snapuserd: Add I/O path support for variable block size
The flow of I/O path is as follows:
1: When there is a I/O request for a given sector, we first
check the in-memory COW operation mapping for that sector.
2: If the mapping of sector to COW operation is found, then the
existing I/O path will work seamlessly. Even if the COW operation
encodes multiple blocks, we will discard the remaining data.
3: If the mapping of sector to COW operation is not found:
a: Find the previous COW operation as the vector has sorted sectors.
b: If the previous COW operation is a REPLACE op:
i: Check if the current sector is encoded in the previous COW
operations compressed block.
ii: If the sector falls within the range of compressed blocks,
retrieve the block offset.
iii: De-compress the COW operation based on the compression
factor.
iv: memcpy the data based on the block offset.
v: cache the COW operation pointer as subsequent I/O requests
are sequential and can just be a memcpy at the correct offset.
c: If the previous COW operation is not a REPLACE op or if the
requested sector does not fall within the compression factor
of the previous COW operation, then fallback and read the data
from base device.
Snapshot-merge:
During merge of REPLACE ops, read the entire op in one shot, de-compress
multiple blocks and write all the blocks in one shot.
Performance:
go/variable-block-vabc-perf covers detail performance runs
on Pixel 6 for full and incremental OTA.
Bug: 319309466
Test: snapuserd_test covers all the I/O path with various block sizes.
About 252 cases with all combinations and tunables.
[==========] 252 tests from 4 test suites ran. (702565 ms total)
[ PASSED ] 252 tests.
On Pixel 6:
=======================================
COW Writer V3:
for i in full, incremental OTA
for j in 4k, 16k, 32k, 64k, 128, 256k
for k in lz4, zstd, gz
install OTA, reboot, verify merge
=======================================
COW Writer V2:
for i in full, incremental OTA
for j in 4k
for k in lz4, zstd, gz
install OTA, reboot, verity merge
=====================================
Change-Id: I4c3b5c3efa0d09677568b4396cc53db0e74e7c99
Signed-off-by: Akilesh Kailash <akailash@google.com>
diff --git a/fs_mgr/libsnapshot/include/libsnapshot/cow_format.h b/fs_mgr/libsnapshot/include/libsnapshot/cow_format.h
index 8f62234..6865b19 100644
--- a/fs_mgr/libsnapshot/include/libsnapshot/cow_format.h
+++ b/fs_mgr/libsnapshot/include/libsnapshot/cow_format.h
@@ -359,5 +359,9 @@
// Return block size used for compression
size_t CowOpCompressionSize(const CowOperation* op, size_t block_size);
+
+// Return the relative offset of the I/O block which the CowOperation
+// multi-block compression
+bool GetBlockOffset(const CowOperation* op, uint64_t io_block, size_t block_size, off_t* offset);
} // namespace snapshot
} // namespace android
diff --git a/fs_mgr/libsnapshot/include/libsnapshot/cow_reader.h b/fs_mgr/libsnapshot/include/libsnapshot/cow_reader.h
index bf4c79f..3f49c69 100644
--- a/fs_mgr/libsnapshot/include/libsnapshot/cow_reader.h
+++ b/fs_mgr/libsnapshot/include/libsnapshot/cow_reader.h
@@ -162,6 +162,9 @@
// Creates a clone of the current CowReader without the file handlers
std::unique_ptr<CowReader> CloneCowReader();
+ // Get the max compression size
+ uint32_t GetMaxCompressionSize();
+
void UpdateMergeOpsCompleted(int num_merge_ops) { header_.num_merge_ops += num_merge_ops; }
private:
diff --git a/fs_mgr/libsnapshot/libsnapshot_cow/cow_compress.cpp b/fs_mgr/libsnapshot/libsnapshot_cow/cow_compress.cpp
index c7a5f05..577cdbd 100644
--- a/fs_mgr/libsnapshot/libsnapshot_cow/cow_compress.cpp
+++ b/fs_mgr/libsnapshot/libsnapshot_cow/cow_compress.cpp
@@ -223,7 +223,7 @@
PLOG(ERROR) << "CompressBlocks: Compression failed";
return false;
}
- if (data.size() > std::numeric_limits<uint16_t>::max()) {
+ if (data.size() > std::numeric_limits<uint32_t>::max()) {
LOG(ERROR) << "Compressed block is too large: " << data.size();
return false;
}
diff --git a/fs_mgr/libsnapshot/libsnapshot_cow/cow_format.cpp b/fs_mgr/libsnapshot/libsnapshot_cow/cow_format.cpp
index 4e1c666..19014c0 100644
--- a/fs_mgr/libsnapshot/libsnapshot_cow/cow_format.cpp
+++ b/fs_mgr/libsnapshot/libsnapshot_cow/cow_format.cpp
@@ -181,5 +181,31 @@
return (block_size << compression_bits);
}
+bool GetBlockOffset(const CowOperation* op, uint64_t io_block, size_t block_size, off_t* offset) {
+ const uint64_t new_block = op->new_block;
+
+ if (op->type() != kCowReplaceOp || io_block < new_block) {
+ LOG(VERBOSE) << "Invalid IO request for block: " << io_block
+ << " CowOperation: new_block: " << new_block;
+ return false;
+ }
+
+ // Get the actual compression size
+ const size_t compression_size = CowOpCompressionSize(op, block_size);
+ // Find the number of blocks spanned
+ const size_t num_blocks = compression_size / block_size;
+ // Find the distance of the I/O block which this
+ // CowOperation encompasses
+ const size_t block_distance = io_block - new_block;
+ // Check if this block is within this range;
+ // if so, return the relative offset
+ if (block_distance < num_blocks) {
+ *offset = block_distance * block_size;
+ return true;
+ }
+
+ return false;
+}
+
} // namespace snapshot
} // namespace android
diff --git a/fs_mgr/libsnapshot/libsnapshot_cow/cow_reader.cpp b/fs_mgr/libsnapshot/libsnapshot_cow/cow_reader.cpp
index d6a03ca..6516499 100644
--- a/fs_mgr/libsnapshot/libsnapshot_cow/cow_reader.cpp
+++ b/fs_mgr/libsnapshot/libsnapshot_cow/cow_reader.cpp
@@ -164,6 +164,21 @@
return PrepMergeOps();
}
+uint32_t CowReader::GetMaxCompressionSize() {
+ switch (header_.prefix.major_version) {
+ case 1:
+ case 2:
+ // Old versions supports only 4KB compression.
+ return header_.block_size;
+ ;
+ case 3:
+ return header_.max_compression_size;
+ default:
+ LOG(ERROR) << "Unknown version: " << header_.prefix.major_version;
+ return 0;
+ }
+}
+
//
// This sets up the data needed for MergeOpIter. MergeOpIter presents
// data in the order we intend to merge in.
diff --git a/fs_mgr/libsnapshot/libsnapshot_cow/snapshot_reader.cpp b/fs_mgr/libsnapshot/libsnapshot_cow/snapshot_reader.cpp
index 4e90a0f..12073fc 100644
--- a/fs_mgr/libsnapshot/libsnapshot_cow/snapshot_reader.cpp
+++ b/fs_mgr/libsnapshot/libsnapshot_cow/snapshot_reader.cpp
@@ -42,10 +42,21 @@
op_iter_->Next();
continue;
}
- if (op->new_block >= ops_.size()) {
- ops_.resize(op->new_block + 1, nullptr);
+
+ size_t num_blocks = 1;
+ if (op->type() == kCowReplaceOp) {
+ num_blocks = (CowOpCompressionSize(op, block_size_) / block_size_);
}
- ops_[op->new_block] = op;
+ if (op->new_block >= ops_.size()) {
+ ops_.resize(op->new_block + num_blocks, nullptr);
+ }
+
+ size_t vec_index = op->new_block;
+ while (num_blocks) {
+ ops_[vec_index] = op;
+ num_blocks -= 1;
+ vec_index += 1;
+ }
op_iter_->Next();
}
}
@@ -172,11 +183,20 @@
} else if (op->type() == kCowZeroOp) {
memset(buffer, 0, bytes_to_read);
} else if (op->type() == kCowReplaceOp) {
- if (cow_->ReadData(op, buffer, bytes_to_read, start_offset) < bytes_to_read) {
- LOG(ERROR) << "CompressedSnapshotReader failed to read replace op";
+ size_t buffer_size = CowOpCompressionSize(op, block_size_);
+ uint8_t temp_buffer[buffer_size];
+ if (cow_->ReadData(op, temp_buffer, buffer_size, 0) < buffer_size) {
+ LOG(ERROR) << "CompressedSnapshotReader failed to read replace op: buffer_size: "
+ << buffer_size << "start_offset: " << start_offset;
errno = EIO;
return -1;
}
+ off_t block_offset{};
+ if (!GetBlockOffset(op, chunk, block_size_, &block_offset)) {
+ LOG(ERROR) << "GetBlockOffset failed";
+ return -1;
+ }
+ std::memcpy(buffer, (char*)temp_buffer + block_offset + start_offset, bytes_to_read);
} else if (op->type() == kCowXorOp) {
borrowed_fd fd = GetSourceFd();
if (fd < 0) {
diff --git a/fs_mgr/libsnapshot/snapuserd/user-space-merge/merge_worker.cpp b/fs_mgr/libsnapshot/snapuserd/user-space-merge/merge_worker.cpp
index 1e7d0c0..bd7eaca 100644
--- a/fs_mgr/libsnapshot/snapuserd/user-space-merge/merge_worker.cpp
+++ b/fs_mgr/libsnapshot/snapuserd/user-space-merge/merge_worker.cpp
@@ -13,10 +13,11 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-#include "merge_worker.h"
+#include <libsnapshot/cow_format.h>
#include <pthread.h>
+#include "merge_worker.h"
#include "snapuserd_core.h"
#include "utility.h"
@@ -37,6 +38,7 @@
int num_ops = *pending_ops;
int nr_consecutive = 0;
bool checkOrderedOp = (replace_zero_vec == nullptr);
+ size_t num_blocks = 1;
do {
if (!cowop_iter_->AtEnd() && num_ops) {
@@ -48,11 +50,15 @@
*source_offset = cow_op->new_block * BLOCK_SZ;
if (!checkOrderedOp) {
replace_zero_vec->push_back(cow_op);
+ if (cow_op->type() == kCowReplaceOp) {
+ // Get the number of blocks this op has compressed
+ num_blocks = (CowOpCompressionSize(cow_op, BLOCK_SZ) / BLOCK_SZ);
+ }
}
cowop_iter_->Next();
- num_ops -= 1;
- nr_consecutive = 1;
+ num_ops -= num_blocks;
+ nr_consecutive = num_blocks;
while (!cowop_iter_->AtEnd() && num_ops) {
const CowOperation* op = cowop_iter_->Get();
@@ -66,11 +72,20 @@
}
if (!checkOrderedOp) {
+ if (op->type() == kCowReplaceOp) {
+ num_blocks = (CowOpCompressionSize(op, BLOCK_SZ) / BLOCK_SZ);
+ if (num_ops < num_blocks) {
+ break;
+ }
+ } else {
+ // zero op
+ num_blocks = 1;
+ }
replace_zero_vec->push_back(op);
}
- nr_consecutive += 1;
- num_ops -= 1;
+ nr_consecutive += num_blocks;
+ num_ops -= num_blocks;
cowop_iter_->Next();
}
}
@@ -108,18 +123,24 @@
for (size_t i = 0; i < replace_zero_vec.size(); i++) {
const CowOperation* cow_op = replace_zero_vec[i];
-
- void* buffer = bufsink_.AcquireBuffer(BLOCK_SZ);
- if (!buffer) {
- SNAP_LOG(ERROR) << "AcquireBuffer failed in MergeReplaceOps";
- return false;
- }
if (cow_op->type() == kCowReplaceOp) {
- if (!reader_->ReadData(cow_op, buffer, BLOCK_SZ)) {
+ size_t buffer_size = CowOpCompressionSize(cow_op, BLOCK_SZ);
+ void* buffer = bufsink_.AcquireBuffer(buffer_size);
+ if (!buffer) {
+ SNAP_LOG(ERROR) << "AcquireBuffer failed in MergeReplaceOps";
+ return false;
+ }
+ // Read the entire compressed buffer spanning multiple blocks
+ if (!reader_->ReadData(cow_op, buffer, buffer_size)) {
SNAP_LOG(ERROR) << "Failed to read COW in merge";
return false;
}
} else {
+ void* buffer = bufsink_.AcquireBuffer(BLOCK_SZ);
+ if (!buffer) {
+ SNAP_LOG(ERROR) << "AcquireBuffer failed in MergeReplaceOps";
+ return false;
+ }
CHECK(cow_op->type() == kCowZeroOp);
memset(buffer, 0, BLOCK_SZ);
}
@@ -137,7 +158,7 @@
return false;
}
- num_ops_merged += linear_blocks;
+ num_ops_merged += replace_zero_vec.size();
if (num_ops_merged >= total_ops_merged_per_commit) {
// Flush the data
diff --git a/fs_mgr/libsnapshot/snapuserd/user-space-merge/read_worker.cpp b/fs_mgr/libsnapshot/snapuserd/user-space-merge/read_worker.cpp
index f1d4065..d40b6d1 100644
--- a/fs_mgr/libsnapshot/snapuserd/user-space-merge/read_worker.cpp
+++ b/fs_mgr/libsnapshot/snapuserd/user-space-merge/read_worker.cpp
@@ -14,10 +14,10 @@
* limitations under the License.
*/
-#include "read_worker.h"
-
+#include <libsnapshot/cow_format.h>
#include <pthread.h>
+#include "read_worker.h"
#include "snapuserd_core.h"
#include "utility.h"
@@ -48,9 +48,10 @@
// Start the replace operation. This will read the
// internal COW format and if the block is compressed,
// it will be de-compressed.
-bool ReadWorker::ProcessReplaceOp(const CowOperation* cow_op, void* buffer) {
- if (!reader_->ReadData(cow_op, buffer, BLOCK_SZ)) {
- SNAP_LOG(ERROR) << "ProcessReplaceOp failed for block " << cow_op->new_block;
+bool ReadWorker::ProcessReplaceOp(const CowOperation* cow_op, void* buffer, size_t buffer_size) {
+ if (!reader_->ReadData(cow_op, buffer, buffer_size)) {
+ SNAP_LOG(ERROR) << "ProcessReplaceOp failed for block " << cow_op->new_block
+ << " buffer_size: " << buffer_size;
return false;
}
return true;
@@ -183,7 +184,13 @@
switch (cow_op->type()) {
case kCowReplaceOp: {
- return ProcessReplaceOp(cow_op, buffer);
+ size_t buffer_size = CowOpCompressionSize(cow_op, BLOCK_SZ);
+ uint8_t chunk[buffer_size];
+ if (!ProcessReplaceOp(cow_op, chunk, buffer_size)) {
+ return false;
+ }
+ std::memcpy(buffer, chunk, BLOCK_SZ);
+ return true;
}
case kCowZeroOp: {
@@ -209,6 +216,13 @@
return false;
}
+ const size_t compression_factor = reader_->GetMaxCompressionSize();
+ if (!compression_factor) {
+ SNAP_LOG(ERROR) << "Compression factor is set to 0 which is invalid.";
+ return false;
+ }
+ decompressed_buffer_ = std::make_unique<uint8_t[]>(compression_factor);
+
backing_store_fd_.reset(open(backing_store_device_.c_str(), O_RDONLY));
if (backing_store_fd_ < 0) {
SNAP_PLOG(ERROR) << "Open Failed: " << backing_store_device_;
@@ -276,6 +290,20 @@
return true;
}
+bool ReadWorker::GetCowOpBlockOffset(const CowOperation* cow_op, uint64_t io_block,
+ off_t* block_offset) {
+ // If this is a replace op, get the block offset of this I/O
+ // block. Multi-block compression is supported only for
+ // Replace ops.
+ //
+ // Note: This can be extended when we support COPY and XOR ops down the
+ // line as the blocks are mostly contiguous.
+ if (cow_op && cow_op->type() == kCowReplaceOp) {
+ return GetBlockOffset(cow_op, io_block, BLOCK_SZ, block_offset);
+ }
+ return false;
+}
+
bool ReadWorker::ReadAlignedSector(sector_t sector, size_t sz) {
size_t remaining_size = sz;
std::vector<std::pair<sector_t, const CowOperation*>>& chunk_vec = snapuserd_->GetChunkVec();
@@ -286,7 +314,7 @@
size_t read_size = std::min(PAYLOAD_BUFFER_SZ, remaining_size);
size_t total_bytes_read = 0;
-
+ const CowOperation* prev_op = nullptr;
while (read_size) {
// We need to check every 4k block to verify if it is
// present in the mapping.
@@ -294,7 +322,7 @@
auto it = std::lower_bound(chunk_vec.begin(), chunk_vec.end(),
std::make_pair(sector, nullptr), SnapshotHandler::compare);
- bool not_found = (it == chunk_vec.end() || it->first != sector);
+ const bool sector_not_found = (it == chunk_vec.end() || it->first != sector);
void* buffer = block_server_->GetResponseBuffer(BLOCK_SZ, size);
if (!buffer) {
@@ -302,15 +330,88 @@
return false;
}
- if (not_found) {
- // Block not found in map - which means this block was not
- // changed as per the OTA. Just route the I/O to the base
- // device.
- if (!ReadDataFromBaseDevice(sector, buffer, size)) {
- SNAP_LOG(ERROR) << "ReadDataFromBaseDevice failed";
- return false;
+ if (sector_not_found) {
+ // Find the 4k block
+ uint64_t io_block = SectorToChunk(sector);
+ // Get the previous iterator. Since the vector is sorted, the
+ // lookup of this sector can fall in a range of blocks if
+ // CowOperation has compressed multiple blocks.
+ if (it != chunk_vec.begin()) {
+ std::advance(it, -1);
}
+ bool is_mapping_present = true;
+
+ // Vector itself is empty. This can happen if the block was not
+ // changed per the OTA or if the merge was already complete but
+ // snapshot table was not yet collapsed.
+ if (it == chunk_vec.end()) {
+ is_mapping_present = false;
+ }
+
+ const CowOperation* cow_op = nullptr;
+ // Relative offset within the compressed multiple blocks
+ off_t block_offset = 0;
+ if (is_mapping_present) {
+ // Get the nearest operation found in the vector
+ cow_op = it->second;
+ is_mapping_present = GetCowOpBlockOffset(cow_op, io_block, &block_offset);
+ }
+
+ // Thus, we have a case wherein sector was not found in the sorted
+ // vector; however, we indeed have a mapping of this sector
+ // embedded in one of the CowOperation which spans multiple
+ // block size.
+ if (is_mapping_present) {
+ // block_offset = 0 would mean that the CowOperation should
+ // already be in the sorted vector. Hence, lookup should
+ // have already found it. If not, this is a bug.
+ if (block_offset == 0) {
+ SNAP_LOG(ERROR)
+ << "GetBlockOffset returned offset 0 for io_block: " << io_block;
+ return false;
+ }
+
+ // Get the CowOperation actual compression size
+ size_t compression_size = CowOpCompressionSize(cow_op, BLOCK_SZ);
+ // Offset cannot be greater than the compression size
+ if (block_offset > compression_size) {
+ SNAP_LOG(ERROR) << "Invalid I/O block found. io_block: " << io_block
+ << " CowOperation-new-block: " << cow_op->new_block
+ << " compression-size: " << compression_size;
+ return false;
+ }
+
+ // Cached copy of the previous iteration. Just retrieve the
+ // data
+ if (prev_op && prev_op->new_block == cow_op->new_block) {
+ std::memcpy(buffer, (char*)decompressed_buffer_.get() + block_offset, size);
+ } else {
+ // Get the data from the disk based on the compression
+ // size
+ if (!ProcessReplaceOp(cow_op, decompressed_buffer_.get(),
+ compression_size)) {
+ return false;
+ }
+ // Copy the data from the decompressed buffer relative
+ // to the i/o block offset.
+ std::memcpy(buffer, (char*)decompressed_buffer_.get() + block_offset, size);
+ // Cache this CowOperation pointer for successive I/O
+ // operation. Since the request is sequential and the
+ // block is already decompressed, subsequest I/O blocks
+ // can fetch the data directly from this decompressed
+ // buffer.
+ prev_op = cow_op;
+ }
+ } else {
+ // Block not found in map - which means this block was not
+ // changed as per the OTA. Just route the I/O to the base
+ // device.
+ if (!ReadDataFromBaseDevice(sector, buffer, size)) {
+ SNAP_LOG(ERROR) << "ReadDataFromBaseDevice failed";
+ return false;
+ }
+ }
ret = size;
} else {
// We found the sector in mapping. Check the type of COW OP and
@@ -341,12 +442,50 @@
return true;
}
+bool ReadWorker::IsMappingPresent(const CowOperation* cow_op, loff_t requested_offset,
+ loff_t cow_op_offset) {
+ const bool replace_op = (cow_op->type() == kCowReplaceOp);
+ if (replace_op) {
+ size_t max_compressed_size = CowOpCompressionSize(cow_op, BLOCK_SZ);
+ if ((requested_offset >= cow_op_offset) &&
+ (requested_offset < (cow_op_offset + max_compressed_size))) {
+ return true;
+ }
+ }
+ return false;
+}
+
int ReadWorker::ReadUnalignedSector(
sector_t sector, size_t size,
std::vector<std::pair<sector_t, const CowOperation*>>::iterator& it) {
SNAP_LOG(DEBUG) << "ReadUnalignedSector: sector " << sector << " size: " << size
<< " Aligned sector: " << it->first;
+ loff_t requested_offset = sector << SECTOR_SHIFT;
+ loff_t final_offset = (it->first) << SECTOR_SHIFT;
+
+ const CowOperation* cow_op = it->second;
+ if (IsMappingPresent(cow_op, requested_offset, final_offset)) {
+ size_t buffer_size = CowOpCompressionSize(cow_op, BLOCK_SZ);
+ uint8_t chunk[buffer_size];
+ // Read the entire decompressed buffer based on the block-size
+ if (!ProcessReplaceOp(cow_op, chunk, buffer_size)) {
+ return -1;
+ }
+ size_t skip_offset = (requested_offset - final_offset);
+ size_t write_sz = std::min(size, buffer_size - skip_offset);
+
+ auto buffer =
+ reinterpret_cast<uint8_t*>(block_server_->GetResponseBuffer(BLOCK_SZ, write_sz));
+ if (!buffer) {
+ SNAP_LOG(ERROR) << "ReadUnalignedSector failed to allocate buffer";
+ return -1;
+ }
+
+ std::memcpy(buffer, (char*)chunk + skip_offset, write_sz);
+ return write_sz;
+ }
+
int num_sectors_skip = sector - it->first;
size_t skip_size = num_sectors_skip << SECTOR_SHIFT;
size_t write_size = std::min(size, BLOCK_SZ - skip_size);
@@ -445,8 +584,11 @@
size_t remaining_size = size;
int ret = 0;
+
+ const CowOperation* cow_op = it->second;
if (!merge_complete && (requested_offset >= final_offset) &&
- (requested_offset - final_offset) < BLOCK_SZ) {
+ (((requested_offset - final_offset) < BLOCK_SZ) ||
+ IsMappingPresent(cow_op, requested_offset, final_offset))) {
// Read the partial un-aligned data
ret = ReadUnalignedSector(sector, remaining_size, it);
if (ret < 0) {
diff --git a/fs_mgr/libsnapshot/snapuserd/user-space-merge/read_worker.h b/fs_mgr/libsnapshot/snapuserd/user-space-merge/read_worker.h
index 1aff50c..43e896a 100644
--- a/fs_mgr/libsnapshot/snapuserd/user-space-merge/read_worker.h
+++ b/fs_mgr/libsnapshot/snapuserd/user-space-merge/read_worker.h
@@ -44,9 +44,12 @@
bool ProcessXorOp(const CowOperation* cow_op, void* buffer);
bool ProcessOrderedOp(const CowOperation* cow_op, void* buffer);
bool ProcessCopyOp(const CowOperation* cow_op, void* buffer);
- bool ProcessReplaceOp(const CowOperation* cow_op, void* buffer);
+ bool ProcessReplaceOp(const CowOperation* cow_op, void* buffer, size_t buffer_size);
bool ProcessZeroOp(void* buffer);
+ bool IsMappingPresent(const CowOperation* cow_op, loff_t requested_offset,
+ loff_t cow_op_offset);
+ bool GetCowOpBlockOffset(const CowOperation* cow_op, uint64_t io_block, off_t* block_offset);
bool ReadAlignedSector(sector_t sector, size_t sz);
bool ReadUnalignedSector(sector_t sector, size_t size);
int ReadUnalignedSector(sector_t sector, size_t size,
@@ -56,6 +59,7 @@
constexpr bool IsBlockAligned(size_t size) { return ((size & (BLOCK_SZ - 1)) == 0); }
constexpr sector_t ChunkToSector(chunk_t chunk) { return chunk << CHUNK_SHIFT; }
+ constexpr chunk_t SectorToChunk(sector_t sector) { return sector >> CHUNK_SHIFT; }
std::string backing_store_device_;
unique_fd backing_store_fd_;
@@ -67,6 +71,7 @@
std::basic_string<uint8_t> xor_buffer_;
std::unique_ptr<void, decltype(&::free)> aligned_buffer_;
+ std::unique_ptr<uint8_t[]> decompressed_buffer_;
};
} // namespace snapshot
diff --git a/fs_mgr/libsnapshot/snapuserd/user-space-merge/snapuserd_test.cpp b/fs_mgr/libsnapshot/snapuserd/user-space-merge/snapuserd_test.cpp
index 8ddb0f4..76b44b4 100644
--- a/fs_mgr/libsnapshot/snapuserd/user-space-merge/snapuserd_test.cpp
+++ b/fs_mgr/libsnapshot/snapuserd/user-space-merge/snapuserd_test.cpp
@@ -64,6 +64,9 @@
struct TestParam {
bool io_uring;
bool o_direct;
+ std::string compression;
+ int block_size;
+ int num_threads;
};
class SnapuserdTestBase : public ::testing::TestWithParam<TestParam> {
@@ -74,6 +77,7 @@
void CreateCowDevice();
void SetDeviceControlName();
std::unique_ptr<ICowWriter> CreateCowDeviceInternal();
+ std::unique_ptr<ICowWriter> CreateV3Cow();
std::unique_ptr<ITestHarness> harness_;
size_t size_ = 10_MiB;
@@ -133,6 +137,24 @@
return CreateCowWriter(kDefaultCowVersion, options, std::move(fd));
}
+std::unique_ptr<ICowWriter> SnapuserdTestBase::CreateV3Cow() {
+ const TestParam params = GetParam();
+
+ CowOptions options;
+ options.op_count_max = 100000;
+ options.compression = params.compression;
+ options.num_compress_threads = params.num_threads;
+ options.batch_write = true;
+ options.compression_factor = params.block_size;
+
+ cow_system_ = std::make_unique<TemporaryFile>();
+
+ unique_fd fd(cow_system_->fd);
+ cow_system_->fd = -1;
+
+ return CreateCowWriter(3, options, std::move(fd));
+}
+
void SnapuserdTestBase::CreateCowDevice() {
unique_fd rnd_fd;
loff_t offset = 0;
@@ -236,6 +258,7 @@
void SetupOrderedOpsInverted();
void SetupCopyOverlap_1();
void SetupCopyOverlap_2();
+ void SetupDeviceForPassthrough();
bool Merge();
void ValidateMerge();
void ReadSnapshotDeviceAndValidate();
@@ -258,6 +281,9 @@
void SimulateDaemonRestart();
+ void CreateCowDeviceWithNoBlockChanges();
+ void ValidateDeviceWithNoBlockChanges();
+
void CreateCowDeviceOrderedOps();
void CreateCowDeviceOrderedOpsInverted();
void CreateCowDeviceWithCopyOverlap_1();
@@ -307,6 +333,12 @@
ASSERT_NO_FATAL_FAILURE(SetupDaemon());
}
+void SnapuserdTest::SetupDeviceForPassthrough() {
+ ASSERT_NO_FATAL_FAILURE(CreateBaseDevice());
+ ASSERT_NO_FATAL_FAILURE(CreateCowDeviceWithNoBlockChanges());
+ ASSERT_NO_FATAL_FAILURE(SetupDaemon());
+}
+
void SnapuserdTest::SetupOrderedOpsInverted() {
ASSERT_NO_FATAL_FAILURE(CreateBaseDevice());
ASSERT_NO_FATAL_FAILURE(CreateCowDeviceOrderedOpsInverted());
@@ -480,6 +512,47 @@
}
}
+void SnapuserdTest::CreateCowDeviceWithNoBlockChanges() {
+ auto writer = CreateCowDeviceInternal();
+ ASSERT_NE(writer, nullptr);
+
+ std::unique_ptr<uint8_t[]> buffer = std::make_unique<uint8_t[]>(BLOCK_SZ);
+ std::memset(buffer.get(), 'A', BLOCK_SZ);
+
+ // This test focusses on not changing all the blocks thereby validating
+ // the pass-through I/O
+
+ // Replace the first block
+ ASSERT_TRUE(writer->AddRawBlocks(1, buffer.get(), BLOCK_SZ));
+
+ // Set zero block of Block 3
+ ASSERT_TRUE(writer->AddZeroBlocks(3, 1));
+
+ ASSERT_TRUE(writer->Finalize());
+ orig_buffer_ = std::make_unique<uint8_t[]>(total_base_size_);
+
+ // Read the entire base device
+ ASSERT_EQ(android::base::ReadFullyAtOffset(base_fd_, orig_buffer_.get(), total_base_size_, 0),
+ true);
+
+ off_t offset = BLOCK_SZ;
+ std::memcpy(orig_buffer_.get() + offset, buffer.get(), BLOCK_SZ);
+ offset = 3 * BLOCK_SZ;
+ std::memset(orig_buffer_.get() + offset, 0, BLOCK_SZ);
+}
+
+void SnapuserdTest::ValidateDeviceWithNoBlockChanges() {
+ unique_fd fd(open(dmuser_dev_->GetPath().c_str(), O_RDONLY));
+ ASSERT_GE(fd, 0);
+ std::unique_ptr<uint8_t[]> snapshot_buffer = std::make_unique<uint8_t[]>(size_);
+ std::memset(snapshot_buffer.get(), 'B', size_);
+
+ // All the I/O request should be a pass through to base device except for
+ // Block 1 and Block 3.
+ ASSERT_EQ(ReadFullyAtOffset(fd, snapshot_buffer.get(), size_, 0), true);
+ ASSERT_EQ(memcmp(snapshot_buffer.get(), orig_buffer_.get(), size_), 0);
+}
+
void SnapuserdTest::CreateCowDeviceWithCopyOverlap_1() {
auto writer = CreateCowDeviceInternal();
ASSERT_NE(writer, nullptr);
@@ -781,6 +854,20 @@
ASSERT_TRUE(Merge());
}
+TEST_P(SnapuserdTest, Snapshot_Passthrough) {
+ if (!harness_->HasUserDevice()) {
+ GTEST_SKIP() << "Skipping snapshot read; not supported";
+ }
+ ASSERT_NO_FATAL_FAILURE(SetupDeviceForPassthrough());
+ // I/O before merge
+ ASSERT_NO_FATAL_FAILURE(ValidateDeviceWithNoBlockChanges());
+ ASSERT_TRUE(Merge());
+ ValidateMerge();
+ // I/O after merge - daemon should read directly
+ // from base device
+ ASSERT_NO_FATAL_FAILURE(ValidateDeviceWithNoBlockChanges());
+}
+
TEST_P(SnapuserdTest, Snapshot_IO_TEST) {
if (!harness_->HasUserDevice()) {
GTEST_SKIP() << "Skipping snapshot read; not supported";
@@ -853,7 +940,7 @@
GTEST_SKIP() << "Skipping snapshot read; not supported";
}
ASSERT_NO_FATAL_FAILURE(SetupCopyOverlap_2());
- ASSERT_NO_FATAL_FAILURE(MergeInterruptAndValidate(2));
+ ASSERT_NO_FATAL_FAILURE(MergeInterruptFixed(300));
ValidateMerge();
}
@@ -881,11 +968,243 @@
ValidateMerge();
}
+class SnapuserdVariableBlockSizeTest : public SnapuserdTest {
+ public:
+ void SetupCowV3ForVariableBlockSize();
+ void ReadSnapshotWithVariableBlockSize();
+
+ protected:
+ void SetUp() override;
+ void TearDown() override;
+
+ void CreateV3CowDeviceForVariableBlockSize();
+};
+
+void SnapuserdVariableBlockSizeTest::SetupCowV3ForVariableBlockSize() {
+ ASSERT_NO_FATAL_FAILURE(CreateBaseDevice());
+ ASSERT_NO_FATAL_FAILURE(CreateV3CowDeviceForVariableBlockSize());
+ ASSERT_NO_FATAL_FAILURE(SetupDaemon());
+}
+
+void SnapuserdVariableBlockSizeTest::CreateV3CowDeviceForVariableBlockSize() {
+ auto writer = CreateV3Cow();
+
+ size_t total_data_to_write = size_;
+
+ size_t total_blocks_to_write = total_data_to_write / BLOCK_SZ;
+ size_t num_blocks_per_op = total_blocks_to_write / 4;
+ size_t source_block = 0;
+
+ size_t seq_len = num_blocks_per_op;
+ uint32_t sequence[seq_len];
+ size_t xor_block_start = seq_len * 3;
+ for (size_t i = 0; i < seq_len; i++) {
+ sequence[i] = xor_block_start + i;
+ }
+ ASSERT_TRUE(writer->AddSequenceData(seq_len, sequence));
+
+ size_t total_replace_blocks = num_blocks_per_op;
+ // Write some data which can be compressed
+ std::string data;
+ data.resize(total_replace_blocks * BLOCK_SZ, '\0');
+ for (size_t i = 0; i < data.size(); i++) {
+ data[i] = static_cast<char>('A' + i / BLOCK_SZ);
+ }
+ // REPLACE ops
+ ASSERT_TRUE(writer->AddRawBlocks(source_block, data.data(), data.size()));
+
+ total_blocks_to_write -= total_replace_blocks;
+ source_block = source_block + total_replace_blocks;
+
+ // ZERO ops
+ size_t total_zero_blocks = total_blocks_to_write / 3;
+ ASSERT_TRUE(writer->AddZeroBlocks(source_block, total_zero_blocks));
+
+ total_blocks_to_write -= total_zero_blocks;
+ source_block = source_block + total_zero_blocks;
+
+ // Generate some random data wherein few blocks cannot be compressed.
+ // This is to test the I/O path for those blocks which aren't compressed.
+ size_t total_random_data_blocks = total_blocks_to_write / 2;
+ unique_fd rnd_fd(open("/dev/random", O_RDONLY));
+
+ ASSERT_GE(rnd_fd, 0);
+ std::string random_buffer;
+ random_buffer.resize(total_random_data_blocks * BLOCK_SZ, '\0');
+ ASSERT_EQ(
+ android::base::ReadFullyAtOffset(rnd_fd, random_buffer.data(), random_buffer.size(), 0),
+ true);
+ // REPLACE ops
+ ASSERT_TRUE(writer->AddRawBlocks(source_block, random_buffer.data(), random_buffer.size()));
+
+ total_blocks_to_write -= total_random_data_blocks;
+ source_block = source_block + total_random_data_blocks;
+
+ // XOR ops will always be 4k blocks
+ std::string xor_buffer;
+ xor_buffer.resize(total_blocks_to_write * BLOCK_SZ, '\0');
+ for (size_t i = 0; i < xor_buffer.size(); i++) {
+ xor_buffer[i] = static_cast<char>('C' + i / BLOCK_SZ);
+ }
+ size_t xor_offset = 21;
+ std::string source_buffer;
+ source_buffer.resize(total_blocks_to_write * BLOCK_SZ, '\0');
+ ASSERT_EQ(android::base::ReadFullyAtOffset(base_fd_, source_buffer.data(), source_buffer.size(),
+ size_ + xor_offset),
+ true);
+ for (size_t i = 0; i < xor_buffer.size(); i++) {
+ xor_buffer[i] ^= source_buffer[i];
+ }
+
+ ASSERT_EQ(xor_block_start, source_block);
+
+ ASSERT_TRUE(writer->AddXorBlocks(source_block, xor_buffer.data(), xor_buffer.size(),
+ (size_ / BLOCK_SZ), xor_offset));
+ // Flush operations
+ ASSERT_TRUE(writer->Finalize());
+
+ // Construct the buffer required for validation
+ orig_buffer_ = std::make_unique<uint8_t[]>(total_base_size_);
+
+ // Read the entire base device
+ ASSERT_EQ(android::base::ReadFullyAtOffset(base_fd_, orig_buffer_.get(), total_base_size_, 0),
+ true);
+
+ // REPLACE ops which are compressed
+ std::memcpy(orig_buffer_.get(), data.data(), data.size());
+ size_t offset = data.size();
+
+ // ZERO ops
+ std::string zero_buffer(total_zero_blocks * BLOCK_SZ, 0);
+ std::memcpy((char*)orig_buffer_.get() + offset, (void*)zero_buffer.c_str(), zero_buffer.size());
+ offset += zero_buffer.size();
+
+ // REPLACE ops - Random buffers which aren't compressed
+ std::memcpy((char*)orig_buffer_.get() + offset, random_buffer.c_str(), random_buffer.size());
+ offset += random_buffer.size();
+
+ // XOR Ops which default to 4k block size compression irrespective of
+ // compression factor
+ ASSERT_EQ(android::base::ReadFullyAtOffset(base_fd_, (char*)orig_buffer_.get() + offset,
+ xor_buffer.size(), size_ + xor_offset),
+ true);
+ for (size_t i = 0; i < xor_buffer.size(); i++) {
+ orig_buffer_.get()[offset + i] = (uint8_t)(orig_buffer_.get()[offset + i] ^ xor_buffer[i]);
+ }
+}
+
+void SnapuserdVariableBlockSizeTest::ReadSnapshotWithVariableBlockSize() {
+ unique_fd fd(open(dmuser_dev_->GetPath().c_str(), O_RDONLY | O_DIRECT));
+ ASSERT_GE(fd, 0);
+
+ void* addr;
+ ssize_t page_size = getpagesize();
+ ASSERT_EQ(posix_memalign(&addr, page_size, size_), 0);
+ std::unique_ptr<void, decltype(&::free)> snapshot_buffer(addr, ::free);
+
+ const TestParam params = GetParam();
+
+ // Issue I/O request with various block sizes
+ size_t num_blocks = size_ / params.block_size;
+ off_t offset = 0;
+ for (size_t i = 0; i < num_blocks; i++) {
+ ASSERT_EQ(ReadFullyAtOffset(fd, (char*)snapshot_buffer.get() + offset, params.block_size,
+ offset),
+ true);
+ offset += params.block_size;
+ }
+ // Validate buffer
+ ASSERT_EQ(memcmp(snapshot_buffer.get(), orig_buffer_.get(), size_), 0);
+
+ // Reset the buffer
+ std::memset(snapshot_buffer.get(), 0, size_);
+
+ // Read one full chunk in a single shot and re-validate.
+ ASSERT_EQ(ReadFullyAtOffset(fd, snapshot_buffer.get(), size_, 0), true);
+ ASSERT_EQ(memcmp(snapshot_buffer.get(), orig_buffer_.get(), size_), 0);
+
+ // Reset the buffer
+ std::memset(snapshot_buffer.get(), 0, size_);
+
+ // Buffered I/O test
+ fd.reset(open(dmuser_dev_->GetPath().c_str(), O_RDONLY));
+ ASSERT_GE(fd, 0);
+
+ // Try not to cache
+ posix_fadvise(fd.get(), 0, size_, POSIX_FADV_DONTNEED);
+
+ size_t num_blocks_per_op = (size_ / BLOCK_SZ) / 4;
+ offset = num_blocks_per_op * BLOCK_SZ;
+ size_t read_size = 1019; // bytes
+ offset -= 111;
+
+ // Issue a un-aligned read which crosses the boundary between a REPLACE block and a ZERO
+ // block.
+ ASSERT_EQ(ReadFullyAtOffset(fd, snapshot_buffer.get(), read_size, offset), true);
+
+ // Validate the data
+ ASSERT_EQ(std::memcmp(snapshot_buffer.get(), (char*)orig_buffer_.get() + offset, read_size), 0);
+
+ offset = (num_blocks_per_op * 3) * BLOCK_SZ;
+ offset -= (BLOCK_SZ - 119);
+ read_size = 8111;
+
+ // Issue an un-aligned read which crosses the boundary between a REPLACE block of random
+ // un-compressed data and a XOR block
+ ASSERT_EQ(ReadFullyAtOffset(fd, snapshot_buffer.get(), read_size, offset), true);
+
+ // Validate the data
+ ASSERT_EQ(std::memcmp(snapshot_buffer.get(), (char*)orig_buffer_.get() + offset, read_size), 0);
+
+ // Reset the buffer
+ std::memset(snapshot_buffer.get(), 0, size_);
+
+ // Read just one byte at an odd offset which is a REPLACE op
+ offset = 19;
+ read_size = 1;
+ ASSERT_EQ(ReadFullyAtOffset(fd, snapshot_buffer.get(), read_size, offset), true);
+ // Validate the data
+ ASSERT_EQ(std::memcmp(snapshot_buffer.get(), (char*)orig_buffer_.get() + offset, read_size), 0);
+
+ // Reset the buffer
+ std::memset(snapshot_buffer.get(), 0, size_);
+
+ // Read a block which has no mapping to a COW operation. This read should be
+ // a pass-through to the underlying base device.
+ offset = size_ + 9342;
+ read_size = 30;
+ ASSERT_EQ(ReadFullyAtOffset(fd, snapshot_buffer.get(), read_size, offset), true);
+ // Validate the data
+ ASSERT_EQ(std::memcmp(snapshot_buffer.get(), (char*)orig_buffer_.get() + offset, read_size), 0);
+}
+
+void SnapuserdVariableBlockSizeTest::SetUp() {
+ ASSERT_NO_FATAL_FAILURE(SnapuserdTest::SetUp());
+}
+
+void SnapuserdVariableBlockSizeTest::TearDown() {
+ SnapuserdTest::TearDown();
+}
+
+TEST_P(SnapuserdVariableBlockSizeTest, Snapshot_Test_Variable_Block_Size) {
+ if (!harness_->HasUserDevice()) {
+ GTEST_SKIP() << "Skipping snapshot read; not supported";
+ }
+ ASSERT_NO_FATAL_FAILURE(SetupCowV3ForVariableBlockSize());
+ ASSERT_NO_FATAL_FAILURE(ReadSnapshotWithVariableBlockSize());
+ ASSERT_TRUE(StartMerge());
+ CheckMergeCompletion();
+ ValidateMerge();
+ ASSERT_NO_FATAL_FAILURE(ReadSnapshotWithVariableBlockSize());
+}
+
class HandlerTest : public SnapuserdTestBase {
protected:
void SetUp() override;
void TearDown() override;
+ void SetUpV2Cow();
+ void InitializeDevice();
AssertionResult ReadSectors(sector_t sector, uint64_t size, void* buffer);
TestBlockServerFactory factory_;
@@ -896,10 +1215,11 @@
std::future<bool> handler_thread_;
};
-void HandlerTest::SetUp() {
- ASSERT_NO_FATAL_FAILURE(SnapuserdTestBase::SetUp());
- ASSERT_NO_FATAL_FAILURE(CreateBaseDevice());
+void HandlerTest::SetUpV2Cow() {
ASSERT_NO_FATAL_FAILURE(CreateCowDevice());
+}
+
+void HandlerTest::InitializeDevice() {
ASSERT_NO_FATAL_FAILURE(SetDeviceControlName());
opener_ = factory_.CreateTestOpener(system_device_ctrl_name_);
@@ -921,6 +1241,13 @@
handler_thread_ = std::async(std::launch::async, &SnapshotHandler::Start, handler_.get());
}
+void HandlerTest::SetUp() {
+ ASSERT_NO_FATAL_FAILURE(SnapuserdTestBase::SetUp());
+ ASSERT_NO_FATAL_FAILURE(CreateBaseDevice());
+ ASSERT_NO_FATAL_FAILURE(SetUpV2Cow());
+ ASSERT_NO_FATAL_FAILURE(InitializeDevice());
+}
+
void HandlerTest::TearDown() {
ASSERT_TRUE(factory_.DeleteQueue(system_device_ctrl_name_));
ASSERT_TRUE(handler_thread_.get());
@@ -986,6 +1313,147 @@
ASSERT_EQ(memcmp(snapuserd_buffer.get(), orig_buffer_.get(), SECTOR_SIZE), 0);
}
+class HandlerTestV3 : public HandlerTest {
+ public:
+ void ReadSnapshotWithVariableBlockSize();
+
+ protected:
+ void SetUp() override;
+ void TearDown() override;
+ void SetUpV3Cow();
+};
+
+void HandlerTestV3::SetUp() {
+ ASSERT_NO_FATAL_FAILURE(SnapuserdTestBase::SetUp());
+ ASSERT_NO_FATAL_FAILURE(CreateBaseDevice());
+ ASSERT_NO_FATAL_FAILURE(SetUpV3Cow());
+ ASSERT_NO_FATAL_FAILURE(InitializeDevice());
+}
+
+void HandlerTestV3::TearDown() {
+ ASSERT_NO_FATAL_FAILURE(HandlerTest::TearDown());
+}
+
+void HandlerTestV3::SetUpV3Cow() {
+ auto writer = CreateV3Cow();
+
+ size_t total_data_to_write = size_;
+
+ size_t total_blocks_to_write = total_data_to_write / BLOCK_SZ;
+ size_t num_blocks_per_op = total_blocks_to_write / 4;
+ size_t source_block = 0;
+
+ size_t total_replace_blocks = num_blocks_per_op;
+ // Write some data which can be compressed
+ std::string data;
+ data.resize(total_replace_blocks * BLOCK_SZ, '\0');
+ for (size_t i = 0; i < data.size(); i++) {
+ data[i] = static_cast<char>('A' + i / BLOCK_SZ);
+ }
+ // REPLACE ops
+ ASSERT_TRUE(writer->AddRawBlocks(source_block, data.data(), data.size()));
+
+ total_blocks_to_write -= total_replace_blocks;
+ source_block = source_block + total_replace_blocks;
+
+ // ZERO ops
+ size_t total_zero_blocks = total_blocks_to_write / 3;
+ ASSERT_TRUE(writer->AddZeroBlocks(source_block, total_zero_blocks));
+
+ total_blocks_to_write -= total_zero_blocks;
+ source_block = source_block + total_zero_blocks;
+
+ // Generate some random data wherein few blocks cannot be compressed.
+ // This is to test the I/O path for those blocks which aren't compressed.
+ size_t total_random_data_blocks = total_blocks_to_write;
+ unique_fd rnd_fd(open("/dev/random", O_RDONLY));
+
+ ASSERT_GE(rnd_fd, 0);
+ std::string random_buffer;
+ random_buffer.resize(total_random_data_blocks * BLOCK_SZ, '\0');
+ ASSERT_EQ(
+ android::base::ReadFullyAtOffset(rnd_fd, random_buffer.data(), random_buffer.size(), 0),
+ true);
+ // REPLACE ops
+ ASSERT_TRUE(writer->AddRawBlocks(source_block, random_buffer.data(), random_buffer.size()));
+ // Flush operations
+ ASSERT_TRUE(writer->Finalize());
+
+ // Construct the buffer required for validation
+ orig_buffer_ = std::make_unique<uint8_t[]>(total_base_size_);
+
+ // Read the entire base device
+ ASSERT_EQ(android::base::ReadFullyAtOffset(base_fd_, orig_buffer_.get(), total_base_size_, 0),
+ true);
+
+ // REPLACE ops which are compressed
+ std::memcpy(orig_buffer_.get(), data.data(), data.size());
+ size_t offset = data.size();
+
+ // ZERO ops
+ std::string zero_buffer(total_zero_blocks * BLOCK_SZ, 0);
+ std::memcpy((char*)orig_buffer_.get() + offset, (void*)zero_buffer.c_str(), zero_buffer.size());
+ offset += zero_buffer.size();
+
+ // REPLACE ops - Random buffers which aren't compressed
+ std::memcpy((char*)orig_buffer_.get() + offset, random_buffer.c_str(), random_buffer.size());
+}
+
+TEST_P(HandlerTestV3, Read) {
+ std::unique_ptr<uint8_t[]> snapuserd_buffer = std::make_unique<uint8_t[]>(size_);
+
+ size_t read_size = SECTOR_SIZE;
+ off_t offset = 0;
+ // Read the first sector
+ ASSERT_TRUE(ReadSectors(1, read_size, snapuserd_buffer.get()));
+ // Validate the data
+ ASSERT_EQ(std::memcmp(snapuserd_buffer.get(), orig_buffer_.get(), read_size), 0);
+
+ // Read the second block at offset 7680 (Sector 15). This will map to the
+ // first COW operation for variable block size
+ offset += (((BLOCK_SZ * 2) - SECTOR_SIZE));
+ read_size = BLOCK_SZ; // Span across two REPLACE ops
+ ASSERT_TRUE(ReadSectors(offset / SECTOR_SIZE, read_size, snapuserd_buffer.get()));
+ // Validate the data
+ ASSERT_EQ(std::memcmp(snapuserd_buffer.get(), (char*)orig_buffer_.get() + offset, read_size),
+ 0);
+
+ // Fill some other data since we are going to read zero blocks
+ std::memset(snapuserd_buffer.get(), 'Z', size_);
+
+ size_t num_blocks_per_op = (size_ / BLOCK_SZ) / 4;
+ offset = num_blocks_per_op * BLOCK_SZ;
+ // Issue read spanning between a REPLACE op and ZERO ops. The starting point
+ // is the last REPLACE op at sector 5118
+ offset -= (SECTOR_SIZE * 2);
+ // This will make sure it falls back to aligned reads after reading the
+ // first unaligned block
+ read_size = BLOCK_SZ * 6;
+ ASSERT_TRUE(ReadSectors(offset / SECTOR_SIZE, read_size, snapuserd_buffer.get()));
+ // Validate the data
+ ASSERT_EQ(std::memcmp(snapuserd_buffer.get(), (char*)orig_buffer_.get() + offset, read_size),
+ 0);
+
+ // Issue I/O request at the last block. The first chunk of (SECTOR_SIZE * 2)
+ // will be from REPLACE op which has random buffers
+ offset = (size_ - (SECTOR_SIZE * 2));
+ // Request will span beyond the COW mapping, thereby fetching data from base
+ // device.
+ read_size = BLOCK_SZ * 8;
+ ASSERT_TRUE(ReadSectors(offset / SECTOR_SIZE, read_size, snapuserd_buffer.get()));
+ // Validate the data
+ ASSERT_EQ(std::memcmp(snapuserd_buffer.get(), (char*)orig_buffer_.get() + offset, read_size),
+ 0);
+
+ // Issue I/O request which are not mapped to any COW operations
+ offset = (size_ + (SECTOR_SIZE * 3));
+ read_size = BLOCK_SZ * 3;
+ ASSERT_TRUE(ReadSectors(offset / SECTOR_SIZE, read_size, snapuserd_buffer.get()));
+ // Validate the data
+ ASSERT_EQ(std::memcmp(snapuserd_buffer.get(), (char*)orig_buffer_.get() + offset, read_size),
+ 0);
+}
+
std::vector<bool> GetIoUringConfigs() {
#if __ANDROID__
if (!android::base::GetBoolProperty("ro.virtual_ab.io_uring.enabled", false)) {
@@ -1018,6 +1486,37 @@
return testParams;
}
+std::vector<TestParam> GetVariableBlockTestConfigs() {
+ std::vector<TestParam> testParams;
+
+ std::vector<int> block_sizes = {4096, 8192, 16384, 32768, 65536, 131072};
+ std::vector<std::string> compression_algo = {"none", "lz4", "zstd", "gz"};
+ std::vector<int> threads = {1, 2};
+ std::vector<bool> uring_configs = GetIoUringConfigs();
+
+ // This should test 96 combination and validates the I/O path
+ for (auto block : block_sizes) {
+ for (auto compression : compression_algo) {
+ for (auto thread : threads) {
+ for (auto io_uring : uring_configs) {
+ TestParam param;
+ param.block_size = block;
+ param.compression = compression;
+ param.num_threads = thread;
+ param.io_uring = io_uring;
+ param.o_direct = false;
+ testParams.push_back(std::move(param));
+ }
+ }
+ }
+ }
+
+ return testParams;
+}
+
+INSTANTIATE_TEST_SUITE_P(Io, SnapuserdVariableBlockSizeTest,
+ ::testing::ValuesIn(GetVariableBlockTestConfigs()));
+INSTANTIATE_TEST_SUITE_P(Io, HandlerTestV3, ::testing::ValuesIn(GetVariableBlockTestConfigs()));
INSTANTIATE_TEST_SUITE_P(Io, SnapuserdTest, ::testing::ValuesIn(GetTestConfigs()));
INSTANTIATE_TEST_SUITE_P(Io, HandlerTest, ::testing::ValuesIn(GetTestConfigs()));