diff --git a/.gitignore b/.gitignore
index 927f7eb..e55789a 100644
--- a/.gitignore
+++ b/.gitignore
@@ -1,4 +1,5 @@
.idea/*
+build/*
cmake-build-*/*
cmake-build-debug/*
scripts/*.txt
@@ -16,3 +17,5 @@ scripts/benchmarks/*.dat
*eps
resources/datasets/google-cluster-data/saber-debs-demo.data
resources/datasets/lrb/datafile3hours.dat
+.DS_Store
+/docs/.DS_Store
diff --git a/Dockerfile b/Dockerfile
index 6e5db3f..b401f9b 100644
--- a/Dockerfile
+++ b/Dockerfile
@@ -1,5 +1,8 @@
FROM ubuntu:bionic
+ENV TZ=Etc/UTC
+RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone
+
RUN apt update && \
apt upgrade -y && \
apt install -y \
@@ -10,7 +13,8 @@ RUN apt update && \
build-essential \
ccache \
flex \
- g++ \
+ gcc \
+ g++ \
git \
libboost-all-dev \
libbz2-dev \
@@ -33,7 +37,12 @@ RUN apt update && \
pkg-config \
python-dev \
zlib1g-dev \
- wget
+ wget \
+ libaio-dev \
+ libibverbs-dev \
+ bpfcc-tools \
+ sysstat \
+ fio
RUN cd && \
apt remove --purge --auto-remove cmake && \
@@ -88,7 +97,81 @@ ENV LIBRARY_PATH=$LLVM_HOME/lib:$LIBRARY_PATH
ENV LD_LIBRARY_PATH=$LD_LIBRARY_PATH:$LIBRARY_PATH
ENV PATH=/usr/lib/ccache:$PATH
+RUN cd && \
+ apt install -y \
+ git \
+ gcc \
+ g++ \
+ autoconf \
+ automake \
+ asciidoc \
+ asciidoctor \
+ bash-completion \
+ xmlto \
+ libtool \
+ pkg-config \
+ libglib2.0-0 \
+ libglib2.0-dev \
+ libfabric1 \
+ libfabric-dev \
+ doxygen \
+ graphviz \
+ pandoc \
+ libncurses5 \
+ libkmod2 \
+ libkmod-dev \
+ libudev-dev \
+ uuid-dev \
+ libjson-c-dev \
+ libkeyutils-dev \
+ systemd \
+ libsystemd-dev
+
+RUN cd && \
+ git clone https://github.com/pmem/ndctl && \
+ cd ndctl && \
+ git checkout c7767834871f7ce50a2abe1da946e9e16fb08eda && \
+ ./autogen.sh && \
+ ./configure CFLAGS='-g -O2' --prefix=/usr/local --sysconfdir=/etc --libdir=/usr/local/lib64 && \
+ make -j$(nproc) && \
+ make install
+
+RUN cd && \
+ apt install -y \
+ autoconf \
+ automake \
+ pkg-config \
+ libglib2.0-dev \
+ libfabric-dev \
+ pandoc \
+ libncurses5-dev
+
+RUN cd && \
+ git clone https://github.com/pmem/pmdk && \
+ cd pmdk && \
+ git checkout 3bc5b0da5a7a5d5752ad2cb4f5f9bf0edfd47d67 && \
+ export PKG_CONFIG_PATH=/usr/local/lib64/pkgconfig:/usr/local/lib/pkgconfig:${PKG_CONFIG_PATH} && \
+ make -j$(nproc) && \
+ PKG_CONFIG_PATH=/usr/local/lib64/pkgconfig:/usr/local/lib/pkgconfig:${PKG_CONFIG_PATH} make install && \
+ echo /usr/local/lib >> /etc/ld.so.conf && \
+ echo /usr/local/lib64 >> /etc/ld.so.conf && \
+ ldconfig
+# echo 'export PKG_CONFIG_PATH=/usr/local/lib64/pkgconfig:/usr/local/lib/pkgconfig:${PKG_CONFIG_PATH}' >> $HOME/.profile && \
+# source $HOME/.profile
+
+ENV PKG_CONFIG_PATH=/usr/local/lib64/pkgconfig:/usr/local/lib/pkgconfig:${PKG_CONFIG_PATH}
+
+RUN cd && \
+ git clone https://github.com/pmem/libpmemobj-cpp.git && \
+ cd libpmemobj-cpp && \
+ git checkout 9f784bba07b94cd36c9eebeaa88c5df4f05045b2 && \
+ mkdir build && \
+ cd build && \
+ cmake -DTESTS_USE_VALGRIND=OFF .. && \
+ make -j$(nproc) && \
+ make install
+
RUN cd && \
git clone https://github.com/lsds/LightSaber.git && \
cd LightSaber && \
- ./scripts/build.sh
+ ./scripts/build.sh
\ No newline at end of file
diff --git a/README.md b/README.md
index 24aae89..d029f4a 100644
--- a/README.md
+++ b/README.md
@@ -1,17 +1,44 @@
-
+
-# LightSaber [![License](https://img.shields.io/github/license/lsds/LightSaber.svg?branch=master)](https://github.com/lsds/LightSaber/blob/master/LICENCE.md)
+[![License](https://img.shields.io/github/license/lsds/LightSaber.svg?branch=master)](https://github.com/lsds/LightSaber/blob/master/LICENCE.md)
+# Introduction
-LightSaber is a stream processing engine that balances parallelism and incremental processing when
-executing window aggregation queries on multi-core CPUs. LightSaber operates on in-order
-streams of data and achieves up to an order of magnitude higher throughput compared to existing systems.
+As an ever-growing amount of data is acquired and analyzed in real-time, stream processing engines have become an essential part of any data processing stack. Given the importance of this class of applications, modern stream processing engines must be designed specifically for the efficient execution on multi-core CPUs. However, it is challenging to analyze conceptually infinite data streams with high throughput and low latency performance while providing fault-tolerance semantics.
+This project offers two systems to help tackle this problem.
-### Getting started
-The `prepare-software.sh` script will guide you through the installation and compilation process of our system locally.
-The script is tested on **Ubuntu 18.04.5 LTS**.
+
+## LightSaber
+
+LightSaber is a stream processing engine that balances parallelism and incremental processing when executing window aggregation queries
+on multi-core CPUs. LightSaber operates on in-order data streams and achieves up to an order of magnitude higher throughput than existing systems.
+
+See application examples and how to configure LightSaber [here](#running-lightsaber).
+
+
+
+
+
+
+## Scabbard
+Scabbard is the first single-node SPE that supports exactly-once fault-tolerance semantics despite limited local I/O bandwidth.
+It tightly couples the persistence operations with the operator graph through a novel persistent operator graph model and
+dynamically reduces the required disk bandwidth at runtime through adaptive data compression.
+Scabbard is based on the query execution engine and compiler from LightSaber.
+
+See application examples and how to configure Scabbard [here](#running-scabbard).
+
+
+
+
+
+## Getting started
+
+The `prepare-software.sh` script will guide you through the installation of our system locally.
+The script is tested on **Ubuntu 18.04.5 LTS**. If an error occurs, you may have to manually
+remove and add the symbolic links of the compiler binaries in `/usr/lib/ccache/`.
```
$ git clone https://github.com/lsds/LightSaber.git
@@ -20,7 +47,7 @@ $ ./scripts/prepare-software.sh
$ ./scripts/build.sh
```
-Otherwise use the Dockerfile:
+Otherwise, use the Dockerfile:
```
$ git clone https://github.com/lsds/LightSaber.git
$ cd LightSaber
@@ -28,81 +55,194 @@ $ docker build --tag="lightsaber" .
$ docker run -ti lightsaber
```
-### Run unit tests
+### Setting up variables before running the code
+When running a query, the **LightSaber system is used by default**.
+**To enable the features of Scabbard, we have to set the variables defined [here](#scabbard-configuration)**.
+
+Skip the next part if you don't want to change the folder where code/data is stored, and you have
+installed LLVM in the `$HOME` directory.
+
+Before running any query, set the path (the default is the `$HOME` directory) where files are stored in the
+SystemConf.cpp file:
+```
+SystemConf::FILE_ROOT_PATH = ...
+```
+and the path for LLVM/Clang source files in src/CMakeLists (the default is the `$HOME` directory):
+```
+set(USER_PATH "...")
+```
+
+### Adding new applications
+When compiling in `Release` mode, add the `-UNDEBUG` flag in the `CMakeLists.txt` to enable `assert`:
```
-$ ./build/test/unit_tests/unit_tests_run
+target_compile_options(exec ... -UNDEBUG)
+```
+
+### Start with unit tests
+```
+$ ./build/test/unit_tests/ds_unit_tests
+$ ./build/test/unit_tests/internals_unit_tests
+$ ./build/test/unit_tests/operators_unit_tests
```
+## Running LightSaber
+
### Running a microbenchmark (e.g., Projection)
```
$ ./build/test/benchmarks/microbenchmarks/TestProjection
```
-### Running an application benchmark with sample data
+### Running a cluster monitoring application with sample data
```
$ ./build/test/benchmarks/applications/cluster_monitoring
```
-### How to cite LightSaber
-* **[SIGMOD]** Georgios Theodorakis, Alexandros Koliousis, Peter R. Pietzuch, and Holger Pirk. LightSaber: Efficient Window Aggregation on Multi-core Processors, SIGMOD, 2020
+
+### Running benchmarks from the paper
+You can find the results in `build/test/benchmarks/applications/`.
```
-@inproceedings{Theodorakis2020,
- author = {Georgios Theodorakis and Alexandros Koliousis and Peter R. Pietzuch and Holger Pirk},
- title = {{LightSaber: Efficient Window Aggregation on Multi-core Processors}},
- booktitle = {Proceedings of the 2020 ACM SIGMOD International Conference on Management of Data},
- series = {SIGMOD '20},
- year = {2020},
- publisher = {ACM},
- address = {Portland, OR, USA},
-}
+$ cd scripts/lightsaber-bench
+$ ./run-benchmarks-lightsaber.sh
```
-#### Other related publications
-* **[EDBT]** Georgios Theodorakis, Peter R. Pietzuch, and Holger Pirk. SlideSide: A fast Incremental Stream Processing Algorithm for Multiple Queries, EDBT, 2020
-* **[ADMS]** Georgios Theodorakis, Alexandros Koliousis, Peter R. Pietzuch, and Holger Pirk. Hammer Slide: Work- and CPU-efficient Streaming Window Aggregation, ADMS, 2018 [[code]](https://github.com/grtheod/Hammerslide)
-* **[SIGMOD]** Alexandros Koliousis, Matthias Weidlich, Raul Castro Fernandez, Alexander Wolf, Paolo Costa, and Peter Pietzuch. Saber: Window-Based Hybrid Stream Processing for Heterogeneous Architectures, SIGMOD, 2016
-
-
-### The LightSaber engine
-
-
-
-
-#### LightSaber configuration
+### LightSaber configuration
Variables in **SystemConf.h** configure the LightSaber runtime. Each of them also corresponds to a command-line argument available to all LightSaber applications:
###### --threads _N_
-
Sets the number of CPU worker threads (`WORKER_THREADS` variable). The default value is `1`. **CPU worker threads are pinned to physical cores**. The threads are pinned to core ids based on the underlying hardware (e.g., if there are multiple sockets with n cores each, the first n threads are pinned in the first socket and so on).
-###### --slots _N_
+###### --batch-size _N_
+Sets the batch size in bytes (`BATCH_SIZE` variable). The default value is `131072`, i.e. 128 KB.
+
+###### --bundle-size _N_
+Sets the bundle size in bytes (`BUNDLE_SIZE` variable), which is used for generating data in-memory.
+It has to be a multiple of the `BATCH_SIZE`. The default value is `131072`, i.e. 128 KB, which is the same as the `BATCH_SIZE`.
+###### --slots _N_
Sets the number of intermediate query result slots (`SLOTS` variable). The default value is `256`.
###### --partial-windows _N_
-
Sets the maximum number of window fragments in a query task (`PARTIAL_WINDOWS` variable). The default value is `1024`.
###### --circular-size _N_
-
Sets the circular buffer size in bytes (`CIRCULAR_BUFFER_SIZE` variable). The default value is `4194304`, i.e. 4 MB.
###### --unbounded-size _N_
-
Sets the intermediate result buffer size in bytes (`UNBOUNDED_BUFFER_SIZE` variable). The default value is `524288`, i.e. 512 KB.
###### --hashtable-size _N_
-
Hash table size (in number of buckets): hash tables hold partial window aggregate results (`HASH_TABLE_SIZE` variable with the default value 512).
###### --performance-monitor-interval _N_
-
-Sets the performance monitor interval, in msec (`PERFORMANCE_MONITOR_INTERVAL` variable). The default value is `1000`, i.e. 1 sec. Controls how often LightSaber prints on standard output performance statistics such as throughput and latency.
+Sets the performance monitor interval in msec (`PERFORMANCE_MONITOR_INTERVAL` variable).
+The default value is `1000`, i.e. 1 sec. Controls how often LightSaber prints on standard output performance statistics such as throughput and latency.
###### --latency `true`|`false`
-
Determines whether LightSaber should measure task latency or not (`LATENCY_ON` variable). The default value is `false`.
+###### --parallel-merge `true`|`false`
+Determines whether LightSaber uses parallel aggregation when merging fragment windows or not (`PARALLEL_MERGE_ON` variable). The default value is `false`.
+
###### To enable NUMA-aware scheduling
-Set the HAVE_NUMA flag in the respective CMakeLists.txt (e.g., in test/benchmarks/applications/CMakeLists.txt) and recompile the code.
\ No newline at end of file
+Set the `HAVE_NUMA` flag in the respective CMakeLists.txt (e.g., in `test/benchmarks/applications/CMakeLists.txt`) and recompile the code.
+
+###### To ingest/output data with TCP
+
+Set the `TCP_INPUT`/`TCP_OUTPUT` flag in the respective CMakeLists.txt (e.g., in `test/benchmarks/applicationsWithCheckpoints/CMakeLists.txt`) and recompile the code.
+Check the `test/benchmarks/applications/RemoteBenchmark` folder for code samples to create TCP sources/sinks.
+
+###### To ingest/output data with RDMA
+
+Set the `RDMA_INPUT`/`RDMA_OUTPUT` flag in the respective CMakeLists.txt (e.g., in `test/benchmarks/applicationsWithCheckpoints/CMakeLists.txt`) and recompile the code.
+Check the `test/benchmarks/applications/RemoteBenchmark` folder for code samples to create RDMA sources/sinks.
+
+
+
+## Running Scabbard
+
+### Running a microbenchmark (e.g., Aggregation) with persistent input streams and 1-sec checkpoints
+```
+$ ./build/test/benchmarks/microbenchmarks/TestPersistentAggregation
+```
+
+### Running a cluster monitoring application with persistence using sample data
+```
+$ ./build/test/benchmarks/applicationsWithCheckpoints/cluster_monitoring_checkpoints --circular-size 33554432 --unbounded-size 524288 --batch-size 524288 --bundle-size 524288 --query 1 --checkpoint-duration 1000 --disk-block-size 65536 --checkpoint-compression true --persist-input true --lineage true --latency true --threads 1
+```
+
+### Running benchmarks from the paper
+You can find the results in `build/test/benchmarks/applicationsWithCheckpoints/`.
+```
+$ cd scripts/scabbard-bench/paper/
+$ ./run-benchmarks-...-FIG_X.sh
+```
+
+### Scabbard configuration
+
+In addition to [LightSaber's system variables](#lightsaber-configuration), we can configure the Scabbard runtime with variables specific its fault-tolerance semantics.
+Each of them also corresponds to a command-line argument available to all Scabbard applications:
+
+###### --compression-monitor-interval _N_
+Sets the query compression decision update interval in msec (`COMPRESSION_MONITOR_INTERVAL` variable). The default value is `4000` i.e. 4 sec.
+
+###### --checkpoint-duration _N_
+Sets the performance monitor interval in msec (`CHECKPOINT_INTERVAL` variable). The default value is `1000`, i.e. 1 sec.
+
+###### --disk-block-size _N_
+Sets the size of blocks on disk in bytes (`BLOCK_SIZE` variable). The default value is `16KB`.
+
+###### --create-merge `true`|`false`
+Determines whether Scabbard is generating merge tasks to avoid resource starvation due to asynchronous execution (`CREATE_MERGE_WITH_CHECKPOINTS` variable). The default value is `false`.
+
+###### --checkpoint-compression `true`|`false`
+Determines whether Scabbard is compressing data before storing them to disk (`CHECKPOINT_COMPRESSION` variable). The default value is `false`.
+
+###### --persist-input `true`|`false`
+Determines whether Scabbard persists its input streams (`PERSIST_INPUT` variable). The default value is `false`.
+
+###### --lineage `true`|`false`
+Enables dependency tracking required for exaclty-once results (`LINEAGE_ON` variable). The default value is `false`.
+
+###### --adaptive-compression `true`|`false`
+Enables adaptive compression (`ADAPTIVE_COMPRESSION_ON` variable). The default value is `false`.
+
+###### --adaptive-interval _N_
+Sets the interval in msec that triggers the code generation of new compression functions based on collected statistics (`ADAPTIVE_COMPRESSION_INTERVAL` variable). The default value is `4000`, i.e. 4 sec.
+
+###### --recover `true`|`false`
+If set true, Scabbard attempts to recover using previous persisted data (`RECOVER` variable). The default value is `false`.
+
+
+## How to cite Scabbard
+* **[VLDB]** Georgios Theodorakis, Fotios Kounelis, Peter R. Pietzuch, and Holger Pirk. Scabbard: Single-Node Fault-Tolerant Stream Processing, VLDB, 2022
+```
+@inproceedings{Theodorakis2022,
+ author = {Georgios Theodorakis and Fotios Kounelis and Peter R. Pietzuch and Holger Pirk},
+ title = {{Scabbard: Single-Node Fault-Tolerant Stream Processing}},
+ series = {VLDB '22},
+ year = {2022},
+ publisher = {ACM},
+}
+```
+
+## How to cite LightSaber
+* **[SIGMOD]** Georgios Theodorakis, Alexandros Koliousis, Peter R. Pietzuch, and Holger Pirk. LightSaber: Efficient Window Aggregation on Multi-core Processors, SIGMOD, 2020
+```
+@inproceedings{Theodorakis2020,
+ author = {Georgios Theodorakis and Alexandros Koliousis and Peter R. Pietzuch and Holger Pirk},
+ title = {{LightSaber: Efficient Window Aggregation on Multi-core Processors}},
+ booktitle = {Proceedings of the 2020 ACM SIGMOD International Conference on Management of Data},
+ series = {SIGMOD '20},
+ year = {2020},
+ publisher = {ACM},
+ address = {Portland, OR, USA},
+}
+```
+
+### Other related publications
+* **[EDBT]** Georgios Theodorakis, Peter R. Pietzuch, and Holger Pirk. SlideSide: A fast Incremental Stream Processing Algorithm for Multiple Queries, EDBT, 2020
+* **[ADMS]** Georgios Theodorakis, Alexandros Koliousis, Peter R. Pietzuch, and Holger Pirk. Hammer Slide: Work- and CPU-efficient Streaming Window Aggregation, ADMS, 2018 [[code]](https://github.com/grtheod/Hammerslide)
+* **[SIGMOD]** Alexandros Koliousis, Matthias Weidlich, Raul Castro Fernandez, Alexander Wolf, Paolo Costa, and Peter Pietzuch. Saber: Window-Based Hybrid Stream Processing for Heterogeneous Architectures, SIGMOD, 2016
+
+
diff --git a/VERSION b/VERSION
index 8a9ecc2..7bcd0e3 100644
--- a/VERSION
+++ b/VERSION
@@ -1 +1 @@
-0.0.1
\ No newline at end of file
+0.0.2
\ No newline at end of file
diff --git a/docs/images/Scabbard_arch.png b/docs/images/Scabbard_arch.png
new file mode 100644
index 0000000..a60b341
Binary files /dev/null and b/docs/images/Scabbard_arch.png differ
diff --git a/docs/images/Scabbard_logo.png b/docs/images/Scabbard_logo.png
new file mode 100644
index 0000000..c37efba
Binary files /dev/null and b/docs/images/Scabbard_logo.png differ
diff --git a/resources/datasets/google-cluster-data/google-cluster-data.txt b/resources/datasets/google-cluster-data/google-cluster-data.txt
index e384529..2d97a7d 100644
--- a/resources/datasets/google-cluster-data/google-cluster-data.txt
+++ b/resources/datasets/google-cluster-data/google-cluster-data.txt
@@ -16289,4 +16289,97 @@
643 6218406404 447 -1 0 472 2 0 0.041 0.021 0.000 0
643 6251706807 25 336051749 1 256 2 2 0.125 0.080 0.000 0
643 515042969 3 -1 5 24 2 0 0.016 0.016 0.000 0
-643 515042969 3 -1 0 24 2 0 0.016 0.016 0.000
\ No newline at end of file
+644 515042969 3 -1 0 24 2 0 0.016 0.016 0.000 0
+644 6218406404 853 294816715 2 472 2 0 0.041 0.021 0.000 0
+644 6218406404 853 -1 0 472 2 0 0.041 0.021 0.000 0
+644 6218406404 192 294816715 2 472 2 0 0.041 0.021 0.000 0
+644 6218406404 192 -1 0 472 2 0 0.041 0.021 0.000 0
+644 6114773114 124 329168536 2 356 0 0 0.013 0.008 0.000 0
+644 6114773114 124 -1 0 356 0 0 0.013 0.008 0.000 0
+644 6218406404 232 329168536 2 472 2 0 0.041 0.021 0.000 0
+644 6218406404 232 -1 0 472 2 0 0.041 0.021 0.000 0
+644 6249832520 94 329168536 2 153 0 0 0.056 0.032 0.000 0
+644 6249832520 94 -1 0 153 0 0 0.056 0.032 0.000 0
+644 6251706807 0 329168536 1 256 2 2 0.125 0.080 0.000 0
+644 515042969 1 -1 5 24 2 0 0.016 0.016 0.000 0
+644 515042969 1 -1 0 24 2 0 0.016 0.016 0.000 0
+645 515042969 23 610044506 1 24 2 0 0.016 0.016 0.000 0
+645 6114773114 1850 227459271 1 356 0 0 0.013 0.008 0.000 0
+645 6221861800 1365 8055696 1 356 0 0 0.013 0.016 0.000 0
+645 6221861800 7720 4023910289 1 356 0 0 0.013 0.016 0.000 0
+645 6221861800 9723 988445 1 356 0 0 0.013 0.016 0.000 0
+645 6238340468 1084 711085 1 340 1 0 0.063 0.040 0.000 0
+645 6238340468 2083 603107 1 340 1 0 0.063 0.040 0.000 0
+645 6250345153 1199 257500619 2 501 0 0 0.069 0.012 0.000 0
+645 6250345153 1199 -1 0 501 0 0 0.069 0.012 0.000 0
+645 6218406404 112 257500619 2 472 2 0 0.041 0.021 0.000 0
+645 6218406404 112 -1 0 472 2 0 0.041 0.021 0.000 0
+645 6251706807 7 257500619 1 256 2 2 0.125 0.080 0.000 0
+645 6238340468 948 854237645 2 340 1 0 0.063 0.040 0.000 0
+645 6238340468 948 -1 0 340 1 0 0.063 0.040 0.000 0
+645 6221861800 14659 854237645 2 356 0 0 0.013 0.016 0.000 0
+645 6221861800 14659 -1 0 356 0 0 0.013 0.016 0.000 0
+645 6218406404 890 854237645 2 472 2 0 0.041 0.021 0.000 0
+645 6218406404 890 -1 0 472 2 0 0.041 0.021 0.000 0
+645 6218406404 1060 854237645 2 472 2 0 0.041 0.021 0.000 0
+645 6218406404 1060 -1 0 472 2 0 0.041 0.021 0.000 0
+645 6218406404 1062 854237645 2 472 2 0 0.041 0.021 0.000 0
+645 6218406404 1062 -1 0 472 2 0 0.041 0.021 0.000 0
+645 6251706807 9 854237645 1 256 2 2 0.125 0.080 0.000 0
+645 4297552690 0 4820156549 3 118 2 9 0.006 0.008 0.000 1
+645 4297552690 0 -1 0 118 2 9 0.006 0.008 0.000 1
+645 515042969 15 -1 5 24 2 0 0.016 0.016 0.000 0
+645 515042969 15 -1 0 24 2 0 0.016 0.016 0.000 0
+645 515042969 1 336055200 1 24 2 0 0.016 0.016 0.000 0
+645 5715747784 17 63676345 1 256 2 0 0.019 0.014 0.001 0
+645 6114773114 124 1095431 1 356 0 0 0.013 0.008 0.000 0
+645 6221861800 694 381139 1 356 0 0 0.013 0.016 0.000 0
+645 6249832520 94 1274930 1 153 0 0 0.056 0.032 0.000 0
+646 515042969 16 -1 5 24 2 0 0.016 0.016 0.000 0
+646 515042969 16 -1 0 24 2 0 0.016 0.016 0.000 0
+646 6251539840 335 38673936 4 328 0 2 0.025 0.025 0.000 1
+646 6221861800 2494 84899412 2 356 0 0 0.013 0.016 0.000 0
+646 6221861800 2494 -1 0 356 0 0 0.013 0.016 0.000 0
+646 6250345153 240 84899412 2 501 0 0 0.069 0.012 0.000 0
+646 6250345153 240 -1 0 501 0 0 0.069 0.012 0.000 0
+646 6114773114 1864 84899412 2 356 0 0 0.013 0.008 0.000 0
+646 6114773114 1864 -1 0 356 0 0 0.013 0.008 0.000 0
+646 6218406404 1020 84899412 2 472 2 0 0.041 0.021 0.000 0
+646 6218406404 1020 -1 0 472 2 0 0.041 0.021 0.000 0
+646 6218406404 268 84899412 2 472 2 0 0.041 0.021 0.000 0
+646 6218406404 268 -1 0 472 2 0 0.041 0.021 0.000 0
+646 6251706807 24 84899412 1 256 2 2 0.125 0.080 0.000 0
+646 6251693920 39 16914033 4 249 0 2 0.063 0.006 0.000 0
+646 4297552690 0 3231094147 1 118 2 9 0.006 0.008 0.000 1
+646 515042969 15 6565184 1 24 2 0 0.016 0.016 0.000 0
+646 6221861800 14659 449615529 1 356 0 0 0.013 0.016 0.000 0
+646 6238340468 948 974071212 1 340 1 0 0.063 0.040 0.000 0
+646 6250345153 1199 4820356727 1 501 0 0 0.069 0.012 0.000 0
+647 5003132532 0 351634303 2 243 2 0 0.125 0.003 0.000 0
+647 5003132532 0 -1 0 243 2 0 0.125 0.003 0.000 0
+647 6221861800 15888 351634303 2 356 0 0 0.013 0.016 0.000 0
+647 6221861800 15888 -1 0 356 0 0 0.013 0.016 0.000 0
+647 6251706807 13 351634303 1 256 2 2 0.125 0.080 0.000 0
+647 515042969 21 -1 5 24 2 0 0.016 0.016 0.000 0
+647 515042969 21 -1 0 24 2 0 0.016 0.016 0.000 0
+647 515042969 13 -1 5 24 2 0 0.016 0.016 0.000 0
+647 515042969 13 -1 0 24 2 0 0.016 0.016 0.000 0
+647 515042969 24 -1 5 24 2 0 0.016 0.016 0.000 0
+647 515042969 24 -1 0 24 2 0 0.016 0.016 0.000 0
+647 515042969 26 -1 5 24 2 0 0.016 0.016 0.000 0
+647 515042969 26 -1 0 24 2 0 0.016 0.016 0.000 0
+647 6251539840 239 6192629659 4 328 0 2 0.025 0.025 0.000 1
+647 515042969 16 927964999 1 24 2 0 0.016 0.016 0.000 0
+647 6114773114 1864 5781854 1 356 0 0 0.013 0.008 0.000 0
+647 6221861800 2494 228617356 1 356 0 0 0.013 0.016 0.000 0
+647 6250345153 240 227418843 1 501 0 0 0.069 0.012 0.000 0
+647 6251693920 45 38707778 4 249 0 2 0.063 0.006 0.000 0
+647 6221861800 7406 336051749 2 356 0 0 0.013 0.016 0.000 0
+647 6221861800 7406 -1 0 356 0 0 0.013 0.016 0.000 0
+647 6250345153 513 336051749 2 501 0 0 0.069 0.012 0.000 0
+647 6250345153 513 -1 0 501 0 0 0.069 0.012 0.000 0
+647 6218406404 447 336051749 2 472 2 0 0.041 0.021 0.000 0
+647 6218406404 447 -1 0 472 2 0 0.041 0.021 0.000 0
+647 6251706807 25 336051749 1 256 2 2 0.125 0.080 0.000 0
+647 515042969 3 -1 5 24 2 0 0.016 0.016 0.000 0
+647 515042969 3 -1 0 24 2 0 0.016 0.016 0.000 0
\ No newline at end of file
diff --git a/resources/datasets/smartgrid/smartgrid-data.txt b/resources/datasets/smartgrid/smartgrid-data.txt
index 19c4a49..a51c576 100644
--- a/resources/datasets/smartgrid/smartgrid-data.txt
+++ b/resources/datasets/smartgrid/smartgrid-data.txt
@@ -33625,4 +33625,4 @@
1377986484 0.000 1 13 7 4
1377986484 3.216 0 13 7 4
1377986484 0.000 1 2 7 4
-1377986484 3.216 0
\ No newline at end of file
+1377986484 3.216 0 2 7 4
\ No newline at end of file
diff --git a/scripts/build.sh b/scripts/build.sh
index f11e06e..5598b26 100755
--- a/scripts/build.sh
+++ b/scripts/build.sh
@@ -8,6 +8,7 @@ cd build
#cmake .. -DCMAKE_BUILD_TYPE=Debug -DBENCHMARK_DOWNLOAD_DEPENDENCIES=ON
cmake .. -DCMAKE_BUILD_TYPE=Release -DBENCHMARK_DOWNLOAD_DEPENDENCIES=ON
make -j$(nproc)
+cd ..
# Run example
#cd $HOME/LightSaber/build/benchmarks/microbenchmarks
diff --git a/scripts/lightsaber-bench/run-benchmarks-lightsaber.sh b/scripts/lightsaber-bench/run-benchmarks-lightsaber.sh
new file mode 100755
index 0000000..a69b1ce
--- /dev/null
+++ b/scripts/lightsaber-bench/run-benchmarks-lightsaber.sh
@@ -0,0 +1,139 @@
+#!/bin/bash
+
+echo "Start running benchmarks for LightSaber"
+
+allThreads=(1 2 3 4 5 6 7 8 9 10 11 12 13 14 15)
+iterations=(1 2 3 4 5)
+t10=(10)
+t14=(14)
+t15=(15)
+
+path="$HOME/LightSaber/build/test/benchmarks/applicationsWithCheckpoints/"
+
+if [ -n "$path" ]; then
+ echo "The build path is set to $path"
+else
+ echo "Set the application build path. Exiting..."
+ exit
+fi
+
+cd $path # If the application doesn't run from the build folder, it breaks. This happens because of the where the code files are generated.
+echo $PWD
+
+echo "CM1" >> ls_bench_res.txt
+for t in ${t15[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./cluster_monitoring_checkpoints \
+ --circular-size 16777216 --unbounded-size 524288 --batch-size 524288 --bundle-size 524288 \
+ --query 1 --latency true --threads $t >> ls_bench_res.txt
+ done
+done
+
+echo "CM2" >> ls_bench_res.txt
+for t in ${t14[@]}; # 8
+do
+ for it in ${iterations[@]};
+ do
+ ./cluster_monitoring_checkpoints \
+ --circular-size 16777216 --unbounded-size 1048576 --batch-size 524288 --bundle-size 524288 \
+ --query 2 --latency true --threads $t >> ls_bench_res.txt
+ done
+done
+
+echo "SG1" >> ls_bench_res.txt
+for t in ${t10[@]}; #5
+do
+ for it in ${iterations[@]};
+ do
+ ./smartgrid_checkpoints \
+ --query 1 --unbounded-size 262144 --batch-size 1048576 --circular-size 33554432 \
+ --bundle-size 1048576 --slots 128 --latency true --threads $t >> ls_bench_res.txt
+# --query 1 --unbounded-size 262144 --batch-size 524288 --circular-size 16777216 \
+# --bundle-size 524288 --slots 128 --latency true --threads $t >> ls_bench_res.txt
+ done
+done
+
+echo "SG2" >> ls_bench_res.txt
+for t in ${t15[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./smartgrid_checkpoints \
+ --query 2 --hashtable-size 512 --unbounded-size 1048576 --circular-size 16777216 \
+ --bundle-size 524288 --slots 128 --batch-size 524288 --unbounded-size 4194304 \
+ --create-merge true --parallel-merge true --latency true --threads $t >> ls_bench_res.txt
+ done
+done
+
+# increase the circular-size or decouple memory buffering from storage
+echo "SG3" >> ls_bench_res.txt
+for t in ${t15[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./smartgrid_checkpoints \
+ --query 3 --hashtable-size 512 --unbounded-size 1048576 --circular-size 16777216 \
+ --bundle-size 524288 --slots 128 --batch-size 524288 --unbounded-size 4194304 \
+ --create-merge true --parallel-merge true --latency true --threads $t >> ls_bench_res.txt
+ done
+done
+
+echo "LRB1" >> ls_bench_res.txt
+for t in ${t15[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./linear_road_benchmark_checkpoints \
+ --unbounded-size 8388608 --circular-size 16777216 \
+ --batch-size 524288 --bundle-size 524288 --query 1 --hashtable-size 256 \
+ --create-merge true --parallel-merge true --latency true --threads $t >> ls_bench_res.txt
+ done
+done
+
+echo "LRB2" >> ls_bench_res.txt
+for t in ${t15[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./linear_road_benchmark_checkpoints \
+ --unbounded-size 16777216 --circular-size 16777216 \
+ --batch-size 262144 --bundle-size 262144 --query 2 \
+ --create-merge true --parallel-merge true --latency true --threads $t >> ls_bench_res.txt
+ done
+done
+
+echo "LRB3" >> ls_bench_res.txt
+for t in ${t14[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./linear_road_benchmark_checkpoints \
+ --unbounded-size 16777216 --circular-size 16777216 \
+ --batch-size 262144 --bundle-size 262144 --query 3 \
+ --create-merge true --parallel-merge true --latency true --threads $t >> ls_bench_res.txt
+ done
+done
+
+echo "YSB" >> ls_bench_res.txt
+for t in ${t15[@]}; #10
+do
+ for it in ${iterations[@]};
+ do
+ ./yahoo_benchmark_checkpoints --circular-size 16777216 --slots 128 --batch-size 1048576 \
+ --bundle-size 1048576 --latency true --threads $t >> ls_bench_res.txt
+ done
+done
+
+echo "NBQ5" >> ls_bench_res.txt
+for t in ${t15[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./nexmark_checkpoints --circular-size 33554432 --batch-size 1048576 --bundle-size 1048576 \
+ --unbounded-size 262144 --latency true --parallel-merge true --threads $t >> ls_bench_res.txt
+ done
+done
+
+echo "All done..."
\ No newline at end of file
diff --git a/scripts/prepare-software.sh b/scripts/prepare-software.sh
index 9b6f664..5cebb74 100755
--- a/scripts/prepare-software.sh
+++ b/scripts/prepare-software.sh
@@ -30,9 +30,20 @@ sudo apt-get install libboost-all-dev
# TBB
sudo apt-get install libtbb-dev
+# KAIO
+sudo apt-get install libaio-dev
+
+# IBVerbs
+sudo apt install libibverbs-dev
+#sudo apt-get install libmlx4–1 infiniband-diags ibutils ibverbs-utils rdmacm-utils
+
# Used for disk profiling
sudo apt install bpfcc-tools
# sudo biolatency-bpfcc -D 30 1
+sudo apt install sysstat
+# iostat -dx 30 2 (iostat -dx 1) // mpstat 30 1 (cpu) // sar -n DEV 30 1 (network)
+sudo apt install fio
+# sudo fio --name=writefile --size=1G --filesize=1G --filename=/home/grt17/dat --bs=1M --nrfiles=1 --direct=1 --sync=0 --randrepeat=0 --rw=write --refill_buffers --end_fsync=1 --iodepth=200 --ioengine=libaio
# CMake 3.16
sudo apt remove --purge --auto-remove cmake
@@ -77,7 +88,7 @@ cmake -DLLVM_ENABLE_PROJECTS=clang -DCMAKE_BUILD_TYPE=Release \
-DLLVM_TARGETS_TO_BUILD=X86 -G "Unix Makefiles" ../llvm
make -j$(nproc)
sudo make install
-echo 'export LLVM_HOME=$(pwd)' >> $HOME/.profile
+echo "export LLVM_HOME=$(pwd)" >> $HOME/.profile
echo 'export PATH=$LLVM_HOME/bin:$PATH' >> $HOME/.profile
echo 'export LIBRARY_PATH=$LLVM_HOME/lib:$LIBRARY_PATH' >> $HOME/.profile
source $HOME/.profile
@@ -92,8 +103,11 @@ sudo apt-get install libgtest-dev
cd /usr/src/gtest
sudo cmake CMakeLists.txt
sudo make -j$(nproc)
-# which one is the correct?
-sudo cp ./lib/*.a /usr/lib
+# cd /usr/src/googletest
+# sudo cmake CMakeLists.txt
+# sudo make -j$(nproc)
+# sudo cp googlemock/gtest/*a /usr/lib
+# sudo cp ./lib/*.a /usr/lib
sudo cp *.a /usr/lib/
sudo mkdir /usr/local/lib/gtest
sudo ln -s /usr/lib/libgtest.a /usr/local/lib/gtest/libgtest.a
@@ -110,6 +124,48 @@ make -j$(nproc)
sudo make install
cd
+# PMDK
+sudo apt install -y git gcc g++ autoconf automake asciidoc asciidoctor bash-completion xmlto libtool pkg-config libglib2.0-0 libglib2.0-dev libfabric1 libfabric-dev doxygen graphviz pandoc libncurses5 libkmod2 libkmod-dev libudev-dev uuid-dev libjson-c-dev libkeyutils-dev
+git clone https://github.com/pmem/ndctl
+cd ndctl
+git checkout c7767834871f7ce50a2abe1da946e9e16fb08eda
+sudo ./autogen.sh
+sudo ./configure CFLAGS='-g -O2' --prefix=/usr/local --sysconfdir=/etc --libdir=/usr/local/lib64
+#sudo ./configure CFLAGS='-g -O2' --prefix=/usr --sysconfdir=/etc --libdir=/usr/lib
+sudo make -j$(nproc)
+sudo make install
+cd
+
+sudo apt install autoconf automake pkg-config libglib2.0-dev libfabric-dev pandoc libncurses5-dev
+git clone https://github.com/pmem/pmdk
+cd pmdk
+git checkout 3bc5b0da5a7a5d5752ad2cb4f5f9bf0edfd47d67
+export PKG_CONFIG_PATH=/usr/local/lib64/pkgconfig:/usr/local/lib/pkgconfig:${PKG_CONFIG_PATH}
+make -j$(nproc)
+sudo PKG_CONFIG_PATH=/usr/local/lib64/pkgconfig:/usr/local/lib/pkgconfig:${PKG_CONFIG_PATH} make install
+sudo sh -c "echo /usr/local/lib >> /etc/ld.so.conf"
+sudo sh -c "echo /usr/local/lib64 >> /etc/ld.so.conf"
+sudo ldconfig
+cd
+# PKG_CONFIG_PATH
+echo 'export PKG_CONFIG_PATH=/usr/local/lib64/pkgconfig:/usr/local/lib/pkgconfig:${PKG_CONFIG_PATH}' >> $HOME/.profile
+source $HOME/.profile
+
+
+git clone https://github.com/pmem/libpmemobj-cpp.git
+cd libpmemobj-cpp
+git checkout 9f784bba07b94cd36c9eebeaa88c5df4f05045b2
+mkdir build
+cd build
+cmake -DTESTS_USE_VALGRIND=OFF ..
+make -j$(nproc)
+sudo make install
+cd
+
+## Set up home directory
+# echo "Setting up the home directory in src/utils/SystemConf.cpp"
+# sed -i '65s#.*#"'$HOME'"#' $HOME/LightSaber/src/CMakeLists.txt
+
# Build LightSaber
#cd $HOME/LightSaber
#mkdir build
@@ -121,4 +177,4 @@ cd
#cd $HOME/LightSaber/build/test/benchmarks/microbenchmarks/
#./TestProjection
-echo "All done..."
\ No newline at end of file
+echo "All done..."
diff --git a/scripts/scabbard-bench/other/run-checkpoint-only-benchmarks-different-confs.sh b/scripts/scabbard-bench/other/run-checkpoint-only-benchmarks-different-confs.sh
new file mode 100755
index 0000000..bcbdf01
--- /dev/null
+++ b/scripts/scabbard-bench/other/run-checkpoint-only-benchmarks-different-confs.sh
@@ -0,0 +1,215 @@
+#!/bin/bash
+
+echo "Start running benchmarks with checkpoints"
+
+allThreads=(1 2 3 4 5 6 7 8 9 10 11 12 13 14 15)
+iterations=(1 2 3 4 5)
+
+path="$HOME/LightSaber/build/test/benchmarks/applicationsWithCheckpoints/"
+
+if [ -n "$path" ]; then
+ echo "The build path is set to $path"
+else
+ echo "Set the application build path. Exiting..."
+ exit
+fi
+
+cd $path # If the application doesn't run from the build folder, it breaks. This happens because of the where the code files are generated.
+echo $PWD
+
+echo "CM1" >> bench_res.txt
+for t in ${allThreads[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./cluster_monitoring_checkpoints \
+ --circular-size 33554432 --unbounded-size 524288 --batch-size 524288 --bundle-size 524288 \
+ --query 1 --disk-block-size 65536 --latency true --threads $t >> bench_res.txt
+ done
+done
+
+echo "CM1-CH" >> bench_res.txt
+for t in ${allThreads[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./cluster_monitoring_checkpoints \
+ --circular-size 33554432 --unbounded-size 524288 --batch-size 524288 --bundle-size 524288 \
+ --query 1 --checkpoint-duration 1000 --disk-block-size 65536 --latency true --threads $t >> bench_res.txt
+ done
+done
+
+echo "CM2" >> bench_res.txt
+for t in ${allThreads[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./cluster_monitoring_checkpoints \
+ --circular-size 16777216 --unbounded-size 524288 --batch-size 524288 --bundle-size 524288 \
+ --query 2 --disk-block-size 131072 --latency true --threads $t >> bench_res.txt
+ done
+done
+
+echo "CM2-CH" >> bench_res.txt
+for t in ${allThreads[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./cluster_monitoring_checkpoints \
+ --circular-size 16777216 --unbounded-size 1048576 --batch-size 524288 --bundle-size 524288 \
+ --query 2 --checkpoint-duration 1000 --disk-block-size 131072 --latency true --threads $t >> bench_res.txt
+ done
+done
+
+echo "SG2" >> bench_res.txt
+for t in ${allThreads[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./smartgrid_checkpoints \
+ --query 2 --hashtable-size 512 --unbounded-size 1048576 --circular-size 16777216 \
+ --bundle-size 524288 --slots 128 --batch-size 524288 --unbounded-size 4194304 \
+ --disk-block-size 4194304 --create-merge true \
+ --parallel-merge true --latency true --threads $t >> bench_res.txt
+ done
+done
+
+
+echo "SG2-CH" >> bench_res.txt
+for t in ${allThreads[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./smartgrid_checkpoints \
+ --query 2 --hashtable-size 512 --unbounded-size 1048576 --circular-size 16777216 \
+ --bundle-size 524288 --slots 128 --batch-size 524288 --unbounded-size 4194304 \
+ --checkpoint-duration 1000 --disk-block-size 4194304 --create-merge true \
+ --parallel-merge true --latency true --threads $t >> bench_res.txt
+ done
+done
+
+echo "SG2-CH-CMP" >> bench_res.txt
+for t in ${allThreads[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./smartgrid_checkpoints \
+ --query 2 --hashtable-size 512 --unbounded-size 1048576 --circular-size 16777216 \
+ --bundle-size 524288 --slots 128 --batch-size 524288 --unbounded-size 4194304 \
+ --checkpoint-duration 1000 --disk-block-size 4194304 --create-merge true \
+ --parallel-merge true --latency true --checkpoint-compression true --threads $t >> bench_res.txt
+ done
+done
+
+echo "LRB1" >> bench_res.txt
+for t in ${allThreads[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./linear_road_benchmark_checkpoints \
+ --unbounded-size 8388608 --circular-size 16777216 \
+ --batch-size 524288 --bundle-size 524288 --query 1 --hashtable-size 256 \
+ --disk-block-size 16777216 \
+ --create-merge true --parallel-merge true --latency true --threads $t >> bench_res.txt
+ done
+done
+
+echo "LRB1-CH" >> bench_res.txt
+for t in ${allThreads[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./linear_road_benchmark_checkpoints \
+ --unbounded-size 8388608 --circular-size 16777216 \
+ --batch-size 524288 --bundle-size 524288 --query 1 --hashtable-size 256 \
+ --checkpoint-duration 1000 --disk-block-size 16777216 \
+ --create-merge true --parallel-merge true --latency true --threads $t >> bench_res.txt
+ done
+done
+
+echo "LRB1-CH-CMP" >> bench_res.txt
+for t in ${allThreads[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./linear_road_benchmark_checkpoints \
+ --unbounded-size 8388608 --circular-size 16777216 \
+ --batch-size 524288 --bundle-size 524288 --query 1 --hashtable-size 256 \
+ --checkpoint-duration 1000 --disk-block-size 16777216 \
+ --create-merge true --parallel-merge true --latency true --checkpoint-compression true --threads $t >> bench_res.txt
+ done
+done
+
+echo "LRB2" >> bench_res.txt
+for t in ${allThreads[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./linear_road_benchmark_checkpoints \
+ --unbounded-size 16777216 --circular-size 16777216 \
+ --batch-size 262144 --bundle-size 262144 --query 2 \
+ --disk-block-size 8388608 \
+ --create-merge true --parallel-merge true --latency true --threads $t >> bench_res.txt
+ done
+done
+
+echo "LRB2-CH" >> bench_res.txt
+for t in ${allThreads[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./linear_road_benchmark_checkpoints \
+ --unbounded-size 16777216 --circular-size 16777216 \
+ --batch-size 262144 --bundle-size 262144 --query 2 \
+ --checkpoint-duration 1000 --disk-block-size 8388608 \
+ --create-merge true --parallel-merge true --latency true --threads $t >> bench_res.txt
+ done
+done
+
+echo "LRB2-CH-CMP" >> bench_res.txt
+for t in ${allThreads[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./linear_road_benchmark_checkpoints \
+ --unbounded-size 16777216 --circular-size 16777216 \
+ --batch-size 262144 --bundle-size 262144 --query 2 \
+ --checkpoint-duration 1000 --disk-block-size 8388608 \
+ --create-merge true --parallel-merge true --latency true --checkpoint-compression true --threads $t >> bench_res.txt
+ done
+done
+
+echo "YSB" >> bench_res.txt
+for t in ${allThreads[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./yahoo_benchmark_checkpoints \
+ --circular-size 33554432 --slots 128 --batch-size 524288 --bundle-size 524288 \
+ --disk-block-size 32768 --latency true --threads $t >> bench_res.txt
+ done
+done
+
+echo "YSB-CH" >> bench_res.txt
+for t in ${allThreads[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./yahoo_benchmark_checkpoints \
+ --circular-size 33554432 --slots 128 --batch-size 524288 --bundle-size 524288 \
+ --latency true --checkpoint-duration 1000 --threads $t >> bench_res.txt
+ done
+done
+
+echo "NBQ5-CH" >> bench_res.txt
+for t in ${allThreads[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./nexmark_checkpoints --circular-size 33554432 --batch-size 1048576 --bundle-size 1048576 \
+ --unbounded-size 262144 --disk-block-size 1048576 --checkpoint-compression true --persist-input tru \
+ --lineage true --latency true --parallel-merge true --checkpoint-duration 1000 --threads $t >> bench_res.txt
+ done
+done
+
+echo "All done..."
\ No newline at end of file
diff --git a/scripts/scabbard-bench/other/run-scalability-benchmarks.sh b/scripts/scabbard-bench/other/run-scalability-benchmarks.sh
new file mode 100755
index 0000000..53bee29
--- /dev/null
+++ b/scripts/scabbard-bench/other/run-scalability-benchmarks.sh
@@ -0,0 +1,43 @@
+#!/bin/bash
+
+echo "Start running scalability benchmarks"
+
+allThreads=(1 2 3 4 5 6 7 8 9 10 11 12 13 14 15)
+iterations=(1 2 3 4 5)
+
+path="$HOME/LightSaber/build/test/benchmarks/applicationsWithCheckpoints/"
+
+if [ -n "$path" ]; then
+ echo "The build path is set to $path"
+else
+ echo "Set the application build path. Exiting..."
+ exit
+fi
+
+cd $path # If the application doesn't run from the build folder, it breaks. This happens because of the where the code files are generated.
+echo $PWD
+
+
+echo "YSB-chk" >> scal_bench_res.txt
+for t in ${allThreads[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./yahoo_benchmark_checkpoints --circular-size 33554432 --slots 128 --batch-size 1048576 \
+ --bundle-size 1048576 --disk-block-size 32768 --latency true --checkpoint-compression tru \
+ --persist-input true --lineage true --threads $t >> scal_bench_res.txt
+ done
+done
+
+echo "YSB-ALL-chk" >> scal_bench_res.txt
+for t in ${allThreads[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./yahoo_benchmark_checkpoints --circular-size 33554432 --slots 128 --batch-size 1048576 \
+ --bundle-size 1048576 --disk-block-size 32768 --latency true --checkpoint-compression true \
+ --persist-input true --lineage true --threads $t >> scal_bench_res.txt
+ done
+done
+
+echo "All done..."
\ No newline at end of file
diff --git a/scripts/scabbard-bench/paper/run-adaptive-benchmarks-FIG12.sh b/scripts/scabbard-bench/paper/run-adaptive-benchmarks-FIG12.sh
new file mode 100755
index 0000000..ba0d148
--- /dev/null
+++ b/scripts/scabbard-bench/paper/run-adaptive-benchmarks-FIG12.sh
@@ -0,0 +1,25 @@
+#!/bin/bash
+
+echo "Start running adaptive benchmarks"
+
+path="$HOME/LightSaber/build/test/benchmarks/applicationsWithCheckpoints/"
+
+if [ -n "$path" ]; then
+ echo "The build path is set to $path"
+else
+ echo "Set the application build path. Exiting..."
+ exit
+fi
+
+cd $path # If the application doesn't run from the build folder, it breaks. This happens because of the where the code files are generated.
+echo $PWD
+
+# run with adaptive compression
+echo "ME1 adaptive" >> adapt_res.txt
+./manufacturing_equipment_checkpoints --unbounded-size 32768 --circular-size 16777216 --batch-size 262144 --bundle-size 262144 --disk-block-size 32768 --latency tru --checkpoint-compression true --persist-input true --lineage true --threads 10 --adaptive-compression true --adaptive-data true >> adapt_res.txt
+
+# run without adaptive compression
+echo "ME1" >> adapt_res.txt
+./manufacturing_equipment_checkpoints --unbounded-size 32768 --circular-size 16777216 --batch-size 262144 --bundle-size 262144 --disk-block-size 32768 --latency tru --checkpoint-compression true --persist-input true --lineage true --threads 10 --adaptive-data true >> adapt_res.txt
+
+echo "All done..."
\ No newline at end of file
diff --git a/scripts/scabbard-bench/paper/run-aws-gp3-benchmarks-FIG14.sh b/scripts/scabbard-bench/paper/run-aws-gp3-benchmarks-FIG14.sh
new file mode 100755
index 0000000..b69778a
--- /dev/null
+++ b/scripts/scabbard-bench/paper/run-aws-gp3-benchmarks-FIG14.sh
@@ -0,0 +1,156 @@
+#!/bin/bash
+
+echo "Start running benchmarks"
+
+# These experiments run with gp3 disks (700 MB/s and 16000 IOPS).
+allThreads=(1 2 3 4 5 6 7 8 9 10 11 12 13 14 15)
+iterations=(1 2 3 4 5)
+t7=(7)
+t8=(8)
+t10=(10)
+t14=(14)
+t15=(15)
+
+path="$HOME/LightSaber/build/test/benchmarks/applicationsWithCheckpoints/"
+#path="/home/ubuntu/tmp/cmake-build-debug-aws/test/benchmarks/applicationsWithCheckpoints/"
+
+if [ -n "$path" ]; then
+ echo "The build path is set to $path"
+else
+ echo "Set the application build path. Exiting..."
+ exit
+fi
+
+cd $path # If the application doesn't run from the build folder, it breaks. This happens because of the where the code files are generated.
+echo $PWD
+
+echo "CM1-ALL" >> scb_bench_res.txt
+for t in ${t15[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./cluster_monitoring_checkpoints \
+ --circular-size 67108864 --unbounded-size 4194304 --batch-size 2097152 --bundle-size 2097152 \
+ --query 1 --checkpoint-duration 1000 --disk-block-size 262144 --checkpoint-compression true \
+ --persist-input true --lineage true --latency true --threads $t >> scb_bench_res.txt
+ done
+done
+
+echo "CM2-ALL" >> scb_bench_res.txt
+for t in ${t8[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./cluster_monitoring_checkpoints \
+ --circular-size 67108864 --unbounded-size 4194304 --batch-size 2097152 --bundle-size 2097152 \
+ --query 2 --checkpoint-duration 1000 --disk-block-size 131072 --latency true --checkpoint-compression true \
+ --checkpoint-duration 1000 --persist-input true --lineage true --threads $t >> scb_bench_res.txt
+ done
+done
+
+echo "SG1-Simple" >> scb_bench_res.txt
+for t in ${t7[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./smartgrid_checkpoints \
+ --query 1 --unbounded-size 262144 --batch-size 1048576 --circular-size 33554432 \
+ --bundle-size 1048576 --slots 128 --latency true --threads $t >> scb_bench_res.txt
+ done
+done
+
+echo "SG1-ALL" >> scb_bench_res.txt
+for t in ${t7[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./smartgrid_checkpoints \
+ --query 1 --unbounded-size 262144 --batch-size 1048576 --circular-size 33554432 \
+ --bundle-size 1048576 --slots 128 --latency true --checkpoint-compression true \
+ --disk-block-size 131072 --persist-input true --lineage true \
+ --checkpoint-duration 1000 --threads $t >> scb_bench_res.txt
+ done
+done
+
+echo "SG2-ALL" >> scb_bench_res.txt
+for t in ${t15[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./smartgrid_checkpoints \
+ --query 2 --hashtable-size 512 --unbounded-size 1048576 --circular-size 16777216 \
+ --bundle-size 524288 --slots 128 --batch-size 524288 --unbounded-size 4194304 \
+ --checkpoint-duration 1000 --disk-block-size 4194304 --create-merge true --persist-input true \
+ --checkpoint-compression true --lineage true --parallel-merge true --latency true --threads $t >> scb_bench_res.txt
+ done
+done
+
+# increase the circular-size or decouple memory buffering from storage
+echo "SG3-ALL" >> scb_bench_res.txt
+for t in ${t15[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./smartgrid_checkpoints \
+ --query 3 --hashtable-size 512 --unbounded-size 1048576 --circular-size 16777216 \
+ --bundle-size 524288 --slots 128 --batch-size 524288 --unbounded-size 4194304 \
+ --checkpoint-duration 1000 --disk-block-size 4194304 --create-merge true --persist-input true \
+ --checkpoint-compression true --lineage tru --parallel-merge true --latency true --threads $t >> scb_bench_res.txt
+ done
+done
+
+echo "LRB1-ALL-CMP" >> scb_bench_res.txt
+for t in ${t15[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./linear_road_benchmark_checkpoints \
+ --unbounded-size 8388608 --circular-size 16777216 \
+ --batch-size 524288 --bundle-size 524288 --query 1 --hashtable-size 256 \
+ --checkpoint-duration 1000 --disk-block-size 16777216 --persist-input true \
+ --checkpoint-compression true --lineage true \
+ --create-merge true --parallel-merge true --latency true --threads $t >> scb_bench_res.txt
+ done
+done
+
+echo "LRB2-ALL-CMP" >> scb_bench_res.txt
+for t in ${t15[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./linear_road_benchmark_checkpoints \
+ --unbounded-size 16777216 --circular-size 16777216 \
+ --batch-size 262144 --bundle-size 262144 --query 2 \
+ --checkpoint-duration 1000 --disk-block-size 8388608 \
+ --persist-input true \
+ --checkpoint-compression true --lineage true \
+ --create-merge true --parallel-merge true --latency true --threads $t >> scb_bench_res.txt
+ done
+done
+
+echo "LRB3-ALL-CMP" >> scb_bench_res.txt
+for t in ${t14[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./linear_road_benchmark_checkpoints \
+ --unbounded-size 16777216 --circular-size 16777216 \
+ --batch-size 262144 --bundle-size 262144 --query 3 \
+ --disk-block-size 8388608 --persist-input true \
+ --checkpoint-compression true --lineage true \
+ --create-merge true --parallel-merge true --latency true --threads $t >> scb_bench_res.txt
+ done
+done
+
+echo "YSB-ALL" >> scb_bench_res.txt
+for t in ${t15[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./yahoo_benchmark_checkpoints --circular-size 67108864 --slots 128 --batch-size 2097152 \
+ --bundle-size 2097152 --disk-block-size 32768 --latency true --checkpoint-compression true \
+ --persist-input true --lineage true --checkpoint-duration 1000 --threads $t >> scb_bench_res.txt
+ done
+done
+
+echo "All done..."
\ No newline at end of file
diff --git a/scripts/scabbard-bench/paper/run-benchmarks-FIG7.sh b/scripts/scabbard-bench/paper/run-benchmarks-FIG7.sh
new file mode 100755
index 0000000..071cfb2
--- /dev/null
+++ b/scripts/scabbard-bench/paper/run-benchmarks-FIG7.sh
@@ -0,0 +1,199 @@
+#!/bin/bash
+
+echo "Start running benchmarks"
+
+allThreads=(1 2 3 4 5 6 7 8 9 10 11 12 13 14 15)
+iterations=(1 2 3 4 5)
+t12=(12)
+t14=(14)
+t15=(15)
+
+path="$HOME/LightSaber/build/test/benchmarks/applicationsWithCheckpoints/"
+
+if [ -n "$path" ]; then
+ echo "The build path is set to $path"
+else
+ echo "Set the application build path. Exiting..."
+ exit
+fi
+
+cd $path # If the application doesn't run from the build folder, it breaks. This happens because of the where the code files are generated.
+echo $PWD
+
+echo "CM1-ALL" >> scb_bench_res.txt
+for t in ${t15[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./cluster_monitoring_checkpoints \
+ --circular-size 33554432 --unbounded-size 524288 --batch-size 524288 --bundle-size 524288 \
+ --query 1 --checkpoint-duration 1000 --disk-block-size 65536 --checkpoint-compression true \
+ --persist-input true --lineage true --latency true --threads $t >> scb_bench_res.txt
+ done
+done
+
+echo "CM2-ALL" >> scb_bench_res.txt
+for t in ${t14[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./cluster_monitoring_checkpoints \
+ --circular-size 16777216 --unbounded-size 1048576 --batch-size 524288 --bundle-size 524288 \
+ --query 2 --checkpoint-duration 1000 --disk-block-size 131072 --latency true --checkpoint-compression true \
+ --checkpoint-duration 1000 --persist-input true --lineage true --threads $t >> scb_bench_res.txt
+ done
+done
+
+echo "SG1-ALL" >> scb_bench_res.txt
+for t in ${t12[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./smartgrid_checkpoints \
+ --query 1 --unbounded-size 262144 --batch-size 1048576 --circular-size 33554432 \
+ --bundle-size 1048576 --slots 128 --latency true --checkpoint-compression true \
+ --disk-block-size 131072 --persist-input true --lineage true \
+ --checkpoint-duration 1000 --threads $t >> scb_bench_res.txt
+# --query 1 --unbounded-size 262144 --batch-size 524288 --circular-size 16777216 \
+# --bundle-size 524288 --slots 128 --latency true --checkpoint-compression true \
+# --disk-block-size 131072 --persist-input true --lineage true \
+ done
+done
+
+echo "SG2-ALL" >> scb_bench_res.txt
+for t in ${t15[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./smartgrid_checkpoints \
+ --query 2 --hashtable-size 512 --unbounded-size 1048576 --circular-size 16777216 \
+ --bundle-size 524288 --slots 128 --batch-size 524288 --unbounded-size 4194304 \
+ --checkpoint-duration 1000 --disk-block-size 4194304 --create-merge true --persist-input true \
+ --checkpoint-compression true --lineage true --parallel-merge true --latency true --threads $t >> scb_bench_res.txt
+ done
+done
+
+# increase the circular-size or decouple memory buffering from storage
+echo "SG3-ALL" >> scb_bench_res.txt
+for t in ${t15[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./smartgrid_checkpoints \
+ --query 3 --hashtable-size 512 --unbounded-size 1048576 --circular-size 16777216 \
+ --bundle-size 524288 --slots 128 --batch-size 524288 --unbounded-size 4194304 \
+ --checkpoint-duration 1000 --disk-block-size 4194304 --create-merge true --persist-input true \
+ --checkpoint-compression true --lineage tru --parallel-merge true --latency true --threads $t >> scb_bench_res.txt
+ done
+done
+
+#echo "LRB1-ALL" >> scb_bench_res.txt
+#for t in ${t15[@]};
+#do
+# for it in ${iterations[@]};
+# do
+# ./linear_road_benchmark_checkpoints \
+# --unbounded-size 8388608 --circular-size 16777216 \
+# --batch-size 524288 --bundle-size 524288 --query 1 --hashtable-size 256 \
+# --checkpoint-duration 1000 --disk-block-size 16777216 --persist-input true \
+# --checkpoint-compression tru --lineage true \
+# --create-merge true --parallel-merge true --latency true --threads $t >> scb_bench_res.txt
+# done
+#done
+
+echo "LRB1-ALL-CMP" >> scb_bench_res.txt
+for t in ${t15[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./linear_road_benchmark_checkpoints \
+ --unbounded-size 8388608 --circular-size 16777216 \
+ --batch-size 524288 --bundle-size 524288 --query 1 --hashtable-size 256 \
+ --checkpoint-duration 1000 --disk-block-size 16777216 --persist-input true \
+ --checkpoint-compression true --lineage true \
+ --create-merge true --parallel-merge true --latency true --threads $t >> scb_bench_res.txt
+ done
+done
+
+#echo "LRB2-ALL" >> scb_bench_res.txt
+#for t in ${t15[@]};
+#do
+# for it in ${iterations[@]};
+# do
+# ./linear_road_benchmark_checkpoints \
+# --unbounded-size 16777216 --circular-size 16777216 \
+# --batch-size 262144 --bundle-size 262144 --query 2 \
+# --checkpoint-duration 1000 --disk-block-size 8388608 \
+# --persist-input true \
+# --checkpoint-compression tru --lineage true \
+# --create-merge true --parallel-merge true --latency true --threads $t >> scb_bench_res.txt
+# done
+#done
+
+echo "LRB2-ALL-CMP" >> scb_bench_res.txt
+for t in ${t15[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./linear_road_benchmark_checkpoints \
+ --unbounded-size 16777216 --circular-size 16777216 \
+ --batch-size 262144 --bundle-size 262144 --query 2 \
+ --checkpoint-duration 1000 --disk-block-size 8388608 \
+ --persist-input true \
+ --checkpoint-compression true --lineage true \
+ --create-merge true --parallel-merge true --latency true --threads $t >> scb_bench_res.txt
+ done
+done
+
+#echo "LRB3-ALL" >> scb_bench_res.txt
+#for t in ${t14[@]};
+#do
+# for it in ${iterations[@]};
+# do
+# ./linear_road_benchmark_checkpoints \
+# --unbounded-size 16777216 --circular-size 16777216 \
+# --batch-size 262144 --bundle-size 262144 --query 3 \
+# --disk-block-size 8388608 \
+# --persist-input true \
+# --checkpoint-compression tru --lineage true \
+# --create-merge true --parallel-merge true --latency true --threads $t >> scb_bench_res.txt
+# done
+#done
+
+echo "LRB3-ALL-CMP" >> scb_bench_res.txt
+for t in ${t14[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./linear_road_benchmark_checkpoints \
+ --unbounded-size 16777216 --circular-size 16777216 \
+ --batch-size 262144 --bundle-size 262144 --query 3 \
+ --disk-block-size 8388608 --persist-input true \
+ --checkpoint-compression true --lineage true \
+ --create-merge true --parallel-merge true --latency true --threads $t >> scb_bench_res.txt
+ done
+done
+
+echo "YSB-ALL" >> scb_bench_res.txt
+for t in ${t15[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./yahoo_benchmark_checkpoints --circular-size 33554432 --slots 128 --batch-size 1048576 \
+ --bundle-size 1048576 --disk-block-size 32768 --latency true --checkpoint-compression true \
+ --persist-input true --lineage true --checkpoint-duration 1000 --threads $t >> scb_bench_res.txt
+ done
+done
+
+echo "NBQ5-ALL" >> scb_bench_res.txt
+for t in ${t15[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./nexmark_checkpoints --circular-size 33554432 --batch-size 1048576 --bundle-size 1048576 \
+ --unbounded-size 262144 --disk-block-size 1048576 --checkpoint-compression true --persist-input true \
+ --lineage true --latency true --parallel-merge true --checkpoint-duration 1000 --threads $t >> scb_bench_res.txt
+ done
+done
+
+echo "All done..."
\ No newline at end of file
diff --git a/scripts/scabbard-bench/paper/run-breakdown-benchmarks-FIG11.sh b/scripts/scabbard-bench/paper/run-breakdown-benchmarks-FIG11.sh
new file mode 100755
index 0000000..23d62fe
--- /dev/null
+++ b/scripts/scabbard-bench/paper/run-breakdown-benchmarks-FIG11.sh
@@ -0,0 +1,109 @@
+#!/bin/bash
+
+echo "Start running benchmarks"
+
+allThreads=(1 2 3 4 5 6 7 8 9 10 11 12 13 14 15)
+iterations=(1 2 3 4 5)
+t12=(12)
+t14=(14)
+t15=(15)
+
+path="$HOME/LightSaber/build/test/benchmarks/applicationsWithCheckpoints/"
+
+if [ -n "$path" ]; then
+ echo "The build path is set to $path"
+else
+ echo "Set the application build path. Exiting..."
+ exit
+fi
+
+cd $path # If the application doesn't run from the build folder, it breaks. This happens because of the where the code files are generated.
+echo $PWD
+
+
+
+#1) no-opt
+echo "CM1-NO-OPT" >> breakdown_bench_res.txt
+for t in ${t15[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./cluster_monitoring_checkpoints \
+ --circular-size 33554432 --unbounded-size 524288 --batch-size 524288 --bundle-size 524288 \
+ --query 1 --checkpoint-duration 1000 --disk-block-size 65536 --checkpoint-compression tru \
+ --persist-input true --lineage true --latency true --threads $t >> breakdown_bench_res.txt
+ done
+done
+#2) only delayed persistence
+# use noCompressInput function and --checkpoint-compression true
+#3) only compression (lossless)
+# use onlyCompressInputLossless function and --checkpoint-compression true
+#4) both => lossless floats
+# use compressGenInput function and --checkpoint-compression true
+#5) => lossy floats
+# use compressInput function and --checkpoint-compression true
+#6) no-disk => --checkpoint-compression true or false
+# use -DNO_DISK flag when compiling
+echo "CM1-OPTS" >> breakdown_bench_res.txt
+for t in ${t15[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./cluster_monitoring_checkpoints \
+ --circular-size 33554432 --unbounded-size 524288 --batch-size 524288 --bundle-size 524288 \
+ --query 1 --checkpoint-duration 1000 --disk-block-size 65536 --checkpoint-compression true \
+ --persist-input true --lineage true --latency true --threads $t >> breakdown_bench_res.txt
+ done
+done
+
+echo "CM2-ALL" >> breakdown_bench_res.txt
+for t in ${t14[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./cluster_monitoring_checkpoints \
+ --circular-size 16777216 --unbounded-size 1048576 --batch-size 524288 --bundle-size 524288 \
+ --query 2 --checkpoint-duration 1000 --disk-block-size 131072 --latency true --checkpoint-compression true \
+ --checkpoint-duration 1000 --persist-input true --lineage true --threads $t >> breakdown_bench_res.txt
+ done
+done
+
+echo "SG1-ALL" >> breakdown_bench_res.txt
+for t in ${t12[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./smartgrid_checkpoints \
+ --query 1 --unbounded-size 262144 --batch-size 1048576 --circular-size 33554432 \
+ --bundle-size 1048576 --slots 128 --latency true --checkpoint-compression true \
+ --disk-block-size 131072 --persist-input true --lineage true \
+ --checkpoint-duration 1000 --threads $t >> breakdown_bench_res.txt
+# --query 1 --unbounded-size 262144 --batch-size 524288 --circular-size 16777216 \
+# --bundle-size 524288 --slots 128 --latency true --checkpoint-compression true \
+# --disk-block-size 131072 --persist-input true --lineage true \
+ done
+done
+
+echo "YSB-ALL" >> breakdown_bench_res.txt
+for t in ${t15[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./yahoo_benchmark_checkpoints --circular-size 33554432 --slots 128 --batch-size 1048576 \
+ --bundle-size 1048576 --disk-block-size 32768 --latency true --checkpoint-compression true \
+ --persist-input true --lineage true --checkpoint-duration 1000 --threads $t >> breakdown_bench_res.txt
+ done
+done
+
+#echo "NBQ5-ALL" >> breakdown_bench_res.txt
+#for t in ${t15[@]};
+#do
+# for it in ${iterations[@]};
+# do
+# ./nexmark_checkpoints --circular-size 33554432 --batch-size 1048576 --bundle-size 1048576 \
+# --unbounded-size 262144 --disk-block-size 1048576 --checkpoint-compression true --persist-input true \
+# --lineage true --latency true --parallel-merge true --checkpoint-duration 1000 --threads $t >> breakdown_bench_res.txt
+# done
+#done
+
+echo "All done..."
\ No newline at end of file
diff --git a/scripts/scabbard-bench/paper/run-checkpoint-only-benchmarks-TABLE4.sh b/scripts/scabbard-bench/paper/run-checkpoint-only-benchmarks-TABLE4.sh
new file mode 100755
index 0000000..2368948
--- /dev/null
+++ b/scripts/scabbard-bench/paper/run-checkpoint-only-benchmarks-TABLE4.sh
@@ -0,0 +1,150 @@
+#!/bin/bash
+
+echo "Start running checkpointing benchmarks"
+
+allThreads=(1 2 3 4 5 6 7 8 9 10 11 12 13 14 15)
+iterations=(1 2 3 4 5)
+t12=(12)
+t14=(14)
+t15=(15)
+
+path="$HOME/LightSaber/build/test/benchmarks/applicationsWithCheckpoints/"
+
+if [ -n "$path" ]; then
+ echo "The build path is set to $path"
+else
+ echo "Set the application build path. Exiting..."
+ exit
+fi
+
+cd $path # If the application doesn't run from the build folder, it breaks. This happens because of the where the code files are generated.
+echo $PWD
+
+echo "CM1" >> bench_res.txt
+for t in ${t15[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./cluster_monitoring_checkpoints \
+ --circular-size 33554432 --unbounded-size 524288 --batch-size 524288 --bundle-size 524288 \
+ --query 1 --disk-block-size 65536 --latency true --checkpoint-duration 1000 --threads $t >> bench_res.txt
+ done
+done
+
+echo "CM2" >> bench_res.txt
+for t in ${t14[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./cluster_monitoring_checkpoints \
+ --circular-size 16777216 --unbounded-size 524288 --batch-size 524288 --bundle-size 524288 \
+ --query 2 --disk-block-size 131072 --latency true --checkpoint-duration 1000 --threads $t >> bench_res.txt
+ done
+done
+
+echo "SG1" >> bench_res.txt
+for t in ${t12[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./smartgrid_checkpoints \
+ --query 1 --unbounded-size 262144 --batch-size 1048576 --circular-size 33554432 \
+ --bundle-size 1048576 --slots 128 --latency true --checkpoint-compression true \
+ --disk-block-size 131072 --persist-input tru --lineage true \
+ --checkpoint-duration 1000 --threads $t >> bench_res.txt
+# --query 1 --unbounded-size 262144 --batch-size 524288 --circular-size 16777216 \
+# --bundle-size 524288 --slots 128 --latency true --checkpoint-compression true \
+# --disk-block-size 131072 --persist-input tru --lineage true \
+# --threads $t >> bench_res.txt
+ done
+done
+
+echo "SG2" >> bench_res.txt
+for t in ${t15[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./smartgrid_checkpoints \
+ --query 2 --hashtable-size 512 --unbounded-size 1048576 --circular-size 16777216 \
+ --bundle-size 524288 --slots 128 --batch-size 524288 --unbounded-size 4194304 \
+ --disk-block-size 4194304 --create-merge true --checkpoint-duration 1000 \
+ --parallel-merge true --latency true --threads $t >> bench_res.txt
+ done
+done
+
+echo "SG3" >> bench_res.txt
+for t in ${t15[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./smartgrid_checkpoints \
+ --query 3 --hashtable-size 512 --unbounded-size 1048576 --circular-size 16777216 \
+ --bundle-size 524288 --slots 128 --batch-size 524288 --unbounded-size 4194304 \
+ --disk-block-size 4194304 --create-merge true --checkpoint-duration 1000\
+ --parallel-merge true --latency true --threads $t >> bench_res.txt
+ done
+done
+
+echo "LRB1" >> bench_res.txt
+for t in ${t15[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./linear_road_benchmark_checkpoints \
+ --unbounded-size 8388608 --circular-size 16777216 \
+ --batch-size 524288 --bundle-size 524288 --query 1 --hashtable-size 256 \
+ --disk-block-size 16777216 --checkpoint-duration 1000 \
+ --create-merge true --parallel-merge true --latency true --threads $t >> bench_res.txt
+ done
+done
+
+echo "LRB2" >> bench_res.txt
+for t in ${t15[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./linear_road_benchmark_checkpoints \
+ --unbounded-size 16777216 --circular-size 16777216 \
+ --batch-size 262144 --bundle-size 262144 --query 2 \
+ --disk-block-size 8388608 --checkpoint-duration 1000 \
+ --create-merge true --parallel-merge true --latency true --threads $t >> bench_res.txt
+ done
+done
+
+echo "LRB3" >> bench_res.txt
+for t in ${t15[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./linear_road_benchmark_checkpoints \
+ --unbounded-size 16777216 --circular-size 16777216 \
+ --batch-size 262144 --bundle-size 262144 --query 3 \
+ --disk-block-size 8388608 --checkpoint-duration 1000 \
+ --create-merge true --parallel-merge true --latency true --threads $t >> bench_res.txt
+ done
+done
+
+
+echo "YSB" >> bench_res.txt
+for t in ${t15[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./yahoo_benchmark_checkpoints \
+ --circular-size 33554432 --slots 128 --batch-size 524288 --bundle-size 524288 \
+ --disk-block-size 32768 --latency true --threads --checkpoint-duration 1000 $t >> bench_res.txt
+ done
+done
+
+echo "NBQ5-ALL" >> bench_res.txt
+for t in ${t15[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./nexmark_checkpoints --circular-size 33554432 --batch-size 1048576 --bundle-size 1048576 \
+ --unbounded-size 262144 --disk-block-size 1048576 --checkpoint-compression true --persist-input tru \
+ --lineage true --latency true --parallel-merge true --checkpoint-duration 1000 --threads $t >> bench_res.txt
+ done
+done
+
+echo "All done..."
\ No newline at end of file
diff --git a/scripts/scabbard-bench/paper/run-flink-benchmarks-FIG9.sh b/scripts/scabbard-bench/paper/run-flink-benchmarks-FIG9.sh
new file mode 100755
index 0000000..7bfd1b8
--- /dev/null
+++ b/scripts/scabbard-bench/paper/run-flink-benchmarks-FIG9.sh
@@ -0,0 +1,46 @@
+#!/bin/bash
+
+echo "Start running scalability benchmarks"
+
+batchSize=(524288 1048576 2097152 4194304)
+iterations=(1 2 3 4 5)
+buffers=(1 2 4 8)
+
+path="$HOME/LightSaber/build/test/benchmarks/kafka-flink/"
+#path="/tmp/tmp.0lki8nQd4R/cmake-build-debug/test/benchmarks/kafka-flink/"
+
+if [ -n "$path" ]; then
+ echo "The build path is set to $path"
+else
+ echo "Set the application build path. Exiting..."
+ exit
+fi
+
+cd $path # If the application doesn't run from the build folder, it breaks. This happens because of the where the code files are generated.
+echo $PWD
+
+echo "YSB" >> flink_bench_res.txt
+for b in ${batchSize[@]};
+do
+ for it in ${iterations[@]};
+ do
+ for buf in ${buffers[@]};
+ do
+ ./yahoo_benchmark_flink --disk-block-size $b --threads 16 --latency true --disk-buffer $buf --use-checkpoints true >> flink_bench_res.txt
+ done
+ done
+done
+
+echo "YSB-Kafka" >> flink_bench_res.txt
+for b in ${batchSize[@]};
+do
+ for it in ${iterations[@]};
+ do
+ for buf in ${buffers[@]};
+ do
+ ./yahoo_benchmark_flink --disk-block-size $b --threads 16 --latency true --disk-buffer $buf --use-checkpoints true --use-kafka true >> flink_bench_res.txt
+ done
+ done
+done
+
+echo "All done..."
\ No newline at end of file
diff --git a/scripts/scabbard-bench/paper/run-ingestion-only-benchmarks-FIG8.sh b/scripts/scabbard-bench/paper/run-ingestion-only-benchmarks-FIG8.sh
new file mode 100755
index 0000000..3eb1c8d
--- /dev/null
+++ b/scripts/scabbard-bench/paper/run-ingestion-only-benchmarks-FIG8.sh
@@ -0,0 +1,156 @@
+#!/bin/bash
+
+echo "Start running ingestion benchmarks"
+
+allThreads=(7 8 9 10 14 15)
+empty=()
+iterations=(1 2 3 4 5)
+
+# Before running the following experiments, set m_doProcessing variable in OperatorKernel to false
+path="$HOME/LightSaber/build/test/benchmarks/applicationsWithCheckpoints/"
+
+if [ -n "$path" ]; then
+ echo "The build path is set to $path"
+else
+ echo "Set the application build path. Exiting..."
+ exit
+fi
+
+cd $path # If the application doesn't run from the build folder, it breaks. This happens because of the where the code files are generated.
+echo $PWD
+
+echo "CM1-ALL" >> ingestion_res.txt
+for t in ${allThreads[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./cluster_monitoring_checkpoints \
+ --circular-size 33554432 --unbounded-size 524288 --batch-size 524288 --bundle-size 524288 \
+ --query 1 --disk-block-size 65536 --checkpoint-compression true \
+ --persist-input true --lineage true --latency true --threads $t >> ingestion_res.txt
+ done
+done
+
+echo "CM2-ALL" >> ingestion_res.txt
+for t in ${allThreads[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./cluster_monitoring_checkpoints \
+ --circular-size 16777216 --unbounded-size 1048576 --batch-size 524288 --bundle-size 524288 \
+ --query 2 --disk-block-size 131072 --latency true --checkpoint-compression true \
+ --persist-input true --lineage true --threads $t >> ingestion_res.txt
+ done
+done
+
+echo "SG1-ALL" >> ingestion_res.txt
+for t in ${allThreads[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./smartgrid_checkpoints \
+ --query 1 --unbounded-size 262144 --batch-size 1048576 --circular-size 33554432 \
+ --bundle-size 1048576 --slots 128 --latency true --checkpoint-compression true \
+ --disk-block-size 131072 --persist-input true --lineage true \
+ --threads $t >> ingestion_res.txt
+# --query 1 --unbounded-size 262144 --batch-size 524288 --circular-size 16777216 \
+# --bundle-size 524288 --slots 128 --latency true --checkpoint-compression true \
+# --disk-block-size 131072 --persist-input true --lineage true \
+# --threads $t >> ingestion_res.txt
+ done
+done
+
+echo "SG2-ALL" >> ingestion_res.txt
+for t in ${allThreads[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./smartgrid_checkpoints \
+ --query 2 --hashtable-size 512 --unbounded-size 1048576 --circular-size 16777216 \
+ --bundle-size 524288 --slots 128 --batch-size 524288 --unbounded-size 4194304 \
+ --disk-block-size 4194304 --create-merge true --persist-input true \
+ --checkpoint-compression true --lineage true --parallel-merge tru --latency true --threads $t >> ingestion_res.txt
+ done
+done
+
+echo "LRB1-ALL" >> ingestion_res.txt
+for t in ${allThreads[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./linear_road_benchmark_checkpoints \
+ --unbounded-size 8388608 --circular-size 16777216 \
+ --batch-size 524288 --bundle-size 524288 --query 1 --hashtable-size 256 \
+ --disk-block-size 16777216 --persist-input true \
+ --checkpoint-compression tru --lineage true \
+ --create-merge true --parallel-merge tru --latency true --threads $t >> ingestion_res.txt
+ done
+done
+
+echo "LRB1-ALL-CMP" >> ingestion_res.txt
+for t in ${allThreads[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./linear_road_benchmark_checkpoints \
+ --unbounded-size 8388608 --circular-size 16777216 \
+ --batch-size 524288 --bundle-size 524288 --query 1 --hashtable-size 256 \
+ --disk-block-size 16777216 --persist-input true \
+ --checkpoint-compression true --lineage true \
+ --create-merge true --parallel-merge tru --latency true --threads $t >> ingestion_res.txt
+ done
+done
+
+echo "LRB2-ALL" >> ingestion_res.txt
+for t in ${allThreads[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./linear_road_benchmark_checkpoints \
+ --unbounded-size 16777216 --circular-size 16777216 \
+ --batch-size 262144 --bundle-size 262144 --query 2 \
+ --disk-block-size 8388608 \
+ --persist-input true \
+ --checkpoint-compression tru --lineage true \
+ --create-merge true --parallel-merge tru --latency true --threads $t >> ingestion_res.txt
+ done
+done
+
+echo "LRB2-ALL-CMP" >> ingestion_res.txt
+for t in ${allThreads[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./linear_road_benchmark_checkpoints \
+ --unbounded-size 16777216 --circular-size 16777216 \
+ --batch-size 262144 --bundle-size 262144 --query 2 \
+ --disk-block-size 8388608 \
+ --persist-input true \
+ --checkpoint-compression true --lineage true \
+ --create-merge true --parallel-merge tru --latency true --threads $t >> ingestion_res.txt
+ done
+done
+
+echo "YSB-ALL" >> ingestion_res.txt
+for t in ${allThreads[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./yahoo_benchmark_checkpoints --circular-size 33554432 --slots 128 --batch-size 1048576 \
+ --bundle-size 1048576 --disk-block-size 32768 --latency true --checkpoint-compression true \
+ --persist-input true --lineage true --threads $t >> ingestion_res.txt
+ done
+done
+
+echo "NBQ5-CH" >> ingestion_res.txt
+for t in ${allThreads[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./nexmark_checkpoints --circular-size 33554432 --batch-size 1048576 --bundle-size 1048576 \
+ --unbounded-size 262144 --disk-block-size 1048576 --checkpoint-compression true --persist-input true \
+ --lineage true --latency true --parallel-merge true --threads $t >> ingestion_res.txt
+ done
+done
+
+echo "All done..."
\ No newline at end of file
diff --git a/scripts/scabbard-bench/paper/run-kafka-benchmarks-FIG9.sh b/scripts/scabbard-bench/paper/run-kafka-benchmarks-FIG9.sh
new file mode 100755
index 0000000..b323d2f
--- /dev/null
+++ b/scripts/scabbard-bench/paper/run-kafka-benchmarks-FIG9.sh
@@ -0,0 +1,104 @@
+#!/bin/bash
+
+echo "Start running scalability benchmarks"
+
+batchSize=(524288 1048576 2097152 4194304)
+iterations=(1 2 3 4 5)
+
+path="$HOME/LightSaber/build/test/benchmarks/kafka-flink/"
+#path="/tmp/tmp.0lki8nQd4R/cmake-build-debug/test/benchmarks/kafka-flink/"
+
+if [ -n "$path" ]; then
+ echo "The build path is set to $path"
+else
+ echo "Set the application build path. Exiting..."
+ exit
+fi
+
+cd $path # If the application doesn't run from the build folder, it breaks. This happens because of the where the code files are generated.
+echo $PWD
+
+echo "CM1" >> kafka_bench_res.txt
+for b in ${batchSize[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./kafka_benchmarks --disk-block-size $b --threads 4 --query 0 --latency true --disk-buffer 16 --batch-size $b --bundle-size $b >> kafka_bench_res.txt
+ #./kafka_benchmarks --disk-block-size $b --threads 4 --query 0 --latency true --disk-buffer 2 --batch-size $b --bundle-size $b >> kafka_bench_res.txt
+ done
+done
+
+echo "CM2" >> kafka_bench_res.txt
+for b in ${batchSize[@]};
+do
+ bb=$((2*b))
+ for it in ${iterations[@]};
+ do
+ ./kafka_benchmarks --disk-block-size $b --threads 10 --query 1 --latency true --disk-buffer 32 --bundle-size $bb --batch-size $bb >> kafka_bench_res.txt
+ #./kafka_benchmarks --disk-block-size $b --threads 10 --query 1 --latency true --disk-buffer 4 --bundle-size $bb --batch-size $bb >> kafka_bench_res.txt
+ done
+done
+
+echo "SG1" >> kafka_bench_res.txt
+for b in ${batchSize[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./kafka_benchmarks --disk-block-size $b --threads 1 --query 2 --latency true --disk-buffer 32 >> kafka_bench_res.txt
+ #./kafka_benchmarks --disk-block-size $b --threads 1 --query 2 --latency true --disk-buffer 2 >> kafka_bench_res.txt
+ done
+done
+
+echo "SG2" >> kafka_bench_res.txt
+for b in ${batchSize[@]};
+do
+ bb=$((8*b))
+ for it in ${iterations[@]};
+ do
+ ./kafka_benchmarks --disk-block-size $b --threads 16 --query 3 --latency true --bundle-size $bb --batch-size $bb >> kafka_bench_res.txt
+ #./kafka_benchmarks --disk-block-size $b --threads 16 --query 3 --latency true --bundle-size $bb --batch-size $bb --disk-buffer 2 >> kafka_bench_res.txt
+ done
+done
+
+echo "LRB1" >> kafka_bench_res.txt
+for b in ${batchSize[@]};
+do
+ bb=$((2*b))
+ for it in ${iterations[@]};
+ do
+ ./kafka_benchmarks --disk-block-size $b --threads 16 --query 5 --latency true --disk-buffer 16 --batch-size $bb --bundle-size $bb >> kafka_bench_res.txt
+ #./kafka_benchmarks --disk-block-size $b --threads 16 --query 5 --latency true --disk-buffer 2 --batch-size $bb --bundle-size $bb >> kafka_bench_res.txt
+ done
+done
+
+echo "LRB2" >> kafka_bench_res.txt
+for b in ${batchSize[@]};
+do
+ bb=$((8*b))
+ for it in ${iterations[@]};
+ do
+ ./kafka_benchmarks --disk-block-size $b --threads 16 --query 6 --latency true --disk-buffer 8 --batch-size $bb --bundle-size $bb >> kafka_bench_res.txt
+ #./kafka_benchmarks --disk-block-size $b --threads 16 --query 6 --latency true --disk-buffer 2 --batch-size $bb --bundle-size $bb >> kafka_bench_res.txt
+ done
+done
+
+echo "YSB" >> kafka_bench_res.txt
+for b in ${batchSize[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./kafka_benchmarks --disk-block-size $b --threads 16 --query 8 --latency true >> kafka_bench_res.txt
+ #./kafka_benchmarks --disk-block-size $b --threads 16 --query 8 --latency true --disk-buffer 1 >> kafka_bench_res.txt
+ done
+done
+
+echo "NXB5" >> kafka_bench_res.txt
+for b in ${batchSize[@]};
+do
+ for it in ${iterations[@]};
+ do
+ ./kafka_benchmarks --disk-block-size $b --threads 16 --query 10 --latency true --disk-buffer 16 >> kafka_bench_res.txt
+ done
+done
+
+echo "All done..."
\ No newline at end of file
diff --git a/scripts/scabbard-bench/paper/run-rdma-benchmarks-FIG13.sh b/scripts/scabbard-bench/paper/run-rdma-benchmarks-FIG13.sh
new file mode 100755
index 0000000..7311ace
--- /dev/null
+++ b/scripts/scabbard-bench/paper/run-rdma-benchmarks-FIG13.sh
@@ -0,0 +1,84 @@
+#!/bin/bash
+
+echo "Start running benchmarks"
+
+allThreads=(1 2 3 4 5 6 7 8 9 10 11 12 13 14 15)
+iterations=(1) # 2 3 4 5)
+t12=(12)
+t14=(14)
+t15=(15)
+
+# sudo nano /sys/devices/system/cpu/smt/control (off)
+# cat /sys/devices/system/cpu/smt/control
+# install pcm https://software.opensuse.org/download/package?package=pcm&project=home%3Aopcm
+# 1) Login as root
+# 2) Execute the command 'modprobe msr'
+
+# qperf (server)
+# qperf -t 10 rc_bw
+# qperf -t 10 --rem_id mlx5_1 rc_bw
+# server: iperf -s
+# client: iperf -c 10.0.0.40
+
+# ibdev2netdev : check status
+# sudo ifconfig ib0 10.0.0.30/24 up
+# sudo ifconfig ib1 11.0.0.31/24 up
+
+# if run in wallaby, use the second socket => change the core mapping in Utils.cpp
+# unset the DHAVE_NUMA FLAG
+# use args.push_back("--gcc-toolchain=/usr/local/gcc/7.5.0"); in OperatorJit.cpp => https://stackoverflow.com/questions/40486053/selecting-a-specific-libstdc-version-with-clang
+
+#path="$HOME/LightSaber/cmake-build-debug/test/benchmarks/applicationsWithCheckpoints/"
+path="/tmp/tmp.Ogl5pzLSii/cmake-build-debug-wallaby/test/benchmarks/applicationsWithCheckpoints/"
+
+if [ -n "$path" ]; then
+ echo "The build path is set to $path"
+else
+ echo "Set the application build path. Exiting..."
+ exit
+fi
+
+cd $path # If the application doesn't run from the build folder, it breaks. This happens because of the where the code files are generated.
+echo $PWD
+
+echo "CM1" >> remote_scb_bench_res.txt
+for t in ${t15[@]};
+do
+ for it in ${iterations[@]};
+ do
+ echo "Starting Sink"
+ nohup ssh -t kea04 '/tmp/tmp.VuzIsR9Kxl/cmake-build-debug-kea04/test/benchmarks/applications/remoteRDMASink --batch-size 1048576 --bundle-size 1048576' &
+
+ echo "Starting app"
+ nohup bash -c "./cluster_monitoring_checkpoints \
+ --circular-size 33554432 --unbounded-size 524288 --batch-size 524288 --bundle-size 524288 \
+ --query 1 --checkpoint-duration 1000 --disk-block-size 65536 --checkpoint-compression true \
+ --persist-input true --lineage true --latency true --threads 15 >> remote_scb_bench_res.txt" &
+
+ echo "Starting Source"
+ nohup ssh -t kea03 '/tmp/tmp.42rD7Z5vpA/cmake-build-debug-kea03/test/benchmarks/applications/remoteRDMASource --batch-size 1048576 --bundle-size 1048576 --query 0' &
+ done
+done
+
+
+echo "YSB" >> remote_scb_bench_res.txt
+for t in ${t15[@]};
+do
+ for it in ${iterations[@]};
+ do
+ echo "Starting Sink"
+ nohup ssh -t kea04 '/tmp/tmp.VuzIsR9Kxl/cmake-build-debug-kea04/test/benchmarks/applications/remoteRDMASink --batch-size 1048576 --bundle-size 1048576' &
+
+ echo "Starting app"
+ nohup bash -c " ./yahoo_benchmark_checkpoints \
+ --circular-size 33554432 --slots 128 --batch-size 1048576 --bundle-size 1048576 \
+ --disk-block-size 32768 --latency true --checkpoint-compression true --persist-input true --lineage true \
+ --checkpoint-duration 1000 --threads 15 >> remote_scb_bench_res.txt" &
+
+ echo "Starting Source"
+ nohup ssh -t kea03 '/tmp/tmp.42rD7Z5vpA/cmake-build-debug-kea03/test/benchmarks/applications/remoteRDMASource --batch-size 1048576 --bundle-size 1048576 --query 8' &
+ done
+done
+
+
+echo "All done..."
\ No newline at end of file
diff --git a/scripts/scabbard-bench/paper/run-recovery-benchmarks-FIG10.sh b/scripts/scabbard-bench/paper/run-recovery-benchmarks-FIG10.sh
new file mode 100755
index 0000000..a479eb4
--- /dev/null
+++ b/scripts/scabbard-bench/paper/run-recovery-benchmarks-FIG10.sh
@@ -0,0 +1,34 @@
+#!/bin/bash
+
+echo "Start running recovery benchmarks"
+
+# Set SystemConf::PERFORMANCE_MONITOR_INTERVAL = 100 or 10
+# Adjust manually the throughput in test/benchmarks/applications/BenchmarkQuery.h by setting the sleep time of line 156
+# Set in LRB1.cpp, line 610 to false => if (true) {...}
+# Increase the sleep time if the recovering application can't access the pmem files locks
+path="$HOME/LightSaber/build/test/benchmarks/applicationsWithCheckpoints/"
+#path="/home/ubuntu/tmp/cmake-build-debug-aws/test/benchmarks/applicationsWithCheckpoints/"
+#path=/tmp/tmp.0lki8nQd4R/cmake-build-debug/test/benchmarks/applicationsWithCheckpoints
+
+if [ -n "$path" ]; then
+ echo "The build path is set to $path"
+else
+ echo "Set the application build path. Exiting..."
+ exit
+fi
+
+cd $path # If the application doesn't run from the build folder, it breaks. This happens because of the where the code files are generated.
+echo $PWD
+
+bash -c "exec -a LRBRecover ./linear_road_benchmark_checkpoints --unbounded-size 8388608 --circular-size 16777216 --batch-size 524288 --bundle-size 524288 --query 1 --hashtable-size 256 --checkpoint-duration 1000 --disk-block-size 16777216 --persist-input true --checkpoint-compression true --lineage true --create-merge true --parallel-merge true --latency true --threads 15 --ingestion 300 --performance-monitor-interval 100 >> recovery.txt" &
+
+sleep 7
+pkill -f LRBRecover
+sleep 0.07
+
+
+bash -c "exec -a LRBRecover ./linear_road_benchmark_checkpoints --unbounded-size 8388608 --circular-size 16777216 --batch-size 524288 --bundle-size 524288 --query 1 --hashtable-size 256 --checkpoint-duration 1000 --disk-block-size 16777216 --persist-input true --checkpoint-compression true --lineage true --create-merge true --parallel-merge true --latency true --recover true --threads 15 --ingestion 300 --performance-monitor-interval 100 >> recovery.txt" &
+sleep 15
+pkill -f LRBRecover
+
+echo "All done..."
\ No newline at end of file
diff --git a/scripts/utils/memcheck.sh b/scripts/utils/memcheck.sh
new file mode 100644
index 0000000..bbe06f1
--- /dev/null
+++ b/scripts/utils/memcheck.sh
@@ -0,0 +1,2 @@
+# pass a name as a parameter
+while true; do ps -eo rss,pid,euser,args:100 --sort %mem | grep -v grep | grep -i $@ | awk '{printf $1/1024 "MB"; $1=""; print }' ; sleep 0.5s ; done
diff --git a/scripts/utils/run_with_huge_pages.sh b/scripts/utils/run_with_huge_pages.sh
new file mode 100644
index 0000000..18ba8cd
--- /dev/null
+++ b/scripts/utils/run_with_huge_pages.sh
@@ -0,0 +1,32 @@
+#!/bin/bash
+# taken from https://github.com/lemire/Code-used-on-Daniel-Lemire-s-blog
+if ! [ $(id -u) = 0 ]; then
+ echo "The script need to be run as root." >&2
+ exit 1
+fi
+
+if [ $SUDO_USER ]; then
+ real_user=$SUDO_USER
+else
+ real_user=$(whoami)
+fi
+
+origval=$(sudo cat /sys/kernel/mm/transparent_hugepage/enabled)
+sudo -u $real_user echo $origval
+set -e
+function cleanup {
+ echo "Restoring hugepages to madvise"
+ echo "madvise" > /sys/kernel/mm/transparent_hugepage/enabled
+}
+trap cleanup EXIT
+
+for mode in "always" ; do # "never
+ sudo -u $real_user echo "mode: " $mode
+ echo $mode > /sys/kernel/mm/transparent_hugepage/enabled
+ echo $(sudo cat /sys/kernel/mm/transparent_hugepage/enabled)
+ $@
+done
+echo "Done."
+
+# echo 2048 | sudo tee /sys/kernel/mm/hugepages/hugepages-2048kB/nr_hugepages
+# cat /proc/meminfo | grep Huge
\ No newline at end of file
diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt
index 52cb646..2323e5a 100644
--- a/src/CMakeLists.txt
+++ b/src/CMakeLists.txt
@@ -46,7 +46,7 @@ target_compile_options(operatorJITLib PRIVATE -Wall -Wextra -O3 -march=native)
SET(HEADERS
utils/Utils.h
- buffers/NUMACircularQueryBuffer.h
+ buffers/NumaCircularQueryBuffer.h
buffers/CircularQueryBuffer.h
buffers/QueryBuffer.h
buffers/PartialWindowResults.h
@@ -75,6 +75,8 @@ SET(HEADERS
cql/predicates/ComparisonPredicate.h
cql/predicates/ANDPredicate.h
cql/predicates/ORPredicate.h
+ dispatcher/ITaskDispatcher.h
+ dispatcher/JoinTaskDispatcher.h
dispatcher/TaskDispatcher.h
monitors/LatencyMonitor.h
monitors/Measurement.h
@@ -96,7 +98,16 @@ SET(HEADERS
utils/WindowDefinition.h
)
SET(CPP_FILES
+ checkpoint/FileBackedCheckpointCoordinator.cpp
+ checkpoint/BlockManager.cpp
+ checkpoint/LineageGraph.cpp
+ cql/expressions/Expression.cpp
+ dispatcher/ITaskDispatcher.cpp
+ dispatcher/JoinTaskDispatcher.cpp
dispatcher/TaskDispatcher.cpp
+ compression/CompressionCodeGenUtils.cpp
+ compression/CompressionStatistics.cpp
+ monitors/CompressionMonitor.cpp
monitors/PerformanceMonitor.cpp
monitors/LatencyMonitor.cpp
monitors/Measurement.cpp
@@ -111,13 +122,25 @@ SET(CPP_FILES
utils/SystemConf.cpp
utils/Utils.cpp
)
+SET(RDMA_CPP_FILES
+ RDMA/infinity/core/Context.cpp
+ RDMA/infinity/memory/Atomic.cpp
+ RDMA/infinity/memory/Buffer.cpp
+ RDMA/infinity/memory/Region.cpp
+ RDMA/infinity/memory/RegionToken.cpp
+ RDMA/infinity/memory/RegisteredMemory.cpp
+ RDMA/infinity/queues/QueuePair.cpp
+ RDMA/infinity/queues/QueuePairFactory.cpp
+ RDMA/infinity/requests/RequestToken.cpp
+ RDMA/infinity/utils/Address.cpp
+ )
SET(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -pthread -lnuma")
-SET( CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -std=c++17 -g -Wall -Wextra -DHAVE_NUMA" )
+SET( CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -std=c++17 -g -Wall -Wextra -DHAVE_NUMA") #--gcc-toolchain=/usr/local/gcc/7.5.0
#-mcx16
-add_executable(LightSaber ${HEADERS} ${CPP_FILES} main.cpp)
+add_executable(LightSaber ${HEADERS} ${CPP_FILES} ${RDMA_CPP_FILES} main.cpp)
target_compile_options(LightSaber PRIVATE -O3 -march=native)
@@ -146,11 +169,25 @@ set_target_properties(operatorJITLib PROPERTIES
CXX_EXTENSIONS OFF
)
+#set(USER_PATH "/media/george/DATA")
+#set(USER_PATH "/home/grt17")
+set(USER_PATH $ENV{HOME})
+set(ENV{LLVM_HOME} "${USER_PATH}/llvm-project/build")
+message(STATUS "LLVM_HOME: $ENV{LLVM_HOME}")
+set(ENV{PATH} "$ENV{LLVM_HOME}/bin:$ENV{PATH}")
+message(STATUS "PATH: $ENV{PATH}")
+set(ENV{LIBRARY_PATH} "$ENV{LLVM_HOME}/lib:$ENV{LIBRARY_PATH}")
+message(STATUS "LIBRARY_PATH: $ENV{LIBRARY_PATH}")
+
# add LLVM libraries, set LLVM_* variables
+set(LLVM_DIR "$ENV{LLVM_HOME}/lib/cmake/llvm")
find_package(LLVM 9.0 REQUIRED PATHS ${LLVM_DIR})
message(STATUS "LLVM_DIR: ${LLVM_DIR}")
message(STATUS "LLVM_PACKAGE_VERSION: ${LLVM_PACKAGE_VERSION}")
+set(LLVM_BUILD_BINARY_DIR $ENV{LLVM_HOME})
+message(STATUS "LLVM_BUILD_BINARY_DIR: ${LLVM_BUILD_BINARY_DIR}")
+
# add Clang libraries
include(${LLVM_BUILD_BINARY_DIR}/lib/cmake/clang/ClangTargets.cmake)
@@ -164,6 +201,10 @@ if (USE_LLD)
target_link_libraries(operatorJITLib PRIVATE "-fuse-ld=lld")
endif ()
+set(LLVM_BUILD_MAIN_SRC_DIR "${USER_PATH}/llvm-project/llvm")
+message(STATUS "LLVM_BUILD_MAIN_SRC_DIR: ${LLVM_BUILD_MAIN_SRC_DIR}")
+
+
# find Clang source directory
if (EXISTS ${LLVM_BUILD_MAIN_SRC_DIR}/tools/clang)
set(CLANG_SRC_DIR_PREFIX tools) # sources in-tree (default build)
diff --git a/src/RDMA/infinity/core/Configuration.h b/src/RDMA/infinity/core/Configuration.h
new file mode 100644
index 0000000..09974e2
--- /dev/null
+++ b/src/RDMA/infinity/core/Configuration.h
@@ -0,0 +1,54 @@
+/**
+ * Core - Configuration
+ *
+ * (c) 2018 Claude Barthels, ETH Zurich
+ * Contact: claudeb@inf.ethz.ch
+ *
+ */
+
+#ifndef CORE_CONFIGURATION_H_
+#define CORE_CONFIGURATION_H_
+
+#include
+
+namespace infinity {
+namespace core {
+
+class Configuration {
+
+public:
+
+ /**
+ * Queue length settings
+ */
+
+ static const uint32_t SEND_COMPLETION_QUEUE_LENGTH = 8 * 1024; //16351; // Must be less than MAX_CQE
+
+ static const uint32_t RECV_COMPLETION_QUEUE_LENGTH = 16351; // Must be less than MAX_CQE
+
+ static const uint32_t SHARED_RECV_QUEUE_LENGTH = 16351; // Must be less than MAX_SRQ_WR
+
+ static const uint32_t MAX_NUMBER_OF_OUTSTANDING_REQUESTS = 16351; // Must be less than (MAX_QP_WR * MAX_QP)
+ // Since we use one single shared receive queue,
+ // this number should be less than MAX_SRQ_WR
+
+ static const uint32_t MAX_NUMBER_OF_SGE_ELEMENTS = 1; // Must be less than MAX_SGE
+
+public:
+
+ /**
+ * System settings
+ */
+
+ static const uint32_t PAGE_SIZE = 4096; // Memory regions will be page aligned by the Infinity library
+
+ static const uint32_t MAX_CONNECTION_USER_DATA_SIZE = 1024; // Size of the user data which can be transmitted when establishing a connection
+
+ static constexpr const char* DEFAULT_IB_DEVICE = "ib0"; // Default name of IB device
+
+};
+
+} /* namespace core */
+} /* namespace infinity */
+
+#endif /* CORE_CONFIGURATION_H_ */
diff --git a/src/RDMA/infinity/core/Context.cpp b/src/RDMA/infinity/core/Context.cpp
new file mode 100644
index 0000000..b2ac92c
--- /dev/null
+++ b/src/RDMA/infinity/core/Context.cpp
@@ -0,0 +1,224 @@
+/**
+ * Core - Context
+ *
+ * (c) 2018 Claude Barthels, ETH Zurich
+ * Contact: claudeb@inf.ethz.ch
+ *
+ */
+
+#include "Context.h"
+
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+
+#include
+
+#define MAX(a,b) ((a) > (b) ? (a) : (b))
+
+namespace infinity {
+namespace core {
+
+/*******************************
+ * Context
+ ******************************/
+
+Context::Context(uint16_t device, uint16_t devicePort) {
+
+ // Get IB device list
+ int32_t numberOfInstalledDevices = 0;
+ ibv_device **ibvDeviceList = ibv_get_device_list(&numberOfInstalledDevices);
+ INFINITY_ASSERT(numberOfInstalledDevices > 0, "[INFINITY][CORE][CONTEXT] No InfiniBand devices found.\n");
+ INFINITY_ASSERT(device < numberOfInstalledDevices, "[INFINITY][CORE][CONTEXT] Requested device %d not found. There are %d devices available.\n",
+ device, numberOfInstalledDevices);
+ INFINITY_ASSERT(ibvDeviceList != NULL, "[INFINITY][CORE][CONTEXT] Device list was NULL.\n");
+
+ // Get IB device
+ this->ibvDevice = ibvDeviceList[device];
+ INFINITY_ASSERT(this->ibvDevice != NULL, "[INFINITY][CORE][CONTEXT] Requested device %d was NULL.\n", device);
+
+ // Open IB device and allocate protection domain
+ this->ibvContext = ibv_open_device(this->ibvDevice);
+ INFINITY_ASSERT(this->ibvContext != NULL, "[INFINITY][CORE][CONTEXT] Could not open device %d.\n", device);
+ this->ibvProtectionDomain = ibv_alloc_pd(this->ibvContext);
+ INFINITY_ASSERT(this->ibvProtectionDomain != NULL, "[INFINITY][CORE][CONTEXT] Could not allocate protection domain.\n");
+
+ // Get the LID
+ ibv_port_attr portAttributes;
+ ibv_query_port(this->ibvContext, devicePort, &portAttributes);
+ this->ibvLocalDeviceId = portAttributes.lid;
+ this->ibvDevicePort = devicePort;
+
+ // Allocate completion queues
+ this->ibvSendCompletionQueue = ibv_create_cq(this->ibvContext, MAX(Configuration::SEND_COMPLETION_QUEUE_LENGTH, 1), NULL, NULL, 0);
+ this->ibvReceiveCompletionQueue = ibv_create_cq(this->ibvContext, MAX(Configuration::RECV_COMPLETION_QUEUE_LENGTH, 1), NULL, NULL, 0);
+
+ // Allocate shared receive queue
+ ibv_srq_init_attr sia;
+ memset(&sia, 0, sizeof(ibv_srq_init_attr));
+ sia.srq_context = this->ibvContext;
+ sia.attr.max_wr = MAX(Configuration::SHARED_RECV_QUEUE_LENGTH, 1);
+ sia.attr.max_sge = 1;
+ this->ibvSharedReceiveQueue = ibv_create_srq(this->ibvProtectionDomain, &sia);
+ INFINITY_ASSERT(this->ibvSharedReceiveQueue != NULL, "[INFINITY][CORE][CONTEXT] Could not allocate shared receive queue.\n");
+
+ // Create a default request token
+ defaultRequestToken = new infinity::requests::RequestToken(this);
+ defaultAtomic = new infinity::memory::Atomic(this);
+
+}
+
+Context::~Context() {
+
+ // Delete default token
+ delete defaultRequestToken;
+ delete defaultAtomic;
+
+ // Destroy shared receive queue
+ int returnValue = ibv_destroy_srq(this->ibvSharedReceiveQueue);
+ INFINITY_ASSERT(returnValue == 0, "[INFINITY][CORE][CONTEXT] Could not delete shared receive queue\n");
+
+ // Destroy completion queues
+ returnValue = ibv_destroy_cq(this->ibvSendCompletionQueue);
+ INFINITY_ASSERT(returnValue == 0, "[INFINITY][CORE][CONTEXT] Could not delete send completion queue\n");
+ returnValue = ibv_destroy_cq(this->ibvReceiveCompletionQueue);
+ INFINITY_ASSERT(returnValue == 0, "[INFINITY][CORE][CONTEXT] Could not delete receive completion queue\n");
+
+ // Destroy protection domain
+ returnValue = ibv_dealloc_pd(this->ibvProtectionDomain);
+ INFINITY_ASSERT(returnValue == 0, "[INFINITY][CORE][CONTEXT] Could not delete protection domain\n");
+
+ // Close device
+ returnValue = ibv_close_device(this->ibvContext);
+ INFINITY_ASSERT(returnValue == 0, "[INFINITY][CORE][CONTEXT] Could not close device\n");
+
+}
+
+void Context::postReceiveBuffer(infinity::memory::Buffer* buffer) {
+
+ INFINITY_ASSERT(buffer->getSizeInBytes() <= std::numeric_limits::max(),
+ "[INFINITY][CORE][CONTEXT] Cannot post receive buffer which is larger than max(uint32_t).\n");
+
+ // Create scatter-getter
+ ibv_sge isge;
+ memset(&isge, 0, sizeof(ibv_sge));
+ isge.addr = buffer->getAddress();
+ isge.length = static_cast(buffer->getSizeInBytes());
+ isge.lkey = buffer->getLocalKey();
+
+ // Create work request
+ ibv_recv_wr wr;
+ memset(&wr, 0, sizeof(ibv_recv_wr));
+ wr.wr_id = reinterpret_cast(buffer);
+ wr.next = NULL;
+ wr.sg_list = &isge;
+ wr.num_sge = 1;
+
+ // Post buffer to shared receive queue
+ ibv_recv_wr *badwr;
+ uint32_t returnValue = ibv_post_srq_recv(this->ibvSharedReceiveQueue, &wr, &badwr);
+ INFINITY_ASSERT(returnValue == 0, "[INFINITY][CORE][CONTEXT] Cannot post buffer to receive queue.\n");
+
+}
+
+bool Context::receive(receive_element_t* receiveElement) {
+
+ return receive(&(receiveElement->buffer), &(receiveElement->bytesWritten), &(receiveElement->immediateValue), &(receiveElement->immediateValueValid), &(receiveElement->queuePair));
+
+}
+
+bool Context::receive(infinity::memory::Buffer** buffer, uint32_t *bytesWritten, uint32_t *immediateValue, bool *immediateValueValid, infinity::queues::QueuePair **queuePair) {
+
+ ibv_wc wc;
+ if (ibv_poll_cq(this->ibvReceiveCompletionQueue, 1, &wc) > 0) {
+
+ if(wc.opcode == IBV_WC_RECV) {
+ *(buffer) = reinterpret_cast(wc.wr_id);
+ *(bytesWritten) = wc.byte_len;
+ } else if (wc.opcode == IBV_WC_RECV_RDMA_WITH_IMM) {
+ *(buffer) = NULL;
+ *(bytesWritten) = wc.byte_len;
+ infinity::memory::Buffer* receiveBuffer = reinterpret_cast(wc.wr_id);
+ this->postReceiveBuffer(receiveBuffer);
+ }
+
+ if(wc.wc_flags & IBV_WC_WITH_IMM) {
+ *(immediateValue) = ntohl(wc.imm_data);
+ *(immediateValueValid) = true;
+ } else {
+ *(immediateValue) = 0;
+ *(immediateValueValid) = false;
+ }
+
+ if(queuePair != NULL) {
+ *(queuePair) = queuePairMap.at(wc.qp_num);
+ }
+
+ return true;
+ }
+
+ return false;
+
+}
+
+bool Context::pollSendCompletionQueue() {
+
+ ibv_wc wc;
+ if (ibv_poll_cq(this->ibvSendCompletionQueue, 1, &wc) > 0) {
+
+ infinity::requests::RequestToken * request = reinterpret_cast(wc.wr_id);
+ if (request != NULL) {
+ request->setCompleted(wc.status == IBV_WC_SUCCESS);
+ }
+
+ if (wc.status == IBV_WC_SUCCESS) {
+ INFINITY_DEBUG("[INFINITY][CORE][CONTEXT] Request completed (id %lu).\n", wc.wr_id);
+ } else {
+ INFINITY_DEBUG("[INFINITY][CORE][CONTEXT] Request failed (id %lu).\n", wc.wr_id);
+ }
+ return true;
+ }
+
+ return false;
+
+}
+
+void Context::registerQueuePair(infinity::queues::QueuePair* queuePair) {
+ this->queuePairMap.insert({queuePair->getQueuePairNumber(), queuePair});
+}
+
+ibv_context* Context::getInfiniBandContext() {
+ return this->ibvContext;
+}
+
+uint16_t Context::getLocalDeviceId() {
+ return this->ibvLocalDeviceId;
+}
+
+uint16_t Context::getDevicePort() {
+ return this->ibvDevicePort;
+}
+
+ibv_pd* Context::getProtectionDomain() {
+ return this->ibvProtectionDomain;
+}
+
+ibv_cq* Context::getSendCompletionQueue() {
+ return this->ibvSendCompletionQueue;
+}
+
+ibv_cq* Context::getReceiveCompletionQueue() {
+ return this->ibvReceiveCompletionQueue;
+}
+
+ibv_srq* Context::getSharedReceiveQueue() {
+ return this->ibvSharedReceiveQueue;
+}
+
+} /* namespace core */
+} /* namespace infinity */
diff --git a/src/RDMA/infinity/core/Context.h b/src/RDMA/infinity/core/Context.h
new file mode 100644
index 0000000..2627fb0
--- /dev/null
+++ b/src/RDMA/infinity/core/Context.h
@@ -0,0 +1,166 @@
+/**
+ * Core - Context
+ *
+ * (c) 2018 Claude Barthels, ETH Zurich
+ * Contact: claudeb@inf.ethz.ch
+ *
+ */
+
+#ifndef CORE_CONTEXT_H_
+#define CORE_CONTEXT_H_
+
+#include
+#include
+#include
+#include
+
+namespace infinity {
+namespace memory {
+class Region;
+class Buffer;
+class Atomic;
+class RegisteredMemory;
+}
+}
+
+namespace infinity {
+namespace queues {
+class QueuePair;
+class QueuePairFactory;
+}
+}
+
+namespace infinity {
+namespace requests {
+class RequestToken;
+}
+}
+
+namespace infinity {
+namespace core {
+
+typedef struct {
+ infinity::memory::Buffer *buffer;
+ uint32_t bytesWritten;
+ uint32_t immediateValue;
+ bool immediateValueValid;
+ infinity::queues::QueuePair *queuePair;
+} receive_element_t;
+
+class Context {
+
+ friend class infinity::memory::Region;
+ friend class infinity::memory::Buffer;
+ friend class infinity::memory::Atomic;
+ friend class infinity::memory::RegisteredMemory;
+ friend class infinity::queues::QueuePair;
+ friend class infinity::queues::QueuePairFactory;
+ friend class infinity::requests::RequestToken;
+
+public:
+
+ /**
+ * Constructors
+ */
+ Context(uint16_t device = 0, uint16_t devicePort = 1);
+
+ /**
+ * Destructor
+ */
+ ~Context();
+
+public:
+
+ /**
+ * Check if receive operation completed
+ */
+ bool receive(receive_element_t *receiveElement);
+ bool receive(infinity::memory::Buffer **buffer, uint32_t *bytesWritten, uint32_t *immediateValue, bool *immediateValueValid, infinity::queues::QueuePair **queuePair = NULL);
+
+ /**
+ * Post a new buffer for receiving messages
+ */
+ void postReceiveBuffer(infinity::memory::Buffer *buffer);
+
+public:
+
+ infinity::requests::RequestToken * defaultRequestToken;
+ infinity::memory::Atomic * defaultAtomic;
+
+protected:
+
+ /**
+ * Returns ibVerbs context
+ */
+ ibv_context * getInfiniBandContext();
+
+ /**
+ * Returns local device id
+ */
+ uint16_t getLocalDeviceId();
+
+ /**
+ * Returns device port
+ */
+ uint16_t getDevicePort();
+
+ /**
+ * Returns ibVerbs protection domain
+ */
+ ibv_pd * getProtectionDomain();
+
+protected:
+
+ /**
+ * Check if send operation completed
+ */
+ bool pollSendCompletionQueue();
+
+ /**
+ * Returns ibVerbs completion queue for sending
+ */
+ ibv_cq * getSendCompletionQueue();
+
+ /**
+ * Returns ibVerbs completion queue for receiving
+ */
+ ibv_cq * getReceiveCompletionQueue();
+
+ /**
+ * Returns ibVerbs shared receive queue
+ */
+ ibv_srq * getSharedReceiveQueue();
+
+protected:
+
+ /**
+ * IB context and protection domain
+ */
+ ibv_context *ibvContext;
+ ibv_pd *ibvProtectionDomain;
+
+ /**
+ * Local device id and port
+ */
+ ibv_device *ibvDevice;
+ uint16_t ibvLocalDeviceId;
+ uint16_t ibvDevicePort;
+
+ /**
+ * IB send and receive completion queues
+ */
+ ibv_cq *ibvSendCompletionQueue;
+ ibv_cq *ibvReceiveCompletionQueue;
+ ibv_srq *ibvSharedReceiveQueue;
+
+protected:
+
+ void registerQueuePair(infinity::queues::QueuePair *queuePair);
+ std::unordered_map queuePairMap;
+
+};
+
+} /* namespace core */
+} /* namespace infinity */
+
+#endif /* CORE_CONTEXT_H_ */
diff --git a/src/RDMA/infinity/infinity.h b/src/RDMA/infinity/infinity.h
new file mode 100644
index 0000000..31d44aa
--- /dev/null
+++ b/src/RDMA/infinity/infinity.h
@@ -0,0 +1,26 @@
+/**
+ * Infinity - A C++ RDMA library for InfiniBand
+ *
+ * (c) 2018 Claude Barthels, ETH Zurich
+ * Contact: claudeb@inf.ethz.ch
+ *
+ */
+
+#ifndef INFINITY_H_
+#define INFINITY_H_
+
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+
+#endif /* INFINITY_H_ */
diff --git a/src/RDMA/infinity/memory/Atomic.cpp b/src/RDMA/infinity/memory/Atomic.cpp
new file mode 100644
index 0000000..5c22e8a
--- /dev/null
+++ b/src/RDMA/infinity/memory/Atomic.cpp
@@ -0,0 +1,51 @@
+/*
+ * Memory - Atomic
+ *
+ * (c) 2018 Claude Barthels, ETH Zurich
+ * Contact: claudeb@inf.ethz.ch
+ *
+ */
+
+#include "Atomic.h"
+
+#include
+
+namespace infinity {
+namespace memory {
+
+Atomic::Atomic(infinity::core::Context* context) {
+
+ this->context = context;
+ this->sizeInBytes = sizeof(uint64_t);
+ this->memoryRegionType = RegionType::ATOMIC;
+
+ this->value = 0;
+ this->data = &value;
+
+ this->ibvMemoryRegion = ibv_reg_mr(this->context->getProtectionDomain(), &(this->value), this->sizeInBytes,
+ IBV_ACCESS_LOCAL_WRITE | IBV_ACCESS_REMOTE_ATOMIC | IBV_ACCESS_REMOTE_READ | IBV_ACCESS_REMOTE_WRITE);
+
+
+}
+
+uint64_t infinity::memory::Atomic::getValue() {
+
+ return this->value;
+
+}
+
+void infinity::memory::Atomic::setValueNonAtomic(uint64_t value) {
+
+ this->value = value;
+
+}
+
+
+Atomic::~Atomic() {
+
+ ibv_dereg_mr(this->ibvMemoryRegion);
+
+}
+
+} /* namespace memory */
+} /* namespace infinity */
diff --git a/src/RDMA/infinity/memory/Atomic.h b/src/RDMA/infinity/memory/Atomic.h
new file mode 100644
index 0000000..d6912a3
--- /dev/null
+++ b/src/RDMA/infinity/memory/Atomic.h
@@ -0,0 +1,42 @@
+/*
+ * Memory - Atomic
+ *
+ * (c) 2018 Claude Barthels, ETH Zurich
+ * Contact: claudeb@inf.ethz.ch
+ *
+ */
+
+#ifndef MEMORY_ATOMIC_H_
+#define MEMORY_ATOMIC_H_
+
+#include
+#include
+#include
+
+namespace infinity {
+namespace memory {
+
+class Atomic : public Region {
+
+public:
+
+ Atomic(infinity::core::Context *context);
+ virtual ~Atomic();
+
+public:
+
+ uint64_t getValue();
+
+ void setValueNonAtomic(uint64_t value);
+
+protected:
+
+ uint64_t value;
+
+
+};
+
+} /* namespace memory */
+} /* namespace infinity */
+
+#endif /* MEMORY_ATOMIC_H_ */
diff --git a/src/RDMA/infinity/memory/Buffer.cpp b/src/RDMA/infinity/memory/Buffer.cpp
new file mode 100644
index 0000000..1b12d1f
--- /dev/null
+++ b/src/RDMA/infinity/memory/Buffer.cpp
@@ -0,0 +1,112 @@
+/*
+ * Memory - Buffer
+ *
+ * (c) 2018 Claude Barthels, ETH Zurich
+ * Contact: claudeb@inf.ethz.ch
+ *
+ */
+
+#include "Buffer.h"
+
+#include
+#include
+#include
+#include
+
+#define MIN(a,b) (((a)<(b)) ? (a) : (b))
+
+namespace infinity {
+namespace memory {
+
+Buffer::Buffer(infinity::core::Context* context, uint64_t sizeInBytes) {
+
+ this->context = context;
+ this->sizeInBytes = sizeInBytes;
+ this->memoryRegionType = RegionType::BUFFER;
+
+ int res = posix_memalign(&(this->data), infinity::core::Configuration::PAGE_SIZE, sizeInBytes);
+ INFINITY_ASSERT(res == 0, "[INFINITY][MEMORY][BUFFER] Cannot allocate and align buffer.\n");
+
+ memset(this->data, 0, sizeInBytes);
+
+ this->ibvMemoryRegion = ibv_reg_mr(this->context->getProtectionDomain(), this->data, this->sizeInBytes,
+ IBV_ACCESS_REMOTE_WRITE | IBV_ACCESS_LOCAL_WRITE | IBV_ACCESS_REMOTE_READ);
+ INFINITY_ASSERT(this->ibvMemoryRegion != NULL, "[INFINITY][MEMORY][BUFFER] Registration failed.\n");
+
+ this->memoryAllocated = true;
+ this->memoryRegistered = true;
+
+}
+
+Buffer::Buffer(infinity::core::Context* context, infinity::memory::RegisteredMemory* memory, uint64_t offset, uint64_t sizeInBytes) {
+
+ this->context = context;
+ this->sizeInBytes = sizeInBytes;
+ this->memoryRegionType = RegionType::BUFFER;
+
+ this->data = reinterpret_cast(memory->getData()) + offset;
+ this->ibvMemoryRegion = memory->getRegion();
+
+ this->memoryAllocated = false;
+ this->memoryRegistered = false;
+
+}
+
+Buffer::Buffer(infinity::core::Context *context, void *memory, uint64_t sizeInBytes) {
+
+ this->context = context;
+ this->sizeInBytes = sizeInBytes;
+ this->memoryRegionType = RegionType::BUFFER;
+
+ this->data = memory;
+ this->ibvMemoryRegion = ibv_reg_mr(this->context->getProtectionDomain(), this->data, this->sizeInBytes,
+ IBV_ACCESS_REMOTE_WRITE | IBV_ACCESS_LOCAL_WRITE | IBV_ACCESS_REMOTE_READ);
+ INFINITY_ASSERT(this->ibvMemoryRegion != NULL, "[INFINITY][MEMORY][BUFFER] Registration failed.\n");
+
+ this->memoryAllocated = false;
+ this->memoryRegistered = true;
+
+}
+
+Buffer::~Buffer() {
+
+ if (this->memoryRegistered) {
+ ibv_dereg_mr(this->ibvMemoryRegion);
+ }
+ if (this->memoryAllocated) {
+ free(this->data);
+ }
+
+}
+
+void* Buffer::getData() {
+ return reinterpret_cast(this->getAddress());
+}
+
+void Buffer::resize(uint64_t newSize, void* newData) {
+
+ void *oldData = this->data;
+ uint32_t oldSize = this->sizeInBytes;
+
+ if (newData == NULL) {
+ newData = this->data;
+ }
+
+ if (oldData != newData) {
+ uint64_t copySize = MIN(newSize, oldSize);
+ memcpy(newData, oldData, copySize);
+ }
+
+ if (memoryRegistered) {
+ ibv_dereg_mr(this->ibvMemoryRegion);
+ this->ibvMemoryRegion = ibv_reg_mr(this->context->getProtectionDomain(), newData, newSize,
+ IBV_ACCESS_REMOTE_WRITE | IBV_ACCESS_LOCAL_WRITE | IBV_ACCESS_REMOTE_READ);
+ this->data = newData;
+ this->sizeInBytes = newSize;
+ } else {
+ INFINITY_ASSERT(false, "[INFINITY][MEMORY][BUFFER] You can only resize memory which has registered by this buffer.\n");
+ }
+}
+
+} /* namespace memory */
+} /* namespace infinity */
diff --git a/src/RDMA/infinity/memory/Buffer.h b/src/RDMA/infinity/memory/Buffer.h
new file mode 100644
index 0000000..c9bd3f2
--- /dev/null
+++ b/src/RDMA/infinity/memory/Buffer.h
@@ -0,0 +1,44 @@
+/*
+ * Memory - Buffer
+ *
+ * (c) 2018 Claude Barthels, ETH Zurich
+ * Contact: claudeb@inf.ethz.ch
+ *
+ */
+
+#ifndef MEMORY_BUFFER_H_
+#define MEMORY_BUFFER_H_
+
+#include
+#include
+#include
+
+namespace infinity {
+namespace memory {
+
+class Buffer : public Region {
+
+public:
+
+ Buffer(infinity::core::Context *context, uint64_t sizeInBytes);
+ Buffer(infinity::core::Context *context, infinity::memory::RegisteredMemory *memory, uint64_t offset, uint64_t sizeInBytes);
+ Buffer(infinity::core::Context *context, void *memory, uint64_t sizeInBytes);
+ ~Buffer();
+
+public:
+
+ void * getData();
+ void resize(uint64_t newSize, void *newData = NULL);
+
+protected:
+
+ bool memoryRegistered;
+ bool memoryAllocated;
+
+
+};
+
+} /* namespace memory */
+} /* namespace infinity */
+
+#endif /* MEMORY_BUFFER_H_ */
diff --git a/src/RDMA/infinity/memory/Region.cpp b/src/RDMA/infinity/memory/Region.cpp
new file mode 100644
index 0000000..929e105
--- /dev/null
+++ b/src/RDMA/infinity/memory/Region.cpp
@@ -0,0 +1,62 @@
+/*
+ * Memory - Region
+ *
+ * (c) 2018 Claude Barthels, ETH Zurich
+ * Contact: claudeb@inf.ethz.ch
+ *
+ */
+
+#include
+#include
+
+#include "Buffer.h"
+
+namespace infinity {
+namespace memory {
+
+Region::~Region() {
+ // To be overwritten in sub class
+}
+
+RegionToken* Region::createRegionToken() {
+ return new RegionToken(this, getMemoryRegionType(), getSizeInBytes(), getAddress(), getLocalKey(), getRemoteKey());
+}
+
+RegionToken * Region::createRegionToken(uint64_t offset) {
+ return new RegionToken(this, getMemoryRegionType(), getRemainingSizeInBytes(offset), getAddressWithOffset(offset), getLocalKey(), getRemoteKey());
+}
+
+RegionToken * Region::createRegionToken(uint64_t offset, uint64_t size) {
+ return new RegionToken(this, getMemoryRegionType(), size, getAddressWithOffset(offset), getLocalKey(), getRemoteKey());
+}
+
+RegionType Region::getMemoryRegionType() {
+ return this->memoryRegionType;
+}
+
+uint64_t Region::getSizeInBytes() {
+ return this->sizeInBytes;
+}
+
+uint64_t Region::getRemainingSizeInBytes(uint64_t offset) {
+ return this->sizeInBytes - offset;
+}
+
+uint64_t Region::getAddress() {
+ return reinterpret_cast(this->data);
+}
+
+uint64_t Region::getAddressWithOffset(uint64_t offset) {
+ return reinterpret_cast(this->data) + offset;
+}
+
+uint32_t Region::getLocalKey() {
+ return this->ibvMemoryRegion->lkey;
+}
+
+uint32_t Region::getRemoteKey() {
+ return this->ibvMemoryRegion->rkey;
+}
+
+} /* namespace memory */
+} /* namespace infinity */
diff --git a/src/RDMA/infinity/memory/Region.h b/src/RDMA/infinity/memory/Region.h
new file mode 100644
index 0000000..5191077
--- /dev/null
+++ b/src/RDMA/infinity/memory/Region.h
@@ -0,0 +1,58 @@
+/*
+ * Memory - Region
+ *
+ * (c) 2018 Claude Barthels, ETH Zurich
+ * Contact: claudeb@inf.ethz.ch
+ *
+ */
+
+#ifndef MEMORY_REGION_H_
+#define MEMORY_REGION_H_
+
+#include
+#include
+#include
+#include
+
+namespace infinity {
+namespace memory {
+
+class RegionToken;
+
+class Region {
+
+public:
+
+ virtual ~Region();
+
+ RegionToken * createRegionToken();
+ RegionToken * createRegionToken(uint64_t offset);
+ RegionToken * createRegionToken(uint64_t offset, uint64_t size);
+
+public:
+
+ RegionType getMemoryRegionType();
+ uint64_t getSizeInBytes();
+ uint64_t getRemainingSizeInBytes(uint64_t offset);
+ uint64_t getAddress();
+ uint64_t getAddressWithOffset(uint64_t offset);
+ uint32_t getLocalKey();
+ uint32_t getRemoteKey();
+
+protected:
+
+ infinity::core::Context* context;
+ RegionType memoryRegionType;
+ ibv_mr *ibvMemoryRegion;
+
+protected:
+
+ void * data;
+ uint64_t sizeInBytes;
+
+};
+
+} /* namespace memory */
+} /* namespace infinity */
+
+#endif /* MEMORY_REGION_H_ */
diff --git a/src/RDMA/infinity/memory/RegionToken.cpp b/src/RDMA/infinity/memory/RegionToken.cpp
new file mode 100644
index 0000000..190054e
--- /dev/null
+++ b/src/RDMA/infinity/memory/RegionToken.cpp
@@ -0,0 +1,72 @@
+/*
+ * Memory - Region Token
+ *
+ * (c) 2018 Claude Barthels, ETH Zurich
+ * Contact: claudeb@inf.ethz.ch
+ *
+ */
+
+#include
+
+namespace infinity {
+namespace memory {
+
+RegionToken::RegionToken() :
+ memoryRegion (NULL),
+ memoryRegionType (UNKNOWN),
+ sizeInBytes(0),
+ address(0),
+ localKey(0),
+ remoteKey(0) {
+
+ // Nothing to do here
+
+}
+
+RegionToken::RegionToken(Region *memoryRegion, RegionType memoryRegionType, uint64_t sizeInBytes, uint64_t address, uint32_t localKey, uint32_t remoteKey) :
+ memoryRegion (memoryRegion),
+ memoryRegionType (memoryRegionType),
+ sizeInBytes(sizeInBytes),
+ address(address),
+ localKey(localKey),
+ remoteKey(remoteKey) {
+
+ // Nothing to do here
+
+}
+
+Region* RegionToken::getMemoryRegion() {
+ return memoryRegion;
+}
+
+RegionType RegionToken::getMemoryRegionType() {
+ return this->memoryRegionType;
+}
+
+uint64_t RegionToken::getSizeInBytes() {
+ return this->sizeInBytes;
+}
+
+uint64_t RegionToken::getRemainingSizeInBytes(uint64_t offset) {
+ return this->sizeInBytes-offset;
+}
+
+uint64_t RegionToken::getAddress() {
+ return address;
+}
+
+uint64_t RegionToken::getAddressWithOffset(uint64_t offset) {
+ return address + offset;
+}
+
+uint32_t RegionToken::getLocalKey() {
+ return this->localKey;
+}
+
+uint32_t RegionToken::getRemoteKey() {
+ return this->remoteKey;
+}
+
+
+} /* namespace memory */
+} /* namespace infinity */
diff --git a/src/RDMA/infinity/memory/RegionToken.h b/src/RDMA/infinity/memory/RegionToken.h
new file mode 100644
index 0000000..b84dbc3
--- /dev/null
+++ b/src/RDMA/infinity/memory/RegionToken.h
@@ -0,0 +1,51 @@
+/*
+ * Memory - Region Token
+ *
+ * (c) 2018 Claude Barthels, ETH Zurich
+ * Contact: claudeb@inf.ethz.ch
+ *
+ */
+
+#ifndef MEMORY_REGIONTOKEN_H_
+#define MEMORY_REGIONTOKEN_H_
+
+#include
+#include
+#include
+
+namespace infinity {
+namespace memory {
+
+class RegionToken {
+
+public:
+
+ RegionToken();
+ RegionToken(Region *memoryRegion, RegionType memoryRegionType, uint64_t sizeInBytes, uint64_t address, uint32_t localKey, uint32_t remoteKey);
+
+public:
+
+ Region * getMemoryRegion();
+ RegionType getMemoryRegionType();
+ uint64_t getSizeInBytes();
+ uint64_t getRemainingSizeInBytes(uint64_t offset);
+ uint64_t getAddress();
+ uint64_t getAddressWithOffset(uint64_t offset);
+ uint32_t getLocalKey();
+ uint32_t getRemoteKey();
+
+protected:
+
+ Region *memoryRegion;
+ const RegionType memoryRegionType;
+ const uint64_t sizeInBytes;
+ const uint64_t address;
+ const uint32_t localKey;
+ const uint32_t remoteKey;
+
+};
+
+} /* namespace memory */
+} /* namespace infinity */
+
+#endif /* MEMORY_REGIONTOKEN_H_ */
diff --git a/src/RDMA/infinity/memory/RegionType.h b/src/RDMA/infinity/memory/RegionType.h
new file mode 100644
index 0000000..2f1ca9e
--- /dev/null
+++ b/src/RDMA/infinity/memory/RegionType.h
@@ -0,0 +1,20 @@
+/*
+ * Memory - Region Type
+ *
+ * (c) 2018 Claude Barthels, ETH Zurich
+ * Contact: claudeb@inf.ethz.ch
+ *
+ */
+
+#ifndef MEMORY_REGIONTYPE_H_
+#define MEMORY_REGIONTYPE_H_
+
+namespace infinity {
+namespace memory {
+
+enum RegionType {BUFFER, ATOMIC, UNKNOWN};
+
+} /* namespace memory */
+} /* namespace infinity */
+
+#endif /* MEMORY_REGIONTYPE_H_ */
diff --git a/src/RDMA/infinity/memory/RegisteredMemory.cpp b/src/RDMA/infinity/memory/RegisteredMemory.cpp
new file mode 100644
index 0000000..2ed7a7d
--- /dev/null
+++ b/src/RDMA/infinity/memory/RegisteredMemory.cpp
@@ -0,0 +1,78 @@
+/*
+ * Memory - Registered Memory
+ *
+ * (c) 2018 Claude Barthels, ETH Zurich
+ * Contact: claudeb@inf.ethz.ch
+ *
+ */
+
+#include "RegisteredMemory.h"
+
+#include
+#include
+#include
+#include
+
+namespace infinity {
+namespace memory {
+
+RegisteredMemory::RegisteredMemory(infinity::core::Context* context, uint64_t sizeInBytes) {
+
+ this->context = context;
+ this->sizeInBytes = sizeInBytes;
+ this->memoryAllocated = true;
+
+ int res = posix_memalign(&(this->data), infinity::core::Configuration::PAGE_SIZE, sizeInBytes);
+ INFINITY_ASSERT(res == 0, "[INFINITY][MEMORY][REGISTERED] Cannot allocate and align buffer.\n");
+
+ memset(this->data, 0, sizeInBytes);
+
+ this->ibvMemoryRegion = ibv_reg_mr(this->context->getProtectionDomain(), this->data, this->sizeInBytes,
+ IBV_ACCESS_REMOTE_WRITE | IBV_ACCESS_LOCAL_WRITE | IBV_ACCESS_REMOTE_READ);
+ INFINITY_ASSERT(this->ibvMemoryRegion != NULL, "[INFINITY][MEMORY][REGISTERED] Registration failed.\n");
+}
+
+RegisteredMemory::RegisteredMemory(infinity::core::Context* context, void *data, uint64_t sizeInBytes) {
+
+ this->context = context;
+ this->sizeInBytes = sizeInBytes;
+ this->memoryAllocated = false;
+
+ this->data = data;
+
+ this->ibvMemoryRegion = ibv_reg_mr(this->context->getProtectionDomain(), this->data, this->sizeInBytes,
+ IBV_ACCESS_REMOTE_WRITE | IBV_ACCESS_LOCAL_WRITE | IBV_ACCESS_REMOTE_READ);
+ INFINITY_ASSERT(this->ibvMemoryRegion != NULL, "[INFINITY][MEMORY][REGISTERED] Registration failed.\n");
+}
+
+
+RegisteredMemory::~RegisteredMemory() {
+
+ ibv_dereg_mr(this->ibvMemoryRegion);
+
+ if(this->memoryAllocated) {
+ free(this->data);
+ }
+
+}
+
+void* RegisteredMemory::getData() {
+
+ return this->data;
+
+}
+
+uint64_t RegisteredMemory::getSizeInBytes() {
+
+ return this->sizeInBytes;
+
+}
+
+ibv_mr* RegisteredMemory::getRegion() {
+
+ return this->ibvMemoryRegion;
+
+}
+
+} /* namespace pool */
+} /* namespace ivory */
diff --git a/src/RDMA/infinity/memory/RegisteredMemory.h b/src/RDMA/infinity/memory/RegisteredMemory.h
new file mode 100644
index 0000000..b2d6fb3
--- /dev/null
+++ b/src/RDMA/infinity/memory/RegisteredMemory.h
@@ -0,0 +1,50 @@
+/*
+ * Memory - Registered Memory
+ *
+ * (c) 2018 Claude Barthels, ETH Zurich
+ * Contact: claudeb@inf.ethz.ch
+ *
+ */
+
+#ifndef INFINITY_MEMORY_REGISTEREDMEMORY_H_
+#define INFINITY_MEMORY_REGISTEREDMEMORY_H_
+
+#include
+
+namespace infinity {
+namespace memory {
+
+class RegisteredMemory {
+
+public:
+
+ RegisteredMemory(infinity::core::Context *context, uint64_t sizeInBytes);
+ RegisteredMemory(infinity::core::Context *context, void *data, uint64_t sizeInBytes);
+ ~RegisteredMemory();
+
+ void * getData();
+
+ uint64_t getSizeInBytes();
+
+ ibv_mr * getRegion();
+
+
+protected:
+
+ infinity::core::Context* context;
+
+ void *data;
+ uint64_t sizeInBytes;
+
+ ibv_mr *ibvMemoryRegion;
+
+protected:
+
+ bool memoryAllocated;
+
+};
+
+} /* namespace infinity */
+} /* namespace memory */
+
+#endif /* INFINITY_MEMORY_REGISTEREDMEMORY_H_ */
diff --git a/src/RDMA/infinity/queues/QueuePair.cpp b/src/RDMA/infinity/queues/QueuePair.cpp
new file mode 100644
index 0000000..9626214
--- /dev/null
+++ b/src/RDMA/infinity/queues/QueuePair.cpp
@@ -0,0 +1,589 @@
+/**
+ * Queues - Queue Pair
+ *
+ * (c) 2018 Claude Barthels, ETH Zurich
+ * Contact: claudeb@inf.ethz.ch
+ *
+ */
+
+#include "QueuePair.h"
+
+#include
+#include
+#include
+#include
+
+#include
+#include
+
+#define MAX(a,b) ((a) > (b) ? (a) : (b))
+
+namespace infinity {
+namespace queues {
+
+int OperationFlags::ibvFlags() {
+ int flags = 0;
+ if (fenced) {
+ flags |= IBV_SEND_FENCE;
+ }
+ if (signaled) {
+ flags |= IBV_SEND_SIGNALED;
+ }
+ if (inlined) {
+ flags |= IBV_SEND_INLINE;
+ }
+ return flags;
+}
+
+QueuePair::QueuePair(infinity::core::Context* context) :
+ context(context) {
+
+ ibv_qp_init_attr qpInitAttributes;
+ memset(&qpInitAttributes, 0, sizeof(qpInitAttributes));
+
+ qpInitAttributes.send_cq = context->getSendCompletionQueue();
+ qpInitAttributes.recv_cq = context->getReceiveCompletionQueue();
+ qpInitAttributes.srq = context->getSharedReceiveQueue();
+ qpInitAttributes.cap.max_send_wr = MAX(infinity::core::Configuration::SEND_COMPLETION_QUEUE_LENGTH, 1);
+ qpInitAttributes.cap.max_send_sge = infinity::core::Configuration::MAX_NUMBER_OF_SGE_ELEMENTS;
+ qpInitAttributes.cap.max_recv_wr = MAX(infinity::core::Configuration::RECV_COMPLETION_QUEUE_LENGTH, 1);
+ qpInitAttributes.cap.max_recv_sge = infinity::core::Configuration::MAX_NUMBER_OF_SGE_ELEMENTS;
+ qpInitAttributes.qp_type = IBV_QPT_RC;
+ qpInitAttributes.sq_sig_all = 0;
+
+ this->ibvQueuePair = ibv_create_qp(context->getProtectionDomain(), &(qpInitAttributes));
+ INFINITY_ASSERT(this->ibvQueuePair != NULL, "[INFINITY][QUEUES][QUEUEPAIR] Cannot create queue pair.\n");
+
+ ibv_qp_attr qpAttributes;
+ memset(&qpAttributes, 0, sizeof(qpAttributes));
+
+ qpAttributes.qp_state = IBV_QPS_INIT;
+ qpAttributes.pkey_index = 0;
+ qpAttributes.port_num = context->getDevicePort();
+ qpAttributes.qp_access_flags = IBV_ACCESS_REMOTE_WRITE | IBV_ACCESS_REMOTE_READ | IBV_ACCESS_LOCAL_WRITE | IBV_ACCESS_REMOTE_ATOMIC;
+
+ int32_t returnValue = ibv_modify_qp(this->ibvQueuePair, &(qpAttributes), IBV_QP_STATE | IBV_QP_PORT | IBV_QP_ACCESS_FLAGS | IBV_QP_PKEY_INDEX);
+
+ INFINITY_ASSERT(returnValue == 0, "[INFINITY][QUEUES][QUEUEPAIR] Cannot transition to INIT state.\n");
+
+ std::random_device randomGenerator;
+ std::uniform_int_distribution range(0, 1<<24);
+ this->sequenceNumber = range(randomGenerator);
+
+ this->userData = NULL;
+ this->userDataSize = 0;
+}
+
+QueuePair::~QueuePair() {
+
+ int32_t returnValue = ibv_destroy_qp(this->ibvQueuePair);
+ INFINITY_ASSERT(returnValue == 0, "[INFINITY][QUEUES][QUEUEPAIR] Cannot delete queue pair.\n");
+
+ if (this->userData != NULL && this->userDataSize != 0) {
+ free(this->userData);
+ this->userDataSize = 0;
+ }
+
+}
+
+void QueuePair::activate(uint16_t remoteDeviceId, uint32_t remoteQueuePairNumber, uint32_t remoteSequenceNumber) {
+
+ ibv_qp_attr qpAttributes;
+ memset(&(qpAttributes), 0, sizeof(qpAttributes));
+
+ qpAttributes.qp_state = IBV_QPS_RTR;
+ qpAttributes.path_mtu = IBV_MTU_4096;
+ qpAttributes.dest_qp_num = remoteQueuePairNumber;
+ qpAttributes.rq_psn = remoteSequenceNumber;
+ qpAttributes.max_dest_rd_atomic = 1;
+ qpAttributes.min_rnr_timer = 12;
+ qpAttributes.ah_attr.is_global = 0;
+ qpAttributes.ah_attr.dlid = remoteDeviceId;
+ qpAttributes.ah_attr.sl = 0;
+ qpAttributes.ah_attr.src_path_bits = 0;
+ qpAttributes.ah_attr.port_num = context->getDevicePort();
+
+ int32_t returnValue = ibv_modify_qp(this->ibvQueuePair, &qpAttributes,
+ IBV_QP_STATE | IBV_QP_AV | IBV_QP_PATH_MTU | IBV_QP_DEST_QPN | IBV_QP_RQ_PSN | IBV_QP_MIN_RNR_TIMER | IBV_QP_MAX_DEST_RD_ATOMIC);
+
+ INFINITY_ASSERT(returnValue == 0, "[INFINITY][QUEUES][QUEUEPAIR] Cannot transition to RTR state.\n");
+
+ qpAttributes.qp_state = IBV_QPS_RTS;
+ qpAttributes.timeout = 14;
+ qpAttributes.retry_cnt = 7;
+ qpAttributes.rnr_retry = 7;
+ qpAttributes.sq_psn = this->getSequenceNumber();
+ qpAttributes.max_rd_atomic = 1;
+
+ returnValue = ibv_modify_qp(this->ibvQueuePair, &qpAttributes,
+ IBV_QP_STATE | IBV_QP_TIMEOUT | IBV_QP_RETRY_CNT | IBV_QP_RNR_RETRY | IBV_QP_SQ_PSN | IBV_QP_MAX_QP_RD_ATOMIC);
+
+ INFINITY_ASSERT(returnValue == 0, "[INFINITY][QUEUES][QUEUEPAIR] Cannot transition to RTS state.\n");
+
+}
+
+void QueuePair::setRemoteUserData(void* userData, uint32_t userDataSize) {
+ if (userDataSize > 0) {
+ this->userData = new char[userDataSize];
+ memcpy(this->userData, userData, userDataSize);
+ this->userDataSize = userDataSize;
+ }
+}
+
+uint16_t QueuePair::getLocalDeviceId() {
+ return this->context->getLocalDeviceId();
+}
+
+uint32_t QueuePair::getQueuePairNumber() {
+ return this->ibvQueuePair->qp_num;
+}
+
+uint32_t QueuePair::getSequenceNumber() {
+ return this->sequenceNumber;
+}
+
+void QueuePair::send(infinity::memory::Buffer* buffer, infinity::requests::RequestToken *requestToken) {
+ send(buffer, 0, buffer->getSizeInBytes(), OperationFlags(), requestToken);
+}
+
+void QueuePair::send(infinity::memory::Buffer* buffer, uint32_t sizeInBytes, infinity::requests::RequestToken *requestToken) {
+ send(buffer, 0, sizeInBytes, OperationFlags(), requestToken);
+}
+
+void QueuePair::send(infinity::memory::Buffer* buffer, uint64_t localOffset, uint32_t sizeInBytes, OperationFlags send_flags,
+ infinity::requests::RequestToken *requestToken) {
+
+ if (requestToken != NULL) {
+ requestToken->reset();
+ requestToken->setRegion(buffer);
+ }
+
+ struct ibv_sge sgElement;
+ struct ibv_send_wr workRequest;
+ struct ibv_send_wr *badWorkRequest;
+
+ memset(&sgElement, 0, sizeof(ibv_sge));
+ sgElement.addr = buffer->getAddress() + localOffset;
+ sgElement.length = sizeInBytes;
+ sgElement.lkey = buffer->getLocalKey();
+
+ INFINITY_ASSERT(sizeInBytes <= buffer->getRemainingSizeInBytes(localOffset),
+ "[INFINITY][QUEUES][QUEUEPAIR] Segmentation fault while creating scatter-getter element.\n");
+
+ memset(&workRequest, 0, sizeof(ibv_send_wr));
+ workRequest.wr_id = reinterpret_cast(requestToken);
+ workRequest.sg_list = &sgElement;
+ workRequest.num_sge = 1;
+ workRequest.opcode = IBV_WR_SEND;
+ workRequest.send_flags = send_flags.ibvFlags();
+ if (requestToken != NULL) {
+ workRequest.send_flags |= IBV_SEND_SIGNALED;
+ }
+
+ int returnValue = ibv_post_send(this->ibvQueuePair, &workRequest, &badWorkRequest);
+
+ INFINITY_ASSERT(returnValue == 0, "[INFINITY][QUEUES][QUEUEPAIR] Posting send request failed. %s.\n", strerror(errno));
+
+ INFINITY_DEBUG("[INFINITY][QUEUES][QUEUEPAIR] Send request created (id %lu).\n", workRequest.wr_id);
+
+}
+
+void QueuePair::sendWithImmediate(infinity::memory::Buffer* buffer, uint64_t localOffset, uint32_t sizeInBytes, uint32_t immediateValue,
+ OperationFlags send_flags, infinity::requests::RequestToken* requestToken) {
+
+ if (requestToken != NULL) {
+ requestToken->reset();
+ requestToken->setRegion(buffer);
+ requestToken->setImmediateValue(immediateValue);
+ }
+
+ struct ibv_sge sgElement;
+ struct ibv_send_wr workRequest;
+ struct ibv_send_wr *badWorkRequest;
+
+ memset(&sgElement, 0, sizeof(ibv_sge));
+ sgElement.addr = buffer->getAddress() + localOffset;
+ sgElement.length = sizeInBytes;
+ sgElement.lkey = buffer->getLocalKey();
+
+ INFINITY_ASSERT(sizeInBytes <= buffer->getRemainingSizeInBytes(localOffset),
+ "[INFINITY][QUEUES][QUEUEPAIR] Segmentation fault while creating scatter-getter element.\n");
+
+ memset(&workRequest, 0, sizeof(ibv_send_wr));
+ workRequest.wr_id = reinterpret_cast(requestToken);
+ workRequest.sg_list = &sgElement;
+ workRequest.num_sge = 1;
+ workRequest.opcode = IBV_WR_SEND_WITH_IMM;
+ workRequest.imm_data = htonl(immediateValue);
+ workRequest.send_flags = send_flags.ibvFlags();
+ if (requestToken != NULL) {
+ workRequest.send_flags |= IBV_SEND_SIGNALED;
+ }
+
+ int returnValue = ibv_post_send(this->ibvQueuePair, &workRequest, &badWorkRequest);
+
+ INFINITY_ASSERT(returnValue == 0, "[INFINITY][QUEUES][QUEUEPAIR] Posting send request failed. %s.\n", strerror(errno));
+
+ INFINITY_DEBUG("[INFINITY][QUEUES][QUEUEPAIR] Send request created (id %lu).\n", workRequest.wr_id);
+
+}
+
+void QueuePair::write(infinity::memory::Buffer* buffer, infinity::memory::RegionToken* destination, infinity::requests::RequestToken *requestToken) {
+ write(buffer, 0, destination, 0, buffer->getSizeInBytes(), OperationFlags(), requestToken);
+ INFINITY_ASSERT(buffer->getSizeInBytes() <= ((uint64_t) UINT32_MAX), "[INFINITY][QUEUES][QUEUEPAIR] Request must be smaller or equal to UINT_32_MAX bytes. This memory region is larger. Please explicitly indicate the size of the data to transfer.\n");
+}
+
+void QueuePair::write(infinity::memory::Buffer* buffer, infinity::memory::RegionToken* destination, uint32_t sizeInBytes,
+ infinity::requests::RequestToken *requestToken) {
+ write(buffer, 0, destination, 0, sizeInBytes, OperationFlags(), requestToken);
+}
+
+void QueuePair::write(infinity::memory::Buffer* buffer, uint64_t localOffset, infinity::memory::RegionToken* destination, uint64_t remoteOffset,
+ uint32_t sizeInBytes, OperationFlags send_flags, infinity::requests::RequestToken *requestToken) {
+
+ if (requestToken != NULL) {
+ requestToken->reset();
+ requestToken->setRegion(buffer);
+ }
+
+ struct ibv_sge sgElement;
+ struct ibv_send_wr workRequest;
+ struct ibv_send_wr *badWorkRequest;
+
+ memset(&sgElement, 0, sizeof(ibv_sge));
+ sgElement.addr = buffer->getAddress() + localOffset;
+ sgElement.length = sizeInBytes;
+ sgElement.lkey = buffer->getLocalKey();
+
+ INFINITY_ASSERT(sizeInBytes <= buffer->getRemainingSizeInBytes(localOffset),
+ "[INFINITY][QUEUES][QUEUEPAIR] Segmentation fault while creating scatter-getter element.\n");
+
+ memset(&workRequest, 0, sizeof(ibv_send_wr));
+ workRequest.wr_id = reinterpret_cast(requestToken);
+ workRequest.sg_list = &sgElement;
+ workRequest.num_sge = 1;
+ workRequest.opcode = IBV_WR_RDMA_WRITE;
+ workRequest.send_flags = send_flags.ibvFlags();
+ if (requestToken != NULL) {
+ workRequest.send_flags |= IBV_SEND_SIGNALED;
+ }
+ workRequest.wr.rdma.remote_addr = destination->getAddress() + remoteOffset;
+ workRequest.wr.rdma.rkey = destination->getRemoteKey();
+
+ INFINITY_ASSERT(sizeInBytes <= destination->getRemainingSizeInBytes(remoteOffset),
+ "[INFINITY][QUEUES][QUEUEPAIR] Segmentation fault while writing to remote memory.\n");
+
+ int returnValue = ibv_post_send(this->ibvQueuePair, &workRequest, &badWorkRequest);
+
+ INFINITY_ASSERT(returnValue == 0, "[INFINITY][QUEUES][QUEUEPAIR] Posting write request failed. %s.\n", strerror(errno));
+
+ INFINITY_DEBUG("[INFINITY][QUEUES][QUEUEPAIR] Write request created (id %lu).\n", workRequest.wr_id);
+
+}
+
+void QueuePair::writeWithImmediate(infinity::memory::Buffer* buffer, uint64_t localOffset, infinity::memory::RegionToken* destination, uint64_t remoteOffset,
+ uint32_t sizeInBytes, uint32_t immediateValue, OperationFlags send_flags, infinity::requests::RequestToken* requestToken) {
+
+ if (requestToken != NULL) {
+ requestToken->reset();
+ requestToken->setRegion(buffer);
+ requestToken->setImmediateValue(immediateValue);
+ }
+
+ struct ibv_sge sgElement;
+ struct ibv_send_wr workRequest;
+ struct ibv_send_wr *badWorkRequest;
+
+ memset(&sgElement, 0, sizeof(ibv_sge));
+ sgElement.addr = buffer->getAddress() + localOffset;
+ sgElement.length = sizeInBytes;
+ sgElement.lkey = buffer->getLocalKey();
+
+ INFINITY_ASSERT(sizeInBytes <= buffer->getRemainingSizeInBytes(localOffset),
+ "[INFINITY][QUEUES][QUEUEPAIR] Segmentation fault while creating scatter-getter element.\n");
+
+ memset(&workRequest, 0, sizeof(ibv_send_wr));
+ workRequest.wr_id = reinterpret_cast(requestToken);
+ workRequest.sg_list = &sgElement;
+ workRequest.num_sge = 1;
+ workRequest.opcode = IBV_WR_RDMA_WRITE_WITH_IMM;
+ workRequest.imm_data = htonl(immediateValue);
+ workRequest.send_flags = send_flags.ibvFlags();
+ if (requestToken != NULL) {
+ workRequest.send_flags |= IBV_SEND_SIGNALED;
+ }
+ workRequest.wr.rdma.remote_addr = destination->getAddress() + remoteOffset;
+ workRequest.wr.rdma.rkey = destination->getRemoteKey();
+
+ INFINITY_ASSERT(sizeInBytes <= destination->getRemainingSizeInBytes(remoteOffset),
+ "[INFINITY][QUEUES][QUEUEPAIR] Segmentation fault while writing to remote memory.\n");
+
+ int returnValue = ibv_post_send(this->ibvQueuePair, &workRequest, &badWorkRequest);
+
+ INFINITY_ASSERT(returnValue == 0, "[INFINITY][QUEUES][QUEUEPAIR] Posting write request failed. %s.\n", strerror(errno));
+
+ INFINITY_DEBUG("[INFINITY][QUEUES][QUEUEPAIR] Write request created (id %lu).\n", workRequest.wr_id);
+
+}
+
+void QueuePair::multiWrite(infinity::memory::Buffer** buffers, uint32_t* sizesInBytes, uint64_t* localOffsets, uint32_t numberOfElements,
+ infinity::memory::RegionToken* destination, uint64_t remoteOffset, OperationFlags send_flags, infinity::requests::RequestToken* requestToken) {
+
+ if (requestToken != NULL) {
+ requestToken->reset();
+ requestToken->setRegion(buffers[0]);
+ }
+
+ struct ibv_sge *sgElements = (ibv_sge *) calloc(numberOfElements, sizeof(ibv_sge));
+ struct ibv_send_wr workRequest;
+ struct ibv_send_wr *badWorkRequest;
+
+ INFINITY_ASSERT(numberOfElements <= infinity::core::Configuration::MAX_NUMBER_OF_SGE_ELEMENTS, "[INFINITY][QUEUES][QUEUEPAIR] Request contains too many SGE.\n");
+
+ uint32_t totalSizeInBytes = 0;
+ for (uint32_t i = 0; i < numberOfElements; ++i) {
+ if (localOffsets != NULL) {
+ sgElements[i].addr = buffers[i]->getAddress() + localOffsets[i];
+ } else {
+ sgElements[i].addr = buffers[i]->getAddress();
+ }
+ if (sizesInBytes != NULL) {
+ sgElements[i].length = sizesInBytes[i];
+ } else {
+ sgElements[i].length = buffers[i]->getSizeInBytes();
+ }
+ totalSizeInBytes += sgElements[i].length;
+ sgElements[i].lkey = buffers[i]->getLocalKey();
+ }
+
+ memset(&workRequest, 0, sizeof(ibv_send_wr));
+ workRequest.wr_id = reinterpret_cast(requestToken);
+ workRequest.sg_list = sgElements;
+ workRequest.num_sge = numberOfElements;
+ workRequest.opcode = IBV_WR_RDMA_WRITE;
+ workRequest.send_flags = send_flags.ibvFlags();
+ if (requestToken != NULL) {
+ workRequest.send_flags |= IBV_SEND_SIGNALED;
+ }
+ workRequest.wr.rdma.remote_addr = destination->getAddress() + remoteOffset;
+ workRequest.wr.rdma.rkey = destination->getRemoteKey();
+
+ INFINITY_ASSERT(totalSizeInBytes <= destination->getRemainingSizeInBytes(remoteOffset),
+ "[INFINITY][QUEUES][QUEUEPAIR] Segmentation fault while writing to remote memory.\n");
+
+ int returnValue = ibv_post_send(this->ibvQueuePair, &workRequest, &badWorkRequest);
+
+ INFINITY_ASSERT(returnValue == 0, "[INFINITY][QUEUES][QUEUEPAIR] Posting write request failed. %s.\n", strerror(errno));
+
+ INFINITY_DEBUG("[INFINITY][QUEUES][QUEUEPAIR] Multi-Write request created (id %lu).\n", workRequest.wr_id);
+}
+
+void QueuePair::multiWriteWithImmediate(infinity::memory::Buffer** buffers, uint32_t* sizesInBytes, uint64_t* localOffsets, uint32_t numberOfElements,
+ infinity::memory::RegionToken* destination, uint64_t remoteOffset, uint32_t immediateValue, OperationFlags send_flags, infinity::requests::RequestToken* requestToken) {
+
+ if (requestToken != NULL) {
+ requestToken->reset();
+ requestToken->setRegion(buffers[0]);
+ requestToken->setImmediateValue(immediateValue);
+ }
+
+ struct ibv_sge *sgElements = (ibv_sge *) calloc(numberOfElements, sizeof(ibv_sge));
+ struct ibv_send_wr workRequest;
+ struct ibv_send_wr *badWorkRequest;
+
+ INFINITY_ASSERT(numberOfElements <= infinity::core::Configuration::MAX_NUMBER_OF_SGE_ELEMENTS, "[INFINITY][QUEUES][QUEUEPAIR] Request contains too many SGE.\n");
+
+ uint32_t totalSizeInBytes = 0;
+ for (uint32_t i = 0; i < numberOfElements; ++i) {
+ if (localOffsets != NULL) {
+ sgElements[i].addr = buffers[i]->getAddress() + localOffsets[i];
+ } else {
+ sgElements[i].addr = buffers[i]->getAddress();
+ }
+ if (sizesInBytes != NULL) {
+ sgElements[i].length = sizesInBytes[i];
+ } else {
+ sgElements[i].length = buffers[i]->getSizeInBytes();
+ }
+ totalSizeInBytes += sgElements[i].length;
+ sgElements[i].lkey = buffers[i]->getLocalKey();
+ }
+
+ memset(&workRequest, 0, sizeof(ibv_send_wr));
+ workRequest.wr_id = reinterpret_cast(requestToken);
+ workRequest.sg_list = sgElements;
+ workRequest.num_sge = numberOfElements;
+ workRequest.opcode = IBV_WR_RDMA_WRITE_WITH_IMM;
+ workRequest.imm_data = htonl(immediateValue);
+ workRequest.send_flags = send_flags.ibvFlags();
+ if (requestToken != NULL) {
+ workRequest.send_flags |= IBV_SEND_SIGNALED;
+ }
+ workRequest.wr.rdma.remote_addr = destination->getAddress() + remoteOffset;
+ workRequest.wr.rdma.rkey = destination->getRemoteKey();
+
+ INFINITY_ASSERT(totalSizeInBytes <= destination->getRemainingSizeInBytes(remoteOffset),
+ "[INFINITY][QUEUES][QUEUEPAIR] Segmentation fault while writing to remote memory.\n");
+
+ int returnValue = ibv_post_send(this->ibvQueuePair, &workRequest, &badWorkRequest);
+
+ INFINITY_ASSERT(returnValue == 0, "[INFINITY][QUEUES][QUEUEPAIR] Posting write request failed. %s.\n", strerror(errno));
+
+ INFINITY_DEBUG("[INFINITY][QUEUES][QUEUEPAIR] Multi-Write request created (id %lu).\n", workRequest.wr_id);
+
+}
+
+void QueuePair::read(infinity::memory::Buffer* buffer, infinity::memory::RegionToken* source, infinity::requests::RequestToken *requestToken) {
+ read(buffer, 0, source, 0, buffer->getSizeInBytes(), OperationFlags(), requestToken);
+ INFINITY_ASSERT(buffer->getSizeInBytes() <= ((uint64_t) UINT32_MAX), "[INFINITY][QUEUES][QUEUEPAIR] Request must be smaller or equal to UINT_32_MAX bytes. This memory region is larger. Please explicitly indicate the size of the data to transfer.\n");
+}
+
+void QueuePair::read(infinity::memory::Buffer* buffer, infinity::memory::RegionToken* source, uint32_t sizeInBytes,
+ infinity::requests::RequestToken *requestToken) {
+ read(buffer, 0, source, 0, sizeInBytes, OperationFlags(), requestToken);
+}
+
+void QueuePair::read(infinity::memory::Buffer* buffer, uint64_t localOffset, infinity::memory::RegionToken* source, uint64_t remoteOffset, uint32_t sizeInBytes,
+ OperationFlags send_flags, infinity::requests::RequestToken *requestToken) {
+
+ if (requestToken != NULL) {
+ requestToken->reset();
+ requestToken->setRegion(buffer);
+ }
+
+ struct ibv_sge sgElement;
+ struct ibv_send_wr workRequest;
+ struct ibv_send_wr *badWorkRequest;
+
+ memset(&sgElement, 0, sizeof(ibv_sge));
+ sgElement.addr = buffer->getAddress() + localOffset;
+ sgElement.length = sizeInBytes;
+ sgElement.lkey = buffer->getLocalKey();
+
+ INFINITY_ASSERT(sizeInBytes <= buffer->getRemainingSizeInBytes(localOffset),
+ "[INFINITY][QUEUES][QUEUEPAIR] Segmentation fault while creating scatter-getter element.\n");
+
+ memset(&workRequest, 0, sizeof(ibv_send_wr));
+ workRequest.wr_id = reinterpret_cast(requestToken);
+ workRequest.sg_list = &sgElement;
+ workRequest.num_sge = 1;
+ workRequest.opcode = IBV_WR_RDMA_READ;
+ workRequest.send_flags = send_flags.ibvFlags();
+ if (requestToken != NULL) {
+ workRequest.send_flags |= IBV_SEND_SIGNALED;
+ }
+ workRequest.wr.rdma.remote_addr = source->getAddress() + remoteOffset;
+ workRequest.wr.rdma.rkey = source->getRemoteKey();
+
+ INFINITY_ASSERT(sizeInBytes <= source->getRemainingSizeInBytes(remoteOffset),
+ "[INFINITY][QUEUES][QUEUEPAIR] Segmentation fault while reading from remote memory.\n");
+
+ int returnValue = ibv_post_send(this->ibvQueuePair, &workRequest, &badWorkRequest);
+
+ INFINITY_ASSERT(returnValue == 0, "[INFINITY][QUEUES][QUEUEPAIR] Posting read request failed. %s.\n", strerror(errno));
+
+ INFINITY_DEBUG("[INFINITY][QUEUES][QUEUEPAIR] Read request created (id %lu).\n", workRequest.wr_id);
+
+}
+
+void QueuePair::compareAndSwap(infinity::memory::RegionToken* destination, infinity::memory::Atomic* previousValue, uint64_t compare, uint64_t swap,
+ OperationFlags send_flags, infinity::requests::RequestToken *requestToken) {
+
+ if (requestToken != NULL) {
+ requestToken->reset();
+ requestToken->setRegion(previousValue);
+ }
+
+ struct ibv_sge sgElement;
+ struct ibv_send_wr workRequest;
+ struct ibv_send_wr *badWorkRequest;
+
+ memset(&sgElement, 0, sizeof(ibv_sge));
+ sgElement.addr = previousValue->getAddress();
+ sgElement.length = previousValue->getSizeInBytes();
+ sgElement.lkey = previousValue->getLocalKey();
+
+ memset(&workRequest, 0, sizeof(ibv_send_wr));
+ workRequest.wr_id = reinterpret_cast(requestToken);
+ workRequest.sg_list = &sgElement;
+ workRequest.num_sge = 1;
+ workRequest.opcode = IBV_WR_ATOMIC_CMP_AND_SWP;
+ workRequest.send_flags = send_flags.ibvFlags();
+ if (requestToken != NULL) {
+ workRequest.send_flags |= IBV_SEND_SIGNALED;
+ }
+ workRequest.wr.atomic.remote_addr = destination->getAddress();
+ workRequest.wr.atomic.rkey = destination->getRemoteKey();
+ workRequest.wr.atomic.compare_add = compare;
+ workRequest.wr.atomic.swap = swap;
+
+ int returnValue = ibv_post_send(this->ibvQueuePair, &workRequest, &badWorkRequest);
+
+ INFINITY_ASSERT(returnValue == 0, "[INFINITY][QUEUES][QUEUEPAIR] Posting cmp-and-swp request failed. %s.\n", strerror(errno));
+
+ INFINITY_DEBUG("[INFINITY][QUEUES][QUEUEPAIR] Cmp-and-swp request created (id %lu).\n", workRequest.wr_id);
+
+}
+
+void QueuePair::compareAndSwap(infinity::memory::RegionToken* destination, uint64_t compare, uint64_t swap, infinity::requests::RequestToken *requestToken) {
+ compareAndSwap(destination, context->defaultAtomic, compare, swap, OperationFlags(), requestToken);
+}
+
+void QueuePair::fetchAndAdd(infinity::memory::RegionToken* destination, uint64_t add, infinity::requests::RequestToken *requestToken) {
+ fetchAndAdd(destination, context->defaultAtomic, add, OperationFlags(), requestToken);
+}
+
+void QueuePair::fetchAndAdd(infinity::memory::RegionToken* destination, infinity::memory::Atomic* previousValue, uint64_t add,
+ OperationFlags send_flags, infinity::requests::RequestToken *requestToken) {
+
+ if (requestToken != NULL) {
+ requestToken->reset();
+ requestToken->setRegion(previousValue);
+ }
+
+ struct ibv_sge sgElement;
+ struct ibv_send_wr workRequest;
+ struct ibv_send_wr *badWorkRequest;
+
+ memset(&sgElement, 0, sizeof(ibv_sge));
+ sgElement.addr = previousValue->getAddress();
+ sgElement.length = previousValue->getSizeInBytes();
+ sgElement.lkey = previousValue->getLocalKey();
+
+ memset(&workRequest, 0, sizeof(ibv_send_wr));
+ workRequest.wr_id = reinterpret_cast(requestToken);
+ workRequest.sg_list = &sgElement;
+ workRequest.num_sge = 1;
+ workRequest.opcode = IBV_WR_ATOMIC_FETCH_AND_ADD;
+ workRequest.send_flags = send_flags.ibvFlags();
+ if (requestToken != NULL) {
+ workRequest.send_flags |= IBV_SEND_SIGNALED;
+ }
+ workRequest.wr.atomic.remote_addr = destination->getAddress();
+ workRequest.wr.atomic.rkey = destination->getRemoteKey();
+ workRequest.wr.atomic.compare_add = add;
+
+ int returnValue = ibv_post_send(this->ibvQueuePair, &workRequest, &badWorkRequest);
+
+ INFINITY_ASSERT(returnValue == 0, "[INFINITY][QUEUES][QUEUEPAIR] Posting fetch-add request failed. %s.\n", strerror(errno));
+
+ INFINITY_DEBUG("[INFINITY][QUEUES][QUEUEPAIR] Fetch-add request created (id %lu).\n", workRequest.wr_id);
+
+}
+
+
+
+bool QueuePair::hasUserData() {
+ return (this->userData != NULL && this->userDataSize != 0);
+}
+
+uint32_t QueuePair::getUserDataSize() {
+ return this->userDataSize;
+}
+
+void* QueuePair::getUserData() {
+ return this->userData;
+}
+
+} /* namespace queues */
+} /* namespace infinity */
diff --git a/src/RDMA/infinity/queues/QueuePair.h b/src/RDMA/infinity/queues/QueuePair.h
new file mode 100644
index 0000000..ff6b148
--- /dev/null
+++ b/src/RDMA/infinity/queues/QueuePair.h
@@ -0,0 +1,157 @@
+/**
+ * Queues - Queue Pair
+ *
+ * (c) 2018 Claude Barthels, ETH Zurich
+ * Contact: claudeb@inf.ethz.ch
+ *
+ */
+
+#ifndef QUEUES_QUEUEPAIR_H_
+#define QUEUES_QUEUEPAIR_H_
+
+#include
+#include
+#include
+#include
+#include
+#include
+
+namespace infinity {
+namespace queues {
+class QueuePairFactory;
+}
+}
+
+namespace infinity {
+namespace queues {
+
+class OperationFlags {
+
+public:
+ bool fenced;
+ bool signaled;
+ bool inlined;
+
+ OperationFlags() : fenced(false), signaled(false), inlined(false) { };
+
+ /**
+ * Turn the bools into a bit field.
+ */
+ int ibvFlags();
+};
+
+class QueuePair {
+
+ friend class infinity::queues::QueuePairFactory;
+
+public:
+
+ /**
+ * Constructor
+ */
+ QueuePair(infinity::core::Context *context);
+
+ /**
+ * Destructor
+ */
+ ~QueuePair();
+
+protected:
+
+ /**
+ * Activation methods
+ */
+
+ void activate(uint16_t remoteDeviceId, uint32_t remoteQueuePairNumber, uint32_t remoteSequenceNumber);
+ void setRemoteUserData(void *userData, uint32_t userDataSize);
+
+public:
+
+ /**
+ * User data received during connection setup
+ */
+
+ bool hasUserData();
+ uint32_t getUserDataSize();
+ void * getUserData();
+
+public:
+
+ /**
+ * Queue pair information
+ */
+
+ uint16_t getLocalDeviceId();
+ uint32_t getQueuePairNumber();
+ uint32_t getSequenceNumber();
+
+public:
+
+ /**
+ * Buffer operations
+ */
+
+ void send(infinity::memory::Buffer *buffer, infinity::requests::RequestToken *requestToken = NULL);
+ void send(infinity::memory::Buffer *buffer, uint32_t sizeInBytes, infinity::requests::RequestToken *requestToken = NULL);
+ void send(infinity::memory::Buffer *buffer, uint64_t localOffset, uint32_t sizeInBytes, OperationFlags flags,
+ infinity::requests::RequestToken *requestToken = NULL);
+
+ void write(infinity::memory::Buffer *buffer, infinity::memory::RegionToken *destination, infinity::requests::RequestToken *requestToken = NULL);
+ void write(infinity::memory::Buffer *buffer, infinity::memory::RegionToken *destination, uint32_t sizeInBytes,
+ infinity::requests::RequestToken *requestToken = NULL);
+ void write(infinity::memory::Buffer *buffer, uint64_t localOffset, infinity::memory::RegionToken *destination, uint64_t remoteOffset, uint32_t sizeInBytes,
+ OperationFlags flags, infinity::requests::RequestToken *requestToken = NULL);
+
+ void read(infinity::memory::Buffer *buffer, infinity::memory::RegionToken *source, infinity::requests::RequestToken *requestToken = NULL);
+ void read(infinity::memory::Buffer *buffer, infinity::memory::RegionToken *source, uint32_t sizeInBytes, infinity::requests::RequestToken *requestToken =
+ NULL);
+ void read(infinity::memory::Buffer *buffer, uint64_t localOffset, infinity::memory::RegionToken *source, uint64_t remoteOffset, uint32_t sizeInBytes,
+ OperationFlags flags, infinity::requests::RequestToken *requestToken = NULL);
+
+public:
+
+ /**
+ * Complex buffer operations
+ */
+
+ void multiWrite(infinity::memory::Buffer **buffers, uint32_t *sizesInBytes, uint64_t *localOffsets, uint32_t numberOfElements,
+ infinity::memory::RegionToken *destination, uint64_t remoteOffset, OperationFlags flags, infinity::requests::RequestToken *requestToken = NULL);
+
+ void sendWithImmediate(infinity::memory::Buffer *buffer, uint64_t localOffset, uint32_t sizeInBytes, uint32_t immediateValue,
+ OperationFlags flags, infinity::requests::RequestToken *requestToken = NULL);
+
+ void writeWithImmediate(infinity::memory::Buffer *buffer, uint64_t localOffset, infinity::memory::RegionToken *destination, uint64_t remoteOffset,
+ uint32_t sizeInBytes, uint32_t immediateValue, OperationFlags flags, infinity::requests::RequestToken *requestToken = NULL);
+
+ void multiWriteWithImmediate(infinity::memory::Buffer **buffers, uint32_t *sizesInBytes, uint64_t *localOffsets, uint32_t numberOfElements,
+ infinity::memory::RegionToken *destination, uint64_t remoteOffset, uint32_t immediateValue, OperationFlags flags, infinity::requests::RequestToken *requestToken = NULL);
+
+public:
+
+ /**
+ * Atomic value operations
+ */
+
+ void compareAndSwap(infinity::memory::RegionToken *destination, uint64_t compare, uint64_t swap, infinity::requests::RequestToken *requestToken = NULL);
+ void compareAndSwap(infinity::memory::RegionToken *destination, infinity::memory::Atomic *previousValue, uint64_t compare, uint64_t swap,
+ OperationFlags flags, infinity::requests::RequestToken *requestToken = NULL);
+ void fetchAndAdd(infinity::memory::RegionToken *destination, uint64_t add, infinity::requests::RequestToken *requestToken = NULL);
+ void fetchAndAdd(infinity::memory::RegionToken *destination, infinity::memory::Atomic *previousValue, uint64_t add,
+ OperationFlags flags, infinity::requests::RequestToken *requestToken = NULL);
+
+protected:
+
+ infinity::core::Context * const context;
+
+ ibv_qp* ibvQueuePair;
+ uint32_t sequenceNumber;
+
+ void *userData;
+ uint32_t userDataSize;
+
+};
+
+} /* namespace queues */
+} /* namespace infinity */
+
+#endif /* QUEUES_QUEUEPAIR_H_ */
diff --git a/src/RDMA/infinity/queues/QueuePairFactory.cpp b/src/RDMA/infinity/queues/QueuePairFactory.cpp
new file mode 100644
index 0000000..3ef04e0
--- /dev/null
+++ b/src/RDMA/infinity/queues/QueuePairFactory.cpp
@@ -0,0 +1,183 @@
+/**
+ * Queues - Queue Pair Factory
+ *
+ * (c) 2018 Claude Barthels, ETH Zurich
+ * Contact: claudeb@inf.ethz.ch
+ *
+ */
+
+#include "QueuePairFactory.h"
+
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+
+namespace infinity {
+namespace queues {
+
+typedef struct {
+
+ uint16_t localDeviceId;
+ uint32_t queuePairNumber;
+ uint32_t sequenceNumber;
+ uint32_t userDataSize;
+ char userData[infinity::core::Configuration::MAX_CONNECTION_USER_DATA_SIZE];
+
+} serializedQueuePair;
+
+QueuePairFactory::QueuePairFactory(infinity::core::Context *context) {
+
+ this->context = context;
+ this->serverSocket = -1;
+
+}
+
+QueuePairFactory::~QueuePairFactory() {
+
+ if (serverSocket >= 0) {
+ close(serverSocket);
+ }
+
+}
+
+void QueuePairFactory::bindToPort(uint16_t port) {
+
+ serverSocket = socket(AF_INET, SOCK_STREAM, 0);
+ INFINITY_ASSERT(serverSocket >= 0, "[INFINITY][QUEUES][FACTORY] Cannot open server socket.\n");
+
+ sockaddr_in serverAddress;
+ memset(&(serverAddress), 0, sizeof(sockaddr_in));
+ serverAddress.sin_family = AF_INET;
+ serverAddress.sin_port = htons(port);
+
+ int32_t enabled = 1;
+ int32_t returnValue = setsockopt(serverSocket, SOL_SOCKET, SO_REUSEADDR, &enabled, sizeof(enabled));
+ INFINITY_ASSERT(returnValue == 0, "[INFINITY][QUEUES][FACTORY] Cannot set socket option to reuse address.\n");
+
+ returnValue = bind(serverSocket, (sockaddr *) &serverAddress, sizeof(sockaddr_in));
+ INFINITY_ASSERT(returnValue == 0, "[INFINITY][QUEUES][FACTORY] Cannot bind to local address and port.\n");
+
+ returnValue = listen(serverSocket, 128);
+ INFINITY_ASSERT(returnValue == 0, "[INFINITY][QUEUES][FACTORY] Cannot listen on server socket.\n");
+
+ char *ipAddressOfDevice = infinity::utils::Address::getIpAddressOfInterface(infinity::core::Configuration::DEFAULT_IB_DEVICE);
+ INFINITY_DEBUG("[INFINITY][QUEUES][FACTORY] Accepting connections on IP address %s and port %d.\n", ipAddressOfDevice, port);
+ free(ipAddressOfDevice);
+
+}
+
+QueuePair * QueuePairFactory::acceptIncomingConnection(void *userData, uint32_t userDataSizeInBytes) {
+
+ INFINITY_ASSERT(userDataSizeInBytes < infinity::core::Configuration::MAX_CONNECTION_USER_DATA_SIZE,
+ "[INFINITY][QUEUES][FACTORY] User data size is too large.\n")
+
+ serializedQueuePair *receiveBuffer = (serializedQueuePair*) calloc(1, sizeof(serializedQueuePair));
+ serializedQueuePair *sendBuffer = (serializedQueuePair*) calloc(1, sizeof(serializedQueuePair));
+
+ int connectionSocket = accept(this->serverSocket, (sockaddr *) NULL, NULL);
+ INFINITY_ASSERT(connectionSocket >= 0, "[INFINITY][QUEUES][FACTORY] Cannot open connection socket.\n");
+
+ int32_t returnValue = recv(connectionSocket, receiveBuffer, sizeof(serializedQueuePair), 0);
+ INFINITY_ASSERT(returnValue == sizeof(serializedQueuePair), "[INFINITY][QUEUES][FACTORY] Incorrect number of bytes received. Expected %lu. Received %d.\n",
+ sizeof(serializedQueuePair), returnValue);
+
+ QueuePair *queuePair = new QueuePair(this->context);
+
+ sendBuffer->localDeviceId = queuePair->getLocalDeviceId();
+ sendBuffer->queuePairNumber = queuePair->getQueuePairNumber();
+ sendBuffer->sequenceNumber = queuePair->getSequenceNumber();
+ sendBuffer->userDataSize = userDataSizeInBytes;
+ memcpy(sendBuffer->userData, userData, userDataSizeInBytes);
+
+ returnValue = send(connectionSocket, sendBuffer, sizeof(serializedQueuePair), 0);
+ INFINITY_ASSERT(returnValue == sizeof(serializedQueuePair),
+ "[INFINITY][QUEUES][FACTORY] Incorrect number of bytes transmitted. Expected %lu. Received %d.\n", sizeof(serializedQueuePair), returnValue);
+
+ INFINITY_DEBUG("[INFINITY][QUEUES][FACTORY] Pairing (%u, %u, %u, %u)-(%u, %u, %u, %u)\n", queuePair->getLocalDeviceId(), queuePair->getQueuePairNumber(),
+ queuePair->getSequenceNumber(), userDataSizeInBytes, receiveBuffer->localDeviceId, receiveBuffer->queuePairNumber, receiveBuffer->sequenceNumber,
+ receiveBuffer->userDataSize);
+
+ queuePair->activate(receiveBuffer->localDeviceId, receiveBuffer->queuePairNumber, receiveBuffer->sequenceNumber);
+ queuePair->setRemoteUserData(receiveBuffer->userData, receiveBuffer->userDataSize);
+
+ this->context->registerQueuePair(queuePair);
+
+ close(connectionSocket);
+ free(receiveBuffer);
+ free(sendBuffer);
+
+ return queuePair;
+
+}
+
+QueuePair * QueuePairFactory::connectToRemoteHost(const char* hostAddress, uint16_t port, void *userData, uint32_t userDataSizeInBytes) {
+
+ INFINITY_ASSERT(userDataSizeInBytes < infinity::core::Configuration::MAX_CONNECTION_USER_DATA_SIZE,
+ "[INFINITY][QUEUES][FACTORY] User data size is too large.\n")
+
+ serializedQueuePair *receiveBuffer = (serializedQueuePair*) calloc(1, sizeof(serializedQueuePair));
+ serializedQueuePair *sendBuffer = (serializedQueuePair*) calloc(1, sizeof(serializedQueuePair));
+
+ sockaddr_in remoteAddress;
+ memset(&(remoteAddress), 0, sizeof(sockaddr_in));
+ remoteAddress.sin_family = AF_INET;
+ inet_pton(AF_INET, hostAddress, &(remoteAddress.sin_addr));
+ remoteAddress.sin_port = htons(port);
+
+ int connectionSocket = socket(AF_INET, SOCK_STREAM, 0);
+ INFINITY_ASSERT(connectionSocket >= 0, "[INFINITY][QUEUES][FACTORY] Cannot open connection socket.\n");
+
+ int returnValue = connect(connectionSocket, (sockaddr *) &(remoteAddress), sizeof(sockaddr_in));
+ INFINITY_ASSERT(returnValue == 0, "[INFINITY][QUEUES][FACTORY] Could not connect to server.\n");
+
+ QueuePair *queuePair = new QueuePair(this->context);
+
+ sendBuffer->localDeviceId = queuePair->getLocalDeviceId();
+ sendBuffer->queuePairNumber = queuePair->getQueuePairNumber();
+ sendBuffer->sequenceNumber = queuePair->getSequenceNumber();
+ sendBuffer->userDataSize = userDataSizeInBytes;
+ memcpy(sendBuffer->userData, userData, userDataSizeInBytes);
+
+ returnValue = send(connectionSocket, sendBuffer, sizeof(serializedQueuePair), 0);
+ INFINITY_ASSERT(returnValue == sizeof(serializedQueuePair),
+ "[INFINITY][QUEUES][FACTORY] Incorrect number of bytes transmitted. Expected %lu. Received %d.\n", sizeof(serializedQueuePair), returnValue);
+
+ returnValue = recv(connectionSocket, receiveBuffer, sizeof(serializedQueuePair), 0);
+ INFINITY_ASSERT(returnValue == sizeof(serializedQueuePair),
+ "[INFINITY][QUEUES][FACTORY] Incorrect number of bytes received. Expected %lu. Received %d.\n", sizeof(serializedQueuePair), returnValue);
+
+ INFINITY_DEBUG("[INFINITY][QUEUES][FACTORY] Pairing (%u, %u, %u, %u)-(%u, %u, %u, %u)\n", queuePair->getLocalDeviceId(), queuePair->getQueuePairNumber(),
+ queuePair->getSequenceNumber(), userDataSizeInBytes, receiveBuffer->localDeviceId, receiveBuffer->queuePairNumber, receiveBuffer->sequenceNumber,
+ receiveBuffer->userDataSize);
+
+ queuePair->activate(receiveBuffer->localDeviceId, receiveBuffer->queuePairNumber, receiveBuffer->sequenceNumber);
+ queuePair->setRemoteUserData(receiveBuffer->userData, receiveBuffer->userDataSize);
+
+ this->context->registerQueuePair(queuePair);
+
+ close(connectionSocket);
+ free(receiveBuffer);
+ free(sendBuffer);
+
+ return queuePair;
+
+}
+
+QueuePair* QueuePairFactory::createLoopback(void *userData, uint32_t userDataSizeInBytes) {
+
+ QueuePair *queuePair = new QueuePair(this->context);
+ queuePair->activate(queuePair->getLocalDeviceId(), queuePair->getQueuePairNumber(), queuePair->getSequenceNumber());
+ queuePair->setRemoteUserData(userData, userDataSizeInBytes);
+
+ this->context->registerQueuePair(queuePair);
+
+ return queuePair;
+
+}
+
+} /* namespace queues */
+} /* namespace infinity */
diff --git a/src/RDMA/infinity/queues/QueuePairFactory.h b/src/RDMA/infinity/queues/QueuePairFactory.h
new file mode 100644
index 0000000..42eb226
--- /dev/null
+++ b/src/RDMA/infinity/queues/QueuePairFactory.h
@@ -0,0 +1,57 @@
+/**
+ * Queues - Queue Pair Factory
+ *
+ * (c) 2018 Claude Barthels, ETH Zurich
+ * Contact: claudeb@inf.ethz.ch
+ *
+ */
+
+#ifndef QUEUES_QUEUEPAIRFACTORY_H_
+#define QUEUES_QUEUEPAIRFACTORY_H_
+
+#include
+#include
+#include
+#include
+
+namespace infinity {
+namespace queues {
+
+class QueuePairFactory {
+public:
+
+ QueuePairFactory(infinity::core::Context *context);
+ ~QueuePairFactory();
+
+ /**
+ * Bind to port for listening to incoming connections
+ */
+ void bindToPort(uint16_t port);
+
+ /**
+ * Accept incoming connection request (passive side)
+ */
+ QueuePair * acceptIncomingConnection(void *userData = NULL, uint32_t userDataSizeInBytes = 0);
+
+ /**
+ * Connect to remote machine (active side)
+ */
+ QueuePair * connectToRemoteHost(const char* hostAddress, uint16_t port, void *userData = NULL, uint32_t userDataSizeInBytes = 0);
+
+ /**
+ * Create loopback queue pair
+ */
+ QueuePair * createLoopback(void *userData = NULL, uint32_t userDataSizeInBytes = 0);
+
+protected:
+
+ infinity::core::Context * context;
+
+ int32_t serverSocket;
+
+};
+
+} /* namespace queues */
+} /* namespace infinity */
+
+#endif /* QUEUES_QUEUEPAIRFACTORY_H_ */
diff --git a/src/RDMA/infinity/requests/RequestToken.cpp b/src/RDMA/infinity/requests/RequestToken.cpp
new file mode 100644
index 0000000..52f446e
--- /dev/null
+++ b/src/RDMA/infinity/requests/RequestToken.cpp
@@ -0,0 +1,101 @@
+/**
+ * Requests - Request Token
+ *
+ * (c) 2018 Claude Barthels, ETH Zurich
+ * Contact: claudeb@inf.ethz.ch
+ *
+ */
+
+#include "RequestToken.h"
+
+namespace infinity {
+namespace requests {
+
+RequestToken::RequestToken(infinity::core::Context *context) :
+ context(context) {
+ this->success.store(false);
+ this->completed.store(false);
+ this->region = NULL;
+ this->userData = NULL;
+ this->userDataValid = false;
+ this->userDataSize = 0;
+ this->immediateValue = 0;
+ this->immediateValueValid = false;
+}
+
+void RequestToken::setCompleted(bool success) {
+ this->success.store(success);
+ this->completed.store(true);
+}
+
+bool RequestToken::checkIfCompleted() {
+ if (this->completed.load()) {
+ return true;
+ } else {
+ this->context->pollSendCompletionQueue();
+ return this->completed.load();
+ }
+}
+
+void RequestToken::waitUntilCompleted() {
+ while (!this->completed.load()) {
+ this->context->pollSendCompletionQueue();
+ }
+}
+
+bool RequestToken::wasSuccessful() {
+ return this->success.load();
+}
+
+void RequestToken::reset() {
+ this->success.store(false);
+ this->completed.store(false);
+ this->region = NULL;
+ this->userData = NULL;
+ this->userDataValid = false;
+ this->userDataSize = 0;
+ this->immediateValue = 0;
+ this->immediateValueValid = false;
+}
+
+void RequestToken::setRegion(infinity::memory::Region* region) {
+ this->region = region;
+}
+
+infinity::memory::Region* RequestToken::getRegion() {
+ return this->region;
+}
+
+void RequestToken::setUserData(void* userData, uint32_t userDataSize) {
+ this->userData = userData;
+ this->userDataSize = userDataSize;
+ this->userDataValid = true;
+}
+
+void* RequestToken::getUserData() {
+ return this->userData;
+}
+
+bool RequestToken::hasUserData() {
+ return this->userDataValid;
+}
+
+uint32_t RequestToken::getUserDataSize() {
+ return this->userDataSize;
+}
+
+void RequestToken::setImmediateValue(uint32_t immediateValue) {
+ this->immediateValue = immediateValue;
+ this->immediateValueValid = true;
+}
+
+uint32_t RequestToken::getImmediateValue() {
+ return this->immediateValue;
+}
+
+bool RequestToken::hasImmediateValue() {
+ return this->immediateValueValid;
+}
+
+} /* namespace requests */
+} /* namespace infinity */
diff --git a/src/RDMA/infinity/requests/RequestToken.h b/src/RDMA/infinity/requests/RequestToken.h
new file mode 100644
index 0000000..1aefb6d
--- /dev/null
+++ b/src/RDMA/infinity/requests/RequestToken.h
@@ -0,0 +1,67 @@
+/**
+ * Requests - Request Token
+ *
+ * (c) 2018 Claude Barthels, ETH Zurich
+ * Contact: claudeb@inf.ethz.ch
+ *
+ */
+
+#ifndef REQUESTS_REQUESTTOKEN_H_
+#define REQUESTS_REQUESTTOKEN_H_
+
+#include
+#include
+#include
+
+#include
+
+namespace infinity {
+namespace requests {
+
+class RequestToken {
+
+public:
+
+ RequestToken(infinity::core::Context *context);
+
+ void reset();
+
+ void setRegion(infinity::memory::Region * region);
+ infinity::memory::Region * getRegion();
+
+ void setCompleted(bool success);
+ bool wasSuccessful();
+
+ bool checkIfCompleted();
+ void waitUntilCompleted();
+
+ void setImmediateValue(uint32_t immediateValue);
+ bool hasImmediateValue();
+ uint32_t getImmediateValue();
+
+ void setUserData(void* userData, uint32_t userDataSize);
+ bool hasUserData();
+ void* getUserData();
+ uint32_t getUserDataSize();
+
+protected:
+
+ infinity::core::Context * const context;
+ infinity::memory::Region * region;
+
+ std::atomic completed;
+ std::atomic success;
+
+ void *userData;
+ uint32_t userDataSize;
+ bool userDataValid;
+
+ uint32_t immediateValue;
+ bool immediateValueValid;
+
+};
+
+} /* namespace requests */
+} /* namespace infinity */
+
+#endif /* REQUESTS_REQUESTTOKEN_H_ */
diff --git a/src/RDMA/infinity/utils/Address.cpp b/src/RDMA/infinity/utils/Address.cpp
new file mode 100644
index 0000000..0dbe56e
--- /dev/null
+++ b/src/RDMA/infinity/utils/Address.cpp
@@ -0,0 +1,57 @@
+/**
+ * Utils - Address
+ *
+ * (c) 2018 Claude Barthels, ETH Zurich
+ * Contact: claudeb@inf.ethz.ch
+ *
+ */
+
+#include "Address.h"
+
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+
+namespace infinity {
+namespace utils {
+
+char* Address::getIpAddressOfInterface(const char* interfaceName) {
+
+ struct ifaddrs *ifAddr;
+ struct ifaddrs *ifa;
+ char *ipAddress = (char*) calloc(16, sizeof(char));
+
+ int returnValue = getifaddrs(&ifAddr);
+ INFINITY_ASSERT(returnValue != -1, "[INFINITY][UTILS][ADDRESS] Cannot read interface list.\n");
+
+ for (ifa = ifAddr; ifa != NULL; ifa = ifa->ifa_next) {
+ if (ifa->ifa_addr == NULL) {
+ continue;
+ }
+ if ((ifa->ifa_addr->sa_family == AF_INET) && (strcasecmp(interfaceName, ifa->ifa_name) == 0)) {
+ sprintf(ipAddress, "%s", inet_ntoa(((struct sockaddr_in *) ifa->ifa_addr)->sin_addr));
+ break;
+ }
+ }
+ INFINITY_ASSERT(ifa != NULL, "[INFINITY][UTILS][ADDRESS] Cannot find interface named %s.\n", interfaceName);
+
+ freeifaddrs(ifAddr);
+
+ return ipAddress;
+
+}
+
+uint32_t Address::getIpAddressAsUint32(const char* ipAddress) {
+
+ uint32_t ipAddressNumbers[4];
+ sscanf(ipAddress, "%d.%d.%d.%d", &ipAddressNumbers[3], &ipAddressNumbers[2], &ipAddressNumbers[1], &ipAddressNumbers[0]);
+ uint32_t ipAddressNumber(ipAddressNumbers[0] | ipAddressNumbers[1] << 8 | ipAddressNumbers[2] << 16 | ipAddressNumbers[3] << 24);
+ return ipAddressNumber;
+}
+
+} /* namespace utils */
+} /* namespace infinity */
diff --git a/src/RDMA/infinity/utils/Address.h b/src/RDMA/infinity/utils/Address.h
new file mode 100644
index 0000000..4d4df0c
--- /dev/null
+++ b/src/RDMA/infinity/utils/Address.h
@@ -0,0 +1,29 @@
+/**
+ * Utils - Address
+ *
+ * (c) 2018 Claude Barthels, ETH Zurich
+ * Contact: claudeb@inf.ethz.ch
+ *
+ */
+
+#ifndef UTILS_ADDRESS_H_
+#define UTILS_ADDRESS_H_
+
+#include
+
+namespace infinity {
+namespace utils {
+
+class Address {
+
+public:
+
+ static char * getIpAddressOfInterface(const char *interfaceName);
+ static uint32_t getIpAddressAsUint32(const char *ipAddress);
+
+};
+
+} /* namespace utils */
+} /* namespace infinity */
+
+#endif /* UTILS_ADDRESS_H_ */
diff --git a/src/RDMA/infinity/utils/Debug.h b/src/RDMA/infinity/utils/Debug.h
new file mode 100644
index 0000000..d18c86f
--- /dev/null
+++ b/src/RDMA/infinity/utils/Debug.h
@@ -0,0 +1,27 @@
+/**
+ * Utils - Debug
+ *
+ * (c) 2018 Claude Barthels, ETH Zurich
+ * Contact: claudeb@inf.ethz.ch
+ *
+ */
+
+#ifndef UTILS_DEBUG_H_
+#define UTILS_DEBUG_H_
+
+#include
+#include
+
+#ifdef INFINITY_DEBUG_ON
+ #define INFINITY_DEBUG(X, ...) {fprintf(stdout, X, ##__VA_ARGS__); fflush(stdout);}
+#else
+ #define INFINITY_DEBUG(X, ...) {}
+#endif
+
+#ifdef INFINITY_ASSERT_ON
+ #define INFINITY_ASSERT(B, X, ...) {if(!(B)) {fprintf(stdout, X, ##__VA_ARGS__); fflush(stdout); exit(-1);}}
+#else
+ #define INFINITY_ASSERT(B, X, ...) {}
+#endif
+
+#endif /* UTILS_DEBUG_H_ */
diff --git a/src/buffers/CircularQueryBuffer.h b/src/buffers/CircularQueryBuffer.h
index 372c7e8..8435a85 100644
--- a/src/buffers/CircularQueryBuffer.h
+++ b/src/buffers/CircularQueryBuffer.h
@@ -1,15 +1,16 @@
#pragma once
#include
-#include
+#include
#include
#include
-#include
-#include
+#include
-#include "QueryBuffer.h"
+#include "buffers/QueryBuffer.h"
+#include "buffers/UnboundedQueryBufferFactory.h"
#include "utils/PaddedLong.h"
#include "utils/SystemConf.h"
+#include "utils/Utils.h"
/*
* \brief This class implements a non-NUMA-aware circular buffer.
@@ -21,44 +22,212 @@ class CircularQueryBuffer : public QueryBuffer {
ByteBuffer m_buffer;
public:
- CircularQueryBuffer(int id, size_t capacity, int tupleSize = 1, bool copyDataOnInsert = true) :
- QueryBuffer(id, capacity, false, tupleSize, copyDataOnInsert), m_buffer(m_capacity) {};
+ CircularQueryBuffer(int id, size_t capacity, int tupleSize = 1,
+ bool copyDataOnInsert = true, size_t batchSize = SystemConf::getInstance().BATCH_SIZE, bool clearFiles = true)
+ : QueryBuffer(id, capacity, false, tupleSize, copyDataOnInsert, batchSize, nullptr, clearFiles),
+ m_buffer(m_capacity){
+ if (SystemConf::getInstance().LINEAGE_ON) {
+ setupForCheckpoints(nullptr);
+ }
+ };
- long put(char *values, long bytes, long latencyMark = -1) override {
+
+ long put(char *values, long bytes, long latencyMark, long retainMark = -1, std::shared_ptr graph = nullptr) override {
if (values == nullptr || bytes <= 0)
- throw std::invalid_argument("error: cannot put null to circular buffer");
+ throw std::invalid_argument("error: cannot put null to circular buffer " + std::to_string(m_id));
- /* Get the end pointer */
+ /* Get the end pointer */
long end = m_endP.m_value.load(std::memory_order_relaxed);
+
/* Find remaining bytes until the circular buffer wraps */
long wrapPoint = (end + bytes - 1) - m_capacity;
if (m_temp.m_value <= wrapPoint) {
+
m_temp.m_value = m_startP.m_value.load(std::memory_order_relaxed);
if (m_temp.m_value <= wrapPoint) {
+ //std::cout << "[DBG] Circular Buffer " << std::to_string(m_id)
+ // << " is full with startP "
+ // << m_startP.m_value.load() << " and endP "
+ // << m_endP.m_value.load() << std::endl;
return -1;
}
}
long index = normalise(end);
- if (m_copyDataOnInsert || m_wraps == 0) { // copy only until the buffer is filled once
- if (bytes > ((long) m_capacity - index)) { /* Copy in two parts */
+
+ if (SystemConf::getInstance().LINEAGE_ON) {
+ auto firstSlotId = index / m_batchSize;
+ auto endSlotId = (index + bytes) / m_batchSize;
+ auto slotId = endSlotId; //firstSlotId;
+ while (slotId <= endSlotId) {
+ auto normSlot = slotId % m_numberOfSlots;
+ if (slotId != endSlotId) {
+ std::lock_guard l (m_slots[normSlot].m_updateLock);
+ if (!m_slots[normSlot].m_graph) {
+ auto newGraph = LineageGraphFactory::getInstance().newInstance();
+ m_slots[normSlot].setLineageGraph(newGraph);
+ }
+ } else {
+ std::lock_guard l (m_slots[normSlot].m_updateLock);
+ if (!graph && !m_slots[normSlot].m_graph) {
+ graph = LineageGraphFactory::getInstance().newInstance();
+ }
+ if (m_slots[normSlot].m_graph && graph) {
+ m_slots[normSlot].m_graph->mergeGraphs(graph);
+ } else if (graph) {
+ m_slots[normSlot].setLineageGraph(graph);
+ }
+ }
+ /*if (!m_slots[normSlot].m_graph) {
+ throw std::runtime_error(
+ "error: the lineage graph is not initialized when inserting for slot " +
+ std::to_string(normSlot));
+ }*/
+ slotId ++;
+ }
+ }
+
+ if (m_copyDataOnInsert ||
+ m_wraps == 0) { // copy only until the buffer is filled once
+ if (bytes > ((long)m_capacity - index)) { /* Copy in two parts */
long right = m_capacity - index;
long left = bytes - (m_capacity - index);
std::memcpy(&m_buffer[index], values, (right) * sizeof(char));
- std::memcpy(&m_buffer[0], &values[m_capacity - index], (left) * sizeof(char));
+ std::memcpy(&m_buffer[0], &values[m_capacity - index],
+ (left) * sizeof(char));
} else {
std::memcpy(&m_buffer[index], values, (bytes) * sizeof(char));
}
}
if (SystemConf::getInstance().LATENCY_ON && !m_copyDataOnInsert) {
- long *buf = (long *) &m_buffer[index];
- buf[0] = Utils::pack(latencyMark, (int) buf[0]);
+ long *buf = (long *)&m_buffer[index];
+ buf[0] = Utils::pack(latencyMark, (int)buf[0]);
}
long p = normalise(end + bytes);
- if (p <= index)
- m_wraps++;
+ if (p <= index) m_wraps++;
m_endP.m_value.store(end + bytes, std::memory_order_relaxed);
+
+ // debug ();
+ return index;
+ }
+
+ long put(std::shared_ptr &input, long latencyMark, long retainMark = -1, std::shared_ptr graph = nullptr) override {
+ auto values = input->getBuffer().data();
+ auto bytes = input->getBuffer().size();
+
+ if (values == nullptr || bytes <= 0)
+ throw std::invalid_argument("error: cannot put null to circular buffer " + std::to_string(m_id));
+
+ /* Get the end pointer */
+ long end = m_endP.m_value.load(std::memory_order_relaxed);
+
+ /* Find remaining bytes until the circular buffer wraps */
+ long wrapPoint = (end + bytes - 1) - m_capacity;
+ if (m_temp.m_value <= wrapPoint) {
+
+ m_temp.m_value = m_startP.m_value.load(std::memory_order_relaxed);
+ if (m_temp.m_value <= wrapPoint) {
+ //std::cout << "[DBG] Circular Buffer " << std::to_string(m_id)
+ // << " is full with startP "
+ // << m_startP.m_value.load() << " and endP "
+ // << m_endP.m_value.load() << std::endl;
+ return -1;
+ }
+ }
+
+ long index = normalise(end);
+
+ if (SystemConf::getInstance().LINEAGE_ON) {
+ throw std::runtime_error("error: lineage not supported during insertion");
+ }
+
+ // always copy
+ if (bytes > ((long)m_capacity - index)) { /* Copy in two parts */
+ long right = m_capacity - index;
+ long left = bytes - (m_capacity - index);
+ std::memcpy(&m_buffer[index], values, (right) * sizeof(char));
+ std::memcpy(&m_buffer[0], &values[m_capacity - index],
+ (left) * sizeof(char));
+ } else {
+ std::memcpy(&m_buffer[index], values, (bytes) * sizeof(char));
+ }
+ if (SystemConf::getInstance().LATENCY_ON && !m_copyDataOnInsert) {
+ long *buf = (long *)&m_buffer[index];
+ buf[0] = Utils::pack(latencyMark, (int)buf[0]);
+ }
+
+ long p = normalise(end + bytes);
+ if (p <= index) m_wraps++;
+ m_endP.m_value.store(end + bytes, std::memory_order_relaxed);
+
+ // free UnboundedQueryBuffer
+ UnboundedQueryBufferFactory::getInstance().freeNB(input->getBufferId(), input);
+
+ // debug ();
+ return index;
+ }
+
+ long put(void *val, long latencyMark, long retainMark = -1, std::shared_ptr graph = nullptr) override {
+ auto *values = (infinity::core::receive_element_t *) val;
+ if (!values) {
+ throw std::runtime_error("error: values is not set");
+ }
+
+ auto bytes = values->buffer->getSizeInBytes();
+ if (bytes <= 0)
+ throw std::invalid_argument("error: cannot put null to circular buffer " + std::to_string(m_id));
+
+ /* Get the end pointer */
+ long end = m_endP.m_value.load(std::memory_order_relaxed);
+
+ /* Find remaining bytes until the circular buffer wraps */
+ long wrapPoint = (end + bytes - 1) - m_capacity;
+ if (m_temp.m_value <= wrapPoint) {
+
+ m_temp.m_value = m_startP.m_value.load(std::memory_order_relaxed);
+ if (m_temp.m_value <= wrapPoint) {
+ //std::cout << "[DBG] Circular Buffer " << std::to_string(m_id)
+ // << " is full with startP "
+ // << m_startP.m_value.load() << " and endP "
+ // << m_endP.m_value.load() << std::endl;
+ return -1;
+ }
+ }
+
+ long index = normalise(end);
+
+ if (SystemConf::getInstance().LINEAGE_ON) {
+ throw std::runtime_error("error: lineage not supported during insertion");
+ }
+
+ // always copy
+ if (bytes > ((long)m_capacity - index)) { /* Copy in two parts */
+ long right = m_capacity - index;
+ long left = bytes - (m_capacity - index);
+ std::memcpy(&m_buffer[index], values->buffer->getData(), (right) * sizeof(char));
+ std::memcpy(&m_buffer[0], (char*)values->buffer->getData() + (m_capacity - index),
+ (left) * sizeof(char));
+ } else {
+ size_t idx = 0;
+ if (m_filter) {
+ m_filterFP((char*) values->buffer->getData(), 0, (int) bytes, &m_buffer[index], 0, reinterpret_cast(idx));
+ } else {
+ std::memcpy(&m_buffer[index], values->buffer->getData(), (bytes) * sizeof(char));
+ }
+ }
+ if (SystemConf::getInstance().LATENCY_ON && !m_copyDataOnInsert) {
+ long *buf = (long *)&m_buffer[index];
+ buf[0] = Utils::pack(latencyMark, (int)buf[0]);
+ }
+
+ long p = normalise(end + bytes);
+ if (p <= index) m_wraps++;
+ m_endP.m_value.store(end + bytes, std::memory_order_relaxed);
+
+ //
+ RDMABufferPool::getInstance().free(values);
+
// debug ();
return index;
}
@@ -75,14 +244,15 @@ class CircularQueryBuffer : public QueryBuffer {
bytes = offset - index + 1;
m_bytesProcessed.fetch_add(bytes, std::memory_order_relaxed);
- m_tuplesProcessed.fetch_add((bytes / (size_t) m_tupleSize), std::memory_order_relaxed);
+ m_tuplesProcessed.fetch_add((bytes / (size_t)m_tupleSize),
+ std::memory_order_relaxed);
m_tasksProcessed.fetch_add(1, std::memory_order_relaxed);
/* Set new start pointer */
m_startP.m_value.store(_start + bytes, std::memory_order_relaxed);
}
- void free(long offset) override {
+ void free(long offset, bool isPersistent = false) override {
long _start = m_startP.m_value.load(std::memory_order_relaxed);
long index = normalise(_start);
long bytes;
@@ -92,50 +262,154 @@ class CircularQueryBuffer : public QueryBuffer {
else
bytes = offset - index + 1;
+ if (m_filesystem) {
+ //auto slotId = normalise(index + bytes) / m_batchSize;
+ //auto oldVal = 3;
+ //if (!m_slots[slotId].m_slot.compare_exchange_weak(oldVal, -1)) {
+ // throw std::runtime_error("error: failed to free values from slot " + std::to_string(slotId));
+ //}
+ //std::cout << "[DBG] free "
+ // << offset << " offset of " << slotId << " slotId with "
+ // << m_slots[slotId].m_slot.load() << " slot " << std::endl;
+ //if (m_slots[slotId].m_slot.load() != -1 && m_slots[slotId].m_slot.load() != 1 &&
+ // m_slots[slotId].m_slot.load() != 3 && !m_slots[slotId].m_ready) {
+ // debugSlots();
+ // throw std::runtime_error("error: wrong value when freeing slot " +
+ // std::to_string(slotId) + " with " +
+ // std::to_string(m_slots[slotId].m_slot.load()));
+ //}
+ //m_slots[slotId].reset();
+ }
+
m_bytesProcessed.fetch_add(bytes, std::memory_order_relaxed);
- m_tuplesProcessed.fetch_add((bytes / (size_t) m_tupleSize), std::memory_order_relaxed);
+ m_tuplesProcessed.fetch_add((bytes / (size_t)m_tupleSize),
+ std::memory_order_relaxed);
m_tasksProcessed.fetch_add(1, std::memory_order_relaxed);
/* Set new start pointer */
m_startP.m_value.store(_start + bytes, std::memory_order_relaxed);
+ //debug ();
}
- ByteBuffer &getBuffer() override {
- return m_buffer;
- }
+ ByteBuffer &getBuffer() override { return m_buffer; }
char *getBufferRaw() override { return m_buffer.data(); }
size_t getBufferCapacity(int id) override {
- (void) id;
+ (void)id;
return m_capacity;
}
long getLong(size_t index) override {
- auto p = (long *) m_buffer.data();
- return p[normalise(index / sizeof(long))];
+ auto p = (long *)m_buffer.data();
+ return p[normalise(index) / sizeof(size_t)];
}
void setLong(size_t index, long value) override {
- auto p = (long *) m_buffer.data();
- p[normalise(index / sizeof(long))] = value;
+ auto p = (long *)m_buffer.data();
+ p[normalise(index) / sizeof(size_t)] = value;
}
- void appendBytesTo(int startPos, int endPos, ByteBuffer &outputBuffer) override {
+ void appendBytesTo(int startPos, int endPos,
+ ByteBuffer &outputBuffer) override {
if (endPos > startPos) {
- std::copy(m_buffer.begin() + startPos, m_buffer.begin() + endPos, outputBuffer.begin());
+ std::copy(m_buffer.begin() + startPos, m_buffer.begin() + endPos,
+ outputBuffer.begin());
} else {
- std::copy(m_buffer.begin() + startPos, m_buffer.end(), outputBuffer.begin());
- std::copy(m_buffer.begin(), m_buffer.begin() + endPos, outputBuffer.begin() + (m_capacity - startPos));
+ std::copy(m_buffer.begin() + startPos, m_buffer.end(),
+ outputBuffer.begin());
+ std::copy(m_buffer.begin(), m_buffer.begin() + endPos,
+ outputBuffer.begin() + (m_capacity - startPos));
}
}
void appendBytesTo(int startPos, int endPos, char *output) override {
if (endPos > startPos) {
- std::memcpy(output, &(m_buffer[startPos]), (endPos - startPos) * sizeof(char));
+ std::memcpy(output, m_buffer.data() + startPos,
+ (endPos - startPos) * sizeof(char));
} else {
- std::memcpy(output, &(m_buffer[startPos]), (m_capacity - startPos) * sizeof(char));
- std::memcpy(output + (m_capacity - startPos), &(m_buffer[0]), (endPos) * sizeof(char));
+ std::memcpy(output, m_buffer.data() + startPos,
+ (m_capacity - startPos) * sizeof(char));
+ std::memcpy(output + (m_capacity - startPos), m_buffer.data(),
+ (endPos) * sizeof(char));
+ }
+ }
+
+ void setupForCheckpoints(std::shared_ptr filesystem) override {
+ m_filesystem = filesystem;
+ // Initialize the slots
+ for (size_t slotId = 0; slotId < m_numberOfSlots; ++slotId) {
+ m_slots[slotId].setId(slotId, m_batchSize, m_buffer.data() + slotId * m_batchSize);
+ }
+ }
+
+ int prepareCheckpoint(long freePtr, tbb::concurrent_queue &readySlots, int &firstSlot, int &lastSlot) override {
+ auto endPtr = m_endP.m_value.load();
+ endPtr = normalise(endPtr);
+ if (freePtr > endPtr)
+ endPtr += m_capacity;
+ int numOfSlots = 0;
+
+ firstSlot = -1;
+ lastSlot = -1;
+ bool isFirst = true;
+ //auto first = freePtr;
+ //std::cout << "[DBG] preparing checkpoint for circular buffer "
+ // << std::to_string(m_id) << " with "
+ // << freePtr << " freePtr " << endPtr << " endPtr "
+ // << (endPtr-freePtr)/m_batchSize << " slots " << std::endl;
+ while (freePtr < endPtr - (long) m_batchSize) {
+ freePtr += (long) m_batchSize;
+ auto slotId = normalise(freePtr) / m_batchSize;
+
+ if (isFirst) {
+ firstSlot = slotId;
+ isFirst = false;
+ }
+ lastSlot = slotId;
+ //std::cout << "[DBG] preparing checkpoint for circular buffer "
+ // << std::to_string(m_id) << " with "
+ // << std::to_string(m_batchSize) << " batchSize "
+ // << std::to_string(first) << " first " << freePtr << " freePtr "
+ // << endPtr << " endPtr " << slotId << " slotId " << std::endl;
+ //if (m_slots[slotId].m_slot.load() != 1 && m_slots[slotId].m_slot.load() != 3) {
+ // debugSlots();
+ // throw std::runtime_error("error: wrong value in slot " +
+ // std::to_string(slotId) + " with " +
+ // std::to_string(m_slots[slotId].m_slot.load()));
+ //}
+ m_slots[slotId].reset();
+ m_slots[slotId].setPreviousSlot(3);
+ m_slots[slotId].m_slot.store(5);
+ m_slots[slotId].setNumberOfResults();
+ readySlots.push(slotId);
+ numOfSlots++;
+ }
+ if (numOfSlots == 0 && freePtr < endPtr) {
+ freePtr += (long) m_batchSize;
+ auto slotId = normalise(freePtr) / m_batchSize;
+ firstSlot = slotId;
+ lastSlot = slotId;
+ //std::cout << "[DBG] preparing checkpoint for circular buffer "
+ // << std::to_string(m_id) << " with "
+ // << std::to_string(m_batchSize) << " batchSize "
+ // << std::to_string(first) << " first " << freePtr << " freePtr "
+ // << endPtr << " endPtr " << slotId << " slotId " << std::endl;
+ m_slots[slotId].reset();
+ m_slots[slotId].setPreviousSlot(3);
+ m_slots[slotId].m_slot.store(5);
+ m_slots[slotId].setNumberOfResults();
+ readySlots.push(slotId);
+ numOfSlots++;
+ }
+ return numOfSlots;
+ }
+
+ void debugSlots(){
+ for (auto &slot : m_slots) {
+ if (slot.m_id == -1) break;
+ std::cout << slot.m_id << " slotId " << slot.m_slot.load()
+ << " slot " << std::endl;
}
}
diff --git a/src/buffers/NUMACircularQueryBuffer.h b/src/buffers/NUMACircularQueryBuffer.h
index 51c1ec2..77f978f 100644
--- a/src/buffers/NUMACircularQueryBuffer.h
+++ b/src/buffers/NUMACircularQueryBuffer.h
@@ -1,16 +1,19 @@
#pragma once
+#include
+
#include
-#include
+#include
#include
#include
-#include
-#include
+#include
-#include "utils/Utils.h"
+#include "buffers/NumaBuffer.h"
+#include "buffers/QueryBuffer.h"
+#include "buffers/UnboundedQueryBufferFactory.h"
#include "utils/PaddedLong.h"
#include "utils/SystemConf.h"
-#include "buffers/QueryBuffer.h"
+#include "utils/Utils.h"
/*
* \brief This class implements a NUMA-aware circular buffer.
@@ -20,33 +23,20 @@
*
* */
-class NUMACircularQueryBuffer : public QueryBuffer {
+class NumaCircularQueryBuffer : public QueryBuffer, public NumaBuffer {
private:
- const int m_cpusPerNode;
- const int m_numaNodes;
- const size_t m_maxBufferCapacity;
- size_t m_minBufferCapacity;
std::vector> m_buffers;
- std::vector m_orderedCpus;
public:
- NUMACircularQueryBuffer(int id, size_t capacity, int tupleSize = 1, bool copyDataOnInsert = true) :
- QueryBuffer(id, capacity, true, tupleSize, copyDataOnInsert),
-#if defined(HAVE_NUMA)
- m_cpusPerNode(Utils::getNumberOfCoresPerSocket()),
- m_numaNodes(
- (numa_available() < 0) ? 1 :
- (int) std::ceil(((double) SystemConf::getInstance().WORKER_THREADS + 1) / m_cpusPerNode)),
-#else
- m_cpusPerNode(SystemConf::getInstance().THREADS),
- m_numaNodes(1),
-#endif
- m_maxBufferCapacity(Utils::getPowerOfTwo(m_capacity / m_numaNodes)),
- m_buffers(m_numaNodes) {
+ NumaCircularQueryBuffer(int id, size_t capacity, int tupleSize = 1, bool copyDataOnInsert = true,
+ size_t batchSize = SystemConf::getInstance().BATCH_SIZE, bool clearFiles = true) :
+ QueryBuffer(id, capacity, true, tupleSize, copyDataOnInsert, batchSize, nullptr, clearFiles),
+ NumaBuffer(capacity, tupleSize), m_buffers(m_numaNodes) {
assert(m_maxBufferCapacity % tupleSize == 0 && "Buffer capacity has to be divisible by the tuple size.");
#if defined(HAVE_NUMA)
int numa_node = -1;
- long bufferSizePerThread = (long) (m_maxBufferCapacity / 8);
+ long bufferSizePerThread = (long) (m_maxBufferCapacity / m_cpusPerNode);
+ bufferSizePerThread = Utils::getPowerOfTwo(bufferSizePerThread);
size_t totalSize = 0;
Utils::getOrderedCores(m_orderedCpus);
for (int i = 0; i < m_numaNodes; ++i) {
@@ -54,11 +44,15 @@ class NUMACircularQueryBuffer : public QueryBuffer {
long bufferSize = (i != m_numaNodes-1) ? (long) m_maxBufferCapacity :
(long) ((SystemConf::getInstance().WORKER_THREADS - i * m_cpusPerNode + 1) % m_cpusPerNode)
* bufferSizePerThread;
- if (bufferSize == 0)
+ if (m_numaNodes == 1) {
+ bufferSize = m_maxBufferCapacity;
+ }
+ if (bufferSize == 0 || bufferSize > m_maxBufferCapacity)
bufferSize = m_maxBufferCapacity;
+ bufferSize = Utils::getPowerOfTwo(bufferSize);
m_buffers[i] = std::make_unique(bufferSize);
m_buffers[i]->data()[0] = 0;
- std::cout << "[DBG] Creating ByteBuffer " << i << std::endl;
+ std::cout << "[DBG] Creating ByteBuffer " << i << " with " << bufferSize << " size" << std::endl;
SystemConf::getInstance().findMemoryNodeForAddress(m_buffers[i].get()->data(), numa_node);
std::cout << "[DBG] ByteBuffer " << i << " is allocated in " << numa_node << " numa node." << std::endl;
/*if (numa_node != i) {
@@ -71,16 +65,28 @@ class NUMACircularQueryBuffer : public QueryBuffer {
}
setMask(totalSize);
setCapacity(totalSize);
+ m_numberOfSlots = totalSize/m_batchSize;
#else
m_buffers[0] = std::make_unique(m_maxBufferCapacity);
+ m_numberOfSlots = m_maxBufferCapacity/m_batchSize;
#endif
+
+ m_emptySlots.store(m_numberOfSlots);
+ //m_slots.resize(m_numberOfSlots);
+
+ //if (m_capacity != 1 && m_capacity % m_batchSize != 0)
+ // throw std::runtime_error("error: the capacity is not a multiple of the slot size");
+
+ if (m_capacity > 2 && m_tupleSize != 1 && SystemConf::getInstance().LINEAGE_ON) {
+ setupForCheckpoints(nullptr);
+ }
};
- long put(char *values, long bytes, long latencyMark = -1) override {
+ long put(char *values, long bytes, long latencyMark, long retainMark = -1, std::shared_ptr graph = nullptr) override {
if (values == nullptr || bytes <= 0)
throw std::invalid_argument("error: cannot put null to circular buffer");
assert((long) m_maxBufferCapacity > bytes && "Buffer capacity has to be at least "
- "the number of bytes writter to avoid spilling to more than two buffers");
+ "the number of bytes writer to avoid spilling to more than two buffers");
/* Get the end pointer */
long end = m_endP.m_value.load(std::memory_order_relaxed);
@@ -94,6 +100,38 @@ class NUMACircularQueryBuffer : public QueryBuffer {
}
long index = normaliseNotPowerOfTwo(end);
+
+ if (SystemConf::getInstance().LINEAGE_ON) {
+ auto firstSlotId = index / m_batchSize;
+ auto endSlotId = (index + bytes) / m_batchSize;
+ auto slotId = firstSlotId;
+ while (slotId <= endSlotId) {
+ auto normSlot = slotId % m_numberOfSlots;
+ if (slotId != endSlotId) {
+ if (!m_slots[normSlot].m_graph) {
+ std::lock_guard l (m_slots[normSlot].m_updateLock);
+ auto newGraph = LineageGraphFactory::getInstance().newInstance();
+ m_slots[normSlot].setLineageGraph(newGraph);
+ }
+ } else {
+ if (!graph && !m_slots[normSlot].m_graph) {
+ graph = LineageGraphFactory::getInstance().newInstance();
+ }
+ if (graph) {
+ std::lock_guard l (m_slots[normSlot].m_updateLock);
+ m_slots[normSlot].setLineageGraph(graph);
+ }
+ }
+
+ /*if (!m_slots[normSlot].m_graph) {
+ throw std::runtime_error(
+ "error: the lineage graph is not initialized when inserting for slot " +
+ std::to_string(normSlot));
+ }*/
+ slotId ++;
+ }
+ }
+
if (m_copyDataOnInsert || m_wraps == 0) { // copy only until the buffer is filled once
int bufferIdx = getBufferIndex(index);
/* TODO: fix the case when the batch spills to multiple buffers */
@@ -122,6 +160,64 @@ class NUMACircularQueryBuffer : public QueryBuffer {
return index;
}
+ long put(std::shared_ptr &input, long latencyMark, long retainMark = -1, std::shared_ptr graph = nullptr) override {
+ auto values = input->getBuffer().data();
+ auto bytes = input->getBuffer().size();
+
+ if (values == nullptr || bytes <= 0)
+ throw std::invalid_argument("error: cannot put null to circular buffer");
+ assert((long) m_maxBufferCapacity > bytes && "Buffer capacity has to be at least "
+ "the number of bytes writer to avoid spilling to more than two buffers");
+
+ /* Get the end pointer */
+ long end = m_endP.m_value.load(std::memory_order_relaxed);
+ /* Find remaining bytes until the circular buffer wraps */
+ long wrapPoint = (end + bytes - 1) - m_capacity;
+ if (m_temp.m_value <= wrapPoint) {
+ m_temp.m_value = m_startP.m_value.load(std::memory_order_relaxed);
+ if (m_temp.m_value <= wrapPoint) {
+ return -1;
+ }
+ }
+
+ long index = normaliseNotPowerOfTwo(end);
+
+ if (SystemConf::getInstance().LINEAGE_ON) {
+ throw std::runtime_error("error: lineage not supported during insertion");
+ }
+
+ // always copy
+ int bufferIdx = getBufferIndex(index);
+ /* TODO: fix the case when the batch spills to multiple buffers */
+ if (bytes > ((long) m_capacity - index)) { /* Copy in two parts */
+ throw std::runtime_error("error: batches should not be spilled to more than one buffer");
+ } else if (bytes > ((long) ((bufferIdx + 1) * m_maxBufferCapacity) - index)) { /* Copy in two parts */
+ throw std::runtime_error("error: batches should not be spilled to more than one buffer");
+ } else {
+ long normIndex = normaliseIndex(index, bufferIdx);
+ std::memcpy(&m_buffers[bufferIdx].get()->data()[normIndex], values, (bytes) * sizeof(char));
+ }
+
+ if (SystemConf::getInstance().LATENCY_ON && !m_copyDataOnInsert) {
+ int bufferIdx = getBufferIndex(index);
+ long normIndex = normaliseIndex(index, bufferIdx);
+ long *buf = (long *) &m_buffers[bufferIdx].get()->data()[normIndex];
+ buf[0] = Utils::pack(latencyMark, (int) buf[0]);
+ }
+
+ //long p = normalise(_end + bytes);
+ long p = normaliseNotPowerOfTwo(end + bytes);
+ if (p <= index)
+ m_wraps++;
+ m_endP.m_value.store(end + bytes, std::memory_order_relaxed);
+
+ // free UnboundedQueryBuffer
+ UnboundedQueryBufferFactory::getInstance().freeNB(input->getBufferId(), input);
+
+ // debug ();
+ return index;
+ }
+
void free() override {
long _start = m_startP.m_value.load(std::memory_order_relaxed);
long offset = normaliseNotPowerOfTwo(SystemConf::getInstance().BATCH_SIZE + _start);
@@ -141,7 +237,7 @@ class NUMACircularQueryBuffer : public QueryBuffer {
m_startP.m_value.store(_start + bytes, std::memory_order_relaxed);
}
- void free(long offset) override {
+ void free(long offset, bool isPersistent = false) override {
long _start = m_startP.m_value.load(std::memory_order_relaxed);
long index = normaliseNotPowerOfTwo(_start);
long bytes;
@@ -159,42 +255,48 @@ class NUMACircularQueryBuffer : public QueryBuffer {
m_startP.m_value.store(_start + bytes, std::memory_order_relaxed);
}
- int getBufferIndex(long index) {
+ int getBufferIndex(const long index) override {
return (int) (index / m_maxBufferCapacity);
}
- long normaliseIndex(long index) {
+ long normaliseIndex(const long index) override {
return (index % m_maxBufferCapacity);
}
- long normaliseIndex(long index, int bufferIdx) {
+ long normaliseIndex(const long index, const int bufferIdx) override {
//return index % maxBufferCapacity;
return (bufferIdx != m_numaNodes - 1) ? (index % m_maxBufferCapacity) : (index % m_minBufferCapacity);
}
- ByteBuffer &getBuffer(long index) {
+ ByteBuffer &getBuffer(long index) override {
//index = normalise(index);
index = normaliseNotPowerOfTwo(index);
int bufferIdx = getBufferIndex(index);
return getBuffer(bufferIdx);
}
- ByteBuffer &getBuffer() override {
- return *m_buffers[0].get();
+ char *getBufferRaw(long index) override {
+ index = normaliseNotPowerOfTwo(index);
+ int bufferIdx = getBufferIndex(index);
+ return getBufferRaw(bufferIdx);
}
- ByteBuffer &getBuffer(int bufferIdx = 0) {
+ ByteBuffer &getBuffer(int bufferIdx) override {
return *m_buffers[bufferIdx].get();
}
- char *getBufferRaw(long index) {
- index = normaliseNotPowerOfTwo(index);
+ char *getBufferRaw(int bufferIdx) override {
+ return m_buffers[bufferIdx].get()->data();
+ }
+
+ int geNumaNodeWithPtr(int index) override {
+ index = (int) normaliseNotPowerOfTwo(index);
int bufferIdx = getBufferIndex(index);
- return getBufferRaw(bufferIdx);
+ return bufferIdx;
}
- char *getBufferRaw(int bufferIdx) {
- return m_buffers[bufferIdx].get()->data();
+ ByteBuffer &getBuffer() override {
+ return *m_buffers[0].get();
}
char *getBufferRaw() override {
@@ -231,15 +333,85 @@ class NUMACircularQueryBuffer : public QueryBuffer {
throw std::runtime_error("error: this method is not supported for the NUMA-aware Circular Buffer");
}
+ void setupForCheckpoints(std::shared_ptr filesystem) override {
+ if (m_capacity != 1 && m_capacity % m_batchSize != 0)
+ throw std::runtime_error("error: the capacity is not a multiple of the slot size");
+ m_filesystem = filesystem;
+ // Initialize the slots
+ for (size_t slotId = 0; slotId < m_numberOfSlots; ++slotId) {
+ m_slots[slotId].setId(slotId, m_batchSize, getBufferRaw((long)(slotId * m_batchSize)));
+ }
+ }
+
+ int prepareCheckpoint(long freePtr, tbb::concurrent_queue &readySlots, int &firstSlot, int &lastSlot) override {
+ auto endPtr = m_endP.m_value.load();
+ endPtr = normaliseNotPowerOfTwo(endPtr);
+ if (freePtr > endPtr)
+ endPtr += m_capacity;
+ int numOfSlots = 0;
+
+ firstSlot = -1;
+ lastSlot = -1;
+ bool isFirst = true;
+ //auto first = freePtr;
+ //std::cout << "[DBG] preparing checkpoint for circular buffer "
+ // << std::to_string(m_id) << " with "
+ // << freePtr << " freePtr " << endPtr << " endPtr "
+ // << (endPtr-freePtr)/m_batchSize << " slots " << std::endl;
+ while (freePtr < endPtr - (long) m_batchSize) {
+ freePtr += (long) m_batchSize;
+ auto slotId = normaliseNotPowerOfTwo(freePtr) / m_batchSize;
+
+ if (isFirst) {
+ firstSlot = slotId;
+ isFirst = false;
+ }
+ lastSlot = slotId;
+ //std::cout << "[DBG] preparing checkpoint for circular buffer "
+ // << std::to_string(m_id) << " with "
+ // << std::to_string(m_batchSize) << " batchSize "
+ // << std::to_string(first) << " first " << freePtr << " freePtr "
+ // << endPtr << " endPtr " << slotId << " slotId " << std::endl;
+ //if (m_slots[slotId].m_slot.load() != 1 && m_slots[slotId].m_slot.load() != 3) {
+ // debugSlots();
+ // throw std::runtime_error("error: wrong value in slot " +
+ // std::to_string(slotId) + " with " +
+ // std::to_string(m_slots[slotId].m_slot.load()));
+ //}
+ m_slots[slotId].reset();
+ m_slots[slotId].setPreviousSlot(3);
+ m_slots[slotId].m_slot.store(5);
+ m_slots[slotId].setNumberOfResults();
+ readySlots.push(slotId);
+ numOfSlots++;
+ }
+ if (numOfSlots == 0 && freePtr < endPtr) {
+ freePtr += (long) m_batchSize;
+ auto slotId = normaliseNotPowerOfTwo(freePtr) / m_batchSize;
+ firstSlot = slotId;
+ lastSlot = slotId;
+ //std::cout << "[DBG] preparing checkpoint for circular buffer "
+ // << std::to_string(m_id) << " with "
+ // << std::to_string(m_batchSize) << " batchSize "
+ // << std::to_string(first) << " first " << freePtr << " freePtr "
+ // << endPtr << " endPtr " << slotId << " slotId " << std::endl;
+ m_slots[slotId].reset();
+ m_slots[slotId].setPreviousSlot(3);
+ m_slots[slotId].m_slot.store(5);
+ m_slots[slotId].setNumberOfResults();
+ readySlots.push(slotId);
+ numOfSlots++;
+ }
+ return numOfSlots;
+ }
+
size_t getBufferCapacity(int id) override {
return (id != m_numaNodes - 1) ? m_maxBufferCapacity : m_minBufferCapacity;
}
- int geNumaNodeWithPtr(int index) {
- index = (int) normaliseNotPowerOfTwo(index);
- int bufferIdx = getBufferIndex(index);
- return bufferIdx;
+ void fixTimestamps(size_t index, long timestamp, long step, long batchSize) {
+ throw std::runtime_error("error: this method is not supported for the NUMA-aware Circular Buffer");
}
- ~NUMACircularQueryBuffer() override = default;
+ ~NumaCircularQueryBuffer() override = default;
};
\ No newline at end of file
diff --git a/src/buffers/NumaBuffer.h b/src/buffers/NumaBuffer.h
new file mode 100644
index 0000000..56ab009
--- /dev/null
+++ b/src/buffers/NumaBuffer.h
@@ -0,0 +1,69 @@
+#pragma once
+
+#include
+#include
+#include
+#include
+#include
+#include
+
+#include "utils/Utils.h"
+#include "utils/PaddedLong.h"
+#include "utils/SystemConf.h"
+
+class Query;
+
+/*
+ * \brief NumaBuffer is used as a base class for implementing a lock-free numa-aware circular buffer.
+ *
+ * */
+
+class NumaBuffer {
+
+ protected:
+ const int m_cpusPerNode;
+ const int m_numaNodes;
+ const size_t m_maxBufferCapacity;
+ size_t m_minBufferCapacity;
+ std::vector m_orderedCpus;
+
+ public:
+ NumaBuffer(size_t capacity, int tupleSize = 1) :
+#if defined(HAVE_NUMA)
+ m_cpusPerNode(Utils::getNumberOfCoresPerSocket()),
+ m_numaNodes(
+ (numa_available() < 0) ? 1 :
+ (int) std::ceil(((double) SystemConf::getInstance().WORKER_THREADS + 1) / m_cpusPerNode)),
+#else
+ m_cpusPerNode(SystemConf::getInstance().THREADS),
+ m_numaNodes(1),
+#endif
+ m_maxBufferCapacity(Utils::getPowerOfTwo(capacity / m_numaNodes)) {
+ assert(m_maxBufferCapacity % tupleSize == 0 && "Buffer capacity has to be divisible by the tuple size.");
+ };
+
+ virtual int getBufferIndex(const long index) {
+ return (int) (index / m_maxBufferCapacity);
+ }
+
+ virtual long normaliseIndex(const long index) {
+ return (index % m_maxBufferCapacity);
+ }
+
+ virtual long normaliseIndex(const long index, const int bufferIdx) {
+ //return index % maxBufferCapacity;
+ return (bufferIdx != m_numaNodes - 1) ? (index % m_maxBufferCapacity) : (index % m_minBufferCapacity);
+ }
+
+ virtual ByteBuffer &getBuffer(long index) = 0;
+
+ virtual char *getBufferRaw(long index) = 0;
+
+ virtual int geNumaNodeWithPtr(int index) = 0;
+
+ virtual ByteBuffer &getBuffer(int bufferIdx = 0) = 0;
+
+ virtual char *getBufferRaw(int bufferIdx = 0) = 0;
+
+ virtual ~NumaBuffer() = default;
+};
\ No newline at end of file
diff --git a/src/buffers/PartialWindowResults.h b/src/buffers/PartialWindowResults.h
index 1cf368a..3b1d7f7 100644
--- a/src/buffers/PartialWindowResults.h
+++ b/src/buffers/PartialWindowResults.h
@@ -48,7 +48,8 @@ class PartialWindowResults {
//std::cout << "[DBG] Thread " << sched_getcpu() << " in " << node << " numa node." << std::endl;
if (type == 1) {
for (size_t i = 0; i < m_bufferPtrs.size(); ++i) {
- m_bufferPtrs[i] = (char *) _mm_malloc(capacity * sizeof(char), 64);
+ //m_bufferPtrs[i] = (char *) _mm_malloc(capacity * sizeof(char), 64);
+ m_bufferPtrs[i] = (char *) _mm_malloc(capacity * sizeof(char), 512);
}
}
//SystemConf::getInstance().findMemoryNodeForAddress(&bufferPtrs[2], node);
@@ -81,6 +82,14 @@ class PartialWindowResults {
"error: getting a buffer of pointers from partial window buffer of this type is not implemented");
}
+ char **getBufferPtrsRaw() {
+ if (m_type == 1)
+ return m_bufferPtrs.data();
+ else
+ throw std::runtime_error(
+ "error: getting a buffer of pointers from partial window buffer of this type is not implemented");
+ }
+
size_t getCapacity() {
return m_capacity;
}
@@ -91,6 +100,9 @@ class PartialWindowResults {
void clear() {
//std::fill(buffer.begin(), buffer.end(), 0);
+ if (m_type == 0) {
+ m_position = 0;
+ }
}
void reset() {
@@ -105,6 +117,7 @@ class PartialWindowResults {
void init() {
m_count = 0;
+ m_position = 0;
}
void nullify() {
@@ -116,7 +129,8 @@ class PartialWindowResults {
void increment() {
if (m_count > (int) m_partialWindows)
- throw std::out_of_range("error: partial window result index out of bounds while incrementing the counter");
+ throw std::out_of_range("error: partial window result index out of bounds while incrementing the counter ("
+ + std::to_string(m_count) + " > " + std::to_string(m_partialWindows) + ")");
m_startPointers[m_count] = (int) getPosition();
m_count++;
}
@@ -125,14 +139,16 @@ class PartialWindowResults {
m_count += cnt;
if (m_count > (int) m_partialWindows)
throw std::out_of_range(
- "error: partial window result index out of bounds while incrementing the counter with a value");
+ "error: partial window result index out of bounds while incrementing the counter ("
+ + std::to_string(m_count) + " > " + std::to_string(m_partialWindows) + ")");
}
void setCount(int cnt) {
m_count = cnt;
if (m_count > (int) m_partialWindows)
throw std::out_of_range(
- "error: partial window result index out of bounds while incrementing the counter with a value");
+ "error: partial window result index out of bounds while incrementing the counter with a value ("
+ + std::to_string(m_count) + " > " + std::to_string(m_partialWindows) + ")");
}
int getStartPointer(int idx) {
diff --git a/src/buffers/PartialWindowResultsFactory.h b/src/buffers/PartialWindowResultsFactory.h
index 60ac7f6..b30b200 100644
--- a/src/buffers/PartialWindowResultsFactory.h
+++ b/src/buffers/PartialWindowResultsFactory.h
@@ -48,6 +48,13 @@ class PartialWindowResultsFactory {
std::make_shared(pid, SystemConf::getInstance().UNBOUNDED_BUFFER_SIZE);
}
partialWindowResults->init();
+
+ /*if (partialWindowResults.use_count() != 1) {
+ std::cout << "warning: the partial result has a reference counter of " + std::to_string(partialWindowResults.use_count()) << std::endl;
+ //throw std::runtime_error("error: the partial result should have only one reference when we free it: " +
+ // std::to_string(partialWindowResults.use_count()));
+ }*/
+
return partialWindowResults;
}
@@ -73,8 +80,14 @@ class PartialWindowResultsFactory {
}
void free(int pid, std::shared_ptr &partialWindowResults) {
- //if (partialWindowResults.use_count() != 1)
- // throw std::runtime_error("error: the partial result should have only one reference when we free it: " + std::to_string(partialWindowResults.use_count()));
+ /*if (partialWindowResults.use_count() != 1) {
+ std::cout << "warning: the partial result has a reference counter of " + std::to_string(partialWindowResults.use_count()) << std::endl;
+ throw std::runtime_error("error: the partial result should have only one reference when we free it: " +
+ std::to_string(partialWindowResults.use_count()));
+ }*/
+ if (pid >= m_numberOfThreads)
+ throw std::runtime_error("error: attempting to free partial window with pid: " + std::to_string(pid)
+ + " >= " + std::to_string(m_numberOfThreads));
if (partialWindowResults->getType() == 0) {
m_poolSeqMem[pid].push(partialWindowResults);
diff --git a/src/buffers/PersistentCircularQueryBuffer.h b/src/buffers/PersistentCircularQueryBuffer.h
new file mode 100644
index 0000000..03b2322
--- /dev/null
+++ b/src/buffers/PersistentCircularQueryBuffer.h
@@ -0,0 +1,1292 @@
+#pragma once
+
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+
+#include "PartialWindowResultsFactory.h"
+#include "UnboundedQueryBufferFactory.h"
+#include "buffers/QueryBuffer.h"
+#include "tasks/Task.h"
+#include "tasks/TaskFactory.h"
+#include "tasks/WindowBatchFactory.h"
+#include "utils/Async.h"
+#include "utils/PaddedLong.h"
+#include "utils/Query.h"
+#include "utils/SystemConf.h"
+#include "utils/TupleSchema.h"
+#include "utils/Utils.h"
+#include "utils/WindowDefinition.h"
+
+/*
+ * \brief This class implements a non-NUMA-aware circular buffer
+ * that persists data to disk asynchronously. It used to implement a P-Stream.
+ *
+ * */
+
+class AckSlotContext : public IAsyncContext {
+ public:
+ AckSlotContext(std::atomic *slot, std::shared_ptr parW = nullptr) : m_slot(slot), m_parW(parW) {}
+
+ protected:
+ Status deepCopyInternal(IAsyncContext *&context_copy) final {
+ return IAsyncContext::deepCopyInternal(*this, context_copy);
+ }
+
+ public:
+ std::atomic *m_slot;
+ std::shared_ptr m_parW;
+};
+
+class PersistentCircularQueryBuffer : public QueryBuffer {
+ private:
+ struct PMem;
+
+ /// The buffer holding the in-memory data
+ ByteBuffer m_buffer;
+
+ // todo: these have to be persisted
+ std::atomic m_nextFreeSlot;
+ /* Lock protecting the acknowledgment of persisting the input to a file */
+ std::mutex m_ackLock;
+ std::atomic m_nextPersistentSlot;
+
+ // Variables for persisting the file pointers
+ const size_t m_poolSize;
+ const std::string m_layout = "";
+ pmem::obj::pool m_pop;
+ pmem::obj::persistent_ptr m_root;
+ std::string m_pmFileName;
+ file_t *m_pmFile;
+
+ // Variables for persisting asynchronously the actual data
+ std::string m_asyncFileName;
+ file_t *m_asyncFile;
+ FileOptions m_asyncFileOptions;
+
+ std::atomic m_wraps = 0;
+
+ // Used for compression
+ std::vector m_copyBuffers;
+
+ bool m_ready = false;
+ const bool m_debug = false;
+
+ public:
+ PersistentCircularQueryBuffer(int id, size_t capacity, int tupleSize = 1,
+ bool copyDataOnInsert = true, size_t batchSize = SystemConf::getInstance().BATCH_SIZE,
+ std::shared_ptr filesystem = nullptr, bool clearFiles = true,
+ bool unbuffered = true, bool delete_on_close = false)
+ : QueryBuffer(id, capacity, false, tupleSize, copyDataOnInsert, batchSize, filesystem, clearFiles),
+ m_buffer(capacity),
+ m_nextFreeSlot(0),
+ m_nextPersistentSlot(0),
+ m_poolSize(PMEMOBJ_MIN_POOL),
+ m_pmFileName("scabbard/queue_pm_" + std::to_string(id)),
+ m_asyncFileName("scabbard/queue_data_" + std::to_string(id)),
+ m_asyncFileOptions(unbuffered, delete_on_close),
+ m_copyBuffers(SystemConf::getInstance().WORKER_THREADS, ByteBuffer(m_batchSize)) {
+ if (m_capacity % m_batchSize != 0)
+ throw std::runtime_error("error: the capacity is not a multiple of the slot size");
+ if (!(m_numberOfSlots && !(m_numberOfSlots & (m_numberOfSlots - 1)))) {
+ throw std::runtime_error ("error: the number of slots has to be a power of two");
+ }
+ try {
+ if (!m_filesystem) {
+ std::cout << "warning: no filesystem passed to the constructor. "
+ "Initializing a new filesystem..." << std::endl;
+ m_filesystem = std::make_shared(SystemConf::FILE_ROOT_PATH, SystemConf::getInstance().WORKER_THREADS);
+ }
+
+ Utils::tryCreateDirectory(m_filesystem->getRootPath() + "scabbard");
+ auto pmPath = m_filesystem->getRootPath() + m_pmFileName;
+ if (Utils::fileExists(pmPath.c_str()) != 0) {
+ m_pop = pmem::obj::pool::create(pmPath.c_str(),
+ "", m_poolSize, CREATE_MODE_RW);
+ m_root = m_pop.root();
+ pmem::obj::make_persistent_atomic(m_pop, m_root->next);
+ pmem::obj::transaction::run(m_pop, [&] { m_root = m_root->next; });
+ m_previousBlockSize = SystemConf::getInstance().BATCH_SIZE;
+ } else {
+ m_pop = pmem::obj::pool::open(pmPath, "");
+ m_root = m_pop.root();
+ m_root = m_root->next;
+ m_previousBlockSize = m_root->m_blockSize.get_ro();
+ }
+ } catch (const pmem::pool_error &e) {
+ std::cerr << "Exception: " << e.what() << std::endl;
+ return;
+ } catch (const pmem::transaction_error &e) {
+ std::cerr << "Exception: " << e.what() << std::endl;
+ return;
+ }
+
+ m_root->m_blockSize.get_rw() = SystemConf::getInstance().BLOCK_SIZE;
+
+ // Initialize the slots
+ for (size_t slotId = 0; slotId < m_numberOfSlots; ++slotId) {
+ m_slots[slotId].setId(slotId, m_batchSize, nullptr);
+ }
+
+ // Open File handlers
+ m_pmFile = m_filesystem->newFile(m_pmFileName); // do I need this?
+ if (!SystemConf::getInstance().LINEAGE_ON) {
+ m_asyncFile = m_filesystem->newFile(m_asyncFileName, m_numberOfSlots * m_batchSize);
+ }
+ m_ready = true;
+ };
+
+ long put(char *values, long bytes, long latencyMark, long retainMark = -1, std::shared_ptr graph = nullptr) override {
+ while (!m_ready)
+ ;
+
+ if (values == nullptr || bytes <= 0)
+ throw std::invalid_argument("error: cannot put null to circular buffer");
+ if (bytes != m_batchSize)
+ throw std::invalid_argument("error: the size of the input must be equal to the slot ("+std::to_string(m_batchSize)+" != "+std::to_string(bytes)+")");
+
+ /* Get the end pointer */
+ long end = m_endP.m_value.load(std::memory_order_relaxed);
+
+ bool hasEmptySlots = (m_emptySlots > 0);
+
+ /* Find remaining bytes until the circular buffer wraps */
+ long wrapPoint = (end + bytes - 1) - m_capacity;
+ if (m_temp.m_value <= wrapPoint || !hasEmptySlots) {
+ m_temp.m_value = m_startP.m_value.load(std::memory_order_relaxed);
+ if (m_temp.m_value <= wrapPoint || !hasEmptySlots) {
+ // std::cout << "[DBG] Circular Buffer is full with startP "
+ // << m_startP.m_value.load() << " and endP "
+ // << m_endP.m_value.load() << std::endl;
+ // check if some async calls have finished
+ m_filesystem->getHandler().tryCompleteMultiple();
+ tryToAcknowledge();
+ return -1;
+ }
+ }
+
+ // create task
+ if (m_debug) {
+ std::cout << "[DBG] CB " + std::to_string(m_id) + " Creating insertion task for slot "
+ + std::to_string(m_nextSlotToWrite)
+ + " with empty slots " + std::to_string(m_emptySlots.load()-1) << std::endl;
+ }
+ createInsertionTask(values, bytes, latencyMark, retainMark, graph);
+ m_emptySlots.fetch_add(-1);
+ // try to forward the end pointer
+ m_filesystem->getHandler().tryCompleteMultiple();
+ tryToAcknowledge();
+ //long index = tryConsumeNextSlot();
+
+ if (m_debug) {
+ debug();
+ }
+ return 0;
+ }
+
+ long put(std::shared_ptr &values, long latencyMark, long retainMark = -1, std::shared_ptr graph = nullptr) override {
+ while (!m_ready)
+ ;
+
+ auto bytes = values->getBuffer().size();
+ if (values == nullptr || bytes <= 0)
+ throw std::invalid_argument("error: cannot put null to circular buffer");
+ if (bytes != m_batchSize)
+ throw std::invalid_argument("error: the size of the input must be equal to the slot ("+std::to_string(m_batchSize)+" != "+std::to_string(bytes)+")");
+
+ /* Get the end pointer */
+ long end = m_endP.m_value.load(std::memory_order_relaxed);
+
+ bool hasEmptySlots = (m_emptySlots > 0);
+
+ /* Find remaining bytes until the circular buffer wraps */
+ long wrapPoint = (end + bytes - 1) - m_capacity;
+ if (m_temp.m_value <= wrapPoint || !hasEmptySlots) {
+ m_temp.m_value = m_startP.m_value.load(std::memory_order_relaxed);
+ if (m_temp.m_value <= wrapPoint || !hasEmptySlots) {
+ // std::cout << "[DBG] Circular Buffer is full with startP "
+ // << m_startP.m_value.load() << " and endP "
+ // << m_endP.m_value.load() << std::endl;
+ // check if some async calls have finished
+ m_filesystem->getHandler().tryCompleteMultiple();
+ tryToAcknowledge();
+ return -1;
+ }
+ }
+
+ // create task
+ if (m_debug) {
+ std::cout << "[DBG] CB " + std::to_string(m_id) + " Creating insertion task for slot "
+ + std::to_string(m_nextSlotToWrite)
+ + " with empty slots " + std::to_string(m_emptySlots.load()-1) << std::endl;
+ }
+ createInsertionTask(values, bytes, latencyMark, retainMark, graph);
+ m_emptySlots.fetch_add(-1);
+ // try to forward the end pointer
+ m_filesystem->getHandler().tryCompleteMultiple();
+ tryToAcknowledge();
+ //long index = tryConsumeNextSlot();
+
+ if (m_debug) {
+ debug();
+ }
+ return 0;
+ }
+
+ long put(void *val, long latencyMark = -1, long retainMark = -1, std::shared_ptr graph = nullptr) override {
+ while (!m_ready)
+ ;
+
+ infinity::core::receive_element_t *values = (infinity::core::receive_element_t *) val;
+ if (!values) {
+ throw std::runtime_error("error: values is not set");
+ }
+
+ auto bytes = values->buffer->getSizeInBytes();
+ if (values == nullptr || bytes <= 0)
+ throw std::invalid_argument("error: cannot put null to circular buffer");
+ if (bytes != m_batchSize)
+ throw std::invalid_argument("error: the size of the input must be equal to the slot ("+std::to_string(m_batchSize)+" != "+std::to_string(bytes)+")");
+
+ /* Get the end pointer */
+ long end = m_endP.m_value.load(std::memory_order_relaxed);
+
+ bool hasEmptySlots = (m_emptySlots > 0);
+
+ /* Find remaining bytes until the circular buffer wraps */
+ long wrapPoint = (end + bytes - 1) - m_capacity;
+ if (m_temp.m_value <= wrapPoint || !hasEmptySlots) {
+ m_temp.m_value = m_startP.m_value.load(std::memory_order_relaxed);
+ if (m_temp.m_value <= wrapPoint || !hasEmptySlots) {
+ // std::cout << "[DBG] Circular Buffer is full with startP "
+ // << m_startP.m_value.load() << " and endP "
+ // << m_endP.m_value.load() << std::endl;
+ // check if some async calls have finished
+ m_filesystem->getHandler().tryCompleteMultiple();
+ tryToAcknowledge();
+ return -1;
+ }
+ }
+
+ // create task
+ if (m_debug) {
+ std::cout << "[DBG] CB " + std::to_string(m_id) + " Creating insertion task for slot "
+ + std::to_string(m_nextSlotToWrite)
+ + " with empty slots " + std::to_string(m_emptySlots.load()-1) << std::endl;
+ }
+ createInsertionTask(values, bytes, latencyMark, retainMark, graph);
+ m_emptySlots.fetch_add(-1);
+ // try to forward the end pointer
+ m_filesystem->getHandler().tryCompleteMultiple();
+ tryToAcknowledge();
+ //long index = tryConsumeNextSlot();
+
+ if (m_debug) {
+ debug();
+ }
+ return 0;
+
+ }
+
+ void putRows(int pid, char *values, long bytes, size_t slot, long latencyMark, long retainMark, int wraps) override {
+ if (values == nullptr || bytes <= 0)
+ throw std::invalid_argument("error: cannot put null to circular buffer");
+
+ auto id = slot;
+ auto slotId = id % m_numberOfSlots;
+ //m_slots[slotId].m_slot.store(0);
+ while (m_slots[slotId].m_slot.load() != 0) {
+ std::cout << "error: inserting data to slot " << slotId
+ << " is blocked with oldVal " << m_slots[slotId].m_slot.load() << std::endl;
+ exit(1);
+ _mm_pause();
+ }
+
+ auto index = slotId * m_batchSize;
+
+ if (m_debug) {
+ std::cout << "[DBG] CB " + std::to_string(m_id) + " Worker writing to slot " + std::to_string(slotId)
+ + " with " + std::to_string(index) + " startPointer"
+ + " and " + std::to_string(index+bytes) + " endPointer " << std::endl;
+ }
+
+ // check if some async calls have finished
+ // m_filesystem->getHandler().tryCompleteMultiple();
+ // tryToAcknowledge();
+
+ if (!retainMark) {
+ if (SystemConf::getInstance().LINEAGE_ON) {
+ if (!m_slots[slotId].m_graph)
+ throw std::runtime_error("error: the lineage graph is not initialized for slot " + std::to_string(slotId));
+ auto bufferId = (m_id % 2 == 0) ? 0 : 1;
+ auto fptr = m_slots[slotId].m_fptr; //m_fileStore->getFilePtr(m_query->getId(), bufferId, slot * m_batchSize);
+ }
+ auto oldVal = 0;
+ m_slots[slotId].m_slot.compare_exchange_weak(oldVal, 1);
+ // check if some async calls have finished
+ m_filesystem->getHandler().tryCompleteMultiple();
+ std::cout << "warning: no retain marker was used" << std::endl;
+ return;
+ }
+
+ // async write here
+ auto copyBuffer = PartialWindowResultsFactory::getInstance().newInstance(pid);
+ AckSlotContext context{&m_slots[slotId].m_slot, copyBuffer};
+ auto callback = [](IAsyncContext *ctxt, Status result,
+ size_t bytes_transferred) {
+ CallbackContext context{ctxt};
+ if (result != Status::Ok) {
+ fprintf(stderr, "AsyncFlushPages(), error: %u\n",
+ static_cast(result));
+ }
+
+ // std::cout << "[DBG] callback setting the slot status with "
+ // << bytes_transferred << " bytes_transferred" << std::endl;
+ // Set the slot status to ready
+ auto oldVal = 0;
+ while (!context->m_slot->compare_exchange_weak(oldVal, 1)) {
+ std::cout << "warning: callback (" << std::this_thread::get_id()
+ << ") blocked with oldVal " << oldVal << std::endl;
+ _mm_pause();
+ }
+ if (context->m_parW) {
+ PartialWindowResultsFactory::getInstance().free(context->m_parW->getThreadId(), context->m_parW);
+ context->m_parW.reset();
+ }
+ };
+
+ if (m_copyDataOnInsert || wraps == 0) { // copy only until the buffer is filled once
+ std::memcpy(&m_buffer[index], values, bytes);
+ }
+ if (SystemConf::getInstance().LATENCY_ON && !m_copyDataOnInsert) {
+ long *buf = (long *)&m_buffer[index];
+ buf[0] = Utils::pack(latencyMark, (int)buf[0]);
+ }
+
+ // memcpy may succeed after the write on disk in a multi-threaded scenario!
+ m_slots[slotId].m_memcpyFinished.store(true);
+
+ int diskBytes = 0;
+ char *diskValues = &m_buffer[index];
+ bool clear = false;
+ if (m_compress) {
+ if (SystemConf::getInstance().ADAPTIVE_COMPRESSION_ON && m_startInstr && id % 128 == 0) {
+ // results
+ uint32_t *dVs = nullptr;
+ double *cVs = nullptr, *mns = nullptr, *mxs = nullptr, *mxDs = nullptr;
+ m_instrFP(pid, values, (int) bytes, dVs, cVs, mns, mxs, mxDs);
+ m_compStats->addStatistics(dVs, cVs, mns, mxs, mxDs);
+ }
+ int metadataSize = 128;
+ if (pid >= m_copyBuffers.size())
+ throw std::runtime_error("error: invalid pid for data insertions with compression");
+
+ //m_compressionFP[m_compPos](pid, values, 0, (int) bytes, m_copyBuffers[pid].data() + metadataSize, diskBytes, (int) m_copyBuffers[pid].size(), clear, -1);
+ m_compressionFP[m_compPos](pid, values, 0, (int) bytes, copyBuffer->getBufferRaw() + metadataSize, diskBytes, (int) copyBuffer->getCapacity(), clear, -1);
+ // if compression fails, fall back to the initial compression scheme
+ if (clear) {
+ //std::cout << "[DBG] falling back to the initial compression scheme" << std::endl;
+ //m_compressionFP[0](pid, values, 0, (int) bytes, m_copyBuffers[pid].data() + metadataSize, diskBytes, (int) m_copyBuffers[pid].size(), clear, -1);
+ m_compressionFP[0](pid, values, 0, (int) bytes, copyBuffer->getBufferRaw() + metadataSize, diskBytes, (int) copyBuffer->getCapacity(), clear, -1);
+ m_compPos = 0;
+ }
+ diskBytes += metadataSize;
+ latencyMark = (SystemConf::getInstance().LATENCY_ON) ? latencyMark : -1;
+ if (clear || m_compPos == 0) {
+ //m_compressionFP[0](pid, values, 0, -1, m_copyBuffers[pid].data(), diskBytes, (int) m_copyBuffers[pid].size(), clear, latencyMark);
+ m_compressionFP[0](pid, values, 0, -1, copyBuffer->getBufferRaw(), diskBytes, (int) copyBuffer->getCapacity(), clear, latencyMark);
+ }
+ //diskValues = m_copyBuffers[pid].data();
+ diskValues = copyBuffer->getBufferRaw();
+
+ m_storedBytes.fetch_add(diskBytes);
+ m_storedCounter.fetch_add(1);
+ //diskBytes = 64 * 1024;//bytes;
+ } else {
+ diskBytes = bytes;
+ }
+ diskBytes = roundOffset(Utils::getPowerOfTwo(diskBytes));
+#if defined(NO_DISK)
+ diskBytes = 0;
+#endif
+ m_root->updateBlockSize(diskBytes);
+
+ if (m_debug) {
+ std::cout << "[DBG] CB " + std::to_string(m_id) + " Worker compressing data for slot " + std::to_string(slotId)
+ + " of " + std::to_string(bytes) + " bytes to "
+ + std::to_string(diskBytes) + " with "
+ + std::to_string((double)bytes/(double)diskBytes) + " ratio " << std::endl;
+ }
+
+ if (diskBytes > m_batchSize)
+ throw std::runtime_error("error: the write exceeds the size of slots in the input log");
+ if (!SystemConf::getInstance().LINEAGE_ON) {
+ assert(m_asyncFile->writeAsync(reinterpret_cast(diskValues),
+ slotId * m_batchSize, diskBytes, callback,
+ context) == Status::Ok);
+ } else {
+ if (!m_slots[slotId].m_graph)
+ throw std::runtime_error("error: the lineage graph is not initialized for slot " + std::to_string(slotId));
+ auto bufferId = (m_id % 2 == 0) ? 0 : 1;
+ auto fptr = m_slots[slotId].m_fptr; //m_fileStore->getFilePtr(m_query->getId(), bufferId, slot * m_batchSize);
+ assert(fptr->writeAsync(reinterpret_cast(diskValues),
+ slotId * m_batchSize, diskBytes, callback,
+ context) == Status::Ok);
+ }
+ // copyBuffer.reset();
+
+ // check if some async calls have finished
+ m_filesystem->getHandler().tryCompleteMultiple();
+ //tryToAcknowledge();
+ }
+
+ void putRows(int pid, std::shared_ptr &values, long bytes, size_t slot, long latencyMark, long retainMark, int wraps) override {
+ if (values == nullptr || bytes <= 0)
+ throw std::invalid_argument("error: cannot put null to circular buffer");
+
+ auto id = slot;
+ auto slotId = id % m_numberOfSlots;
+ //m_slots[slotId].m_slot.store(0);
+ while (m_slots[slotId].m_slot.load() != 0) {
+ std::cout << "error: inserting data to slot " << slotId
+ << " is blocked with oldVal " << m_slots[slotId].m_slot.load() << std::endl;
+ exit(1);
+ _mm_pause();
+ }
+
+ auto index = slotId * m_batchSize;
+
+ if (m_debug) {
+ std::cout << "[DBG] CB " + std::to_string(m_id) + " Worker writing to slot " + std::to_string(slotId)
+ + " with " + std::to_string(index) + " startPointer"
+ + " and " + std::to_string(index+bytes) + " endPointer " << std::endl;
+ }
+
+ // check if some async calls have finished
+ // m_filesystem->getHandler().tryCompleteMultiple();
+ // tryToAcknowledge();
+
+ if (!retainMark) {
+ if (SystemConf::getInstance().LINEAGE_ON) {
+ if (!m_slots[slotId].m_graph)
+ throw std::runtime_error("error: the lineage graph is not initialized for slot " + std::to_string(slotId));
+ auto bufferId = (m_id % 2 == 0) ? 0 : 1;
+ auto fptr = m_slots[slotId].m_fptr; //m_fileStore->getFilePtr(m_query->getId(), bufferId, slot * m_batchSize);
+ }
+ auto oldVal = 0;
+ m_slots[slotId].m_slot.compare_exchange_weak(oldVal, 1);
+ // check if some async calls have finished
+ m_filesystem->getHandler().tryCompleteMultiple();
+ std::cout << "warning: no retain marker was used" << std::endl;
+ return;
+ }
+
+ // async write here
+ auto copyBuffer = PartialWindowResultsFactory::getInstance().newInstance(pid);
+ AckSlotContext context{&m_slots[slotId].m_slot, copyBuffer};
+ auto callback = [](IAsyncContext *ctxt, Status result,
+ size_t bytes_transferred) {
+ CallbackContext context{ctxt};
+ if (result != Status::Ok) {
+ fprintf(stderr, "AsyncFlushPages(), error: %u\n",
+ static_cast(result));
+ }
+
+ // std::cout << "[DBG] callback setting the slot status with "
+ // << bytes_transferred << " bytes_transferred" << std::endl;
+ // Set the slot status to ready
+ auto oldVal = 0;
+ while (!context->m_slot->compare_exchange_weak(oldVal, 1)) {
+ std::cout << "warning: callback (" << std::this_thread::get_id()
+ << ") blocked with oldVal " << oldVal << std::endl;
+ _mm_pause();
+ }
+ if (context->m_parW) {
+ PartialWindowResultsFactory::getInstance().free(context->m_parW->getThreadId(), context->m_parW);
+ context->m_parW.reset();
+ }
+ };
+
+ if (m_copyDataOnInsert || wraps == 0) { // copy only until the buffer is filled once
+ std::memcpy(&m_buffer[index], values->getBuffer().data(), bytes);
+ }
+ if (SystemConf::getInstance().LATENCY_ON && !m_copyDataOnInsert) {
+ long *buf = (long *)&m_buffer[index];
+ buf[0] = Utils::pack(latencyMark, (int)buf[0]);
+ }
+
+ // memcpy may succeed after the write on disk in a multi-threaded scenario!
+ m_slots[slotId].m_memcpyFinished.store(true);
+
+ int diskBytes = 0;
+ char *diskValues = &m_buffer[index];
+ bool clear = false;
+ if (m_compress) {
+ if (SystemConf::getInstance().ADAPTIVE_COMPRESSION_ON && m_startInstr && id % 128 == 0) {
+ // results
+ uint32_t *dVs = nullptr;
+ double *cVs = nullptr, *mns = nullptr, *mxs = nullptr, *mxDs = nullptr;
+ m_instrFP(pid, values->getBuffer().data(), (int) bytes, dVs, cVs, mns, mxs, mxDs);
+ m_compStats->addStatistics(dVs, cVs, mns, mxs, mxDs);
+ }
+ int metadataSize = 128;
+ if (pid >= m_copyBuffers.size())
+ throw std::runtime_error("error: invalid pid for data insertions with compression");
+
+ //m_compressionFP[m_compPos](pid, values, 0, (int) bytes, m_copyBuffers[pid].data() + metadataSize, diskBytes, (int) m_copyBuffers[pid].size(), clear, -1);
+ m_compressionFP[m_compPos](pid, values->getBuffer().data(), 0, (int) bytes, copyBuffer->getBufferRaw() + metadataSize, diskBytes, (int) copyBuffer->getCapacity(), clear, -1);
+ // if compression fails, fall back to the initial compression scheme
+ if (clear) {
+ //std::cout << "[DBG] falling back to the initial compression scheme" << std::endl;
+ //m_compressionFP[0](pid, values, 0, (int) bytes, m_copyBuffers[pid].data() + metadataSize, diskBytes, (int) m_copyBuffers[pid].size(), clear, -1);
+ m_compressionFP[0](pid, values->getBuffer().data(), 0, (int) bytes, copyBuffer->getBufferRaw() + metadataSize, diskBytes, (int) copyBuffer->getCapacity(), clear, -1);
+ m_compPos = 0;
+ }
+ diskBytes += metadataSize;
+ latencyMark = (SystemConf::getInstance().LATENCY_ON) ? latencyMark : -1;
+ if (clear || m_compPos == 0) {
+ //m_compressionFP[0](pid, values, 0, -1, m_copyBuffers[pid].data(), diskBytes, (int) m_copyBuffers[pid].size(), clear, latencyMark);
+ m_compressionFP[0](pid, values->getBuffer().data(), 0, -1, copyBuffer->getBufferRaw(), diskBytes, (int) copyBuffer->getCapacity(), clear, latencyMark);
+ }
+ //diskValues = m_copyBuffers[pid].data();
+ diskValues = copyBuffer->getBufferRaw();
+
+ m_storedBytes.fetch_add(diskBytes);
+ m_storedCounter.fetch_add(1);
+ //diskBytes = 64 * 1024;//bytes;
+
+ // free UnboundedQueryBuffer
+ UnboundedQueryBufferFactory::getInstance().freeNB(values->getBufferId(), values);
+ } else {
+ diskBytes = bytes;
+ }
+ diskBytes = roundOffset(Utils::getPowerOfTwo(diskBytes));
+#if defined(NO_DISK)
+ diskBytes = 0;
+#endif
+ m_root->updateBlockSize(diskBytes);
+
+ if (m_debug) {
+ std::cout << "[DBG] CB " + std::to_string(m_id) + " Worker compressing data for slot " + std::to_string(slotId)
+ + " of " + std::to_string(bytes) + " bytes to "
+ + std::to_string(diskBytes) + " with "
+ + std::to_string((double)bytes/(double)diskBytes) + " ratio " << std::endl;
+ }
+
+ if (diskBytes > m_batchSize)
+ throw std::runtime_error("error: the write exceeds the size of slots in the input log");
+ if (!SystemConf::getInstance().LINEAGE_ON) {
+ assert(m_asyncFile->writeAsync(reinterpret_cast(diskValues),
+ slotId * m_batchSize, diskBytes, callback,
+ context) == Status::Ok);
+ } else {
+ if (!m_slots[slotId].m_graph)
+ throw std::runtime_error("error: the lineage graph is not initialized for slot " + std::to_string(slotId));
+ auto bufferId = (m_id % 2 == 0) ? 0 : 1;
+ auto fptr = m_slots[slotId].m_fptr; //m_fileStore->getFilePtr(m_query->getId(), bufferId, slot * m_batchSize);
+ assert(fptr->writeAsync(reinterpret_cast(diskValues),
+ slotId * m_batchSize, diskBytes, callback,
+ context) == Status::Ok);
+ }
+ // copyBuffer.reset();
+
+ // check if some async calls have finished
+ m_filesystem->getHandler().tryCompleteMultiple();
+ //tryToAcknowledge();
+ }
+
+ void putRows(int pid, void *val, long bytes, size_t slot, long latencyMark, long retainMark, int wraps) override {
+ auto *values = (infinity::core::receive_element_t *) val;
+
+ if (values == nullptr || bytes <= 0)
+ throw std::invalid_argument("error: cannot put null to circular buffer");
+
+ /*for (auto ii = 0; ii < bytes; ii+=8) {
+ __builtin_prefetch(((char*) values->buffer->getData() + ii), 1, 3);
+ }*/
+
+ auto id = slot;
+ auto slotId = id % m_numberOfSlots;
+ //m_slots[slotId].m_slot.store(0);
+ while (m_slots[slotId].m_slot.load() != 0) {
+ std::cout << "error: inserting data to slot " << slotId
+ << " is blocked with oldVal " << m_slots[slotId].m_slot.load() << std::endl;
+ exit(1);
+ _mm_pause();
+ }
+
+ auto index = slotId * m_batchSize;
+
+ if (m_debug) {
+ std::cout << "[DBG] CB " + std::to_string(m_id) + " Worker writing to slot " + std::to_string(slotId)
+ + " with " + std::to_string(index) + " startPointer"
+ + " and " + std::to_string(index+bytes) + " endPointer " << std::endl;
+ }
+
+ // check if some async calls have finished
+ // m_filesystem->getHandler().tryCompleteMultiple();
+ // tryToAcknowledge();
+
+ if (!retainMark) {
+ if (SystemConf::getInstance().LINEAGE_ON) {
+ if (!m_slots[slotId].m_graph)
+ throw std::runtime_error("error: the lineage graph is not initialized for slot " + std::to_string(slotId));
+ auto bufferId = (m_id % 2 == 0) ? 0 : 1;
+ auto fptr = m_slots[slotId].m_fptr; //m_fileStore->getFilePtr(m_query->getId(), bufferId, slot * m_batchSize);
+ }
+ auto oldVal = 0;
+ m_slots[slotId].m_slot.compare_exchange_weak(oldVal, 1);
+ // check if some async calls have finished
+ m_filesystem->getHandler().tryCompleteMultiple();
+ std::cout << "warning: no retain marker was used" << std::endl;
+ return;
+ }
+
+ // async write here
+ auto copyBuffer = PartialWindowResultsFactory::getInstance().newInstance(pid);
+ AckSlotContext context{&m_slots[slotId].m_slot, copyBuffer};
+ auto callback = [](IAsyncContext *ctxt, Status result,
+ size_t bytes_transferred) {
+ CallbackContext context{ctxt};
+ if (result != Status::Ok) {
+ fprintf(stderr, "AsyncFlushPages(), error: %u\n",
+ static_cast(result));
+ }
+
+ // std::cout << "[DBG] callback setting the slot status with "
+ // << bytes_transferred << " bytes_transferred" << std::endl;
+ // Set the slot status to ready
+ auto oldVal = 0;
+ while (!context->m_slot->compare_exchange_weak(oldVal, 1)) {
+ std::cout << "warning: callback (" << std::this_thread::get_id()
+ << ") blocked with oldVal " << oldVal << std::endl;
+ _mm_pause();
+ }
+ if (context->m_parW) {
+ PartialWindowResultsFactory::getInstance().free(context->m_parW->getThreadId(), context->m_parW);
+ context->m_parW.reset();
+ }
+ };
+
+ if (m_copyDataOnInsert || wraps == 0) { // copy only until the buffer is filled once
+ //std::memcpy(&m_buffer[index], values->buffer->getData(), bytes);
+ size_t idx = 0;
+ if (m_filter) {
+ m_filterFP((char*) values->buffer->getData(), 0, (int) bytes, &m_buffer[index], 0, reinterpret_cast(idx));
+ } else {
+ std::memcpy(&m_buffer[index], values->buffer->getData(), bytes);
+ }
+ }
+ //if (wraps == 0 && !m_copyDataOnInsert) {
+ // std::cout << " I am not copying data " << std::endl;
+ //}
+ if (SystemConf::getInstance().LATENCY_ON && !m_copyDataOnInsert) {
+ long *buf = (long *)&m_buffer[index];
+ buf[0] = Utils::pack(latencyMark, (int)buf[0]);
+ }
+
+ // memcpy may succeed after the write on disk in a multi-threaded scenario!
+ m_slots[slotId].m_memcpyFinished.store(true);
+
+ int diskBytes = 0;
+ char *diskValues = &m_buffer[index];
+ bool clear = false;
+ if (m_compress) {
+ if (SystemConf::getInstance().ADAPTIVE_COMPRESSION_ON && m_startInstr && id % 128 == 0) {
+ // results
+ uint32_t *dVs = nullptr;
+ double *cVs = nullptr, *mns = nullptr, *mxs = nullptr, *mxDs = nullptr;
+ m_instrFP(pid, (char*) values->buffer->getData(), (int) bytes, dVs, cVs, mns, mxs, mxDs);
+ m_compStats->addStatistics(dVs, cVs, mns, mxs, mxDs);
+ }
+ int metadataSize = 128;
+ if (pid >= m_copyBuffers.size())
+ throw std::runtime_error("error: invalid pid for data insertions with compression");
+
+ //m_compressionFP[m_compPos](pid, values, 0, (int) bytes, m_copyBuffers[pid].data() + metadataSize, diskBytes, (int) m_copyBuffers[pid].size(), clear, -1);
+ m_compressionFP[m_compPos](pid, (char*) values->buffer->getData(), 0, (int) bytes, copyBuffer->getBufferRaw() + metadataSize, diskBytes, (int) copyBuffer->getCapacity(), clear, -1);
+ // if compression fails, fall back to the initial compression scheme
+ if (clear) {
+ //std::cout << "[DBG] falling back to the initial compression scheme" << std::endl;
+ //m_compressionFP[0](pid, values, 0, (int) bytes, m_copyBuffers[pid].data() + metadataSize, diskBytes, (int) m_copyBuffers[pid].size(), clear, -1);
+ m_compressionFP[0](pid, (char*) values->buffer->getData(), 0, (int) bytes, copyBuffer->getBufferRaw() + metadataSize, diskBytes, (int) copyBuffer->getCapacity(), clear, -1);
+ m_compPos = 0;
+ }
+ diskBytes += metadataSize;
+ latencyMark = (SystemConf::getInstance().LATENCY_ON) ? latencyMark : -1;
+ if (clear || m_compPos == 0) {
+ //m_compressionFP[0](pid, values, 0, -1, m_copyBuffers[pid].data(), diskBytes, (int) m_copyBuffers[pid].size(), clear, latencyMark);
+ m_compressionFP[0](pid, (char*) values->buffer->getData(), 0, -1, copyBuffer->getBufferRaw(), diskBytes, (int) copyBuffer->getCapacity(), clear, latencyMark);
+ }
+ //diskValues = m_copyBuffers[pid].data();
+ diskValues = copyBuffer->getBufferRaw();
+
+ m_storedBytes.fetch_add(diskBytes);
+ m_storedCounter.fetch_add(1);
+ //diskBytes = 64 * 1024;//bytes;
+ } else {
+ diskBytes = bytes;
+ }
+ // free rdma buffer
+ RDMABufferPool::getInstance().free(values);
+
+ diskBytes = roundOffset(Utils::getPowerOfTwo(diskBytes));
+#if defined(NO_DISK)
+ diskBytes = 0;
+#endif
+ m_root->updateBlockSize(diskBytes);
+
+ if (m_debug) {
+ std::cout << "[DBG] CB " + std::to_string(m_id) + " Worker compressing data for slot " + std::to_string(slotId)
+ + " of " + std::to_string(bytes) + " bytes to "
+ + std::to_string(diskBytes) + " with "
+ + std::to_string((double)bytes/(double)diskBytes) + " ratio " << std::endl;
+ }
+
+ if (diskBytes > m_batchSize)
+ throw std::runtime_error("error: the write exceeds the size of slots in the input log");
+ if (!SystemConf::getInstance().LINEAGE_ON) {
+ assert(m_asyncFile->writeAsync(reinterpret_cast(diskValues),
+ slotId * m_batchSize, diskBytes, callback,
+ context) == Status::Ok);
+ } else {
+ if (!m_slots[slotId].m_graph)
+ throw std::runtime_error("error: the lineage graph is not initialized for slot " + std::to_string(slotId));
+ auto bufferId = (m_id % 2 == 0) ? 0 : 1;
+ auto fptr = m_slots[slotId].m_fptr; //m_fileStore->getFilePtr(m_query->getId(), bufferId, slot * m_batchSize);
+ assert(fptr->writeAsync(reinterpret_cast(diskValues),
+ slotId * m_batchSize, diskBytes, callback,
+ context) == Status::Ok);
+ }
+ // copyBuffer.reset();
+
+ // check if some async calls have finished
+ m_filesystem->getHandler().tryCompleteMultiple();
+ //tryToAcknowledge();
+ }
+
+ long recover(int &bytes) override {
+ if (!SystemConf::getInstance().LINEAGE_ON) {
+ throw std::runtime_error("error: lineage must be enabled for recovery");
+ }
+ if (m_numberOfSlotsToRecover == 0) {
+ return -1;
+ }
+
+ if (m_emptySlots <= 0) {
+ m_filesystem->getHandler().tryCompleteMultiple();
+ tryToAcknowledge();
+ return 0;
+ }
+
+ /* Get the slot to write */
+ auto slot = getNextSlotToWrite();
+ if (slot >= m_numberOfSlots)
+ m_wraps = 1;
+ auto slotId = slot % m_numberOfSlots;
+ auto index = slotId * m_batchSize;
+ auto end = (m_compress) ? slotId * SystemConf::getInstance().BLOCK_SIZE : slotId * m_batchSize;
+ long readEnd = m_pop.root()->m_startP.get_ro().load() + end;
+ long readIndex = normalise(readEnd);
+
+ auto graph = LineageGraphFactory::getInstance().newInstance();
+
+ m_slots[slotId].setLineageGraph(graph);
+ auto bufferId = (m_id % 2 == 0) ? 0 : 1;
+ auto fptr = m_fileStore->getUnsafeFilePtr(m_query->getId(), bufferId, readEnd, m_pop.root()->m_startId.get_ro().load());
+ m_slots[slotId].m_fptr = fptr;
+
+ AckSlotContext context{&m_slots[slotId].m_slot};
+ auto callback = [](IAsyncContext *ctxt, Status result,
+ size_t bytes_transferred) {
+ CallbackContext context{ctxt};
+ if (result != Status::Ok) {
+ fprintf(stderr, "AsyncFlushPages(), error: %u\n",
+ static_cast(result));
+ }
+
+ // std::cout << "[DBG] callback setting the slot status with "
+ // << bytes_transferred << " bytes_transferred" << std::endl;
+ // Set the slot status to ready
+ auto oldVal = 0;
+ while (!context->m_slot->compare_exchange_weak(oldVal, 1)) {
+ std::cout << "warning: callback (" << std::this_thread::get_id()
+ << ") blocked with oldVal " << oldVal << std::endl;
+ _mm_pause();
+ }
+ };
+
+ auto oldVal = -1;
+ while (!m_slots[slotId].m_slot.compare_exchange_weak(oldVal, 0)) {
+ std::cout << "warning: (during recovery) adding data to slot " << slotId
+ << " is blocked with oldVal " << oldVal << std::endl;
+ _mm_pause();
+ }
+
+ bytes = (!m_compress) ? m_batchSize : std::min((size_t)m_previousBlockSize, m_batchSize);
+ if (m_copyDataOnInsert) {
+ assert(fptr->readAsync(readIndex, &m_buffer[index], bytes,
+ callback, context) == Status::Ok);
+ } else {
+ //std::cout << "reading async data for slot " << slotId << std::endl;
+ assert(fptr->readAsync(0, &m_buffer[index], bytes,
+ callback, context) == Status::Ok);
+ }
+
+ m_slots[slotId].m_memcpyFinished.store(true);
+
+ // reduce number of slots
+ m_numberOfSlotsToRecover--;
+ m_emptySlots.fetch_add(-1);
+
+ if (m_debug) {
+ debug();
+ }
+
+ m_filesystem->getHandler().tryCompleteMultiple();
+ tryToAcknowledge();
+ return 0;
+ }
+
+ void prepareRecovery() override {
+ if (!m_copyDataOnInsert) {
+ if (m_numberOfSlotsToRecover < m_numberOfSlots) {
+ long readEnd = m_pop.root()->m_startP.get_ro().load();
+ long readIndex = normalise(readEnd);
+ auto bufferId = (m_id % 2 == 0) ? 0 : 1;
+ auto fptr = m_fileStore->getUnsafeFilePtr(m_query->getId(), bufferId, readEnd, m_pop.root()->m_startId.get_ro().load());
+ if (m_compress) {
+ auto sIndex = m_numberOfSlotsToRecover * m_batchSize;
+ auto readBytes = std::min((size_t)m_previousBlockSize, m_batchSize);
+ assert(fptr->readSync(0, m_copyBuffers[0].data(), readBytes) == Status::Ok);
+ int writePos = 0;
+ bool copy = false;
+ auto latency = (SystemConf::getInstance().LATENCY_ON) ? 0 : -1;
+ m_decompressionFP[m_compPos](0, &m_buffer[sIndex], 0, m_batchSize, m_copyBuffers[0].data(), writePos, SystemConf::getInstance().BLOCK_SIZE, copy, latency);
+ if (writePos != SystemConf::getInstance().BATCH_SIZE)
+ throw std::runtime_error("error: the write position is not equal to the batch size after decompression");
+ //std::cout << "[DBG] decompressing for slot " << m_numberOfSlotsToRecover << " and index " << sIndex << std::endl;
+ for (size_t slotId = m_numberOfSlotsToRecover + 1; slotId < m_numberOfSlots; ++slotId) {
+ auto index = slotId * m_batchSize;
+ std::memcpy(&m_buffer[index], &m_buffer[sIndex], m_batchSize);
+ //std::cout << "[DBG] copying for slot " << slotId << " and index " << index << std::endl;
+ }
+ } else {
+ assert(fptr->readSync(0, m_copyBuffers[0].data(), m_batchSize) == Status::Ok);
+ for (size_t slotId = m_numberOfSlotsToRecover; slotId < m_numberOfSlots; ++slotId) {
+ auto index = slotId * m_batchSize;
+ std::memcpy(&m_buffer[index], m_copyBuffers[0].data(), m_batchSize);
+ }
+ }
+ }
+ m_wraps = 1;
+ }
+
+ }
+
+ bool tryConsumeNextSlot(long &index, int &length, bool recover = false) {
+ if (recover) {
+ m_filesystem->getHandler().tryCompleteMultiple();
+ tryToAcknowledge();
+ }
+ bool found = false;
+ if (isSlotReady(m_readerSlot)) {
+ if (m_debug) {
+ std::cout << "[DBG] CB " + std::to_string(m_id) + " Creating processing task for slot " + std::to_string(m_readerSlot)
+ + " with reader slot " + std::to_string(m_readerSlot) << std::endl;
+ }
+ m_slots[m_readerSlot].m_slot.store(3);
+ index = m_readerSlot * m_batchSize;
+ length = m_batchSize;
+ m_readerSlot++;
+ if (m_readerSlot == getNumberOfSlots()) {
+ m_readerSlot = 0;
+ }
+ found = true;
+ }
+ return found;
+ }
+
+ void free() override {
+ throw std::invalid_argument("error: this operator is not supported yet");
+ }
+
+ void free(long offset, bool isPersistent = false) override {
+ if (SystemConf::getInstance().LINEAGE_ON && !isPersistent)
+ return;
+
+ long _start = m_startP.m_value.load(std::memory_order_relaxed);
+
+ long index = normalise(_start);
+ long bytes;
+ /* Measurements */
+ if (offset <= index)
+ bytes = m_capacity - index + offset + 1;
+ else
+ bytes = offset - index + 1;
+
+ m_bytesProcessed.fetch_add(bytes, std::memory_order_relaxed);
+ m_tuplesProcessed.fetch_add((bytes / (size_t)m_tupleSize),
+ std::memory_order_relaxed);
+ m_tasksProcessed.fetch_add(1, std::memory_order_relaxed);
+
+ if (m_debug) {
+ std::cout << "[DBG] CB " + std::to_string(m_id) + " Freeing slot " + std::to_string(m_nextFreeSlot)
+ + " with empty slots " + std::to_string(m_emptySlots) << std::endl;
+ }
+
+ m_slots[m_nextFreeSlot].m_memcpyFinished.store(false);
+ m_slots[m_nextFreeSlot].m_slot.store(-1);
+ m_emptySlots.fetch_add(1);
+
+ m_nextFreeSlot++;
+ if (m_nextFreeSlot == m_numberOfSlots) m_nextFreeSlot = 0;
+
+ /* Set new start pointer */
+ m_startP.m_value.store(_start + bytes, std::memory_order_relaxed);
+ m_root->m_bytesProcessed.get_rw().fetch_add(bytes,
+ std::memory_order_release);
+ //m_root->m_startP.get_rw().store(_start + bytes, std::memory_order_release);
+ }
+
+ ByteBuffer &getBuffer() override { return m_buffer; }
+
+ char *getBufferRaw() override { return m_buffer.data(); }
+
+ size_t getBufferCapacity(int id) override {
+ (void)id;
+ return m_capacity;
+ }
+
+ long getLong(size_t index) override {
+ auto p = (long *)m_buffer.data();
+ return p[normalise(index) / sizeof(size_t)];
+ }
+
+ void setLong(size_t index, long value) override {
+ auto p = (long *)m_buffer.data();
+ p[normalise(index) / sizeof(size_t)] = value;
+ }
+
+ void appendBytesTo(int startPos, int endPos,
+ ByteBuffer &outputBuffer) override {
+ if (endPos > startPos) {
+ std::copy(m_buffer.begin() + startPos, m_buffer.begin() + endPos,
+ outputBuffer.begin());
+ } else {
+ std::copy(m_buffer.begin() + startPos, m_buffer.end(),
+ outputBuffer.begin());
+ std::copy(m_buffer.begin(), m_buffer.begin() + endPos,
+ outputBuffer.begin() + (m_capacity - startPos));
+ }
+ }
+
+ void appendBytesTo(int startPos, int endPos, char *output) override {
+ if (endPos > startPos) {
+ std::memcpy(output, m_buffer.data() + startPos,
+ (endPos - startPos) * sizeof(char));
+ } else {
+ std::memcpy(output, m_buffer.data() + startPos,
+ (m_capacity - startPos) * sizeof(char));
+ std::memcpy(output + (m_capacity - startPos), m_buffer.data(),
+ (endPos) * sizeof(char));
+ }
+ }
+
+ void createInsertionTask(char *values, long bytes, long latencyMark = -1, long retainMark = -1, std::shared_ptr graph = nullptr) {
+ auto batch = WindowBatchFactory::getInstance().newInstance(
+ 0, 0, -1, -1, m_query, nullptr,
+ &m_query->getWindowDefinition(), m_query->getSchema(),
+ -1);
+ batch->setTaskType(TaskType::INSERT);
+ auto task =
+ TaskFactory::getInstance().newInstance(0, batch, nullptr, TaskType::INSERT);
+ auto slot = getNextSlotToWrite();
+ auto slotId = slot % m_numberOfSlots;
+ if (slot >= m_numberOfSlots) {
+ m_wraps = 1;
+ }
+ if (SystemConf::getInstance().LINEAGE_ON) {
+ if (!graph)
+ graph = LineageGraphFactory::getInstance().newInstance();
+ m_slots[slotId].setLineageGraph(graph);
+ graph.reset();
+ if (!m_slots[slotId].m_graph)
+ throw std::runtime_error("error: the lineage graph is not initialized before task creation for slot " + std::to_string(slot));
+ auto bufferId = (m_id % 2 == 0) ? 0 : 1;
+ auto fptr = m_fileStore->getFilePtr(m_query->getId(), bufferId, slot * m_batchSize);
+ m_slots[slotId].m_fptr = fptr;
+ }
+
+ auto oldVal = -1;
+ while (!m_slots[slotId].m_slot.compare_exchange_weak(oldVal, 0)) {
+ std::cout << "warning: adding data to slot " << slotId
+ << " is blocked with oldVal " << oldVal << std::endl;
+ _mm_pause();
+ }
+
+ if (m_debug) {
+ std::cout << "slot " << slotId << " is set to " << m_slots[slotId].m_slot.load() << std::endl;
+ }
+ task->setInsertion(values, bytes, slot, latencyMark, retainMark, m_wraps);
+ while (!m_query->getTaskQueue()->try_enqueue(task))
+ ;
+ }
+
+ void createInsertionTask(std::shared_ptr &values, long bytes, long latencyMark = -1, long retainMark = -1, std::shared_ptr graph = nullptr) {
+ auto batch = WindowBatchFactory::getInstance().newInstance(
+ 0, 0, -1, -1, m_query, nullptr,
+ &m_query->getWindowDefinition(), m_query->getSchema(),
+ -1);
+ batch->setTaskType(TaskType::INSERT);
+ auto task =
+ TaskFactory::getInstance().newInstance(0, batch, nullptr, TaskType::INSERT);
+ auto slot = getNextSlotToWrite();
+ auto slotId = slot % m_numberOfSlots;
+ if (slot >= m_numberOfSlots) {
+ m_wraps = 1;
+ }
+ if (SystemConf::getInstance().LINEAGE_ON) {
+ if (!graph)
+ graph = LineageGraphFactory::getInstance().newInstance();
+ m_slots[slotId].setLineageGraph(graph);
+ graph.reset();
+ if (!m_slots[slotId].m_graph)
+ throw std::runtime_error("error: the lineage graph is not initialized before task creation for slot " + std::to_string(slot));
+ auto bufferId = (m_id % 2 == 0) ? 0 : 1;
+ auto fptr = m_fileStore->getFilePtr(m_query->getId(), bufferId, slot * m_batchSize);
+ m_slots[slotId].m_fptr = fptr;
+ }
+
+ auto oldVal = -1;
+ while (!m_slots[slotId].m_slot.compare_exchange_weak(oldVal, 0)) {
+ std::cout << "warning: adding data to slot " << slotId
+ << " is blocked with oldVal " << oldVal << std::endl;
+ _mm_pause();
+ }
+
+ if (m_debug) {
+ std::cout << "slot " << slotId << " is set to " << m_slots[slotId].m_slot.load() << std::endl;
+ }
+ task->setInsertion(values, bytes, slot, latencyMark, retainMark, m_wraps);
+ while (!m_query->getTaskQueue()->try_enqueue(task))
+ ;
+ }
+
+ void createInsertionTask(void *values, long bytes, long latencyMark = -1, long retainMark = -1, std::shared_ptr graph = nullptr) {
+ auto batch = WindowBatchFactory::getInstance().newInstance(
+ 0, 0, -1, -1, m_query, nullptr,
+ &m_query->getWindowDefinition(), m_query->getSchema(),
+ -1);
+ batch->setTaskType(TaskType::INSERT);
+ auto task =
+ TaskFactory::getInstance().newInstance(0, batch, nullptr, TaskType::INSERT);
+ auto slot = getNextSlotToWrite();
+ auto slotId = slot % m_numberOfSlots;
+ if (slot >= m_numberOfSlots) {
+ m_wraps = 1;
+ }
+ if (SystemConf::getInstance().LINEAGE_ON) {
+ if (!graph)
+ graph = LineageGraphFactory::getInstance().newInstance();
+ m_slots[slotId].setLineageGraph(graph);
+ graph.reset();
+ if (!m_slots[slotId].m_graph)
+ throw std::runtime_error("error: the lineage graph is not initialized before task creation for slot " + std::to_string(slot));
+ auto bufferId = (m_id % 2 == 0) ? 0 : 1;
+ auto fptr = m_fileStore->getFilePtr(m_query->getId(), bufferId, slot * m_batchSize);
+ m_slots[slotId].m_fptr = fptr;
+ }
+
+ auto oldVal = -1;
+ while (!m_slots[slotId].m_slot.compare_exchange_weak(oldVal, 0)) {
+ std::cout << "warning: adding data to slot " << slotId
+ << " is blocked with oldVal " << oldVal << std::endl;
+ _mm_pause();
+ }
+
+ if (m_debug) {
+ std::cout << "slot " << slotId << " is set to " << m_slots[slotId].m_slot.load() << std::endl;
+ }
+ task->setInsertion((void *)values, bytes, slot, latencyMark, retainMark, m_wraps);
+ while (!m_query->getTaskQueue()->try_enqueue(task))
+ ;
+ }
+
+ bool isPersistent() override { return true; }
+
+ void updateFileEndPtr(long id) override {
+ m_root->m_endId.get_rw().store(id);
+ }
+
+ void updateFileStartPtr(long id, long offset) override {
+ if (m_debug) {
+ std::cout << "[DBG] CB " + std::to_string(m_id) + " garbage collection: updating the start id to " + std::to_string(id)
+ + " and the offset to " + std::to_string(offset) + " (prev offset: "
+ + std::to_string(m_prevFreeOffset) + ")" << std::endl;
+ }
+ if (offset < m_prevFreeOffset) {
+ //throw std::runtime_error("error: trying to free an invalid offset " +
+ // std::to_string(offset) + " < " +
+ // std::to_string(m_prevFreeOffset));
+ //std::cout << "warning: trying to free an invalid offset in cqbuffer " +
+ // std::to_string(offset) + " < " + std::to_string(m_prevFreeOffset) << std::endl;
+ return;
+ }
+ m_prevFreeOffset = offset;
+
+ m_root->m_startId.get_rw().store(id);
+ m_root->m_startP.get_rw().store(offset);
+ }
+
+ void updateStepAndOffset(long step, long offset) {
+ m_root->m_step.get_rw().store(step);
+ m_root->m_offset.get_rw().store(offset);
+ }
+
+ void getStepAndOffset(long &step, long &offset) {
+ step = m_root->m_step.get_ro().load();
+ offset = m_root->m_startP.get_ro().load(); //m_root->m_offset.get_ro().load();
+ }
+
+ size_t getBytesProcessed() override { return m_root->m_bytesProcessed.get_ro(); }
+
+ size_t getUnsafeStartPointer() override {
+ return m_root->m_startP.get_ro().load(std::memory_order_relaxed);
+ }
+
+ void incrementUnsafeStartPointer(size_t offset) override {
+ auto start = m_root->m_startP.get_ro().load();
+ auto bufferId = (m_id % 2 == 0) ? 0 : 1;
+ m_fileStore->freePersistent(m_query->getId(), bufferId, start + offset);
+ }
+
+ size_t getUnsafeEndPointer() override {
+ return m_root->m_endP.get_ro().load(std::memory_order_relaxed);
+ }
+
+ size_t getUnsafeRemainingBytes() override {
+ auto start = m_root->m_startP.get_ro().load();
+ auto end = m_root->m_endP.get_ro().load();
+ return end-start;
+ }
+
+ bool isSlotReady(size_t slotId) {
+ checkSlotNumber(slotId);
+ return m_slots[slotId].m_slot.load() == 2 && m_slots[slotId].m_memcpyFinished;
+ }
+
+ void checkSlotNumber(size_t slotId) {
+ if (slotId >= m_numberOfSlots)
+ throw std::invalid_argument("error: slotId >= m_numberOfSlots");
+ }
+
+ size_t getSlotId(size_t index) { return std::floor(index / m_batchSize); }
+
+ void clearPersistentMemory() {
+ m_pop.close();
+ m_filesystem->eraseFiles();
+ }
+
+ ~PersistentCircularQueryBuffer() override {
+ m_pop.close();
+ };
+
+ private:
+ void tryToAcknowledge() {
+ try {
+ if (!m_ackLock.try_lock()) return;
+
+ while (true) {
+ m_filesystem->getHandler().tryCompleteMultiple();
+ auto slotId = m_nextPersistentSlot.load();
+ if (m_slots[slotId].m_slot.load() != 1) {
+ break;
+ }
+
+ if (m_debug) {
+ std::cout << "[DBG] CB " + std::to_string(m_id) + " Acknowledging slot " + std::to_string(slotId) << std::endl;
+ }
+
+ // m_endP.fetch_add(_4MB);
+ if (m_numberOfSlotsToFree == 0) {
+ m_root->m_endP.get_rw().fetch_add(m_batchSize, std::memory_order_release);
+ } else {
+ if (m_compress) {
+ auto index = slotId * m_batchSize;
+ int writePos = 0;
+ bool copy = true;
+ auto latency = (SystemConf::getInstance().LATENCY_ON) ? 0 : -1;
+ m_decompressionFP[m_compPos](0, &m_buffer[index], 0, m_batchSize, m_copyBuffers[0].data(), writePos, SystemConf::getInstance().BLOCK_SIZE, copy, latency);
+ if (writePos != SystemConf::getInstance().BATCH_SIZE)
+ throw std::runtime_error("error: the write position is not equal to the batch size after decompression");
+ //std::cout << "[DBG] decompressing for slot " << slotId << " and index " << index << std::endl;
+ }
+ m_numberOfSlotsToFree--;
+ }
+ m_endP.m_value.fetch_add(m_batchSize, std::memory_order_relaxed);
+
+ m_nextPersistentSlot.fetch_add(1);
+ m_slots[slotId].m_slot.store(2);
+
+ if (m_nextPersistentSlot.load() == m_numberOfSlots)
+ m_nextPersistentSlot.store(0);
+ }
+
+ m_ackLock.unlock();
+ } catch (std::exception &e) {
+ std::cout << e.what() << std::endl;
+ }
+ }
+
+ struct PMem {
+ pmem::obj::p> m_startP;
+ pmem::obj::p> m_endP;
+ pmem::obj::p> m_bytesProcessed;
+ pmem::obj::p> m_step;
+ pmem::obj::p> m_offset;
+ pmem::obj::p> m_startId;
+ pmem::obj::p> m_endId;
+ pmem::obj::p> m_blockSize;
+ pmem::obj::persistent_ptr next;
+ PMem() {
+ m_startP.get_rw() = 0L;
+ m_endP.get_rw() = 0L;
+ m_bytesProcessed.get_rw() = 0L;
+ m_blockSize.get_rw() = 0L;
+ };
+
+ void updateBlockSize(long const& value) {
+ auto prev_value = m_blockSize.get_ro().load();
+ while(prev_value < value &&
+ !m_blockSize.get_rw().compare_exchange_weak(prev_value, value))
+ {}
+ }
+
+ /** Copy constructor is deleted */
+ PMem(const PMem &) = delete;
+ /** Assignment operator is deleted */
+ PMem &operator=(const PMem &) = delete;
+ };
+};
\ No newline at end of file
diff --git a/src/buffers/PersistentNumaCircularQueryBuffer.h b/src/buffers/PersistentNumaCircularQueryBuffer.h
new file mode 100644
index 0000000..4179f69
--- /dev/null
+++ b/src/buffers/PersistentNumaCircularQueryBuffer.h
@@ -0,0 +1,1413 @@
+#pragma once
+
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include