Write verity first, then do fs verification
Old behavior:
Read partition, for each block:
Update hasher
Update verity writer
before reading hashtree/verity:
write hashtree/verity to disk
Read the last verity blocks.
Finalize hasher, verity hashes.
The old bahvior tries to minimize fs read by only read once and feed
data to hasher and verity writer. However, in VABC, reading/writing are
handled very differently. Read can be done via regular fd, but writes
must go through special COW API. As we have seen in b/186196758, using
COW API in filesystem hashing can lead to inconsistent read and boot
failure. Therefore, we've decided to write verity first using COW API,
then read/hash partition using regular fd. This does mean that we need
to read everything twice, but we think this is a worth while tradeoff.
As verity writes can take 5 minutes, but reading the entire partition
again only takes <10 seconds.
New behavior:
Read partition, for each block:
Update verity writer
Finalize verity writer, write verity to disk
launch snapuserd, open a regular fd.
Read partition, for each block:
Update hasher
Finaliaze hasher, verity hashes.
Test: th
Test: Manual testing on pixel of the following scenario:
1. Verity enabled, VABC enabled, pause/resume multiple times
2. Verity disabled, VABC enabled, pause/resume multiple times
3. Verity Enabled, VABC enabled, pause/resume multiple times
Bug: 186196758
Change-Id: I2477c2dc4da5b921e84b48a54d0d8a877c1a52ef
diff --git a/payload_consumer/file_descriptor.cc b/payload_consumer/file_descriptor.cc
index 7c69c1b..da76327 100644
--- a/payload_consumer/file_descriptor.cc
+++ b/payload_consumer/file_descriptor.cc
@@ -139,7 +139,9 @@
}
bool EintrSafeFileDescriptor::Close() {
- CHECK_GE(fd_, 0);
+ if (fd_ < 0) {
+ return false;
+ }
// https://stackoverflow.com/questions/705454/does-linux-guarantee-the-contents-of-a-file-is-flushed-to-disc-after-close
// |close()| doesn't imply |fsync()|, we need to do it manually.
fsync(fd_);
diff --git a/payload_consumer/filesystem_verifier_action.cc b/payload_consumer/filesystem_verifier_action.cc
index 8d496a6..bec5aca 100644
--- a/payload_consumer/filesystem_verifier_action.cc
+++ b/payload_consumer/filesystem_verifier_action.cc
@@ -35,6 +35,7 @@
#include <brillo/secure_blob.h>
#include <brillo/streams/file_stream.h>
+#include "common/error_code.h"
#include "payload_generator/delta_diff_generator.h"
#include "update_engine/common/utils.h"
#include "update_engine/payload_consumer/file_descriptor.h"
@@ -77,6 +78,7 @@
namespace {
const off_t kReadFileBufferSize = 128 * 1024;
+constexpr float VERITY_PROGRESS_PERCENT = 0.6;
} // namespace
void FilesystemVerifierAction::PerformAction() {
@@ -102,7 +104,6 @@
}
void FilesystemVerifierAction::TerminateProcessing() {
- brillo::MessageLoop::current()->CancelTask(pending_task_id_);
cancelled_ = true;
Cleanup(ErrorCode::kSuccess); // error code is ignored if canceled_ is true.
}
@@ -134,11 +135,29 @@
}
}
-bool FilesystemVerifierAction::InitializeFdVABC() {
+void FilesystemVerifierAction::UpdatePartitionProgress(double progress) {
+ // WE don't consider sizes of each partition. Every partition
+ // has the same length on progress bar.
+ // TODO(zhangkelvin) Take sizes of each partition into account
+ UpdateProgress((progress + partition_index_) /
+ install_plan_.partitions.size());
+}
+
+bool FilesystemVerifierAction::InitializeFdVABC(bool should_write_verity) {
const InstallPlan::Partition& partition =
install_plan_.partitions[partition_index_];
- if (!ShouldWriteVerity()) {
+ if (!should_write_verity) {
+ // In VABC, we cannot map/unmap partitions w/o first closing ALL fds first.
+ // Since this function might be called inside a ScheduledTask, the closure
+ // might have a copy of partition_fd_ when executing this function. Which
+ // means even if we do |partition_fd_.reset()| here, there's a chance that
+ // underlying fd isn't closed until we return. This is unacceptable, we need
+ // to close |partition_fd| right away.
+ if (partition_fd_) {
+ partition_fd_->Close();
+ partition_fd_.reset();
+ }
// In VABC, if we are not writing verity, just map all partitions,
// and read using regular fd on |postinstall_mount_device| .
// All read will go through snapuserd, which provides a consistent
@@ -152,8 +171,6 @@
dynamic_control_->MapAllPartitions();
return InitializeFd(partition.readonly_target_path);
}
-
- // FilesystemVerifierAction need the read_fd_.
partition_fd_ =
dynamic_control_->OpenCowFd(partition.name, partition.source_path, true);
if (!partition_fd_) {
@@ -180,6 +197,112 @@
return true;
}
+void FilesystemVerifierAction::WriteVerityAndHashPartition(
+ FileDescriptorPtr fd,
+ const off64_t start_offset,
+ const off64_t end_offset,
+ void* buffer,
+ const size_t buffer_size) {
+ if (start_offset >= end_offset) {
+ LOG_IF(WARNING, start_offset > end_offset)
+ << "start_offset is greater than end_offset : " << start_offset << " > "
+ << end_offset;
+ if (!verity_writer_->Finalize(fd, fd)) {
+ LOG(ERROR) << "Failed to write verity data";
+ Cleanup(ErrorCode::kVerityCalculationError);
+ return;
+ }
+ if (dynamic_control_->UpdateUsesSnapshotCompression()) {
+ // spin up snapuserd to read fs
+ if (!InitializeFdVABC(false)) {
+ LOG(ERROR) << "Failed to map all partitions";
+ Cleanup(ErrorCode::kFilesystemVerifierError);
+ return;
+ }
+ }
+ HashPartition(partition_fd_, 0, partition_size_, buffer, buffer_size);
+ return;
+ }
+ const auto cur_offset = fd->Seek(start_offset, SEEK_SET);
+ if (cur_offset != start_offset) {
+ PLOG(ERROR) << "Failed to seek to offset: " << start_offset;
+ Cleanup(ErrorCode::kVerityCalculationError);
+ return;
+ }
+ const auto read_size =
+ std::min<size_t>(buffer_size, end_offset - start_offset);
+ const auto bytes_read = fd->Read(buffer, read_size);
+ if (bytes_read < 0 || static_cast<size_t>(bytes_read) != read_size) {
+ PLOG(ERROR) << "Failed to read offset " << start_offset << " expected "
+ << read_size << " bytes, actual: " << bytes_read;
+ Cleanup(ErrorCode::kVerityCalculationError);
+ return;
+ }
+ if (!verity_writer_->Update(
+ start_offset, static_cast<const uint8_t*>(buffer), read_size)) {
+ LOG(ERROR) << "VerityWriter::Update() failed";
+ Cleanup(ErrorCode::kVerityCalculationError);
+ return;
+ }
+ UpdatePartitionProgress((start_offset + bytes_read) * 1.0f / partition_size_ *
+ VERITY_PROGRESS_PERCENT);
+ CHECK(pending_task_id_.PostTask(
+ FROM_HERE,
+ base::BindOnce(&FilesystemVerifierAction::WriteVerityAndHashPartition,
+ base::Unretained(this),
+ fd,
+ start_offset + bytes_read,
+ end_offset,
+ buffer,
+ buffer_size)));
+}
+
+void FilesystemVerifierAction::HashPartition(FileDescriptorPtr fd,
+ const off64_t start_offset,
+ const off64_t end_offset,
+ void* buffer,
+ const size_t buffer_size) {
+ if (start_offset >= end_offset) {
+ LOG_IF(WARNING, start_offset > end_offset)
+ << "start_offset is greater than end_offset : " << start_offset << " > "
+ << end_offset;
+ FinishPartitionHashing();
+ return;
+ }
+ const auto cur_offset = fd->Seek(start_offset, SEEK_SET);
+ if (cur_offset != start_offset) {
+ PLOG(ERROR) << "Failed to seek to offset: " << start_offset;
+ Cleanup(ErrorCode::kFilesystemVerifierError);
+ return;
+ }
+ const auto read_size =
+ std::min<size_t>(buffer_size, end_offset - start_offset);
+ const auto bytes_read = fd->Read(buffer, read_size);
+ if (bytes_read < 0 || static_cast<size_t>(bytes_read) != read_size) {
+ PLOG(ERROR) << "Failed to read offset " << start_offset << " expected "
+ << read_size << " bytes, actual: " << bytes_read;
+ Cleanup(ErrorCode::kFilesystemVerifierError);
+ return;
+ }
+ if (!hasher_->Update(buffer, read_size)) {
+ LOG(ERROR) << "Hasher updated failed on offset" << start_offset;
+ Cleanup(ErrorCode::kFilesystemVerifierError);
+ return;
+ }
+ const auto progress = (start_offset + bytes_read) * 1.0f / partition_size_;
+ UpdatePartitionProgress(progress * (1 - VERITY_PROGRESS_PERCENT) +
+ VERITY_PROGRESS_PERCENT);
+ CHECK(pending_task_id_.PostTask(
+ FROM_HERE,
+ base::BindOnce(&FilesystemVerifierAction::HashPartition,
+ base::Unretained(this),
+ fd,
+ start_offset + bytes_read,
+ end_offset,
+ buffer,
+ buffer_size)));
+}
+
void FilesystemVerifierAction::StartPartitionHashing() {
if (partition_index_ == install_plan_.partitions.size()) {
if (!install_plan_.untouched_dynamic_partitions.empty()) {
@@ -208,7 +331,7 @@
<< partition.name << ") on device " << part_path;
auto success = false;
if (IsVABC(partition)) {
- success = InitializeFdVABC();
+ success = InitializeFdVABC(ShouldWriteVerity());
} else {
if (part_path.empty()) {
if (partition_size_ == 0) {
@@ -243,17 +366,19 @@
filesystem_data_end_ = partition.fec_offset;
}
if (ShouldWriteVerity()) {
+ LOG(INFO) << "Verity writes enabled on partition " << partition.name;
if (!verity_writer_->Init(partition)) {
LOG(INFO) << "Verity writes enabled on partition " << partition.name;
Cleanup(ErrorCode::kVerityCalculationError);
return;
}
+ WriteVerityAndHashPartition(
+ partition_fd_, 0, filesystem_data_end_, buffer_.data(), buffer_.size());
} else {
LOG(INFO) << "Verity writes disabled on partition " << partition.name;
+ HashPartition(
+ partition_fd_, 0, partition_size_, buffer_.data(), buffer_.size());
}
-
- // Start the first read.
- ScheduleFileSystemRead();
}
bool FilesystemVerifierAction::IsVABC(
@@ -298,106 +423,6 @@
(partition.hash_tree_size > 0 || partition.fec_size > 0);
}
-void FilesystemVerifierAction::ReadVerityAndFooter() {
- if (ShouldWriteVerity()) {
- if (!verity_writer_->Finalize(partition_fd_, partition_fd_)) {
- LOG(ERROR) << "Failed to write hashtree/FEC data.";
- Cleanup(ErrorCode::kFilesystemVerifierError);
- return;
- }
- }
- // Since we handed our |read_fd_| to verity_writer_ during |Finalize()|
- // call, fd's position could have been changed. Re-seek.
- partition_fd_->Seek(filesystem_data_end_, SEEK_SET);
- auto bytes_to_read = partition_size_ - filesystem_data_end_;
- while (bytes_to_read > 0) {
- const auto read_size = std::min<size_t>(buffer_.size(), bytes_to_read);
- auto bytes_read = partition_fd_->Read(buffer_.data(), read_size);
- if (bytes_read <= 0) {
- PLOG(ERROR) << "Failed to read hash tree " << bytes_read;
- Cleanup(ErrorCode::kFilesystemVerifierError);
- return;
- }
- if (!hasher_->Update(buffer_.data(), bytes_read)) {
- LOG(ERROR) << "Unable to update the hash.";
- Cleanup(ErrorCode::kError);
- return;
- }
- bytes_to_read -= bytes_read;
- }
- FinishPartitionHashing();
-}
-
-void FilesystemVerifierAction::ScheduleFileSystemRead() {
- // We can only start reading anything past |hash_tree_offset| after we have
- // already read all the data blocks that the hash tree covers. The same
- // applies to FEC.
-
- size_t bytes_to_read = std::min(static_cast<uint64_t>(buffer_.size()),
- filesystem_data_end_ - offset_);
- if (!bytes_to_read) {
- ReadVerityAndFooter();
- return;
- }
- partition_fd_->Seek(offset_, SEEK_SET);
- auto bytes_read = partition_fd_->Read(buffer_.data(), bytes_to_read);
- if (bytes_read < 0) {
- LOG(ERROR) << "Unable to schedule an asynchronous read from the stream. "
- << bytes_read;
- Cleanup(ErrorCode::kError);
- } else {
- // We could just invoke |OnReadDoneCallback()|, it works. But |PostTask|
- // is used so that users can cancel updates.
- pending_task_id_ = brillo::MessageLoop::current()->PostTask(
- base::Bind(&FilesystemVerifierAction::OnReadDone,
- base::Unretained(this),
- bytes_read));
- }
-}
-
-void FilesystemVerifierAction::OnReadDone(size_t bytes_read) {
- if (cancelled_) {
- Cleanup(ErrorCode::kError);
- return;
- }
- if (bytes_read == 0) {
- LOG(ERROR) << "Failed to read the remaining " << partition_size_ - offset_
- << " bytes from partition "
- << install_plan_.partitions[partition_index_].name;
- Cleanup(ErrorCode::kFilesystemVerifierError);
- return;
- }
-
- if (!hasher_->Update(buffer_.data(), bytes_read)) {
- LOG(ERROR) << "Unable to update the hash.";
- Cleanup(ErrorCode::kError);
- return;
- }
-
- // WE don't consider sizes of each partition. Every partition
- // has the same length on progress bar.
- // TODO(zhangkelvin) Take sizes of each partition into account
-
- UpdateProgress(
- (static_cast<double>(offset_) / partition_size_ + partition_index_) /
- install_plan_.partitions.size());
- if (ShouldWriteVerity()) {
- if (!verity_writer_->Update(offset_, buffer_.data(), bytes_read)) {
- LOG(ERROR) << "Unable to update verity";
- Cleanup(ErrorCode::kVerityCalculationError);
- return;
- }
- }
-
- offset_ += bytes_read;
- if (offset_ == filesystem_data_end_) {
- ReadVerityAndFooter();
- return;
- }
-
- ScheduleFileSystemRead();
-}
-
void FilesystemVerifierAction::FinishPartitionHashing() {
if (!hasher_->Finalize()) {
LOG(ERROR) << "Unable to finalize the hash.";
@@ -469,6 +494,7 @@
hasher_.reset();
buffer_.clear();
if (partition_fd_) {
+ partition_fd_->Close();
partition_fd_.reset();
}
StartPartitionHashing();
diff --git a/payload_consumer/filesystem_verifier_action.h b/payload_consumer/filesystem_verifier_action.h
index 68a9432..850abda 100644
--- a/payload_consumer/filesystem_verifier_action.h
+++ b/payload_consumer/filesystem_verifier_action.h
@@ -22,12 +22,14 @@
#include <memory>
#include <string>
+#include <utility>
#include <vector>
#include <brillo/message_loops/message_loop.h>
#include "update_engine/common/action.h"
#include "update_engine/common/hash_calculator.h"
+#include "update_engine/common/scoped_task_id.h"
#include "update_engine/payload_consumer/file_descriptor.h"
#include "update_engine/payload_consumer/install_plan.h"
#include "update_engine/payload_consumer/verity_writer_interface.h"
@@ -84,6 +86,16 @@
private:
friend class FilesystemVerifierActionTestDelegate;
+ void WriteVerityAndHashPartition(FileDescriptorPtr fd,
+ const off64_t start_offset,
+ const off64_t end_offset,
+ void* buffer,
+ const size_t buffer_size);
+ void HashPartition(FileDescriptorPtr fd,
+ const off64_t start_offset,
+ const off64_t end_offset,
+ void* buffer,
+ const size_t buffer_size);
// Return true if we need to write verity bytes.
bool ShouldWriteVerity();
@@ -97,17 +109,6 @@
size_t GetPartitionSize() const;
- // Schedules the asynchronous read of the filesystem part of this
- // partition(not including hashtree/verity).
- void ScheduleFileSystemRead();
-
- // Read the verity part of this partition.(hash tree and FEC)
- void ReadVerityAndFooter();
-
- // Called from the main loop when a single read from |src_stream_| succeeds or
- // fails, calling OnReadDoneCallback() and OnReadErrorCallback() respectively.
- void OnReadDone(size_t bytes_read);
-
// When the read is done, finalize the hash checking of the current partition
// and continue checking the next one.
void FinishPartitionHashing();
@@ -120,9 +121,13 @@
// Invoke delegate callback to report progress, if delegate is not null
void UpdateProgress(double progress);
+ // Updates progress of current partition. |progress| should be in range [0,
+ // 1], and it will be scaled appropriately with # of partitions.
+ void UpdatePartitionProgress(double progress);
+
// Initialize read_fd_ and write_fd_
bool InitializeFd(const std::string& part_path);
- bool InitializeFdVABC();
+ bool InitializeFdVABC(bool should_write_verity);
// The type of the partition that we are verifying.
VerifierStep verifier_step_ = VerifierStep::kVerifyTargetHash;
@@ -167,8 +172,7 @@
// Callback that should be cancelled on |TerminateProcessing|. Usually this
// points to pending read callbacks from async stream.
- brillo::MessageLoop::TaskId pending_task_id_{
- brillo::MessageLoop::kTaskIdNull};
+ ScopedTaskId pending_task_id_;
DISALLOW_COPY_AND_ASSIGN(FilesystemVerifierAction);
};
diff --git a/payload_consumer/filesystem_verifier_action_unittest.cc b/payload_consumer/filesystem_verifier_action_unittest.cc
index ce2f437..f2f2954 100644
--- a/payload_consumer/filesystem_verifier_action_unittest.cc
+++ b/payload_consumer/filesystem_verifier_action_unittest.cc
@@ -30,6 +30,7 @@
#include <fec/ecc.h>
#include <gtest/gtest.h>
#include <libsnapshot/snapshot_writer.h>
+#include <sys/stat.h>
#include "update_engine/common/dynamic_partition_control_stub.h"
#include "update_engine/common/hash_calculator.h"
@@ -78,6 +79,13 @@
fec_data_.resize(fec_size);
hash_tree_data_.resize(hash_tree_size);
+ // Globally readable writable, as we want to write data
+ ASSERT_EQ(0, fchmod(source_part_.fd(), 0666))
+ << " Failed to set " << source_part_.path() << " as writable "
+ << strerror(errno);
+ ASSERT_EQ(0, fchmod(target_part_.fd(), 0666))
+ << " Failed to set " << target_part_.path() << " as writable "
+ << strerror(errno);
brillo::Blob part_data(PARTITION_SIZE);
test_utils::FillWithData(&part_data);
ASSERT_TRUE(utils::WriteFile(
@@ -193,9 +201,9 @@
};
ScopedTempFile FilesystemVerifierActionTest::source_part_{
- "source_part.XXXXXX", false, PARTITION_SIZE};
+ "source_part.XXXXXX", true, PARTITION_SIZE};
ScopedTempFile FilesystemVerifierActionTest::target_part_{
- "target_part.XXXXXX", false, PARTITION_SIZE};
+ "target_part.XXXXXX", true, PARTITION_SIZE};
static void EnableVABC(MockDynamicPartitionControl* dynamic_control,
const std::string& part_name) {
@@ -295,16 +303,12 @@
FilesystemVerifierActionTestDelegate delegate;
processor_.set_delegate(&delegate);
- loop_.PostTask(FROM_HERE,
- base::Bind(
- [](ActionProcessor* processor, bool terminate_early) {
- processor->StartProcessing();
- if (terminate_early) {
- processor->StopProcessing();
- }
- },
- base::Unretained(&processor_),
- terminate_early));
+ loop_.PostTask(base::Bind(&ActionProcessor::StartProcessing,
+ base::Unretained(&processor_)));
+ if (terminate_early) {
+ loop_.PostTask(base::Bind(&ActionProcessor::StopProcessing,
+ base::Unretained(&processor_)));
+ }
loop_.Run();
if (!terminate_early) {
@@ -560,21 +564,33 @@
NiceMock<MockDynamicPartitionControl> dynamic_control;
EnableVABC(&dynamic_control, part.name);
+ auto open_cow = [part]() {
+ auto cow_fd = std::make_shared<EintrSafeFileDescriptor>();
+ EXPECT_TRUE(cow_fd->Open(part.readonly_target_path.c_str(), O_RDWR))
+ << "Failed to open part " << part.readonly_target_path
+ << strerror(errno);
+ return cow_fd;
+ };
EXPECT_CALL(dynamic_control, UpdateUsesSnapshotCompression())
.Times(AtLeast(1));
- auto cow_fd = std::make_shared<EintrSafeFileDescriptor>();
- ASSERT_TRUE(cow_fd->Open(part.readonly_target_path.c_str(), O_RDWR));
+ auto cow_fd = open_cow();
+ if (HasFailure()) {
+ return;
+ }
+
if (enable_verity) {
ON_CALL(dynamic_control, OpenCowFd(part.name, {part.source_path}, _))
- .WillByDefault(Return(cow_fd));
+ .WillByDefault(open_cow);
EXPECT_CALL(dynamic_control, OpenCowFd(part.name, {part.source_path}, _))
.Times(AtLeast(1));
- // If we are writing verity, fs verification shouldn't try to open fd
- // against |postinstall_mount_device| or |target_path| at all. It should
- // just use whatever file descriptor returned by OpenCowFd(). Therefore set
- // a fake path to prevent fs verification from trying to open it.
- part.readonly_target_path = "/dev/fake_postinstall_mount_device";
+
+ // fs verification isn't supposed to write to |readonly_target_path|. All
+ // writes should go through fd returned by |OpenCowFd|. Therefore we set
+ // target part as read-only to make sure.
+ ASSERT_EQ(0, chmod(part.readonly_target_path.c_str(), 0444))
+ << " Failed to set " << part.readonly_target_path << " as read-only "
+ << strerror(errno);
} else {
// Since we are not writing verity, we should not attempt to OpenCowFd()
// reads should go through regular file descriptors on mapped partitions.
@@ -592,11 +608,9 @@
FilesystemVerifierActionTestDelegate delegate;
processor_.set_delegate(&delegate);
- loop_.PostTask(
- FROM_HERE,
- base::Bind(
- [](ActionProcessor* processor) { processor->StartProcessing(); },
- base::Unretained(&processor_)));
+ loop_.PostTask(FROM_HERE,
+ base::Bind(&ActionProcessor::StartProcessing,
+ base::Unretained(&processor_)));
loop_.Run();
ASSERT_FALSE(processor_.IsRunning());