diff --git a/CHANGELOG.md b/CHANGELOG.md index 0a2310203..fa910bc88 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,6 +1,19 @@ Changelog ========= +## 1.2.0-SNAPSHOT (current master) + +### new features + +* `OSMEntitySnapshot` now also returns the `lastContributionTimestamp` for each snapshot ([#495]) + +### other changes + +* `CellIterator` is now decoupled from implementation of the "Grid" ([#495]) + +[#495]: https://github.com/GIScience/oshdb/pull/495 + + ## 1.1.1 * update ignite dependency to [2.14.0-heigit1] ([#491]) diff --git a/oshdb-api-ignite/src/main/java/org/heigit/ohsome/oshdb/api/mapreducer/backend/MapReducerIgniteAffinityCall.java b/oshdb-api-ignite/src/main/java/org/heigit/ohsome/oshdb/api/mapreducer/backend/MapReducerIgniteAffinityCall.java index bcbd18dc2..ac8538707 100644 --- a/oshdb-api-ignite/src/main/java/org/heigit/ohsome/oshdb/api/mapreducer/backend/MapReducerIgniteAffinityCall.java +++ b/oshdb-api-ignite/src/main/java/org/heigit/ohsome/oshdb/api/mapreducer/backend/MapReducerIgniteAffinityCall.java @@ -13,7 +13,6 @@ import java.util.Optional; import java.util.TreeMap; import java.util.function.Function; -import java.util.stream.Collectors; import java.util.stream.LongStream; import java.util.stream.Stream; import javax.annotation.Nonnull; @@ -40,6 +39,7 @@ import org.heigit.ohsome.oshdb.util.CellId; import org.heigit.ohsome.oshdb.util.TableNames; import org.heigit.ohsome.oshdb.util.celliterator.CellIterator; +import org.heigit.ohsome.oshdb.util.celliterator.OSHEntitySource; import org.heigit.ohsome.oshdb.util.exceptions.OSHDBTimeoutException; import org.heigit.ohsome.oshdb.util.function.SerializableBiFunction; import org.heigit.ohsome.oshdb.util.function.SerializableBinaryOperator; @@ -139,8 +139,8 @@ private static <T> T asyncGetHandleTimeouts(IgniteFuture<T> async, Long timeout) // When a timeout happens remotely, the exception might be burried in (few) layers of // "ignite exceptions". This recursively unwinds these and throws the original exception. Throwable unwrapped = unwrapNestedIgniteException(e); - if (unwrapped instanceof OSHDBTimeoutException) { - throw (OSHDBTimeoutException) unwrapped; + if (unwrapped instanceof OSHDBTimeoutException timeoutException) { + throw timeoutException; } else { throw e; } @@ -150,8 +150,8 @@ private static <T> T asyncGetHandleTimeouts(IgniteFuture<T> async, Long timeout) /** Recursively unwinds nested ignite exceptions. */ private static Throwable unwrapNestedIgniteException(IgniteException e) { Throwable cause = e.getCause(); - if (cause instanceof IgniteException && e != cause) { - return unwrapNestedIgniteException((IgniteException) cause); + if (cause instanceof IgniteException igniteException && e != cause) { + return unwrapNestedIgniteException(igniteException); } return cause; } @@ -193,13 +193,13 @@ private <S> S reduce( .mapToObj(cellLongId -> asyncGetHandleTimeouts( compute.affinityCallAsync(cacheName, cellLongId, () -> { @SuppressWarnings("SerializableStoresNonSerializable") - GridOSHEntity oshEntityCell = cache.localPeek(cellLongId); + GridOSHEntity cell = cache.localPeek(cellLongId); S ret; - if (oshEntityCell == null) { + if (cell == null) { ret = identitySupplier.get(); } else { - ret = cellProcessor.apply(oshEntityCell, cellIterator); + ret = cellProcessor.apply(OSHEntitySource.fromGridOSHEntity(cell), cellIterator); } onClose.run(); return ret; @@ -263,19 +263,19 @@ cacheName, cellIdRangeToCellIds(), cellIdRanges, cellProcessor, cellIterator this.timeout ).stream() .flatMap(Collection::stream) - .collect(Collectors.toList()); + .toList(); Collections.shuffle(cellsWithData); Stream<X> resultForType = cellsWithData.parallelStream() .filter(ignored -> this.isActive()) .map(cellLongId -> asyncGetHandleTimeouts( compute.affinityCallAsync(cacheName, cellLongId, () -> { - GridOSHEntity oshEntityCell = cache.localPeek(cellLongId); + GridOSHEntity cell = cache.localPeek(cellLongId); Collection<X> ret; - if (oshEntityCell == null) { + if (cell == null) { ret = Collections.<X>emptyList(); } else { - ret = cellProcessor.apply(oshEntityCell, cellIterator) - .collect(Collectors.toList()); + ret = cellProcessor.apply(OSHEntitySource.fromGridOSHEntity(cell), cellIterator) + .toList(); } onClose.run(); return ret; @@ -444,10 +444,11 @@ public Collection<Long> call() { // test if cell exists and contains any relevant data GridOSHEntity cell = localCache.localPeek(cellLongId); return cell != null - && cellProcessor.apply(cell, cellIterator).anyMatch(ignored -> true); + && cellProcessor.apply(OSHEntitySource.fromGridOSHEntity(cell), cellIterator) + .anyMatch(ignored -> true); }) .boxed() - .collect(Collectors.toList()); + .toList(); } } @@ -488,13 +489,14 @@ public Collection<Long> call() { MapReducerIgniteScanQuery.cellKeyInRange(key, cellIdRangesByLevel) ).setPartition(part), cacheEntry -> { Object data = cacheEntry.getValue(); - GridOSHEntity oshEntityCell; - if (data instanceof BinaryObject) { - oshEntityCell = ((BinaryObject) data).deserialize(); + GridOSHEntity cell; + if (data instanceof BinaryObject binaryData) { + cell = binaryData.deserialize(); } else { - oshEntityCell = (GridOSHEntity) data; + cell = (GridOSHEntity) data; } - Stream<?> cellStream = cellProcessor.apply(oshEntityCell, this.cellIterator); + Stream<?> cellStream = cellProcessor.apply( + OSHEntitySource.fromGridOSHEntity(cell), this.cellIterator); if (cellStream.anyMatch(ignored -> true)) { return Optional.of(cacheEntry.getKey()); } else { @@ -510,7 +512,7 @@ public Collection<Long> call() { } }) .flatMap(Collection::stream) - .collect(Collectors.toList()); + .toList(); } } } diff --git a/oshdb-api-ignite/src/main/java/org/heigit/ohsome/oshdb/api/mapreducer/backend/MapReducerIgniteLocalPeek.java b/oshdb-api-ignite/src/main/java/org/heigit/ohsome/oshdb/api/mapreducer/backend/MapReducerIgniteLocalPeek.java index a4e24304d..0b2b55fca 100644 --- a/oshdb-api-ignite/src/main/java/org/heigit/ohsome/oshdb/api/mapreducer/backend/MapReducerIgniteLocalPeek.java +++ b/oshdb-api-ignite/src/main/java/org/heigit/ohsome/oshdb/api/mapreducer/backend/MapReducerIgniteLocalPeek.java @@ -32,6 +32,7 @@ import org.heigit.ohsome.oshdb.util.CellId; import org.heigit.ohsome.oshdb.util.TableNames; import org.heigit.ohsome.oshdb.util.celliterator.CellIterator; +import org.heigit.ohsome.oshdb.util.celliterator.OSHEntitySource; import org.heigit.ohsome.oshdb.util.exceptions.OSHDBTimeoutException; import org.heigit.ohsome.oshdb.util.function.OSHEntityFilter; import org.heigit.ohsome.oshdb.util.function.OSMEntityFilter; @@ -105,7 +106,7 @@ protected Stream<X> flatMapStreamCellsOSMEntitySnapshotGroupedById( private List<String> cacheNames(String prefix) { return this.typeFilter.stream().map(TableNames::forOSMType).filter(Optional::isPresent) - .map(Optional::get).map(tn -> tn.toString(prefix)).collect(Collectors.toList()); + .map(Optional::get).map(tn -> tn.toString(prefix)).toList(); } @Override @@ -281,7 +282,8 @@ S execute(Ignite node, CellProcessor<S> cellProcessor) { // filter out cache misses === empty oshdb cells or not "local" data .filter(Objects::nonNull) .filter(ignored -> this.isActive()) - .map(cell -> cellProcessor.apply(cell, this.cellIterator)) + .map(cell -> + cellProcessor.apply(OSHEntitySource.fromGridOSHEntity(cell), this.cellIterator)) .reduce(identitySupplier.get(), combiner); } } diff --git a/oshdb-api-ignite/src/main/java/org/heigit/ohsome/oshdb/api/mapreducer/backend/MapReducerIgniteScanQuery.java b/oshdb-api-ignite/src/main/java/org/heigit/ohsome/oshdb/api/mapreducer/backend/MapReducerIgniteScanQuery.java index ac69019a1..e214ab2e0 100644 --- a/oshdb-api-ignite/src/main/java/org/heigit/ohsome/oshdb/api/mapreducer/backend/MapReducerIgniteScanQuery.java +++ b/oshdb-api-ignite/src/main/java/org/heigit/ohsome/oshdb/api/mapreducer/backend/MapReducerIgniteScanQuery.java @@ -13,7 +13,6 @@ import java.util.TreeMap; import java.util.UUID; import java.util.function.Function; -import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; @@ -39,6 +38,7 @@ import org.heigit.ohsome.oshdb.util.CellId; import org.heigit.ohsome.oshdb.util.TableNames; import org.heigit.ohsome.oshdb.util.celliterator.CellIterator; +import org.heigit.ohsome.oshdb.util.celliterator.OSHEntitySource; import org.heigit.ohsome.oshdb.util.exceptions.OSHDBTimeoutException; import org.heigit.ohsome.oshdb.util.function.OSHEntityFilter; import org.heigit.ohsome.oshdb.util.function.OSMEntityFilter; @@ -342,13 +342,14 @@ S execute(Ignite node, CellProcessor<S> cellProcessor) { } // iterate over the history of all OSM objects in the current cell Object data = cacheEntry.getValue(); - GridOSHEntity oshEntityCell; - if (data instanceof BinaryObject) { - oshEntityCell = ((BinaryObject) data).deserialize(); + GridOSHEntity cell; + if (data instanceof BinaryObject binaryData) { + cell = binaryData.deserialize(); } else { - oshEntityCell = (GridOSHEntity) data; + cell = (GridOSHEntity) data; } - return cellProcessor.apply(oshEntityCell, this.cellIterator); + return cellProcessor.apply( + OSHEntitySource.fromGridOSHEntity(cell), this.cellIterator); } ) ) { @@ -503,7 +504,7 @@ private static <V, R, M, S, P extends Geometry & Polygonal> S mapReduceOnIgniteC null ); S ret; - if (!oshdb.timeoutInMilliseconds().isPresent()) { + if (oshdb.timeoutInMilliseconds().isEmpty()) { ret = result.get(); } else { try { @@ -534,13 +535,14 @@ private static <X> Stream<X> mapStreamOnIgniteCache( ).setPageSize(SCAN_QUERY_PAGE_SIZE), cacheEntry -> { // iterate over the history of all OSM objects in the current cell Object data = cacheEntry.getValue(); - GridOSHEntity oshEntityCell; - if (data instanceof BinaryObject) { - oshEntityCell = ((BinaryObject) data).deserialize(); + GridOSHEntity cell; + if (data instanceof BinaryObject binaryData) { + cell = binaryData.deserialize(); } else { - oshEntityCell = (GridOSHEntity) data; + cell = (GridOSHEntity) data; } - return cellProcessor.apply(oshEntityCell, cellIterator).collect(Collectors.toList()); + return cellProcessor.apply(OSHEntitySource.fromGridOSHEntity(cell), cellIterator) + .toList(); } ); // todo: ignite scan query doesn't support timeouts -> implement ourself? 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 d3b320892..c62eab363 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 @@ -10,8 +10,8 @@ import javax.annotation.Nonnull; import org.heigit.ohsome.oshdb.api.object.OSMContributionImpl; import org.heigit.ohsome.oshdb.api.object.OSMEntitySnapshotImpl; -import org.heigit.ohsome.oshdb.grid.GridOSHEntity; import org.heigit.ohsome.oshdb.util.celliterator.CellIterator; +import org.heigit.ohsome.oshdb.util.celliterator.OSHEntitySource; import org.heigit.ohsome.oshdb.util.function.SerializableBiFunction; import org.heigit.ohsome.oshdb.util.function.SerializableFunction; import org.heigit.ohsome.oshdb.util.function.SerializableSupplier; @@ -19,7 +19,7 @@ import org.heigit.ohsome.oshdb.util.mappable.OSMEntitySnapshot; class Kernels implements Serializable { - interface CellProcessor<S> extends SerializableBiFunction<GridOSHEntity, CellIterator, S> {} + interface CellProcessor<S> extends SerializableBiFunction<OSHEntitySource, CellIterator, S> {} interface CancelableProcessStatus { default <T> boolean isActive(T ignored) { @@ -57,10 +57,10 @@ static <R, S> CellProcessor<S> getOSMContributionCellReducer( SerializableBiFunction<S, R, S> accumulator, CancelableProcessStatus process ) { - return (oshEntityCell, cellIterator) -> { + return (source, cellIterator) -> { // iterate over the history of all OSM objects in the current cell AtomicReference<S> accInternal = new AtomicReference<>(identitySupplier.get()); - cellIterator.iterateByContribution(oshEntityCell) + cellIterator.iterateByContribution(source) .takeWhile(process::isActive) .forEach(contribution -> { OSMContribution osmContribution = new OSMContributionImpl(contribution); @@ -86,11 +86,11 @@ static <R, S> CellProcessor<S> getOSMContributionGroupingCellReducer( SerializableBiFunction<S, R, S> accumulator, CancelableProcessStatus process ) { - return (oshEntityCell, cellIterator) -> { + return (source, cellIterator) -> { AtomicReference<S> accInternal = new AtomicReference<>(identitySupplier.get()); // iterate over the history of all OSM objects in the current cell List<OSMContribution> contributions = new ArrayList<>(); - cellIterator.iterateByContribution(oshEntityCell) + cellIterator.iterateByContribution(source) .takeWhile(process::isActive) .forEach(contribution -> { OSMContribution thisContribution = new OSMContributionImpl(contribution); @@ -131,10 +131,10 @@ static <R, S> CellProcessor<S> getOSMEntitySnapshotCellReducer( SerializableBiFunction<S, R, S> accumulator, CancelableProcessStatus process ) { - return (oshEntityCell, cellIterator) -> { + return (source, cellIterator) -> { // iterate over the history of all OSM objects in the current cell AtomicReference<S> accInternal = new AtomicReference<>(identitySupplier.get()); - cellIterator.iterateByTimestamps(oshEntityCell) + cellIterator.iterateByTimestamps(source) .takeWhile(process::isActive) .forEach(data -> { OSMEntitySnapshot snapshot = new OSMEntitySnapshotImpl(data); @@ -161,11 +161,11 @@ static <R, S> CellProcessor<S> getOSMEntitySnapshotGroupingCellReducer( SerializableBiFunction<S, R, S> accumulator, CancelableProcessStatus process ) { - return (oshEntityCell, cellIterator) -> { + return (source, cellIterator) -> { // iterate over the history of all OSM objects in the current cell AtomicReference<S> accInternal = new AtomicReference<>(identitySupplier.get()); List<OSMEntitySnapshot> osmEntitySnapshots = new ArrayList<>(); - cellIterator.iterateByTimestamps(oshEntityCell) + cellIterator.iterateByTimestamps(source) .takeWhile(process::isActive) .forEach(data -> { OSMEntitySnapshot thisSnapshot = new OSMEntitySnapshotImpl(data); @@ -204,9 +204,9 @@ static <S> CellProcessor<Stream<S>> getOSMContributionCellStreamer( SerializableFunction<OSMContribution, S> mapper, CancelableProcessStatus process ) { - return (oshEntityCell, cellIterator) -> { + return (source, cellIterator) -> { // iterate over the history of all OSM objects in the current cell - return cellIterator.iterateByContribution(oshEntityCell) + return cellIterator.iterateByContribution(source) .takeWhile(process::isActive) .map(OSMContributionImpl::new) .map(mapper); @@ -225,11 +225,11 @@ static <S> CellProcessor<Stream<S>> getOSMContributionGroupingCellStreamer( SerializableFunction<List<OSMContribution>, Iterable<S>> mapper, CancelableProcessStatus process ) { - return (oshEntityCell, cellIterator) -> { + return (source, cellIterator) -> { // iterate over the history of all OSM objects in the current cell List<OSMContribution> contributions = new ArrayList<>(); List<S> result = new LinkedList<>(); - cellIterator.iterateByContribution(oshEntityCell) + cellIterator.iterateByContribution(source) .takeWhile(process::isActive) .map(OSMContributionImpl::new) .forEach(contribution -> { @@ -261,9 +261,9 @@ static <S> CellProcessor<Stream<S>> getOSMEntitySnapshotCellStreamer( SerializableFunction<OSMEntitySnapshot, S> mapper, CancelableProcessStatus process ) { - return (oshEntityCell, cellIterator) -> { + return (source, cellIterator) -> { // iterate over the history of all OSM objects in the current cell - return cellIterator.iterateByTimestamps(oshEntityCell) + return cellIterator.iterateByTimestamps(source) .takeWhile(process::isActive) .map(OSMEntitySnapshotImpl::new) .map(mapper); @@ -282,11 +282,11 @@ static <S> CellProcessor<Stream<S>> getOSMEntitySnapshotGroupingCellStreamer( SerializableFunction<List<OSMEntitySnapshot>, Iterable<S>> mapper, CancelableProcessStatus process ) { - return (oshEntityCell, cellIterator) -> { + return (source, cellIterator) -> { // iterate over the history of all OSM objects in the current cell List<OSMEntitySnapshot> snapshots = new ArrayList<>(); List<S> result = new LinkedList<>(); - cellIterator.iterateByTimestamps(oshEntityCell) + cellIterator.iterateByTimestamps(source) .takeWhile(process::isActive) .map(OSMEntitySnapshotImpl::new) .forEach(contribution -> { diff --git a/oshdb-api/src/main/java/org/heigit/ohsome/oshdb/api/mapreducer/backend/MapReducerJdbcMultithread.java b/oshdb-api/src/main/java/org/heigit/ohsome/oshdb/api/mapreducer/backend/MapReducerJdbcMultithread.java index 6f75856c3..f046f8cf6 100644 --- a/oshdb-api/src/main/java/org/heigit/ohsome/oshdb/api/mapreducer/backend/MapReducerJdbcMultithread.java +++ b/oshdb-api/src/main/java/org/heigit/ohsome/oshdb/api/mapreducer/backend/MapReducerJdbcMultithread.java @@ -10,6 +10,7 @@ import org.heigit.ohsome.oshdb.api.mapreducer.backend.Kernels.CellProcessor; import org.heigit.ohsome.oshdb.index.XYGridTree.CellIdRange; import org.heigit.ohsome.oshdb.util.celliterator.CellIterator; +import org.heigit.ohsome.oshdb.util.celliterator.OSHEntitySource; import org.heigit.ohsome.oshdb.util.function.SerializableBiFunction; import org.heigit.ohsome.oshdb.util.function.SerializableBinaryOperator; import org.heigit.ohsome.oshdb.util.function.SerializableFunction; @@ -69,7 +70,7 @@ private <S> S reduce( .filter(ignored -> this.isActive()) .flatMap(this::getOshCellsStream) .filter(ignored -> this.isActive()) - .map(oshCell -> processor.apply(oshCell, cellIterator)) + .map(cell -> processor.apply(OSHEntitySource.fromGridOSHEntity(cell), cellIterator)) .reduce(identitySupplier.get(), combiner); } @@ -91,7 +92,7 @@ private Stream<X> stream( .filter(ignored -> this.isActive()) .flatMap(this::getOshCellsStream) .filter(ignored -> this.isActive()) - .flatMap(oshCell -> processor.apply(oshCell, cellIterator)); + .flatMap(cell -> processor.apply(OSHEntitySource.fromGridOSHEntity(cell), cellIterator)); } // === map-reduce operations === diff --git a/oshdb-api/src/main/java/org/heigit/ohsome/oshdb/api/mapreducer/backend/MapReducerJdbcSinglethread.java b/oshdb-api/src/main/java/org/heigit/ohsome/oshdb/api/mapreducer/backend/MapReducerJdbcSinglethread.java index 756c3f0e6..f5d13e1cd 100644 --- a/oshdb-api/src/main/java/org/heigit/ohsome/oshdb/api/mapreducer/backend/MapReducerJdbcSinglethread.java +++ b/oshdb-api/src/main/java/org/heigit/ohsome/oshdb/api/mapreducer/backend/MapReducerJdbcSinglethread.java @@ -12,6 +12,7 @@ import org.heigit.ohsome.oshdb.grid.GridOSHEntity; import org.heigit.ohsome.oshdb.index.XYGridTree.CellIdRange; import org.heigit.ohsome.oshdb.util.celliterator.CellIterator; +import org.heigit.ohsome.oshdb.util.celliterator.OSHEntitySource; import org.heigit.ohsome.oshdb.util.function.SerializableBiFunction; import org.heigit.ohsome.oshdb.util.function.SerializableBinaryOperator; import org.heigit.ohsome.oshdb.util.function.SerializableFunction; @@ -73,8 +74,9 @@ private <S> S reduce( pstmt.setLong(3, cellIdRange.getEnd().getId()); try (var oshCellsRawData = pstmt.executeQuery()) { while (oshCellsRawData.next()) { - GridOSHEntity oshCellRawData = readOshCellRawData(oshCellsRawData); - result = combiner.apply(result, cellProcessor.apply(oshCellRawData, cellIterator)); + GridOSHEntity cell = readOshCellRawData(oshCellsRawData); + result = combiner.apply(result, cellProcessor.apply( + OSHEntitySource.fromGridOSHEntity(cell), cellIterator)); } } } @@ -95,7 +97,8 @@ private Stream<X> stream( return Streams.stream(this.getCellIdRanges()) .flatMap(this::getOshCellsStream) - .flatMap(oshCellRawData -> cellProcessor.apply(oshCellRawData, cellIterator)); + .flatMap(cell -> cellProcessor.apply( + OSHEntitySource.fromGridOSHEntity(cell), cellIterator)); } // === map-reduce operations === diff --git a/oshdb-api/src/main/java/org/heigit/ohsome/oshdb/api/object/OSMContributionImpl.java b/oshdb-api/src/main/java/org/heigit/ohsome/oshdb/api/object/OSMContributionImpl.java index 65fb838c6..1672b53a4 100644 --- a/oshdb-api/src/main/java/org/heigit/ohsome/oshdb/api/object/OSMContributionImpl.java +++ b/oshdb-api/src/main/java/org/heigit/ohsome/oshdb/api/object/OSMContributionImpl.java @@ -75,52 +75,52 @@ public OSMContributionImpl( @Override public OSHDBTimestamp getTimestamp() { - return data.timestamp; + return data.timestamp(); } @Override public Geometry getGeometryBefore() { - return data.previousGeometry.get(); + return data.previousGeometry().get(); } @Override public Geometry getGeometryUnclippedBefore() { - return data.unclippedPreviousGeometry.get(); + return data.unclippedPreviousGeometry().get(); } @Override public Geometry getGeometryAfter() { - return data.geometry.get(); + return data.geometry().get(); } @Override public Geometry getGeometryUnclippedAfter() { - return data.unclippedGeometry.get(); + return data.unclippedGeometry().get(); } @Override public OSMEntity getEntityBefore() { - return data.previousOsmEntity; + return data.previousOsmEntity(); } @Override public OSMEntity getEntityAfter() { - return data.osmEntity; + return data.osmEntity(); } @Override public OSHEntity getOSHEntity() { - return data.oshEntity; + return data.oshEntity(); } @Override public boolean is(ContributionType contributionType) { - return data.activities.contains(contributionType); + return data.activities().contains(contributionType); } @Override public EnumSet<ContributionType> getContributionTypes() { - return data.activities.get(); + return data.activities().get(); } @@ -175,7 +175,7 @@ public int getContributorUserId() { @Override public long getChangesetId() { - return data.changeset; + return data.changeset(); } @Override diff --git a/oshdb-api/src/main/java/org/heigit/ohsome/oshdb/api/object/OSMEntitySnapshotImpl.java b/oshdb-api/src/main/java/org/heigit/ohsome/oshdb/api/object/OSMEntitySnapshotImpl.java index 21ca9157e..610e375bb 100644 --- a/oshdb-api/src/main/java/org/heigit/ohsome/oshdb/api/object/OSMEntitySnapshotImpl.java +++ b/oshdb-api/src/main/java/org/heigit/ohsome/oshdb/api/object/OSMEntitySnapshotImpl.java @@ -15,8 +15,7 @@ * * <p>Alongside the entity and the timestamp, also the entity's geometry is provided.</p> */ -public class OSMEntitySnapshotImpl implements - org.heigit.ohsome.oshdb.util.mappable.OSMEntitySnapshot { +public class OSMEntitySnapshotImpl implements OSMEntitySnapshot { private final IterateByTimestampEntry data; public OSMEntitySnapshotImpl(IterateByTimestampEntry data) { @@ -39,6 +38,7 @@ public OSMEntitySnapshotImpl( ) { this.data = new IterateByTimestampEntry( other.getTimestamp(), + other.getLastContributionTimestamp(), other.getEntity(), other.getOSHEntity(), reclippedGeometry, @@ -48,27 +48,32 @@ public OSMEntitySnapshotImpl( @Override public OSHDBTimestamp getTimestamp() { - return data.timestamp; + return data.timestamp(); + } + + @Override + public OSHDBTimestamp getLastContributionTimestamp() { + return data.lastModificationTimestamp(); } @Override public Geometry getGeometry() { - return data.geometry.get(); + return data.geometry().get(); } @Override public Geometry getGeometryUnclipped() { - return data.unclippedGeometry.get(); + return data.unclippedGeometry().get(); } @Override public OSMEntity getEntity() { - return data.osmEntity; + return data.osmEntity(); } @Override public OSHEntity getOSHEntity() { - return data.oshEntity; + return data.oshEntity(); } /** diff --git a/oshdb-api/src/test/java/org/heigit/ohsome/oshdb/api/tests/TestAutoAggregation.java b/oshdb-api/src/test/java/org/heigit/ohsome/oshdb/api/tests/TestAutoAggregation.java index e5a3275b1..ad4fe1f55 100644 --- a/oshdb-api/src/test/java/org/heigit/ohsome/oshdb/api/tests/TestAutoAggregation.java +++ b/oshdb-api/src/test/java/org/heigit/ohsome/oshdb/api/tests/TestAutoAggregation.java @@ -194,7 +194,8 @@ protected <R, S> S flatMapReduceCellsOSMEntitySnapshotGroupedById( private static OSMEntitySnapshot snapshot(OSHNode node) { var timestamp = timestamps.get().first(); - var data = new IterateByTimestampEntry(timestamp, node.getVersions().iterator().next(), node, + var data = new IterateByTimestampEntry(timestamp, null, + node.getVersions().iterator().next(), node, new LazyEvaluatedObject<>(point), new LazyEvaluatedObject<>(point)); return new OSMEntitySnapshotImpl(data); diff --git a/oshdb-filter/src/test/java/org/heigit/ohsome/oshdb/filter/ApplyOSMEntitySnapshotTest.java b/oshdb-filter/src/test/java/org/heigit/ohsome/oshdb/filter/ApplyOSMEntitySnapshotTest.java index ad44b3cfe..e119c81c9 100644 --- a/oshdb-filter/src/test/java/org/heigit/ohsome/oshdb/filter/ApplyOSMEntitySnapshotTest.java +++ b/oshdb-filter/src/test/java/org/heigit/ohsome/oshdb/filter/ApplyOSMEntitySnapshotTest.java @@ -35,6 +35,11 @@ public OSHDBTimestamp getTimestamp() { throw new UnsupportedOperationException(UNSUPPORTED_IN_TEST); } + @Override + public OSHDBTimestamp getLastContributionTimestamp() { + throw new UnsupportedOperationException(UNSUPPORTED_IN_TEST); + } + @Override public Geometry getGeometry() { return this.geometry; 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 b1b0f3b1b..6f36d12c5 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 @@ -21,8 +21,6 @@ import org.heigit.ohsome.oshdb.OSHDBBoundingBox; import org.heigit.ohsome.oshdb.OSHDBTemporal; import org.heigit.ohsome.oshdb.OSHDBTimestamp; -import org.heigit.ohsome.oshdb.grid.GridOSHEntity; -import org.heigit.ohsome.oshdb.index.XYGrid; import org.heigit.ohsome.oshdb.osh.OSHEntities; import org.heigit.ohsome.oshdb.osh.OSHEntity; import org.heigit.ohsome.oshdb.osm.OSMEntity; @@ -30,7 +28,6 @@ import org.heigit.ohsome.oshdb.osm.OSMRelation; import org.heigit.ohsome.oshdb.osm.OSMType; import org.heigit.ohsome.oshdb.osm.OSMWay; -import org.heigit.ohsome.oshdb.util.CellId; import org.heigit.ohsome.oshdb.util.function.OSHEntityFilter; import org.heigit.ohsome.oshdb.util.function.OSMEntityFilter; import org.heigit.ohsome.oshdb.util.geometry.Geo; @@ -59,8 +56,8 @@ * Allows to iterate through the contents of OSH grid cells. * * <p>There are two modes of iterating through a cell: 1) iterate at specific timestamps - * (entity snapshots) {@link #iterateByTimestamps(GridOSHEntity)} or 2) iterate through - * all (minor) versions of each entity {@link #iterateByContribution(GridOSHEntity)}. + * (entity snapshots) {@link #iterateByTimestamps(OSHEntitySource)} or 2) iterate through + * all (minor) versions of each entity {@link #iterateByContribution(OSHEntitySource)}. */ public class CellIterator implements Serializable { private static final Logger LOG = LoggerFactory.getLogger(CellIterator.class); @@ -208,43 +205,31 @@ public CellIterator( } /** - * Holds the result of a single item returned by {@link #iterateByTimestamps(GridOSHEntity)}. + * Holds the result of a single item returned by {@link #iterateByTimestamps(OSHEntitySource)}. + * + * @param timestamp timestamp of the snapshot + * @param lastModificationTimestamp last modification timestamp before the snapshot's timestamp + * @param osmEntity the exact version of the OSM object + * @param oshEntity the whole version history of the OSM object + * @param geometry an object which holds the geometry of the OSM object, or a method to build it + * on request, clipped to the query area of interest + * @param unclippedGeometry holds the full unclipped geometry of the OSM object + * or a function to build it */ - public static class IterateByTimestampEntry { - public final OSHDBTimestamp timestamp; - public final OSMEntity osmEntity; - public final OSHEntity oshEntity; - public final LazyEvaluatedObject<Geometry> geometry; - public final LazyEvaluatedObject<Geometry> unclippedGeometry; - - /** - * Properties associated with each entity snapshot. - * - * @param timestamp timestamp of the snapshot - * @param osmEntity the exact version of the OSM object - * @param oshEntity the whole version history of the OSM object - * @param geom an object which holds the geometry of the OSM object, or a method to build it - * on request, clipped to the query area of interest - * @param unclippedGeom holds the full unclipped geometry of the OSM object - * or a function to build it - */ - public IterateByTimestampEntry( - OSHDBTimestamp timestamp, @Nonnull OSMEntity osmEntity, @Nonnull OSHEntity oshEntity, - LazyEvaluatedObject<Geometry> geom, LazyEvaluatedObject<Geometry> unclippedGeom - ) { - this.timestamp = timestamp; - this.osmEntity = osmEntity; - this.oshEntity = oshEntity; - this.geometry = geom; - this.unclippedGeometry = unclippedGeom; - } - } + public record IterateByTimestampEntry( + OSHDBTimestamp timestamp, + OSHDBTimestamp lastModificationTimestamp, + @Nonnull OSMEntity osmEntity, + @Nonnull OSHEntity oshEntity, + LazyEvaluatedObject<Geometry> geometry, + LazyEvaluatedObject<Geometry> unclippedGeometry + ) {} /** - * Helper method to easily iterate over all entities in a cell that match a given condition/filter + * Helper method to easily iterate over all entities that match a given condition/filter * as they existed at the given timestamps. * - * @param cell the data cell + * @param source a provider of a stream of OSHEntity objects and a corresponding bounding box * * @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 @@ -252,20 +237,20 @@ public IterateByTimestampEntry( * optimize away recalculating expensive geometry operations on unchanged feature * geometries later on in the code. */ - public Stream<IterateByTimestampEntry> iterateByTimestamps(GridOSHEntity cell) { - var cellBoundingBox = XYGrid.getBoundingBox(new CellId(cell.getLevel(), cell.getId()), true); + public Stream<IterateByTimestampEntry> iterateByTimestamps(OSHEntitySource source) { + var cellBoundingBox = source.getBoundingBox(); final boolean allFullyInside = fullyInside(cellBoundingBox); if (!allFullyInside && isBoundByPolygon && bboxOutsidePolygon.test(cellBoundingBox)) { return Stream.empty(); } - return iterateByTimestamps(cell.getEntities(), allFullyInside); + return iterateByTimestamps(source.getData(), allFullyInside); } /** - * Helper method to easily iterate over all entities in a cell that match a given condition/filter + * Helper method to easily iterate over all entities that match a given condition/filter * as they existed at the given timestamps. * - * @param cellData the entities to iterate through + * @param oshData the entities to iterate through * @param allFullyInside indicator that exact geometry inclusion checks can be skipped * * @return a stream of matching filtered OSMEntities with their clipped Geometries at each @@ -274,9 +259,9 @@ public Stream<IterateByTimestampEntry> iterateByTimestamps(GridOSHEntity cell) { * optimize away recalculating expensive geometry operations on unchanged feature * geometries later on in the code. */ - public Stream<IterateByTimestampEntry> iterateByTimestamps(Iterable<? extends OSHEntity> cellData, + private Stream<IterateByTimestampEntry> iterateByTimestamps(Stream<? extends OSHEntity> oshData, boolean allFullyInside) { - return Streams.stream(cellData).flatMap(oshEntity -> { + return oshData.flatMap(oshEntity -> { if (!oshEntityPreFilter.test(oshEntity) || !allFullyInside && ( !oshEntity.getBoundable().intersects(boundingBox) @@ -295,15 +280,18 @@ public Stream<IterateByTimestampEntry> iterateByTimestamps(Iterable<? extends OS // optimize loop by requesting modification timestamps first, and skip geometry calculations // where not needed SortedMap<OSHDBTimestamp, List<OSHDBTimestamp>> queryTs = new TreeMap<>(); + SortedMap<OSHDBTimestamp, OSHDBTimestamp> lastModificationTimestamps = new TreeMap<>(); if (!includeOldStyleMultipolygons) { List<OSHDBTimestamp> modTs = OSHEntityTimeUtils.getModificationTimestamps(oshEntity, osmEntityFilter); int j = 0; + OSHDBTimestamp lastModificationTimestamp = null; for (OSHDBTimestamp requestedT : timestamps) { boolean needToRequest = false; while (j < modTs.size() && modTs.get(j).getEpochSecond() <= requestedT.getEpochSecond()) { needToRequest = true; + lastModificationTimestamp = modTs.get(j); j++; } if (needToRequest) { @@ -311,6 +299,7 @@ public Stream<IterateByTimestampEntry> iterateByTimestamps(Iterable<? extends OS } else if (queryTs.size() > 0) { queryTs.get(queryTs.lastKey()).add(requestedT); } + lastModificationTimestamps.put(requestedT, lastModificationTimestamp); } } else { // todo: make this work with old style multipolygons!!?! @@ -331,17 +320,16 @@ public Stream<IterateByTimestampEntry> iterateByTimestamps(Iterable<? extends OS // skip because this entity is deleted at this timestamp continue; } - if (osmEntity instanceof OSMWay && ((OSMWay) osmEntity).getMembers().length == 0 - || osmEntity instanceof OSMRelation - && ((OSMRelation) osmEntity).getMembers().length == 0) { + if (osmEntity instanceof OSMWay osmWay && osmWay.getMembers().length == 0 + || osmEntity instanceof OSMRelation osmRelation + && osmRelation.getMembers().length == 0) { // skip way/relation with zero nodes/members continue; } boolean isOldStyleMultipolygon = false; - if (includeOldStyleMultipolygons && osmEntity instanceof OSMRelation - && tagInterpreter.isOldStyleMultipolygon((OSMRelation) osmEntity)) { - final OSMRelation rel = (OSMRelation) osmEntity; + if (includeOldStyleMultipolygons && osmEntity instanceof OSMRelation rel + && tagInterpreter.isOldStyleMultipolygon(rel)) { for (int i = 0; i < rel.getMembers().length; i++) { final OSMMember relMember = rel.getMembers()[i]; if (relMember.getType() == OSMType.WAY @@ -396,16 +384,17 @@ public Stream<IterateByTimestampEntry> iterateByTimestamps(Iterable<? extends OS }); } + var lastModificationTimestamp = lastModificationTimestamps.get(timestamp); if (fullyInside || !geom.get().isEmpty()) { LazyEvaluatedObject<Geometry> fullGeom = fullyInside ? geom : new LazyEvaluatedObject<>( () -> OSHDBGeometryBuilder.getGeometry(osmEntity, timestamp, tagInterpreter)); - results.add( - new IterateByTimestampEntry(timestamp, osmEntity, oshEntity, geom, fullGeom) + results.add(new IterateByTimestampEntry( + timestamp, lastModificationTimestamp, osmEntity, oshEntity, geom, fullGeom) ); // add skipped timestamps (where nothing has changed from the last timestamp) to result for (OSHDBTimestamp additionalT : queryTs.get(timestamp)) { - results.add( - new IterateByTimestampEntry(additionalT, osmEntity, oshEntity, geom, fullGeom) + results.add(new IterateByTimestampEntry( + additionalT, lastModificationTimestamp, osmEntity, oshEntity, geom, fullGeom) ); } } @@ -470,97 +459,72 @@ private Geometry createEmptyGeometryLike(Geometry geometry) { } /** - * Holds the result of a single item returned by {@link #iterateByContribution(GridOSHEntity)}. + * Holds the result of a single item returned by {@link #iterateByContribution(OSHEntitySource)}. + * + * @param timestamp the timestamp when the OSM object was modified + * @param osmEntity the version of the OSM object after the modification + * @param previousOsmEntity the version of the OSM object before the modification + * @param oshEntity the full version history of the OSM object + * @param geometry the geometry of the OSM object (or a function to build it) of the state + * of the OSM object after the modification, clipped to the query + * area of interest + * @param previousGeometry the geometry of the OSM object (or a function to build it) of + * the state of the OSM object before the modification, clipped to the + * the query area of interest + * @param unclippedGeometry same as {@link #geometry}, but not clipped to the query area + * @param unclippedPreviousGeometry same as {@link #previousGeometry}, but not clipped to + * the query area + * @param activities a set of contribution types this modification can be classified with + * @param changeset the changeset id this data modification is a part of */ - public static class IterateAllEntry { - public final OSHDBTimestamp timestamp; - @Nonnull - public final OSMEntity osmEntity; - public final OSMEntity previousOsmEntity; - public final OSHEntity oshEntity; - public final LazyEvaluatedObject<Geometry> geometry; - public final LazyEvaluatedObject<Geometry> previousGeometry; - public final LazyEvaluatedObject<Geometry> unclippedGeometry; - public final LazyEvaluatedObject<Geometry> unclippedPreviousGeometry; - public final LazyEvaluatedContributionTypes activities; - public final long changeset; - - /** - * Properties associated with each contribution object. - * - * @param timestamp the timestamp when the OSM object was modified - * @param osmEntity the version of the OSM object after the modification - * @param previousOsmEntity the version of the OSM object before the modification - * @param oshEntity the full version history of the OSM object - * @param geometry the geometry of the OSM object (or a function to build it) of the state - * of the OSM object after the modification, clipped to the query - * area of interest - * @param previousGeometry the geometry of the OSM object (or a function to build it) of - * the state of the OSM object before the modification, clipped to the - * the query area of interest - * @param unclippedGeometry same as {@link #geometry}, but not clipped to the query area - * @param previousUnclippedGeometry same as {@link #previousGeometry}, but not clipped to - * the query area - * @param activities a set of contribution types this modification can be classified with - * @param changeset the changeset id this data modification is a part of - */ - public IterateAllEntry( - OSHDBTimestamp timestamp, - @Nonnull OSMEntity osmEntity, OSMEntity previousOsmEntity, @Nonnull OSHEntity oshEntity, - LazyEvaluatedObject<Geometry> geometry, LazyEvaluatedObject<Geometry> previousGeometry, - LazyEvaluatedObject<Geometry> unclippedGeometry, - LazyEvaluatedObject<Geometry> previousUnclippedGeometry, - LazyEvaluatedContributionTypes activities, - long changeset - ) { - this.timestamp = timestamp; - this.osmEntity = osmEntity; - this.previousOsmEntity = previousOsmEntity; - this.oshEntity = oshEntity; - this.geometry = geometry; - this.previousGeometry = previousGeometry; - this.unclippedGeometry = unclippedGeometry; - this.unclippedPreviousGeometry = previousUnclippedGeometry; - this.activities = activities; - this.changeset = changeset; - } - } + public record IterateAllEntry( + OSHDBTimestamp timestamp, + @Nonnull OSMEntity osmEntity, + OSMEntity previousOsmEntity, + OSHEntity oshEntity, + LazyEvaluatedObject<Geometry> geometry, + LazyEvaluatedObject<Geometry> previousGeometry, + LazyEvaluatedObject<Geometry> unclippedGeometry, + LazyEvaluatedObject<Geometry> unclippedPreviousGeometry, + LazyEvaluatedContributionTypes activities, + long changeset + ) {} /** - * Helper method to easily iterate over all entity modifications in a cell that match a given + * Helper method to easily iterate over all entity modifications that match a given * condition/filter. * - * @param cell the data cell + * @param source a provider of a stream of OSHEntity objects and a corresponding bounding box * * @return a stream of matching filtered OSMEntities with their clipped Geometries and timestamp * intervals. */ - public Stream<IterateAllEntry> iterateByContribution(GridOSHEntity cell) { - var cellBoundingBox = XYGrid.getBoundingBox(new CellId(cell.getLevel(), cell.getId()), true); + public Stream<IterateAllEntry> iterateByContribution(OSHEntitySource source) { + var cellBoundingBox = source.getBoundingBox(); final boolean allFullyInside = fullyInside(cellBoundingBox); if (!allFullyInside && isBoundByPolygon && bboxOutsidePolygon.test(cellBoundingBox)) { return Stream.empty(); } - return iterateByContribution(cell.getEntities(), allFullyInside); + return iterateByContribution(source.getData(), allFullyInside); } /** - * Helper method to easily iterate over all entity modifications in a cell that match a given + * Helper method to easily iterate over all entity modifications that match a given * condition/filter. * - * @param cellData the entities to iterate through + * @param oshData the entities to iterate through * @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<IterateAllEntry> iterateByContribution(Iterable<? extends OSHEntity> cellData, + private Stream<IterateAllEntry> iterateByContribution(Stream<? extends OSHEntity> oshData, boolean allFullyInside) { if (includeOldStyleMultipolygons) { //todo: remove this by finishing the functionality below throw new UnsupportedOperationException("this is not yet properly implemented (probably)"); } - return Streams.stream(cellData) + return oshData .filter(oshEntity -> allFullyInside || oshEntity.getBoundable().intersects(boundingBox)) .filter(oshEntityPreFilter) .filter(oshEntity -> allFullyInside || !isBoundByPolygon @@ -621,8 +585,8 @@ public IterateAllEntry next() { private IterateAllEntry getNext() { while (pos < osmEntityAtTimestamps.size()) { - OSHDBTimestamp timestamp = modTs.get(pos); - OSMEntity osmEntity = osmEntityAtTimestamps.get(pos); + final OSHDBTimestamp timestamp = modTs.get(pos); + final OSMEntity osmEntity = osmEntityAtTimestamps.get(pos); pos++; // todo: replace with variable outside of osmEntitiyLoop (than we can also get rid of diff --git a/oshdb-util/src/main/java/org/heigit/ohsome/oshdb/util/celliterator/OSHEntitySource.java b/oshdb-util/src/main/java/org/heigit/ohsome/oshdb/util/celliterator/OSHEntitySource.java new file mode 100644 index 000000000..cac69feb3 --- /dev/null +++ b/oshdb-util/src/main/java/org/heigit/ohsome/oshdb/util/celliterator/OSHEntitySource.java @@ -0,0 +1,52 @@ +package org.heigit.ohsome.oshdb.util.celliterator; + +import com.google.common.collect.Streams; +import java.util.stream.Stream; +import org.heigit.ohsome.oshdb.OSHDBBoundingBox; +import org.heigit.ohsome.oshdb.grid.GridOSHEntity; +import org.heigit.ohsome.oshdb.index.XYGrid; +import org.heigit.ohsome.oshdb.osh.OSHEntity; +import org.heigit.ohsome.oshdb.util.CellId; + +/** + * A source of OSH entities. + */ +public interface OSHEntitySource { + + /** + * Returns a stream of OSH entities. + * + * @return a stream of OSH entities + */ + Stream<? extends OSHEntity> getData(); + + /** + * Returns the bounding box of the entities returned by `getData()`. + * + * <p>By convention this bbox must contain the bboxes of all OSH entities returned by this source. + * It should be the minimal bbox encompassing all of the entities.</p> + * + * @return A bounding box enclosing all OSH entities of this source + */ + OSHDBBoundingBox getBoundingBox(); + + /** + * A helper method which transforms a grid cell to an OSH entity source. + * + * @param cell A grid cell containing OSH entities + * @return A source object which will return the entities of the given grid cell and its bbox + */ + static OSHEntitySource fromGridOSHEntity(GridOSHEntity cell) { + return new OSHEntitySource() { + @Override + public Stream<? extends OSHEntity> getData() { + return Streams.stream(cell.getEntities()); + } + + @Override + public OSHDBBoundingBox getBoundingBox() { + return XYGrid.getBoundingBox(new CellId(cell.getLevel(), cell.getId()), true); + } + }; + } +} diff --git a/oshdb-util/src/main/java/org/heigit/ohsome/oshdb/util/geometry/OSHDBGeometryBuilder.java b/oshdb-util/src/main/java/org/heigit/ohsome/oshdb/util/geometry/OSHDBGeometryBuilder.java index be17271b9..c889f0169 100644 --- a/oshdb-util/src/main/java/org/heigit/ohsome/oshdb/util/geometry/OSHDBGeometryBuilder.java +++ b/oshdb-util/src/main/java/org/heigit/ohsome/oshdb/util/geometry/OSHDBGeometryBuilder.java @@ -80,15 +80,13 @@ public static Geometry getGeometry( "cannot produce geometry of entity for timestamp before this entity's version's timestamp" ); } - if (entity instanceof OSMNode) { - OSMNode node = (OSMNode) entity; + if (entity instanceof OSMNode node) { if (node.isVisible()) { return geometryFactory.createPoint(new Coordinate(node.getLongitude(), node.getLatitude())); } else { return geometryFactory.createPoint((Coordinate) null); } - } else if (entity instanceof OSMWay) { - OSMWay way = (OSMWay) entity; + } else if (entity instanceof OSMWay way) { if (!way.isVisible()) { LOG.info("way/{} is deleted - falling back to empty (line) geometry", way.getId()); return geometryFactory.createLineString((CoordinateSequence) null); diff --git a/oshdb-util/src/main/java/org/heigit/ohsome/oshdb/util/mappable/OSMEntitySnapshot.java b/oshdb-util/src/main/java/org/heigit/ohsome/oshdb/util/mappable/OSMEntitySnapshot.java index 8048c0d98..b07f8c516 100644 --- a/oshdb-util/src/main/java/org/heigit/ohsome/oshdb/util/mappable/OSMEntitySnapshot.java +++ b/oshdb-util/src/main/java/org/heigit/ohsome/oshdb/util/mappable/OSMEntitySnapshot.java @@ -16,6 +16,13 @@ public interface OSMEntitySnapshot extends OSHDBMapReducible, Comparable<OSMEnti */ OSHDBTimestamp getTimestamp(); + /** + * The timestamp when the entity of the snapshot was last modified before the snapshot timestamp. + * + * @return last modification timestamp as an OSHDBTimestamp object + */ + OSHDBTimestamp getLastContributionTimestamp(); + /** * The geometry of this entity at the snapshot's timestamp clipped to the requested area of * interest. diff --git a/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByContributionNodesTest.java b/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByContributionNodesTest.java index 2f0e48ce2..5726139aa 100644 --- a/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByContributionNodesTest.java +++ b/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByContributionNodesTest.java @@ -8,9 +8,7 @@ import java.io.IOException; import java.util.EnumSet; import java.util.List; -import java.util.stream.Collectors; import org.heigit.ohsome.oshdb.OSHDBBoundingBox; -import org.heigit.ohsome.oshdb.grid.GridOSHEntity; import org.heigit.ohsome.oshdb.grid.GridOSHNodes; import org.heigit.ohsome.oshdb.index.XYGrid; import org.heigit.ohsome.oshdb.util.celliterator.CellIterator.IterateAllEntry; @@ -27,7 +25,7 @@ import org.locationtech.jts.geom.Polygon; /** - * Tests the {@link CellIterator#iterateByContribution(GridOSHEntity)} method on nodes. + * Tests the {@link CellIterator#iterateByContribution(OSHEntitySource)} method on nodes. */ class IterateByContributionNodesTest { private final GridOSHNodes oshdbDataGridCell; @@ -59,29 +57,29 @@ void testGeometryChange() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(1).activities.get() + result.get(1).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(2).activities.get() + result.get(2).activities().get() ); - assertEquals(1, result.get(0).changeset); - assertNull(result.get(0).previousGeometry.get()); - Geometry geom = result.get(0).geometry.get(); + assertEquals(1, result.get(0).changeset()); + assertNull(result.get(0).previousGeometry().get()); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof Point); - assertEquals(result.get(0).geometry.get(), result.get(1).previousGeometry.get()); - assertNotEquals(result.get(1).geometry.get(), result.get(1).previousGeometry.get()); - assertEquals(result.get(1).osmEntity.getTags(), result.get(0).osmEntity.getTags()); + assertEquals(result.get(0).geometry().get(), result.get(1).previousGeometry().get()); + assertNotEquals(result.get(1).geometry().get(), result.get(1).previousGeometry().get()); + assertEquals(result.get(1).osmEntity().getTags(), result.get(0).osmEntity().getTags()); } @Test @@ -99,25 +97,25 @@ void testTagChange() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.TAG_CHANGE), - result.get(1).activities.get() + result.get(1).activities().get() ); assertEquals( EnumSet.of(ContributionType.TAG_CHANGE), - result.get(2).activities.get() + result.get(2).activities().get() ); - assertEquals(3, result.get(0).changeset); - assertNotEquals(result.get(1).osmEntity.getTags(), result.get(0).osmEntity.getTags()); - assertNotEquals(result.get(2).osmEntity.getTags(), result.get(1).osmEntity.getTags()); + assertEquals(3, result.get(0).changeset()); + assertNotEquals(result.get(1).osmEntity().getTags(), result.get(0).osmEntity().getTags()); + assertNotEquals(result.get(2).osmEntity().getTags(), result.get(1).osmEntity().getTags()); } @Test @@ -135,31 +133,31 @@ void testVisibleChange() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(5, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.DELETION), - result.get(1).activities.get() + result.get(1).activities().get() ); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(2).activities.get() + result.get(2).activities().get() ); assertEquals( EnumSet.of(ContributionType.DELETION), - result.get(3).activities.get() + result.get(3).activities().get() ); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(4).activities.get() + result.get(4).activities().get() ); - assertEquals(6, result.get(0).changeset); + assertEquals(6, result.get(0).changeset()); } @Test @@ -182,39 +180,39 @@ void testMultipleChanges() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(6, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.TAG_CHANGE, ContributionType.GEOMETRY_CHANGE), - result.get(1).activities.get() + result.get(1).activities().get() ); assertEquals( EnumSet.of(ContributionType.DELETION), - result.get(2).activities.get() + result.get(2).activities().get() ); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(3).activities.get() + result.get(3).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(4).activities.get() + result.get(4).activities().get() ); assertEquals( EnumSet.of(ContributionType.TAG_CHANGE), - result.get(5).activities.get() + result.get(5).activities().get() ); - assertEquals(11, result.get(0).changeset); - assertNotEquals(result.get(1).osmEntity.getTags(), result.get(0).osmEntity.getTags()); - assertNotEquals(result.get(3).osmEntity.getTags(), result.get(1).osmEntity.getTags()); - assertEquals(result.get(4).osmEntity.getTags(), result.get(3).osmEntity.getTags()); - assertNotEquals(result.get(5).osmEntity.getTags(), result.get(4).osmEntity.getTags()); + assertEquals(11, result.get(0).changeset()); + assertNotEquals(result.get(1).osmEntity().getTags(), result.get(0).osmEntity().getTags()); + assertNotEquals(result.get(3).osmEntity().getTags(), result.get(1).osmEntity().getTags()); + assertEquals(result.get(4).osmEntity().getTags(), result.get(3).osmEntity().getTags()); + assertNotEquals(result.get(5).osmEntity().getTags(), result.get(4).osmEntity().getTags()); } @Test @@ -232,8 +230,8 @@ void testBboxMinAndMaxNotCorrect() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertTrue(result.isEmpty()); } @@ -252,8 +250,8 @@ void testBboxMinExactlyAtDataMinMaxExcluded() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertTrue(result.isEmpty()); } @@ -272,8 +270,8 @@ void testBboxMaxExactlyAtDataMaxMinExcluded() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertTrue(result.isEmpty()); } @@ -292,8 +290,8 @@ void testBboxMinMaxExactlyAtDataMinMax() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); } @@ -312,25 +310,25 @@ void testTagChangeTagFilterWithSuccess() { osmEntity -> osmEntity.getTags().hasTagKey(osmXmlTestData.keys().get("shop")), false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(4, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.DELETION), - result.get(1).activities.get() + result.get(1).activities().get() ); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(2).activities.get() + result.get(2).activities().get() ); assertEquals( EnumSet.of(ContributionType.DELETION), - result.get(3).activities.get() + result.get(3).activities().get() ); } @@ -348,21 +346,21 @@ void testTagChangeTagFilterDisused() { osmEntity -> osmEntity.getTags().hasTagKey(osmXmlTestData.keys().get("disused:shop")), false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.DELETION), - result.get(1).activities.get() + result.get(1).activities().get() ); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(2).activities.get() + result.get(2).activities().get() ); } @@ -380,28 +378,28 @@ void testMoreComplicatedFilter() { osmEntity -> osmEntity.getTags().hasTagKey(osmXmlTestData.keys().get("shop")), false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(4, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.DELETION), - result.get(1).activities.get() + result.get(1).activities().get() ); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(2).activities.get() + result.get(2).activities().get() ); } @Test void testTagChangeTagFilterWithoutSuccess() { // check if results are correct if we filter for a special tag - // tag not in data + // case: tag not in data List<IterateAllEntry> result = (new CellIterator( new OSHDBTimestamps( "2000-01-01T00:00:00Z", @@ -410,11 +408,12 @@ void testTagChangeTagFilterWithoutSuccess() { OSHDBBoundingBox.bboxWgs84Coordinates(-180.0, -90.0, 180.0, 90.0), areaDecider, oshEntity -> oshEntity.getId() == 5, - osmEntity -> osmEntity.getTags().hasTagKey(osmXmlTestData.keys().getOrDefault("amenity", -1)), + osmEntity -> osmEntity.getTags().hasTagKey( + osmXmlTestData.keys().getOrDefault("amenity", -1)), false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertTrue(result.isEmpty()); } @@ -441,8 +440,8 @@ void testPolygonIntersectingDataPartly() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(2, result.size()); } @@ -471,8 +470,8 @@ void testTagFilterAndPolygonIntersectingDataPartly() { osmEntity -> osmEntity.getTags().hasTagKey(osmXmlTestData.keys().get("shop")), false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); // result size =2 because if tag filtered for disappears it's a deletion assertEquals(2, result.size()); // one version with tag shop } @@ -498,11 +497,13 @@ void testCoordinatesRelativeToPolygon() throws IOException { oshEntity -> oshEntity.getId() >= 10 && oshEntity.getId() < 20, osmEntity -> true, false - )).iterateByContribution(GridOSHFactory.getGridOSHNodes(osmXmlTestData, 6, (new XYGrid(6)) - .getId(1.0, 1.0)/* approx. 0, 0, 5.6, 5.6*/)).collect(Collectors.toList()); + )).iterateByContribution(OSHEntitySource.fromGridOSHEntity( + GridOSHFactory.getGridOSHNodes(osmXmlTestData, 6, (new XYGrid(6)) + .getId(1.0, 1.0)/* approx. 0, 0, 5.6, 5.6*/) + )).toList(); assertEquals(2, result.size()); - assertEquals(13, result.get(0).osmEntity.getId()); - assertEquals(14, result.get(1).osmEntity.getId()); + assertEquals(13, result.get(0).osmEntity().getId()); + assertEquals(14, result.get(1).osmEntity().getId()); } } diff --git a/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByContributionNotOsmTypeSpecificTest.java b/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByContributionNotOsmTypeSpecificTest.java index 2a2d7b5ae..275097414 100644 --- a/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByContributionNotOsmTypeSpecificTest.java +++ b/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByContributionNotOsmTypeSpecificTest.java @@ -7,8 +7,6 @@ import java.io.IOException; import java.util.Collections; import java.util.List; -import java.util.stream.Collectors; -import org.heigit.ohsome.oshdb.grid.GridOSHEntity; import org.heigit.ohsome.oshdb.grid.GridOSHRelations; import org.heigit.ohsome.oshdb.osh.OSHRelation; import org.heigit.ohsome.oshdb.util.celliterator.CellIterator.IterateAllEntry; @@ -23,8 +21,8 @@ import org.locationtech.jts.geom.Polygon; /** - * Tests the {@link CellIterator#iterateByContribution(GridOSHEntity)} method on special situations - * which are related to OSHDB grid cells. + * Tests the {@link CellIterator#iterateByContribution(OSHEntitySource)} method on special + * situations which are related to OSHDB grid cells. */ class IterateByContributionNotOsmTypeSpecificTest { @@ -68,8 +66,8 @@ void testCellOutsidePolygon() throws IOException { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertTrue(resultPoly.isEmpty()); } @@ -98,8 +96,8 @@ void testCellCoveringPolygon() throws IOException { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertTrue(resultPoly.isEmpty()); } @@ -129,8 +127,8 @@ void testCellFullyInsidePolygon() throws IOException { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertFalse(resultPoly.isEmpty()); } diff --git a/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByContributionRelationsTest.java b/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByContributionRelationsTest.java index a176edb1c..4590b6915 100644 --- a/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByContributionRelationsTest.java +++ b/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByContributionRelationsTest.java @@ -3,6 +3,7 @@ import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.IOException; @@ -10,7 +11,6 @@ import java.util.List; import java.util.stream.Collectors; import org.heigit.ohsome.oshdb.OSHDBBoundingBox; -import org.heigit.ohsome.oshdb.grid.GridOSHEntity; import org.heigit.ohsome.oshdb.grid.GridOSHRelations; import org.heigit.ohsome.oshdb.util.celliterator.CellIterator.IterateAllEntry; import org.heigit.ohsome.oshdb.util.celliterator.helpers.GridOSHFactory; @@ -27,7 +27,7 @@ import org.locationtech.jts.geom.Polygon; /** - * Tests the {@link CellIterator#iterateByContribution(GridOSHEntity)} method on relations. + * Tests the {@link CellIterator#iterateByContribution(OSHEntitySource)} method on relations. */ class IterateByContributionRelationsTest { private GridOSHRelations oshdbDataGridCell; @@ -68,31 +68,31 @@ void testGeometryChange() { osmEntity -> true, // osmEntityFilter: true -> get all false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); // one creation and two geometry changes should give a result with 3 elements assertEquals(3, result.size()); // check if the contribution types are correct assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(1).activities.get() + result.get(1).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(2).activities.get() + result.get(2).activities().get() ); // check if changeset number si correct - assertEquals(300, result.get(0).changeset); + assertEquals(300, result.get(0).changeset()); // check if geometry types of in every contribution are correct - Geometry geom = result.get(0).geometry.get(); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof MultiPolygon); - Geometry geom3 = result.get(1).geometry.get(); + Geometry geom3 = result.get(1).geometry().get(); assertTrue(geom3 instanceof MultiPolygon); - Geometry geom4 = result.get(2).geometry.get(); + Geometry geom4 = result.get(2).geometry().get(); assertTrue(geom4 instanceof MultiPolygon); } @@ -111,23 +111,23 @@ void testVisibleChange() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.DELETION), - result.get(1).activities.get() + result.get(1).activities().get() ); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(2).activities.get() + result.get(2).activities().get() ); - assertEquals(303, result.get(0).changeset); + assertEquals(303, result.get(0).changeset()); } @Test @@ -145,7 +145,7 @@ void testWaysNotExistent() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) ).collect(Collectors.toList()); }); } @@ -164,23 +164,23 @@ void testTagChange() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.TAG_CHANGE), - result.get(1).activities.get() + result.get(1).activities().get() ); assertEquals( EnumSet.of(ContributionType.TAG_CHANGE), - result.get(2).activities.get() + result.get(2).activities().get() ); - assertEquals(307, result.get(0).changeset); + assertEquals(307, result.get(0).changeset()); } @Test @@ -198,29 +198,29 @@ void testGeometryChangeOfNodeRefsInWays() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(8, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(1).activities.get() + result.get(1).activities().get() ); - assertEquals(310, result.get(0).changeset); + assertEquals(310, result.get(0).changeset()); - assertEquals(null, result.get(0).previousGeometry.get()); - Geometry geom = result.get(0).geometry.get(); + assertNull(result.get(0).previousGeometry().get()); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof Polygon); - Geometry geom3 = result.get(1).geometry.get(); + Geometry geom3 = result.get(1).geometry().get(); assertTrue(geom3 instanceof Polygon); - assertNotEquals(result.get(1).geometry.get(), result.get(1).previousGeometry.get()); - assertEquals(result.get(2).geometry.get(), result.get(2).previousGeometry.get()); + assertNotEquals(result.get(1).geometry().get(), result.get(1).previousGeometry().get()); + assertEquals(result.get(2).geometry().get(), result.get(2).previousGeometry().get()); } @Test @@ -238,28 +238,28 @@ void testGeometryChangeOfNodeCoordinatesInWay() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(1).activities.get() + result.get(1).activities().get() ); - assertEquals(312, result.get(0).changeset); + assertEquals(312, result.get(0).changeset()); - assertEquals(null, result.get(0).previousGeometry.get()); - Geometry geom = result.get(0).geometry.get(); + assertNull(result.get(0).previousGeometry().get()); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof Polygon); - Geometry geom3 = result.get(1).geometry.get(); + Geometry geom3 = result.get(1).geometry().get(); assertTrue(geom3 instanceof Polygon); - assertNotEquals(result.get(1).geometry.get(), result.get(1).previousGeometry.get()); - assertNotEquals(result.get(2).geometry.get(), result.get(2).previousGeometry.get()); + assertNotEquals(result.get(1).geometry().get(), result.get(1).previousGeometry().get()); + assertNotEquals(result.get(2).geometry().get(), result.get(2).previousGeometry().get()); } @Test @@ -277,29 +277,29 @@ void testGeometryChangeOfNodeCoordinatesInRelationAndWay() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(1).activities.get() + result.get(1).activities().get() ); - assertEquals(313, result.get(0).changeset); + assertEquals(313, result.get(0).changeset()); - assertEquals(null, result.get(0).previousGeometry.get()); - Geometry geom = result.get(0).geometry.get(); + assertNull(result.get(0).previousGeometry().get()); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof Polygon); - Geometry geom3 = result.get(1).geometry.get(); + Geometry geom3 = result.get(1).geometry().get(); assertTrue(geom3 instanceof Polygon); - assertNotEquals(result.get(1).geometry.get(), result.get(1).previousGeometry.get()); - assertEquals(result.get(2).geometry.get(), result.get(2).previousGeometry.get()); + assertNotEquals(result.get(1).geometry().get(), result.get(1).previousGeometry().get()); + assertEquals(result.get(2).geometry().get(), result.get(2).previousGeometry().get()); } @Test @@ -316,25 +316,26 @@ void testGeometryCollection() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); - assertEquals(314, result.get(0).changeset); - assertEquals(null, result.get(0).previousGeometry.get()); - Geometry geom = result.get(0).geometry.get(); + assertEquals(314, result.get(0).changeset()); + assertNull(result.get(0).previousGeometry().get()); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof GeometryCollection); - Geometry geom1 = result.get(1).geometry.get(); + Geometry geom1 = result.get(1).geometry().get(); assertTrue(geom1 instanceof GeometryCollection); - Geometry geom2 = result.get(2).geometry.get(); + Geometry geom2 = result.get(2).geometry().get(); assertTrue(geom2 instanceof GeometryCollection); } + @SuppressWarnings("ResultOfMethodCallIgnored") @Test void testNodesOfWaysNotExistent() { // relation 2 way members nodes do not exist @@ -350,8 +351,8 @@ void testNodesOfWaysNotExistent() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); }); } @@ -369,47 +370,47 @@ void testVisibleChangeOfNodeInWay() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(5, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(1).activities.get() + result.get(1).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(2).activities.get() + result.get(2).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(3).activities.get() + result.get(3).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(4).activities.get() + result.get(4).activities().get() ); - assertEquals(316, result.get(0).changeset); + assertEquals(316, result.get(0).changeset()); - assertEquals(null, result.get(0).previousGeometry.get()); - Geometry geom = result.get(0).geometry.get(); + assertNull(result.get(0).previousGeometry().get()); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof Polygon); - Geometry geom2 = result.get(1).geometry.get(); + Geometry geom2 = result.get(1).geometry().get(); assertTrue(geom2 instanceof Polygon); - Geometry geom3 = result.get(2).geometry.get(); + Geometry geom3 = result.get(2).geometry().get(); assertTrue(geom3 instanceof Polygon); - Geometry geom4 = result.get(3).geometry.get(); + Geometry geom4 = result.get(3).geometry().get(); assertTrue(geom4 instanceof Polygon); - Geometry geom5 = result.get(4).geometry.get(); + Geometry geom5 = result.get(4).geometry().get(); assertTrue(geom5 instanceof Polygon); - assertNotEquals(result.get(1).geometry.get(), result.get(1).previousGeometry.get()); - assertNotEquals(result.get(2).geometry.get(), result.get(2).previousGeometry.get()); + assertNotEquals(result.get(1).geometry().get(), result.get(1).previousGeometry().get()); + assertNotEquals(result.get(2).geometry().get(), result.get(2).previousGeometry().get()); } @Test @@ -426,17 +427,17 @@ void testTagChangeOfNodeInWay() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(4, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); - assertEquals(317, result.get(0).changeset); - assertEquals(null, result.get(0).previousGeometry.get()); + assertEquals(317, result.get(0).changeset()); + assertNull(result.get(0).previousGeometry().get()); } @Test @@ -453,26 +454,26 @@ void testVisibleChangeOfWay() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(1).activities.get() + result.get(1).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(2).activities.get() + result.get(2).activities().get() ); - assertEquals(318, result.get(0).changeset); - assertEquals(null, result.get(0).previousGeometry.get()); - assertTrue(result.get(1).geometry.get().isEmpty()); + assertEquals(318, result.get(0).changeset()); + assertNull(result.get(0).previousGeometry().get()); + assertTrue(result.get(1).geometry().get().isEmpty()); } @Test @@ -490,24 +491,24 @@ void testVisibleChangeOfOneWayOfOuterRing() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(2, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(1).activities.get() + result.get(1).activities().get() ); - Geometry geom = result.get(0).geometry.get(); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof Polygon); - Geometry geom2 = result.get(1).geometry.get(); + Geometry geom2 = result.get(1).geometry().get(); assertTrue(geom2 instanceof GeometryCollection); - assertEquals(319, result.get(0).changeset); - assertEquals(null, result.get(0).previousGeometry.get()); + assertEquals(319, result.get(0).changeset()); + assertNull(result.get(0).previousGeometry().get()); } @Test @@ -524,20 +525,20 @@ void testTagChangeOfWay() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); - Geometry geom = result.get(0).geometry.get(); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof Polygon); - Geometry geom2 = result.get(2).geometry.get(); + Geometry geom2 = result.get(2).geometry().get(); assertTrue(geom2 instanceof Polygon); - assertEquals(320, result.get(0).changeset); - assertEquals(null, result.get(0).previousGeometry.get()); + assertEquals(320, result.get(0).changeset()); + assertNull(result.get(0).previousGeometry().get()); } @Test @@ -554,25 +555,25 @@ void testOneOfTwoPolygonDisappears() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(2, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(1).activities.get() + result.get(1).activities().get() ); - Geometry geom = result.get(0).geometry.get(); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof MultiPolygon); - Geometry geom2 = result.get(1).geometry.get(); + Geometry geom2 = result.get(1).geometry().get(); assertTrue(geom2 instanceof Polygon); - assertEquals(321, result.get(0).changeset); - assertEquals(null, result.get(0).previousGeometry.get()); - assertNotEquals(result.get(1).geometry.get(), result.get(1).previousGeometry.get()); + assertEquals(321, result.get(0).changeset()); + assertNull(result.get(0).previousGeometry().get()); + assertNotEquals(result.get(1).geometry().get(), result.get(1).previousGeometry().get()); } @Test @@ -589,30 +590,30 @@ void testWaySplitUpInTwo() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(1).activities.get() + result.get(1).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(2).activities.get() + result.get(2).activities().get() ); - Geometry geom = result.get(0).geometry.get(); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof Polygon); - Geometry geom1 = result.get(1).geometry.get(); + Geometry geom1 = result.get(1).geometry().get(); assertTrue(geom1 instanceof GeometryCollection); - Geometry geom2 = result.get(2).geometry.get(); + Geometry geom2 = result.get(2).geometry().get(); assertTrue(geom2 instanceof Polygon); - assertEquals(323, result.get(0).changeset); - assertEquals(null, result.get(0).previousGeometry.get()); - assertNotEquals(result.get(1).geometry.get(), result.get(1).previousGeometry.get()); + assertEquals(323, result.get(0).changeset()); + assertNull(result.get(0).previousGeometry().get()); + assertNotEquals(result.get(1).geometry().get(), result.get(1).previousGeometry().get()); } @@ -639,8 +640,8 @@ void testPolygonIntersectingDataPartly() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); } @@ -667,8 +668,8 @@ void testPolygonIntersectingDataOnlyAtBorderLine() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertTrue(result.isEmpty()); } @@ -695,8 +696,8 @@ void testPolygonIntersectingDataCompletely() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); } @@ -723,8 +724,8 @@ void testPolygonNotIntersectingData() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertTrue(result.isEmpty()); } @@ -744,18 +745,18 @@ void testNodeChangeOutsideBbox() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); - assertTrue(result.get(1).activities.get().isEmpty()); + assertTrue(result.get(1).activities().get().isEmpty()); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(2).activities.get() + result.get(2).activities().get() ); - assertEquals(7, result.get(1).geometry.get().getNumPoints()); + assertEquals(7, result.get(1).geometry().get().getNumPoints()); } @Test @@ -781,8 +782,8 @@ void testPolygonIntersectingDataCompletelyTimeIntervalAfterChanges() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertTrue(result.isEmpty()); } @@ -800,8 +801,8 @@ void testTimeIntervalAfterChanges() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertTrue(result.isEmpty()); } @@ -821,8 +822,8 @@ void testBboxOutsidePolygon() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertTrue(result.isEmpty()); } @@ -844,20 +845,20 @@ void testUnclippedGeom() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); // full geom of same timestamp with unclippedPreviousGeometry and unclippedGeometry - assertEquals(result.get(1).unclippedPreviousGeometry.get().getArea(), - result.get(0).unclippedGeometry.get().getArea(), DELTA); + assertEquals(result.get(1).unclippedPreviousGeometry().get().getArea(), + result.get(0).unclippedGeometry().get().getArea(), DELTA); // geom of requested area vs full geom after modification - assertNotEquals(result.get(0).geometry.get().getArea(), - result.get(0).unclippedGeometry.get().getArea()); + assertNotEquals(result.get(0).geometry().get().getArea(), + result.get(0).unclippedGeometry().get().getArea()); // full geom changed - assertNotEquals(result.get(1).unclippedGeometry.get().getArea(), - result.get(0).unclippedGeometry.get().getArea()); - assertNotEquals(result.get(1).unclippedGeometry.get().getArea(), - result.get(2).unclippedGeometry.get().getArea()); + assertNotEquals(result.get(1).unclippedGeometry().get().getArea(), + result.get(0).unclippedGeometry().get().getArea()); + assertNotEquals(result.get(1).unclippedGeometry().get().getArea(), + result.get(2).unclippedGeometry().get().getArea()); } @@ -886,8 +887,8 @@ void testSelfIntersectingPolygonClipped() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertTrue(result.isEmpty()); } @@ -906,15 +907,15 @@ void testMembersDisappear() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(4, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); - assertTrue(result.get(3).geometry.get().isEmpty()); + assertTrue(result.get(3).geometry().get().isEmpty()); } @Test @@ -931,12 +932,12 @@ void testTimeIntervalAfterDeletionInVersion2() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(1, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); } @@ -955,12 +956,12 @@ void testTimeIntervalAfterDeletionInCurrentVersion() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(1, result.size()); assertEquals( EnumSet.of(ContributionType.DELETION), - result.get(0).activities.get() + result.get(0).activities().get() ); } @@ -979,13 +980,13 @@ void testExcludingVersion2() { osmEntity -> !(osmEntity.getVersion() == 2), false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(1, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); } @@ -1014,17 +1015,17 @@ void testMembersDisappearClipped() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(4, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.DELETION), - result.get(3).activities.get() + result.get(3).activities().get() ); } @@ -1052,13 +1053,13 @@ void testTimeIntervalAfterDeletionInVersion2Clipped() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(1, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); } @@ -1086,13 +1087,13 @@ void testTimeIntervalAfterDeletionInCurrentVersionClipped() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(1, result.size()); assertEquals( EnumSet.of(ContributionType.DELETION), - result.get(0).activities.get() + result.get(0).activities().get() ); } @@ -1119,13 +1120,13 @@ void testExcludingVersion2Clipped() { osmEntity -> !(osmEntity.getVersion() == 2), false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(1, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); } @@ -1153,8 +1154,8 @@ void testClippingPolygonIsVeryBig() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); } diff --git a/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByContributionTest.java b/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByContributionTest.java index 4b9673aad..f2a762922 100644 --- a/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByContributionTest.java +++ b/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByContributionTest.java @@ -5,7 +5,6 @@ import java.io.ObjectInputStream; import java.util.List; import java.util.TreeSet; -import java.util.stream.Collectors; import org.h2.jdbcx.JdbcConnectionPool; import org.heigit.ohsome.oshdb.OSHDBBoundingBox; import org.heigit.ohsome.oshdb.OSHDBTimestamp; @@ -19,7 +18,7 @@ import org.junit.jupiter.api.Test; /** - * Tests the {@link CellIterator#iterateByContribution(GridOSHEntity)} method. + * Tests the {@link CellIterator#iterateByContribution(OSHEntitySource)} method. */ class IterateByContributionTest { private static JdbcConnectionPool source; @@ -73,11 +72,11 @@ void testIssue108() throws Exception { osmEntity -> true, false )).iterateByContribution( - oshCellRawData - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshCellRawData) + ).toList(); countTotal += result.size(); for (IterateAllEntry entry : result) { - if (entry.activities.contains(ContributionType.CREATION)) { + if (entry.activities().contains(ContributionType.CREATION)) { countCreated++; } else { countOther++; diff --git a/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByContributionTypeNotMultipolygonTest.java b/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByContributionTypeNotMultipolygonTest.java index 7f8c7556e..6e6445e6d 100644 --- a/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByContributionTypeNotMultipolygonTest.java +++ b/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByContributionTypeNotMultipolygonTest.java @@ -3,14 +3,13 @@ import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.IOException; import java.util.EnumSet; import java.util.List; -import java.util.stream.Collectors; import org.heigit.ohsome.oshdb.OSHDBBoundingBox; -import org.heigit.ohsome.oshdb.grid.GridOSHEntity; import org.heigit.ohsome.oshdb.grid.GridOSHRelations; import org.heigit.ohsome.oshdb.util.celliterator.CellIterator.IterateAllEntry; import org.heigit.ohsome.oshdb.util.celliterator.helpers.GridOSHFactory; @@ -26,7 +25,7 @@ import org.locationtech.jts.geom.Polygon; /** - * Tests the {@link CellIterator#iterateByContribution(GridOSHEntity)} method on relations except + * Tests the {@link CellIterator#iterateByContribution(OSHEntitySource)} method on relations except * multipolygon relations. */ class IterateByContributionTypeNotMultipolygonTest { @@ -59,28 +58,28 @@ void testGeometryChange() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(1).activities.get() + result.get(1).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(2).activities.get() + result.get(2).activities().get() ); - assertEquals(300, result.get(0).changeset); - Geometry geom = result.get(0).geometry.get(); + assertEquals(300, result.get(0).changeset()); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof GeometryCollection); - Geometry geom3 = result.get(1).geometry.get(); + Geometry geom3 = result.get(1).geometry().get(); assertTrue(geom3 instanceof GeometryCollection); - Geometry geom4 = result.get(2).geometry.get(); + Geometry geom4 = result.get(2).geometry().get(); assertTrue(geom4 instanceof GeometryCollection); } @@ -99,25 +98,26 @@ void testVisibleChange() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.DELETION), - result.get(1).activities.get() + result.get(1).activities().get() ); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(2).activities.get() + result.get(2).activities().get() ); - assertEquals(303, result.get(0).changeset); + assertEquals(303, result.get(0).changeset()); } + @SuppressWarnings("ResultOfMethodCallIgnored") @Test void testWaysNotExistent() { // relation with two ways, both missing @@ -133,8 +133,8 @@ void testWaysNotExistent() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); }); } @@ -152,23 +152,23 @@ void testTagChange() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.TAG_CHANGE), - result.get(1).activities.get() + result.get(1).activities().get() ); assertEquals( EnumSet.of(ContributionType.TAG_CHANGE), - result.get(2).activities.get() + result.get(2).activities().get() ); - assertEquals(307, result.get(0).changeset); + assertEquals(307, result.get(0).changeset()); } @Test @@ -186,29 +186,29 @@ void testGeometryChangeOfNodeRefsInWays() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(8, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(1).activities.get() + result.get(1).activities().get() ); - assertEquals(310, result.get(0).changeset); + assertEquals(310, result.get(0).changeset()); - assertEquals(null, result.get(0).previousGeometry.get()); - Geometry geom = result.get(0).geometry.get(); + assertNull(result.get(0).previousGeometry().get()); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof GeometryCollection); - Geometry geom3 = result.get(1).geometry.get(); + Geometry geom3 = result.get(1).geometry().get(); assertTrue(geom3 instanceof GeometryCollection); - assertNotEquals(result.get(1).geometry.get(), result.get(1).previousGeometry.get()); - assertEquals(result.get(2).geometry.get(), result.get(2).previousGeometry.get()); + assertNotEquals(result.get(1).geometry().get(), result.get(1).previousGeometry().get()); + assertEquals(result.get(2).geometry().get(), result.get(2).previousGeometry().get()); } @Test @@ -226,29 +226,29 @@ void testGeometryChangeOfNodeCoordinatesInWay() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(1).activities.get() + result.get(1).activities().get() ); - assertEquals(312, result.get(0).changeset); + assertEquals(312, result.get(0).changeset()); - assertEquals(null, result.get(0).previousGeometry.get()); - Geometry geom = result.get(0).geometry.get(); + assertNull(result.get(0).previousGeometry().get()); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof GeometryCollection); - Geometry geom3 = result.get(1).geometry.get(); + Geometry geom3 = result.get(1).geometry().get(); assertTrue(geom3 instanceof GeometryCollection); - assertNotEquals(result.get(1).geometry.get(), result.get(1).previousGeometry.get()); - assertNotEquals(result.get(2).geometry.get(), result.get(2).previousGeometry.get()); + assertNotEquals(result.get(1).geometry().get(), result.get(1).previousGeometry().get()); + assertNotEquals(result.get(2).geometry().get(), result.get(2).previousGeometry().get()); } @Test @@ -266,29 +266,29 @@ void testGeometryChangeOfNodeCoordinatesInRelationAndWay() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(1).activities.get() + result.get(1).activities().get() ); - assertEquals(313, result.get(0).changeset); + assertEquals(313, result.get(0).changeset()); - assertEquals(null, result.get(0).previousGeometry.get()); - Geometry geom = result.get(0).geometry.get(); + assertNull(result.get(0).previousGeometry().get()); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof GeometryCollection); - Geometry geom3 = result.get(1).geometry.get(); + Geometry geom3 = result.get(1).geometry().get(); assertTrue(geom3 instanceof GeometryCollection); - assertNotEquals(result.get(1).geometry.get(), result.get(1).previousGeometry.get()); - assertNotEquals(result.get(2).geometry.get(), result.get(2).previousGeometry.get()); + assertNotEquals(result.get(1).geometry().get(), result.get(1).previousGeometry().get()); + assertNotEquals(result.get(2).geometry().get(), result.get(2).previousGeometry().get()); } @Test @@ -305,25 +305,26 @@ void testGeometryCollection() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); - assertEquals(314, result.get(0).changeset); - assertEquals(null, result.get(0).previousGeometry.get()); - Geometry geom = result.get(0).geometry.get(); + assertEquals(314, result.get(0).changeset()); + assertNull(result.get(0).previousGeometry().get()); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof GeometryCollection); - Geometry geom1 = result.get(1).geometry.get(); + Geometry geom1 = result.get(1).geometry().get(); assertTrue(geom1 instanceof GeometryCollection); - Geometry geom2 = result.get(2).geometry.get(); + Geometry geom2 = result.get(2).geometry().get(); assertTrue(geom2 instanceof GeometryCollection); } + @SuppressWarnings("ResultOfMethodCallIgnored") @Test void testNodesOfWaysNotExistent() { // relation 2 way members nodes do not exist @@ -339,8 +340,8 @@ void testNodesOfWaysNotExistent() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); }); } @@ -358,47 +359,47 @@ void testVisibleChangeOfNodeInWay() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(5, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(1).activities.get() + result.get(1).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(2).activities.get() + result.get(2).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(3).activities.get() + result.get(3).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(4).activities.get() + result.get(4).activities().get() ); - assertEquals(316, result.get(0).changeset); + assertEquals(316, result.get(0).changeset()); - assertEquals(null, result.get(0).previousGeometry.get()); - Geometry geom = result.get(0).geometry.get(); + assertNull(result.get(0).previousGeometry().get()); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof GeometryCollection); - Geometry geom2 = result.get(1).geometry.get(); + Geometry geom2 = result.get(1).geometry().get(); assertTrue(geom2 instanceof GeometryCollection); - Geometry geom3 = result.get(2).geometry.get(); + Geometry geom3 = result.get(2).geometry().get(); assertTrue(geom3 instanceof GeometryCollection); - Geometry geom4 = result.get(3).geometry.get(); + Geometry geom4 = result.get(3).geometry().get(); assertTrue(geom4 instanceof GeometryCollection); - Geometry geom5 = result.get(4).geometry.get(); + Geometry geom5 = result.get(4).geometry().get(); assertTrue(geom5 instanceof GeometryCollection); - assertNotEquals(result.get(1).geometry.get(), result.get(1).previousGeometry.get()); - assertNotEquals(result.get(2).geometry.get(), result.get(2).previousGeometry.get()); + assertNotEquals(result.get(1).geometry().get(), result.get(1).previousGeometry().get()); + assertNotEquals(result.get(2).geometry().get(), result.get(2).previousGeometry().get()); } @Test @@ -415,17 +416,17 @@ void testTagChangeOfNodeInWay() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(4, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); - assertEquals(317, result.get(0).changeset); - assertEquals(null, result.get(0).previousGeometry.get()); + assertEquals(317, result.get(0).changeset()); + assertNull(result.get(0).previousGeometry().get()); } @Test @@ -442,17 +443,17 @@ void testVisibleChangeOfWay() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); - assertEquals(318, result.get(0).changeset); - assertEquals(null, result.get(0).previousGeometry.get()); + assertEquals(318, result.get(0).changeset()); + assertNull(result.get(0).previousGeometry().get()); } @Test @@ -470,24 +471,24 @@ void testVisibleChangeOfOneWayOfOuterRing() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(2, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(1).activities.get() + result.get(1).activities().get() ); - Geometry geom = result.get(0).geometry.get(); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof GeometryCollection); - Geometry geom2 = result.get(1).geometry.get(); + Geometry geom2 = result.get(1).geometry().get(); assertTrue(geom2 instanceof GeometryCollection); - assertEquals(319, result.get(0).changeset); - assertEquals(null, result.get(0).previousGeometry.get()); + assertEquals(319, result.get(0).changeset()); + assertNull(result.get(0).previousGeometry().get()); } @Test @@ -504,20 +505,20 @@ void testTagChangeOfWay() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); - Geometry geom = result.get(0).geometry.get(); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof GeometryCollection); - Geometry geom2 = result.get(2).geometry.get(); + Geometry geom2 = result.get(2).geometry().get(); assertTrue(geom2 instanceof GeometryCollection); - assertEquals(320, result.get(0).changeset); - assertEquals(null, result.get(0).previousGeometry.get()); + assertEquals(320, result.get(0).changeset()); + assertNull(result.get(0).previousGeometry().get()); } @Test @@ -534,25 +535,25 @@ void testOneOfTwoPolygonDisappears() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(2, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(1).activities.get() + result.get(1).activities().get() ); - Geometry geom = result.get(0).geometry.get(); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof GeometryCollection); - Geometry geom2 = result.get(1).geometry.get(); + Geometry geom2 = result.get(1).geometry().get(); assertTrue(geom2 instanceof GeometryCollection); - assertEquals(321, result.get(0).changeset); - assertEquals(null, result.get(0).previousGeometry.get()); - assertNotEquals(result.get(1).geometry.get(), result.get(1).previousGeometry.get()); + assertEquals(321, result.get(0).changeset()); + assertNull(result.get(0).previousGeometry().get()); + assertNotEquals(result.get(1).geometry().get(), result.get(1).previousGeometry().get()); } @Test @@ -569,30 +570,30 @@ void testWaySplitUpInTwo() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(1).activities.get() + result.get(1).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(2).activities.get() + result.get(2).activities().get() ); - Geometry geom = result.get(0).geometry.get(); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof GeometryCollection); - Geometry geom1 = result.get(1).geometry.get(); + Geometry geom1 = result.get(1).geometry().get(); assertTrue(geom1 instanceof GeometryCollection); - Geometry geom2 = result.get(2).geometry.get(); + Geometry geom2 = result.get(2).geometry().get(); assertTrue(geom2 instanceof GeometryCollection); - assertEquals(323, result.get(0).changeset); - assertEquals(null, result.get(0).previousGeometry.get()); - assertNotEquals(result.get(1).geometry.get(), result.get(1).previousGeometry.get()); + assertEquals(323, result.get(0).changeset()); + assertNull(result.get(0).previousGeometry().get()); + assertNotEquals(result.get(1).geometry().get(), result.get(1).previousGeometry().get()); } @@ -619,8 +620,8 @@ void testPolygonIntersectingDataPartly() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); } @@ -647,8 +648,8 @@ void testPolygonIntersectingDataCompletely() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); } @@ -675,8 +676,8 @@ void testPolygonNotIntersectingData() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertTrue(result.isEmpty()); } @@ -696,19 +697,19 @@ void testNodeChangeOutsideBbox() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); - assertTrue(result.get(1).activities.get().isEmpty()); + assertTrue(result.get(1).activities().get().isEmpty()); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(2).activities.get() + result.get(2).activities().get() ); - assertEquals(7, result.get(1).geometry.get().getNumPoints()); + assertEquals(7, result.get(1).geometry().get().getNumPoints()); } @Test @@ -734,8 +735,8 @@ void testPolygonIntersectingDataCompletelyTimeIntervalAfterChanges() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertTrue(result.isEmpty()); } @@ -753,8 +754,8 @@ void testTimeIntervalAfterChanges() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertTrue(result.isEmpty()); } @@ -774,8 +775,8 @@ void testBboxOutsidePolygon() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertTrue(result.isEmpty()); } @@ -802,9 +803,9 @@ void testSelfIntersectingPolygonClipped() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); - Geometry geom1 = result.get(0).geometry.get(); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); + Geometry geom1 = result.get(0).geometry().get(); assertTrue(geom1 instanceof GeometryCollection); assertEquals(1, result.size()); } @@ -824,13 +825,13 @@ void testMembersDisappear() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(4, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); } @@ -849,8 +850,8 @@ void testMembersDisappearAndPreviousIsNull() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertTrue(result.isEmpty()); } @@ -877,12 +878,12 @@ void testTimeIntervalAfterDeletionInCurrentVersionClipped() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(1, result.size()); assertEquals( EnumSet.of(ContributionType.DELETION), - result.get(0).activities.get() + result.get(0).activities().get() ); } } diff --git a/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByContributionWaysTest.java b/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByContributionWaysTest.java index 49446019a..94047a5c2 100644 --- a/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByContributionWaysTest.java +++ b/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByContributionWaysTest.java @@ -2,14 +2,13 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.IOException; import java.util.EnumSet; import java.util.List; -import java.util.stream.Collectors; import org.heigit.ohsome.oshdb.OSHDBBoundingBox; -import org.heigit.ohsome.oshdb.grid.GridOSHEntity; import org.heigit.ohsome.oshdb.grid.GridOSHWays; import org.heigit.ohsome.oshdb.util.celliterator.CellIterator.IterateAllEntry; import org.heigit.ohsome.oshdb.util.celliterator.helpers.GridOSHFactory; @@ -23,7 +22,7 @@ import org.locationtech.jts.geom.Polygon; /** - * Tests the {@link CellIterator#iterateByContribution(GridOSHEntity)} method on ways. + * Tests the {@link CellIterator#iterateByContribution(OSHEntitySource)} method on ways. */ class IterateByContributionWaysTest { private GridOSHWays oshdbDataGridCell; @@ -55,38 +54,38 @@ void testGeometryChange() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(1).activities.get() + result.get(1).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(2).activities.get() + result.get(2).activities().get() ); - assertEquals(31, result.get(0).changeset); + assertEquals(31, result.get(0).changeset()); - assertEquals(4, result.get(0).geometry.get().getNumPoints()); - assertEquals(8, result.get(1).geometry.get().getNumPoints()); - assertEquals(9, result.get(2).geometry.get().getNumPoints()); + assertEquals(4, result.get(0).geometry().get().getNumPoints()); + assertEquals(8, result.get(1).geometry().get().getNumPoints()); + assertEquals(9, result.get(2).geometry().get().getNumPoints()); - assertEquals(null, result.get(0).previousGeometry.get()); - Geometry geom = result.get(0).geometry.get(); + assertNull(result.get(0).previousGeometry().get()); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof LineString); - Geometry geom2 = result.get(1).geometry.get(); + Geometry geom2 = result.get(1).geometry().get(); assertTrue(geom2 instanceof LineString); - Geometry geom3 = result.get(2).geometry.get(); + Geometry geom3 = result.get(2).geometry().get(); assertTrue(geom3 instanceof LineString); - assertNotEquals(result.get(1).geometry.get(), result.get(1).previousGeometry.get()); - assertNotEquals(result.get(2).geometry.get(), result.get(2).previousGeometry.get()); + assertNotEquals(result.get(1).geometry().get(), result.get(1).previousGeometry().get()); + assertNotEquals(result.get(2).geometry().get(), result.get(2).previousGeometry().get()); } @Test @@ -104,40 +103,40 @@ void testGeometryChangeOfNodeInWay() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(4, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(1).activities.get() + result.get(1).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(2).activities.get() + result.get(2).activities().get() ); - assertEquals(34, result.get(0).changeset); + assertEquals(34, result.get(0).changeset()); - assertEquals(2, result.get(0).geometry.get().getNumPoints()); - assertEquals(2, result.get(1).geometry.get().getNumPoints()); - assertEquals(3, result.get(3).geometry.get().getNumPoints()); + assertEquals(2, result.get(0).geometry().get().getNumPoints()); + assertEquals(2, result.get(1).geometry().get().getNumPoints()); + assertEquals(3, result.get(3).geometry().get().getNumPoints()); - assertEquals(null, result.get(0).previousGeometry.get()); - Geometry geom = result.get(0).geometry.get(); + assertNull(result.get(0).previousGeometry().get()); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof LineString); - Geometry geom3 = result.get(1).geometry.get(); + Geometry geom3 = result.get(1).geometry().get(); assertTrue(geom3 instanceof LineString); - Geometry geom4 = result.get(3).geometry.get(); + Geometry geom4 = result.get(3).geometry().get(); assertTrue(geom4 instanceof LineString); - assertNotEquals(result.get(1).geometry.get(), result.get(1).previousGeometry.get()); - assertNotEquals(result.get(2).geometry.get(), result.get(2).previousGeometry.get()); - assertNotEquals(result.get(3).geometry.get(), result.get(3).previousGeometry.get()); + assertNotEquals(result.get(1).geometry().get(), result.get(1).previousGeometry().get()); + assertNotEquals(result.get(2).geometry().get(), result.get(2).previousGeometry().get()); + assertNotEquals(result.get(3).geometry().get(), result.get(3).previousGeometry().get()); } @Test @@ -155,23 +154,23 @@ void testVisibleChange() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.DELETION), - result.get(1).activities.get() + result.get(1).activities().get() ); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(2).activities.get() + result.get(2).activities().get() ); - assertEquals(36, result.get(0).changeset); + assertEquals(36, result.get(0).changeset()); } @Test @@ -188,38 +187,38 @@ void testTagChange() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.TAG_CHANGE, ContributionType.GEOMETRY_CHANGE), - result.get(1).activities.get() + result.get(1).activities().get() ); assertEquals( EnumSet.of(ContributionType.TAG_CHANGE), - result.get(2).activities.get() + result.get(2).activities().get() ); - assertEquals(39, result.get(0).changeset); + assertEquals(39, result.get(0).changeset()); - assertEquals(3, result.get(0).geometry.get().getNumPoints()); - assertEquals(5, result.get(1).geometry.get().getNumPoints()); - assertEquals(5, result.get(2).geometry.get().getNumPoints()); + assertEquals(3, result.get(0).geometry().get().getNumPoints()); + assertEquals(5, result.get(1).geometry().get().getNumPoints()); + assertEquals(5, result.get(2).geometry().get().getNumPoints()); - assertEquals(null, result.get(0).previousGeometry.get()); - Geometry geom = result.get(0).geometry.get(); + assertNull(result.get(0).previousGeometry().get()); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof LineString); - Geometry geom2 = result.get(1).geometry.get(); + Geometry geom2 = result.get(1).geometry().get(); assertTrue(geom2 instanceof LineString); - Geometry geom3 = result.get(2).geometry.get(); + Geometry geom3 = result.get(2).geometry().get(); assertTrue(geom3 instanceof LineString); - assertNotEquals(result.get(1).geometry.get(), result.get(1).previousGeometry.get()); - assertEquals(result.get(2).geometry.get(), result.get(2).previousGeometry.get()); + assertNotEquals(result.get(1).geometry().get(), result.get(1).previousGeometry().get()); + assertEquals(result.get(2).geometry().get(), result.get(2).previousGeometry().get()); } @Test @@ -239,33 +238,33 @@ void testMultipleChangesOnNodesOfWay() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(6, result.size()); - assertEquals(2, result.get(0).geometry.get().getNumPoints()); - assertEquals(3, result.get(1).geometry.get().getNumPoints()); - assertEquals(2, result.get(2).geometry.get().getNumPoints()); - assertEquals(3, result.get(3).geometry.get().getNumPoints()); - assertEquals(3, result.get(4).geometry.get().getNumPoints()); + assertEquals(2, result.get(0).geometry().get().getNumPoints()); + assertEquals(3, result.get(1).geometry().get().getNumPoints()); + assertEquals(2, result.get(2).geometry().get().getNumPoints()); + assertEquals(3, result.get(3).geometry().get().getNumPoints()); + assertEquals(3, result.get(4).geometry().get().getNumPoints()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(1).activities.get() + result.get(1).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(2).activities.get() + result.get(2).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(3).activities.get() + result.get(3).activities().get() ); - assertEquals(42, result.get(0).changeset); + assertEquals(42, result.get(0).changeset()); } @@ -283,37 +282,37 @@ void testMultipleChanges() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(6, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.TAG_CHANGE, ContributionType.GEOMETRY_CHANGE), - result.get(1).activities.get() + result.get(1).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(2).activities.get() + result.get(2).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(3).activities.get() + result.get(3).activities().get() ); assertEquals( EnumSet.of(ContributionType.DELETION), - result.get(4).activities.get() + result.get(4).activities().get() ); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(5).activities.get() + result.get(5).activities().get() ); - assertEquals(44, result.get(0).changeset); - assertNotEquals(result.get(1).osmEntity.getTags(), result.get(0).osmEntity.getTags()); - assertEquals(result.get(5).osmEntity.getTags(), result.get(3).osmEntity.getTags()); + assertEquals(44, result.get(0).changeset()); + assertNotEquals(result.get(1).osmEntity().getTags(), result.get(0).osmEntity().getTags()); + assertEquals(result.get(5).osmEntity().getTags(), result.get(3).osmEntity().getTags()); } @Test @@ -330,30 +329,30 @@ void testPolygonAreaYesTagDisappears() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(2, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.TAG_CHANGE, ContributionType.GEOMETRY_CHANGE), - result.get(1).activities.get() + result.get(1).activities().get() ); - assertEquals(48, result.get(0).changeset); + assertEquals(48, result.get(0).changeset()); - assertEquals(5, result.get(0).geometry.get().getNumPoints()); - assertEquals(5, result.get(1).geometry.get().getNumPoints()); + assertEquals(5, result.get(0).geometry().get().getNumPoints()); + assertEquals(5, result.get(1).geometry().get().getNumPoints()); - assertEquals(null, result.get(0).previousGeometry.get()); - Geometry geom = result.get(0).geometry.get(); + assertNull(result.get(0).previousGeometry().get()); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof Polygon); - Geometry geom2 = result.get(1).geometry.get(); + Geometry geom2 = result.get(1).geometry().get(); assertTrue(geom2 instanceof LineString); - assertNotEquals(result.get(1).geometry.get(), result.get(1).previousGeometry.get()); + assertNotEquals(result.get(1).geometry().get(), result.get(1).previousGeometry().get()); } @Test @@ -370,30 +369,30 @@ void testPolygonAreaYesNodeDisappears() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(2, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(1).activities.get() + result.get(1).activities().get() ); - assertEquals(50, result.get(0).changeset); + assertEquals(50, result.get(0).changeset()); - assertEquals(5, result.get(0).geometry.get().getNumPoints()); - assertEquals(4, result.get(1).geometry.get().getNumPoints()); + assertEquals(5, result.get(0).geometry().get().getNumPoints()); + assertEquals(4, result.get(1).geometry().get().getNumPoints()); - assertEquals(null, result.get(0).previousGeometry.get()); - Geometry geom = result.get(0).geometry.get(); + assertNull(result.get(0).previousGeometry().get()); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof Polygon); - Geometry geom2 = result.get(1).geometry.get(); + Geometry geom2 = result.get(1).geometry().get(); assertTrue(geom2 instanceof LineString); - assertNotEquals(result.get(1).geometry.get(), result.get(1).previousGeometry.get()); + assertNotEquals(result.get(1).geometry().get(), result.get(1).previousGeometry().get()); } @Test @@ -411,17 +410,17 @@ void testTimestampInclusion() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); // should be 2: entity has created at start time, modified in between and at end time (excluded) assertEquals(2, result.size()); - assertEquals(61, result.get(0).changeset); + assertEquals(61, result.get(0).changeset()); } @Test void testTwoNodesChangedAtSameTimeDifferentChangesets() { // way with two nodes, nodes changed lat lon, both at same time, different changesets - // which changeset is shown in result.get(1).changeset? -> from node 20, not 21 + // which changeset is shown in result.get(1).changeset()? -> from node 20, not 21 List<IterateAllEntry> result = (new CellIterator( new OSHDBTimestamps( "2000-01-01T00:00:00Z", @@ -433,20 +432,20 @@ void testTwoNodesChangedAtSameTimeDifferentChangesets() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(2, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(1).activities.get() + result.get(1).activities().get() ); // randomly 332 or 334 - //assertEquals(332, result.get(1).changeset); + //assertEquals(332, result.get(1).changeset()); } @Test @@ -464,16 +463,16 @@ void testNodeChangeOutsideBboxIsNotGeometryChange() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(2, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); - assertTrue(result.get(1).activities.get().isEmpty()); - assertEquals(3, result.get(1).geometry.get().getNumPoints()); + assertTrue(result.get(1).activities().get().isEmpty()); + assertEquals(3, result.get(1).geometry().get().getNumPoints()); } @Test @@ -492,20 +491,20 @@ void testNodeChangeOutsideBboxAffectsPartOfLineStringInBbox() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); - assertTrue(result.get(1).activities.get().isEmpty()); + assertTrue(result.get(1).activities().get().isEmpty()); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(2).activities.get() + result.get(2).activities().get() ); - assertEquals(3, result.get(1).geometry.get().getNumPoints()); + assertEquals(3, result.get(1).geometry().get().getNumPoints()); } @Test @@ -523,16 +522,16 @@ void testTagChangeOfNodeInWay() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); - assertTrue(result.get(1).activities.get().isEmpty()); - assertTrue(result.get(2).activities.get().isEmpty()); + assertTrue(result.get(1).activities().get().isEmpty()); + assertTrue(result.get(2).activities().get().isEmpty()); } @Test @@ -549,21 +548,21 @@ void testNodeRefsDeletedInVersion2() { osmEntity -> true, false )).iterateByContribution( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); assertEquals( EnumSet.of(ContributionType.CREATION), - result.get(0).activities.get() + result.get(0).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(1).activities.get() + result.get(1).activities().get() ); assertEquals( EnumSet.of(ContributionType.GEOMETRY_CHANGE), - result.get(2).activities.get() + result.get(2).activities().get() ); } } diff --git a/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByTimestampNotOsmTypeSpecificTest.java b/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByTimestampNotOsmTypeSpecificTest.java index c9f7651c4..d78e5d00d 100644 --- a/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByTimestampNotOsmTypeSpecificTest.java +++ b/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByTimestampNotOsmTypeSpecificTest.java @@ -1,5 +1,6 @@ package org.heigit.ohsome.oshdb.util.celliterator; +import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.IOException; @@ -11,7 +12,6 @@ import java.util.Map.Entry; import java.util.TreeMap; import java.util.stream.Collectors; -import org.heigit.ohsome.oshdb.grid.GridOSHEntity; import org.heigit.ohsome.oshdb.grid.GridOSHRelations; import org.heigit.ohsome.oshdb.impl.osh.OSHNodeImpl; import org.heigit.ohsome.oshdb.impl.osh.OSHRelationImpl; @@ -34,7 +34,7 @@ import org.locationtech.jts.geom.Polygon; /** - * Tests the {@link CellIterator#iterateByTimestamps(GridOSHEntity)} method on special situations + * Tests the {@link CellIterator#iterateByTimestamps(OSHEntitySource)} method on special situations * which are related to OSHDB grid cells. */ class IterateByTimestampNotOsmTypeSpecificTest { @@ -107,8 +107,8 @@ void testCellOutsidePolygon() throws IOException { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertTrue(result.isEmpty()); } @@ -137,8 +137,8 @@ void testCellCoveringPolygon() throws IOException { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertTrue(result.isEmpty()); } @@ -168,8 +168,8 @@ void testCellFullyInsidePolygon() throws IOException { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); - assertTrue(!result.isEmpty()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); + assertFalse(result.isEmpty()); } } diff --git a/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByTimestampsNodesTest.java b/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByTimestampsNodesTest.java index 2f1c80352..10e5cc7ca 100644 --- a/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByTimestampsNodesTest.java +++ b/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByTimestampsNodesTest.java @@ -7,9 +7,7 @@ import java.io.IOException; import java.util.List; -import java.util.stream.Collectors; import org.heigit.ohsome.oshdb.OSHDBBoundingBox; -import org.heigit.ohsome.oshdb.grid.GridOSHEntity; import org.heigit.ohsome.oshdb.grid.GridOSHNodes; import org.heigit.ohsome.oshdb.index.XYGrid; import org.heigit.ohsome.oshdb.util.celliterator.CellIterator.IterateByTimestampEntry; @@ -24,7 +22,7 @@ import org.locationtech.jts.geom.Polygon; /** - * Tests the {@link CellIterator#iterateByTimestamps(GridOSHEntity)} method on OSM nodes. + * Tests the {@link CellIterator#iterateByTimestamps(OSHEntitySource)} method on OSM nodes. */ class IterateByTimestampsNodesTest { private final GridOSHNodes oshdbDataGridCell; @@ -58,14 +56,14 @@ void testGeometryChange() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(11, result.size()); - assertNotEquals(result.get(1).geometry.get().getCoordinates(), - result.get(0).geometry.get().getCoordinates()); - assertNotEquals(result.get(2).geometry.get().getCoordinates(), - result.get(1).geometry.get().getCoordinates()); + assertNotEquals(result.get(1).geometry().get().getCoordinates(), + result.get(0).geometry().get().getCoordinates()); + assertNotEquals(result.get(2).geometry().get().getCoordinates(), + result.get(1).geometry().get().getCoordinates()); } @Test @@ -84,20 +82,20 @@ void testTagChange() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(12, result.size()); - assertNotEquals(result.get(1).osmEntity.getTags(), result.get(0).osmEntity.getTags()); - assertEquals(result.get(2).osmEntity.getTags(), result.get(1).osmEntity.getTags()); - assertEquals(result.get(3).osmEntity.getTags(), result.get(2).osmEntity.getTags()); - assertEquals(result.get(4).osmEntity.getTags(), result.get(3).osmEntity.getTags()); - assertEquals(result.get(5).osmEntity.getTags(), result.get(4).osmEntity.getTags()); - assertEquals(result.get(6).osmEntity.getTags(), result.get(5).osmEntity.getTags()); - assertNotEquals(result.get(7).osmEntity.getTags(), result.get(6).osmEntity.getTags()); - assertEquals(result.get(8).osmEntity.getTags(), result.get(7).osmEntity.getTags()); - assertEquals(result.get(9).osmEntity.getTags(), result.get(8).osmEntity.getTags()); - assertEquals(result.get(10).osmEntity.getTags(), result.get(9).osmEntity.getTags()); - assertEquals(result.get(11).osmEntity.getTags(), result.get(10).osmEntity.getTags()); + assertNotEquals(result.get(1).osmEntity().getTags(), result.get(0).osmEntity().getTags()); + assertEquals(result.get(2).osmEntity().getTags(), result.get(1).osmEntity().getTags()); + assertEquals(result.get(3).osmEntity().getTags(), result.get(2).osmEntity().getTags()); + assertEquals(result.get(4).osmEntity().getTags(), result.get(3).osmEntity().getTags()); + assertEquals(result.get(5).osmEntity().getTags(), result.get(4).osmEntity().getTags()); + assertEquals(result.get(6).osmEntity().getTags(), result.get(5).osmEntity().getTags()); + assertNotEquals(result.get(7).osmEntity().getTags(), result.get(6).osmEntity().getTags()); + assertEquals(result.get(8).osmEntity().getTags(), result.get(7).osmEntity().getTags()); + assertEquals(result.get(9).osmEntity().getTags(), result.get(8).osmEntity().getTags()); + assertEquals(result.get(10).osmEntity().getTags(), result.get(9).osmEntity().getTags()); + assertEquals(result.get(11).osmEntity().getTags(), result.get(10).osmEntity().getTags()); } @Test @@ -116,8 +114,8 @@ void testVisibleChange() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(5, result.size()); } @@ -142,32 +140,32 @@ void testMultipleChanges() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(11, result.size()); - assertNotEquals(result.get(1).geometry.get().getCoordinates(), - result.get(0).geometry.get().getCoordinates()); - assertArrayEquals(result.get(2).geometry.get().getCoordinates(), - result.get(1).geometry.get().getCoordinates()); - assertNotEquals(result.get(3).geometry.get().getCoordinates(), - result.get(2).geometry.get().getCoordinates()); - assertArrayEquals(result.get(5).geometry.get().getCoordinates(), - result.get(3).geometry.get().getCoordinates()); - assertNotEquals(result.get(6).geometry.get().getCoordinates(), - result.get(3).geometry.get().getCoordinates()); - assertArrayEquals(result.get(9).geometry.get().getCoordinates(), - result.get(6).geometry.get().getCoordinates()); - assertNotEquals(result.get(1).osmEntity.getTags(), - result.get(0).osmEntity.getTags()); - assertEquals(result.get(2).osmEntity.getTags(), - result.get(1).osmEntity.getTags()); - assertNotEquals(result.get(3).osmEntity.getTags(), - result.get(2).osmEntity.getTags()); - assertEquals(result.get(5).osmEntity.getTags(), - result.get(4).osmEntity.getTags()); - assertNotEquals(result.get(9).osmEntity.getTags(), - result.get(6).osmEntity.getTags()); + assertNotEquals(result.get(1).geometry().get().getCoordinates(), + result.get(0).geometry().get().getCoordinates()); + assertArrayEquals(result.get(2).geometry().get().getCoordinates(), + result.get(1).geometry().get().getCoordinates()); + assertNotEquals(result.get(3).geometry().get().getCoordinates(), + result.get(2).geometry().get().getCoordinates()); + assertArrayEquals(result.get(5).geometry().get().getCoordinates(), + result.get(3).geometry().get().getCoordinates()); + assertNotEquals(result.get(6).geometry().get().getCoordinates(), + result.get(3).geometry().get().getCoordinates()); + assertArrayEquals(result.get(9).geometry().get().getCoordinates(), + result.get(6).geometry().get().getCoordinates()); + assertNotEquals(result.get(1).osmEntity().getTags(), + result.get(0).osmEntity().getTags()); + assertEquals(result.get(2).osmEntity().getTags(), + result.get(1).osmEntity().getTags()); + assertNotEquals(result.get(3).osmEntity().getTags(), + result.get(2).osmEntity().getTags()); + assertEquals(result.get(5).osmEntity().getTags(), + result.get(4).osmEntity().getTags()); + assertNotEquals(result.get(9).osmEntity().getTags(), + result.get(6).osmEntity().getTags()); } @Test @@ -185,8 +183,8 @@ void testTagChangeTagFilterWithSuccess() { osmEntity -> osmEntity.getTags().hasTagKey(osmXmlTestData.keys().get("shop")), false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(7, result.size()); } @@ -202,11 +200,12 @@ void testTagChangeTagFilterWithoutSuccess() { OSHDBBoundingBox.bboxWgs84Coordinates(-180.0, -90.0, 180.0, 90.0), areaDecider, oshEntity -> oshEntity.getId() == 5, - osmEntity -> osmEntity.getTags().hasTagKey(osmXmlTestData.keys().getOrDefault("amenity", -1)), + osmEntity -> osmEntity.getTags().hasTagKey( + osmXmlTestData.keys().getOrDefault("amenity", -1)), false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertTrue(result.isEmpty()); } @@ -234,8 +233,8 @@ void testTagFilterAndPolygonIntersectingDataPartly() { osmEntity -> osmEntity.getTags().hasTagKey(osmXmlTestData.keys().get("shop")), false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(1, result.size()); } @@ -261,14 +260,45 @@ void testCoordinatesRelativeToPolygon() throws IOException { oshEntity -> oshEntity.getId() >= 10 && oshEntity.getId() < 20, osmEntity -> true, false - )).iterateByTimestamps( + )).iterateByTimestamps(OSHEntitySource.fromGridOSHEntity( GridOSHFactory.getGridOSHNodes(osmXmlTestData, 6, (new XYGrid(6)) .getId(1.0, 1.0)/* approx. 0, 0, 5.6, 5.6*/) - ).collect(Collectors.toList()); + )).toList(); assertEquals(3, result.size()); - assertEquals(13, result.get(0).osmEntity.getId()); - assertEquals(13, result.get(1).osmEntity.getId()); - assertEquals(14, result.get(2).osmEntity.getId()); + assertEquals(13, result.get(0).osmEntity().getId()); + assertEquals(13, result.get(1).osmEntity().getId()); + assertEquals(14, result.get(2).osmEntity().getId()); + } + + @Test + void testLastContributionTimestamp() { + // node 3: creation and 4 visible changes, but no geometry and no tag changes + + List<IterateByTimestampEntry> result = (new CellIterator( + new OSHDBTimestamps( + "2000-06-01T00:00:00Z", + "2018-06-01T00:00:00Z", + "P1Y" + ).get(), + OSHDBBoundingBox.bboxWgs84Coordinates(-180.0, -90.0, 180.0, 90.0), + areaDecider, + oshEntity -> oshEntity.getId() == 3, + osmEntity -> true, + false + )).iterateByTimestamps( + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); + assertEquals(5, result.size()); + assertEquals("2007-06-01T00:00:00", result.get(0).timestamp().toString()); + assertEquals("2007-01-01T00:00:00", result.get(0).lastModificationTimestamp().toString()); + assertEquals("2014-06-01T00:00:00", result.get(1).timestamp().toString()); + assertEquals("2014-01-01T00:00:00", result.get(1).lastModificationTimestamp().toString()); + assertEquals("2016-06-01T00:00:00", result.get(2).timestamp().toString()); + assertEquals("2016-01-01T00:00:00", result.get(2).lastModificationTimestamp().toString()); + assertEquals("2017-06-01T00:00:00", result.get(3).timestamp().toString()); + assertEquals("2016-01-01T00:00:00", result.get(3).lastModificationTimestamp().toString()); + assertEquals("2018-06-01T00:00:00", result.get(4).timestamp().toString()); + assertEquals("2016-01-01T00:00:00", result.get(4).lastModificationTimestamp().toString()); } } diff --git a/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByTimestampsRelationsTest.java b/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByTimestampsRelationsTest.java index ccb263dbd..9b25f4773 100644 --- a/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByTimestampsRelationsTest.java +++ b/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByTimestampsRelationsTest.java @@ -9,7 +9,6 @@ import java.util.List; import java.util.stream.Collectors; import org.heigit.ohsome.oshdb.OSHDBBoundingBox; -import org.heigit.ohsome.oshdb.grid.GridOSHEntity; import org.heigit.ohsome.oshdb.grid.GridOSHRelations; import org.heigit.ohsome.oshdb.util.celliterator.CellIterator.IterateByTimestampEntry; import org.heigit.ohsome.oshdb.util.celliterator.helpers.GridOSHFactory; @@ -26,7 +25,7 @@ import org.locationtech.jts.geom.Polygon; /** - * Tests the {@link CellIterator#iterateByTimestamps(GridOSHEntity)} method on OSM relations. + * Tests the {@link CellIterator#iterateByTimestamps(OSHEntitySource)} method on OSM relations. */ class IterateByTimestampsRelationsTest { private final GridOSHRelations oshdbDataGridCell; @@ -59,17 +58,17 @@ void testGeometryChange() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(12, result.size()); - assertEquals(300, result.get(0).osmEntity.getChangesetId()); - Geometry geom = result.get(0).geometry.get(); + assertEquals(300, result.get(0).osmEntity().getChangesetId()); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof MultiPolygon); - Geometry geom3 = result.get(1).geometry.get(); + Geometry geom3 = result.get(1).geometry().get(); assertTrue(geom3 instanceof MultiPolygon); - Geometry geom4 = result.get(2).geometry.get(); + Geometry geom4 = result.get(2).geometry().get(); assertTrue(geom4 instanceof MultiPolygon); } @@ -89,11 +88,11 @@ void testVisibleChange() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(9, result.size()); - assertEquals(303, result.get(0).osmEntity.getChangesetId()); + assertEquals(303, result.get(0).osmEntity().getChangesetId()); } @SuppressWarnings("ResultOfMethodCallIgnored") @@ -113,7 +112,7 @@ void testWaysNotExistent() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) ).collect(Collectors.toList()); }); } @@ -133,11 +132,11 @@ void testTagChange() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(14, result.size()); - assertEquals(307, result.get(0).osmEntity.getChangesetId()); + assertEquals(307, result.get(0).osmEntity().getChangesetId()); } @Test @@ -156,20 +155,20 @@ void testGeometryChangeOfNodeRefsInWays() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(10, result.size()); - assertEquals(310, result.get(0).osmEntity.getChangesetId()); + assertEquals(310, result.get(0).osmEntity().getChangesetId()); - Geometry geom = result.get(0).geometry.get(); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof Polygon); - Geometry geom3 = result.get(1).geometry.get(); + Geometry geom3 = result.get(1).geometry().get(); assertTrue(geom3 instanceof Polygon); - assertNotEquals(result.get(1).geometry.get(), result.get(0).geometry.get()); - assertNotEquals(result.get(2).geometry.get(), result.get(1).geometry.get()); - assertEquals(result.get(3).geometry.get(), result.get(2).geometry.get()); + assertNotEquals(result.get(1).geometry().get(), result.get(0).geometry().get()); + assertNotEquals(result.get(2).geometry().get(), result.get(1).geometry().get()); + assertEquals(result.get(3).geometry().get(), result.get(2).geometry().get()); } @Test @@ -188,19 +187,19 @@ void testGeometryChangeOfNodeCoordinatesInWay() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(10, result.size()); - assertEquals(312, result.get(0).osmEntity.getChangesetId()); + assertEquals(312, result.get(0).osmEntity().getChangesetId()); - Geometry geom = result.get(0).geometry.get(); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof Polygon); - Geometry geom3 = result.get(1).geometry.get(); + Geometry geom3 = result.get(1).geometry().get(); assertTrue(geom3 instanceof Polygon); - assertNotEquals(result.get(1).geometry.get(), result.get(0).geometry.get()); - assertNotEquals(result.get(6).geometry.get(), result.get(1).geometry.get()); + assertNotEquals(result.get(1).geometry().get(), result.get(0).geometry().get()); + assertNotEquals(result.get(6).geometry().get(), result.get(1).geometry().get()); } @Test @@ -219,14 +218,14 @@ void testGeometryChangeOfNodeCoordinatesInRelationAndWay() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(10, result.size()); - assertEquals(313, result.get(0).osmEntity.getChangesetId()); + assertEquals(313, result.get(0).osmEntity().getChangesetId()); - assertNotEquals(result.get(1).geometry.get(), result.get(0).geometry.get()); - assertEquals(result.get(6).geometry.get(), result.get(5).geometry.get()); + assertNotEquals(result.get(1).geometry().get(), result.get(0).geometry().get()); + assertEquals(result.get(6).geometry().get(), result.get(5).geometry().get()); } @Test @@ -244,15 +243,15 @@ void testGeometryCollection() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(10, result.size()); - assertEquals(314, result.get(0).osmEntity.getChangesetId()); - Geometry geom = result.get(0).geometry.get(); + assertEquals(314, result.get(0).osmEntity().getChangesetId()); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof GeometryCollection); - Geometry geom2 = result.get(9).geometry.get(); + Geometry geom2 = result.get(9).geometry().get(); assertTrue(geom2 instanceof GeometryCollection); } @@ -273,7 +272,7 @@ void testNodesOfWaysNotExistent() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) ).collect(Collectors.toList()); }); } @@ -293,25 +292,25 @@ void testVisibleChangeOfNodeInWay() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(10, result.size()); - assertEquals(316, result.get(0).osmEntity.getChangesetId()); + assertEquals(316, result.get(0).osmEntity().getChangesetId()); - Geometry geom = result.get(0).geometry.get(); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof Polygon); - Geometry geom2 = result.get(1).geometry.get(); + Geometry geom2 = result.get(1).geometry().get(); assertTrue(geom2 instanceof Polygon); - Geometry geom3 = result.get(2).geometry.get(); + Geometry geom3 = result.get(2).geometry().get(); assertTrue(geom3 instanceof Polygon); - Geometry geom4 = result.get(3).geometry.get(); + Geometry geom4 = result.get(3).geometry().get(); assertTrue(geom4 instanceof Polygon); - Geometry geom5 = result.get(9).geometry.get(); + Geometry geom5 = result.get(9).geometry().get(); assertTrue(geom5 instanceof Polygon); - assertNotEquals(result.get(1).geometry.get(), result.get(0).geometry.get()); - assertEquals(result.get(2).geometry.get(), result.get(1).geometry.get()); + assertNotEquals(result.get(1).geometry().get(), result.get(0).geometry().get()); + assertEquals(result.get(2).geometry().get(), result.get(1).geometry().get()); } @Test @@ -329,11 +328,11 @@ void testTagChangeOfNodeInWay() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(13, result.size()); - assertEquals(317, result.get(0).osmEntity.getChangesetId()); + assertEquals(317, result.get(0).osmEntity().getChangesetId()); } @Test @@ -351,13 +350,13 @@ void testVisibleChangeOfWay() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(10, result.size()); - assertEquals(318, result.get(0).osmEntity.getChangesetId()); + assertEquals(318, result.get(0).osmEntity().getChangesetId()); - assertTrue(result.get(6).geometry.get().isEmpty()); + assertTrue(result.get(6).geometry().get().isEmpty()); } @Test @@ -376,15 +375,15 @@ void testVisibleChangeOfOneWayOfOuterRing() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(10, result.size()); - Geometry geom = result.get(0).geometry.get(); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof Polygon); - Geometry geom2 = result.get(7).geometry.get(); + Geometry geom2 = result.get(7).geometry().get(); assertTrue(geom2 instanceof GeometryCollection); - assertEquals(319, result.get(0).osmEntity.getChangesetId()); + assertEquals(319, result.get(0).osmEntity().getChangesetId()); } @Test @@ -402,15 +401,15 @@ void testTagChangeOfWay() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(12, result.size()); - Geometry geom = result.get(0).geometry.get(); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof Polygon); - Geometry geom2 = result.get(2).geometry.get(); + Geometry geom2 = result.get(2).geometry().get(); assertTrue(geom2 instanceof Polygon); - assertEquals(320, result.get(0).osmEntity.getChangesetId()); + assertEquals(320, result.get(0).osmEntity().getChangesetId()); } @Test @@ -428,16 +427,16 @@ void testOneOfTwoPolygonDisappears() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(12, result.size()); - Geometry geom = result.get(0).geometry.get(); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof MultiPolygon); - Geometry geom2 = result.get(9).geometry.get(); + Geometry geom2 = result.get(9).geometry().get(); assertTrue(geom2 instanceof Polygon); - assertEquals(321, result.get(0).osmEntity.getChangesetId()); - assertNotEquals(result.get(9).geometry.get(), result.get(0).geometry.get()); + assertEquals(321, result.get(0).osmEntity().getChangesetId()); + assertNotEquals(result.get(9).geometry().get(), result.get(0).geometry().get()); } @Test @@ -455,18 +454,18 @@ void testWaySplitUpInTwo() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(12, result.size()); - Geometry geom = result.get(0).geometry.get(); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof Polygon); - Geometry geom1 = result.get(1).geometry.get(); + Geometry geom1 = result.get(1).geometry().get(); assertTrue(geom1 instanceof GeometryCollection); - Geometry geom2 = result.get(9).geometry.get(); + Geometry geom2 = result.get(9).geometry().get(); assertTrue(geom2 instanceof Polygon); - assertEquals(323, result.get(0).osmEntity.getChangesetId()); - assertNotEquals(result.get(9).geometry.get(), result.get(0).geometry.get()); + assertEquals(323, result.get(0).osmEntity().getChangesetId()); + assertNotEquals(result.get(9).geometry().get(), result.get(0).geometry().get()); } @Test @@ -494,8 +493,8 @@ void testPolygonIntersectingDataPartly() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(10, result.size()); } @@ -523,8 +522,8 @@ void testPolygonIntersectingDataOnlyAtBorderLine() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(0, result.size()); } @@ -553,8 +552,8 @@ void testPolygonIntersectingDataCompletely() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(10, result.size()); } @@ -583,8 +582,8 @@ void testPolygonNotIntersectingData() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertTrue(resultPoly.isEmpty()); } @@ -605,10 +604,10 @@ void testNodeChangeOutsideBbox() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); - assertNotEquals(result.get(3).geometry.get(), result.get(0).geometry.get()); + assertNotEquals(result.get(3).geometry().get(), result.get(0).geometry().get()); } @Test @@ -635,8 +634,8 @@ void testPolygonIntersectingDataCompletelyTimeIntervalAfterChanges() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); } @@ -656,8 +655,8 @@ void testTimeIntervalAfterChanges() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); } @@ -676,8 +675,8 @@ void testBboxOutsidePolygon() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertTrue(resultPoly.isEmpty()); } @@ -699,17 +698,17 @@ void testUnclippedGeom() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); // geom of requested area vs full geom after modification - assertNotEquals(result.get(0).geometry.get().getArea(), - result.get(0).unclippedGeometry.get().getArea()); + assertNotEquals(result.get(0).geometry().get().getArea(), + result.get(0).unclippedGeometry().get().getArea()); // full geom changed - assertNotEquals(result.get(2).unclippedGeometry.get().getArea(), - result.get(0).unclippedGeometry.get().getArea()); - assertNotEquals(result.get(2).unclippedGeometry.get().getArea(), - result.get(4).unclippedGeometry.get().getArea()); + assertNotEquals(result.get(2).unclippedGeometry().get().getArea(), + result.get(0).unclippedGeometry().get().getArea()); + assertNotEquals(result.get(2).unclippedGeometry().get().getArea(), + result.get(4).unclippedGeometry().get().getArea()); } @Test @@ -738,8 +737,8 @@ void testSelfIntersectingPolygonClipped() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertTrue(result.isEmpty()); } @@ -759,8 +758,8 @@ void testMembersDisappear() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); } @@ -780,8 +779,8 @@ void testTimeIntervalAfterDeletionInVersion2() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(5, result.size()); } @@ -801,8 +800,8 @@ void testTimeIntervalAfterDeletionInCurrentVersion() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(0, result.size()); } @@ -831,8 +830,8 @@ void testMembersDisappearClipped() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); } @@ -861,8 +860,8 @@ void testTimeIntervalAfterDeletionInVersion2Clipped() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(5, result.size()); } @@ -891,8 +890,8 @@ void testTimeIntervalAfterDeletionInCurrentVersionClipped() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(0, result.size()); } @@ -921,8 +920,8 @@ void testExcludingVersion2Clipped() { osmEntity -> !(osmEntity.getVersion() == 2), false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(3, result.size()); } @@ -951,8 +950,8 @@ void testClippingPolygonIsVeryBig() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(12, result.size()); } diff --git a/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByTimestampsWaysTest.java b/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByTimestampsWaysTest.java index 6902ab632..f1f373a97 100644 --- a/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByTimestampsWaysTest.java +++ b/oshdb-util/src/test/java/org/heigit/ohsome/oshdb/util/celliterator/IterateByTimestampsWaysTest.java @@ -6,9 +6,7 @@ import java.io.IOException; import java.util.List; -import java.util.stream.Collectors; import org.heigit.ohsome.oshdb.OSHDBBoundingBox; -import org.heigit.ohsome.oshdb.grid.GridOSHEntity; import org.heigit.ohsome.oshdb.grid.GridOSHWays; import org.heigit.ohsome.oshdb.util.celliterator.CellIterator.IterateByTimestampEntry; import org.heigit.ohsome.oshdb.util.celliterator.helpers.GridOSHFactory; @@ -22,7 +20,7 @@ import org.locationtech.jts.geom.Polygon; /** - * Tests the {@link CellIterator#iterateByTimestamps(GridOSHEntity)} method on OSM ways. + * Tests the {@link CellIterator#iterateByTimestamps(OSHEntitySource)} method on OSM ways. */ class IterateByTimestampsWaysTest { @@ -57,23 +55,23 @@ void testGeometryChange() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(10, result.size()); - assertEquals(result.get(1).osmEntity.getTags(), result.get(0).osmEntity.getTags()); - assertEquals(4, result.get(0).geometry.get().getNumPoints()); - assertEquals(8, result.get(1).geometry.get().getNumPoints()); - assertEquals(9, result.get(2).geometry.get().getNumPoints()); + assertEquals(result.get(1).osmEntity().getTags(), result.get(0).osmEntity().getTags()); + assertEquals(4, result.get(0).geometry().get().getNumPoints()); + assertEquals(8, result.get(1).geometry().get().getNumPoints()); + assertEquals(9, result.get(2).geometry().get().getNumPoints()); - Geometry geom = result.get(0).geometry.get(); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof LineString); - Geometry geom2 = result.get(1).geometry.get(); + Geometry geom2 = result.get(1).geometry().get(); assertTrue(geom2 instanceof LineString); - Geometry geom3 = result.get(2).geometry.get(); + Geometry geom3 = result.get(2).geometry().get(); assertTrue(geom3 instanceof LineString); - assertEquals(31, result.get(0).osmEntity.getChangesetId()); - assertNotEquals(result.get(1).geometry.get(), result.get(0).geometry.get()); - assertNotEquals(result.get(2).geometry.get(), result.get(1).geometry.get()); + assertEquals(31, result.get(0).osmEntity().getChangesetId()); + assertNotEquals(result.get(1).geometry().get(), result.get(0).geometry().get()); + assertNotEquals(result.get(2).geometry().get(), result.get(1).geometry().get()); } @Test @@ -92,18 +90,18 @@ void testGeometryChangeOfNodeInWay() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(10, result.size()); - assertEquals(34, result.get(0).osmEntity.getChangesetId()); - assertEquals(35, result.get(8).osmEntity.getChangesetId()); + assertEquals(34, result.get(0).osmEntity().getChangesetId()); + assertEquals(35, result.get(8).osmEntity().getChangesetId()); - assertNotEquals(result.get(1).geometry.get(), result.get(0).geometry.get()); - assertNotEquals(result.get(2).geometry.get(), result.get(1).geometry.get()); - assertEquals(result.get(5).geometry.get(), result.get(4).geometry.get()); - assertNotEquals(result.get(9).geometry.get(), result.get(1).geometry.get()); + assertNotEquals(result.get(1).geometry().get(), result.get(0).geometry().get()); + assertNotEquals(result.get(2).geometry().get(), result.get(1).geometry().get()); + assertEquals(result.get(5).geometry().get(), result.get(4).geometry().get()); + assertNotEquals(result.get(9).geometry().get(), result.get(1).geometry().get()); } @Test @@ -123,12 +121,12 @@ void testVisibleChange() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(10, result.size()); - assertEquals(36, result.get(0).osmEntity.getChangesetId()); - assertEquals(38, result.get(9).osmEntity.getChangesetId()); + assertEquals(36, result.get(0).osmEntity().getChangesetId()); + assertEquals(38, result.get(9).osmEntity().getChangesetId()); } @Test @@ -147,24 +145,24 @@ void testTagChange() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(11, result.size()); - assertEquals(3, result.get(0).geometry.get().getNumPoints()); - assertEquals(5, result.get(2).geometry.get().getNumPoints()); - assertEquals(5, result.get(10).geometry.get().getNumPoints()); + assertEquals(3, result.get(0).geometry().get().getNumPoints()); + assertEquals(5, result.get(2).geometry().get().getNumPoints()); + assertEquals(5, result.get(10).geometry().get().getNumPoints()); - Geometry geom = result.get(0).geometry.get(); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof LineString); - Geometry geom2 = result.get(2).geometry.get(); + Geometry geom2 = result.get(2).geometry().get(); assertTrue(geom2 instanceof LineString); - Geometry geom3 = result.get(10).geometry.get(); + Geometry geom3 = result.get(10).geometry().get(); assertTrue(geom3 instanceof LineString); - assertNotEquals(result.get(2).osmEntity.getTags(), result.get(0).osmEntity.getTags()); - assertNotEquals(result.get(10).osmEntity.getTags(), result.get(2).osmEntity.getTags()); - assertNotEquals(result.get(2).geometry.get(), result.get(0).geometry.get()); - assertEquals(result.get(10).geometry.get(), result.get(2).geometry.get()); + assertNotEquals(result.get(2).osmEntity().getTags(), result.get(0).osmEntity().getTags()); + assertNotEquals(result.get(10).osmEntity().getTags(), result.get(2).osmEntity().getTags()); + assertNotEquals(result.get(2).geometry().get(), result.get(0).geometry().get()); + assertEquals(result.get(10).geometry().get(), result.get(2).geometry().get()); } @Test @@ -185,17 +183,17 @@ void testMultipleChangesOnNodesOfWay() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(8, result.size()); - assertEquals(2, result.get(0).geometry.get().getNumPoints()); - assertEquals(3, result.get(3).geometry.get().getNumPoints()); - assertEquals(2, result.get(4).geometry.get().getNumPoints()); - - assertEquals(42, result.get(0).osmEntity.getChangesetId()); - assertEquals(result.get(1).geometry.get(), result.get(0).geometry.get()); - assertNotEquals(result.get(3).geometry.get(), result.get(1).geometry.get()); - assertNotEquals(result.get(4).geometry.get(), result.get(3).geometry.get()); + assertEquals(2, result.get(0).geometry().get().getNumPoints()); + assertEquals(3, result.get(3).geometry().get().getNumPoints()); + assertEquals(2, result.get(4).geometry().get().getNumPoints()); + + assertEquals(42, result.get(0).osmEntity().getChangesetId()); + assertEquals(result.get(1).geometry().get(), result.get(0).geometry().get()); + assertNotEquals(result.get(3).geometry().get(), result.get(1).geometry().get()); + assertNotEquals(result.get(4).geometry().get(), result.get(3).geometry().get()); } @Test @@ -213,15 +211,15 @@ void testMultipleChanges() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(7, result.size()); - assertNotEquals(result.get(2).osmEntity.getTags(), result.get(0).osmEntity.getTags()); - assertEquals(result.get(6).osmEntity.getTags(), result.get(2).osmEntity.getTags()); - assertEquals(result.get(1).geometry.get(), result.get(0).geometry.get()); - assertNotEquals(result.get(3).geometry.get(), result.get(1).geometry.get()); - assertNotEquals(result.get(6).geometry.get(), result.get(3).geometry.get()); + assertNotEquals(result.get(2).osmEntity().getTags(), result.get(0).osmEntity().getTags()); + assertEquals(result.get(6).osmEntity().getTags(), result.get(2).osmEntity().getTags()); + assertEquals(result.get(1).geometry().get(), result.get(0).geometry().get()); + assertNotEquals(result.get(3).geometry().get(), result.get(1).geometry().get()); + assertNotEquals(result.get(6).geometry().get(), result.get(3).geometry().get()); } @Test @@ -239,19 +237,19 @@ void testPolygonAreaYesTagDisappears() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(11, result.size()); - assertEquals(5, result.get(0).geometry.get().getNumPoints()); - assertEquals(5, result.get(1).geometry.get().getNumPoints()); + assertEquals(5, result.get(0).geometry().get().getNumPoints()); + assertEquals(5, result.get(1).geometry().get().getNumPoints()); - Geometry geom = result.get(0).geometry.get(); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof Polygon); - Geometry geom2 = result.get(8).geometry.get(); + Geometry geom2 = result.get(8).geometry().get(); assertTrue(geom2 instanceof LineString); - assertNotEquals(result.get(8).osmEntity.getTags(), result.get(0).osmEntity.getTags()); - assertNotEquals(result.get(8).geometry.get(), result.get(0).geometry.get()); + assertNotEquals(result.get(8).osmEntity().getTags(), result.get(0).osmEntity().getTags()); + assertNotEquals(result.get(8).geometry().get(), result.get(0).geometry().get()); } @Test @@ -269,18 +267,18 @@ void testPolygonAreaYesNodeDisappears() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(11, result.size()); - assertEquals(5, result.get(0).geometry.get().getNumPoints()); - assertEquals(4, result.get(8).geometry.get().getNumPoints()); + assertEquals(5, result.get(0).geometry().get().getNumPoints()); + assertEquals(4, result.get(8).geometry().get().getNumPoints()); - Geometry geom = result.get(0).geometry.get(); + Geometry geom = result.get(0).geometry().get(); assertTrue(geom instanceof Polygon); - Geometry geom2 = result.get(8).geometry.get(); + Geometry geom2 = result.get(8).geometry().get(); assertTrue(geom2 instanceof LineString); - assertEquals(result.get(8).osmEntity.getTags(), result.get(0).osmEntity.getTags()); - assertNotEquals(result.get(8).geometry.get(), result.get(0).geometry.get()); + assertEquals(result.get(8).osmEntity().getTags(), result.get(0).osmEntity().getTags()); + assertNotEquals(result.get(8).geometry().get(), result.get(0).geometry().get()); } @Test @@ -299,8 +297,8 @@ void testTimestampInclusion() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(9, result.size()); } @@ -320,11 +318,11 @@ void testNodeChangeOutsideBboxIsNotGeometryChange() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); assertEquals(2, result.size()); - assertEquals(result.get(0).geometry.get(), result.get(1).geometry.get()); + assertEquals(result.get(0).geometry().get(), result.get(1).geometry().get()); } @Test @@ -345,13 +343,13 @@ void testNodeChangeOutsideBboxAffectsPartOfLineStringInBbox() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); - assertNotEquals(result.get(0).geometry.get(), result.get(3).geometry.get()); + assertNotEquals(result.get(0).geometry().get(), result.get(3).geometry().get()); assertEquals(4, result.size()); - assertEquals(3, result.get(1).geometry.get().getNumPoints()); - assertEquals(4, result.get(0).unclippedGeometry.get().getNumPoints()); + assertEquals(3, result.get(1).geometry().get().getNumPoints()); + assertEquals(4, result.get(0).unclippedGeometry().get().getNumPoints()); } @Test @@ -369,9 +367,9 @@ void testNodeRefsDeletedInVersion2() { osmEntity -> true, false )).iterateByTimestamps( - oshdbDataGridCell - ).collect(Collectors.toList()); - assertEquals(3, result.get(0).geometry.get().getNumPoints()); + OSHEntitySource.fromGridOSHEntity(oshdbDataGridCell) + ).toList(); + assertEquals(3, result.get(0).geometry().get().getNumPoints()); // only 4 timestamps in result, because after 03/2012 no more node refs assertEquals(4, result.size()); }