From 0c00d9cc17f40ff3cee09e34494484d3b8d84391 Mon Sep 17 00:00:00 2001 From: Rafael Troilo Date: Thu, 9 Sep 2021 15:37:23 +0200 Subject: [PATCH 1/4] celliterator refactoring, breaking for-loop up into iterator breaking for-loop up into iterator opening up iterateBy to accept Iterables instead of GridOSHEntity --- .../oshdb/util/celliterator/CellIterator.java | 234 +++++++++--------- 1 file changed, 118 insertions(+), 116 deletions(-) diff --git a/oshdb-util/src/main/java/org/heigit/ohsome/oshdb/util/celliterator/CellIterator.java b/oshdb-util/src/main/java/org/heigit/ohsome/oshdb/util/celliterator/CellIterator.java index 64578b6d6..95ef43cb5 100644 --- a/oshdb-util/src/main/java/org/heigit/ohsome/oshdb/util/celliterator/CellIterator.java +++ b/oshdb-util/src/main/java/org/heigit/ohsome/oshdb/util/celliterator/CellIterator.java @@ -1,14 +1,17 @@ package org.heigit.ohsome.oshdb.util.celliterator; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Streams; import java.io.Serializable; import java.util.ArrayList; +import java.util.Collections; import java.util.EnumSet; import java.util.Iterator; import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.NoSuchElementException; import java.util.SortedMap; import java.util.SortedSet; import java.util.TreeMap; @@ -261,8 +264,24 @@ public Stream iterateByTimestamps(GridOSHEntity cell) { } else { allFullyInside = false; } + return iterateByTimestamps(cell.getEntities(), allFullyInside); + } - Iterable cellData = cell.getEntities(); + /** + * Helper method to easily iterate over all entities in a cell that match a given condition/filter + * as they existed at the given timestamps. + * + * @param cellData the entities to iterate through + * @param allFullyInside + * + * @return a stream of matching filtered OSMEntities with their clipped Geometries at each + * timestamp. If an object has not been modified between timestamps, the output may + * contain the *same* Geometry object in the output multiple times. This can be used to + * optimize away recalculating expensive geometry operations on unchanged feature + * geometries later on in the code. + */ + public Stream iterateByTimestamps(Iterable cellData, + boolean allFullyInside) { return Streams.stream(cellData).flatMap(oshEntity -> { if (!oshEntityPreFilter.test(oshEntity) || !allFullyInside && ( @@ -526,15 +545,12 @@ public IterateAllEntry( * intervals. */ public Stream iterateByContribution(GridOSHEntity cell) { - OSHDBTimestampInterval timeInterval = new OSHDBTimestampInterval(timestamps); - + var cellBoundingBox = XYGrid.getBoundingBox(new CellId( + cell.getLevel(), + cell.getId() + ), true); final boolean allFullyInside; if (isBoundByPolygon) { - // if cell is fully inside bounding box/polygon we can skip all entity-based inclusion checks - OSHDBBoundingBox cellBoundingBox = XYGrid.getBoundingBox(new CellId( - cell.getLevel(), - cell.getId() - ), true); if (bboxOutsidePolygon.test(cellBoundingBox)) { return Stream.empty(); } @@ -542,58 +558,93 @@ public Stream iterateByContribution(GridOSHEntity cell) { } else { allFullyInside = false; } + return iterateByContribution(cell.getEntities(), allFullyInside); + } + + /** + * Helper method to easily iterate over all entity modifications in a cell that match a given + * condition/filter. + * + * @param cellData the entities to iterate through + * @param allFullyInside + * + * @return a stream of matching filtered OSMEntities with their clipped Geometries and timestamp + * intervals. + */ + public Stream iterateByContribution(Iterable cellData, + boolean allFullyInside) { + OSHDBTimestampInterval timeInterval = new OSHDBTimestampInterval(timestamps); if (includeOldStyleMultipolygons) { //todo: remove this by finishing the functionality below throw new UnsupportedOperationException("this is not yet properly implemented (probably)"); } + return Streams.stream(cellData) + .filter(oshEntity -> allFullyInside || (oshEntity.intersects(boundingBox) + && !(isBoundByPolygon && bboxOutsidePolygon.test(oshEntity)))) + .filter(oshEntityPreFilter) + .flatMap(oshEntity -> { + var fullyInside = allFullyInside || ( + oshEntity.coveredBy(boundingBox) + && (!isBoundByPolygon || bboxInPolygon.test(oshEntity)) + ); + var contribs = new ContributionIterator(oshEntity, timeInterval, fullyInside); + return Streams.stream(contribs); + }); + } - Iterable cellData = cell.getEntities(); + private class ContributionIterator implements Iterator { + private final OSHEntity oshEntity; + private final OSHDBTimestampInterval timeInterval; + private final boolean fullyInside; + private final Map changesetTs; + private final List modTs; + private final List osmEntityAtTimestamps; - return Streams.stream(cellData).flatMap(oshEntity -> { - if (!oshEntityPreFilter.test(oshEntity) - || !allFullyInside && ( - !oshEntity.intersects(boundingBox) - || (isBoundByPolygon && bboxOutsidePolygon.test(oshEntity)) - )) { - // this osh entity doesn't match the prefilter or is fully outside the requested - // area of interest -> skip it - return Stream.empty(); - } - if (Streams.stream(oshEntity.getVersions()).noneMatch(osmEntityFilter)) { - // none of this osh entity's versions matches the filter -> skip it - return Stream.empty(); - } - - boolean fullyInside = allFullyInside || ( - oshEntity.coveredBy(boundingBox) - && (!isBoundByPolygon || bboxInPolygon.test(oshEntity)) - ); + private int pos = 0; + private IterateAllEntry prev; + private IterateAllEntry next; - Map changesetTs = OSHEntityTimeUtils.getChangesetTimestamps(oshEntity); - List modTs = + private ContributionIterator(OSHEntity oshEntity, OSHDBTimestampInterval timeInterval, + boolean fullyInside) { + this.oshEntity = oshEntity; + this.timeInterval = timeInterval; + this.fullyInside = fullyInside; + this.changesetTs = OSHEntityTimeUtils.getChangesetTimestamps(oshEntity); + this.modTs = OSHEntityTimeUtils.getModificationTimestamps(oshEntity, osmEntityFilter, changesetTs); - if (modTs.isEmpty() || !timeInterval.intersects( new OSHDBTimestampInterval(modTs.get(0), modTs.get(modTs.size() - 1)) )) { // ignore osh entity because it's edit history is fully outside of the given time interval // of interest - return Stream.empty(); + this.osmEntityAtTimestamps = Collections.emptyList(); + } else { + this.osmEntityAtTimestamps = getVersionsByTimestamps(oshEntity, modTs); } + } - List osmEntityAtTimestamps = getVersionsByTimestamps(oshEntity, modTs); - - List results = new LinkedList<>(); + @Override + public boolean hasNext() { + return next != null || (next = getNext()) != null; + } - IterateAllEntry prev = null; + @Override + public IterateAllEntry next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + var ret = next; + next = null; + return ret; + } - osmEntityLoop: - for (int j = 0; j < osmEntityAtTimestamps.size(); j++) { - OSHDBTimestamp timestamp = modTs.get(j); - OSMEntity osmEntity = osmEntityAtTimestamps.get(j); + private IterateAllEntry getNext() { + while (pos < osmEntityAtTimestamps.size()) { + OSHDBTimestamp timestamp = modTs.get(pos); + OSMEntity osmEntity = osmEntityAtTimestamps.get(pos); + pos++; - // prev = results.size() > 0 ? results.get(results.size()-1) : null; // todo: replace with variable outside of osmEntitiyLoop (than we can also get rid of // the `|| prev.osmEntity.getId() != osmEntity.getId()`'s below) boolean skipOutput = false; @@ -638,81 +689,33 @@ public Stream iterateByContribution(GridOSHEntity cell) { // timestamp inteval). But if the oshdb-api would at some point have to support non- // contiguous timestamp intervals, this case could be needed. if (!skipOutput) { - results.add(prev); + return prev; } } - continue osmEntityLoop; + continue; } - // todo check old style mp code!!1!!!11! - boolean isOldStyleMultipolygon = false; - if (includeOldStyleMultipolygons && osmEntity instanceof OSMRelation - && tagInterpreter.isOldStyleMultipolygon((OSMRelation) osmEntity)) { - final OSMRelation rel = (OSMRelation) osmEntity; - for (int i = 0; i < rel.getMembers().length; i++) { - final OSMMember relMember = rel.getMembers()[i]; - if (relMember.getType() == OSMType.WAY - && tagInterpreter.isMultipolygonOuterMember(relMember)) { - OSMEntity way = OSHEntities.getByTimestamp(relMember.getEntity(), timestamp); - if (!osmEntityFilter.test(way)) { - // skip this old-style-multipolygon because it doesn't match our filter - continue osmEntityLoop; - } else { - // we know this multipolygon only has exactly one outer way, so we can abort the - // loop and actually - // "continue" with the calculations ^-^ - isOldStyleMultipolygon = true; - break; - } + if (!osmEntityFilter.test(osmEntity)) { + // this entity doesn't match our filter (anymore) + // TODO?: separate/additional activity type (e.g. "RECYCLED" ??) and still construct + // geometries for these? + if (prev != null && !prev.activities.contains(ContributionType.DELETION)) { + prev = new IterateAllEntry(timestamp, + osmEntity, prev.osmEntity, oshEntity, + new LazyEvaluatedObject<>((Geometry) null), prev.geometry, + new LazyEvaluatedObject<>((Geometry) null), prev.unclippedGeometry, + new LazyEvaluatedContributionTypes(EnumSet.of(ContributionType.DELETION)), + changesetTs.get(timestamp) + ); + if (!skipOutput) { + return prev; } } - } else { - if (!osmEntityFilter.test(osmEntity)) { - // this entity doesn't match our filter (anymore) - // TODO?: separate/additional activity type (e.g. "RECYCLED" ??) and still construct - // geometries for these? - if (prev != null && !prev.activities.contains(ContributionType.DELETION)) { - prev = new IterateAllEntry(timestamp, - osmEntity, prev.osmEntity, oshEntity, - new LazyEvaluatedObject<>((Geometry) null), prev.geometry, - new LazyEvaluatedObject<>((Geometry) null), prev.unclippedGeometry, - new LazyEvaluatedContributionTypes(EnumSet.of(ContributionType.DELETION)), - changesetTs.get(timestamp) - ); - if (!skipOutput) { - results.add(prev); - } - } - continue osmEntityLoop; - } + continue; } try { - LazyEvaluatedObject geom; - if (!isOldStyleMultipolygon) { - geom = constructClippedGeometry(osmEntity, timestamp, fullyInside); - } else { - // old style multipolygons: return only the inner holes of the geometry -> this is then - // used to "fix" the results obtained from calculating the geometry on the object's - // outer way which doesn't know about the inner members of the multipolygon relation - // todo: check if this is all valid? - GeometryFactory gf = new GeometryFactory(); - geom = new LazyEvaluatedObject<>(() -> { - Geometry geometry = - OSHDBGeometryBuilder.getGeometry(osmEntity, timestamp, tagInterpreter); - Polygon poly = (Polygon) geometry; - Polygon[] interiorRings = new Polygon[poly.getNumInteriorRing()]; - for (int i = 0; i < poly.getNumInteriorRing(); i++) { - interiorRings[i] = - new Polygon((LinearRing) poly.getInteriorRingN(i), new LinearRing[]{}, gf); - } - geometry = new MultiPolygon(interiorRings, gf); - if (!fullyInside) { - geometry = Geo.clip(geometry, boundingBox); - } - return geometry; - }); - } + var geom = constructClippedGeometry(osmEntity, timestamp, fullyInside); LazyEvaluatedContributionTypes activity; if (!fullyInside && geom.get().isEmpty()) { @@ -726,10 +729,10 @@ public Stream iterateByContribution(GridOSHEntity cell) { changesetTs.get(timestamp) ); if (!skipOutput) { - results.add(prev); + return prev; } } - continue osmEntityLoop; + continue; } else if (prev == null || prev.activities.contains(ContributionType.DELETION)) { activity = new LazyEvaluatedContributionTypes(EnumSet.of(ContributionType.CREATION)); // todo: special case when an object gets specific tag/condition again after having @@ -762,10 +765,10 @@ public Stream iterateByContribution(GridOSHEntity cell) { }); } - IterateAllEntry result; - LazyEvaluatedObject unclippedGeom = new LazyEvaluatedObject<>(() -> + var unclippedGeom = new LazyEvaluatedObject<>(() -> OSHDBGeometryBuilder.getGeometry(osmEntity, timestamp, tagInterpreter) ); + IterateAllEntry result; if (prev != null) { result = new IterateAllEntry(timestamp, osmEntity, prev.osmEntity, oshEntity, @@ -784,10 +787,10 @@ public Stream iterateByContribution(GridOSHEntity cell) { ); } + prev = result; if (!skipOutput) { - results.add(result); + return result; } - prev = result; } catch (IllegalArgumentException err) { // maybe some corner case where JTS doesn't support operations on a broken geometry LOG.info("Entity {}/{} skipped because of invalid geometry at timestamp {}", @@ -799,9 +802,8 @@ public Stream iterateByContribution(GridOSHEntity cell) { err.toString()); } } - // stream this oshEntity's results - return results.stream(); - }); + return null; + } } /** From 6eb129120fd9edce9e843141c148f3a4b73405e0 Mon Sep 17 00:00:00 2001 From: Rafael Troilo Date: Thu, 9 Sep 2021 15:40:39 +0200 Subject: [PATCH 2/4] replace filter with takeWhile --- .../oshdb/api/mapreducer/backend/Kernels.java | 20 +++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/oshdb-api/src/main/java/org/heigit/ohsome/oshdb/api/mapreducer/backend/Kernels.java b/oshdb-api/src/main/java/org/heigit/ohsome/oshdb/api/mapreducer/backend/Kernels.java index 7b7c33d8e..ca567ae75 100644 --- a/oshdb-api/src/main/java/org/heigit/ohsome/oshdb/api/mapreducer/backend/Kernels.java +++ b/oshdb-api/src/main/java/org/heigit/ohsome/oshdb/api/mapreducer/backend/Kernels.java @@ -22,6 +22,10 @@ class Kernels implements Serializable { interface CellProcessor extends SerializableBiFunction {} interface CancelableProcessStatus { + default boolean isActive(T ignore) { + return isActive(); + } + boolean isActive(); } @@ -57,7 +61,7 @@ static CellProcessor getOSMContributionCellReducer( // iterate over the history of all OSM objects in the current cell AtomicReference accInternal = new AtomicReference<>(identitySupplier.get()); cellIterator.iterateByContribution(oshEntityCell) - .filter(ignored -> process.isActive()) + .takeWhile(process::isActive) .forEach(contribution -> { OSMContribution osmContribution = new OSMContributionImpl(contribution); accInternal.set(accumulator.apply(accInternal.get(), mapper.apply(osmContribution))); @@ -87,7 +91,7 @@ static CellProcessor getOSMContributionGroupingCellReducer( // iterate over the history of all OSM objects in the current cell List contributions = new ArrayList<>(); cellIterator.iterateByContribution(oshEntityCell) - .filter(ignored -> process.isActive()) + .takeWhile(process::isActive) .forEach(contribution -> { OSMContribution thisContribution = new OSMContributionImpl(contribution); if (contributions.size() > 0 @@ -131,7 +135,7 @@ static CellProcessor getOSMEntitySnapshotCellReducer( // iterate over the history of all OSM objects in the current cell AtomicReference accInternal = new AtomicReference<>(identitySupplier.get()); cellIterator.iterateByTimestamps(oshEntityCell) - .filter(ignored -> process.isActive()) + .takeWhile(process::isActive) .forEach(data -> { OSMEntitySnapshot snapshot = new OSMEntitySnapshotImpl(data); // immediately fold the result @@ -162,7 +166,7 @@ static CellProcessor getOSMEntitySnapshotGroupingCellReducer( AtomicReference accInternal = new AtomicReference<>(identitySupplier.get()); List osmEntitySnapshots = new ArrayList<>(); cellIterator.iterateByTimestamps(oshEntityCell) - .filter(ignored -> process.isActive()) + .takeWhile(process::isActive) .forEach(data -> { OSMEntitySnapshot thisSnapshot = new OSMEntitySnapshotImpl(data); if (osmEntitySnapshots.size() > 0 @@ -203,7 +207,7 @@ static CellProcessor> getOSMContributionCellStreamer( return (oshEntityCell, cellIterator) -> { // iterate over the history of all OSM objects in the current cell return cellIterator.iterateByContribution(oshEntityCell) - .filter(ignored -> process.isActive()) + .takeWhile(process::isActive) .map(OSMContributionImpl::new) .map(mapper); }; @@ -226,7 +230,7 @@ static CellProcessor> getOSMContributionGroupingCellStreamer( List contributions = new ArrayList<>(); List result = new LinkedList<>(); cellIterator.iterateByContribution(oshEntityCell) - .filter(ignored -> process.isActive()) + .takeWhile(process::isActive) .map(OSMContributionImpl::new) .forEach(contribution -> { if (contributions.size() > 0 && contribution.getEntityAfter().getId() @@ -260,7 +264,7 @@ static CellProcessor> getOSMEntitySnapshotCellStreamer( return (oshEntityCell, cellIterator) -> { // iterate over the history of all OSM objects in the current cell return cellIterator.iterateByTimestamps(oshEntityCell) - .filter(ignored -> process.isActive()) + .takeWhile(process::isActive) .map(OSMEntitySnapshotImpl::new) .map(mapper); }; @@ -283,7 +287,7 @@ static CellProcessor> getOSMEntitySnapshotGroupingCellStreamer( List snapshots = new ArrayList<>(); List result = new LinkedList<>(); cellIterator.iterateByTimestamps(oshEntityCell) - .filter(ignored -> process.isActive()) + .takeWhile(process::isActive) .map(OSMEntitySnapshotImpl::new) .forEach(contribution -> { if (snapshots.size() > 0 && contribution.getEntity().getId() From 75ff657e1aadc1bc1a60b688e5ab66a6784e72b2 Mon Sep 17 00:00:00 2001 From: Rafael Troilo Date: Thu, 9 Sep 2021 18:26:32 +0200 Subject: [PATCH 3/4] review --- .../oshdb/api/mapreducer/backend/Kernels.java | 2 +- .../oshdb/util/celliterator/CellIterator.java | 80 ++++++++----------- .../oshdb/util/osh/OSHEntityTimeUtils.java | 3 + 3 files changed, 38 insertions(+), 47 deletions(-) diff --git a/oshdb-api/src/main/java/org/heigit/ohsome/oshdb/api/mapreducer/backend/Kernels.java b/oshdb-api/src/main/java/org/heigit/ohsome/oshdb/api/mapreducer/backend/Kernels.java index ca567ae75..d3b320892 100644 --- a/oshdb-api/src/main/java/org/heigit/ohsome/oshdb/api/mapreducer/backend/Kernels.java +++ b/oshdb-api/src/main/java/org/heigit/ohsome/oshdb/api/mapreducer/backend/Kernels.java @@ -22,7 +22,7 @@ class Kernels implements Serializable { interface CellProcessor extends SerializableBiFunction {} interface CancelableProcessStatus { - default boolean isActive(T ignore) { + default boolean isActive(T ignored) { return isActive(); } diff --git a/oshdb-util/src/main/java/org/heigit/ohsome/oshdb/util/celliterator/CellIterator.java b/oshdb-util/src/main/java/org/heigit/ohsome/oshdb/util/celliterator/CellIterator.java index 95ef43cb5..3f295a92e 100644 --- a/oshdb-util/src/main/java/org/heigit/ohsome/oshdb/util/celliterator/CellIterator.java +++ b/oshdb-util/src/main/java/org/heigit/ohsome/oshdb/util/celliterator/CellIterator.java @@ -1,6 +1,5 @@ package org.heigit.ohsome.oshdb.util.celliterator; -import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Streams; import java.io.Serializable; @@ -18,6 +17,7 @@ import java.util.TreeSet; import java.util.stream.Stream; import javax.annotation.Nonnull; +import org.heigit.ohsome.oshdb.OSHDBBoundable; import org.heigit.ohsome.oshdb.OSHDBBoundingBox; import org.heigit.ohsome.oshdb.OSHDBTemporal; import org.heigit.ohsome.oshdb.OSHDBTimestamp; @@ -66,6 +66,7 @@ public class CellIterator implements Serializable { private static final Logger LOG = LoggerFactory.getLogger(CellIterator.class); private final TreeSet timestamps; + private final OSHDBTimestampInterval timeInterval; private final OSHDBBoundingBox boundingBox; private boolean isBoundByPolygon; private FastBboxInPolygon bboxInPolygon; @@ -76,6 +77,7 @@ public class CellIterator implements Serializable { private final OSMEntityFilter osmEntityFilter; private final boolean includeOldStyleMultipolygons; + /** * Creates a cell iterator from a bounding box and a bounding polygon. * @@ -193,6 +195,7 @@ public CellIterator( boolean includeOldStyleMultipolygons ) { this.timestamps = new TreeSet<>(timestamps); + this.timeInterval = new OSHDBTimestampInterval(this.timestamps); this.boundingBox = boundingBox; this.isBoundByPolygon = false; // todo: maybe replace this with a "dummy" polygonClipper? this.bboxInPolygon = null; @@ -250,19 +253,10 @@ public IterateByTimestampEntry( * geometries later on in the code. */ public Stream iterateByTimestamps(GridOSHEntity cell) { - final boolean allFullyInside; - if (isBoundByPolygon) { - // if cell is fully inside bounding box/polygon we can skip all entity-based inclusion checks - OSHDBBoundingBox cellBoundingBox = XYGrid.getBoundingBox(new CellId( - cell.getLevel(), - cell.getId() - ), true); - if (bboxOutsidePolygon.test(cellBoundingBox)) { - return Stream.empty(); - } - allFullyInside = bboxInPolygon.test(cellBoundingBox); - } else { - allFullyInside = false; + var cellBoundingBox = XYGrid.getBoundingBox(new CellId(cell.getLevel(), cell.getId()), true); + final boolean allFullyInside = fullyInside(cellBoundingBox); + if (!allFullyInside && isBoundByPolygon && bboxOutsidePolygon.test(cellBoundingBox)) { + return Stream.empty(); } return iterateByTimestamps(cell.getEntities(), allFullyInside); } @@ -272,7 +266,7 @@ public Stream iterateByTimestamps(GridOSHEntity cell) { * as they existed at the given timestamps. * * @param cellData the entities to iterate through - * @param allFullyInside + * @param allFullyInside indicator that exact geometry inclusion checks can be skipped * * @return a stream of matching filtered OSMEntities with their clipped Geometries at each * timestamp. If an object has not been modified between timestamps, the output may @@ -296,10 +290,7 @@ public Stream iterateByTimestamps(Iterable skip it return Stream.empty(); } - boolean fullyInside = allFullyInside || ( - oshEntity.coveredBy(boundingBox) - && (!isBoundByPolygon || bboxInPolygon.test(oshEntity)) - ); + boolean fullyInside = allFullyInside || fullyInside(oshEntity); // optimize loop by requesting modification timestamps first, and skip geometry calculations // where not needed @@ -545,18 +536,10 @@ public IterateAllEntry( * intervals. */ public Stream iterateByContribution(GridOSHEntity cell) { - var cellBoundingBox = XYGrid.getBoundingBox(new CellId( - cell.getLevel(), - cell.getId() - ), true); - final boolean allFullyInside; - if (isBoundByPolygon) { - if (bboxOutsidePolygon.test(cellBoundingBox)) { - return Stream.empty(); - } - allFullyInside = bboxInPolygon.test(cellBoundingBox); - } else { - allFullyInside = false; + var cellBoundingBox = XYGrid.getBoundingBox(new CellId(cell.getLevel(), cell.getId()), true); + final boolean allFullyInside = fullyInside(cellBoundingBox); + if (!allFullyInside && isBoundByPolygon && bboxOutsidePolygon.test(cellBoundingBox)) { + return Stream.empty(); } return iterateByContribution(cell.getEntities(), allFullyInside); } @@ -566,36 +549,31 @@ public Stream iterateByContribution(GridOSHEntity cell) { * condition/filter. * * @param cellData the entities to iterate through - * @param allFullyInside + * @param allFullyInside indicator that exact geometry inclusion checks can be skipped * * @return a stream of matching filtered OSMEntities with their clipped Geometries and timestamp * intervals. */ public Stream iterateByContribution(Iterable cellData, boolean allFullyInside) { - OSHDBTimestampInterval timeInterval = new OSHDBTimestampInterval(timestamps); - if (includeOldStyleMultipolygons) { //todo: remove this by finishing the functionality below throw new UnsupportedOperationException("this is not yet properly implemented (probably)"); } return Streams.stream(cellData) - .filter(oshEntity -> allFullyInside || (oshEntity.intersects(boundingBox) - && !(isBoundByPolygon && bboxOutsidePolygon.test(oshEntity)))) + .filter(oshEntity -> allFullyInside || oshEntity.intersects(boundingBox)) .filter(oshEntityPreFilter) + .filter( + oshEntity -> allFullyInside || !isBoundByPolygon || !bboxOutsidePolygon.test(oshEntity)) .flatMap(oshEntity -> { - var fullyInside = allFullyInside || ( - oshEntity.coveredBy(boundingBox) - && (!isBoundByPolygon || bboxInPolygon.test(oshEntity)) - ); - var contribs = new ContributionIterator(oshEntity, timeInterval, fullyInside); + var fullyInside = allFullyInside || fullyInside(oshEntity); + var contribs = new ContributionIterator(oshEntity, fullyInside); return Streams.stream(contribs); }); } private class ContributionIterator implements Iterator { private final OSHEntity oshEntity; - private final OSHDBTimestampInterval timeInterval; private final boolean fullyInside; private final Map changesetTs; private final List modTs; @@ -605,10 +583,8 @@ private class ContributionIterator implements Iterator { private IterateAllEntry prev; private IterateAllEntry next; - private ContributionIterator(OSHEntity oshEntity, OSHDBTimestampInterval timeInterval, - boolean fullyInside) { + private ContributionIterator(OSHEntity oshEntity, boolean fullyInside) { this.oshEntity = oshEntity; - this.timeInterval = timeInterval; this.fullyInside = fullyInside; this.changesetTs = OSHEntityTimeUtils.getChangesetTimestamps(oshEntity); this.modTs = @@ -626,7 +602,11 @@ private ContributionIterator(OSHEntity oshEntity, OSHDBTimestampInterval timeInt @Override public boolean hasNext() { - return next != null || (next = getNext()) != null; + if (next != null) { + return true; + } + next = getNext(); + return next != null; } @Override @@ -806,6 +786,14 @@ private IterateAllEntry getNext() { } } + private boolean fullyInside(OSHDBBoundable bbox) { + if (isBoundByPolygon) { + return bboxInPolygon.test(bbox); + } else { + return bbox.coveredBy(boundingBox); + } + } + /** * Returns a list of corresponding osm OSMEntity "versions" of the given OSHEntity which are * valid at the given timestamps. The resulting list will contain the same number of entries diff --git a/oshdb-util/src/main/java/org/heigit/ohsome/oshdb/util/osh/OSHEntityTimeUtils.java b/oshdb-util/src/main/java/org/heigit/ohsome/oshdb/util/osh/OSHEntityTimeUtils.java index 3c0eec6f6..752ce1add 100644 --- a/oshdb-util/src/main/java/org/heigit/ohsome/oshdb/util/osh/OSHEntityTimeUtils.java +++ b/oshdb-util/src/main/java/org/heigit/ohsome/oshdb/util/osh/OSHEntityTimeUtils.java @@ -176,6 +176,9 @@ private static List getModificationTimestamps( OSHEntity osh, boolean recurse, Predicate osmEntityFilter) { // first, store this ways direct modifications (i.e. the "major" versions' timestamps) var entityTimestamps = getModificationTimestampsReverseNonRecursed(osh, osmEntityFilter); + if (entityTimestamps.isEmpty()) { + return entityTimestamps; + } if (!recurse || osh instanceof OSHNode) { return Lists.reverse(entityTimestamps); } From 17c0155015c3af889fe3c7be7bedabd9e94d5579 Mon Sep 17 00:00:00 2001 From: Rafael Troilo Date: Wed, 15 Sep 2021 17:09:52 +0200 Subject: [PATCH 4/4] Update CHANGELOG.md --- CHANGELOG.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 7d4dc0fe7..344781770 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -10,9 +10,11 @@ Changelog ### bugfixes * fix a bug which causes queries using the geometry filters `length:` and `area:` to fail when executed on an ignite cluster ([#426]) +* fix a bug which cause ComputeJobs to keep processing for a while despite they are already canceled ([#428]) [#419]: https://github.com/GIScience/oshdb/pull/419 [#426]: https://github.com/GIScience/oshdb/pull/426 +[#428]: https://github.com/GIScience/oshdb/pull/428 ## 0.7.1