planetiler/flatmap-core/src/main/java/com/onthegomap/flatmap/mbtiles/MbtilesWriter.java

385 wiersze
15 KiB
Java
Czysty Zwykły widok Historia

2021-10-20 01:57:47 +00:00
package com.onthegomap.flatmap.mbtiles;
import static com.onthegomap.flatmap.util.Gzip.gzip;
2021-04-10 09:25:42 +00:00
2021-09-10 00:46:20 +00:00
import com.onthegomap.flatmap.VectorTile;
2021-08-06 09:56:24 +00:00
import com.onthegomap.flatmap.collection.FeatureGroup;
2021-09-10 00:46:20 +00:00
import com.onthegomap.flatmap.config.FlatmapConfig;
2021-09-18 01:12:24 +00:00
import com.onthegomap.flatmap.config.MbtilesMetadata;
2021-07-26 00:49:58 +00:00
import com.onthegomap.flatmap.geo.TileCoord;
2021-08-06 09:56:24 +00:00
import com.onthegomap.flatmap.stats.Counter;
2021-10-20 01:57:47 +00:00
import com.onthegomap.flatmap.stats.ProcessInfo;
2021-08-06 09:56:24 +00:00
import com.onthegomap.flatmap.stats.ProgressLoggers;
import com.onthegomap.flatmap.stats.Stats;
2021-09-10 00:46:20 +00:00
import com.onthegomap.flatmap.stats.Timer;
import com.onthegomap.flatmap.util.DiskBacked;
2021-08-06 09:56:24 +00:00
import com.onthegomap.flatmap.util.FileUtils;
import com.onthegomap.flatmap.util.Format;
import com.onthegomap.flatmap.util.LayerStats;
2021-07-26 00:49:58 +00:00
import com.onthegomap.flatmap.worker.WorkQueue;
2021-08-05 11:09:52 +00:00
import com.onthegomap.flatmap.worker.WorkerPipeline;
2021-04-12 10:54:52 +00:00
import java.io.IOException;
2021-05-01 20:08:20 +00:00
import java.nio.file.Path;
2021-07-26 00:49:58 +00:00
import java.util.ArrayDeque;
2021-07-29 01:47:13 +00:00
import java.util.ArrayList;
2021-06-06 12:00:04 +00:00
import java.util.LinkedHashMap;
2021-07-29 01:47:13 +00:00
import java.util.List;
2021-06-06 12:00:04 +00:00
import java.util.Map;
2021-07-26 00:49:58 +00:00
import java.util.Queue;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.atomic.AtomicReference;
2021-06-06 12:00:04 +00:00
import java.util.concurrent.atomic.LongAccumulator;
2021-04-12 10:54:52 +00:00
import java.util.function.Consumer;
import java.util.function.Supplier;
2021-06-04 11:22:40 +00:00
import java.util.stream.IntStream;
2021-06-06 12:00:04 +00:00
import java.util.stream.Stream;
2021-04-12 10:54:52 +00:00
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
2021-04-10 09:25:42 +00:00
2021-09-10 00:46:20 +00:00
/**
* Final stage of the map generation process that encodes vector tiles using {@link VectorTile} and writes them to an
* {@link Mbtiles} file.
*/
2021-04-10 09:25:42 +00:00
public class MbtilesWriter {
2021-04-25 21:08:01 +00:00
private static final Logger LOGGER = LoggerFactory.getLogger(MbtilesWriter.class);
2021-09-18 01:12:24 +00:00
private static final long MAX_FEATURES_PER_BATCH = 10_000;
private static final long MAX_TILES_PER_BATCH = 1_000;
2021-06-06 12:00:04 +00:00
private final Counter.Readable featuresProcessed;
private final Counter memoizedTiles;
2021-05-08 10:53:37 +00:00
private final Mbtiles db;
2021-09-10 00:46:20 +00:00
private final FlatmapConfig config;
2021-04-12 10:54:52 +00:00
private final Stats stats;
2021-05-13 10:25:06 +00:00
private final LayerStats layerStats;
2021-06-06 12:00:04 +00:00
private final Counter.Readable[] tilesByZoom;
private final Counter.Readable[] totalTileSizesByZoom;
private final LongAccumulator[] maxTileSizesByZoom;
2021-07-18 09:57:48 +00:00
private final FeatureGroup features;
2021-07-26 00:49:58 +00:00
private final AtomicReference<TileCoord> lastTileWritten = new AtomicReference<>();
2021-09-18 01:12:24 +00:00
private final MbtilesMetadata mbtilesMetadata;
2021-06-04 11:22:40 +00:00
2021-09-18 01:12:24 +00:00
private MbtilesWriter(FeatureGroup features, Mbtiles db, FlatmapConfig config, MbtilesMetadata mbtilesMeatadata,
Stats stats, LayerStats layerStats) {
2021-07-18 09:57:48 +00:00
this.features = features;
2021-05-08 10:53:37 +00:00
this.db = db;
2021-05-01 20:08:20 +00:00
this.config = config;
2021-09-18 01:12:24 +00:00
this.mbtilesMetadata = mbtilesMeatadata;
2021-04-12 10:54:52 +00:00
this.stats = stats;
2021-05-13 10:25:06 +00:00
this.layerStats = layerStats;
2021-09-10 00:46:20 +00:00
tilesByZoom = IntStream.rangeClosed(0, config.maxzoom())
.mapToObj(i -> Counter.newSingleThreadCounter())
2021-06-06 12:00:04 +00:00
.toArray(Counter.Readable[]::new);
2021-09-10 00:46:20 +00:00
totalTileSizesByZoom = IntStream.rangeClosed(0, config.maxzoom())
.mapToObj(i -> Counter.newMultiThreadCounter())
2021-06-06 12:00:04 +00:00
.toArray(Counter.Readable[]::new);
2021-09-10 00:46:20 +00:00
maxTileSizesByZoom = IntStream.rangeClosed(0, config.maxzoom())
.mapToObj(i -> new LongAccumulator(Long::max, 0))
2021-06-06 12:00:04 +00:00
.toArray(LongAccumulator[]::new);
memoizedTiles = stats.longCounter("mbtiles_memoized_tiles");
featuresProcessed = stats.longCounter("mbtiles_features_processed");
Map<String, Counter.Readable> countsByZoom = new LinkedHashMap<>();
for (int zoom = config.minzoom(); zoom <= config.maxzoom(); zoom++) {
countsByZoom.put(Integer.toString(zoom), tilesByZoom[zoom]);
}
stats.counter("mbtiles_tiles_written", "zoom", () -> countsByZoom);
2021-04-12 10:54:52 +00:00
}
2021-09-10 00:46:20 +00:00
/** Reads all {@code features}, encodes them in parallel, and writes to {@code outputPath}. */
2021-09-18 01:12:24 +00:00
public static void writeOutput(FeatureGroup features, Path outputPath, MbtilesMetadata mbtilesMetadata,
FlatmapConfig config, Stats stats) {
2021-06-04 09:22:27 +00:00
try (Mbtiles output = Mbtiles.newWriteToFileDatabase(outputPath)) {
2021-09-18 01:12:24 +00:00
writeOutput(features, output, () -> FileUtils.fileSize(outputPath), mbtilesMetadata, config, stats);
2021-05-01 20:08:20 +00:00
} catch (IOException e) {
2021-05-08 10:53:37 +00:00
throw new IllegalStateException("Unable to write to " + outputPath, e);
2021-05-01 20:08:20 +00:00
}
2021-05-08 10:53:37 +00:00
}
2021-09-10 00:46:20 +00:00
/** Reads all {@code features}, encodes them in parallel, and writes to {@code output}. */
2021-09-18 01:12:24 +00:00
public static void writeOutput(FeatureGroup features, Mbtiles output, DiskBacked fileSize,
MbtilesMetadata mbtilesMetadata, FlatmapConfig config, Stats stats) {
2021-08-10 10:55:30 +00:00
var timer = stats.startStage("mbtiles");
2021-09-18 01:12:24 +00:00
MbtilesWriter writer = new MbtilesWriter(features, output, config, mbtilesMetadata, stats,
2021-07-18 09:57:48 +00:00
features.layerStats());
2021-04-12 10:54:52 +00:00
2021-08-05 11:09:52 +00:00
var pipeline = WorkerPipeline.start("mbtiles", stats);
2021-07-26 00:49:58 +00:00
2021-10-20 01:57:47 +00:00
// a larger tile queue size helps keep cores busy, but needs a lot of RAM
// 5k works fine with 100GB of RAM, so adjust the queue size down from there
// but no less than 100
int queueSize = Math.max(
100,
(int) (5_000d * ProcessInfo.getMaxMemoryBytes() / 100_000_000_000d)
);
2021-07-26 00:49:58 +00:00
2021-08-05 11:09:52 +00:00
WorkerPipeline<TileBatch> encodeBranch, writeBranch = null;
2021-07-27 02:01:55 +00:00
if (config.emitTilesInOrder()) {
2021-09-10 00:46:20 +00:00
/*
* To emit tiles in order, fork the input queue and send features to both the encoder and writer. The writer
* waits on them to be encoded in the order they were received, and the encoder processes them in parallel.
* One batch might take a long time to process, so make the queues very big to avoid idle encoding CPUs.
*/
2021-07-27 12:09:06 +00:00
WorkQueue<TileBatch> writerQueue = new WorkQueue<>("mbtiles_writer_queue", queueSize, 1, stats);
2021-08-05 11:09:52 +00:00
encodeBranch = pipeline
2021-08-05 11:58:13 +00:00
.<TileBatch>fromGenerator("reader", next -> {
2021-09-10 00:46:20 +00:00
writer.readFeaturesAndBatch(batch -> {
2021-08-05 11:58:13 +00:00
next.accept(batch);
writerQueue.accept(batch); // also send immediately to writer
});
writerQueue.close();
2021-09-10 00:46:20 +00:00
// use only 1 thread since readFeaturesAndBatch needs to be single-threaded
2021-08-05 11:58:13 +00:00
}, 1)
2021-07-26 11:27:56 +00:00
.addBuffer("reader_queue", queueSize)
2021-07-27 02:01:55 +00:00
.sinkTo("encoder", config.threads(), writer::tileEncoderSink);
2021-07-26 11:27:56 +00:00
// the tile writer will wait on the result of each batch to ensure tiles are written in order
2021-08-05 11:09:52 +00:00
writeBranch = pipeline.readFromQueue(writerQueue)
2021-09-10 00:46:20 +00:00
// use only 1 thread since tileWriter needs to be single-threaded
2021-07-26 11:27:56 +00:00
.sinkTo("writer", 1, writer::tileWriter);
} else {
2021-09-10 00:46:20 +00:00
/*
* If we don't need to emit tiles in order, just send the features to the encoder, and when it finishes with
* a tile send that to the writer.
*/
2021-08-05 11:09:52 +00:00
encodeBranch = pipeline
2021-09-10 00:46:20 +00:00
// use only 1 thread since readFeaturesAndBatch needs to be single-threaded
.fromGenerator("reader", writer::readFeaturesAndBatch, 1)
2021-07-27 02:01:55 +00:00
.addBuffer("reader_queue", queueSize)
.addWorker("encoder", config.threads(), writer::tileEncoder)
.addBuffer("writer_queue", queueSize)
2021-09-10 00:46:20 +00:00
// use only 1 thread since tileWriter needs to be single-threaded
2021-07-27 02:01:55 +00:00
.sinkTo("writer", 1, writer::tileWriter);
2021-07-26 11:27:56 +00:00
}
2021-04-12 10:54:52 +00:00
2021-09-10 00:46:20 +00:00
var loggers = ProgressLoggers.create()
.addRatePercentCounter("features", features.numFeaturesWritten(), writer.featuresProcessed)
2021-10-20 01:57:47 +00:00
.addFileSize(features)
2021-06-06 12:00:04 +00:00
.addRateCounter("tiles", writer::tilesEmitted)
2021-05-08 10:53:37 +00:00
.addFileSize(fileSize)
2021-08-10 10:55:30 +00:00
.newLine()
2021-04-12 10:54:52 +00:00
.addProcessStats()
2021-08-10 10:55:30 +00:00
.newLine()
2021-08-05 11:09:52 +00:00
.addPipelineStats(encodeBranch)
.addPipelineStats(writeBranch)
2021-07-26 00:49:58 +00:00
.newLine()
2021-09-10 00:46:20 +00:00
.add(writer::getLastTileLogDetails);
2021-04-12 10:54:52 +00:00
2021-07-26 00:49:58 +00:00
encodeBranch.awaitAndLog(loggers, config.logInterval());
2021-07-27 02:01:55 +00:00
if (writeBranch != null) {
writeBranch.awaitAndLog(loggers, config.logInterval());
}
2021-07-26 00:49:58 +00:00
writer.printTileStats();
2021-06-08 00:55:23 +00:00
timer.stop();
2021-04-12 10:54:52 +00:00
}
2021-09-10 00:46:20 +00:00
private String getLastTileLogDetails() {
TileCoord lastTile = lastTileWritten.get();
String blurb;
if (lastTile == null) {
blurb = "n/a";
} else {
var extentForZoom = config.bounds().tileExtents().getForZoom(lastTile.z());
int zMinX = extentForZoom.minX();
int zMaxX = extentForZoom.maxX();
2021-10-20 01:57:47 +00:00
blurb = "%d/%d/%d (z%d %s%%) %s".formatted(
2021-09-10 00:46:20 +00:00
lastTile.z(), lastTile.x(), lastTile.y(),
lastTile.z(), (100 * (lastTile.x() + 1 - zMinX)) / (zMaxX - zMinX),
lastTile.getDebugUrl()
);
}
return "last tile: " + blurb;
}
private void readFeaturesAndBatch(Consumer<TileBatch> next) {
2021-07-18 09:57:48 +00:00
int currentZoom = Integer.MIN_VALUE;
2021-07-26 00:49:58 +00:00
TileBatch batch = new TileBatch();
2021-07-26 11:27:56 +00:00
long featuresInThisBatch = 0;
long tilesInThisBatch = 0;
2021-07-18 09:57:48 +00:00
for (var feature : features) {
2021-09-10 00:46:20 +00:00
int z = feature.tileCoord().z();
if (z != currentZoom) {
LOGGER.trace("Starting z" + z);
2021-07-18 09:57:48 +00:00
currentZoom = z;
}
2021-07-29 11:07:58 +00:00
long thisTileFeatures = feature.getNumFeaturesToEmit();
if (tilesInThisBatch > 0 &&
(tilesInThisBatch >= MAX_TILES_PER_BATCH ||
((featuresInThisBatch + thisTileFeatures) > MAX_FEATURES_PER_BATCH))) {
2021-07-26 00:49:58 +00:00
next.accept(batch);
batch = new TileBatch();
2021-07-26 11:27:56 +00:00
featuresInThisBatch = 0;
tilesInThisBatch = 0;
2021-07-26 00:49:58 +00:00
}
featuresInThisBatch += thisTileFeatures;
2021-07-27 12:09:06 +00:00
tilesInThisBatch++;
2021-07-29 01:47:13 +00:00
batch.in.add(feature);
2021-07-26 00:49:58 +00:00
}
if (!batch.in.isEmpty()) {
next.accept(batch);
2021-07-18 09:57:48 +00:00
}
}
2021-09-10 00:46:20 +00:00
private void tileEncoderSink(Supplier<TileBatch> prev) throws IOException {
2021-07-27 02:01:55 +00:00
tileEncoder(prev, batch -> {
2021-09-10 00:46:20 +00:00
// no next step
2021-07-27 02:01:55 +00:00
});
}
2021-09-10 00:46:20 +00:00
private void tileEncoder(Supplier<TileBatch> prev, Consumer<TileBatch> next) throws IOException {
2021-07-26 00:49:58 +00:00
TileBatch batch;
2021-09-10 00:46:20 +00:00
/*
* To optimize emitting many identical consecutive tiles (like large ocean areas), memoize output to avoid
* recomputing if the input hasn't changed.
*/
2021-04-12 10:54:52 +00:00
byte[] lastBytes = null, lastEncoded = null;
2021-07-26 00:49:58 +00:00
while ((batch = prev.get()) != null) {
Queue<Mbtiles.TileEntry> result = new ArrayDeque<>(batch.size());
2021-07-29 01:47:13 +00:00
FeatureGroup.TileFeatures last = null;
2021-09-10 00:46:20 +00:00
// each batch contains tile ordered by z asc, x asc, y desc
2021-07-29 01:47:13 +00:00
for (int i = 0; i < batch.in.size(); i++) {
FeatureGroup.TileFeatures tileFeatures = batch.in.get(i);
2021-07-29 11:07:58 +00:00
featuresProcessed.incBy(tileFeatures.getNumFeaturesProcessed());
2021-07-26 00:49:58 +00:00
byte[] bytes, encoded;
if (tileFeatures.hasSameContents(last)) {
bytes = lastBytes;
encoded = lastEncoded;
memoizedTiles.inc();
} else {
2021-09-10 00:46:20 +00:00
VectorTile en = tileFeatures.getVectorTileEncoder();
2021-07-26 00:49:58 +00:00
encoded = en.encode();
2021-10-20 01:57:47 +00:00
bytes = gzip(encoded);
2021-07-26 00:49:58 +00:00
last = tileFeatures;
lastEncoded = encoded;
lastBytes = bytes;
if (encoded.length > 1_000_000) {
2021-09-10 00:46:20 +00:00
LOGGER.warn(tileFeatures.tileCoord() + " " + (encoded.length / 1024) + "kb uncompressed");
2021-07-26 00:49:58 +00:00
}
2021-04-12 10:54:52 +00:00
}
2021-09-10 00:46:20 +00:00
int zoom = tileFeatures.tileCoord().z();
2021-07-26 00:49:58 +00:00
int encodedLength = encoded.length;
totalTileSizesByZoom[zoom].incBy(encodedLength);
maxTileSizesByZoom[zoom].accumulate(encodedLength);
2021-09-10 00:46:20 +00:00
result.add(new Mbtiles.TileEntry(tileFeatures.tileCoord(), bytes));
2021-04-12 10:54:52 +00:00
}
2021-09-10 00:46:20 +00:00
// hand result off to writer
2021-07-26 00:49:58 +00:00
batch.out.complete(result);
2021-07-27 02:01:55 +00:00
next.accept(batch);
2021-04-12 10:54:52 +00:00
}
}
2021-07-26 00:49:58 +00:00
private void tileWriter(Supplier<TileBatch> tileBatches) throws ExecutionException, InterruptedException {
2021-09-10 00:46:20 +00:00
db.createTables();
2021-05-08 10:53:37 +00:00
if (!config.deferIndexCreation()) {
2021-09-10 00:46:20 +00:00
db.addTileIndex();
2021-05-08 10:53:37 +00:00
} else {
2021-09-10 00:46:20 +00:00
LOGGER.info("Deferring index creation. Add later by executing: " + Mbtiles.ADD_TILE_INDEX_SQL);
2021-05-08 10:53:37 +00:00
}
2021-04-12 10:54:52 +00:00
2021-05-08 10:53:37 +00:00
db.metadata()
2021-09-18 01:12:24 +00:00
.setName(mbtilesMetadata.name())
2021-05-08 10:53:37 +00:00
.setFormat("pbf")
2021-09-18 01:12:24 +00:00
.setDescription(mbtilesMetadata.description())
.setAttribution(mbtilesMetadata.attribution())
.setVersion(mbtilesMetadata.version())
.setType(mbtilesMetadata.type())
2021-09-10 00:46:20 +00:00
.setBoundsAndCenter(config.bounds().latLon())
2021-05-08 10:53:37 +00:00
.setMinzoom(config.minzoom())
.setMaxzoom(config.maxzoom())
2021-05-13 10:25:06 +00:00
.setJson(layerStats.getTileStats());
2021-05-08 10:53:37 +00:00
2021-07-26 00:49:58 +00:00
TileCoord lastTile = null;
2021-09-10 00:46:20 +00:00
Timer time = null;
int currentZ = Integer.MIN_VALUE;
2021-05-08 10:53:37 +00:00
try (var batchedWriter = db.newBatchedTileWriter()) {
2021-07-26 00:49:58 +00:00
TileBatch batch;
while ((batch = tileBatches.get()) != null) {
Queue<Mbtiles.TileEntry> tiles = batch.out.get();
Mbtiles.TileEntry tile;
while ((tile = tiles.poll()) != null) {
TileCoord tileCoord = tile.tile();
2021-09-10 00:46:20 +00:00
assert lastTile == null || lastTile.compareTo(tileCoord) < 0 : "Tiles out of order %s before %s"
.formatted(lastTile, tileCoord);
2021-07-26 00:49:58 +00:00
lastTile = tile.tile();
int z = tileCoord.z();
2021-09-10 00:46:20 +00:00
if (z != currentZ) {
if (time == null) {
LOGGER.info("Starting z" + z);
} else {
LOGGER.info("Finished z" + currentZ + " in " + time.stop() + ", now starting z" + z);
}
time = Timer.start();
currentZ = z;
}
2021-07-26 00:49:58 +00:00
batchedWriter.write(tile.tile(), tile.bytes());
stats.wroteTile(z, tile.bytes().length);
tilesByZoom[z].inc();
}
lastTileWritten.set(lastTile);
2021-05-01 20:08:20 +00:00
}
2021-05-08 10:53:37 +00:00
}
2021-05-01 20:08:20 +00:00
2021-10-20 01:57:47 +00:00
if (time != null) {
LOGGER.info("Finished z" + currentZ + " in " + time.stop());
}
2021-05-08 10:53:37 +00:00
if (config.optimizeDb()) {
db.vacuumAnalyze();
2021-05-01 20:08:20 +00:00
}
2021-06-04 11:22:40 +00:00
}
private void printTileStats() {
LOGGER.debug("Tile stats:");
long sumSize = 0;
long sumCount = 0;
long maxMax = 0;
for (int z = config.minzoom(); z <= config.maxzoom(); z++) {
long totalCount = tilesByZoom[z].get();
long totalSize = totalTileSizesByZoom[z].get();
sumSize += totalSize;
sumCount += totalCount;
long maxSize = maxTileSizesByZoom[z].get();
LOGGER.debug("z" + z +
" avg:" + Format.formatStorage(totalSize / Math.max(totalCount, 1), false) +
" max:" + Format.formatStorage(maxSize, false));
}
LOGGER.debug("all" +
" avg:" + Format.formatStorage(sumSize / Math.max(sumCount, 1), false) +
" max:" + Format.formatStorage(maxMax, false));
2021-06-06 12:00:04 +00:00
LOGGER.debug(" # features: " + Format.formatInteger(featuresProcessed.get()));
LOGGER.debug(" # tiles: " + Format.formatInteger(this.tilesEmitted()));
}
private long tilesEmitted() {
2021-09-10 00:46:20 +00:00
return Stream.of(tilesByZoom).mapToLong(c -> c.get()).sum();
2021-04-12 10:54:52 +00:00
}
2021-09-18 01:12:24 +00:00
/**
* Container for a batch of tiles to be processed together in the encoder and writer threads.
* <p>
* The cost of encoding a tile may vary dramatically by its size (depending on the profile) so batches are sized
* dynamically to put as little as 1 large tile, or as many as 10,000 small tiles in a batch to keep encoding threads
* busy.
*
* @param in the tile data to encode
* @param out the future that encoder thread completes to hand finished tile off to writer thread
*/
private static record TileBatch(
List<FeatureGroup.TileFeatures> in,
CompletableFuture<Queue<Mbtiles.TileEntry>> out
) {
TileBatch() {
this(new ArrayList<>(), new CompletableFuture<>());
}
public int size() {
return in.size();
}
public boolean isEmpty() {
return in.isEmpty();
2021-04-12 10:54:52 +00:00
}
2021-04-10 09:25:42 +00:00
}
}