Skip to content

Commit

Permalink
[Inference] Memory modification for ShrinkMemory. (#28355)
Browse files Browse the repository at this point in the history
  • Loading branch information
jiweibo authored Nov 4, 2020
1 parent 95b1868 commit 0511469
Show file tree
Hide file tree
Showing 19 changed files with 223 additions and 2 deletions.
1 change: 1 addition & 0 deletions paddle/fluid/memory/allocation/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@ cc_library(locked_allocator SRCS locked_allocator.cc DEPS allocator)
cc_library(buffered_allocator SRCS buffered_allocator.cc DEPS allocator)
cc_library(best_fit_allocator SRCS best_fit_allocator.cc DEPS allocator)
cc_library(naive_best_fit_allocator SRCS naive_best_fit_allocator.cc DEPS allocator buddy_allocator profiler)
cc_test(naive_best_fit_allocator_test SRCS naive_best_fit_allocator_test.cc DEPS naive_best_fit_allocator)
cc_test(buffered_allocator_test SRCS buffered_allocator_test.cc DEPS locked_allocator buffered_allocator cpu_allocator best_fit_allocator)

if (WITH_MKLDNN)
Expand Down
3 changes: 3 additions & 0 deletions paddle/fluid/memory/allocation/allocator.h
Original file line number Diff line number Diff line change
Expand Up @@ -178,12 +178,15 @@ class Allocator {
FreeImpl(allocation);
}

inline void Release(const platform::Place& place) { ReleaseImpl(place); }

// True if the `Allocate` is thread safe.
virtual bool IsAllocThreadSafe() const;

protected:
virtual Allocation* AllocateImpl(size_t size) = 0;
virtual void FreeImpl(Allocation* allocation);
virtual void ReleaseImpl(const platform::Place& place) {}
};

using AllocationDeleter = Allocator::AllocationDeleter;
Expand Down
5 changes: 5 additions & 0 deletions paddle/fluid/memory/allocation/allocator_facade.cc
Original file line number Diff line number Diff line change
Expand Up @@ -287,6 +287,11 @@ AllocationPtr AllocatorFacade::Alloc(const platform::Place& place,
return m_->GetAllocator(place, size)->Allocate(size);
}

void AllocatorFacade::Release(const platform::Place& place) {
m_->GetAllocator(place, /* A non-zero num to choose allocator_ */ 1)
->Release(place);
}

} // namespace allocation
} // namespace memory
} // namespace paddle
3 changes: 3 additions & 0 deletions paddle/fluid/memory/allocation/allocator_facade.h
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,9 @@ class AllocatorFacade {
// Allocate a unique allocation.
AllocationPtr Alloc(const platform::Place& place, size_t size);

// Release unused memory pool.
void Release(const platform::Place& place);

// TODO(yy): Allocate a Copy-On-Write allocation?
private:
AllocatorFacade();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,9 @@ class AutoGrowthBestFitAllocator : public Allocator {

void FreeImpl(Allocation *allocation) override;

// Release the memory block which is not used in pool.
void ReleaseImpl(const platform::Place &place) override { FreeIdleChunks(); }

private:
void FreeIdleChunks();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,7 @@ static void TestFreeIdleChunk(bool free_idle_chunk,
} else {
ASSERT_EQ(recorded_allocator->AllocatedSize(), memory_size + alignment);
}
ag_allocator->Release(platform::CPUPlace());
}
}

Expand Down
51 changes: 51 additions & 0 deletions paddle/fluid/memory/allocation/naive_best_fit_allocator.cc
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,9 @@ void *Alloc(const Place &place, size_t size);
template <typename Place>
void Free(const Place &place, void *p, size_t size);

template <typename Place>
void Release(const Place &place);

template <typename Place>
size_t Used(const Place &place);

Expand Down Expand Up @@ -99,6 +102,11 @@ void Free<platform::CPUPlace>(const platform::CPUPlace &place, void *p,
GetCPUBuddyAllocator()->Free(p);
}

template <>
void Release<platform::CPUPlace>(const platform::CPUPlace &place) {
GetCPUBuddyAllocator()->Release();
}

template <>
size_t Used<platform::CPUPlace>(const platform::CPUPlace &place) {
return GetCPUBuddyAllocator()->Used();
Expand Down Expand Up @@ -186,6 +194,17 @@ void Free<platform::XPUPlace>(const platform::XPUPlace &place, void *p,
#endif
}

template <>
void Release<platform::XPUPlace>(const platform::XPUPlace &place) {
#ifdef PADDLE_WITH_XPU
PADDLE_THROW(
platform::errors::PermissionDenied("Release XPU pool is not supported."));
#else
PADDLE_THROW(
platform::errors::PermissionDenied("'XPUPlace' is not supported."));
#endif
}

template <>
size_t Used<platform::XPUPlace>(const platform::XPUPlace &place) {
#ifdef PADDLE_WITH_XPU
Expand Down Expand Up @@ -313,6 +332,16 @@ void Free<platform::CUDAPlace>(const platform::CUDAPlace &place, void *p,
#endif
}

template <>
void Release<platform::CUDAPlace>(const platform::CUDAPlace &place) {
#ifdef PADDLE_WITH_CUDA
GetGPUBuddyAllocator(place.device)->Release();
#else
PADDLE_THROW(platform::errors::PermissionDenied(
"'CUDAPlace' is not supported in CPU only device."));
#endif
}

#ifdef PADDLE_WITH_CUDA
BuddyAllocator *GetCUDAPinnedBuddyAllocator() {
static std::once_flag init_flag;
Expand Down Expand Up @@ -371,6 +400,17 @@ void Free<platform::CUDAPinnedPlace>(const platform::CUDAPinnedPlace &place,
#endif
}

template <>
void Release<platform::CUDAPinnedPlace>(
const platform::CUDAPinnedPlace &place) {
#ifdef PADDLE_WITH_CUDA
GetCUDAPinnedBuddyAllocator()->Release();
#else
PADDLE_THROW(platform::errors::PermissionDenied(
"'CUDAPinnedPlace' is not supported in CPU only device."));
#endif
}

struct AllocVisitor : public boost::static_visitor<void *> {
inline explicit AllocVisitor(size_t size) : size_(size) {}

Expand All @@ -397,6 +437,13 @@ struct FreeVisitor : public boost::static_visitor<void> {
size_t size_;
};

struct ReleaseVisitor : public boost::static_visitor<void> {
template <typename Place>
inline void operator()(const Place &place) const {
Release<Place>(place);
}
};

size_t Usage::operator()(const platform::CPUPlace &cpu) const {
return Used(cpu);
}
Expand Down Expand Up @@ -439,6 +486,10 @@ void NaiveBestFitAllocator::FreeImpl(Allocation *allocation) {
delete allocation;
}

void NaiveBestFitAllocator::ReleaseImpl(const platform::Place &place) {
boost::apply_visitor(legacy::ReleaseVisitor(), place);
}

} // namespace allocation
} // namespace memory
} // namespace paddle
1 change: 1 addition & 0 deletions paddle/fluid/memory/allocation/naive_best_fit_allocator.h
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ class NaiveBestFitAllocator : public Allocator {
protected:
Allocation *AllocateImpl(size_t size) override;
void FreeImpl(Allocation *allocation) override;
void ReleaseImpl(const platform::Place &place) override;

private:
platform::Place place_;
Expand Down
74 changes: 74 additions & 0 deletions paddle/fluid/memory/allocation/naive_best_fit_allocator_test.cc
Original file line number Diff line number Diff line change
@@ -0,0 +1,74 @@
// Copyright (c) 2020 PaddlePaddle Authors. All Rights Reserved.
//
// 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 "paddle/fluid/memory/allocation/naive_best_fit_allocator.h"

#include <algorithm>
#include <chrono> // NOLINT
#include <condition_variable> // NOLINT
#include <mutex> // NOLINT
#include <string>
#include <thread> // NOLINT
#include <vector>

#include "gtest/gtest.h"

namespace paddle {
namespace memory {
namespace allocation {

TEST(NaiveBestFitAllocatorTest, CpuAlloc) {
NaiveBestFitAllocator alloc{platform::CPUPlace()};
{
size_t size = (1 << 20);
auto allocation = alloc.Allocate(size);
}
alloc.Release(platform::CPUPlace());

size_t size = (1 << 20);
auto allocation = alloc.Allocate(size);
alloc.Release(platform::CPUPlace());
}

#ifdef PADDLE_WITH_CUDA
TEST(NaiveBestFitAllocatorTest, GpuAlloc) {
NaiveBestFitAllocator alloc{platform::CUDAPlace(0)};
{
size_t size = (1 << 20);
auto allocation = alloc.Allocate(size);
}
alloc.Release(platform::CUDAPlace(0));

size_t size = (1 << 20);
auto allocation = alloc.Allocate(size);
alloc.Release(platform::CUDAPlace(0));
}

TEST(NaiveBestFitAllocatorTest, CudaPinnedAlloc) {
NaiveBestFitAllocator alloc{platform::CUDAPinnedPlace()};
{
size_t size = (1 << 20);
auto allocation = alloc.Allocate(size);
}
alloc.Release(platform::CUDAPinnedPlace());

size_t size = (1 << 20);
auto allocation = alloc.Allocate(size);
alloc.Release(platform::CUDAPinnedPlace());
}
#endif

} // namespace allocation
} // namespace memory
} // namespace paddle
3 changes: 3 additions & 0 deletions paddle/fluid/memory/allocation/retry_allocator.h
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,9 @@ class RetryAllocator : public Allocator {
protected:
void FreeImpl(Allocation* allocation) override;
Allocation* AllocateImpl(size_t size) override;
void ReleaseImpl(const platform::Place& place) override {
underlying_allocator_->Release(place);
}

private:
std::shared_ptr<Allocator> underlying_allocator_;
Expand Down
2 changes: 2 additions & 0 deletions paddle/fluid/memory/allocation/retry_allocator_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -96,6 +96,7 @@ TEST(RetryAllocator, RetryAllocator) {
bool is_all_equal = std::all_of(addresses.begin(), addresses.end(),
[val](void *p) { return p == val; });
ASSERT_TRUE(is_all_equal);
allocator->Release(platform::CPUPlace());
}
}

Expand Down Expand Up @@ -135,6 +136,7 @@ TEST(RetryAllocator, RetryAllocatorLastAllocFailure) {
auto allocation = allocator.Allocate(allocate_size);
ASSERT_TRUE(false);
allocation.reset();
allocator.Release(p);
} catch (BadAlloc &ex) {
ASSERT_TRUE(std::string(ex.what()).find("Cannot allocate") !=
std::string::npos);
Expand Down
2 changes: 2 additions & 0 deletions paddle/fluid/memory/allocation/thread_local_allocator.cc
Original file line number Diff line number Diff line change
Expand Up @@ -72,6 +72,8 @@ void ThreadLocalAllocatorImpl::FreeImpl(ThreadLocalAllocation* allocation) {
delete allocation;
}

void ThreadLocalAllocatorImpl::ReleaseImpl() { buddy_allocator_->Release(); }

} // namespace allocation
} // namespace memory
} // namespace paddle
4 changes: 4 additions & 0 deletions paddle/fluid/memory/allocation/thread_local_allocator.h
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@ class ThreadLocalAllocatorImpl
explicit ThreadLocalAllocatorImpl(const platform::Place& p);
ThreadLocalAllocation* AllocateImpl(size_t size);
void FreeImpl(ThreadLocalAllocation* allocation);
void ReleaseImpl();

private:
std::unique_ptr<memory::detail::BuddyAllocator> buddy_allocator_;
Expand Down Expand Up @@ -91,6 +92,9 @@ class ThreadLocalCUDAAllocator : public Allocator {
auto allocator_impl = tl_allocation->GetAllocator();
allocator_impl->FreeImpl(tl_allocation);
}
void ReleaseImpl(const platform::Place& p) override {
return ThreadLocalCUDAAllocatorPool::Instance().Get(gpu_id_)->ReleaseImpl();
}

private:
int gpu_id_;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,7 @@ TEST(ThreadLocalAllocator, cross_scope_release) {
auto tl_allocator_impl =
ThreadLocalCUDAAllocatorPool::Instance().Get(devices[j]);
allocator_addresses[j][i] = tl_allocator_impl.get();
memory::Release(platform::CUDAPlace(devices[j]));
}
});
}
Expand Down
41 changes: 39 additions & 2 deletions paddle/fluid/memory/detail/buddy_allocator.cc
Original file line number Diff line number Diff line change
Expand Up @@ -39,9 +39,10 @@ BuddyAllocator::~BuddyAllocator() {
while (!pool_.empty()) {
auto block = static_cast<MemoryBlock*>(std::get<2>(*pool_.begin()));
auto desc = cache_.LoadDesc(block);
VLOG(10) << "Free from block (" << block << ", " << desc->get_size() << ")";
VLOG(10) << "Free from block (" << block << ", " << desc->get_total_size()
<< ")";

system_allocator_->Free(block, desc->get_size(), desc->get_index());
system_allocator_->Free(block, desc->get_total_size(), desc->get_index());
cache_.Invalidate(block);
pool_.erase(pool_.begin());
}
Expand Down Expand Up @@ -161,6 +162,39 @@ void BuddyAllocator::Free(void* p) {
IndexSizeAddress(desc->get_index(), desc->get_total_size(), block));
}

void BuddyAllocator::Release() {
std::lock_guard<std::mutex> lock(mutex_);
int num = 0;
uint64_t bytes = 0;
bool del_flag = false;
for (auto iter = pool_.begin(); iter != pool_.end();) {
auto remain_size = std::get<1>(*iter);
auto remain_ptr = std::get<2>(*iter);
for (auto& chunk : chunks_) {
auto init_size = std::get<1>(chunk);
auto init_ptr = std::get<2>(chunk);

if (init_size == remain_size && init_ptr == remain_ptr) {
++num;
bytes += init_size;
total_free_ -= init_size;
auto block = static_cast<MemoryBlock*>(std::get<2>(chunk));
system_allocator_->Free(init_ptr, init_size, std::get<0>(chunk));
cache_.Invalidate(block);
del_flag = true;
break;
}
}

if (del_flag) {
iter = pool_.erase(iter);
} else {
iter++;
}
}
VLOG(10) << "Release " << num << " chunk, Free " << bytes << " bytes.";
}

size_t BuddyAllocator::Used() { return total_used_; }
size_t BuddyAllocator::GetMinChunkSize() { return min_chunk_size_; }
size_t BuddyAllocator::GetMaxChunkSize() { return max_chunk_size_; }
Expand Down Expand Up @@ -213,6 +247,9 @@ BuddyAllocator::PoolSet::iterator BuddyAllocator::RefillPool(

total_free_ += allocate_bytes;

// record the chunk.
chunks_.insert(IndexSizeAddress(index, allocate_bytes, p));

// dump the block into pool
return pool_.insert(IndexSizeAddress(index, allocate_bytes, p)).first;
}
Expand Down
7 changes: 7 additions & 0 deletions paddle/fluid/memory/detail/buddy_allocator.h
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,8 @@ class BuddyAllocator {
public:
void* Alloc(size_t unaligned_size);
void Free(void* ptr);
// Release the unused memory pool, a real free operation for the OS.
void Release();
size_t Used();
size_t GetMinChunkSize();
size_t GetMaxChunkSize();
Expand Down Expand Up @@ -92,6 +94,11 @@ class BuddyAllocator {
*/
PoolSet pool_;

/**
* \brief Record the allocated chunks when Refill pool.
*/
PoolSet chunks_;

private:
/*! Unify the metadata format between GPU and CPU allocations */
MetadataCache cache_;
Expand Down
Loading

0 comments on commit 0511469

Please sign in to comment.