AU: reuse scratch blocks in delta generation, tolerate insufficient scratch.
Changes the delta generator cycle cutting algorithm. The old algorithm
looked for scratch space on the disk, then allocated that scratch
sequentially to operations that needed it, bailing if it ran out of
scratch.
The new algorithm first allocates non-existent blocks (those at
kTempBlockStart and higher) at first to break cycles. It then comes up
with a valid topological order for all nodes. It then tries to find
real blocks for all the non-existent temp blocks allocated. For each
cut edge ( A->B => A->N<-B ), there are 3 nodes of importance: the old
source (A) the old dst (B) and the new node (N). N writes to temp
blocks, then B reads from those temp blocks. The new algorithm starts
at node B and scans the topological order up, knowing that any
dependency from a found node to B would be valid, as it doesn't
require a change in the topo order. If a node is found, which has
blocks written, and these blocks aren't in a read-before dependence
from the found node to another node, we use those blocks as temp
space. Notice that after we find temp blocks for a cut, a future cut
could use the blocks written by N of this cut, thus allowing temp
blocks to be reused.
Another change this algorithm makes is that if no node is found for
supplying temp blocks, the cut is resolved by making the old dst node
(B) a full operation, and moving it to the end of the topological
order (so it may supply temp blocks to other cuts).
Thus, if there is insufficient scratch, we lose compression ratio
rather than failing.
In a resent image that used a lot of scratch, I found this new algo
didn't have to convert any ops to full, as reuing scratch was enough.
This CL does perform a regression. Our filesystems do not take up the
full space in the partition. The existing delta generator makes use of
that extra space for scratch, but this new algo doesn't. We could fix
this new algorithm by creating a dummy node that writes to this extra
space at the end of the partition, then removing it from the update
file so the client doesn't actually do that write. If the reviewer is
okay with it, I'll file an Issue for this.
BUG=None
TEST=Attached unittests, create/perform delta w/ new algo
Review URL: http://codereview.chromium.org/3597014
diff --git a/delta_diff_generator.cc b/delta_diff_generator.cc
index 4d67c4e..676629a 100644
--- a/delta_diff_generator.cc
+++ b/delta_diff_generator.cc
@@ -10,6 +10,7 @@
#include <sys/types.h>
#include <algorithm>
+#include <map>
#include <set>
#include <string>
#include <utility>
@@ -22,6 +23,7 @@
#include "update_engine/bzip.h"
#include "update_engine/cycle_breaker.h"
#include "update_engine/extent_mapper.h"
+#include "update_engine/extent_ranges.h"
#include "update_engine/file_writer.h"
#include "update_engine/filesystem_iterator.h"
#include "update_engine/graph_types.h"
@@ -33,6 +35,7 @@
#include "update_engine/utils.h"
using std::make_pair;
+using std::map;
using std::max;
using std::min;
using std::set;
@@ -135,14 +138,16 @@
return true;
}
-// For a given regular file which must exist at new_root + path, and may
-// exist at old_root + path, creates a new InstallOperation and adds it to
-// the graph. Also, populates the 'blocks' array as necessary.
-// Also, writes the data necessary to send the file down to the client
-// into data_fd, which has length *data_file_size. *data_file_size is
-// updated appropriately.
-// Returns true on success.
+// For a given regular file which must exist at new_root + path, and
+// may exist at old_root + path, creates a new InstallOperation and
+// adds it to the graph. Also, populates the |blocks| array as
+// necessary, if |blocks| is non-NULL. Also, writes the data
+// necessary to send the file down to the client into data_fd, which
+// has length *data_file_size. *data_file_size is updated
+// appropriately. If |existing_vertex| is no kInvalidIndex, use that
+// rather than allocating a new vertex. Returns true on success.
bool DeltaReadFile(Graph* graph,
+ Vertex::Index existing_vertex,
vector<Block>* blocks,
const string& old_root,
const string& new_root,
@@ -167,15 +172,20 @@
*data_file_size += data.size();
// Now, insert into graph and blocks vector
- graph->resize(graph->size() + 1);
- graph->back().op = operation;
- CHECK(graph->back().op.has_type());
- graph->back().file_name = path;
+ Vertex::Index vertex = existing_vertex;
+ if (vertex == Vertex::kInvalidIndex) {
+ graph->resize(graph->size() + 1);
+ vertex = graph->size() - 1;
+ }
+ (*graph)[vertex].op = operation;
+ CHECK((*graph)[vertex].op.has_type());
+ (*graph)[vertex].file_name = path;
- TEST_AND_RETURN_FALSE(AddInstallOpToBlocksVector(graph->back().op,
- blocks,
- *graph,
- graph->size() - 1));
+ if (blocks)
+ TEST_AND_RETURN_FALSE(AddInstallOpToBlocksVector((*graph)[vertex].op,
+ blocks,
+ *graph,
+ vertex));
return true;
}
@@ -205,6 +215,7 @@
LOG(INFO) << "Encoding file " << fs_iter.GetPartialPath();
TEST_AND_RETURN_FALSE(DeltaReadFile(graph,
+ Vertex::kInvalidIndex,
blocks,
old_root,
new_root,
@@ -215,74 +226,23 @@
return true;
}
-// Attempts to find |block_count| blocks to use as scratch space. Returns true
-// on success. Right now we return exactly as many blocks as are required.
-//
-// TODO(adlr): Consider returning all scratch blocks, even if there are extras,
-// to make it easier for a scratch allocator to find contiguous regions for
-// specific scratch writes.
-bool FindScratchSpace(const vector<Block>& blocks,
- vector<Block>::size_type block_count,
- vector<Extent>* out) {
- // Scan |blocks| for blocks that are neither read, nor written. If we don't
- // find enough of those, look past the end of |blocks| till the end of the
- // partition. If we don't find |block_count| scratch blocks, return false.
- //
- // TODO(adlr): Return blocks that are written by operations that don't have
- // incoming edges (and thus, can be deferred until all old blocks are read by
- // other operations).
- vector<Extent> ret;
- vector<Block>::size_type blocks_found = 0;
- const size_t kPartitionBlocks = kRootFSPartitionSize / kBlockSize;
- for (vector<Block>::size_type i = 0;
- i < kPartitionBlocks && blocks_found < block_count; i++) {
- if (i >= blocks.size() ||
- (blocks[i].reader == Vertex::kInvalidIndex &&
- blocks[i].writer == Vertex::kInvalidIndex)) {
- graph_utils::AppendBlockToExtents(&ret, i);
- blocks_found++;
- }
- }
- LOG(INFO) << "found " << blocks_found << " scratch blocks";
- if (blocks_found == block_count) {
- out->swap(ret);
- return true;
- }
- return false;
-}
-
-// This class takes a collection of Extents and allows the client to
-// allocate space from these extents. The client must not request more
-// space then exists in the source extents. Space is allocated from the
-// beginning of the source extents on; no consideration is paid to
-// fragmentation.
-class LinearExtentAllocator {
+// This class allocates non-existent temp blocks, starting from
+// kTempBlockStart. Other code is responsible for converting these
+// temp blocks into real blocks, as the client can't read or write to
+// these blocks.
+class DummyExtentAllocator {
public:
- explicit LinearExtentAllocator(const vector<Extent>& extents)
- : extents_(extents),
- extent_index_(0),
- extent_blocks_allocated_(0) {}
+ explicit DummyExtentAllocator()
+ : next_block_(kTempBlockStart) {}
vector<Extent> Allocate(const uint64_t block_count) {
- vector<Extent> ret;
- for (uint64_t blocks = 0; blocks < block_count; blocks++) {
- CHECK_LT(extent_index_, extents_.size());
- CHECK_LT(extent_blocks_allocated_, extents_[extent_index_].num_blocks());
- graph_utils::AppendBlockToExtents(
- &ret,
- extents_[extent_index_].start_block() + extent_blocks_allocated_);
- extent_blocks_allocated_++;
- if (extent_blocks_allocated_ >= extents_[extent_index_].num_blocks()) {
- extent_blocks_allocated_ = 0;
- extent_index_++;
- }
- }
+ vector<Extent> ret(1);
+ ret[0].set_start_block(next_block_);
+ ret[0].set_num_blocks(block_count);
+ next_block_ += block_count;
return ret;
}
private:
- const vector<Extent> extents_;
- vector<Extent>::size_type extent_index_; // current Extent
- // number of blocks allocated from the current extent
- uint64_t extent_blocks_allocated_;
+ uint64_t next_block_;
};
// Reads blocks from image_path that are not yet marked as being written
@@ -296,7 +256,8 @@
int blobs_fd,
off_t* blobs_length,
const string& image_path,
- DeltaArchiveManifest_InstallOperation* out_op) {
+ Vertex* vertex) {
+ DeltaArchiveManifest_InstallOperation* out_op = &vertex->op;
int image_fd = open(image_path.c_str(), O_RDONLY, 000);
TEST_AND_RETURN_FALSE_ERRNO(image_fd >= 0);
ScopedFdCloser image_fd_closer(&image_fd);
@@ -324,6 +285,9 @@
for (vector<Block>::size_type i = 0; i < blocks.size(); i++) {
if (blocks[i].writer != Vertex::kInvalidIndex)
continue;
+ if (blocks[i].reader != Vertex::kInvalidIndex) {
+ graph_utils::AddReadBeforeDep(vertex, blocks[i].reader, i);
+ }
graph_utils::AppendBlockToExtents(&extents, i);
block_count++;
}
@@ -377,6 +341,8 @@
out_op->set_type(DeltaArchiveManifest_InstallOperation_Type_REPLACE_BZ);
out_op->set_data_offset(*blobs_length);
out_op->set_data_length(compressed_data.size());
+ LOG(INFO) << "Rootfs non-data blocks compressed take up "
+ << compressed_data.size();
*blobs_length += compressed_data.size();
out_op->set_dst_length(kBlockSize * block_count);
DeltaDiffGenerator::StoreExtents(extents, out_op->mutable_dst_extents());
@@ -618,81 +584,85 @@
return true;
}
-void DeltaDiffGenerator::SubstituteBlocks(
- DeltaArchiveManifest_InstallOperation* op,
- const vector<Extent>& remove_extents,
- const vector<Extent>& replace_extents) {
- // First, expand out the blocks that op reads from
- vector<uint64_t> read_blocks;
- for (int i = 0; i < op->src_extents_size(); i++) {
- const Extent& extent = op->src_extents(i);
+namespace {
+
+// Takes a collection (vector or RepeatedPtrField) of Extent and
+// returns a vector of the blocks referenced, in order.
+template<typename T>
+vector<uint64_t> ExpandExtents(const T& extents) {
+ vector<uint64_t> ret;
+ for (size_t i = 0, e = static_cast<size_t>(extents.size()); i != e; ++i) {
+ const Extent extent = graph_utils::GetElement(extents, i);
if (extent.start_block() == kSparseHole) {
- read_blocks.resize(read_blocks.size() + extent.num_blocks(), kSparseHole);
+ ret.resize(ret.size() + extent.num_blocks(), kSparseHole);
} else {
for (uint64_t block = extent.start_block();
block < (extent.start_block() + extent.num_blocks()); block++) {
- read_blocks.push_back(block);
+ ret.push_back(block);
}
}
}
+ return ret;
+}
+
+// Takes a vector of blocks and returns an equivalent vector of Extent
+// objects.
+vector<Extent> CompressExtents(const vector<uint64_t>& blocks) {
+ vector<Extent> new_extents;
+ for (vector<uint64_t>::const_iterator it = blocks.begin(), e = blocks.end();
+ it != e; ++it) {
+ graph_utils::AppendBlockToExtents(&new_extents, *it);
+ }
+ return new_extents;
+}
+
+} // namespace {}
+
+void DeltaDiffGenerator::SubstituteBlocks(
+ Vertex* vertex,
+ const vector<Extent>& remove_extents,
+ const vector<Extent>& replace_extents) {
+ // First, expand out the blocks that op reads from
+ vector<uint64_t> read_blocks = ExpandExtents(vertex->op.src_extents());
{
// Expand remove_extents and replace_extents
- vector<uint64_t> remove_extents_expanded;
- for (vector<Extent>::const_iterator it = remove_extents.begin();
- it != remove_extents.end(); ++it) {
- const Extent& extent = *it;
- for (uint64_t block = extent.start_block();
- block < (extent.start_block() + extent.num_blocks()); block++) {
- remove_extents_expanded.push_back(block);
- }
- }
- vector<uint64_t> replace_extents_expanded;
- for (vector<Extent>::const_iterator it = replace_extents.begin();
- it != replace_extents.end(); ++it) {
- const Extent& extent = *it;
- for (uint64_t block = extent.start_block();
- block < (extent.start_block() + extent.num_blocks()); block++) {
- replace_extents_expanded.push_back(block);
- }
- }
+ vector<uint64_t> remove_extents_expanded =
+ ExpandExtents(remove_extents);
+ vector<uint64_t> replace_extents_expanded =
+ ExpandExtents(replace_extents);
CHECK_EQ(remove_extents_expanded.size(), replace_extents_expanded.size());
+ map<uint64_t, uint64_t> conversion;
for (vector<uint64_t>::size_type i = 0;
i < replace_extents_expanded.size(); i++) {
- vector<uint64_t>::size_type index = 0;
- CHECK(utils::VectorIndexOf(read_blocks,
- remove_extents_expanded[i],
- &index));
- CHECK(read_blocks[index] == remove_extents_expanded[i]);
- read_blocks[index] = replace_extents_expanded[i];
+ conversion[remove_extents_expanded[i]] = replace_extents_expanded[i];
+ }
+ utils::ApplyMap(&read_blocks, conversion);
+ for (Vertex::EdgeMap::iterator it = vertex->out_edges.begin(),
+ e = vertex->out_edges.end(); it != e; ++it) {
+ vector<uint64_t> write_before_deps_expanded =
+ ExpandExtents(it->second.write_extents);
+ utils::ApplyMap(&write_before_deps_expanded, conversion);
+ it->second.write_extents = CompressExtents(write_before_deps_expanded);
}
}
// Convert read_blocks back to extents
- op->clear_src_extents();
- vector<Extent> new_extents;
- for (vector<uint64_t>::const_iterator it = read_blocks.begin();
- it != read_blocks.end(); ++it) {
- graph_utils::AppendBlockToExtents(&new_extents, *it);
- }
- DeltaDiffGenerator::StoreExtents(new_extents, op->mutable_src_extents());
+ vertex->op.clear_src_extents();
+ vector<Extent> new_extents = CompressExtents(read_blocks);
+ DeltaDiffGenerator::StoreExtents(new_extents,
+ vertex->op.mutable_src_extents());
}
bool DeltaDiffGenerator::CutEdges(Graph* graph,
- const vector<Block>& blocks,
- const set<Edge>& edges) {
- // First, find enough scratch space for the edges we'll be cutting.
- vector<Block>::size_type blocks_required = 0;
- for (set<Edge>::const_iterator it = edges.begin(); it != edges.end(); ++it) {
- blocks_required += graph_utils::EdgeWeight(*graph, *it);
- }
- vector<Extent> scratch_extents;
- LOG(INFO) << "requesting " << blocks_required << " blocks of scratch";
- TEST_AND_RETURN_FALSE(
- FindScratchSpace(blocks, blocks_required, &scratch_extents));
- LinearExtentAllocator scratch_allocator(scratch_extents);
+ const set<Edge>& edges,
+ vector<CutEdgeVertexes>* out_cuts) {
+ DummyExtentAllocator scratch_allocator;
+ vector<CutEdgeVertexes> cuts;
+ cuts.reserve(edges.size());
uint64_t scratch_blocks_used = 0;
for (set<Edge>::const_iterator it = edges.begin();
it != edges.end(); ++it) {
+ cuts.resize(cuts.size() + 1);
vector<Extent> old_extents =
(*graph)[it->first].out_edges[it->second].extents;
// Choose some scratch space
@@ -700,21 +670,33 @@
LOG(INFO) << "using " << graph_utils::EdgeWeight(*graph, *it)
<< " scratch blocks ("
<< scratch_blocks_used << ")";
- vector<Extent> scratch =
+ cuts.back().tmp_extents =
scratch_allocator.Allocate(graph_utils::EdgeWeight(*graph, *it));
// create vertex to copy original->scratch
+ cuts.back().new_vertex = graph->size();
graph->resize(graph->size() + 1);
+ cuts.back().old_src = it->first;
+ cuts.back().old_dst = it->second;
+
+ EdgeProperties& cut_edge_properties =
+ (*graph)[it->first].out_edges.find(it->second)->second;
+
+ // This should never happen, as we should only be cutting edges between
+ // real file nodes, and write-before relationships are created from
+ // a real file node to a temp copy node:
+ CHECK(cut_edge_properties.write_extents.empty())
+ << "Can't cut edge that has write-before relationship.";
// make node depend on the copy operation
(*graph)[it->first].out_edges.insert(make_pair(graph->size() - 1,
- EdgeProperties()));
+ cut_edge_properties));
// Set src/dst extents and other proto variables for copy operation
graph->back().op.set_type(DeltaArchiveManifest_InstallOperation_Type_MOVE);
DeltaDiffGenerator::StoreExtents(
- (*graph)[it->first].out_edges[it->second].extents,
+ cut_edge_properties.extents,
graph->back().op.mutable_src_extents());
- DeltaDiffGenerator::StoreExtents(scratch,
+ DeltaDiffGenerator::StoreExtents(cuts.back().tmp_extents,
graph->back().op.mutable_dst_extents());
graph->back().op.set_src_length(
graph_utils::EdgeWeight(*graph, *it) * kBlockSize);
@@ -722,23 +704,26 @@
// make the dest node read from the scratch space
DeltaDiffGenerator::SubstituteBlocks(
- &((*graph)[it->second].op),
+ &((*graph)[it->second]),
(*graph)[it->first].out_edges[it->second].extents,
- scratch);
+ cuts.back().tmp_extents);
// delete the old edge
CHECK_EQ(1, (*graph)[it->first].out_edges.erase(it->second));
// Add an edge from dst to copy operation
- (*graph)[it->second].out_edges.insert(make_pair(graph->size() - 1,
- EdgeProperties()));
+ EdgeProperties write_before_edge_properties;
+ write_before_edge_properties.write_extents = cuts.back().tmp_extents;
+ (*graph)[it->second].out_edges.insert(
+ make_pair(graph->size() - 1, write_before_edge_properties));
}
+ out_cuts->swap(cuts);
return true;
}
// Stores all Extents in 'extents' into 'out'.
void DeltaDiffGenerator::StoreExtents(
- vector<Extent>& extents,
+ const vector<Extent>& extents,
google::protobuf::RepeatedPtrField<Extent>* out) {
for (vector<Extent>::const_iterator it = extents.begin();
it != extents.end(); ++it) {
@@ -774,6 +759,244 @@
}
}
+namespace {
+
+class SortCutsByTopoOrderLess {
+ public:
+ SortCutsByTopoOrderLess(vector<vector<Vertex::Index>::size_type>& table)
+ : table_(table) {}
+ bool operator()(const CutEdgeVertexes& a, const CutEdgeVertexes& b) {
+ return table_[a.old_dst] < table_[b.old_dst];
+ }
+ private:
+ vector<vector<Vertex::Index>::size_type>& table_;
+};
+
+} // namespace {}
+
+void DeltaDiffGenerator::GenerateReverseTopoOrderMap(
+ vector<Vertex::Index>& op_indexes,
+ vector<vector<Vertex::Index>::size_type>* reverse_op_indexes) {
+ vector<vector<Vertex::Index>::size_type> table(op_indexes.size());
+ for (vector<Vertex::Index>::size_type i = 0, e = op_indexes.size();
+ i != e; ++i) {
+ Vertex::Index node = op_indexes[i];
+ if (table.size() < (node + 1)) {
+ table.resize(node + 1);
+ }
+ table[node] = i;
+ }
+ reverse_op_indexes->swap(table);
+}
+
+void DeltaDiffGenerator::SortCutsByTopoOrder(vector<Vertex::Index>& op_indexes,
+ vector<CutEdgeVertexes>* cuts) {
+ // first, make a reverse lookup table.
+ vector<vector<Vertex::Index>::size_type> table;
+ GenerateReverseTopoOrderMap(op_indexes, &table);
+ SortCutsByTopoOrderLess less(table);
+ sort(cuts->begin(), cuts->end(), less);
+}
+
+void DeltaDiffGenerator::MoveFullOpsToBack(Graph* graph,
+ vector<Vertex::Index>* op_indexes) {
+ vector<Vertex::Index> ret;
+ vector<Vertex::Index> full_ops;
+ ret.reserve(op_indexes->size());
+ for (vector<Vertex::Index>::size_type i = 0, e = op_indexes->size(); i != e;
+ ++i) {
+ DeltaArchiveManifest_InstallOperation_Type type =
+ (*graph)[(*op_indexes)[i]].op.type();
+ if (type == DeltaArchiveManifest_InstallOperation_Type_REPLACE ||
+ type == DeltaArchiveManifest_InstallOperation_Type_REPLACE_BZ) {
+ full_ops.push_back((*op_indexes)[i]);
+ } else {
+ ret.push_back((*op_indexes)[i]);
+ }
+ }
+ LOG(INFO) << "Stats: " << full_ops.size() << " full ops out of "
+ << (full_ops.size() + ret.size()) << " total ops.";
+ ret.insert(ret.end(), full_ops.begin(), full_ops.end());
+ op_indexes->swap(ret);
+}
+
+namespace {
+
+template<typename T>
+bool TempBlocksExistInExtents(const T& extents) {
+ for (int i = 0, e = extents.size(); i < e; ++i) {
+ Extent extent = graph_utils::GetElement(extents, i);
+ uint64_t start = extent.start_block();
+ uint64_t num = extent.num_blocks();
+ if (start == kSparseHole)
+ continue;
+ if (start >= kTempBlockStart ||
+ (start + num) >= kTempBlockStart) {
+ LOG(ERROR) << "temp block!";
+ LOG(ERROR) << "start: " << start << ", num: " << num;
+ LOG(ERROR) << "kTempBlockStart: " << kTempBlockStart;
+ LOG(ERROR) << "returning true";
+ return true;
+ }
+ // check for wrap-around, which would be a bug:
+ CHECK(start <= (start + num));
+ }
+ return false;
+}
+
+} // namespace {}
+
+bool DeltaDiffGenerator::AssignTempBlocks(
+ Graph* graph,
+ const string& new_root,
+ int data_fd,
+ off_t* data_file_size,
+ vector<Vertex::Index>* op_indexes,
+ vector<vector<Vertex::Index>::size_type>* reverse_op_indexes,
+ vector<CutEdgeVertexes>& cuts) {
+ CHECK(!cuts.empty());
+ for (vector<CutEdgeVertexes>::size_type i = cuts.size() - 1, e = 0;
+ true ; --i) {
+ LOG(INFO) << "Fixing temp blocks in cut " << i
+ << ": old dst: " << cuts[i].old_dst << " new vertex: "
+ << cuts[i].new_vertex;
+ const uint64_t blocks_needed =
+ graph_utils::BlocksInExtents(cuts[i].tmp_extents);
+ LOG(INFO) << "Scanning for usable blocks (" << blocks_needed << " needed)";
+ // For now, just look for a single op w/ sufficient blocks, not
+ // considering blocks from outgoing read-before deps.
+ Vertex::Index node = cuts[i].old_dst;
+ DeltaArchiveManifest_InstallOperation_Type node_type =
+ (*graph)[node].op.type();
+ if (node_type == DeltaArchiveManifest_InstallOperation_Type_REPLACE ||
+ node_type == DeltaArchiveManifest_InstallOperation_Type_REPLACE_BZ) {
+ LOG(INFO) << "This was already converted to full, so skipping.";
+ // Delete the temp node and pointer to it from old src
+ if (!(*graph)[cuts[i].old_src].out_edges.erase(cuts[i].new_vertex)) {
+ LOG(INFO) << "Odd. node " << cuts[i].old_src << " didn't point to "
+ << cuts[i].new_vertex;
+ }
+ (*graph)[cuts[i].new_vertex].valid = false;
+ vector<Vertex::Index>::size_type new_topo_idx =
+ (*reverse_op_indexes)[cuts[i].new_vertex];
+ op_indexes->erase(op_indexes->begin() + new_topo_idx);
+ GenerateReverseTopoOrderMap(*op_indexes, reverse_op_indexes);
+ continue;
+ }
+ bool found_node = false;
+ for (vector<Vertex::Index>::size_type j = (*reverse_op_indexes)[node] + 1,
+ je = op_indexes->size(); j < je; ++j) {
+ Vertex::Index test_node = (*op_indexes)[j];
+ // See if this node has sufficient blocks
+ ExtentRanges ranges;
+ ranges.AddRepeatedExtents((*graph)[test_node].op.dst_extents());
+ ranges.SubtractExtent(ExtentForRange(
+ kTempBlockStart, kSparseHole - kTempBlockStart));
+ ranges.SubtractRepeatedExtents((*graph)[test_node].op.src_extents());
+ // For now, for simplicity, subtract out all blocks in read-before
+ // dependencies.
+ for (Vertex::EdgeMap::const_iterator edge_i =
+ (*graph)[test_node].out_edges.begin(),
+ edge_e = (*graph)[test_node].out_edges.end();
+ edge_i != edge_e; ++edge_i) {
+ ranges.SubtractExtents(edge_i->second.extents);
+ }
+
+ uint64_t blocks_found = ranges.blocks();
+ if (blocks_found < blocks_needed) {
+ if (blocks_found > 0)
+ LOG(INFO) << "insufficient blocks found in topo node " << j
+ << " (node " << (*op_indexes)[j] << "). Found only "
+ << blocks_found;
+ continue;
+ }
+ found_node = true;
+ LOG(INFO) << "Found sufficient blocks in topo node " << j
+ << " (node " << (*op_indexes)[j] << ")";
+ // Sub in the blocks, and make the node supplying the blocks
+ // depend on old_dst.
+ vector<Extent> real_extents =
+ ranges.GetExtentsForBlockCount(blocks_needed);
+
+ // Fix the old dest node w/ the real blocks
+ SubstituteBlocks(&(*graph)[node],
+ cuts[i].tmp_extents,
+ real_extents);
+
+ // Fix the new node w/ the real blocks. Since the new node is just a
+ // copy operation, we can replace all the dest extents w/ the real
+ // blocks.
+ DeltaArchiveManifest_InstallOperation *op =
+ &(*graph)[cuts[i].new_vertex].op;
+ op->clear_dst_extents();
+ StoreExtents(real_extents, op->mutable_dst_extents());
+
+ // Add an edge from the real-block supplier to the old dest block.
+ graph_utils::AddReadBeforeDepExtents(&(*graph)[test_node],
+ node,
+ real_extents);
+ break;
+ }
+ if (!found_node) {
+ // convert to full op
+ LOG(WARNING) << "Failed to find enough temp blocks for cut " << i
+ << " with old dest (graph node " << node
+ << "). Converting to a full op, at the expense of a "
+ << "good compression ratio.";
+ TEST_AND_RETURN_FALSE(ConvertCutToFullOp(graph,
+ cuts[i],
+ new_root,
+ data_fd,
+ data_file_size));
+ // move the full op to the back
+ vector<Vertex::Index> new_op_indexes;
+ for (vector<Vertex::Index>::const_iterator iter_i = op_indexes->begin(),
+ iter_e = op_indexes->end(); iter_i != iter_e; ++iter_i) {
+ if ((*iter_i == cuts[i].old_dst) || (*iter_i == cuts[i].new_vertex))
+ continue;
+ new_op_indexes.push_back(*iter_i);
+ }
+ new_op_indexes.push_back(cuts[i].old_dst);
+ op_indexes->swap(new_op_indexes);
+
+ GenerateReverseTopoOrderMap(*op_indexes, reverse_op_indexes);
+ }
+ if (i == e) {
+ // break out of for() loop
+ break;
+ }
+ }
+ return true;
+}
+
+bool DeltaDiffGenerator::NoTempBlocksRemain(const Graph& graph) {
+ size_t idx = 0;
+ for (Graph::const_iterator it = graph.begin(), e = graph.end(); it != e;
+ ++it, ++idx) {
+ if (!it->valid)
+ continue;
+ const DeltaArchiveManifest_InstallOperation& op = it->op;
+ if (TempBlocksExistInExtents(op.dst_extents()) ||
+ TempBlocksExistInExtents(op.src_extents())) {
+ LOG(INFO) << "bad extents in node " << idx;
+ LOG(INFO) << "so yeah";
+ return false;
+ }
+
+ // Check out-edges:
+ for (Vertex::EdgeMap::const_iterator jt = it->out_edges.begin(),
+ je = it->out_edges.end(); jt != je; ++jt) {
+ if (TempBlocksExistInExtents(jt->second.extents) ||
+ TempBlocksExistInExtents(jt->second.write_extents)) {
+ LOG(INFO) << "bad out edge in node " << idx;
+ LOG(INFO) << "so yeah";
+ return false;
+ }
+ }
+ }
+ return true;
+}
+
bool DeltaDiffGenerator::ReorderDataBlobs(
DeltaArchiveManifest* manifest,
const std::string& data_blobs_path,
@@ -814,6 +1037,88 @@
return true;
}
+bool DeltaDiffGenerator::ConvertCutToFullOp(Graph* graph,
+ const CutEdgeVertexes& cut,
+ const string& new_root,
+ int data_fd,
+ off_t* data_file_size) {
+ // Drop all incoming edges, keep all outgoing edges
+
+ // Keep all outgoing edges
+ Vertex::EdgeMap out_edges = (*graph)[cut.old_dst].out_edges;
+ graph_utils::DropWriteBeforeDeps(&out_edges);
+
+ TEST_AND_RETURN_FALSE(DeltaReadFile(graph,
+ cut.old_dst,
+ NULL,
+ "/-!@:&*nonexistent_path",
+ new_root,
+ (*graph)[cut.old_dst].file_name,
+ data_fd,
+ data_file_size));
+
+ (*graph)[cut.old_dst].out_edges = out_edges;
+
+ // Right now we don't have doubly-linked edges, so we have to scan
+ // the whole graph.
+ graph_utils::DropIncomingEdgesTo(graph, cut.old_dst);
+
+ // Delete temp node
+ (*graph)[cut.old_src].out_edges.erase(cut.new_vertex);
+ CHECK((*graph)[cut.old_dst].out_edges.find(cut.new_vertex) ==
+ (*graph)[cut.old_dst].out_edges.end());
+ (*graph)[cut.new_vertex].valid = false;
+ return true;
+}
+
+bool DeltaDiffGenerator::ConvertGraphToDag(Graph* graph,
+ const string& new_root,
+ int fd,
+ off_t* data_file_size,
+ vector<Vertex::Index>* final_order) {
+ CycleBreaker cycle_breaker;
+ LOG(INFO) << "Finding cycles...";
+ set<Edge> cut_edges;
+ cycle_breaker.BreakCycles(*graph, &cut_edges);
+ LOG(INFO) << "done finding cycles";
+ CheckGraph(*graph);
+
+ // Calculate number of scratch blocks needed
+
+ LOG(INFO) << "Cutting cycles...";
+ vector<CutEdgeVertexes> cuts;
+ TEST_AND_RETURN_FALSE(CutEdges(graph, cut_edges, &cuts));
+ LOG(INFO) << "done cutting cycles";
+ LOG(INFO) << "There are " << cuts.size() << " cuts.";
+ CheckGraph(*graph);
+
+ LOG(INFO) << "Creating initial topological order...";
+ TopologicalSort(*graph, final_order);
+ LOG(INFO) << "done with initial topo order";
+ CheckGraph(*graph);
+
+ LOG(INFO) << "Moving full ops to the back";
+ MoveFullOpsToBack(graph, final_order);
+ LOG(INFO) << "done moving full ops to back";
+
+ vector<vector<Vertex::Index>::size_type> inverse_final_order;
+ GenerateReverseTopoOrderMap(*final_order, &inverse_final_order);
+
+ if (!cuts.empty())
+ TEST_AND_RETURN_FALSE(AssignTempBlocks(graph,
+ new_root,
+ fd,
+ data_file_size,
+ final_order,
+ &inverse_final_order,
+ cuts));
+ LOG(INFO) << "Making sure all temp blocks have been allocated";
+ graph_utils::DumpGraph(*graph);
+ CHECK(NoTempBlocksRemain(*graph));
+ LOG(INFO) << "done making sure all temp blocks are allocated";
+ return true;
+}
+
bool DeltaDiffGenerator::GenerateDeltaUpdateFile(
const string& old_root,
const string& old_image,
@@ -857,7 +1162,7 @@
vector<DeltaArchiveManifest_InstallOperation> kernel_ops;
- DeltaArchiveManifest_InstallOperation final_op;
+ vector<Vertex::Index> final_order;
{
int fd;
TEST_AND_RETURN_FALSE(
@@ -871,13 +1176,15 @@
new_root,
fd,
&data_file_size));
+ LOG(INFO) << "done reading normal files";
CheckGraph(graph);
+ graph.resize(graph.size() + 1);
TEST_AND_RETURN_FALSE(ReadUnwrittenBlocks(blocks,
fd,
&data_file_size,
new_image,
- &final_op));
+ &graph.back()));
// Read kernel partition
TEST_AND_RETURN_FALSE(DeltaCompressKernelPartition(old_kernel_part,
@@ -885,42 +1192,26 @@
&kernel_ops,
fd,
&data_file_size));
+
+ LOG(INFO) << "done reading kernel";
+ CheckGraph(graph);
+
+ LOG(INFO) << "Creating edges...";
+ CreateEdges(&graph, blocks);
+ LOG(INFO) << "Done creating edges";
+ CheckGraph(graph);
+
+ TEST_AND_RETURN_FALSE(ConvertGraphToDag(&graph,
+ new_root,
+ fd,
+ &data_file_size,
+ &final_order));
}
- CheckGraph(graph);
-
- LOG(INFO) << "Creating edges...";
- CreateEdges(&graph, blocks);
- CheckGraph(graph);
-
- CycleBreaker cycle_breaker;
- LOG(INFO) << "Finding cycles...";
- set<Edge> cut_edges;
- cycle_breaker.BreakCycles(graph, &cut_edges);
- CheckGraph(graph);
-
- // Calculate number of scratch blocks needed
-
- LOG(INFO) << "Cutting cycles...";
- TEST_AND_RETURN_FALSE(CutEdges(&graph, blocks, cut_edges));
- CheckGraph(graph);
-
- vector<Vertex::Index> final_order;
- LOG(INFO) << "Ordering...";
- TopologicalSort(graph, &final_order);
- CheckGraph(graph);
// Convert to protobuf Manifest object
DeltaArchiveManifest manifest;
CheckGraph(graph);
InstallOperationsToManifest(graph, final_order, kernel_ops, &manifest);
- {
- // Write final operation
- DeltaArchiveManifest_InstallOperation* op =
- manifest.add_install_operations();
- *op = final_op;
- CHECK(op->has_type());
- LOG(INFO) << "final op length: " << op->data_length();
- }
CheckGraph(graph);
manifest.set_block_size(kBlockSize);
diff --git a/delta_diff_generator.h b/delta_diff_generator.h
index 31f9e64..9362b55 100644
--- a/delta_diff_generator.h
+++ b/delta_diff_generator.h
@@ -20,6 +20,22 @@
namespace chromeos_update_engine {
+// This struct stores all relevant info for an edge that is cut between
+// nodes old_src -> old_dst by creating new vertex new_vertex. The new
+// relationship is:
+// old_src -(read before)-> new_vertex <-(write before)- old_dst
+// new_vertex is a MOVE operation that moves some existing blocks into
+// temp space. The temp extents are, by necessity, stored in new_vertex
+// (as dst extents) and old_dst (as src extents), but they are also broken
+// out into tmp_extents, as the nodes themselves may contain many more
+// extents.
+struct CutEdgeVertexes {
+ Vertex::Index new_vertex;
+ Vertex::Index old_src;
+ Vertex::Index old_dst;
+ std::vector<Extent> tmp_extents;
+};
+
class DeltaDiffGenerator {
public:
// Represents a disk block on the install partition.
@@ -57,6 +73,19 @@
// These functions are public so that the unit tests can access them:
+ // Takes a graph, which is not a DAG, which represents the files just
+ // read from disk, and converts it into a DAG by breaking all cycles
+ // and finding temp space to resolve broken edges.
+ // The final order of the nodes is given in |final_order|
+ // Some files may need to be reread from disk, thus |fd| and
+ // |data_file_size| are be passed.
+ // Returns true on success.
+ static bool ConvertGraphToDag(Graph* graph,
+ const std::string& new_root,
+ int fd,
+ off_t* data_file_size,
+ std::vector<Vertex::Index>* final_order);
+
// Reads old_filename (if it exists) and a new_filename and determines
// the smallest way to encode this file for the diff. It stores
// necessary data in out_data and fills in out_op.
@@ -78,7 +107,7 @@
// contains blocks 6, 2, 3, 5, and replace blocks contains
// 12, 13, 14, 15, then op will be changed to read from:
// 1, 13, 14, 4, 15, 12, 7, 8
- static void SubstituteBlocks(DeltaArchiveManifest_InstallOperation* op,
+ static void SubstituteBlocks(Vertex* vertex,
const std::vector<Extent>& remove_extents,
const std::vector<Extent>& replace_extents);
@@ -90,17 +119,32 @@
// but not on B. Free space is found by looking in 'blocks'.
// Returns true on success.
static bool CutEdges(Graph* graph,
- const std::vector<Block>& blocks,
- const std::set<Edge>& edges);
+ const std::set<Edge>& edges,
+ std::vector<CutEdgeVertexes>* out_cuts);
// Stores all Extents in 'extents' into 'out'.
- static void StoreExtents(std::vector<Extent>& extents,
+ static void StoreExtents(const std::vector<Extent>& extents,
google::protobuf::RepeatedPtrField<Extent>* out);
// Creates all the edges for the graph. Writers of a block point to
// readers of the same block. This is because for an edge A->B, B
// must complete before A executes.
static void CreateEdges(Graph* graph, const std::vector<Block>& blocks);
+
+ // Given a topologically sorted graph |op_indexes| and |graph|, alters
+ // |op_indexes| to move all the full operations to the end of the vector.
+ // Full operations should not be depended on, so this is safe.
+ static void MoveFullOpsToBack(Graph* graph,
+ std::vector<Vertex::Index>* op_indexes);
+
+ // Sorts the vector |cuts| by its |cuts[].old_dest| member. Order is
+ // determined by the order of elements in op_indexes.
+ static void SortCutsByTopoOrder(std::vector<Vertex::Index>& op_indexes,
+ std::vector<CutEdgeVertexes>* cuts);
+
+ // Returns true iff there are no extents in the graph that refer to temp
+ // blocks. Temp blocks are in the range [kTempBlockStart, kSparseHole).
+ static bool NoTempBlocksRemain(const Graph& graph);
// Install operations in the manifest may reference data blobs, which
// are in data_blobs_path. This function creates a new data blobs file
@@ -112,6 +156,42 @@
static bool ReorderDataBlobs(DeltaArchiveManifest* manifest,
const std::string& data_blobs_path,
const std::string& new_data_blobs_path);
+
+ // Handles allocation of temp blocks to a cut edge by converting the
+ // dest node to a full op. This removes the need for temp blocks, but
+ // comes at the cost of a worse compression ratio.
+ // For example, say we have A->B->A. It would first be cut to form:
+ // A->B->N<-A, where N copies blocks to temp space. If there are no
+ // temp blocks, this function can be called to convert it to the form:
+ // A->B. Now, A is a full operation.
+ static bool ConvertCutToFullOp(Graph* graph,
+ const CutEdgeVertexes& cut,
+ const std::string& new_root,
+ int data_fd,
+ off_t* data_file_size);
+
+ // Takes |op_indexes|, which is effectively a mapping from order in
+ // which the op is performed -> graph vertex index, and produces the
+ // reverse: a mapping from graph vertex index -> op_indexes index.
+ static void GenerateReverseTopoOrderMap(
+ std::vector<Vertex::Index>& op_indexes,
+ std::vector<std::vector<Vertex::Index>::size_type>* reverse_op_indexes);
+
+ // Takes a |graph|, which has edges that must be cut, as listed in
+ // |cuts|. Cuts the edges. Maintains a list in which the operations
+ // will be performed (in |op_indexes|) and the reverse (in
+ // |reverse_op_indexes|). Cutting edges requires scratch space, and
+ // if insufficient scratch is found, the file is reread and will be
+ // send down (either as REPLACE or REPLACE_BZ). Returns true on
+ // success.
+ static bool AssignTempBlocks(
+ Graph* graph,
+ const std::string& new_root,
+ int data_fd,
+ off_t* data_file_size,
+ std::vector<Vertex::Index>* op_indexes,
+ std::vector<std::vector<Vertex::Index>::size_type>* reverse_op_indexes,
+ std::vector<CutEdgeVertexes>& cuts);
private:
// This should never be constructed
diff --git a/delta_diff_generator_unittest.cc b/delta_diff_generator_unittest.cc
index 11f7735..0d7b0fe 100644
--- a/delta_diff_generator_unittest.cc
+++ b/delta_diff_generator_unittest.cc
@@ -7,6 +7,7 @@
#include <fcntl.h>
#include <unistd.h>
#include <set>
+#include <sstream>
#include <string>
#include <utility>
#include <vector>
@@ -15,15 +16,18 @@
#include "update_engine/cycle_breaker.h"
#include "update_engine/delta_diff_generator.h"
#include "update_engine/delta_performer.h"
+#include "update_engine/extent_ranges.h"
#include "update_engine/graph_types.h"
#include "update_engine/graph_utils.h"
#include "update_engine/subprocess.h"
#include "update_engine/test_utils.h"
+#include "update_engine/topological_sort.h"
#include "update_engine/utils.h"
using std::make_pair;
using std::set;
using std::string;
+using std::stringstream;
using std::vector;
namespace chromeos_update_engine {
@@ -161,13 +165,14 @@
vector<Extent> replace_blocks;
AppendExtent(&replace_blocks, 10, 2);
AppendExtent(&replace_blocks, 13, 2);
- DeltaArchiveManifest_InstallOperation op;
+ Vertex vertex;
+ DeltaArchiveManifest_InstallOperation& op = vertex.op;
OpAppendExtent(&op, 4, 3);
OpAppendExtent(&op, kSparseHole, 4); // Sparse hole in file
OpAppendExtent(&op, 3, 1);
OpAppendExtent(&op, 7, 3);
- DeltaDiffGenerator::SubstituteBlocks(&op, remove_blocks, replace_blocks);
+ DeltaDiffGenerator::SubstituteBlocks(&vertex, remove_blocks, replace_blocks);
EXPECT_EQ(7, op.src_extents_size());
EXPECT_EQ(11, op.src_extents(0).start_block());
@@ -254,7 +259,8 @@
EXPECT_TRUE(cut_edges.end() != cut_edges.find(make_pair<Vertex::Index>(1,
0)));
- EXPECT_TRUE(DeltaDiffGenerator::CutEdges(&graph, blocks, cut_edges));
+ vector<CutEdgeVertexes> cuts;
+ EXPECT_TRUE(DeltaDiffGenerator::CutEdges(&graph, cut_edges, &cuts));
EXPECT_EQ(3, graph.size());
@@ -262,21 +268,17 @@
EXPECT_EQ(DeltaArchiveManifest_InstallOperation_Type_MOVE,
graph.back().op.type());
EXPECT_EQ(2, graph.back().op.src_extents_size());
- EXPECT_EQ(2, graph.back().op.dst_extents_size());
- EXPECT_EQ(0, graph.back().op.dst_extents(0).start_block());
- EXPECT_EQ(1, graph.back().op.dst_extents(0).num_blocks());
- EXPECT_EQ(8, graph.back().op.dst_extents(1).start_block());
- EXPECT_EQ(1, graph.back().op.dst_extents(1).num_blocks());
+ EXPECT_EQ(1, graph.back().op.dst_extents_size());
+ EXPECT_EQ(kTempBlockStart, graph.back().op.dst_extents(0).start_block());
+ EXPECT_EQ(2, graph.back().op.dst_extents(0).num_blocks());
EXPECT_TRUE(graph.back().out_edges.empty());
// Check that old node reads from new blocks
- EXPECT_EQ(3, graph[0].op.src_extents_size());
- EXPECT_EQ(0, graph[0].op.src_extents(0).start_block());
- EXPECT_EQ(1, graph[0].op.src_extents(0).num_blocks());
- EXPECT_EQ(8, graph[0].op.src_extents(1).start_block());
+ EXPECT_EQ(2, graph[0].op.src_extents_size());
+ EXPECT_EQ(kTempBlockStart, graph[0].op.src_extents(0).start_block());
+ EXPECT_EQ(2, graph[0].op.src_extents(0).num_blocks());
+ EXPECT_EQ(7, graph[0].op.src_extents(1).start_block());
EXPECT_EQ(1, graph[0].op.src_extents(1).num_blocks());
- EXPECT_EQ(7, graph[0].op.src_extents(2).start_block());
- EXPECT_EQ(1, graph[0].op.src_extents(2).num_blocks());
// And that the old dst extents haven't changed
EXPECT_EQ(2, graph[0].op.dst_extents_size());
@@ -348,4 +350,209 @@
unlink(new_blobs.c_str());
}
+TEST_F(DeltaDiffGeneratorTest, MoveFullOpsToBackTest) {
+ Graph graph(4);
+ graph[0].file_name = "A";
+ graph[0].op.set_type(DeltaArchiveManifest_InstallOperation_Type_REPLACE);
+ graph[1].file_name = "B";
+ graph[1].op.set_type(DeltaArchiveManifest_InstallOperation_Type_BSDIFF);
+ graph[2].file_name = "C";
+ graph[2].op.set_type(DeltaArchiveManifest_InstallOperation_Type_REPLACE_BZ);
+ graph[3].file_name = "D";
+ graph[3].op.set_type(DeltaArchiveManifest_InstallOperation_Type_MOVE);
+
+ vector<Vertex::Index> vect(graph.size());
+
+ for (vector<Vertex::Index>::size_type i = 0; i < vect.size(); ++i) {
+ vect[i] = i;
+ }
+ DeltaDiffGenerator::MoveFullOpsToBack(&graph, &vect);
+ EXPECT_EQ(vect.size(), graph.size());
+ EXPECT_EQ(graph[vect[0]].file_name, "B");
+ EXPECT_EQ(graph[vect[1]].file_name, "D");
+ EXPECT_EQ(graph[vect[2]].file_name, "A");
+ EXPECT_EQ(graph[vect[3]].file_name, "C");
+}
+
+namespace {
+
+#define OP_BSDIFF DeltaArchiveManifest_InstallOperation_Type_BSDIFF
+#define OP_MOVE DeltaArchiveManifest_InstallOperation_Type_MOVE
+#define OP_REPLACE DeltaArchiveManifest_InstallOperation_Type_REPLACE
+#define OP_REPLACE_BZ DeltaArchiveManifest_InstallOperation_Type_REPLACE_BZ
+
+void GenVertex(Vertex* out,
+ const vector<Extent>& src_extents,
+ const vector<Extent>& dst_extents,
+ const string& path,
+ DeltaArchiveManifest_InstallOperation_Type type) {
+ out->op.set_type(type);
+ out->file_name = path;
+ DeltaDiffGenerator::StoreExtents(src_extents, out->op.mutable_src_extents());
+ DeltaDiffGenerator::StoreExtents(dst_extents, out->op.mutable_dst_extents());
+}
+
+vector<Extent> VectOfExt(uint64_t start_block, uint64_t num_blocks) {
+ return vector<Extent>(1, ExtentForRange(start_block, num_blocks));
+}
+
+EdgeProperties EdgeWithReadDep(const vector<Extent>& extents) {
+ EdgeProperties ret;
+ ret.extents = extents;
+ return ret;
+}
+
+EdgeProperties EdgeWithWriteDep(const vector<Extent>& extents) {
+ EdgeProperties ret;
+ ret.write_extents = extents;
+ return ret;
+}
+
+template<typename T>
+void DumpVect(const vector<T>& vect) {
+ std::stringstream ss(stringstream::out);
+ for (typename vector<T>::const_iterator it = vect.begin(), e = vect.end();
+ it != e; ++it) {
+ ss << *it << ", ";
+ }
+ LOG(INFO) << "{" << ss.str() << "}";
+}
+
+} // namespace {}
+
+TEST_F(DeltaDiffGeneratorTest, RunAsRootAssignTempBlocksTest) {
+ Graph graph(9);
+ const vector<Extent> empt; // empty
+ const string kFilename = "/foo";
+
+ // Some scratch space:
+ GenVertex(&graph[0], empt, VectOfExt(200, 1), "", OP_REPLACE);
+ GenVertex(&graph[1], empt, VectOfExt(210, 10), "", OP_REPLACE);
+ GenVertex(&graph[2], empt, VectOfExt(220, 1), "", OP_REPLACE);
+
+ // A cycle that requires 10 blocks to break:
+ GenVertex(&graph[3], VectOfExt(10, 11), VectOfExt(0, 9), "", OP_BSDIFF);
+ graph[3].out_edges[4] = EdgeWithReadDep(VectOfExt(0, 9));
+ GenVertex(&graph[4], VectOfExt(0, 9), VectOfExt(10, 11), "", OP_BSDIFF);
+ graph[4].out_edges[3] = EdgeWithReadDep(VectOfExt(10, 11));
+
+ // A cycle that requires 9 blocks to break:
+ GenVertex(&graph[5], VectOfExt(40, 11), VectOfExt(30, 10), "", OP_BSDIFF);
+ graph[5].out_edges[6] = EdgeWithReadDep(VectOfExt(30, 10));
+ GenVertex(&graph[6], VectOfExt(30, 10), VectOfExt(40, 11), "", OP_BSDIFF);
+ graph[6].out_edges[5] = EdgeWithReadDep(VectOfExt(40, 11));
+
+ // A cycle that requires 40 blocks to break (which is too many):
+ GenVertex(&graph[7],
+ VectOfExt(120, 50),
+ VectOfExt(60, 40),
+ "",
+ OP_BSDIFF);
+ graph[7].out_edges[8] = EdgeWithReadDep(VectOfExt(60, 40));
+ GenVertex(&graph[8],
+ VectOfExt(60, 40),
+ VectOfExt(120, 50),
+ kFilename,
+ OP_BSDIFF);
+ graph[8].out_edges[7] = EdgeWithReadDep(VectOfExt(120, 50));
+
+ graph_utils::DumpGraph(graph);
+
+ vector<Vertex::Index> final_order;
+
+
+ // Prepare the filesystem with the minimum required for this to work
+ string temp_dir;
+ EXPECT_TRUE(utils::MakeTempDirectory("/tmp/AssignTempBlocksTest.XXXXXX",
+ &temp_dir));
+ ScopedDirRemover temp_dir_remover(temp_dir);
+
+ const size_t kBlockSize = 4096;
+ vector<char> temp_data(kBlockSize * 50);
+ FillWithData(&temp_data);
+ EXPECT_TRUE(WriteFileVector(temp_dir + kFilename, temp_data));
+ ScopedPathUnlinker filename_unlinker(temp_dir + kFilename);
+
+ int fd;
+ EXPECT_TRUE(utils::MakeTempFile("/tmp/AssignTempBlocksTestData.XXXXXX",
+ NULL,
+ &fd));
+ ScopedFdCloser fd_closer(&fd);
+ off_t data_file_size = 0;
+
+
+ EXPECT_TRUE(DeltaDiffGenerator::ConvertGraphToDag(&graph,
+ temp_dir,
+ fd,
+ &data_file_size,
+ &final_order));
+
+
+ Graph expected_graph(12);
+ GenVertex(&expected_graph[0], empt, VectOfExt(200, 1), "", OP_REPLACE);
+ GenVertex(&expected_graph[1], empt, VectOfExt(210, 10), "", OP_REPLACE);
+ GenVertex(&expected_graph[2], empt, VectOfExt(220, 1), "", OP_REPLACE);
+ GenVertex(&expected_graph[3],
+ VectOfExt(10, 11),
+ VectOfExt(0, 9),
+ "",
+ OP_BSDIFF);
+ expected_graph[3].out_edges[9] = EdgeWithReadDep(VectOfExt(0, 9));
+ GenVertex(&expected_graph[4],
+ VectOfExt(60, 9),
+ VectOfExt(10, 11),
+ "",
+ OP_BSDIFF);
+ expected_graph[4].out_edges[3] = EdgeWithReadDep(VectOfExt(10, 11));
+ expected_graph[4].out_edges[9] = EdgeWithWriteDep(VectOfExt(60, 9));
+ GenVertex(&expected_graph[5],
+ VectOfExt(40, 11),
+ VectOfExt(30, 10),
+ "",
+ OP_BSDIFF);
+ expected_graph[5].out_edges[10] = EdgeWithReadDep(VectOfExt(30, 10));
+
+ GenVertex(&expected_graph[6],
+ VectOfExt(60, 10),
+ VectOfExt(40, 11),
+ "",
+ OP_BSDIFF);
+ expected_graph[6].out_edges[5] = EdgeWithReadDep(VectOfExt(40, 11));
+ expected_graph[6].out_edges[10] = EdgeWithWriteDep(VectOfExt(60, 10));
+
+ GenVertex(&expected_graph[7],
+ VectOfExt(120, 50),
+ VectOfExt(60, 40),
+ "",
+ OP_BSDIFF);
+ expected_graph[7].out_edges[6] = EdgeWithReadDep(VectOfExt(60, 10));
+
+ GenVertex(&expected_graph[8], empt, VectOfExt(0, 50), "/foo", OP_REPLACE_BZ);
+ expected_graph[8].out_edges[7] = EdgeWithReadDep(VectOfExt(120, 50));
+
+ GenVertex(&expected_graph[9],
+ VectOfExt(0, 9),
+ VectOfExt(60, 9),
+ "",
+ OP_MOVE);
+
+ GenVertex(&expected_graph[10],
+ VectOfExt(30, 10),
+ VectOfExt(60, 10),
+ "",
+ OP_MOVE);
+ expected_graph[10].out_edges[4] = EdgeWithReadDep(VectOfExt(60, 9));
+
+ EXPECT_EQ(12, graph.size());
+ EXPECT_FALSE(graph.back().valid);
+ for (Graph::size_type i = 0; i < graph.size() - 1; i++) {
+ EXPECT_TRUE(graph[i].out_edges == expected_graph[i].out_edges);
+ if (i == 8) {
+ // special case
+ } else {
+ // EXPECT_TRUE(graph[i] == expected_graph[i]) << "i = " << i;
+ }
+ }
+}
+
} // namespace chromeos_update_engine