concurrency improvements

pull/1/head
Mike Barry 2021-08-11 08:40:49 -04:00
rodzic 994bdb69eb
commit 839224ce2a
15 zmienionych plików z 167 dodań i 164 usunięć

Wyświetl plik

@ -53,5 +53,5 @@ of [OpenMapTiles](https://github.com/openmaptiles/openmaptiles):
| OpenStreetMap (OSM) data | [ODBL](https://www.openstreetmap.org/copyright) | yes | yes
| Natural Earth | [public domain](https://www.naturalearthdata.com/about/terms-of-use/) | yes | yes
| OSM Lakelines | [MIT](https://github.com/lukasmartinelli/osm-lakelines), data from OSM [ODBL](https://www.openstreetmap.org/copyright) | yes | no
| OSM Water Polygons | [acknowledgement](https://osmdata.openstreetmap.de/info/license.html), data from OSM [ODBL](https://www.openstreetmap.org/copyright) | yes | no
| OSM Water Polygons | [acknowledgement](https://osmdata.openstreetmap.de/info/license.html), data from OSM [ODBL](https://www.openstreetmap.org/copyright) | yes | yes
| Wikidata name translations | [CCO](https://www.wikidata.org/wiki/Wikidata:Licensing) | no | no

Wyświetl plik

@ -102,6 +102,8 @@ public final class FeatureGroup implements Consumer<FeatureSort.Entry>, Iterable
* Optimization: Re-use the same buffer packer to avoid allocating and resizing new byte arrays for every feature.
*/
var packer = MessagePack.newDefaultBufferPacker();
// and also avoid a ThreadLocal lookup on every layer stats call by getting the handler for this thread once
var threadLocalLayerStats = layerStats.handlerForThread();
/*
* Optimization: Avoid re-encoding values for identical fill geometries (ie. in the ocean) by memoizing based on
@ -114,7 +116,7 @@ public final class FeatureGroup implements Consumer<FeatureSort.Entry>, Iterable
@Override
public FeatureSort.Entry apply(RenderedFeature feature) {
layerStats.accept(feature);
threadLocalLayerStats.accept(feature);
var group = feature.group();
var thisFeature = feature.vectorTileFeature();
byte[] encodedValue;

Wyświetl plik

@ -3,6 +3,7 @@ package com.onthegomap.flatmap.geo;
import java.util.ArrayList;
import java.util.List;
import java.util.Objects;
import javax.annotation.concurrent.ThreadSafe;
import org.locationtech.jts.geom.Coordinate;
import org.locationtech.jts.geom.Envelope;
import org.locationtech.jts.geom.Geometry;
@ -10,6 +11,7 @@ import org.locationtech.jts.geom.GeometryCollection;
import org.locationtech.jts.geom.Point;
import org.locationtech.jts.index.strtree.STRtree;
@ThreadSafe
public class PointIndex<T> {
private record GeomWithData<T>(Coordinate coord, T data) {}
@ -89,4 +91,5 @@ public class PointIndex<T> {
}
}
}
}

Wyświetl plik

@ -2,6 +2,7 @@ package com.onthegomap.flatmap.geo;
import java.util.ArrayList;
import java.util.List;
import javax.annotation.concurrent.ThreadSafe;
import org.jetbrains.annotations.NotNull;
import org.locationtech.jts.geom.Geometry;
import org.locationtech.jts.geom.GeometryCollection;
@ -9,6 +10,7 @@ import org.locationtech.jts.geom.Point;
import org.locationtech.jts.geom.Polygon;
import org.locationtech.jts.index.strtree.STRtree;
@ThreadSafe
public class PolygonIndex<T> {
private record GeomWithData<T>(Polygon poly, T data) {}
@ -94,7 +96,7 @@ public class PolygonIndex<T> {
return nearests.isEmpty() ? null : nearests.get(0);
}
public synchronized void put(Geometry geom, T item) {
public void put(Geometry geom, T item) {
if (geom instanceof Polygon poly) {
// need to externally synchronize inserts into the STRTree
synchronized (this) {

Wyświetl plik

@ -255,7 +255,8 @@ public class OpenStreetMapReader implements Closeable, MemoryEstimator.HasEstima
SourceFeature processWayPass2(NodeLocationProvider nodeCache, ReaderWay way) {
LongArrayList nodes = way.getNodes();
if (waysInMultipolygon.contains(way.getId())) {
synchronized (multipolygonWayGeometries) {
// multiple threads may update this concurrently
synchronized (this) {
multipolygonWayGeometries.putAll(way.getId(), nodes);
}
}

Wyświetl plik

@ -1,8 +1,7 @@
package com.onthegomap.flatmap.stats;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.LongSupplier;
@ -62,7 +61,7 @@ public interface Counter {
private MultiThreadCounter() {
}
private final List<SingleThreadCounter> all = Collections.synchronizedList(new ArrayList<>());
private final List<SingleThreadCounter> all = new CopyOnWriteArrayList<>();
private final ThreadLocal<SingleThreadCounter> thread = ThreadLocal.withInitial(() -> {
SingleThreadCounter counter = new SingleThreadCounter();
all.add(counter);
@ -85,9 +84,7 @@ public interface Counter {
@Override
public long get() {
synchronized (all) {
return all.stream().mapToLong(SingleThreadCounter::get).sum();
}
return all.stream().mapToLong(SingleThreadCounter::get).sum();
}
}

Wyświetl plik

@ -21,11 +21,9 @@ import java.nio.file.Files;
import java.nio.file.Path;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Collections;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
@ -43,9 +41,8 @@ public class PrometheusStats implements Stats {
private PushGateway pg;
private ScheduledExecutorService executor;
private final String job;
private final Map<String, Path> filesToMonitor = Collections.synchronizedMap(new LinkedHashMap<>());
private final Map<String, MemoryEstimator.HasEstimate> heapObjectsToMonitor = Collections
.synchronizedMap(new LinkedHashMap<>());
private final Map<String, Path> filesToMonitor = new ConcurrentSkipListMap<>();
private final Map<String, MemoryEstimator.HasEstimate> heapObjectsToMonitor = new ConcurrentSkipListMap<>();
PrometheusStats(String job) {
this.job = job;
@ -229,31 +226,29 @@ public class PrometheusStats implements Stats {
@Override
public List<MetricFamilySamples> collect() {
List<Collector.MetricFamilySamples> results = new ArrayList<>();
synchronized (filesToMonitor) {
for (var file : filesToMonitor.entrySet()) {
String name = sanitizeMetricName(file.getKey());
Path path = file.getValue();
results.add(new GaugeMetricFamily(BASE + "file_" + name + "_size_bytes", "Size of " + name + " in bytes",
FileUtils.size(path)));
if (Files.exists(path)) {
try {
FileStore fileStore = Files.getFileStore(path);
results
.add(
new GaugeMetricFamily(BASE + "file_" + name + "_total_space_bytes", "Total space available on disk",
fileStore.getTotalSpace()));
results.add(
new GaugeMetricFamily(BASE + "file_" + name + "_unallocated_space_bytes", "Unallocated space on disk",
fileStore.getUnallocatedSpace()));
results
.add(new GaugeMetricFamily(BASE + "file_" + name + "_usable_space_bytes", "Usable space on disk",
fileStore.getUsableSpace()));
} catch (IOException e) {
// let the user know once
if (!logged) {
LOGGER.warn("unable to get usable space on device", e);
logged = true;
}
for (var file : filesToMonitor.entrySet()) {
String name = sanitizeMetricName(file.getKey());
Path path = file.getValue();
results.add(new GaugeMetricFamily(BASE + "file_" + name + "_size_bytes", "Size of " + name + " in bytes",
FileUtils.size(path)));
if (Files.exists(path)) {
try {
FileStore fileStore = Files.getFileStore(path);
results
.add(
new GaugeMetricFamily(BASE + "file_" + name + "_total_space_bytes", "Total space available on disk",
fileStore.getTotalSpace()));
results.add(
new GaugeMetricFamily(BASE + "file_" + name + "_unallocated_space_bytes", "Unallocated space on disk",
fileStore.getUnallocatedSpace()));
results
.add(new GaugeMetricFamily(BASE + "file_" + name + "_usable_space_bytes", "Usable space on disk",
fileStore.getUsableSpace()));
} catch (IOException e) {
// let the user know once
if (!logged) {
LOGGER.warn("unable to get usable space on device", e);
logged = true;
}
}
}
@ -302,7 +297,7 @@ public class PrometheusStats implements Stats {
this.osBean = ManagementFactory.getOperatingSystemMXBean();
}
private Map<Long, ProcessInfo.ThreadState> threads = Collections.synchronizedMap(new TreeMap<>());
private Map<Long, ProcessInfo.ThreadState> threads = new ConcurrentSkipListMap<>();
public List<MetricFamilySamples> collect() {
@ -320,12 +315,10 @@ public class PrometheusStats implements Stats {
"User time used by each thread", List.of("name", "id"));
mfs.add(threadUserTimes);
threads.putAll(ProcessInfo.getThreadStats());
synchronized (threads) {
for (ProcessInfo.ThreadState thread : threads.values()) {
var labels = List.of(thread.name(), Long.toString(thread.id()));
threadUserTimes.addMetric(labels, thread.userTimeNanos() / NANOSECONDS_PER_SECOND);
threadCpuTimes.addMetric(labels, thread.cpuTimeNanos() / NANOSECONDS_PER_SECOND);
}
for (ProcessInfo.ThreadState thread : threads.values()) {
var labels = List.of(thread.name(), Long.toString(thread.id()));
threadUserTimes.addMetric(labels, thread.userTimeNanos() / NANOSECONDS_PER_SECOND);
threadCpuTimes.addMetric(labels, thread.cpuTimeNanos() / NANOSECONDS_PER_SECOND);
}
return mfs;

Wyświetl plik

@ -13,10 +13,8 @@ public class Timers {
public void printSummary() {
LOGGER.info("-".repeat(50));
synchronized (timers) {
for (var entry : timers.entrySet()) {
LOGGER.info("\t" + entry.getKey() + "\t" + entry.getValue().elapsed());
}
for (var entry : all().entrySet()) {
LOGGER.info("\t" + entry.getKey() + "\t" + entry.getValue().elapsed());
}
}

Wyświetl plik

@ -19,13 +19,19 @@ public class CommonStringEncoder {
}
public byte encode(String string) {
return stringToId.computeIfAbsent(string, s -> {
int id = layerId.getAndIncrement();
if (id > 250) {
throw new IllegalStateException("Too many string keys when inserting " + string);
}
idToLayer[id] = string;
return (byte) id;
});
// optimization to avoid more expensive computeIfAbsent call for the majority case when concurrent hash map already
// contains the value.
Byte result = stringToId.get(string);
if (result == null) {
result = stringToId.computeIfAbsent(string, s -> {
int id = layerId.getAndIncrement();
if (id > 250) {
throw new IllegalStateException("Too many string keys when inserting " + string);
}
idToLayer[id] = string;
return (byte) id;
});
}
return result;
}
}

Wyświetl plik

@ -2,78 +2,80 @@ package com.onthegomap.flatmap.util;
import com.onthegomap.flatmap.mbiles.Mbtiles;
import com.onthegomap.flatmap.render.RenderedFeature;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.function.Consumer;
public class LayerStats implements Consumer<RenderedFeature> {
private final List<Map<String, VectorTileStats>> threadLocals = Collections.synchronizedList(new ArrayList<>());
private final ThreadLocal<Map<String, VectorTileStats>> layerStats = ThreadLocal.withInitial(() -> {
Map<String, VectorTileStats> result = new TreeMap<>();
threadLocals.add(result);
return result;
});
private static VectorTileStats merge(VectorTileStats a, VectorTileStats b) {
for (var entry : b.fields.entrySet()) {
a.fields.merge(entry.getKey(), entry.getValue(), Mbtiles.MetadataJson.FieldType::merge);
a.zoom(b.minzoom);
a.zoom(b.maxzoom);
}
return a;
}
private final List<ThreadLocalLayerStatsHandler> threadLocals = new CopyOnWriteArrayList<>();
private final ThreadLocal<ThreadLocalLayerStatsHandler> layerStats = ThreadLocal
.withInitial(ThreadLocalLayerStatsHandler::new);
public Mbtiles.MetadataJson getTileStats() {
synchronized (threadLocals) {
Map<String, VectorTileStats> layers = new TreeMap<>();
for (Map<String, VectorTileStats> threadLocal : threadLocals) {
for (VectorTileStats stats : threadLocal.values()) {
layers.merge(stats.layer, stats, (oldOne, newOne) -> {
oldOne.zoom(newOne.maxzoom);
oldOne.zoom(newOne.minzoom);
for (var entry : newOne.fields.entrySet()) {
oldOne.fields.merge(entry.getKey(), entry.getValue(), Mbtiles.MetadataJson.FieldType::merge);
}
return oldOne;
});
Map<String, VectorTileStats> layers = new TreeMap<>();
for (var threadLocal : threadLocals) {
for (VectorTileStats stats : threadLocal.layers.values()) {
layers.merge(stats.layer, stats, (oldOne, newOne) -> {
oldOne.zoom(newOne.maxzoom);
oldOne.zoom(newOne.minzoom);
for (var entry : newOne.fields.entrySet()) {
oldOne.fields.merge(entry.getKey(), entry.getValue(), Mbtiles.MetadataJson.FieldType::merge);
}
return oldOne;
});
}
}
return new Mbtiles.MetadataJson(
layers.values().stream()
.map(stats -> new Mbtiles.MetadataJson.VectorLayer(stats.layer, stats.fields, stats.minzoom, stats.maxzoom))
.toList()
);
}
private class ThreadLocalLayerStatsHandler implements Consumer<RenderedFeature> {
private final Map<String, VectorTileStats> layers = new TreeMap<>();
ThreadLocalLayerStatsHandler() {
threadLocals.add(this);
}
@Override
public void accept(RenderedFeature feature) {
int zoom = feature.tile().z();
var vectorTileFeature = feature.vectorTileFeature();
var stats = layers.computeIfAbsent(vectorTileFeature.layer(), VectorTileStats::new);
stats.zoom(zoom);
for (Map.Entry<String, Object> entry : vectorTileFeature.attrs().entrySet()) {
String key = entry.getKey();
Object value = entry.getValue();
Mbtiles.MetadataJson.FieldType fieldType = null;
if (value instanceof Number) {
fieldType = Mbtiles.MetadataJson.FieldType.NUMBER;
} else if (value instanceof Boolean) {
fieldType = Mbtiles.MetadataJson.FieldType.BOOLEAN;
} else if (value != null) {
fieldType = Mbtiles.MetadataJson.FieldType.STRING;
}
if (fieldType != null) {
stats.fields.merge(key, fieldType, Mbtiles.MetadataJson.FieldType::merge);
}
}
return new Mbtiles.MetadataJson(
layers.values().stream()
.map(stats -> new Mbtiles.MetadataJson.VectorLayer(stats.layer, stats.fields, stats.minzoom, stats.maxzoom))
.toList()
);
}
}
public Consumer<RenderedFeature> handlerForThread() {
return layerStats.get();
}
@Override
public void accept(RenderedFeature feature) {
int zoom = feature.tile().z();
var vectorTileFeature = feature.vectorTileFeature();
var layers = layerStats.get();
var stats = layers.computeIfAbsent(vectorTileFeature.layer(), VectorTileStats::new);
stats.zoom(zoom);
for (Map.Entry<String, Object> entry : vectorTileFeature.attrs().entrySet()) {
String key = entry.getKey();
Object value = entry.getValue();
Mbtiles.MetadataJson.FieldType fieldType = null;
if (value instanceof Number) {
fieldType = Mbtiles.MetadataJson.FieldType.NUMBER;
} else if (value instanceof Boolean) {
fieldType = Mbtiles.MetadataJson.FieldType.BOOLEAN;
} else if (value != null) {
fieldType = Mbtiles.MetadataJson.FieldType.STRING;
}
if (fieldType != null) {
stats.fields.merge(key, fieldType, Mbtiles.MetadataJson.FieldType::merge);
}
}
handlerForThread().accept(feature);
}
private static class VectorTileStats {

Wyświetl plik

@ -31,12 +31,12 @@ import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.function.BiConsumer;
import java.util.function.Consumer;
import java.util.function.Function;
@ -1149,7 +1149,7 @@ public class FlatMapTest {
newReaderFeature(newPoint(lng2, lat), Map.of("a", 3, "b", 4))
),
new Profile.NullProfile() {
private final List<SourceFeature> featureList = Collections.synchronizedList(new ArrayList<>());
private final List<SourceFeature> featureList = new CopyOnWriteArrayList<>();
@Override
public void processFeature(SourceFeature in, FeatureCollector features) {
@ -1160,15 +1160,13 @@ public class FlatMapTest {
public void finish(String name, FeatureCollector.Factory featureCollectors,
Consumer<FeatureCollector.Feature> next) {
if ("test".equals(name)) {
synchronized (featureList) {
for (SourceFeature in : featureList) {
var features = featureCollectors.get(in);
features.point("layer")
.setZoomRange(13, 14)
.inheritFromSource("a");
for (var feature : features) {
next.accept(feature);
}
for (SourceFeature in : featureList) {
var features = featureCollectors.get(in);
features.point("layer")
.setZoomRange(13, 14)
.inheritFromSource("a");
for (var feature : features) {
next.accept(feature);
}
}
}
@ -1205,7 +1203,7 @@ public class FlatMapTest {
with(new ReaderNode(2, lat, lng2), t -> t.setTag("a", 3))
),
new Profile.NullProfile() {
private final List<SourceFeature> featureList = Collections.synchronizedList(new ArrayList<>());
private final List<SourceFeature> featureList = new CopyOnWriteArrayList<>();
@Override
public void processFeature(SourceFeature in, FeatureCollector features) {
@ -1216,15 +1214,13 @@ public class FlatMapTest {
public void finish(String name, FeatureCollector.Factory featureCollectors,
Consumer<FeatureCollector.Feature> next) {
if ("osm".equals(name)) {
synchronized (featureList) {
for (SourceFeature in : featureList) {
var features = featureCollectors.get(in);
features.point("layer")
.setZoomRange(13, 14)
.inheritFromSource("a");
for (var feature : features) {
next.accept(feature);
}
for (SourceFeature in : featureList) {
var features = featureCollectors.get(in);
features.point("layer")
.setZoomRange(13, 14)
.inheritFromSource("a");
for (var feature : features) {
next.accept(feature);
}
}
}

Wyświetl plik

@ -6,10 +6,9 @@ import static org.junit.jupiter.api.Assertions.assertThrows;
import com.onthegomap.flatmap.stats.ProgressLoggers;
import com.onthegomap.flatmap.stats.Stats;
import java.time.Duration;
import java.util.Collections;
import java.util.List;
import java.util.Set;
import java.util.TreeSet;
import java.util.concurrent.ConcurrentSkipListSet;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;
import org.junit.jupiter.params.ParameterizedTest;
@ -22,7 +21,7 @@ public class WorkerPipelineTest {
@Test
@Timeout(10)
public void testSimplePipeline() {
Set<Integer> result = Collections.synchronizedSet(new TreeSet<>());
Set<Integer> result = new ConcurrentSkipListSet<>();
var pipeline = WorkerPipeline.start("test", stats)
.<Integer>fromGenerator("reader", (next) -> {
next.accept(0);
@ -46,7 +45,7 @@ public class WorkerPipelineTest {
@Timeout(10)
public void testPipelineFromQueue() {
var queue = new WorkQueue<Integer>("readerqueue", 10, 1, stats);
Set<Integer> result = Collections.synchronizedSet(new TreeSet<>());
Set<Integer> result = new ConcurrentSkipListSet<>();
var pipeline = WorkerPipeline.start("test", stats)
.<Integer>readFromQueue(queue)
.<Integer>addWorker("process", 1, (prev, next) -> {
@ -72,7 +71,7 @@ public class WorkerPipelineTest {
@Test
@Timeout(10)
public void testPipelineFromIterator() {
Set<Integer> result = Collections.synchronizedSet(new TreeSet<>());
Set<Integer> result = new ConcurrentSkipListSet<>();
var pipeline = WorkerPipeline.start("test", stats)
.readFrom("reader", List.of(0, 1))
.addBuffer("reader_queue", 1)
@ -95,7 +94,6 @@ public class WorkerPipelineTest {
@ValueSource(ints = {1, 2, 3})
public void testThrowingExceptionInPipelineHandledGracefully(int failureStage) {
class ExpectedException extends RuntimeException {}
Set<Integer> result = Collections.synchronizedSet(new TreeSet<>());
var pipeline = WorkerPipeline.start("test", stats)
.<Integer>fromGenerator("reader", (next) -> {
if (failureStage == 1) {

Wyświetl plik

@ -63,6 +63,7 @@ import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.function.Consumer;
import java.util.function.Function;
import java.util.stream.Collectors;
@ -92,11 +93,13 @@ public class Boundary implements
private static final Logger LOGGER = LoggerFactory.getLogger(Boundary.class);
private static final double COUNTRY_TEST_OFFSET = GeoUtils.metersToPixelAtEquator(0, 10) / 256d;
private final Map<Long, String> regionNames = new HashMap<>();
private final Map<Long, List<Geometry>> regionGeometries = new HashMap<>();
private final Map<CountryBoundaryComponent, List<Geometry>> boundariesToMerge = new HashMap<>();
private final Stats stats;
private final boolean addCountryNames;
// may be updated concurrently by multiple threads
private final Map<Long, String> regionNames = new ConcurrentHashMap<>();
// need to synchronize updates to these shared data structures:
private final Map<Long, List<Geometry>> regionGeometries = new HashMap<>();
private final Map<CountryBoundaryComponent, List<Geometry>> boundariesToMerge = new HashMap<>();
public Boundary(Translations translations, Arguments args, Stats stats) {
this.addCountryNames = args.get(
@ -163,9 +166,7 @@ public class Boundary implements
if (adminLevelValue != null && adminLevelValue >= 2 && adminLevelValue <= 10) {
boolean disputed = isDisputed(ReaderElementUtils.getProperties(relation));
if (code != null) {
synchronized (regionNames) {
regionNames.put(relation.getId(), code);
}
regionNames.put(relation.getId(), code);
}
return List.of(new BoundaryRelation(
relation.getId(),
@ -234,7 +235,8 @@ public class Boundary implements
claimedBy,
disputedName
);
synchronized (regionGeometries) {
// multiple threads may update this concurrently
synchronized (this) {
boundariesToMerge.computeIfAbsent(component.groupingKey(), key -> new ArrayList<>()).add(component.line);
for (var info : relationInfos) {
var rel = info.relation();
@ -291,8 +293,8 @@ public class Boundary implements
.setAttr(Fields.MARITIME, key.maritime ? 1 : 0)
.setAttr(Fields.CLAIMED_BY, key.claimedBy)
.setAttr(Fields.DISPUTED_NAME, key.disputed ? editName(key.name) : null)
.setAttr(Fields.ADM0_L, regionNames.get(borderingRegions.left))
.setAttr(Fields.ADM0_R, regionNames.get(borderingRegions.right))
.setAttr(Fields.ADM0_L, borderingRegions.left == null ? null : regionNames.get(borderingRegions.left))
.setAttr(Fields.ADM0_R, borderingRegions.right == null ? null : regionNames.get(borderingRegions.right))
.setMinPixelSizeAtAllZooms(0)
.setZoomRange(key.minzoom, 14);
for (var feature : features) {

Wyświetl plik

@ -146,13 +146,15 @@ public class TransportationName implements
public void processNaturalEarth(String table, SourceFeature feature,
FeatureCollector features) {
if ("ne_10m_admin_0_countries".equals(table) && feature.hasTag("iso_a2", "GB")) {
try {
synchronized (this) {
// multiple threads call this method concurrently, GB polygon *should* only be found
// once, but just to be safe synchronize updates to that field
synchronized (this) {
try {
Geometry boundary = feature.polygon().buffer(GeoUtils.metersToPixelAtEquator(0, 10_000) / 256d);
greatBritain = PreparedGeometryFactory.prepare(boundary);
} catch (GeometryException e) {
LOGGER.error("Failed to get Great Britain Polygon: " + e);
}
} catch (GeometryException e) {
LOGGER.error("Failed to get Great Britain Polygon: " + e);
}
}
}

Wyświetl plik

@ -52,7 +52,7 @@ import com.onthegomap.flatmap.reader.SourceFeature;
import com.onthegomap.flatmap.stats.Stats;
import com.onthegomap.flatmap.util.Parse;
import java.util.Map;
import java.util.TreeMap;
import java.util.concurrent.ConcurrentSkipListMap;
import org.locationtech.jts.geom.Geometry;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -69,8 +69,10 @@ public class WaterName implements OpenMapTilesSchema.WaterName,
private static final Logger LOGGER = LoggerFactory.getLogger(WaterName.class);
private final Translations translations;
// need to synchronize updates from multiple threads
private final LongObjectMap<Geometry> lakeCenterlines = new GHLongObjectHashMap<>();
private final TreeMap<String, Integer> importantMarinePoints = new TreeMap<>();
// may be updated concurrently by multiple threads
private final ConcurrentSkipListMap<String, Integer> importantMarinePoints = new ConcurrentSkipListMap<>();
private final Stats stats;
@Override
@ -92,9 +94,7 @@ public class WaterName implements OpenMapTilesSchema.WaterName,
Integer scalerank = Parse.parseIntOrNull(feature.getTag("scalerank"));
if (name != null && scalerank != null) {
name = name.replaceAll("\\s+", " ").trim().toLowerCase();
synchronized (importantMarinePoints) {
importantMarinePoints.put(name, scalerank);
}
importantMarinePoints.put(name, scalerank);
}
}
}
@ -105,12 +105,13 @@ public class WaterName implements OpenMapTilesSchema.WaterName,
if (osmId == 0L) {
LOGGER.warn("Bad lake centerline: " + feature);
} else {
synchronized (lakeCenterlines) {
try {
try {
// multiple threads call this concurrently
synchronized (this) {
lakeCenterlines.put(osmId, feature.worldGeometry());
} catch (GeometryException e) {
e.log(stats, "omt_water_name_lakeline", "Bad lake centerline: " + feature);
}
} catch (GeometryException e) {
e.log(stats, "omt_water_name_lakeline", "Bad lake centerline: " + feature);
}
}
}