aboutsummaryrefslogtreecommitdiffhomepage
path: root/tensorflow/compiler
diff options
context:
space:
mode:
authorGravatar Mark Heffernan <meheff@google.com>2018-05-01 14:52:40 -0700
committerGravatar TensorFlower Gardener <gardener@tensorflow.org>2018-05-01 14:55:30 -0700
commitfdcdf752dca18d479932119a2445e0129fcd54a9 (patch)
treea952abdb7f4a385370cfbe9a579f1800135db772 /tensorflow/compiler
parent325d0ef21a48bea1cc618a2bd24a9776de417ce5 (diff)
Fix bug in peak buffer accounting in buffer assignment.
Buffer assignment keeps track of the set of logical buffers which are live at the point of peak memory usage for each allocation. Previously colocated buffers were not properly accounted for. This CL addresses this problem. PiperOrigin-RevId: 195001567
Diffstat (limited to 'tensorflow/compiler')
-rw-r--r--tensorflow/compiler/xla/service/buffer_assignment.cc196
-rw-r--r--tensorflow/compiler/xla/service/buffer_assignment.h23
-rw-r--r--tensorflow/compiler/xla/service/buffer_assignment_test.cc77
3 files changed, 169 insertions, 127 deletions
diff --git a/tensorflow/compiler/xla/service/buffer_assignment.cc b/tensorflow/compiler/xla/service/buffer_assignment.cc
index dbe45e932c..94ccfedf62 100644
--- a/tensorflow/compiler/xla/service/buffer_assignment.cc
+++ b/tensorflow/compiler/xla/service/buffer_assignment.cc
@@ -292,112 +292,6 @@ BufferAllocationProto BufferAllocation::ToProto() const {
return proto;
}
-std::pair<int64, std::vector<const LogicalBuffer*>>
-BufferAllocation::ComputePeakMemoryLogicalBuffers() const {
- if (HeapTraces().empty()) {
- // Just return the largest LogicalBuffer in the allocation.
- const LogicalBuffer* largest_buffer = nullptr;
- int64 largest_size = 0;
- for (const auto& pair : assigned_buffers()) {
- const LogicalBuffer* buffer = pair.first;
- int64 size = pair.second.size;
- if (largest_buffer == nullptr) {
- largest_buffer = buffer;
- largest_size = size;
- continue;
- }
- // Tie-break with LogicalBuffer::Id so the return value is stable relative
- // to changing addresses.
- if (size > largest_size ||
- ((size == largest_size) && (largest_buffer->id() > buffer->id()))) {
- largest_buffer = buffer;
- largest_size = size;
- }
- }
- CHECK(largest_buffer != nullptr)
- << "No logical buffers in allocation: " << ToString();
- return {largest_size, {largest_buffer}};
- }
-
- // Create a map from LogicalBuffer::Id to LogicalBuffer* for the logical
- // buffers in this allocation.
- tensorflow::gtl::FlatMap<LogicalBuffer::Id, const LogicalBuffer*>
- id_to_buffer;
- tensorflow::gtl::FlatMap<const LogicalBuffer*, int64> buffer_sizes;
- for (const auto& pair : assigned_buffers()) {
- const LogicalBuffer* buffer = pair.first;
- const OffsetSize& offset_size = pair.second;
- id_to_buffer[buffer->id()] = buffer;
- buffer_sizes[buffer] = offset_size.size;
- }
-
- // Returns how much the given event increases the total size of live
- // buffers. Can be negative.
- auto memory_delta = [this, &id_to_buffer, &buffer_sizes](
- const HeapSimulatorTrace::Event& event) -> int64 {
- const LogicalBuffer* buffer = id_to_buffer.at(event.buffer_id());
- const int64 buffer_size = buffer_sizes.at(buffer);
- if (event.kind() == HeapSimulatorTrace::Event::ALLOC) {
- return buffer_size;
- } else if (event.kind() == HeapSimulatorTrace::Event::SHARE_WITH) {
- // Sharing a buffer does not change the live set size for the purposes of
- // the heap simulator. Even though the shared-with buffer may be smaller,
- // the entire allocation remains live.
- return 0;
- } else if (event.kind() == HeapSimulatorTrace::Event::FREE) {
- return -1 * buffer_size;
- }
- LOG(FATAL) << "Unknown event kind: " << event.kind();
- };
-
- int64 total_max_live_size = 0;
- std::vector<const LogicalBuffer*> live_buffers_vector;
- for (const HeapSimulatorTrace& heap_trace : HeapTraces()) {
- // First compute the size of the maximal live set.
- int64 max_live_size = 0;
- int64 live_size = 0;
- for (const auto& event : heap_trace.events()) {
- live_size += memory_delta(event);
- if (max_live_size < live_size) {
- max_live_size = live_size;
- }
- }
-
- // Next gather the set of logical buffers live at the earliest point of
- // maximal live set size.
- tensorflow::gtl::FlatSet<const LogicalBuffer*> live_buffers;
- live_size = 0;
- for (const auto& event : heap_trace.events()) {
- const LogicalBuffer* buffer = id_to_buffer.at(event.buffer_id());
- if (event.kind() == HeapSimulatorTrace::Event::ALLOC) {
- InsertOrDie(&live_buffers, buffer);
- } else if (event.kind() == HeapSimulatorTrace::Event::SHARE_WITH) {
- // Nothing to do.
- } else if (event.kind() == HeapSimulatorTrace::Event::FREE) {
- CHECK(ContainsKey(live_buffers, buffer));
- live_buffers.erase(buffer);
- }
-
- live_size += memory_delta(event);
- if (live_size == max_live_size) {
- break;
- }
- }
- CHECK_EQ(live_size, max_live_size);
- total_max_live_size += max_live_size;
-
- live_buffers_vector.insert(live_buffers_vector.end(), live_buffers.begin(),
- live_buffers.end());
- }
-
- // Stabily sort the live buffers.
- std::sort(live_buffers_vector.begin(), live_buffers_vector.end(),
- [](const LogicalBuffer* a, const LogicalBuffer* b) {
- return a->id() < b->id();
- });
- return {total_max_live_size, live_buffers_vector};
-}
-
string BufferAllocation::ToString() const {
string output;
Appendf(&output, "allocation %lld: %p, size %lld", index_, this, size());
@@ -610,6 +504,7 @@ BufferAllocation* BufferAssignment::NewAllocation(const LogicalBuffer& buffer,
BufferAllocation* allocation =
NewEmptyAllocation(size, is_thread_local, is_reusable, buffer.color());
AddAssignment(allocation, buffer, /*offset=*/0, size);
+ allocation->peak_buffers_.push_back(&buffer);
return allocation;
}
@@ -680,6 +575,10 @@ void BufferAssignment::CombineTempAllocations() {
CHECK_EQ(temp_allocation.HeapTraces().size(), 1);
combined_allocation->AddHeapTrace(temp_allocation.HeapTraces().front());
}
+ combined_allocation->peak_buffers_.insert(
+ combined_allocation->peak_buffers_.end(),
+ temp_allocation.peak_buffers_.begin(),
+ temp_allocation.peak_buffers_.end());
}
// Replace all existing temporary allocations with the new combined
// allocations.
@@ -1228,6 +1127,89 @@ Status BufferAssigner::AssignBuffersWithSequentialOrdering(
return Status::OK();
}
+namespace {
+
+// Computes and returns the set of logical buffers live at the point of maximal
+// liveness in the given heap trace. LogicalBuffers are (stabily) sorted by id.
+std::vector<const LogicalBuffer*> ComputePeakMemoryLogicalBuffers(
+ const BufferAllocation& allocation, const HeapSimulatorTrace& heap_trace) {
+ // Create a map from LogicalBuffer::Id to LogicalBuffer* for the logical
+ // buffers in this allocation.
+ tensorflow::gtl::FlatMap<LogicalBuffer::Id, const LogicalBuffer*>
+ id_to_buffer;
+ tensorflow::gtl::FlatMap<const LogicalBuffer*, int64> buffer_sizes;
+ for (const auto& pair : allocation.assigned_buffers()) {
+ const LogicalBuffer* buffer = pair.first;
+ const BufferAllocation::OffsetSize& offset_size = pair.second;
+ id_to_buffer[buffer->id()] = buffer;
+ buffer_sizes[buffer] = offset_size.size;
+ }
+
+ // Returns how much the given event increases the total size of live
+ // buffers. Can be negative.
+ auto memory_delta = [&id_to_buffer, &buffer_sizes](
+ const HeapSimulatorTrace::Event& event) -> int64 {
+ const LogicalBuffer* buffer = id_to_buffer.at(event.buffer_id());
+ const int64 buffer_size = buffer_sizes.at(buffer);
+ if (event.kind() == HeapSimulatorTrace::Event::ALLOC) {
+ return buffer_size;
+ } else if (event.kind() == HeapSimulatorTrace::Event::SHARE_WITH) {
+ // Sharing a buffer does not change the live set size for the purposes of
+ // the heap simulator. Even though the shared-with buffer may be smaller,
+ // the entire allocation remains live.
+ return 0;
+ } else if (event.kind() == HeapSimulatorTrace::Event::FREE) {
+ return -1 * buffer_size;
+ }
+ LOG(FATAL) << "Unknown event kind: " << event.kind();
+ };
+
+ // First compute the size of the maximal live set.
+ int64 max_live_size = 0;
+ int64 live_size = 0;
+ for (const auto& event : heap_trace.events()) {
+ live_size += memory_delta(event);
+ if (max_live_size < live_size) {
+ max_live_size = live_size;
+ }
+ }
+
+ // Next gather the set of logical buffers live at the earliest point of
+ // maximal live set size.
+ tensorflow::gtl::FlatSet<const LogicalBuffer*> live_buffers;
+ live_size = 0;
+ for (const auto& event : heap_trace.events()) {
+ const LogicalBuffer* buffer = id_to_buffer.at(event.buffer_id());
+ if (event.kind() == HeapSimulatorTrace::Event::ALLOC) {
+ InsertOrDie(&live_buffers, buffer);
+ } else if (event.kind() == HeapSimulatorTrace::Event::SHARE_WITH) {
+ // Nothing to do.
+ } else if (event.kind() == HeapSimulatorTrace::Event::FREE) {
+ CHECK(ContainsKey(live_buffers, buffer));
+ live_buffers.erase(buffer);
+ }
+
+ live_size += memory_delta(event);
+ if (live_size == max_live_size) {
+ break;
+ }
+ }
+ CHECK_EQ(live_size, max_live_size);
+
+ std::vector<const LogicalBuffer*> live_buffers_vector;
+ live_buffers_vector.insert(live_buffers_vector.end(), live_buffers.begin(),
+ live_buffers.end());
+
+ // Stabily sort the live buffers.
+ std::sort(live_buffers_vector.begin(), live_buffers_vector.end(),
+ [](const LogicalBuffer* a, const LogicalBuffer* b) {
+ return a->id() < b->id();
+ });
+ return live_buffers_vector;
+}
+
+} // namespace
+
void BufferAssigner::AssignBuffersFromHeapSimulator(
const HeapSimulator::Result& result, BufferAssignment* assignment,
LogicalBuffer::Color color) {
@@ -1246,6 +1228,8 @@ void BufferAssigner::AssignBuffersFromHeapSimulator(
const HeapSimulator::Chunk& chunk = buffer_chunk.second;
assignment->AddAssignment(allocation, buffer, chunk.offset, chunk.size);
}
+ allocation->peak_buffers_ =
+ ComputePeakMemoryLogicalBuffers(*allocation, result.debug_trace);
VLOG(1) << "Ran heap simulation for allocation: " << allocation->ToString();
allocation->AddHeapTrace(result.debug_trace);
diff --git a/tensorflow/compiler/xla/service/buffer_assignment.h b/tensorflow/compiler/xla/service/buffer_assignment.h
index 3086d0e2ca..15fd905e8d 100644
--- a/tensorflow/compiler/xla/service/buffer_assignment.h
+++ b/tensorflow/compiler/xla/service/buffer_assignment.h
@@ -206,17 +206,15 @@ class BufferAllocation {
return heap_traces_;
}
- // Compute and return the LogicalBuffers which are live at the point of peak
- // memory usage for the given allocation. The point of peak memory usage is
- // the point at which the total size of all live logical buffers is
- // maximal. If peak memory is reached at multiple points, the set of logical
- // buffers live at the earliest maximal point is returned. The vector is
- // stabily asserted by LogicalBuffer::Index.
- //
- // The return value is a pair of total size of the logical buffers at peak,
- // and the buffers themselves.
- std::pair<int64, std::vector<const LogicalBuffer*>>
- ComputePeakMemoryLogicalBuffers() const;
+ // Returns the LogicalBuffers which are live at the point of peak memory usage
+ // for this allocation. The point of peak memory usage is the point at which
+ // the total size of all live logical buffers is maximal. If peak memory is
+ // reached at multiple points, the set of logical buffers live at the earliest
+ // maximal point is returned. The vector is stabily sorted by
+ // LogicalBuffer::Index.
+ const std::vector<const LogicalBuffer*>& PeakMemoryLogicalBuffers() const {
+ return peak_buffers_;
+ }
// Get the number of bytes lost to fragmentation. This is equal to the
// difference between the size of the allocation and the size of the maximal
@@ -291,6 +289,9 @@ class BufferAllocation {
int64 fragmentation_bytes_ = 0;
std::vector<HeapSimulatorTrace> heap_traces_;
+
+ // Set of buffers live at the point of peak memory usage for this allocation.
+ std::vector<const LogicalBuffer*> peak_buffers_;
};
// Add stream operators for nicer output of CHECK/RET_CHECK failures.
diff --git a/tensorflow/compiler/xla/service/buffer_assignment_test.cc b/tensorflow/compiler/xla/service/buffer_assignment_test.cc
index 3ec9795a65..40cf6483aa 100644
--- a/tensorflow/compiler/xla/service/buffer_assignment_test.cc
+++ b/tensorflow/compiler/xla/service/buffer_assignment_test.cc
@@ -1519,12 +1519,8 @@ TEST_F(BufferAssignmentTest, TrivialPeakBuffers) {
// single logical buffer should be exactly the logical buffer in that
// allocation.
const BufferAllocation& mul_buffer = GetTopLevelAllocation(*buffers, mul);
- int64 peak_size;
- std::vector<const LogicalBuffer*> peak_buffers;
-
- std::tie(peak_size, peak_buffers) =
- mul_buffer.ComputePeakMemoryLogicalBuffers();
- EXPECT_EQ(peak_size, ShapeUtil::ByteSizeOf(f32vec100_));
+ const std::vector<const LogicalBuffer*>& peak_buffers =
+ mul_buffer.PeakMemoryLogicalBuffers();
ASSERT_EQ(peak_buffers.size(), 1);
EXPECT_EQ(peak_buffers[0]->instruction(), mul);
}
@@ -1555,6 +1551,7 @@ TEST_F(BufferAssignmentTest, PeakBuffers) {
HloInstruction::CreateConcatenate(concat_shape, {rev, neg}, 0));
// Make the root tiny so no interior nodes can share its buffer.
auto root = builder.AddInstruction(HloInstruction::CreateSlice(
+
ShapeUtil::MakeShape(F32, {1}), concat, {0}, {1}, {1}));
auto module = CreateNewModule();
@@ -1569,12 +1566,10 @@ TEST_F(BufferAssignmentTest, PeakBuffers) {
EXPECT_TRUE(buffer.IsPreallocatedTempBuffer());
ASSERT_EQ(buffer.assigned_buffers().size(), 4);
- int64 peak_size;
- std::vector<const LogicalBuffer*> peak_buffers;
- std::tie(peak_size, peak_buffers) = buffer.ComputePeakMemoryLogicalBuffers();
+ const std::vector<const LogicalBuffer*>& peak_buffers =
+ buffer.PeakMemoryLogicalBuffers();
// The peak live set should be concat and its inputs.
- EXPECT_EQ(peak_size, ShapeUtil::ByteSizeOf(ShapeUtil::MakeShape(F32, {400})));
ASSERT_EQ(peak_buffers.size(), 3);
std::vector<const HloInstruction*> peak_instructions;
for (const LogicalBuffer* logical_buffer : peak_buffers) {
@@ -1583,6 +1578,68 @@ TEST_F(BufferAssignmentTest, PeakBuffers) {
EXPECT_THAT(peak_instructions, UnorderedElementsAre(rev, neg, concat));
}
+TEST_F(BufferAssignmentTest, PeakBuffersWhile) {
+ auto module = CreateNewModule();
+ const Shape shape = ShapeUtil::MakeShape(F32, {123, 123});
+ HloComputation* condition;
+ {
+ auto b = HloComputation::Builder(TestName() + ".cond");
+ b.AddInstruction(HloInstruction::CreateParameter(0, shape, "x"));
+ b.AddInstruction(
+ HloInstruction::CreateConstant(Literal::CreateR0<bool>(true)));
+ condition = module->AddEmbeddedComputation(b.Build());
+ }
+ HloComputation* body;
+ {
+ auto b = HloComputation::Builder(TestName() + ".body");
+ auto param =
+ b.AddInstruction(HloInstruction::CreateParameter(0, shape, "x"));
+ b.AddInstruction(
+ HloInstruction::CreateUnary(shape, HloOpcode::kNegate, param));
+ body = module->AddEmbeddedComputation(b.Build());
+ }
+ auto builder = HloComputation::Builder(TestName());
+ auto param =
+ builder.AddInstruction(HloInstruction::CreateParameter(0, shape, "p0"));
+ auto copy = builder.AddInstruction(
+ HloInstruction::CreateUnary(shape, HloOpcode::kCopy, param));
+ auto while_op = builder.AddInstruction(
+ HloInstruction::CreateWhile(shape, condition, body, copy));
+ // This broadcast should get a temporary allocation which is merged with the
+ // allocation for the while. Peak buffers should include the while and the
+ // broadcast.
+ auto bcast = builder.AddInstruction(HloInstruction::CreateBroadcast(
+ ShapeUtil::MakeShape(F32, {123, 123, 123}), while_op, {0, 1}));
+ builder.AddInstruction(HloInstruction::CreateReverse(
+ ShapeUtil::MakeShape(F32, {123, 123, 123}), bcast, {0}));
+ module->AddEntryComputation(builder.Build());
+
+ auto buffers = RunBufferAssignment(module.get());
+ const BufferAllocation& buffer = GetTopLevelAllocation(*buffers, bcast);
+ const std::vector<const LogicalBuffer*>& peak_buffers =
+ buffer.PeakMemoryLogicalBuffers();
+ ASSERT_EQ(peak_buffers.size(), 2);
+
+ // The peak buffers should include the broadcast and one of the colocated
+ // buffers of the while (body param, condition param, body root, or the while
+ // itself).
+ const LogicalBuffer* bcast_buffer;
+ const LogicalBuffer* nonbcast_buffer;
+ if (peak_buffers[0]->instruction() == bcast) {
+ bcast_buffer = peak_buffers[0];
+ nonbcast_buffer = peak_buffers[1];
+ } else {
+ bcast_buffer = peak_buffers[1];
+ nonbcast_buffer = peak_buffers[0];
+ }
+ EXPECT_EQ(bcast_buffer->instruction(), bcast);
+ EXPECT_TRUE(
+ nonbcast_buffer->instruction() == while_op ||
+ nonbcast_buffer->instruction() == body->parameter_instruction(0) ||
+ nonbcast_buffer->instruction() == body->root_instruction() ||
+ nonbcast_buffer->instruction() == condition->parameter_instruction(0));
+}
+
class WhileBufferAssignmentTest : public HloTestBase {
protected:
std::unique_ptr<HloComputation> BuildWhileConditionComputation(