Split TensorFlow type-related methods out of Allocator.

This CL reduces the dependencies of tensorflow::Allocator, by removing methods related to TensorFlow-specific element types (ResourceHandle, Variant) out of the Allocator base class and into a TypedAllocator utility class.

In addition, this CL makes the following related changes to tidy up the Allocator interface and implementation:

* Allocator::ShouldAllocateEmptyTensors() becomes Allocator::AllocatesOpaqueHandles(), and the result of this method is used (instead of virtual dispatch) to determine whether or not to run the complex type constructors/destructors when allocating/deleting a Tensor buffer.

* The virtual Allocator::Run{String,Resource,Variant}{Ctor,Dtor}() methods are removed, and their dynamic logic moved to TypedAllocator.

* AllocationAttributes is separated out into its own header.

* A method forwarding bug in AllocatorWrapper is fixed.

* A few unused methods are deleted and IWYU-related errors are fixed.

PiperOrigin-RevId: 247210317
This commit is contained in:
Derek Murray 2019-05-08 07:38:22 -07:00 committed by TensorFlower Gardener
parent 66b193faee
commit 61c8837163
14 changed files with 241 additions and 182 deletions

View File

@ -58,18 +58,13 @@ class XlaCompilationAllocator : public Allocator {
// Make sure that even tensors with 0 elements have allocated
// buffers, so they get ids to track.
bool ShouldAllocateEmptyTensors() const override { return true; }
private:
// Don't run any constructors or destructors for complex objects,
// since there is no backing store for the tensor to run them
// on. strings are the only complex objects currently stored in
// Tensors. If others are added, this set of overrides must be
// extended to include them.
void RunStringCtor(string* p, size_t n) override {}
void RunStringDtor(string* p, size_t n) override {}
void RunResourceCtor(ResourceHandle* p, size_t n) override {}
void RunResourceDtor(ResourceHandle* p, size_t n) override {}
//
// NOTE: It is the caller's responsibility to track whether an allocated
// object is a buffer or an opaque handle. In particular, when this allocator
// is used, the caller must not run any constructors or destructors for
// complex objects, since there is no backing store for the tensor in which to
// place their outputs.
bool AllocatesOpaqueHandle() const override { return true; }
};
XlaCompilationDevice::XlaCompilationDevice(const SessionOptions& options,

View File

@ -944,6 +944,7 @@ tf_cuda_library(
"framework/tracking_allocator.h",
"framework/type_index.h",
"framework/type_traits.h",
"framework/typed_allocator.h",
"framework/types.h",
"public/version.h",
"util/activation_mode.h",

View File

@ -24,6 +24,7 @@ limitations under the License.
#include "tensorflow/core/common_runtime/gpu/gpu_id.h"
#include "tensorflow/core/common_runtime/gpu/gpu_id_utils.h"
#include "tensorflow/core/common_runtime/gpu/gpu_init.h"
#include "tensorflow/core/framework/typed_allocator.h"
#include "tensorflow/core/lib/core/threadpool.h"
#include "tensorflow/core/lib/gtl/inlined_vector.h"
#include "tensorflow/core/lib/random/simple_philox.h"
@ -153,18 +154,18 @@ TEST(GPUBFCAllocatorTest, ExerciseCoalescing) {
GPUBFCAllocator a(sub_allocator, 1 << 30, "GPU_0_bfc");
CheckStats(&a, 0, 0, 0, 0);
float* first_ptr = a.Allocate<float>(1024);
float* first_ptr = TypedAllocator::Allocate<float>(&a, 1024, {});
a.DeallocateRaw(first_ptr);
CheckStats(&a, 1, 0, 4096, 4096);
for (int i = 0; i < 1024; ++i) {
// Allocate several buffers of different sizes, and then clean them
// all up. We should be able to repeat this endlessly without
// causing fragmentation and growth.
float* t1 = a.Allocate<float>(1024);
float* t1 = TypedAllocator::Allocate<float>(&a, 1024, {});
int64* t2 = a.Allocate<int64>(1048576);
double* t3 = a.Allocate<double>(2048);
float* t4 = a.Allocate<float>(10485760);
int64* t2 = TypedAllocator::Allocate<int64>(&a, 1048576, {});
double* t3 = TypedAllocator::Allocate<double>(&a, 2048, {});
float* t4 = TypedAllocator::Allocate<float>(&a, 10485760, {});
a.DeallocateRaw(t1);
a.DeallocateRaw(t2);
@ -179,7 +180,7 @@ TEST(GPUBFCAllocatorTest, ExerciseCoalescing) {
// At the end, we should have coalesced all memory into one region
// starting at the beginning, so validate that allocating a pointer
// starts from this region.
float* first_ptr_after = a.Allocate<float>(1024);
float* first_ptr_after = TypedAllocator::Allocate<float>(&a, 1024, {});
EXPECT_EQ(first_ptr, first_ptr_after);
a.DeallocateRaw(first_ptr_after);
}
@ -190,7 +191,7 @@ TEST(GPUBFCAllocatorTest, AllocateZeroBufSize) {
GpuIdUtil::ExecutorForPlatformGpuId(platform_gpu_id).ValueOrDie(),
platform_gpu_id, false /*use_unified_memory*/, {}, {});
GPUBFCAllocator a(sub_allocator, 1 << 30, "GPU_0_bfc");
float* ptr = a.Allocate<float>(0);
float* ptr = TypedAllocator::Allocate<float>(&a, 0, {});
EXPECT_EQ(nullptr, ptr);
}
@ -209,7 +210,7 @@ TEST(GPUBFCAllocatorTest, AllocatedVsRequested) {
GpuIdUtil::ExecutorForPlatformGpuId(platform_gpu_id).ValueOrDie(),
platform_gpu_id, false /*use_unified_memory*/, {}, {});
GPUBFCAllocator a(sub_allocator, 1 << 30, "GPU_0_bfc");
float* t1 = a.Allocate<float>(1);
float* t1 = TypedAllocator::Allocate<float>(&a, 1, {});
EXPECT_EQ(4, a.RequestedSize(t1));
EXPECT_EQ(256, a.AllocatedSize(t1));
a.DeallocateRaw(t1);
@ -223,8 +224,8 @@ TEST(GPUBFCAllocatorTest, TestCustomMemoryLimit) {
// Configure a 1MiB byte limit
GPUBFCAllocator a(sub_allocator, 1 << 20, "GPU_0_bfc");
float* first_ptr = a.Allocate<float>(1 << 6);
float* second_ptr = a.Allocate<float>(1 << 20);
float* first_ptr = TypedAllocator::Allocate<float>(&a, 1 << 6, {});
float* second_ptr = TypedAllocator::Allocate<float>(&a, 1 << 20, {});
EXPECT_NE(nullptr, first_ptr);
EXPECT_EQ(nullptr, second_ptr);

View File

@ -25,6 +25,7 @@ limitations under the License.
#include "tensorflow/core/common_runtime/gpu/gpu_id.h"
#include "tensorflow/core/common_runtime/gpu/gpu_id_utils.h"
#include "tensorflow/core/common_runtime/gpu/gpu_init.h"
#include "tensorflow/core/framework/typed_allocator.h"
#include "tensorflow/core/lib/gtl/inlined_vector.h"
#include "tensorflow/core/platform/logging.h"
#include "tensorflow/core/platform/stream_executor.h"
@ -47,7 +48,8 @@ TEST(GPUDebugAllocatorTest, OverwriteDetection_None) {
for (int s : {8}) {
std::vector<int64> cpu_array(s);
memset(&cpu_array[0], 0, cpu_array.size() * sizeof(int64));
int64* gpu_array = a.Allocate<int64>(cpu_array.size());
int64* gpu_array =
TypedAllocator::Allocate<int64>(&a, cpu_array.size(), {});
se::DeviceMemory<int64> gpu_array_ptr{se::DeviceMemoryBase{gpu_array}};
ASSERT_TRUE(stream_exec->SynchronousMemcpy(&gpu_array_ptr, &cpu_array[0],
s * sizeof(int64)));
@ -74,7 +76,8 @@ TEST(GPUDebugAllocatorTest, OverwriteDetection_Header) {
std::vector<int64> cpu_array(s);
memset(&cpu_array[0], 0, cpu_array.size() * sizeof(int64));
int64* gpu_array = a.Allocate<int64>(cpu_array.size());
int64* gpu_array =
TypedAllocator::Allocate<int64>(&a, cpu_array.size(), {});
se::DeviceMemory<int64> gpu_array_ptr{
se::DeviceMemoryBase{gpu_array}};
@ -110,7 +113,8 @@ TEST(GPUDebugAllocatorTest, OverwriteDetection_Footer) {
std::vector<int64> cpu_array(s);
memset(&cpu_array[0], 0, cpu_array.size() * sizeof(int64));
int64* gpu_array = a.Allocate<int64>(cpu_array.size());
int64* gpu_array =
TypedAllocator::Allocate<int64>(&a, cpu_array.size(), {});
se::DeviceMemory<int64> gpu_array_ptr{
se::DeviceMemoryBase{gpu_array}};
@ -145,7 +149,7 @@ TEST(GPUDebugAllocatorTest, ResetToNan) {
std::vector<float> cpu_array_result(1024);
// Allocate 1024 floats
float* gpu_array = a.Allocate<float>(cpu_array.size());
float* gpu_array = TypedAllocator::Allocate<float>(&a, cpu_array.size(), {});
se::DeviceMemory<float> gpu_array_ptr{se::DeviceMemoryBase{gpu_array}};
ASSERT_TRUE(stream_exec->SynchronousMemcpy(&cpu_array[0], gpu_array_ptr,
cpu_array.size() * sizeof(float)));
@ -192,7 +196,7 @@ TEST(GPUDebugAllocatorTest, ResetToNanWithHeaderFooter) {
std::vector<float> cpu_array_result(1024);
// Allocate 1024 floats
float* gpu_array = a.Allocate<float>(cpu_array.size());
float* gpu_array = TypedAllocator::Allocate<float>(&a, cpu_array.size(), {});
se::DeviceMemory<float> gpu_array_ptr{se::DeviceMemoryBase{gpu_array}};
ASSERT_TRUE(stream_exec->SynchronousMemcpy(&cpu_array[0], gpu_array_ptr,
cpu_array.size() * sizeof(float)));
@ -241,7 +245,7 @@ TEST(GPUDebugAllocatorTest, AllocatedVsRequested) {
new GPUDebugAllocator(new GPUBFCAllocator(sub_allocator, 1 << 30, ""),
platform_gpu_id),
platform_gpu_id);
float* t1 = a.Allocate<float>(1);
float* t1 = TypedAllocator::Allocate<float>(&a, 1, {});
EXPECT_EQ(4, a.RequestedSize(t1));
EXPECT_EQ(256, a.AllocatedSize(t1));
a.DeallocateRaw(t1);

View File

@ -151,7 +151,8 @@ void GPUUtil::SetProtoFromGPU(const Tensor& tensor, Device* dev,
if (total_bytes > 0) {
tracing::ScopedAnnotation annotation("SetProtoFromGPU");
alloc = GPUProcessState::singleton()->GetGpuHostAllocator(0);
buf = alloc->Allocate<char>(total_bytes);
buf = static_cast<char*>(
alloc->AllocateRaw(Allocator::kAllocatorAlignment, total_bytes));
if (LogMemory::IsEnabled()) {
LogMemory::RecordRawAllocation("SetProtoFromGPU",
LogMemory::PROTO_BUFFER_STEP_ID,
@ -178,7 +179,7 @@ void GPUUtil::SetProtoFromGPU(const Tensor& tensor, Device* dev,
LogMemory::PROTO_BUFFER_STEP_ID,
buf, alloc, false);
}
alloc->Deallocate<char>(buf, total_bytes);
alloc->DeallocateRaw(buf);
}
done(Status::OK());
});

View File

@ -106,7 +106,6 @@ class ScopedAllocatorInstance : public Allocator {
}
void DeallocateRaw(void* p) LOCKS_EXCLUDED(mu_) override;
bool TracksAllocationSizes() const override { return false; }
bool ShouldAllocateEmptyTensors() const override { return false; }
size_t RequestedSize(const void* ptr) const override { return 0; }
size_t AllocatedSize(const void* ptr) const override { return 0; }
int64 AllocationId(const void* ptr) const override { return 0; }

View File

@ -23,6 +23,7 @@ limitations under the License.
#include "tensorflow/core/lib/core/errors.h"
#include "tensorflow/core/lib/core/status.h"
#include "tensorflow/core/lib/gtl/flatmap.h"
#include "tensorflow/core/platform/mutex.h"
// gRPC response caching. Most WorkerService methods cannot be retried directly
// as they will fail or deadlock. To enable retrying, we can instead cache

View File

@ -18,7 +18,6 @@ limitations under the License.
#include "tensorflow/core/framework/allocator_registry.h"
#include "tensorflow/core/framework/log_memory.h"
#include "tensorflow/core/framework/tracking_allocator.h"
#include "tensorflow/core/framework/variant.h"
#include "tensorflow/core/lib/strings/stringprintf.h"
#include "tensorflow/core/platform/mem.h"
#include "tensorflow/core/platform/mutex.h"
@ -41,22 +40,6 @@ constexpr size_t Allocator::kAllocatorAlignment;
Allocator::~Allocator() {}
void RunResourceCtor(ResourceHandle* p, size_t n) {
for (size_t i = 0; i < n; ++p, ++i) new (p) ResourceHandle();
}
void RunResourceDtor(ResourceHandle* p, size_t n) {
for (size_t i = 0; i < n; ++p, ++i) p->~ResourceHandle();
}
void Allocator::RunVariantCtor(Variant* p, size_t n) {
for (size_t i = 0; i < n; ++p, ++i) new (p) Variant();
}
void Allocator::RunVariantDtor(Variant* p, size_t n) {
for (size_t i = 0; i < n; ++p, ++i) p->~Variant();
}
// If true, cpu allocator collects more stats.
static bool cpu_allocator_collect_stats = false;
// If true, cpu allocator collects full stats.

View File

@ -18,23 +18,20 @@ limitations under the License.
#include <stdlib.h>
#include <functional>
#include <limits>
#include "absl/strings/string_view.h"
#include "absl/types/optional.h"
#include "tensorflow/core/framework/numeric_types.h"
#include "tensorflow/core/framework/resource_handle.h"
#include "tensorflow/core/framework/type_traits.h"
#include "tensorflow/core/platform/logging.h"
#include "tensorflow/core/platform/macros.h"
#include "tensorflow/core/platform/mutex.h"
#include "tensorflow/core/platform/numa.h"
#include "tensorflow/core/platform/types.h"
namespace tensorflow {
class Variant;
// Attributes for a single allocation call. Different calls to the same
// allocator could potentially have different allocation attributes.
struct AllocationAttributes {
@ -129,51 +126,25 @@ class Allocator {
// REQUIRES: "ptr" was previously returned by a call to AllocateRaw
virtual void DeallocateRaw(void* ptr) = 0;
// Convenience functions to do typed allocation. C++ constructors
// and destructors are invoked for complex types if necessary,
// depending on the concrete Allocator implementation. May return
// NULL if the tensor has too many elements to represent in a single
// allocation.
template <typename T>
T* Allocate(size_t num_elements) {
return Allocate<T>(num_elements, AllocationAttributes());
}
template <typename T>
T* Allocate(size_t num_elements,
const AllocationAttributes& allocation_attr) {
// TODO(jeff): Do we need to allow clients to pass in alignment
// requirements?
if (num_elements > (std::numeric_limits<size_t>::max() / sizeof(T))) {
return NULL;
}
void* p = AllocateRaw(kAllocatorAlignment, sizeof(T) * num_elements,
allocation_attr);
T* typed_p = reinterpret_cast<T*>(p);
if (typed_p) RunCtor<T>(typed_p, num_elements);
return typed_p;
}
template <typename T>
void Deallocate(T* ptr, size_t num_elements) {
if (ptr) {
RunDtor<T>(ptr, num_elements);
DeallocateRaw(ptr);
}
}
// Returns true if this allocator tracks the sizes of allocations.
// RequestedSize and AllocatedSize must be overridden if
// TracksAllocationSizes is overridden to return true.
virtual bool TracksAllocationSizes() const { return false; }
// Returns true if this allocator requires tensors with 0 elements
// to allocate buffers. This is false for most allocators, but may
// be used by special-case allocators that want to track tensor
// usage.
virtual bool ShouldAllocateEmptyTensors() const { return false; }
// Returns true if this allocator allocates an opaque handle rather than the
// requested number of bytes.
//
// This method returns false for most allocators, but may be used by
// special-case allocators that track tensor usage. If this method returns
// true, AllocateRaw() should be invoked for all values of `num_bytes`,
// including 0.
//
// NOTE: It is the caller's responsibility to track whether an allocated
// object is a buffer or an opaque handle. In particular, when this method
// returns `true`, users of this allocator must not run any constructors or
// destructors for complex objects, since there is no backing store for the
// tensor in which to place their outputs.
virtual bool AllocatesOpaqueHandle() const { return false; }
// Returns the user-requested size of the data allocated at
// 'ptr'. Note that the actual buffer allocated might be larger
@ -232,80 +203,8 @@ class Allocator {
virtual void ClearStats() {}
virtual void SetSafeFrontier(uint64 count) {}
private:
// No constructors or destructors are run for simple types
template <typename T>
void RunCtor(T* p, size_t n) {
static_assert(is_simple_type<T>::value, "T is not a simple type.");
}
template <typename T>
void RunDtor(T* p, size_t n) {}
// custom constructors and destructors that can be overridden for
// non-standard allocators
// Runs string's default constructor for p[0], p[1], ..., p[n-1].
virtual void RunStringCtor(string* p, size_t n) {
for (size_t i = 0; i < n; ++p, ++i) new (p) string();
}
// Runs string's default destructor for p[0], p[1], ..., p[n-1].
virtual void RunStringDtor(string* p, size_t n) {
for (size_t i = 0; i < n; ++p, ++i) p->~string();
}
virtual void RunResourceCtor(ResourceHandle* p, size_t n) {
for (size_t i = 0; i < n; ++p, ++i) new (p) ResourceHandle();
}
// Runs string's default destructor for p[0], p[1], ..., p[n-1].
virtual void RunResourceDtor(ResourceHandle* p, size_t n) {
for (size_t i = 0; i < n; ++p, ++i) p->~ResourceHandle();
}
virtual void RunVariantCtor(Variant* p, size_t n);
virtual void RunVariantDtor(Variant* p, size_t n);
// TODO(jeff): Maybe provide some interface to give info about
// current allocation state (total number of bytes available for
// allocation, number of bytes free on device, etc.)
};
// Allocator-specific constructors and destructors are used for
// strings
template <>
inline void Allocator::RunCtor(string* p, size_t n) {
RunStringCtor(p, n);
}
template <>
inline void Allocator::RunDtor(string* p, size_t n) {
RunStringDtor(p, n);
}
template <>
inline void Allocator::RunCtor(ResourceHandle* p, size_t n) {
RunResourceCtor(p, n);
}
template <>
inline void Allocator::RunDtor(ResourceHandle* p, size_t n) {
RunResourceDtor(p, n);
}
template <>
inline void Allocator::RunCtor(Variant* p, size_t n) {
RunVariantCtor(p, n);
}
template <>
inline void Allocator::RunDtor(Variant* p, size_t n) {
RunVariantDtor(p, n);
}
// An implementation of Allocator that delegates all calls to another Allocator.
//
// Useful to clients who want to override part of the functionality of another
@ -336,8 +235,8 @@ class AllocatorWrapper : public Allocator {
return wrapped_->TracksAllocationSizes();
}
bool ShouldAllocateEmptyTensors() const override {
return wrapped_->TracksAllocationSizes();
bool AllocatesOpaqueHandle() const override {
return wrapped_->AllocatesOpaqueHandle();
}
size_t RequestedSize(const void* ptr) const override {

View File

@ -18,6 +18,7 @@ limitations under the License.
#include <algorithm>
#include <vector>
#include "tensorflow/core/framework/typed_allocator.h"
#include "tensorflow/core/platform/logging.h"
#include "tensorflow/core/platform/test.h"
#include "tensorflow/core/platform/test_benchmark.h"
@ -102,14 +103,14 @@ TEST(CPUAllocatorTest, Simple) {
a->DeallocateRaw(ptrs[i]);
}
CheckStats(a, 1023, 0, 552640, 1024);
float* t1 = a->Allocate<float>(1024);
double* t2 = a->Allocate<double>(1048576);
float* t1 = TypedAllocator::Allocate<float>(a, 1024, {});
double* t2 = TypedAllocator::Allocate<double>(a, 1048576, {});
CheckStats(a, 1025, 1048576 * sizeof(double) + 1024 * sizeof(float),
1048576 * sizeof(double) + 1024 * sizeof(float),
1048576 * sizeof(double));
a->Deallocate(t1, 1024);
a->Deallocate(t2, 1048576);
TypedAllocator::Deallocate(a, t1, 1024);
TypedAllocator::Deallocate(a, t2, 1048576);
CheckStats(a, 1025, 0, 1048576 * sizeof(double) + 1024 * sizeof(float),
1048576 * sizeof(double));
@ -130,7 +131,8 @@ TEST(CPUAllocatorTest, AllocateOverflowMaxSizeT) {
// The maximum size_t value will definitely overflow.
size_t count_to_allocate = std::numeric_limits<size_t>::max();
TestStruct* const test_pointer = a->Allocate<TestStruct>(count_to_allocate);
TestStruct* const test_pointer =
TypedAllocator::Allocate<TestStruct>(a, count_to_allocate, {});
CHECK_EQ(test_pointer, reinterpret_cast<TestStruct*>(NULL));
}
@ -141,7 +143,8 @@ TEST(CPUAllocatorTest, AllocateOverflowSmallest) {
// count_to_allocate is the smallest count that will cause overflow.
const size_t count_to_allocate =
(std::numeric_limits<size_t>::max() / sizeof(TestStruct)) + 1;
TestStruct* const test_pointer = a->Allocate<TestStruct>(count_to_allocate);
TestStruct* const test_pointer =
TypedAllocator::Allocate<TestStruct>(a, count_to_allocate, {});
CHECK_EQ(test_pointer, reinterpret_cast<TestStruct*>(NULL));
}

View File

@ -35,6 +35,7 @@ limitations under the License.
#include "tensorflow/core/framework/tensor.pb.h"
#include "tensorflow/core/framework/tensor_description.pb.h"
#include "tensorflow/core/framework/type_traits.h"
#include "tensorflow/core/framework/typed_allocator.h"
#include "tensorflow/core/framework/types.h"
#include "tensorflow/core/framework/variant.h"
#include "tensorflow/core/framework/variant_encode_decode.h"
@ -443,12 +444,14 @@ struct ProtoHelper<Eigen::half> {
template <typename T>
Buffer<T>::Buffer(Allocator* a, int64 n)
: BufferBase(a, a->Allocate<T>(n)), elem_(n) {}
: BufferBase(a, TypedAllocator::Allocate<T>(a, n, AllocationAttributes())),
elem_(n) {}
template <typename T>
Buffer<T>::Buffer(Allocator* a, int64 n,
const AllocationAttributes& allocation_attr)
: BufferBase(a, a->Allocate<T>(n, allocation_attr)), elem_(n) {}
: BufferBase(a, TypedAllocator::Allocate<T>(a, n, allocation_attr)),
elem_(n) {}
template <typename T>
Buffer<T>::~Buffer() {
@ -456,7 +459,7 @@ Buffer<T>::~Buffer() {
if (LogMemory::IsEnabled()) {
RecordDeallocation();
}
alloc_->Deallocate<T>(static_cast<T*>(data()), elem_);
TypedAllocator::Deallocate<T>(alloc_, static_cast<T*>(data()), elem_);
}
}
@ -734,7 +737,7 @@ Tensor::Tensor(Allocator* a, DataType type, const TensorShape& shape)
: shape_(shape), buf_(nullptr) {
set_dtype(type);
CHECK_NOTNULL(a);
if (shape_.num_elements() > 0 || a->ShouldAllocateEmptyTensors()) {
if (shape_.num_elements() > 0 || a->AllocatesOpaqueHandle()) {
CASES(type, buf_ = new Buffer<T>(a, shape.num_elements()));
}
if (buf_ != nullptr && buf_->data() != nullptr && LogMemory::IsEnabled()) {
@ -748,7 +751,7 @@ Tensor::Tensor(Allocator* a, DataType type, const TensorShape& shape,
: shape_(shape), buf_(nullptr) {
set_dtype(type);
CHECK_NOTNULL(a);
if (shape_.num_elements() > 0 || a->ShouldAllocateEmptyTensors()) {
if (shape_.num_elements() > 0 || a->AllocatesOpaqueHandle()) {
CASES(type, buf_ = new Buffer<T>(a, shape.num_elements(), allocation_attr));
}
if (!allocation_attr.allocation_will_be_logged && buf_ != nullptr &&

View File

@ -0,0 +1,32 @@
/* Copyright 2015 The TensorFlow 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 "tensorflow/core/framework/typed_allocator.h"
#include "tensorflow/core/framework/variant.h"
namespace tensorflow {
/* static */
void TypedAllocator::RunVariantCtor(Variant* p, size_t n) {
for (size_t i = 0; i < n; ++p, ++i) new (p) Variant();
}
/* static */
void TypedAllocator::RunVariantDtor(Variant* p, size_t n) {
for (size_t i = 0; i < n; ++p, ++i) p->~Variant();
}
} // namespace tensorflow

View File

@ -0,0 +1,133 @@
/* Copyright 2015 The TensorFlow 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.
==============================================================================*/
#ifndef TENSORFLOW_CORE_FRAMEWORK_TYPED_ALLOCATOR_H_
#define TENSORFLOW_CORE_FRAMEWORK_TYPED_ALLOCATOR_H_
#include <limits>
#include "tensorflow/core/framework/allocator.h"
#include "tensorflow/core/framework/resource_handle.h"
#include "tensorflow/core/platform/types.h"
namespace tensorflow {
class Variant;
// Convenience functions to do typed allocation. C++ constructors
// and destructors are invoked for complex types if necessary.
class TypedAllocator {
public:
// May return NULL if the tensor has too many elements to represent in a
// single allocation.
template <typename T>
static T* Allocate(Allocator* raw_allocator, size_t num_elements,
const AllocationAttributes& allocation_attr) {
// TODO(jeff): Do we need to allow clients to pass in alignment
// requirements?
if (num_elements > (std::numeric_limits<size_t>::max() / sizeof(T))) {
return nullptr;
}
void* p =
raw_allocator->AllocateRaw(Allocator::kAllocatorAlignment,
sizeof(T) * num_elements, allocation_attr);
T* typed_p = reinterpret_cast<T*>(p);
if (typed_p) RunCtor<T>(raw_allocator, typed_p, num_elements);
return typed_p;
}
template <typename T>
static void Deallocate(Allocator* raw_allocator, T* ptr,
size_t num_elements) {
if (ptr) {
RunDtor<T>(raw_allocator, ptr, num_elements);
raw_allocator->DeallocateRaw(ptr);
}
}
private:
// No constructors or destructors are run for simple types
template <typename T>
static void RunCtor(Allocator* raw_allocator, T* p, size_t n) {
static_assert(is_simple_type<T>::value, "T is not a simple type.");
}
template <typename T>
static void RunDtor(Allocator* raw_allocator, T* p, size_t n) {}
static void RunVariantCtor(Variant* p, size_t n);
static void RunVariantDtor(Variant* p, size_t n);
};
template <>
/* static */
inline void TypedAllocator::RunCtor(Allocator* raw_allocator, string* p,
size_t n) {
if (!raw_allocator->AllocatesOpaqueHandle()) {
for (size_t i = 0; i < n; ++p, ++i) new (p) string();
}
}
template <>
/* static */
inline void TypedAllocator::RunDtor(Allocator* raw_allocator, string* p,
size_t n) {
if (!raw_allocator->AllocatesOpaqueHandle()) {
for (size_t i = 0; i < n; ++p, ++i) p->~string();
}
}
template <>
/* static */
inline void TypedAllocator::RunCtor(Allocator* raw_allocator, ResourceHandle* p,
size_t n) {
if (!raw_allocator->AllocatesOpaqueHandle()) {
for (size_t i = 0; i < n; ++p, ++i) new (p) ResourceHandle();
}
}
template <>
/* static */
inline void TypedAllocator::RunDtor(Allocator* raw_allocator, ResourceHandle* p,
size_t n) {
if (!raw_allocator->AllocatesOpaqueHandle()) {
for (size_t i = 0; i < n; ++p, ++i) p->~ResourceHandle();
}
}
template <>
/* static */
inline void TypedAllocator::RunCtor(Allocator* raw_allocator, Variant* p,
size_t n) {
if (!raw_allocator->AllocatesOpaqueHandle()) {
RunVariantCtor(p, n);
}
}
template <>
/* static */
inline void TypedAllocator::RunDtor(Allocator* raw_allocator, Variant* p,
size_t n) {
if (!raw_allocator->AllocatesOpaqueHandle()) {
RunVariantDtor(p, n);
}
}
} // namespace tensorflow
#endif // TENSORFLOW_CORE_FRAMEWORK_TYPED_ALLOCATOR_H_

View File

@ -15,11 +15,12 @@ limitations under the License.
#include "tensorflow/lite/delegates/flex/buffer_map.h"
#include "tensorflow/c/c_api_internal.h"
#include "tensorflow/core/framework/allocation_description.pb.h"
#include "tensorflow/core/framework/log_memory.h"
#include "tensorflow/core/framework/typed_allocator.h"
#include "tensorflow/lite/delegates/flex/util.h"
#include "tensorflow/lite/string.h"
#include "tensorflow/lite/string_util.h"
#include "tensorflow/core/framework/allocation_description.pb.h"
#include "tensorflow/core/framework/log_memory.h"
namespace tflite {
namespace flex {
@ -99,8 +100,9 @@ class StringTfLiteTensorBuffer : public BaseTfLiteTensorBuffer {
~StringTfLiteTensorBuffer() override {
LogDeallocation();
tensorflow::cpu_allocator()->Deallocate<string>(
static_cast<string*>(data()), num_strings_);
tensorflow::TypedAllocator::Deallocate<tensorflow::string>(
tensorflow::cpu_allocator(), static_cast<tensorflow::string*>(data()),
num_strings_);
}
size_t size() const override { return num_strings_ * sizeof(string); }
@ -109,7 +111,9 @@ class StringTfLiteTensorBuffer : public BaseTfLiteTensorBuffer {
StringTfLiteTensorBuffer(const TfLiteTensor* tensor, int num_strings)
: BaseTfLiteTensorBuffer(
num_strings != 0
? tensorflow::cpu_allocator()->Allocate<string>(num_strings)
? tensorflow::TypedAllocator::Allocate<tensorflow::string>(
tensorflow::cpu_allocator(), num_strings,
tensorflow::AllocationAttributes())
: nullptr),
num_strings_(num_strings) {
LogAllocation();