planetiler/flatmap-core/src/main/java/com/onthegomap/flatmap/reader/osm/OsmReader.java

631 wiersze
25 KiB
Java
Czysty Zwykły widok Historia

2021-08-06 09:56:24 +00:00
package com.onthegomap.flatmap.reader.osm;
2021-04-10 09:25:42 +00:00
2021-09-10 00:46:20 +00:00
import static com.onthegomap.flatmap.util.MemoryEstimator.estimateSize;
import static com.onthegomap.flatmap.util.MemoryEstimator.estimateSizeWithoutKeys;
import static com.onthegomap.flatmap.util.MemoryEstimator.estimateSizeWithoutValues;
2021-06-18 11:21:43 +00:00
import com.carrotsearch.hppc.IntObjectHashMap;
2021-05-13 10:25:06 +00:00
import com.carrotsearch.hppc.LongArrayList;
2021-04-12 10:05:32 +00:00
import com.carrotsearch.hppc.LongHashSet;
2021-06-18 11:21:43 +00:00
import com.carrotsearch.hppc.ObjectIntHashMap;
import com.graphhopper.coll.GHIntObjectHashMap;
2021-04-12 10:05:32 +00:00
import com.graphhopper.coll.GHLongHashSet;
import com.graphhopper.coll.GHLongObjectHashMap;
2021-06-18 11:21:43 +00:00
import com.graphhopper.coll.GHObjectIntHashMap;
2021-04-12 10:05:32 +00:00
import com.graphhopper.reader.ReaderElement;
2021-04-23 11:26:02 +00:00
import com.graphhopper.reader.ReaderElementUtils;
2021-04-12 10:05:32 +00:00
import com.graphhopper.reader.ReaderNode;
import com.graphhopper.reader.ReaderRelation;
import com.graphhopper.reader.ReaderWay;
2021-05-08 10:53:37 +00:00
import com.onthegomap.flatmap.FeatureCollector;
2021-04-12 10:05:32 +00:00
import com.onthegomap.flatmap.Profile;
2021-08-06 09:56:24 +00:00
import com.onthegomap.flatmap.collection.FeatureGroup;
import com.onthegomap.flatmap.collection.LongLongMap;
import com.onthegomap.flatmap.collection.LongLongMultimap;
2021-09-10 00:46:20 +00:00
import com.onthegomap.flatmap.collection.SortableFeature;
import com.onthegomap.flatmap.config.FlatmapConfig;
2021-04-25 11:42:13 +00:00
import com.onthegomap.flatmap.geo.GeoUtils;
2021-05-28 10:08:13 +00:00
import com.onthegomap.flatmap.geo.GeometryException;
2021-08-06 09:56:24 +00:00
import com.onthegomap.flatmap.reader.SourceFeature;
2021-05-20 09:59:18 +00:00
import com.onthegomap.flatmap.render.FeatureRenderer;
2021-08-06 09:56:24 +00:00
import com.onthegomap.flatmap.stats.Counter;
import com.onthegomap.flatmap.stats.ProgressLoggers;
import com.onthegomap.flatmap.stats.Stats;
import com.onthegomap.flatmap.util.MemoryEstimator;
2021-08-05 11:09:52 +00:00
import com.onthegomap.flatmap.worker.WorkerPipeline;
2021-04-10 09:25:42 +00:00
import java.io.Closeable;
2021-04-12 10:59:34 +00:00
import java.io.IOException;
2021-05-30 11:42:06 +00:00
import java.util.ArrayList;
2021-04-12 10:05:32 +00:00
import java.util.List;
2021-06-06 12:00:04 +00:00
import java.util.Map;
2021-04-12 10:05:32 +00:00
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicLong;
2021-06-23 01:46:42 +00:00
import java.util.function.Consumer;
2021-05-30 11:42:06 +00:00
import org.locationtech.jts.geom.Coordinate;
import org.locationtech.jts.geom.CoordinateList;
2021-05-13 10:25:06 +00:00
import org.locationtech.jts.geom.CoordinateSequence;
2021-05-30 11:42:06 +00:00
import org.locationtech.jts.geom.CoordinateXY;
2021-04-12 10:05:32 +00:00
import org.locationtech.jts.geom.Geometry;
2021-09-10 00:46:20 +00:00
import org.locationtech.jts.geom.LineString;
import org.locationtech.jts.geom.MultiPolygon;
import org.locationtech.jts.geom.Point;
import org.locationtech.jts.geom.Polygon;
2021-05-30 11:42:06 +00:00
import org.locationtech.jts.geom.impl.CoordinateArraySequence;
2021-05-13 10:25:06 +00:00
import org.locationtech.jts.geom.impl.PackedCoordinateSequence;
2021-05-30 11:42:06 +00:00
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
2021-04-10 09:25:42 +00:00
2021-09-10 00:46:20 +00:00
/**
* Utility that constructs {@link SourceFeature SourceFeatures} from the raw nodes, ways, and relations contained in an
* {@link OsmInputFile}.
*/
2021-08-14 09:55:00 +00:00
public class OsmReader implements Closeable, MemoryEstimator.HasEstimate {
2021-04-10 09:25:42 +00:00
2021-08-14 09:55:00 +00:00
private static final Logger LOGGER = LoggerFactory.getLogger(OsmReader.class);
2021-09-10 00:46:20 +00:00
private static final int ROLE_BITS = 16;
private static final int MAX_ROLES = (1 << ROLE_BITS) - 10;
private static final int ROLE_SHIFT = 64 - ROLE_BITS;
private static final int ROLE_MASK = (1 << ROLE_BITS) - 1;
private static final long NOT_ROLE_MASK = (1L << ROLE_SHIFT) - 1L;
2021-05-28 10:08:13 +00:00
private final OsmSource osmInputFile;
2021-04-12 10:05:32 +00:00
private final Stats stats;
2021-09-10 00:46:20 +00:00
private final LongLongMap nodeLocationDb;
2021-06-06 12:00:04 +00:00
private final Counter.Readable PASS1_NODES = Counter.newSingleThreadCounter();
private final Counter.Readable PASS1_WAYS = Counter.newSingleThreadCounter();
private final Counter.Readable PASS1_RELATIONS = Counter.newSingleThreadCounter();
2021-05-01 20:08:20 +00:00
private final Profile profile;
2021-05-28 10:08:13 +00:00
private final String name;
2021-09-10 00:46:20 +00:00
private final AtomicLong relationInfoSizes = new AtomicLong(0);
2021-04-12 10:05:32 +00:00
// need a few large objects to process ways in relations, should be small enough to keep in memory
// for routes (750k rels 40m ways) and boundaries (650k rels, 8m ways)
// need to store route info to use later when processing ways
// <~500mb
2021-09-10 00:46:20 +00:00
private GHLongObjectHashMap<OsmRelationInfo> relationInfo = new GHLongObjectHashMap<>();
2021-04-12 10:05:32 +00:00
// ~800mb, ~1.6GB when sorting
2021-05-04 11:07:16 +00:00
private LongLongMultimap wayToRelations = LongLongMultimap.newSparseUnorderedMultimap();
2021-04-12 10:05:32 +00:00
// for multipolygons need to store way info (20m ways, 800m nodes) to use when processing relations (4.5m)
// ~300mb
private LongHashSet waysInMultipolygon = new GHLongHashSet();
// ~7GB
2021-05-04 11:07:16 +00:00
private LongLongMultimap multipolygonWayGeometries = LongLongMultimap.newDensedOrderedMultimap();
2021-09-10 00:46:20 +00:00
// keep track of data needed to encode/decode role strings into a long
private final ObjectIntHashMap<String> roleIds = new GHObjectIntHashMap<>();
private final IntObjectHashMap<String> roleIdsReverse = new GHIntObjectHashMap<>();
private final AtomicLong roleSizes = new AtomicLong(0);
/**
* Constructs a new {@code OsmReader} from
*
* @param name ID for this reader to use in stats and logs
* @param osmInputFile the file to read raw nodes, ways, and relations from
* @param nodeLocationDb store that will temporarily hold node locations (encoded as a long) between passes to
* reconstruct way geometries
* @param profile logic that defines what map features to emit for each source feature
* @param stats to keep track of counters and timings
*/
public OsmReader(String name, OsmSource osmInputFile, LongLongMap nodeLocationDb, Profile profile,
2021-05-28 10:08:13 +00:00
Stats stats) {
this.name = name;
2021-04-12 10:05:32 +00:00
this.osmInputFile = osmInputFile;
2021-09-10 00:46:20 +00:00
this.nodeLocationDb = nodeLocationDb;
2021-04-12 10:05:32 +00:00
this.stats = stats;
2021-05-01 20:08:20 +00:00
this.profile = profile;
2021-06-05 12:02:51 +00:00
stats.monitorInMemoryObject("osm_relations", this);
2021-06-06 12:00:04 +00:00
stats.counter("osm_pass1_elements_processed", "type", () -> Map.of(
"nodes", PASS1_NODES,
"ways", PASS1_WAYS,
"relations", PASS1_RELATIONS
));
2021-04-12 10:05:32 +00:00
}
2021-09-10 00:46:20 +00:00
/**
* Pre-processes all OSM elements before {@link #pass2(FeatureGroup, FlatmapConfig)} is used to emit map features.
* <p>
* Stores node locations for pass2 to use to reconstruct way geometries.
* <p>
* Also stores the result of {@link Profile#preprocessOsmRelation(OsmElement.Relation)} so that pass2 can know the
* relevant relations that a way belongs to.
*
* @param config user-provided arguments to control the number of threads, and log interval
*/
public void pass1(FlatmapConfig config) {
2021-08-10 10:55:30 +00:00
var timer = stats.startStage("osm_pass1");
2021-07-24 00:20:51 +00:00
String pbfParsePrefix = "pbfpass1";
2021-07-24 14:38:06 +00:00
int parseThreads = Math.max(1, config.threads() - 2);
2021-08-05 11:09:52 +00:00
var pipeline = WorkerPipeline.start("osm_pass1", stats)
2021-07-24 14:38:06 +00:00
.fromGenerator("pbf", osmInputFile.read("pbfpass1", parseThreads))
2021-04-16 11:13:05 +00:00
.addBuffer("reader_queue", 50_000, 10_000)
2021-09-10 00:46:20 +00:00
.sinkToConsumer("process", 1, this::processPass1Element);
2021-04-12 10:05:32 +00:00
2021-09-10 00:46:20 +00:00
var loggers = ProgressLoggers.create()
2021-08-10 10:55:30 +00:00
.addRateCounter("nodes", PASS1_NODES, true)
2021-10-20 01:57:47 +00:00
.addFileSizeAndRam(nodeLocationDb)
2021-08-10 10:55:30 +00:00
.addRateCounter("ways", PASS1_WAYS, true)
.addRateCounter("rels", PASS1_RELATIONS, true)
.newLine()
2021-04-12 10:05:32 +00:00
.addProcessStats()
2021-05-04 12:02:22 +00:00
.addInMemoryObject("hppc", this)
2021-08-10 10:55:30 +00:00
.newLine()
2021-10-20 01:57:47 +00:00
.addThreadPoolStats("parse", pbfParsePrefix + "-pool")
2021-08-05 11:09:52 +00:00
.addPipelineStats(pipeline);
pipeline.awaitAndLog(loggers, config.logInterval());
2021-06-08 00:55:23 +00:00
timer.stop();
2021-04-10 09:25:42 +00:00
}
2021-09-10 00:46:20 +00:00
void processPass1Element(ReaderElement readerElement) {
// only a single thread calls this with elements ordered by ID, so it's safe to manipulate these
// shared data structures which are not thread safe
2021-10-20 01:57:47 +00:00
if (readerElement.getId() < 0) {
throw new IllegalArgumentException("Negative OSM element IDs not supported: " + readerElement);
}
2021-05-28 10:08:13 +00:00
if (readerElement instanceof ReaderNode node) {
2021-06-06 12:00:04 +00:00
PASS1_NODES.inc();
2021-09-10 00:46:20 +00:00
// TODO allow limiting node storage to only ones that profile cares about
nodeLocationDb.put(node.getId(), GeoUtils.encodeFlatLocation(node.getLon(), node.getLat()));
} else if (readerElement instanceof ReaderWay) {
2021-06-06 12:00:04 +00:00
PASS1_WAYS.inc();
2021-05-28 10:08:13 +00:00
} else if (readerElement instanceof ReaderRelation rel) {
2021-06-06 12:00:04 +00:00
PASS1_RELATIONS.inc();
2021-09-10 00:46:20 +00:00
// don't leak graphhopper classes out through public API
2021-08-14 09:55:00 +00:00
OsmElement.Relation osmRelation = OsmElement.fromGraphopper(rel);
2021-09-10 00:46:20 +00:00
List<OsmRelationInfo> infos = profile.preprocessOsmRelation(osmRelation);
2021-05-28 10:08:13 +00:00
if (infos != null) {
2021-09-10 00:46:20 +00:00
for (OsmRelationInfo info : infos) {
2021-05-28 10:08:13 +00:00
relationInfo.put(rel.getId(), info);
relationInfoSizes.addAndGet(info.estimateMemoryUsageBytes());
for (ReaderRelation.Member member : rel.getMembers()) {
2021-06-24 09:16:30 +00:00
int type = member.getType();
2021-10-20 01:57:47 +00:00
// TODO handle nodes in relations and super-relations
if (type == ReaderRelation.Member.WAY) {
2021-09-10 00:46:20 +00:00
wayToRelations.put(member.getRef(), encodeRelationMembership(member.getRole(), rel.getId()));
2021-05-28 10:08:13 +00:00
}
}
}
}
2021-09-10 00:46:20 +00:00
// TODO allow limiting multipolygon storage to only ones that profile cares about
2021-05-28 10:08:13 +00:00
if (rel.hasTag("type", "multipolygon")) {
for (ReaderRelation.Member member : rel.getMembers()) {
if (member.getType() == ReaderRelation.Member.WAY) {
waysInMultipolygon.add(member.getRef());
}
}
}
}
}
2021-09-10 00:46:20 +00:00
/**
* Constructs geometries from OSM elements and emits map features as defined by the {@link Profile}.
*
* @param writer consumer that will store finished features
* @param config user-provided arguments to control the number of threads, and log interval
*/
public void pass2(FeatureGroup writer, FlatmapConfig config) {
2021-08-10 10:55:30 +00:00
var timer = stats.startStage("osm_pass2");
2021-10-20 01:57:47 +00:00
int threads = config.threads();
int readerThreads = Math.max(threads / 4, 1);
int processThreads = threads - (threads >= 4 ? 1 : 0);
2021-06-06 12:00:04 +00:00
Counter.MultiThreadCounter nodesProcessed = Counter.newMultiThreadCounter();
Counter.MultiThreadCounter waysProcessed = Counter.newMultiThreadCounter();
Counter.MultiThreadCounter relsProcessed = Counter.newMultiThreadCounter();
stats.counter("osm_pass2_elements_processed", "type", () -> Map.of(
"nodes", nodesProcessed,
"ways", waysProcessed,
"relations", relsProcessed
));
2021-09-10 00:46:20 +00:00
// since multiple threads process OSM elements, and we must process all ways before processing any relations,
// use a count down latch to wait for all threads to finish processing ways
2021-04-12 10:05:32 +00:00
CountDownLatch waysDone = new CountDownLatch(processThreads);
2021-07-24 00:20:51 +00:00
String parseThreadPrefix = "pbfpass2";
2021-08-05 11:09:52 +00:00
var pipeline = WorkerPipeline.start("osm_pass2", stats)
2021-07-24 00:20:51 +00:00
.fromGenerator("pbf", osmInputFile.read(parseThreadPrefix, readerThreads))
2021-04-16 11:13:05 +00:00
.addBuffer("reader_queue", 50_000, 1_000)
2021-09-10 00:46:20 +00:00
.<SortableFeature>addWorker("process", processThreads, (prev, next) -> {
// avoid contention trying to get the thread-local counters by getting them once when thread starts
2021-06-06 12:00:04 +00:00
Counter nodes = nodesProcessed.counterForThread();
Counter ways = waysProcessed.counterForThread();
Counter rels = relsProcessed.counterForThread();
2021-04-12 11:14:05 +00:00
ReaderElement readerElement;
2021-06-06 12:00:04 +00:00
var featureCollectors = new FeatureCollector.Factory(config, stats);
2021-09-10 00:46:20 +00:00
NodeLocationProvider nodeLocations = newNodeLocationProvider();
FeatureRenderer renderer = createFeatureRenderer(writer, config, next);
2021-04-12 11:14:05 +00:00
while ((readerElement = prev.get()) != null) {
SourceFeature feature = null;
if (readerElement instanceof ReaderNode node) {
2021-06-06 12:00:04 +00:00
nodes.inc();
2021-05-28 10:08:13 +00:00
feature = processNodePass2(node);
2021-04-12 11:14:05 +00:00
} else if (readerElement instanceof ReaderWay way) {
2021-06-06 12:00:04 +00:00
ways.inc();
2021-09-10 00:46:20 +00:00
feature = processWayPass2(way, nodeLocations);
2021-04-12 11:14:05 +00:00
} else if (readerElement instanceof ReaderRelation rel) {
// ensure all ways finished processing before we start relations
if (waysDone.getCount() > 0) {
waysDone.countDown();
waysDone.await();
}
2021-06-06 12:00:04 +00:00
rels.inc();
2021-09-10 00:46:20 +00:00
feature = processRelationPass2(rel, nodeLocations);
2021-04-12 10:05:32 +00:00
}
2021-09-10 00:46:20 +00:00
// render features specified by profile and hand them off to next step that will
// write them intermediate storage
2021-04-12 11:14:05 +00:00
if (feature != null) {
2021-05-13 10:25:06 +00:00
FeatureCollector features = featureCollectors.get(feature);
2021-04-12 11:14:05 +00:00
profile.processFeature(feature, features);
2021-05-18 10:53:12 +00:00
for (FeatureCollector.Feature renderable : features) {
2021-05-28 10:08:13 +00:00
renderer.accept(renderable);
2021-04-12 11:14:05 +00:00
}
2021-04-12 10:05:32 +00:00
}
}
2021-04-12 11:14:05 +00:00
// just in case a worker skipped over all relations
waysDone.countDown();
}).addBuffer("feature_queue", 50_000, 1_000)
2021-09-10 00:46:20 +00:00
// FeatureGroup writes need to be single-threaded
.sinkToConsumer("write", 1, writer);
2021-04-12 10:05:32 +00:00
2021-09-10 00:46:20 +00:00
var logger = ProgressLoggers.create()
2021-05-28 10:08:13 +00:00
.addRatePercentCounter("nodes", PASS1_NODES.get(), nodesProcessed)
2021-10-20 01:57:47 +00:00
.addFileSizeAndRam(nodeLocationDb)
2021-05-28 10:08:13 +00:00
.addRatePercentCounter("ways", PASS1_WAYS.get(), waysProcessed)
.addRatePercentCounter("rels", PASS1_RELATIONS.get(), relsProcessed)
2021-09-10 00:46:20 +00:00
.addRateCounter("features", writer::numFeaturesWritten)
.addFileSize(writer)
2021-08-10 10:55:30 +00:00
.newLine()
2021-04-12 10:05:32 +00:00
.addProcessStats()
2021-05-04 12:02:22 +00:00
.addInMemoryObject("hppc", this)
2021-08-10 10:55:30 +00:00
.newLine()
2021-07-24 14:38:06 +00:00
.addThreadPoolStats("parse", parseThreadPrefix + "-pool")
2021-08-05 11:09:52 +00:00
.addPipelineStats(pipeline);
2021-04-12 10:05:32 +00:00
2021-08-05 11:09:52 +00:00
pipeline.awaitAndLog(logger, config.logInterval());
2021-06-23 01:46:42 +00:00
profile.finish(name,
new FeatureCollector.Factory(config, stats),
2021-09-10 00:46:20 +00:00
createFeatureRenderer(writer, config, writer));
2021-06-08 00:55:23 +00:00
timer.stop();
2021-04-10 09:25:42 +00:00
}
2021-09-10 00:46:20 +00:00
private FeatureRenderer createFeatureRenderer(FeatureGroup writer, FlatmapConfig config,
Consumer<SortableFeature> next) {
2021-06-23 01:46:42 +00:00
var encoder = writer.newRenderedFeatureEncoder();
return new FeatureRenderer(
config,
rendered -> next.accept(encoder.apply(rendered)),
stats
);
}
2021-09-10 00:46:20 +00:00
SourceFeature processNodePass2(ReaderNode node) {
// nodes are simple because they already contain their location
return new NodeSourceFeature(node);
2021-05-28 10:08:13 +00:00
}
2021-09-10 00:46:20 +00:00
SourceFeature processWayPass2(ReaderWay way, NodeLocationProvider nodeLocations) {
// ways contain an ordered list of node IDs, so we need to join that with node locations
// from pass1 to reconstruct the geometry.
2021-05-28 10:08:13 +00:00
LongArrayList nodes = way.getNodes();
2021-05-30 11:42:06 +00:00
if (waysInMultipolygon.contains(way.getId())) {
2021-09-10 00:46:20 +00:00
// if this is part of a multipolygon, store the node IDs for this way ID so that when
// we get to the multipolygon we can go from way IDs -> node IDs -> node locations.
synchronized (this) { // multiple threads may update this concurrently
2021-05-30 11:42:06 +00:00
multipolygonWayGeometries.putAll(way.getId(), nodes);
}
}
2021-05-28 10:08:13 +00:00
boolean closed = nodes.size() > 1 && nodes.get(0) == nodes.get(nodes.size() - 1);
2021-09-10 00:46:20 +00:00
// area tag used to differentiate between whether a closed way should be treated as a polygon or linestring
2021-05-28 10:08:13 +00:00
String area = way.getTag("area");
2021-09-10 00:46:20 +00:00
List<RelationMember<OsmRelationInfo>> rels = getRelationMembershipForWay(way.getId());
return new WaySourceFeature(way, closed, area, nodeLocations, rels);
2021-06-24 09:16:30 +00:00
}
2021-09-10 00:46:20 +00:00
SourceFeature processRelationPass2(ReaderRelation rel, NodeLocationProvider nodeLocations) {
// Relation info gets used during way processing, except multipolygons which we have to process after we've
// stored all the node IDs for each way.
if (rel.hasTag("type", "multipolygon")) {
List<RelationMember<OsmRelationInfo>> parentRelations = getRelationMembershipForWay(rel.getId());
return new MultipolygonSourceFeature(rel, nodeLocations, parentRelations);
} else {
return null;
}
}
private List<RelationMember<OsmRelationInfo>> getRelationMembershipForWay(long wayId) {
LongArrayList relationIds = wayToRelations.get(wayId);
List<RelationMember<OsmRelationInfo>> rels = null;
2021-05-31 10:21:53 +00:00
if (!relationIds.isEmpty()) {
rels = new ArrayList<>(relationIds.size());
for (int r = 0; r < relationIds.size(); r++) {
2021-06-18 11:21:43 +00:00
long encoded = relationIds.get(r);
2021-09-10 00:46:20 +00:00
// encoded ID uses the upper few bits of the long to encode the role
RelationMembership parsed = decodeRelationMembership(encoded);
OsmRelationInfo rel = relationInfo.get(parsed.relationId);
2021-05-31 10:21:53 +00:00
if (rel != null) {
2021-06-18 11:21:43 +00:00
rels.add(new RelationMember<>(parsed.role, rel));
2021-05-31 10:21:53 +00:00
}
}
}
2021-06-24 09:16:30 +00:00
return rels;
2021-05-28 10:08:13 +00:00
}
2021-05-04 12:02:22 +00:00
@Override
public long estimateMemoryUsageBytes() {
long size = 0;
2021-09-10 00:46:20 +00:00
size += estimateSize(waysInMultipolygon);
size += estimateSize(multipolygonWayGeometries);
size += estimateSize(wayToRelations);
size += estimateSizeWithoutValues(relationInfo);
size += MemoryEstimator.estimateSizeWithoutValues(roleIdsReverse);
size += estimateSizeWithoutKeys(roleIds);
2021-06-18 11:21:43 +00:00
size += roleSizes.get();
2021-05-04 12:02:22 +00:00
size += relationInfoSizes.get();
return size;
2021-04-10 09:25:42 +00:00
}
2021-04-12 10:05:32 +00:00
@Override
2021-04-12 10:59:34 +00:00
public void close() throws IOException {
2021-04-12 10:05:32 +00:00
multipolygonWayGeometries = null;
wayToRelations = null;
waysInMultipolygon = null;
relationInfo = null;
2021-09-10 00:46:20 +00:00
nodeLocationDb.close();
2021-06-18 11:21:43 +00:00
roleIds.release();
roleIdsReverse.release();
2021-04-10 09:25:42 +00:00
}
2021-09-10 00:46:20 +00:00
NodeLocationProvider newNodeLocationProvider() {
return new NodeDbLocationProvider();
}
2021-06-18 11:21:43 +00:00
2021-09-10 00:46:20 +00:00
public interface NodeLocationProvider {
2021-06-18 11:21:43 +00:00
2021-09-10 00:46:20 +00:00
default CoordinateSequence getWayGeometry(LongArrayList nodeIds) {
CoordinateList coordList = new CoordinateList();
for (var cursor : nodeIds) {
coordList.add(getCoordinate(cursor.value));
2021-06-18 11:21:43 +00:00
}
2021-09-10 00:46:20 +00:00
return new CoordinateArraySequence(coordList.toCoordinateArray());
2021-06-18 11:21:43 +00:00
}
2021-09-10 00:46:20 +00:00
Coordinate getCoordinate(long id);
}
2021-04-10 09:25:42 +00:00
2021-09-10 00:46:20 +00:00
/**
* Member of a relation extracted from OSM input data.
*
* @param <T> type of the user-defined class storing information about the relation
* @param role "role" of the relation member
* @param relation user-provided data about the relation from pass1
*/
public static record RelationMember<T extends OsmRelationInfo>(String role, T relation) {}
/** Raw relation membership data that gets encoded/decoded into a long. */
private record RelationMembership(String role, long relationId) {}
/** Returns the role and relation ID packed into a long. */
private RelationMembership decodeRelationMembership(long encoded) {
int role = (int) ((encoded >>> ROLE_SHIFT) & ROLE_MASK);
return new RelationMembership(roleIdsReverse.get(role), encoded & NOT_ROLE_MASK);
}
2021-06-18 11:21:43 +00:00
2021-09-10 00:46:20 +00:00
/** Packs a string role and relation into a compact long for storage. */
private long encodeRelationMembership(String role, long relationId) {
int roleId = roleIds.getOrDefault(role, -1);
if (roleId == -1) {
roleSizes.addAndGet(estimateSize(role));
roleId = roleIds.size() + 1;
roleIds.put(role, roleId);
roleIdsReverse.put(roleId, role);
if (roleId > MAX_ROLES) {
throw new IllegalStateException("Too many roles to encode: " + role);
}
2021-04-12 10:05:32 +00:00
}
2021-09-10 00:46:20 +00:00
return relationId | ((long) roleId << ROLE_SHIFT);
2021-04-12 10:05:32 +00:00
}
2021-09-10 00:46:20 +00:00
/**
* A source feature generated from OSM elements. Inferring the geometry can be expensive, so each sublass is
* constructed with the inputs necessary to create the geometry, but the geometry is constructed lazily on read.
*/
private abstract class OsmFeature extends SourceFeature {
2021-04-23 11:26:02 +00:00
2021-09-10 00:46:20 +00:00
private final boolean polygon;
private final boolean line;
private final boolean point;
private Geometry latLonGeom;
private Geometry worldGeom;
2021-05-25 10:05:41 +00:00
2021-09-10 00:46:20 +00:00
public OsmFeature(ReaderElement elem, boolean point, boolean line, boolean polygon,
List<RelationMember<OsmRelationInfo>> relationInfo) {
2021-08-14 09:55:00 +00:00
super(ReaderElementUtils.getTags(elem), name, null, relationInfo, elem.getId());
2021-05-25 10:05:41 +00:00
this.point = point;
this.line = line;
this.polygon = polygon;
2021-04-23 11:26:02 +00:00
}
@Override
2021-05-28 10:08:13 +00:00
public Geometry latLonGeometry() throws GeometryException {
return latLonGeom != null ? latLonGeom : (latLonGeom = GeoUtils.worldToLatLonCoords(worldGeometry()));
2021-05-13 10:25:06 +00:00
}
@Override
2021-05-28 10:08:13 +00:00
public Geometry worldGeometry() throws GeometryException {
2021-05-13 10:25:06 +00:00
return worldGeom != null ? worldGeom : (worldGeom = computeWorldGeometry());
2021-04-23 11:26:02 +00:00
}
2021-05-13 10:25:06 +00:00
2021-05-28 10:08:13 +00:00
protected abstract Geometry computeWorldGeometry() throws GeometryException;
2021-05-25 10:05:41 +00:00
@Override
public boolean isPoint() {
return point;
}
@Override
public boolean canBeLine() {
return line;
}
@Override
public boolean canBePolygon() {
return polygon;
}
2021-04-23 11:26:02 +00:00
}
2021-09-10 00:46:20 +00:00
/** A {@link Point} created from an OSM node. */
private class NodeSourceFeature extends OsmFeature {
2021-04-12 10:05:32 +00:00
2021-05-13 10:25:06 +00:00
private final double lon;
private final double lat;
NodeSourceFeature(ReaderNode node) {
2021-05-31 10:21:53 +00:00
super(node, true, false, false, null);
2021-05-13 10:25:06 +00:00
this.lon = node.getLon();
this.lat = node.getLat();
2021-04-12 10:05:32 +00:00
}
@Override
2021-05-13 10:25:06 +00:00
protected Geometry computeWorldGeometry() {
2021-05-16 10:42:57 +00:00
return GeoUtils.point(
2021-05-13 10:25:06 +00:00
GeoUtils.getWorldX(lon),
GeoUtils.getWorldY(lat)
2021-05-16 10:42:57 +00:00
);
2021-05-13 10:25:06 +00:00
}
@Override
public boolean isPoint() {
return true;
2021-04-12 10:05:32 +00:00
}
2021-05-30 12:02:38 +00:00
@Override
public String toString() {
2021-06-07 11:46:03 +00:00
return "OsmNode[" + id() + ']';
2021-05-30 12:02:38 +00:00
}
2021-04-12 10:05:32 +00:00
}
2021-09-10 00:46:20 +00:00
/**
* A {@link LineString} or {@link Polygon} created from an OSM way.
* <p>
* Unclosed rings are always interpreted as linestrings. Closed rings can be interpreted as either a polygon or a
* linestring unless {@code area=yes} tag prevents them from being a linestring or {@code area=no} tag prevents them
* from being a polygon.
*/
private class WaySourceFeature extends OsmFeature {
2021-04-12 10:05:32 +00:00
2021-09-10 00:46:20 +00:00
private final NodeLocationProvider nodeLocations;
2021-05-13 10:25:06 +00:00
private final LongArrayList nodeIds;
2021-09-10 00:46:20 +00:00
public WaySourceFeature(ReaderWay way, boolean closed, String area, NodeLocationProvider nodeLocations,
List<RelationMember<OsmRelationInfo>> relationInfo) {
2021-05-28 10:08:13 +00:00
super(way, false,
2021-09-10 00:46:20 +00:00
(!closed || !"yes".equals(area)) && way.getNodes().size() >= 2, // line
(closed && !"no".equals(area)) && way.getNodes().size() >= 4, // polygon
2021-05-31 10:21:53 +00:00
relationInfo
2021-05-28 10:08:13 +00:00
);
2021-05-13 10:25:06 +00:00
this.nodeIds = way.getNodes();
2021-09-10 00:46:20 +00:00
this.nodeLocations = nodeLocations;
2021-04-12 10:05:32 +00:00
}
@Override
2021-05-28 10:08:13 +00:00
protected Geometry computeLine() throws GeometryException {
try {
2021-09-10 00:46:20 +00:00
CoordinateSequence coords = nodeLocations.getWayGeometry(nodeIds);
2021-05-28 10:08:13 +00:00
return GeoUtils.JTS_FACTORY.createLineString(coords);
} catch (IllegalArgumentException e) {
2021-06-07 11:46:03 +00:00
throw new GeometryException("osm_invalid_line", "Error building line for way " + id() + ": " + e);
2021-05-28 10:08:13 +00:00
}
}
@Override
protected Geometry computePolygon() throws GeometryException {
try {
2021-09-10 00:46:20 +00:00
CoordinateSequence coords = nodeLocations.getWayGeometry(nodeIds);
2021-05-28 10:08:13 +00:00
return GeoUtils.JTS_FACTORY.createPolygon(coords);
} catch (IllegalArgumentException e) {
2021-06-07 11:46:03 +00:00
throw new GeometryException("osm_invalid_polygon", "Error building polygon for way " + id() + ": " + e);
2021-05-28 10:08:13 +00:00
}
}
@Override
protected Geometry computeWorldGeometry() throws GeometryException {
return canBePolygon() ? polygon() : line();
2021-04-12 10:05:32 +00:00
}
2021-05-30 12:02:38 +00:00
@Override
public String toString() {
2021-06-07 11:46:03 +00:00
return "OsmWay[" + id() + ']';
2021-05-30 12:02:38 +00:00
}
2021-04-12 10:05:32 +00:00
}
2021-09-10 00:46:20 +00:00
/**
* A {@link MultiPolygon} created from an OSM relation where {@code type=multipolygon}.
* <p>
* Delegates complex reconstruction work to {@link OsmMultipolygon}.
*/
private class MultipolygonSourceFeature extends OsmFeature {
2021-04-12 10:05:32 +00:00
2021-05-30 11:42:06 +00:00
private final ReaderRelation relation;
2021-09-10 00:46:20 +00:00
private final NodeLocationProvider nodeLocations;
2021-04-12 10:05:32 +00:00
2021-09-10 00:46:20 +00:00
public MultipolygonSourceFeature(ReaderRelation relation, NodeLocationProvider nodeLocations,
List<RelationMember<OsmRelationInfo>> parentRelations) {
2021-06-24 09:16:30 +00:00
super(relation, false, false, true, parentRelations);
2021-05-30 11:42:06 +00:00
this.relation = relation;
2021-09-10 00:46:20 +00:00
this.nodeLocations = nodeLocations;
2021-04-12 10:05:32 +00:00
}
2021-05-13 10:25:06 +00:00
@Override
2021-05-30 11:42:06 +00:00
protected Geometry computeWorldGeometry() throws GeometryException {
List<LongArrayList> rings = new ArrayList<>(relation.getMembers().size());
for (ReaderRelation.Member member : relation.getMembers()) {
2021-05-30 12:14:07 +00:00
String role = member.getRole();
2021-05-30 11:42:06 +00:00
LongArrayList poly = multipolygonWayGeometries.get(member.getRef());
2021-07-17 10:16:03 +00:00
if (member.getType() == ReaderRelation.Member.WAY) {
2021-05-30 12:14:07 +00:00
if (poly != null && !poly.isEmpty()) {
rings.add(poly);
} else {
LOGGER.warn("Missing " + role + " OsmWay[" + member.getRef() + "] for multipolygon " + this);
}
2021-05-30 11:42:06 +00:00
}
}
2021-09-10 00:46:20 +00:00
return OsmMultipolygon.build(rings, nodeLocations, id());
2021-05-13 10:25:06 +00:00
}
2021-05-30 12:02:38 +00:00
@Override
public String toString() {
2021-06-07 11:46:03 +00:00
return "OsmRelation[" + id() + ']';
2021-05-30 12:02:38 +00:00
}
2021-05-13 10:25:06 +00:00
}
2021-09-10 00:46:20 +00:00
/**
* A thin layer on top of {@link LongLongMap} that decodes node locations stored as {@code long} values.
*/
private class NodeDbLocationProvider implements NodeLocationProvider {
2021-05-13 10:25:06 +00:00
2021-05-30 11:42:06 +00:00
@Override
public Coordinate getCoordinate(long id) {
2021-09-10 00:46:20 +00:00
long encoded = nodeLocationDb.get(id);
2021-07-22 09:20:23 +00:00
if (encoded == LongLongMap.MISSING_VALUE) {
throw new IllegalArgumentException("Missing location for node: " + id);
2021-05-30 11:42:06 +00:00
}
2021-07-22 09:20:23 +00:00
return new CoordinateXY(GeoUtils.decodeWorldX(encoded), GeoUtils.decodeWorldY(encoded));
2021-05-30 11:42:06 +00:00
}
@Override
2021-05-13 10:25:06 +00:00
public CoordinateSequence getWayGeometry(LongArrayList nodeIds) {
int num = nodeIds.size();
CoordinateSequence seq = new PackedCoordinateSequence.Double(nodeIds.size(), 2, 0);
for (int i = 0; i < num; i++) {
2021-09-10 00:46:20 +00:00
long encoded = nodeLocationDb.get(nodeIds.get(i));
2021-07-22 09:20:23 +00:00
if (encoded == LongLongMap.MISSING_VALUE) {
throw new IllegalArgumentException("Missing location for node: " + nodeIds.get(i));
2021-05-13 10:25:06 +00:00
}
2021-07-22 09:20:23 +00:00
seq.setOrdinate(i, 0, GeoUtils.decodeWorldX(encoded));
seq.setOrdinate(i, 1, GeoUtils.decodeWorldY(encoded));
2021-05-13 10:25:06 +00:00
}
return seq;
}
2021-07-22 09:20:23 +00:00
}
2021-04-10 09:25:42 +00:00
}