sort way members before inserting, and collect common keys

osm-mirror
Mike Barry 2023-04-08 05:51:50 -04:00
rodzic e381f54f62
commit 64ff5dbae8
8 zmienionych plików z 1517 dodań i 50 usunięć

Wyświetl plik

@ -0,0 +1,34 @@
package com.onthegomap.planetiler.osmmirror;
import com.carrotsearch.hppc.ObjectIntHashMap;
import com.carrotsearch.hppc.ObjectIntMap;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.Objects;
public class CommonTags {
// (curl 'https://taginfo.openstreetmap.org/api/4/tags/popular?sortname=count_all&sortorder=desc&page=1&rp=999&qtype=tag' | jq -r '.data[].key + "\n" + .data[].value') | awk '!x[$0]++' > planetiler-core/src/main/resources/common_tags.txt
// (curl 'https://taginfo.openstreetmap.org/api/4/tags/popular?sortname=count_all&sortorder=desc&page=1&rp=999&qtype=tag' | jq -r '.data[].key + "\n" + .data[].value'; curl 'https://taginfo.openstreetmap.org/api/4/keys/all?include=prevalent_values&sortname=count_all&sortorder=desc&page=1&rp=999&qtype=key' | jq -r '.data[].key') | sort | uniq > planetiler-core/src/main/resources/common_tags.txt
private final String[] forward;
private final ObjectIntMap<String> reverse;
public CommonTags() {
try (var is = CommonTags.class.getResourceAsStream("/common_tags.txt")) {
forward = new String(Objects.requireNonNull(is).readAllBytes(), StandardCharsets.UTF_8).split("\n");
reverse = new ObjectIntHashMap<>(forward.length);
for (int i = 0; i < forward.length; i++) {
reverse.putIfAbsent(forward[i], i);
}
} catch (IOException e) {
throw new RuntimeException(e);
}
}
public String decode(int key) {
return forward[key];
}
public int encode(String string) {
return reverse.getOrDefault(string, -1);
}
}

Wyświetl plik

@ -0,0 +1,290 @@
package com.onthegomap.planetiler.osmmirror;
import static com.onthegomap.planetiler.util.MemoryEstimator.CLASS_HEADER_BYTES;
import static com.onthegomap.planetiler.util.MemoryEstimator.LONG_BYTES;
import static com.onthegomap.planetiler.util.MemoryEstimator.POINTER_BYTES;
import com.onthegomap.planetiler.stats.ProcessInfo;
import com.onthegomap.planetiler.stats.ProgressLoggers;
import com.onthegomap.planetiler.stats.Stats;
import com.onthegomap.planetiler.stats.Timer;
import com.onthegomap.planetiler.util.CloseableIterator;
import com.onthegomap.planetiler.util.FileUtils;
import com.onthegomap.planetiler.worker.WorkerPipeline;
import java.io.IOException;
import java.nio.channels.FileChannel;
import java.nio.file.Path;
import java.nio.file.StandardOpenOption;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Comparator;
import java.util.Iterator;
import java.util.List;
import java.util.PriorityQueue;
import java.util.concurrent.atomic.AtomicLong;
import org.msgpack.core.MessagePack;
import org.msgpack.core.MessagePacker;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public interface LongLongSorter extends Iterable<LongLongSorter.Result> {
Logger LOGGER = LoggerFactory.getLogger(LongLongSorter.class);
void put(long a, long b);
long count();
record Result(long a, long b) implements Comparable<Result> {
@Override
public int compareTo(Result o) {
int comp = Long.compare(a, o.a);
if (comp == 0) {
comp = Long.compare(b, o.b);
}
return comp;
}
}
class DiskBacked implements LongLongSorter {
private static final long BYTES_PER_ENTRY = CLASS_HEADER_BYTES + LONG_BYTES * 2 + POINTER_BYTES;
private final long limit;
private final Stats stats;
private final int maxWorkers;
long count = 0;
boolean prepared;
List<Chunk> chunks = new ArrayList<>();
Chunk chunk;
@Override
public long count() {
return chunks.stream().mapToLong(d -> d.count).sum();
}
class Chunk implements Iterable<Result> {
private final MessagePacker packer;
Chunk() {
path = tmpDir.resolve("chunk-" + chunks.size());
chunks.add(this);
try {
packer =
MessagePack.newDefaultPacker(FileChannel.open(path, StandardOpenOption.CREATE, StandardOpenOption.WRITE));
} catch (IOException e) {
throw new RuntimeException(e);
}
}
Path path;
int count = 0;
public void sort() {
Result[] array = new Result[count];
int i = 0;
try (var iter = iterator()) {
while (iter.hasNext()) {
array[i++] = iter.next();
}
}
Arrays.parallelSort(array);
try (
var p = MessagePack
.newDefaultPacker(FileChannel.open(path, StandardOpenOption.TRUNCATE_EXISTING, StandardOpenOption.WRITE))
) {
for (var item : array) {
p.packLong(item.a).packLong(item.b);
}
} catch (IOException e) {
throw new RuntimeException(e);
}
}
@Override
public CloseableIterator<Result> iterator() {
try {
var unpacker = MessagePack.newDefaultUnpacker(FileChannel.open(path, StandardOpenOption.READ));
return new CloseableIterator<>() {
int i = 0;
@Override
public void close() {
try {
unpacker.close();
} catch (IOException e) {
throw new RuntimeException(e);
}
}
@Override
public boolean hasNext() {
return i < count;
}
@Override
public Result next() {
try {
i++;
return new Result(unpacker.unpackLong(), unpacker.unpackLong());
} catch (IOException e) {
throw new RuntimeException(e);
}
}
};
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
private final Path tmpDir;
public DiskBacked(Path nodeToWayTmp, Stats stats, long limit, int maxWorkers) {
tmpDir = nodeToWayTmp;
FileUtils.createDirectory(nodeToWayTmp);
chunk = new Chunk();
this.stats = stats;
this.limit = limit;
this.maxWorkers = maxWorkers;
}
public DiskBacked(Path nodeToWayTmp, Stats stats, int maxWorkers) {
this(nodeToWayTmp, stats, 1_000_000_000 / BYTES_PER_ENTRY, maxWorkers);
}
private void prepare() {
if (!prepared) {
var timer = Timer.start();
try {
chunk.packer.close();
} catch (IOException e) {
throw new RuntimeException(e);
}
LOGGER.info("Sorting {} way members...", count());
int sortThreads = Math.min(maxWorkers,
Math.max(1, (int) (ProcessInfo.getMaxMemoryBytes() / 2 / BYTES_PER_ENTRY / limit)));
AtomicLong done = new AtomicLong(0);
var pipeline = WorkerPipeline.start("sort", stats)
.readFromTiny("chunks", chunks)
.sinkToConsumer("sort", sortThreads, (chunk) -> {
chunk.sort();
done.incrementAndGet();
});
ProgressLoggers logger = ProgressLoggers.create()
.addPercentCounter("sort", chunks.size(), done)
.newLine()
.addPipelineStats(pipeline)
.newLine()
.addProcessStats();
pipeline.awaitAndLog(logger, Duration.ofSeconds(10));
LOGGER.info("Sorted way members in {}", timer.stop());
prepared = true;
}
}
class PeekableIterator implements Comparable<PeekableIterator>, CloseableIterator<Result> {
private final CloseableIterator<Result> iterator;
Result next;
public PeekableIterator(CloseableIterator<Result> iterator) {
this.iterator = iterator;
next = iterator.next();
}
@Override
public int compareTo(PeekableIterator o) {
return next.compareTo(o.next);
}
@Override
public void close() {
iterator.close();
}
@Override
public boolean hasNext() {
return next != null;
}
@Override
public Result next() {
var result = next;
next = iterator.hasNext() ? iterator.next() : null;
return result;
}
}
@Override
public CloseableIterator<Result> iterator() {
prepare();
PriorityQueue<PeekableIterator> pq = new PriorityQueue<>();
for (var c : chunks) {
if (c.count > 0) {
pq.offer(new PeekableIterator(c.iterator()));
}
}
return new CloseableIterator<>() {
@Override
public void close() {
pq.forEach(PeekableIterator::close);
}
@Override
public boolean hasNext() {
return !pq.isEmpty();
}
@Override
public Result next() {
var item = pq.poll();
var result = item.next();
if (item.hasNext()) {
pq.offer(item);
} else {
item.close();
}
return result;
}
};
}
public void put(long a, long b) {
if (prepared) {
throw new IllegalStateException("Already prepared");
}
try {
if (chunk.count > limit) {
chunk.packer.close();
chunk = new Chunk();
}
chunk.packer.packLong(a).packLong(b);
chunk.count++;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
class InMemory implements LongLongSorter {
private List<Result> items = new ArrayList<>();
@Override
public Iterator<Result> iterator() {
items.sort(Comparator.naturalOrder());
return items.iterator();
}
public void put(long a, long b) {
items.add(new Result(a, b));
}
@Override
public long count() {
return items.size();
}
}
}

Wyświetl plik

@ -28,8 +28,8 @@ public interface OsmMirror extends AutoCloseable {
return new InMemoryOsmMirror();
}
static OsmMirror newSqliteWrite(Path path) {
return SqliteOsmMirror.newWriteToFileDatabase(path, Arguments.of());
static OsmMirror newSqliteWrite(Path path, int maxWorkers) {
return SqliteOsmMirror.newWriteToFileDatabase(path, Arguments.of(), maxWorkers);
}
static OsmMirror newSqliteMemory() {
@ -55,7 +55,11 @@ public interface OsmMirror extends AutoCloseable {
OsmInputFile in = new OsmInputFile(input);
var blockCounter = new AtomicLong();
boolean id = !type.contains("sqlite");
try (var blocks = in.get()) {
try (
var blocks = in.get();
OsmMirror out = newWriter(type, output.resolve("test.db"), processThreads);
var writer = out.newBulkWriter()
) {
record Batch(CompletableFuture<List<Serialized<? extends OsmElement>>> results, OsmBlockSource.Block block) {}
var queue = new WorkQueue<Batch>("batches", processThreads * 2, 1, stats);
@ -100,11 +104,7 @@ public interface OsmMirror extends AutoCloseable {
var writeBranch = pipeline.readFromQueue(queue)
.sinkTo("write", 1, prev -> {
try (
OsmMirror out = newWriter(type, output.resolve("test.db"));
var writer = out.newBulkWriter();
var phaserForWorker = phaser.forWorker()
) {
try (var phaserForWorker = phaser.forWorker()) {
System.err.println("Using " + out.getClass().getSimpleName());
for (var batch : prev) {
for (var item : batch.results.get()) {
@ -142,11 +142,11 @@ public interface OsmMirror extends AutoCloseable {
}
}
static OsmMirror newWriter(String type, Path path) {
static OsmMirror newWriter(String type, Path path, int maxWorkers) {
return switch (type) {
case "mapdb" -> newMapdbWrite(path);
case "mapdb-memory" -> newMapdbMemory();
case "sqlite" -> newSqliteWrite(path);
case "sqlite" -> newSqliteWrite(path, maxWorkers);
case "sqlite-memory" -> newSqliteMemory();
case "memory" -> newInMemory();
case "lmdb" -> newLmdbWrite(path);
@ -155,10 +155,10 @@ public interface OsmMirror extends AutoCloseable {
};
}
static OsmMirror newReader(String type, Path path) {
static OsmMirror newReader(String type, Path path, int maxWorkers) {
return switch (type) {
case "mapdb" -> MapDbOsmMirror.newReadFromFile(path);
case "sqlite" -> newSqliteWrite(path);
case "sqlite" -> newSqliteWrite(path, maxWorkers);
case "lmdb" -> newLmdbWrite(path);
default -> throw new IllegalArgumentException("Unrecognized type: " + type);
};

Wyświetl plik

@ -3,7 +3,6 @@ package com.onthegomap.planetiler.osmmirror;
import com.carrotsearch.hppc.LongArrayList;
import com.onthegomap.planetiler.reader.osm.OsmElement;
import java.io.IOException;
import java.io.UncheckedIOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
@ -13,44 +12,41 @@ import org.msgpack.core.MessageBufferPacker;
import org.msgpack.core.MessagePack;
import org.msgpack.core.MessagePacker;
import org.msgpack.core.MessageUnpacker;
import org.msgpack.value.Value;
import org.msgpack.value.ValueFactory;
public class OsmMirrorUtil {
private static final CommonTags tags = new CommonTags();
public static byte[] encodeTags(Map<String, Object> tags) {
if (tags.isEmpty()) {
return null;
static Value encodeString(Object string) {
if (string == null) {
return ValueFactory.newNil();
}
try (var msgPack = MessagePack.newDefaultBufferPacker()) {
packTags(tags, msgPack);
return msgPack.toByteArray();
} catch (IOException e) {
throw new UncheckedIOException(e);
String str = string.toString();
int encoded = tags.encode(str);
return encoded < 0 ? ValueFactory.newString(str) : ValueFactory.newInteger(encoded);
}
static String decodeString(Value value) {
if (value.isStringValue()) {
return value.asStringValue().asString();
} else if (value.isIntegerValue()) {
return tags.decode(value.asIntegerValue().asInt());
} else {
return null;
}
}
private static void packTags(Map<String, Object> tags, MessagePacker msgPack) throws IOException {
msgPack.packMapHeader(tags.size());
for (var entry : tags.entrySet()) {
msgPack.packString(entry.getKey());
if (entry.getValue() == null) {
msgPack.packNil();
} else {
msgPack.packString(entry.getValue().toString());
if (!tags.isEmpty()) {
msgPack.packMapHeader(tags.size());
for (var entry : tags.entrySet()) {
msgPack.packValue(encodeString(entry.getKey()));
msgPack.packValue(encodeString(entry.getValue()));
}
}
}
public static Map<String, Object> parseTags(byte[] bytes) {
if (bytes == null || bytes.length == 0) {
return Map.of();
}
try (var msgPack = MessagePack.newDefaultUnpacker(bytes)) {
return unpackTags(msgPack);
} catch (IOException e) {
throw new UncheckedIOException(e);
}
}
private static Map<String, Object> unpackTags(MessageUnpacker msgPack) throws IOException {
if (!msgPack.hasNext()) {
return Map.of();
@ -59,8 +55,8 @@ public class OsmMirrorUtil {
Map<String, Object> result = new HashMap<>(length);
for (int i = 0; i < length; i++) {
result.put(
msgPack.unpackString(),
msgPack.unpackString()
decodeString(msgPack.unpackValue()),
decodeString(msgPack.unpackValue())
);
}
return result;

Wyświetl plik

@ -7,7 +7,12 @@ import com.google.common.collect.Iterators;
import com.onthegomap.planetiler.config.Arguments;
import com.onthegomap.planetiler.mbtiles.Mbtiles;
import com.onthegomap.planetiler.reader.osm.OsmElement;
import com.onthegomap.planetiler.stats.ProgressLoggers;
import com.onthegomap.planetiler.stats.Stats;
import com.onthegomap.planetiler.stats.Timer;
import com.onthegomap.planetiler.util.CloseableIterator;
import com.onthegomap.planetiler.util.FileUtils;
import com.onthegomap.planetiler.worker.Worker;
import java.io.IOException;
import java.nio.file.Path;
import java.sql.Connection;
@ -15,10 +20,12 @@ import java.sql.DriverManager;
import java.sql.PreparedStatement;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.time.Duration;
import java.util.Iterator;
import java.util.List;
import java.util.NoSuchElementException;
import java.util.Objects;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Function;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -30,10 +37,16 @@ public class SqliteOsmMirror implements OsmMirror {
private static final Logger LOGGER = LoggerFactory.getLogger(SqliteOsmMirror.class);
private final Connection connection;
private final Path path;
private final Stats stats;
private final int maxWorkers;
private SqliteOsmMirror(Connection connection) {
private SqliteOsmMirror(Connection connection, Path path, Stats stats, int maxWorkers) {
this.connection = connection;
this.path = path;
createTables();
this.stats = stats;
this.maxWorkers = maxWorkers;
}
private static Connection newConnection(String url, SQLiteConfig defaults, Arguments args) {
@ -53,7 +66,7 @@ public class SqliteOsmMirror implements OsmMirror {
}
}
public static SqliteOsmMirror newWriteToFileDatabase(Path path, Arguments options) {
public static SqliteOsmMirror newWriteToFileDatabase(Path path, Arguments options, int maxWorkers) {
Objects.requireNonNull(path);
SQLiteConfig sqliteConfig = new SQLiteConfig();
sqliteConfig.setJournalMode(SQLiteConfig.JournalMode.OFF);
@ -62,13 +75,13 @@ public class SqliteOsmMirror implements OsmMirror {
sqliteConfig.setLockingMode(SQLiteConfig.LockingMode.EXCLUSIVE);
sqliteConfig.setTempStore(SQLiteConfig.TempStore.MEMORY);
var connection = newConnection("jdbc:sqlite:" + path.toAbsolutePath(), sqliteConfig, options);
return new SqliteOsmMirror(connection);
return new SqliteOsmMirror(connection, path, options.getStats(), maxWorkers);
}
public static SqliteOsmMirror newInMemoryDatabase() {
SQLiteConfig sqliteConfig = new SQLiteConfig();
var connection = newConnection("jdbc:sqlite::memory:", sqliteConfig, Arguments.of());
return new SqliteOsmMirror(connection);
return new SqliteOsmMirror(connection, null, Stats.inMemory(), 1);
}
@ -351,6 +364,8 @@ public class SqliteOsmMirror implements OsmMirror {
private final ChildToParentWriter nodeToRelWriter = new ChildToParentWriter(connection, "node_to_relation");
private final ChildToParentWriter wayToRelWriter = new ChildToParentWriter(connection, "way_to_relation");
private final ChildToParentWriter relToRelWriter = new ChildToParentWriter(connection, "relation_to_relation");
private final LongLongSorter nodeToWay = path == null ? new LongLongSorter.InMemory() :
new LongLongSorter.DiskBacked(path.resolveSibling("node_to_way_tmp"), stats, maxWorkers);
@Override
public void putNode(Serialized.Node node) {
@ -360,13 +375,13 @@ public class SqliteOsmMirror implements OsmMirror {
@Override
public void putWay(Serialized.Way way) {
wayWriter.write(way);
// TODO write way members separately, then insert in order into table in close
long wayId = way.item().id();
var nodes = way.item().nodes();
LongSet written = new LongHashSet();
for (int i = 0; i < nodes.size(); i++) {
long id = nodes.get(i);
if (written.add(id)) {
wayMemberWriter.write(new ParentChild(nodes.get(i), way.item().id()));
nodeToWay.put(id, wayId);
}
}
}
@ -396,7 +411,44 @@ public class SqliteOsmMirror implements OsmMirror {
@Override
public void close() throws IOException {
// TODO sort way members, then insert in order into wayMembers
LongArrayList values = new LongArrayList();
var iter = nodeToWay.iterator();
var counter = new AtomicLong(0);
var worker = new Worker("writer", stats, 1, () -> {
long lastKey = -1;
LOGGER.info("Inserting {} sorted way members...", nodeToWay.count());
var start = Timer.start();
while (iter.hasNext()) {
var pair = iter.next();
long key = pair.a();
if (key != lastKey) {
for (int i = 0; i < values.size(); i++) {
wayMemberWriter.write(new ParentChild(lastKey, values.get(i)));
counter.incrementAndGet();
}
values.clear();
lastKey = key;
}
values.add(pair.b());
}
if (!values.isEmpty()) {
for (int i = 0; i < values.size(); i++) {
wayMemberWriter.write(new ParentChild(lastKey, values.get(i)));
counter.incrementAndGet();
}
}
LOGGER.info("Inserted sorted way members in {}", start.stop());
});
ProgressLoggers loggers = ProgressLoggers.create()
.addRatePercentCounter("way_members", nodeToWay.count(), counter, true)
.addFileSize(() -> FileUtils.size(path))
.newLine()
.addThreadPoolStats("writer", worker)
.newLine()
.addProcessStats();
worker.awaitAndLog(loggers, Duration.ofSeconds(10));
nodeWriter.close();
wayWriter.close();
wayMemberWriter.close();

Wyświetl plik

@ -0,0 +1,994 @@
building
yes
highway
surface
natural
source
waterway
oneway
building:levels
power
service
access
addr:country
wall
tiger:reviewed
tiger:cfcc
lanes
landuse
source:date
layer
lit
barrier
type
bridge
amenity
created_by
leaf_type
intermittent
addr:state
source:geometry
footway
maxspeed
foot
bicycle
roof:shape
bus
public_transport
addr:housenumber
tunnel
crossing
railway
leaf_cycle
tracktype
import
building:part
leisure
building:units
parking
entrance
roof:levels
boundary
place
gauge
tiger:name_type
emergency
tiger:separated
area
water
attribution
building:flats
generator:source
NHD:RESOLUTION
yh:TOTYUMONO
generator:method
yh:STRUCTURE
generator:type
yh:TYPE
wheelchair
noexit
generator:output:electricity
smoothness
source:addr
tactile_paving
addr:city
addr:TW:dataset
NHD:FType
addr:interpolation
religion
denotation
building:ruian:type
material
electrified
sidewalk
crossing_ref
fee
yh:WIDTH
operator
ford
tourism
segregated
height
backrest
usage
horse
direction
cycleway:both
motor_vehicle
frequency
wetland
source_ref
gnis:ftype
shelter
fire_hydrant:type
capacity
junction
crossing:island
admin_level
noname
yh:WIDTH_RANK
covered
shop
bench
roof:material
location
restriction
NHD:FCode
mml:class
cables
addr:province:en
addr:province
man_made
LINZ:source_version
lanes:backward
tiger:source
LINZ:dataset
building:use
lane_markings
kerb
sport
traffic_signals
golf
operator:wikidata
lanes:forward
incline
source:conscriptionnumber
level
residential
information
NHD:FTYPE
width
building:material
source:maxspeed
voltage
opening_hours
lamp_type
sac_scale
source:tracer
hgv
toll
passenger_lines
crop
support
source:geometry:date
roof:orientation
it:fvg:ctrn:revision
water_source
design
boat
gnis:fcode
bin
tower:type
it:fvg:ctrn:code
seasonal
nysgissam:review
historic
addr:street:type
takeaway
recycling_type
meadow
motorcycle
addr:street
traffic_calming
at_bev:addr_date
note:ja
hiking
note
addr:inclusion
landcover
fire_hydrant:diameter
geobase:acquisitionTechnique
mtb:scale
tiger:name_direction_prefix
cycleway
operator:type
CLC:year
living_street
lamp_mount
designation
traffic_signals:direction
parking_space
start_date
raba:id
aeroway
NHS
denomination
house
public_transport:version
bicycle_parking
fire_hydrant:position
route
motorcar
internet_access
building:levels:underground
substation
naptan:verified
cycleway:right
roof:colour
ref
handrail
embankment
construction
office
traffic_sign
supervised
building:condition
healthcare
trail_visibility
ramp
nhd:ftype
train
network:type
source:data
source:imagery
source:shape
shoulder
indoor
building:colour
structure
circuits
addr:street:prefix
lcn
maxspeed:type
building:fireproof
project:eurosha_2012
LINZ2OSM:source_version
LINZ2OSM:layer
payment:cash
railway:etcs
atm
colour
track
asphalt
tree
BAG
stream
unclassified
1
tower
microsoft/BuildingFootprints
Bing
driveway
private
path
DE
no
cadastre-dgi-fr source : Direction Générale des Impôts - Cadastre. Mise à jour : 2010
pole
A41
unpaved
wood
2
cadastre-dgi-fr source : Direction Générale des Impôts - Cadastre. Mise à jour : 2011
farmland
cadastre-dgi-fr source : Direction Générale des Impôts - Cadastre. Mise à jour : 2012
tertiary
2014-03-24
bing
NRCan-CanVec-10.0
maxar
fence
detached
garage
multipolygon
parking_aisle
apartments
grass
forest
digitalglobe
secondary
JOSM
broadleaved
NY
gate
YahooJapan/ALPSMAP
www.geoportal.gov.pl
-1
cadastre-dgi-fr source : Direction Générale des Finances Publiques - Cadastre. Mise à jour : 2014
paved
Kartverket N50
scrub
Maxar_private;Ecopia.AI
cuzk:ruian
50
gabled
street_lamp
ditch
primary
bus_stop
30
cadastre-dgi-fr source : Direction Générale des Finances Publiques - Cadastre. Mise à jour : 2013
cadastre-dgi-fr source : Direction Générale des Finances Publiques - Cadastre. Mise à jour : 2015
NHD
platform
shed
2014-02-11
ground
hedge
3
4
culvert
5
6
concrete
Danmarks Adresseregister
DK
EG
NRCan-CanVec-7.0
paving_stones
marked
rail
survey
7
US
8
deciduous
grade3
kapor2
hut
10
turning_circle
9
AT
pitch
industrial
cadastre-dgi-fr source : Direction Générale des Impôts - Cadastre. Mise à jour : 2013
NRCan-CanVec-8.0
pmsp
generator
11
CZ
12
grade2
farm_auxiliary
gravel
swimming_pool
cadastre-dgi-fr source : Direction Générale des Finances Publiques - Cadastre. Mise à jour : 2017
administrative
hamlet
1435
extraction vectorielle v1 cadastre-dgi-fr source : Direction Générale des Impôts - Cadas. Mise à jour : 2010
14
Rd
fire_hydrant
grade4
cadastre-dgi-fr source : Direction Générale des Finances Publiques - Cadastre. Mise à jour : 2016
river
15
unmarked
pond
13
2014-05-07
locality
cadastre-dgi-fr source : Direction Générale des Impôts - Cadastre. Mise à jour : 2009
roof
uncontrolled
16
designated
PGS
solar
High
trunk
供用中
CanVec 6.0 - NRCan
alley
photovoltaic
village
St
地上
17
drain
A74
steps
solar_photovoltaic_panel
18
その他一般道
TW
terrace
3dShapes
CA
grassland
grade5
20
cadastre-dgi-fr source : Direction Générale des Impôts - Cadastre ; mise à jour : 2009
stop
19
place_of_worship
RU
FL
tree_row
restaurant
dirt
21
orchard
customers
22
school
grade1
Ayuntamiento de Tijuana
good
emuia.gugik.gov.pl
GSImaps/std
farmyard
臺中市
137998
Regione Emilia Romagna
stop_position
23
StreamRiver
24
odd
even
flat
christian
coastline
motorway
park
urban
commercial
SK
semidetached_house
contact_line
25
both
CT
40
zebra
RABA-KGZ
26
Dr
3.0m〜5.5m
Enedis
cidi
cuzk:km
garden
3.5
27
60
main
28
forward
canal
level_crossing
Akros
GEOBASES 2015
Ave
gugik.gov.pl
29
peak
lake
retail
switch
31
give_way
cadastre-dgi-fr source : Direction Générale des Finances Publiques - Cadastre. Mise à jour : 2018
budovy201004
bog
32
http://wiki.osm.org/wiki/GSI_KIBAN
evergreen
garages
bare_rock
AZ
line
compacted
excellent
80
backward
motorway_link
underground
1.5m〜3.0m
33
playground
nls.fi database import
roundabout
34
Japan GSI ortho Imagery
minor_line
MD
needleleaved
permissive
CH
IT
35
pillar
pedestrian
EE
36
SE
esri/LINZ_NZ_Buildings
cliff
isolated_dwelling
barn
esri
GSImaps/ort
GSI/KIBAN 2500; NARO
waste_basket
outbuilding
convenience
37
cadastre-dgi-fr source : Direction Générale des Finances Publiques - Cadastre. Mise à jour : 2019
0
KSJ2
38
roof_tiles
30 mph
SanGIS Addresses Public Domain (http://www.sangis.org/)
http://wiki.osm.org/wiki/Attribution#LINZ
http://www.linz.govt.nz/topography/topo-maps/
no_u_turn
46003
geoimage.at
GPS
IBGE
ME
GSI/KIBAN 2500
cabin
39
vineyard
36311
hipped
bollard
Cairo
القاهرة
sand
greenhouse
41
storage_tank
Plan polyvalent Douala
70
lpis
2012-06-06
pier
tiger_import_dch_v0.6_20070829
42
mainland
Cavite
NC
Ln
tiger_import_dch_v0.6_20070813
lowered
intermediate
LV
reservoir
yard
Yahoo
43
44
GB
46006
Amsterdam
soccer
KY
Geobase_Import_2009
signal
bunker
45
25 mph
Q3587594
100
up
46
tiger_import_dch_v0.6_20070830
Maa-amet 2019
trunk_link
cafe
fast_food
fuel
spur
tiger_import_dch_v0.6_20070809
rural
guidepost
47
VA
right
48
metal
LakePond
cemetery
ArcGIS Exporter
bad
tee
BE
heath
staircase
Batangas
mixed
cadastre-dgi-fr source : Direction Générale des Impôts - Cadastre ; mise à jour : 2008
islet
tennis
49
cadastre-dgi-fr source : Direction Générale des Finances Publiques - Cadastre. Mise à jour : 2020
EMUiA (emuia.gugik.gov.pl)
brick
2013-11-26
51
Regione Liguria
sign
25000
plot
supermarket
52
24/7
lift_gate
CO
electric
Berlin
associatedStreet
retaining_wall
no_left_turn
primary_link
neighbourhood
Maxar Premium Imagery
39004
scanaerial
53
55
54
Ct
7.0
destination
ORT10LT
NV NMD2018
AR
PA
GeoBase®
rice
CANVEC
hotel
الجيزه
Giza
Brisbane
2018-12-14
Instituto Geográfico Nacional
Brisbane City Council
2015-01-01..2019-11-05
recycling
cadastre-dgi-fr source : Direction Générale des Finances Publiques - Cadastre. Mise à jour : 2021
stop_area
toilets
bank
swamp
cement_block
56
BDOrtho IGN
wind
across
http://nlftp.mlit.go.jp/ksj/jpgis/datalist/KsjTmplt-W05.html
1A0
cadastre-dgi-fr source : Direction Générale des Impôts - Cadastre. Mise à jour : 2014
survey_point
pharmacy
down
only_straight_on
IN
grape
90
h-frame
57
avenue
steel
City of Ottawa
church
post_box
abandoned
58
clothes
farm
static_caravan
M-NCPPC
communication
Kartverket
WI
4AED
wind_turbine
59
limited
20 mph
warehouse
Akros Global Health
Natural Resources Canada
sobzeros
sett
no matching street nearby
61
201011
allotments
Rotterdam
siding
secondary_link
memorial
2014-05-20
Avenue
TT
NHD_import_v0.2_20090219152958
container
street_side
62
bungalow
agricultural
1.5m未満
FR
45 mph
Hauptstraße
bump
San Jose
cutline
fine_gravel
2017-10-01
63
国土数値情報(河川データ)平成18年国土交通省
National-Land Numerical Information (River) 2006, MLIT Japan
Miami
65
64
actual
trees
Maa-amet 2013
Louisville
's-Gravenhage
ES
Bing 2014
Regione Toscana
N
E
WA
1989-07-01
lane
public
building_passage
2006
W
cadastre-dgi-fr source : Direction Générale des Impôts - Cadastre. Mise à jour : 2020
66
esri_USDOT_Arizona
Indianapolis
S
marsh
hairdresser
cadastre-dgi-fr source : Direction Générale des Finances Publiques - Cadastre. Mise à jour : 2022
bent_mast
board
kindergarten
St. Louis County GIS Service Center (https://data.stlouisco.com/pages/open-data 2019-08)
EMUiA (emuia.geoportal.gov.pl)
public_footpath
PL
67
46007
BRG_MAPATHON_2020
drinking_water
scree
hump
68
OS_OpenData_StreetView
35 mph
use_sidepath
normal
1970
GURS
1300
pipeline
taxiway
Portland
picnic_table
San Diego
catholic
branch
terraced
69
muslim
71
PGS(could be inacurately)
A31
horizontal_axis
Rochester
40 mph
Fortaleza
stands
green
manhole
RTE
SC
ID
wlan
القليوبية
Al Qalyubia
72
cadastre-dgi-fr source : Direction Générale des Impôts - Cadastre. Mise à jour : 2022
1520
mast
Montevideo
Phoenix
plaster
Hamburg
separate
MGC
minor_distribution
silo
1950
75
catastro
left
along
cidi;bing
1972
Mapbox
2014-05-24
74
73
1975
basketball
grey
three-level
TX
PNOA
tertiary_link
2014-01-22
NJ
road
mountain_hiking
Ortofotomozaika SR
1960
fairway
USGS National Hydrographic Database
geoportal.gov.pl:BDOT
basin
76
2019/07/02
UY
maxar Premium
15000
1980
pyramidal
government
city_limit
www.sii.cl, Cartografía Digital SII Mapas
cadastre-dgi-fr source : Direction Générale des Impôts - Cadastre. Mise à jour : 2021
car_repair
dry_stone
Geoportal Berlin / Hauskoordinaten
rock
sports_centre
Toronto
OR
http://www.bakersfieldcity.us/gis/downloads/gis_spatial_data.htm
artwork
77
almien_coastlines
Bakersfield_GIS
attraction
buffer_stop
vending_machine
esri/Orange_County_CA_Buildings_v2
GSI/KIBAN 2500;NARO
2017-01-13
1973
20000
Buffalo
الاسكندرية
Alexandria
Seattle
esri/Orange_County_CA_Buildings_v3
company
quarry
Baltimore
78
esri_USDOT_Arizona;microsoft/BuildingFootprints
bakery
outdoor
hunting_stand
©IGN 2010 dans le cadre de la cartographie réglementaire
naptan_import
motorway_junction
DigitalGlobe Premium Imagery
A51
HiRes aerial imagery
basemap.at
Maxar Premium Imagery (Beta)
bar
1965
1500
79
dam
very_bad
viewpoint
KZ
Direction Générale des Finances Publiques - cadastre
101
81
works
1100
Regione_del_Veneto_LR28_16.7.1976_Formazione_CTR_auth_39164-5700-1100_23.01.2009
GeobaseNHN_Import_2009
120
node_network
1974
https://gis.ny.gov/gisdata
ESRI Sat & NYS Sat Imagery
Modified MS Footprints https://cugir.library.cornell.edu
1995-07-01
tiger_import_dch_v0.6_20070810
LINZ
82
esri/Indianapolis_IN_Buildings
hospital
Office of Geographic and Environmental Information (MassGIS)
Union européenne - SOeS, CORINE Land Cover, 2006.
waste_disposal
CL
white
lattice
South
85
red
cadastre-dgi-fr source : Direction Générale des Impôts - Cadastre. Mise à jour : 2019
CanVec_Import_2009
post_office
Maps4BW, LGL, www.lgl-bw.de
Warwicks CC Aerial Imagery 2013
London
akros
84
petroleum_well
83
straight_mast
1968
1955
none
Nova Scotia
yahoo
улица Ленина
1978
Birmingham
http://sdi.opolskie.pl/mapa_bazowa/request.aspx
1990
Auckland Council
86
1988
16.7
surveillance
2015-04-13
ac_buildings
http://www.aucklandcouncil.govt.nz/EN/ratesbuildingproperty/propertyinformation/GIS_maps/Pages/opendata.aspx
dibavod
drive-through
102
1994-07-01
1985
utility_pole
1971
GNS
pub
stone
Советская улица
UZ
88
ruins
King County GIS;data.seattle.gov
4ABA
1976
milestone
87
1969
civic
Wien
Kreis_Viersen_Katasteramt_2012_06
IL
55 mph
no_right_turn
table
1987
water_well
1989
SNCF Réseau
2014-05-30
5.5m〜13.0m
brown
1986
105
Philadelphia
GA
site
clinic
3000

Wyświetl plik

@ -0,0 +1,89 @@
package com.onthegomap.planetiler.osmmirror;
import static org.junit.jupiter.api.Assertions.assertEquals;
import com.onthegomap.planetiler.stats.Stats;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.List;
import java.util.stream.StreamSupport;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
class LongLongSorterTest {
@TempDir
Path temp;
LongLongSorter fixture;
@BeforeEach
void before() {
fixture = new LongLongSorter.DiskBacked(temp, Stats.inMemory(), 2);
}
@Test
void test1() {
fixture.put(1, 1);
assertEquals(
List.of(
new LongLongSorter.Result(1, 1)
),
StreamSupport.stream(fixture.spliterator(), false).toList()
);
}
@Test
void test() {
fixture.put(1, 2);
fixture.put(1, 1);
assertEquals(
List.of(
new LongLongSorter.Result(1, 1),
new LongLongSorter.Result(1, 2)
),
StreamSupport.stream(fixture.spliterator(), false).toList()
);
}
@Test
void test2() {
fixture.put(1, 2);
fixture.put(1, 3);
fixture.put(1, 1);
assertEquals(
List.of(
new LongLongSorter.Result(1, 1),
new LongLongSorter.Result(1, 2),
new LongLongSorter.Result(1, 3)
),
StreamSupport.stream(fixture.spliterator(), false).toList()
);
}
@Test
void test3() {
assertEquals(
List.of(),
StreamSupport.stream(fixture.spliterator(), false).toList()
);
}
@Test
void testBig() {
List<LongLongSorter.Result> expected = new ArrayList<>();
for (long a = 10; a >= 0; a--) {
for (long b = 10; b >= 0; b--) {
fixture.put(a, b);
}
}
for (long a = 0; a <= 10; a++) {
for (long b = 0; b <= 10; b++) {
expected.add(new LongLongSorter.Result(a, b));
}
}
assertEquals(
expected,
StreamSupport.stream(fixture.spliterator(), false).toList()
);
}
}

Wyświetl plik

@ -41,8 +41,9 @@ abstract class OsmMirrorTest {
@Test
void testInsertWay() throws IOException {
var way = new OsmElement.Way(2, Map.of("key", "value"), LongArrayList.from(1), infoVersion(0));
var way2 = new OsmElement.Way(3, Map.of("key", "value"), LongArrayList.from(1, 1), infoVersion(0));
var way = new OsmElement.Way(2, Map.of("building", "yes"), LongArrayList.from(1), infoVersion(0));
var way2 = new OsmElement.Way(3, Map.of("random random random key??!?!", "not a normal value!!!"),
LongArrayList.from(1, 1), infoVersion(0));
try (var writer = fixture.newBulkWriter()) {
writer.putWay(new Serialized.Way(way, serializeId));
writer.putWay(new Serialized.Way(way2, serializeId));
@ -94,6 +95,17 @@ abstract class OsmMirrorTest {
serializeId = false;
}
}
static class SqliteFileTest extends OsmMirrorTest {
@TempDir
Path path;
@BeforeEach
void setup() {
fixture = OsmMirror.newSqliteWrite(path.resolve("test.db"), 2);
serializeId = false;
}
}
static class MapdbTest extends OsmMirrorTest {
@BeforeEach