From fb1d0e3bd6233111ce2af9f05ec1cce1fc318d84 Mon Sep 17 00:00:00 2001 From: Michael Barry Date: Wed, 22 May 2024 05:55:57 -0400 Subject: [PATCH] Iniitial geoparquet support (#888) --- NOTICE.md | 3 + README.md | 2 + .../benchmarks/BenchmarkParquetRead.java | 28 + planetiler-core/pom.xml | 22 +- .../com/onthegomap/planetiler/Planetiler.java | 52 ++ .../onthegomap/planetiler/geo/GeoUtils.java | 20 +- .../planetiler/reader/NaturalEarthReader.java | 3 +- .../planetiler/reader/parquet/GeoArrow.java | 104 ++++ .../reader/parquet/GeoParquetMetadata.java | 200 +++++++ .../reader/parquet/GeometryReader.java | 63 ++ .../planetiler/reader/parquet/Interval.java | 42 ++ .../reader/parquet/ParquetFeature.java | 77 +++ .../reader/parquet/ParquetInputFile.java | 233 ++++++++ .../parquet/ParquetPrimitiveConverter.java | 214 +++++++ .../reader/parquet/ParquetReader.java | 217 +++++++ .../parquet/ParquetRecordConverter.java | 391 +++++++++++++ .../onthegomap/planetiler/stats/Stats.java | 13 +- .../onthegomap/planetiler/stats/Timers.java | 20 +- .../onthegomap/planetiler/util/FileUtils.java | 10 +- .../com/onthegomap/planetiler/util/Glob.java | 55 ++ .../hadoop/io/compress/CompressionCodec.java | 16 + .../apache/hadoop/io/compress/GzipCodec.java | 10 + .../apache/hadoop/io/compress/Lz4Codec.java | 9 + .../parquet/filter2/predicate/Filters.java | 20 + .../src/main/resources/log4j2.properties | 7 + .../planetiler/PlanetilerTests.java | 43 ++ .../reader/parquet/GeoArrowTest.java | 231 ++++++++ .../parquet/GeoParquetMetadataTest.java | 447 +++++++++++++++ .../reader/parquet/ParquetConverterTest.java | 536 ++++++++++++++++++ .../reader/parquet/ParquetInputFileTest.java | 188 ++++++ .../reader/parquet/ParquetReaderTest.java | 84 +++ .../planetiler/util/FileUtilsTest.java | 16 + .../onthegomap/planetiler/util/GlobTest.java | 66 +++ .../src/test/resources/log4j2-test.properties | 4 + .../resources/parquet/all_data_types.parquet | Bin 0 -> 3946 bytes .../parquet/boston.customgeometryname.parquet | Bin 0 -> 13032 bytes .../boston.geoarrow_from_gdal_new.parquet | Bin 0 -> 13863 bytes .../boston.geoarrow_from_gdal_old.parquet | Bin 0 -> 13462 bytes .../resources/parquet/boston.gzip.parquet | Bin 0 -> 14196 bytes .../parquet/boston.lz4hadoop.parquet | Bin 0 -> 13544 bytes .../resources/parquet/boston.lz4raw.parquet | Bin 0 -> 12976 bytes .../resources/parquet/boston.none.parquet | Bin 0 -> 13028 bytes .../src/test/resources/parquet/boston.parquet | Bin 0 -> 20576 bytes .../resources/parquet/boston.snappy.parquet | Bin 0 -> 13019 bytes .../resources/parquet/boston.zstd.parquet | Bin 0 -> 13543 bytes .../java/com/onthegomap/planetiler/Main.java | 3 + .../examples/overture/OvertureBasemap.java | 66 +++ 47 files changed, 3492 insertions(+), 23 deletions(-) create mode 100644 planetiler-benchmarks/src/main/java/com/onthegomap/planetiler/benchmarks/BenchmarkParquetRead.java create mode 100644 planetiler-core/src/main/java/com/onthegomap/planetiler/reader/parquet/GeoArrow.java create mode 100644 planetiler-core/src/main/java/com/onthegomap/planetiler/reader/parquet/GeoParquetMetadata.java create mode 100644 planetiler-core/src/main/java/com/onthegomap/planetiler/reader/parquet/GeometryReader.java create mode 100644 planetiler-core/src/main/java/com/onthegomap/planetiler/reader/parquet/Interval.java create mode 100644 planetiler-core/src/main/java/com/onthegomap/planetiler/reader/parquet/ParquetFeature.java create mode 100644 planetiler-core/src/main/java/com/onthegomap/planetiler/reader/parquet/ParquetInputFile.java create mode 100644 planetiler-core/src/main/java/com/onthegomap/planetiler/reader/parquet/ParquetPrimitiveConverter.java create mode 100644 planetiler-core/src/main/java/com/onthegomap/planetiler/reader/parquet/ParquetReader.java create mode 100644 planetiler-core/src/main/java/com/onthegomap/planetiler/reader/parquet/ParquetRecordConverter.java create mode 100644 planetiler-core/src/main/java/com/onthegomap/planetiler/util/Glob.java create mode 100644 planetiler-core/src/main/java/org/apache/hadoop/io/compress/CompressionCodec.java create mode 100644 planetiler-core/src/main/java/org/apache/hadoop/io/compress/GzipCodec.java create mode 100644 planetiler-core/src/main/java/org/apache/hadoop/io/compress/Lz4Codec.java create mode 100644 planetiler-core/src/main/java/org/apache/parquet/filter2/predicate/Filters.java create mode 100644 planetiler-core/src/test/java/com/onthegomap/planetiler/reader/parquet/GeoArrowTest.java create mode 100644 planetiler-core/src/test/java/com/onthegomap/planetiler/reader/parquet/GeoParquetMetadataTest.java create mode 100644 planetiler-core/src/test/java/com/onthegomap/planetiler/reader/parquet/ParquetConverterTest.java create mode 100644 planetiler-core/src/test/java/com/onthegomap/planetiler/reader/parquet/ParquetInputFileTest.java create mode 100644 planetiler-core/src/test/java/com/onthegomap/planetiler/reader/parquet/ParquetReaderTest.java create mode 100644 planetiler-core/src/test/java/com/onthegomap/planetiler/util/GlobTest.java create mode 100644 planetiler-core/src/test/resources/parquet/all_data_types.parquet create mode 100644 planetiler-core/src/test/resources/parquet/boston.customgeometryname.parquet create mode 100644 planetiler-core/src/test/resources/parquet/boston.geoarrow_from_gdal_new.parquet create mode 100644 planetiler-core/src/test/resources/parquet/boston.geoarrow_from_gdal_old.parquet create mode 100644 planetiler-core/src/test/resources/parquet/boston.gzip.parquet create mode 100644 planetiler-core/src/test/resources/parquet/boston.lz4hadoop.parquet create mode 100644 planetiler-core/src/test/resources/parquet/boston.lz4raw.parquet create mode 100644 planetiler-core/src/test/resources/parquet/boston.none.parquet create mode 100644 planetiler-core/src/test/resources/parquet/boston.parquet create mode 100644 planetiler-core/src/test/resources/parquet/boston.snappy.parquet create mode 100644 planetiler-core/src/test/resources/parquet/boston.zstd.parquet create mode 100644 planetiler-examples/src/main/java/com/onthegomap/planetiler/examples/overture/OvertureBasemap.java diff --git a/NOTICE.md b/NOTICE.md index 25f81916d1..9a0546ed2e 100644 --- a/NOTICE.md +++ b/NOTICE.md @@ -18,6 +18,7 @@ The `planetiler-core` module includes the following software: , [EPSG](https://github.com/geotools/geotools/blob/main/licenses/EPSG.md)) - org.msgpack:msgpack-core (Apache license) - org.xerial:sqlite-jdbc (Apache license) + - org.xerial.snappy:snappy-java (Apache license) - com.ibm.icu:icu4j ([ICU license](https://github.com/unicode-org/icu/blob/main/icu4c/LICENSE)) - com.google.guava:guava (Apache license) - com.google.protobuf:protobuf-java (BSD 3-Clause License) @@ -29,6 +30,7 @@ The `planetiler-core` module includes the following software: - org.snakeyaml:snakeyaml-engine (Apache license) - org.commonmark:commonmark (BSD 2-clause license) - org.tukaani:xz (public domain) + - blue.strategic.parquet:parquet-floor (Apache license) - Adapted code: - `DouglasPeuckerSimplifier` from [JTS](https://github.com/locationtech/jts) (EDL) - `OsmMultipolygon` from [imposm3](https://github.com/omniscale/imposm3) (Apache license) @@ -65,4 +67,5 @@ The `planetiler-core` module includes the following software: | 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 | yes | | Wikidata name translations | [CCO](https://www.wikidata.org/wiki/Wikidata:Licensing) | no | no | +| Overture Maps | [Various](https://docs.overturemaps.org/attribution) | no | yes | diff --git a/README.md b/README.md index 0eb3e7f5c7..993284ecf8 100644 --- a/README.md +++ b/README.md @@ -334,6 +334,8 @@ Planetiler is made possible by these awesome open source projects: Google's [Common Expression Language](https://github.com/google/cel-spec) that powers dynamic expressions embedded in schema config files. - [PMTiles](https://github.com/protomaps/PMTiles) optimized tile storage format +- [Apache Parquet](https://github.com/apache/parquet-mr) to support reading geoparquet files in java (with dependencies + minimized by [parquet-floor](https://github.com/strategicblue/parquet-floor)) See [NOTICE.md](NOTICE.md) for a full list and license details. diff --git a/planetiler-benchmarks/src/main/java/com/onthegomap/planetiler/benchmarks/BenchmarkParquetRead.java b/planetiler-benchmarks/src/main/java/com/onthegomap/planetiler/benchmarks/BenchmarkParquetRead.java new file mode 100644 index 0000000000..67fd05c571 --- /dev/null +++ b/planetiler-benchmarks/src/main/java/com/onthegomap/planetiler/benchmarks/BenchmarkParquetRead.java @@ -0,0 +1,28 @@ +package com.onthegomap.planetiler.benchmarks; + +import com.onthegomap.planetiler.config.Arguments; +import com.onthegomap.planetiler.config.Bounds; +import com.onthegomap.planetiler.reader.parquet.ParquetInputFile; +import java.nio.file.Path; + +public class BenchmarkParquetRead { + + public static void main(String[] args) { + var arguments = Arguments.fromArgs(args); + var path = + arguments.inputFile("parquet", "parquet file to read", Path.of("data", "sources", "locality.zstd.parquet")); + long c = 0; + var file = new ParquetInputFile("parquet", "locality", path, null, Bounds.WORLD, null, tags -> tags.get("id")); + for (int i = 0; i < 20; i++) { + long start = System.currentTimeMillis(); + for (var block : file.get()) { + for (var item : block) { + c += item.tags().size(); + } + } + System.err.println(System.currentTimeMillis() - start); + } + + System.err.println(c); + } +} diff --git a/planetiler-core/pom.xml b/planetiler-core/pom.xml index cd5514da67..fb69511d09 100644 --- a/planetiler-core/pom.xml +++ b/planetiler-core/pom.xml @@ -154,12 +154,24 @@ geopackage ${geopackage.version} - - org.slf4j - slf4j-nop - - + + org.slf4j + slf4j-nop + + + + + org.xerial.snappy + snappy-java + 1.1.10.5 + + + blue.strategic.parquet + parquet-floor + 1.41 + + diff --git a/planetiler-core/src/main/java/com/onthegomap/planetiler/Planetiler.java b/planetiler-core/src/main/java/com/onthegomap/planetiler/Planetiler.java index 7ad554095c..59371024a5 100644 --- a/planetiler-core/src/main/java/com/onthegomap/planetiler/Planetiler.java +++ b/planetiler-core/src/main/java/com/onthegomap/planetiler/Planetiler.java @@ -13,9 +13,11 @@ import com.onthegomap.planetiler.reader.GeoPackageReader; import com.onthegomap.planetiler.reader.NaturalEarthReader; import com.onthegomap.planetiler.reader.ShapefileReader; +import com.onthegomap.planetiler.reader.SourceFeature; import com.onthegomap.planetiler.reader.osm.OsmInputFile; import com.onthegomap.planetiler.reader.osm.OsmNodeBoundsProvider; import com.onthegomap.planetiler.reader.osm.OsmReader; +import com.onthegomap.planetiler.reader.parquet.ParquetReader; import com.onthegomap.planetiler.stats.ProcessInfo; import com.onthegomap.planetiler.stats.Stats; import com.onthegomap.planetiler.stats.Timers; @@ -39,9 +41,12 @@ import java.nio.file.Path; import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.function.Function; +import java.util.regex.Pattern; import java.util.stream.IntStream; +import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -469,6 +474,53 @@ public Planetiler addNaturalEarthSource(String name, Path defaultPath, String de config, profile, stats, keepUnzipped))); } + + /** + * Adds a new geoparquet source that will be processed when + * {@link #run()} is called. + * + * @param name string to use in stats and logs to identify this stage + * @param paths paths to the geoparquet files to read. + * @param hivePartitioning Set to true to parse extra feature tags from the file path, for example + * {@code {them="buildings", type="part"}} from + * {@code base/theme=buildings/type=part/file.parquet} + * @param getId function that extracts a unique vector tile feature ID from each input feature, string or + * binary features will be hashed to a {@code long}. + * @param getLayer function that extracts {@link SourceFeature#getSourceLayer()} from the properties of each + * input feature + * @return this runner instance for chaining + * @see GeoPackageReader + */ + public Planetiler addParquetSource(String name, List paths, boolean hivePartitioning, + Function, Object> getId, Function, Object> getLayer) { + // TODO handle auto-downloading + for (var path : paths) { + inputPaths.add(new InputPath(name, path, false)); + } + var separator = Pattern.quote(paths.isEmpty() ? "/" : paths.getFirst().getFileSystem().getSeparator()); + String prefix = StringUtils.getCommonPrefix(paths.stream().map(Path::toString).toArray(String[]::new)) + .replaceAll(separator + "[^" + separator + "]*$", ""); + return addStage(name, "Process features in " + (prefix.isEmpty() ? (paths.size() + " files") : prefix), + ifSourceUsed(name, () -> new ParquetReader(name, profile, stats, getId, getLayer, hivePartitioning) + .process(paths, featureGroup, config))); + } + + /** + * Alias for {@link #addParquetSource(String, List, boolean, Function, Function)} using the default layer and ID + * extractors. + */ + public Planetiler addParquetSource(String name, List paths, boolean hivePartitioning) { + return addParquetSource(name, paths, hivePartitioning, null, null); + } + + /** + * Alias for {@link #addParquetSource(String, List, boolean, Function, Function)} without hive partitioning and using + * the default layer and ID extractors. + */ + public Planetiler addParquetSource(String name, List paths) { + return addParquetSource(name, paths, false); + } + /** * Adds a new stage that will be invoked when {@link #run()} is called. * diff --git a/planetiler-core/src/main/java/com/onthegomap/planetiler/geo/GeoUtils.java b/planetiler-core/src/main/java/com/onthegomap/planetiler/geo/GeoUtils.java index f6d0aceb7b..540933b4b8 100644 --- a/planetiler-core/src/main/java/com/onthegomap/planetiler/geo/GeoUtils.java +++ b/planetiler-core/src/main/java/com/onthegomap/planetiler/geo/GeoUtils.java @@ -17,6 +17,8 @@ import org.locationtech.jts.geom.LineSegment; import org.locationtech.jts.geom.LineString; import org.locationtech.jts.geom.LinearRing; +import org.locationtech.jts.geom.MultiLineString; +import org.locationtech.jts.geom.MultiPoint; import org.locationtech.jts.geom.MultiPolygon; import org.locationtech.jts.geom.Point; import org.locationtech.jts.geom.Polygon; @@ -27,6 +29,7 @@ import org.locationtech.jts.geom.util.GeometryFixer; import org.locationtech.jts.geom.util.GeometryTransformer; import org.locationtech.jts.io.WKBReader; +import org.locationtech.jts.io.WKTReader; import org.locationtech.jts.precision.GeometryPrecisionReducer; /** @@ -40,8 +43,9 @@ public class GeoUtils { /** Rounding precision for 256x256px tiles encoded using 4096 values. */ public static final PrecisionModel TILE_PRECISION = new PrecisionModel(4096d / 256d); public static final GeometryFactory JTS_FACTORY = new GeometryFactory(PackedCoordinateSequenceFactory.DOUBLE_FACTORY); - public static final WKBReader WKB_READER = new WKBReader(JTS_FACTORY); + public static final Geometry EMPTY_GEOMETRY = JTS_FACTORY.createGeometryCollection(); + public static final CoordinateSequence EMPTY_COORDINATE_SEQUENCE = new PackedCoordinateSequence.Double(0, 2, 0); public static final Point EMPTY_POINT = JTS_FACTORY.createPoint(); public static final LineString EMPTY_LINE = JTS_FACTORY.createLineString(); public static final Polygon EMPTY_POLYGON = JTS_FACTORY.createPolygon(); @@ -247,11 +251,11 @@ public static Point point(Coordinate coord) { return JTS_FACTORY.createPoint(coord); } - public static Geometry createMultiLineString(List lineStrings) { + public static MultiLineString createMultiLineString(List lineStrings) { return JTS_FACTORY.createMultiLineString(lineStrings.toArray(EMPTY_LINE_STRING_ARRAY)); } - public static Geometry createMultiPolygon(List polygon) { + public static MultiPolygon createMultiPolygon(List polygon) { return JTS_FACTORY.createMultiPolygon(polygon.toArray(EMPTY_POLYGON_ARRAY)); } @@ -370,7 +374,7 @@ public static CoordinateSequence coordinateSequence(double... coords) { return new PackedCoordinateSequence.Double(coords, 2, 0); } - public static Geometry createMultiPoint(List points) { + public static MultiPoint createMultiPoint(List points) { return JTS_FACTORY.createMultiPoint(points.toArray(EMPTY_POINT_ARRAY)); } @@ -548,6 +552,14 @@ public static int minZoomForPixelSize(double worldGeometrySize, double minPixelS PlanetilerConfig.MAX_MAXZOOM); } + public static WKBReader wkbReader() { + return new WKBReader(JTS_FACTORY); + } + + public static WKTReader wktReader() { + return new WKTReader(JTS_FACTORY); + } + /** Helper class to sort polygons by area of their outer shell. */ private record PolyAndArea(Polygon poly, double area) implements Comparable { diff --git a/planetiler-core/src/main/java/com/onthegomap/planetiler/reader/NaturalEarthReader.java b/planetiler-core/src/main/java/com/onthegomap/planetiler/reader/NaturalEarthReader.java index 2711fdb243..d2d4892610 100644 --- a/planetiler-core/src/main/java/com/onthegomap/planetiler/reader/NaturalEarthReader.java +++ b/planetiler-core/src/main/java/com/onthegomap/planetiler/reader/NaturalEarthReader.java @@ -172,6 +172,7 @@ public void readFeatures(Consumer next) throws Exception { } } if (geometryColumn >= 0) { + var wkbReader = GeoUtils.wkbReader(); while (rs.next()) { byte[] geometry = rs.getBytes(geometryColumn + 1); if (geometry == null) { @@ -179,7 +180,7 @@ public void readFeatures(Consumer next) throws Exception { } // create the feature and pass to next stage - Geometry latLonGeometry = GeoUtils.WKB_READER.read(geometry); + Geometry latLonGeometry = wkbReader.read(geometry); SimpleFeature readerGeometry = SimpleFeature.create(latLonGeometry, HashMap.newHashMap(column.length - 1), sourceName, table, ++id); for (int c = 0; c < column.length; c++) { diff --git a/planetiler-core/src/main/java/com/onthegomap/planetiler/reader/parquet/GeoArrow.java b/planetiler-core/src/main/java/com/onthegomap/planetiler/reader/parquet/GeoArrow.java new file mode 100644 index 0000000000..32b404ca74 --- /dev/null +++ b/planetiler-core/src/main/java/com/onthegomap/planetiler/reader/parquet/GeoArrow.java @@ -0,0 +1,104 @@ +package com.onthegomap.planetiler.reader.parquet; + +import com.onthegomap.planetiler.geo.GeoUtils; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import org.locationtech.jts.geom.CoordinateSequence; +import org.locationtech.jts.geom.LineString; +import org.locationtech.jts.geom.LinearRing; +import org.locationtech.jts.geom.MultiLineString; +import org.locationtech.jts.geom.MultiPoint; +import org.locationtech.jts.geom.MultiPolygon; +import org.locationtech.jts.geom.Point; +import org.locationtech.jts.geom.Polygon; +import org.locationtech.jts.geom.impl.PackedCoordinateSequence; + +/** + * Utilities for converting nested geoarrow + * coordinate lists to JTS geometries. + */ +class GeoArrow { + private GeoArrow() {} + + // TODO create packed coordinate arrays while reading parquet values to avoid creating so many intermediate objects + static MultiPolygon multipolygon(List>> list) { + return GeoUtils.createMultiPolygon(map(list, GeoArrow::polygon)); + } + + static Polygon polygon(List> input) { + return GeoUtils.createPolygon(ring(input.getFirst()), input.stream().skip(1).map(GeoArrow::ring).toList()); + } + + static MultiPoint multipoint(List input) { + return GeoUtils.createMultiPoint(map(input, GeoArrow::point)); + } + + static Point point(Object input) { + int dims = input instanceof List l ? l.size() : input instanceof Map m ? m.size() : 0; + CoordinateSequence result = + new PackedCoordinateSequence.Double(1, dims, dims == 4 ? 1 : 0); + coordinate(input, result, 0); + return GeoUtils.JTS_FACTORY.createPoint(result); + } + + static MultiLineString multilinestring(List> input) { + return GeoUtils.createMultiLineString(map(input, GeoArrow::linestring)); + } + + static LineString linestring(List input) { + return GeoUtils.JTS_FACTORY.createLineString(coordinateSequence(input)); + } + + + private static CoordinateSequence coordinateSequence(List input) { + if (input.isEmpty()) { + return GeoUtils.EMPTY_COORDINATE_SEQUENCE; + } + Object first = input.getFirst(); + int dims = first instanceof List l ? l.size() : first instanceof Map m ? m.size() : 0; + CoordinateSequence result = + new PackedCoordinateSequence.Double(input.size(), dims, dims == 4 ? 1 : 0); + for (int i = 0; i < input.size(); i++) { + Object item = input.get(i); + coordinate(item, result, i); + } + return result; + } + + private static LinearRing ring(List input) { + return GeoUtils.JTS_FACTORY.createLinearRing(coordinateSequence(input)); + } + + private static void coordinate(Object input, CoordinateSequence result, int index) { + switch (input) { + case List list -> { + List l = (List) list; + for (int i = 0; i < l.size(); i++) { + result.setOrdinate(index, i, l.get(i).doubleValue()); + } + } + case Map map -> { + Map m = (Map) map; + + for (var entry : m.entrySet()) { + int ordinateIndex = switch (entry.getKey()) { + case "x" -> 0; + case "y" -> 1; + case "z" -> 2; + case "m" -> 3; + case null, default -> throw new IllegalArgumentException("Bad coordinate key: " + entry.getKey()); + }; + result.setOrdinate(index, ordinateIndex, entry.getValue().doubleValue()); + } + } + default -> throw new IllegalArgumentException("Expecting map or list, got: " + input); + } + } + + private static List map(List in, Function remap) { + return in.stream().map(remap).toList(); + } + +} diff --git a/planetiler-core/src/main/java/com/onthegomap/planetiler/reader/parquet/GeoParquetMetadata.java b/planetiler-core/src/main/java/com/onthegomap/planetiler/reader/parquet/GeoParquetMetadata.java new file mode 100644 index 0000000000..bbf4dab036 --- /dev/null +++ b/planetiler-core/src/main/java/com/onthegomap/planetiler/reader/parquet/GeoParquetMetadata.java @@ -0,0 +1,200 @@ +package com.onthegomap.planetiler.reader.parquet; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.PropertyNamingStrategies; +import com.fasterxml.jackson.databind.annotation.JsonNaming; +import com.onthegomap.planetiler.config.Bounds; +import com.onthegomap.planetiler.geo.GeoUtils; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.function.BiFunction; +import org.apache.parquet.filter2.predicate.FilterApi; +import org.apache.parquet.filter2.predicate.FilterPredicate; +import org.apache.parquet.filter2.predicate.Filters; +import org.apache.parquet.hadoop.metadata.FileMetaData; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; +import org.locationtech.jts.geom.Envelope; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Struct for deserializing a + * geoparquet + * metadata json string into. + */ +@JsonNaming(PropertyNamingStrategies.SnakeCaseStrategy.class) +public record GeoParquetMetadata( + String version, + String primaryColumn, + Map columns +) { + + private static final Logger LOGGER = LoggerFactory.getLogger(GeoParquetMetadata.class); + + private static final ObjectMapper mapper = + new ObjectMapper().configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + + public record CoveringBbox( + List xmin, + List ymin, + List xmax, + List ymax + ) {} + + public record Covering( + CoveringBbox bbox + ) {} + + @JsonNaming(PropertyNamingStrategies.SnakeCaseStrategy.class) + public record ColumnMetadata( + String encoding, + List geometryTypes, + Object crs, + String orientation, + String edges, + List bbox, + Double epoch, + Covering covering + ) { + ColumnMetadata(String encoding) { + this(encoding, List.of()); + } + + ColumnMetadata(String encoding, List geometryTypes) { + this(encoding, geometryTypes, null, null, null, null, null, null); + } + + public Envelope envelope() { + return (bbox == null || bbox.size() != 4) ? GeoUtils.WORLD_LAT_LON_BOUNDS : + new Envelope(bbox.get(0), bbox.get(2), bbox.get(1), bbox.get(3)); + } + + /** + * Returns a parquet filter that filters records read to only those where the covering bbox overlaps {@code bounds} + * or null if unable to infer that from the metadata. + *

+ * If covering bbox metadata is missing from geoparquet metadata, it will try to use bbox.xmin, bbox.xmax, + * bbox.ymin, and bbox.ymax if present. + */ + public FilterPredicate bboxFilter(MessageType schema, Bounds bounds) { + if (!bounds.isWorld()) { + var covering = covering(); + // if covering metadata missing, use default bbox:{xmin,xmax,ymin,ymax} + if (covering == null) { + if (hasNumericField(schema, "bbox.xmin") && + hasNumericField(schema, "bbox.xmax") && + hasNumericField(schema, "bbox.ymin") && + hasNumericField(schema, "bbox.ymax")) { + covering = new GeoParquetMetadata.Covering(new GeoParquetMetadata.CoveringBbox( + List.of("bbox.xmin"), + List.of("bbox.ymin"), + List.of("bbox.xmax"), + List.of("bbox.ymax") + )); + } else if (hasNumericField(schema, "bbox", "xmin") && + hasNumericField(schema, "bbox", "xmax") && + hasNumericField(schema, "bbox", "ymin") && + hasNumericField(schema, "bbox", "ymax")) { + covering = new GeoParquetMetadata.Covering(new GeoParquetMetadata.CoveringBbox( + List.of("bbox", "xmin"), + List.of("bbox", "ymin"), + List.of("bbox", "xmax"), + List.of("bbox", "ymax") + )); + } + } + if (covering != null) { + var latLonBounds = bounds.latLon(); + // TODO apply projection + var coveringBbox = covering.bbox(); + var coordinateType = + schema.getColumnDescription(coveringBbox.xmax().toArray(String[]::new)) + .getPrimitiveType() + .getPrimitiveTypeName(); + BiFunction, Number, FilterPredicate> gtEq = switch (coordinateType) { + case DOUBLE -> (p, v) -> FilterApi.gtEq(Filters.doubleColumn(p), v.doubleValue()); + case FLOAT -> (p, v) -> FilterApi.gtEq(Filters.floatColumn(p), v.floatValue()); + default -> throw new UnsupportedOperationException(); + }; + BiFunction, Number, FilterPredicate> ltEq = switch (coordinateType) { + case DOUBLE -> (p, v) -> FilterApi.ltEq(Filters.doubleColumn(p), v.doubleValue()); + case FLOAT -> (p, v) -> FilterApi.ltEq(Filters.floatColumn(p), v.floatValue()); + default -> throw new UnsupportedOperationException(); + }; + return FilterApi.and( + FilterApi.and( + gtEq.apply(coveringBbox.xmax(), latLonBounds.getMinX()), + ltEq.apply(coveringBbox.xmin(), latLonBounds.getMaxX()) + ), + FilterApi.and( + gtEq.apply(coveringBbox.ymax(), latLonBounds.getMinY()), + ltEq.apply(coveringBbox.ymin(), latLonBounds.getMaxY()) + ) + ); + } + } + return null; + } + } + + public ColumnMetadata primaryColumnMetadata() { + return Objects.requireNonNull(columns.get(primaryColumn), + "No geoparquet metadata for primary column " + primaryColumn); + } + + + /** + * Extracts geoparquet metadata from the {@code "geo"} key value metadata field for the file, or tries to generate a + * default one if missing that uses geometry, wkb_geometry, or wkt_geometry column. + */ + public static GeoParquetMetadata parse(FileMetaData metadata) throws IOException { + String string = metadata.getKeyValueMetaData().get("geo"); + if (string != null) { + try { + return mapper.readValue(string, GeoParquetMetadata.class); + } catch (JsonProcessingException e) { + LOGGER.warn("Invalid geoparquet metadata", e); + } + } + // fallback + for (var field : metadata.getSchema().asGroupType().getFields()) { + if (field.isPrimitive() && + field.asPrimitiveType().getPrimitiveTypeName() == PrimitiveType.PrimitiveTypeName.BINARY) { + switch (field.getName()) { + case "geometry", "wkb_geometry" -> { + return new GeoParquetMetadata("1.0.0", field.getName(), Map.of( + field.getName(), new ColumnMetadata("WKB"))); + } + case "wkt_geometry" -> { + return new GeoParquetMetadata("1.0.0", field.getName(), Map.of( + field.getName(), new ColumnMetadata("WKT"))); + } + default -> { + //ignore + } + } + } + } + throw new IOException( + "No valid geometry columns found: " + metadata.getSchema().asGroupType().getFields().stream().map( + Type::getName).toList()); + } + + private static boolean hasNumericField(MessageType root, String... path) { + if (root.containsPath(path)) { + var type = root.getType(path); + if (!type.isPrimitive()) { + return false; + } + var typeName = type.asPrimitiveType().getPrimitiveTypeName(); + return typeName == PrimitiveType.PrimitiveTypeName.DOUBLE || typeName == PrimitiveType.PrimitiveTypeName.FLOAT; + } + return false; + } +} diff --git a/planetiler-core/src/main/java/com/onthegomap/planetiler/reader/parquet/GeometryReader.java b/planetiler-core/src/main/java/com/onthegomap/planetiler/reader/parquet/GeometryReader.java new file mode 100644 index 0000000000..67deed9ced --- /dev/null +++ b/planetiler-core/src/main/java/com/onthegomap/planetiler/reader/parquet/GeometryReader.java @@ -0,0 +1,63 @@ +package com.onthegomap.planetiler.reader.parquet; + +import com.onthegomap.planetiler.geo.GeoUtils; +import com.onthegomap.planetiler.geo.GeometryException; +import com.onthegomap.planetiler.reader.WithTags; +import com.onthegomap.planetiler.util.FunctionThatThrows; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.locationtech.jts.geom.Geometry; + +/** + * Decodes geometries from a parquet record based on the {@link GeoParquetMetadata} provided. + */ +class GeometryReader { + private final Map> converters = new HashMap<>(); + private final String geometryColumn; + + GeometryReader(GeoParquetMetadata geoparquet) { + this.geometryColumn = geoparquet.primaryColumn(); + for (var entry : geoparquet.columns().entrySet()) { + String column = entry.getKey(); + GeoParquetMetadata.ColumnMetadata columnInfo = entry.getValue(); + FunctionThatThrows converter = switch (columnInfo.encoding()) { + case "WKB" -> obj -> obj instanceof byte[] bytes ? GeoUtils.wkbReader().read(bytes) : null; + case "WKT" -> obj -> obj instanceof String string ? GeoUtils.wktReader().read(string) : null; + case "multipolygon", "geoarrow.multipolygon" -> + obj -> obj instanceof List list ? GeoArrow.multipolygon((List>>) list) : null; + case "polygon", "geoarrow.polygon" -> + obj -> obj instanceof List list ? GeoArrow.polygon((List>) list) : null; + case "multilinestring", "geoarrow.multilinestring" -> + obj -> obj instanceof List list ? GeoArrow.multilinestring((List>) list) : null; + case "linestring", "geoarrow.linestring" -> + obj -> obj instanceof List list ? GeoArrow.linestring((List) list) : null; + case "multipoint", "geoarrow.multipoint" -> + obj -> obj instanceof List list ? GeoArrow.multipoint((List) list) : null; + case "point", "geoarrow.point" -> GeoArrow::point; + default -> throw new IllegalArgumentException("Unhandled type: " + columnInfo.encoding()); + }; + + converters.put(column, converter); + } + } + + Geometry readPrimaryGeometry(WithTags tags) throws GeometryException { + return readGeometry(tags, geometryColumn); + } + + Geometry readGeometry(WithTags tags, String column) throws GeometryException { + var value = tags.getTag(column); + var converter = converters.get(column); + if (value == null) { + throw new GeometryException("no_parquet_column", "Missing geometry column column " + column); + } else if (converter == null) { + throw new GeometryException("no_converter", "No geometry converter for " + column); + } + try { + return converter.apply(value); + } catch (Exception e) { + throw new GeometryException("error_reading", "Error reading " + column, e); + } + } +} diff --git a/planetiler-core/src/main/java/com/onthegomap/planetiler/reader/parquet/Interval.java b/planetiler-core/src/main/java/com/onthegomap/planetiler/reader/parquet/Interval.java new file mode 100644 index 0000000000..f1cb4d464b --- /dev/null +++ b/planetiler-core/src/main/java/com/onthegomap/planetiler/reader/parquet/Interval.java @@ -0,0 +1,42 @@ +package com.onthegomap.planetiler.reader.parquet; + +import java.time.Duration; +import java.time.Period; +import java.time.temporal.Temporal; +import java.time.temporal.TemporalAmount; +import java.time.temporal.TemporalUnit; +import java.util.List; +import java.util.stream.Stream; + +/** + * Represents a parquet + * interval datatype which has a month, day, and millisecond part. + *

+ * Built-in java {@link TemporalAmount} implementations can only store a period or duration amount, but not both. + */ +public record Interval(Period period, Duration duration) implements TemporalAmount { + + public static Interval of(int months, long days, long millis) { + return new Interval(Period.ofMonths(months).plusDays(days), Duration.ofMillis(millis)); + } + + @Override + public long get(TemporalUnit unit) { + return period.get(unit) + duration.get(unit); + } + + @Override + public List getUnits() { + return Stream.concat(period.getUnits().stream(), duration.getUnits().stream()).toList(); + } + + @Override + public Temporal addTo(Temporal temporal) { + return temporal.plus(period).plus(duration); + } + + @Override + public Temporal subtractFrom(Temporal temporal) { + return temporal.minus(period).minus(duration); + } +} diff --git a/planetiler-core/src/main/java/com/onthegomap/planetiler/reader/parquet/ParquetFeature.java b/planetiler-core/src/main/java/com/onthegomap/planetiler/reader/parquet/ParquetFeature.java new file mode 100644 index 0000000000..6cbc826244 --- /dev/null +++ b/planetiler-core/src/main/java/com/onthegomap/planetiler/reader/parquet/ParquetFeature.java @@ -0,0 +1,77 @@ +package com.onthegomap.planetiler.reader.parquet; + +import com.onthegomap.planetiler.geo.GeoUtils; +import com.onthegomap.planetiler.geo.GeometryException; +import com.onthegomap.planetiler.reader.SourceFeature; +import java.nio.file.Path; +import java.util.List; +import java.util.Map; +import org.locationtech.jts.geom.Geometry; +import org.locationtech.jts.geom.Lineal; +import org.locationtech.jts.geom.Polygonal; +import org.locationtech.jts.geom.Puntal; + +/** + * A single record read from a geoparquet file. + */ +public class ParquetFeature extends SourceFeature { + + private final GeometryReader geometryParser; + private final Path filename; + private Geometry latLon; + private Geometry world; + + ParquetFeature(String source, String sourceLayer, Path filename, long id, GeometryReader geometryParser, + Map tags) { + super(tags, source, sourceLayer, List.of(), id); + this.geometryParser = geometryParser; + this.filename = filename; + } + + public Path getFilename() { + return filename; + } + + @Override + public Geometry latLonGeometry() throws GeometryException { + return latLon == null ? latLon = geometryParser.readPrimaryGeometry(this) : latLon; + } + + @Override + public Geometry worldGeometry() throws GeometryException { + return world != null ? world : + (world = GeoUtils.sortPolygonsByAreaDescending(GeoUtils.latLonToWorldCoords(latLonGeometry()))); + } + + @Override + public boolean isPoint() { + try { + return latLonGeometry() instanceof Puntal; + } catch (GeometryException e) { + throw new IllegalStateException(e); + } + } + + @Override + public boolean canBePolygon() { + try { + return latLonGeometry() instanceof Polygonal; + } catch (GeometryException e) { + throw new IllegalStateException(e); + } + } + + @Override + public boolean canBeLine() { + try { + return latLonGeometry() instanceof Lineal; + } catch (GeometryException e) { + throw new IllegalStateException(e); + } + } + + @Override + public String toString() { + return tags().toString(); + } +} diff --git a/planetiler-core/src/main/java/com/onthegomap/planetiler/reader/parquet/ParquetInputFile.java b/planetiler-core/src/main/java/com/onthegomap/planetiler/reader/parquet/ParquetInputFile.java new file mode 100644 index 0000000000..d6b16b42e2 --- /dev/null +++ b/planetiler-core/src/main/java/com/onthegomap/planetiler/reader/parquet/ParquetInputFile.java @@ -0,0 +1,233 @@ +package com.onthegomap.planetiler.reader.parquet; + +import blue.strategic.parquet.ParquetReader; +import com.google.common.collect.Iterators; +import com.onthegomap.planetiler.config.Bounds; +import com.onthegomap.planetiler.geo.GeometryException; +import com.onthegomap.planetiler.reader.SourceFeature; +import com.onthegomap.planetiler.util.Hashing; +import java.io.Closeable; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.nio.file.Path; +import java.util.Collections; +import java.util.Iterator; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Objects; +import java.util.function.Function; +import java.util.function.ToLongFunction; +import java.util.stream.IntStream; +import org.apache.parquet.ParquetReadOptions; +import org.apache.parquet.column.page.PageReadStore; +import org.apache.parquet.filter2.compat.FilterCompat; +import org.apache.parquet.filter2.predicate.FilterApi; +import org.apache.parquet.filter2.predicate.FilterPredicate; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.io.ColumnIOFactory; +import org.apache.parquet.io.InputFile; +import org.apache.parquet.io.MessageColumnIO; +import org.locationtech.jts.geom.Envelope; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Reads {@link SourceFeature SourceFeatures} from a single + * geoparquet file. + */ +public class ParquetInputFile { + + private static final Logger LOGGER = LoggerFactory.getLogger(ParquetInputFile.class); + private final ParquetMetadata metadata; + private final InputFile inputFile; + private final Path path; + private final FilterCompat.Filter filter; + private final String source; + private final ToLongFunction> idGenerator; + private final String layer; + private final long count; + private final int blockCount; + private final GeometryReader geometryReader; + private final Map extraFields; + private Envelope postFilterBounds = null; + private boolean outOfBounds = false; + + public ParquetInputFile(String source, String layer, Path path) { + this(source, layer, path, null, Bounds.WORLD, null, null); + } + + public ParquetInputFile(String source, String layer, Path path, FilterPredicate filter, Bounds bounds, + Map extraFields, Function, Object> idGenerator) { + this.idGenerator = idGenerator == null ? null : map -> hashToLong(idGenerator.apply(map)); + this.layer = layer; + this.source = source; + this.path = path; + inputFile = ParquetReader.makeInputFile(path.toFile()); + this.extraFields = extraFields; + try (var file = open()) { + metadata = file.getFooter(); + var fileMetadata = metadata.getFileMetaData(); + var geoparquet = GeoParquetMetadata.parse(fileMetadata); + this.geometryReader = new GeometryReader(geoparquet); + if (!bounds.isWorld()) { + if (!geoparquet.primaryColumnMetadata().envelope().intersects(bounds.latLon())) { + outOfBounds = true; + } else { + var bboxFilter = geoparquet.primaryColumnMetadata().bboxFilter(fileMetadata.getSchema(), bounds); + if (bboxFilter != null) { + filter = filter == null ? bboxFilter : FilterApi.and(filter, bboxFilter); + } else { + LOGGER.warn("No covering column specified in geoparquet metadata, fall back to post-filtering"); + postFilterBounds = bounds.latLon(); + } + } + } + count = outOfBounds ? 0 : file.getFilteredRecordCount(); + blockCount = outOfBounds ? 0 : metadata.getBlocks().size(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + this.filter = filter == null ? FilterCompat.NOOP : FilterCompat.get(filter); + } + + private static long hashToLong(Object o) { + return switch (o) { + case String s -> Hashing.fnv1a64(s.getBytes(StandardCharsets.UTF_8)); + case byte[] bs -> Hashing.fnv1a64(bs); + case Integer i -> i; + case Long l -> l; + case Float f -> Float.floatToIntBits(f); + case Double d -> Double.doubleToLongBits(d); + case null -> 0; + default -> Hashing.fnv1a64(o.toString().getBytes(StandardCharsets.UTF_8)); + }; + } + + public boolean hasFilter() { + return FilterCompat.isFilteringRequired(filter); + } + + public boolean isOutOfBounds() { + return outOfBounds; + } + + public BlockReader get() { + if (outOfBounds) { + return Collections::emptyIterator; + } + long fileHash = Hashing.fnv1a64(path.toString().getBytes(StandardCharsets.UTF_8)); + var schema = metadata.getFileMetaData().getSchema(); + var columnIOFactory = new ColumnIOFactory(metadata.getFileMetaData().getCreatedBy(), false); + return () -> IntStream.range(0, metadata.getBlocks().size()).mapToObj(blockIndex -> { + long blockHash = Hashing.fnv1a64(fileHash, ByteBuffer.allocate(4).putInt(blockIndex).array()); + // happens in reader thread + return (Block) new Block() { + @Override + public Path getFileName() { + return path; + } + + @Override + public String layer() { + return layer; + } + + @Override + public Iterator iterator() { + PageReadStore group; + try (var reader = open()) { + group = reader.readFilteredRowGroup(blockIndex); + if (group == null) { + return Collections.emptyIterator(); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + MessageColumnIO columnIO = columnIOFactory.getColumnIO(schema); + var recordReader = columnIO.getRecordReader(group, new ParquetRecordConverter(schema), filter); + long total = group.getRowCount(); + return Iterators.filter(new Iterator<>() { + long i = 0; + + @Override + public boolean hasNext() { + return i < total; + } + + @Override + public ParquetFeature next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + i++; + + var item = recordReader.read(); + + if (item == null) { + return null; + } + + if (extraFields != null) { + item.putAll(extraFields); + } + + var feature = new ParquetFeature( + source, + layer, + path, + idGenerator != null ? idGenerator.applyAsLong(item) : + Hashing.fnv1a64(blockHash, ByteBuffer.allocate(8).putLong(i).array()), + geometryReader, + item + ); + + if (postFilterBounds != null) { + try { + if (!feature.latLonGeometry().getEnvelopeInternal().intersects(postFilterBounds)) { + return null; + } + } catch (GeometryException e) { + LOGGER.warn("Error reading geometry to post-filter bounds", e); + return null; + } + } + + return feature; + } + }, Objects::nonNull); + } + }; + }).iterator(); + } + + private ParquetFileReader open() throws IOException { + return ParquetFileReader.open(inputFile, ParquetReadOptions.builder() + .withRecordFilter(filter) + .build()); + } + + public long getCount() { + return count; + } + + public long getBlockCount() { + return blockCount; + } + + public interface BlockReader extends Iterable, Closeable { + + @Override + default void close() throws IOException {} + } + + public interface Block extends Iterable { + + Path getFileName(); + + String layer(); + } + +} diff --git a/planetiler-core/src/main/java/com/onthegomap/planetiler/reader/parquet/ParquetPrimitiveConverter.java b/planetiler-core/src/main/java/com/onthegomap/planetiler/reader/parquet/ParquetPrimitiveConverter.java new file mode 100644 index 0000000000..e021bf9d84 --- /dev/null +++ b/planetiler-core/src/main/java/com/onthegomap/planetiler/reader/parquet/ParquetPrimitiveConverter.java @@ -0,0 +1,214 @@ +package com.onthegomap.planetiler.reader.parquet; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.Period; +import java.time.ZoneOffset; +import java.time.temporal.ChronoUnit; +import java.util.UUID; +import java.util.function.Function; +import java.util.function.LongFunction; +import org.apache.parquet.column.Dictionary; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.io.api.PrimitiveConverter; +import org.apache.parquet.schema.LogicalTypeAnnotation; +import org.apache.parquet.schema.PrimitiveType; + +/** + * Converts typed primitive values from parquet records to java objects: + * + *

    + *
  • {@link PrimitiveType.PrimitiveTypeName#FLOAT} -> {@link Float} + *
  • {@link PrimitiveType.PrimitiveTypeName#DOUBLE} -> {@link Double} + *
  • {@link PrimitiveType.PrimitiveTypeName#INT32} -> {@link Integer} + *
  • {@link PrimitiveType.PrimitiveTypeName#INT64} -> {@link Long} + *
  • {@link PrimitiveType.PrimitiveTypeName#BOOLEAN} -> {@link Boolean} + *
  • {@link PrimitiveType.PrimitiveTypeName#INT96} -> {@link Instant} + *
  • {@link LogicalTypeAnnotation.DateLogicalTypeAnnotation} -> {@link LocalDate} + *
  • {@link LogicalTypeAnnotation.TimeLogicalTypeAnnotation} -> {@link LocalTime} + *
  • {@link LogicalTypeAnnotation.TimestampLogicalTypeAnnotation} -> {@link Instant} + *
  • {@link LogicalTypeAnnotation.UUIDLogicalTypeAnnotation} -> {@link UUID} + *
  • {@link LogicalTypeAnnotation.DecimalLogicalTypeAnnotation} -> {@link Double} + *
  • {@link LogicalTypeAnnotation.StringLogicalTypeAnnotation} -> {@link String} + *
  • {@link LogicalTypeAnnotation.JsonLogicalTypeAnnotation} -> {@link String} + *
  • {@link LogicalTypeAnnotation.EnumLogicalTypeAnnotation} -> {@link String} + *
  • {@link PrimitiveType.PrimitiveTypeName#BINARY} -> {@code byte[]} + *
+ */ +class ParquetPrimitiveConverter extends PrimitiveConverter { + private final PrimitiveType.PrimitiveTypeName primitiveType; + private final ParquetRecordConverter.Context context; + private Dictionary dictionary; + + ParquetPrimitiveConverter(ParquetRecordConverter.Context context) { + this.context = context; + this.primitiveType = context.type.asPrimitiveType().getPrimitiveTypeName(); + } + + static ParquetPrimitiveConverter of(ParquetRecordConverter.Context context) { + var primitiveType = context.type().asPrimitiveType().getPrimitiveTypeName(); + return switch (primitiveType) { + case FLOAT, DOUBLE, BOOLEAN -> new ParquetPrimitiveConverter(context); + case INT64, INT32 -> switch (context.type().getLogicalTypeAnnotation()) { + case null -> new ParquetPrimitiveConverter(context); + case LogicalTypeAnnotation.IntLogicalTypeAnnotation ignored -> + new ParquetPrimitiveConverter(context); + case LogicalTypeAnnotation.DecimalLogicalTypeAnnotation decimal -> { + var multiplier = Math.pow(10, -decimal.getScale()); + yield new IntegerConverter(context, value -> multiplier * value); + } + case LogicalTypeAnnotation.DateLogicalTypeAnnotation ignored -> + new IntegerConverter(context, LocalDate::ofEpochDay); + case LogicalTypeAnnotation.TimeLogicalTypeAnnotation time -> { + var unit = getUnit(time.getUnit()); + yield new IntegerConverter(context, value -> LocalTime.ofNanoOfDay(Duration.of(value, unit).toNanos())); + } + case LogicalTypeAnnotation.TimestampLogicalTypeAnnotation timestamp -> { + var unit = getUnit(timestamp.getUnit()); + yield new IntegerConverter(context, value -> Instant.ofEpochMilli(Duration.of(value, unit).toMillis())); + } + default -> throw new UnsupportedOperationException( + "Unsupported logical type for " + primitiveType + ": " + context.type().getLogicalTypeAnnotation()); + }; + case INT96 -> new BinaryConverer(context, value -> { + var buf = value.toByteBuffer().order(ByteOrder.LITTLE_ENDIAN); + LocalTime timeOfDay = LocalTime.ofNanoOfDay(buf.getLong()); + LocalDate day = LocalDate.ofEpochDay(buf.getInt() - 2440588L); + return LocalDateTime.of(day, timeOfDay).toInstant(ZoneOffset.UTC); + }); + case FIXED_LEN_BYTE_ARRAY, BINARY -> switch (context.type().getLogicalTypeAnnotation()) { + case LogicalTypeAnnotation.UUIDLogicalTypeAnnotation ignored -> new BinaryConverer(context, binary -> { + ByteBuffer byteBuffer = binary.toByteBuffer(); + long msb = byteBuffer.getLong(); + long lsb = byteBuffer.getLong(); + return new UUID(msb, lsb); + }); + case LogicalTypeAnnotation.IntervalLogicalTypeAnnotation ignored -> new BinaryConverer(context, binary -> { + ByteBuffer byteBuffer = binary.toByteBuffer().order(ByteOrder.LITTLE_ENDIAN); + int months = byteBuffer.getInt(); + int days = byteBuffer.getInt(); + int millis = byteBuffer.getInt(); + return new Interval(Period.ofMonths(months).plusDays(days), Duration.ofMillis(millis)); + }); + case LogicalTypeAnnotation.DecimalLogicalTypeAnnotation decimal -> { + int scale = -decimal.getScale(); + yield new BinaryConverer(context, + binary -> new BigDecimal(new BigInteger(binary.getBytes()), scale).doubleValue()); + } + case LogicalTypeAnnotation.StringLogicalTypeAnnotation ignored -> + new BinaryConverer(context, Binary::toStringUsingUTF8); + case LogicalTypeAnnotation.EnumLogicalTypeAnnotation ignored -> + new BinaryConverer(context, Binary::toStringUsingUTF8); + case LogicalTypeAnnotation.JsonLogicalTypeAnnotation ignores -> + new BinaryConverer(context, Binary::toStringUsingUTF8); + case null, default -> new ParquetPrimitiveConverter(context); + }; + }; + } + + private static ChronoUnit getUnit(LogicalTypeAnnotation.TimeUnit unit) { + return switch (unit) { + case MILLIS -> ChronoUnit.MILLIS; + case MICROS -> ChronoUnit.MICROS; + case NANOS -> ChronoUnit.NANOS; + }; + } + + void add(Object value) { + context.accept(value); + } + + @Override + public void addFloat(float value) { + add((double) value); + } + + @Override + public void addDouble(double value) { + add(value); + } + + @Override + public void addInt(int value) { + add(value); + } + + @Override + public void addLong(long value) { + add(value); + } + + @Override + public void addBoolean(boolean value) { + add(value); + } + + @Override + public void addBinary(Binary value) { + add(value.getBytes()); + } + + @Override + public void addValueFromDictionary(int idx) { + switch (primitiveType) { + case INT64 -> addLong(dictionary.decodeToLong(idx)); + case INT32 -> addInt(dictionary.decodeToInt(idx)); + case BOOLEAN -> addBoolean(dictionary.decodeToBoolean(idx)); + case FLOAT -> addFloat(dictionary.decodeToFloat(idx)); + case DOUBLE -> addDouble(dictionary.decodeToDouble(idx)); + case BINARY, FIXED_LEN_BYTE_ARRAY, INT96 -> addBinary(dictionary.decodeToBinary(idx)); + } + } + + @Override + public void setDictionary(Dictionary dictionary) { + this.dictionary = dictionary; + } + + @Override + public boolean hasDictionarySupport() { + return true; + } + + private static class BinaryConverer extends ParquetPrimitiveConverter { + + private final Function remapper; + + BinaryConverer(ParquetRecordConverter.Context context, Function remapper) { + super(context); + this.remapper = remapper; + } + + @Override + public void addBinary(Binary value) { + add(remapper.apply(value)); + } + } + + + private static class IntegerConverter extends ParquetPrimitiveConverter { + private final LongFunction remapper; + + IntegerConverter(ParquetRecordConverter.Context context, LongFunction remapper) { + super(context); + this.remapper = remapper; + } + + @Override + public void addLong(long value) { + add(remapper.apply(value)); + } + + @Override + public void addInt(int value) { + addLong(value); + } + } +} diff --git a/planetiler-core/src/main/java/com/onthegomap/planetiler/reader/parquet/ParquetReader.java b/planetiler-core/src/main/java/com/onthegomap/planetiler/reader/parquet/ParquetReader.java new file mode 100644 index 0000000000..ff24ce025a --- /dev/null +++ b/planetiler-core/src/main/java/com/onthegomap/planetiler/reader/parquet/ParquetReader.java @@ -0,0 +1,217 @@ +package com.onthegomap.planetiler.reader.parquet; + +import static io.prometheus.client.Collector.NANOSECONDS_PER_SECOND; + +import com.onthegomap.planetiler.FeatureCollector; +import com.onthegomap.planetiler.Profile; +import com.onthegomap.planetiler.collection.FeatureGroup; +import com.onthegomap.planetiler.collection.SortableFeature; +import com.onthegomap.planetiler.config.PlanetilerConfig; +import com.onthegomap.planetiler.reader.SourceFeature; +import com.onthegomap.planetiler.render.FeatureRenderer; +import com.onthegomap.planetiler.stats.Counter; +import com.onthegomap.planetiler.stats.ProgressLoggers; +import com.onthegomap.planetiler.stats.Stats; +import com.onthegomap.planetiler.util.Format; +import com.onthegomap.planetiler.worker.WorkerPipeline; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.file.Path; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Reads {@link SourceFeature SourceFeatures} from one or more + * geoparquet files. + *

+ * If files don't contain geoparquet metadata then try to get geometry from "geometry" "wkb_geometry" or "wkt_geometry" + * fields. + */ +public class ParquetReader { + public static final String DEFAULT_LAYER = "features"; + + private static final Logger LOGGER = LoggerFactory.getLogger(ParquetReader.class); + private final String sourceName; + private final Function, Object> idGenerator; + private final Function, Object> layerGenerator; + private final Profile profile; + private final Stats stats; + private final boolean hivePartitioning; + + public ParquetReader( + String sourceName, + Profile profile, + Stats stats + ) { + this(sourceName, profile, stats, null, null, false); + } + + public ParquetReader( + String sourceName, + Profile profile, + Stats stats, + Function, Object> getId, + Function, Object> getLayer, + boolean hivePartitioning + ) { + this.sourceName = sourceName; + this.layerGenerator = getLayer; + this.idGenerator = getId; + this.profile = profile; + this.stats = stats; + this.hivePartitioning = hivePartitioning; + } + + static Map getHivePartitionFields(Path path) { + Map fields = new HashMap<>(); + for (var part : path) { + var string = part.toString(); + if (string.contains("=")) { + var parts = string.split("="); + fields.put(parts[0], parts[1]); + } + } + return fields.isEmpty() ? null : fields; + } + + public void process(List sourcePath, FeatureGroup writer, PlanetilerConfig config) { + var timer = stats.startStage(sourceName); + var inputFiles = sourcePath.stream() + .filter(d -> !"_SUCCESS".equals(d.getFileName().toString())) + .map(path -> { + var hivePartitionFields = hivePartitioning ? getHivePartitionFields(path) : null; + String layer = getLayerName(path); + return new ParquetInputFile(sourceName, layer, path, null, config.bounds(), hivePartitionFields, idGenerator); + }) + .filter(file -> !file.isOutOfBounds()) + .toList(); + // don't show % complete on features when a filter is present because to determine total # elements would + // take an expensive initial query, and % complete on blocks gives a good enough proxy + long featureCount = inputFiles.stream().anyMatch(ParquetInputFile::hasFilter) ? 0 : + inputFiles.stream().mapToLong(ParquetInputFile::getCount).sum(); + long blockCount = inputFiles.stream().mapToLong(ParquetInputFile::getBlockCount).sum(); + int processThreads = config.featureProcessThreads(); + int writeThreads = config.featureWriteThreads(); + var blocksRead = Counter.newMultiThreadCounter(); + var featuresRead = Counter.newMultiThreadCounter(); + var featuresWritten = Counter.newMultiThreadCounter(); + Map workingOn = new ConcurrentHashMap<>(); + var inputBlocks = inputFiles.stream().mapMulti((file, next) -> { + try (var blockReader = file.get()) { + for (var block : blockReader) { + next.accept(block); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + }).toList(); + + var pipeline = WorkerPipeline.start(sourceName, stats) + .readFromTiny("blocks", inputBlocks) + .addWorker("process", processThreads, (prev, next) -> { + var blocks = blocksRead.counterForThread(); + var elements = featuresRead.counterForThread(); + var featureCollectors = new FeatureCollector.Factory(config, stats); + try (FeatureRenderer renderer = newFeatureRenderer(writer, config, next)) { + for (var block : prev) { + String layer = block.layer(); + workingOn.merge(layer, 1, Integer::sum); + for (var sourceFeature : block) { + FeatureCollector features = featureCollectors.get(sourceFeature); + try { + profile.processFeature(sourceFeature, features); + for (FeatureCollector.Feature renderable : features) { + renderer.accept(renderable); + } + } catch (Exception e) { + LOGGER.error("Error processing {}", sourceFeature, e); + } + elements.inc(); + } + blocks.inc(); + workingOn.merge(layer, -1, Integer::sum); + } + } + }) + .addBuffer("write_queue", 50_000, 1_000) + .sinkTo("write", writeThreads, prev -> { + var features = featuresWritten.counterForThread(); + try (var threadLocalWriter = writer.writerForThread()) { + for (var item : prev) { + features.inc(); + threadLocalWriter.accept(item); + } + } + }); + + var loggers = ProgressLoggers.create() + .addRatePercentCounter("read", featureCount, featuresRead, true) + .addRatePercentCounter("blocks", blockCount, blocksRead, false) + .addRateCounter("write", featuresWritten) + .addFileSize(writer) + .newLine() + .add(() -> workingOn.entrySet().stream() + .sorted(Map.Entry.comparingByValue().reversed()) + .filter(d -> d.getValue() > 0) + .map(d -> d.getKey() + ": " + d.getValue()) + .collect(Collectors.joining(", "))) + .newLine() + .addProcessStats() + .newLine() + .addPipelineStats(pipeline); + + pipeline.awaitAndLog(loggers, config.logInterval()); + + if (LOGGER.isInfoEnabled()) { + var format = Format.defaultInstance(); + long count = featuresRead.get(); + var elapsed = timer.elapsed(); + LOGGER.info("Processed {} parquet features ({}/s, {} blocks, {} files) in {}", + format.integer(count), + format.numeric(count * NANOSECONDS_PER_SECOND / elapsed.wall().toNanos()), + format.integer(blocksRead.get()), + format.integer(inputFiles.size()), + elapsed + ); + } + timer.stop(); + + // hook for profile to do any post-processing after this source is read + try ( + var threadLocalWriter = writer.writerForThread(); + var featureRenderer = newFeatureRenderer(writer, config, threadLocalWriter) + ) { + profile.finish(sourceName, new FeatureCollector.Factory(config, stats), featureRenderer); + } catch (IOException e) { + LOGGER.warn("Error closing writer", e); + } + } + + private String getLayerName(Path path) { + String layer = DEFAULT_LAYER; + if (hivePartitioning) { + var fields = getHivePartitionFields(path); + layer = layerGenerator.apply(fields == null ? Map.of() : fields) instanceof Object o ? o.toString() : layer; + } + return layer; + } + + private FeatureRenderer newFeatureRenderer(FeatureGroup writer, PlanetilerConfig config, + Consumer next) { + @SuppressWarnings("java:S2095") // closed by FeatureRenderer + var encoder = writer.newRenderedFeatureEncoder(); + return new FeatureRenderer( + config, + rendered -> next.accept(encoder.apply(rendered)), + stats, + encoder + ); + } +} diff --git a/planetiler-core/src/main/java/com/onthegomap/planetiler/reader/parquet/ParquetRecordConverter.java b/planetiler-core/src/main/java/com/onthegomap/planetiler/reader/parquet/ParquetRecordConverter.java new file mode 100644 index 0000000000..2cdae2b7ff --- /dev/null +++ b/planetiler-core/src/main/java/com/onthegomap/planetiler/reader/parquet/ParquetRecordConverter.java @@ -0,0 +1,391 @@ +package com.onthegomap.planetiler.reader.parquet; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import org.apache.parquet.io.api.Converter; +import org.apache.parquet.io.api.GroupConverter; +import org.apache.parquet.io.api.RecordMaterializer; +import org.apache.parquet.schema.LogicalTypeAnnotation; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.Type; + +/** + * Simple converter for parquet datatypes that maps all structs to {@code Map} and handles deserializing + * list and map nested + * types into java {@link List Lists} and {@link Map Maps}. + */ +public class ParquetRecordConverter extends RecordMaterializer> { + + private final StructConverter root; + private Map map; + + ParquetRecordConverter(MessageType schema) { + root = new StructConverter(new Context(schema)) { + @Override + public void start() { + var group = new MapGroup(schema.getFieldCount()); + context.current = group; + map = group.getMap(); + } + }; + } + + @Override + public Map getCurrentRecord() { + return map; + } + + @Override + public void skipCurrentRecord() { + root.context.current = null; + } + + @Override + public GroupConverter getRootConverter() { + return root; + } + + + interface Group { + // TODO handle repeated when processing schema, not elements + void add(Object key, Object value, boolean repeated); + + Object value(); + } + + private static class ListConverter extends StructConverter { + + ListConverter(Context context) { + super(context); + } + + @Override + protected Converter makeConverter(Context child) { + if ((child.named("list") || child.named("array")) && child.onlyField("element")) { + return new ListElementConverter(child.hoist()); + } + return super.makeConverter(child); + } + + @Override + public void start() { + context.current = new ListGroup(); + context.acceptCurrentValue(); + } + } + + private static class ListElementConverter extends StructConverter { + + ListElementConverter(Context context) { + super(context); + } + + @Override + public void start() { + context.current = new ItemGroup(); + } + + @Override + public void end() { + context.acceptCurrentValue(); + } + } + + private static class MapConverter extends StructConverter { + + MapConverter(Context context) { + super(context); + } + + @Override + protected Converter makeConverter(Context child) { + if (context.getFieldCount() == 1) { + Type type = child.type; + String onlyFieldName = type.getName().toLowerCase(Locale.ROOT); + if (!type.isPrimitive() && type.asGroupType().getFieldCount() == 2 && + (onlyFieldName.equals("key_value") || onlyFieldName.equals("map"))) { + return new MapEntryConverter(child.repeated(false)); + } + } + return super.makeConverter(child); + } + + @Override + public void start() { + context.current = new MapGroup(); + context.acceptCurrentValue(); + } + } + + private static class MapEntryConverter extends StructConverter { + MapEntryGroup entry; + + MapEntryConverter(Context context) { + super(context); + } + + @Override + public void start() { + context.current = entry = new MapEntryGroup(); + } + + @Override + public void end() { + if (entry.v != null && entry.k != null) { + context.accept(entry.k, entry.v); + } + } + } + + static class StructConverter extends GroupConverter { + + final Context context; + private final Converter[] converters; + + StructConverter(Context context) { + this.context = context; + int count = context.type.asGroupType().getFieldCount(); + converters = new Converter[count]; + for (int i = 0; i < count; i++) { + converters[i] = makeConverter(context.field(i)); + } + } + + protected Converter makeConverter(Context child) { + Type type = child.type; + LogicalTypeAnnotation logical = type.getLogicalTypeAnnotation(); + if (!type.isPrimitive()) { + return switch (logical) { + case LogicalTypeAnnotation.ListLogicalTypeAnnotation ignored -> + // If the repeated field is not a group, then its type is the element type and elements are required. + // If the repeated field is a group with multiple fields, then its type is the element type and elements are required. + // If the repeated field is a group with one field and is named either array or uses the LIST-annotated group's name with _tuple appended then the repeated type is the element type and elements are required. + // Otherwise, the repeated field's type is the element type with the repeated field's repetition. + new ListConverter(child); + case LogicalTypeAnnotation.MapLogicalTypeAnnotation ignored -> + // The outer-most level must be a group annotated with MAP that contains a single field named key_value. The repetition of this level must be either optional or required and determines whether the list is nullable. + // The middle level, named key_value, must be a repeated group with a key field for map keys and, optionally, a value field for map values. + // The key field encodes the map's key type. This field must have repetition required and must always be present. + // The value field encodes the map's value type and repetition. This field can be required, optional, or omitted. + new MapConverter(child); + case LogicalTypeAnnotation.MapKeyValueTypeAnnotation ignored -> + new MapConverter(child); + case null, default -> new StructConverter(child); + }; + } + return ParquetPrimitiveConverter.of(child); + } + + @Override + public Converter getConverter(int fieldIndex) { + return converters[fieldIndex]; + } + + @Override + public void start() { + context.current = new MapGroup(context.getFieldCount()); + context.acceptCurrentValue(); + } + + @Override + public void end() { + // by default, don't need to do anything + } + } + + private static class MapGroup implements Group { + + private final Map map; + + MapGroup() { + this(10); + } + + MapGroup(int size) { + map = HashMap.newHashMap(size * 2); + } + + @Override + public void add(Object key, Object value, boolean repeated) { + if (repeated) { + List items = (List) map.computeIfAbsent(key, n -> new ArrayList<>()); + items.add(value); + } else { + if (map.put(key, value) != null) { + throw new IllegalStateException("Multiple values for " + key); + } + } + } + + @Override + public String toString() { + return "MapGroup" + map; + } + + public Map getMap() { + return (Map) (Map) map; + } + + @Override + public Object value() { + return map; + } + } + + private static class ListGroup implements Group { + + private final List list = new ArrayList<>(); + + @Override + public void add(Object key, Object value, boolean repeated) { + list.add(value); + } + + @Override + public String toString() { + return "ListGroup" + list; + } + + @Override + public Object value() { + return list; + } + } + + private static class ItemGroup implements Group { + + private Object item; + + @Override + public void add(Object key, Object value, boolean repeated) { + if (repeated) { + if (item == null) { + item = new ArrayList<>(); + } + ((List) item).add(value); + } else { + item = value; + } + } + + @Override + public String toString() { + return "ItemGroup{" + item + '}'; + } + + @Override + public Object value() { + return item; + } + } + + private static class MapEntryGroup implements Group { + + private Object k; + private Object v; + + @Override + public void add(Object key, Object value, boolean repeated) { + if ("key".equals(key)) { + k = value; + } else if ("value".equals(key)) { + v = value; + } else if (k == null) { + k = value; + } else { + v = value; + } + } + + @Override + public String toString() { + return "MapEntryGroup{" + k + '=' + v + '}'; + } + + @Override + public Object value() { + throw new UnsupportedOperationException(); + } + } + + /** Constructs java objects from parquet records at read-time. */ + static final class Context { + + final Context parent; + final String fieldOnParent; + final Type type; + final boolean repeated; + private final int fieldCount; + Group current; + + Context(Context parent, String fieldOnParent, Type type, boolean repeated) { + this.parent = parent; + this.fieldOnParent = fieldOnParent; + this.type = type; + this.repeated = repeated; + this.fieldCount = type.isPrimitive() ? 0 : type.asGroupType().getFieldCount(); + } + + public Context(Context newParent, Type type) { + this(newParent, type.getName(), type, type.isRepetition(Type.Repetition.REPEATED)); + } + + public Context(MessageType schema) { + this(null, schema); + } + + public Context field(int i) { + return new Context(this, type.asGroupType().getType(i)); + } + + /** Returns a new context that flattens-out this level of the hierarchy and writes values into the parent field. */ + public Context hoist() { + return new Context(parent, parent.fieldOnParent, type, repeated); + } + + public void acceptCurrentValue() { + accept(current.value()); + } + + public void accept(Object value) { + parent.current.add(fieldOnParent, value, repeated); + } + + public int getFieldCount() { + return fieldCount; + } + + public void accept(Object k, Object v) { + parent.current.add(k, v, repeated); + } + + public Context repeated(boolean newRepeated) { + return new Context(parent, fieldOnParent, type, newRepeated); + } + + public boolean named(String name) { + return type.getName().equalsIgnoreCase(name); + } + + boolean onlyField(String name) { + return !type.isPrimitive() && fieldCount == 1 && + type.asGroupType().getFieldName(0).equalsIgnoreCase(name); + } + + public Type type() { + return type; + } + + @Override + public String toString() { + return "Context[" + + "parent=" + parent + ", " + + "fieldOnParent=" + fieldOnParent + ", " + + "type=" + type + ", " + + "repeated=" + repeated + ']'; + } + } +} diff --git a/planetiler-core/src/main/java/com/onthegomap/planetiler/stats/Stats.java b/planetiler-core/src/main/java/com/onthegomap/planetiler/stats/Stats.java index c18ac1bbc2..4570c0a832 100644 --- a/planetiler-core/src/main/java/com/onthegomap/planetiler/stats/Stats.java +++ b/planetiler-core/src/main/java/com/onthegomap/planetiler/stats/Stats.java @@ -94,10 +94,15 @@ default Timers.Finishable startStage(String name, boolean log) { LogUtil.setStage(name); } var timer = timers().startTimer(name, log); - return () -> { - timer.stop(); - if (log) { - LogUtil.clearStage(); + return new Timers.Finishable() { + @Override + public void stop() { + timer.stop(); + } + + @Override + public ProcessTime elapsed() { + return timer.elapsed(); } }; } diff --git a/planetiler-core/src/main/java/com/onthegomap/planetiler/stats/Timers.java b/planetiler-core/src/main/java/com/onthegomap/planetiler/stats/Timers.java index 3ce21270b7..8e22e727c8 100644 --- a/planetiler-core/src/main/java/com/onthegomap/planetiler/stats/Timers.java +++ b/planetiler-core/src/main/java/com/onthegomap/planetiler/stats/Timers.java @@ -103,12 +103,20 @@ public Finishable startTimer(String name, boolean logStart) { LOGGER.info(""); LOGGER.info("Starting..."); } - return () -> { - LOGGER.info("Finished in {}", timers.get(name).timer.stop()); - for (var details : getStageDetails(name, true)) { - LOGGER.info(" {}", details); + return new Finishable() { + @Override + public void stop() { + LOGGER.info("Finished in {}", timers.get(name).timer.stop()); + for (var details : getStageDetails(name, true)) { + LOGGER.info(" {}", details); + } + currentStage.set(last); + } + + @Override + public ProcessTime elapsed() { + return timer.elapsed(); } - currentStage.set(last); }; } @@ -129,6 +137,8 @@ public Map all() { /** A handle that callers can use to indicate a task has finished. */ public interface Finishable { void stop(); + + ProcessTime elapsed(); } record ThreadInfo(ProcessInfo.ThreadState state, String prefix, Duration elapsed) {} diff --git a/planetiler-core/src/main/java/com/onthegomap/planetiler/util/FileUtils.java b/planetiler-core/src/main/java/com/onthegomap/planetiler/util/FileUtils.java index 1cbbe13875..431cc3ddea 100644 --- a/planetiler-core/src/main/java/com/onthegomap/planetiler/util/FileUtils.java +++ b/planetiler-core/src/main/java/com/onthegomap/planetiler/util/FileUtils.java @@ -12,6 +12,7 @@ import java.nio.file.FileStore; import java.nio.file.FileSystem; import java.nio.file.FileSystems; +import java.nio.file.FileVisitOption; import java.nio.file.Files; import java.nio.file.NoSuchFileException; import java.nio.file.Path; @@ -49,7 +50,7 @@ public static Stream walkFileSystem(FileSystem fileSystem) { return StreamSupport.stream(fileSystem.getRootDirectories().spliterator(), false) .flatMap(rootDirectory -> { try { - return Files.walk(rootDirectory); + return Files.walk(rootDirectory, FileVisitOption.FOLLOW_LINKS); } catch (IOException e) { LOGGER.error("Unable to walk " + rootDirectory + " in " + fileSystem, e); return Stream.empty(); @@ -82,9 +83,9 @@ public static List walkPathWithPattern(Path basePath, String pattern, .toList(); } } else if (Files.isDirectory(basePath)) { - try (var walk = Files.walk(basePath)) { + try (var walk = Files.walk(basePath, FileVisitOption.FOLLOW_LINKS)) { return walk - .filter(path -> matcher.matches(path.getFileName())) + .filter(path -> matcher.matches(path.getFileName()) || matcher.matches(basePath.relativize(path))) .flatMap(path -> { if (FileUtils.hasExtension(path, "zip")) { return walkZipFile.apply(path).stream(); @@ -109,9 +110,10 @@ public static List walkPathWithPattern(Path basePath, String pattern, * @param pattern pattern to match filenames against, as described in {@link FileSystem#getPathMatcher(String)}. */ public static List walkPathWithPattern(Path basePath, String pattern) { - return walkPathWithPattern(basePath, pattern, zipPath -> List.of(zipPath)); + return walkPathWithPattern(basePath, pattern, List::of); } + /** Returns true if {@code path} ends with ".extension" (case-insensitive). */ public static boolean hasExtension(Path path, String extension) { return path.toString().toLowerCase().endsWith("." + extension.toLowerCase()); diff --git a/planetiler-core/src/main/java/com/onthegomap/planetiler/util/Glob.java b/planetiler-core/src/main/java/com/onthegomap/planetiler/util/Glob.java new file mode 100644 index 0000000000..f17d65eca0 --- /dev/null +++ b/planetiler-core/src/main/java/com/onthegomap/planetiler/util/Glob.java @@ -0,0 +1,55 @@ +package com.onthegomap.planetiler.util; + +import java.nio.file.Path; +import java.util.Arrays; +import java.util.List; +import java.util.regex.Pattern; + + +/** + * Utility for constructing base+glob paths for matching many files + */ +public record Glob(Path base, String pattern) { + + private static final Pattern GLOB_PATTERN = Pattern.compile("[?*{\\[].*$"); + + /** Wrap a base path with no globs in it yet. */ + public static Glob of(Path path) { + return new Glob(path, null); + } + + /** Resolves a subdirectory using parts separated by the platform file separator. */ + public Glob resolve(String... subPath) { + String separator = "/"; + if (pattern != null) { + return new Glob(base, pattern + separator + String.join(separator, subPath)); + } else if (subPath == null || subPath.length == 0) { + return this; + } else if (GLOB_PATTERN.matcher(subPath[0]).find()) { + return new Glob(base, String.join(separator, subPath)); + } else { + return of(base.resolve(subPath[0])).resolve(Arrays.copyOfRange(subPath, 1, subPath.length)); + } + } + + /** Parse a string containing platform-specific file separators into a base+glob pattern. */ + public static Glob parse(String path) { + var matcher = GLOB_PATTERN.matcher(path); + if (!matcher.find()) { + return of(Path.of(path)); + } + matcher.reset(); + String base = matcher.replaceAll(""); + String separator = Path.of(base).getFileSystem().getSeparator(); + int idx = base.lastIndexOf(separator); + if (idx > 0) { + base = base.substring(0, idx); + } + return of(Path.of(base)).resolve(path.substring(idx + 1).split(Pattern.quote(separator))); + } + + /** Search the filesystem for all files beneath {@link #base()} matching {@link #pattern()}. */ + public List find() { + return pattern == null ? List.of(base) : FileUtils.walkPathWithPattern(base, pattern); + } +} diff --git a/planetiler-core/src/main/java/org/apache/hadoop/io/compress/CompressionCodec.java b/planetiler-core/src/main/java/org/apache/hadoop/io/compress/CompressionCodec.java new file mode 100644 index 0000000000..48d29d1331 --- /dev/null +++ b/planetiler-core/src/main/java/org/apache/hadoop/io/compress/CompressionCodec.java @@ -0,0 +1,16 @@ +package org.apache.hadoop.io.compress; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; + +/** Fix interface from parquet floor so we can extend it with {@link GzipCodec} and {@link Lz4Codec} */ +public interface CompressionCodec { + Decompressor createDecompressor(); + + Compressor createCompressor(); + + CompressionInputStream createInputStream(InputStream is, Decompressor d) throws IOException; + + CompressionOutputStream createOutputStream(OutputStream os, Compressor c) throws IOException; +} diff --git a/planetiler-core/src/main/java/org/apache/hadoop/io/compress/GzipCodec.java b/planetiler-core/src/main/java/org/apache/hadoop/io/compress/GzipCodec.java new file mode 100644 index 0000000000..287ded26b9 --- /dev/null +++ b/planetiler-core/src/main/java/org/apache/hadoop/io/compress/GzipCodec.java @@ -0,0 +1,10 @@ +package org.apache.hadoop.io.compress; + +import io.airlift.compress.gzip.JdkGzipCodec; + +/** + * Make {@link JdkGzipCodec} available at the location expected by + * {@link org.apache.parquet.hadoop.metadata.CompressionCodecName} to allow deserializing parquet files that use gzip + * compression. + */ +public class GzipCodec extends JdkGzipCodec {} diff --git a/planetiler-core/src/main/java/org/apache/hadoop/io/compress/Lz4Codec.java b/planetiler-core/src/main/java/org/apache/hadoop/io/compress/Lz4Codec.java new file mode 100644 index 0000000000..cab23a1515 --- /dev/null +++ b/planetiler-core/src/main/java/org/apache/hadoop/io/compress/Lz4Codec.java @@ -0,0 +1,9 @@ +package org.apache.hadoop.io.compress; + +/** + * Make {@link io.airlift.compress.lz4.Lz4Codec} available at the location expected by + * {@link org.apache.parquet.hadoop.metadata.CompressionCodecName} to allow deserializing parquet files that use lz4 + * compression. + */ +@SuppressWarnings("java:S2176") +public class Lz4Codec extends io.airlift.compress.lz4.Lz4Codec {} diff --git a/planetiler-core/src/main/java/org/apache/parquet/filter2/predicate/Filters.java b/planetiler-core/src/main/java/org/apache/parquet/filter2/predicate/Filters.java new file mode 100644 index 0000000000..b268257327 --- /dev/null +++ b/planetiler-core/src/main/java/org/apache/parquet/filter2/predicate/Filters.java @@ -0,0 +1,20 @@ +package org.apache.parquet.filter2.predicate; + +import java.util.List; +import org.apache.parquet.hadoop.metadata.ColumnPath; + +/** + * Create {@link Operators.DoubleColumn} and {@link Operators.FloatColumn} instances with dots in the column names since + * their constructors are package-private. + */ +public class Filters { + private Filters() {} + + public static Operators.DoubleColumn doubleColumn(List parts) { + return new Operators.DoubleColumn(ColumnPath.get(parts.toArray(String[]::new))); + } + + public static Operators.FloatColumn floatColumn(List parts) { + return new Operators.FloatColumn(ColumnPath.get(parts.toArray(String[]::new))); + } +} diff --git a/planetiler-core/src/main/resources/log4j2.properties b/planetiler-core/src/main/resources/log4j2.properties index 00e993bb1a..7d23fc3a9a 100644 --- a/planetiler-core/src/main/resources/log4j2.properties +++ b/planetiler-core/src/main/resources/log4j2.properties @@ -7,3 +7,10 @@ packages=com.onthegomap.planetiler.util.log4j rootLogger.level=debug rootLogger.appenderRefs=stdout rootLogger.appenderRef.stdout.ref=STDOUT + +logger.apache.name=org.apache +logger.apache.level=warn + +# suppress warning about unreadable duckdb statistics +logger.apachecorrupt.name=org.apache.parquet.CorruptStatistics +logger.apachecorrupt.level=error diff --git a/planetiler-core/src/test/java/com/onthegomap/planetiler/PlanetilerTests.java b/planetiler-core/src/test/java/com/onthegomap/planetiler/PlanetilerTests.java index ffe58fffcd..d8d71292f3 100644 --- a/planetiler-core/src/test/java/com/onthegomap/planetiler/PlanetilerTests.java +++ b/planetiler-core/src/test/java/com/onthegomap/planetiler/PlanetilerTests.java @@ -2248,6 +2248,49 @@ public void processFeature(SourceFeature source, FeatureCollector features) { } } + @ParameterizedTest + @ValueSource(strings = { + "", + "--write-threads=2 --process-threads=2 --feature-read-threads=2 --threads=4" + }) + void testPlanetilerRunnerParquet(String args) throws Exception { + Path mbtiles = tempDir.resolve("output.mbtiles"); + + Planetiler.create(Arguments.fromArgs((args + " --tmpdir=" + tempDir.resolve("data")).split("\\s+"))) + .setProfile(new Profile.NullProfile() { + @Override + public void processFeature(SourceFeature source, FeatureCollector features) { + features.polygon("buildings") + .setZoomRange(0, 14) + .setMinPixelSize(0) + .setAttr("id", source.getString("id")); + } + }) + .addParquetSource("parquet", List.of(TestUtils.pathToResource("parquet").resolve("boston.parquet"))) + .setOutput(mbtiles) + .run(); + + try (Mbtiles db = Mbtiles.newReadOnlyDatabase(mbtiles)) { + Set uniqueIds = new HashSet<>(); + long featureCount = 0; + var tileMap = TestUtils.getTileMap(db); + for (int z = 14; z >= 11; z--) { + var coord = TileCoord.aroundLngLat(-71.07448, 42.35626, z); + assertTrue(tileMap.containsKey(coord), "contain " + coord); + } + for (var tile : tileMap.values()) { + for (var feature : tile) { + feature.geometry().validate(); + featureCount++; + uniqueIds.add((String) feature.attrs().get("id")); + } + } + + assertTrue(featureCount > 0); + assertEquals(3, uniqueIds.size()); + } + } + private void runWithProfile(Path tempDir, Profile profile, boolean force) throws Exception { Planetiler.create(Arguments.of("tmpdir", tempDir, "force", Boolean.toString(force))) .setProfile(profile) diff --git a/planetiler-core/src/test/java/com/onthegomap/planetiler/reader/parquet/GeoArrowTest.java b/planetiler-core/src/test/java/com/onthegomap/planetiler/reader/parquet/GeoArrowTest.java new file mode 100644 index 0000000000..ca7e1e8b85 --- /dev/null +++ b/planetiler-core/src/test/java/com/onthegomap/planetiler/reader/parquet/GeoArrowTest.java @@ -0,0 +1,231 @@ +package com.onthegomap.planetiler.reader.parquet; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +import com.onthegomap.planetiler.geo.GeoUtils; +import java.util.List; +import java.util.Map; +import org.junit.jupiter.api.Test; +import org.locationtech.jts.geom.Geometry; +import org.locationtech.jts.io.ParseException; + +class GeoArrowTest { + @Test + void testPointXY() throws ParseException { + assertSame( + "POINT(1 2)", + GeoArrow.point(Map.of("x", 1, "y", 2)), + GeoArrow.point(List.of(1, 2)) + ); + } + + @Test + void testPointXYZ() throws ParseException { + assertSame( + "POINT Z(1 2 3)", + GeoArrow.point(Map.of("x", 1, "y", 2, "z", 3)), + GeoArrow.point(List.of(1, 2, 3)) + ); + } + + @Test + void testPointXYZM() throws ParseException { + assertSame( + "POINT ZM(1 2 3 4)", + GeoArrow.point(Map.of("x", 1, "y", 2, "z", 3, "m", 4)), + GeoArrow.point(List.of(1, 2, 3, 4)) + ); + } + + @Test + void testLine() throws ParseException { + assertSame( + "LINESTRING(1 2, 3 4)", + GeoArrow.linestring(List.of( + Map.of("x", 1, "y", 2), + Map.of("x", 3, "y", 4) + )), + GeoArrow.linestring(List.of( + List.of(1, 2), + List.of(3, 4) + )) + ); + } + + @Test + void testLineZ() throws ParseException { + assertSame( + "LINESTRING Z(1 2 3, 4 5 6)", + GeoArrow.linestring(List.of( + Map.of("x", 1, "y", 2, "z", 3), + Map.of("x", 4, "y", 5, "z", 6) + )), + GeoArrow.linestring(List.of( + List.of(1, 2, 3), + List.of(4, 5, 6) + )) + ); + } + + @Test + void testLineZM() throws ParseException { + assertSame( + "LINESTRING ZM(1 2 3 4, 5 6 7 8)", + GeoArrow.linestring(List.of( + Map.of("x", 1, "y", 2, "z", 3, "m", 4), + Map.of("x", 5, "y", 6, "z", 7, "m", 8) + )), + GeoArrow.linestring(List.of( + List.of(1, 2, 3, 4), + List.of(5, 6, 7, 8) + )) + ); + } + + @Test + void testPolygon() throws ParseException { + assertSame( + "POLYGON((0 0, 0 1, 1 1, 1 0, 0 0))", + GeoArrow.polygon(List.of( + List.of( + Map.of("x", 0, "y", 0), + Map.of("x", 0, "y", 1), + Map.of("x", 1, "y", 1), + Map.of("x", 1, "y", 0), + Map.of("x", 0, "y", 0) + ))), + GeoArrow.polygon(List.of( + List.of( + List.of(0, 0), + List.of(0, 1), + List.of(1, 1), + List.of(1, 0), + List.of(0, 0) + ) + )) + ); + } + + @Test + void testPolygonWithHole() throws ParseException { + assertSame( + "POLYGON((-2 -2, 2 -2, 0 2, -2 -2), (-1 -1, 1 -1, 0 1, -1 -1))", + GeoArrow.polygon(List.of( + List.of( + Map.of("x", -2, "y", -2), + Map.of("x", 2, "y", -2), + Map.of("x", 0, "y", 2), + Map.of("x", -2, "y", -2) + ), + List.of( + Map.of("x", -1, "y", -1), + Map.of("x", 1, "y", -1), + Map.of("x", 0, "y", 1), + Map.of("x", -1, "y", -1) + ) + )), + GeoArrow.polygon(List.of( + List.of( + List.of(-2, -2), + List.of(2, -2), + List.of(0, 2), + List.of(-2, -2) + ), + List.of( + List.of(-1, -1), + List.of(1, -1), + List.of(0, 1), + List.of(-1, -1) + ) + )) + ); + } + + @Test + void testMultipoint() throws ParseException { + assertSame( + "MULTIPOINT(1 2, 3 4)", + GeoArrow.multipoint(List.of( + Map.of("x", 1, "y", 2), + Map.of("x", 3, "y", 4) + )), + GeoArrow.multipoint(List.of( + List.of(1, 2), + List.of(3, 4) + )) + ); + } + + @Test + void testMultilinestring() throws ParseException { + assertSame( + "MULTILINESTRING((1 2, 3 4), (5 6, 7 8))", + GeoArrow.multilinestring(List.of( + List.of( + Map.of("x", 1, "y", 2), + Map.of("x", 3, "y", 4) + ), + List.of( + Map.of("x", 5, "y", 6), + Map.of("x", 7, "y", 8) + ) + )), + GeoArrow.multilinestring(List.of( + List.of( + List.of(1, 2), + List.of(3, 4) + ), + List.of( + List.of(5, 6), + List.of(7, 8) + ) + )) + ); + } + + @Test + void testMultipolygon() throws ParseException { + assertSame( + "MULTIPOLYGON(((0 0, 1 0, 1 1, 0 1, 0 0)), ((2 0, 3 0, 3 1, 2 1, 2 0)))", + GeoArrow.multipolygon(List.of( + List.of(List.of( + Map.of("x", 0, "y", 0), + Map.of("x", 1, "y", 0), + Map.of("x", 1, "y", 1), + Map.of("x", 0, "y", 1), + Map.of("x", 0, "y", 0) + )), + List.of(List.of( + Map.of("x", 2, "y", 0), + Map.of("x", 3, "y", 0), + Map.of("x", 3, "y", 1), + Map.of("x", 2, "y", 1), + Map.of("x", 2, "y", 0) + )) + )), + GeoArrow.multipolygon(List.of( + List.of(List.of( + List.of(0, 0), + List.of(1, 0), + List.of(1, 1), + List.of(0, 1), + List.of(0, 0) + )), + List.of(List.of( + List.of(2, 0), + List.of(3, 0), + List.of(3, 1), + List.of(2, 1), + List.of(2, 0) + )) + )) + ); + } + + private static void assertSame(String wkt, Geometry... geometry) throws ParseException { + Geometry expected = GeoUtils.wktReader().read(wkt); + for (int i = 0; i < geometry.length; i++) { + assertEquals(expected, geometry[i], "geometry #" + i); + } + } +} diff --git a/planetiler-core/src/test/java/com/onthegomap/planetiler/reader/parquet/GeoParquetMetadataTest.java b/planetiler-core/src/test/java/com/onthegomap/planetiler/reader/parquet/GeoParquetMetadataTest.java new file mode 100644 index 0000000000..ddcf0daa69 --- /dev/null +++ b/planetiler-core/src/test/java/com/onthegomap/planetiler/reader/parquet/GeoParquetMetadataTest.java @@ -0,0 +1,447 @@ +package com.onthegomap.planetiler.reader.parquet; + +import static com.onthegomap.planetiler.geo.GeoUtils.createMultiPoint; +import static com.onthegomap.planetiler.geo.GeoUtils.point; +import static org.apache.parquet.filter2.predicate.FilterApi.*; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import com.onthegomap.planetiler.config.Bounds; +import com.onthegomap.planetiler.geo.GeometryException; +import com.onthegomap.planetiler.reader.WithTags; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import org.apache.parquet.filter2.predicate.Filters; +import org.apache.parquet.hadoop.metadata.FileMetaData; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Types; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestFactory; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; +import org.junit.jupiter.params.provider.ValueSource; +import org.locationtech.jts.geom.Envelope; +import org.locationtech.jts.io.WKBWriter; +import org.locationtech.jts.io.WKTWriter; + +class GeoParquetMetadataTest { + // https://github.com/opengeospatial/geoparquet/blob/main/examples/example_metadata.json + private static final String EXAMPLE_METADATA = """ + { + "columns": { + "geometry": { + "bbox": [ + -180.0, + -90.0, + 180.0, + 83.6451 + ], + "covering": { + "bbox": { + "xmax": [ + "bbox", + "xmax" + ], + "xmin": [ + "bbox", + "xmin" + ], + "ymax": [ + "bbox", + "ymax" + ], + "ymin": [ + "bbox", + "ymin" + ] + } + }, + "crs": { + "$schema": "https://proj.org/schemas/v0.6/projjson.schema.json", + "area": "World.", + "bbox": { + "east_longitude": 180, + "north_latitude": 90, + "south_latitude": -90, + "west_longitude": -180 + }, + "coordinate_system": { + "axis": [ + { + "abbreviation": "Lon", + "direction": "east", + "name": "Geodetic longitude", + "unit": "degree" + }, + { + "abbreviation": "Lat", + "direction": "north", + "name": "Geodetic latitude", + "unit": "degree" + } + ], + "subtype": "ellipsoidal" + }, + "datum_ensemble": { + "accuracy": "2.0", + "ellipsoid": { + "inverse_flattening": 298.257223563, + "name": "WGS 84", + "semi_major_axis": 6378137 + }, + "id": { + "authority": "EPSG", + "code": 6326 + }, + "members": [ + { + "id": { + "authority": "EPSG", + "code": 1166 + }, + "name": "World Geodetic System 1984 (Transit)" + }, + { + "id": { + "authority": "EPSG", + "code": 1152 + }, + "name": "World Geodetic System 1984 (G730)" + }, + { + "id": { + "authority": "EPSG", + "code": 1153 + }, + "name": "World Geodetic System 1984 (G873)" + }, + { + "id": { + "authority": "EPSG", + "code": 1154 + }, + "name": "World Geodetic System 1984 (G1150)" + }, + { + "id": { + "authority": "EPSG", + "code": 1155 + }, + "name": "World Geodetic System 1984 (G1674)" + }, + { + "id": { + "authority": "EPSG", + "code": 1156 + }, + "name": "World Geodetic System 1984 (G1762)" + }, + { + "id": { + "authority": "EPSG", + "code": 1309 + }, + "name": "World Geodetic System 1984 (G2139)" + } + ], + "name": "World Geodetic System 1984 ensemble" + }, + "id": { + "authority": "OGC", + "code": "CRS84" + }, + "name": "WGS 84 (CRS84)", + "scope": "Not known.", + "type": "GeographicCRS" + }, + "edges": "planar", + "encoding": "WKB", + "geometry_types": [ + "Polygon", + "MultiPolygon" + ] + } + }, + "primary_column": "geometry", + "version": "1.1.0-dev" + } + """; + + @Test + void testParseBasicMetadata() throws IOException { + var parsed = GeoParquetMetadata.parse(new FileMetaData( + Types.buildMessage() + .required(PrimitiveType.PrimitiveTypeName.BINARY) + .named("geometry") + .named("root"), + Map.of("geo", EXAMPLE_METADATA), + "")); + assertEquals("geometry", parsed.primaryColumn()); + assertEquals("1.1.0-dev", parsed.version()); + assertEquals("planar", parsed.primaryColumnMetadata().edges()); + assertEquals("WKB", parsed.primaryColumnMetadata().encoding()); + assertEquals(new Envelope(-180.0, 180.0, -90.0, 83.6451), parsed.primaryColumnMetadata().envelope()); + assertEquals(new GeoParquetMetadata.CoveringBbox( + List.of("bbox", "xmin"), + List.of("bbox", "ymin"), + List.of("bbox", "xmax"), + List.of("bbox", "ymax") + ), parsed.primaryColumnMetadata().covering().bbox()); + assertEquals(List.of("Polygon", "MultiPolygon"), parsed.primaryColumnMetadata().geometryTypes()); + assertTrue(parsed.primaryColumnMetadata().crs() instanceof Map); + } + + @Test + void testFailsWhenNoGeometry() { + var fileMetadata = new FileMetaData( + Types.buildMessage() + .required(PrimitiveType.PrimitiveTypeName.BINARY) + .named("not_geometry") + .named("root"), + Map.of(), + ""); + assertThrows(IOException.class, () -> GeoParquetMetadata.parse(fileMetadata)); + } + + @Test + void testFailsWhenBadGeometryType() { + var fileMetadata = new FileMetaData( + Types.buildMessage() + .required(PrimitiveType.PrimitiveTypeName.INT32) + .named("geometry") + .named("root"), + Map.of(), + ""); + assertThrows(IOException.class, () -> GeoParquetMetadata.parse(fileMetadata)); + } + + @Test + void testInfersDefaultGeometry() throws IOException { + var parsed = GeoParquetMetadata.parse(new FileMetaData( + Types.buildMessage() + .required(PrimitiveType.PrimitiveTypeName.BINARY) + .named("geometry") + .named("root"), + Map.of(), + "")); + assertEquals("geometry", parsed.primaryColumn()); + assertEquals("WKB", parsed.primaryColumnMetadata().encoding()); + assertEquals(Bounds.WORLD.latLon(), parsed.primaryColumnMetadata().envelope()); + assertNull(parsed.primaryColumnMetadata().covering()); + } + + @Test + void testGeometryReaderFromMetadata() throws IOException, GeometryException { + var parsed = GeoParquetMetadata.parse(new FileMetaData( + Types.buildMessage() + .required(PrimitiveType.PrimitiveTypeName.BINARY) + .named("geometry") + .named("root"), + Map.of("geo", EXAMPLE_METADATA), + "")); + assertEquals(point(1, 2), new GeometryReader(parsed).readPrimaryGeometry(WithTags.from(Map.of( + "geometry", new WKBWriter().write(point(1, 2)) + )))); + } + + @Test + void testGeometryReaderFromMetadataDifferentName() throws IOException, GeometryException { + var parsed = GeoParquetMetadata.parse(new FileMetaData( + Types.buildMessage() + .required(PrimitiveType.PrimitiveTypeName.BINARY) + .named("other") + .named("root"), + Map.of("geo", """ + { + "primary_column": "other", + "columns": { + "other": { + "encoding": "WKB" + } + } + } + """), + "")); + assertEquals(point(1, 2), new GeometryReader(parsed).readPrimaryGeometry(WithTags.from(Map.of( + "other", new WKBWriter().write(point(1, 2)) + )))); + } + + @ParameterizedTest + @ValueSource(strings = {"wkb_geometry", "geometry"}) + void testReadWKBGeometryNoMetadata(String name) throws IOException, GeometryException { + var parsed = GeoParquetMetadata.parse(new FileMetaData( + Types.buildMessage() + .required(PrimitiveType.PrimitiveTypeName.BINARY) + .named(name) + .named("root"), + Map.of(), + "")); + assertEquals(point(1, 2), new GeometryReader(parsed).readPrimaryGeometry(WithTags.from(Map.of( + name, new WKBWriter().write(point(1, 2)) + )))); + } + + @Test + void testReadWKTGeometryNoMetadata() throws IOException, GeometryException { + var parsed = GeoParquetMetadata.parse(new FileMetaData( + Types.buildMessage() + .required(PrimitiveType.PrimitiveTypeName.BINARY) + .named("wkt_geometry") + .named("root"), + Map.of(), + "")); + assertEquals(point(1, 2), new GeometryReader(parsed).readPrimaryGeometry(WithTags.from(Map.of( + "wkt_geometry", new WKTWriter().write(point(1, 2)) + )))); + } + + @TestFactory + void testReadGeoArrowPoint() throws IOException, GeometryException { + var parsed = GeoParquetMetadata.parse(new FileMetaData( + Types.buildMessage().named("root"), + Map.of("geo", """ + { + "primary_column": "geoarrow", + "columns": { + "geoarrow": { + "encoding": "point" + } + } + } + """), + "")); + assertEquals(point(1, 2), new GeometryReader(parsed).readPrimaryGeometry(WithTags.from(Map.of( + "geoarrow", Map.of("x", 1, "y", 2) + )))); + } + + @TestFactory + void testReadGeoArrowMultiPoint() throws IOException, GeometryException { + var parsed = GeoParquetMetadata.parse(new FileMetaData( + Types.buildMessage().named("root"), + Map.of("geo", """ + { + "primary_column": "geoarrow", + "columns": { + "geoarrow": { + "encoding": "multipolygon" + } + } + } + """), + "")); + assertEquals(createMultiPoint(List.of(point(1, 2))), + new GeometryReader(parsed).readPrimaryGeometry(WithTags.from(Map.of( + "geoarrow", List.of(Map.of("x", 1, "y", 2)) + )))); + } + + @ParameterizedTest + @CsvSource({ + "bbox, true, DOUBLE", + "bbox, true, FLOAT", + "custom_bbox, true, DOUBLE", + "custom_bbox, true, FLOAT", + "bbox, false, DOUBLE", + "bbox, false, FLOAT", + }) + void testBboxFilterFromMetadata(String bbox, boolean hasMetadata, PrimitiveType.PrimitiveTypeName type) + throws IOException { + var schema = Types.buildMessage() + .required(PrimitiveType.PrimitiveTypeName.BINARY) + .named("geometry") + .requiredGroup() + .required(type).named("xmin") + .required(type).named("xmax") + .required(type).named("ymin") + .required(type).named("ymax") + .named(bbox) + .named("root"); + var parsed = GeoParquetMetadata.parse(new FileMetaData( + schema, + hasMetadata ? Map.of("geo", EXAMPLE_METADATA.replaceAll("\"bbox\",", "\"" + bbox + "\",")) : Map.of(), + "")); + var expected = type == PrimitiveType.PrimitiveTypeName.FLOAT ? + and( + and(gtEq(floatColumn(bbox + ".xmax"), 1f), ltEq(floatColumn(bbox + ".xmin"), 2f)), + and(gtEq(floatColumn(bbox + ".ymax"), 3f), ltEq(floatColumn(bbox + ".ymin"), 4f)) + ) : + and( + and(gtEq(doubleColumn(bbox + ".xmax"), 1.0), ltEq(doubleColumn(bbox + ".xmin"), 2.0)), + and(gtEq(doubleColumn(bbox + ".ymax"), 3.0), ltEq(doubleColumn(bbox + ".ymin"), 4.0)) + ); + assertEquals(expected, parsed.primaryColumnMetadata().bboxFilter(schema, new Bounds(new Envelope(1, 2, 3, 4)))); + } + + @ParameterizedTest + @CsvSource({ + "bbox, true, DOUBLE", + "bbox, true, FLOAT", + "custom_bbox, true, DOUBLE", + "custom_bbox, true, FLOAT", + "bbox, false, DOUBLE", + "bbox, false, FLOAT", + }) + void testBboxFilterFromMetadataOldGdalStyle(String bbox, boolean hasMetadata, PrimitiveType.PrimitiveTypeName type) + throws IOException { + var schema = Types.buildMessage() + .required(PrimitiveType.PrimitiveTypeName.BINARY) + .named("geometry") + .required(type).named(bbox + ".xmin") + .required(type).named(bbox + ".xmax") + .required(type).named(bbox + ".ymin") + .required(type).named(bbox + ".ymax") + .named("root"); + var parsed = GeoParquetMetadata.parse(new FileMetaData( + schema, + hasMetadata ? Map.of("geo", """ + { + "primary_column": "geometry", + "columns": { + "geometry": { + "covering": { + "bbox": { + "xmin": ["bbox.xmin"], + "xmax": ["bbox.xmax"], + "ymin": ["bbox.ymin"], + "ymax": ["bbox.ymax"] + } + } + } + } + } + """.replace("bbox.", bbox + ".")) : Map.of(), + "")); + var expected = type == PrimitiveType.PrimitiveTypeName.FLOAT ? + and( + and(gtEq(Filters.floatColumn(List.of(bbox + ".xmax")), 1f), + ltEq(Filters.floatColumn(List.of(bbox + ".xmin")), 2f)), + and(gtEq(Filters.floatColumn(List.of(bbox + ".ymax")), 3f), + ltEq(Filters.floatColumn(List.of(bbox + ".ymin")), 4f)) + ) : + and( + and(gtEq(Filters.doubleColumn(List.of(bbox + ".xmax")), 1.0), + ltEq(Filters.doubleColumn(List.of(bbox + ".xmin")), 2.0)), + and(gtEq(Filters.doubleColumn(List.of(bbox + ".ymax")), 3.0), + ltEq(Filters.doubleColumn(List.of(bbox + ".ymin")), 4.0)) + ); + assertEquals(expected, parsed.primaryColumnMetadata().bboxFilter(schema, new Bounds(new Envelope(1, 2, 3, 4)))); + } + + @Test + void testNoBboxFilterFromDefault() throws IOException { + var schema = Types.buildMessage() + .required(PrimitiveType.PrimitiveTypeName.BINARY) + .named("geometry") + .named("root"); + var parsed = GeoParquetMetadata.parse(new FileMetaData( + schema, + Map.of(), + "")); + assertNull(parsed.primaryColumnMetadata().bboxFilter(schema, new Bounds(new Envelope(1, 2, 3, 4)))); + } +} diff --git a/planetiler-core/src/test/java/com/onthegomap/planetiler/reader/parquet/ParquetConverterTest.java b/planetiler-core/src/test/java/com/onthegomap/planetiler/reader/parquet/ParquetConverterTest.java new file mode 100644 index 0000000000..25a6840031 --- /dev/null +++ b/planetiler-core/src/test/java/com/onthegomap/planetiler/reader/parquet/ParquetConverterTest.java @@ -0,0 +1,536 @@ +package com.onthegomap.planetiler.reader.parquet; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +import com.google.common.collect.Lists; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalTime; +import java.time.Period; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.function.Consumer; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.io.api.PrimitiveConverter; +import org.apache.parquet.schema.LogicalTypeAnnotation; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; +import org.apache.parquet.schema.Types; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; + +class ParquetConverterTest { + @Test + void testIntPrimitive() { + testPrimitive( + PrimitiveType.PrimitiveTypeName.INT32, + converter -> converter.addInt(1), + 1 + ); + } + + @ParameterizedTest + @CsvSource({ + "32, true, 100, 100", + "32, true, 2147483647, 2147483647", + "32, true, -2147483648, -2147483648", + "32, false, 100, 100", + "16, true, 100, 100", + "8, true, 256, 256", + }) + void testIntPrimitiveWithAnnotation(int bitWidth, boolean isSigned, int input, int expected) { + testAnnotatedPrimitive( + PrimitiveType.PrimitiveTypeName.INT32, + LogicalTypeAnnotation.intType(bitWidth, isSigned), + converter -> converter.addInt(input), + expected + ); + } + + @Test + void testLongPrimitive() { + testPrimitive( + PrimitiveType.PrimitiveTypeName.INT64, + converter -> converter.addLong(1), + 1L + ); + } + + @ParameterizedTest + @CsvSource({ + "64, true, 9223372036854775807, 9223372036854775807", + "64, false, 9223372036854775807, 9223372036854775807", + "64, true, -9223372036854775808, -9223372036854775808", + "64, true, 1, 1", + }) + void testLongPrimitiveWithAnnotation(int bitWidth, boolean isSigned, long input, long expected) { + testAnnotatedPrimitive( + PrimitiveType.PrimitiveTypeName.INT64, + LogicalTypeAnnotation.intType(bitWidth, isSigned), + converter -> converter.addLong(input), + expected + ); + } + + @ParameterizedTest + @CsvSource({ + "0, 1, 10, 10", + "1, 9, 10, 1", + "2, 9, 10, 0.1", + }) + void testIntDecimal(int scale, int precision, int value, double expected) { + testAnnotatedPrimitive( + PrimitiveType.PrimitiveTypeName.INT32, + LogicalTypeAnnotation.decimalType(scale, precision), + converter -> converter.addInt(value), + expected + ); + } + + @ParameterizedTest + @CsvSource({ + "0, 1, 10, 10", + "1, 18, 10, 1", + "2, 18, 10, 0.1", + }) + void testLongDecimal(int scale, int precision, long value, double expected) { + testAnnotatedPrimitive( + PrimitiveType.PrimitiveTypeName.INT64, + LogicalTypeAnnotation.decimalType(scale, precision), + converter -> converter.addLong(value), + expected + ); + } + + @Test + void testBooleanPrimitive() { + testPrimitive( + PrimitiveType.PrimitiveTypeName.BOOLEAN, + converter -> converter.addBoolean(true), + true + ); + } + + @Test + void testFloatPrimitive() { + testPrimitive( + PrimitiveType.PrimitiveTypeName.FLOAT, + converter -> converter.addFloat(1f), + 1.0 + ); + } + + @Test + void testDoublePrimitive() { + testPrimitive( + PrimitiveType.PrimitiveTypeName.DOUBLE, + converter -> converter.addDouble(1.5), + 1.5 + ); + } + + @Test + void testInt96Timestamp() { + testPrimitive( + PrimitiveType.PrimitiveTypeName.INT96, + converter -> converter.addBinary(Binary.fromConstantByteArray(new byte[]{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0})), + Instant.parse("-4713-11-24T00:00:00Z") + ); + } + + @Test + void testDate() { + testAnnotatedPrimitive( + PrimitiveType.PrimitiveTypeName.INT32, + LogicalTypeAnnotation.dateType(), + converter -> converter.addInt(2), + LocalDate.of(1970, 1, 3) + ); + } + + @Test + void testTime() { + testAnnotatedPrimitive( + PrimitiveType.PrimitiveTypeName.INT32, + LogicalTypeAnnotation.timeType(true, LogicalTypeAnnotation.TimeUnit.MILLIS), + converter -> converter.addInt(61_000), + LocalTime.of(0, 1, 1) + ); + testAnnotatedPrimitive( + PrimitiveType.PrimitiveTypeName.INT64, + LogicalTypeAnnotation.timeType(true, LogicalTypeAnnotation.TimeUnit.MICROS), + converter -> converter.addLong(61_000_000), + LocalTime.of(0, 1, 1) + ); + testAnnotatedPrimitive( + PrimitiveType.PrimitiveTypeName.INT64, + LogicalTypeAnnotation.timeType(true, LogicalTypeAnnotation.TimeUnit.NANOS), + converter -> converter.addLong(61_000_000_000L), + LocalTime.of(0, 1, 1) + ); + } + + @ParameterizedTest + @CsvSource({ + "true, MILLIS, 61000, 1970-01-01T00:01:01Z", + "true, MICROS, 61000000, 1970-01-01T00:01:01Z", + "true, NANOS, 61000000000, 1970-01-01T00:01:01Z", + }) + void testTimestamp(boolean utc, LogicalTypeAnnotation.TimeUnit unit, long input, String output) { + testAnnotatedPrimitive( + PrimitiveType.PrimitiveTypeName.INT64, + LogicalTypeAnnotation.timestampType(utc, unit), + converter -> converter.addLong(input), + Instant.parse(output) + ); + } + + @Test + void testString() { + testAnnotatedPrimitive( + PrimitiveType.PrimitiveTypeName.BINARY, + LogicalTypeAnnotation.stringType(), + converter -> converter.addBinary(Binary.fromString("abcdef")), + "abcdef" + ); + } + + @Test + void testEnum() { + testAnnotatedPrimitive( + PrimitiveType.PrimitiveTypeName.BINARY, + LogicalTypeAnnotation.enumType(), + converter -> converter.addBinary(Binary.fromString("value")), + "value" + ); + } + + @Test + void testJson() { + testAnnotatedPrimitive( + PrimitiveType.PrimitiveTypeName.BINARY, + LogicalTypeAnnotation.jsonType(), + converter -> converter.addBinary(Binary.fromString("[1,2,3]")), + "[1,2,3]" + ); + } + + @Test + void testUUID() { + testAnnotatedPrimitive( + PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY, + LogicalTypeAnnotation.uuidType(), + 16, + converter -> converter + .addBinary(Binary.fromConstantByteArray( + new byte[]{0x00, 0x11, 0x22, 0x33, 0x44, 0x55, 0x66, 0x77, (byte) 0x88, (byte) 0x99, (byte) 0xaa, (byte) 0xbb, + (byte) 0xcc, (byte) 0xdd, (byte) 0xee, (byte) 0xff})), + UUID.fromString("00112233-4455-6677-8899-aabbccddeeff") + ); + } + + @Test + void testInterval() { + testAnnotatedPrimitive( + PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY, + LogicalTypeAnnotation.IntervalLogicalTypeAnnotation.getInstance(), + 12, + converter -> converter.addBinary(Binary.fromConstantByteBuffer(ByteBuffer.allocate(12) + .order(ByteOrder.LITTLE_ENDIAN) + .putInt(1) + .putInt(2) + .putInt(3) + .flip())), + new Interval( + Period.ofMonths(1).plusDays(2), + Duration.ofMillis(3) + ) + ); + } + + @Test + void testOptionalMissing() { + var materializer = new ParquetRecordConverter(Types.buildMessage() + .optional(PrimitiveType.PrimitiveTypeName.INT32).named("value") + .named("message")); + var rootConverter = materializer.getRootConverter(); + rootConverter.start(); + rootConverter.end(); + assertEquals(Map.of(), materializer.getCurrentRecord()); + } + + @Test + void testListFromSimpleRepeatedElement() { + var materializer = new ParquetRecordConverter(Types.buildMessage() + .repeated(PrimitiveType.PrimitiveTypeName.INT32).named("value") + .named("message")); + + + var rootConverter = materializer.getRootConverter(); + rootConverter.start(); + rootConverter.end(); + assertEquals(Map.of(), materializer.getCurrentRecord()); + + rootConverter.start(); + rootConverter.getConverter(0).asPrimitiveConverter().addInt(1); + rootConverter.end(); + assertEquals(Map.of("value", List.of(1)), materializer.getCurrentRecord()); + + rootConverter.start(); + rootConverter.getConverter(0).asPrimitiveConverter().addInt(1); + rootConverter.getConverter(0).asPrimitiveConverter().addInt(2); + rootConverter.end(); + assertEquals(Map.of("value", List.of(1, 2)), materializer.getCurrentRecord()); + } + + @Test + void testListFromListElementStructs() { + var materializer = new ParquetRecordConverter(Types.buildMessage() + .requiredList().optionalElement(PrimitiveType.PrimitiveTypeName.INT32).named("value") + .named("message")); + + var root = materializer.getRootConverter(); + var value = root.getConverter(0).asGroupConverter(); + var list = value.getConverter(0).asGroupConverter(); + var element = list.getConverter(0).asPrimitiveConverter(); + root.start(); + value.start(); + value.end(); + root.end(); + assertEquals(Map.of("value", List.of()), materializer.getCurrentRecord()); + + root.start(); + value.start(); + list.start(); + element.addInt(1); + list.end(); + list.start(); + list.end(); + list.start(); + element.addInt(3); + list.end(); + value.end(); + root.end(); + assertEquals(Map.of("value", Lists.newArrayList(1, null, 3)), materializer.getCurrentRecord()); + } + + @Test + void testListRepeatedAtTopAndBottomLevel() { + var materializer = new ParquetRecordConverter(Types.buildMessage() + .list(Type.Repetition.REPEATED).element(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REPEATED) + .named("value") + .named("message")); + + var root = materializer.getRootConverter(); + var value = root.getConverter(0).asGroupConverter(); + var list = value.getConverter(0).asGroupConverter(); + var element = list.getConverter(0).asPrimitiveConverter(); + root.start(); + value.start(); + value.end(); + value.start(); + list.start(); + element.addInt(1); + element.addInt(2); + list.end(); + list.start(); + element.addInt(3); + list.end(); + value.end(); + root.end(); + assertEquals(Map.of("value", List.of(List.of(), List.of(List.of(1, 2), List.of(3)))), + materializer.getCurrentRecord()); + } + + @Test + void testNestedList() { + var materializer = new ParquetRecordConverter(Types.buildMessage() + .optionalList() + .optionalListElement() + .optionalElement(PrimitiveType.PrimitiveTypeName.INT32) + .named("value") + .named("root")); + + //message root { + // optional group value (LIST) { + // repeated group list { + // optional group element (LIST) { + // repeated group list { + // optional int32 element; + // } + // } + // } + // } + //} + + var root = materializer.getRootConverter(); + var value = root.getConverter(0).asGroupConverter(); + var outerList = value.getConverter(0).asGroupConverter(); + var outerElement = outerList.getConverter(0).asGroupConverter(); + var innerList = outerElement.getConverter(0).asGroupConverter(); + var innerElement = innerList.getConverter(0).asPrimitiveConverter(); + root.start(); + root.end(); + assertEquals(Map.of(), materializer.getCurrentRecord()); + + root.start(); + value.start(); + value.end(); + root.end(); + assertEquals(Map.of("value", List.of()), materializer.getCurrentRecord()); + + root.start(); + value.start(); + outerList.start(); + outerList.end(); + + outerList.start(); + outerElement.start(); + + innerList.start(); + innerElement.addInt(1); + innerList.end(); + + innerList.start(); + innerList.end(); + + innerList.start(); + innerElement.addInt(2); + innerList.end(); + + outerElement.end(); + outerList.end(); + value.end(); + root.end(); + + assertEquals(Map.of( + "value", Lists.newArrayList(null, Lists.newArrayList(1, null, 2)) + ), materializer.getCurrentRecord()); + } + + @Test + void testMapConverter() { + var materializer = new ParquetRecordConverter(Types.buildMessage() + .optionalMap() + .key(PrimitiveType.PrimitiveTypeName.INT32) + .optionalValue(PrimitiveType.PrimitiveTypeName.INT64) + .named("value") + .named("root")); + + //message root { + // optional group value (MAP) { + // repeated group key_value { + // required int32 key; + // optional int64 value; + // } + // } + //} + + var root = materializer.getRootConverter(); + var map = root.getConverter(0).asGroupConverter(); + var keyValue = map.getConverter(0).asGroupConverter(); + var key = keyValue.getConverter(0).asPrimitiveConverter(); + var value = keyValue.getConverter(1).asPrimitiveConverter(); + + root.start(); + root.end(); + assertEquals(Map.of(), materializer.getCurrentRecord()); + + root.start(); + map.start(); + map.end(); + root.end(); + assertEquals(Map.of("value", Map.of()), materializer.getCurrentRecord()); + + root.start(); + map.start(); + keyValue.start(); + key.addInt(1); + keyValue.end(); + map.end(); + root.end(); + assertEquals(Map.of("value", Map.of()), materializer.getCurrentRecord()); + + root.start(); + map.start(); + keyValue.start(); + key.addInt(1); + value.addLong(2); + keyValue.end(); + map.end(); + root.end(); + assertEquals(Map.of("value", Map.of(1, 2L)), materializer.getCurrentRecord()); + + root.start(); + map.start(); + keyValue.start(); + key.addInt(1); + value.addLong(2); + keyValue.end(); + keyValue.start(); + key.addInt(3); + value.addLong(4); + keyValue.end(); + map.end(); + root.end(); + assertEquals(Map.of("value", Map.of(1, 2L, 3, 4L)), materializer.getCurrentRecord()); + } + + @Test + void testRepeatedMap() { + var materializer = new ParquetRecordConverter(Types.buildMessage() + .map(Type.Repetition.REPEATED) + .key(PrimitiveType.PrimitiveTypeName.INT32) + .optionalValue(PrimitiveType.PrimitiveTypeName.INT64) + .named("value") + .named("root")); + + var root = materializer.getRootConverter(); + var map = root.getConverter(0).asGroupConverter(); + var keyValue = map.getConverter(0).asGroupConverter(); + var key = keyValue.getConverter(0).asPrimitiveConverter(); + + root.start(); + map.start(); + keyValue.start(); + key.addInt(1); + keyValue.end(); + map.end(); + root.end(); + assertEquals(Map.of("value", List.of(Map.of())), materializer.getCurrentRecord()); + } + + private void testPrimitive(PrimitiveType.PrimitiveTypeName type, Consumer consumer, + Object expected) { + var materializer = new ParquetRecordConverter(Types.buildMessage() + .required(type).named("value") + .named("message")); + var rootConverter = materializer.getRootConverter(); + rootConverter.start(); + consumer.accept(rootConverter.getConverter(0).asPrimitiveConverter()); + rootConverter.end(); + assertEquals(Map.of("value", expected), materializer.getCurrentRecord()); + } + + private void testAnnotatedPrimitive(PrimitiveType.PrimitiveTypeName type, LogicalTypeAnnotation annotation, + Consumer consumer, Object expected) { + testAnnotatedPrimitive(type, annotation, 0, consumer, expected); + } + + private void testAnnotatedPrimitive(PrimitiveType.PrimitiveTypeName type, LogicalTypeAnnotation annotation, + int length, Consumer consumer, Object expected) { + var materializer = new ParquetRecordConverter(Types.buildMessage() + .required(type).as(annotation).length(length).named("value") + .named("message")); + var rootConverter = materializer.getRootConverter(); + rootConverter.start(); + consumer.accept(rootConverter.getConverter(0).asPrimitiveConverter()); + rootConverter.end(); + assertEquals(Map.of("value", expected), materializer.getCurrentRecord()); + } +} diff --git a/planetiler-core/src/test/java/com/onthegomap/planetiler/reader/parquet/ParquetInputFileTest.java b/planetiler-core/src/test/java/com/onthegomap/planetiler/reader/parquet/ParquetInputFileTest.java new file mode 100644 index 0000000000..994b1dc42b --- /dev/null +++ b/planetiler-core/src/test/java/com/onthegomap/planetiler/reader/parquet/ParquetInputFileTest.java @@ -0,0 +1,188 @@ +package com.onthegomap.planetiler.reader.parquet; + +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.DynamicTest.dynamicTest; + +import com.onthegomap.planetiler.TestUtils; +import com.onthegomap.planetiler.config.Bounds; +import com.onthegomap.planetiler.util.Glob; +import java.nio.file.Path; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalTime; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.function.Consumer; +import org.apache.parquet.filter2.predicate.FilterApi; +import org.junit.jupiter.api.DynamicTest; +import org.junit.jupiter.api.TestFactory; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; +import org.locationtech.jts.geom.Envelope; + +class ParquetInputFileTest { + + static List bostons() { + return Glob.of(TestUtils.pathToResource("parquet")).resolve("boston*.parquet").find(); + } + + @ParameterizedTest + @MethodSource("bostons") + void testReadBoston(Path path) { + for (int i = 0; i < 3; i++) { + Set ids = new HashSet<>(); + for (var block : new ParquetInputFile("parquet", "layer", path) + .get()) { + for (var item : block) { + ids.add(item.getString("id")); + } + } + assertEquals(3, ids.size(), "iter " + i); + } + } + + @ParameterizedTest + @MethodSource("bostons") + void testReadBostonWithBboxFilterCovering(Path path) { + Set ids = new HashSet<>(); + for (var block : new ParquetInputFile("parquet", "layer", path, null, + new Bounds(new Envelope(-71.0747653629, -71.0741656634, 42.3560968301, 42.3564346282)), null, null) + .get()) { + for (var item : block) { + ids.add(item.getString("id")); + } + } + assertEquals(3, ids.size()); + } + + @ParameterizedTest + @MethodSource("bostons") + void testReadBostonWithBboxFilterCoveringAndOtherFilter(Path path) { + Set ids = new HashSet<>(); + for (var block : new ParquetInputFile("parquet", "layer", path, FilterApi.gt(FilterApi.doubleColumn("height"), 3d), + new Bounds(new Envelope(-71.0747653629, -71.0741656634, 42.3560968301, 42.3564346282)), null, null) + .get()) { + for (var item : block) { + ids.add(item.getString("id")); + } + } + assertEquals(1, ids.size()); + } + + @ParameterizedTest + @MethodSource("bostons") + void testReadBostonWithBboxFilterNotCovering(Path path) { + Set ids = new HashSet<>(); + for (var block : new ParquetInputFile("parquet", "layer", path, null, + new Bounds(new Envelope(-72.0747653629, -72.0741656634, 42.3560968301, 42.3564346282)), null, null) + .get()) { + for (var item : block) { + ids.add(item.getString("id")); + } + } + assertEquals(0, ids.size()); + } + + @TestFactory + @SuppressWarnings("java:S5961") + List testReadAllDataTypes() { + + /* + ┌──────────────────────┬────────────────────────────────┬─────────┬─────────┬─────────┬─────────┐ + │ column_name │ column_type │ null │ key │ default │ extra │ + │ varchar │ varchar │ varchar │ varchar │ varchar │ varchar │ + ├──────────────────────┼────────────────────────────────┼─────────┼─────────┼─────────┼─────────┤ + │ geometry │ BLOB │ YES │ │ │ │ ST_AsWKB(ST_Point(1, 2)) + │ bigint │ BIGINT │ YES │ │ │ │ 9223372036854775807 + │ blob │ BLOB │ YES │ │ │ │ '1011' + │ boolean │ BOOLEAN │ YES │ │ │ │ true + │ date │ DATE │ YES │ │ │ │ '2000-01-01' + │ decimal │ DECIMAL(18,3) │ YES │ │ │ │ 123456.789 + │ double │ DOUBLE │ YES │ │ │ │ 123456.789 + │ hugeint │ HUGEINT │ YES │ │ │ │ 92233720368547758079223372036854775807 + │ integer │ INTEGER │ YES │ │ │ │ 123 + │ interval │ INTERVAL │ YES │ │ │ │ INTERVAL 1 MONTH + INTERVAL 1 DAY + INTERVAL 1 SECOND + │ real │ FLOAT │ YES │ │ │ │ 123.456 + │ smallint │ SMALLINT │ YES │ │ │ │ 1234 + │ time │ TIME │ YES │ │ │ │ 2000-01-01 05:30:10.123 + │ timestamp_with_tim… │ TIMESTAMP WITH TIME ZONE │ YES │ │ │ │ 2000-01-01 05:30:10.123 EST + │ timestamp │ TIMESTAMP │ YES │ │ │ │ 2000-01-01 05:30:10.123456 EST + │ tinyint │ TINYINT │ YES │ │ │ │ 123 + │ ubigint │ UBIGINT │ YES │ │ │ │ 9223372036854775807 + │ uhugeint │ UHUGEINT │ YES │ │ │ │ 92233720368547758079223372036854775807 + │ uinteger │ UINTEGER │ YES │ │ │ │ 123 + │ usmallint │ USMALLINT │ YES │ │ │ │ 123 + │ utinyint │ UTINYINT │ YES │ │ │ │ 123 + │ uuid │ UUID │ YES │ │ │ │ 606362d9-012a-4949-b91a-1ab439951671 + │ varchar │ VARCHAR │ YES │ │ │ │ "string" + │ list │ INTEGER[] │ YES │ │ │ │ [1,2,3,4] + │ map │ MAP(INTEGER, VARCHAR) │ YES │ │ │ │ map([1,2,3],['one','two','three']) + │ array │ INTEGER[3] │ YES │ │ │ │ [1,2,3] + │ struct │ STRUCT(i INTEGER, j VARCHAR) │ YES │ │ │ │ {'i': 42, 'j': 'a'} + │ complex │ MAP(VARCHAR, STRUCT(i INTEGE… │ YES │ │ │ │ [MAP(['a', 'b'], [[], [{'i': 43, 'j': 'a'}, {'i': 43, 'j': 'b'}]])]; + ├──────────────────────┴────────────────────────────────┴─────────┴─────────┴─────────┴─────────┤ + │ 29 rows 6 columns │ + └───────────────────────────────────────────────────────────────────────────────────────────────┘ + + */ + Map map = null; + int i = 0; + for (var block : new ParquetInputFile("parquet", "layer", + TestUtils.pathToResource("parquet").resolve("all_data_types.parquet")) + .get()) { + for (var item : block) { + map = item.tags(); + assertEquals(0, i++); + } + } + assertNotNull(map); + return List.of( + testEquals(map, "bigint", 9223372036854775807L), + test(map, "blob", v -> assertArrayEquals("1011".getBytes(), (byte[]) v)), + testEquals(map, "boolean", true), + testEquals(map, "date", LocalDate.of(2000, 1, 1)), + testEquals(map, "decimal", 123456.789), + testEquals(map, "double", 123456.789), + testEquals(map, "hugeint", 92233720368547758079223372036854775807.0), + testEquals(map, "integer", 123), + testEquals(map, "interval", Interval.of(1, 2, 3_000)), + test(map, "real", v -> assertEquals(123.456, (double) v, 1e-3)), + testEquals(map, "smallint", 1234), + testEquals(map, "time", LocalTime.parse("05:30:10.123")), + testEquals(map, "timestamp_with_timezone", Instant.parse("2000-01-01T10:30:10.123Z")), + testEquals(map, "timestamp", Instant.parse("2000-01-01T10:30:10.123Z")), + testEquals(map, "tinyint", 123), + testEquals(map, "ubigint", 9223372036854775807L), + testEquals(map, "uhugeint", 92233720368547758079223372036854775807.0), + testEquals(map, "uinteger", 123), + testEquals(map, "usmallint", 123), + testEquals(map, "utinyint", 123), + testEquals(map, "uuid", UUID.fromString("606362d9-012a-4949-b91a-1ab439951671")), + testEquals(map, "varchar", "string"), + testEquals(map, "list_of_items", List.of(1, 2, 3, 4)), + testEquals(map, "map", Map.of(1, "one", 2, "two", 3, "three")), + testEquals(map, "array", List.of(1, 2, 3)), + testEquals(map, "struct", Map.of("i", 42, "j", "a")), + testEquals(map, "complex", List.of(Map.of( + "a", List.of(), + "b", List.of( + Map.of("i", 43, "j", "a"), + Map.of("i", 43, "j", "b") + ) + ))) + ); + } + + private static DynamicTest testEquals(Map map, String key, Object expected) { + return test(map, key, v -> assertEquals(expected, map.get(key))); + } + + private static DynamicTest test(Map map, String key, Consumer test) { + return dynamicTest(key, () -> test.accept(map.get(key))); + } +} diff --git a/planetiler-core/src/test/java/com/onthegomap/planetiler/reader/parquet/ParquetReaderTest.java b/planetiler-core/src/test/java/com/onthegomap/planetiler/reader/parquet/ParquetReaderTest.java new file mode 100644 index 0000000000..b01238c2da --- /dev/null +++ b/planetiler-core/src/test/java/com/onthegomap/planetiler/reader/parquet/ParquetReaderTest.java @@ -0,0 +1,84 @@ +package com.onthegomap.planetiler.reader.parquet; + +import static com.onthegomap.planetiler.TestUtils.newPoint; +import static com.onthegomap.planetiler.TestUtils.round; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; + +import com.onthegomap.planetiler.FeatureCollector; +import com.onthegomap.planetiler.Profile; +import com.onthegomap.planetiler.TestUtils; +import com.onthegomap.planetiler.collection.FeatureGroup; +import com.onthegomap.planetiler.config.PlanetilerConfig; +import com.onthegomap.planetiler.geo.GeoUtils; +import com.onthegomap.planetiler.geo.GeometryException; +import com.onthegomap.planetiler.geo.TileOrder; +import com.onthegomap.planetiler.reader.SourceFeature; +import com.onthegomap.planetiler.stats.Stats; +import com.onthegomap.planetiler.util.Glob; +import com.onthegomap.planetiler.util.Parse; +import java.nio.file.Path; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CopyOnWriteArrayList; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; +import org.locationtech.jts.geom.Geometry; + +class ParquetReaderTest { + private final PlanetilerConfig config = PlanetilerConfig.defaults(); + private final Stats stats = Stats.inMemory(); + + static List bostons() { + return Glob.of(TestUtils.pathToResource("parquet")).resolve("boston*.parquet").find(); + } + + @ParameterizedTest + @MethodSource("bostons") + @Timeout(30) + void testReadOvertureParquet(Path path) { + List ids = new CopyOnWriteArrayList<>(); + List geoms = new CopyOnWriteArrayList<>(); + + var profile = new Profile.NullProfile() { + volatile double height = 0; + + @Override + public synchronized void processFeature(SourceFeature sourceFeature, FeatureCollector features) { + try { + ids.add(sourceFeature.getString("id")); + height += Parse.parseDoubleOrNull(sourceFeature.getTag("height")) instanceof Double d ? d : 0; + geoms.add(sourceFeature.latLonGeometry()); + } catch (GeometryException e) { + throw new RuntimeException(e); + } + } + }; + var reader = new ParquetReader("source", profile, stats); + reader.process(List.of(path), + FeatureGroup.newInMemoryFeatureGroup(TileOrder.TMS, profile, config, stats), PlanetilerConfig.defaults()); + assertEquals(List.of( + "08b2a306638a0fff02001c5b97636c80", + "08b2a306638a0fff0200a75c80c3d54b", + "08b2a306638a0fff0200d1814977faca" + ), ids.stream().sorted().toList()); + var center = GeoUtils.combine(geoms.toArray(Geometry[]::new)).getCentroid(); + assertEquals(newPoint(-71.07448, 42.35626), round(center)); + assertEquals(4.7, profile.height); + } + + + @Test + void testHivePartitionFields() { + assertNull(ParquetReader.getHivePartitionFields(Path.of(""))); + assertNull(ParquetReader.getHivePartitionFields(Path.of("a"))); + assertNull(ParquetReader.getHivePartitionFields(Path.of("a", "b"))); + assertEquals(Map.of("c", "d"), ParquetReader.getHivePartitionFields(Path.of("a", "b", "c=d"))); + assertEquals(Map.of("c", "d", "e", "f"), + ParquetReader.getHivePartitionFields(Path.of("a", "b", "c=d", "e=f"))); + assertEquals(Map.of("a", "b", "c", "d", "e", "f"), + ParquetReader.getHivePartitionFields(Path.of("a=b", "b", "c=d", "e=f"))); + } +} diff --git a/planetiler-core/src/test/java/com/onthegomap/planetiler/util/FileUtilsTest.java b/planetiler-core/src/test/java/com/onthegomap/planetiler/util/FileUtilsTest.java index 2f480cbdec..9d08d2ba36 100644 --- a/planetiler-core/src/test/java/com/onthegomap/planetiler/util/FileUtilsTest.java +++ b/planetiler-core/src/test/java/com/onthegomap/planetiler/util/FileUtilsTest.java @@ -119,6 +119,13 @@ void testWalkPathWithPatternDirectory() throws IOException { txtFiles.stream().sorted().toList(), matchingPaths.stream().sorted().toList() ); + + matchingPaths = Glob.of(parent).resolve("*.txt").find(); + + assertEquals( + txtFiles.stream().sorted().toList(), + matchingPaths.stream().sorted().toList() + ); } @Test @@ -140,6 +147,9 @@ void testWalkPathWithPatternDirectoryZip() throws IOException { // Otherwise, the files inside the zip should be returned. assertEquals(List.of(zipFile.resolve("inner.txt")), FileUtils.walkPathWithPattern(parent, "*.zip", mockWalkZipFile)); + + + assertEquals(List.of(zipFile), Glob.of(parent).resolve("*.zip").find()); } @Test @@ -148,6 +158,12 @@ void testWalkPathWithPatternSingleZip() { var matchingPaths = FileUtils.walkPathWithPattern(zipPath, "stations.sh[px]"); + assertEquals( + List.of("/shapefile/stations.shp", "/shapefile/stations.shx"), + matchingPaths.stream().map(Path::toString).sorted().toList()); + + matchingPaths = Glob.of(zipPath).resolve("stations.sh[px]").find(); + assertEquals( List.of("/shapefile/stations.shp", "/shapefile/stations.shx"), matchingPaths.stream().map(Path::toString).sorted().toList()); diff --git a/planetiler-core/src/test/java/com/onthegomap/planetiler/util/GlobTest.java b/planetiler-core/src/test/java/com/onthegomap/planetiler/util/GlobTest.java new file mode 100644 index 0000000000..c7f7034d4e --- /dev/null +++ b/planetiler-core/src/test/java/com/onthegomap/planetiler/util/GlobTest.java @@ -0,0 +1,66 @@ +package com.onthegomap.planetiler.util; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +import java.io.IOException; +import java.nio.file.FileSystems; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.List; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; + +class GlobTest { + @TempDir + Path tmpDir; + + @ParameterizedTest + @CsvSource(value = { + "a/b/c; a/b/c;", + "a/b/*; a/b; *", + "a/*/b; a; */b", + "*/b/*; ; */b/*", + "/*/test; /; */test", + "a/b={c,d}/other; a; b={c,d}/other", + "./a/b=?/other; ./a; b=?/other", + }, delimiter = ';') + void testParsePathWithPattern(String input, String base, String pattern) { + var separator = FileSystems.getDefault().getSeparator(); + input = input.replace("/", separator); + base = base == null ? "" : base.replace("/", separator); + assertEquals( + new Glob(Path.of(base), pattern), + Glob.parse(input) + ); + } + + @Test + void testWalkPathWithPattern() throws IOException { + var path = tmpDir.resolve("a").resolve("b").resolve("c.txt"); + FileUtils.createParentDirectories(path); + Files.writeString(path, "test"); + assertEquals(List.of(path), Glob.of(tmpDir).resolve("a", "*", "c.txt").find()); + assertEquals(List.of(path), Glob.of(tmpDir).resolve("*", "*", "c.txt").find()); + assertEquals(List.of(path), Glob.of(tmpDir).resolve("a", "b", "c.txt").find()); + } + + @Test + void testResolve() { + var base = Glob.of(Path.of("a", "b")); + assertEquals(new Glob(Path.of("a", "b", "c"), null), base.resolve("c")); + assertEquals(new Glob(Path.of("a", "b", "c", "d"), null), base.resolve("c", "d")); + assertEquals(new Glob(Path.of("a", "b"), "*/d"), base.resolve("*", "d")); + assertEquals(new Glob(tmpDir, "*/*/c.txt"), + Glob.of(tmpDir).resolve("*", "*", "c.txt")); + } + + @Test + void testParseAbsoluteString() { + var base = Glob.of(Path.of("a", "b")).resolve("*", "d"); + var separator = base.base().getFileSystem().getSeparator(); + assertEquals(new Glob(base.base().toAbsolutePath(), base.pattern()), + Glob.parse(base.base().toAbsolutePath() + separator + base.pattern())); + } +} diff --git a/planetiler-core/src/test/resources/log4j2-test.properties b/planetiler-core/src/test/resources/log4j2-test.properties index 3140f703c0..7e7dcb0b5b 100644 --- a/planetiler-core/src/test/resources/log4j2-test.properties +++ b/planetiler-core/src/test/resources/log4j2-test.properties @@ -7,3 +7,7 @@ packages=com.onthegomap.planetiler.util.log4j rootLogger.level=warn rootLogger.appenderRefs=stdout rootLogger.appenderRef.stdout.ref=STDOUT + +# suppress warning about unreadable duckdb statistics +logger.apachecorrupt.name=org.apache.parquet.CorruptStatistics +logger.apachecorrupt.level=error diff --git a/planetiler-core/src/test/resources/parquet/all_data_types.parquet b/planetiler-core/src/test/resources/parquet/all_data_types.parquet new file mode 100644 index 0000000000000000000000000000000000000000..ac6776b6071d3d17e957e72ae85edf06d8edbfff GIT binary patch literal 3946 zcmbVPO=u)V6t3>+Om`-en1rpe6T%vnnI^;mCy*@TGOlX~E4u^7s0xKvQOJan`SW?`;^!P-X6S4LJhmvn z|B25#eE}MnFXR`{-58yWFzBXW0DtrZH6u?_8S&)KJRyB~>77?}l-3vU-QMj_(wTqz zODE60zxC_oA71S1|7}5^p>q*&lPkV#c!<3(N4SRG5$yl|_i|tQA0ah;)&Fz-+mA4H zeF0w|+%rW3KDNW^bVHfe&*uAIUp=S>%1IQvO+)zi{EJ>FF0E~Zu`Nd=)sLg6Dq67cIMeGy-cq%@eW z3ImNBB-N+^U@p?t2spbI11FHR)pQg+=U!_K!NA>VI}Vr)^br#U;GU^c;SIqZ!02=& z10!=h16-vIPxEv!M>Xj1;LS}TPgN16X&8%*O&Ji!C1Da(0?-6E8q$oXVFv#IT%czn zVb9D(z)C6)*a)>cb^nu5H(v zZZ;#HYUO&X4F2GWUT(GOj@=|wg^^aZT_>9sMtaq$)Eag@n}RSixlL?QW(olO12Z zJ~@*EZ{QCDcoJtjuHD!zU8}h}B^E);we-d72ui^?q)l_PdPPrne^h&F-TX)`+WKt_=lS+H*OG|w` zaG`Dx2@k1D0aK@DrtZ&LWQL}vD47cPHC36)w%CSc6io`XRer((+h>}QxkthSF}l~d zd;dM+&d;)Ct6VeTke5cV+A?pdEIB~m6% zih$`b0Zj5H5%!R&dp5=-kC}k#ZlZV^5PW?^yz=Y`L)IWvPdhZMIx z&n{X=T1an*bmF-caDO}m+;|@mEpmsWn5^KC3vBPAEnZhdi?-p&w-sBCpNHHIH{3xI zvL|mcVGpy{JsacccY+4gM@TL2Kw%H5yJut6;h7td9qNxx<}aatq=u;is>q^Ys=Ag? zq)PV>^gzXd0V_D*T@*VsIQIj)Bee5F6sm$^5Xd?lM(%DY0w30jz~K9V28Jvp0sh@o zZwF$#@;bw@FR^v&8^rp8#Q`G=4~HzX^VZ==(Zu5(Zg27zQh9798gn6Dt@z0wp83pR zud%ZA3$SW%-BanSjfnnSe_{A?U?4h(ajZNFYMRjj;!*;0wzW1{G7O1?zMVE%DYUTHK%EC>lXv zdxhH;Skb!m1f1SBe8GzUM)8+r@r|XvD~lRiXP2y>kf{tz1>Qyjzxe4%?+kn25iw4xZtiVlfP1gtigZT{{!OA Bw8a1b literal 0 HcmV?d00001 diff --git a/planetiler-core/src/test/resources/parquet/boston.customgeometryname.parquet b/planetiler-core/src/test/resources/parquet/boston.customgeometryname.parquet new file mode 100644 index 0000000000000000000000000000000000000000..f6dbabb4a1ce7de6068ae6adba86afbd21a02150 GIT binary patch literal 13032 zcmc&*eQaY@c7M-t94C{3FwFBh)IgdUZUf^a&);#%(mdOV<0RurGMP9rrQz9rah!Sn zz;@y|2#Hp7wB0`tswf(5y9l)^MF=4@TD9A5#ZJ+5R8=fnO{LY8wg|Oog%Cp1?QT`A z5b8PiJ^MX7PBKe8Q!~%@{W$0R&bjB~-s>6o@VjzB+v8Nnk4!7UuTNjh!?@TjxN79q<`@AkM zOyd~tBxkh)(>{1iHZLb|-o9JvTh}!_%y})rh@T zBaT~`Aze$;#z3|C6O)ZIE7}Yh6N&teaVML}QKkNrwY_e_X|>U&H+sl-dH|`$qKV<*NIwD;zkugDJbRJih}DQ=$XIJNBfWgjUZf2+ zef>E3q8E^AESea$L~7=G=QslmZDTLUL?=GMS+vNs8nIV-M}}GpzI6{M(Z-q>@?oEi zGpNDVwJjA(nY`khFR_-(EcxtS^4`4wPor<*))L+u`rNZ5TY=iiI~~Kkci8Dz;vM4- z_qfA9!t;E*-^Tlf9nk3e-&hxxH#-m~QJ!K`tYmriV611~dVRhTx?BU3Am zwB!Ov8gk9elUlAhwpB=pWofOP$x77gzc5&C8pzxCljl!h*7-W(TAIhndFL?sKw|EH z8ZCPtJcVMuAQj6QsWiGFWzrjEQ1~Hk(9W5)WP9J9NoZQ;-Z5Oy4grt{s zYyav==5y=#4yN_T1vZb0$nMb&!zIRI53;Zmo5UT z#+`?8=y<~fYhx4Wdro)2dxx1}4GekXblsvuojEZpmAVSWOjayn0GKS_XCVg2o2SU9 zgEsGBtXslwGNFwd(2}v&1VHX_qX|t61qG{3J|J~S(+VV0GM~-nbKMuE>e{v_Z%G~S zjp9pKHpmNv{PYnU@VXuOG;`FqX((R_Clf;*4cCBe{5XQG3BcDIoP8grqZv%!|7IWz>F|E`F#Bte9 zetD*yIDS5I8#tQQN`_Rjuml90+3Ny;@8mHGl*;mON3Bd0RCbSE^Ene9vxl1{^>3aZF4-8^e2 zodvV&8>$W?`O;H|3KL|pRHBgHVIj$omp$Z~8>>+(KDQIs_i!hg@=BLg$tw)G<a0unf84w{cAWBanft-5X+Q ztsoZ5igxpsCGyeO<4v^vFdhCwe6Zc=<4)ERyxI1n_wjXNqS?e&b8STXW-$)6=&hT* zgT(~@*>Bdp)@o8WP+DgHaGbn3Zqrz5-}YhNaxEHm-7IR=1j=8G<2Dl zCdkizYaf@P2WS0`;UGb;1~gr}Q4rO&?wbE0kZVFa zUga11vi6AI_4W&+KcLUm;rFIIF4JM}_5! z88}CY4dOnZBOlD|D-LQ&Edh29V3A3~IVrct@XSyr9k41Nb(@=5Jg@)FISq(nbIr#u zv6dHE@==t$6~!9U$mXc#errK9_c2M_mYNgBt$A`|eqZIU8@a0;wFA|k8H`Uc-RxD| z3uA)oyt;7hy5sThe=#vR{2lJ{)andTlx zfkrFc0z#1o*|pmR6pz5_w+t!=+F3iK>-)6xY^}Ek$aRNEc7D^qsmCY4 zW|*@p*tDY*XGAi_ibtxN41}ocrZpzY`QtGCdQ=SR$wbeveoWAJHcRPJx6iHQaHx9C zl8PLvO2Rjn@u5Hd3%1qr4Da)+eZYZO=;;+ominJoVp_tZWqV1YX zpMI>dQ0Yf>m7>;-8aJhIMf9FjTcbeny7X7;Mn?MNq;SzfjIo&dZT(-K`hWY4f!v9o z1IV4YxhHqxlAYX%`|vw(hfVJOz3|<+vn6-$4wc;1TgqG+nPIb!$OX5X+pmKJqS6{)do50Z3Q{n z1w`Ag3CIyo{^ZDq`F5Vd$x#m0 z#|IrF{0Kj2A5?!Q!@IPtYz_wM2UZwVI!lA&&+59N0YCiZ@h%;v$7TaC5&!aKaQw+( zAF%bgy$**DeyGQn$L@BHxIA8myWZurtJ9o5x6`k_)M;+`VPEa>J!K!%-NJyVWPU3L zw{S^0pS)Pflq8Ug2WzQZtge--1qpCIIXIt}tLc1h@TupXdycX83X6;9V&lqz+WPzh zqxkZrX9Pi5fCpc*g5VYcX<ugv7)F@HIdsbp$n$!f$aRcE$S;SD*NP00x-pE;K+W#TcP|E?>8kag4Mj>C zO$6v7o^wkVl}DrDVP2))zm!UW%1lWygEbU(g;hbATY$BMMTNN`;jAD;(?LN9=5eZo%QGoJ8wQ~5z-64pWeSvVz0T2P zO+!0GS2l}6!Z1+~C;+;ip^$rYbPT#CC#e$z==v-EfHQ@Q@C5YhgPr*^^P{6H^P8ii zIDO$PB|UrM+@ z-^dcspK+sn7!3p#;avf#_$O^DAd*@Cc3k#1t{+uDNa=lb`3ipE4lfOVNGCrYmaFkt zlow&K0;CT zPsFCs-wLZSG-@D%^y`PwQ~Sp^R-IE9dGR-n7ok?0&b))9!3w6gm9f6;&Ep? zsn{=%^sYL6r1kn?Z`e^t$o?$EiAuZVQ|tvwApbzftJ*Jgfc?s1%%k~`5D1^yR_#Ao zrw<-LzXf?{J)Yg!PPk@vL1_LAq)rqsoX58Suks5BJJ`m+gI;`7USa&G`>+tq*5%LE z<%e6#r||>p9$im{r>d!Gh+!ePiuAklMRolZ`%xh}{|iw2?Z^;6C6qrl4@9`8!X#aV z(K@~2KdtGbC6Em|BR$Bg*6cqSrSTf1OGly8d=$d`5*R}mn0N%?0z84O)tEz0qU<2j z`&U)_1ws_0ki?may(lXr66nzMj3W_Vrt&W!ezztc3zw24l=?&G&!cJ~>KZv5SCA?E zs6Q{4;(#TQTY&rl{Ki76KTWFfM@^$pAw9KcU{{PSV%$MKMfsz(^*FbB3*xdJwV>ddQ<37!^;Vp z4}(0)dj=lGKY1EIx_pK;ih7btPhAh~gS9nT30*|`@Vx6+vQG%-L%ae`C6!dLKvI{X zR>1FSeNg-dJCs)lKSbxUN0-e{yIFGTZ?fJH)$AA}ynqyhCEUP`F9^#5HdKLnvj z>-RzV6LPmFi!qSMM$xyNqu|Q$85BZw`hEP1<}a*$P)8Tn#TYeh9_d%+A(YUE7WT_RVKUbi3SD@bU5)~I1LwThpc{+Oo2sM>VFIVDWsFkq) zf_gbg|sh3NA;Wd(V*nsmY*FOB|G!HpI4)2Vd2;$%y}j@mm)+sY=J z<{P@`vj{jq_g8dlqQnawU(L>J(=~*L+|$8{o|Cmlb^ z4^gydpc^p5D~g~9y24055Cr=uil7LBp!3qg^@6NFigW?GVaT>F2(qFGionSFg{}yK zJuCW`|Cg;d{Vj?mI!zpXqa0hlYLBz|mp5 zS;|<=wgKCXKHIO%^fyiP;QbOh>;fZERx<@(N8v_M&>pl7Em+OOWR@Z5Lg2$zn?7Q@ z1$&im8#QWZLHEHCWfZ0XivH|?!)jIsabQ<0SEOQrjO6ns+cgvYY(M=|i{m!jx~lgz zqfggmJIG`i);-4f$5{6|hE8_M>x2K}4r-YhVh zSW_N9LG0pFyTY`U9+}z^(v=Hn(&EFbhjbqvu9Y%;Rotvfd6Aglmf7~EnSN`Ke(?Z$ z^IpVtHP74q)_2TW$iHuX|C29+TIVh~OXXroELR)ODW}ur1YP)wScS$?Ni3{Y%c59a ztGoR^k8d*Q_MP?vNGKMvQbsHYqI0THljX}UU^|>Rk{*@B%K3>c zQOa#qS&14wyB1j+S&%Y(*~w-p)4avy44tPe7m`1@PK{2oeh(XDS?07acAfh0Jmp|m zPr&Ut?O8pKq1HaRlJ}~-K~Oj4FW6r;-7D+6u#0YP)$#@#wK%|FtQPB|g}d1NLVJ|` z%OiS}9jFv*WkIZv2z&dG?Z*e`_aCBf9mWXztsPrp_X5>*`Z%q$mi&JA^bxRiq*%xl zWq#z0RIKc@XF|Va^hgVWy|oGSN-O#mac8EamxfgE;J)Cff}Y z{f%MzJ-g$MoV#idseA!+#MKU`(nHpV^{nk5fX)MUlAS4ve$}DtGNkAt6!UpCL9g3V z^c4sF?jJaSY`>ze9n(b}Rz#I+vKpl*>4%TeHy-OLiLrVYaG8QZ+0P$_K@@A%bWM^o zQX$t5O+ng~gbvS8qlXQ6_I`o|*v<8+&q9B1L|3d|kok%-ab7gpuAAt0j?>Q^$BEO8 z&wfpE!>O-~{j#_#$|M5N7=PxZfAp~)#%Q{fB3lo;iGu&I1e3e&UK8vePB^U8@JVo< z{ooc~*(~wpsxrD|+dBQi-c&2bi0Y|gBnDCvROvE z^ft}jXToW4opx9csoR19Wz`^~zizVKG|^u>N&o7ELuaW!^<7$*`-2A7QRO`=oBL zLwTvNX)qdw%v(>;H=pPcGGu>uw@KkWmp;~mD2R-;8JaJ;=x1E_L(aFy^@%rxo9v9Z zDHP>mxhd*1iGIaHzsEYd^V)9!f7!1G@Iyp;MuLrIRXY!05WnrCU-k90%SV_k zzYOR@JxYWX%aT~A@-53^D6UL3F=;xm5Iry6<>nm_- zwv#3IQ}V6rr`+FtZRO_Qf9Lq){|NWLE&Ox%HMsw0?)_EM(V-_IfA^?qWauycjV_r6 zhi<%i>Ay|(q0QN$FPaAbIeUyEmB5a9y-pWwcp_68;`JJQ|9H5HT3{9q&DgITk+dFEOF6p0-Ws0uV+LhC1Z{O^ujp^sI4e-Y zHdevVx0}isTegLoOdMqUCe8?J-EtWrhCPqgjY_s{MikK%JDC z=zFUv?Wa=o;43)~rustfqo6|HdyRzNSJ{K!ZzY4?Uvc8Tmk{WERt4yNAN23%37y{0 zq5C)Lyh{Hj9rfr3dMcwI=mCm;pl2QWfgVe2SI2Av^q}qaCmw}L%PITe926d>lrxI` zfBG-M%V4>dFI1dUmz?;n_>!~QD8YMSC%j9B_cG4QE~om!-vEmis$xzoBhc_V+<>4H zt1mJ;VJ>H-mR4oWuPw!pS<9=mPLd{u@`9@qp*K(Rp6c!h$or#CR1i+y?-(EkGc;e# zbO9T+3Kfg8$QN+1O6}i(zbf)Fu|o6RgaK%fh4uWN#0VtC{_eEK8odES_g7nGy(>;j zJ-jytDNOeUH!H^m6wL(f)m{UNt|~zymDsLeL%q}hS0r4%OxgROG-Q9K?~)$hPWQNb zoPn46>dRbwl>u`DCWBT*BELUq&X;B{%o_MLMG;?Q0&>14SEW)>ZsdvuAi%WjO>;@?TrNB9jID3%#lkZlt$#5bt>`pSpo{vJ)_0Ukr(poE z-t1q`WI*P2+LMxnyj)8)qJFW_)FBWpY*l$2r)(UHoZZOJ?~2nfUKO5Pt4m_ST|%2B z8U~xhgUWcoehPdZ<$1x5WQ@^#MozPCIWsr!OS1D)+LJ>YC~%np{&ICAnY)dA1)fa0 zgFv?nW02E@)vZiK^&j{nlkhRAB#bc<+~z%*lE9ut8M@!%K}Jl5WC6wwy2pr5;<`_e zcNQZMLn{LM*10G)AaLMLf-s6O3?y!OXBwUY?Qy^-R*3&Ib89Uqe%!2;J zUhZ@}n1@*071SJ|`=C{ybV9Br*)5q*WQv*THXr2UTj^;MJ1{?e*;IaSH|>euc6_P) z?Dj@tRZgx7m?ItYshiH2%XbCH zTP=Kz`hpBt6SBAWt(qga3M7G00p@9CsSw)+n(D?1w}Gi)18<>~2*-tYLXb~t+*!yV zE3lS@RH?v%xn7|yi!jMj&#s+>7cp0DO5FtcBbJ*q>Y#VbW$>#`}Lw=s{ zlM@q@*bEaUusFkqJmQ1GKjg!FPnw~?q=G-g6>$AUaoi$NB0>Qz3!s^UIQ(tZwgcAGJ z!BHgn!#B~Fh*tDRfd+dKaCq`bAYE*uSM00PLm;U9LM51URnnZ^1|tOpP#VV@3S*#f zoPb}-c!{ZrK8g}DlS-h^I4+LzyhxA!nTZkoAUd!?zEM?0Xjl3zqz`U`WTStHe1m`S zG2}yjxm}N9pBxwk^i)I%5Y3393B`uuZ+*Pj2OnfThe}QRC$eO`(_CmA_RYGL&KWW3h z7=l8{!53j2tEWI?H_?APsQOUxgPn2mcgGF??jKW5z22QVq2+5JPY$2 z_gbZmM4gKvJ$om5^oLHr!3Ha^j-hxih4j8A{d9N#?I68z4TN_1tm=P`TSHFaMYSE` zlzh?Eel6=QP!%Bj$$Za+C!6$J$j>YR|5SeXc<}j6q1k9H^P5DtG zRAFk6kB^Z(8P+Qr! z*n7)DJ)A>*XJHO1`BstFZZDEoB5nK697^U^fO2B>PV{8FDt#dY`vy1c{YX4`93y5``V zR)~=t15+8C3-LkDBV^oayf^bNsN5z0_V6F>8Mugy27u6l6t1#_+aFPOHWKC*Mm*#2 zw>hqnS_NtlBk-%_PuIlim{2N>@#S){KB5*4N8Ip-IpYkxh=!j)-1q-f?PCk8jN>bfhGTv?S|g@nJj0-SI@7bbOQ# zNu*~W8_*yP1|ul2BFNgI+q!Njf*=Tt&P4;)3%pnl3)meBJj3$C>}`MaB8vKczwiC~e&6@|edjE2OKz*hdedaRY?`(9TB#!xrMxmkQ4|vt zSl+|<{hlDttgWpvEW_{tpBQ9BPsZmJhSJ9=SH>N5dqaW18ZYwC&rmM6=o3N#zsC<~ zR?2$RYIgzC!|>^K_^IKcZcL3l>|HU8TtTw~$IbA&*JSQFPSG#)IIIJMKC1WdpxrEI ztY%xU?OKoRS7!PLCc5uY2_1BSlPIg1f?tQ>i=qIwTOAFT4Y*kP@;hGmJO!V2%k35% zwOerBMhzRFc5A-X>#+7|*7OTPVP|wFC+CS#e`d0M-$Z|ZAN?N|$6XlSTT>@!;eyY8 z_;hNDqjn35yn5l zx>p$YxZ6AK4vjJllOA+1fe|+hvg2MK{ERxN1!jZ^jxfFzcL1=)J${1Nz6O43*sd~d zrAMZAoaxF1G-=5-H;?MM=3up$;VaUGiky>3)PHWa-89p0JVn271hdY=i0kT}Tl=gZ znHzcLw)yt+S3s?^7oEjYp(vFqHRrg~>2iWDd|9dhaJnevmn$Vns?704m?=qOp_I8G zXJBsI?e}?n!I0ZG;fE=)kYAHCQeKpt1!Neh9SeDA?#?~b{yIx^sYWU1@ zba`l2&hRBCyGEJf7MC-8j$e15(?%D(%w9%cK>g=$HZ z$|S;W9kBh|e){?#ea(gu_G>@3#O`^j>-KS4=|=MVjfq3x>u4dLDJcBVDY;PIY|n&# z%kZHV1bZt8^lCorF`LX3{olv+r8<=7b5ePqBgvRyUNZ+)5m=vmx2-T3U)C6`C^%D7LF zwj_l_06OCX7k$OGi!+)_N|7V7-9*8UjT$E3`tq76`e&mKD>e8GES|l8oiAS~@}-Ju z-GXg}zB6*Njdl>w;Xm!Ddf0lju?zYfKY1UYBNOc|w)Zt=^uKJD;Vz?1xA&NE8@%Lp zSPy8&f?l<15bLj+Y}ZZnug2)V8*}I^^}lvu-gPZn4lyQBphJ5Lj@rp@n|as(>Du;~ zEEN5TM_;M~`D*UMno=l~R9~v+m|w8;`z)Mec4gRA!w1-_pZRo$9e~5=1%uNtWZv-6 zulRNe8FCM~-=y$%K)1CY1rb}Dq4`39{>c9*NHAVd_NWT?) z6rw&F*F`<4iORuAtR(aLp8MW7{nGfOkaq2)F6~gMP*}UrA?puL(x3e9uChk0Y$MtI zW)d@YmKPK$(+__YYb8DhhbpB#l>O%6{^q2;SEVpUagUk4{S{rY14Oa%I$uTFKsVk!TT#xY= zO}5J>`c8y?FM>6umCat={ob_h?t>D)CABAv%hUAjsa=hKp~rd$&Joa&fs%XObUX6- z)syabZ!BK_i?i3QuZ>Ji{P4$LI`+2!A3OBaTdxBAm-)Y+yaDikuDFij{J&&xFX8;H zkIgUsF~DDr{`FDQgzJ-k{{27t69B*dk95&=Sb6dKKmFv}0A9O!@!t{qM?YpxBYa_c z;G3qy17CAcHWTCe(b+FPy$-YA`QQWERCJZU^dA2=0DkJMyl6W7d;j%*Ao>n~{*9k* zeG_QEJ^aQ`ar&1}|Mnq-Va3wc=Uvv5T2sm1*HFUVhiafzP(Qnj!di~Ji3h;`Tfxi6 zf|oz;yZmv`jP>zk|GxgAm$;WhL*MzXZS(U#wfyyeXHFqb_VYp_fhuVvbZkgc&neX4m@MV z?$LIQ%f@-OUE2#cxV)GzGSq}iLojp|j4W(V3H56JELobM`jV*<%Arp;kFEhru z)}mmXF=XY&Fe=%WE>+PTX{|wl;ti>$F^r6iiGcE%?ac^;J4XOFxb#D*ZV6o%G{qP128}i})BifAs%;H;w*vYenr5 zK5&bwvj{!r-pAhB8T4K{6pyrvKtEh#U@zDG$A2AvJXdIX54U3a;V-)Mp6;&nUazC{ z-fozW{2oZ}@oGo!`L^~buUYh7ZchJS&;PbhPS|?sKHKZh9;K+kllFsI=pl@&mjU|^ z_g{qfI!ZN{FFVICI`Os7MQ5c}gjYOHcr64iR_7&`v-xV-FuNr23cScDR|O4yS}E`q z4{N~ZcOC>E?-mb+f8YGIcaH|{v3y0!N+nd*@Mg|{pc}8fQ|X*D8XsRiJ&(*RzCqmVI~wK;w8nO!0YhJ}wkBU!oR-$y?ii#;w)=FmiDf`(Skc~iGN9N;&`-5BOo370NW_8w?5*uQ4J=xoNf(>=XwjL zWyJiZ3MYI(yHpf%VuN;F%*VGfHD4u@*kSl&80eL}zfnPwET?Re@BETa zjGjUNtz>Xbz?W;w+u*DCmK1@1s+LfyZFtM+#B`agp=VXj%qZLGWmwzIbXj1hnYntb zjeZ(^9*F639)#YvNBKbiaPp0D3%(=k5!)!p%eZ!lj3UbM5UiP7jN;m^uEF|N&<{r3 zGoGaypYY{ld8Qo8!@9$or;}Uzvn^EuGuh59$NGFs$KeO28*SL~OY?Ii>m`$>{ zA40Xe+m~KXvD2GrH3x2qkRx~ZR};6AN-il7Gob%n@F5QhYR?macp(evcXA4A;|8YTX@t2=go8*rhxIZN zLsn-Hjvzc)1{_jIxtR_B*jNytsX0^xlQ}$q2oKPpD0oPIH`GEL<1!1Su`z!eeV7<& zphqSuKOSO8AYdeWVry*7jr54VxDAeg(v81}P*pXekE%ow@?C{Y&((>(C*WuSaCizG zr=AQ{B2|=#gXlvpszC0}J1o_aVGiX5>mhCZHUAM+ zh%1Ru%!ypMoQ7O0;K1Algg&k*7bZz)p-HdppRV-kSpaQam_&J-W3bhc$1d?o^0kng zhI_zVRm`t~{5ft0{pFt2!Cx3U;kw{it)cIEKW>3Xqfhrmj!N{v`5P zsPA;2qwl%0)EaD_&tM4tec@NDQe{Le j7Dq_yU`Xpp47q*Sl7Kh7@J|gM?GN?g5JfG(|B>`R6Imw- literal 0 HcmV?d00001 diff --git a/planetiler-core/src/test/resources/parquet/boston.gzip.parquet b/planetiler-core/src/test/resources/parquet/boston.gzip.parquet new file mode 100644 index 0000000000000000000000000000000000000000..9ca613914cbf705ecf23e86be254203620fb500f GIT binary patch literal 14196 zcmc&*dvqIBdLPO1BaRb7*by?TCC0$5!4FBJhb{B+$g(BblHay$K^jY=u`OG#)ysBx z2hJW&F*JlffGrd@&B@83VK-2^A&i(H9`=0mSZ=4>sPe&{0cV+bBGACU^QyVF2)BR->xh_f0-%L|ilohw^ zRDD!^QoH`+pWS)y2lCTzJjPn@)Ax^}hrS3qt2Wp6#F9^ZqkZI_cT&IKds}704ez^7 z{QAaCvzuQC!Zb8R|A4-}4(8qfza?rDRa2#H@-be$)@0H*d9`C>V_HV5_3N5+26JQM zn3waa3XafmMxVLSq&IO*T9{aZ$@xY{M>)T5$V*W>KKp2}9G;@*iqv#PrI9MxP^pp! z{Io(@qI|hX`E(IFC_`l{1*5VKv`5i$EanFI0(lj*if+!=rM~aC*ZJiK542QJ1F`Lm z6$MnIQe1)_JfKn-;8RJ}WVH&2>m%Y?o@6%?akrPM=`umw8imgnnNiGy0%0QcPMPwA z41HgWb``77z~byh=?77E6&*YQqMS}X(^*)QQX)#7D2mitUXEUby(lj}0;16O{k2R-4bL1pURadxh$yd$qDZafMR{vOz9>0XdFdkb z`FapVniPv7wF+CMjMh47`J%kTK|E8G`q$Pg6aWlDYa`t(We^qaq=Zk*LTbTXURLmP z6m<*tH+$B=L-Ze)sA;*F6iQQkEFOpmF|u8zJT60rHll@#K?G^WMe`T5#vU2{RDtCA zFbDyOjX7r2{=1Q>&}pA%49_ypAJc2K_wBgr^~ZjDnf$Z4y-w%dWg|yh-g(9H&MSt4 zZ>smacKF?jx>9QQRY#8v0FDCbQw5Xi>a~saI^BSlX*L+)r$MXLdQ|v{wqDy*uQd+n z8k=?IX1$3#DWsU(bh%w6rOUO+sNb95wx3pL~b9Zf@R5ZQEn}r;A_!I{m|p*!I$RBpKuQxFQ&cCrbGs zAL7G_GI8cvP(+#X6`ArrIr`!<^k0=&5e3;6l=Md0pD*br{OIV{8vBv84cBzI>c4T- z9haBiTk>LQ?J586tCTWo8@rI)R9N8h!G2qRl{4V?##HONqkOnO5##xU%Nso_u}^GT zM(k2>0B@X65UJmlDR;`z;mgtCA7QD_z`LOAqjV~NIsC*sI$G0c&>KzvxX)lnYCn;E zQ$II9_dwkPa}{c*_qL)dxp*sA%2AK*zWE}&+(RH-MYm^JU0zc$-NcymjG1AyEk;$v zw9aJI8=K5JV~eSf7Gw`Gk5@S7udSdCHa$}hK1ZR2E0@v3ni$VTV*V{;15u{jCRgs4 zqhr#m z%Y~d)*oDivNO&yZ=ffP1<}&4|a<2?Mj?lYTt4(EvvvM`vk)M^ZyGGx5qw}>l90Ok; zo!1;Kp*|dWg}oR+L5$Lt(Up{84;MaNW^eKbN8B>d;rUYMA^S zF)qU2nf z^2P$*FwRmXN`mxN9onW_U8MIK&jM*#&>J34ddG$JltBHd0llqXU8wIho(0s!#{b%c z_L^1~=hMxZls2=p@qK(^n&-ng%eCYQub9z7)9S*0^oFxguObeqSc3WU>(Q6jEeTUg z4;0uoK)yVrsdrp^WZv=GeCK#^>|y=u%)_!@(a*n%w>^-IbN0)yWUq}R6TW01=nsU) zRmEH&6@Xpw;+2K-`^RJy)qnR}WUu^uOGX+X33}skf_k4!xgbM(ZbC2Ih&Ka;WM47Wq)59HEj?@S~(^ z$af*7P}Ht__)7`T_pEA~s=OMqwQ9|TH@+q6jU@!F^eAW0$F0{dnynIs!#}Ysuz_as z7uD--Em7R7re50nT?8-oi*2VWJ2H>2jV;ohQc+W&Hy`?#AFrr2*IcR~_B>|G1eK~p z*d!B;!BV-KP%pr~?jkOR^{^#qztfKhISUxg^b( z!})(pmiCFie&IyBoJ;sCE-x*&Y5eoxeBLdoZ4HJ))FwsGPrA{8u2n;`txrlb6}b54 z=eKZ?U?e61+S!Xf>{&HHy9Ojc7m0vEa2$r6MplYB*pKe&Uo}kgLlUNSu}EZWOFp6# zgJ{p-l874UzWkMhgR{21+j-dG`bhCy`L*|NjO+V1#;MlJzO!8l(YH7m!HH*0bT;M< z!Lc)_DjE0sg1oX-i^;w1!iBB z0R5|5qySw{Ao>Gv@JpntLnKI_z8QV)SyI0%9i;pIQqtN60x1#;Ks~~n3+m0yXm0b8 zpkN!2zv6IkjI(PxTt|wi5A0|^wo#s;shZ+(s3(L~!dJeHVXu%~SDW)nXayTfA^a1W z@@*NK8%4j|f;SAg6R(PS4{?%t*Ye&JpPztzXqRtEHNL8#A1ul*VcgH(MU7RBDSU4i z-EDiY^1Tx?zd8Q&b)Da~eA`#`zk8lrIQ>M+={eVdmkzg_zW9;XXJ2Pe9&WzkN%kbW z>8;<(|M(B@KYIC1E$?i)|A_pd-+kq2DgMK5(}%P2k1yTx!3o95-~8=A-~QA(|M7qO z@`wM^a@(Ev{OLt@A#i>8o2u`870feVeE;l7^!UI2%NHM1UHbQ%_dib^y6MrGj+a#j zBelI-WuI<}HQw}xb%&lDO5ON+#Z`CS@x-gQY zzmmC{a;kx?QLZ8{Ea)~trpsa3eNt&C-CGXiS;ZDSyKs7;ON6=8B@79i zSRMdua%Y`@O)3@83QR_+aLFLe149%}XwoFY$#^;W%0&#~nK^sH%;_|{b1|1v@w`=t z4~<1nYf|7t&N(^rkuyZP2U}0rwu39ey|kaE$YkOURWNhb=1zE;wTlu)Nc$Lpd(KWG z%@gZx@rs`PEO%wiev-9rrMt1ji`EK(bzZhkQ(*R4$uZrHC(6!B!Yr8xqm(6;>dJ*H z|KDC{q6_hA5nYIveCR^FZbKL1#n%OR5rtO&4bJMkJwdDYvSM}L2dsSM9$na~`HGge z(aP4L(aIN$(FLgNqLnVeq6=IrMJrs?L@QZVJl7&1I*&pgy5My*bV2G`=zOYB=saph z==>@?=)CJO=sc?@Xr(I{=zQx9==`e#Xa$e#=)4Zt=o}B0%9m@EC8$jK(;I$3!Ru31 z5u8a?U&(H*7XPlNy181{pl#4r*Hw#ej&WTv84AOL^zjv*34QVE=3Av+xQDmp5&X6U z2D>{gFb@AsU3K%#^^GuTqrs%pHNv0V;LBKNU>e}>7<7hA7gHw=V;T*NS-iwy2KW;l zv8Qo!U3FGf2Uu~DWEd)qTrk2-O$XvU@J&PAl9-M10Q2VRo=9+ZJQA+neEaRUQ>qHK zudmzPES#=Yrz$o0YGgZDmhFXKe08#{fwhdY?JPUawvLlNizVmzH12H6Ja1*;F>b>} zW+BfVEIW-Kz&D=6+JqnDfo!({-!(eIo*)-g={uqxw^2JX+$YKk*!4q6 zzinuCgqcZ=FbTbPxGRY9Niqh<{Py6qXV?|!4#oq6HglYJ`%}I^hktBXX9fjYu`2O+ z)-r-`4XyCe&2lW;Gi`&%F0dh+U7u@%IMzBQfuF`z3#z~qi~1w+ zNn95?;ez}P`m+r&9ycbkPS*kv?qmD$hqWn9FZxeGddN=`oF1DJ=z(8HTK~OTVlQGO zA>K(o>c;uRPVxshDu5&SjT4Z!x8gRT*QNE><{}_17+==;1@!kx-CGrav$~H6M3;A^;ZhsIrS(~*9gdALkACV9E;hcc=>A>St zX?}fe1Oyo*=#!|D7)gi+$jQ0{to>lb0n9G~>LLDV?IZC+J_LWpQI_Qtc#UH%pr1aR zo42yvn7(O5lba6<`g4|kTzEXhE8rxo04njrUSX4h6~PMYgTR`EHtB_>6!^zH!&7W; zk6xqE5155~Ag(8Q@lWthL;i(4++l?L0pn}AG+wJPKk-*VUc!p#kA#N3E9D>Vn8a}q z#Qd5tJz2lHNd6HvX+rA306VeQ!}<;v3H5^hZzb}xJ&?+<*u7`UpOCvl9*6s33%h@fvRaXPX|5#oc)7sbP2p9A(IZ6RJr z8}_bdVpCZEVG^@TjFV8N~I%j_HX!ORQIfpAoOUgtQaZSYbRH z&EOaO#YP6;@xmqK%L(vvA|9&)n+ft-d*M!4uV%>D#rhZU;%Gf)2OoZO^GCnynKm<)9F%_s23$smPLs{s$M=MQM!DT zPsu@mEJ%&57=mEv0wYTUBj~yy2!bFmunZkiE%5S3;U+;=3?9}6#TFESF&Ju=2E`C$ zJLmf@c`qr_c5TVK@a4VV2*ki7!*W-4$ zp%>5b$9d8ZygT7GTHRc4cO#x!9+rkZv|IrgTpe&9x}=9+qfXy)k|Qs-SoycCEcHA0jt80JqAij_+;b{sb`n4(|8i-isSsECy^t z{%WZS?jIl9i@V0&KR!x6IRdy`?apfg|Ak>~CsA9Mm(ByGsFA=uY+ z{43ggwgA+DFOOK@A=m`_fXzK1*cWW}({|@+yZ4mMW{X(4Yn}l+R0^k^F8F8Vc8#_H z+t7f`wP5!^uhR}U?O5+1H`i-nOl#`*ZqmkNoMLNb-0 z!{vtF{H31!<~Vun0Ze6gBDJBl1b%E9GaAeNU4ZKwyO@5s`{2&zGTB6~kWAzU*AvOL z^#bVsO4Hh&xDAS%%sV>sXFBp53wh3hiZ>M3P_u9I^BT<_2KnFG?K}dH*=!iew@r=hal?Q*z=ymr^HyOX;-B)A=dR}gH&u1@Y5o8a)+ z9m5V#`XoQ3ajL%ea9PvUa#+g-b>?We9N%qWzQqCBVr}Q>RQ39|ZlQR4E)mP*;wyBD zHkqH*n_t$G7rM#w4_n-1f`Mc)lbma00a^Zy4ZQ>eY>fzm*e2F9Jfn7Zc4`kBu zj4X9eB{TU=Kv}ywGV~s)>0qfX0zF)8ZO{U5EhbK<=g43Dan<0xt!XKh$hT#4$&{4C z@nbZ<|9`3tt2$mriq2E{3$i_yNwhacN8ASevKA3aug%P2KngGD#CGw z&=||4Qkiu7W@3A#B+11@3%pUdQS${I`F$_>d9M|C{vmj1Vydn*6+D{JY2;{}VknR= z`fiD+mExk9;^rUd$i+T#{jr0@^zx}HrX^;j)?87p%Z+U%DVLPem^F-MifdRUDbTD)!dEIM4_BW zq-*9q)u$gg$tCAOg8agB8>TblA?vB2E_%pycT-R{-mjT$aS8C?Zu=p4WQv7oF)7EB z>9tlU#*(FE3^HHMx)G+lLz;e;R@w`C!&S3pY%!4Qr>oGRpDg9`6yjSt^S|iG7hdv1 zFV3Ki*!>~N zEjkYVnYmv2q#AP?IXaG9{kE0ox*maKXlY-U@+(;>S718NnWN8_HGaGsv)H0l^Yg%+T6I~Lkp>HuN3r1xM`jw76 zJ4~+piB;vSzBQ52&|i(VB05yHE~G35tYyDueXpjp2YZXoz>$l7)tepZVrpeo&SY}T z&TKdG(g^v~2fLA`d>i6zGI`!ceEq|iYRwMVeXeLMib3}ID7iA)B*^fHqT#aWHsroB zu3DlUUwSC#zH1bKDh03<&@tx7(oNKV1# z3np51NMD&EFHRkR@IRYTg&)afGOH_lM86v(F9e&4KE?YqiT95ny1RBwxcXU>qH-dR zV>G*+lTxs~k?RKuthd{%jTR%LXfQrt*LH1A#o0k|=GUbxn;zcKnLpN%PiM)6SzN+5 zV7bpjEREvixCWcj^d2X1y{)vt!XEYhG3xLrpJr0BDrx6 z=f+xY`|O(6mQ~jrN=T(decJkDiM+Mc)H3fI_?;GYFRMEN+Ft8<683bvJ-DC1P`>!~14m9k`HKIa(|_B|4WH;=`0sA+>iq87 zzeDR^U(o;OuZOu~5~tre{&#`Lpz4*s`qN*1YUg&ce{=V z-0-aJXFr9>9iA8xn-$%k+D{-~1+9C_%QzZ>S}E+Q3px@H5&Jj*tCbd|x*g+BO> z>RQuUB9lrKl)c@hR`VMM^6YoWtC2_h_nu34ePwfT7GplPo=8c2KgS<_M2|<~k8smg zkmb@_=vgvv;&>CZ&@<)gtF)!O;>lIg-w{lv?cxq3pROXe#9XL5OPM<-61`A#A z!rBC6SB`s7JOZm8`zRb}W#yDaTc;i%RBAhbT(eC_?riDJ8yWA)j#ve!>_7pVe!8Dw zQ+K1h0m(QE?n)~%5F*=5s7%-f?N<7AC>WHjo0eg9+oY|mm(r$e78=RnQT2)?nH;)o zui-~m$(0)P9$T)$nhD<3H6=%xDGN8|X3a{7wQ3e3tcSWnVA5-*?utybUH#0eZL6L5 zv@M#crc#X>*Cl->dQUd4P@s5CPOMZTBW;ReT&g3CGaI|CU0y=_ z@6$qZzz2O~pU>#__2BFd55Ml_EbHJ;mk#C(h}^|F4!QHQ7;;zVDdcX?M9AHpYwY`R z1G&p119I2f{{7scle^hvlRMl`n&0g+8;QyM6UT8lW9qlGz*f|9L-$mV@`oEer+e(D zY^Q8J{XNR}M?AkPrqa+**|kC&tIPMCeoC!{clf-T!EXf6IXvZqe)v~TJ*S@-@BmYf z(`~nV;4d}sa`iifQx2Eg?yOb`{Yp2%;}pEgrF3(`A7Lmpo+tZzG>0`HDwZjx;h--j zXJVV>WIh3MD^OPo+t~!be4=MABX6%|(mhW;{q)nE<*+zEKN~*HHr1BjcMam@5hq1a z48e`plqfny-?qDU z@y#=lj>xg}Og1Vwle5W*t?3Q17#x|%Po#a@QrNy8OUtXvf;S%z+lvz$nPfy*EJeeM zwpe;IIXyDD5f!%V(Xi|M#75RLk&4Sv!7j(gCta+(SOnTlDeOuk?V_zPGLbANms69a z#K?p@ylry?&pOL9=R-npVbl@yi)D77%g^{Hiox^epgy!UbG}k9ZcmR?X|{oGdv;;d zrQB7T8Naw4^jGU0OJ~c;sADo43A=2|OY@4ZfZefJj0YCCmxZm;vQThHOEWU!Q+1SA z^Wm1Rsh8 z@f^IaVc#IQLln>*Q)|%G=R~)Juq>9L>_3OT%b*(rWptAh%OhBxfs!~vg-H17r%tB+ z4$f^14x+n5AzC?(?JNlB8)6azs&823hI4Sx8|#}J9E7q@+@iGrT!{}_6sPeK36BYp zMQT)5EzQu2Bf zL5))Sp1vWg4Ui9JU5yux4exrUGPdM-@`4Hbk96G*g42x5${7hB;a%1^e z8vwuCz2)VlNm~rZ+T;ev8w~b=DUh*ZKRP~X=t3o2kiWrxfkhz_Mk0T?4~Vc%Jc~d6 zp$fecKMm<2Kdr|c=t`gmd6O0Uhiuee)Jbf-(|k0G`DC2t4+vBMNAbJXA#b0@GNrdy z>^B;uAbp5WGk;JbhCh~G%tH=@W?u;LudDJmk%%S5w>IQu?p?#6cxr-ppU|)ELZ{6sryUOI3MI@(U^z*jkUK;`Zfm zT*jgp@DG8K=WZd7>Id>LC!jqiuz#r{Z+?`5pq-dKsVS+G*m!`PY%jpv4^CV_c^NPd zjZgnP9Y6Gk#WRjFO;F$*C;GrX^O&2@i?c{Sv^-dw4;#j_#y&yp591YZQc(bv#$kvp zQqT}0n;)2HI<#pmOr;<{7g^dALvxP7LC1oZr5{QIf? z;vA$hR6BID{0X@`6@f8;Ybt={k3uM;{0T8wrEd~n2xT1RKFHsZF>f{kMR8;U=)JK) zC4V)EXBs*Yn0J?9Z4?LpWB>JA$wU6*Fn2qAu(q6cVNo1E5AyoTb@DX!roCt?i7)v! zajsm#xf;gN_fVb}>8ZU(sF6o#e@#44{|b?%$@6TDu!f5>a7_%>PnCm&*F7KHe#P0@l$93c&|DDD%ZQ z(o=n!%vV&N3+G-+It6nq>n~=j_*uLJXb0@iE|xFXAknvbFM zyB(*7PC5I#i+R|*b;I8#eXp1(48*e80V$Wul)IJ9VYl6ddqen~0$(2Q%|A}}5Xa5I H|8e?1pvEHk literal 0 HcmV?d00001 diff --git a/planetiler-core/src/test/resources/parquet/boston.lz4raw.parquet b/planetiler-core/src/test/resources/parquet/boston.lz4raw.parquet new file mode 100644 index 0000000000000000000000000000000000000000..b3ba777df8f0c5d6a7903daa9e6ed098b80610df GIT binary patch literal 12976 zcmc&*4QyN2b$+BMik9s}N&Gk}V^d9zZ0$j3iXOIkgO5+zxbNZAr427&rWK2ek& zf68CUL4a&Xjjad_Bgg_{YYSt@wxTEu!wAwe9b7H&jAC$+p$iJH+kzkq3d1N27juIQ z2)3Pb-+O!{MOj8fdyGiF`*Y6u&d)vf-a{wE&)Zo8dqc;*rki1nEb});s__yWnlKx|zP7W;|>TjD{GQ<=`+M)*-gV{I+dX+{k6{gw>t7xJGpJJ-hapH)@fMdv4 zYc=3}`(Pu^t$KT2?IdSA0jI{Hf#04uKLQNDhxavj@5G5C76XnUU#-=E^VLUo;;ggh z8;8k9hXAL>p@CmhoF=w&hSk#rJqi*rb{}VtXzS5pz+U+i`Drb9Iw!9&osl8$cUf7z zvcx*dV!n{d%0YQfXMRIRK75p1d=$WGxb;|C6StAZNo!ICAH`8a4nby(M zxfUF0$_0?rCYUQnv;=doluHOjaiy3_i!|JC>&?H?lQ*9rHy^`P^JSzpwGPL+XE5VH zF22)=hJ6g)xqLP!=8Gw@FtjG7l50iK_bIk_fHi3e_OTt4(6Y=O2Hn}F$4`?%*QO`G z{e!AnlO2U@DIXUL1}Rl2c8HRg7BfYY!f6W>HJP_{=AY}xXJ+!285M0Rrl}@hXMI&o zx(r($=z9Ij!n-8Jg&V%{F++z!smakf!c z7xRjZb9n5IQ3ojbG}~9>G=1NpO3hTup;|VmWt;3fTwrb10}S0NZauY!q6hL~JeyCf z&@Itq{!_jA3my6RE9AD->b|$crnN zRQ+MY(090Qf~CF*^cULTqs_$V^bGmV(^Y%+wP%F1Sm?;*Q)wZO;cqm*t21BGk+Vn1 z^Jey8uGY2-rt(CGMmBco9 zqVQtoRq~r-4PrpX4eXW%c z-g&kPVM(?@d%h$!B&v?GkWUF2T+&7}!npMex&BNmVO$-!4;Us%$RtZbQr;L4$LIaz z*Zr-;@!|3Nz|r87KRZrt4Ym@*N4Dz9-rU_MF|k+?#Z29fQgzVlHgb2Um6*;r?!z#K z0;HQNhW9w~U2azlHr83)wGJGF)sQU}W2KaoNM(}k(2S?bsW_yZy7LrlAqQ*bS=#9= zn5~|r+6?5^t}0R(mxMxrBD$nAzoR2}o#ZVi?nKSl+|ROpgzaj`Dmgi;Fyy*NJH)k1 zVp)`^;8W%$^16rIb?;&dhE;en^a^LuG4N+*`sg>PCiB-ZQ(_R61TeP2zhinUE= zV95EeYhLQil+r7!QZ}2H?I@qNfA~Ck^?5jL@5-+!zJ}fB9&z&BQO#PNa9dibu^0x+ zC*L4fzp+cO&?|E7G2tF^e#>i`9zZ!%(`rm!=E)m;3&gxPrmeR_SfogZEAgz9%~vJ8 zGDbe}wLsF@aZS>rilj7L7GQ7EmfDAY@{Yd+!fsAz!Vc%N+0~UDqApI5a}&FYI?i&{ z1ZTqiKPki{F>wH%a$9*J4aae*F^J*Zc%V9HG0GG*@e8+#^HUnGPKv9rCgkL-`y-wC zvW~nxNq!T+BWM#=`wZRJPiUe02o;`4!6~??URN=EZ%mW-rgyc<{V$VSGn%4ZR8}^h zf?JbNC+*SyN$(tPDoMB$Wp)^y>1(G07WqrHw^4^z^Ur5B7_z;!;69@> ze@92YKTB>0aZjn|vQJar3~8o5C<O(|MBQofA$k-pTGRX;iJ&L>i?I_-}Nw~M+Z**R}XW2 z{!a2gVDzuf=>PNMQRY#B(QhC5ao}0#`u<=1$-jSUXSQ?XU;iAM-}}YCZJ!AFC<9$+`{w_=@YBo8Hu>c*FMz~<5dP__g)Zh! z`9O>AjG3{5 z442+QYlp0fVNEbXYlziHVN-d;N?7#~d-X6^t?tM?YpWd^bG29il_lfMb12YYp~pPf z%R#nUK0xsZtXf7;IM7b@7GB$@U0BuLP9U$jCL(wFz{whA-s%-i1E*Y(0GobhK*pwB zmskUmF&4ZwsWK3vd}7v^$ajpr^y^eGD7TMVhSjS;ZD*sD4&{j4Ob&~xtCp1I(9>%j zKYDspd(eCGu@252@T{G|7|KjJ_{eUqIa$b~H52+JDDidwjxYN~+)o)C- zBQ;xt+Kn1FByCyrj_j&Zp!k}cr*vhUu%nXq~&!t()tn_X>m1- zJlMh(X>EgY2u0LH1jlAp5H~w7e`p_IcGO z`@X)n^5#wUbDJh#@*Ziv(r-2rllfQA9%0}oBTE}xK`pm+FZC)v@b`}N+K=0g+j<9j zl^?>eZdOWXU_iaC!kFAw=pA`U>xF0dSucyVa+n-B;e&bjuUUFWUKsQMQ;*Yaw|n51 zeSEnFoZNAT%WZd7ySM>m8s~9xUgf1sbHb1IN{{Eof!>;;3y6wmOBpD*#ieX~qmn9! zAXkODQryak0P}_3P*&PXW;4AnzWnmbjO7qNKR*{4kuTGh&%1~4<>9A!o)5ztUnh9p z$@`N01kWe=u_W#D`Rc|i*g0MuALC&d+wh`X==c=RS8xEnk%@m8hjAi5=fo$^R3<9L zGqbrE=SOB4RI1t!GnFZm}GTEZX9kjnvH8 z^m>fjw8tW@(^Km?&r~`g#W=f^n4EUW?Zpz%ZU_-q25A><#j&YWCAFNME{kJR?#PzS z5j^Rv%$^Q&!BgXopr5bE?}fswf2tHby#W2;&DqmxKfg6IR;Aejx~;iW8!qKtrJ42f zTS0%d-?4PEl8QN|bJ2*)w!Ad2=nB{!i={+hach~|EH86Khp;p&AwEqT{x7!r103(DpB&@3QE|Kn308#P%4rXQ9Q9Q6U08`l*wtzk{Lmp&@j4 zI7~Yyaa;}p`h=MHfaV*ybKN;K~JI?#KA{j0M56#X0JQ?DuVix7KKJT0xo5Y#xOAKrsL zgj1vGiS^~kytKsG*Dzj%$V7pkMEd^19`wO_`q^?U;N8ML{shtw#Z`KKjrKPB4|5V9 zi%6vJ@3QS!%v=4_>KE%JmJDq^C3R*xO9Bg80JrC^0QU>%gyCuZvgym_g0pd zrfqRtYt!o>Zz$Lgra;Du{pk9np$nbxg8U8k3oLTc2om`#{Xm3${7L-q53BS_{4}M9 z{InK#peunMqj+mzm}+HX8aLHZD1 z&Hh1&Wc=~WVgYg>4Ew@}e@&CWfkZS($nq(xe(VgQ{l_RiYE+pYO-<3rrUY0@2_EtX z@T*k$$?GqG^fmd5+H-O?7sYi4uOiOxzc2h$-g0`;mf#kpSkj+Fc@v@2b@94G9-1Nm zI!EkMEEDGE%gX$uj~`O<<-PosNKe3iNwL~+zEG97OMXG60-I~Gbi%$INl4h_1O8!9 za$yg7R6mf1IRWiCh4ag*y!mkof=-g{Nli(eB(Dd^$@U`b{ouq?C@%}}R{)Id|?SlLP^ZVl}Ucbzr#;dF^ z6-Dhwm&W95Ik7aoj_W`|dBaFg_pe!+f8;}&oO%%835T8&uYddVXq*cSw$m6^R1Ft9E|63gF-%^t@J3D!ydZ}?}qR*3jE^uaQ>M8200=4e^veu DVt*3= literal 0 HcmV?d00001 diff --git a/planetiler-core/src/test/resources/parquet/boston.none.parquet b/planetiler-core/src/test/resources/parquet/boston.none.parquet new file mode 100644 index 0000000000000000000000000000000000000000..17fc8cb7af6bcacf1c2876963bbd691f142e01a9 GIT binary patch literal 13028 zcmc&*eQYDwabHptkEgqHE@v-Gxr#08>{r?PS}wmt)pl+vN~At0i8`Gq(JF zeuSUW83t0rIiP`I1OaLou45Phniz&*7(w8=P2pS**Zadrkb_$U*RQ~6gWwPpMtf}n z7dRN5dArLcDe`o%r0X7+-1jjvznM30-g~pUC1KggTG$&V_{_6bmN~;P{S3o!;|X4L zabB-$T;w)4H#wf;M4v|*=OkCs<4$zylXQ+d-F}~MQ;vor z!yx%I+}1uXGuYowb|YU$+a8Tv8HO>#Z8e#@9%ab2u0Hl;uZOXo?sb?mNtUr&?H_f) zhkVmaY=F3%mVL{t6KM4(@Ak??s6%4hyy+TcgyB@}v9qqHMC|^;6`HQ-IQ7 z(88`QN*imNXU%lBgFJNEGwc~-7CJ0gt9~OpqXbXt%raRS^7-jL)~rpeV^z6W%H&mN z{Eo@~9TR!^G`aZ|fM!s)@M;U~Z4>);Jrp^Bmgjiq2YSK%%c^98wB&d8;)w46z$ z%b@H->_fw>&4{mG-O~vzNr$z|jDu^ln7Yj5M~=Em;bWzIr6|cIi;^jokI9Ohm2+jA z#%C9lwAnA4;6pC|0l8&ICEE&VtHsmo)4CR?K=@)o&Ydq8Ww{&?3q4SA;e6HIV`(>e8suFaK~KjgG;%i6t3~1`6FU!H3*@S+(L*9Ofy!*92@15nfRTo0YJm5xOgr5i5 z2OHqq>JX`Sbur|(kMyxUYT#(^TRl=fx0y-GISHqb&HkMIIr4)?$hRNq^V*oU00!6r zJ))r-h4Q&%UJ?7}GWpUrAZp%ZSO!lub#OEn(U5rC7?YVHS0AmLaJpmRm7CK1|W#H zhRI9AhX&%-=v{zl8H#sC$t$CWhT!*HeHQO@9*TrquE}z)Y3Zns%X=L8E_Y~vUUS`r zNh~FZD|GrmYh!6(<_{Il5^t zeHNQcT^8~iPaP&SD`Kfc;k<9Mzi%QhddLTETyWZPxr-TmAA4Wh@WGIO@ENi{ zl6{F>8Go{cvKP9;KU@-E6LXq9t%vWQ{|v7)=UOf6u<0$vlif1drdC7MWy0`%<&W!L z>agmYqI%r^V1j&mqR(JxJau5*b_g1_(>#h6Gc45uXQ7P-YoV9?>g(jn*E@I# z9(@~k0G*}$n_y^p9Hme#>#q2YKwc3JEacruV~U-`Ttrgdkn&0%_K-02sLcNa$c?~( z<=hS$a=xO;$-?FzRv%+_T@8}Ag9jG%$+RKrR5735+}I=N^=b0OR7XLh?5G~haA8l0 z2}MpGho{1BQOv>y8xXg|GV|c^dZWXtQW!1^hlJZ<15^(MRZ5EmHN3uSvcGF0uZPL? z5N;{ka5|8;-kLSM^*$;(nSniRSwDl~Api9&d2_a-LGIRLugx1uo~DxW#S9#P#3os< z%#&Ml9c959p~p8|I#O`JfunMZxtYOk+Tc)sG)r4ec-FqYV1Q69ZTRygll_v3d>$bm zM{wn6W^y1S|ILzN0d$lwIAJo>MWG62LC?yLu@}gyz<}JzWkr5e}(d&Uo!vu zr>*pD1w8PVp(lXg%|H8-fB7{M6v#jRDU`qW^MAg42^s#^^EW=i@*?-spJ4gblhYg6 z-_G=X3;Q?Hw-a#r<`>WW1PERupM3Hb5d5w9@xvu7Un&pWgz}Y;?`OY<@~hQfY%_R#O3+(Zu^(R5G&E8+U6^)`&O(wE7hP6QpO)AzO#U1St(_{5V ztTjVlJ?*3NY)pL^%=PR6B%G}CPohAJgKh_5@dw#@J^{reuzFrW<3KC*qj_VUafH=t zdw{&*=!o3ajVWtYdFv-P1Dtkt0&M2_VHKNkj$$oH#yIdirOQBw>bBZoq8>r^)2~Ov zpdC&c8P*RAjg_rZj%i!zc5+x$UALqvhi-J6_|d(lUW22jZUZ3>!n3hyV<N3%Gt;=L%Tl2cs*wV1L=+&rkOW;>U?}^bm1&VKooq9Dg zY7A~Fm--BY9Yb$l@9neyZ|@PvQKbFJQDo`KQ6$yLQRKjnAa_mfeG>TIBx%XLOPrE> znzuakJmeAQ3J){&cesf>^66ox<;dYD*~lTM%*fFuz{ugIvko=w^d*x?2T2kgeIn;* z5-{WdX%+J2lO^OpDG+kd#0EK7y5i6i3*><50&?Iaz+s-?$wAKL)0g9$C9S3o^ibW?(6=?56U$foETM;RVKV zQdnMIj7_MAX~*ZiV|e+5SwRq@aN{*62yP*e5~c+qB}}GhT_DiZUc<_%di$gR%~*zu za-r=rf>6T-_(dkcacstp!lE0Wd^5SYBIOne3ErJq%*^b}ZwZyiex1E`voZU+BJI+MRb8%*?;G4-Nl?3lplHpm8T3)RH z?Y0>6g>k)Tjh?+c}c;7lcQ@dDIG zcNQ+{^}_D_WSwRg=yn%ZwmsTir&$OJyOChM-nDkVmPxo~3-OqTTVGq&bOr3L)k-q7 zy1UNrRM+{kOI%w}5TBu=vY8AkwfI^jv#69Z7efA$97|Rcnc3v#n$r&k45BHqzYti* zNB1OrEeet#EY(8L9Dy+u%EASBOkv*$42QtuaLlEkYru`e62-DmgL3c!j$IyyF;v50 za$|WC%L`BvCaDk+AA@v|>3Bz$w#LSAxT8^88OCnKja#ottR;Rro^YyQV7N~um$w~$k3*0 zKTZEu`E=AY`DO4uManAa1cI8P^yB-`FJaesc6w_)wydo2&NTX~7@IB$VWb}_?L!}F zreCNgLjGN>6XuY9Ow#FvG_7qNKa5F2BBoT<_;?A%g3<<-s;a$&DXQ<(Qc<&Cq_h1( z&O$;ld`Or|cy;?n>+~~sp!YZvtNtwDuSz*kE41ZP>4gB!9wEG<@tX=#f5M5U$3uJ) zaOwo=J}k@`^7D21>+R)ZZ3xEQ>94J?&2kdXwb?C@Hx?NJQy^o_esq3P*M&;BAbx}W zLaTf{hD5>I5D;OVa2~&c<2t?OKW*tDKBXlW4kgfoyjk7;QI3um9VB(W(|ELq@gz*+ z2RJH#qxd~(h}##jOzEAv{iY%mWB~CstRIv}#V_SnOArI0IS@tsX+!=t644-`%BQS` zurh-7pQQMxQB{64G{s{(3Sg-u1&ANOuU6-$&c6`SH^eV$&+tMaj`I$#GWH+3EBsX6 zdUlmd@~cWB6-=SL>7|QJ{<=pVnj!=`$DB$c7ZsMPTK`l)7}MhAoyIGford+2V&$;E zSeMr!zMxW}opd6bbgsve3KoS>a2%Apu#Y^dAIQU)fcBih{#9Mx@)QL@JE``hrlfHIMUPgYk|feb(5xs z9t1dtUJvU#Ty(0p_5UE%Us!@rhH6Le6h9$$XX7vja8CuWf(dYCls_Xx>hvA_3!#Kz z?SuFopY|8xP!uM&fZi{SY4NLrKU3F$Cr<<&0bXZ2r1m0 z!O%#sf6^~?y_$wOqS~9IPDeK>>inSm3fLX?IWT^-tj-r&hP|twjw!7Fz{DTX=8L|6 zYV~(YFT*}srVs+4gSK9Tk)G<)VZEaAJh=8!(m7aT)&4@Ej$iec5bc2d)k}?+Y4CHZ zp5TNrQ}qo-;YnSuc4*&L`>S|ytp-wRO%CHVg>{|6?P7H0qe literal 0 HcmV?d00001 diff --git a/planetiler-core/src/test/resources/parquet/boston.parquet b/planetiler-core/src/test/resources/parquet/boston.parquet new file mode 100644 index 0000000000000000000000000000000000000000..297da5e9dcddef6611890c15bf122a804e345b8b GIT binary patch literal 20576 zcmd@+du$u`c}I$tWkqfkC&$qln`n%4P-wEpa*YEo=S>{%pw1vKHqMtJ@(7iNuilU_JLli}^lY)zP zvtF-zl4sY~*I5_K@)MrqB%5@nJmUh)SvfPiwOyPn5 zxN`f3X>i~o`~C01pKlDzUN*4j}n?L*XvG18)8)4pg=UagAFZsWE zpkx|cxK?)j44Q8RCw}xX!23q^CmUX%<0r2Se-lPee&POKz$jS@qt|h9i_LrjxTKEE zE&zNU`2_?&Fft77265STfp4G(rW0n6zE<4eII@G>wc-azZ&0J>26Q#*ON;rOST5Fx zGJUJh`oeMM@A{aJEvQYq$ZDV(xoWk^|5vzfsmC_ju4?hXh>Lmlt#SC?1K;LNw^^{) zX2EeQHEd|<;xrSheD!WS-KSXPutgB^+m`KIRwnZNBa`(zCg#QEj{f25WcU%cQ>Ut zVzb~F0@hp2l-{;BQ(9}3=?66P3JsL%AesOkOzFP>6~BP*E%@%H6h~|p97Di*tC`Z5 zPw%F*L8fomn5S((sSctEU`tB-=z#^=OoEhU0C1u+K1Eyg2xGHguku8IdJEo83@K5= z)JriR{kokt*Mr9Ks#q*#^NMnQ-DJIPVm`f(`TP5TJbir=w3g((WTKzdqUf4rgViq9 zwf0H1;Z$HF*a0Z#$2Uknho}?pcC}1B!8{2YvJ#NpW&*?eu zJvKNz>GHZ=K9`F<@40)>&br(aPWO2?FnNX^Hf)-Ccwoyg)iz)R15i&*D|?3))?+_G zk(hD)iw7`zq9`Ww#nfez<@H&gHd{Y2G0#5CyfAF{?vJr0uaM3y0OhEV=t=IQ?>BJJ z2c0KbgNF4`%)29Yx?k}ct@W!vnU~kIDN#Tf3GJLXOf~~O$>-*1OM%L3unamXP=9nhY;bStN6HCVn z#cYnRXEv{!zy_Ebqs%Ag?B0VwwH^Xx=7>G;~zA?Rq@lQ(_Li@t6W@!TVtZEx?rCxTes5=FTo zsAwB>>r2c_U+N}JZ#;Sk!B|S*a5Ne|`ZDu3U*3}jmL69Fm+kSpz#8(EvQWuNsjQqn z2F+x)noWY!X-KLdiri(GXFEn_!E7y|>aj3i=QL7bg2b0fgrrwZ*4ItUOXr!}zlqVP z6`aG^_2=kQO@8ILP4O#m{txJ>+%ZY4iW0%RW=$|Z^fTY_?;#9|t^=h=ZNg@v;GdNm zCVzpsIsy~L+?ux2)ai%71KCbw_|oM9Uo0!!EnC-^p9dakS_`=^h7+U@(w#y2Ue&?( z*?;^a{G2-9EMi-qI-EyFE(HL>@Y(7}H?t7)yRx1KEWg8M_Wrf7=6i z%e`pGy0Nmci+T(8+Q`#t9yYXeYI{r;h@bQNR`tu3+~svCpD!x1R1)tG=a{$VAo1Q4 zU`q;{qR@WY^47rTM& zyDK`jlf`^~{qinczYH_)hWErZO1o5N_iHiC1=74AiK!FtR;U&E9AxKGvnoRxc|ses z^(rfjS`4R*57u#tb!(O$9F40F{Yh62-grmy1y;OdbWN+`Vxj{ts;#YQq2h*daTKO`f1q*FUA z8>v;#n#Ml;Sf=*&!*at}2%#%E72T`AR*xY%arILG&}Lp3QE1ao1hfU0Q8qjos5p>B zB`MT_C~5RT{Povqs3+J)fLgX|>}{n#!mlhAA`jSfgNXhzTlAlwGq2H;y&>TjPkq;i=uoag!%fQ19{> z>Iy_}uwu0v0U2$Kvi6>~pZ?^*|FtU~=Kt-b0COkm_spFrwKI33D$d-A0`?s!Iy0Yr zA@{Q>vNE4t(UbX1%Z^>IA>QE{UpMP@-7UQ_U9X}s-7knSU9N30ceYH$bhqNv)v8dN zg`Q5zH+R0ga%V~+Ob1mCrv1VN(@`yg>9oASbXGCw`njIz@TksoeERO@v6<=QBy7Dg zYVBqEtgnpT3zxr=tp|Q!WBXmx6-WK*-7z)faE`L0tYc)zQNN+WDpn;Y@&+R&76V_3Bm1X3rPX zqkMsf^ZIBB>b#qyl9=X`wfUtrpBFBjZMIz#>KxdPxktxGos{k2T{B8wkwyX5S1Mw8 zw3y!-g}Ly9C64M zNy2KDV+t0EBMud&W9rJ)t0UT4RRB5!AuGe}-{B$PPj|G{hane#=A$ieK+%&&whl?} zc8=Gv9d$9*McT$(J38hkt{$2%Rn4`H%*tgkEfxbRI!#j-(FtpG0T?wEg6Z zMIM!+UIAJoN~w+RqnzsRDKwC%$w#88{@aE20cgJoXdi@rYp=sdm>u||sc+1W}2b_}& z@7+|it%>zwo(55;|B${%z2|Np0ufaE%j;D|4S=>YEoI`+6toll!@FY*;UPuVp3Wb^ zTaxnVQK`~{9=QY7mmk^HYInRk3?ZMFL>{BdA(ctFRn!4!n{~+b`sJt&N|`_+cWdp# zFkF-AKn};!6Oe>Iuq%UBlJIG`YF>jMsbU^Jz^$xYj7}*RYqqDR#_%(dj&K|ohA)0D zaNIcOPjj;zm*xU#(&zU#jBjD*j5fX)UR2r~=SL`h4%*X~0x&JdmB>tWeD-2>Y-}4^ zYA<;uk9B+s?vT8Yo85{>m-2~K&rUKpTjOhJpV_Ah#Y^m&A@ zkiLfa#9D|IV}5CVBcDyWS8A!~5<8#W$|iD>Bm`G><~Irg%&EkqPBMp&hO-NS#x;D@ zBjXxDj+BJpY*q+f%3hQa)}_*X&MBpWvztIi24GFB$Jmk)Ucwcr3egCgls6mT#&c37 z9t(Neg&WU>s$yWiG%v4sl7t?JKy%VoVs+j-FE0VU$VLjsbBRJCy1fZUFK^Uc$_lyJ zvN9)A=A+{cG$nG|)p(5XUr9x`OYvx^gmY7KA!lM0_y<1X(UnXpI0O8bee0{+6Z1Jw zMu>tuvY~MyIy;X1XmsQfCBZdI#P z3HGg=fPF{=XW2N=ExMh)g*hY7QaK(=cgAxBU1qeAGa1A^iL9s44ptlFwF7%s5OT?OVebg;NG=FroSNYzSl&%7|#!IhY4=N|0ABDKV{K9_q6*?k4^_$46oA++y~k zR5to(7uahd4mN7E=LY_BZTQvPc1FmhobhN%>MC4W*^92i%!Q=XTuMqb#J&bO?j!Gf zHWBnWNz4HqLOboM9=O2fJN4h)viJ7m+y>-GI}N^}b6*Sgk&pyqzLSNxCa*}4U)4}Q z+Oazx=#07ju-}jeHsmd#s^Hp|;);&66O;3as3h}qjk%;zciN#B)|6u|%&B~}Ci!+O z3jtz7%$1S&Xe!?nCyyR;8uLQPGd=5=TMBNH<7oLf^s|u1OG(V}+VVSzOx!iQnTYrx zzHL_tobJW<9Gp2ImyE1K%wQpR1HZXt@X^Y7qLaKNne$Z>k`Hpy6)BmMGEMs3aee}N zl8d>Q@^L)>81);^2Wq@)NA5D(M`Aq@lrRTkcjvS#)s%8}Xm4NZ@ZszSXBWs@Vk-r~ z)p%CvG*76@{!ZwP6bE0`HCqBd%mp^#OzVeSGYMx#Kj?f$T9B8vAeI;5@3s)|;QMw8 z&SwiCyHw4ipM&uHePP-)Ha2+)-ry*XAM(1huo12-lJhwp>i2@5kdpEe%Ac%sEem^{ z*ONHmfb)82O$vLKlo~?V2j_K}Kt&<~%lHbpnto2&$fxHwW-5z;am?A$^Rk*VO6w4N zOR1={0_Xbld@dykE;xtJk@GaPm#UD%l?55{pR6D9AHz7z2ON?Bb3lJF=kv}7La|lI zh!z4|RzI&x>*O2@c^SYE<|RMClw?H@x$IgjQiAhV82pYW0eQf@T5cr+J+DHIPBTHKWc+2Bdk%Or-X=phsvuBsb z#$X`8Eof5e{{<1#LtISc!TFWf3V0^Hek()M#oeXgba8uA8M zm8=$3`EP0KGYF4sz22D909*Vr)B_cDB}(lbUkl@S60K% zh0g*g=<@qp<0qWG&*@03bv7fs92q^~pLI3Gja#CEhJUyYz=loAqz{9Qe zYxN)SA1J@Ax<0A)NBot+{)jx2c;->Ia}(!B)%_3eM?b-rg<79G5sg&1S(N`-Rex1| zZN^7L0ku#G>X?-j*zcI??-tbko7r!_T=D3(lhAr$4e*V{oA_(iZ=wot7OEuNkxarx zwu55{e`}Ks_Dz)7=${FH+F6}j-a!2-)#>%mtPp`2@DWiG1d2U?PmVf?KMXDJjE#B0 zrx1PSmV&r{0esZ)p=#<%fpvb+M}2(+*N<-;Krf>9>++e~!TDj-Qw3F$n}!*vGP)5d z@j*eY+p`is%aRxlY$3lM#832DnNR39#1j(Q=G44klXH>zI}nP9{Vo#t#`uS*6#F6i zkaYQ+$=nj!!OR%=yNk;ZIte}&C4rY8@FDaO(hJy}!y(1gTk8Kb;`ePKJe;_n|HK-C zR_g;a@hWUJ;!g+C>iRw67x7;h`yh^P#=wqX6_-Q$CpG@}&}X7wU3G~*_&s>vfy=oI;ByuJA%3g;>=CbU{+xt&FZjkefo4VDxJ&T1pS}?M zV$hipVI9~h@E=#~6#<3)GlyJX5P%(6A4I(-_6}P$lg7jpPcABVP6YK1_z1i|1UppZv4#3Ukf}pBSD?77FM1BK**BsQ#<7A*Tm^TIquSkP7~r!_Vd)_1}{ewG97n!T$lp(rJhQ literal 0 HcmV?d00001 diff --git a/planetiler-core/src/test/resources/parquet/boston.snappy.parquet b/planetiler-core/src/test/resources/parquet/boston.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..5c230ab5f002d06ece6e8439472845116002b860 GIT binary patch literal 13019 zcmc&*du(H6cE8tg94C{3FwFHjTLWok_!@|lTwlN96lkvP#CbW1GnqIsrQ!OK#L4wz z{Y(ZSZC7-(-9HejC>repp;n~`A%sS&cH6DkDVmO|ie;;*w3^Zup%$$WLL<7}t+p$K zdd_$6^|c*mG9{U*iS7G6&N;vH_|Erz$HStym(T0nkP_1GO)+*y!X5W{Jx&iy z;~4HVXSD;<9(bE<9%i7g71Mei)|NBsxdO7Tvj+Gyv4*y@4EcJSjqC1pGp3$it0A4_ z3>K5+?KaD=4CF6ZVm>aRy>>7X!xcQffMRq9Vz%j+z8y`B8rXuAc&!;$s_3Ol?z z;JqIy_F9cNj!5-ZGt$q@`;kh!bx6PHAlEtosm`K_;o(TH%8&Rzf%jYR-j5VVtVSF| z#(Jw6>D3d>NcS3TdcBK$y9Yp08giH;?@%0YxTKjS+)YTzzdEc-ZLaPmU+jR z!!_pc4f8x7>$UOTAqR8{V=g!R4cnMSeuy6(;@!&*FU%Tqdg#PvA9$DntHQLA9+_Hk zq$L+X(vWLzoYr&AiE=(El~U`abS6ck{-(il!$97=pM3okW}Po1uBCYd&NGL}2NHAd z(`ecI;GHkz@~J{8ohpuOrP7;QB~bW5uHVj?^<;bBzDa0W=H5~CXPW^(%|^D(KrUKq zYE5+%bLB!JRW!=!VyPn~r!uK*$*gkP14Yf2E3D=3S@ObH$lDfFw56DqnmobzHBBDc zJqh9$@~P~(QX!Qpg{6EKbeumII@dRsPD+Iq`gG}|){_mSK)GM5%fE4mRvq9!shBEJ zrEaj64_WfoSIPBWRO%M8TB`Omx1wzY{A0Mrst%9a=^pht+~c11o`F%p;}m>?z>mA{ z>#^~I)9Y}KJ3+~_+(4bv3vl(au6Dkt-zE z>F#K@e8XV*oF%`0fPC`-o9EUNTj~ni;2a>wfW%DlRqnw$4b0GalCO)fHilgLO&iyx zWR2GG)s@I)H`2*eHi1jYYEevW7WcdLLYk<6d zhJ4&_^Bl&yCHy86+PFSF8GB3s}jAej=mOeUA@+)3@NS0uTd zYJ(?=FK*c)&k^#ohi$;?cI4B{Q9Gugd}TSA7#e8kQ}T}khXQD)FerqiWt%)VKz{aH z#|h!5kJccpDmG{@l;x&e)lrcOX(@}V+GIf(-+GjM`p9v@_{e@6FwC-)-7HI+%HDuD zF4@Vi2aglSFNSXeN0U!p8z$cxK28u{@U^wQd$3R9sZuqS$~J5&H3wbh$&37PVtUni z8-_6!A>q_8yf2Wy5e~$_b1rS;vi|M~SPi*yDPB&?$#izJ9h!-BC7pnz({QMQUF1aF zJZn3h1+!}hsx~9}++BkT6J)7aq>$cXA<2*z-Q=nZccWH(Zf9NJ!<}l%D;-uPuQ22@ zuYQzkms6FLOeHT{V&wZ?@{;!eV=$b`l%aPzE6c#2g&CmVpqwgzWyocpjbnNrh74rw z+>(mxd8trRv|F?+lMhB8YohIi>F^)ogX2yQcUnvEX4{Y7#n-9vW)oY@+KB#SF%Goo zt(m=z#RUJ!Z`Y#MYElnSdS-urjJ!T((^=|I2QY8xi-uD-i&`}Wh}DR_%BRIJP}kDY zwy{RoKcCQD)s-!0);Hu_uAtabIe7o{aq{NlaPU5mVM~NfyU?wo<=t`JU|n!~TCX!2 zhRh4&$-NLAgXIEH2+;BSHgt3p>Rwe;2CfdUP3ddyn-k=d;BiQMC8SF`S;*x!*7wQ!Xp&r;JW$p! zCuqs;E30@G*p%XOD%lB7`MrXafz!F%9LjKv?5qu1O$tSQ6vHj!`m_$LivlZdNqHp& zzrk9*&yr84$jeiBByGX#r~&-mtRBGkQ0d7u9E(fZ1r~$&k7mg)W)8H=|6#Sf7S>hm zp~7;7G~A=4263Oxk@w~f6bE~WmH<2ZaFf}Dds23v;hBMUI$%}4YBo2qcwhhP1s#ZD zb3Mi{vX140qSr`*s`<2MmYmUc$aC3ZO=-b?tC;trEe|P4gAN>`yFI~R>4{DsL1xrT`cu~49#~!-go~S7`{~b$+ic8{N&An@4@KF2Oj(-j8a+{y@bTz(}vp$ zsMt_-xVoY{(Y%?;Wl|;eBJxVRLV7*YLD1!fVo=nN8wpt3(=Wtr35HFndTlt zfkrDm0m2>+vbEv@ibr6zB7@3-c4~L@`ab$0ELjg0DZlftF8!WfIG*Vg;_ng6$67|5OY zF@W5Ox;?oQC3bQrs^NE_f=%vzRru~yY{}iLppv^N~t&7G;QkRy~#$d|8zkRuf~$Wdz*iw;!#w;)Fa2Xr4tas8 z*X40Iyzm=6zT9?~Fzj@D9IjfIU{|LJUYFogU+Od${HCw=c%QQO*A-wuR3cZ-LIExz z=Mp>BbTI{THK^;Qy?hE_KH0yRllL}r+5V@VefC+#+AS_EEkwtZ%e3`Zy(9SYiZh}p zM&OOFSy6O}{!MXG6gS1dChhb48^){HIZ+!Ah%k(8cu_8Nd|DK%H~`PGPJU{ChS($gpyNQotF`xuA{t> zoRX`t)o^-2E~d|ie8p5WS&64-k{hcI9~dx*ro{Q8e+?g90r*%D6Qa0S4Z(02d?=R0 z^YFNdbHm^cQ9yUhZo*W*3*8dIwpfLB@I3l1hi(j2(M>LF2e3U4EipiaNcb3}PNx12 zFK&;Fpt~ax+Bt>eN)XT|#3Y7v-zc5iu8|R6VsLR}1loRam-Yg1H9lxloWrkJbW(_| z_;RtRoRy{`;!Z`Cw;2d)^!_DE&`b|;EQ*PYuM(4e7%PG&Mdj}$feDt_SJfPVqos{9>kbw*xuQvZV~<8DR% zv^c$(Rqc1awf&Im)J!EV#5NMb)>a|}{`7AHy)Qhtq1aE=zga%@nkv5pu_q@o@>U!{ zO;GyLL+BTAYAiFky%t@PR|Urw#;X*aEQ(V|KUh43KHNw@Ux|l&d)Oz=BK=50qZhYm zZ?pd}CyDW>TwWDoMeqfs^)FTwdx;ZN--*S7YQONo_6s@kaaoTcaU$-~>>sJoPv3&x z?TD}VGJwC5$bwp-ZLdNv`f>G$Q_Cv9i6D(9TzFbO#3vD#j=vVe;;b$|SChZiT0ZuM z!0!%Ub!~NqPvBac*#>zd;XyD3GFI(J*C!2K=!6&KZ?IozMTkX_C|Df?BJ2~-;YV;( zqgUgnB|YS)t%MU@3G^UuMzeo}r~aZ&Qr0`oM+=xwrfB|vKm~9Vzk3Vv_62NHdWUAe zi7*A}M|^er2PIPQC$cL=$bm5Ik0AanUH%Rd(IlbBr>q9CGmQ2RP<+&=3O|~fV$oe0 zu#}S`keKeoIiM5_^G_L%nF|rR^<3*a1-TCE?#Je z*M0KP6d}+#>X75vh`3Zy=WqJO5j9`l%3sOMB|6g7Zx!+{S2F}7hUQt<3C9C z7Z)Lwq1usKrgjjr%j0x7=YaocTUjr(4QE#$^(pNCz{D3;*Nb+3YL0hGpMrCAibC*%4(fg} zh4fUP1NJK_&y9O8C7p#mR+%s6YxtFT3DF5SUwJ9{atq>|swXy0k+w?iDaqTiTC5&RB&| zP@L;?4v!AI?0w}TT;BTN-}Zd1oGJ|^^7$dDP{>vL)XQO?!;N=C_!$L$alAW!%>RI# JMfl%Z{ujH5{#pP4 literal 0 HcmV?d00001 diff --git a/planetiler-core/src/test/resources/parquet/boston.zstd.parquet b/planetiler-core/src/test/resources/parquet/boston.zstd.parquet new file mode 100644 index 0000000000000000000000000000000000000000..c90b5924c30cc6096bb9ecbae266efc30fec8c84 GIT binary patch literal 13543 zcmc&*4Qw0dc|KAUP04cNI6jW*ltz)0uS)bo^7toucI%@kQIteUlx0z3kSa&=L{U2a zgujx4Z0V30?GOZR(HX(&21bx&MKBCOk)>%0q^^+}SepJUD++IGi(m_iASjHiLyN5N zioNf5cRZ3JE!UR3gO0rK?|q;5=X<~Jdyksq=k2V4y{cnh&`q;OmN~*O{r_P7-(Jb3 zXWnk>Wf;ab65|Ai&FyxK2)5PLRU2os2_9E`#1?lXT+Ud+K4N!zJ)Tt|F4*I)nAhWW zxS>_gFw-Y4ZtGcwJ<5_nAbuEbV~?BZ?`=l2mX4)qk6P*gjNYWgA940 zt%vREb}`1o-4=Z+!FHLA=C|6+=iA5?9Wm_{@9sg+8N=!s_&5w-3e z$a9ySaNi4eqaG{E!Sa#E}nNDS>h%f5QAm2~h z$+`BP8_>OJ+H$>0o#ziR%(I!^{z(TTlM;W{UG_c6L4#k8tC3M?q2|=-@ttZ?wz=? z#bUrVq_34);C}h;ow)1t{qkMp{X>9TqtrmRDQ*+{scBZPMF_(Kz#lHadxU*Zo8J}# z*29d*Jc_ml{WyW8b(jd<;@Q*K%w;oJ_+%W<|-F12jq93H#llmjGxkagGC zRo~xLsp)U&s->oie&oxld2x!i^>i?Fv3m7^U8L{Gi}7qeu|gMVlljl}=9l&4`Cjsu z_xHGO%($rnAP7x^096lWbTl%3iXE&G$fSZklu|)cpKN2uxd(gLPB{uS_sGt8HnW;a zh?zJ}MU(k)^D{c~<}vc@u^zXHX^N?jb=F19d%h%lUm%-FWF?_@DwQp40Ow6LCDBZFGWWnza7{WP&bsjHk+}IE1{q9V4uF2W$FS+G#H} zl+^4N+YIFW<5lF)PZA0RittsP`6oK^k(WH{#pSaZxj!U|+{N}bi)shFYE)y$uTN;( z)^MC2L8-UKmArkuJX|_XslA&oDoD7q26y8qHuasV&-*+0fsl zR>i5BrcH-S(0RV<_!gtGRn*p=Pd-Q9{9KPlS$k^1qp7``9YSEGZ*mC1T8jZc&H8@& zEths+Z_^nV@|<6@W@o09URjm0*}SZ0`5ba?l>D0y&LJ)7HpSaw)w~J+`sTQ%%}zLX zuGDB0o$SLg^5$3zC&T1K)9uhr@O^PY(?kdIrJAVv_SFD+C9n@{&rNFcx(icD32`N! zm9qIN=PN<-d~hE)KQpayzFXm(hEo@;vD%b=bBa7WwGYgHAJUi~%4f5yD?3=f79wXu zEm;qGSK(_C2AOmk2a%3@>GdT)umwA4~FR}Abs zZMEmv$B!{TH!?@N9;3$y4884_!f#)(pLpZisj;EIW-mSZH&Fh^WA}ge+faV~oA(|% z2IaT?|C0GzY(F-*@ZZ=z|8er?Q2yFk{eQmy*wJqV9)ZFOfAxi*eSpMr6tiius2uDrwwqr!kf%RS&PN{}+?kc`zHW0d7Goj4CZ+{; zkYT$X(qm%$A!fPHFae*(PKY!9+WVAZUTLV;GQX_DGHEqS2UcEa$QBRW!-H{YyLrmY@_ zHE_!D6R_#02W4#9QIs{{FvfyMQk4fnlur{H5&3+!n|z%L2Ib^dOR#!us;z9~azHsQ zG~>gf=&B}VK6K|^N0085)f%*(yk&)b6Fh7CN`?|sHgB?-Yj#6&Yt3dv?xF1xWbSp# zx5^WB*SNaYw$(3wwJkO4jarQoH$;A!^^TCNjzIP`fl;kSLbY*9rc#|@jM>=T(|!He z|J!RNvLEk!$bP(#A^Y*RgzU$wjD2|FK<<3f{?4S($(>7LlRKI*-ury-K4)O}G<~|q zsZX-^X-l#XDMzx`=|Zy4sXnsDX}Ud4p50;s>~`{5`=4RjpF9w`ja&}7^?VGut(*$E z-Mk37z1+s0=N!mw<_*YgpZ)jpm`-lzs7-G1JZXMwz-%NY^GlAS47`=Iw82r-a!vPS zpYpSfzTZujlO21EkOzo*oNl|_13%Ni%Qfiah8-@q-C3>T29<7{$H{q>OX=o>UtlOTp2r6JYBDy! zDxNK6AlVm}vhj^dsvrVi73xZHD<=ZXNBico(pEB?>3i&{r=DUgUHts~Y~;9nsJ494 zZNwLbt_6k;xRoh2-XvQ*JlnQMR@_Ypw;2Vkf zN3a1=q8w zD7RRSMHX%G%tmT@baFk$ZQ5fI*LH9{=Lx10QjD`piHS*^SzhLf4q<6V zLVTKx(rRKtszjGUsadIzIuq~~#7LqXOHC$Lm+W3ppdVF<{dwOqJ~~I?W0sHe{9Gjf z%^|QMU*ylgV-ov@z#Ke>=9o%CSDzEj62>xLfwKP$+AfP`3{=oePArdNc?L@SC}kqx zqn{d?+B-D2ZndJh!(m!Ef$g#r&?m&i2Q=HrmFrHc)f*p}vs$6-<2Pw709X8j7WrxX ziblq{=%P0pjYt_`BFt};6@JOlkUHKsPX{!P2S4Wdc-mWzN?!DpavbDcL%bg3kM1}& zPK6lN}%++LS($aPvH20!Y<=O_2XyCv4D3A>-Z@g zZ;h+t`88VGXg`ced@LfBmbhpEY(dBR=E}0Z_%SN)*j!%GU#PYI0`^=?(tL;?i@8<( zhpXd*HyH1-#}>V5z+aAMK&-&JM;_1naQ5&M3ktn4KlLY^cxpVvCmyGcuj<46l*T_> z<-gpVKh_4o?sjiwd1=xX$GJAS4*aa40Z;`JR`f^bCv{z@gbU&~=r6FyMI$)KUm1Ww zSjV5n5C4cdUh$u%;~_q+#T{r$7!Uj=RsDx;)Lzs`@_eW9Xcpti1dSiyr~r=QcdbF( z-o`Q=Z&&p<7NQ`1h_7b-po3)m@yucYVjwj8!iaxO_2c*_$j~T^r9`nElRPZ zKZ*Rt=eFzobq7CGMF3=u*riw|%+HsV{z)HiRpRB1?3GB5!+J@v+OWS+<<}yD^ywP`IZrNBQQUE1Koa}KN3vEY^CfwG?LnEw=S1>!Hn;Yk<7 z59mJ-SMmB~`qW=#c_}NZKRPugX3B}B$#t9u67n0t@pS!~q47uFq{*QN0ZyUU!}<;v zo$5{f-%sV|=OC1!*x?(+Pl(;=D9i!eQvoc03|txc2l-HSd<*|VC=;;uLHv%6dvj4J z@}ui8-W#_n@vDVDQ`d>Yy1NW}qXgI=`>*ZB59v?9+U@Yc-g4fBMSfx%_zhGV_|@o} z_M)LAvEp9YM4h)AU`jTr}`eHN*<;CYy5%Qmy0ei?*{Gjx5*d6vcU_V-x=L;>v-qlNO3hO@*@rIQ7qVAs>{hf{%VIM70 z2tJTOSuZAVJe8-#dPVuUaP6f-r(lhh`}4Ufe%W6Fv;+2+FF9VWfuB?Hcstlkme(JK zCwaZvq`N#MtzPhc6BH^0D)4#FPc=JA_O?SO=91$0%X z_Q{isg$wxk>0Zb1$gp#;w^V?`TQB@R>4}n99E#_1Lqa~Et@J8~!(O`!&xY_i1>QW~ PnLkE%AH&SSzc~FL7n-vQ literal 0 HcmV?d00001 diff --git a/planetiler-dist/src/main/java/com/onthegomap/planetiler/Main.java b/planetiler-dist/src/main/java/com/onthegomap/planetiler/Main.java index becc425ba0..5b8eee3241 100644 --- a/planetiler-dist/src/main/java/com/onthegomap/planetiler/Main.java +++ b/planetiler-dist/src/main/java/com/onthegomap/planetiler/Main.java @@ -10,6 +10,7 @@ import com.onthegomap.planetiler.examples.OsmQaTiles; import com.onthegomap.planetiler.examples.ToiletsOverlay; import com.onthegomap.planetiler.examples.ToiletsOverlayLowLevelApi; +import com.onthegomap.planetiler.examples.overture.OvertureBasemap; import com.onthegomap.planetiler.mbtiles.Verify; import com.onthegomap.planetiler.util.CompareArchives; import com.onthegomap.planetiler.util.TileSizeStats; @@ -54,6 +55,8 @@ public class Main { entry("example-bikeroutes", BikeRouteOverlay::main), entry("example-toilets", ToiletsOverlay::main), entry("example-toilets-lowlevel", ToiletsOverlayLowLevelApi::main), + entry("example-overture", OvertureBasemap::main), + entry("overture", OvertureBasemap::main), entry("example-qa", OsmQaTiles::main), entry("osm-qa", OsmQaTiles::main), diff --git a/planetiler-examples/src/main/java/com/onthegomap/planetiler/examples/overture/OvertureBasemap.java b/planetiler-examples/src/main/java/com/onthegomap/planetiler/examples/overture/OvertureBasemap.java new file mode 100644 index 0000000000..3723c5a1be --- /dev/null +++ b/planetiler-examples/src/main/java/com/onthegomap/planetiler/examples/overture/OvertureBasemap.java @@ -0,0 +1,66 @@ +package com.onthegomap.planetiler.examples.overture; + +import com.onthegomap.planetiler.FeatureCollector; +import com.onthegomap.planetiler.Planetiler; +import com.onthegomap.planetiler.Profile; +import com.onthegomap.planetiler.config.Arguments; +import com.onthegomap.planetiler.reader.SourceFeature; +import com.onthegomap.planetiler.util.Glob; +import java.nio.file.Path; + +/** + * Example basemap using Overture Maps data. + */ +public class OvertureBasemap implements Profile { + + @Override + public void processFeature(SourceFeature source, FeatureCollector features) { + String layer = source.getSourceLayer(); + switch (layer) { + case "building" -> features.polygon("building") + .setMinZoom(13) + .inheritAttrFromSource("height") + .inheritAttrFromSource("roof_color"); + case null, default -> { + // ignore for now + } + } + } + + @Override + public String name() { + return "Overture"; + } + + @Override + public String description() { + return "A basemap generated from Overture data"; + } + + @Override + public String attribution() { + return """ + © OpenStreetMap + © Overture Maps Foundation + """ + .replace("\n", " ") + .trim(); + } + + public static void main(String[] args) throws Exception { + run(Arguments.fromArgsOrConfigFile(args)); + } + + static void run(Arguments args) throws Exception { + Path base = args.inputFile("base", "overture base directory", Path.of("data", "overture")); + Planetiler.create(args) + .setProfile(new OvertureBasemap()) + .addParquetSource("overture-buildings", + Glob.of(base).resolve("*", "type=building", "*.parquet").find(), + true, // hive-partitioning + fields -> fields.get("id"), // hash the ID field to generate unique long IDs + fields -> fields.get("type")) // extract "type={}" from the filename to get layer + .overwriteOutput(Path.of("data", "overture.pmtiles")) + .run(); + } +}