diff --git a/src/main/java/com/onthegomap/flatmap/OpenMapTilesMain.java b/src/main/java/com/onthegomap/flatmap/OpenMapTilesMain.java index 4bee883a..393c46da 100644 --- a/src/main/java/com/onthegomap/flatmap/OpenMapTilesMain.java +++ b/src/main/java/com/onthegomap/flatmap/OpenMapTilesMain.java @@ -92,7 +92,8 @@ public class OpenMapTilesMain { profile, stats) ); - try (var osmReader = new OpenStreetMapReader(osmInputFile, nodeLocations, profile, stats)) { + try (var osmReader = new OpenStreetMapReader(OpenMapTilesProfile.OSM_SOURCE, osmInputFile, nodeLocations, profile, + stats)) { stats.time("osm_pass1", () -> osmReader.pass1(config)); stats.time("osm_pass2", () -> osmReader.pass2(featureMap, config)); } diff --git a/src/main/java/com/onthegomap/flatmap/SourceFeature.java b/src/main/java/com/onthegomap/flatmap/SourceFeature.java index ce936641..e44e3aec 100644 --- a/src/main/java/com/onthegomap/flatmap/SourceFeature.java +++ b/src/main/java/com/onthegomap/flatmap/SourceFeature.java @@ -50,24 +50,33 @@ public abstract class SourceFeature { private Geometry linearGeometry = null; - public Geometry line() throws GeometryException { + protected Geometry computeLine() throws GeometryException { + Geometry world = worldGeometry(); + return world instanceof Lineal ? world : GeoUtils.polygonToLineString(world); + } + + public final Geometry line() throws GeometryException { if (!canBeLine()) { throw new GeometryException("cannot be line"); } if (linearGeometry == null) { - Geometry world = worldGeometry(); - linearGeometry = world instanceof Lineal ? world : GeoUtils.polygonToLineString(world); + linearGeometry = computeLine(); } return linearGeometry; } + private Geometry polygonGeometry = null; - public Geometry polygon() throws GeometryException { + protected Geometry computePolygon() throws GeometryException { + return worldGeometry(); + } + + public final Geometry polygon() throws GeometryException { if (!canBePolygon()) { throw new GeometryException("cannot be polygon"); } - return polygonGeometry != null ? polygonGeometry : (polygonGeometry = worldGeometry()); + return polygonGeometry != null ? polygonGeometry : (polygonGeometry = computePolygon()); } private double area = Double.NaN; @@ -79,7 +88,8 @@ public abstract class SourceFeature { private double length = Double.NaN; public double length() throws GeometryException { - return Double.isNaN(length) ? (length = worldGeometry().getLength()) : length; + return Double.isNaN(length) ? (length = + (isPoint() || canBePolygon() || canBeLine()) ? worldGeometry().getLength() : 0) : length; } public Object getTag(String key) { diff --git a/src/main/java/com/onthegomap/flatmap/collections/LongLongMap.java b/src/main/java/com/onthegomap/flatmap/collections/LongLongMap.java index bff92001..6d368833 100644 --- a/src/main/java/com/onthegomap/flatmap/collections/LongLongMap.java +++ b/src/main/java/com/onthegomap/flatmap/collections/LongLongMap.java @@ -1,5 +1,6 @@ package com.onthegomap.flatmap.collections; +import com.graphhopper.coll.GHLongLongHashMap; import com.onthegomap.flatmap.FileUtils; import java.io.Closeable; import java.io.IOException; @@ -14,6 +15,8 @@ import org.mapdb.volume.Volume; public interface LongLongMap extends Closeable { + long MISSING_VALUE = Long.MIN_VALUE; + void put(long key, long value); long get(long key); @@ -52,6 +55,34 @@ public interface LongLongMap extends Closeable { return new MapdbSortedTable(volume, () -> 0); } + static LongLongMap newInMemoryHashMap() { + return new HppcMap(); + } + + class HppcMap implements LongLongMap { + + private final com.carrotsearch.hppc.LongLongMap underlying = new GHLongLongHashMap(); + + @Override + public void put(long key, long value) { + underlying.put(key, value); + } + + @Override + public long get(long key) { + return underlying.getOrDefault(key, MISSING_VALUE); + } + + @Override + public long fileSize() { + return 0; + } + + @Override + public void close() throws IOException { + } + } + class MapdbSortedTable implements LongLongMap { private final SortedTableMap.Sink mapSink; @@ -88,7 +119,7 @@ public interface LongLongMap extends Closeable { @Override public long get(long key) { - return getMap().getOrDefault(key, Long.MIN_VALUE); + return getMap().getOrDefault(key, MISSING_VALUE); } @Override diff --git a/src/main/java/com/onthegomap/flatmap/profiles/OpenMapTilesProfile.java b/src/main/java/com/onthegomap/flatmap/profiles/OpenMapTilesProfile.java index 07f054e6..a3ca6537 100644 --- a/src/main/java/com/onthegomap/flatmap/profiles/OpenMapTilesProfile.java +++ b/src/main/java/com/onthegomap/flatmap/profiles/OpenMapTilesProfile.java @@ -15,6 +15,7 @@ public class OpenMapTilesProfile implements Profile { public static final String LAKE_CENTERLINE_SOURCE = "lake_centerlines"; public static final String WATER_POLYGON_SOURCE = "water_polygons"; public static final String NATURAL_EARTH_SOURCE = "natural_earth"; + public static final String OSM_SOURCE = "osm"; private static final Logger LOGGER = LoggerFactory.getLogger(OpenMapTilesProfile.class); @Override diff --git a/src/main/java/com/onthegomap/flatmap/read/OpenStreetMapReader.java b/src/main/java/com/onthegomap/flatmap/read/OpenStreetMapReader.java index df127bcf..791e921d 100644 --- a/src/main/java/com/onthegomap/flatmap/read/OpenStreetMapReader.java +++ b/src/main/java/com/onthegomap/flatmap/read/OpenStreetMapReader.java @@ -20,6 +20,7 @@ import com.onthegomap.flatmap.collections.FeatureSort; import com.onthegomap.flatmap.collections.LongLongMap; import com.onthegomap.flatmap.collections.LongLongMultimap; import com.onthegomap.flatmap.geo.GeoUtils; +import com.onthegomap.flatmap.geo.GeometryException; import com.onthegomap.flatmap.monitoring.ProgressLoggers; import com.onthegomap.flatmap.monitoring.Stats; import com.onthegomap.flatmap.render.FeatureRenderer; @@ -35,13 +36,14 @@ import org.locationtech.jts.geom.impl.PackedCoordinateSequence; public class OpenStreetMapReader implements Closeable, MemoryEstimator.HasEstimate { - private final OsmInputFile osmInputFile; + private final OsmSource osmInputFile; private final Stats stats; private final LongLongMap nodeDb; - private final AtomicLong TOTAL_NODES = new AtomicLong(0); - private final AtomicLong TOTAL_WAYS = new AtomicLong(0); - private final AtomicLong TOTAL_RELATIONS = new AtomicLong(0); + private final AtomicLong PASS1_NODES = new AtomicLong(0); + private final AtomicLong PASS1_WAYS = new AtomicLong(0); + private final AtomicLong PASS1_RELATIONS = new AtomicLong(0); private final Profile profile; + private final String name; // 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) @@ -57,7 +59,13 @@ public class OpenStreetMapReader implements Closeable, MemoryEstimator.HasEstima // ~7GB private LongLongMultimap multipolygonWayGeometries = LongLongMultimap.newDensedOrderedMultimap(); - public OpenStreetMapReader(OsmInputFile osmInputFile, LongLongMap nodeDb, Profile profile, Stats stats) { + public OpenStreetMapReader(OsmSource osmInputFile, LongLongMap nodeDb, Profile profile, Stats stats) { + this("osm", osmInputFile, nodeDb, profile, stats); + } + + public OpenStreetMapReader(String name, OsmSource osmInputFile, LongLongMap nodeDb, Profile profile, + Stats stats) { + this.name = name; this.osmInputFile = osmInputFile; this.nodeDb = nodeDb; this.stats = stats; @@ -68,41 +76,13 @@ public class OpenStreetMapReader implements Closeable, MemoryEstimator.HasEstima var topology = Topology.start("osm_pass1", stats) .fromGenerator("pbf", osmInputFile.read(config.threads() - 1)) .addBuffer("reader_queue", 50_000, 10_000) - .sinkToConsumer("process", 1, (readerElement) -> { - if (readerElement instanceof ReaderNode node) { - TOTAL_NODES.incrementAndGet(); - nodeDb.put(node.getId(), GeoUtils.encodeFlatLocation(node.getLon(), node.getLat())); - } else if (readerElement instanceof ReaderWay) { - TOTAL_WAYS.incrementAndGet(); - } else if (readerElement instanceof ReaderRelation rel) { - TOTAL_RELATIONS.incrementAndGet(); - List infos = profile.preprocessOsmRelation(rel); - if (infos != null) { - for (RelationInfo info : infos) { - relationInfo.put(rel.getId(), info); - relationInfoSizes.addAndGet(info.estimateMemoryUsageBytes()); - for (ReaderRelation.Member member : rel.getMembers()) { - if (member.getType() == ReaderRelation.Member.WAY) { - wayToRelations.put(member.getRef(), rel.getId()); - } - } - } - } - if (rel.hasTag("type", "multipolygon")) { - for (ReaderRelation.Member member : rel.getMembers()) { - if (member.getType() == ReaderRelation.Member.WAY) { - waysInMultipolygon.add(member.getRef()); - } - } - } - } - }); + .sinkToConsumer("process", 1, this::processPass1); var loggers = new ProgressLoggers("osm_pass1") - .addRateCounter("nodes", TOTAL_NODES) + .addRateCounter("nodes", PASS1_NODES) .addFileSize(nodeDb::fileSize) - .addRateCounter("ways", TOTAL_WAYS) - .addRateCounter("rels", TOTAL_RELATIONS) + .addRateCounter("ways", PASS1_WAYS) + .addRateCounter("rels", PASS1_RELATIONS) .addProcessStats() .addInMemoryObject("hppc", this) .addThreadPoolStats("parse", "pool-") @@ -110,6 +90,36 @@ public class OpenStreetMapReader implements Closeable, MemoryEstimator.HasEstima topology.awaitAndLog(loggers, config.logInterval()); } + void processPass1(ReaderElement readerElement) { + if (readerElement instanceof ReaderNode node) { + PASS1_NODES.incrementAndGet(); + nodeDb.put(node.getId(), GeoUtils.encodeFlatLocation(node.getLon(), node.getLat())); + } else if (readerElement instanceof ReaderWay) { + PASS1_WAYS.incrementAndGet(); + } else if (readerElement instanceof ReaderRelation rel) { + PASS1_RELATIONS.incrementAndGet(); + List infos = profile.preprocessOsmRelation(rel); + if (infos != null) { + for (RelationInfo info : infos) { + relationInfo.put(rel.getId(), info); + relationInfoSizes.addAndGet(info.estimateMemoryUsageBytes()); + for (ReaderRelation.Member member : rel.getMembers()) { + if (member.getType() == ReaderRelation.Member.WAY) { + wayToRelations.put(member.getRef(), rel.getId()); + } + } + } + } + if (rel.hasTag("type", "multipolygon")) { + for (ReaderRelation.Member member : rel.getMembers()) { + if (member.getType() == ReaderRelation.Member.WAY) { + waysInMultipolygon.add(member.getRef()); + } + } + } + } + } + public void pass2(FeatureGroup writer, CommonParams config) { int readerThreads = Math.max(config.threads() / 4, 1); int processThreads = config.threads() - 1; @@ -124,7 +134,7 @@ public class OpenStreetMapReader implements Closeable, MemoryEstimator.HasEstima .addWorker("process", processThreads, (prev, next) -> { ReaderElement readerElement; var featureCollectors = new FeatureCollector.Factory(config); - NodeGeometryCache nodeCache = new NodeGeometryCache(); + NodeGeometryCache nodeCache = newNodeGeometryCache(); var encoder = writer.newRenderedFeatureEncoder(); FeatureRenderer renderer = new FeatureRenderer( config, @@ -134,13 +144,10 @@ public class OpenStreetMapReader implements Closeable, MemoryEstimator.HasEstima SourceFeature feature = null; if (readerElement instanceof ReaderNode node) { nodesProcessed.incrementAndGet(); - feature = new NodeSourceFeature(node); + feature = processNodePass2(node); } else if (readerElement instanceof ReaderWay way) { waysProcessed.incrementAndGet(); - LongArrayList nodes = way.getNodes(); - boolean closed = nodes.size() > 1 && nodes.get(0) == nodes.get(nodes.size() - 1); - String area = way.getTag("area"); - feature = new WaySourceFeature(way, closed, area, nodeCache); + feature = processWayPass2(nodeCache, way); } else if (readerElement instanceof ReaderRelation rel) { // ensure all ways finished processing before we start relations if (waysDone.getCount() > 0) { @@ -148,15 +155,13 @@ public class OpenStreetMapReader implements Closeable, MemoryEstimator.HasEstima waysDone.await(); } relsProcessed.incrementAndGet(); - if (rel.hasTag("type", "multipolygon")) { - feature = new MultipolygonSourceFeature(rel); - } + feature = processRelationPass2(rel); } if (feature != null) { FeatureCollector features = featureCollectors.get(feature); profile.processFeature(feature, features); for (FeatureCollector.Feature renderable : features) { - renderer.renderFeature(renderable); + renderer.accept(renderable); } } nodeCache.reset(); @@ -168,10 +173,10 @@ public class OpenStreetMapReader implements Closeable, MemoryEstimator.HasEstima .sinkToConsumer("write", 1, writer); var logger = new ProgressLoggers("osm_pass2") - .addRatePercentCounter("nodes", TOTAL_NODES.get(), nodesProcessed) + .addRatePercentCounter("nodes", PASS1_NODES.get(), nodesProcessed) .addFileSize(nodeDb::fileSize) - .addRatePercentCounter("ways", TOTAL_WAYS.get(), waysProcessed) - .addRatePercentCounter("rels", TOTAL_RELATIONS.get(), relsProcessed) + .addRatePercentCounter("ways", PASS1_WAYS.get(), waysProcessed) + .addRatePercentCounter("rels", PASS1_RELATIONS.get(), relsProcessed) .addRateCounter("features", () -> writer.sorter().size()) .addFileSize(writer::getStorageSize) .addProcessStats() @@ -182,6 +187,21 @@ public class OpenStreetMapReader implements Closeable, MemoryEstimator.HasEstima topology.awaitAndLog(logger, config.logInterval()); } + SourceFeature processRelationPass2(ReaderRelation rel) { + return rel.hasTag("type", "multipolygon") ? new MultipolygonSourceFeature(rel) : null; + } + + SourceFeature processWayPass2(NodeGeometryCache nodeCache, ReaderWay way) { + LongArrayList nodes = way.getNodes(); + boolean closed = nodes.size() > 1 && nodes.get(0) == nodes.get(nodes.size() - 1); + String area = way.getTag("area"); + return new WaySourceFeature(way, closed, area, nodeCache); + } + + SourceFeature processNodePass2(ReaderNode node) { + return new NodeSourceFeature(node); + } + @Override public long estimateMemoryUsageBytes() { long size = 0; @@ -211,34 +231,36 @@ public class OpenStreetMapReader implements Closeable, MemoryEstimator.HasEstima } } - private static abstract class ProxyFeature extends SourceFeature { + private abstract class ProxyFeature extends SourceFeature { - private final boolean polygon; - private final boolean line; - private final boolean point; + final boolean polygon; + final boolean line; + final boolean point; + final long osmId; public ProxyFeature(ReaderElement elem, boolean point, boolean line, boolean polygon) { - super(ReaderElementUtils.getProperties(elem), null, null); + super(ReaderElementUtils.getProperties(elem), name, null); this.point = point; this.line = line; this.polygon = polygon; + this.osmId = elem.getId(); } private Geometry latLonGeom; @Override - public Geometry latLonGeometry() { - return latLonGeom != null ? latLonGeom : (latLonGeom = GeoUtils.latLonToWorldCoords(worldGeometry())); + public Geometry latLonGeometry() throws GeometryException { + return latLonGeom != null ? latLonGeom : (latLonGeom = GeoUtils.worldToLatLonCoords(worldGeometry())); } private Geometry worldGeom; @Override - public Geometry worldGeometry() { + public Geometry worldGeometry() throws GeometryException { return worldGeom != null ? worldGeom : (worldGeom = computeWorldGeometry()); } - protected abstract Geometry computeWorldGeometry(); + protected abstract Geometry computeWorldGeometry() throws GeometryException; @Override public boolean isPoint() { @@ -256,7 +278,7 @@ public class OpenStreetMapReader implements Closeable, MemoryEstimator.HasEstima } } - private static class NodeSourceFeature extends ProxyFeature { + private class NodeSourceFeature extends ProxyFeature { private final double lon; private final double lat; @@ -281,20 +303,43 @@ public class OpenStreetMapReader implements Closeable, MemoryEstimator.HasEstima } } - private static class WaySourceFeature extends ProxyFeature { + private class WaySourceFeature extends ProxyFeature { private final NodeGeometryCache nodeCache; private final LongArrayList nodeIds; public WaySourceFeature(ReaderWay way, boolean closed, String area, NodeGeometryCache nodeCache) { - super(way, false, !closed || !"yes".equals(area), closed && !"no".equals(area)); + super(way, false, + (!closed || !"yes".equals(area)) && way.getNodes().size() >= 2, + (closed && !"no".equals(area)) && way.getNodes().size() >= 4 + ); this.nodeIds = way.getNodes(); this.nodeCache = nodeCache; } @Override - protected Geometry computeWorldGeometry() { - return null; + protected Geometry computeLine() throws GeometryException { + try { + CoordinateSequence coords = nodeCache.getWayGeometry(nodeIds); + return GeoUtils.JTS_FACTORY.createLineString(coords); + } catch (IllegalArgumentException e) { + throw new GeometryException("Error building line for way " + osmId + ": " + e); + } + } + + @Override + protected Geometry computePolygon() throws GeometryException { + try { + CoordinateSequence coords = nodeCache.getWayGeometry(nodeIds); + return GeoUtils.JTS_FACTORY.createPolygon(coords); + } catch (IllegalArgumentException e) { + throw new GeometryException("Error building polygon for way " + osmId + ": " + e); + } + } + + @Override + protected Geometry computeWorldGeometry() throws GeometryException { + return canBePolygon() ? polygon() : line(); } @Override @@ -303,7 +348,7 @@ public class OpenStreetMapReader implements Closeable, MemoryEstimator.HasEstima } } - private static class MultipolygonSourceFeature extends ProxyFeature { + private class MultipolygonSourceFeature extends ProxyFeature { public MultipolygonSourceFeature(ReaderRelation relation) { super(relation, false, false, true); @@ -320,7 +365,11 @@ public class OpenStreetMapReader implements Closeable, MemoryEstimator.HasEstima } } - private class NodeGeometryCache { + NodeGeometryCache newNodeGeometryCache() { + return new NodeGeometryCache(); + } + + class NodeGeometryCache { private final LongDoubleHashMap xs = new LongDoubleHashMap(); private final LongDoubleHashMap ys = new LongDoubleHashMap(); @@ -335,6 +384,10 @@ public class OpenStreetMapReader implements Closeable, MemoryEstimator.HasEstima worldX = xs.getOrDefault(id, Double.NaN); if (Double.isNaN(worldX)) { long encoded = nodeDb.get(id); + if (encoded == LongLongMap.MISSING_VALUE) { + throw new IllegalArgumentException("Missing location for node: " + id); + } + xs.put(id, worldX = GeoUtils.decodeWorldX(encoded)); ys.put(id, worldY = GeoUtils.decodeWorldY(encoded)); } else { diff --git a/src/main/java/com/onthegomap/flatmap/read/OsmInputFile.java b/src/main/java/com/onthegomap/flatmap/read/OsmInputFile.java index b7fd9bf8..6bf69d13 100644 --- a/src/main/java/com/onthegomap/flatmap/read/OsmInputFile.java +++ b/src/main/java/com/onthegomap/flatmap/read/OsmInputFile.java @@ -24,7 +24,7 @@ import org.openstreetmap.osmosis.osmbinary.Osmformat.HeaderBBox; import org.openstreetmap.osmosis.osmbinary.Osmformat.HeaderBlock; import org.openstreetmap.osmosis.osmbinary.file.FileFormatException; -public class OsmInputFile implements BoundsProvider { +public class OsmInputFile implements BoundsProvider, OsmSource { private final Path path; @@ -83,6 +83,7 @@ public class OsmInputFile implements BoundsProvider { } } + @Override public Topology.SourceStep read(int threads) { return next -> readTo(next, threads); } diff --git a/src/main/java/com/onthegomap/flatmap/read/OsmSource.java b/src/main/java/com/onthegomap/flatmap/read/OsmSource.java new file mode 100644 index 00000000..757b06be --- /dev/null +++ b/src/main/java/com/onthegomap/flatmap/read/OsmSource.java @@ -0,0 +1,9 @@ +package com.onthegomap.flatmap.read; + +import com.graphhopper.reader.ReaderElement; +import com.onthegomap.flatmap.worker.Topology; + +public interface OsmSource { + + Topology.SourceStep read(int threads); +} diff --git a/src/main/java/com/onthegomap/flatmap/read/Reader.java b/src/main/java/com/onthegomap/flatmap/read/Reader.java index a851653c..b2feb7ae 100644 --- a/src/main/java/com/onthegomap/flatmap/read/Reader.java +++ b/src/main/java/com/onthegomap/flatmap/read/Reader.java @@ -50,7 +50,7 @@ public abstract class Reader implements Closeable { if (sourceFeature.latLonGeometry().getEnvelopeInternal().intersects(latLonBounds)) { profile.processFeature(sourceFeature, features); for (FeatureCollector.Feature renderable : features) { - renderer.renderFeature(renderable); + renderer.accept(renderable); } } } diff --git a/src/main/java/com/onthegomap/flatmap/render/FeatureRenderer.java b/src/main/java/com/onthegomap/flatmap/render/FeatureRenderer.java index bd7ec2c7..d676cc95 100644 --- a/src/main/java/com/onthegomap/flatmap/render/FeatureRenderer.java +++ b/src/main/java/com/onthegomap/flatmap/render/FeatureRenderer.java @@ -30,7 +30,7 @@ import org.locationtech.jts.simplify.DouglasPeuckerSimplifier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class FeatureRenderer { +public class FeatureRenderer implements Consumer { private static final AtomicLong idGen = new AtomicLong(0); @@ -52,7 +52,8 @@ public class FeatureRenderer { this.consumer = consumer; } - public void renderFeature(FeatureCollector.Feature feature) { + @Override + public void accept(FeatureCollector.Feature feature) { renderGeometry(feature.getGeometry(), feature); } diff --git a/src/main/java/com/onthegomap/flatmap/worker/Topology.java b/src/main/java/com/onthegomap/flatmap/worker/Topology.java index 04cc5ca4..77260706 100644 --- a/src/main/java/com/onthegomap/flatmap/worker/Topology.java +++ b/src/main/java/com/onthegomap/flatmap/worker/Topology.java @@ -160,4 +160,5 @@ public record Topology( }); } } + } diff --git a/src/test/java/com/onthegomap/flatmap/FlatMapTest.java b/src/test/java/com/onthegomap/flatmap/FlatMapTest.java index de7d6168..22bb5b02 100644 --- a/src/test/java/com/onthegomap/flatmap/FlatMapTest.java +++ b/src/test/java/com/onthegomap/flatmap/FlatMapTest.java @@ -5,7 +5,10 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertTrue; +import com.graphhopper.reader.ReaderElement; +import com.graphhopper.reader.ReaderNode; import com.graphhopper.reader.ReaderRelation; +import com.graphhopper.reader.ReaderWay; import com.onthegomap.flatmap.collections.FeatureGroup; import com.onthegomap.flatmap.collections.FeatureSort; import com.onthegomap.flatmap.collections.LongLongMap; @@ -14,6 +17,7 @@ import com.onthegomap.flatmap.geo.TileCoord; import com.onthegomap.flatmap.monitoring.Stats; import com.onthegomap.flatmap.profiles.OpenMapTilesProfile; import com.onthegomap.flatmap.read.OpenStreetMapReader; +import com.onthegomap.flatmap.read.OsmSource; import com.onthegomap.flatmap.read.Reader; import com.onthegomap.flatmap.read.ReaderFeature; import com.onthegomap.flatmap.worker.Topology; @@ -22,10 +26,10 @@ import com.onthegomap.flatmap.write.MbtilesWriter; import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; -import java.sql.SQLException; import java.util.List; import java.util.Map; import java.util.function.BiConsumer; +import java.util.function.Consumer; import java.util.function.Function; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; @@ -75,11 +79,32 @@ public class FlatMapTest { }.process(featureGroup, config); } - private FlatMapResults runWithReaderFeatures( + private void processOsmFeatures(FeatureGroup featureGroup, Profile profile, CommonParams config, + List osmElements) throws IOException { + OsmSource elems = threads -> next -> { + // process the same order they come in from an OSM file + osmElements.stream().filter(e -> e.getType() == ReaderElement.FILEHEADER).forEachOrdered(next); + osmElements.stream().filter(e -> e.getType() == ReaderElement.NODE).forEachOrdered(next); + osmElements.stream().filter(e -> e.getType() == ReaderElement.WAY).forEachOrdered(next); + osmElements.stream().filter(e -> e.getType() == ReaderElement.RELATION).forEachOrdered(next); + }; + var nodeMap = LongLongMap.newInMemorySortedTable(); + try (var reader = new OpenStreetMapReader(elems, nodeMap, profile, new Stats.InMemory())) { + reader.pass1(config); + reader.pass2(featureGroup, config); + } + } + + private interface Runner { + + void run(FeatureGroup featureGroup, Profile profile, CommonParams config) throws Exception; + } + + private FlatMapResults run( Map args, - List features, + Runner runner, BiConsumer profileFunction - ) throws IOException, SQLException { + ) throws Exception { CommonParams config = CommonParams.from(Arguments.of(args)); var translations = Translations.defaultProvider(List.of()); var profile1 = new OpenMapTilesProfile(); @@ -87,7 +112,7 @@ public class FlatMapTest { FeatureSort featureDb = FeatureSort.newInMemory(); FeatureGroup featureGroup = new FeatureGroup(featureDb, profile1, stats); var profile = TestProfile.processSourceFeatures(profileFunction); - processReaderFeatures(featureGroup, profile, config, features); + runner.run(featureGroup, profile, config); featureGroup.sorter().sort(); try (Mbtiles db = Mbtiles.newInMemoryDatabase()) { MbtilesWriter.writeOutput(featureGroup, db, () -> 0L, profile, config, stats); @@ -99,8 +124,32 @@ public class FlatMapTest { } } + private FlatMapResults runWithReaderFeatures( + Map args, + List features, + BiConsumer profileFunction + ) throws Exception { + return run( + args, + (featureGroup, profile, config) -> processReaderFeatures(featureGroup, profile, config, features), + profileFunction + ); + } + + private FlatMapResults runWithOsmElements( + Map args, + List features, + BiConsumer profileFunction + ) throws Exception { + return run( + args, + (featureGroup, profile, config) -> processOsmFeatures(featureGroup, profile, config, features), + profileFunction + ); + } + @Test - public void testMetadataButNoPoints() throws IOException, SQLException { + public void testMetadataButNoPoints() throws Exception { var results = runWithReaderFeatures( Map.of("threads", "1"), List.of(), @@ -132,7 +181,7 @@ public class FlatMapTest { } @Test - public void testSinglePoint() throws IOException, SQLException { + public void testSinglePoint() throws Exception { double x = 0.5 + Z14_WIDTH / 2; double y = 0.5 + Z14_WIDTH / 2; double lat = GeoUtils.getWorldLat(y); @@ -180,7 +229,7 @@ public class FlatMapTest { } @Test - public void testMultiPoint() throws IOException, SQLException { + public void testMultiPoint() throws Exception { double x1 = 0.5 + Z14_WIDTH / 2; double y1 = 0.5 + Z14_WIDTH / 2; double x2 = x1 + Z13_WIDTH / 256d; @@ -231,7 +280,7 @@ public class FlatMapTest { } @Test - public void testLabelGridLimit() throws IOException, SQLException { + public void testLabelGridLimit() throws Exception { double y = 0.5 + Z14_WIDTH / 2; double lat = GeoUtils.getWorldLat(y); @@ -271,7 +320,7 @@ public class FlatMapTest { } @Test - public void testLineString() throws IOException, SQLException { + public void testLineString() throws Exception { double x1 = 0.5 + Z14_WIDTH / 2; double y1 = 0.5 + Z14_WIDTH / 2; double x2 = x1 + Z14_WIDTH; @@ -309,7 +358,7 @@ public class FlatMapTest { } @Test - public void testMultiLineString() throws IOException, SQLException { + public void testMultiLineString() throws Exception { double x1 = 0.5 + Z14_WIDTH / 2; double y1 = 0.5 + Z14_WIDTH / 2; double x2 = x1 + Z14_WIDTH; @@ -368,7 +417,7 @@ public class FlatMapTest { } @Test - public void testPolygonWithHoleSpanningMultipleTiles() throws IOException, SQLException { + public void testPolygonWithHoleSpanningMultipleTiles() throws Exception { List outerPoints = z14CoordinateList( 0.5, 0.5, 3.5, 0.5, @@ -479,7 +528,7 @@ public class FlatMapTest { } @Test - public void testFullWorldPolygon() throws IOException, SQLException { + public void testFullWorldPolygon() throws Exception { List outerPoints = worldCoordinateList( Z14_WIDTH / 2, Z14_WIDTH / 2, 1 - Z14_WIDTH / 2, Z14_WIDTH / 2, @@ -517,7 +566,7 @@ public class FlatMapTest { "njshore.wkb, 10571" }) public void testComplexShorelinePolygons__TAKES_A_MINUTE_OR_TWO(String fileName, int expected) - throws IOException, SQLException, ParseException { + throws Exception, ParseException { MultiPolygon geometry = (MultiPolygon) new WKBReader() .read(new InputStreamInStream(Files.newInputStream(Path.of("src", "test", "resources", fileName)))); assertNotNull(geometry); @@ -539,7 +588,7 @@ public class FlatMapTest { } @Test - public void testReorderNestedMultipolygons() throws IOException, SQLException { + public void testReorderNestedMultipolygons() throws Exception { List outerPoints1 = worldRectangle(10d / 256, 240d / 256); List innerPoints1 = worldRectangle(20d / 256, 230d / 256); List outerPoints2 = worldRectangle(30d / 256, 220d / 256); @@ -576,6 +625,180 @@ public class FlatMapTest { assertEquals(2, multiPolygon.getNumGeometries()); } + @Test + public void testOsmPoint() throws Exception { + var results = runWithOsmElements( + Map.of("threads", "1"), + List.of( + with(new ReaderNode(1, 0, 0), t -> t.setTag("attr", "value")) + ), + (in, features) -> { + if (in.isPoint()) { + features.point("layer") + .setZoomRange(0, 0) + .setAttr("name", "name value") + .inheritFromSource("attr"); + } + } + ); + + assertSubmap(Map.of( + TileCoord.ofXYZ(0, 0, 0), List.of( + feature(newPoint(128, 128), Map.of( + "attr", "value", + "name", "name value" + )) + ) + ), results.tiles); + } + + private static T with(T elem, Consumer fn) { + fn.accept(elem); + return elem; + } + + @Test + public void testOsmLine() throws Exception { + var results = runWithOsmElements( + Map.of("threads", "1"), + List.of( + new ReaderNode(1, 0, 0), + new ReaderNode(2, GeoUtils.getWorldLat(0.75), GeoUtils.getWorldLon(0.75)), + with(new ReaderWay(3), way -> { + way.setTag("attr", "value"); + way.getNodes().add(1, 2); + }) + ), + (in, features) -> { + if (in.canBeLine()) { + features.line("layer") + .setZoomRange(0, 0) + .setAttr("name", "name value") + .inheritFromSource("attr"); + } + } + ); + + assertSubmap(Map.of( + TileCoord.ofXYZ(0, 0, 0), List.of( + feature(newLineString(128, 128, 192, 192), Map.of( + "attr", "value", + "name", "name value" + )) + ) + ), results.tiles); + } + + @Test + public void testOsmLineOrPolygon() throws Exception { + var results = runWithOsmElements( + Map.of("threads", "1"), + List.of( + new ReaderNode(1, GeoUtils.getWorldLat(0.25), GeoUtils.getWorldLon(0.25)), + new ReaderNode(2, GeoUtils.getWorldLat(0.25), GeoUtils.getWorldLon(0.75)), + new ReaderNode(3, GeoUtils.getWorldLat(0.75), GeoUtils.getWorldLon(0.75)), + new ReaderNode(4, GeoUtils.getWorldLat(0.75), GeoUtils.getWorldLon(0.25)), + new ReaderNode(5, GeoUtils.getWorldLat(0.75), GeoUtils.getWorldLon(0.25)), + with(new ReaderWay(6), way -> { + way.setTag("attr", "value"); + way.getNodes().add(1, 2, 3, 4, 5); + }) + ), + (in, features) -> { + if (in.canBeLine()) { + features.line("layer") + .setZoomRange(0, 0) + .setAttr("name", "name value1") + .inheritFromSource("attr"); + } + if (in.canBePolygon()) { + features.polygon("layer") + .setZoomRange(0, 0) + .setAttr("name", "name value2") + .inheritFromSource("attr"); + } + } + ); + + assertSubmap(Map.of( + TileCoord.ofXYZ(0, 0, 0), List.of( + feature(newLineString( + 128, 128, + 192, 128, + 192, 192, + 128, 192, + 128, 128 + ), Map.of( + "attr", "value", + "name", "name value1" + )), + feature(rectangle(128, 192), Map.of( + "attr", "value", + "name", "name value2" + )) + ) + ), results.tiles); + } + + @Test + public void testOsmMultipolygon() throws Exception { + var results = runWithOsmElements( + Map.of("threads", "1"), + List.of( + new ReaderNode(1, GeoUtils.getWorldLat(0.25), GeoUtils.getWorldLon(0.25)), + new ReaderNode(2, GeoUtils.getWorldLat(0.25), GeoUtils.getWorldLon(0.75)), + new ReaderNode(3, GeoUtils.getWorldLat(0.75), GeoUtils.getWorldLon(0.75)), + new ReaderNode(4, GeoUtils.getWorldLat(0.75), GeoUtils.getWorldLon(0.25)), + new ReaderNode(5, GeoUtils.getWorldLat(0.75), GeoUtils.getWorldLon(0.25)), + + new ReaderNode(6, GeoUtils.getWorldLat(0.3), GeoUtils.getWorldLon(0.3)), + new ReaderNode(7, GeoUtils.getWorldLat(0.3), GeoUtils.getWorldLon(0.7)), + new ReaderNode(8, GeoUtils.getWorldLat(0.7), GeoUtils.getWorldLon(0.7)), + new ReaderNode(9, GeoUtils.getWorldLat(0.7), GeoUtils.getWorldLon(0.3)), + new ReaderNode(10, GeoUtils.getWorldLat(0.7), GeoUtils.getWorldLon(0.3)), + + new ReaderNode(11, GeoUtils.getWorldLat(0.4), GeoUtils.getWorldLon(0.4)), + new ReaderNode(12, GeoUtils.getWorldLat(0.4), GeoUtils.getWorldLon(0.6)), + new ReaderNode(13, GeoUtils.getWorldLat(0.6), GeoUtils.getWorldLon(0.6)), + new ReaderNode(14, GeoUtils.getWorldLat(0.6), GeoUtils.getWorldLon(0.4)), + new ReaderNode(15, GeoUtils.getWorldLat(0.6), GeoUtils.getWorldLon(0.4)), + + with(new ReaderWay(16), way -> way.getNodes().add(1, 2, 3, 4, 5)), + with(new ReaderWay(17), way -> way.getNodes().add(6, 7, 8, 9, 10)), + with(new ReaderWay(18), way -> way.getNodes().add(11, 12, 13, 14, 15)), + + with(new ReaderRelation(19), rel -> { + rel.setTag("type", "multipolygon"); + rel.setTag("attr", "value"); + rel.add(new ReaderRelation.Member(ReaderRelation.Member.WAY, 16, "outer")); + rel.add(new ReaderRelation.Member(ReaderRelation.Member.WAY, 17, "inner")); + rel.add(new ReaderRelation.Member(ReaderRelation.Member.WAY, 18, "outer")); + }) + ), + (in, features) -> { + if (in.canBePolygon()) { + features.polygon("layer") + .setZoomRange(0, 0) + .setAttr("name", "name value") + .inheritFromSource("attr"); + } + } + ); + + assertSubmap(Map.of( + TileCoord.ofXYZ(0, 0, 0), List.of( + feature(newMultiPolygon( + rectangle(0.25 * 256, 0.75 * 256), + rectangle(0.3 * 256, 0.7 * 256), + rectangle(0.4 * 256, 0.6 * 256) + ), Map.of( + "attr", "value", + "name", "name value" + )) + ) + ), results.tiles); + } + private Map.Entry> newTileEntry(int x, int y, int z, List features) { return Map.entry(TileCoord.ofXYZ(x, y, z), features); diff --git a/src/test/java/com/onthegomap/flatmap/TestUtils.java b/src/test/java/com/onthegomap/flatmap/TestUtils.java index 69047303..05cdfcd6 100644 --- a/src/test/java/com/onthegomap/flatmap/TestUtils.java +++ b/src/test/java/com/onthegomap/flatmap/TestUtils.java @@ -414,8 +414,18 @@ public class TestUtils { ); } - public static void assertSameNormalizedFeature(Geometry expected, Geometry actual) { - assertEquals(new NormGeometry(expected), new NormGeometry(actual)); + public static void assertSameNormalizedFeature(Geometry expected, Geometry actual, Geometry... otherActuals) { + assertEquals(new NormGeometry(expected), new NormGeometry(actual), "arg 2 != arg 1"); + if (otherActuals != null && otherActuals.length > 0) { + for (int i = 0; i < otherActuals.length; i++) { + assertEquals(new NormGeometry(expected), new NormGeometry(otherActuals[i]), + "arg " + Integer.toString(i + 3) + " != arg 1"); + } + } + } + + public static void assertPointOnSurface(Geometry surface, Geometry actual) { + assertTrue(surface.covers(actual), actual + "\nis not inside\n" + surface); } public static void assertTopologicallyEquivalentFeatures( diff --git a/src/test/java/com/onthegomap/flatmap/read/OpenStreetMapReaderTest.java b/src/test/java/com/onthegomap/flatmap/read/OpenStreetMapReaderTest.java new file mode 100644 index 00000000..d873fc83 --- /dev/null +++ b/src/test/java/com/onthegomap/flatmap/read/OpenStreetMapReaderTest.java @@ -0,0 +1,430 @@ +package com.onthegomap.flatmap.read; + +import static com.onthegomap.flatmap.TestUtils.*; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import com.graphhopper.reader.ReaderNode; +import com.graphhopper.reader.ReaderRelation; +import com.graphhopper.reader.ReaderWay; +import com.onthegomap.flatmap.Profile; +import com.onthegomap.flatmap.SourceFeature; +import com.onthegomap.flatmap.TestUtils; +import com.onthegomap.flatmap.collections.LongLongMap; +import com.onthegomap.flatmap.geo.GeoUtils; +import com.onthegomap.flatmap.geo.GeometryException; +import com.onthegomap.flatmap.monitoring.Stats; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import org.jetbrains.annotations.NotNull; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; + +public class OpenStreetMapReaderTest { + + public final OsmSource osmSource = threads -> next -> { + }; + private final Stats stats = new Stats.InMemory(); + private final Profile profile = new Profile.NullProfile(); + private final LongLongMap longLongMap = LongLongMap.newInMemoryHashMap(); + + private static Profile newProfile(Function> processRelation) { + return new Profile.NullProfile() { + @Override + public List preprocessOsmRelation(ReaderRelation relation) { + return processRelation.apply(relation); + } + }; + } + + @Test + public void testPoint() throws GeometryException { + OpenStreetMapReader reader = new OpenStreetMapReader( + osmSource, + longLongMap, + profile, + stats + ); + var node = new ReaderNode(1, 0, 0); + node.setTag("key", "value"); + reader.processPass1(node); + SourceFeature feature = reader.processNodePass2(node); + assertTrue(feature.isPoint()); + assertFalse(feature.canBePolygon()); + assertFalse(feature.canBeLine()); + assertSameNormalizedFeature( + newPoint(0.5, 0.5), + feature.worldGeometry(), + feature.centroid(), + feature.pointOnSurface(), + GeoUtils.latLonToWorldCoords(feature.latLonGeometry()) + ); + assertEquals(0, feature.area()); + assertEquals(0, feature.length()); + assertThrows(GeometryException.class, feature::line); + assertThrows(GeometryException.class, feature::polygon); + assertEquals(Map.of("key", "value"), feature.properties()); + } + + @Test + public void testLine() throws GeometryException { + OpenStreetMapReader reader = new OpenStreetMapReader( + osmSource, + longLongMap, + profile, + stats + ); + var nodeCache = reader.newNodeGeometryCache(); + var node1 = new ReaderNode(1, 0, 0); + var node2 = node(2, 0.75, 0.75); + var way = new ReaderWay(3); + way.getNodes().add(node1.getId(), node2.getId()); + way.setTag("key", "value"); + + reader.processPass1(node1); + reader.processPass1(node2); + reader.processPass1(way); + + SourceFeature feature = reader.processWayPass2(nodeCache, way); + assertTrue(feature.canBeLine()); + assertFalse(feature.isPoint()); + assertFalse(feature.canBePolygon()); + + assertSameNormalizedFeature( + newLineString( + 0.5, 0.5, + 0.75, 0.75 + ), + feature.worldGeometry(), + feature.line(), + GeoUtils.latLonToWorldCoords(feature.latLonGeometry()) + ); + assertThrows(GeometryException.class, feature::polygon); + assertEquals( + newPoint(0.625, 0.625), + feature.centroid() + ); + assertPointOnSurface(feature); + + assertEquals(0, feature.area()); + assertEquals(Math.sqrt(2 * 0.25 * 0.25), feature.length(), 1e-5); + assertEquals(Map.of("key", "value"), feature.properties()); + } + + @Test + public void testPolygonAreaNotSpecified() throws GeometryException { + OpenStreetMapReader reader = new OpenStreetMapReader( + osmSource, + longLongMap, + profile, + stats + ); + var nodeCache = reader.newNodeGeometryCache(); + var node1 = node(1, 0.5, 0.5); + var node2 = node(2, 0.5, 0.75); + var node3 = node(3, 0.75, 0.75); + var node4 = node(4, 0.75, 0.5); + var way = new ReaderWay(3); + way.getNodes().add(1, 2, 3, 4, 1); + way.setTag("key", "value"); + + reader.processPass1(node1); + reader.processPass1(node2); + reader.processPass1(node3); + reader.processPass1(node4); + reader.processPass1(way); + + SourceFeature feature = reader.processWayPass2(nodeCache, way); + assertTrue(feature.canBeLine()); + assertFalse(feature.isPoint()); + assertTrue(feature.canBePolygon()); + + assertSameNormalizedFeature( + rectangle(0.5, 0.75), + feature.worldGeometry(), + feature.polygon(), + GeoUtils.latLonToWorldCoords(feature.latLonGeometry()) + ); + assertSameNormalizedFeature( + rectangle(0.5, 0.75).getExteriorRing(), + feature.line() + ); + assertEquals( + newPoint(0.625, 0.625), + feature.centroid() + ); + assertPointOnSurface(feature); + + assertEquals(0.25 * 0.25, feature.area()); + assertEquals(1, feature.length()); + } + + @Test + public void testPolygonAreaYes() throws GeometryException { + OpenStreetMapReader reader = new OpenStreetMapReader( + osmSource, + longLongMap, + profile, + stats + ); + var nodeCache = reader.newNodeGeometryCache(); + var node1 = node(1, 0.5, 0.5); + var node2 = node(2, 0.5, 0.75); + var node3 = node(3, 0.75, 0.75); + var node4 = node(4, 0.75, 0.5); + var way = new ReaderWay(3); + way.getNodes().add(1, 2, 3, 4, 1); + way.setTag("area", "yes"); + + reader.processPass1(node1); + reader.processPass1(node2); + reader.processPass1(node3); + reader.processPass1(node4); + reader.processPass1(way); + + SourceFeature feature = reader.processWayPass2(nodeCache, way); + assertFalse(feature.canBeLine()); + assertFalse(feature.isPoint()); + assertTrue(feature.canBePolygon()); + + assertSameNormalizedFeature( + rectangle(0.5, 0.75), + feature.worldGeometry(), + feature.polygon(), + GeoUtils.latLonToWorldCoords(feature.latLonGeometry()) + ); + assertThrows(GeometryException.class, feature::line); + assertEquals( + newPoint(0.625, 0.625), + feature.centroid() + ); + assertPointOnSurface(feature); + + assertEquals(0.25 * 0.25, feature.area()); + assertEquals(1, feature.length()); + } + + @Test + public void testPolygonAreaNo() throws GeometryException { + OpenStreetMapReader reader = new OpenStreetMapReader( + osmSource, + longLongMap, + profile, + stats + ); + var nodeCache = reader.newNodeGeometryCache(); + var node1 = node(1, 0.5, 0.5); + var node2 = node(2, 0.5, 0.75); + var node3 = node(3, 0.75, 0.75); + var node4 = node(4, 0.75, 0.5); + var way = new ReaderWay(5); + way.getNodes().add(1, 2, 3, 4, 1); + way.setTag("area", "no"); + + reader.processPass1(node1); + reader.processPass1(node2); + reader.processPass1(node3); + reader.processPass1(node4); + reader.processPass1(way); + + SourceFeature feature = reader.processWayPass2(nodeCache, way); + assertTrue(feature.canBeLine()); + assertFalse(feature.isPoint()); + assertFalse(feature.canBePolygon()); + + assertSameNormalizedFeature( + rectangle(0.5, 0.75).getExteriorRing(), + feature.worldGeometry(), + feature.line(), + GeoUtils.latLonToWorldCoords(feature.latLonGeometry()) + ); + assertThrows(GeometryException.class, feature::polygon); + assertEquals( + newPoint(0.625, 0.625), + feature.centroid() + ); + assertPointOnSurface(feature); + + assertEquals(0, feature.area()); + assertEquals(1, feature.length()); + } + + @Test + public void testLineWithTooFewPoints() throws GeometryException { + OpenStreetMapReader reader = new OpenStreetMapReader( + osmSource, + longLongMap, + profile, + stats + ); + var node1 = node(1, 0.5, 0.5); + var way = new ReaderWay(3); + way.getNodes().add(1); + + reader.processPass1(node1); + reader.processPass1(way); + + SourceFeature feature = reader.processWayPass2(reader.newNodeGeometryCache(), way); + assertFalse(feature.canBeLine()); + assertFalse(feature.isPoint()); + assertFalse(feature.canBePolygon()); + + assertThrows(GeometryException.class, feature::worldGeometry); + assertThrows(GeometryException.class, feature::latLonGeometry); + assertThrows(GeometryException.class, feature::line); + assertThrows(GeometryException.class, feature::centroid); + assertThrows(GeometryException.class, feature::pointOnSurface); + + assertEquals(0, feature.area()); + assertEquals(0, feature.length()); + } + + @Test + public void testPolygonWithTooFewPoints() throws GeometryException { + OpenStreetMapReader reader = new OpenStreetMapReader( + osmSource, + longLongMap, + profile, + stats + ); + var node1 = node(1, 0.5, 0.5); + var node2 = node(2, 0.5, 0.75); + var way = new ReaderWay(3); + way.getNodes().add(1, 2, 1); + + reader.processPass1(node1); + reader.processPass1(node2); + reader.processPass1(way); + + SourceFeature feature = reader.processWayPass2(reader.newNodeGeometryCache(), way); + assertTrue(feature.canBeLine()); + assertFalse(feature.isPoint()); + assertFalse(feature.canBePolygon()); + + assertSameNormalizedFeature( + newLineString(0.5, 0.5, 0.5, 0.75, 0.5, 0.5), + feature.worldGeometry(), + feature.line(), + GeoUtils.latLonToWorldCoords(feature.latLonGeometry()) + ); + assertSameNormalizedFeature( + newPoint(0.5, 0.625), + feature.centroid() + ); + assertPointOnSurface(feature); + + assertEquals(0, feature.area()); + assertEquals(0.5, feature.length()); + } + + private static void assertPointOnSurface(SourceFeature feature) throws GeometryException { + TestUtils.assertPointOnSurface(feature.worldGeometry(), feature.pointOnSurface()); + } + + @Test + public void testInvalidPolygon() throws GeometryException { + OpenStreetMapReader reader = new OpenStreetMapReader( + osmSource, + longLongMap, + profile, + stats + ); + + reader.processPass1(node(1, 0.5, 0.5)); + reader.processPass1(node(2, 0.75, 0.5)); + reader.processPass1(node(3, 0.5, 0.75)); + reader.processPass1(node(4, 0.75, 0.75)); + var way = new ReaderWay(6); + way.setTag("area", "yes"); + way.getNodes().add(1, 2, 3, 4, 1); + reader.processPass1(way); + + SourceFeature feature = reader.processWayPass2(reader.newNodeGeometryCache(), way); + assertFalse(feature.canBeLine()); + assertFalse(feature.isPoint()); + assertTrue(feature.canBePolygon()); + + assertSameNormalizedFeature( + newPolygon( + 0.5, 0.5, + 0.75, 0.5, + 0.5, 0.75, + 0.75, 0.75, + 0.5, 0.5 + ), + feature.worldGeometry(), + GeoUtils.latLonToWorldCoords(feature.latLonGeometry()) + ); + assertThrows(GeometryException.class, feature::line); + assertSameNormalizedFeature( + newPoint(0.625, 0.625), + feature.centroid() + ); + assertPointOnSurface(feature); + + assertEquals(0, feature.area()); + assertEquals(1.207, feature.length(), 1e-2); + } + + @NotNull + private ReaderNode node(long id, double x, double y) { + return new ReaderNode(id, GeoUtils.getWorldLat(y), GeoUtils.getWorldLon(x)); + } + + @Test + @Disabled + public void testLineReferencingNonexistentNode() { + OpenStreetMapReader reader = new OpenStreetMapReader( + osmSource, + longLongMap, + profile, + stats + ); + var way = new ReaderWay(321); + way.getNodes().add(123, 2222, 333, 444, 123); + reader.processPass1(way); + + SourceFeature feature = reader.processWayPass2(reader.newNodeGeometryCache(), way); + assertTrue(feature.canBeLine()); + assertFalse(feature.isPoint()); + assertTrue(feature.canBePolygon()); + + GeometryException exception = assertThrows(GeometryException.class, feature::line); + assertTrue(exception.getMessage().contains("321") && exception.getMessage().contains("123"), + "Exception message did not contain way and missing node ID: " + exception.getMessage() + ); + assertThrows(GeometryException.class, feature::worldGeometry); + assertThrows(GeometryException.class, feature::centroid); + assertThrows(GeometryException.class, feature::polygon); + assertThrows(GeometryException.class, feature::pointOnSurface); + assertThrows(GeometryException.class, feature::area); + assertThrows(GeometryException.class, feature::length); + } + + @Test + @Disabled + public void testMultiPolygon() { + } + + @Test + @Disabled + public void testMultiPolygonInfersCorrectParents() { + } + + @Test + @Disabled + public void testInvalidMultiPolygon() { + } + + @Test + @Disabled + public void testMultiPolygonRefersToNonexistentWay() { + } + + // TODO what about: + // - relation info / storage size + // - multilevel multipolygon relationship containers +} diff --git a/src/test/java/com/onthegomap/flatmap/render/FeatureRendererTest.java b/src/test/java/com/onthegomap/flatmap/render/FeatureRendererTest.java index 216070af..357ea813 100644 --- a/src/test/java/com/onthegomap/flatmap/render/FeatureRendererTest.java +++ b/src/test/java/com/onthegomap/flatmap/render/FeatureRendererTest.java @@ -48,7 +48,7 @@ public class FeatureRendererTest { private Map> renderGeometry(FeatureCollector.Feature feature) { Map> result = new TreeMap<>(); new FeatureRenderer(config, rendered -> result.computeIfAbsent(rendered.tile(), tile -> new HashSet<>()) - .add(decodeSilently(rendered.vectorTileFeature().geometry()))).renderFeature(feature); + .add(decodeSilently(rendered.vectorTileFeature().geometry()))).accept(feature); result.values().forEach(gs -> gs.forEach(TestUtils::validateGeometry)); return result; } @@ -56,7 +56,7 @@ public class FeatureRendererTest { private Map> renderFeatures(FeatureCollector.Feature feature) { Map> result = new TreeMap<>(); new FeatureRenderer(config, rendered -> result.computeIfAbsent(rendered.tile(), tile -> new HashSet<>()) - .add(rendered)).renderFeature(feature); + .add(rendered)).accept(feature); result.values() .forEach(gs -> gs.forEach(f -> TestUtils.validateGeometry(decodeSilently(f.vectorTileFeature().geometry())))); return result; @@ -807,7 +807,7 @@ public class FeatureRendererTest { .setBufferPixels(0); AtomicLong num = new AtomicLong(0); new FeatureRenderer(config, rendered1 -> num.incrementAndGet()) - .renderFeature(feature); + .accept(feature); assertEquals(num.get(), Math.pow(4, maxZoom)); }