kopia lustrzana https://github.com/onthegomap/planetiler
Parallel temp feature writes using `--write-threads` argument (#213)
rodzic
3589fc61f3
commit
726e6d0107
|
@ -72,14 +72,14 @@ jobs:
|
|||
|
||||
- name: 'Run branch'
|
||||
run: |
|
||||
rm -f data/out.mbtiles
|
||||
rm -rf data/out.mbtiles data/tmp
|
||||
cp branch/planetiler-dist/target/*with-deps.jar run.jar
|
||||
java -Xms${{ env.RAM }} -Xmx${{ env.RAM }} -jar run.jar --area="${{ env.AREA }}" "${{ env.BOUNDS_ARG }}" --mbtiles=data/out.mbtiles 2>&1 | tee log
|
||||
ls -alh run.jar | tee -a log
|
||||
cat log | strip-ansi > build-info/branchlogs.txt
|
||||
- name: 'Run base'
|
||||
run: |
|
||||
rm -f data/out.mbtiles
|
||||
rm -rf data/out.mbtiles data/tmp
|
||||
cp base/planetiler-dist/target/*with-deps.jar run.jar
|
||||
java -Xms${{ env.RAM }} -Xmx${{ env.RAM }} -jar run.jar --area="${{ env.AREA }}" "${{ env.BOUNDS_ARG }}" --mbtiles=data/out.mbtiles 2>&1 | tee log
|
||||
ls -alh run.jar | tee -a log
|
||||
|
|
|
@ -39,9 +39,10 @@ public class BenchmarkExternalMergeSort {
|
|||
var config = PlanetilerConfig.defaults();
|
||||
try {
|
||||
List<Results> results = new ArrayList<>();
|
||||
for (int limit : List.of(500_000_000, 2_000_000_000)) {
|
||||
results.add(run(path, number, limit, false, true, true, config));
|
||||
results.add(run(path, number, limit, true, true, true, config));
|
||||
int limit = 2_000_000_000;
|
||||
for (int writers : List.of(1, 2, 4)) {
|
||||
results.add(run(path, writers, number, limit, false, true, true, config));
|
||||
results.add(run(path, writers, number, limit, true, true, true, config));
|
||||
}
|
||||
for (var result : results) {
|
||||
System.err.println(result);
|
||||
|
@ -51,17 +52,18 @@ public class BenchmarkExternalMergeSort {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
private record Results(
|
||||
String write, String read, String sort,
|
||||
int chunks, long items, int chunkSizeLimit, boolean gzip, boolean mmap, boolean parallelSort,
|
||||
int chunks,
|
||||
int writeWorkers, int readWorkers,
|
||||
long items, int chunkSizeLimit, boolean gzip, boolean mmap, boolean parallelSort,
|
||||
boolean madvise
|
||||
) {}
|
||||
|
||||
private static Results run(Path tmpDir, long items, int chunkSizeLimit, boolean mmap, boolean parallelSort,
|
||||
private static Results run(Path tmpDir, int writeWorkers, long items, int chunkSizeLimit, boolean mmap,
|
||||
boolean parallelSort,
|
||||
boolean madvise, PlanetilerConfig config) {
|
||||
boolean gzip = false;
|
||||
int writeWorkers = 1;
|
||||
int sortWorkers = Runtime.getRuntime().availableProcessors();
|
||||
int readWorkers = 1;
|
||||
FileUtils.delete(tmpDir);
|
||||
|
@ -86,6 +88,8 @@ public class BenchmarkExternalMergeSort {
|
|||
FORMAT.numeric(items * NANOSECONDS_PER_SECOND / readTimer.elapsed().wall().toNanos()) + "/s",
|
||||
FORMAT.duration(sortTimer.elapsed().wall()),
|
||||
sorter.chunks(),
|
||||
writeWorkers,
|
||||
readWorkers,
|
||||
items,
|
||||
chunkSizeLimit,
|
||||
gzip,
|
||||
|
@ -116,12 +120,14 @@ public class BenchmarkExternalMergeSort {
|
|||
private static void doWrites(int writeWorkers, long items, ExternalMergeSort sorter) {
|
||||
var counters = Counter.newMultiThreadCounter();
|
||||
var writer = new Worker("write", Stats.inMemory(), writeWorkers, () -> {
|
||||
var counter = counters.counterForThread();
|
||||
var random = ThreadLocalRandom.current();
|
||||
long toWrite = items / writeWorkers;
|
||||
for (long i = 0; i < toWrite; i++) {
|
||||
sorter.add(new SortableFeature(random.nextLong(), TEST_DATA));
|
||||
counter.inc();
|
||||
try (var writerForThread = sorter.writerForThread()) {
|
||||
var counter = counters.counterForThread();
|
||||
var random = ThreadLocalRandom.current();
|
||||
long toWrite = items / writeWorkers;
|
||||
for (long i = 0; i < toWrite; i++) {
|
||||
writerForThread.accept(new SortableFeature(random.nextLong(), TEST_DATA));
|
||||
counter.inc();
|
||||
}
|
||||
}
|
||||
});
|
||||
ProgressLoggers loggers = ProgressLoggers.create()
|
||||
|
|
|
@ -7,7 +7,9 @@ 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.BinPack;
|
||||
import com.onthegomap.planetiler.util.ByteBufferUtil;
|
||||
import com.onthegomap.planetiler.util.CloseableConusmer;
|
||||
import com.onthegomap.planetiler.util.FileUtils;
|
||||
import com.onthegomap.planetiler.worker.WorkerPipeline;
|
||||
import java.io.BufferedInputStream;
|
||||
|
@ -26,14 +28,17 @@ import java.nio.file.Files;
|
|||
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.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.NoSuchElementException;
|
||||
import java.util.PriorityQueue;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.concurrent.Semaphore;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.function.Supplier;
|
||||
import java.util.zip.Deflater;
|
||||
|
@ -63,7 +68,8 @@ class ExternalMergeSort implements FeatureSort {
|
|||
private final int chunkSizeLimit;
|
||||
private final int workers;
|
||||
private final AtomicLong features = new AtomicLong(0);
|
||||
private final List<Chunk> chunks = new ArrayList<>();
|
||||
private final List<Chunk> chunks = new CopyOnWriteArrayList<>();
|
||||
private final AtomicInteger chunkNum = new AtomicInteger(0);
|
||||
private final boolean gzip;
|
||||
private final PlanetilerConfig config;
|
||||
private final int readerLimit;
|
||||
|
@ -72,7 +78,6 @@ class ExternalMergeSort implements FeatureSort {
|
|||
private final boolean parallelSort;
|
||||
private final boolean madvise;
|
||||
private final AtomicBoolean madviseFailed = new AtomicBoolean(false);
|
||||
private Chunk currentChunk;
|
||||
private volatile boolean sorted = false;
|
||||
|
||||
ExternalMergeSort(Path tempDir, PlanetilerConfig config, Stats stats) {
|
||||
|
@ -118,7 +123,6 @@ class ExternalMergeSort implements FeatureSort {
|
|||
try {
|
||||
FileUtils.deleteDirectory(dir);
|
||||
Files.createDirectories(dir);
|
||||
newChunk();
|
||||
} catch (IOException e) {
|
||||
throw new UncheckedIOException(e);
|
||||
}
|
||||
|
@ -134,17 +138,8 @@ class ExternalMergeSort implements FeatureSort {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void add(SortableFeature item) {
|
||||
try {
|
||||
assert !sorted;
|
||||
features.incrementAndGet();
|
||||
currentChunk.add(item);
|
||||
if (currentChunk.bytesInMemory > chunkSizeLimit) {
|
||||
newChunk();
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new UncheckedIOException(e);
|
||||
}
|
||||
public CloseableConusmer<SortableFeature> writerForThread() {
|
||||
return new ThreadLocalWriter();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -160,9 +155,9 @@ class ExternalMergeSort implements FeatureSort {
|
|||
@Override
|
||||
public void sort() {
|
||||
assert !sorted;
|
||||
if (currentChunk != null) {
|
||||
for (var chunk : chunks) {
|
||||
try {
|
||||
currentChunk.close();
|
||||
chunk.close();
|
||||
} catch (IOException e) {
|
||||
// ok
|
||||
}
|
||||
|
@ -175,12 +170,32 @@ class ExternalMergeSort implements FeatureSort {
|
|||
AtomicLong sorting = new AtomicLong(0);
|
||||
AtomicLong doneCounter = new AtomicLong(0);
|
||||
|
||||
// we may end up with many small chunks because each thread-local writer starts a new one
|
||||
// so group together smaller chunks that can be sorted together in-memory to minimize the
|
||||
// number of chunks that the reader needs to deal with
|
||||
List<List<ExternalMergeSort.Chunk>> groups = BinPack.pack(
|
||||
chunks,
|
||||
chunkSizeLimit,
|
||||
chunk -> chunk.bytesInMemory
|
||||
);
|
||||
|
||||
LOGGER.info("Grouped {} chunks into {}", chunks.size(), groups.size());
|
||||
|
||||
var pipeline = WorkerPipeline.start("sort", stats)
|
||||
.readFromTiny("item_queue", chunks)
|
||||
.sinkToConsumer("worker", workers, chunk -> {
|
||||
.readFromTiny("item_queue", groups)
|
||||
.sinkToConsumer("worker", workers, group -> {
|
||||
try {
|
||||
readSemaphore.acquire();
|
||||
var toSort = time(reading, chunk::readAll);
|
||||
var chunk = group.get(0);
|
||||
var others = group.stream().skip(1).toList();
|
||||
var toSort = time(reading, () -> {
|
||||
// merge all chunks into first one, and remove the others
|
||||
var result = chunk.readAllAndMergeIn(others);
|
||||
for (var other : others) {
|
||||
other.remove();
|
||||
}
|
||||
return result;
|
||||
});
|
||||
readSemaphore.release();
|
||||
|
||||
time(sorting, toSort::sort);
|
||||
|
@ -223,6 +238,10 @@ class ExternalMergeSort implements FeatureSort {
|
|||
public Iterator<SortableFeature> iterator() {
|
||||
assert sorted;
|
||||
|
||||
if (chunks.isEmpty()) {
|
||||
return Collections.emptyIterator();
|
||||
}
|
||||
|
||||
// k-way merge to interleave all the sorted chunks
|
||||
PriorityQueue<Reader<?>> queue = new PriorityQueue<>(chunks.size());
|
||||
for (Chunk chunk : chunks) {
|
||||
|
@ -250,15 +269,6 @@ class ExternalMergeSort implements FeatureSort {
|
|||
};
|
||||
}
|
||||
|
||||
private void newChunk() throws IOException {
|
||||
Path chunkPath = dir.resolve("chunk" + (chunks.size() + 1));
|
||||
chunkPath.toFile().deleteOnExit();
|
||||
if (currentChunk != null) {
|
||||
currentChunk.close();
|
||||
}
|
||||
chunks.add(currentChunk = new Chunk(chunkPath));
|
||||
}
|
||||
|
||||
public int chunks() {
|
||||
return chunks.size();
|
||||
}
|
||||
|
@ -400,6 +410,50 @@ class ExternalMergeSort implements FeatureSort {
|
|||
abstract SortableFeature readNextFeature();
|
||||
}
|
||||
|
||||
/** Writer that a single thread can use to write features independent of writers used in other threads. */
|
||||
@NotThreadSafe
|
||||
private class ThreadLocalWriter implements CloseableConusmer<SortableFeature> {
|
||||
private Chunk currentChunk;
|
||||
|
||||
private ThreadLocalWriter() {
|
||||
try {
|
||||
newChunk();
|
||||
} catch (IOException e) {
|
||||
throw new UncheckedIOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void accept(SortableFeature item) {
|
||||
assert !sorted;
|
||||
try {
|
||||
features.incrementAndGet();
|
||||
currentChunk.add(item);
|
||||
if (currentChunk.bytesInMemory > chunkSizeLimit) {
|
||||
newChunk();
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new UncheckedIOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private void newChunk() throws IOException {
|
||||
Path chunkPath = dir.resolve("chunk" + chunkNum.incrementAndGet());
|
||||
FileUtils.deleteOnExit(chunkPath);
|
||||
if (currentChunk != null) {
|
||||
currentChunk.close();
|
||||
}
|
||||
chunks.add(currentChunk = new Chunk(chunkPath));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (currentChunk != null) {
|
||||
currentChunk.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** Write features to the chunk file through a memory-mapped file. */
|
||||
private class WriterMmap implements Writer {
|
||||
private final FileChannel channel;
|
||||
|
@ -467,19 +521,34 @@ class ExternalMergeSort implements FeatureSort {
|
|||
itemCount++;
|
||||
}
|
||||
|
||||
private SortableChunk readAll() {
|
||||
try (var iterator = newReader()) {
|
||||
SortableFeature[] featuresToSort = new SortableFeature[itemCount];
|
||||
int i = 0;
|
||||
while (iterator.hasNext()) {
|
||||
featuresToSort[i] = iterator.next();
|
||||
i++;
|
||||
private SortableChunk readAllAndMergeIn(Collection<Chunk> others) {
|
||||
// first, grow this chunk
|
||||
int newItems = itemCount;
|
||||
int newBytes = bytesInMemory;
|
||||
for (var other : others) {
|
||||
if (Integer.MAX_VALUE - newItems < other.itemCount) {
|
||||
throw new IllegalStateException("Too many items in merged chunk: " + itemCount + "+" +
|
||||
others.stream().map(c -> c.itemCount).toList());
|
||||
}
|
||||
if (i != itemCount) {
|
||||
throw new IllegalStateException("Expected " + itemCount + " features in " + path + " got " + i);
|
||||
if (Integer.MAX_VALUE - newBytes < other.bytesInMemory) {
|
||||
throw new IllegalStateException("Too big merged chunk: " + bytesInMemory + "+" +
|
||||
others.stream().map(c -> c.bytesInMemory).toList());
|
||||
}
|
||||
return new SortableChunk(featuresToSort);
|
||||
newItems += other.itemCount;
|
||||
newBytes += other.bytesInMemory;
|
||||
}
|
||||
// then read items from all chunks into memory
|
||||
SortableChunk result = new SortableChunk(newItems);
|
||||
result.readAll(this);
|
||||
itemCount = newItems;
|
||||
bytesInMemory = newBytes;
|
||||
for (var other : others) {
|
||||
result.readAll(other);
|
||||
}
|
||||
if (result.i != itemCount) {
|
||||
throw new IllegalStateException("Expected " + itemCount + " features in " + path + " got " + result.i);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
private Writer newWriter(Path path) {
|
||||
|
@ -495,15 +564,21 @@ class ExternalMergeSort implements FeatureSort {
|
|||
writer.close();
|
||||
}
|
||||
|
||||
public void remove() {
|
||||
chunks.remove(this);
|
||||
FileUtils.delete(path);
|
||||
}
|
||||
|
||||
/**
|
||||
* A container for all features in a chunk read into memory for sorting.
|
||||
*/
|
||||
private class SortableChunk {
|
||||
|
||||
private SortableFeature[] featuresToSort;
|
||||
private int i = 0;
|
||||
|
||||
private SortableChunk(SortableFeature[] featuresToSort) {
|
||||
this.featuresToSort = featuresToSort;
|
||||
private SortableChunk(int itemCount) {
|
||||
this.featuresToSort = new SortableFeature[itemCount];
|
||||
}
|
||||
|
||||
public SortableChunk sort() {
|
||||
|
@ -526,6 +601,14 @@ class ExternalMergeSort implements FeatureSort {
|
|||
throw new UncheckedIOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private void readAll(Chunk chunk) {
|
||||
try (var iterator = chunk.newReader()) {
|
||||
while (iterator.hasNext()) {
|
||||
featuresToSort[i++] = iterator.next();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -9,6 +9,7 @@ import com.onthegomap.planetiler.geo.GeometryType;
|
|||
import com.onthegomap.planetiler.geo.TileCoord;
|
||||
import com.onthegomap.planetiler.render.RenderedFeature;
|
||||
import com.onthegomap.planetiler.stats.Stats;
|
||||
import com.onthegomap.planetiler.util.CloseableConusmer;
|
||||
import com.onthegomap.planetiler.util.CommonStringEncoder;
|
||||
import com.onthegomap.planetiler.util.DiskBacked;
|
||||
import com.onthegomap.planetiler.util.LayerStats;
|
||||
|
@ -44,8 +45,7 @@ import org.slf4j.LoggerFactory;
|
|||
* supported (see {@link CommonStringEncoder})
|
||||
*/
|
||||
@NotThreadSafe
|
||||
public final class FeatureGroup implements Consumer<SortableFeature>, Iterable<FeatureGroup.TileFeatures>,
|
||||
DiskBacked {
|
||||
public final class FeatureGroup implements Iterable<FeatureGroup.TileFeatures>, DiskBacked {
|
||||
|
||||
public static final int SORT_KEY_BITS = 23;
|
||||
public static final int SORT_KEY_MAX = (1 << (SORT_KEY_BITS - 1)) - 1;
|
||||
|
@ -246,10 +246,9 @@ public final class FeatureGroup implements Consumer<SortableFeature>, Iterable<F
|
|||
return (byte) ((geometry.geomType().asByte() & 0xff) | (geometry.scale() << 3));
|
||||
}
|
||||
|
||||
/** Writes a serialized binary feature to intermediate storage. */
|
||||
@Override
|
||||
public void accept(SortableFeature entry) {
|
||||
sorter.add(entry);
|
||||
/** Returns a new feature writer that can be used for a single thread. */
|
||||
public CloseableConusmer<SortableFeature> writerForThread() {
|
||||
return sorter.writerForThread();
|
||||
}
|
||||
|
||||
private volatile boolean prepared = false;
|
||||
|
|
|
@ -1,8 +1,10 @@
|
|||
package com.onthegomap.planetiler.collection;
|
||||
|
||||
import com.onthegomap.planetiler.util.CloseableConusmer;
|
||||
import com.onthegomap.planetiler.util.DiskBacked;
|
||||
import com.onthegomap.planetiler.util.MemoryEstimator;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
@ -26,7 +28,7 @@ interface FeatureSort extends Iterable<SortableFeature>, DiskBacked, MemoryEstim
|
|||
|
||||
/** Returns a feature sorter that sorts all features in memory. Suitable for toy examples (unit tests). */
|
||||
static FeatureSort newInMemory() {
|
||||
List<SortableFeature> list = new ArrayList<>();
|
||||
List<SortableFeature> list = Collections.synchronizedList(new ArrayList<>());
|
||||
return new FeatureSort() {
|
||||
@Override
|
||||
public void sort() {
|
||||
|
@ -39,8 +41,8 @@ interface FeatureSort extends Iterable<SortableFeature>, DiskBacked, MemoryEstim
|
|||
}
|
||||
|
||||
@Override
|
||||
public void add(SortableFeature newEntry) {
|
||||
list.add(newEntry);
|
||||
public CloseableConusmer<SortableFeature> writerForThread() {
|
||||
return list::add;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -74,6 +76,9 @@ interface FeatureSort extends Iterable<SortableFeature>, DiskBacked, MemoryEstim
|
|||
return list;
|
||||
}
|
||||
|
||||
void add(SortableFeature newEntry);
|
||||
|
||||
/**
|
||||
* Returns a new writer that can be used to write features from a single thread independent of writers used from other
|
||||
* threads.
|
||||
*/
|
||||
CloseableConusmer<SortableFeature> writerForThread();
|
||||
}
|
||||
|
|
|
@ -12,6 +12,8 @@ public record PlanetilerConfig(
|
|||
Arguments arguments,
|
||||
Bounds bounds,
|
||||
int threads,
|
||||
int featureWriteThreads,
|
||||
int featureProcessThreads,
|
||||
Duration logInterval,
|
||||
int minzoom,
|
||||
int maxzoom,
|
||||
|
@ -75,10 +77,20 @@ public record PlanetilerConfig(
|
|||
"default storage type for temporary data, one of " + Stream.of(Storage.values()).map(
|
||||
Storage::id).toList(),
|
||||
fallbackTempStorage);
|
||||
int threads = arguments.threads();
|
||||
int featureWriteThreads =
|
||||
arguments.getInteger("write_threads", "number of threads to use when writing temp features",
|
||||
// defaults: <48 cpus=1 writer, 48-80=2 writers, 80-112=3 writers, 112-144=4 writers, ...
|
||||
Math.max(1, (threads - 16) / 32 + 1));
|
||||
int featureProcessThreads =
|
||||
arguments.getInteger("process_threads", "number of threads to use when processing input features",
|
||||
Math.max(threads < 4 ? threads : (threads - featureWriteThreads), 1));
|
||||
return new PlanetilerConfig(
|
||||
arguments,
|
||||
new Bounds(arguments.bounds("bounds", "bounds")),
|
||||
arguments.threads(),
|
||||
threads,
|
||||
featureWriteThreads,
|
||||
featureProcessThreads,
|
||||
arguments.getDuration("loginterval", "time between logs", "10s"),
|
||||
arguments.getInteger("minzoom", "minimum zoom level", MIN_MINZOOM),
|
||||
arguments.getInteger("maxzoom", "maximum zoom level (limit 14)", MAX_MAXZOOM),
|
||||
|
|
|
@ -11,6 +11,7 @@ import com.onthegomap.planetiler.stats.ProgressLoggers;
|
|||
import com.onthegomap.planetiler.stats.Stats;
|
||||
import com.onthegomap.planetiler.worker.WorkerPipeline;
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.function.Consumer;
|
||||
import org.locationtech.jts.geom.Envelope;
|
||||
|
@ -48,7 +49,8 @@ public abstract class SimpleReader implements Closeable {
|
|||
public final void process(FeatureGroup writer, PlanetilerConfig config) {
|
||||
var timer = stats.startStage(sourceName);
|
||||
long featureCount = getCount();
|
||||
int threads = config.threads();
|
||||
int writeThreads = config.featureWriteThreads();
|
||||
int processThreads = config.featureProcessThreads();
|
||||
Envelope latLonBounds = config.bounds().latLon();
|
||||
AtomicLong featuresRead = new AtomicLong(0);
|
||||
AtomicLong featuresWritten = new AtomicLong(0);
|
||||
|
@ -56,7 +58,7 @@ public abstract class SimpleReader implements Closeable {
|
|||
var pipeline = WorkerPipeline.start(sourceName, stats)
|
||||
.fromGenerator("read", read())
|
||||
.addBuffer("read_queue", 1000)
|
||||
.<SortableFeature>addWorker("process", threads, (prev, next) -> {
|
||||
.<SortableFeature>addWorker("process", processThreads, (prev, next) -> {
|
||||
var featureCollectors = new FeatureCollector.Factory(config, stats);
|
||||
try (FeatureRenderer renderer = newFeatureRenderer(writer, config, next)) {
|
||||
for (SourceFeature sourceFeature : prev) {
|
||||
|
@ -78,9 +80,13 @@ public abstract class SimpleReader implements Closeable {
|
|||
// output large batches since each input may map to many tiny output features (i.e. slicing ocean tiles)
|
||||
// which turns enqueueing into the bottleneck
|
||||
.addBuffer("write_queue", 50_000, 1_000)
|
||||
.sinkToConsumer("write", 1, item -> {
|
||||
featuresWritten.incrementAndGet();
|
||||
writer.accept(item);
|
||||
.sinkTo("write", writeThreads, prev -> {
|
||||
try (var threadLocalWriter = writer.writerForThread()) {
|
||||
for (var item : prev) {
|
||||
featuresWritten.incrementAndGet();
|
||||
threadLocalWriter.accept(item);
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
var loggers = ProgressLoggers.create()
|
||||
|
@ -95,8 +101,13 @@ public abstract class SimpleReader implements Closeable {
|
|||
pipeline.awaitAndLog(loggers, config.logInterval());
|
||||
|
||||
// hook for profile to do any post-processing after this source is read
|
||||
try (var featureRenderer = newFeatureRenderer(writer, config, writer)) {
|
||||
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);
|
||||
}
|
||||
timer.stop();
|
||||
}
|
||||
|
|
|
@ -304,8 +304,8 @@ public class OsmReader implements Closeable, MemoryEstimator.HasEstimate {
|
|||
*/
|
||||
public void pass2(FeatureGroup writer, PlanetilerConfig config) {
|
||||
var timer = stats.startStage("osm_pass2");
|
||||
int threads = config.threads();
|
||||
int processThreads = Math.max(threads < 4 ? threads : (threads - 1), 1);
|
||||
int writeThreads = config.featureWriteThreads();
|
||||
int processThreads = config.featureProcessThreads();
|
||||
Counter.MultiThreadCounter blocksProcessed = Counter.newMultiThreadCounter();
|
||||
// track relation count separately because they get enqueued onto the distributor near the end
|
||||
Counter.MultiThreadCounter relationsProcessed = Counter.newMultiThreadCounter();
|
||||
|
@ -323,7 +323,7 @@ public class OsmReader implements Closeable, MemoryEstimator.HasEstimate {
|
|||
|
||||
var pipeline = WorkerPipeline.start("osm_pass2", stats)
|
||||
.fromGenerator("read", osmBlockSource::forEachBlock)
|
||||
.addBuffer("pbf_blocks", Math.max(10, threads / 2))
|
||||
.addBuffer("pbf_blocks", Math.max(10, processThreads / 2))
|
||||
.<SortableFeature>addWorker("process", processThreads, (prev, next) -> {
|
||||
// avoid contention trying to get the thread-local counters by getting them once when thread starts
|
||||
Counter blocks = blocksProcessed.counterForThread();
|
||||
|
@ -369,7 +369,13 @@ public class OsmReader implements Closeable, MemoryEstimator.HasEstimate {
|
|||
}
|
||||
}).addBuffer("feature_queue", 50_000, 1_000)
|
||||
// FeatureGroup writes need to be single-threaded
|
||||
.sinkToConsumer("write", 1, writer);
|
||||
.sinkTo("write", writeThreads, prev -> {
|
||||
try (var writerForThread = writer.writerForThread()) {
|
||||
for (var item : prev) {
|
||||
writerForThread.accept(item);
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
var logger = ProgressLoggers.create()
|
||||
.addRatePercentCounter("nodes", pass1Phaser.nodes(), pass2Phaser::nodes, true)
|
||||
|
@ -393,7 +399,10 @@ public class OsmReader implements Closeable, MemoryEstimator.HasEstimate {
|
|||
|
||||
timer.stop();
|
||||
|
||||
try (var renderer = createFeatureRenderer(writer, config, writer)) {
|
||||
try (
|
||||
var writerForThread = writer.writerForThread();
|
||||
var renderer = createFeatureRenderer(writer, config, writerForThread)
|
||||
) {
|
||||
profile.finish(name, new FeatureCollector.Factory(config, stats), renderer);
|
||||
} catch (Exception e) {
|
||||
LOGGER.error("Error calling profile.finish", e);
|
||||
|
@ -740,7 +749,7 @@ public class OsmReader implements Closeable, MemoryEstimator.HasEstimate {
|
|||
|
||||
@Override
|
||||
protected Geometry computeWorldGeometry() throws GeometryException {
|
||||
return canBePolygon() ? polygon() : line();
|
||||
return super.canBePolygon() ? polygon() : line();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -0,0 +1,46 @@
|
|||
package com.onthegomap.planetiler.util;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.function.ToLongFunction;
|
||||
|
||||
/**
|
||||
* Implements a best-effort 1-D bin packing using the
|
||||
* <a href="https://en.wikipedia.org/wiki/First-fit-decreasing_bin_packing#Other_variants">best-fit decreasing</a>
|
||||
* algorithm.
|
||||
*/
|
||||
public class BinPack {
|
||||
private BinPack() {}
|
||||
|
||||
/**
|
||||
* Returns {@code items} grouped into an approximately minimum number of bins under {@code maxBinSize} according to
|
||||
* {@code getSize} function.
|
||||
*/
|
||||
public static <T> List<List<T>> pack(Collection<T> items, long maxBinSize, ToLongFunction<T> getSize) {
|
||||
class Bin {
|
||||
long size = 0;
|
||||
final List<T> items = new ArrayList<>();
|
||||
}
|
||||
var descendingItems = items.stream().sorted(Comparator.comparingLong(getSize).reversed()).toList();
|
||||
List<Bin> bins = new ArrayList<>();
|
||||
for (var item : descendingItems) {
|
||||
long size = getSize.applyAsLong(item);
|
||||
var bestBin = bins.stream()
|
||||
.filter(b -> maxBinSize - b.size >= size)
|
||||
// Instead of using the first bin that this element fits in, use the "fullest" bin.
|
||||
// This makes the algorithm "best-fit decreasing" instead of "first-fit decreasing"
|
||||
.max(Comparator.comparingLong(bin -> bin.size));
|
||||
Bin bin;
|
||||
if (bestBin.isPresent()) {
|
||||
bin = bestBin.get();
|
||||
} else {
|
||||
bins.add(bin = new Bin());
|
||||
}
|
||||
bin.items.add(item);
|
||||
bin.size += size;
|
||||
}
|
||||
return bins.stream().map(bin -> bin.items).toList();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,12 @@
|
|||
package com.onthegomap.planetiler.util;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.function.Consumer;
|
||||
|
||||
@FunctionalInterface
|
||||
public interface CloseableConusmer<T> extends Consumer<T>, Closeable {
|
||||
|
||||
@Override
|
||||
default void close() throws IOException {}
|
||||
}
|
|
@ -56,7 +56,7 @@ public class WorkQueue<T> implements AutoCloseable, IterableOnce<T>, Consumer<T>
|
|||
* @param stats stats to monitor this with
|
||||
*/
|
||||
public WorkQueue(String name, int capacity, int maxBatch, Stats stats) {
|
||||
this.pendingBatchesCapacity = capacity / maxBatch;
|
||||
this.pendingBatchesCapacity = Math.max(1, capacity / maxBatch);
|
||||
this.batchSize = maxBatch;
|
||||
itemQueue = new ArrayBlockingQueue<>(pendingBatchesCapacity);
|
||||
|
||||
|
|
|
@ -1579,7 +1579,11 @@ class PlanetilerTests {
|
|||
Files.copy(originalOsm, tempOsm);
|
||||
Planetiler.create(Arguments.fromArgs(
|
||||
"--tmpdir", tempDir.toString(),
|
||||
"--free-osm-after-read"
|
||||
"--free-osm-after-read",
|
||||
// ensure we exercise the multi-threaded code
|
||||
"--write-threads=2",
|
||||
"--process-threads=2",
|
||||
"--threads=4"
|
||||
))
|
||||
.setProfile(new Profile.NullProfile() {
|
||||
@Override
|
||||
|
|
|
@ -13,6 +13,7 @@ import com.onthegomap.planetiler.geo.GeometryType;
|
|||
import com.onthegomap.planetiler.geo.TileCoord;
|
||||
import com.onthegomap.planetiler.render.RenderedFeature;
|
||||
import com.onthegomap.planetiler.stats.Stats;
|
||||
import com.onthegomap.planetiler.util.CloseableConusmer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
@ -30,6 +31,7 @@ class FeatureGroupTest {
|
|||
|
||||
private final FeatureSort sorter = FeatureSort.newInMemory();
|
||||
private FeatureGroup features = new FeatureGroup(sorter, new Profile.NullProfile(), Stats.inMemory());
|
||||
private CloseableConusmer<SortableFeature> featureWriter = features.writerForThread();
|
||||
|
||||
@Test
|
||||
void testEmpty() {
|
||||
|
@ -65,7 +67,7 @@ class FeatureGroupTest {
|
|||
sortKey,
|
||||
hasGroup ? Optional.of(new RenderedFeature.Group(group, limit)) : Optional.empty()
|
||||
);
|
||||
features.accept(features.newRenderedFeatureEncoder().apply(feature));
|
||||
featureWriter.accept(features.newRenderedFeatureEncoder().apply(feature));
|
||||
}
|
||||
|
||||
private Map<Integer, Map<String, List<Feature>>> getFeatures() {
|
||||
|
@ -212,6 +214,7 @@ class FeatureGroupTest {
|
|||
return items;
|
||||
}
|
||||
}, Stats.inMemory());
|
||||
featureWriter = features.writerForThread();
|
||||
putWithGroup(
|
||||
1, "layer", Map.of("id", 3), newPoint(5, 6), 2, 1, 2
|
||||
);
|
||||
|
|
|
@ -9,6 +9,7 @@ import java.util.ArrayList;
|
|||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
import java.util.stream.Stream;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.io.TempDir;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
|
@ -21,10 +22,11 @@ class FeatureSortTest {
|
|||
@TempDir
|
||||
Path tmpDir;
|
||||
|
||||
private static SortableFeature newEntry(int i) {
|
||||
private SortableFeature newEntry(int i) {
|
||||
return new SortableFeature(Long.MIN_VALUE + i, new byte[]{(byte) i, (byte) (1 + i)});
|
||||
}
|
||||
|
||||
|
||||
private FeatureSort newSorter(int workers, int chunkSizeLimit, boolean gzip, boolean mmap) {
|
||||
return new ExternalMergeSort(tmpDir, workers, chunkSizeLimit, gzip, mmap, true, true, config,
|
||||
Stats.inMemory());
|
||||
|
@ -32,7 +34,7 @@ class FeatureSortTest {
|
|||
|
||||
@Test
|
||||
void testEmpty() {
|
||||
FeatureSort sorter = newSorter(1, 100, false, false);
|
||||
var sorter = newSorter(1, 100, false, false);
|
||||
sorter.sort();
|
||||
assertEquals(List.of(), sorter.toList());
|
||||
}
|
||||
|
@ -40,7 +42,8 @@ class FeatureSortTest {
|
|||
@Test
|
||||
void testSingle() {
|
||||
FeatureSort sorter = newSorter(1, 100, false, false);
|
||||
sorter.add(newEntry(1));
|
||||
var writer = sorter.writerForThread();
|
||||
writer.accept(newEntry(1));
|
||||
sorter.sort();
|
||||
assertEquals(List.of(newEntry(1)), sorter.toList());
|
||||
}
|
||||
|
@ -48,8 +51,9 @@ class FeatureSortTest {
|
|||
@Test
|
||||
void testTwoItemsOneChunk() {
|
||||
FeatureSort sorter = newSorter(1, 100, false, false);
|
||||
sorter.add(newEntry(2));
|
||||
sorter.add(newEntry(1));
|
||||
var writer = sorter.writerForThread();
|
||||
writer.accept(newEntry(2));
|
||||
writer.accept(newEntry(1));
|
||||
sorter.sort();
|
||||
assertEquals(List.of(newEntry(1), newEntry(2)), sorter.toList());
|
||||
}
|
||||
|
@ -57,8 +61,9 @@ class FeatureSortTest {
|
|||
@Test
|
||||
void testTwoItemsTwoChunks() {
|
||||
FeatureSort sorter = newSorter(1, 0, false, false);
|
||||
sorter.add(newEntry(2));
|
||||
sorter.add(newEntry(1));
|
||||
var writer = sorter.writerForThread();
|
||||
writer.accept(newEntry(2));
|
||||
writer.accept(newEntry(1));
|
||||
sorter.sort();
|
||||
assertEquals(List.of(newEntry(1), newEntry(2)), sorter.toList());
|
||||
}
|
||||
|
@ -66,14 +71,45 @@ class FeatureSortTest {
|
|||
@Test
|
||||
void testTwoWorkers() {
|
||||
FeatureSort sorter = newSorter(2, 0, false, false);
|
||||
sorter.add(newEntry(4));
|
||||
sorter.add(newEntry(3));
|
||||
sorter.add(newEntry(2));
|
||||
sorter.add(newEntry(1));
|
||||
var writer = sorter.writerForThread();
|
||||
writer.accept(newEntry(4));
|
||||
writer.accept(newEntry(3));
|
||||
writer.accept(newEntry(2));
|
||||
writer.accept(newEntry(1));
|
||||
sorter.sort();
|
||||
assertEquals(List.of(newEntry(1), newEntry(2), newEntry(3), newEntry(4)), sorter.toList());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testTwoWriters() {
|
||||
FeatureSort sorter = newSorter(2, 0, false, false);
|
||||
var writer1 = sorter.writerForThread();
|
||||
var writer2 = sorter.writerForThread();
|
||||
writer1.accept(newEntry(4));
|
||||
writer1.accept(newEntry(3));
|
||||
writer2.accept(newEntry(2));
|
||||
writer2.accept(newEntry(1));
|
||||
sorter.sort();
|
||||
assertEquals(List.of(newEntry(1), newEntry(2), newEntry(3), newEntry(4)), sorter.toList());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testMultipleWritersThatGetCombined() {
|
||||
FeatureSort sorter = newSorter(2, 2_000_000, false, false);
|
||||
var writer1 = sorter.writerForThread();
|
||||
var writer2 = sorter.writerForThread();
|
||||
var writer3 = sorter.writerForThread();
|
||||
writer1.accept(newEntry(4));
|
||||
writer1.accept(newEntry(3));
|
||||
writer2.accept(newEntry(2));
|
||||
writer2.accept(newEntry(1));
|
||||
writer3.accept(newEntry(5));
|
||||
writer3.accept(newEntry(6));
|
||||
sorter.sort();
|
||||
assertEquals(Stream.of(1, 2, 3, 4, 5, 6).map(this::newEntry).toList(),
|
||||
sorter.toList());
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@CsvSource({
|
||||
"false,false",
|
||||
|
@ -90,7 +126,8 @@ class FeatureSortTest {
|
|||
}
|
||||
Collections.shuffle(shuffled, new Random(0));
|
||||
FeatureSort sorter = newSorter(2, 20_000, gzip, mmap);
|
||||
shuffled.forEach(sorter::add);
|
||||
var writer = sorter.writerForThread();
|
||||
shuffled.forEach(writer);
|
||||
sorter.sort();
|
||||
assertEquals(sorted, sorter.toList());
|
||||
}
|
||||
|
|
|
@ -0,0 +1,56 @@
|
|||
package com.onthegomap.planetiler.util;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.stream.Stream;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.CsvSource;
|
||||
|
||||
class BinBackTest {
|
||||
|
||||
@ParameterizedTest
|
||||
@CsvSource(value = {
|
||||
"3;[];[]",
|
||||
"2;[1];[[1]]",
|
||||
"2;[2];[[2]]",
|
||||
"2;[3];[[3]]",
|
||||
"3;[1,2,3];[[3], [2, 1]]",
|
||||
"5;[1,2,3];[[3,2],[1]]",
|
||||
"6;[1,2,3];[[3,2,1]]",
|
||||
"2;[1,2,3];[[3],[2],[1]]",
|
||||
"1;[1,2,3];[[3],[2],[1]]",
|
||||
}, delimiter = ';')
|
||||
void test(int limit, String inputString, String expectedString) {
|
||||
List<Long> input = parseList(inputString);
|
||||
List<List<Long>> expected = parseListList(expectedString);
|
||||
// make sure we parsed correctly
|
||||
assertEqualsIgnoringWhitespace(inputString, input, "failed to parse input");
|
||||
assertEqualsIgnoringWhitespace(expectedString, expected, "failed to parse expected");
|
||||
|
||||
assertEquals(expected, BinPack.pack(input, limit, i -> i));
|
||||
}
|
||||
|
||||
private static List<Long> parseList(String string) {
|
||||
return Stream.of(string.replaceAll("[\\[\\]]", "").split(","))
|
||||
.map(String::strip)
|
||||
.filter(s -> !s.isBlank())
|
||||
.map(Long::parseLong)
|
||||
.toList();
|
||||
}
|
||||
|
||||
private static List<List<Long>> parseListList(String string) {
|
||||
return Stream.of(string.replaceAll("((^\\[)|(]$))", "").split("]\\s*,\\s*\\["))
|
||||
.map(BinBackTest::parseList)
|
||||
.filter(l -> !l.isEmpty())
|
||||
.toList();
|
||||
}
|
||||
|
||||
private static void assertEqualsIgnoringWhitespace(Object expected, Object actual, String message) {
|
||||
assertEquals(
|
||||
expected.toString().replaceAll("\\s", ""),
|
||||
actual.toString().replaceAll("\\s", ""),
|
||||
message
|
||||
);
|
||||
}
|
||||
}
|
|
@ -1,4 +1,4 @@
|
|||
sonar.issue.ignore.multicriteria=js1659,js3358,js1172,js106,js125,js2699,js3776,js1121,js107
|
||||
sonar.issue.ignore.multicriteria=js1659,js3358,js1172,js106,js125,js2699,js3776,js1121,js107,js1192
|
||||
# subjective
|
||||
sonar.issue.ignore.multicriteria.js1659.ruleKey=java:S1659
|
||||
sonar.issue.ignore.multicriteria.js1659.resourceKey=**/*.java
|
||||
|
@ -14,6 +14,8 @@ sonar.issue.ignore.multicriteria.js1121.ruleKey=java:S1121
|
|||
sonar.issue.ignore.multicriteria.js1121.resourceKey=**/*.java
|
||||
sonar.issue.ignore.multicriteria.js107.ruleKey=java:S107
|
||||
sonar.issue.ignore.multicriteria.js107.resourceKey=**/*.java
|
||||
sonar.issue.ignore.multicriteria.js1192.ruleKey=java:S1192
|
||||
sonar.issue.ignore.multicriteria.js1192.resourceKey=**/*.java
|
||||
|
||||
# layer constructors need same signatures
|
||||
sonar.issue.ignore.multicriteria.js1172.ruleKey=java:S1172
|
||||
|
|
Ładowanie…
Reference in New Issue