Finish PMTiles write implementation and basic reader (#502)

pull/513/head
Brandon Liu 2023-03-14 01:56:11 +08:00 zatwierdzone przez GitHub
rodzic 68b04f5295
commit d5b404d8e2
Nie znaleziono w bazie danych klucza dla tego podpisu
ID klucza GPG: 4AEE18F83AFDEB23
9 zmienionych plików z 1030 dodań i 69 usunięć

Wyświetl plik

@ -42,6 +42,7 @@ The `planetiler-core` module includes the following software:
- `osmformat.proto` and `fileformat.proto` (generates `Osmformat.java` and `Fileformat.java`)
from [openstreetmap/OSM-binary](https://github.com/openstreetmap/OSM-binary/tree/master/osmpbf) (MIT License)
- `VarInt` from [Bazel](https://github.com/bazelbuild/bazel) (Apache license)
- `SeekableInMemoryByteChannel` from [Apache Commons compress](https://commons.apache.org/proper/commons-compress/apidocs/org/apache/commons/compress/utils/SeekableInMemoryByteChannel.html) (Apache License)
- Maven Dependencies:
- org.snakeyaml:snakeyaml-engine (Apache license)
- org.commonmark:commonmark (BSD 2-clause license)

Wyświetl plik

@ -30,7 +30,7 @@ public class BenchmarkPmtiles {
var timer = Timer.start();
var result = Pmtiles.deserializeDirectory(Pmtiles.serializeDirectory(entries));
var result = Pmtiles.directoryFromBytes(Pmtiles.directoryToBytes(entries));
assert (result.size() == entries.size());
System.err.println(

Wyświetl plik

@ -62,4 +62,14 @@ public record TileArchiveMetadata(
}
return this;
}
public Map<String, String> getAll() {
var allKvs = new LinkedHashMap<String, String>(planetilerSpecific);
allKvs.put("name", this.name);
allKvs.put("description", this.description);
allKvs.put("attribution", this.attribution);
allKvs.put("version", this.version);
allKvs.put("type", this.type);
return allKvs;
}
}

Wyświetl plik

@ -1,73 +1,30 @@
package com.onthegomap.planetiler.pmtiles;
import static com.fasterxml.jackson.annotation.JsonInclude.Include.NON_ABSENT;
import com.carrotsearch.hppc.ByteArrayList;
import com.fasterxml.jackson.annotation.JsonAnyGetter;
import com.fasterxml.jackson.annotation.JsonAnySetter;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.datatype.jdk8.Jdk8Module;
import com.onthegomap.planetiler.reader.FileFormatException;
import com.onthegomap.planetiler.util.LayerStats;
import com.onthegomap.planetiler.util.VarInt;
import java.io.IOException;
import java.nio.BufferUnderflowException;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
/**
* PMTiles is a single-file tile archive format designed for efficient access on cloud storage.
*
* @see <a href="https://github.com/protomaps/PMTiles/blob/main/spec/v3/spec.md">PMTiles Specification</a>
*/
public final class Pmtiles {
static final int HEADER_LEN = 127;
public static final class Entry implements Comparable<Entry> {
private long tileId;
private long offset;
private int length;
private int runLength;
public Entry(long tileId, long offset, int length, int runLength) {
this.tileId = tileId;
this.offset = offset;
this.length = length;
this.runLength = runLength;
}
public long tileId() {
return tileId;
}
public long offset() {
return offset;
}
public long length() {
return length;
}
public long runLength() {
return runLength;
}
@Override
public boolean equals(Object o) {
return this == o || (o instanceof Entry other &&
tileId == other.tileId &&
offset == other.offset &&
length == other.length &&
runLength == other.runLength);
}
@Override
public int hashCode() {
return Objects.hash(tileId, offset, length, runLength);
}
@Override
public int compareTo(Entry that) {
return Long.compare(this.tileId, that.tileId);
}
}
public class Pmtiles {
public enum Compression {
UNKNOWN((byte) 0),
NONE((byte) 1),
@ -117,6 +74,8 @@ public final class Pmtiles {
}
}
static final int HEADER_LEN = 127;
public record Header(
byte specVersion,
long rootDirOffset,
@ -244,6 +203,56 @@ public final class Pmtiles {
}
}
public static final class Entry implements Comparable<Entry> {
private long tileId;
private long offset;
private int length;
protected int runLength;
public Entry(long tileId, long offset, int length, int runLength) {
this.tileId = tileId;
this.offset = offset;
this.length = length;
this.runLength = runLength;
}
public long tileId() {
return tileId;
}
public long offset() {
return offset;
}
public int length() {
return length;
}
public int runLength() {
return runLength;
}
@Override
public boolean equals(Object o) {
return this == o || (o instanceof Entry other &&
tileId == other.tileId &&
offset == other.offset &&
length == other.length &&
runLength == other.runLength);
}
@Override
public int hashCode() {
return Objects.hash(tileId, offset, length, runLength);
}
@Override
public int compareTo(Entry that) {
return Long.compare(this.tileId, that.tileId);
}
}
/**
* Convert a range of entries from a directory to bytes.
*
@ -252,8 +261,8 @@ public final class Pmtiles {
* @param end the end index, exclusive.
* @return the uncompressed bytes of the directory.
*/
public static byte[] serializeDirectory(List<Entry> slice, int start, int end) {
return serializeDirectory(start == 0 && end == slice.size() ? slice : slice.subList(start, end));
public static byte[] directoryToBytes(List<Entry> slice, int start, int end) {
return directoryToBytes(start == 0 && end == slice.size() ? slice : slice.subList(start, end));
}
/**
@ -262,7 +271,7 @@ public final class Pmtiles {
* @param slice a list of entries sorted by ascending {@code tileId} with size > 0.
* @return the uncompressed bytes of the directory.
*/
public static byte[] serializeDirectory(List<Entry> slice) {
public static byte[] directoryToBytes(List<Entry> slice) {
ByteArrayList dir = new ByteArrayList();
VarInt.putVarLong(slice.size(), dir);
@ -281,7 +290,7 @@ public final class Pmtiles {
VarInt.putVarLong(entry.length, dir);
}
Pmtiles.Entry last = null;
Entry last = null;
for (var entry : slice) {
if (last != null && entry.offset == last.offset + last.length) {
VarInt.putVarLong(0, dir);
@ -294,7 +303,7 @@ public final class Pmtiles {
return dir.toArray();
}
public static List<Entry> deserializeDirectory(byte[] bytes) {
public static List<Entry> directoryFromBytes(byte[] bytes) {
ByteBuffer buffer = ByteBuffer.wrap(bytes);
int numEntries = (int) VarInt.getVarLong(buffer);
ArrayList<Entry> result = new ArrayList<>(numEntries);
@ -324,4 +333,41 @@ public final class Pmtiles {
}
return result;
}
private static final ObjectMapper objectMapper = new ObjectMapper()
.registerModules(new Jdk8Module())
.setSerializationInclusion(NON_ABSENT);
/**
* Arbitrary application-specific JSON metadata in the archive.
* <p>
* stores name, attribution, created_at, planetiler build SHA, vector_layers, etc.
*/
public record JsonMetadata(
@JsonProperty("vector_layers") List<LayerStats.VectorLayer> vectorLayers,
@JsonAnyGetter @JsonAnySetter Map<String, String> otherMetadata
) {
@JsonCreator
JsonMetadata(@JsonProperty("vector_layers") List<LayerStats.VectorLayer> vectorLayers) {
this(vectorLayers, new HashMap<>());
}
public byte[] toBytes() {
try {
return objectMapper.writeValueAsBytes(this);
} catch (JsonProcessingException e) {
throw new IllegalArgumentException("Unable to encode as string: " + this, e);
}
}
public static JsonMetadata fromBytes(byte[] bytes) {
try {
return objectMapper.readValue(bytes, JsonMetadata.class);
} catch (IOException e) {
throw new IllegalStateException("Invalid metadata json: " + bytes, e);
}
}
}
}

Wyświetl plik

@ -0,0 +1,167 @@
package com.onthegomap.planetiler.pmtiles;
import com.onthegomap.planetiler.archive.ReadableTileArchive;
import com.onthegomap.planetiler.geo.TileCoord;
import com.onthegomap.planetiler.util.CloseableIterator;
import com.onthegomap.planetiler.util.Gzip;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.SeekableByteChannel;
import java.util.Iterator;
import java.util.List;
import java.util.stream.IntStream;
import java.util.stream.Stream;
public class ReadablePmtiles implements ReadableTileArchive {
private final SeekableByteChannel channel;
private final Pmtiles.Header header;
public ReadablePmtiles(SeekableByteChannel channel) throws IOException {
this.channel = channel;
this.header = Pmtiles.Header.fromBytes(getBytes(0, Pmtiles.HEADER_LEN));
}
private synchronized byte[] getBytes(long start, int length) throws IOException {
channel.position(start);
var buf = ByteBuffer.allocate(length);
channel.read(buf);
return buf.array();
}
/**
* Finds the relevant entry for a tileId in a list of entries.
* <p>
* If there is an exact match for tileId, return that. Else if the tileId matches an entry's tileId + runLength,
* return that. Else if the preceding entry is a directory (runLength = 0), return that. Else return null.
*/
public static Pmtiles.Entry findTile(List<Pmtiles.Entry> entries, long tileId) {
int m = 0;
int n = entries.size() - 1;
while (m <= n) {
int k = (n + m) >> 1;
long cmp = tileId - entries.get(k).tileId();
if (cmp > 0) {
m = k + 1;
} else if (cmp < 0) {
n = k - 1;
} else {
return entries.get(k);
}
}
if (n >= 0 && (entries.get(n).runLength() == 0 || tileId - entries.get(n).tileId() < entries.get(n).runLength())) {
return entries.get(n);
}
return null;
}
@Override
@SuppressWarnings("java:S1168")
public byte[] getTile(int x, int y, int z) {
try {
var tileId = TileCoord.ofXYZ(x, y, z).hilbertEncoded();
long dirOffset = header.rootDirOffset();
int dirLength = (int) header.rootDirLength();
for (int depth = 0; depth <= 3; depth++) {
byte[] dirBytes = getBytes(dirOffset, dirLength);
if (header.internalCompression() == Pmtiles.Compression.GZIP) {
dirBytes = Gzip.gunzip(dirBytes);
}
var dir = Pmtiles.directoryFromBytes(dirBytes);
var entry = findTile(dir, tileId);
if (entry != null) {
if (entry.runLength() > 0) {
return getBytes(header.tileDataOffset() + entry.offset(), entry.length());
} else {
dirOffset = header.leafDirectoriesOffset() + entry.offset();
dirLength = entry.length();
}
} else {
return null;
}
}
} catch (IOException e) {
throw new IllegalStateException("Could not get tile", e);
}
return null;
}
public Pmtiles.Header getHeader() {
return header;
}
public Pmtiles.JsonMetadata getJsonMetadata() throws IOException {
var buf = getBytes(header.jsonMetadataOffset(), (int) header.jsonMetadataLength());
if (header.internalCompression() == Pmtiles.Compression.GZIP) {
buf = Gzip.gunzip(buf);
}
return Pmtiles.JsonMetadata.fromBytes(buf);
}
private static class TileCoordIterator implements CloseableIterator<TileCoord> {
private final Stream<TileCoord> stream;
private final Iterator<TileCoord> iterator;
public TileCoordIterator(Stream<TileCoord> stream) {
this.stream = stream;
this.iterator = stream.iterator();
}
@Override
public void close() {
stream.close();
}
@Override
public boolean hasNext() {
return this.iterator.hasNext();
}
@Override
public TileCoord next() {
return this.iterator.next();
}
}
private List<Pmtiles.Entry> readDir(long offset, int length) throws IOException {
var buf = getBytes(offset, length);
if (header.internalCompression() == Pmtiles.Compression.GZIP) {
buf = Gzip.gunzip(buf);
}
return Pmtiles.directoryFromBytes(buf);
}
// Warning: this will only work on z15 or less pmtiles which planetiler creates
private Stream<TileCoord> getTileCoords(List<Pmtiles.Entry> dir) {
return dir.stream().flatMap(entry -> {
try {
return entry.runLength() == 0 ?
getTileCoords(readDir(header.leafDirectoriesOffset() + entry.offset(), entry.length())) : IntStream
.range((int) entry.tileId(), (int) entry.tileId() + entry.runLength()).mapToObj(TileCoord::hilbertDecode);
} catch (IOException e) {
throw new IllegalStateException(e);
}
});
}
@Override
public CloseableIterator<TileCoord> getAllTileCoords() {
List<Pmtiles.Entry> rootDir;
try {
rootDir = readDir(header.rootDirOffset(), (int) header.rootDirLength());
return new TileCoordIterator(getTileCoords(rootDir));
} catch (IOException e) {
throw new IllegalStateException(e);
}
}
@Override
public void close() throws IOException {
channel.close();
}
}

Wyświetl plik

@ -0,0 +1,318 @@
package com.onthegomap.planetiler.pmtiles;
import com.carrotsearch.hppc.ByteArrayList;
import com.carrotsearch.hppc.LongLongHashMap;
import com.onthegomap.planetiler.archive.TileArchiveMetadata;
import com.onthegomap.planetiler.archive.TileEncodingResult;
import com.onthegomap.planetiler.archive.WriteableTileArchive;
import com.onthegomap.planetiler.collection.Hppc;
import com.onthegomap.planetiler.config.PlanetilerConfig;
import com.onthegomap.planetiler.geo.GeoUtils;
import com.onthegomap.planetiler.geo.TileCoord;
import com.onthegomap.planetiler.geo.TileOrder;
import com.onthegomap.planetiler.util.Format;
import com.onthegomap.planetiler.util.Gzip;
import com.onthegomap.planetiler.util.LayerStats;
import com.onthegomap.planetiler.util.SeekableInMemoryByteChannel;
import java.io.IOException;
import java.io.UncheckedIOException;
import java.nio.ByteBuffer;
import java.nio.channels.FileChannel;
import java.nio.channels.SeekableByteChannel;
import java.nio.file.Path;
import java.nio.file.StandardOpenOption;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
import java.util.OptionalLong;
import org.locationtech.jts.geom.Envelope;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* PMTiles is a single-file tile archive format designed for efficient access on cloud storage.
*
* @see <a href="https://github.com/protomaps/PMTiles/blob/main/spec/v3/spec.md">PMTiles Specification</a>
*/
public final class WriteablePmtiles implements WriteableTileArchive {
static final int INIT_SECTION = 16384;
private static final Logger LOGGER = LoggerFactory.getLogger(WriteablePmtiles.class);
final LongLongHashMap hashToOffset = Hppc.newLongLongHashMap();
final ArrayList<Pmtiles.Entry> entries = new ArrayList<>();
private final SeekableByteChannel out;
private long currentOffset = 0;
private long numUnhashedTiles = 0;
private long numAddressedTiles = 0;
private LayerStats layerStats;
private TileArchiveMetadata tileArchiveMetadata;
private boolean isClustered = true;
private WriteablePmtiles(SeekableByteChannel channel) throws IOException {
this.out = channel;
out.write(ByteBuffer.allocate(INIT_SECTION));
}
private static Directories makeDirectoriesWithLeaves(List<Pmtiles.Entry> subEntries, int leafSize, int attemptNum)
throws IOException {
LOGGER.info("Building directories with {} entries per leaf, attempt {}...", leafSize, attemptNum);
ArrayList<Pmtiles.Entry> rootEntries = new ArrayList<>();
ByteArrayList leavesOutputStream = new ByteArrayList();
int leavesLength = 0;
int numLeaves = 0;
for (int i = 0; i < subEntries.size(); i += leafSize) {
numLeaves++;
int end = i + leafSize;
if (i + leafSize > subEntries.size()) {
end = subEntries.size();
}
byte[] leafBytes = Pmtiles.directoryToBytes(subEntries, i, end);
leafBytes = Gzip.gzip(leafBytes);
rootEntries.add(new Pmtiles.Entry(subEntries.get(i).tileId(), leavesLength, leafBytes.length, 0));
leavesOutputStream.add(leafBytes);
leavesLength += leafBytes.length;
}
byte[] rootBytes = Pmtiles.directoryToBytes(rootEntries);
rootBytes = Gzip.gzip(rootBytes);
LOGGER.info("Built directories with {} leaves, {}B root directory", rootEntries.size(), rootBytes.length);
return new Directories(rootBytes, leavesOutputStream.toArray(), numLeaves, leafSize, attemptNum);
}
/**
* Serialize all entries into bytes, choosing the # of leaf directories to ensure the header+root fits in 16 KB.
*
* @param entries a sorted ObjectArrayList of all entries in the tileset.
* @return byte arrays of the root and all leaf directories, and the # of leaves.
* @throws IOException if compression fails
*/
protected static Directories makeDirectories(List<Pmtiles.Entry> entries) throws IOException {
int maxEntriesRootOnly = 16384;
int attemptNum = 1;
if (entries.size() < maxEntriesRootOnly) {
byte[] testBytes = Pmtiles.directoryToBytes(entries, 0, entries.size());
testBytes = Gzip.gzip(testBytes);
if (testBytes.length < INIT_SECTION - Pmtiles.HEADER_LEN) {
return new Directories(testBytes, new byte[0], 0, 0, attemptNum);
}
}
double estimatedLeafSize = entries.size() / 3_500d;
int leafSize = (int) Math.max(estimatedLeafSize, 4096);
while (true) {
Directories temp = makeDirectoriesWithLeaves(entries, leafSize, attemptNum++);
if (temp.root.length < INIT_SECTION - Pmtiles.HEADER_LEN) {
return temp;
}
leafSize *= 1.2;
}
}
public static WriteablePmtiles newWriteToFile(Path path) throws IOException {
return new WriteablePmtiles(
FileChannel.open(path, StandardOpenOption.CREATE, StandardOpenOption.READ, StandardOpenOption.WRITE));
}
public static WriteablePmtiles newWriteToMemory(SeekableInMemoryByteChannel bytes) throws IOException {
return new WriteablePmtiles(bytes);
}
@Override
public TileOrder tileOrder() {
return TileOrder.HILBERT;
}
@Override
public void initialize(PlanetilerConfig config, TileArchiveMetadata tileArchiveMetadata, LayerStats layerStats) {
this.layerStats = layerStats;
this.tileArchiveMetadata = tileArchiveMetadata;
}
@Override
public void finish(PlanetilerConfig config) {
if (!isClustered) {
LOGGER.info("Tile data was not written in order, sorting entries...");
Collections.sort(entries);
LOGGER.info("Done sorting.");
}
try {
Directories directories = makeDirectories(entries);
byte[] jsonBytes = new Pmtiles.JsonMetadata(layerStats.getTileStats(), tileArchiveMetadata.getAll()).toBytes();
jsonBytes = Gzip.gzip(jsonBytes);
Envelope envelope = config.bounds().latLon();
Pmtiles.Header header = new Pmtiles.Header(
(byte) 3,
Pmtiles.HEADER_LEN,
directories.root.length,
INIT_SECTION + currentOffset,
jsonBytes.length,
INIT_SECTION + currentOffset + jsonBytes.length,
directories.leaves.length,
INIT_SECTION,
currentOffset,
numAddressedTiles,
entries.size(),
hashToOffset.size() + numUnhashedTiles,
isClustered,
Pmtiles.Compression.GZIP,
Pmtiles.Compression.GZIP,
Pmtiles.TileType.MVT,
(byte) config.minzoom(),
(byte) config.maxzoom(),
(int) (envelope.getMinX() * 10_000_000),
(int) (envelope.getMinY() * 10_000_000),
(int) (envelope.getMaxX() * 10_000_000),
(int) (envelope.getMaxY() * 10_000_000),
(byte) Math.ceil(GeoUtils.getZoomFromLonLatBounds(envelope)),
(int) ((envelope.getMinX() + envelope.getMaxX()) / 2 * 10_000_000),
(int) ((envelope.getMinY() + envelope.getMaxY()) / 2 * 10_000_000)
);
LOGGER.info("Writing metadata and leaf directories...");
out.write(ByteBuffer.wrap(jsonBytes));
out.write(ByteBuffer.wrap(directories.leaves));
LOGGER.info("Writing header...");
out.position(0);
out.write(ByteBuffer.wrap(header.toBytes()));
out.write(ByteBuffer.wrap(directories.root));
Format format = Format.defaultInstance();
if (LOGGER.isInfoEnabled()) {
LOGGER.info("# addressed tiles: {}", numAddressedTiles);
LOGGER.info("# of tile entries: {}", entries.size());
LOGGER.info("# of tile contents: {}", (hashToOffset.size() + numUnhashedTiles));
LOGGER.info("Root directory: {}B", format.storage(directories.root.length, false));
LOGGER.info("# leaves: {}", directories.numLeaves);
if (directories.numLeaves > 0) {
LOGGER.info("Leaf directories: {}B", format.storage(directories.leaves.length, false));
LOGGER
.info("Avg leaf size: {}B", format.storage(directories.leaves.length / directories.numLeaves, false));
}
LOGGER
.info("Total dir bytes: {}B", format.storage(directories.root.length + directories.leaves.length, false));
double tot = (double) directories.root.length + directories.leaves.length;
LOGGER.info("Average bytes per addressed tile: {}", tot / numAddressedTiles);
}
} catch (IOException e) {
LOGGER.error(e.getMessage());
}
}
@Override
public void close() throws IOException {
out.close();
}
public WriteableTileArchive.TileWriter newTileWriter() {
return new DeduplicatingTileWriter();
}
public record Directories(byte[] root, byte[] leaves, int numLeaves, int leafSize, int numAttempts) {
@Override
public boolean equals(Object o) {
return o instanceof Directories that &&
numLeaves == that.numLeaves &&
Arrays.equals(root, that.root) &&
Arrays.equals(leaves, that.leaves) &&
leafSize == that.leafSize &&
numAttempts == that.numAttempts;
}
@Override
public int hashCode() {
int result = Objects.hash(numLeaves, leafSize, numAttempts);
result = 31 * result + Arrays.hashCode(root);
result = 31 * result + Arrays.hashCode(leaves);
return result;
}
@Override
public String toString() {
return "Directories{" +
"root=" + Arrays.toString(root) +
", leaves=" + Arrays.toString(leaves) +
", numLeaves=" + numLeaves +
", leafSize=" + leafSize +
", numAttempts=" + numAttempts +
'}';
}
}
private class DeduplicatingTileWriter implements TileWriter {
Pmtiles.Entry lastEntry = null;
@Override
public void write(TileEncodingResult encodingResult) {
numAddressedTiles++;
boolean writeTileData;
long offset;
OptionalLong tileDataHashOpt = encodingResult.tileDataHash();
var data = encodingResult.tileData();
TileCoord coord = encodingResult.coord();
long tileId = coord.hilbertEncoded();
if (!entries.isEmpty()) {
if (tileId < lastEntry.tileId()) {
isClustered = false;
} else if (tileId == lastEntry.tileId()) {
LOGGER.error("Duplicate tile detected in writer");
}
}
if (tileDataHashOpt.isPresent()) {
long tileDataHash = tileDataHashOpt.getAsLong();
if (hashToOffset.containsKey(tileDataHash)) {
offset = hashToOffset.get(tileDataHash);
writeTileData = false;
if (lastEntry != null && lastEntry.tileId() + lastEntry.runLength() == tileId &&
lastEntry.offset() == offset) {
lastEntry.runLength++;
return;
}
} else {
hashToOffset.put(tileDataHash, currentOffset);
offset = currentOffset;
writeTileData = true;
}
} else {
numUnhashedTiles++;
offset = currentOffset;
writeTileData = true;
}
var newEntry = new Pmtiles.Entry(tileId, offset, data.length, 1);
entries.add(newEntry);
lastEntry = newEntry;
if (writeTileData) {
try {
out.write(ByteBuffer.wrap(data));
} catch (IOException e) {
throw new UncheckedIOException(e);
}
currentOffset += data.length;
}
}
@Override
public void close() {
// no cleanup needed.
}
}
}

Wyświetl plik

@ -0,0 +1,221 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*
*/
package com.onthegomap.planetiler.util;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.ClosedChannelException;
import java.nio.channels.SeekableByteChannel;
import java.util.Arrays;
import java.util.concurrent.atomic.AtomicBoolean;
/**
* A {@link SeekableByteChannel} implementation that wraps a byte[].
*
* <p>
* When this channel is used for writing an internal buffer grows to accommodate incoming data. The natural size limit
* is the value of {@link Integer#MAX_VALUE} and it is not possible to {@link #position(long) set the position} or
* {@link #truncate truncate} to a value bigger than that. Internal buffer can be accessed via
* {@link SeekableInMemoryByteChannel#array()}. Adapted from <a href=
* "https://commons.apache.org/proper/commons-compress/jacoco/org.apache.commons.compress.utils/SeekableInMemoryByteChannel.java.html">Apache
* Commons Compress</a>.
* </p>
*
* @since 1.13
* @NotThreadSafe
*/
public class SeekableInMemoryByteChannel implements SeekableByteChannel {
private static final int NAIVE_RESIZE_LIMIT = Integer.MAX_VALUE >> 1;
private byte[] data;
private final AtomicBoolean closed = new AtomicBoolean();
private int position, size;
/**
* Constructor taking a byte array.
*
* <p>
* This constructor is intended to be used with pre-allocated buffer or when reading from a given byte array.
* </p>
*
* @param data input data or pre-allocated array.
*/
public SeekableInMemoryByteChannel(final byte[] data) {
this.data = data;
size = data.length;
}
/**
* Constructor taking a size of storage to be allocated.
*
* <p>
* Creates a channel and allocates internal storage of a given size.
* </p>
*
* @param size size of internal buffer to allocate, in bytes.
*/
public SeekableInMemoryByteChannel(final int size) {
this(new byte[size]);
}
/**
* Returns this channel's position.
*
* <p>
* This method violates the contract of {@link SeekableByteChannel#position()} as it will not throw any exception when
* invoked on a closed channel. Instead it will return the position the channel had when close has been called.
* </p>
*/
@Override
public long position() {
return position;
}
@Override
public SeekableByteChannel position(final long newPosition) throws IOException {
ensureOpen();
if (newPosition < 0L || newPosition > Integer.MAX_VALUE) {
throw new IOException("Position has to be in range 0.. " + Integer.MAX_VALUE);
}
position = (int) newPosition;
return this;
}
/**
* Returns the current size of entity to which this channel is connected.
*
* <p>
* This method violates the contract of {@link SeekableByteChannel#size} as it will not throw any exception when
* invoked on a closed channel. Instead it will return the size the channel had when close has been called.
* </p>
*/
@Override
public long size() {
return size;
}
/**
* Truncates the entity, to which this channel is connected, to the given size.
*
* <p>
* This method violates the contract of {@link SeekableByteChannel#truncate} as it will not throw any exception when
* invoked on a closed channel.
* </p>
*
* @throws IllegalArgumentException if size is negative or bigger than the maximum of a Java integer
*/
@Override
public SeekableByteChannel truncate(final long newSize) {
if (newSize < 0L || newSize > Integer.MAX_VALUE) {
throw new IllegalArgumentException("Size has to be in range 0.. " + Integer.MAX_VALUE);
}
if (size > newSize) {
size = (int) newSize;
}
if (position > newSize) {
position = (int) newSize;
}
return this;
}
@Override
public int read(final ByteBuffer buf) throws IOException {
ensureOpen();
int wanted = buf.remaining();
final int possible = size - position;
if (possible <= 0) {
return -1;
}
if (wanted > possible) {
wanted = possible;
}
buf.put(data, position, wanted);
position += wanted;
return wanted;
}
@Override
public void close() {
closed.set(true);
}
@Override
public boolean isOpen() {
return !closed.get();
}
@Override
public int write(final ByteBuffer b) throws IOException {
ensureOpen();
int wanted = b.remaining();
final int possibleWithoutResize = size - position;
if (wanted > possibleWithoutResize) {
final int newSize = position + wanted;
if (newSize < 0) { // overflow
resize(Integer.MAX_VALUE);
wanted = Integer.MAX_VALUE - position;
} else {
resize(newSize);
}
}
b.get(data, position, wanted);
position += wanted;
if (size < position) {
size = position;
}
return wanted;
}
/**
* Obtains the array backing this channel.
*
* <p>
* NOTE: The returned buffer is not aligned with containing data, use {@link #size()} to obtain the size of data
* stored in the buffer.
* </p>
*
* @return internal byte array.
*/
public byte[] array() {
return data;
}
private void resize(final int newLength) {
int len = data.length;
if (len <= 0) {
len = 1;
}
if (newLength < NAIVE_RESIZE_LIMIT) {
while (len < newLength) {
len <<= 1;
}
} else { // avoid overflow
len = newLength;
}
data = Arrays.copyOf(data, len);
}
private void ensureOpen() throws ClosedChannelException {
if (!isOpen()) {
throw new ClosedChannelException();
}
}
}

Wyświetl plik

@ -1,12 +1,29 @@
package com.onthegomap.planetiler.pmtiles;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.*;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.onthegomap.planetiler.Profile;
import com.onthegomap.planetiler.TestUtils;
import com.onthegomap.planetiler.archive.TileArchiveMetadata;
import com.onthegomap.planetiler.archive.TileEncodingResult;
import com.onthegomap.planetiler.config.PlanetilerConfig;
import com.onthegomap.planetiler.geo.TileCoord;
import com.onthegomap.planetiler.reader.FileFormatException;
import com.onthegomap.planetiler.util.LayerStats;
import com.onthegomap.planetiler.util.SeekableInMemoryByteChannel;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.FileChannel;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.OptionalLong;
import java.util.Set;
import java.util.stream.Collectors;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
class PmtilesTest {
@ -105,7 +122,7 @@ class PmtilesTest {
ArrayList<Pmtiles.Entry> in = new ArrayList<>();
in.add(new Pmtiles.Entry(0, 0, 1, 1));
List<Pmtiles.Entry> out = Pmtiles.deserializeDirectory(Pmtiles.serializeDirectory(in));
List<Pmtiles.Entry> out = Pmtiles.directoryFromBytes(Pmtiles.directoryToBytes(in));
assertEquals(in, out);
}
@ -118,9 +135,9 @@ class PmtilesTest {
in.add(new Pmtiles.Entry(1, 1, 1, 1));
in.add(new Pmtiles.Entry(2, 3, 1, 1));
List<Pmtiles.Entry> out = Pmtiles.deserializeDirectory(Pmtiles.serializeDirectory(in));
List<Pmtiles.Entry> out = Pmtiles.directoryFromBytes(Pmtiles.directoryToBytes(in));
assertEquals(in, out);
out = Pmtiles.deserializeDirectory(Pmtiles.serializeDirectory(in, 0, in.size()));
out = Pmtiles.directoryFromBytes(Pmtiles.directoryToBytes(in, 0, in.size()));
assertEquals(in, out);
}
@ -133,7 +150,188 @@ class PmtilesTest {
in.add(new Pmtiles.Entry(1, 1, 1, 1));
in.add(new Pmtiles.Entry(2, 3, 1, 1));
List<Pmtiles.Entry> out = Pmtiles.deserializeDirectory(Pmtiles.serializeDirectory(in, 1, 2));
List<Pmtiles.Entry> out = Pmtiles.directoryFromBytes(
Pmtiles.directoryToBytes(in, 1, 2));
assertEquals(1, out.size());
}
@Test
void testBuildDirectoriesRootOnly() throws IOException {
ArrayList<Pmtiles.Entry> in = new ArrayList<>();
for (int i = 0; i < 1000; i++) {
in.add(new Pmtiles.Entry(i, i * 100, 100, 1));
}
var result = WriteablePmtiles.makeDirectories(in);
assertEquals(1, result.numAttempts());
}
@Test
void testBuildDirectoriesLeavesNotTooSmall() throws IOException {
ArrayList<Pmtiles.Entry> in = new ArrayList<>();
for (int i = 0; i < 100000; i++) {
in.add(new Pmtiles.Entry(i, i * 100, 100, 1));
}
var result = WriteablePmtiles.makeDirectories(in);
assertTrue(result.leafSize() >= 4096, "entries in leaf: " + result.leafSize());
}
@Test
void testWritePmtilesSingleEntry() throws IOException {
var bytes = new SeekableInMemoryByteChannel(0);
var in = WriteablePmtiles.newWriteToMemory(bytes);
var config = PlanetilerConfig.defaults();
in.initialize(config, new TileArchiveMetadata(new Profile.NullProfile()), new LayerStats());
var writer = in.newTileWriter();
writer.write(new TileEncodingResult(TileCoord.ofXYZ(0, 0, 1), new byte[]{0xa, 0x2}, OptionalLong.empty()));
in.finish(config);
var reader = new ReadablePmtiles(bytes);
var header = reader.getHeader();
assertEquals(1, header.numAddressedTiles());
assertEquals(1, header.numTileContents());
assertEquals(1, header.numTileEntries());
assertArrayEquals(new byte[]{0xa, 0x2}, reader.getTile(0, 0, 1));
assertNull(reader.getTile(0, 0, 0));
assertNull(reader.getTile(0, 0, 2));
Set<TileCoord> coordset = reader.getAllTileCoords().stream().collect(Collectors.toSet());
assertEquals(1, coordset.size());
}
@Test
void testWritePmtilesToFileWithMetadata(@TempDir Path tempDir) throws IOException {
try (var in = WriteablePmtiles.newWriteToFile(tempDir.resolve("tmp.pmtiles"))) {
var config = PlanetilerConfig.defaults();
in.initialize(config,
new TileArchiveMetadata("MyName", "MyDescription", "MyAttribution", "MyVersion", "baselayer", new HashMap<>()),
new LayerStats());
var writer = in.newTileWriter();
writer.write(new TileEncodingResult(TileCoord.ofXYZ(0, 0, 0), new byte[]{0xa, 0x2}, OptionalLong.empty()));
in.finish(config);
}
var reader = new ReadablePmtiles(FileChannel.open(tempDir.resolve("tmp.pmtiles")));
assertArrayEquals(new byte[]{0xa, 0x2}, reader.getTile(0, 0, 0));
var metadata = reader.getJsonMetadata();
assertEquals("MyName", metadata.otherMetadata().get("name"));
assertEquals("MyDescription", metadata.otherMetadata().get("description"));
assertEquals("MyAttribution", metadata.otherMetadata().get("attribution"));
assertEquals("MyVersion", metadata.otherMetadata().get("version"));
assertEquals("baselayer", metadata.otherMetadata().get("type"));
}
@Test
void testPmtilesMetadataTopLevelKeys() throws IOException {
var hashMap = new HashMap<String, String>();
hashMap.put("testkey", "testvalue");
var metadata = new Pmtiles.JsonMetadata(List.of(), hashMap);
var bytes = metadata.toBytes();
ObjectMapper mapper = new ObjectMapper();
var node = mapper.readTree(bytes);
assertEquals("testvalue", node.get("testkey").asText());
}
@Test
void testReadPmtilesFromTippecanoe() throws IOException {
// '{"type":"Polygon","coordinates":[[[0,0],[0,1],[1,1],[1,0],[0,0]]]}' | ./tippecanoe -zg -o box1degree.pmtiles
var reader = new ReadablePmtiles(FileChannel.open(TestUtils.pathToResource("box1degree.pmtiles")));
var header = reader.getHeader();
assertTrue(header.maxZoom() <= 15);
assertNotNull(reader.getTile(0, 0, 0));
}
@Test
void testWritePmtilesDuplication() throws IOException {
var bytes = new SeekableInMemoryByteChannel(0);
var in = WriteablePmtiles.newWriteToMemory(bytes);
var config = PlanetilerConfig.defaults();
in.initialize(config, new TileArchiveMetadata(new Profile.NullProfile()), new LayerStats());
var writer = in.newTileWriter();
writer.write(new TileEncodingResult(TileCoord.ofXYZ(0, 0, 0), new byte[]{0xa, 0x2}, OptionalLong.of(42)));
writer.write(new TileEncodingResult(TileCoord.ofXYZ(0, 0, 1), new byte[]{0xa, 0x2}, OptionalLong.of(42)));
writer.write(new TileEncodingResult(TileCoord.ofXYZ(0, 0, 2), new byte[]{0xa, 0x2}, OptionalLong.of(42)));
in.finish(config);
var reader = new ReadablePmtiles(bytes);
var header = reader.getHeader();
assertEquals(3, header.numAddressedTiles());
assertEquals(1, header.numTileContents());
assertEquals(2, header.numTileEntries()); // z0 and z1 are contiguous
assertArrayEquals(new byte[]{0xa, 0x2}, reader.getTile(0, 0, 0));
assertArrayEquals(new byte[]{0xa, 0x2}, reader.getTile(0, 0, 1));
assertArrayEquals(new byte[]{0xa, 0x2}, reader.getTile(0, 0, 2));
Set<TileCoord> coordset = reader.getAllTileCoords().stream().collect(Collectors.toSet());
assertEquals(3, coordset.size());
}
@Test
void testWritePmtilesUnclustered() throws IOException {
var bytes = new SeekableInMemoryByteChannel(0);
var in = WriteablePmtiles.newWriteToMemory(bytes);
var config = PlanetilerConfig.defaults();
in.initialize(config, new TileArchiveMetadata(new Profile.NullProfile()), new LayerStats());
var writer = in.newTileWriter();
writer.write(new TileEncodingResult(TileCoord.ofXYZ(0, 0, 1), new byte[]{0xa, 0x2}, OptionalLong.of(42)));
writer.write(new TileEncodingResult(TileCoord.ofXYZ(0, 0, 0), new byte[]{0xa, 0x2}, OptionalLong.of(42)));
in.finish(config);
var reader = new ReadablePmtiles(bytes);
var header = reader.getHeader();
assertEquals(2, header.numAddressedTiles());
assertEquals(1, header.numTileContents());
assertEquals(2, header.numTileEntries());
assertFalse(header.clustered());
assertArrayEquals(new byte[]{0xa, 0x2}, reader.getTile(0, 0, 0));
assertArrayEquals(new byte[]{0xa, 0x2}, reader.getTile(0, 0, 1));
Set<TileCoord> coordset = reader.getAllTileCoords().stream().collect(Collectors.toSet());
assertEquals(2, coordset.size());
}
@Test
void testWritePmtilesLeafDirectories() throws IOException {
var bytes = new SeekableInMemoryByteChannel(0);
var in = WriteablePmtiles.newWriteToMemory(bytes);
var config = PlanetilerConfig.defaults();
in.initialize(config, new TileArchiveMetadata(new Profile.NullProfile()), new LayerStats());
var writer = in.newTileWriter();
int ENTRIES = 20000;
for (int i = 0; i < ENTRIES; i++) {
writer.write(new TileEncodingResult(TileCoord.hilbertDecode(i), ByteBuffer.allocate(4).putInt(i).array(),
OptionalLong.empty()));
}
in.finish(config);
var reader = new ReadablePmtiles(bytes);
var header = reader.getHeader();
assertEquals(ENTRIES, header.numAddressedTiles());
assertEquals(ENTRIES, header.numTileContents());
assertEquals(ENTRIES, header.numTileEntries());
assertTrue(header.leafDirectoriesLength() > 0);
for (int i = 0; i < ENTRIES; i++) {
var coord = TileCoord.hilbertDecode(i);
assertArrayEquals(ByteBuffer.allocate(4).putInt(i).array(), reader.getTile(coord.x(), coord.y(), coord.z()),
"tileCoord=%s did not match".formatted(coord.toString()));
}
Set<TileCoord> coordset = reader.getAllTileCoords().stream().collect(Collectors.toSet());
assertEquals(ENTRIES, coordset.size());
for (int i = 0; i < ENTRIES; i++) {
var coord = TileCoord.hilbertDecode(i);
assertTrue(coordset.contains(coord), "tileCoord=%s not in result".formatted(coord.toString()));
}
}
}

Plik binarny nie jest wyświetlany.