Calculate verity hash tree.
Calculate verity hash tree in FilesystemVerifierAction based on configs
specified in protobuf, and write it to target partition before reading
from those blocks.
A new error code kVerityCalculationError was added to report error if
it fails.
Bug: 28171891
Test: update_engine_unittests
Change-Id: I492885a0655bf51043902f578720fffd87e6a3cf
diff --git a/Android.mk b/Android.mk
index e141ae1..ba66d67 100644
--- a/Android.mk
+++ b/Android.mk
@@ -109,8 +109,10 @@
libbspatch \
libbrotli \
libpuffpatch \
+ libverity_tree \
$(ue_update_metadata_protos_exported_static_libraries)
ue_libpayload_consumer_exported_shared_libraries := \
+ libbase \
libcrypto \
$(ue_update_metadata_protos_exported_shared_libraries)
@@ -148,6 +150,7 @@
payload_consumer/payload_metadata.cc \
payload_consumer/payload_verifier.cc \
payload_consumer/postinstall_runner_action.cc \
+ payload_consumer/verity_writer_android.cc \
payload_consumer/xz_extent_writer.cc
ifeq ($(local_use_fec),1)
@@ -941,6 +944,7 @@
payload_consumer/file_writer_unittest.cc \
payload_consumer/filesystem_verifier_action_unittest.cc \
payload_consumer/postinstall_runner_action_unittest.cc \
+ payload_consumer/verity_writer_android_unittest.cc \
payload_consumer/xz_extent_writer_unittest.cc \
payload_generator/ab_generator_unittest.cc \
payload_generator/blob_file_writer_unittest.cc \
diff --git a/common/error_code.h b/common/error_code.h
index 86b7a3e..252cc42 100644
--- a/common/error_code.h
+++ b/common/error_code.h
@@ -79,6 +79,7 @@
kNoUpdate = 53,
kRollbackNotPossible = 54,
kFirstActiveOmahaPingSentPersistenceError = 55,
+ kVerityCalculationError = 56,
// VERY IMPORTANT! When adding new error codes:
//
diff --git a/common/error_code_utils.cc b/common/error_code_utils.cc
index 930dafe..a1607f5 100644
--- a/common/error_code_utils.cc
+++ b/common/error_code_utils.cc
@@ -156,6 +156,8 @@
return "ErrorCode::kRollbackNotPossible";
case ErrorCode::kFirstActiveOmahaPingSentPersistenceError:
return "ErrorCode::kFirstActiveOmahaPingSentPersistenceError";
+ case ErrorCode::kVerityCalculationError:
+ return "ErrorCode::kVerityCalculationError";
// Don't add a default case to let the compiler warn about newly added
// error codes which should be added here.
}
diff --git a/metrics_utils.cc b/metrics_utils.cc
index e7b2347..e1aa744 100644
--- a/metrics_utils.cc
+++ b/metrics_utils.cc
@@ -68,6 +68,7 @@
case ErrorCode::kDownloadWriteError:
case ErrorCode::kFilesystemCopierError:
case ErrorCode::kFilesystemVerifierError:
+ case ErrorCode::kVerityCalculationError:
return metrics::AttemptResult::kOperationExecutionError;
case ErrorCode::kDownloadMetadataSignatureMismatch:
@@ -224,6 +225,7 @@
case ErrorCode::kNoUpdate:
case ErrorCode::kRollbackNotPossible:
case ErrorCode::kFirstActiveOmahaPingSentPersistenceError:
+ case ErrorCode::kVerityCalculationError:
break;
// Special flags. These can't happen (we mask them out above) but
diff --git a/payload_consumer/delta_performer.cc b/payload_consumer/delta_performer.cc
index 7831c0f..e700255 100644
--- a/payload_consumer/delta_performer.cc
+++ b/payload_consumer/delta_performer.cc
@@ -607,6 +607,8 @@
// Clear the download buffer.
DiscardBuffer(false, metadata_size_);
+ block_size_ = manifest_.block_size();
+
// This populates |partitions_| and the |install_plan.partitions| with the
// list of partitions from the manifest.
if (!ParseManifestPartitions(error))
@@ -868,6 +870,45 @@
install_part.target_size = info.size();
install_part.target_hash.assign(info.hash().begin(), info.hash().end());
+ install_part.block_size = block_size_;
+ if (partition.has_hash_tree_extent()) {
+ Extent extent = partition.hash_tree_data_extent();
+ install_part.hash_tree_data_offset = extent.start_block() * block_size_;
+ install_part.hash_tree_data_size = extent.num_blocks() * block_size_;
+ extent = partition.hash_tree_extent();
+ install_part.hash_tree_offset = extent.start_block() * block_size_;
+ install_part.hash_tree_size = extent.num_blocks() * block_size_;
+ uint64_t hash_tree_data_end =
+ install_part.hash_tree_data_offset + install_part.hash_tree_data_size;
+ if (install_part.hash_tree_offset < hash_tree_data_end) {
+ LOG(ERROR) << "Invalid hash tree extents, hash tree data ends at "
+ << hash_tree_data_end << ", but hash tree starts at "
+ << install_part.hash_tree_offset;
+ *error = ErrorCode::kDownloadNewPartitionInfoError;
+ return false;
+ }
+ install_part.hash_tree_algorithm = partition.hash_tree_algorithm();
+ install_part.hash_tree_salt.assign(partition.hash_tree_salt().begin(),
+ partition.hash_tree_salt().end());
+ }
+ if (partition.has_fec_extent()) {
+ Extent extent = partition.fec_data_extent();
+ install_part.fec_data_offset = extent.start_block() * block_size_;
+ install_part.fec_data_size = extent.num_blocks() * block_size_;
+ extent = partition.fec_extent();
+ install_part.fec_offset = extent.start_block() * block_size_;
+ install_part.fec_size = extent.num_blocks() * block_size_;
+ uint64_t fec_data_end =
+ install_part.fec_data_offset + install_part.fec_data_size;
+ if (install_part.fec_offset < fec_data_end) {
+ LOG(ERROR) << "Invalid fec extents, fec data ends at " << fec_data_end
+ << ", but fec starts at " << install_part.fec_offset;
+ *error = ErrorCode::kDownloadNewPartitionInfoError;
+ return false;
+ }
+ install_part.fec_roots = partition.fec_roots();
+ }
+
install_plan_->partitions.push_back(install_part);
}
@@ -1840,7 +1881,6 @@
bool DeltaPerformer::PrimeUpdateState() {
CHECK(manifest_valid_);
- block_size_ = manifest_.block_size();
int64_t next_operation = kUpdateStateOperationInvalid;
if (!prefs_->GetInt64(kPrefsUpdateStateNextOperation, &next_operation) ||
diff --git a/payload_consumer/filesystem_verifier_action.cc b/payload_consumer/filesystem_verifier_action.cc
index 6a379e5..a5e1e61 100644
--- a/payload_consumer/filesystem_verifier_action.cc
+++ b/payload_consumer/filesystem_verifier_action.cc
@@ -29,10 +29,7 @@
#include <brillo/data_encoding.h>
#include <brillo/streams/file_stream.h>
-#include "update_engine/common/boot_control_interface.h"
#include "update_engine/common/utils.h"
-#include "update_engine/payload_consumer/delta_performer.h"
-#include "update_engine/payload_consumer/payload_constants.h"
using brillo::data_encoding::Base64Encode;
using std::string;
@@ -87,24 +84,26 @@
Cleanup(ErrorCode::kSuccess);
return;
}
- InstallPlan::Partition& partition =
+ const InstallPlan::Partition& partition =
install_plan_.partitions[partition_index_];
string part_path;
switch (verifier_step_) {
case VerifierStep::kVerifySourceHash:
part_path = partition.source_path;
- remaining_size_ = partition.source_size;
+ partition_size_ = partition.source_size;
break;
case VerifierStep::kVerifyTargetHash:
part_path = partition.target_path;
- remaining_size_ = partition.target_size;
+ partition_size_ = partition.target_size;
break;
}
LOG(INFO) << "Hashing partition " << partition_index_ << " ("
<< partition.name << ") on device " << part_path;
- if (part_path.empty())
- return Cleanup(ErrorCode::kFilesystemVerifierError);
+ if (part_path.empty()) {
+ Cleanup(ErrorCode::kFilesystemVerifierError);
+ return;
+ }
brillo::ErrorPtr error;
src_stream_ = brillo::FileStream::Open(
@@ -115,33 +114,54 @@
if (!src_stream_) {
LOG(ERROR) << "Unable to open " << part_path << " for reading";
- return Cleanup(ErrorCode::kFilesystemVerifierError);
+ Cleanup(ErrorCode::kFilesystemVerifierError);
+ return;
}
buffer_.resize(kReadFileBufferSize);
- read_done_ = false;
- hasher_.reset(new HashCalculator());
+ hasher_ = std::make_unique<HashCalculator>();
+
+ offset_ = 0;
+ if (verifier_step_ == VerifierStep::kVerifyTargetHash) {
+ if (!verity_writer_->Init(partition)) {
+ Cleanup(ErrorCode::kVerityCalculationError);
+ return;
+ }
+ }
// Start the first read.
ScheduleRead();
}
void FilesystemVerifierAction::ScheduleRead() {
- size_t bytes_to_read = std::min(static_cast<int64_t>(buffer_.size()),
- remaining_size_);
+ const InstallPlan::Partition& partition =
+ install_plan_.partitions[partition_index_];
+
+ // 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.
+ uint64_t read_end = partition_size_;
+ if (partition.hash_tree_size != 0 &&
+ offset_ < partition.hash_tree_data_offset + partition.hash_tree_data_size)
+ read_end = std::min(read_end, partition.hash_tree_offset);
+ if (partition.fec_size != 0 &&
+ offset_ < partition.fec_data_offset + partition.fec_data_size)
+ read_end = std::min(read_end, partition.fec_offset);
+ size_t bytes_to_read =
+ std::min(static_cast<uint64_t>(buffer_.size()), read_end - offset_);
if (!bytes_to_read) {
- OnReadDoneCallback(0);
+ FinishPartitionHashing();
return;
}
bool read_async_ok = src_stream_->ReadAsync(
- buffer_.data(),
- bytes_to_read,
- base::Bind(&FilesystemVerifierAction::OnReadDoneCallback,
- base::Unretained(this)),
- base::Bind(&FilesystemVerifierAction::OnReadErrorCallback,
- base::Unretained(this)),
- nullptr);
+ buffer_.data(),
+ bytes_to_read,
+ base::Bind(&FilesystemVerifierAction::OnReadDoneCallback,
+ base::Unretained(this)),
+ base::Bind(&FilesystemVerifierAction::OnReadErrorCallback,
+ base::Unretained(this)),
+ nullptr);
if (!read_async_ok) {
LOG(ERROR) << "Unable to schedule an asynchronous read from the stream.";
@@ -150,31 +170,39 @@
}
void FilesystemVerifierAction::OnReadDoneCallback(size_t bytes_read) {
+ if (cancelled_) {
+ Cleanup(ErrorCode::kError);
+ return;
+ }
+
if (bytes_read == 0) {
- read_done_ = true;
- } else {
- remaining_size_ -= bytes_read;
- CHECK(!read_done_);
- if (!hasher_->Update(buffer_.data(), bytes_read)) {
- LOG(ERROR) << "Unable to update the hash.";
- Cleanup(ErrorCode::kError);
+ 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;
+ }
+
+ if (verifier_step_ == VerifierStep::kVerifyTargetHash) {
+ if (!verity_writer_->Update(offset_, buffer_.data(), bytes_read)) {
+ Cleanup(ErrorCode::kVerityCalculationError);
return;
}
}
- // We either terminate the current partition or have more data to read.
- if (cancelled_)
- return Cleanup(ErrorCode::kError);
+ offset_ += bytes_read;
- if (read_done_ || remaining_size_ == 0) {
- if (remaining_size_ != 0) {
- LOG(ERROR) << "Failed to read the remaining " << remaining_size_
- << " bytes from partition "
- << install_plan_.partitions[partition_index_].name;
- return Cleanup(ErrorCode::kFilesystemVerifierError);
- }
- return FinishPartitionHashing();
+ if (offset_ == partition_size_) {
+ FinishPartitionHashing();
+ return;
}
+
ScheduleRead();
}
@@ -188,7 +216,8 @@
void FilesystemVerifierAction::FinishPartitionHashing() {
if (!hasher_->Finalize()) {
LOG(ERROR) << "Unable to finalize the hash.";
- return Cleanup(ErrorCode::kError);
+ Cleanup(ErrorCode::kError);
+ return;
}
InstallPlan::Partition& partition =
install_plan_.partitions[partition_index_];
@@ -202,7 +231,8 @@
<< "' partition verification failed.";
if (partition.source_hash.empty()) {
// No need to verify source if it is a full payload.
- return Cleanup(ErrorCode::kNewRootfsVerificationError);
+ Cleanup(ErrorCode::kNewRootfsVerificationError);
+ return;
}
// If we have not verified source partition yet, now that the target
// partition does not match, and it's not a full payload, we need to
@@ -238,7 +268,8 @@
"-binary | openssl base64";
LOG(INFO) << "To get the checksum of partitions in a bin file, "
<< "run: .../src/scripts/sha256_partitions.sh .../file.bin";
- return Cleanup(ErrorCode::kDownloadStateInitializationError);
+ Cleanup(ErrorCode::kDownloadStateInitializationError);
+ return;
}
// The action will skip kVerifySourceHash step if target partition hash
// matches, if we are in this step, it means target hash does not match,
@@ -246,7 +277,8 @@
// code to reflect the error in target partition.
// We only need to verify the source partition which the target hash does
// not match, the rest of the partitions don't matter.
- return Cleanup(ErrorCode::kNewRootfsVerificationError);
+ Cleanup(ErrorCode::kNewRootfsVerificationError);
+ return;
}
// Start hashing the next partition, if any.
hasher_.reset();
diff --git a/payload_consumer/filesystem_verifier_action.h b/payload_consumer/filesystem_verifier_action.h
index a21fc2a..83d6668 100644
--- a/payload_consumer/filesystem_verifier_action.h
+++ b/payload_consumer/filesystem_verifier_action.h
@@ -29,6 +29,7 @@
#include "update_engine/common/action.h"
#include "update_engine/common/hash_calculator.h"
#include "update_engine/payload_consumer/install_plan.h"
+#include "update_engine/payload_consumer/verity_writer_interface.h"
// This action will hash all the partitions of the target slot involved in the
// update. The hashes are then verified against the ones in the InstallPlan.
@@ -50,7 +51,9 @@
class FilesystemVerifierAction : public InstallPlanAction {
public:
- FilesystemVerifierAction() = default;
+ FilesystemVerifierAction()
+ : verity_writer_(verity_writer::CreateVerityWriter()) {}
+ ~FilesystemVerifierAction() override = default;
void PerformAction() override;
void TerminateProcessing() override;
@@ -95,7 +98,6 @@
// Buffer for storing data we read.
brillo::Blob buffer_;
- bool read_done_{false}; // true if reached EOF on the input stream.
bool cancelled_{false}; // true if the action has been cancelled.
// The install plan we're passed in via the input pipe.
@@ -104,10 +106,18 @@
// Calculates the hash of the data.
std::unique_ptr<HashCalculator> hasher_;
- // Reads and hashes this many bytes from the head of the input stream. This
- // field is initialized from the corresponding InstallPlan::Partition size,
- // when the partition starts to be hashed.
- int64_t remaining_size_{0};
+ // Write verity data of the current partition.
+ std::unique_ptr<VerityWriterInterface> verity_writer_;
+
+ // Reads and hashes this many bytes from the head of the input stream. When
+ // the partition starts to be hashed, this field is initialized from the
+ // corresponding InstallPlan::Partition size which is the total size
+ // update_engine is expected to write, and may be smaller than the size of the
+ // partition in gpt.
+ uint64_t partition_size_{0};
+
+ // The byte offset that we are reading in the current partition.
+ uint64_t offset_{0};
DISALLOW_COPY_AND_ASSIGN(FilesystemVerifierAction);
};
diff --git a/payload_consumer/filesystem_verifier_action_unittest.cc b/payload_consumer/filesystem_verifier_action_unittest.cc
index 91a7da4..9eeecb1 100644
--- a/payload_consumer/filesystem_verifier_action_unittest.cc
+++ b/payload_consumer/filesystem_verifier_action_unittest.cc
@@ -16,32 +16,23 @@
#include "update_engine/payload_consumer/filesystem_verifier_action.h"
-#include <fcntl.h>
-
#include <memory>
-#include <set>
#include <string>
#include <utility>
-#include <vector>
#include <base/bind.h>
#include <base/posix/eintr_wrapper.h>
-#include <base/strings/string_util.h>
-#include <base/strings/stringprintf.h>
#include <brillo/message_loops/fake_message_loop.h>
#include <brillo/message_loops/message_loop_utils.h>
-#include <gmock/gmock.h>
+#include <brillo/secure_blob.h>
#include <gtest/gtest.h>
#include "update_engine/common/hash_calculator.h"
#include "update_engine/common/test_utils.h"
#include "update_engine/common/utils.h"
-#include "update_engine/payload_consumer/payload_constants.h"
using brillo::MessageLoop;
-using std::set;
using std::string;
-using std::vector;
namespace chromeos_update_engine {
@@ -59,6 +50,7 @@
bool DoTest(bool terminate_early, bool hash_fail);
brillo::FakeMessageLoop loop_{nullptr};
+ ActionProcessor processor_;
};
class FilesystemVerifierActionTestDelegate : public ActionProcessorDelegate {
@@ -153,12 +145,11 @@
BondActions(feeder_action.get(), copier_action.get());
BondActions(copier_action.get(), collector_action.get());
- ActionProcessor processor;
FilesystemVerifierActionTestDelegate delegate;
- processor.set_delegate(&delegate);
- processor.EnqueueAction(std::move(feeder_action));
- processor.EnqueueAction(std::move(copier_action));
- processor.EnqueueAction(std::move(collector_action));
+ processor_.set_delegate(&delegate);
+ processor_.EnqueueAction(std::move(feeder_action));
+ processor_.EnqueueAction(std::move(copier_action));
+ processor_.EnqueueAction(std::move(collector_action));
loop_.PostTask(FROM_HERE,
base::Bind(
@@ -168,7 +159,7 @@
processor->StopProcessing();
}
},
- base::Unretained(&processor),
+ base::Unretained(&processor_),
terminate_early));
loop_.Run();
@@ -219,10 +210,9 @@
};
TEST_F(FilesystemVerifierActionTest, MissingInputObjectTest) {
- ActionProcessor processor;
FilesystemVerifierActionTest2Delegate delegate;
- processor.set_delegate(&delegate);
+ processor_.set_delegate(&delegate);
auto copier_action = std::make_unique<FilesystemVerifierAction>();
auto collector_action =
@@ -230,19 +220,18 @@
BondActions(copier_action.get(), collector_action.get());
- processor.EnqueueAction(std::move(copier_action));
- processor.EnqueueAction(std::move(collector_action));
- processor.StartProcessing();
- EXPECT_FALSE(processor.IsRunning());
+ processor_.EnqueueAction(std::move(copier_action));
+ processor_.EnqueueAction(std::move(collector_action));
+ processor_.StartProcessing();
+ EXPECT_FALSE(processor_.IsRunning());
EXPECT_TRUE(delegate.ran_);
EXPECT_EQ(ErrorCode::kError, delegate.code_);
}
TEST_F(FilesystemVerifierActionTest, NonExistentDriveTest) {
- ActionProcessor processor;
FilesystemVerifierActionTest2Delegate delegate;
- processor.set_delegate(&delegate);
+ processor_.set_delegate(&delegate);
InstallPlan install_plan;
InstallPlan::Partition part;
@@ -258,15 +247,16 @@
feeder_action->set_obj(install_plan);
+ BondActions(feeder_action.get(), verifier_action.get());
BondActions(verifier_action.get(), collector_action.get());
- processor.EnqueueAction(std::move(feeder_action));
- processor.EnqueueAction(std::move(verifier_action));
- processor.EnqueueAction(std::move(collector_action));
- processor.StartProcessing();
- EXPECT_FALSE(processor.IsRunning());
+ processor_.EnqueueAction(std::move(feeder_action));
+ processor_.EnqueueAction(std::move(verifier_action));
+ processor_.EnqueueAction(std::move(collector_action));
+ processor_.StartProcessing();
+ EXPECT_FALSE(processor_.IsRunning());
EXPECT_TRUE(delegate.ran_);
- EXPECT_EQ(ErrorCode::kError, delegate.code_);
+ EXPECT_EQ(ErrorCode::kFilesystemVerifierError, delegate.code_);
}
TEST_F(FilesystemVerifierActionTest, RunAsRootVerifyHashTest) {
@@ -286,4 +276,74 @@
while (loop_.RunOnce(false)) {}
}
+#ifdef __ANDROID__
+TEST_F(FilesystemVerifierActionTest, VerityHashTreeTest) {
+ FilesystemVerifierActionTestDelegate delegate;
+ processor_.set_delegate(&delegate);
+
+ test_utils::ScopedTempFile part_file("part_file.XXXXXX");
+ constexpr size_t filesystem_size = 200 * 4096;
+ constexpr size_t part_size = 256 * 4096;
+ brillo::Blob part_data(filesystem_size, 0x1);
+ part_data.resize(part_size);
+ ASSERT_TRUE(test_utils::WriteFileVector(part_file.path(), part_data));
+ string target_path;
+ test_utils::ScopedLoopbackDeviceBinder target_device(
+ part_file.path(), true, &target_path);
+
+ InstallPlan install_plan;
+ InstallPlan::Partition part;
+ part.name = "part";
+ part.target_path = target_path;
+ part.target_size = part_size;
+ part.block_size = 4096;
+ part.hash_tree_algorithm = "sha1";
+ part.hash_tree_data_offset = 0;
+ part.hash_tree_data_size = filesystem_size;
+ part.hash_tree_offset = filesystem_size;
+ part.hash_tree_size = 3 * 4096;
+ // for i in {1..$((200 * 4096))}; do echo -n -e '\x1' >> part; done
+ // avbtool add_hashtree_footer --image part --partition_size $((256 * 4096))
+ // --partition_name part --do_not_generate_fec
+ // --do_not_append_vbmeta_image --output_vbmeta_image vbmeta
+ // truncate -s $((256 * 4096)) part
+ // sha256sum part | xxd -r -p | hexdump -v -e '/1 "0x%02x, "'
+ part.target_hash = {0xf0, 0x2c, 0x81, 0xf5, 0xec, 0x30, 0xa6, 0x99,
+ 0x1b, 0x41, 0x72, 0x16, 0x38, 0x48, 0xe5, 0x68,
+ 0x06, 0x7c, 0x3b, 0x88, 0xb5, 0x97, 0xa9, 0x29,
+ 0xa5, 0x7d, 0xdd, 0xa5, 0x9f, 0x5c, 0x15, 0x84};
+ // avbtool info_image --image vbmeta | grep Salt | cut -d':' -f 2 |
+ // xxd -r -p | hexdump -v -e '/1 "0x%02x, "'
+ part.hash_tree_salt = {0x9e, 0xcb, 0xf8, 0xd5, 0x0b, 0xb4, 0x43,
+ 0x0a, 0x7a, 0x10, 0xad, 0x96, 0xd7, 0x15,
+ 0x70, 0xba, 0xed, 0x27, 0xe2, 0xae};
+ install_plan.partitions = {part};
+
+ auto feeder_action = std::make_unique<ObjectFeederAction<InstallPlan>>();
+ auto verifier_action = std::make_unique<FilesystemVerifierAction>();
+ auto collector_action =
+ std::make_unique<ObjectCollectorAction<InstallPlan>>();
+
+ feeder_action->set_obj(install_plan);
+
+ BondActions(feeder_action.get(), verifier_action.get());
+ BondActions(verifier_action.get(), collector_action.get());
+
+ processor_.EnqueueAction(std::move(feeder_action));
+ processor_.EnqueueAction(std::move(verifier_action));
+ processor_.EnqueueAction(std::move(collector_action));
+
+ loop_.PostTask(
+ FROM_HERE,
+ base::Bind(
+ [](ActionProcessor* processor) { processor->StartProcessing(); },
+ base::Unretained(&processor_)));
+ loop_.Run();
+
+ EXPECT_FALSE(processor_.IsRunning());
+ EXPECT_TRUE(delegate.ran());
+ EXPECT_EQ(ErrorCode::kSuccess, delegate.code());
+}
+#endif // __ANDROID__
+
} // namespace chromeos_update_engine
diff --git a/payload_consumer/install_plan.h b/payload_consumer/install_plan.h
index 929cad3..85c97b5 100644
--- a/payload_consumer/install_plan.h
+++ b/payload_consumer/install_plan.h
@@ -101,6 +101,7 @@
std::string target_path;
uint64_t target_size{0};
brillo::Blob target_hash;
+ uint32_t block_size{0};
// Whether we should run the postinstall script from this partition and the
// postinstall parameters.
@@ -108,6 +109,21 @@
std::string postinstall_path;
std::string filesystem_type;
bool postinstall_optional{false};
+
+ // Verity hash tree and FEC config. See update_metadata.proto for details.
+ // All offsets and sizes are in bytes.
+ uint64_t hash_tree_data_offset{0};
+ uint64_t hash_tree_data_size{0};
+ uint64_t hash_tree_offset{0};
+ uint64_t hash_tree_size{0};
+ std::string hash_tree_algorithm;
+ brillo::Blob hash_tree_salt;
+
+ uint64_t fec_data_offset{0};
+ uint64_t fec_data_size{0};
+ uint64_t fec_offset{0};
+ uint64_t fec_size{0};
+ uint32_t fec_roots{0};
};
std::vector<Partition> partitions;
diff --git a/payload_consumer/verity_writer_android.cc b/payload_consumer/verity_writer_android.cc
new file mode 100644
index 0000000..b374817
--- /dev/null
+++ b/payload_consumer/verity_writer_android.cc
@@ -0,0 +1,100 @@
+//
+// Copyright (C) 2018 The Android Open Source Project
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+//
+
+#include "update_engine/payload_consumer/verity_writer_android.h"
+
+#include <fcntl.h>
+
+#include <algorithm>
+#include <memory>
+
+#include <base/logging.h>
+#include <base/posix/eintr_wrapper.h>
+
+#include "update_engine/common/utils.h"
+
+namespace chromeos_update_engine {
+
+namespace verity_writer {
+std::unique_ptr<VerityWriterInterface> CreateVerityWriter() {
+ return std::make_unique<VerityWriterAndroid>();
+}
+} // namespace verity_writer
+
+bool VerityWriterAndroid::Init(const InstallPlan::Partition& partition) {
+ partition_ = &partition;
+
+ if (partition_->hash_tree_size != 0) {
+ auto hash_function =
+ HashTreeBuilder::HashFunction(partition_->hash_tree_algorithm);
+ if (hash_function == nullptr) {
+ LOG(ERROR) << "Verity hash algorithm not supported: "
+ << partition_->hash_tree_algorithm;
+ return false;
+ }
+ hash_tree_builder_ = std::make_unique<HashTreeBuilder>(
+ partition_->block_size, hash_function);
+ TEST_AND_RETURN_FALSE(hash_tree_builder_->Initialize(
+ partition_->hash_tree_data_size, partition_->hash_tree_salt));
+ if (hash_tree_builder_->CalculateSize(partition_->hash_tree_data_size) !=
+ partition_->hash_tree_size) {
+ LOG(ERROR) << "Verity hash tree size does not match, stored: "
+ << partition_->hash_tree_size << ", calculated: "
+ << hash_tree_builder_->CalculateSize(
+ partition_->hash_tree_data_size);
+ return false;
+ }
+ }
+ return true;
+}
+
+bool VerityWriterAndroid::Update(uint64_t offset,
+ const uint8_t* buffer,
+ size_t size) {
+ if (partition_->hash_tree_size != 0) {
+ uint64_t hash_tree_data_end =
+ partition_->hash_tree_data_offset + partition_->hash_tree_data_size;
+ uint64_t start_offset = std::max(offset, partition_->hash_tree_data_offset);
+ uint64_t end_offset = std::min(offset + size, hash_tree_data_end);
+ if (start_offset < end_offset) {
+ TEST_AND_RETURN_FALSE(hash_tree_builder_->Update(
+ buffer + start_offset - offset, end_offset - start_offset));
+
+ if (end_offset == hash_tree_data_end) {
+ // All hash tree data blocks has been hashed, write hash tree to disk.
+ int fd = HANDLE_EINTR(open(partition_->target_path.c_str(), O_WRONLY));
+ if (fd < 0) {
+ PLOG(ERROR) << "Failed to open " << partition_->target_path
+ << " to write verity data.";
+ return false;
+ }
+ ScopedFdCloser fd_closer(&fd);
+
+ LOG(INFO) << "Writing verity hash tree to " << partition_->target_path;
+ TEST_AND_RETURN_FALSE(hash_tree_builder_->BuildHashTree());
+ TEST_AND_RETURN_FALSE(hash_tree_builder_->WriteHashTreeToFd(
+ fd, partition_->hash_tree_offset));
+ hash_tree_builder_.reset();
+ }
+ }
+ }
+ if (partition_->fec_size != 0) {
+ // TODO(senj): Update FEC data.
+ }
+ return true;
+}
+
+} // namespace chromeos_update_engine
diff --git a/payload_consumer/verity_writer_android.h b/payload_consumer/verity_writer_android.h
new file mode 100644
index 0000000..c26663b
--- /dev/null
+++ b/payload_consumer/verity_writer_android.h
@@ -0,0 +1,46 @@
+//
+// Copyright (C) 2018 The Android Open Source Project
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+//
+
+#ifndef UPDATE_ENGINE_PAYLOAD_CONSUMER_VERITY_WRITER_ANDROID_H_
+#define UPDATE_ENGINE_PAYLOAD_CONSUMER_VERITY_WRITER_ANDROID_H_
+
+#include <memory>
+
+#include <verity/hash_tree_builder.h>
+
+#include "update_engine/payload_consumer/verity_writer_interface.h"
+
+namespace chromeos_update_engine {
+
+class VerityWriterAndroid : public VerityWriterInterface {
+ public:
+ VerityWriterAndroid() = default;
+ ~VerityWriterAndroid() override = default;
+
+ bool Init(const InstallPlan::Partition& partition) override;
+ bool Update(uint64_t offset, const uint8_t* buffer, size_t size) override;
+
+ private:
+ const InstallPlan::Partition* partition_ = nullptr;
+
+ std::unique_ptr<HashTreeBuilder> hash_tree_builder_;
+
+ DISALLOW_COPY_AND_ASSIGN(VerityWriterAndroid);
+};
+
+} // namespace chromeos_update_engine
+
+#endif // UPDATE_ENGINE_PAYLOAD_CONSUMER_VERITY_WRITER_ANDROID_H_
diff --git a/payload_consumer/verity_writer_android_unittest.cc b/payload_consumer/verity_writer_android_unittest.cc
new file mode 100644
index 0000000..56335c1
--- /dev/null
+++ b/payload_consumer/verity_writer_android_unittest.cc
@@ -0,0 +1,100 @@
+//
+// Copyright (C) 2018 The Android Open Source Project
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+//
+
+#include "update_engine/payload_consumer/verity_writer_android.h"
+
+#include <brillo/secure_blob.h>
+#include <gtest/gtest.h>
+
+#include "update_engine/common/test_utils.h"
+#include "update_engine/common/utils.h"
+
+namespace chromeos_update_engine {
+
+class VerityWriterAndroidTest : public ::testing::Test {
+ protected:
+ void SetUp() override {
+ partition_.target_path = temp_file_.path();
+ partition_.block_size = 4096;
+ partition_.hash_tree_data_offset = 0;
+ partition_.hash_tree_data_size = 4096;
+ partition_.hash_tree_offset = 4096;
+ partition_.hash_tree_size = 4096;
+ partition_.hash_tree_algorithm = "sha1";
+ }
+
+ VerityWriterAndroid verity_writer_;
+ InstallPlan::Partition partition_;
+ test_utils::ScopedTempFile temp_file_;
+};
+
+TEST_F(VerityWriterAndroidTest, SimpleTest) {
+ brillo::Blob part_data(8192);
+ test_utils::WriteFileVector(partition_.target_path, part_data);
+ ASSERT_TRUE(verity_writer_.Init(partition_));
+ EXPECT_TRUE(verity_writer_.Update(0, part_data.data(), 4096));
+ EXPECT_TRUE(verity_writer_.Update(4096, part_data.data() + 4096, 4096));
+ brillo::Blob actual_part;
+ utils::ReadFile(partition_.target_path, &actual_part);
+ // dd if=/dev/zero bs=4096 count=1 2>/dev/null | sha1sum | xxd -r -p |
+ // hexdump -v -e '/1 "0x%02x, "'
+ brillo::Blob hash = {0x1c, 0xea, 0xf7, 0x3d, 0xf4, 0x0e, 0x53,
+ 0x1d, 0xf3, 0xbf, 0xb2, 0x6b, 0x4f, 0xb7,
+ 0xcd, 0x95, 0xfb, 0x7b, 0xff, 0x1d};
+ memcpy(part_data.data() + 4096, hash.data(), hash.size());
+ EXPECT_EQ(part_data, actual_part);
+}
+
+TEST_F(VerityWriterAndroidTest, NoOpTest) {
+ partition_.hash_tree_data_size = 0;
+ partition_.hash_tree_size = 0;
+ brillo::Blob part_data(4096);
+ ASSERT_TRUE(verity_writer_.Init(partition_));
+ EXPECT_TRUE(verity_writer_.Update(0, part_data.data(), part_data.size()));
+ EXPECT_TRUE(verity_writer_.Update(4096, part_data.data(), part_data.size()));
+ EXPECT_TRUE(verity_writer_.Update(8192, part_data.data(), part_data.size()));
+}
+
+TEST_F(VerityWriterAndroidTest, InvalidHashAlgorithmTest) {
+ partition_.hash_tree_algorithm = "sha123";
+ EXPECT_FALSE(verity_writer_.Init(partition_));
+}
+
+TEST_F(VerityWriterAndroidTest, WrongHashTreeSizeTest) {
+ partition_.hash_tree_size = 8192;
+ EXPECT_FALSE(verity_writer_.Init(partition_));
+}
+
+TEST_F(VerityWriterAndroidTest, SHA256Test) {
+ partition_.hash_tree_algorithm = "sha256";
+ brillo::Blob part_data(8192);
+ test_utils::WriteFileVector(partition_.target_path, part_data);
+ ASSERT_TRUE(verity_writer_.Init(partition_));
+ EXPECT_TRUE(verity_writer_.Update(0, part_data.data(), 4096));
+ EXPECT_TRUE(verity_writer_.Update(4096, part_data.data() + 4096, 4096));
+ brillo::Blob actual_part;
+ utils::ReadFile(partition_.target_path, &actual_part);
+ // dd if=/dev/zero bs=4096 count=1 2>/dev/null | sha256sum | xxd -r -p |
+ // hexdump -v -e '/1 "0x%02x, "'
+ brillo::Blob hash = {0xad, 0x7f, 0xac, 0xb2, 0x58, 0x6f, 0xc6, 0xe9,
+ 0x66, 0xc0, 0x04, 0xd7, 0xd1, 0xd1, 0x6b, 0x02,
+ 0x4f, 0x58, 0x05, 0xff, 0x7c, 0xb4, 0x7c, 0x7a,
+ 0x85, 0xda, 0xbd, 0x8b, 0x48, 0x89, 0x2c, 0xa7};
+ memcpy(part_data.data() + 4096, hash.data(), hash.size());
+ EXPECT_EQ(part_data, actual_part);
+}
+
+} // namespace chromeos_update_engine
diff --git a/payload_consumer/verity_writer_interface.h b/payload_consumer/verity_writer_interface.h
new file mode 100644
index 0000000..a3ecef3
--- /dev/null
+++ b/payload_consumer/verity_writer_interface.h
@@ -0,0 +1,53 @@
+//
+// Copyright (C) 2018 The Android Open Source Project
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+//
+
+#ifndef UPDATE_ENGINE_PAYLOAD_CONSUMER_VERITY_WRITER_INTERFACE_H_
+#define UPDATE_ENGINE_PAYLOAD_CONSUMER_VERITY_WRITER_INTERFACE_H_
+
+#include <cstdint>
+#include <memory>
+
+#include <base/macros.h>
+
+#include "update_engine/payload_consumer/install_plan.h"
+
+namespace chromeos_update_engine {
+
+class VerityWriterInterface {
+ public:
+ virtual ~VerityWriterInterface() = default;
+
+ virtual bool Init(const InstallPlan::Partition& partition) = 0;
+ // Update partition data at [offset : offset + size) stored in |buffer|.
+ // Data not in |hash_tree_data_extent| or |fec_data_extent| is ignored.
+ // Will write verity data to the target partition once all the necessary
+ // blocks has passed.
+ virtual bool Update(uint64_t offset, const uint8_t* buffer, size_t size) = 0;
+
+ protected:
+ VerityWriterInterface() = default;
+
+ private:
+ DISALLOW_COPY_AND_ASSIGN(VerityWriterInterface);
+};
+
+namespace verity_writer {
+std::unique_ptr<VerityWriterInterface> CreateVerityWriter();
+}
+
+} // namespace chromeos_update_engine
+
+#endif // UPDATE_ENGINE_PAYLOAD_CONSUMER_VERITY_WRITER_INTERFACE_H_
diff --git a/payload_consumer/verity_writer_stub.cc b/payload_consumer/verity_writer_stub.cc
new file mode 100644
index 0000000..a0e2467
--- /dev/null
+++ b/payload_consumer/verity_writer_stub.cc
@@ -0,0 +1,39 @@
+//
+// Copyright (C) 2018 The Android Open Source Project
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+//
+
+#include "update_engine/payload_consumer/verity_writer_stub.h"
+
+#include <memory>
+
+namespace chromeos_update_engine {
+
+namespace verity_writer {
+std::unique_ptr<VerityWriterInterface> CreateVerityWriter() {
+ return std::make_unique<VerityWriterStub>();
+}
+} // namespace verity_writer
+
+bool VerityWriterStub::Init(const InstallPlan::Partition& partition) {
+ return partition.hash_tree_size == 0 && partition.fec_size == 0;
+}
+
+bool VerityWriterStub::Update(uint64_t offset,
+ const uint8_t* buffer,
+ size_t size) {
+ return true;
+}
+
+} // namespace chromeos_update_engine
diff --git a/payload_consumer/verity_writer_stub.h b/payload_consumer/verity_writer_stub.h
new file mode 100644
index 0000000..ea5e574
--- /dev/null
+++ b/payload_consumer/verity_writer_stub.h
@@ -0,0 +1,38 @@
+//
+// Copyright (C) 2018 The Android Open Source Project
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+//
+
+#ifndef UPDATE_ENGINE_PAYLOAD_CONSUMER_VERITY_WRITER_STUB_H_
+#define UPDATE_ENGINE_PAYLOAD_CONSUMER_VERITY_WRITER_STUB_H_
+
+#include "update_engine/payload_consumer/verity_writer_interface.h"
+
+namespace chromeos_update_engine {
+
+class VerityWriterStub : public VerityWriterInterface {
+ public:
+ VerityWriterStub() = default;
+ ~VerityWriterStub() override = default;
+
+ bool Init(const InstallPlan::Partition& partition) override;
+ bool Update(uint64_t offset, const uint8_t* buffer, size_t size) override;
+
+ private:
+ DISALLOW_COPY_AND_ASSIGN(VerityWriterStub);
+};
+
+} // namespace chromeos_update_engine
+
+#endif // UPDATE_ENGINE_PAYLOAD_CONSUMER_VERITY_WRITER_STUB_H_
diff --git a/payload_state.cc b/payload_state.cc
index c4eb950..36f120a 100644
--- a/payload_state.cc
+++ b/payload_state.cc
@@ -305,6 +305,7 @@
case ErrorCode::kUnsupportedMajorPayloadVersion:
case ErrorCode::kUnsupportedMinorPayloadVersion:
case ErrorCode::kPayloadTimestampError:
+ case ErrorCode::kVerityCalculationError:
IncrementUrlIndex();
break;
diff --git a/update_engine.gyp b/update_engine.gyp
index ee2471a..45b0a84 100644
--- a/update_engine.gyp
+++ b/update_engine.gyp
@@ -187,6 +187,7 @@
'payload_consumer/payload_metadata.cc',
'payload_consumer/payload_verifier.cc',
'payload_consumer/postinstall_runner_action.cc',
+ 'payload_consumer/verity_writer_stub.cc',
'payload_consumer/xz_extent_writer.cc',
],
'conditions': [
diff --git a/update_manager/chromeos_policy.cc b/update_manager/chromeos_policy.cc
index 71fec40..587ac67 100644
--- a/update_manager/chromeos_policy.cc
+++ b/update_manager/chromeos_policy.cc
@@ -87,6 +87,7 @@
case ErrorCode::kUnsupportedMajorPayloadVersion:
case ErrorCode::kUnsupportedMinorPayloadVersion:
case ErrorCode::kPayloadTimestampError:
+ case ErrorCode::kVerityCalculationError:
LOG(INFO) << "Advancing download URL due to error "
<< chromeos_update_engine::utils::ErrorCodeToString(err_code)
<< " (" << static_cast<int>(err_code) << ")";
diff --git a/update_metadata.proto b/update_metadata.proto
index a0f278b..f90ec3c 100644
--- a/update_metadata.proto
+++ b/update_metadata.proto
@@ -248,6 +248,29 @@
// Whether a failure in the postinstall step for this partition should be
// ignored.
optional bool postinstall_optional = 9;
+
+ // On minor version 6 or newer, these fields are supported:
+
+ // The extent for data covered by verity hash tree.
+ optional Extent hash_tree_data_extent = 10;
+
+ // The extent to store verity hash tree.
+ optional Extent hash_tree_extent = 11;
+
+ // The hash algorithm used in verity hash tree.
+ optional string hash_tree_algorithm = 12;
+
+ // The salt used for verity hash tree.
+ optional bytes hash_tree_salt = 13;
+
+ // The extent for data covered by FEC.
+ optional Extent fec_data_extent = 14;
+
+ // The extent to store FEC.
+ optional Extent fec_extent = 15;
+
+ // The number of FEC roots.
+ optional uint32 fec_roots = 16 [default = 2];
}
message DeltaArchiveManifest {