diff --git a/.gitignore b/.gitignore index afa5d6e..94dd23a 100644 --- a/.gitignore +++ b/.gitignore @@ -1,3 +1,4 @@ *.csv .DS_Store *.jar +flink-processor/target/ diff --git a/README.md b/README.md index 3e22444..ec30479 100644 --- a/README.md +++ b/README.md @@ -62,3 +62,35 @@ Kafka producer Flink Processor Flink Processor + +# BerlinMOD-9 × 3 streaming forms — the parity matrix on Flink + +The streaming-side parity matrix runs all nine BerlinMOD reference queries (Q1..Q9) in three streaming forms each on this runtime: **continuous** (always-on, per-event emission), **windowed** (tumbling 10-second aggregation), and **snapshot** (5-second tick — the parity-oracle form whose output at watermark T equals the batch BerlinMOD-Q result on data up to T). + +| Q | Topic | Continuous | Windowed | Snapshot | +|---|---|---|---|---| +| Q1 | "which vehicles have appeared in the stream?" | ✓ | ✓ | ✓ | +| Q2 | "where is vehicle X at time T?" | ✓ | ✓ | ✓ | +| Q3 | "vehicles within d of P at time T?" | ✓ | ✓ | ✓ | +| Q4 | "vehicles entered region R, and when?" | ✓ | ✓ | ✓ | +| Q5 | "pairs of vehicles meeting near P" | ✓ | ✓ | ✓ | +| Q6 | "cumulative distance per vehicle" | ✓ | ✓ | ✓ | +| Q7 | "first passage of vehicles through POIs" | ✓ | ✓ | ✓ | +| Q8 | "vehicles close to a road segment" | ✓ | ✓ | ✓ | +| Q9 | "distance between vehicles X and Y at time T" | ✓ | ✓ | ✓ | + +**27 / 27 cells** = the full MobilityFlink parity-matrix row. Each cell has a dedicated `Q{Continuous,Windowed,Snapshot}Function` class in [`flink-processor/src/main/java/berlinmod/`](flink-processor/src/main/java/berlinmod/) and is locally verified via the companion `BerlinMODQLocalTest` driver running on a Flink mini-cluster. + +The streaming snapshot form converges to the batch BerlinMOD result on the same scale-factor corpus, anchored against the cross-platform outputs in [MobilityDB-BerlinMOD](https://github.com/MobilityDB/MobilityDB-BerlinMOD). + +Spatial predicates route through [`MEOSBridge`](flink-processor/src/main/java/berlinmod/MEOSBridge.java), which holds no spatial mathematics of its own: it builds the MEOS temporal instants and geographies and delegates the computation to libmeos. The within-distance predicate uses MEOS `edwithin_tgeo_geo` — the vehicle position as a `tgeogpoint` instant, tested against the query geography in metres on the WGS84 spheroid; region containment uses `eintersects_tgeo_geo` between the point's `tgeompoint` instant and the region polygon; and the pairwise and cumulative distances use `geog_distance`. + +The Kafka-source entry points for Q2 and Q3 are [`BerlinMODQ2Main`](flink-processor/src/main/java/berlinmod/BerlinMODQ2Main.java) and [`BerlinMODQ3Main`](flink-processor/src/main/java/berlinmod/BerlinMODQ3Main.java); the companion producer is [`python-producer-berlinmod.py`](kafka-producer/python-producer-berlinmod.py). Generate a BerlinMOD CSV with the upstream generator (`meos/examples/data/generate_berlinmod_trips.sql` in MobilityDB) at any scale factor and feed it to the producer. The form-by-form definition with default parameters lives in [`doc/berlinmod-q3-streaming-forms.md`](doc/berlinmod-q3-streaming-forms.md). + +### Sibling parity work in the ecosystem + +- [MobilityKafka#1](https://github.com/MobilityDB/MobilityKafka/pull/1) — the same 27-cell row on Kafka Streams +- [MobilityNebula#15](https://github.com/MobilityDB/MobilityNebula/pull/15) — 27 / 27 cells on NebulaStream scaffold (with [#16](https://github.com/MobilityDB/MobilityNebula/pull/16) adding `TEMPORAL_LENGTH` for Q6 and [#17](https://github.com/MobilityDB/MobilityNebula/pull/17) adding `PAIR_MEETING` + `CROSS_DISTANCE` for Q5/Q9, all calling MEOS C ABI directly) +- [MobilityDB-BerlinMOD#29](https://github.com/MobilityDB/MobilityDB-BerlinMOD/pull/29) — the batch BerlinMOD-9 cross-platform timings (the snapshot form's gold-answer source) +- [MobilityDB/.github#10](https://github.com/MobilityDB/.github/pull/10) — the ecosystem-profile description of the stream-layers tier + diff --git a/doc/berlinmod-q3-streaming-forms.md b/doc/berlinmod-q3-streaming-forms.md new file mode 100644 index 0000000..1a69682 --- /dev/null +++ b/doc/berlinmod-q3-streaming-forms.md @@ -0,0 +1,107 @@ +# BerlinMOD-Q3 streaming forms + +This document defines what **BerlinMOD-Q3** means in each of the three +streaming forms the parity contract specifies for the MobilityFlink / +MobilityKafka / MobilityNebula trio (see the planned-tier section of the +[ecosystem profile](https://github.com/MobilityDB/.github)). + +## The batch query + +> *Which vehicles were within distance `d` of point `P` at time `T`?* + +Parameters: a point `P = (lon, lat)`, a radius `d` in metres, and a time `T`. +Returns: the set of `vehicle_id`s whose trajectory passed within `d` of `P` at `T`. + +The batch reference implementation lives in +[MobilityDB-BerlinMOD](https://github.com/MobilityDB/MobilityDB-BerlinMOD) and +runs against the three SQL surfaces (MobilityDB / MobilityDuck / MobilitySpark) +with byte-identical results — the batch oracle for the snapshot streaming form +below. + +## The three streaming forms + +### 1. Continuous form + +> *"At every moment, which vehicles are currently within `d` of `P`?"* + +For each incoming GPS event `(vehicle_id, t, lon, lat)`: + +- Evaluate the radius predicate `distance((lon, lat), P) ≤ d`. +- Emit `(vehicle_id, t, near)` per event. + +No window; output updates per event. Watermark-independent. + +Use case: real-time geofence alerting where each event matters. + +Implemented by [`Q3ContinuousFunction`](../flink-processor/src/main/java/berlinmod/Q3ContinuousFunction.java). + +### 2. Windowed form + +> *"Per N-second tumbling window, how many distinct vehicles were +> within `d` of `P` at any time during the window?"* + +Tumbling event-time window of size `W` (default `W = 10s`). For each window: + +- Collect all events whose timestamp falls in the window. +- Compute the distinct set `{vehicle_id : ∃ event in window with distance ≤ d}`. +- Emit `(window_start, window_end, distinct_count)`. + +Use case: time-bucketed dashboards, near-real-time aggregates. + +Implemented by [`Q3WindowedFunction`](../flink-processor/src/main/java/berlinmod/Q3WindowedFunction.java). + +### 3. Snapshot form — **the parity oracle** + +> *"At time `T`, which vehicles are within `d` of `P`?"* + +Watermark-driven. Per vehicle, maintain `lastKnownPosition` state. At each +snapshot tick (event-time timer at multiples of `snapshotTickMillis`, +default `5000 ms`): + +- For each vehicle's most recent `(lon, lat)`, evaluate the radius predicate. +- Emit `(T, vehicle_id)` for every vehicle satisfying the predicate at `T`. + +As the watermark advances to `T = max(event_times)`, the streaming snapshot +output **equals the batch BerlinMOD-Q3 result** on the same scale-factor +corpus. This is the parity property the contract enforces: + +``` +streaming-Q3-snapshot(T) ≡ batch-BerlinMOD-Q3 on data up to T + (same SF, same P, same d) +``` + +Use case: lambda-architecture style verification — streaming pipeline's +output must converge to the batch reference. + +Implemented by [`Q3SnapshotFunction`](../flink-processor/src/main/java/berlinmod/Q3SnapshotFunction.java). + +## Default parameters + +The `BerlinMODQ3Main` entry point uses: + +| Parameter | Value | Source | +|---|---|---| +| `P` (lon, lat) | (4.3517, 50.8503) — Brussels city centre | Default centre for the BerlinMOD-Brussels corpus | +| `d` (radius) | 5 000 m | Within-city-centre scale | +| `W` (window size) | 10 s | Same as the AIS example for consistency | +| Snapshot tick | 5 s | Half the window for finer parity-oracle granularity | +| Topic | `berlinmod` | Single shared topic across the three forms | + +## Predicate implementation + +The within-distance predicate evaluates through the MEOS `edwithin_tgeo_geo` +operator — the same call used by `MobilityNebula/Queries/Query1.yaml`. The +vehicle position is built as a `tgeogpoint` instant and tested against the +query geography in metres on the WGS84 spheroid. All spatial predicates route +through [`MEOSBridge`](../flink-processor/src/main/java/berlinmod/MEOSBridge.java), +which holds no spatial mathematics of its own: it constructs the MEOS inputs +(temporal instants and geographies) and delegates the computation to libmeos. + +## Companion producer + +The BerlinMOD CSV → Kafka producer lives at +[`kafka-producer/python-producer-berlinmod.py`](../kafka-producer/python-producer-berlinmod.py). +Generate a BerlinMOD CSV at scale factor SF with the upstream generator +(`meos/examples/data/generate_berlinmod_trips.sql` in MobilityDB), name the +columns `(t, vehicle_id, lon, lat)`, and the producer streams it to the +`berlinmod` topic. diff --git a/flink-processor/docs/benchmark-results.md b/flink-processor/docs/benchmark-results.md new file mode 100644 index 0000000..0254a9a --- /dev/null +++ b/flink-processor/docs/benchmark-results.md @@ -0,0 +1,89 @@ +# BerlinMOD streaming-matrix throughput + +Throughput of the BerlinMOD-9 × 3-form streaming matrix (9 queries × +{continuous, windowed, snapshot} = 27 cells) on the Flink local mini-cluster +over the real BerlinMOD instants corpus. The spatial predicates evaluate through +MEOS: within-distance through `edwithin_tgeo_geo`, region containment through +`eintersects_tgeo_geo`, and distances through `geog_distance` (see +[`MEOSBridge`](../src/main/java/berlinmod/MEOSBridge.java)). + +## Method + +The corpus is the BerlinMOD `berlinmod_instants.csv` produced by the BerlinMOD +generator — 216 075 instants, 5 vehicles, over ~11 days. Instants are stored in +EPSG:3857 and reprojected to EPSG:4326 through MEOS `geo_transform` at load (see +[`BerlinMODCorpus`](../src/main/java/berlinmod/BerlinMODCorpus.java)); the +per-query parameters (point `P` = corpus centroid, region box, road segment, +points of interest, target vehicle ids) and the window/tick granularity are +derived from the corpus so each spatial cell is selective and the matrix +produces a comparable number of windows. Each cell runs as its own Flink job +terminated by a counting sink; throughput is input events ÷ wall-clock and +`output rows` is the sink cardinality. Parallelism 1, Flink 1.16, Java 21, +16-core x86-64 Linux; libmeos built `-DMEOS=ON -DCBUFFER=ON -DNPOINT=ON +-DPOSE=ON -DRGEO=ON`. + +Run from `flink-processor/`: + +``` +LD_LIBRARY_PATH= java \ + --add-opens=java.base/java.lang=ALL-UNNAMED \ + --add-opens=java.base/java.util=ALL-UNNAMED \ + --add-opens=java.base/java.lang.reflect=ALL-UNNAMED \ + --add-opens=java.base/java.io=ALL-UNNAMED \ + --add-opens=java.base/java.time=ALL-UNNAMED \ + -cp target/classes:jar/JMEOS.jar: \ + berlinmod.BerlinMODBenchmark --csv +``` + +## Results — real BerlinMOD instants (216 075 events) + +| Cell | Events in | Output rows | Wall (ms) | Throughput (ev/s) | +|---|---:|---:|---:|---:| +| Q1-continuous | 216075 | 5 | 2508 | 86,154 | +| Q1-windowed | 216075 | 86 | 1294 | 166,982 | +| Q1-snapshot | 216075 | 274 | 1056 | 204,616 | +| Q2-continuous | 216075 | 61170 | 1074 | 201,187 | +| Q2-windowed | 216075 | 50 | 1027 | 210,394 | +| Q2-snapshot | 216075 | 71 | 985 | 219,365 | +| Q3-continuous | 216075 | 216075 | 2928 | 73,796 | +| Q3-windowed | 216075 | 86 | 2507 | 86,189 | +| Q3-snapshot | 216075 | 0 | 926 | 233,342 | +| Q4-continuous | 216075 | 62 | 3254 | 66,403 | +| Q4-windowed | 216075 | 98 | 3234 | 66,814 | +| Q4-snapshot | 216075 | 1944 | 3223 | 67,042 | +| Q5-continuous | 216075 | 73063 | 9161 | 23,586 | +| Q5-windowed | 216075 | 6 | 954 | 226,494 | +| Q5-snapshot | 216075 | 0 | 915 | 236,148 | +| Q6-continuous | 216075 | 216075 | 2382 | 90,712 | +| Q6-windowed | 216075 | 203 | 2637 | 81,940 | +| Q6-snapshot | 216075 | 274 | 2214 | 97,595 | +| Q7-continuous | 216075 | 5 | 3973 | 54,386 | +| Q7-windowed | 216075 | 53 | 5004 | 43,180 | +| Q7-snapshot | 216075 | 288 | 3931 | 54,967 | +| Q8-continuous | 216075 | 216075 | 2883 | 74,948 | +| Q8-windowed | 216075 | 86 | 2864 | 75,445 | +| Q8-snapshot | 216075 | 126 | 928 | 232,839 | +| Q9-continuous | 216075 | 107870 | 1858 | 116,294 | +| Q9-windowed | 216075 | 22 | 924 | 233,847 | +| Q9-snapshot | 216075 | 95 | 992 | 217,818 | + +## Parity — streaming continuous form ≡ batch MEOS predicate + +The continuous form emits `predicate(event)` for every event, so it is checked +event-for-event against a batch pass over the same corpus through the same +`MEOSBridge` call ([`BerlinMODParity`](../src/main/java/berlinmod/BerlinMODParity.java)). +Both spatial-membership queries match exactly. + +| Query | Events | Streaming-true | Batch-true | Mismatches | Parity | +|---|---:|---:|---:|---:|---| +| Q3 (within `d` of `P`) | 216075 | 56086 | 56086 | 0 | exact | +| Q8 (within `d` of segment) | 216075 | 118498 | 118498 | 0 | exact | + +## Characteristics + +Q5-continuous enumerates every meeting pair across all vehicles on each event +(O(V²) per event, keyed to a single subtask); it is the lowest throughput of the +matrix. The snapshot form is a sampled form — it evaluates each vehicle's +last-known position at tick instants — so a within-`P` snapshot can be empty when +no vehicle is within `d` of `P` at a tick boundary even though the continuous +form reports near-`P` events between boundaries. diff --git a/flink-processor/jar/JMEOS.jar b/flink-processor/jar/JMEOS.jar index 3c22044..066eb81 100644 Binary files a/flink-processor/jar/JMEOS.jar and b/flink-processor/jar/JMEOS.jar differ diff --git a/flink-processor/lib/libmeos.so b/flink-processor/lib/libmeos.so new file mode 100755 index 0000000..24ebd35 Binary files /dev/null and b/flink-processor/lib/libmeos.so differ diff --git a/flink-processor/pom.xml b/flink-processor/pom.xml index f4c288f..6f6b4e4 100755 --- a/flink-processor/pom.xml +++ b/flink-processor/pom.xml @@ -17,6 +17,11 @@ 3.2.0 2.17.2 21 + + true + ${project.basedir}/lib @@ -214,6 +219,252 @@ + + + org.apache.maven.plugins + maven-surefire-plugin + 3.2.5 + + + ${meos.enabled} + + + ${meos.lib.dir} + + --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED + + + + + + + cbuffer-exclude-off + + CBUFFEROFF + + + + + maven-compiler-plugin + + + **/meos/MeosOpsTCbuffer.java + **/meos/MeosOpsFreeCbuffer.java + **/meos/MeosOpsCbufferSet.java + + + **/MeosCbufferSmokeTest.java + + + + + + + + cbuffer-exclude-zero + + CBUFFER0 + + + + + maven-compiler-plugin + + + **/meos/MeosOpsTCbuffer.java + **/meos/MeosOpsFreeCbuffer.java + **/meos/MeosOpsCbufferSet.java + + + **/MeosCbufferSmokeTest.java + + + + + + + + pose-exclude-off + + POSEOFF + + + + + maven-compiler-plugin + + + **/meos/MeosOpsTPose.java + **/meos/MeosOpsFreePose.java + **/meos/MeosOpsPoseSet.java + + + **/MeosPoseSmokeTest.java + + + + + + + + pose-exclude-zero + + POSE0 + + + + + maven-compiler-plugin + + + **/meos/MeosOpsTPose.java + **/meos/MeosOpsFreePose.java + **/meos/MeosOpsPoseSet.java + + + **/MeosPoseSmokeTest.java + + + + + + + + rgeo-exclude-off + + RGEOOFF + + + + + maven-compiler-plugin + + + **/meos/MeosOpsTRGeometry.java + **/meos/MeosOpsTRGeometryInst.java + **/meos/MeosOpsFreeRgeo.java + + + + + + + + rgeo-exclude-zero + + RGEO0 + + + + + maven-compiler-plugin + + + **/meos/MeosOpsTRGeometry.java + **/meos/MeosOpsTRGeometryInst.java + **/meos/MeosOpsFreeRgeo.java + + + + + + + + h3-exclude-off + + H3OFF + + + + + maven-compiler-plugin + + + **/meos/MeosOpsTh3index.java + **/meos/MeosOpsFreeH3.java + + + + + + + + h3-exclude-zero + + H30 + + + + + maven-compiler-plugin + + + **/meos/MeosOpsTh3index.java + **/meos/MeosOpsFreeH3.java + + + + + + + + npoint-exclude-off + + NPOINTOFF + + + + + maven-compiler-plugin + + + **/meos/MeosOpsTNpoint.java + **/meos/MeosOpsTNpointInst.java + **/meos/MeosOpsFreeNpoint.java + **/meos/MeosOpsNpointSet.java + + + **/MeosNpointSmokeTest.java + + + + + + + + npoint-exclude-zero + + NPOINT0 + + + + + maven-compiler-plugin + + + **/meos/MeosOpsTNpoint.java + **/meos/MeosOpsTNpointInst.java + **/meos/MeosOpsFreeNpoint.java + **/meos/MeosOpsNpointSet.java + + + **/MeosNpointSmokeTest.java + + + + + + + \ No newline at end of file diff --git a/flink-processor/src/main/java/aisdata/AISData.java b/flink-processor/src/main/java/aisdata/AISData.java index 3153362..4d9035b 100644 --- a/flink-processor/src/main/java/aisdata/AISData.java +++ b/flink-processor/src/main/java/aisdata/AISData.java @@ -1,3 +1,28 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + package aisdata; public class AISData { diff --git a/flink-processor/src/main/java/aisdata/AISDataDeserializationSchema.java b/flink-processor/src/main/java/aisdata/AISDataDeserializationSchema.java index e61abf8..099cbf2 100644 --- a/flink-processor/src/main/java/aisdata/AISDataDeserializationSchema.java +++ b/flink-processor/src/main/java/aisdata/AISDataDeserializationSchema.java @@ -1,3 +1,28 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + package aisdata; import com.fasterxml.jackson.core.JsonParser; diff --git a/flink-processor/src/main/java/aisdata/AISTestSource.java b/flink-processor/src/main/java/aisdata/AISTestSource.java index 9cbd32e..a40dfbd 100644 --- a/flink-processor/src/main/java/aisdata/AISTestSource.java +++ b/flink-processor/src/main/java/aisdata/AISTestSource.java @@ -1,3 +1,28 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + package aisdata; import org.apache.flink.streaming.api.functions.source.SourceFunction; diff --git a/flink-processor/src/main/java/aisdata/Main.java b/flink-processor/src/main/java/aisdata/Main.java index 148caa0..6835591 100644 --- a/flink-processor/src/main/java/aisdata/Main.java +++ b/flink-processor/src/main/java/aisdata/Main.java @@ -1,3 +1,28 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + package aisdata; import java.time.Duration; @@ -56,7 +81,9 @@ public static void main(String[] args) throws Exception { // Initialize MEOS with proper error handling try { logger.info("Initializing MEOS library"); - functions.meos_initialize("UTC", errorHandler); + // JMEOS 1.4 split: no-arg meos_initialize() + separate tz + error-handler entry points + functions.meos_initialize(); + functions.meos_initialize_timezone("UTC"); final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); diff --git a/flink-processor/src/main/java/aisdata/TrajectoryWindowFunction.java b/flink-processor/src/main/java/aisdata/TrajectoryWindowFunction.java index e047143..d84babb 100644 --- a/flink-processor/src/main/java/aisdata/TrajectoryWindowFunction.java +++ b/flink-processor/src/main/java/aisdata/TrajectoryWindowFunction.java @@ -1,3 +1,28 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + package aisdata; import org.apache.flink.api.java.tuple.Tuple3; @@ -31,7 +56,9 @@ public class TrajectoryWindowFunction extends public void open(Configuration parameters) throws Exception { super.open(parameters); errorHandler = new error_handler(); // Initialize error handler here - functions.meos_initialize("UTC", errorHandler); + // JMEOS 1.4 split: no-arg meos_initialize() + separate tz + error-handler entry points + functions.meos_initialize(); + functions.meos_initialize_timezone("UTC"); logger.info("MEOS initialized in TrajectoryWindowFunction.open()"); } diff --git a/flink-processor/src/main/java/berlinmod/BerlinMODBenchmark.java b/flink-processor/src/main/java/berlinmod/BerlinMODBenchmark.java new file mode 100644 index 0000000..e56c1b0 --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/BerlinMODBenchmark.java @@ -0,0 +1,178 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; +import org.apache.flink.streaming.api.windowing.time.Time; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.LongAdder; +import java.util.function.Function; + +/** + * Throughput benchmark for the BerlinMOD-9 × 3-form streaming matrix. + * + *

Runs all 27 cells (9 queries × {continuous, windowed, snapshot} = 27 cells) + * on the Flink local mini-cluster, with the spatial predicates evaluating + * through MEOS (see {@link MEOSBridge}). Each cell runs its own job terminated by + * a counting sink; the harness records input events, output rows, wall-clock, + * and throughput (events per second), then prints a Markdown results table. + * + *

The corpus is either the real BerlinMOD instants ({@code --csv }) or a + * deterministic synthetic corpus ({@code --vehicles}/{@code --events}); the + * per-query parameters and the window/tick granularity are derived from the + * corpus by {@link BerlinMODCorpus}. + * + *

Usage (from {@code flink-processor/}, with an extended libmeos on the + * loader path and the Flink-on-Java-21 {@code --add-opens} flags): + *

+ *   java … berlinmod.BerlinMODBenchmark --csv <berlinmod_instants.csv> [--max N] [--only Q3]
+ *   java … berlinmod.BerlinMODBenchmark --vehicles 50 --events 600 [--only continuous]
+ * 
+ */ +public final class BerlinMODBenchmark { + + private static final ConcurrentHashMap COUNTS = new ConcurrentHashMap<>(); + + private BerlinMODBenchmark() { /* utility */ } + + public static void main(String[] args) throws Exception { + String csv = null, only = null; + int maxRows = 0, vehicles = 50, events = 600; + for (int i = 0; i < args.length; i++) { + switch (args[i]) { + case "--csv": csv = args[++i]; break; + case "--max": maxRows = Integer.parseInt(args[++i]); break; + case "--vehicles": vehicles = Integer.parseInt(args[++i]); break; + case "--events": events = Integer.parseInt(args[++i]); break; + case "--only": only = args[++i]; break; + default: break; + } + } + + List corpus = csv != null + ? BerlinMODCorpus.fromInstantsCsv(csv, maxRows) + : BerlinMODCorpus.synthetic(vehicles, events); + int n = corpus.size(); + BerlinMODCorpus.Params p = BerlinMODCorpus.derive(corpus); + System.out.printf("Corpus: %s, %d events; window=%ds tick=%dms; P=(%.5f,%.5f) targets=%d/%d/%d%n", + csv != null ? "real BerlinMOD instants" : "synthetic", + n, p.windowSeconds, p.snapshotTickMillis, p.pLon, p.pLat, p.targetId, p.xId, p.yId); + + Map, DataStream>> cells = new LinkedHashMap<>(); + cells.put("Q1-continuous", t -> t.keyBy(BerlinMODTrip::getVehicleId).process(new Q1ContinuousFunction())); + cells.put("Q1-windowed", t -> t.windowAll(tumble(p)).process(new Q1WindowedFunction())); + cells.put("Q1-snapshot", t -> t.keyBy(BerlinMODTrip::getVehicleId).process(new Q1SnapshotFunction(p.snapshotTickMillis))); + cells.put("Q2-continuous", t -> t.process(new Q2ContinuousFunction(p.targetId))); + cells.put("Q2-windowed", t -> t.windowAll(tumble(p)).process(new Q2WindowedFunction(p.targetId))); + cells.put("Q2-snapshot", t -> t.keyBy(BerlinMODTrip::getVehicleId).process(new Q2SnapshotFunction(p.targetId, p.snapshotTickMillis))); + cells.put("Q3-continuous", t -> t.process(new Q3ContinuousFunction(p.pLon, p.pLat, p.radiusMetres))); + cells.put("Q3-windowed", t -> t.windowAll(tumble(p)).process(new Q3WindowedFunction(p.pLon, p.pLat, p.radiusMetres))); + cells.put("Q3-snapshot", t -> t.keyBy(BerlinMODTrip::getVehicleId).process(new Q3SnapshotFunction(p.pLon, p.pLat, p.radiusMetres, p.snapshotTickMillis))); + cells.put("Q4-continuous", t -> t.keyBy(BerlinMODTrip::getVehicleId).process(new Q4ContinuousFunction(p.xmin, p.ymin, p.xmax, p.ymax))); + cells.put("Q4-windowed", t -> t.windowAll(tumble(p)).process(new Q4WindowedFunction(p.xmin, p.ymin, p.xmax, p.ymax))); + cells.put("Q4-snapshot", t -> t.keyBy(BerlinMODTrip::getVehicleId).process(new Q4SnapshotFunction(p.xmin, p.ymin, p.xmax, p.ymax, p.snapshotTickMillis))); + cells.put("Q5-continuous", t -> t.keyBy(x -> 0).process(new Q5ContinuousFunction(p.pLon, p.pLat, p.radiusMetres, p.dMeetMetres))); + cells.put("Q5-windowed", t -> t.windowAll(tumble(p)).process(new Q5WindowedFunction(p.pLon, p.pLat, p.radiusMetres, p.dMeetMetres))); + cells.put("Q5-snapshot", t -> t.keyBy(x -> 0).process(new Q5SnapshotFunction(p.pLon, p.pLat, p.radiusMetres, p.dMeetMetres, p.snapshotTickMillis))); + cells.put("Q6-continuous", t -> t.keyBy(BerlinMODTrip::getVehicleId).process(new Q6ContinuousFunction())); + cells.put("Q6-windowed", t -> t.keyBy(BerlinMODTrip::getVehicleId).window(tumble(p)).process(new Q6WindowedFunction())); + cells.put("Q6-snapshot", t -> t.keyBy(BerlinMODTrip::getVehicleId).process(new Q6SnapshotFunction(p.snapshotTickMillis))); + cells.put("Q7-continuous", t -> t.keyBy(BerlinMODTrip::getVehicleId).process(new Q7ContinuousFunction(p.pois))); + cells.put("Q7-windowed", t -> t.windowAll(tumble(p)).process(new Q7WindowedFunction(p.pois))); + cells.put("Q7-snapshot", t -> t.keyBy(BerlinMODTrip::getVehicleId).process(new Q7SnapshotFunction(p.pois, p.snapshotTickMillis))); + cells.put("Q8-continuous", t -> t.process(new Q8ContinuousFunction(p.s1Lon, p.s1Lat, p.s2Lon, p.s2Lat, p.radiusMetres))); + cells.put("Q8-windowed", t -> t.windowAll(tumble(p)).process(new Q8WindowedFunction(p.s1Lon, p.s1Lat, p.s2Lon, p.s2Lat, p.radiusMetres))); + cells.put("Q8-snapshot", t -> t.keyBy(BerlinMODTrip::getVehicleId).process(new Q8SnapshotFunction(p.s1Lon, p.s1Lat, p.s2Lon, p.s2Lat, p.radiusMetres, p.snapshotTickMillis))); + cells.put("Q9-continuous", t -> t.keyBy(x -> 0).process(new Q9ContinuousFunction(p.xId, p.yId))); + cells.put("Q9-windowed", t -> t.windowAll(tumble(p)).process(new Q9WindowedFunction(p.xId, p.yId))); + cells.put("Q9-snapshot", t -> t.keyBy(x -> 0).process(new Q9SnapshotFunction(p.xId, p.yId, p.snapshotTickMillis))); + + List rows = new ArrayList<>(); + for (Map.Entry, DataStream>> cell : cells.entrySet()) { + if (only != null && !cell.getKey().contains(only)) { + continue; + } + long[] r = runCell(cell.getKey(), cell.getValue(), corpus); + double secs = r[1] / 1000.0; + double tput = secs > 0 ? n / secs : 0; + rows.add(new String[]{cell.getKey(), String.valueOf(n), String.valueOf(r[0]), + String.valueOf(r[1]), String.format("%,.0f", tput)}); + System.out.printf(" %-14s out=%-8d %6d ms %,.0f ev/s%n", cell.getKey(), r[0], r[1], tput); + } + + System.out.println(); + System.out.println("| Cell | Events in | Output rows | Wall (ms) | Throughput (ev/s) |"); + System.out.println("|---|---:|---:|---:|---:|"); + for (String[] r : rows) { + System.out.printf("| %s | %s | %s | %s | %s |%n", r[0], r[1], r[2], r[3], r[4]); + } + } + + /** @return {outputRows, wallMillis} for one cell. */ + private static long[] runCell(String name, + Function, DataStream> wiring, + List corpus) throws Exception { + COUNTS.put(name, new LongAdder()); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + DataStream trips = env.fromCollection(corpus) + .assignTimestampsAndWatermarks(WatermarkStrategy + .forBoundedOutOfOrderness(Duration.ofSeconds(1)) + .withTimestampAssigner((e, ts) -> e.getTimestamp())); + @SuppressWarnings("unchecked") + DataStream out = (DataStream) wiring.apply(trips); + out.addSink(new CountingSink(name)); + long t0 = System.nanoTime(); + env.execute(name); + long wall = (System.nanoTime() - t0) / 1_000_000L; + return new long[]{COUNTS.get(name).sum(), wall}; + } + + private static TumblingEventTimeWindows tumble(BerlinMODCorpus.Params p) { + return TumblingEventTimeWindows.of(Time.seconds(p.windowSeconds)); + } + + /** Counts records into the shared per-cell {@link LongAdder}. */ + private static final class CountingSink extends RichSinkFunction { + private final String cell; + CountingSink(String cell) { this.cell = cell; } + @Override public void open(Configuration cfg) { } + @Override public void invoke(Object value, Context context) { + COUNTS.get(cell).increment(); + } + } +} diff --git a/flink-processor/src/main/java/berlinmod/BerlinMODCorpus.java b/flink-processor/src/main/java/berlinmod/BerlinMODCorpus.java new file mode 100644 index 0000000..282259b --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/BerlinMODCorpus.java @@ -0,0 +1,181 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import functions.GeneratedFunctions; +import jnr.ffi.Pointer; +import org.mobilitydb.flink.meos.wirings.MeosWiringRuntime; + +import java.nio.file.Files; +import java.nio.file.Paths; +import java.time.OffsetDateTime; +import java.time.format.DateTimeFormatter; +import java.time.format.DateTimeFormatterBuilder; +import java.time.temporal.ChronoField; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.TreeSet; +import java.util.stream.Stream; + +/** + * Corpus loader and query-parameter derivation for the BerlinMOD streaming + * benchmark. + * + *

Supplies either a deterministic synthetic corpus or the real BerlinMOD + * instants corpus read from the {@code berlinmod_instants.csv} produced by the + * BerlinMOD generator. Real instants are stored in EPSG:3857; they are + * reprojected to EPSG:4326 through MEOS {@code geo_transform} at load — the + * loader holds no projection mathematics of its own. + * + *

{@link Params} fixes the per-query parameters from the corpus itself (its + * centroid, bounding box, vehicle ids, and time span) so every spatial cell is + * selective and the windowing granularity yields a comparable number of windows + * regardless of the corpus time span. + */ +public final class BerlinMODCorpus { + + private static final DateTimeFormatter TS = new DateTimeFormatterBuilder() + .appendPattern("yyyy-MM-dd HH:mm:ss") + .optionalStart().appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, true).optionalEnd() + .appendOffset("+HH", "Z") + .toFormatter(); + + private BerlinMODCorpus() { /* utility */ } + + /** Query parameters derived from a corpus. */ + public static final class Params { + public final double pLon, pLat, radiusMetres, dMeetMetres; + public final double xmin, ymin, xmax, ymax; + public final double s1Lon, s1Lat, s2Lon, s2Lat; + public final List pois; + public final int targetId, xId, yId; + public final long windowSeconds, snapshotTickMillis; + + Params(double pLon, double pLat, double radiusMetres, double dMeetMetres, + double xmin, double ymin, double xmax, double ymax, + double s1Lon, double s1Lat, double s2Lon, double s2Lat, + List pois, int targetId, int xId, int yId, + long windowSeconds, long snapshotTickMillis) { + this.pLon = pLon; this.pLat = pLat; this.radiusMetres = radiusMetres; this.dMeetMetres = dMeetMetres; + this.xmin = xmin; this.ymin = ymin; this.xmax = xmax; this.ymax = ymax; + this.s1Lon = s1Lon; this.s1Lat = s1Lat; this.s2Lon = s2Lon; this.s2Lat = s2Lat; + this.pois = pois; this.targetId = targetId; this.xId = xId; this.yId = yId; + this.windowSeconds = windowSeconds; this.snapshotTickMillis = snapshotTickMillis; + } + } + + /** Deterministic synthetic corpus: vehicles on a disc around Brussels centre, + * drifting per event, with monotonically increasing timestamps. */ + public static List synthetic(int vehicles, int perVehicle) { + final double centreLon = 4.3517, centreLat = 50.8503, spread = 0.12; + final long t0 = 1_735_711_200_000L, spanMillis = 600_000L; + int total = vehicles * perVehicle; + long step = Math.max(1L, spanMillis / total); + List events = new ArrayList<>(total); + long g = 0; + for (int e = 0; e < perVehicle; e++) { + for (int v = 0; v < vehicles; v++) { + double ang = (v * 2.399963) % (2 * Math.PI); + double rad = spread * ((v % 17) / 17.0); + double drift = 0.0005 * Math.sin((e + v) * 0.13); + events.add(make(100 + v, t0 + g * step, + centreLon + rad * Math.cos(ang) + drift, + centreLat + rad * Math.sin(ang) + drift)); + g++; + } + } + return events; + } + + /** Real BerlinMOD instants from {@code berlinmod_instants.csv} + * (columns {@code tripid,vehid,day,seqno,geom,t}), reprojected 3857→4326 + * through MEOS, sorted by timestamp. {@code maxRows <= 0} loads all rows. */ + public static List fromInstantsCsv(String path, int maxRows) throws Exception { + MeosWiringRuntime.ensureInitializedOnThread(); + List events = new ArrayList<>(); + try (Stream lines = Files.lines(Paths.get(path))) { + java.util.Iterator it = lines.iterator(); + if (it.hasNext()) { + it.next(); // header + } + while (it.hasNext() && (maxRows <= 0 || events.size() < maxRows)) { + String[] f = it.next().split(","); + int vid = Integer.parseInt(f[1].trim()); + long ms = OffsetDateTime.parse(f[5].trim(), TS).toInstant().toEpochMilli(); + Pointer g4326 = GeneratedFunctions.geo_transform( + GeneratedFunctions.geom_in(f[4].trim(), -1), 4326); + String txt = GeneratedFunctions.geo_as_text(g4326, 7); // POINT(lon lat) + String[] xy = txt.substring(txt.indexOf('(') + 1, txt.indexOf(')')).trim().split("\\s+"); + events.add(make(vid, ms, Double.parseDouble(xy[0]), Double.parseDouble(xy[1]))); + } + } + events.sort((a, b) -> Long.compare(a.getTimestamp(), b.getTimestamp())); + return events; + } + + /** Derive selective per-query parameters and a window/tick granularity that + * yields ~200 windows over the corpus time span. */ + public static Params derive(List corpus) { + double sumLon = 0, sumLat = 0, minLon = Double.MAX_VALUE, minLat = Double.MAX_VALUE, + maxLon = -Double.MAX_VALUE, maxLat = -Double.MAX_VALUE, minT = Double.MAX_VALUE, maxT = -Double.MAX_VALUE; + TreeSet ids = new TreeSet<>(); + for (BerlinMODTrip t : corpus) { + sumLon += t.getLon(); sumLat += t.getLat(); + minLon = Math.min(minLon, t.getLon()); maxLon = Math.max(maxLon, t.getLon()); + minLat = Math.min(minLat, t.getLat()); maxLat = Math.max(maxLat, t.getLat()); + minT = Math.min(minT, t.getTimestamp()); maxT = Math.max(maxT, t.getTimestamp()); + ids.add(t.getVehicleId()); + } + int n = corpus.size(); + double cLon = sumLon / n, cLat = sumLat / n; + double exLon = maxLon - minLon, exLat = maxLat - minLat; + List idList = new ArrayList<>(ids); + int targetId = idList.get(idList.size() / 2); + int xId = idList.get(0); + int yId = idList.get(Math.min(idList.size() - 1, idList.size() / 2)); + long span = (long) (maxT - minT); + long windowSeconds = Math.max(1L, span / 1000 / 200); + long tickMillis = Math.max(1000L, windowSeconds * 1000L / 2); + List pois = Arrays.asList( + new PointOfInterest(1, cLon, cLat, 2_000.0), + new PointOfInterest(2, cLon + 0.1 * exLon, cLat + 0.1 * exLat, 1_000.0), + new PointOfInterest(3, cLon - 0.1 * exLon, cLat - 0.1 * exLat, 2_000.0)); + return new Params(cLon, cLat, 5_000.0, 5_000.0, + cLon - 0.25 * exLon, cLat - 0.25 * exLat, cLon + 0.25 * exLon, cLat + 0.25 * exLat, + minLon + 0.25 * exLon, cLat, maxLon - 0.25 * exLon, cLat, + pois, targetId, xId, yId, windowSeconds, tickMillis); + } + + private static BerlinMODTrip make(int vid, long t, double lon, double lat) { + BerlinMODTrip trip = new BerlinMODTrip(); + trip.setVehicleId(vid); + trip.setTimestamp(t); + trip.setLon(lon); + trip.setLat(lat); + return trip; + } +} diff --git a/flink-processor/src/main/java/berlinmod/BerlinMODDeserializationSchema.java b/flink-processor/src/main/java/berlinmod/BerlinMODDeserializationSchema.java new file mode 100644 index 0000000..a3b9b9e --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/BerlinMODDeserializationSchema.java @@ -0,0 +1,89 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.TypeExtractor; + +import java.io.IOException; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.format.DateTimeFormatter; + +/** + * JSON → {@link BerlinMODTrip} deserializer for the Kafka {@code berlinmod} topic. + * + *

Expected JSON shape per record: + *

+ *   { "t": "2007-05-28 06:00:00", "vehicle_id": 42, "lon": 4.36, "lat": 50.84 }
+ * 
+ * + *

The timestamp format is the same {@code yyyy-MM-dd HH:mm:ss} the BerlinMOD + * generator emits in {@code generate_berlinmod_trips.sql}; we parse it as UTC + * to match the AIS pipeline's event-time convention. + */ +public class BerlinMODDeserializationSchema implements DeserializationSchema { + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final DateTimeFormatter TS_FORMATTER = + DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss"); + + public BerlinMODDeserializationSchema() { + OBJECT_MAPPER.configure(JsonParser.Feature.ALLOW_NON_NUMERIC_NUMBERS, true); + OBJECT_MAPPER.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + } + + @Override + public BerlinMODTrip deserialize(byte[] message) throws IOException { + JsonNode node = OBJECT_MAPPER.readTree(message); + BerlinMODTrip trip = new BerlinMODTrip(); + trip.setTimestamp(parseTimestamp(node.get("t").asText())); + trip.setVehicleId(node.get("vehicle_id").asInt()); + trip.setLon(node.get("lon").asDouble()); + trip.setLat(node.get("lat").asDouble()); + return trip; + } + + private long parseTimestamp(String s) { + LocalDateTime dt = LocalDateTime.parse(s, TS_FORMATTER); + return dt.atZone(ZoneId.of("UTC")).toInstant().toEpochMilli(); + } + + @Override + public boolean isEndOfStream(BerlinMODTrip nextElement) { + return false; + } + + @Override + public TypeInformation getProducedType() { + return TypeExtractor.getForClass(BerlinMODTrip.class); + } +} diff --git a/flink-processor/src/main/java/berlinmod/BerlinMODParity.java b/flink-processor/src/main/java/berlinmod/BerlinMODParity.java new file mode 100644 index 0000000..29a798f --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/BerlinMODParity.java @@ -0,0 +1,154 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.function.BiPredicate; + +/** + * Snapshot-form parity check for the BerlinMOD streaming benchmark. + * + *

The streaming parity contract is that a streaming query computes the same + * result as a batch evaluation of the same MEOS predicate. This driver verifies + * it on the continuous form, which is timing-independent: the continuous form + * emits {@code predicate(event)} for every event, and a batch pass over the same + * corpus computes {@code predicate(event)} directly through the same + * {@link MEOSBridge} call. The two must agree event-for-event. + * + *

Checked queries: Q3 (within {@code d} of point {@code P}, MEOS + * {@code edwithin_tgeo_geo}) and Q8 (within {@code d} of a road segment, MEOS + * {@code edwithin_tgeo_geo} against a line). The corpus and parameters are the + * same as {@link BerlinMODBenchmark}. + * + *

+ *   java … berlinmod.BerlinMODParity --csv <berlinmod_instants.csv> [--max N]
+ *   java … berlinmod.BerlinMODParity --vehicles 50 --events 600
+ * 
+ */ +public final class BerlinMODParity { + + // Continuous-form outputs in arrival order (parallelism 1, no keyBy → stream + // order equals corpus order), so element i corresponds to corpus event i. + private static final ConcurrentLinkedQueue STREAMED = new ConcurrentLinkedQueue<>(); + + private BerlinMODParity() { /* utility */ } + + public static void main(String[] args) throws Exception { + String csv = null; + int maxRows = 0, vehicles = 50, events = 600; + for (int i = 0; i < args.length; i++) { + switch (args[i]) { + case "--csv": csv = args[++i]; break; + case "--max": maxRows = Integer.parseInt(args[++i]); break; + case "--vehicles": vehicles = Integer.parseInt(args[++i]); break; + case "--events": events = Integer.parseInt(args[++i]); break; + default: break; + } + } + List corpus = csv != null + ? BerlinMODCorpus.fromInstantsCsv(csv, maxRows) + : BerlinMODCorpus.synthetic(vehicles, events); + BerlinMODCorpus.Params p = BerlinMODCorpus.derive(corpus); + System.out.printf("Corpus: %s, %d events; P=(%.5f,%.5f) r=%.0fm%n", + csv != null ? "real BerlinMOD instants" : "synthetic", corpus.size(), + p.pLon, p.pLat, p.radiusMetres); + + MeosWiringInit(); + BiPredicate q3 = (lon, lat) -> + MEOSBridge.dwithinMetres(lon, lat, p.pLon, p.pLat, p.radiusMetres); + BiPredicate q8 = (lon, lat) -> + MEOSBridge.dwithinSegmentMetres(lon, lat, p.s1Lon, p.s1Lat, p.s2Lon, p.s2Lat, p.radiusMetres); + + List rows = new ArrayList<>(); + rows.add(check("Q3", corpus, + t -> t.process(new Q3ContinuousFunction(p.pLon, p.pLat, p.radiusMetres)), q3)); + rows.add(check("Q8", corpus, + t -> t.process(new Q8ContinuousFunction(p.s1Lon, p.s1Lat, p.s2Lon, p.s2Lat, p.radiusMetres)), q8)); + + System.out.println(); + System.out.println("| Query | Events | Streaming-true | Batch-true | Mismatches | Parity |"); + System.out.println("|---|---:|---:|---:|---:|---|"); + for (String[] r : rows) { + System.out.printf("| %s | %s | %s | %s | %s | %s |%n", r[0], r[1], r[2], r[3], r[4], r[5]); + } + } + + private static String[] check(String query, List corpus, + java.util.function.Function, DataStream>> wiring, + BiPredicate batch) throws Exception { + STREAMED.clear(); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + DataStream trips = env.fromCollection(corpus) + .assignTimestampsAndWatermarks(WatermarkStrategy + .forBoundedOutOfOrderness(Duration.ofSeconds(1)) + .withTimestampAssigner((e, ts) -> e.getTimestamp())); + wiring.apply(trips).addSink(new CollectSink()); + env.execute("parity-" + query); + + Boolean[] streamed = STREAMED.toArray(new Boolean[0]); + long streamingTrue = 0, batchTrue = 0, mismatches = 0; + for (int i = 0; i < corpus.size(); i++) { + boolean expected = batch.test(corpus.get(i).getLon(), corpus.get(i).getLat()); + if (expected) { + batchTrue++; + } + if (i < streamed.length && streamed[i]) { + streamingTrue++; + } + if (i >= streamed.length || streamed[i].booleanValue() != expected) { + mismatches++; + } + } + boolean parity = mismatches == 0 && streamed.length == corpus.size(); + System.out.printf(" %s: events=%d streaming-out=%d streaming-true=%d batch-true=%d mismatches=%d parity=%s%n", + query, corpus.size(), streamed.length, streamingTrue, batchTrue, mismatches, parity ? "YES" : "NO"); + return new String[]{query, String.valueOf(corpus.size()), String.valueOf(streamingTrue), + String.valueOf(batchTrue), String.valueOf(mismatches), parity ? "exact" : "MISMATCH"}; + } + + private static void MeosWiringInit() { + org.mobilitydb.flink.meos.wirings.MeosWiringRuntime.ensureInitializedOnThread(); + } + + /** Records each continuous-form output's {@code near} flag in arrival order. */ + private static final class CollectSink extends RichSinkFunction> { + @Override public void open(Configuration cfg) { } + @Override public void invoke(Tuple3 v, Context context) { + STREAMED.add(v.f2); + } + } +} diff --git a/flink-processor/src/main/java/berlinmod/BerlinMODQ1LocalTest.java b/flink-processor/src/main/java/berlinmod/BerlinMODQ1LocalTest.java new file mode 100644 index 0000000..29dfead --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/BerlinMODQ1LocalTest.java @@ -0,0 +1,120 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; + +/** + * Local end-to-end test driver for the BerlinMOD-Q1 three streaming forms. + * + *

Same 21-event synthetic corpus as Q2/Q3 local tests. Q1 has no spatial + * predicate and no per-event filter parameter — it simply enumerates vehicles + * seen. + * + *

Expected output: + *

    + *
  • Q1-continuous: 3 lines, one per distinct vehicle (firstSeenTime)
  • + *
  • Q1-windowed: 2 windows, each with distinctCount=3
  • + *
  • Q1-snapshot: 9 lines (3 ticks × 3 vehicles all seen by source-close)
  • + *
+ */ +public class BerlinMODQ1LocalTest { + + private static final Logger LOG = LoggerFactory.getLogger(BerlinMODQ1LocalTest.class); + + private static final long WINDOW_SIZE_SECONDS = 10L; + private static final long SNAPSHOT_TICK_MILLIS = 5_000L; + private static final long T0 = 1_735_711_200_000L; + + public static void main(String[] args) throws Exception { + LOG.info("BerlinMODQ1LocalTest starting; window={}s tick={}ms", + WINDOW_SIZE_SECONDS, SNAPSHOT_TICK_MILLIS); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + List events = buildEvents(); + DataStreamSource raw = env.fromCollection(events); + DataStream trips = raw.assignTimestampsAndWatermarks( + WatermarkStrategy + .forBoundedOutOfOrderness(Duration.ofSeconds(1)) + .withTimestampAssigner((e, t) -> e.getTimestamp())); + + DataStream> cont = trips + .keyBy(BerlinMODTrip::getVehicleId) + .process(new Q1ContinuousFunction()); + cont.print("Q1-continuous"); + + DataStream> win = trips + .windowAll(TumblingEventTimeWindows.of(Time.seconds(WINDOW_SIZE_SECONDS))) + .process(new Q1WindowedFunction()); + win.print("Q1-windowed"); + + DataStream> snap = trips + .keyBy(BerlinMODTrip::getVehicleId) + .process(new Q1SnapshotFunction(SNAPSHOT_TICK_MILLIS)); + snap.print("Q1-snapshot"); + + env.execute("BerlinMODQ1LocalTest"); + LOG.info("BerlinMODQ1LocalTest done"); + } + + private static List buildEvents() { + List events = new ArrayList<>(); + for (int i = 0; i <= 12; i += 2) { + events.add(make(100, T0 + i * 1000L, 4.3517, 50.8503)); + } + for (int i = 1; i <= 13; i += 2) { + events.add(make(200, T0 + i * 1000L, 4.3060, 50.8270)); + } + for (int i = 0; i <= 12; i += 2) { + events.add(make(300, T0 + i * 1000L, 4.2000, 50.7500)); + } + return events; + } + + private static BerlinMODTrip make(int vid, long t, double lon, double lat) { + BerlinMODTrip trip = new BerlinMODTrip(); + trip.setVehicleId(vid); + trip.setTimestamp(t); + trip.setLon(lon); + trip.setLat(lat); + return trip; + } +} diff --git a/flink-processor/src/main/java/berlinmod/BerlinMODQ2LocalTest.java b/flink-processor/src/main/java/berlinmod/BerlinMODQ2LocalTest.java new file mode 100644 index 0000000..6890d7d --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/BerlinMODQ2LocalTest.java @@ -0,0 +1,129 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.java.tuple.Tuple4; +import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; + +/** + * Local end-to-end test driver for the BerlinMOD-Q2 three streaming forms. + * + *

Same structural shape as {@link BerlinMODQ3LocalTest} but exercises Q2 ("where is vehicle X + * at time T?") with {@code X = 200} (the Anderlecht vehicle). Same 3-vehicle / + * 21-event synthetic corpus. + * + *

Expected output shape (with {@code X = 200}): + *

    + *
  • Q2-continuous: 7 events (the 7 vehicle-200 events; vehicles 100 and 300 filtered out)
  • + *
  • Q2-windowed: 2 windows of size 10 s, each emitting the last vehicle-200 position seen in the window
  • + *
  • Q2-snapshot: 3 ticks × 1 emission each = 3 lines (vehicle 200's last-known position at each 5 s tick)
  • + *
+ * + *

Run after {@code mvn package} with: + *

+ *   java -cp target/flink-kafka2postgres-1.0-SNAPSHOT.jar berlinmod.BerlinMODQ2LocalTest
+ * 
+ */ +public class BerlinMODQ2LocalTest { + + private static final Logger LOG = LoggerFactory.getLogger(BerlinMODQ2LocalTest.class); + + private static final int TARGET_VEHICLE_ID = 200; + private static final long WINDOW_SIZE_SECONDS = 10L; + private static final long SNAPSHOT_TICK_MILLIS = 5_000L; + private static final long T0 = 1_735_711_200_000L; // 2025-01-01 06:00:00 UTC + + public static void main(String[] args) throws Exception { + LOG.info("BerlinMODQ2LocalTest starting; X={} window={}s tick={}ms", + TARGET_VEHICLE_ID, WINDOW_SIZE_SECONDS, SNAPSHOT_TICK_MILLIS); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); // deterministic output ordering + + List events = buildEvents(); + DataStreamSource raw = env.fromCollection(events); + DataStream trips = raw.assignTimestampsAndWatermarks( + WatermarkStrategy + .forBoundedOutOfOrderness(Duration.ofSeconds(1)) + .withTimestampAssigner((e, t) -> e.getTimestamp())); + + DataStream cont = trips + .process(new Q2ContinuousFunction(TARGET_VEHICLE_ID)); + cont.map(t -> String.format("v=%d t=%d (%.4f,%.4f)", + t.getVehicleId(), t.getTimestamp(), t.getLon(), t.getLat())) + .print("Q2-continuous"); + + DataStream> win = trips + .windowAll(TumblingEventTimeWindows.of(Time.seconds(WINDOW_SIZE_SECONDS))) + .process(new Q2WindowedFunction(TARGET_VEHICLE_ID)); + win.print("Q2-windowed"); + + DataStream> snap = trips + .keyBy(BerlinMODTrip::getVehicleId) + .process(new Q2SnapshotFunction(TARGET_VEHICLE_ID, SNAPSHOT_TICK_MILLIS)); + snap.print("Q2-snapshot"); + + env.execute("BerlinMODQ2LocalTest"); + LOG.info("BerlinMODQ2LocalTest done"); + } + + private static List buildEvents() { + List events = new ArrayList<>(); + // Same synthetic corpus as Q3LocalTest, so any user can run both and + // see them work over identical inputs. + for (int i = 0; i <= 12; i += 2) { + events.add(make(100, T0 + i * 1000L, 4.3517, 50.8503)); + } + for (int i = 1; i <= 13; i += 2) { + events.add(make(200, T0 + i * 1000L, 4.3060, 50.8270)); + } + for (int i = 0; i <= 12; i += 2) { + events.add(make(300, T0 + i * 1000L, 4.2000, 50.7500)); + } + return events; + } + + private static BerlinMODTrip make(int vid, long t, double lon, double lat) { + BerlinMODTrip trip = new BerlinMODTrip(); + trip.setVehicleId(vid); + trip.setTimestamp(t); + trip.setLon(lon); + trip.setLat(lat); + return trip; + } +} diff --git a/flink-processor/src/main/java/berlinmod/BerlinMODQ2Main.java b/flink-processor/src/main/java/berlinmod/BerlinMODQ2Main.java new file mode 100644 index 0000000..aae3729 --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/BerlinMODQ2Main.java @@ -0,0 +1,133 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.common.eventtime.SerializableTimestampAssigner; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.serialization.SimpleStringSchema; +import org.apache.flink.api.java.tuple.Tuple4; +import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.connector.kafka.source.KafkaSource; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; + +/** + * Entry point for the BerlinMOD-Q2 scaffold on MobilityFlink. + * + *

Runs the three streaming forms of BerlinMOD-Q2 ("where is vehicle X at + * time T?") side by side over the same Kafka input topic {@code berlinmod}: + *

    + *
  • {@link Q2ContinuousFunction} — emit every event of vehicle X as it arrives
  • + *
  • {@link Q2WindowedFunction} — last-known (lon, lat) of vehicle X per N-second tumbling window
  • + *
  • {@link Q2SnapshotFunction} — vehicle X's last-known (lon, lat) at each watermark tick; + * the parity-oracle form (≡ batch BerlinMOD-Q2 at the same scale factor)
  • + *
+ * + *

The queried vehicle id and other defaults match + * {@code doc/berlinmod-q3-streaming-forms.md}. The companion local test driver + * is {@link BerlinMODQ2LocalTest}. + */ +public class BerlinMODQ2Main { + + private static final Logger LOG = LoggerFactory.getLogger(BerlinMODQ2Main.class); + + // Default Q2 parameters — query vehicle 200 (Anderlecht), 10 s windows, + // 5 s snapshot tick. Matches the synthetic-corpus defaults. + private static final int TARGET_VEHICLE_ID = 200; + private static final long WINDOW_SIZE_SECONDS = 10L; + private static final long SNAPSHOT_TICK_MILLIS = 5_000L; + private static final String KAFKA_TOPIC = "berlinmod"; + private static final String KAFKA_BOOTSTRAP = "kafka:29092"; + private static final String CONSUMER_GROUP = "flink_berlinmod_q2"; + + public static void main(String[] args) throws Exception { + LOG.info("BerlinMODQ2Main starting; X={} window={}s tick={}ms", + TARGET_VEHICLE_ID, WINDOW_SIZE_SECONDS, SNAPSHOT_TICK_MILLIS); + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + KafkaSource kafkaSource = KafkaSource.builder() + .setBootstrapServers(KAFKA_BOOTSTRAP) + .setGroupId(CONSUMER_GROUP) + .setTopics(KAFKA_TOPIC) + .setStartingOffsets(OffsetsInitializer.earliest()) + .setValueOnlyDeserializer(new SimpleStringSchema()) + .build(); + + DataStream raw = env.fromSource( + kafkaSource, WatermarkStrategy.noWatermarks(), "Kafka Source (berlinmod)"); + + DataStream trips = raw + .map(new DeserializeBerlinMODMapFunction()) + .assignTimestampsAndWatermarks( + WatermarkStrategy.forBoundedOutOfOrderness(Duration.ofSeconds(10)) + .withTimestampAssigner(new BerlinMODTimestampAssigner()) + .withIdleness(Duration.ofMinutes(1)) + ); + + // Continuous form — per-event pass-through for the queried vehicle + DataStream continuous = trips + .process(new Q2ContinuousFunction(TARGET_VEHICLE_ID)); + continuous.print("Q2-continuous"); + + // Windowed form — last-known (lon, lat) per tumbling window + DataStream> windowed = trips + .windowAll(TumblingEventTimeWindows.of(Time.seconds(WINDOW_SIZE_SECONDS))) + .process(new Q2WindowedFunction(TARGET_VEHICLE_ID)); + windowed.print("Q2-windowed"); + + // Snapshot form — keyed by vehicle, emits queried vehicle's last position at each tick + DataStream> snapshot = trips + .keyBy(BerlinMODTrip::getVehicleId) + .process(new Q2SnapshotFunction(TARGET_VEHICLE_ID, SNAPSHOT_TICK_MILLIS)); + snapshot.print("Q2-snapshot"); + + env.execute("BerlinMOD-Q2 (continuous / windowed / snapshot)"); + LOG.info("BerlinMODQ2Main done"); + } + + public static class DeserializeBerlinMODMapFunction implements MapFunction { + @Override + public BerlinMODTrip map(String value) throws Exception { + return new BerlinMODDeserializationSchema().deserialize(value.getBytes()); + } + } + + public static class BerlinMODTimestampAssigner implements SerializableTimestampAssigner { + @Override + public long extractTimestamp(BerlinMODTrip element, long recordTimestamp) { + return element.getTimestamp(); + } + } +} diff --git a/flink-processor/src/main/java/berlinmod/BerlinMODQ3LocalTest.java b/flink-processor/src/main/java/berlinmod/BerlinMODQ3LocalTest.java new file mode 100644 index 0000000..3cee721 --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/BerlinMODQ3LocalTest.java @@ -0,0 +1,139 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; + +/** + * Local end-to-end test driver for the BerlinMOD-Q3 three streaming forms. + * + *

Runs the same three form functions {@link BerlinMODQ3Main} runs (continuous, + * windowed, snapshot) but reads from a hardcoded synthetic event list via + * {@code env.fromCollection(...)} instead of from Kafka. This lets the scaffold + * be verified on any machine with Java + Maven, without Docker, a Kafka broker, + * the MEOS native lib, or any JMEOS call. + * + *

Synthetic corpus: 3 vehicles, 21 events over 14 simulated seconds — + *

    + *
  • Vehicle 100 — sits on Brussels city centre {@code P}, distance 0 m, near
  • + *
  • Vehicle 200 — Anderlecht, ~4.1 km from {@code P}, near (within the 5 km radius)
  • + *
  • Vehicle 300 — Forest, ~15.4 km from {@code P}, not near (outside the 5 km radius)
  • + *
+ * + *

Expected output shape: + *

    + *
  • Q3-continuous: 21 lines, {@code near=true} for vehicles 100 and 200, {@code false} for 300
  • + *
  • Q3-windowed: 2 windows of size 10 s, each with {@code distinctCount=2} (vehicles 100 and 200)
  • + *
  • Q3-snapshot: 3 ticks × 2 near vehicles = 6 lines (vehicles 100 and 200 at each of the three 5 s ticks)
  • + *
+ * + *

Run after {@code mvn package} with: + *

+ *   java -cp target/flink-kafka2postgres-1.0-SNAPSHOT.jar berlinmod.BerlinMODQ3LocalTest
+ * 
+ */ +public class BerlinMODQ3LocalTest { + + private static final Logger LOG = LoggerFactory.getLogger(BerlinMODQ3LocalTest.class); + + private static final double P_LON = 4.3517; + private static final double P_LAT = 50.8503; + private static final double RADIUS_METRES = 5_000.0; + private static final long WINDOW_SIZE_SECONDS = 10L; + private static final long SNAPSHOT_TICK_MILLIS = 5_000L; + private static final long T0 = 1_735_711_200_000L; // 2025-01-01 06:00:00 UTC + + public static void main(String[] args) throws Exception { + LOG.info("BerlinMODQ3LocalTest starting; P=({}, {}) radius={}m window={}s tick={}ms", + P_LON, P_LAT, RADIUS_METRES, WINDOW_SIZE_SECONDS, SNAPSHOT_TICK_MILLIS); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); // deterministic output ordering for the test + + List events = buildEvents(); + DataStreamSource raw = env.fromCollection(events); + DataStream trips = raw.assignTimestampsAndWatermarks( + WatermarkStrategy + .forBoundedOutOfOrderness(Duration.ofSeconds(1)) + .withTimestampAssigner((e, t) -> e.getTimestamp())); + + DataStream> cont = trips + .process(new Q3ContinuousFunction(P_LON, P_LAT, RADIUS_METRES)); + cont.print("Q3-continuous"); + + DataStream> win = trips + .windowAll(TumblingEventTimeWindows.of(Time.seconds(WINDOW_SIZE_SECONDS))) + .process(new Q3WindowedFunction(P_LON, P_LAT, RADIUS_METRES)); + win.print("Q3-windowed"); + + DataStream> snap = trips + .keyBy(BerlinMODTrip::getVehicleId) + .process(new Q3SnapshotFunction(P_LON, P_LAT, RADIUS_METRES, SNAPSHOT_TICK_MILLIS)); + snap.print("Q3-snapshot"); + + env.execute("BerlinMODQ3LocalTest"); + LOG.info("BerlinMODQ3LocalTest done"); + } + + private static List buildEvents() { + List events = new ArrayList<>(); + // Vehicle 100 — Brussels city centre (= P), 7 events at t0, t0+2s, …, t0+12s + for (int i = 0; i <= 12; i += 2) { + events.add(make(100, T0 + i * 1000L, 4.3517, 50.8503)); + } + // Vehicle 200 — Anderlecht ~4.1 km from P, 7 events at t0+1s, t0+3s, …, t0+13s + for (int i = 1; i <= 13; i += 2) { + events.add(make(200, T0 + i * 1000L, 4.3060, 50.8270)); + } + // Vehicle 300 — Forest ~15.4 km from P, 7 events at t0, t0+2s, …, t0+12s + for (int i = 0; i <= 12; i += 2) { + events.add(make(300, T0 + i * 1000L, 4.2000, 50.7500)); + } + return events; + } + + private static BerlinMODTrip make(int vid, long t, double lon, double lat) { + BerlinMODTrip trip = new BerlinMODTrip(); + trip.setVehicleId(vid); + trip.setTimestamp(t); + trip.setLon(lon); + trip.setLat(lat); + return trip; + } +} diff --git a/flink-processor/src/main/java/berlinmod/BerlinMODQ3Main.java b/flink-processor/src/main/java/berlinmod/BerlinMODQ3Main.java new file mode 100644 index 0000000..7c44e08 --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/BerlinMODQ3Main.java @@ -0,0 +1,136 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.common.eventtime.SerializableTimestampAssigner; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.serialization.SimpleStringSchema; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.connector.kafka.source.KafkaSource; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; + +/** + * Entry point for the BerlinMOD-Q3 scaffold on MobilityFlink. + * + *

Runs the three streaming forms of BerlinMOD-Q3 side by side over the same + * Kafka input topic {@code berlinmod}: + *

    + *
  • {@link Q3ContinuousFunction} — per-event near/not-near
  • + *
  • {@link Q3WindowedFunction} — distinct-count per N-second tumbling window
  • + *
  • {@link Q3SnapshotFunction} — set of vehicles near P at each watermark tick + * (the parity-oracle form; ≡ batch BerlinMOD-Q3 at the same scale factor)
  • + *
+ * + *

Reference point P, radius {@code d}, window size and snapshot tick are + * the hardcoded defaults from the BerlinMOD-Q3 streaming-forms spec (see + * {@code doc/berlinmod-q3-streaming-forms.md}). The Kafka producer is the + * companion {@code kafka-producer/python-producer-berlinmod.py}. + */ +public class BerlinMODQ3Main { + + private static final Logger LOG = LoggerFactory.getLogger(BerlinMODQ3Main.class); + + // Default Q3 parameters — Brussels city centre, 5 km radius, 10 s windows, + // 5 s snapshot tick. Matches the defaults in the spec doc. + private static final double P_LON = 4.3517; + private static final double P_LAT = 50.8503; + private static final double RADIUS_METRES = 5_000.0; + private static final long WINDOW_SIZE_SECONDS = 10L; + private static final long SNAPSHOT_TICK_MILLIS = 5_000L; + private static final String KAFKA_TOPIC = "berlinmod"; + private static final String KAFKA_BOOTSTRAP = "kafka:29092"; + private static final String CONSUMER_GROUP = "flink_berlinmod_q3"; + + public static void main(String[] args) throws Exception { + LOG.info("BerlinMODQ3Main starting; P=({}, {}) radius={}m window={}s tick={}ms", + P_LON, P_LAT, RADIUS_METRES, WINDOW_SIZE_SECONDS, SNAPSHOT_TICK_MILLIS); + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + KafkaSource kafkaSource = KafkaSource.builder() + .setBootstrapServers(KAFKA_BOOTSTRAP) + .setGroupId(CONSUMER_GROUP) + .setTopics(KAFKA_TOPIC) + .setStartingOffsets(OffsetsInitializer.earliest()) + .setValueOnlyDeserializer(new SimpleStringSchema()) + .build(); + + DataStream raw = env.fromSource( + kafkaSource, WatermarkStrategy.noWatermarks(), "Kafka Source (berlinmod)"); + + DataStream trips = raw + .map(new DeserializeBerlinMODMapFunction()) + .assignTimestampsAndWatermarks( + WatermarkStrategy.forBoundedOutOfOrderness(Duration.ofSeconds(10)) + .withTimestampAssigner(new BerlinMODTimestampAssigner()) + .withIdleness(Duration.ofMinutes(1)) + ); + + // Continuous form — per-event near/not-near + DataStream> continuous = trips + .process(new Q3ContinuousFunction(P_LON, P_LAT, RADIUS_METRES)); + continuous.print("Q3-continuous"); + + // Windowed form — distinct count per tumbling window + DataStream> windowed = trips + .windowAll(TumblingEventTimeWindows.of(Time.seconds(WINDOW_SIZE_SECONDS))) + .process(new Q3WindowedFunction(P_LON, P_LAT, RADIUS_METRES)); + windowed.print("Q3-windowed"); + + // Snapshot form — keyed by vehicle, emits set of vehicles near P at each tick + DataStream> snapshot = trips + .keyBy(BerlinMODTrip::getVehicleId) + .process(new Q3SnapshotFunction(P_LON, P_LAT, RADIUS_METRES, SNAPSHOT_TICK_MILLIS)); + snapshot.print("Q3-snapshot"); + + env.execute("BerlinMOD-Q3 (continuous / windowed / snapshot)"); + LOG.info("BerlinMODQ3Main done"); + } + + public static class DeserializeBerlinMODMapFunction implements MapFunction { + @Override + public BerlinMODTrip map(String value) throws Exception { + return new BerlinMODDeserializationSchema().deserialize(value.getBytes()); + } + } + + public static class BerlinMODTimestampAssigner implements SerializableTimestampAssigner { + @Override + public long extractTimestamp(BerlinMODTrip element, long recordTimestamp) { + return element.getTimestamp(); + } + } +} diff --git a/flink-processor/src/main/java/berlinmod/BerlinMODQ4LocalTest.java b/flink-processor/src/main/java/berlinmod/BerlinMODQ4LocalTest.java new file mode 100644 index 0000000..583c258 --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/BerlinMODQ4LocalTest.java @@ -0,0 +1,155 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.api.java.tuple.Tuple4; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; + +/** + * Local end-to-end test driver for the BerlinMOD-Q4 three streaming forms. + * + *

Region R = bounding box {@code (4.30, 50.84, 4.36, 50.86)} — a rectangle + * around Brussels city centre. The synthetic corpus is designed to produce + * multiple outside → inside transitions so the entry-detection logic + * is exercised non-trivially: + * + *

    + *
  • Vehicle 100 sits inside R for all 7 events (no transitions).
  • + *
  • Vehicle 200 oscillates: outside at t=1, inside at t=3 (entry), + * outside at t=5, inside at t=7 (entry), outside at t=9, inside at + * t=11 (entry), outside at t=13 → three entries.
  • + *
  • Vehicle 300 stays in Forest (outside R) for all 7 events.
  • + *
+ * + *

Expected output: + *

    + *
  • Q4-continuous: 3 entries (v200's three outside → inside transitions)
  • + *
  • Q4-windowed: per the intra-window scoping convention — window + * [0, 10 s) contains v100's first-seen-inside event AND v200's two entries + * (t=3, t=7); window [10, 20 s) contains v100's first-event-in-window + * AND v200's third entry (t=11). 5 emissions total.
  • + *
  • Q4-snapshot: cumulative entries up to each tick. Tick 5: 1 + * (v200 t=3). Tick 10: 2 (v200 t=3, t=7). Tick 15: 3 (v200 t=3, t=7, + * t=11). v100 contributes 0 (always inside, no transition). v300 + * contributes 0. 6 emissions total (1+2+3).
  • + *
+ */ +public class BerlinMODQ4LocalTest { + + private static final Logger LOG = LoggerFactory.getLogger(BerlinMODQ4LocalTest.class); + + // Region R — Brussels centre rectangle + private static final double XMIN = 4.30; + private static final double YMIN = 50.84; + private static final double XMAX = 4.36; + private static final double YMAX = 50.86; + + private static final long WINDOW_SIZE_SECONDS = 10L; + private static final long SNAPSHOT_TICK_MILLIS = 5_000L; + private static final long T0 = 1_735_711_200_000L; + + public static void main(String[] args) throws Exception { + LOG.info("BerlinMODQ4LocalTest starting; R=({},{},{},{}) window={}s tick={}ms", + XMIN, YMIN, XMAX, YMAX, WINDOW_SIZE_SECONDS, SNAPSHOT_TICK_MILLIS); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + List events = buildEvents(); + DataStreamSource raw = env.fromCollection(events); + DataStream trips = raw.assignTimestampsAndWatermarks( + WatermarkStrategy + .forBoundedOutOfOrderness(Duration.ofSeconds(1)) + .withTimestampAssigner((e, t) -> e.getTimestamp())); + + DataStream> cont = trips + .keyBy(BerlinMODTrip::getVehicleId) + .process(new Q4ContinuousFunction(XMIN, YMIN, XMAX, YMAX)); + cont.print("Q4-continuous"); + + DataStream> win = trips + .windowAll(TumblingEventTimeWindows.of(Time.seconds(WINDOW_SIZE_SECONDS))) + .process(new Q4WindowedFunction(XMIN, YMIN, XMAX, YMAX)); + win.print("Q4-windowed"); + + DataStream> snap = trips + .keyBy(BerlinMODTrip::getVehicleId) + .process(new Q4SnapshotFunction(XMIN, YMIN, XMAX, YMAX, SNAPSHOT_TICK_MILLIS)); + snap.print("Q4-snapshot"); + + env.execute("BerlinMODQ4LocalTest"); + LOG.info("BerlinMODQ4LocalTest done"); + } + + private static List buildEvents() { + List events = new ArrayList<>(); + // v100 always inside R + for (int i = 0; i <= 12; i += 2) { + events.add(make(100, T0 + i * 1000L, 4.3517, 50.8503)); + } + // v200 oscillates in/out: out, IN, out, IN, out, IN, out + double[][] v200Path = { + {4.3060, 50.8270}, // t=1 out (lat<50.84) + {4.3060, 50.8500}, // t=3 IN + {4.3060, 50.8300}, // t=5 out + {4.3060, 50.8500}, // t=7 IN + {4.3060, 50.8100}, // t=9 out + {4.3060, 50.8500}, // t=11 IN + {4.3060, 50.8300}, // t=13 out + }; + int idx = 0; + for (int i = 1; i <= 13; i += 2, idx++) { + events.add(make(200, T0 + i * 1000L, v200Path[idx][0], v200Path[idx][1])); + } + // v300 always outside R + for (int i = 0; i <= 12; i += 2) { + events.add(make(300, T0 + i * 1000L, 4.2000, 50.7500)); + } + return events; + } + + private static BerlinMODTrip make(int vid, long t, double lon, double lat) { + BerlinMODTrip trip = new BerlinMODTrip(); + trip.setVehicleId(vid); + trip.setTimestamp(t); + trip.setLon(lon); + trip.setLat(lat); + return trip; + } +} diff --git a/flink-processor/src/main/java/berlinmod/BerlinMODQ5LocalTest.java b/flink-processor/src/main/java/berlinmod/BerlinMODQ5LocalTest.java new file mode 100644 index 0000000..c1f35e7 --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/BerlinMODQ5LocalTest.java @@ -0,0 +1,134 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.java.tuple.Tuple4; +import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; + +/** + * Local end-to-end test driver for the BerlinMOD-Q5 three streaming forms. + * + *

Same stationary-vehicle corpus as Q1/Q2/Q3/Q9. Reference point P = + * Brussels city centre (4.3517, 50.8503); {@code dP = 5 km} (vehicle near P); + * {@code dMeet = 5 km} (pair-meeting threshold). + * + *

Pairs: + *

    + *
  • (100, 200) — both near P; distance 4.1 km ≤ dMeet → MEET
  • + *
  • (100, 300) — v300 not near P → don't qualify
  • + *
  • (200, 300) — v300 not near P → don't qualify
  • + *
+ * + *

Expected output (only the (100, 200) pair qualifies): + *

    + *
  • Q5-continuous: pair (100, 200) emits on every event from t=1 + * onward (the first t=0 events of v100 and v300 happen before v200 is + * known, so no pair exists yet). 21 - 2 = 19 emissions.
  • + *
  • Q5-windowed: each of the two 10-second windows contains + * events for v100 and v200 — both qualify, the pair meets. 2 emissions.
  • + *
  • Q5-snapshot: 3 ticks × 1 meeting pair = 3 emissions.
  • + *
+ */ +public class BerlinMODQ5LocalTest { + + private static final Logger LOG = LoggerFactory.getLogger(BerlinMODQ5LocalTest.class); + + private static final double P_LON = 4.3517; + private static final double P_LAT = 50.8503; + private static final double D_P_METRES = 5_000.0; + private static final double D_MEET_METRES = 5_000.0; + private static final long WINDOW_SIZE_SECONDS = 10L; + private static final long SNAPSHOT_TICK_MILLIS = 5_000L; + private static final long T0 = 1_735_711_200_000L; + + public static void main(String[] args) throws Exception { + LOG.info("BerlinMODQ5LocalTest starting; P=({}, {}) dP={}m dMeet={}m", + P_LON, P_LAT, D_P_METRES, D_MEET_METRES); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + List events = buildEvents(); + DataStreamSource raw = env.fromCollection(events); + DataStream trips = raw.assignTimestampsAndWatermarks( + WatermarkStrategy + .forBoundedOutOfOrderness(Duration.ofSeconds(1)) + .withTimestampAssigner((e, t) -> e.getTimestamp())); + + DataStream> cont = trips + .keyBy(t -> 0) + .process(new Q5ContinuousFunction(P_LON, P_LAT, D_P_METRES, D_MEET_METRES)); + cont.print("Q5-continuous"); + + DataStream> win = trips + .windowAll(TumblingEventTimeWindows.of(Time.seconds(WINDOW_SIZE_SECONDS))) + .process(new Q5WindowedFunction(P_LON, P_LAT, D_P_METRES, D_MEET_METRES)); + win.print("Q5-windowed"); + + DataStream> snap = trips + .keyBy(t -> 0) + .process(new Q5SnapshotFunction(P_LON, P_LAT, D_P_METRES, D_MEET_METRES, SNAPSHOT_TICK_MILLIS)); + snap.print("Q5-snapshot"); + + env.execute("BerlinMODQ5LocalTest"); + LOG.info("BerlinMODQ5LocalTest done"); + } + + private static List buildEvents() { + List events = new ArrayList<>(); + for (int i = 0; i <= 12; i += 2) { + events.add(make(100, T0 + i * 1000L, 4.3517, 50.8503)); + } + for (int i = 1; i <= 13; i += 2) { + events.add(make(200, T0 + i * 1000L, 4.3060, 50.8270)); + } + for (int i = 0; i <= 12; i += 2) { + events.add(make(300, T0 + i * 1000L, 4.2000, 50.7500)); + } + return events; + } + + private static BerlinMODTrip make(int vid, long t, double lon, double lat) { + BerlinMODTrip trip = new BerlinMODTrip(); + trip.setVehicleId(vid); + trip.setTimestamp(t); + trip.setLon(lon); + trip.setLat(lat); + return trip; + } +} diff --git a/flink-processor/src/main/java/berlinmod/BerlinMODQ6LocalTest.java b/flink-processor/src/main/java/berlinmod/BerlinMODQ6LocalTest.java new file mode 100644 index 0000000..1bd455f --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/BerlinMODQ6LocalTest.java @@ -0,0 +1,146 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.api.java.tuple.Tuple4; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; + +/** + * Local end-to-end test driver for the BerlinMOD-Q6 three streaming forms. + * + *

Unlike Q1/Q2/Q3 which use a stationary-vehicles corpus, Q6 needs vehicles + * that actually move so the cumulative-distance arithmetic produces non-zero + * output. The synthetic corpus here drifts each vehicle by a fixed bearing + * per event: + * + *

    + *
  • Vehicle 100 drifts east ~100 m per 2 s event (0.001423° lon at lat 50.85)
  • + *
  • Vehicle 200 drifts south ~50 m per 2 s event (0.000450° lat)
  • + *
  • Vehicle 300 drifts west ~200 m per 2 s event (0.002846° lon)
  • + *
+ * + *

With 7 events per vehicle (6 inter-event steps), per-vehicle totals are + * approximately: + * + *

    + *
  • v100: 6 × 100 m = 600 m
  • + *
  • v200: 6 × 50 m = 300 m
  • + *
  • v300: 6 × 200 m = 1200 m
  • + *
+ * + *

Expected output: + * + *

    + *
  • Q6-continuous: 21 lines, cumulative metres rising monotonically per vehicle
  • + *
  • Q6-windowed: 6 windowed emissions (2 windows × 3 vehicles)
  • + *
  • Q6-snapshot: 9 emissions (3 ticks × 3 vehicles, all-source-closed)
  • + *
+ */ +public class BerlinMODQ6LocalTest { + + private static final Logger LOG = LoggerFactory.getLogger(BerlinMODQ6LocalTest.class); + + private static final long WINDOW_SIZE_SECONDS = 10L; + private static final long SNAPSHOT_TICK_MILLIS = 5_000L; + private static final long T0 = 1_735_711_200_000L; + + // Drift per event step + private static final double V100_DLON = 100.0 / (111_000.0 * Math.cos(Math.toRadians(50.85))); + private static final double V200_DLAT = -50.0 / 111_000.0; + private static final double V300_DLON = -200.0 / (111_000.0 * Math.cos(Math.toRadians(50.85))); + + public static void main(String[] args) throws Exception { + LOG.info("BerlinMODQ6LocalTest starting; window={}s tick={}ms", + WINDOW_SIZE_SECONDS, SNAPSHOT_TICK_MILLIS); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + List events = buildEvents(); + DataStreamSource raw = env.fromCollection(events); + DataStream trips = raw.assignTimestampsAndWatermarks( + WatermarkStrategy + .forBoundedOutOfOrderness(Duration.ofSeconds(1)) + .withTimestampAssigner((e, t) -> e.getTimestamp())); + + DataStream> cont = trips + .keyBy(BerlinMODTrip::getVehicleId) + .process(new Q6ContinuousFunction()); + cont.print("Q6-continuous"); + + DataStream> win = trips + .keyBy(BerlinMODTrip::getVehicleId) + .window(TumblingEventTimeWindows.of(Time.seconds(WINDOW_SIZE_SECONDS))) + .process(new Q6WindowedFunction()); + win.print("Q6-windowed"); + + DataStream> snap = trips + .keyBy(BerlinMODTrip::getVehicleId) + .process(new Q6SnapshotFunction(SNAPSHOT_TICK_MILLIS)); + snap.print("Q6-snapshot"); + + env.execute("BerlinMODQ6LocalTest"); + LOG.info("BerlinMODQ6LocalTest done"); + } + + private static List buildEvents() { + List events = new ArrayList<>(); + int step = 0; + for (int i = 0; i <= 12; i += 2, step++) { + events.add(make(100, T0 + i * 1000L, 4.3517 + step * V100_DLON, 50.8503)); + } + step = 0; + for (int i = 1; i <= 13; i += 2, step++) { + events.add(make(200, T0 + i * 1000L, 4.3060, 50.8270 + step * V200_DLAT)); + } + step = 0; + for (int i = 0; i <= 12; i += 2, step++) { + events.add(make(300, T0 + i * 1000L, 4.2000 + step * V300_DLON, 50.7500)); + } + return events; + } + + private static BerlinMODTrip make(int vid, long t, double lon, double lat) { + BerlinMODTrip trip = new BerlinMODTrip(); + trip.setVehicleId(vid); + trip.setTimestamp(t); + trip.setLon(lon); + trip.setLat(lat); + return trip; + } +} diff --git a/flink-processor/src/main/java/berlinmod/BerlinMODQ7LocalTest.java b/flink-processor/src/main/java/berlinmod/BerlinMODQ7LocalTest.java new file mode 100644 index 0000000..39b84cd --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/BerlinMODQ7LocalTest.java @@ -0,0 +1,141 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.api.java.tuple.Tuple4; +import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; +import java.util.Arrays; +import java.util.ArrayList; +import java.util.List; + +/** + * Local end-to-end test driver for the BerlinMOD-Q7 three streaming forms. + * + *

Same stationary-vehicle corpus as Q1/Q2/Q3/Q5/Q9. POI list: + *

    + *
  • POI 1 = Brussels city centre (4.3517, 50.8503), radius 2000 m
  • + *
  • POI 2 = Anderlecht (4.3060, 50.8270), radius 1000 m
  • + *
  • POI 3 = south of Brussels (4.2100, 50.7600), radius 2000 m
  • + *
+ * + *

Per (vehicle, POI) match-up: + *

    + *
  • v100 is inside POI 1 (0 m), outside POI 2 (~4.1 km) and POI 3 (~13 km)
  • + *
  • v200 is inside POI 2 (0 m), outside POI 1 and POI 3
  • + *
  • v300 is inside POI 3 (~1.3 km), outside POI 1 and POI 2
  • + *
+ * + *

Expected output: + *

    + *
  • Q7-continuous: 3 emissions — first-passages on each vehicle's + * very first event (v100 t=0 → POI 1; v200 t=1 → POI 2; v300 t=0 → + * POI 3)
  • + *
  • Q7-windowed: per-window intra-window first-passages — + * window [0, 10 s) sees all 3 first-passages; window [10, 20 s) sees + * all 3 again (intra-window scoping has no cross-window memory). 6 lines.
  • + *
  • Q7-snapshot: 3 ticks × 3 cumulative (vehicle, POI) first-passages = 9 lines
  • + *
+ */ +public class BerlinMODQ7LocalTest { + + private static final Logger LOG = LoggerFactory.getLogger(BerlinMODQ7LocalTest.class); + + private static final long WINDOW_SIZE_SECONDS = 10L; + private static final long SNAPSHOT_TICK_MILLIS = 5_000L; + private static final long T0 = 1_735_711_200_000L; + + private static final List POIS = Arrays.asList( + new PointOfInterest(1, 4.3517, 50.8503, 2_000.0), + new PointOfInterest(2, 4.3060, 50.8270, 1_000.0), + new PointOfInterest(3, 4.2100, 50.7600, 2_000.0)); + + public static void main(String[] args) throws Exception { + LOG.info("BerlinMODQ7LocalTest starting; #POIs={} window={}s tick={}ms", + POIS.size(), WINDOW_SIZE_SECONDS, SNAPSHOT_TICK_MILLIS); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + List events = buildEvents(); + DataStreamSource raw = env.fromCollection(events); + DataStream trips = raw.assignTimestampsAndWatermarks( + WatermarkStrategy + .forBoundedOutOfOrderness(Duration.ofSeconds(1)) + .withTimestampAssigner((e, t) -> e.getTimestamp())); + + DataStream> cont = trips + .keyBy(BerlinMODTrip::getVehicleId) + .process(new Q7ContinuousFunction(POIS)); + cont.print("Q7-continuous"); + + DataStream> win = trips + .windowAll(TumblingEventTimeWindows.of(Time.seconds(WINDOW_SIZE_SECONDS))) + .process(new Q7WindowedFunction(POIS)); + win.print("Q7-windowed"); + + DataStream> snap = trips + .keyBy(BerlinMODTrip::getVehicleId) + .process(new Q7SnapshotFunction(POIS, SNAPSHOT_TICK_MILLIS)); + snap.print("Q7-snapshot"); + + env.execute("BerlinMODQ7LocalTest"); + LOG.info("BerlinMODQ7LocalTest done"); + } + + private static List buildEvents() { + List events = new ArrayList<>(); + for (int i = 0; i <= 12; i += 2) { + events.add(make(100, T0 + i * 1000L, 4.3517, 50.8503)); + } + for (int i = 1; i <= 13; i += 2) { + events.add(make(200, T0 + i * 1000L, 4.3060, 50.8270)); + } + for (int i = 0; i <= 12; i += 2) { + events.add(make(300, T0 + i * 1000L, 4.2000, 50.7500)); + } + return events; + } + + private static BerlinMODTrip make(int vid, long t, double lon, double lat) { + BerlinMODTrip trip = new BerlinMODTrip(); + trip.setVehicleId(vid); + trip.setTimestamp(t); + trip.setLon(lon); + trip.setLat(lat); + return trip; + } +} diff --git a/flink-processor/src/main/java/berlinmod/BerlinMODQ8LocalTest.java b/flink-processor/src/main/java/berlinmod/BerlinMODQ8LocalTest.java new file mode 100644 index 0000000..296b408 --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/BerlinMODQ8LocalTest.java @@ -0,0 +1,132 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; + +/** + * Local end-to-end test driver for the BerlinMOD-Q8 three streaming forms. + * + *

Same stationary-vehicle corpus as the other Qs. Road segment runs + * from (4.30, 50.83) to (4.36, 50.87) — a diagonal across the Brussels- + * centre region. With a {@code d = 5 km} proximity threshold: + * + *

    + *
  • v100 at (4.3517, 50.8503) — ~1.1 km from segment → near
  • + *
  • v200 at (4.3060, 50.8270) — ~0.5 km from segment → near
  • + *
  • v300 at (4.2000, 50.7500) — ~13 km from segment → not near
  • + *
+ * + *

Expected output shape: + *

    + *
  • Q8-continuous: 21 events (14 near=true for v100/v200, 7 near=false for v300)
  • + *
  • Q8-windowed: 2 windows, each with {@code distinctCount=2} (vehicles 100 and 200)
  • + *
  • Q8-snapshot: 3 ticks × 2 near vehicles = 6 emissions
  • + *
+ * + *

Same shape as Q3 with a segment-distance predicate substituted for the + * point-radius one — the algebraic pattern parity intentional. + */ +public class BerlinMODQ8LocalTest { + + private static final Logger LOG = LoggerFactory.getLogger(BerlinMODQ8LocalTest.class); + + // Road segment endpoints + private static final double S1_LON = 4.30, S1_LAT = 50.83; + private static final double S2_LON = 4.36, S2_LAT = 50.87; + private static final double RADIUS_METRES = 5_000.0; + private static final long WINDOW_SIZE_SECONDS = 10L; + private static final long SNAPSHOT_TICK_MILLIS = 5_000L; + private static final long T0 = 1_735_711_200_000L; + + public static void main(String[] args) throws Exception { + LOG.info("BerlinMODQ8LocalTest starting; segment=({},{}) → ({},{}) d={}m", + S1_LON, S1_LAT, S2_LON, S2_LAT, RADIUS_METRES); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + List events = buildEvents(); + DataStreamSource raw = env.fromCollection(events); + DataStream trips = raw.assignTimestampsAndWatermarks( + WatermarkStrategy + .forBoundedOutOfOrderness(Duration.ofSeconds(1)) + .withTimestampAssigner((e, t) -> e.getTimestamp())); + + DataStream> cont = trips + .process(new Q8ContinuousFunction(S1_LON, S1_LAT, S2_LON, S2_LAT, RADIUS_METRES)); + cont.print("Q8-continuous"); + + DataStream> win = trips + .windowAll(TumblingEventTimeWindows.of(Time.seconds(WINDOW_SIZE_SECONDS))) + .process(new Q8WindowedFunction(S1_LON, S1_LAT, S2_LON, S2_LAT, RADIUS_METRES)); + win.print("Q8-windowed"); + + DataStream> snap = trips + .keyBy(BerlinMODTrip::getVehicleId) + .process(new Q8SnapshotFunction(S1_LON, S1_LAT, S2_LON, S2_LAT, RADIUS_METRES, SNAPSHOT_TICK_MILLIS)); + snap.print("Q8-snapshot"); + + env.execute("BerlinMODQ8LocalTest"); + LOG.info("BerlinMODQ8LocalTest done"); + } + + private static List buildEvents() { + List events = new ArrayList<>(); + for (int i = 0; i <= 12; i += 2) { + events.add(make(100, T0 + i * 1000L, 4.3517, 50.8503)); + } + for (int i = 1; i <= 13; i += 2) { + events.add(make(200, T0 + i * 1000L, 4.3060, 50.8270)); + } + for (int i = 0; i <= 12; i += 2) { + events.add(make(300, T0 + i * 1000L, 4.2000, 50.7500)); + } + return events; + } + + private static BerlinMODTrip make(int vid, long t, double lon, double lat) { + BerlinMODTrip trip = new BerlinMODTrip(); + trip.setVehicleId(vid); + trip.setTimestamp(t); + trip.setLon(lon); + trip.setLat(lat); + return trip; + } +} diff --git a/flink-processor/src/main/java/berlinmod/BerlinMODQ9LocalTest.java b/flink-processor/src/main/java/berlinmod/BerlinMODQ9LocalTest.java new file mode 100644 index 0000000..9a34363 --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/BerlinMODQ9LocalTest.java @@ -0,0 +1,132 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; + +/** + * Local end-to-end test driver for the BerlinMOD-Q9 three streaming forms. + * + *

Same stationary-vehicle synthetic corpus as Q1/Q2/Q3 (3 vehicles, 21 + * events). Queried pair X = 100 (Brussels city centre), Y = 200 (Anderlecht); + * their actual distance is ~4.1 km — the expected output for every emission. + * + *

Expected output: + *

    + *
  • Q9-continuous: 13 lines — emitted whenever either X or Y has + * a new event AND the other has been seen at least once. v100 fires + * first at t=0; v200's first event at t=1 produces the first paired + * emission; subsequent 12 events (alternating) each produce one + * emission.
  • + *
  • Q9-windowed: 2 windows — both contain X and Y events, each + * emits the X-Y distance using last seen-in-window positions.
  • + *
  • Q9-snapshot: 3 ticks × 1 emission each = 3 lines.
  • + *
+ */ +public class BerlinMODQ9LocalTest { + + private static final Logger LOG = LoggerFactory.getLogger(BerlinMODQ9LocalTest.class); + + private static final int X_VEHICLE_ID = 100; + private static final int Y_VEHICLE_ID = 200; + private static final long WINDOW_SIZE_SECONDS = 10L; + private static final long SNAPSHOT_TICK_MILLIS = 5_000L; + private static final long T0 = 1_735_711_200_000L; + + public static void main(String[] args) throws Exception { + LOG.info("BerlinMODQ9LocalTest starting; X={} Y={} window={}s tick={}ms", + X_VEHICLE_ID, Y_VEHICLE_ID, WINDOW_SIZE_SECONDS, SNAPSHOT_TICK_MILLIS); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + List events = buildEvents(); + DataStreamSource raw = env.fromCollection(events); + DataStream trips = raw.assignTimestampsAndWatermarks( + WatermarkStrategy + .forBoundedOutOfOrderness(Duration.ofSeconds(1)) + .withTimestampAssigner((e, t) -> e.getTimestamp())); + + // Pre-filter to {X, Y} and key by a constant so the shared X+Y state + // lives in a single subtask. + DataStream xy = trips + .filter(t -> t.getVehicleId() == X_VEHICLE_ID || t.getVehicleId() == Y_VEHICLE_ID); + + DataStream> cont = xy + .keyBy(t -> 0) + .process(new Q9ContinuousFunction(X_VEHICLE_ID, Y_VEHICLE_ID)); + cont.print("Q9-continuous"); + + DataStream> win = xy + .windowAll(TumblingEventTimeWindows.of(Time.seconds(WINDOW_SIZE_SECONDS))) + .process(new Q9WindowedFunction(X_VEHICLE_ID, Y_VEHICLE_ID)); + win.print("Q9-windowed"); + + DataStream> snap = xy + .keyBy(t -> 0) + .process(new Q9SnapshotFunction(X_VEHICLE_ID, Y_VEHICLE_ID, SNAPSHOT_TICK_MILLIS)); + snap.print("Q9-snapshot"); + + env.execute("BerlinMODQ9LocalTest"); + LOG.info("BerlinMODQ9LocalTest done"); + } + + private static List buildEvents() { + List events = new ArrayList<>(); + for (int i = 0; i <= 12; i += 2) { + events.add(make(100, T0 + i * 1000L, 4.3517, 50.8503)); + } + for (int i = 1; i <= 13; i += 2) { + events.add(make(200, T0 + i * 1000L, 4.3060, 50.8270)); + } + for (int i = 0; i <= 12; i += 2) { + events.add(make(300, T0 + i * 1000L, 4.2000, 50.7500)); + } + return events; + } + + private static BerlinMODTrip make(int vid, long t, double lon, double lat) { + BerlinMODTrip trip = new BerlinMODTrip(); + trip.setVehicleId(vid); + trip.setTimestamp(t); + trip.setLon(lon); + trip.setLat(lat); + return trip; + } +} diff --git a/flink-processor/src/main/java/berlinmod/BerlinMODTrip.java b/flink-processor/src/main/java/berlinmod/BerlinMODTrip.java new file mode 100644 index 0000000..1f65487 --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/BerlinMODTrip.java @@ -0,0 +1,73 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +/** + * Plain data class for a single GPS event from a BerlinMOD trip. + * + *

Matches the {@code aisdata.AISData} field set but uses the BerlinMOD vehicle + * identifier {@code vehicleId} instead of an AIS {@code mmsi} and drops the + * AIS-specific {@code speed}/{@code course} channels (BerlinMOD's generator + * does not export those for the streaming form). + */ +public class BerlinMODTrip { + private long timestamp; // epoch milliseconds (event time) + private int vehicleId; + private double lon; + private double lat; + + public long getTimestamp() { + return timestamp; + } + + public void setTimestamp(long timestamp) { + this.timestamp = timestamp; + } + + public int getVehicleId() { + return vehicleId; + } + + public void setVehicleId(int vehicleId) { + this.vehicleId = vehicleId; + } + + public double getLon() { + return lon; + } + + public void setLon(double lon) { + this.lon = lon; + } + + public double getLat() { + return lat; + } + + public void setLat(double lat) { + this.lat = lat; + } +} diff --git a/flink-processor/src/main/java/berlinmod/MEOSBridge.java b/flink-processor/src/main/java/berlinmod/MEOSBridge.java new file mode 100644 index 0000000..c74480f --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/MEOSBridge.java @@ -0,0 +1,158 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import functions.GeneratedFunctions; +import jnr.ffi.Pointer; +import org.mobilitydb.flink.meos.wirings.MeosWiringRuntime; + +/** + * Thin wiring from the BerlinMOD streaming-form predicates to MEOS via JMEOS. + * + *

Every spatial predicate exercised by the BerlinMOD-9 × 3-form scaffold + * flows through this class, and every predicate evaluates through MEOS. The + * within-distance predicate is the canonical temporal operator + * {@code edwithin_tgeo_geo} — ever-within between the vehicle's {@code tgeogpoint} + * instant and the query geography, in metres on the WGS84 spheroid — the same + * MEOS operator the streaming-form parity contract names. Distances are + * {@code geog_distance} over WGS84 geographies. This class holds no spatial + * mathematics of its own: it constructs the MEOS inputs and delegates the + * computation to libmeos. + * + *

{@link MeosWiringRuntime#ensureInitializedOnThread()} initialises MEOS on + * the calling task thread (idempotent per thread) before the first call, since + * MEOS keeps its session state per OS thread. + */ +public final class MEOSBridge { + + private MEOSBridge() { + // utility + } + + // ---------------------------------------------------------------------- + // Public predicate surface — all evaluation delegated to MEOS. + // ---------------------------------------------------------------------- + + /** + * @return {@code true} iff the WGS84 spheroidal distance from + * {@code (lon1, lat1)} to {@code (lon2, lat2)} is at most + * {@code radiusMetres}, via MEOS {@code edwithin_tgeo_geo} between + * the {@code (lon1, lat1)} {@code tgeogpoint} instant and the + * {@code (lon2, lat2)} point geography. + */ + public static boolean dwithinMetres(double lon1, double lat1, + double lon2, double lat2, + double radiusMetres) { + MeosWiringRuntime.ensureInitializedOnThread(); + return GeneratedFunctions.edwithin_tgeo_geo( + tgeogInst(lon1, lat1), pointGeog(lon2, lat2), radiusMetres) == 1; + } + + /** + * @return {@code true} iff the WGS84 spheroidal distance from + * {@code (pLon, pLat)} to the LineString {@code (s1, s2)} is at most + * {@code radiusMetres}, via MEOS {@code edwithin_tgeo_geo} between + * the point {@code tgeogpoint} instant and the line geography. + */ + public static boolean dwithinSegmentMetres(double pLon, double pLat, + double s1Lon, double s1Lat, + double s2Lon, double s2Lat, + double radiusMetres) { + MeosWiringRuntime.ensureInitializedOnThread(); + return GeneratedFunctions.edwithin_tgeo_geo( + tgeogInst(pLon, pLat), + lineGeog(s1Lon, s1Lat, s2Lon, s2Lat), radiusMetres) == 1; + } + + /** + * @return {@code true} iff {@code (lon, lat)} lies in the axis-aligned box + * {@code [xmin, xmax] × [ymin, ymax]}, via MEOS + * {@code eintersects_tgeo_geo} between the point's {@code tgeompoint} + * instant and the box polygon (planar, SRID 4326). + */ + public static boolean intersectsBox(double lon, double lat, + double xmin, double ymin, + double xmax, double ymax) { + MeosWiringRuntime.ensureInitializedOnThread(); + return GeneratedFunctions.eintersects_tgeo_geo( + tgeomInst(lon, lat), boxPolygon(xmin, ymin, xmax, ymax)) == 1; + } + + /** + * @return the WGS84 spheroidal distance in metres between two points, via + * MEOS {@code geog_distance}. + */ + public static double distanceMetres(double lon1, double lat1, + double lon2, double lat2) { + MeosWiringRuntime.ensureInitializedOnThread(); + return GeneratedFunctions.geog_distance(pointGeog(lon1, lat1), pointGeog(lon2, lat2)); + } + + /** + * @return the WGS84 spheroidal distance in metres from {@code (pLon, pLat)} + * to the LineString {@code (s1, s2)}, via MEOS {@code geog_distance}. + */ + public static double distanceSegmentMetres(double pLon, double pLat, + double s1Lon, double s1Lat, + double s2Lon, double s2Lat) { + MeosWiringRuntime.ensureInitializedOnThread(); + return GeneratedFunctions.geog_distance( + pointGeog(pLon, pLat), lineGeog(s1Lon, s1Lat, s2Lon, s2Lat)); + } + + // ---------------------------------------------------------------------- + // Internal helpers — construct the MEOS temporal / geography inputs. + // ---------------------------------------------------------------------- + + private static Pointer tgeogInst(double lon, double lat) { + return GeneratedFunctions.tgeogpoint_in( + String.format("SRID=4326;Point(%.7f %.7f)@2000-01-01", lon, lat)); + } + + private static Pointer tgeomInst(double lon, double lat) { + return GeneratedFunctions.tgeompoint_in( + String.format("SRID=4326;Point(%.7f %.7f)@2000-01-01", lon, lat)); + } + + private static Pointer boxPolygon(double xmin, double ymin, + double xmax, double ymax) { + return GeneratedFunctions.geom_in(String.format( + "SRID=4326;Polygon((%.7f %.7f, %.7f %.7f, %.7f %.7f, %.7f %.7f, %.7f %.7f))", + xmin, ymin, xmax, ymin, xmax, ymax, xmin, ymax, xmin, ymin), -1); + } + + private static Pointer pointGeog(double lon, double lat) { + return GeneratedFunctions.geom_to_geog( + GeneratedFunctions.geom_in(String.format("SRID=4326;Point(%.7f %.7f)", lon, lat), -1)); + } + + private static Pointer lineGeog(double s1Lon, double s1Lat, + double s2Lon, double s2Lat) { + return GeneratedFunctions.geom_to_geog( + GeneratedFunctions.geom_in(String.format( + "SRID=4326;LineString(%.7f %.7f, %.7f %.7f)", s1Lon, s1Lat, s2Lon, s2Lat), -1)); + } +} diff --git a/flink-processor/src/main/java/berlinmod/PointOfInterest.java b/flink-processor/src/main/java/berlinmod/PointOfInterest.java new file mode 100644 index 0000000..251050f --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/PointOfInterest.java @@ -0,0 +1,49 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import java.io.Serializable; + +/** + * Simple point-of-interest record for BerlinMOD-Q7 — a (lon, lat) plus a + * proximity radius in metres and an integer id. Serializable for use in + * Flink operator state and configuration. + */ +public final class PointOfInterest implements Serializable { + private static final long serialVersionUID = 1L; + + public final int id; + public final double lon; + public final double lat; + public final double radiusMetres; + + public PointOfInterest(int id, double lon, double lat, double radiusMetres) { + this.id = id; + this.lon = lon; + this.lat = lat; + this.radiusMetres = radiusMetres; + } +} diff --git a/flink-processor/src/main/java/berlinmod/Q1ContinuousFunction.java b/flink-processor/src/main/java/berlinmod/Q1ContinuousFunction.java new file mode 100644 index 0000000..e4476f2 --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/Q1ContinuousFunction.java @@ -0,0 +1,66 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.KeyedProcessFunction; +import org.apache.flink.util.Collector; + +/** + * BerlinMOD-Q1 — continuous form. + * + *

"Which vehicles have appeared in the stream?" + * + *

Emits {@code (vehicleId, firstSeenTimestamp)} the first time each vehicle + * is seen; subsequent events for the same vehicle are deduplicated via keyed + * state. Keyed by vehicleId. + */ +public class Q1ContinuousFunction + extends KeyedProcessFunction> { + + private transient ValueState seen; + + @Override + public void open(Configuration parameters) { + seen = getRuntimeContext().getState( + new ValueStateDescriptor<>("q1SeenVehicle", Boolean.class)); + } + + @Override + public void processElement( + BerlinMODTrip trip, + Context ctx, + Collector> out) throws Exception { + Boolean s = seen.value(); + if (s == null || !s) { + out.collect(new Tuple2<>(trip.getVehicleId(), trip.getTimestamp())); + seen.update(true); + } + } +} diff --git a/flink-processor/src/main/java/berlinmod/Q1SnapshotFunction.java b/flink-processor/src/main/java/berlinmod/Q1SnapshotFunction.java new file mode 100644 index 0000000..7e46b96 --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/Q1SnapshotFunction.java @@ -0,0 +1,84 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.KeyedProcessFunction; +import org.apache.flink.util.Collector; + +/** + * BerlinMOD-Q1 — snapshot form. + * + *

"At time T, which vehicles have appeared in the stream up to T?" + * + *

Keyed by vehicleId. On each event, mark the vehicle as seen and register + * an event-time timer at the next snapshot tick. When the timer fires at time + * T, emit {@code (T, vehicleId)} for each vehicle that has been seen by T. + * + *

This is the parity-oracle form: at watermark T, the streaming output is + * the set of vehicleIds whose first event occurred at or before T, which + * equals the batch BerlinMOD-Q1 result on data up to T. + */ +public class Q1SnapshotFunction + extends KeyedProcessFunction> { + + private final long snapshotTickMillis; + private transient ValueState seen; + + public Q1SnapshotFunction(long snapshotTickMillis) { + this.snapshotTickMillis = snapshotTickMillis; + } + + @Override + public void open(Configuration parameters) { + seen = getRuntimeContext().getState( + new ValueStateDescriptor<>("q1SnapshotSeen", Boolean.class)); + } + + @Override + public void processElement( + BerlinMODTrip trip, + Context ctx, + Collector> out) throws Exception { + seen.update(true); + long nextTick = ((trip.getTimestamp() / snapshotTickMillis) + 1) * snapshotTickMillis; + ctx.timerService().registerEventTimeTimer(nextTick); + } + + @Override + public void onTimer( + long timestamp, + OnTimerContext ctx, + Collector> out) throws Exception { + Boolean s = seen.value(); + if (Boolean.TRUE.equals(s)) { + out.collect(new Tuple2<>(timestamp, ctx.getCurrentKey())); + } + } +} diff --git a/flink-processor/src/main/java/berlinmod/Q1WindowedFunction.java b/flink-processor/src/main/java/berlinmod/Q1WindowedFunction.java new file mode 100644 index 0000000..d06bc97 --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/Q1WindowedFunction.java @@ -0,0 +1,58 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.streaming.api.functions.windowing.ProcessAllWindowFunction; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.util.Collector; + +import java.util.HashSet; +import java.util.Set; + +/** + * BerlinMOD-Q1 — windowed form. + * + *

"Per N-second tumbling window, how many distinct vehicles appeared + * in the window?" + * + *

Emits {@code (windowStart, windowEnd, distinctCount)} per window. + */ +public class Q1WindowedFunction + extends ProcessAllWindowFunction, TimeWindow> { + + @Override + public void process( + Context ctx, + Iterable elements, + Collector> out) { + Set distinct = new HashSet<>(); + for (BerlinMODTrip trip : elements) { + distinct.add(trip.getVehicleId()); + } + out.collect(new Tuple3<>(ctx.window().getStart(), ctx.window().getEnd(), (long) distinct.size())); + } +} diff --git a/flink-processor/src/main/java/berlinmod/Q2ContinuousFunction.java b/flink-processor/src/main/java/berlinmod/Q2ContinuousFunction.java new file mode 100644 index 0000000..fa5621f --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/Q2ContinuousFunction.java @@ -0,0 +1,65 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.streaming.api.functions.ProcessFunction; +import org.apache.flink.util.Collector; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * BerlinMOD-Q2 — continuous form. + * + *

"Where is vehicle X right now?" + * + *

For each incoming GPS event {@link BerlinMODTrip}, emit it unchanged if it + * belongs to the queried vehicle, otherwise drop. No windowing, no state — + * a per-event filter against {@code targetVehicleId}. + */ +public class Q2ContinuousFunction extends ProcessFunction { + + private static final Logger LOG = LoggerFactory.getLogger(Q2ContinuousFunction.class); + + private final int targetVehicleId; + + public Q2ContinuousFunction(int targetVehicleId) { + this.targetVehicleId = targetVehicleId; + } + + @Override + public void processElement( + BerlinMODTrip trip, + Context ctx, + Collector out) { + if (trip.getVehicleId() == targetVehicleId) { + out.collect(trip); + if (LOG.isDebugEnabled()) { + LOG.debug("Q2-continuous: vehicle={} t={} ({}, {})", + trip.getVehicleId(), trip.getTimestamp(), trip.getLon(), trip.getLat()); + } + } + } +} diff --git a/flink-processor/src/main/java/berlinmod/Q2SnapshotFunction.java b/flink-processor/src/main/java/berlinmod/Q2SnapshotFunction.java new file mode 100644 index 0000000..c1e4dd3 --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/Q2SnapshotFunction.java @@ -0,0 +1,108 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.typeinfo.TypeHint; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.api.java.tuple.Tuple4; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.KeyedProcessFunction; +import org.apache.flink.util.Collector; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * BerlinMOD-Q2 — snapshot form. + * + *

"At time T, where is vehicle X?" + * + *

This is the parity-oracle form: streaming output at watermark T must + * equal the batch BerlinMOD-Q2 result on the same data up to T (the most + * recent known position of vehicle X on or before T). + * + *

Keyed by vehicleId (so the operator scales naturally if the queried + * vehicle changes, and so reuse across multiple queried vehicles is a + * fan-out keying choice rather than a code change). For events whose key + * matches {@code targetVehicleId}, update last-known state and register an + * event-time timer for the next snapshot tick. When the timer fires, emit + * {@code (T, lon, lat, t_of_last_event)}. + */ +public class Q2SnapshotFunction + extends KeyedProcessFunction> { + + private static final Logger LOG = LoggerFactory.getLogger(Q2SnapshotFunction.class); + + private final int targetVehicleId; + private final long snapshotTickMillis; + + private transient ValueState> lastKnown; // (lon, lat, ts) + + public Q2SnapshotFunction(int targetVehicleId, long snapshotTickMillis) { + this.targetVehicleId = targetVehicleId; + this.snapshotTickMillis = snapshotTickMillis; + } + + @Override + public void open(Configuration parameters) { + TypeInformation> tInfo = + TypeInformation.of(new TypeHint>() {}); + ValueStateDescriptor> desc = + new ValueStateDescriptor<>("q2LastKnownPosition", tInfo); + lastKnown = getRuntimeContext().getState(desc); + } + + @Override + public void processElement( + BerlinMODTrip trip, + Context ctx, + Collector> out) throws Exception { + if (trip.getVehicleId() != targetVehicleId) { + return; + } + lastKnown.update(new Tuple3<>(trip.getLon(), trip.getLat(), trip.getTimestamp())); + long nextTick = ((trip.getTimestamp() / snapshotTickMillis) + 1) * snapshotTickMillis; + ctx.timerService().registerEventTimeTimer(nextTick); + } + + @Override + public void onTimer( + long timestamp, + OnTimerContext ctx, + Collector> out) throws Exception { + Tuple3 p = lastKnown.value(); + if (p == null) { + return; + } + out.collect(new Tuple4<>(timestamp, p.f0, p.f1, p.f2)); + if (LOG.isDebugEnabled()) { + LOG.debug("Q2-snapshot: T={} vehicle={} ({}, {}) at t={}", + timestamp, ctx.getCurrentKey(), p.f0, p.f1, p.f2); + } + } +} diff --git a/flink-processor/src/main/java/berlinmod/Q2WindowedFunction.java b/flink-processor/src/main/java/berlinmod/Q2WindowedFunction.java new file mode 100644 index 0000000..a9dc4dd --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/Q2WindowedFunction.java @@ -0,0 +1,83 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.streaming.api.functions.windowing.ProcessAllWindowFunction; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.util.Collector; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * BerlinMOD-Q2 — windowed form. + * + *

"Per N-second tumbling window, what is vehicle X's most recent + * position seen within the window?" + * + *

For each window, filter to events matching {@code targetVehicleId}, keep + * the event with the largest timestamp, and emit + * {@code (windowStart, windowEnd, vehicleId, lon, lat)}. If the vehicle had + * no events in the window, emit nothing. + */ +public class Q2WindowedFunction + extends ProcessAllWindowFunction, TimeWindow> { + + private static final Logger LOG = LoggerFactory.getLogger(Q2WindowedFunction.class); + + private final int targetVehicleId; + + public Q2WindowedFunction(int targetVehicleId) { + this.targetVehicleId = targetVehicleId; + } + + @Override + public void process( + Context ctx, + Iterable elements, + Collector> out) { + BerlinMODTrip latest = null; + for (BerlinMODTrip trip : elements) { + if (trip.getVehicleId() != targetVehicleId) { + continue; + } + if (latest == null || trip.getTimestamp() > latest.getTimestamp()) { + latest = trip; + } + } + if (latest != null) { + out.collect(new Tuple5<>( + ctx.window().getStart(), + ctx.window().getEnd(), + latest.getVehicleId(), + latest.getLon(), + latest.getLat())); + LOG.info("Q2-windowed: [{}, {}) vehicle={} last=({}, {})", + ctx.window().getStart(), ctx.window().getEnd(), + latest.getVehicleId(), latest.getLon(), latest.getLat()); + } + } +} diff --git a/flink-processor/src/main/java/berlinmod/Q3ContinuousFunction.java b/flink-processor/src/main/java/berlinmod/Q3ContinuousFunction.java new file mode 100644 index 0000000..8dcad66 --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/Q3ContinuousFunction.java @@ -0,0 +1,72 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.streaming.api.functions.ProcessFunction; +import org.apache.flink.util.Collector; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * BerlinMOD-Q3 — continuous form. + * + *

"At every moment, which vehicles are currently within {@code d} metres + * of point P?" + * + *

For each incoming GPS event {@link BerlinMODTrip}, evaluate the radius + * predicate and emit {@code (vehicleId, eventTimeMillis, isNear)} per event. + * No windowing — output updates per event, watermark-independent. + * + *

Predicate: {@link MEOSBridge#dwithinMetres} — MEOS + * {@code edwithin_tgeo_geo} over WGS84 geographies. + */ +public class Q3ContinuousFunction extends ProcessFunction> { + + private static final Logger LOG = LoggerFactory.getLogger(Q3ContinuousFunction.class); + + private final double pLon; + private final double pLat; + private final double radiusMetres; + + public Q3ContinuousFunction(double pLon, double pLat, double radiusMetres) { + this.pLon = pLon; + this.pLat = pLat; + this.radiusMetres = radiusMetres; + } + + @Override + public void processElement( + BerlinMODTrip trip, + Context ctx, + Collector> out) { + boolean near = MEOSBridge.dwithinMetres(trip.getLon(), trip.getLat(), pLon, pLat, radiusMetres); + out.collect(new Tuple3<>(trip.getVehicleId(), trip.getTimestamp(), near)); + if (LOG.isDebugEnabled()) { + LOG.debug("Q3-continuous: vehicle={} ts={} near={}", trip.getVehicleId(), trip.getTimestamp(), near); + } + } +} diff --git a/flink-processor/src/main/java/berlinmod/Q3SnapshotFunction.java b/flink-processor/src/main/java/berlinmod/Q3SnapshotFunction.java new file mode 100644 index 0000000..f9f57ce --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/Q3SnapshotFunction.java @@ -0,0 +1,115 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.typeinfo.TypeHint; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.KeyedProcessFunction; +import org.apache.flink.util.Collector; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * BerlinMOD-Q3 — snapshot form. + * + *

"At time T, which vehicles are within {@code d} metres of point P?" + * + *

This is the parity-oracle form: streaming output at watermark T + * must equal the batch BerlinMOD-Q3 result on the same data up to T. + * + *

Keyed by vehicleId. Maintains a per-vehicle {@code lastKnownPosition} + * state. On each event, update the state, then register an event-time timer + * for the snapshot tick. When the timer fires at time T, evaluate the radius + * predicate against the most recent known position and emit + * {@code (T, vehicleId)} if the vehicle is within {@code d} of P at that + * snapshot. + * + *

Predicate: {@link MEOSBridge#dwithinMetres} — MEOS + * {@code edwithin_tgeo_geo} over WGS84 geographies. The snapshot-form output + * at watermark T is equal to the batch BerlinMOD-Q3 result up to T. + */ +public class Q3SnapshotFunction + extends KeyedProcessFunction> { + + private static final Logger LOG = LoggerFactory.getLogger(Q3SnapshotFunction.class); + + private final double pLon; + private final double pLat; + private final double radiusMetres; + private final long snapshotTickMillis; + + private transient ValueState> lastKnown; // (lon, lat, ts) + + public Q3SnapshotFunction( + double pLon, double pLat, double radiusMetres, long snapshotTickMillis) { + this.pLon = pLon; + this.pLat = pLat; + this.radiusMetres = radiusMetres; + this.snapshotTickMillis = snapshotTickMillis; + } + + @Override + public void open(Configuration parameters) { + TypeInformation> tInfo = + TypeInformation.of(new TypeHint>() {}); + ValueStateDescriptor> desc = + new ValueStateDescriptor<>("lastKnownPosition", tInfo); + lastKnown = getRuntimeContext().getState(desc); + } + + @Override + public void processElement( + BerlinMODTrip trip, + Context ctx, + Collector> out) throws Exception { + lastKnown.update(new Tuple3<>(trip.getLon(), trip.getLat(), trip.getTimestamp())); + long nextTick = ((trip.getTimestamp() / snapshotTickMillis) + 1) * snapshotTickMillis; + ctx.timerService().registerEventTimeTimer(nextTick); + } + + @Override + public void onTimer( + long timestamp, + OnTimerContext ctx, + Collector> out) throws Exception { + Tuple3 p = lastKnown.value(); + if (p == null) { + return; + } + if (MEOSBridge.dwithinMetres(p.f0, p.f1, pLon, pLat, radiusMetres)) { + Integer vehicleId = ctx.getCurrentKey(); + out.collect(new Tuple2<>(timestamp, vehicleId)); + if (LOG.isDebugEnabled()) { + LOG.debug("Q3-snapshot: T={} vehicle={}", timestamp, vehicleId); + } + } + } +} diff --git a/flink-processor/src/main/java/berlinmod/Q3WindowedFunction.java b/flink-processor/src/main/java/berlinmod/Q3WindowedFunction.java new file mode 100644 index 0000000..bac04e1 --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/Q3WindowedFunction.java @@ -0,0 +1,83 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.streaming.api.functions.windowing.ProcessAllWindowFunction; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.util.Collector; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashSet; +import java.util.Set; + +/** + * BerlinMOD-Q3 — windowed form. + * + *

"Per N-second window, how many distinct vehicles were within + * {@code d} metres of point P at any time during the window?" + * + *

Tumbling event-time window of configurable size. For each window, scan + * all events whose timestamp falls in the window, count distinct vehicleIds + * for which at least one event satisfies the radius predicate, and emit + * {@code (windowStart, windowEnd, distinctCount)}. + * + *

Predicate: {@link MEOSBridge#dwithinMetres} — MEOS + * {@code edwithin_tgeo_geo} over WGS84 geographies. + */ +public class Q3WindowedFunction + extends ProcessAllWindowFunction, TimeWindow> { + + private static final Logger LOG = LoggerFactory.getLogger(Q3WindowedFunction.class); + + private final double pLon; + private final double pLat; + private final double radiusMetres; + + public Q3WindowedFunction(double pLon, double pLat, double radiusMetres) { + this.pLon = pLon; + this.pLat = pLat; + this.radiusMetres = radiusMetres; + } + + @Override + public void process( + Context ctx, + Iterable elements, + Collector> out) { + Set distinctNear = new HashSet<>(); + for (BerlinMODTrip trip : elements) { + if (MEOSBridge.dwithinMetres(trip.getLon(), trip.getLat(), pLon, pLat, radiusMetres)) { + distinctNear.add(trip.getVehicleId()); + } + } + long count = distinctNear.size(); + out.collect(new Tuple3<>(ctx.window().getStart(), ctx.window().getEnd(), count)); + LOG.info("Q3-windowed: [{}, {}) distinct-near={}", + ctx.window().getStart(), ctx.window().getEnd(), count); + } +} diff --git a/flink-processor/src/main/java/berlinmod/Q4ContinuousFunction.java b/flink-processor/src/main/java/berlinmod/Q4ContinuousFunction.java new file mode 100644 index 0000000..9fd23e2 --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/Q4ContinuousFunction.java @@ -0,0 +1,86 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.KeyedProcessFunction; +import org.apache.flink.util.Collector; + +/** + * BerlinMOD-Q4 — continuous form. + * + *

"Which vehicles entered region R (transition outside → inside), + * and when?" + * + *

Keyed by vehicleId. Per-vehicle state tracks the last seen + * inside-or-outside flag for R. On each event, computes the current + * inside-or-outside, and if the transition is outside→inside, emits + * {@code (vehicleId, entryTime)}. + * + *

Predicate: {@link MEOSBridge#intersectsBox} — MEOS + * {@code eintersects_tgeo_geo} between the point's {@code tgeompoint} instant + * and the region polygon. + */ +public class Q4ContinuousFunction + extends KeyedProcessFunction> { + + private final double xmin, ymin, xmax, ymax; + private transient ValueState wasInside; + + public Q4ContinuousFunction(double xmin, double ymin, double xmax, double ymax) { + this.xmin = xmin; + this.ymin = ymin; + this.xmax = xmax; + this.ymax = ymax; + } + + @Override + public void open(Configuration parameters) { + wasInside = getRuntimeContext().getState( + new ValueStateDescriptor<>("q4WasInside", Boolean.class)); + } + + @Override + public void processElement( + BerlinMODTrip trip, + Context ctx, + Collector> out) throws Exception { + boolean isInside = inBox(trip.getLon(), trip.getLat()); + Boolean prev = wasInside.value(); + boolean prevInside = prev != null && prev; + if (isInside && !prevInside) { + out.collect(new Tuple2<>(trip.getVehicleId(), trip.getTimestamp())); + } + wasInside.update(isInside); + } + + private boolean inBox(double lon, double lat) { + return MEOSBridge.intersectsBox(lon, lat, xmin, ymin, xmax, ymax); + } +} diff --git a/flink-processor/src/main/java/berlinmod/Q4SnapshotFunction.java b/flink-processor/src/main/java/berlinmod/Q4SnapshotFunction.java new file mode 100644 index 0000000..cf91444 --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/Q4SnapshotFunction.java @@ -0,0 +1,107 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.KeyedProcessFunction; +import org.apache.flink.util.Collector; + +/** + * BerlinMOD-Q4 — snapshot form. + * + *

"At time T, what is the list of (vehicleId, entryTime) pairs for all + * vehicles that entered region R at or before T?" + * + *

This is the parity-oracle form: streaming output at watermark T must + * equal the batch BerlinMOD-Q4 result on the same data up to T. + * + *

Keyed by vehicleId. Per-vehicle state: a {@code wasInside} flag plus a + * {@code ListState} of recorded entry times. On each event, detect + * outside → inside transitions and append entry time. Register an event-time + * timer at the next snapshot tick. On timer fire at T, emit one + * {@code (T, vehicleId, entryTime)} per recorded entry. + */ +public class Q4SnapshotFunction + extends KeyedProcessFunction> { + + private final double xmin, ymin, xmax, ymax; + private final long snapshotTickMillis; + private transient ValueState wasInside; + private transient ListState entries; + + public Q4SnapshotFunction(double xmin, double ymin, double xmax, double ymax, long snapshotTickMillis) { + this.xmin = xmin; + this.ymin = ymin; + this.xmax = xmax; + this.ymax = ymax; + this.snapshotTickMillis = snapshotTickMillis; + } + + @Override + public void open(Configuration parameters) { + wasInside = getRuntimeContext().getState( + new ValueStateDescriptor<>("q4SnapWasInside", Boolean.class)); + entries = getRuntimeContext().getListState( + new ListStateDescriptor<>("q4SnapEntries", Long.class)); + } + + @Override + public void processElement( + BerlinMODTrip trip, + Context ctx, + Collector> out) throws Exception { + boolean curr = inBox(trip.getLon(), trip.getLat()); + Boolean prev = wasInside.value(); + boolean prevInside = prev != null && prev; + if (curr && !prevInside) { + entries.add(trip.getTimestamp()); + } + wasInside.update(curr); + long nextTick = ((trip.getTimestamp() / snapshotTickMillis) + 1) * snapshotTickMillis; + ctx.timerService().registerEventTimeTimer(nextTick); + } + + @Override + public void onTimer( + long timestamp, + OnTimerContext ctx, + Collector> out) throws Exception { + for (Long entry : entries.get()) { + if (entry <= timestamp) { + out.collect(new Tuple3<>(timestamp, ctx.getCurrentKey(), entry)); + } + } + } + + private boolean inBox(double lon, double lat) { + return MEOSBridge.intersectsBox(lon, lat, xmin, ymin, xmax, ymax); + } +} diff --git a/flink-processor/src/main/java/berlinmod/Q4WindowedFunction.java b/flink-processor/src/main/java/berlinmod/Q4WindowedFunction.java new file mode 100644 index 0000000..9f70036 --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/Q4WindowedFunction.java @@ -0,0 +1,99 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.java.tuple.Tuple4; +import org.apache.flink.streaming.api.functions.windowing.ProcessAllWindowFunction; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.util.Collector; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * BerlinMOD-Q4 — windowed form. + * + *

"Per N-second tumbling window, which vehicles entered region R during + * the window, and at what time?" + * + *

Scans all events in the window, sorted per-vehicle by time, and detects + * outside → inside transitions within the window. Emits one + * {@code (windowStart, windowEnd, vehicleId, entryTime)} per detected entry. + * + *

Note: a vehicle's "outside" state at window start is inferred only from + * the window's first event (no cross-window state). This intra-window + * scoping matches BerlinMOD-Q4's "first N entries during a period" form. + */ +public class Q4WindowedFunction + extends ProcessAllWindowFunction, TimeWindow> { + + private final double xmin, ymin, xmax, ymax; + + public Q4WindowedFunction(double xmin, double ymin, double xmax, double ymax) { + this.xmin = xmin; + this.ymin = ymin; + this.xmax = xmax; + this.ymax = ymax; + } + + @Override + public void process( + Context ctx, + Iterable elements, + Collector> out) { + Map> perVehicle = new HashMap<>(); + for (BerlinMODTrip trip : elements) { + perVehicle.computeIfAbsent(trip.getVehicleId(), k -> new ArrayList<>()).add(trip); + } + for (Map.Entry> e : perVehicle.entrySet()) { + List sorted = e.getValue(); + sorted.sort((a, b) -> Long.compare(a.getTimestamp(), b.getTimestamp())); + boolean prevInside = false; // intra-window only — treats first event as the prior + for (int i = 0; i < sorted.size(); i++) { + BerlinMODTrip t = sorted.get(i); + boolean curr = inBox(t.getLon(), t.getLat()); + if (i == 0) { + if (curr) { + // first event already inside — count as entry per the + // intra-window scoping convention (no prior visibility) + out.collect(new Tuple4<>(ctx.window().getStart(), ctx.window().getEnd(), + e.getKey(), t.getTimestamp())); + } + } else if (curr && !prevInside) { + out.collect(new Tuple4<>(ctx.window().getStart(), ctx.window().getEnd(), + e.getKey(), t.getTimestamp())); + } + prevInside = curr; + } + } + } + + private boolean inBox(double lon, double lat) { + return MEOSBridge.intersectsBox(lon, lat, xmin, ymin, xmax, ymax); + } +} diff --git a/flink-processor/src/main/java/berlinmod/Q5ContinuousFunction.java b/flink-processor/src/main/java/berlinmod/Q5ContinuousFunction.java new file mode 100644 index 0000000..a38a624 --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/Q5ContinuousFunction.java @@ -0,0 +1,119 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.common.state.MapState; +import org.apache.flink.api.common.state.MapStateDescriptor; +import org.apache.flink.api.common.typeinfo.TypeHint; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple4; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.KeyedProcessFunction; +import org.apache.flink.util.Collector; + +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * BerlinMOD-Q5 — continuous form. + * + *

"Which pairs of vehicles are currently meeting near point P?" + * + *

A pair {@code (a, b)} meets near P when both vehicles are within + * {@code dP} metres of {@code P} and the distance between them is at most + * {@code dMeet} metres. + * + *

Caller is expected to key the input stream by a constant so the shared + * cross-vehicle last-known state lives in a single subtask. Per-event: + * update the last-known position of the event's vehicle, then enumerate all + * known pairs and emit {@code (a, b, eventTime, distanceMetres)} for every + * currently-meeting pair (with {@code a < b} for stable identity). + * + *

Predicate: {@link MEOSBridge#dwithinMetres} (MEOS + * {@code edwithin_tgeo_geo}) for the near-P filter and + * {@link MEOSBridge#distanceMetres} (MEOS {@code geog_distance}) for the + * pairwise meeting distance. + */ +public class Q5ContinuousFunction + extends KeyedProcessFunction> { + + private final double pLon, pLat, dPMetres, dMeetMetres; + private transient MapState> lastPos; + + public Q5ContinuousFunction(double pLon, double pLat, double dPMetres, double dMeetMetres) { + this.pLon = pLon; + this.pLat = pLat; + this.dPMetres = dPMetres; + this.dMeetMetres = dMeetMetres; + } + + @Override + public void open(Configuration parameters) { + TypeInformation> vInfo = + TypeInformation.of(new TypeHint>() {}); + lastPos = getRuntimeContext().getMapState( + new MapStateDescriptor<>("q5LastPos", TypeInformation.of(Integer.class), vInfo)); + } + + @Override + public void processElement( + BerlinMODTrip trip, + Context ctx, + Collector> out) throws Exception { + lastPos.put(trip.getVehicleId(), new Tuple2<>(trip.getLon(), trip.getLat())); + + // Snapshot the map and pick pairs near P + Map> snap = new HashMap<>(); + for (Map.Entry> e : lastPos.entries()) { + snap.put(e.getKey(), e.getValue()); + } + List>> nearP = new ArrayList<>(); + for (Map.Entry> e : snap.entrySet()) { + Tuple2 p = e.getValue(); + if (MEOSBridge.dwithinMetres(p.f0, p.f1, pLon, pLat, dPMetres)) { + nearP.add(e); + } + } + nearP.sort(Comparator.comparingInt(Map.Entry::getKey)); + + for (int i = 0; i < nearP.size(); i++) { + for (int j = i + 1; j < nearP.size(); j++) { + Tuple2 a = nearP.get(i).getValue(); + Tuple2 b = nearP.get(j).getValue(); + double d = MEOSBridge.distanceMetres(a.f0, a.f1, b.f0, b.f1); + if (d <= dMeetMetres) { + out.collect(new Tuple4<>( + nearP.get(i).getKey(), nearP.get(j).getKey(), + trip.getTimestamp(), d)); + } + } + } + } +} diff --git a/flink-processor/src/main/java/berlinmod/Q5SnapshotFunction.java b/flink-processor/src/main/java/berlinmod/Q5SnapshotFunction.java new file mode 100644 index 0000000..f4f4aad --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/Q5SnapshotFunction.java @@ -0,0 +1,122 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.common.state.MapState; +import org.apache.flink.api.common.state.MapStateDescriptor; +import org.apache.flink.api.common.typeinfo.TypeHint; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple4; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.KeyedProcessFunction; +import org.apache.flink.util.Collector; + +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * BerlinMOD-Q5 — snapshot form. + * + *

"At time T, which pairs of vehicles are meeting near P (using each + * vehicle's most-recent-known position on or before T)?" + * + *

This is the parity-oracle form: streaming output at watermark T must + * equal the batch BerlinMOD-Q5 result on the same data up to T. + * + *

Caller is expected to key the stream by a constant; the + * cross-vehicle last-known state is a {@link MapState}. On each event, + * update last-known and register an event-time timer at the next snapshot + * tick. On timer fire at T, snapshot the map and emit all meeting pairs. + */ +public class Q5SnapshotFunction + extends KeyedProcessFunction> { + + private final double pLon, pLat, dPMetres, dMeetMetres; + private final long snapshotTickMillis; + private transient MapState> lastPos; + + public Q5SnapshotFunction(double pLon, double pLat, double dPMetres, + double dMeetMetres, long snapshotTickMillis) { + this.pLon = pLon; + this.pLat = pLat; + this.dPMetres = dPMetres; + this.dMeetMetres = dMeetMetres; + this.snapshotTickMillis = snapshotTickMillis; + } + + @Override + public void open(Configuration parameters) { + TypeInformation> vInfo = + TypeInformation.of(new TypeHint>() {}); + lastPos = getRuntimeContext().getMapState( + new MapStateDescriptor<>("q5SnapLastPos", TypeInformation.of(Integer.class), vInfo)); + } + + @Override + public void processElement( + BerlinMODTrip trip, + Context ctx, + Collector> out) throws Exception { + lastPos.put(trip.getVehicleId(), new Tuple2<>(trip.getLon(), trip.getLat())); + long nextTick = ((trip.getTimestamp() / snapshotTickMillis) + 1) * snapshotTickMillis; + ctx.timerService().registerEventTimeTimer(nextTick); + } + + @Override + public void onTimer( + long timestamp, + OnTimerContext ctx, + Collector> out) throws Exception { + Map> snap = new HashMap<>(); + for (Map.Entry> e : lastPos.entries()) { + snap.put(e.getKey(), e.getValue()); + } + List>> nearP = new ArrayList<>(); + for (Map.Entry> e : snap.entrySet()) { + Tuple2 p = e.getValue(); + if (MEOSBridge.dwithinMetres(p.f0, p.f1, pLon, pLat, dPMetres)) { + nearP.add(e); + } + } + nearP.sort(Comparator.comparingInt(Map.Entry::getKey)); + + for (int i = 0; i < nearP.size(); i++) { + for (int j = i + 1; j < nearP.size(); j++) { + Tuple2 a = nearP.get(i).getValue(); + Tuple2 b = nearP.get(j).getValue(); + double d = MEOSBridge.distanceMetres(a.f0, a.f1, b.f0, b.f1); + if (d <= dMeetMetres) { + out.collect(new Tuple4<>(timestamp, + nearP.get(i).getKey(), nearP.get(j).getKey(), d)); + } + } + } + } +} diff --git a/flink-processor/src/main/java/berlinmod/Q5WindowedFunction.java b/flink-processor/src/main/java/berlinmod/Q5WindowedFunction.java new file mode 100644 index 0000000..ddb11e1 --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/Q5WindowedFunction.java @@ -0,0 +1,100 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.streaming.api.functions.windowing.ProcessAllWindowFunction; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.util.Collector; + +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * BerlinMOD-Q5 — windowed form. + * + *

"Per N-second tumbling window, which pairs of vehicles met near P + * (using each vehicle's last-seen-in-window position)?" + * + *

Within each window, take each vehicle's latest position from the + * window's events. Run the same near-P-and-within-meet-distance pair check + * as the continuous form. Emit {@code (windowStart, windowEnd, a, b, + * distanceMetres)} per meeting pair. + */ +public class Q5WindowedFunction + extends ProcessAllWindowFunction, TimeWindow> { + + private final double pLon, pLat, dPMetres, dMeetMetres; + + public Q5WindowedFunction(double pLon, double pLat, double dPMetres, double dMeetMetres) { + this.pLon = pLon; + this.pLat = pLat; + this.dPMetres = dPMetres; + this.dMeetMetres = dMeetMetres; + } + + @Override + public void process( + Context ctx, + Iterable elements, + Collector> out) { + // Last position per vehicle within the window + Map latest = new HashMap<>(); + for (BerlinMODTrip trip : elements) { + BerlinMODTrip prev = latest.get(trip.getVehicleId()); + if (prev == null || trip.getTimestamp() > prev.getTimestamp()) { + latest.put(trip.getVehicleId(), trip); + } + } + + // Filter to vehicles near P + List>> nearP = new ArrayList<>(); + for (Map.Entry e : latest.entrySet()) { + BerlinMODTrip t = e.getValue(); + if (MEOSBridge.dwithinMetres(t.getLon(), t.getLat(), pLon, pLat, dPMetres)) { + nearP.add(new HashMap.SimpleEntry<>(e.getKey(), new Tuple2<>(t.getLon(), t.getLat()))); + } + } + nearP.sort(Comparator.comparingInt(Map.Entry::getKey)); + + for (int i = 0; i < nearP.size(); i++) { + for (int j = i + 1; j < nearP.size(); j++) { + Tuple2 a = nearP.get(i).getValue(); + Tuple2 b = nearP.get(j).getValue(); + double d = MEOSBridge.distanceMetres(a.f0, a.f1, b.f0, b.f1); + if (d <= dMeetMetres) { + out.collect(new Tuple5<>( + ctx.window().getStart(), ctx.window().getEnd(), + nearP.get(i).getKey(), nearP.get(j).getKey(), d)); + } + } + } + } +} diff --git a/flink-processor/src/main/java/berlinmod/Q6ContinuousFunction.java b/flink-processor/src/main/java/berlinmod/Q6ContinuousFunction.java new file mode 100644 index 0000000..bd47809 --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/Q6ContinuousFunction.java @@ -0,0 +1,83 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.typeinfo.TypeHint; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.KeyedProcessFunction; +import org.apache.flink.util.Collector; + +/** + * BerlinMOD-Q6 — continuous form. + * + *

"What is each vehicle's cumulative distance travelled so far?" + * + *

Keyed by vehicleId. For each event, computes the distance from the + * previous-known position (or 0 if first event), adds it to the cumulative + * total, and emits {@code (vehicleId, t, cumulativeMetres)}. + * + *

Distance: {@link MEOSBridge#distanceMetres} — MEOS {@code geog_distance} + * between consecutive WGS84 positions. + */ +public class Q6ContinuousFunction + extends KeyedProcessFunction> { + + private transient ValueState> lastPos; // lon, lat + private transient ValueState totalDist; + + @Override + public void open(Configuration parameters) { + TypeInformation> posType = + TypeInformation.of(new TypeHint>() {}); + lastPos = getRuntimeContext().getState( + new ValueStateDescriptor<>("q6LastPos", posType)); + totalDist = getRuntimeContext().getState( + new ValueStateDescriptor<>("q6TotalDist", Double.class)); + } + + @Override + public void processElement( + BerlinMODTrip trip, + Context ctx, + Collector> out) throws Exception { + Tuple2 prev = lastPos.value(); + Double total = totalDist.value(); + if (total == null) { + total = 0.0; + } + if (prev != null) { + total += MEOSBridge.distanceMetres(prev.f0, prev.f1, trip.getLon(), trip.getLat()); + } + lastPos.update(new Tuple2<>(trip.getLon(), trip.getLat())); + totalDist.update(total); + out.collect(new Tuple3<>(trip.getVehicleId(), trip.getTimestamp(), total)); + } +} diff --git a/flink-processor/src/main/java/berlinmod/Q6SnapshotFunction.java b/flink-processor/src/main/java/berlinmod/Q6SnapshotFunction.java new file mode 100644 index 0000000..4e818fb --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/Q6SnapshotFunction.java @@ -0,0 +1,101 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.typeinfo.TypeHint; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.KeyedProcessFunction; +import org.apache.flink.util.Collector; + +/** + * BerlinMOD-Q6 — snapshot form. + * + *

"At time T, what is each vehicle's total distance travelled up to T?" + * + *

This is the parity-oracle form: streaming output at watermark T must + * equal the batch BerlinMOD-Q6 result on the same data up to T. + * + *

Keyed by vehicleId. Per event, update {@code lastPos}/{@code totalDist} + * state (matching {@link Q6ContinuousFunction}) and register an event-time + * timer at the next snapshot tick. On timer fire at T, emit + * {@code (T, vehicleId, totalMetres)}. + */ +public class Q6SnapshotFunction + extends KeyedProcessFunction> { + + private final long snapshotTickMillis; + private transient ValueState> lastPos; + private transient ValueState totalDist; + + public Q6SnapshotFunction(long snapshotTickMillis) { + this.snapshotTickMillis = snapshotTickMillis; + } + + @Override + public void open(Configuration parameters) { + TypeInformation> posType = + TypeInformation.of(new TypeHint>() {}); + lastPos = getRuntimeContext().getState( + new ValueStateDescriptor<>("q6SnapLastPos", posType)); + totalDist = getRuntimeContext().getState( + new ValueStateDescriptor<>("q6SnapTotalDist", Double.class)); + } + + @Override + public void processElement( + BerlinMODTrip trip, + Context ctx, + Collector> out) throws Exception { + Tuple2 prev = lastPos.value(); + Double total = totalDist.value(); + if (total == null) { + total = 0.0; + } + if (prev != null) { + total += MEOSBridge.distanceMetres(prev.f0, prev.f1, trip.getLon(), trip.getLat()); + } + lastPos.update(new Tuple2<>(trip.getLon(), trip.getLat())); + totalDist.update(total); + long nextTick = ((trip.getTimestamp() / snapshotTickMillis) + 1) * snapshotTickMillis; + ctx.timerService().registerEventTimeTimer(nextTick); + } + + @Override + public void onTimer( + long timestamp, + OnTimerContext ctx, + Collector> out) throws Exception { + Double total = totalDist.value(); + if (total != null) { + out.collect(new Tuple3<>(timestamp, ctx.getCurrentKey(), total)); + } + } +} diff --git a/flink-processor/src/main/java/berlinmod/Q6WindowedFunction.java b/flink-processor/src/main/java/berlinmod/Q6WindowedFunction.java new file mode 100644 index 0000000..0f37f40 --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/Q6WindowedFunction.java @@ -0,0 +1,70 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.java.tuple.Tuple4; +import org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.util.Collector; + +import java.util.ArrayList; +import java.util.List; + +/** + * BerlinMOD-Q6 — windowed form. + * + *

"Per N-second tumbling window, per vehicle, how far did the vehicle + * travel during the window?" + * + *

Keyed by vehicleId, tumbling event-time window. Within each window, + * sort events by timestamp and accumulate great-circle distances between + * consecutive points. Emit {@code (windowStart, windowEnd, vehicleId, + * distanceMetres)}. + */ +public class Q6WindowedFunction + extends ProcessWindowFunction, Integer, TimeWindow> { + + @Override + public void process( + Integer vehicleId, + Context ctx, + Iterable elements, + Collector> out) { + List sorted = new ArrayList<>(); + for (BerlinMODTrip trip : elements) { + sorted.add(trip); + } + sorted.sort((a, b) -> Long.compare(a.getTimestamp(), b.getTimestamp())); + double total = 0.0; + for (int i = 1; i < sorted.size(); i++) { + BerlinMODTrip prev = sorted.get(i - 1); + BerlinMODTrip curr = sorted.get(i); + total += MEOSBridge.distanceMetres(prev.getLon(), prev.getLat(), + curr.getLon(), curr.getLat()); + } + out.collect(new Tuple4<>(ctx.window().getStart(), ctx.window().getEnd(), vehicleId, total)); + } +} diff --git a/flink-processor/src/main/java/berlinmod/Q7ContinuousFunction.java b/flink-processor/src/main/java/berlinmod/Q7ContinuousFunction.java new file mode 100644 index 0000000..08d74cb --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/Q7ContinuousFunction.java @@ -0,0 +1,82 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.common.state.MapState; +import org.apache.flink.api.common.state.MapStateDescriptor; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.KeyedProcessFunction; +import org.apache.flink.util.Collector; + +import java.util.List; + +/** + * BerlinMOD-Q7 — continuous form. + * + *

"For each (vehicle, POI) pair, when did the vehicle first come within + * the POI's radius?" + * + *

Keyed by vehicleId. State is a per-vehicle {@code MapState}. On each event, walk the POI list; if the vehicle is + * within a POI's radius AND no first-passage has been recorded for that + * (vehicle, POI), record it and emit {@code (vehicleId, poiId, firstPassageTime)}. + */ +public class Q7ContinuousFunction + extends KeyedProcessFunction> { + + private final List pois; + private transient MapState firstPassed; + + public Q7ContinuousFunction(List pois) { + this.pois = pois; + } + + @Override + public void open(Configuration parameters) { + firstPassed = getRuntimeContext().getMapState( + new MapStateDescriptor<>("q7FirstPassed", + TypeInformation.of(Integer.class), + TypeInformation.of(Long.class))); + } + + @Override + public void processElement( + BerlinMODTrip trip, + Context ctx, + Collector> out) throws Exception { + for (PointOfInterest poi : pois) { + if (firstPassed.contains(poi.id)) { + continue; + } + if (MEOSBridge.dwithinMetres(trip.getLon(), trip.getLat(), poi.lon, poi.lat, poi.radiusMetres)) { + firstPassed.put(poi.id, trip.getTimestamp()); + out.collect(new Tuple3<>(trip.getVehicleId(), poi.id, trip.getTimestamp())); + } + } + } +} diff --git a/flink-processor/src/main/java/berlinmod/Q7SnapshotFunction.java b/flink-processor/src/main/java/berlinmod/Q7SnapshotFunction.java new file mode 100644 index 0000000..32a6e2f --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/Q7SnapshotFunction.java @@ -0,0 +1,110 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.common.state.MapState; +import org.apache.flink.api.common.state.MapStateDescriptor; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple4; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.KeyedProcessFunction; +import org.apache.flink.util.Collector; + +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +/** + * BerlinMOD-Q7 — snapshot form. + * + *

"At time T, for each (vehicle, POI), the first time the vehicle came + * within the POI's radius on or before T." + * + *

This is the parity-oracle form: streaming output at watermark T must + * equal the batch BerlinMOD-Q7 result on the same data up to T. + * + *

Keyed by vehicleId. Per-vehicle {@code MapState}. + * On each event, detect new first-passages (matching {@link Q7ContinuousFunction}) + * and register an event-time timer at the next snapshot tick. On timer fire + * at T, emit one {@code (T, vehicleId, poiId, firstPassageTime)} per recorded + * first-passage with {@code firstPassageTime ≤ T}. + */ +public class Q7SnapshotFunction + extends KeyedProcessFunction> { + + private final List pois; + private final long snapshotTickMillis; + private transient MapState firstPassed; + + public Q7SnapshotFunction(List pois, long snapshotTickMillis) { + this.pois = pois; + this.snapshotTickMillis = snapshotTickMillis; + } + + @Override + public void open(Configuration parameters) { + firstPassed = getRuntimeContext().getMapState( + new MapStateDescriptor<>("q7SnapFirstPassed", + TypeInformation.of(Integer.class), + TypeInformation.of(Long.class))); + } + + @Override + public void processElement( + BerlinMODTrip trip, + Context ctx, + Collector> out) throws Exception { + for (PointOfInterest poi : pois) { + if (firstPassed.contains(poi.id)) { + continue; + } + if (MEOSBridge.dwithinMetres(trip.getLon(), trip.getLat(), + poi.lon, poi.lat, poi.radiusMetres)) { + firstPassed.put(poi.id, trip.getTimestamp()); + } + } + long nextTick = ((trip.getTimestamp() / snapshotTickMillis) + 1) * snapshotTickMillis; + ctx.timerService().registerEventTimeTimer(nextTick); + } + + @Override + public void onTimer( + long timestamp, + OnTimerContext ctx, + Collector> out) throws Exception { + // Iterate in poiId order for deterministic output + Map sorted = new TreeMap<>(Comparator.naturalOrder()); + for (Map.Entry e : firstPassed.entries()) { + sorted.put(e.getKey(), e.getValue()); + } + for (Map.Entry e : sorted.entrySet()) { + if (e.getValue() <= timestamp) { + out.collect(new Tuple4<>(timestamp, ctx.getCurrentKey(), e.getKey(), e.getValue())); + } + } + } +} diff --git a/flink-processor/src/main/java/berlinmod/Q7WindowedFunction.java b/flink-processor/src/main/java/berlinmod/Q7WindowedFunction.java new file mode 100644 index 0000000..d273064 --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/Q7WindowedFunction.java @@ -0,0 +1,94 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.streaming.api.functions.windowing.ProcessAllWindowFunction; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.util.Collector; + +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * BerlinMOD-Q7 — windowed form. + * + *

"Per N-second tumbling window, for each (vehicle, POI), what was the + * vehicle's first event in the window that placed it inside the POI's + * radius?" + * + *

Intra-window scoping (no cross-window first-passage state). Per window: + * group events by vehicleId, sort by time, walk and for each POI emit the + * timestamp of the first event in the window where the vehicle is inside + * that POI. + */ +public class Q7WindowedFunction + extends ProcessAllWindowFunction, TimeWindow> { + + private final List pois; + + public Q7WindowedFunction(List pois) { + this.pois = pois; + } + + @Override + public void process( + Context ctx, + Iterable elements, + Collector> out) { + Map> perVehicle = new HashMap<>(); + for (BerlinMODTrip trip : elements) { + perVehicle.computeIfAbsent(trip.getVehicleId(), k -> new ArrayList<>()).add(trip); + } + // For deterministic output, iterate vehicles in id order + List vehicleIds = new ArrayList<>(perVehicle.keySet()); + vehicleIds.sort(Comparator.naturalOrder()); + for (Integer vid : vehicleIds) { + List sorted = perVehicle.get(vid); + sorted.sort((a, b) -> Long.compare(a.getTimestamp(), b.getTimestamp())); + Set emittedPois = new HashSet<>(); + for (BerlinMODTrip trip : sorted) { + for (PointOfInterest poi : pois) { + if (emittedPois.contains(poi.id)) { + continue; + } + if (MEOSBridge.dwithinMetres(trip.getLon(), trip.getLat(), + poi.lon, poi.lat, poi.radiusMetres)) { + emittedPois.add(poi.id); + out.collect(new Tuple5<>( + ctx.window().getStart(), ctx.window().getEnd(), + vid, poi.id, trip.getTimestamp())); + } + } + } + } + } +} diff --git a/flink-processor/src/main/java/berlinmod/Q8ContinuousFunction.java b/flink-processor/src/main/java/berlinmod/Q8ContinuousFunction.java new file mode 100644 index 0000000..6d0a277 --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/Q8ContinuousFunction.java @@ -0,0 +1,71 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.streaming.api.functions.ProcessFunction; +import org.apache.flink.util.Collector; + +/** + * BerlinMOD-Q8 — continuous form. + * + *

"Which vehicles are currently within {@code d} metres of a given + * road segment?" + * + *

For each incoming GPS event {@link BerlinMODTrip}, evaluate the + * point-to-segment distance and emit {@code (vehicleId, eventTime, near)} + * per event. No windowing — same shape as {@link Q3ContinuousFunction} but + * with a segment-distance predicate instead of a point-radius one. + * + *

Predicate: {@link MEOSBridge#dwithinSegmentMetres} — MEOS + * {@code edwithin_tgeo_geo} against a LineString geography. + */ +public class Q8ContinuousFunction extends ProcessFunction> { + + private final double s1Lon, s1Lat, s2Lon, s2Lat, radiusMetres; + + public Q8ContinuousFunction(double s1Lon, double s1Lat, + double s2Lon, double s2Lat, + double radiusMetres) { + this.s1Lon = s1Lon; + this.s1Lat = s1Lat; + this.s2Lon = s2Lon; + this.s2Lat = s2Lat; + this.radiusMetres = radiusMetres; + } + + @Override + public void processElement( + BerlinMODTrip trip, + Context ctx, + Collector> out) { + boolean near = MEOSBridge.dwithinSegmentMetres( + trip.getLon(), trip.getLat(), + s1Lon, s1Lat, s2Lon, s2Lat, + radiusMetres); + out.collect(new Tuple3<>(trip.getVehicleId(), trip.getTimestamp(), near)); + } +} diff --git a/flink-processor/src/main/java/berlinmod/Q8SnapshotFunction.java b/flink-processor/src/main/java/berlinmod/Q8SnapshotFunction.java new file mode 100644 index 0000000..034bb52 --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/Q8SnapshotFunction.java @@ -0,0 +1,103 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.typeinfo.TypeHint; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.KeyedProcessFunction; +import org.apache.flink.util.Collector; + +/** + * BerlinMOD-Q8 — snapshot form. + * + *

"At time T, which vehicles are within {@code d} metres of the road + * segment (using their last-known position on or before T)?" + * + *

This is the parity-oracle form: streaming output at watermark T must + * equal the batch BerlinMOD-Q8 result on the same data up to T. + * + *

Keyed by vehicleId. State: per-vehicle last-known {@code (lon, lat, t)}. + * On each event, update state and register an event-time timer at the next + * snapshot tick. On timer fire at T, evaluate the segment-distance predicate + * against the latest stored position and emit {@code (T, vehicleId)} for + * each near vehicle. + */ +public class Q8SnapshotFunction + extends KeyedProcessFunction> { + + private final double s1Lon, s1Lat, s2Lon, s2Lat, radiusMetres; + private final long snapshotTickMillis; + private transient ValueState> lastKnown; + + public Q8SnapshotFunction(double s1Lon, double s1Lat, + double s2Lon, double s2Lat, + double radiusMetres, + long snapshotTickMillis) { + this.s1Lon = s1Lon; + this.s1Lat = s1Lat; + this.s2Lon = s2Lon; + this.s2Lat = s2Lat; + this.radiusMetres = radiusMetres; + this.snapshotTickMillis = snapshotTickMillis; + } + + @Override + public void open(Configuration parameters) { + TypeInformation> tInfo = + TypeInformation.of(new TypeHint>() {}); + lastKnown = getRuntimeContext().getState( + new ValueStateDescriptor<>("q8LastKnown", tInfo)); + } + + @Override + public void processElement( + BerlinMODTrip trip, + Context ctx, + Collector> out) throws Exception { + lastKnown.update(new Tuple3<>(trip.getLon(), trip.getLat(), trip.getTimestamp())); + long nextTick = ((trip.getTimestamp() / snapshotTickMillis) + 1) * snapshotTickMillis; + ctx.timerService().registerEventTimeTimer(nextTick); + } + + @Override + public void onTimer( + long timestamp, + OnTimerContext ctx, + Collector> out) throws Exception { + Tuple3 p = lastKnown.value(); + if (p == null) { + return; + } + if (MEOSBridge.dwithinSegmentMetres(p.f0, p.f1, s1Lon, s1Lat, s2Lon, s2Lat, radiusMetres)) { + out.collect(new Tuple2<>(timestamp, ctx.getCurrentKey())); + } + } +} diff --git a/flink-processor/src/main/java/berlinmod/Q8WindowedFunction.java b/flink-processor/src/main/java/berlinmod/Q8WindowedFunction.java new file mode 100644 index 0000000..85a1fac --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/Q8WindowedFunction.java @@ -0,0 +1,78 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.streaming.api.functions.windowing.ProcessAllWindowFunction; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.util.Collector; + +import java.util.HashSet; +import java.util.Set; + +/** + * BerlinMOD-Q8 — windowed form. + * + *

"Per N-second tumbling window, how many distinct vehicles were within + * {@code d} metres of the road segment at any time during the window?" + * + *

Tumbling event-time window. Walk all events in the window, count + * distinct vehicleIds for which at least one event satisfies the + * segment-distance predicate. Emit {@code (windowStart, windowEnd, + * distinctCount)}. + */ +public class Q8WindowedFunction + extends ProcessAllWindowFunction, TimeWindow> { + + private final double s1Lon, s1Lat, s2Lon, s2Lat, radiusMetres; + + public Q8WindowedFunction(double s1Lon, double s1Lat, + double s2Lon, double s2Lat, + double radiusMetres) { + this.s1Lon = s1Lon; + this.s1Lat = s1Lat; + this.s2Lon = s2Lon; + this.s2Lat = s2Lat; + this.radiusMetres = radiusMetres; + } + + @Override + public void process( + Context ctx, + Iterable elements, + Collector> out) { + Set distinctNear = new HashSet<>(); + for (BerlinMODTrip trip : elements) { + if (MEOSBridge.dwithinSegmentMetres( + trip.getLon(), trip.getLat(), + s1Lon, s1Lat, s2Lon, s2Lat, + radiusMetres)) { + distinctNear.add(trip.getVehicleId()); + } + } + out.collect(new Tuple3<>(ctx.window().getStart(), ctx.window().getEnd(), (long) distinctNear.size())); + } +} diff --git a/flink-processor/src/main/java/berlinmod/Q9ContinuousFunction.java b/flink-processor/src/main/java/berlinmod/Q9ContinuousFunction.java new file mode 100644 index 0000000..2cd4aac --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/Q9ContinuousFunction.java @@ -0,0 +1,94 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.typeinfo.TypeHint; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple4; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.KeyedProcessFunction; +import org.apache.flink.util.Collector; + +/** + * BerlinMOD-Q9 — continuous form. + * + *

"What is the current distance between vehicles X and Y?" + * + *

Driven by events from either X or Y. State holds the last-known position + * of each as {@code Tuple4(xLon, xLat, yLon, yLat)} (with sentinel + * {@code Double.NaN} for unseen). On each event, update the corresponding + * pair of slots; if both are known, emit {@code (eventTime, distanceMetres)}. + * + *

Caller is expected to filter the stream to {@code vehicleId ∈ {X, Y}} + * and key by a constant so the single shared state lives in one subtask. + * (Two-vehicle Q9 is single-task by construction; a generalised "all-pairs" + * variant would be a different operator.) + */ +public class Q9ContinuousFunction + extends KeyedProcessFunction> { + + private final int xVehicleId; + private final int yVehicleId; + private transient ValueState> xy; + + public Q9ContinuousFunction(int xVehicleId, int yVehicleId) { + this.xVehicleId = xVehicleId; + this.yVehicleId = yVehicleId; + } + + @Override + public void open(Configuration parameters) { + TypeInformation> tInfo = + TypeInformation.of(new TypeHint>() {}); + xy = getRuntimeContext().getState(new ValueStateDescriptor<>("q9xy", tInfo)); + } + + @Override + public void processElement( + BerlinMODTrip trip, + Context ctx, + Collector> out) throws Exception { + Tuple4 s = xy.value(); + if (s == null) { + s = new Tuple4<>(Double.NaN, Double.NaN, Double.NaN, Double.NaN); + } + if (trip.getVehicleId() == xVehicleId) { + s = new Tuple4<>(trip.getLon(), trip.getLat(), s.f2, s.f3); + } else if (trip.getVehicleId() == yVehicleId) { + s = new Tuple4<>(s.f0, s.f1, trip.getLon(), trip.getLat()); + } else { + return; + } + xy.update(s); + if (!Double.isNaN(s.f0) && !Double.isNaN(s.f2)) { + double d = MEOSBridge.distanceMetres(s.f0, s.f1, s.f2, s.f3); + out.collect(new Tuple2<>(trip.getTimestamp(), d)); + } + } +} diff --git a/flink-processor/src/main/java/berlinmod/Q9SnapshotFunction.java b/flink-processor/src/main/java/berlinmod/Q9SnapshotFunction.java new file mode 100644 index 0000000..afedd24 --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/Q9SnapshotFunction.java @@ -0,0 +1,105 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.typeinfo.TypeHint; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple4; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.KeyedProcessFunction; +import org.apache.flink.util.Collector; + +/** + * BerlinMOD-Q9 — snapshot form. + * + *

"At time T, what is the distance between vehicles X and Y (using + * their most-recent-known positions on or before T)?" + * + *

This is the parity-oracle form: streaming output at watermark T must + * equal the batch BerlinMOD-Q9 result on the same data up to T. + * + *

Shared single-key state matches {@link Q9ContinuousFunction}. On each + * X or Y event, update the corresponding pair of slots and register an + * event-time timer at the next snapshot tick. On timer fire at T, emit + * {@code (T, distanceMetres)} if both X and Y have been seen by T. + */ +public class Q9SnapshotFunction + extends KeyedProcessFunction> { + + private final int xVehicleId; + private final int yVehicleId; + private final long snapshotTickMillis; + private transient ValueState> xy; + + public Q9SnapshotFunction(int xVehicleId, int yVehicleId, long snapshotTickMillis) { + this.xVehicleId = xVehicleId; + this.yVehicleId = yVehicleId; + this.snapshotTickMillis = snapshotTickMillis; + } + + @Override + public void open(Configuration parameters) { + TypeInformation> tInfo = + TypeInformation.of(new TypeHint>() {}); + xy = getRuntimeContext().getState(new ValueStateDescriptor<>("q9SnapXy", tInfo)); + } + + @Override + public void processElement( + BerlinMODTrip trip, + Context ctx, + Collector> out) throws Exception { + Tuple4 s = xy.value(); + if (s == null) { + s = new Tuple4<>(Double.NaN, Double.NaN, Double.NaN, Double.NaN); + } + if (trip.getVehicleId() == xVehicleId) { + s = new Tuple4<>(trip.getLon(), trip.getLat(), s.f2, s.f3); + } else if (trip.getVehicleId() == yVehicleId) { + s = new Tuple4<>(s.f0, s.f1, trip.getLon(), trip.getLat()); + } else { + return; + } + xy.update(s); + long nextTick = ((trip.getTimestamp() / snapshotTickMillis) + 1) * snapshotTickMillis; + ctx.timerService().registerEventTimeTimer(nextTick); + } + + @Override + public void onTimer( + long timestamp, + OnTimerContext ctx, + Collector> out) throws Exception { + Tuple4 s = xy.value(); + if (s != null && !Double.isNaN(s.f0) && !Double.isNaN(s.f2)) { + double d = MEOSBridge.distanceMetres(s.f0, s.f1, s.f2, s.f3); + out.collect(new Tuple2<>(timestamp, d)); + } + } +} diff --git a/flink-processor/src/main/java/berlinmod/Q9WindowedFunction.java b/flink-processor/src/main/java/berlinmod/Q9WindowedFunction.java new file mode 100644 index 0000000..bcdfa48 --- /dev/null +++ b/flink-processor/src/main/java/berlinmod/Q9WindowedFunction.java @@ -0,0 +1,79 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package berlinmod; + +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.streaming.api.functions.windowing.ProcessAllWindowFunction; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.util.Collector; + +/** + * BerlinMOD-Q9 — windowed form. + * + *

"Per N-second tumbling window, what is the distance between vehicles + * X and Y at the end of the window (their last-seen positions within + * the window)?" + * + *

Scans the window's events, keeps the latest X and the latest Y + * positions, and emits {@code (windowStart, windowEnd, distanceMetres)} if + * both X and Y were seen in the window. If either was missing, emits nothing + * (no triangulation against earlier windows — the windowed form is strictly + * intra-window). + */ +public class Q9WindowedFunction + extends ProcessAllWindowFunction, TimeWindow> { + + private final int xVehicleId; + private final int yVehicleId; + + public Q9WindowedFunction(int xVehicleId, int yVehicleId) { + this.xVehicleId = xVehicleId; + this.yVehicleId = yVehicleId; + } + + @Override + public void process( + Context ctx, + Iterable elements, + Collector> out) { + BerlinMODTrip latestX = null, latestY = null; + for (BerlinMODTrip trip : elements) { + if (trip.getVehicleId() == xVehicleId + && (latestX == null || trip.getTimestamp() > latestX.getTimestamp())) { + latestX = trip; + } else if (trip.getVehicleId() == yVehicleId + && (latestY == null || trip.getTimestamp() > latestY.getTimestamp())) { + latestY = trip; + } + } + if (latestX != null && latestY != null) { + double d = MEOSBridge.distanceMetres( + latestX.getLon(), latestX.getLat(), + latestY.getLon(), latestY.getLat()); + out.collect(new Tuple3<>(ctx.window().getStart(), ctx.window().getEnd(), d)); + } + } +} diff --git a/flink-processor/src/main/java/org/mobilitydb/flink/meos/wirings/MeosBoundedStateMap.java b/flink-processor/src/main/java/org/mobilitydb/flink/meos/wirings/MeosBoundedStateMap.java new file mode 100644 index 0000000..4776f11 --- /dev/null +++ b/flink-processor/src/main/java/org/mobilitydb/flink/meos/wirings/MeosBoundedStateMap.java @@ -0,0 +1,183 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package org.mobilitydb.flink.meos.wirings; + +import jnr.ffi.Pointer; +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.KeyedProcessFunction; +import org.apache.flink.util.Collector; + +import java.io.Serializable; + +/** + * DataStream wiring for the {@code bounded-state} streaming tier of + * the generated {@code org.mobilitydb.meos.MeosOps*} facades. + * + *

The {@code bounded-state} tier is "per-event with bounded per-key + * state, the state IS a MEOS handle". The canonical example is a + * per-key accumulator that keeps the running MEOS value alive across + * events (e.g. per-vehicle running trajectory, per-key tbox union). + * + *

Why state lives as bytes, not as a {@code Pointer}. A + * {@code jnr.ffi.Pointer} is a raw native-memory address. It is not + * portable across JVM restarts; Flink would not be able to checkpoint + * or replay state. The wiring stores the state as a {@code byte[]} + * (typically the MEOS-WKB serialization of the temporal value), with + * adopter-supplied serialize / deserialize / step lambdas mediating + * the round-trip through MEOS: + * + *

{@code
+ *  byte[] state               -- the per-key serialized MEOS value
+ *      ↓ deserialize (MEOS-WKB → Pointer)
+ *  Pointer prev               -- the in-flight MEOS handle
+ *      ↓ step(prev, event) → (newPointer, output)
+ *  Pointer next, OUT out      -- the new in-flight handle + per-event output
+ *      ↓ serialize (Pointer → MEOS-WKB)
+ *  byte[] newState            -- the per-key serialized new MEOS value
+ * }
+ * + *

This serde discipline is the same one MobilityDuck's persistent + * state machines use; it survives Flink savepoint / checkpoint / + * rescaling correctly because state crossing the operator boundary is + * always the MEOS-WKB bytes — never a raw pointer. + * + *

Typical usage — per-vehicle running tbox union via + * {@code MeosOpsFreeCore.union_tbox_tbox} (stateless on its own; + * stateful when applied as a running fold): + * + *

{@code
+ * DataStream in = ...;          // (vehicleId, tbox)
+ * DataStream out = in
+ *     .keyBy(VehicleTbox::vehicleId)
+ *     .process(new MeosBoundedStateMap(
+ *         /* serialize *(/   ptr -> MeosOpsTBox.tbox_as_wkb(ptr, (byte) 4).array(),
+ *         /* deserialize *(/ bytes -> MeosOpsTBox.tbox_from_wkb(Pointer.wrap(...), bytes.length),
+ *         /* step *(/        (prev, evt) -> {
+ *             Pointer eventTbox = evt.toMeosTbox();
+ *             Pointer merged = (prev == null) ? eventTbox
+ *                                              : MeosOpsFreeCore.union_tbox_tbox(prev, eventTbox);
+ *             RunningTbox result = new RunningTbox(evt.vehicleId(), MeosOpsTBox.tbox_as_hexwkb(merged, (byte) 4, null));
+ *             return new MeosStep<>(merged, result);
+ *         }));
+ * }
+ * + *

The first event for a key sees {@code prev == null} (no prior + * state); the wiring handles that case by skipping the + * {@code deserialize} call. On subsequent events, the state is + * re-hydrated, mutated, re-serialized. + * + *

Coverage: bounded-state is the second-largest tier in the + * v4 baseline (797 of 2,097 emitted methods — 513 OO-classified + 284 + * free-fn). Any of them can be wrapped through this single class — + * adopters provide the three lambdas, the wiring handles all of the + * Flink state plumbing. + * + *

State serializer: this implementation uses Flink's built-in + * {@code byte[]} primitive-array serializer (no custom Kryo / Avro / Pojo + * registration needed). The state size per key is bounded by the + * MEOS-WKB size of the running value — sub-KB for typical + * accumulator scenarios. + * + * @param the key type ({@code keyBy} extractor return type) + * @param the input event type + * @param the output type emitted per event + */ +public final class MeosBoundedStateMap + extends KeyedProcessFunction { + + /** Serializable Pointer → bytes serializer (typically MEOS-WKB). */ + @FunctionalInterface + public interface PointerSerialize extends Serializable { + byte[] toBytes(Pointer pointer) throws Exception; + } + + /** Serializable bytes → Pointer deserializer (typically MEOS-WKB). */ + @FunctionalInterface + public interface PointerDeserialize extends Serializable { + Pointer fromBytes(byte[] bytes) throws Exception; + } + + /** Per-event step: (prior MEOS handle, event) → (new handle, output). */ + @FunctionalInterface + public interface MeosStepFn extends Serializable { + MeosStep apply(Pointer prior, IN event) throws Exception; + } + + /** Tuple returned by the step lambda. */ + public static final class MeosStep implements Serializable { + private static final long serialVersionUID = 1L; + public final Pointer newState; + public final OUT output; + public MeosStep(Pointer newState, OUT output) { + this.newState = newState; + this.output = output; + } + } + + private final PointerSerialize serialize; + private final PointerDeserialize deserialize; + private final MeosStepFn step; + + private transient ValueState handleState; + + public MeosBoundedStateMap(PointerSerialize serialize, + PointerDeserialize deserialize, + MeosStepFn step) { + this.serialize = serialize; + this.deserialize = deserialize; + this.step = step; + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + MeosWiringRuntime.ensureInitializedOnThread(); + ValueStateDescriptor descriptor = new ValueStateDescriptor<>( + "meos-bounded-state", + PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO); + handleState = getRuntimeContext().getState(descriptor); + } + + @Override + public void processElement(IN event, + KeyedProcessFunction.Context ctx, + Collector out) throws Exception { + byte[] priorBytes = handleState.value(); + Pointer prior = (priorBytes == null) ? null : deserialize.fromBytes(priorBytes); + + MeosStep stepResult = step.apply(prior, event); + + byte[] newBytes = serialize.toBytes(stepResult.newState); + handleState.update(newBytes); + + if (stepResult.output != null) { + out.collect(stepResult.output); + } + } +} diff --git a/flink-processor/src/main/java/org/mobilitydb/flink/meos/wirings/MeosCrossStreamJoin.java b/flink-processor/src/main/java/org/mobilitydb/flink/meos/wirings/MeosCrossStreamJoin.java new file mode 100644 index 0000000..dfefd6a --- /dev/null +++ b/flink-processor/src/main/java/org/mobilitydb/flink/meos/wirings/MeosCrossStreamJoin.java @@ -0,0 +1,140 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package org.mobilitydb.flink.meos.wirings; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.co.ProcessJoinFunction; +import org.apache.flink.util.Collector; + +import java.io.Serializable; + +/** + * DataStream wiring for the {@code cross-stream} streaming tier of + * the generated {@code org.mobilitydb.meos.MeosOps*} facades. + * + *

The {@code cross-stream} tier is "pairwise across two streams, + * pre-keyed by the same K, time-bounded match window". Canonical + * examples are spatial-relations between two trajectories + * ({@code edwithin_tgeo_tgeo}, {@code eintersects_tgeo_tgeo}) and + * distance functions on two temporals + * ({@code nad_tgeo_tgeo}, {@code mindistance_tgeo_tgeo}). + * + *

Wraps any cross-stream MeosOps call as a Flink + * {@link ProcessJoinFunction} (the operator backing + * {@code KeyedStream.intervalJoin(other)}). The wiring receives one + * left event and one right event per match, both already paired by + * Flink's interval-join machinery, and the adopter's lambda computes + * the pairwise output via the matching MeosOps call. + * + *

Typical usage — per-vehicle-pair "did they come within + * 100m of each other in the last 5 minutes?" via + * {@code MeosOpsTGeo.edwithin_tgeo_tgeo} (tier = {@code cross-stream}): + * + *

{@code
+ * KeyedStream a = streamA.keyBy(VehiclePosition::regionId);
+ * KeyedStream b = streamB.keyBy(VehiclePosition::regionId);
+ *
+ * DataStream meetings = a
+ *     .intervalJoin(b)
+ *         .between(Time.minutes(-5), Time.minutes(5))
+ *         .process(new MeosCrossStreamJoin(
+ *             (left, right, ctx) -> {
+ *                 Pointer leftT  = left.toTGeoPointer();
+ *                 Pointer rightT = right.toTGeoPointer();
+ *                 if (MeosOpsTGeo.edwithin_tgeo_tgeo(leftT, rightT, 100.0) != 0) {
+ *                     return new MeetingEvent(left.id(), right.id(), ctx.getLeftTimestamp());
+ *                 }
+ *                 return null;  // no output for non-matches
+ *             }));
+ * }
+ * + *

The interval-join is keyed (both streams must be pre-keyed by + * the same K, and only events sharing a key are considered for + * pairing). The match window is time-bounded + * ({@code .between(lowerBound, upperBound)}) and event-time aware — + * watermarks drive when matches are emitted. + * + *

Slim adopter signature — same {@code ContextLike}-style + * pattern as {@link MeosWindowedAggregate}: the lambda receives the + * matched left + right events and a slim context exposing the + * left/right timestamps (the bits a MEOS cross-stream call typically + * needs), keeping the wiring lambda free of Flink internals. + * + *

Coverage: 140 of the 2,097 emitted methods (~7%) qualify + * as {@code cross-stream} per the v4 baseline — all of them wrappable + * through this single class. With this PR, every streamable tier in + * the baseline has a generic wiring class; 1,957 of 2,097 (93%) of + * the generated MeosOps* methods are wirable through 4 classes + * without per-method registration. + * + * @param the left-stream event type + * @param the right-stream event type + * @param the per-match output type + */ +public final class MeosCrossStreamJoin + extends ProcessJoinFunction { + + /** Serializable per-match MEOS pairwise call. */ + @FunctionalInterface + public interface JoinFn extends Serializable { + OUT join(L left, R right, ContextLike ctx) throws Exception; + } + + /** + * Slimmer alternative to Flink's {@code ProcessJoinFunction.Context} + * — exposes only the bits a MEOS pairwise call typically needs. + */ + public interface ContextLike { + long getLeftTimestamp(); + long getRightTimestamp(); + } + + private final JoinFn joinFn; + + public MeosCrossStreamJoin(JoinFn joinFn) { + this.joinFn = joinFn; + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + MeosWiringRuntime.ensureInitializedOnThread(); + } + + @Override + public void processElement(L left, R right, + ProcessJoinFunction.Context context, + Collector out) throws Exception { + ContextLike ctx = new ContextLike() { + @Override public long getLeftTimestamp() { return context.getLeftTimestamp(); } + @Override public long getRightTimestamp() { return context.getRightTimestamp(); } + }; + OUT result = joinFn.join(left, right, ctx); + if (result != null) { + out.collect(result); + } + } +} diff --git a/flink-processor/src/main/java/org/mobilitydb/flink/meos/wirings/MeosStatelessFilter.java b/flink-processor/src/main/java/org/mobilitydb/flink/meos/wirings/MeosStatelessFilter.java new file mode 100644 index 0000000..0ac8163 --- /dev/null +++ b/flink-processor/src/main/java/org/mobilitydb/flink/meos/wirings/MeosStatelessFilter.java @@ -0,0 +1,112 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package org.mobilitydb.flink.meos.wirings; + +import org.apache.flink.api.common.functions.FilterFunction; +import org.apache.flink.api.common.functions.RichFilterFunction; +import org.apache.flink.configuration.Configuration; + +import java.io.Serializable; + +/** + * DataStream wiring for the {@code stateless} streaming tier of the + * generated {@code org.mobilitydb.meos.MeosOps*} facades — the + * predicate-shaped sibling of {@link MeosStatelessMap}. + * + *

Wraps any {@code MeosOps*.f(...)} call that returns {@code boolean} + * (or {@code int} interpreted as a 0/1 flag, common in JMEOS' int-coded + * predicates) and whose streaming tier is {@code stateless} (per + * {@code tools/codegen/meos-ops-manifest.json}) as a Flink + * {@link FilterFunction}. No per-key state; each event filtered + * independently. + * + *

Typical usage: scalar-predicate filter against the + * generated {@code MeosOpsTBox.overlaps_tbox_tbox} (tier = + * {@code stateless}): + * + *

{@code
+ * DataStream in = ...;
+ * DataStream overlapping = in.filter(
+ *     new MeosStatelessFilter<>(
+ *         pair -> MeosOpsTBox.overlaps_tbox_tbox(pair.a, pair.b)));
+ * }
+ * + *

For int-coded predicates (JMEOS returns {@code int} for some MEOS + * predicates rather than {@code boolean}), use + * {@link #fromIntPredicate}: + * + *

{@code
+ * DataStream in = ...;
+ * DataStream adj = in.filter(
+ *     MeosStatelessFilter.fromIntPredicate(
+ *         pair -> MeosOpsFreeGeo.adjacent_stbox_stbox(pair.a, pair.b)));
+ * }
+ * + * @param the record type being filtered + */ +public final class MeosStatelessFilter extends RichFilterFunction { + + /** Serializable boolean-returning per-event MEOS predicate. */ + @FunctionalInterface + public interface MeosPredicate extends Serializable { + boolean test(IN event) throws Exception; + } + + /** Serializable int-returning per-event MEOS predicate (0/1 flag). */ + @FunctionalInterface + public interface MeosIntPredicate extends Serializable { + int test(IN event) throws Exception; + } + + private final MeosPredicate predicate; + + public MeosStatelessFilter(MeosPredicate predicate) { + this.predicate = predicate; + } + + /** + * Adapt an {@code int}-returning generated MEOS predicate (treating + * non-zero as {@code true}) into a Flink {@code FilterFunction}. + */ + public static MeosStatelessFilter fromIntPredicate(MeosIntPredicate p) { + return new MeosStatelessFilter<>(event -> p.test(event) != 0); + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + MeosWiringRuntime.ensureInitializedOnThread(); + } + + @Override + public boolean filter(IN event) throws Exception { + // When chained to a legacy source, records are processed on the source's + // emitter thread rather than the thread open() ran on; the ThreadLocal + // guard makes this a cheap no-op after the first call per thread. + MeosWiringRuntime.ensureInitializedOnThread(); + return predicate.test(event); + } +} diff --git a/flink-processor/src/main/java/org/mobilitydb/flink/meos/wirings/MeosStatelessMap.java b/flink-processor/src/main/java/org/mobilitydb/flink/meos/wirings/MeosStatelessMap.java new file mode 100644 index 0000000..7ac2f6c --- /dev/null +++ b/flink-processor/src/main/java/org/mobilitydb/flink/meos/wirings/MeosStatelessMap.java @@ -0,0 +1,118 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package org.mobilitydb.flink.meos.wirings; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.configuration.Configuration; + +import java.io.Serializable; + +/** + * DataStream wiring for the {@code stateless} streaming tier of the + * generated {@code org.mobilitydb.meos.MeosOps*} facades. + * + *

Wraps any {@code MeosOps*.f(...)} call whose streaming tier is + * {@code stateless} (per {@code tools/codegen/meos-ops-manifest.json} + * or {@code meos-ops-free-manifest.json}) as a Flink + * {@link MapFunction}. No per-key state is allocated; each event is + * mapped independently. The wrapped call: + * + *

    + *
  • does no MEOS-handle state across events (per the + * {@code stateless} tier contract);
  • + *
  • does not touch the time domain (no window required);
  • + *
  • may delegate to MEOS via the bundled JMEOS jar when + * {@code MeosOpsRuntime.MEOS_AVAILABLE} — otherwise the + * generated facade throws {@code UnsupportedOperationException}.
  • + *
+ * + *

Typical usage: register a stateless MEOS predicate / arithmetic + * call as a per-event map step in a DataStream pipeline. Example with + * the generated {@code MeosOpsTBox.overlaps_tbox_tbox} (tier = + * {@code stateless}, per the codegen manifest): + * + *

{@code
+ * DataStream in = ...;            // (tboxA, tboxB)
+ * DataStream overlap = in.map(
+ *     new MeosStatelessMap<>(
+ *         pair -> MeosOpsTBox.overlaps_tbox_tbox(pair.a, pair.b)));
+ * }
+ * + *

Tier coverage: as of the codegen state on the parent PR, + * 804 of the 2,097 generated methods are {@code stateless} (92 OO- + * classified + 712 free-fn). Any of those can be wrapped through this + * single class without per-method boilerplate. + * + *

Coexistence with {@code berlinmod.MEOSBridge}: this is the + * low-level catalog-shaped wiring; {@code MEOSBridge} stays as + * the high-level query-shaped wiring for the BerlinMOD-9 suite. + * Both share the same {@code MeosOpsRuntime.MEOS_AVAILABLE} discipline. + * + * @param the input record type + * @param the output type returned by the wrapped MEOS call + */ +public final class MeosStatelessMap extends RichMapFunction { + + /** + * Serializable per-event MEOS call. Implementations forward to a + * generated {@code MeosOps*.f(...)} static method, returning the + * Java type that the generated facade exposes. + */ + @FunctionalInterface + public interface MeosCall extends Serializable { + OUT apply(IN event) throws Exception; + } + + private final MeosCall call; + + /** + * @param call serializable lambda forwarding to a stateless + * generated MEOS facade method. The lambda must be + * serializable (Java 8+ lambdas implementing a + * {@link Serializable} functional interface are). + */ + public MeosStatelessMap(MeosCall call) { + this.call = call; + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + // No per-key state in the stateless tier; the only per-operator + // concern is MEOS' per-thread session, initialized on this task thread. + MeosWiringRuntime.ensureInitializedOnThread(); + } + + @Override + public OUT map(IN event) throws Exception { + // When chained to a legacy source, records are processed on the source's + // emitter thread rather than the thread open() ran on; the ThreadLocal + // guard makes this a cheap no-op after the first call per thread. + MeosWiringRuntime.ensureInitializedOnThread(); + return call.apply(event); + } +} diff --git a/flink-processor/src/main/java/org/mobilitydb/flink/meos/wirings/MeosWindowedAggregate.java b/flink-processor/src/main/java/org/mobilitydb/flink/meos/wirings/MeosWindowedAggregate.java new file mode 100644 index 0000000..7091253 --- /dev/null +++ b/flink-processor/src/main/java/org/mobilitydb/flink/meos/wirings/MeosWindowedAggregate.java @@ -0,0 +1,148 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package org.mobilitydb.flink.meos.wirings; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction; +import org.apache.flink.streaming.api.windowing.windows.Window; +import org.apache.flink.util.Collector; + +import java.io.Serializable; + +/** + * DataStream wiring for the {@code windowed} streaming tier of the + * generated {@code org.mobilitydb.meos.MeosOps*} facades. + * + *

The {@code windowed} tier is "output cardinality changes; needs a + * window". The canonical examples are + * {@code temporal_length(tgeo)} (one length per trajectory window), + * {@code temporal_twavg(tnumber)} (one time-weighted average per + * window), and the per-class {@code _trajectory} / {@code _time} / + * {@code _timespan} accessors that reduce a full sequence to a single + * derived value. + * + *

Wraps any windowed MeosOps call as a Flink + * {@link ProcessWindowFunction}: per-window, the adopter receives the + * full iterable of events in the window, applies whatever MEOS + * sequence-derived operation is appropriate, and emits a single + * per-window output. The wiring handles the + * {@code ProcessWindowFunction} boilerplate (context, collector) so + * adopters write a single serializable lambda. + * + *

State considerations: unlike + * {@link MeosBoundedStateMap}, the {@code windowed} tier does not + * keep MEOS handles across event boundaries — each window's MEOS + * value is built fresh from the iterable on window close (or + * watermark trigger), used to compute the output, and discarded. The + * iterable's events are Flink-side data; MEOS handles are short-lived + * per-window. + * + *

Typical usage — per-vehicle per-tumbling-window + * trajectory length via {@code MeosOpsTemporal.temporal_length} (tier + * = {@code windowed}): + * + *

{@code
+ * DataStream events = ...;     // (vehicleId, lon, lat, timestamp)
+ * DataStream lengths = events
+ *     .assignTimestampsAndWatermarks(...)
+ *     .keyBy(VehiclePoint::vehicleId)
+ *     .window(TumblingEventTimeWindows.of(Time.minutes(10)))
+ *     .process(new MeosWindowedAggregate(
+ *         (window, events, ctx) -> {
+ *             Pointer trajectory = buildTrajectoryFromPoints(events);  // adopter helper
+ *             double length = MeosOpsTemporal.temporal_length(trajectory);
+ *             return new VehicleLength(ctx.getCurrentKey(), window.getStart(), length);
+ *         }));
+ * }
+ * + *

The window-close path is event-time-aware: when Flink determines + * the window is complete (via watermark), it invokes the lambda once + * with the full iterable, the window metadata, and a context giving + * access to the key. The adopter returns a single output value. + * + *

Coverage: 161 of the 2,097 emitted methods (~8%) qualify + * as {@code windowed} per the v4 baseline — all of them wrappable + * through this single class. + * + * @param the key type + * @param the input event type within the window + * @param the per-window output type + * @param the window type ({@code TimeWindow}, {@code GlobalWindow}, etc.) + */ +public final class MeosWindowedAggregate + extends ProcessWindowFunction { + + /** + * Serializable per-window MEOS aggregate. The lambda receives the + * window metadata, the full iterable of in-window events, and a + * context (for key access). It returns a single per-window output + * value, or {@code null} to emit nothing. + */ + @FunctionalInterface + public interface WindowFn extends Serializable { + OUT aggregate(W window, Iterable events, ContextLike ctx) throws Exception; + } + + /** + * Slimmer alternative to Flink's {@code ProcessWindowFunction.Context} + * — exposes only the bits a MEOS aggregate typically needs (key + + * current processing time + current watermark). Keeps the wiring + * lambda free of Flink internals. + */ + public interface ContextLike { + K getCurrentKey(); + long getCurrentProcessingTime(); + long getCurrentWatermark(); + } + + private final WindowFn windowFn; + + public MeosWindowedAggregate(WindowFn windowFn) { + this.windowFn = windowFn; + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + MeosWiringRuntime.ensureInitializedOnThread(); + } + + @Override + public void process(K key, + ProcessWindowFunction.Context context, + Iterable elements, + Collector out) throws Exception { + ContextLike ctx = new ContextLike() { + @Override public K getCurrentKey() { return key; } + @Override public long getCurrentProcessingTime() { return context.currentProcessingTime(); } + @Override public long getCurrentWatermark() { return context.currentWatermark(); } + }; + OUT result = windowFn.aggregate(context.window(), elements, ctx); + if (result != null) { + out.collect(result); + } + } +} diff --git a/flink-processor/src/main/java/org/mobilitydb/flink/meos/wirings/MeosWiringRuntime.java b/flink-processor/src/main/java/org/mobilitydb/flink/meos/wirings/MeosWiringRuntime.java new file mode 100644 index 0000000..c99bf16 --- /dev/null +++ b/flink-processor/src/main/java/org/mobilitydb/flink/meos/wirings/MeosWiringRuntime.java @@ -0,0 +1,62 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package org.mobilitydb.flink.meos.wirings; + +import functions.GeneratedFunctions; + +/** + * Per-thread MEOS initialization for the {@code org.mobilitydb.flink.meos.wirings} + * operators. + * + *

MEOS keeps its timezone / session state per OS thread. Each Flink + * subtask runs on its own task thread, so every wiring operator must + * initialize MEOS on that thread from its {@code open()} — the JVM-wide + * probe in {@code MeosOpsRuntime} only covers the thread that first + * touches a facade class (typically the job's main thread), not the task + * threads where the operators actually run. + * + *

{@link #ensureInitializedOnThread()} is idempotent per thread (guarded + * by a {@link ThreadLocal}), so it is safe to call from every operator's + * {@code open()} even when operators are chained onto the same thread. It + * installs a no-op error handler so a MEOS-side error surfaces as a thrown + * exception rather than terminating the JVM. + */ +public final class MeosWiringRuntime { + + private static final ThreadLocal INITIALIZED = + ThreadLocal.withInitial(() -> Boolean.FALSE); + + private MeosWiringRuntime() { /* utility */ } + + /** Initialize MEOS on the calling thread exactly once. */ + public static void ensureInitializedOnThread() { + if (!INITIALIZED.get()) { + GeneratedFunctions.meos_initialize_error_handler((level, code, message) -> { }); + GeneratedFunctions.meos_initialize(); + INITIALIZED.set(Boolean.TRUE); + } + } +} diff --git a/flink-processor/src/main/java/org/mobilitydb/flink/meos/wirings/README.md b/flink-processor/src/main/java/org/mobilitydb/flink/meos/wirings/README.md new file mode 100644 index 0000000..b79ec19 --- /dev/null +++ b/flink-processor/src/main/java/org/mobilitydb/flink/meos/wirings/README.md @@ -0,0 +1,89 @@ +# DataStream wirings for the generated MEOS facades + +This package supplies thin, generic Flink-DataStream wrappers around +the generated `org.mobilitydb.meos.MeosOps*` facades, organized +per **streaming tier** (per +`tools/codegen/meos-ops-manifest.json` + `tools/codegen/meos-ops-free-manifest.json`): + +| Tier | Wiring class(es) here | Status in this package | +|---|---|---| +| `stateless` | [`MeosStatelessMap`](MeosStatelessMap.java) (generic `MapFunction`) · [`MeosStatelessFilter`](MeosStatelessFilter.java) (generic `FilterFunction`) | ✅ shipped | +| `bounded-state` | [`MeosBoundedStateMap`](MeosBoundedStateMap.java) (generic `KeyedProcessFunction` with `ValueState` per key — state crosses the operator boundary as MEOS-WKB/WKT bytes so checkpoints/rescaling/savepoints are safe; raw `Pointer` never leaves the JVM-local operator instance) | ✅ shipped | +| `windowed` | [`MeosWindowedAggregate`](MeosWindowedAggregate.java) (generic `ProcessWindowFunction`; window-close-only aggregation; no MEOS handles persist across window boundaries) | ✅ shipped | +| `cross-stream` | [`MeosCrossStreamJoin`](MeosCrossStreamJoin.java) (generic `ProcessJoinFunction` over `KeyedStream.intervalJoin(other)`; time-bounded match window; same-key pairing) | ✅ shipped | +| `io-meta` | covered transitively by the stateless wirings (no state, no window) | n/a | +| `sequence-only` | inherently non-streamable — no wiring | n/a | + +The wirings are **generic**: each takes a serializable lambda +forwarding to whichever generated `MeosOps*.f(...)` method the adopter +needs. No per-method boilerplate, no per-method registration — +adopters wire the entire ~800-method `stateless` slice through +`MeosStatelessMap` / `MeosStatelessFilter` without touching this +package. + +## Why DataStream rather than Table API + +The repo's existing pipeline (`berlinmod/`, `aisdata/`) is +DataStream-API only. Sticking to DataStream avoids adding the +~50 MB `flink-table-planner` runtime dependency to the build matrix. +A Table-API-shaped sibling +(`MeosOpsTableCatalogRegistrar` / `MeosScalarUDF` / `MeosAggregateFunction`) +is a clean follow-up if/when the repo adopts Table API for other +reasons. + +## How a generated MEOS call becomes a Flink operator + +The pattern is the same across all four tiers: + +```java +// 1. Pick the generated MeosOps method +// (Javadoc tier marker tells you which wiring to use) +boolean overlap = MeosOpsTBox.overlaps_tbox_tbox(boxA, boxB); // tier = stateless + +// 2. Wrap with the matching wiring +MeosStatelessFilter filter = MeosStatelessFilter.fromIntPredicate( + pair -> MeosOpsTBox.overlaps_tbox_tbox(pair.a, pair.b)); + +// 3. Apply to the DataStream +DataStream overlapping = stream.filter(filter); +``` + +`MEOS_AVAILABLE` is probed once per JVM by `MeosOpsRuntime`'s static +initializer (shared across all `MeosOps*` and `MeosOpsFree*` +facades). When unavailable, every generated method throws +`UnsupportedOperationException` with a clear message — the wiring +layer doesn't have to handle that itself. + +## End-to-end runnable demo + +[`demo/MeosWiringsDemoJob.java`](demo/MeosWiringsDemoJob.java) walks +through a 3-stage DataStream pipeline using two of the generated +facades wired through `MeosStatelessMap` + `MeosStatelessFilter`: + +1. Parse a stream of TBox WKT strings via + `MeosOpsFreeCore.tbox_in` (io-meta, no state). +2. Filter to those overlapping a fixed query box via + `MeosOpsTBox.overlaps_tbox_tbox` (stateless predicate). +3. Serialize each survivor to hex-WKB via + `MeosOpsTBox.tbox_as_hexwkb` (io-meta, no state). + +Run with: + +```bash +mvn -q exec:java \ + -Dexec.mainClass=org.mobilitydb.flink.meos.wirings.demo.MeosWiringsDemoJob \ + -Dmeos.enabled=true +``` + +Output (expected): two `overlapping-tbox-hex` lines (the two input +boxes that overlap the query box), one disjoint box dropped, one +`MeosWirings stateless tier demo` job completion line. + +## Coexistence with `berlinmod.MEOSBridge` + +`MEOSBridge.java` is the BerlinMOD-specific, hand-written bridge for +the 9-query streaming-form parity matrix — high-level and +query-shaped. The wirings here are low-level and catalog-shaped — +applicable to any of the ~800 stateless or 800 bounded-state +generated facade methods, not just the BerlinMOD-9 subset. Both +share the same `MEOS_AVAILABLE` discipline (`MeosOpsRuntime`). diff --git a/flink-processor/src/main/java/org/mobilitydb/flink/meos/wirings/demo/MeosAllTiersCapstoneDemo.java b/flink-processor/src/main/java/org/mobilitydb/flink/meos/wirings/demo/MeosAllTiersCapstoneDemo.java new file mode 100644 index 0000000..a9ebaed --- /dev/null +++ b/flink-processor/src/main/java/org/mobilitydb/flink/meos/wirings/demo/MeosAllTiersCapstoneDemo.java @@ -0,0 +1,246 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package org.mobilitydb.flink.meos.wirings.demo; + +import jnr.ffi.Pointer; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple4; +import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.KeyedStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.mobilitydb.meos.MeosOpsFreeCore; +import org.mobilitydb.meos.MeosOpsTBox; +import org.mobilitydb.flink.meos.wirings.MeosBoundedStateMap; +import org.mobilitydb.flink.meos.wirings.MeosCrossStreamJoin; +import org.mobilitydb.flink.meos.wirings.MeosStatelessFilter; +import org.mobilitydb.flink.meos.wirings.MeosWindowedAggregate; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.util.Arrays; + +/** + * Capstone end-to-end demo composing ALL FOUR tier wirings in a single + * Flink DataStream pipeline. + * + *

Proves the wirings compose into a realistic pipeline shape, not + * just work in isolation. Each tier-wiring class drives one stage of + * the pipeline: + * + *

{@code
+ *  Stream A (vehicles)                Stream B (queries)
+ *       │                                  │
+ *  ① MeosStatelessFilter                   │
+ *      (keep events in regions of interest)│
+ *       │                                  │
+ *  ② MeosBoundedStateMap                   │
+ *      (per-vehicle running tbox union)    │
+ *       │                                  │
+ *  ③ MeosWindowedAggregate                 │
+ *      (30s tumbling per-vehicle aggregate)│
+ *       │                                  │
+ *  └─────────────┐                  ┌──────┘
+ *                ↓                  ↓
+ *  ④ MeosCrossStreamJoin
+ *       (interval-join: vehicle aggregates vs region queries
+ *        within ±1m time bound, match by region key)
+ *                ↓
+ *           output
+ * }
+ * + *

The pipeline answers: "for each region, which vehicles had an + * aggregate trajectory (running union) overlapping the region's + * query bbox during the latest 30-second window?" + * + *

Tier per stage: + *

    + *
  1. Stateless filter — drop events outside any region of + * interest (per-event predicate, no state).
  2. + *
  3. Bounded-state map — per-vehicle running tbox union + * (MEOS handle persisted across events as byte[] state).
  4. + *
  5. Windowed aggregate — per-vehicle 30s tumbling tbox + * (window-close-only aggregation, no handle persistence across + * windows).
  6. + *
  7. Cross-stream join — interval-join vehicle aggregates + * against region queries (pre-keyed by region, ±1m bound).
  8. + *
+ * + *

Run with: + * + *

{@code
+ * mvn -q exec:java \
+ *     -Dexec.mainClass=org.mobilitydb.flink.meos.wirings.demo.MeosAllTiersCapstoneDemo \
+ *     -Dmeos.enabled=true
+ * }
+ */ +public final class MeosAllTiersCapstoneDemo { + + private static final Logger LOG = LoggerFactory.getLogger(MeosAllTiersCapstoneDemo.class); + + /** Region IDs we care about — the stateless filter drops events outside this set. */ + private static final java.util.Set REGIONS_OF_INTEREST = + new java.util.HashSet<>(Arrays.asList(1, 2)); + + /** Vehicle event stream — (vehicleId, regionId, eventTboxWKT, eventTimeMs). */ + private static final Tuple4[] VEHICLE_EVENTS = new Tuple4[]{ + // window 1: [0s, 30s) + Tuple4.of(10, 1, "TBOX XT([0,2],[2026-01-01,2026-01-01 00:00:15])", ts("00:00:00")), + Tuple4.of(10, 1, "TBOX XT([1,3],[2026-01-01 00:00:15,2026-01-01 00:00:25])", ts("00:00:15")), + Tuple4.of(20, 2, "TBOX XT([10,12],[2026-01-01,2026-01-01 00:00:15])", ts("00:00:05")), + Tuple4.of(99, 9, "TBOX XT([90,92],[2026-01-01,2026-01-01 00:00:15])", ts("00:00:08")), // region 9 — dropped by stage 1 + Tuple4.of(20, 2, "TBOX XT([11,13],[2026-01-01 00:00:15,2026-01-01 00:00:25])", ts("00:00:20")), + // window 2: [30s, 60s) + Tuple4.of(10, 1, "TBOX XT([0,4],[2026-01-01 00:00:30,2026-01-01 00:00:45])", ts("00:00:30")), + Tuple4.of(20, 2, "TBOX XT([10,15],[2026-01-01 00:00:30,2026-01-01 00:00:45])", ts("00:00:35")), + }; + + /** Region query stream — (regionId, queryTboxWKT, eventTimeMs). */ + private static final Tuple2[] REGION_QUERIES = new Tuple2[]{ + Tuple2.of(1, "TBOX XT([1,3],[2026-01-01 00:00:10,2026-01-01 00:00:25])"), + Tuple2.of(2, "TBOX XT([11,13],[2026-01-01 00:00:10,2026-01-01 00:00:25])"), + Tuple2.of(1, "TBOX XT([2,4],[2026-01-01 00:00:35,2026-01-01 00:00:50])"), + Tuple2.of(2, "TBOX XT([12,14],[2026-01-01 00:00:35,2026-01-01 00:00:50])"), + }; + + private static long ts(String hms) { + String[] parts = hms.split(":"); + long secs = Integer.parseInt(parts[0]) * 3600L + + Integer.parseInt(parts[1]) * 60L + + Integer.parseInt(parts[2]); + return 1767225600000L + secs * 1000L; + } + + public static void main(String[] args) throws Exception { + if (!MeosOpsTBox.MEOS_AVAILABLE) { + LOG.error("MEOS not available — the demo requires libmeos."); + System.exit(1); + } + + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + // ── Stream A: vehicle events ──────────────────────────────────────── + DataStream> rawEvents = + env.fromCollection(Arrays.asList(VEHICLE_EVENTS)) + .assignTimestampsAndWatermarks( + WatermarkStrategy + .>forBoundedOutOfOrderness(Duration.ofSeconds(1)) + .withTimestampAssigner((e, t) -> e.f3)); + + // ── ① STATELESS FILTER ── keep only events in regions of interest ── + DataStream> inRegion = + rawEvents.filter(new MeosStatelessFilter>( + evt -> REGIONS_OF_INTEREST.contains(evt.f1))); + + // ── ② BOUNDED-STATE MAP ── per-vehicle running tbox union ────────── + // State holds the MEOS-WKT text of the per-vehicle running union; + // emit (vehicleId, regionId, runningUnionWKT, eventTimeMs) per event. + DataStream> runningUnion = inRegion + .keyBy(t -> t.f0) // key by vehicleId + .process(new MeosBoundedStateMap, Tuple4>( + ptr -> MeosOpsTBox.tbox_out(ptr, 6).getBytes(StandardCharsets.UTF_8), + bytes -> MeosOpsTBox.tbox_in(new String(bytes, StandardCharsets.UTF_8)), + (prior, evt) -> { + Pointer eventTbox = MeosOpsTBox.tbox_in(evt.f2); + Pointer newUnion = (prior == null) + ? eventTbox + : MeosOpsFreeCore.union_tbox_tbox(prior, eventTbox, /*strict=*/false); + Tuple4 output = + Tuple4.of(evt.f0, evt.f1, MeosOpsTBox.tbox_out(newUnion, 6), evt.f3); + return new MeosBoundedStateMap.MeosStep<>(newUnion, output); + })) + .returns(org.apache.flink.api.common.typeinfo.TypeInformation.of( + new org.apache.flink.api.common.typeinfo.TypeHint>() {})); + + // ── ③ WINDOWED AGGREGATE ── per-vehicle 30s tumbling tbox union ───── + // Within each 30s window: take the FINAL running-union value per + // vehicle as the per-window summary. + DataStream> windowed = runningUnion + .keyBy(t -> t.f0) // key by vehicleId + .window(TumblingEventTimeWindows.of(Time.seconds(30))) + .process(new MeosWindowedAggregate< + Integer, + Tuple4, + Tuple4, + TimeWindow + >((window, events, ctx) -> { + // Emit the LAST event in the window (the running union at window close). + Tuple4 last = null; + for (Tuple4 e : events) { + last = e; + } + return last; + })) + .returns(org.apache.flink.api.common.typeinfo.TypeInformation.of( + new org.apache.flink.api.common.typeinfo.TypeHint>() {})); + + // ── Stream B: region queries (keyed by regionId for the join) ─────── + DataStream> queryStream = + env.fromCollection(Arrays.asList(REGION_QUERIES)) + .assignTimestampsAndWatermarks( + WatermarkStrategy + .>forBoundedOutOfOrderness(Duration.ofSeconds(1)) + .withTimestampAssigner((e, t) -> ts("00:00:20"))); // single query-time + + // ── ④ CROSS-STREAM JOIN ── vehicle aggregates × region queries ────── + // Pre-key both by regionId; interval-join within ±1m time bound. + // Per matched pair, emit (regionId, vehicleId, aggUnionWKT, queryWKT, vehicleTs). + KeyedStream, Integer> vehiclesKeyed = + windowed.keyBy(t -> t.f1); // key by regionId + KeyedStream, Integer> queriesKeyed = + queryStream.keyBy(q -> q.f0); + + DataStream> overlaps = + vehiclesKeyed.intervalJoin(queriesKeyed) + .between(Time.minutes(-1), Time.minutes(1)) + .process(new MeosCrossStreamJoin< + Tuple4, + Tuple2, + Tuple5 + >((vehAgg, query, ctx) -> { + Pointer aggTbox = MeosOpsTBox.tbox_in(vehAgg.f2); + Pointer queryTbox = MeosOpsTBox.tbox_in(query.f1); + if (MeosOpsFreeCore.overlaps_tbox_tbox(aggTbox, queryTbox)) { + return Tuple5.of(vehAgg.f1, vehAgg.f0, vehAgg.f2, query.f1, vehAgg.f3); + } + return null; + })) + .returns(org.apache.flink.api.common.typeinfo.TypeInformation.of( + new org.apache.flink.api.common.typeinfo.TypeHint>() {})); + + overlaps.print("capstone-output"); + + env.execute("MeosWirings capstone (all 4 tiers composed)"); + } +} diff --git a/flink-processor/src/main/java/org/mobilitydb/flink/meos/wirings/demo/MeosBoundedStateDemoJob.java b/flink-processor/src/main/java/org/mobilitydb/flink/meos/wirings/demo/MeosBoundedStateDemoJob.java new file mode 100644 index 0000000..bec7c25 --- /dev/null +++ b/flink-processor/src/main/java/org/mobilitydb/flink/meos/wirings/demo/MeosBoundedStateDemoJob.java @@ -0,0 +1,139 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package org.mobilitydb.flink.meos.wirings.demo; + +import jnr.ffi.Pointer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.KeyedStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.mobilitydb.meos.MeosOpsFreeCore; +import org.mobilitydb.meos.MeosOpsTBox; +import org.mobilitydb.flink.meos.wirings.MeosBoundedStateMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.charset.StandardCharsets; +import java.util.Arrays; + +/** + * End-to-end runnable demo of the {@code bounded-state} tier wiring. + * + *

Pipeline: + *

    + *
  1. Stream of {@code (vehicleId, eventTboxWKT)} for 2 vehicles, 3 + * events each.
  2. + *
  3. {@code keyBy(vehicleId)} so per-vehicle state isolates.
  4. + *
  5. Per-vehicle running tbox union via + * {@link MeosBoundedStateMap}: state holds the MEOS-WKT text of + * the current union; on each event, deserialize → call + * {@code MeosOpsFreeCore.union_tbox_tbox} → re-serialize.
  6. + *
  7. Emit {@code (vehicleId, runningUnionTboxWKT)} per event.
  8. + *
+ * + *

What the demo proves: + *

    + *
  • Checkpoint-safe state — state crosses the operator + * boundary as {@code byte[]} (MEOS-WKT here, MEOS-WKB in + * production); no raw native pointers in checkpoints.
  • + *
  • Per-key isolation — vehicle 1's running union does not + * leak into vehicle 2's, and vice versa.
  • + *
  • First-event correctness — the wiring handles + * {@code prior == null} on the first event for each key by + * skipping deserialize and seeding state with the first event's + * tbox.
  • + *
+ * + *

Run with: + * + *

{@code
+ * mvn -q exec:java \
+ *     -Dexec.mainClass=org.mobilitydb.flink.meos.wirings.demo.MeosBoundedStateDemoJob \
+ *     -Dmeos.enabled=true
+ * }
+ * + *

Expected output: 6 lines (3 per vehicle), each showing the growing + * union tbox after that event. + */ +public final class MeosBoundedStateDemoJob { + + private static final Logger LOG = LoggerFactory.getLogger(MeosBoundedStateDemoJob.class); + + /** 6 events across 2 vehicles — the running union grows monotonically per key. */ + private static final Tuple2[] EVENTS = new Tuple2[]{ + Tuple2.of(1, "TBOX XT([0,2],[2026-01-01,2026-01-01 01:00])"), + Tuple2.of(2, "TBOX XT([10,12],[2026-01-01,2026-01-01 01:00])"), + Tuple2.of(1, "TBOX XT([3,5],[2026-01-01 01:00,2026-01-01 02:00])"), + Tuple2.of(2, "TBOX XT([13,15],[2026-01-01 01:00,2026-01-01 02:00])"), + Tuple2.of(1, "TBOX XT([1,4],[2026-01-01 02:00,2026-01-01 03:00])"), + Tuple2.of(2, "TBOX XT([11,14],[2026-01-01 02:00,2026-01-01 03:00])"), + }; + + public static void main(String[] args) throws Exception { + if (!MeosOpsTBox.MEOS_AVAILABLE) { + LOG.error("MEOS not available — the demo requires libmeos."); + System.exit(1); + } + + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + DataStream> events = + env.fromCollection(Arrays.asList(EVENTS)); + + KeyedStream, Integer> keyed = + events.keyBy(t -> t.f0); + + // Wire the per-vehicle running union via MeosBoundedStateMap. + // State is the MEOS-WKT text of the current union (byte[] form); + // each event deserializes, unions with the event tbox, re-serializes. + DataStream> runningUnion = keyed.process( + new MeosBoundedStateMap, Tuple2>( + /* serialize: Pointer → byte[] */ + ptr -> MeosOpsTBox.tbox_out(ptr, 6).getBytes(StandardCharsets.UTF_8), + /* deserialize: byte[] → Pointer */ + bytes -> MeosOpsTBox.tbox_in(new String(bytes, StandardCharsets.UTF_8)), + /* step: (prior union, this event) → (new union, output) */ + (prior, evt) -> { + Pointer eventTbox = MeosOpsTBox.tbox_in(evt.f1); + // First event for a key: prior is null — seed with the event's tbox. + // Subsequent events: union prior with the new event's tbox. + Pointer newUnion = (prior == null) + ? eventTbox + : MeosOpsFreeCore.union_tbox_tbox(prior, eventTbox, /*strict=*/false); + Tuple2 output = + Tuple2.of(evt.f0, MeosOpsTBox.tbox_out(newUnion, 6)); + return new MeosBoundedStateMap.MeosStep<>(newUnion, output); + })) + .returns(org.apache.flink.api.common.typeinfo.TypeInformation.of( + new org.apache.flink.api.common.typeinfo.TypeHint>() {})); + + runningUnion.print("running-union"); + + env.execute("MeosWirings bounded-state tier demo"); + } +} diff --git a/flink-processor/src/main/java/org/mobilitydb/flink/meos/wirings/demo/MeosCrossStreamDemoJob.java b/flink-processor/src/main/java/org/mobilitydb/flink/meos/wirings/demo/MeosCrossStreamDemoJob.java new file mode 100644 index 0000000..4b727dc --- /dev/null +++ b/flink-processor/src/main/java/org/mobilitydb/flink/meos/wirings/demo/MeosCrossStreamDemoJob.java @@ -0,0 +1,160 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package org.mobilitydb.flink.meos.wirings.demo; + +import jnr.ffi.Pointer; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.java.tuple.Tuple4; +import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.KeyedStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.mobilitydb.meos.MeosOpsFreeCore; +import org.mobilitydb.meos.MeosOpsTBox; +import org.mobilitydb.flink.meos.wirings.MeosCrossStreamJoin; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; +import java.util.Arrays; + +/** + * End-to-end runnable demo of the {@code cross-stream} tier wiring. + * + *

Pipeline: + *

    + *
  1. Two parallel streams, each carrying {@code (regionId, + * vehicleId, tboxWKT, eventTimeMs)}, sharing the {@code regionId} + * key so cross-stream pairing is per-region.
  2. + *
  3. {@code keyBy(regionId)} on both, then + * {@code .intervalJoin().between(-1m, +1m)} so each event in + * stream A is matched with events in stream B within ±1 minute + * in the same region.
  4. + *
  5. {@link MeosCrossStreamJoin}: for each matched pair, test + * whether the two tboxes overlap via + * {@code MeosOpsFreeCore.overlaps_tbox_tbox}; if yes, emit + * {@code (regionId, vehAId, vehBId, leftTs, rightTs)}.
  6. + *
+ * + *

What the demo proves: + *

    + *
  • Interval-join semantics — only pairs within the time + * bound are matched; outside-window events are skipped.
  • + *
  • Per-key isolation — events in region 1 don't match + * events in region 2, even if their timestamps overlap.
  • + *
  • Pairwise MEOS call — the wiring lambda receives both + * matched events; the adopter calls any cross-stream MeosOps + * method on the pair (here {@code overlaps_tbox_tbox}, which + * is technically stateless on box pairs but the join-pairing + * is what makes it cross-stream).
  • + *
+ * + *

Run with: + * + *

{@code
+ * mvn -q exec:java \
+ *     -Dexec.mainClass=org.mobilitydb.flink.meos.wirings.demo.MeosCrossStreamDemoJob \
+ *     -Dmeos.enabled=true
+ * }
+ */ +public final class MeosCrossStreamDemoJob { + + private static final Logger LOG = LoggerFactory.getLogger(MeosCrossStreamDemoJob.class); + + /** Stream A — vehicle events, 3 per region across 2 regions. */ + private static final Tuple4[] EVENTS_A = new Tuple4[]{ + Tuple4.of(1, 10, "TBOX XT([0,5],[2026-01-01,2026-01-01 00:00:30])", ts("00:00:00")), + Tuple4.of(2, 20, "TBOX XT([100,105],[2026-01-01,2026-01-01 00:00:30])", ts("00:00:05")), + Tuple4.of(1, 11, "TBOX XT([10,15],[2026-01-01 00:00:30,2026-01-01 00:01:00])", ts("00:00:30")), + }; + + /** Stream B — different vehicles, 3 per region across 2 regions. */ + private static final Tuple4[] EVENTS_B = new Tuple4[]{ + Tuple4.of(1, 30, "TBOX XT([3,8],[2026-01-01,2026-01-01 00:00:30])", ts("00:00:10")), // overlaps with A:(1,10) + Tuple4.of(2, 40, "TBOX XT([200,205],[2026-01-01,2026-01-01 00:00:30])", ts("00:00:15")), // disjoint from A:(2,20) + Tuple4.of(1, 31, "TBOX XT([12,17],[2026-01-01 00:00:30,2026-01-01 00:01:00])", ts("00:00:40")), // overlaps with A:(1,11) + }; + + private static long ts(String hms) { + String[] parts = hms.split(":"); + long secs = Integer.parseInt(parts[0]) * 3600L + + Integer.parseInt(parts[1]) * 60L + + Integer.parseInt(parts[2]); + return 1767225600000L + secs * 1000L; // 2026-01-01T00:00:00 UTC in ms + } + + public static void main(String[] args) throws Exception { + if (!MeosOpsTBox.MEOS_AVAILABLE) { + LOG.error("MEOS not available — the demo requires libmeos."); + System.exit(1); + } + + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + DataStream> a = + env.fromCollection(Arrays.asList(EVENTS_A)) + .assignTimestampsAndWatermarks( + WatermarkStrategy + .>forBoundedOutOfOrderness(Duration.ofSeconds(1)) + .withTimestampAssigner((e, ts) -> e.f3)); + DataStream> b = + env.fromCollection(Arrays.asList(EVENTS_B)) + .assignTimestampsAndWatermarks( + WatermarkStrategy + .>forBoundedOutOfOrderness(Duration.ofSeconds(1)) + .withTimestampAssigner((e, ts) -> e.f3)); + + KeyedStream, Integer> aKeyed = a.keyBy(t -> t.f0); + KeyedStream, Integer> bKeyed = b.keyBy(t -> t.f0); + + // Interval-join: pair events in A with events in B within ±1 minute, same region key. + DataStream> overlaps = + aKeyed.intervalJoin(bKeyed) + .between(Time.minutes(-1), Time.minutes(1)) + .process(new MeosCrossStreamJoin< + Tuple4, // L + Tuple4, // R + Tuple5 // OUT: (region, vehA, vehB, lts, rts) + >((left, right, ctx) -> { + Pointer leftTbox = MeosOpsTBox.tbox_in(left.f2); + Pointer rightTbox = MeosOpsTBox.tbox_in(right.f2); + if (MeosOpsFreeCore.overlaps_tbox_tbox(leftTbox, rightTbox)) { + return Tuple5.of(left.f0, left.f1, right.f1, + ctx.getLeftTimestamp(), ctx.getRightTimestamp()); + } + return null; + })) + .returns(org.apache.flink.api.common.typeinfo.TypeInformation.of( + new org.apache.flink.api.common.typeinfo.TypeHint>() {})); + + overlaps.print("cross-stream-overlap"); + + env.execute("MeosWirings cross-stream tier demo"); + } +} diff --git a/flink-processor/src/main/java/org/mobilitydb/flink/meos/wirings/demo/MeosWindowedDemoJob.java b/flink-processor/src/main/java/org/mobilitydb/flink/meos/wirings/demo/MeosWindowedDemoJob.java new file mode 100644 index 0000000..52b79e2 --- /dev/null +++ b/flink-processor/src/main/java/org/mobilitydb/flink/meos/wirings/demo/MeosWindowedDemoJob.java @@ -0,0 +1,157 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package org.mobilitydb.flink.meos.wirings.demo; + +import jnr.ffi.Pointer; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.api.java.tuple.Tuple4; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.mobilitydb.meos.MeosOpsFreeCore; +import org.mobilitydb.meos.MeosOpsTBox; +import org.mobilitydb.flink.meos.wirings.MeosWindowedAggregate; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; +import java.util.Arrays; + +/** + * End-to-end runnable demo of the {@code windowed} tier wiring. + * + *

Pipeline: + *

    + *
  1. Stream of {@code (vehicleId, tboxWKT, eventTimeMs)} for 2 + * vehicles, 4 events each.
  2. + *
  3. {@code assignTimestampsAndWatermarks} so event-time windows + * fire on a bounded-out-of-orderness schedule.
  4. + *
  5. {@code keyBy(vehicleId)} → 30-second tumbling event-time + * window.
  6. + *
  7. Per-window {@link MeosWindowedAggregate}: union all in-window + * event tboxes into a single per-window aggregate tbox via + * repeated {@code MeosOpsFreeCore.union_tbox_tbox}, emit + * {@code (vehicleId, windowStart, eventCount, aggregateTboxWKT)}.
  8. + *
+ * + *

What the demo proves: + *

    + *
  • Window-close timing — events outside the window are + * excluded; events within are aggregated together.
  • + *
  • Per-key isolation — vehicle 1's window aggregate does + * not include vehicle 2's events, and vice versa.
  • + *
  • Stateless aggregation — unlike {@code bounded-state}, + * no MEOS handle persists across window boundaries; each window + * builds its aggregate from scratch from the iterable.
  • + *
+ * + *

Run with: + * + *

{@code
+ * mvn -q exec:java \
+ *     -Dexec.mainClass=org.mobilitydb.flink.meos.wirings.demo.MeosWindowedDemoJob \
+ *     -Dmeos.enabled=true
+ * }
+ * + *

Expected output: 4 lines (2 windows × 2 vehicles), each showing + * the aggregate tbox spanning that window's events for that vehicle. + */ +public final class MeosWindowedDemoJob { + + private static final Logger LOG = LoggerFactory.getLogger(MeosWindowedDemoJob.class); + + /** 8 events across 2 vehicles, two 30s windows each. */ + private static final Tuple3[] EVENTS = new Tuple3[]{ + // window 1: [0s, 30s) + Tuple3.of(1, "TBOX XT([0,2],[2026-01-01,2026-01-01 00:00:10])", ts("00:00:00")), + Tuple3.of(2, "TBOX XT([10,12],[2026-01-01,2026-01-01 00:00:10])", ts("00:00:05")), + Tuple3.of(1, "TBOX XT([3,5],[2026-01-01 00:00:10,2026-01-01 00:00:20])", ts("00:00:10")), + Tuple3.of(2, "TBOX XT([13,15],[2026-01-01 00:00:10,2026-01-01 00:00:20])", ts("00:00:15")), + // window 2: [30s, 60s) + Tuple3.of(1, "TBOX XT([1,4],[2026-01-01 00:00:30,2026-01-01 00:00:40])", ts("00:00:30")), + Tuple3.of(2, "TBOX XT([11,14],[2026-01-01 00:00:30,2026-01-01 00:00:40])", ts("00:00:35")), + Tuple3.of(1, "TBOX XT([2,3],[2026-01-01 00:00:40,2026-01-01 00:00:50])", ts("00:00:40")), + Tuple3.of(2, "TBOX XT([12,13],[2026-01-01 00:00:40,2026-01-01 00:00:50])", ts("00:00:45")), + }; + + /** Convert "HH:MM:SS" relative to 2026-01-01T00:00:00 into epoch milliseconds. */ + private static long ts(String hms) { + String[] parts = hms.split(":"); + long secs = Integer.parseInt(parts[0]) * 3600L + + Integer.parseInt(parts[1]) * 60L + + Integer.parseInt(parts[2]); + return 1767225600000L + secs * 1000L; // 2026-01-01T00:00:00 UTC in ms + } + + public static void main(String[] args) throws Exception { + if (!MeosOpsTBox.MEOS_AVAILABLE) { + LOG.error("MEOS not available — the demo requires libmeos."); + System.exit(1); + } + + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + DataStream> events = + env.fromCollection(Arrays.asList(EVENTS)) + .assignTimestampsAndWatermarks( + WatermarkStrategy + .>forBoundedOutOfOrderness(Duration.ofSeconds(1)) + .withTimestampAssigner((e, ts) -> e.f2)); + + DataStream> aggregates = events + .keyBy(t -> t.f0) + .window(TumblingEventTimeWindows.of(Time.seconds(30))) + .process(new MeosWindowedAggregate< + Integer, // K + Tuple3, // IN + Tuple4, // OUT + TimeWindow // W + >((window, inWindowEvents, ctx) -> { + Pointer agg = null; + int count = 0; + for (Tuple3 evt : inWindowEvents) { + Pointer evtTbox = MeosOpsTBox.tbox_in(evt.f1); + agg = (agg == null) + ? evtTbox + : MeosOpsFreeCore.union_tbox_tbox(agg, evtTbox, /*strict=*/false); + count++; + } + String aggWkt = (agg == null) ? "(empty)" : MeosOpsTBox.tbox_out(agg, 6); + return Tuple4.of(ctx.getCurrentKey(), window.getStart(), count, aggWkt); + })) + .returns(org.apache.flink.api.common.typeinfo.TypeInformation.of( + new org.apache.flink.api.common.typeinfo.TypeHint>() {})); + + aggregates.print("window-aggregate"); + + env.execute("MeosWirings windowed tier demo"); + } +} diff --git a/flink-processor/src/main/java/org/mobilitydb/flink/meos/wirings/demo/MeosWiringsDemoJob.java b/flink-processor/src/main/java/org/mobilitydb/flink/meos/wirings/demo/MeosWiringsDemoJob.java new file mode 100644 index 0000000..99b06c5 --- /dev/null +++ b/flink-processor/src/main/java/org/mobilitydb/flink/meos/wirings/demo/MeosWiringsDemoJob.java @@ -0,0 +1,126 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package org.mobilitydb.flink.meos.wirings.demo; + +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.mobilitydb.meos.MeosOpsFreeCore; +import org.mobilitydb.meos.MeosOpsTBox; +import org.mobilitydb.flink.meos.wirings.MeosStatelessFilter; +import org.mobilitydb.flink.meos.wirings.MeosStatelessMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; + +/** + * End-to-end runnable demo showing how the generated + * {@code org.mobilitydb.meos.MeosOps*} facades wire into a Flink + * {@code DataStream} pipeline through the + * {@code org.mobilitydb.flink.meos.wirings} helpers. + * + *

The demo: + *

    + *
  1. Builds a small in-memory stream of TBox-WKT strings.
  2. + *
  3. Parses each into a JMEOS {@code Pointer} via + * {@code MeosOpsTBox.tbox_in} (tier = {@code io-meta}).
  4. + *
  5. Filters to those that overlap with a fixed query TBox via + * {@code MeosOpsTBox.overlaps_tbox_tbox} wrapped as a + * {@link MeosStatelessFilter} (tier = {@code stateless}).
  6. + *
  7. Maps each surviving TBox to its serialized WKB hex via + * {@code MeosOpsTBox.tbox_as_hexwkb} wrapped as a + * {@link MeosStatelessMap} (tier = {@code io-meta} but no per-key + * state, so the {@code stateless} wiring works for it too).
  8. + *
+ * + *

Run with: + * + *

{@code
+ * mvn -q exec:java \
+ *     -Dexec.mainClass=org.mobilitydb.flink.meos.wirings.demo.MeosWiringsDemoJob \
+ *     -Dmeos.enabled=true   # require libmeos loadable
+ * }
+ * + *

If libmeos is not loadable on the runtime (or + * {@code -Dmeos.enabled=false}), every wrapped MeosOps + * call throws {@code UnsupportedOperationException} with a clear + * message — the demo prints the throw shape and exits non-zero. + */ +public final class MeosWiringsDemoJob { + + private static final Logger LOG = LoggerFactory.getLogger(MeosWiringsDemoJob.class); + + /** A small box covering (xmin=0, ymin=0, xmax=10, ymax=10). */ + private static final String QUERY_TBOX_WKT = "TBOX XT([0,10],[2026-01-01,2026-01-02])"; + + /** Three input boxes — two overlap the query box, one doesn't. */ + private static final String[] INPUT_TBOX_WKTS = { + "TBOX XT([5,15],[2026-01-01,2026-01-02])", // overlaps + "TBOX XT([20,30],[2026-01-01,2026-01-02])", // disjoint + "TBOX XT([3,8],[2026-01-01,2026-01-02])", // overlaps + }; + + public static void main(String[] args) throws Exception { + // Probe MEOS availability (the static initializer in MeosOpsRuntime + // fires the first time any MeosOps class is touched). + if (!MeosOpsTBox.MEOS_AVAILABLE) { + LOG.error("MEOS not available — the demo requires libmeos. " + + "Set -Dmeos.enabled=true and ensure libmeos is loadable."); + System.exit(1); + } + + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + DataStream tboxWkts = env.fromCollection(Arrays.asList(INPUT_TBOX_WKTS)); + + // The records crossing operator boundaries are serialized MEOS values + // (WKT text) — never raw native pointers, which are process-local and + // not serializable across Flink tasks. Each operator parses to a + // transient MEOS handle, calls MEOS, and re-serializes. + + // Stage 1: parse each WKT and re-serialize via tbox_out (stateless io-meta). + DataStream normalized = tboxWkts.map( + new MeosStatelessMap( + wkt -> MeosOpsTBox.tbox_out(MeosOpsTBox.tbox_in(wkt), 6))) + .returns(Types.STRING); + + // Stage 2: filter to those overlapping the query box (stateless). + // The query box is the constant WKT operand, parsed inside the predicate; + // overlaps_tbox_tbox lives on MeosOpsFreeCore (free fn, not OO-classified). + DataStream overlapping = normalized.filter( + new MeosStatelessFilter( + wkt -> MeosOpsFreeCore.overlaps_tbox_tbox( + MeosOpsTBox.tbox_in(wkt), + MeosOpsTBox.tbox_in(QUERY_TBOX_WKT)))); + + overlapping.print("overlapping-tbox"); + + env.execute("MeosWirings stateless tier demo"); + } +} diff --git a/flink-processor/src/test/java/berlinmod/BerlinMODSetSetJoinTest.java b/flink-processor/src/test/java/berlinmod/BerlinMODSetSetJoinTest.java new file mode 100644 index 0000000..3dba8f6 --- /dev/null +++ b/flink-processor/src/test/java/berlinmod/BerlinMODSetSetJoinTest.java @@ -0,0 +1,120 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ +package berlinmod; + +import functions.GeneratedFunctions; +import jnr.ffi.Pointer; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.condition.EnabledIfSystemProperty; +import org.mobilitydb.meos.MeosSetSetJoin; + +import java.util.HashSet; +import java.util.Set; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; + +/** + * Verifies the BerlinMOD trip-level NxN spatial join (the kernel-pruned + * {@link MeosSetSetJoin} set-set family) against an independent per-pair scalar + * baseline ({@code edwithin_tgeo_tgeo} / {@code eintersects_tgeo_tgeo}). The two + * code paths must agree exactly on which trip pairs ever meet / are always + * disjoint. Runs only with {@code -Dmeos.enabled=true} and an extended libmeos + * on the library path. + */ +@EnabledIfSystemProperty(named = "meos.enabled", matches = "true") +class BerlinMODSetSetJoinTest { + + // Four trajectory trips: T1 crosses T0's path mid-window; T3 coincides with + // T0; T2 is far from everything. + private static final String[] TRIPS = { + "[POINT(0 0)@2000-01-01, POINT(10 0)@2000-01-02]", + "[POINT(5 -100)@2000-01-01, POINT(5 100)@2000-01-02]", + "[POINT(100 100)@2000-01-01, POINT(110 100)@2000-01-02]", + "[POINT(0 0)@2000-01-01, POINT(10 0)@2000-01-02]", + }; + private static final double MEET_DIST = 1.0; + + private static Pointer[] trips; + + @BeforeAll + static void init() { + GeneratedFunctions.meos_initialize_error_handler((level, code, message) -> { }); + GeneratedFunctions.meos_initialize(); + trips = new Pointer[TRIPS.length]; + for (int i = 0; i < TRIPS.length; i++) trips[i] = GeneratedFunctions.tgeompoint_in(TRIPS[i]); + } + + @AfterAll + static void fini() { + GeneratedFunctions.meos_finalize(); + } + + private static Set pairSet(int[][] pairs) { + Set s = new HashSet<>(); + for (int[] p : pairs) s.add(((long) p[0] << 32) | (p[1] & 0xffffffffL)); + return s; + } + + @Test + void eDwithinPairsMatchesScalarBaseline() { + Set kernel = pairSet(MeosSetSetJoin.eDwithinPairs(trips, trips, MEET_DIST)); + Set baseline = new HashSet<>(); + for (int i = 0; i < trips.length; i++) + for (int j = 0; j < trips.length; j++) + if (GeneratedFunctions.edwithin_tgeo_tgeo(trips[i], trips[j], MEET_DIST) == 1) + baseline.add(((long) i << 32) | j); + assertEquals(baseline, kernel, "set-set eDwithinPairs must equal the per-pair edwithin scalar"); + // T0/T3 coincide and T1 crosses T0 — the join is non-empty. + org.junit.jupiter.api.Assertions.assertFalse(kernel.isEmpty()); + } + + @Test + void aDisjointPairsMatchesScalarBaseline() { + Set kernel = pairSet(MeosSetSetJoin.aDisjointPairs(trips, trips)); + Set baseline = new HashSet<>(); + for (int i = 0; i < trips.length; i++) + for (int j = 0; j < trips.length; j++) + if (GeneratedFunctions.eintersects_tgeo_tgeo(trips[i], trips[j]) == 0) + baseline.add(((long) i << 32) | j); + assertEquals(baseline, kernel, "set-set aDisjointPairs must equal the never-intersecting scalar baseline"); + } + + @Test + void tDwithinPairsSupersetOfEverWithinWithPeriods() { + MeosSetSetJoin.TDwithin t = MeosSetSetJoin.tDwithinPairs(trips, trips, MEET_DIST); + Set tdw = pairSet(t.pairs); + Set ever = pairSet(MeosSetSetJoin.eDwithinPairs(trips, trips, MEET_DIST)); + // Continuous tDwithin also reports transient trajectory crossings (e.g. T0/T1 + // coincide at the mid-window crossing) that the ever-within predicate misses, + // so the within-interval pairs are a superset of the ever-within pairs. + org.junit.jupiter.api.Assertions.assertTrue(tdw.containsAll(ever), + "every ever-within pair has a within-interval"); + for (int k = 0; k < t.pairs.length; k++) + assertNotNull(t.periodsHexwkb[k], "every within pair carries its in-range period spanset"); + } +} diff --git a/flink-processor/src/test/java/org/mobilitydb/flink/meos/MeosCbufferSmokeTest.java b/flink-processor/src/test/java/org/mobilitydb/flink/meos/MeosCbufferSmokeTest.java new file mode 100644 index 0000000..4bad5d4 --- /dev/null +++ b/flink-processor/src/test/java/org/mobilitydb/flink/meos/MeosCbufferSmokeTest.java @@ -0,0 +1,67 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package org.mobilitydb.flink.meos; + +import org.mobilitydb.meos.*; + +import functions.GeneratedFunctions; +import jnr.ffi.Pointer; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.condition.EnabledIfSystemProperty; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; + +/** + * Runtime check that the cbuffer facade family calls into libmeos and returns + * correct results. Compiled and run only when the build includes the cbuffer + * family ({@code -DCBUFFER=ON}); the family requires a libmeos built with + * {@code -DCBUFFER=ON}. + */ +@EnabledIfSystemProperty(named = "meos.enabled", matches = "true") +class MeosCbufferSmokeTest { + + @BeforeAll + static void init() { + GeneratedFunctions.meos_initialize_error_handler((level, code, message) -> { }); + GeneratedFunctions.meos_initialize(); + } + + @AfterAll + static void finalizeMeos() { + GeneratedFunctions.meos_finalize(); + } + + @Test + void cbuffer() { + Pointer cb = MeosOpsFreeCbuffer.cbuffer_make(MeosOpsFreeGeo.geom_in("POINT(1 1)", 0), 0.5); + assertNotNull(cb); + assertEquals(0.5, MeosOpsFreeCbuffer.cbuffer_radius(cb), 1e-9); + assertNotNull(MeosOpsFreeCbuffer.cbuffer_out(cb, 6)); + } +} diff --git a/flink-processor/src/test/java/org/mobilitydb/flink/meos/MeosFacadeSmokeTest.java b/flink-processor/src/test/java/org/mobilitydb/flink/meos/MeosFacadeSmokeTest.java new file mode 100644 index 0000000..539dc1c --- /dev/null +++ b/flink-processor/src/test/java/org/mobilitydb/flink/meos/MeosFacadeSmokeTest.java @@ -0,0 +1,93 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package org.mobilitydb.flink.meos; + +import org.mobilitydb.meos.*; + +import functions.GeneratedFunctions; +import jnr.ffi.Pointer; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.condition.EnabledIfSystemProperty; + +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Runtime check that the always-built MEOS facade families (core and geo) call + * into libmeos and return correct results. Each constructs a value through a + * {@code MeosOps*} facade method and reads it back. Runs only with + * {@code -Dmeos.enabled=true} and a libmeos on the load path. The + * optional families have their own gated smoke tests + * ({@link MeosCbufferSmokeTest}, {@link MeosNpointSmokeTest}, + * {@link MeosPoseSmokeTest}), each compiled only when its build flag includes + * the family. + */ +@EnabledIfSystemProperty(named = "meos.enabled", matches = "true") +class MeosFacadeSmokeTest { + + @BeforeAll + static void init() { + // No-op error handler so a parse error returns rather than terminating the JVM. + GeneratedFunctions.meos_initialize_error_handler((level, code, message) -> { }); + GeneratedFunctions.meos_initialize(); + } + + @AfterAll + static void finalizeMeos() { + GeneratedFunctions.meos_finalize(); + } + + @Test + void coreTbox() { + Pointer tbox = MeosOpsTBox.tbox_in("TBOX X([1, 2])"); + assertNotNull(tbox); + assertTrue(MeosOpsTBox.tbox_out(tbox, 6).contains("TBOX")); + } + + @Test + void coreIntspan() { + Pointer span = MeosOpsIntSpan.intspan_in("[1, 5)"); + assertNotNull(span); + String out = MeosOpsIntSpan.intspan_out(span); + assertTrue(out.contains("1") && out.contains("5")); + } + + @Test + void geoStbox() { + Pointer stbox = MeosOpsSTBox.stbox_in("STBOX X((1,1),(2,2))"); + assertNotNull(stbox); + assertTrue(MeosOpsSTBox.stbox_out(stbox, 6).contains("STBOX")); + } + + @Test + void geoGeometry() { + Pointer geom = MeosOpsFreeGeo.geom_in("POINT(1 1)", 0); + assertNotNull(geom); + assertTrue(MeosOpsFreeGeo.geo_as_text(geom, 6).toUpperCase().contains("POINT")); + } +} diff --git a/flink-processor/src/test/java/org/mobilitydb/flink/meos/MeosNpointSmokeTest.java b/flink-processor/src/test/java/org/mobilitydb/flink/meos/MeosNpointSmokeTest.java new file mode 100644 index 0000000..0cfaf41 --- /dev/null +++ b/flink-processor/src/test/java/org/mobilitydb/flink/meos/MeosNpointSmokeTest.java @@ -0,0 +1,66 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package org.mobilitydb.flink.meos; + +import org.mobilitydb.meos.*; + +import functions.GeneratedFunctions; +import jnr.ffi.Pointer; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.condition.EnabledIfSystemProperty; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; + +/** + * Runtime check that the npoint facade family calls into libmeos and returns + * correct results. Compiled and run when the build includes the npoint family + * (the default; dropped with {@code -DNPOINT=OFF}). + */ +@EnabledIfSystemProperty(named = "meos.enabled", matches = "true") +class MeosNpointSmokeTest { + + @BeforeAll + static void init() { + GeneratedFunctions.meos_initialize_error_handler((level, code, message) -> { }); + GeneratedFunctions.meos_initialize(); + } + + @AfterAll + static void finalizeMeos() { + GeneratedFunctions.meos_finalize(); + } + + @Test + void npoint() { + Pointer np = MeosOpsFreeNpoint.npoint_make(1, 0.5); + assertNotNull(np); + assertEquals(1, MeosOpsFreeNpoint.npoint_route(np)); + assertEquals(0.5, MeosOpsFreeNpoint.npoint_position(np), 1e-9); + } +} diff --git a/flink-processor/src/test/java/org/mobilitydb/flink/meos/MeosPoseSmokeTest.java b/flink-processor/src/test/java/org/mobilitydb/flink/meos/MeosPoseSmokeTest.java new file mode 100644 index 0000000..3876167 --- /dev/null +++ b/flink-processor/src/test/java/org/mobilitydb/flink/meos/MeosPoseSmokeTest.java @@ -0,0 +1,67 @@ +/***************************************************************************** + * + * This MobilityDB code is provided under The PostgreSQL License. + * Copyright (c) 2020-2026, Université libre de Bruxelles and MobilityDB + * contributors + * + * Permission to use, copy, modify, and distribute this software and its + * documentation for any purpose, without fee, and without a written + * agreement is hereby granted, provided that the above copyright notice and + * this paragraph and the following two paragraphs appear in all copies. + * + * IN NO EVENT SHALL UNIVERSITE LIBRE DE BRUXELLES BE LIABLE TO ANY PARTY FOR + * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING + * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION, + * EVEN IF UNIVERSITE LIBRE DE BRUXELLES HAS BEEN ADVISED OF THE POSSIBILITY + * OF SUCH DAMAGE. + * + * UNIVERSITE LIBRE DE BRUXELLES SPECIFICALLY DISCLAIMS ANY WARRANTIES, + * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS ON + * AN "AS IS" BASIS, AND UNIVERSITE LIBRE DE BRUXELLES HAS NO OBLIGATIONS TO + * PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + * + *****************************************************************************/ + +package org.mobilitydb.flink.meos; + +import org.mobilitydb.meos.*; + +import functions.GeneratedFunctions; +import jnr.ffi.Pointer; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.condition.EnabledIfSystemProperty; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; + +/** + * Runtime check that the pose facade family calls into libmeos and returns + * correct results. Compiled and run only when the build includes the pose + * family ({@code -DPOSE=ON}); the family requires a libmeos built with + * {@code -DPOSE=ON}. + */ +@EnabledIfSystemProperty(named = "meos.enabled", matches = "true") +class MeosPoseSmokeTest { + + @BeforeAll + static void init() { + GeneratedFunctions.meos_initialize_error_handler((level, code, message) -> { }); + GeneratedFunctions.meos_initialize(); + } + + @AfterAll + static void finalizeMeos() { + GeneratedFunctions.meos_finalize(); + } + + @Test + void pose() { + Pointer pose = MeosOpsFreePose.pose_in("Pose(Point(1 1), 0.5)"); + assertNotNull(pose); + assertNotNull(MeosOpsFreePose.pose_out(pose, 6)); + assertEquals(0.5, MeosOpsFreePose.pose_rotation(pose), 1e-9); + } +} diff --git a/flink-processor/target/classes/aisdata/AISData.class b/flink-processor/target/classes/aisdata/AISData.class deleted file mode 100644 index 32b9d60..0000000 Binary files a/flink-processor/target/classes/aisdata/AISData.class and /dev/null differ diff --git a/flink-processor/target/classes/aisdata/AISDataDeserializationSchema.class b/flink-processor/target/classes/aisdata/AISDataDeserializationSchema.class deleted file mode 100644 index 8ed1fde..0000000 Binary files a/flink-processor/target/classes/aisdata/AISDataDeserializationSchema.class and /dev/null differ diff --git a/flink-processor/target/classes/aisdata/AISTestSource.class b/flink-processor/target/classes/aisdata/AISTestSource.class deleted file mode 100644 index a372cbe..0000000 Binary files a/flink-processor/target/classes/aisdata/AISTestSource.class and /dev/null differ diff --git a/flink-processor/target/classes/aisdata/Main$AISDataTimestampAssigner.class b/flink-processor/target/classes/aisdata/Main$AISDataTimestampAssigner.class deleted file mode 100644 index 2c88edc..0000000 Binary files a/flink-processor/target/classes/aisdata/Main$AISDataTimestampAssigner.class and /dev/null differ diff --git a/flink-processor/target/classes/aisdata/Main$AISDataToTuple4MapFunction.class b/flink-processor/target/classes/aisdata/Main$AISDataToTuple4MapFunction.class deleted file mode 100644 index 9762130..0000000 Binary files a/flink-processor/target/classes/aisdata/Main$AISDataToTuple4MapFunction.class and /dev/null differ diff --git a/flink-processor/target/classes/aisdata/Main$DeserializeAISDataMapFunction.class b/flink-processor/target/classes/aisdata/Main$DeserializeAISDataMapFunction.class deleted file mode 100644 index d4f3c47..0000000 Binary files a/flink-processor/target/classes/aisdata/Main$DeserializeAISDataMapFunction.class and /dev/null differ diff --git a/flink-processor/target/classes/aisdata/Main$LogKafkaMessagesMapFunction.class b/flink-processor/target/classes/aisdata/Main$LogKafkaMessagesMapFunction.class deleted file mode 100644 index 2b7b258..0000000 Binary files a/flink-processor/target/classes/aisdata/Main$LogKafkaMessagesMapFunction.class and /dev/null differ diff --git a/flink-processor/target/classes/aisdata/Main.class b/flink-processor/target/classes/aisdata/Main.class deleted file mode 100644 index 26e68e2..0000000 Binary files a/flink-processor/target/classes/aisdata/Main.class and /dev/null differ diff --git a/flink-processor/target/classes/aisdata/TrajectoryWindowFunction.class b/flink-processor/target/classes/aisdata/TrajectoryWindowFunction.class deleted file mode 100644 index 95fc00d..0000000 Binary files a/flink-processor/target/classes/aisdata/TrajectoryWindowFunction.class and /dev/null differ diff --git a/flink-processor/target/classes/log4j.properties b/flink-processor/target/classes/log4j.properties deleted file mode 100644 index 0db67fa..0000000 --- a/flink-processor/target/classes/log4j.properties +++ /dev/null @@ -1,5 +0,0 @@ -log4j.rootLogger= INFO, console - -log4j.appender.console=org.apache.log4j.ConsoleAppender -log4j.appender.console.layout=org.apache.log4j.PatternLayout -log4j.appender.console.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c %x - %m%n \ No newline at end of file diff --git a/flink-processor/target/maven-archiver/pom.properties b/flink-processor/target/maven-archiver/pom.properties deleted file mode 100644 index f358c8c..0000000 --- a/flink-processor/target/maven-archiver/pom.properties +++ /dev/null @@ -1,3 +0,0 @@ -artifactId=flink-kafka2postgres -groupId=Mariana -version=1.0-SNAPSHOT diff --git a/flink-processor/target/maven-status/maven-compiler-plugin/compile/default-compile/createdFiles.lst b/flink-processor/target/maven-status/maven-compiler-plugin/compile/default-compile/createdFiles.lst deleted file mode 100644 index b76b8d5..0000000 --- a/flink-processor/target/maven-status/maven-compiler-plugin/compile/default-compile/createdFiles.lst +++ /dev/null @@ -1,9 +0,0 @@ -aisdata/TrajectoryWindowFunction.class -aisdata/Main$DeserializeAISDataMapFunction.class -aisdata/Main$AISDataToTuple4MapFunction.class -aisdata/AISTestSource.class -aisdata/Main$AISDataTimestampAssigner.class -aisdata/Main.class -aisdata/AISData.class -aisdata/Main$LogKafkaMessagesMapFunction.class -aisdata/AISDataDeserializationSchema.class diff --git a/flink-processor/target/maven-status/maven-compiler-plugin/compile/default-compile/inputFiles.lst b/flink-processor/target/maven-status/maven-compiler-plugin/compile/default-compile/inputFiles.lst deleted file mode 100644 index 7e01bde..0000000 --- a/flink-processor/target/maven-status/maven-compiler-plugin/compile/default-compile/inputFiles.lst +++ /dev/null @@ -1,5 +0,0 @@ -/Users/marianaduarte/MobilityFlink/flink-processor/src/main/java/aisdata/AISData.java -/Users/marianaduarte/MobilityFlink/flink-processor/src/main/java/aisdata/AISDataDeserializationSchema.java -/Users/marianaduarte/MobilityFlink/flink-processor/src/main/java/aisdata/AISTestSource.java -/Users/marianaduarte/MobilityFlink/flink-processor/src/main/java/aisdata/Main.java -/Users/marianaduarte/MobilityFlink/flink-processor/src/main/java/aisdata/TrajectoryWindowFunction.java diff --git a/kafka-producer/python-producer-berlinmod.py b/kafka-producer/python-producer-berlinmod.py new file mode 100644 index 0000000..83cb1fb --- /dev/null +++ b/kafka-producer/python-producer-berlinmod.py @@ -0,0 +1,46 @@ +""" +BerlinMOD Kafka producer for the MobilityFlink BerlinMOD-Q3 scaffold. + +Reads a BerlinMOD CSV (produced by the BerlinMOD generator — +`meos/examples/data/generate_berlinmod_trips.sql` in MobilityDB, at any +scale factor) and emits one JSON record per row to the Kafka topic +`berlinmod`. + +Expected CSV columns (in order): + t -- "YYYY-MM-DD HH:MM:SS" + vehicle_id -- int + lon -- float + lat -- float + +Companion of `python-producer.py` (the existing AIS producer) — same +shape, different schema and topic. +""" + +from json import dumps +from kafka import KafkaProducer +import pandas as pd + +KAFKA_BOOTSTRAP = "kafka:29092" +TOPIC = "berlinmod" +CSV_PATH = "berlinmod-trips.csv" + + +def gen_data(): + df = pd.read_csv(CSV_PATH) + producer = KafkaProducer( + bootstrap_servers=KAFKA_BOOTSTRAP, + value_serializer=lambda x: dumps(x).encode("utf-8"), + ) + for _, row in df.iterrows(): + record = { + "t": row["t"], + "vehicle_id": int(row["vehicle_id"]), + "lon": float(row["lon"]), + "lat": float(row["lat"]), + } + producer.send(topic=TOPIC, value=record) + producer.flush() + + +if __name__ == "__main__": + gen_data()