Emit XOR ops at OTA generation time

1. Add a new merge operation type: COW_XOR
2. When running bsdiff, extract similar blocks and record them in
annotated operations
3. During merge sequence generation, include COW_XOR ops as well

Test: treehugger
Bug: 177104308

Change-Id: If894736859b572e761e83be5a9498235caa68866
diff --git a/payload_consumer/install_operation_executor_unittest.cc b/payload_consumer/install_operation_executor_unittest.cc
index c90b28d..2a99782 100644
--- a/payload_consumer/install_operation_executor_unittest.cc
+++ b/payload_consumer/install_operation_executor_unittest.cc
@@ -49,20 +49,7 @@
   return out;
 }
 
-namespace {
-template <typename Container>
-size_t GetNthBlock(const Container& extents, const size_t n) {
-  size_t cur_block_count = 0;
-  for (const auto& extent : extents) {
-    if (cur_block_count + extent.num_blocks() >= n) {
-      return extent.start_block() + (n - cur_block_count);
-    }
-    cur_block_count += extent.num_blocks();
-  }
-  return std::numeric_limits<size_t>::max();
-}
-
-}  // namespace
+namespace {}  // namespace
 
 class InstallOperationExecutorTest : public ::testing::Test {
  public:
diff --git a/payload_generator/ab_generator.cc b/payload_generator/ab_generator.cc
index d9b9d88..25cafe3 100644
--- a/payload_generator/ab_generator.cc
+++ b/payload_generator/ab_generator.cc
@@ -54,7 +54,7 @@
                                                        new_part,
                                                        hard_chunk_blocks,
                                                        soft_chunk_blocks,
-                                                       config.version,
+                                                       config,
                                                        blob_file));
   LOG(INFO) << "done reading " << new_part.name;
 
diff --git a/payload_generator/annotated_operation.h b/payload_generator/annotated_operation.h
index 653bab0..c57f249 100644
--- a/payload_generator/annotated_operation.h
+++ b/payload_generator/annotated_operation.h
@@ -19,6 +19,7 @@
 
 #include <ostream>  // NOLINT(readability/streams)
 #include <string>
+#include <vector>
 
 #include <brillo/secure_blob.h>
 
@@ -35,6 +36,11 @@
   // The InstallOperation, as defined by the protobuf.
   InstallOperation op;
 
+  // Array of blocks which should be converted to XOR during OTA install.
+  // All elements in this array should have |merge_op.type() == COW_XOR|.
+  // This information is typically derived from BSDIFF patch data.
+  std::vector<CowMergeOperation> xor_ops;
+
   // Writes |blob| to the end of |blob_file|. It sets the data_offset and
   // data_length in AnnotatedOperation to match the offset and size of |blob|
   // in |blob_file|.
diff --git a/payload_generator/delta_diff_utils.cc b/payload_generator/delta_diff_utils.cc
index 3c025e1..077fbfe 100644
--- a/payload_generator/delta_diff_utils.cc
+++ b/payload_generator/delta_diff_utils.cc
@@ -35,15 +35,17 @@
 #include <memory>
 #include <numeric>
 #include <utility>
+#include <vector>
 
 #include <base/files/file_util.h>
 #include <base/format_macros.h>
 #include <base/strings/string_util.h>
 #include <base/strings/stringprintf.h>
 #include <base/threading/simple_thread.h>
-#include <base/time/time.h>
 #include <brillo/data_encoding.h>
 #include <bsdiff/bsdiff.h>
+#include <bsdiff/control_entry.h>
+#include <bsdiff/patch_reader.h>
 #include <bsdiff/patch_writer_factory.h>
 #include <puffin/utils.h>
 
@@ -131,6 +133,44 @@
   }
   return distances.back();
 }
+
+static bool ShouldCreateNewOp(const std::vector<CowMergeOperation>& ops,
+                              size_t src_block,
+                              size_t dst_block,
+                              size_t src_offset) {
+  if (ops.empty()) {
+    return true;
+  }
+  const auto& op = ops.back();
+  if (op.src_offset() != src_offset) {
+    return true;
+  }
+  const auto& src_extent = op.src_extent();
+  const auto& dst_extent = op.dst_extent();
+  return src_extent.start_block() + src_extent.num_blocks() != src_block ||
+         dst_extent.start_block() + dst_extent.num_blocks() != dst_block;
+}
+
+static void AppendXorBlock(std::vector<CowMergeOperation>* ops,
+                           size_t src_block,
+                           size_t dst_block,
+                           size_t src_offset) {
+  if (ShouldCreateNewOp(*ops, src_block, dst_block, src_offset)) {
+    auto& op = ops->emplace_back();
+    op.mutable_src_extent()->set_start_block(src_block);
+    op.mutable_src_extent()->set_num_blocks(1);
+    op.mutable_dst_extent()->set_start_block(dst_block);
+    op.mutable_dst_extent()->set_num_blocks(1);
+    op.set_src_offset(src_offset);
+  } else {
+    auto& op = ops->back();
+    auto& src_extent = *op.mutable_src_extent();
+    auto& dst_extent = *op.mutable_dst_extent();
+    src_extent.set_num_blocks(src_extent.num_blocks() + 1);
+    dst_extent.set_num_blocks(dst_extent.num_blocks() + 1);
+  }
+}
+
 }  // namespace
 
 namespace diff_utils {
@@ -142,7 +182,7 @@
  public:
   FileDeltaProcessor(const string& old_part,
                      const string& new_part,
-                     const PayloadVersion& version,
+                     const PayloadGenerationConfig& config,
                      const vector<Extent>& old_extents,
                      const vector<Extent>& new_extents,
                      const vector<puffin::BitExtent>& old_deflates,
@@ -152,7 +192,7 @@
                      BlobFileWriter* blob_file)
       : old_part_(old_part),
         new_part_(new_part),
-        version_(version),
+        config_(config),
         old_extents_(old_extents),
         new_extents_(new_extents),
         new_extents_blocks_(utils::BlocksInExtents(new_extents)),
@@ -179,7 +219,7 @@
  private:
   const string& old_part_;  // NOLINT(runtime/member_string_references)
   const string& new_part_;  // NOLINT(runtime/member_string_references)
-  const PayloadVersion& version_;
+  const PayloadGenerationConfig& config_;
 
   // The block ranges of the old/new file within the src/tgt image
   const vector<Extent> old_extents_;
@@ -213,7 +253,7 @@
                      new_deflates_,
                      name_,
                      chunk_blocks_,
-                     version_,
+                     config_,
                      blob_file_)) {
     LOG(ERROR) << "Failed to generate delta for " << name_ << " ("
                << new_extents_blocks_ << " blocks)";
@@ -222,7 +262,7 @@
   }
 
   if (!ABGenerator::FragmentOperations(
-          version_, &file_aops_, new_part_, blob_file_)) {
+          config_.version, &file_aops_, new_part_, blob_file_)) {
     LOG(ERROR) << "Failed to fragment operations for " << name_;
     failed_ = true;
     return;
@@ -273,8 +313,9 @@
                         const PartitionConfig& new_part,
                         ssize_t hard_chunk_blocks,
                         size_t soft_chunk_blocks,
-                        const PayloadVersion& version,
+                        const PayloadGenerationConfig& config,
                         BlobFileWriter* blob_file) {
+  const auto& version = config.version;
   ExtentRanges old_visited_blocks;
   ExtentRanges new_visited_blocks;
 
@@ -297,7 +338,7 @@
                                                 old_part.size / kBlockSize,
                                                 new_part.size / kBlockSize,
                                                 soft_chunk_blocks,
-                                                version,
+                                                config,
                                                 blob_file,
                                                 &old_visited_blocks,
                                                 &new_visited_blocks,
@@ -355,7 +396,7 @@
 
     file_delta_processors.emplace_back(old_part.path,
                                        new_part.path,
-                                       version,
+                                       config,
                                        std::move(old_file_extents),
                                        std::move(new_file_extents),
                                        old_file.deflates,
@@ -385,7 +426,7 @@
     file_delta_processors.emplace_back(
         old_part.path,
         new_part.path,
-        version,
+        config,
         std::move(old_unvisited),
         std::move(new_unvisited),
         vector<puffin::BitExtent>{},  // old_deflates,
@@ -424,11 +465,12 @@
                              size_t old_num_blocks,
                              size_t new_num_blocks,
                              ssize_t chunk_blocks,
-                             const PayloadVersion& version,
+                             const PayloadGenerationConfig& config,
                              BlobFileWriter* blob_file,
                              ExtentRanges* old_visited_blocks,
                              ExtentRanges* new_visited_blocks,
                              ExtentRanges* old_zero_blocks) {
+  const auto& version = config.version;
   vector<BlockMapping::BlockId> old_block_ids;
   vector<BlockMapping::BlockId> new_block_ids;
   TEST_AND_RETURN_FALSE(MapPartitionBlocks(old_part,
@@ -516,7 +558,7 @@
                                           {},        // new_deflates
                                           "<zeros>",
                                           chunk_blocks,
-                                          version,
+                                          config,
                                           blob_file));
     }
   }
@@ -575,10 +617,9 @@
                    const vector<puffin::BitExtent>& new_deflates,
                    const string& name,
                    ssize_t chunk_blocks,
-                   const PayloadVersion& version,
+                   const PayloadGenerationConfig& config,
                    BlobFileWriter* blob_file) {
   brillo::Blob data;
-  InstallOperation operation;
 
   uint64_t total_blocks = utils::BlocksInExtents(new_extents);
   if (chunk_blocks == 0) {
@@ -602,30 +643,29 @@
     NormalizeExtents(&old_extents_chunk);
     NormalizeExtents(&new_extents_chunk);
 
+    // Now, insert into the list of operations.
+    AnnotatedOperation aop;
     TEST_AND_RETURN_FALSE(ReadExtentsToDiff(old_part,
                                             new_part,
                                             old_extents_chunk,
                                             new_extents_chunk,
                                             old_deflates,
                                             new_deflates,
-                                            version,
+                                            config,
                                             &data,
-                                            &operation));
+                                            &aop));
 
     // Check if the operation writes nothing.
-    if (operation.dst_extents_size() == 0) {
+    if (aop.op.dst_extents_size() == 0) {
       LOG(ERROR) << "Empty non-MOVE operation";
       return false;
     }
 
-    // Now, insert into the list of operations.
-    AnnotatedOperation aop;
     aop.name = name;
     if (static_cast<uint64_t>(chunk_blocks) < total_blocks) {
       aop.name = base::StringPrintf(
           "%s:%" PRIu64, name.c_str(), block_offset / chunk_blocks);
     }
-    aop.op = operation;
 
     // Write the data
     TEST_AND_RETURN_FALSE(aop.SetOperationBlob(data, blob_file));
@@ -688,16 +728,84 @@
   return true;
 }
 
+// Decide which blocks are similar from bsdiff patch.
+// Blocks included in out_op->xor_map will be converted to COW_XOR during OTA
+// installation
+bool PopulateXorOps(AnnotatedOperation* aop, const uint8_t* data, size_t size) {
+  bsdiff::BsdiffPatchReader patch_reader;
+  TEST_AND_RETURN_FALSE(patch_reader.Init(data, size));
+  ControlEntry entry;
+  size_t new_off = 0;
+  int64_t old_off = 0;
+  auto& xor_ops = aop->xor_ops;
+  size_t total_xor_blocks = 0;
+  const auto new_file_size =
+      utils::BlocksInExtents(aop->op.dst_extents()) * kBlockSize;
+  while (new_off < new_file_size) {
+    if (!patch_reader.ParseControlEntry(&entry)) {
+      LOG(ERROR)
+          << "Exhausted bsdiff patch data before reaching end of new file. "
+             "Current position: "
+          << new_off << " new file size: " << new_file_size;
+      return false;
+    }
+    if (old_off >= 0) {
+      auto dst_off_aligned = utils::RoundUp(new_off, kBlockSize);
+      const auto skip = dst_off_aligned - new_off;
+      auto src_off = old_off + skip;
+      const size_t chunk_size =
+          entry.diff_size - std::min(skip, entry.diff_size);
+      const auto xor_blocks = (chunk_size + kBlockSize / 2) / kBlockSize;
+      total_xor_blocks += xor_blocks;
+      // Append chunk_size/kBlockSize number of XOR blocks, subject to rounding
+      // rules: if decimal part of that division is >= 0.5, round up.
+      for (size_t i = 0; i < xor_blocks; i++) {
+        AppendXorBlock(
+            &xor_ops,
+            GetNthBlock(aop->op.src_extents(), src_off / kBlockSize),
+            GetNthBlock(aop->op.dst_extents(), dst_off_aligned / kBlockSize),
+            src_off % kBlockSize);
+        src_off += kBlockSize;
+        dst_off_aligned += kBlockSize;
+      }
+    }
+
+    old_off += entry.diff_size + entry.offset_increment;
+    new_off += entry.diff_size + entry.extra_size;
+  }
+
+  for (auto& op : xor_ops) {
+    CHECK_EQ(op.src_extent().num_blocks(), op.dst_extent().num_blocks());
+    // If |src_offset| is greater than 0, then we are reading 1
+    // extra block at the end of src_extent. This dependency must
+    // be honored during merge sequence generation, or we can end
+    // up with a corrupted device after merge.
+    if (op.src_offset() > 0) {
+      op.mutable_src_extent()->set_num_blocks(op.dst_extent().num_blocks() + 1);
+    }
+  }
+
+  if (xor_ops.size() > 0) {
+    // TODO(177104308) Filter out duplicate blocks in XOR op
+    LOG(INFO) << "Added " << total_xor_blocks << " XOR blocks, "
+              << total_xor_blocks * 100.0f / new_off * kBlockSize
+              << "% of blocks in this InstallOp are XOR";
+  }
+  return true;
+}
+
 bool ReadExtentsToDiff(const string& old_part,
                        const string& new_part,
                        const vector<Extent>& old_extents,
                        const vector<Extent>& new_extents,
                        const vector<puffin::BitExtent>& old_deflates,
                        const vector<puffin::BitExtent>& new_deflates,
-                       const PayloadVersion& version,
+                       const PayloadGenerationConfig& config,
                        brillo::Blob* out_data,
-                       InstallOperation* out_op) {
-  InstallOperation operation;
+                       AnnotatedOperation* out_op) {
+  const auto& version = config.version;
+  AnnotatedOperation aop;
+  InstallOperation& operation = aop.op;
 
   // We read blocks from old_extents and write blocks to new_extents.
   uint64_t blocks_to_read = utils::BlocksInExtents(old_extents);
@@ -721,9 +829,10 @@
     puffdiff_allowed = false;
   }
 
-  // Make copies of the extents so we can modify them.
-  vector<Extent> src_extents = old_extents;
-  vector<Extent> dst_extents = new_extents;
+  const vector<Extent>& src_extents = old_extents;
+  const vector<Extent>& dst_extents = new_extents;
+  // All operations have dst_extents.
+  StoreExtents(dst_extents, operation.mutable_dst_extents());
 
   // Read in bytes from new data.
   brillo::Blob new_data;
@@ -786,11 +895,15 @@
                                                   nullptr));
 
         TEST_AND_RETURN_FALSE(utils::ReadFile(patch.value(), &bsdiff_delta));
+
         CHECK_GT(bsdiff_delta.size(), static_cast<brillo::Blob::size_type>(0));
         if (IsDiffOperationBetter(operation,
                                   data_blob.size(),
                                   bsdiff_delta.size(),
                                   src_extents.size())) {
+          if (config.enable_vabc_xor) {
+            PopulateXorOps(&aop, bsdiff_delta);
+          }
           operation.set_type(operation_type);
           data_blob = std::move(bsdiff_delta);
         }
@@ -860,11 +973,9 @@
   if (!IsNoSourceOperation(operation.type())) {
     StoreExtents(src_extents, operation.mutable_src_extents());
   }
-  // All operations have dst_extents.
-  StoreExtents(dst_extents, operation.mutable_dst_extents());
 
   *out_data = std::move(data_blob);
-  *out_op = operation;
+  *out_op = aop;
   return true;
 }
 
diff --git a/payload_generator/delta_diff_utils.h b/payload_generator/delta_diff_utils.h
index c75d16d..f284530 100644
--- a/payload_generator/delta_diff_utils.h
+++ b/payload_generator/delta_diff_utils.h
@@ -48,7 +48,7 @@
                         const PartitionConfig& new_part,
                         ssize_t hard_chunk_blocks,
                         size_t soft_chunk_blocks,
-                        const PayloadVersion& version,
+                        const PayloadGenerationConfig& version,
                         BlobFileWriter* blob_file);
 
 // Create operations in |aops| for identical blocks that moved around in the old
@@ -67,7 +67,7 @@
                              size_t old_num_blocks,
                              size_t new_num_blocks,
                              ssize_t chunk_blocks,
-                             const PayloadVersion& version,
+                             const PayloadGenerationConfig& version,
                              BlobFileWriter* blob_file,
                              ExtentRanges* old_visited_blocks,
                              ExtentRanges* new_visited_blocks,
@@ -90,7 +90,7 @@
                    const std::vector<puffin::BitExtent>& new_deflates,
                    const std::string& name,
                    ssize_t chunk_blocks,
-                   const PayloadVersion& version,
+                   const PayloadGenerationConfig& config,
                    BlobFileWriter* blob_file);
 
 // Reads the blocks |old_extents| from |old_part| (if it exists) and the
@@ -108,9 +108,9 @@
                        const std::vector<Extent>& new_extents,
                        const std::vector<puffin::BitExtent>& old_deflates,
                        const std::vector<puffin::BitExtent>& new_deflates,
-                       const PayloadVersion& version,
+                       const PayloadGenerationConfig& version,
                        brillo::Blob* out_data,
-                       InstallOperation* out_op);
+                       AnnotatedOperation* out_op);
 
 // Generates the best allowed full operation to produce |new_data|. The allowed
 // operations are based on |payload_version|. The operation blob will be stored
@@ -149,6 +149,15 @@
     const std::map<std::string, FilesystemInterface::File>& old_files_map,
     const std::string& new_file_name);
 
+// Read BSDIFF patch data in |data|, compute list of blocks that can be COW_XOR,
+// store these blocks in |aop|.
+bool PopulateXorOps(AnnotatedOperation* aop, const uint8_t* data, size_t size);
+
+inline bool PopulateXorOps(AnnotatedOperation* aop,
+                           const brillo::Blob& patch_data) {
+  return PopulateXorOps(aop, patch_data.data(), patch_data.size());
+}
+
 }  // namespace diff_utils
 
 }  // namespace chromeos_update_engine
diff --git a/payload_generator/delta_diff_utils_unittest.cc b/payload_generator/delta_diff_utils_unittest.cc
index c652357..76d7624 100644
--- a/payload_generator/delta_diff_utils_unittest.cc
+++ b/payload_generator/delta_diff_utils_unittest.cc
@@ -133,7 +133,7 @@
                                                old_part_.size / block_size_,
                                                new_part_.size / block_size_,
                                                chunk_blocks,
-                                               version,
+                                               {.version = version},
                                                &blob_file,
                                                &old_visited_blocks_,
                                                &new_visited_blocks_,
@@ -170,8 +170,8 @@
       new_part_,
       -1,
       -1,
-      PayloadVersion(kMaxSupportedMajorPayloadVersion,
-                     kVerityMinorPayloadVersion),
+      {.version = PayloadVersion(kMaxSupportedMajorPayloadVersion,
+                                 kVerityMinorPayloadVersion)},
       &blob_file));
   for (const auto& aop : aops_) {
     new_visited_blocks_.AddRepeatedExtents(aop.op.dst_extents());
@@ -207,7 +207,8 @@
         WriteExtents(new_part_.path, new_extents, kBlockSize, data_to_test));
 
     brillo::Blob data;
-    InstallOperation op;
+    AnnotatedOperation aop;
+    InstallOperation& op = aop.op;
     ASSERT_TRUE(diff_utils::ReadExtentsToDiff(
         old_part_.path,
         new_part_.path,
@@ -215,9 +216,10 @@
         new_extents,
         {},  // old_deflates
         {},  // new_deflates
-        PayloadVersion(kBrilloMajorPayloadVersion, kSourceMinorPayloadVersion),
+        {.version = PayloadVersion(kBrilloMajorPayloadVersion,
+                                   kSourceMinorPayloadVersion)},
         &data,
-        &op));
+        &aop));
     ASSERT_FALSE(data.empty());
 
     ASSERT_TRUE(op.has_type());
@@ -249,7 +251,7 @@
   ASSERT_TRUE(WriteExtents(new_part_.path, new_extents, kBlockSize, data_blob));
 
   brillo::Blob data;
-  InstallOperation op;
+  AnnotatedOperation aop;
   ASSERT_TRUE(diff_utils::ReadExtentsToDiff(
       old_part_.path,
       new_part_.path,
@@ -257,9 +259,11 @@
       new_extents,
       {},  // old_deflates
       {},  // new_deflates
-      PayloadVersion(kBrilloMajorPayloadVersion, kSourceMinorPayloadVersion),
+      {.version = PayloadVersion(kBrilloMajorPayloadVersion,
+                                 kSourceMinorPayloadVersion)},
       &data,
-      &op));
+      &aop));
+  InstallOperation& op = aop.op;
   ASSERT_TRUE(data.empty());
 
   ASSERT_TRUE(op.has_type());
@@ -283,7 +287,7 @@
   ASSERT_TRUE(WriteExtents(new_part_.path, new_extents, kBlockSize, data_blob));
 
   brillo::Blob data;
-  InstallOperation op;
+  AnnotatedOperation aop;
   ASSERT_TRUE(diff_utils::ReadExtentsToDiff(
       old_part_.path,
       new_part_.path,
@@ -291,9 +295,11 @@
       new_extents,
       {},  // old_deflates
       {},  // new_deflates
-      PayloadVersion(kBrilloMajorPayloadVersion, kSourceMinorPayloadVersion),
+      {.version = PayloadVersion(kBrilloMajorPayloadVersion,
+                                 kSourceMinorPayloadVersion)},
       &data,
-      &op));
+      &aop));
+  auto& op = aop.op;
   ASSERT_FALSE(data.empty());
   ASSERT_TRUE(op.has_type());
   ASSERT_EQ(InstallOperation::SOURCE_BSDIFF, op.type());
@@ -312,7 +318,7 @@
   ASSERT_TRUE(WriteExtents(new_part_.path, extents, kBlockSize, data_blob));
 
   brillo::Blob data;
-  InstallOperation op;
+  AnnotatedOperation aop;
   ASSERT_TRUE(diff_utils::ReadExtentsToDiff(
       old_part_.path,
       new_part_.path,
@@ -320,10 +326,11 @@
       extents,
       {},  // old_deflates
       {},  // new_deflates
-      PayloadVersion(kMaxSupportedMajorPayloadVersion,
-                     kMaxSupportedMinorPayloadVersion),
+      {.version = PayloadVersion(kMaxSupportedMajorPayloadVersion,
+                                 kMaxSupportedMinorPayloadVersion)},
       &data,
-      &op));
+      &aop));
+  auto& op = aop.op;
   ASSERT_FALSE(data.empty());
   ASSERT_TRUE(op.has_type());
   ASSERT_EQ(InstallOperation::REPLACE_BZ, op.type());
diff --git a/payload_generator/extent_utils.h b/payload_generator/extent_utils.h
index e9afa98..1cf7c77 100644
--- a/payload_generator/extent_utils.h
+++ b/payload_generator/extent_utils.h
@@ -127,6 +127,18 @@
 
 std::ostream& operator<<(std::ostream& out, const Extent& extent);
 
+template <typename Container>
+size_t GetNthBlock(const Container& extents, const size_t n) {
+  size_t cur_block_count = 0;
+  for (const auto& extent : extents) {
+    if (cur_block_count + extent.num_blocks() >= n) {
+      return extent.start_block() + (n - cur_block_count);
+    }
+    cur_block_count += extent.num_blocks();
+  }
+  return std::numeric_limits<size_t>::max();
+}
+
 }  // namespace chromeos_update_engine
 
 #endif  // UPDATE_ENGINE_PAYLOAD_GENERATOR_EXTENT_UTILS_H_
diff --git a/payload_generator/generate_delta_main.cc b/payload_generator/generate_delta_main.cc
index b04fec0..1919a7e 100644
--- a/payload_generator/generate_delta_main.cc
+++ b/payload_generator/generate_delta_main.cc
@@ -425,6 +425,9 @@
       disable_vabc,
       false,
       "Whether to disable Virtual AB Compression when installing the OTA");
+  DEFINE_bool(enable_vabc_xor,
+              false,
+              "Whether to use Virtual AB Compression XOR feature");
   DEFINE_string(
       apex_info_file, "", "Path to META/apex_info.pb found in target build");
 
@@ -543,6 +546,8 @@
     payload_config.apex_info_file = FLAGS_apex_info_file;
   }
 
+  payload_config.enable_vabc_xor = FLAGS_enable_vabc_xor;
+
   if (!FLAGS_new_partitions.empty()) {
     LOG_IF(FATAL, !FLAGS_new_image.empty() || !FLAGS_new_kernel.empty())
         << "--new_image and --new_kernel are deprecated, please use "
diff --git a/payload_generator/merge_sequence_generator_unittest.cc b/payload_generator/merge_sequence_generator_unittest.cc
index b8507ed..579cca8 100644
--- a/payload_generator/merge_sequence_generator_unittest.cc
+++ b/payload_generator/merge_sequence_generator_unittest.cc
@@ -51,7 +51,7 @@
 };
 
 TEST_F(MergeSequenceGeneratorTest, Create) {
-  std::vector<AnnotatedOperation> aops{{"file1", {}}, {"file2", {}}};
+  std::vector<AnnotatedOperation> aops{{"file1", {}, {}}, {"file2", {}, {}}};
   aops[0].op.set_type(InstallOperation::SOURCE_COPY);
   *aops[0].op.add_src_extents() = ExtentForRange(10, 10);
   *aops[0].op.add_dst_extents() = ExtentForRange(30, 10);
@@ -81,7 +81,7 @@
   *(op.add_src_extents()) = ExtentForRange(8, 4);
   *(op.add_dst_extents()) = ExtentForRange(10, 8);
 
-  AnnotatedOperation aop{"file1", op};
+  AnnotatedOperation aop{"file1", op, {}};
   auto generator = MergeSequenceGenerator::Create({aop});
   ASSERT_TRUE(generator);
   std::vector<CowMergeOperation> expected = {
diff --git a/payload_generator/payload_generation_config.h b/payload_generator/payload_generation_config.h
index 9c8c59f..c15ac78 100644
--- a/payload_generator/payload_generation_config.h
+++ b/payload_generator/payload_generation_config.h
@@ -233,6 +233,9 @@
 
   // Path to apex_info.pb, extracted from target_file.zip
   std::string apex_info_file;
+
+  // Whether to enable VABC xor op
+  bool enable_vabc_xor = false;
 };
 
 }  // namespace chromeos_update_engine
diff --git a/scripts/brillo_update_payload b/scripts/brillo_update_payload
index d1a1781..0be2b0d 100755
--- a/scripts/brillo_update_payload
+++ b/scripts/brillo_update_payload
@@ -202,6 +202,8 @@
   DEFINE_string full_boot "" "Will include full boot image"
   DEFINE_string disable_vabc "" \
     "Optional: Disables Virtual AB Compression when installing the OTA"
+  DEFINE_string enable_vabc_xor "" \
+    "Optional: Enable the use of Virtual AB Compression XOR feature"
 fi
 if [[ "${COMMAND}" == "hash" || "${COMMAND}" == "sign" ]]; then
   DEFINE_string unsigned_payload "" "Path to the input unsigned payload."
@@ -708,6 +710,11 @@
     fi
   fi
 
+  if [[ -n "${FLAGS_enable_vabc_xor}" ]]; then
+    GENERATOR_ARGS+=(
+      --enable_vabc_xor="${FLAGS_enable_vabc_xor}" )
+  fi
+
   if [[ -n "${FLAGS_disable_vabc}" ]]; then
     GENERATOR_ARGS+=(
       --disable_vabc="${FLAGS_disable_vabc}" )