Compute Library
 24.07
How to Add a New Operator

Adding new operators

Introduction

In Compute Library there are two main parts or modules:

  • The core library consists of a low-level collection of algorithms implemented in C++ and optimized for Arm CPUs and GPUs. The core module is designed to be embedded in other projects and it doesn't perform any memory management or scheduling.
  • The runtime library is a wrapper of the core library and provides other additional features like memory management, multithreaded execution of workloads and allocation of the intermediate tensors.

The library can be integrated in an existing external library or application that provides its own scheduler or a specific memory manager. In that case, the right solution is to use only the core library which means that the user must also manage all the memory allocation not only for the input/output tensor but also for the intermediate tensors/variables necessary. On the other hand, if the user doesn't want to care about allocation and multithreading then the right choice is to use the functions from the runtime library.

Apart from these components that get linked into the application, the sources also include the validation test suite and the C++ reference implementations against which all the operators are validated.

Supporting new operators

Following are the steps involved in adding support for a new operator in Compute Library

  • Add new data types (if required)
  • Add the kernel to the core library.
  • Add the function to the runtime library.
  • Add validation tests.
    • Add the reference implementation.
    • Add the fixture
    • register the tests.

Adding new data types

Compute Library declares a few new datatypes related to its domain, kernels, and functions in the library process Tensors and Images (Computer Vision functions). Tensors are multi-dimensional arrays with a maximum of Coordinates::num_max_dimensions dimensions; depending on the number of dimensions tensors can be interpreted as various objects. A scalar can be represented as a zero-dimensional tensor and a vector of numbers can be represented as a one-dimensional tensor. Furthermore, an image is just a 2D tensor, a 3D tensor can be seen as an array of images and a 4D tensor as a 2D array of images, etc. All the datatype classes or structures are grouped in the core library folder arm_compute/core like the ITensor, ITensorInfo (all the information of a tensor), TensorShape and simpler types are in arm_compute/core/CoreTypes.h.

If an operator handles a new datatype, it must be added to the library. While adding a new data type to the library, it's necessary to implement the function to enable printing, the to_string() method and the output stream insertion (<<) operator. Every datatype implements these two functions in utils/TypePrinter.h

A quick example, in CoreTypes.h we add:

/** Supported tensor data layouts */
enum class DataLayout
{
UNKNOWN, /**< Unknown data layout */
NCHW, /**< Num samples, channels, height, width */
NHWC, /**< Num samples, height, width, channels */
NCDHW, /**< Num samples, channels, depth, height, width */
NDHWC /**< Num samples, depth, height, width, channels */
};

And for printing:

/** Formatted output of the DataLayout type.
*
* @param[out] os Output stream.
* @param[in] data_layout Type to output.
*
* @return Modified output stream.
*/
inline ::std::ostream &operator<<(::std::ostream &os, const DataLayout &data_layout)
{
switch (data_layout)
{
os << "UNKNOWN";
break;
os << "NHWC";
break;
os << "NCHW";
break;
os << "NDHWC";
break;
os << "NCDHW";
break;
default:
ARM_COMPUTE_ERROR("NOT_SUPPORTED!");
}
return os;
}
/** Formatted output of the DataLayout type.
*
* @param[in] data_layout Type to output.
*
* @return Formatted string.
*/
inline std::string to_string(const arm_compute::DataLayout &data_layout)
{
std::stringstream str;
return str.str();
}

In Compute Library, we use namespaces to group all the operators, functions, classes and interfaces. The main namespace to use is arm_compute. In the test suite, the test framework and the individual tests use nested namespaces like test::validation or test::benchmark to group the different purposes of various parts of the suite. Utility functions like conversion or type cast operators, that are shared by multiple operators are in arm_compute/core/Utils.h. Non-inlined function definitions go in the corresponding .cpp files in the src folder. Similarly, all common functions that process shapes, like calculating output shapes of an operator or shape conversions etc are in arm_compute/core/utils/misc/ShapeCalculator.h.

Add a kernel

As we mentioned at the beginning, the kernel is the implementation of the operator or algorithm partially using a specific programming language related to the backend we want to use. Adding a kernel in the library means implementing the algorithm in a SIMD technology like Arm® Neon™ or OpenCL. All kernels in Compute Library must implement a common interface IKernel or one of the specific subinterfaces. IKernel is the common interface for all the kernels in the core library, it contains the main methods for configure and run the kernel itself, such as window() that return the maximum window the kernel can be executed on or is_parallelisable() for indicate whether or not the kernel is parallelizable. If the kernel is parallelizable then the window returned by the window() method can be split into sub-windows which can then be run in parallel, in the other case, only the window returned by window() can be passed to the run method. There are specific interfaces for OpenCL and Neon™: ICLKernel, INEKernel (using INEKernel = ICPPKernel).

  • ICLKernel is the common interface for all the OpenCL kernels. It implements the inherited methods and adds all the methods necessary to configure the CL kernel, such as set/return the Local-Workgroup-Size hint, add single, array or tensor argument, set the targeted GPU architecture according to the CL device. All these methods are used during the configuration and the run of the operator.
  • INEKernel inherits from IKernel as well and it's the common interface for all kernels implemented in Neon™, it adds just the run and the name methods.

There are two others implementation of IKernel called ICLSimpleKernel and INESimpleKernel, they are the interface for simple kernels that have just one input tensor and one output tensor. Creating a new kernel implies adding new files:

  • src/core/CL/kernels/CLReshapeLayerKernel.h
  • src/core/CL/cl_kernels/reshape_layer.cl
  • src/core/CL/kernels/CLReshapeLayerKernel.cpp
  • src/core/CL/CLKernelLibrary.cpp

Neon™ kernel

  • arm_compute/core/NEON/kernels/NEReshapeLayerKernel.h
  • src/core/NEON/kernels/NEReshapeLayerKernel.cpp

We must register the new layer in the respective libraries:

These files contain the list of all kernels available in the corresponding Compute Library's backend, for example CLKernels:

...
#include "src/core/CL/kernels/CLMinMaxLayerKernel.h"
#include "src/core/CL/kernels/CLMinMaxLocationKernel.h"
...
#include "src/core/CL/kernels/CLReshapeLayerKernel.h"
...

For OpenCL we need to update the CLKernelLibrary.cpp, adding the appropriate code to embed the .cl kernel in the library. The OpenCL code can be compiled offline and embed in the library as binary. The essential operation we want to do with a kernel will be

  • create the kernel object
  • initialize the kernel with the input/output and any other parameters that may be required
  • retrieve the execution window of the kernel and run the whole kernel window in the current thread or use the multithreading.

Each kernel will have to implement the method:

  • validate: is a static function that checks if the given info will lead to a valid configuration of the kernel.
  • configure: configure the kernel, its window, accessor, valid region, etc for the given set of tensors and other parameters.
  • run: execute the kernel in the window

The structure of the kernel .cpp file should be similar to the next ones. For OpenCL:

namespace arm_compute
{
namespace opencl
{
namespace kernels
{
namespace
{
Status validate_arguments(const ITensorInfo *src, const ITensorInfo *dst)
{
if (dst->tensor_shape().total_size() != 0)
{
ARM_COMPUTE_RETURN_ERROR_ON(src->tensor_shape().total_size() != dst->tensor_shape().total_size());
}
return Status{};
}
} // namespace
{
}
void ClReshapeKernel::configure(const CLCompileContext &compile_context, const ITensorInfo *src, ITensorInfo *dst)
{
auto padding_info = get_padding_info({src, dst});
// Create kernel
std::set<std::string> build_opts = {"-DDATA_TYPE=" + get_cl_unsigned_type_from_element_size(src->element_size())};
_kernel = create_kernel(compile_context, "reshape_layer", build_opts);
// Add static arguments
const cl_int2 src_shape = {
{static_cast<cl_int>(src->tensor_shape()[0]), static_cast<cl_int>(src->tensor_shape()[1])}};
const cl_int2 dst_shape = {
{static_cast<cl_int>(dst->tensor_shape()[0]), static_cast<cl_int>(dst->tensor_shape()[1])}};
unsigned int idx = 2 * num_arguments_per_3D_tensor(); // Skip the src and dst parameters
_kernel.setArg<cl_int2>(idx++, src_shape);
_kernel.setArg<cl_int2>(idx++, dst_shape);
// Configure kernel window
Window win = calculate_max_window(*dst);
ICLKernel::configure_internal(win);
}
Status ClReshapeKernel::validate(const ITensorInfo *src, const ITensorInfo *dst)
{
return Status{};
}
void ClReshapeKernel::run_op(ITensorPack &tensors, const Window &window, cl::CommandQueue &queue)
{
Window slice = window_collapsed.first_slice_window_3D();
const auto src =
utils::cast::polymorphic_downcast<const ICLTensor *>(tensors.get_const_tensor(TensorType::ACL_SRC));
auto dst = utils::cast::polymorphic_downcast<ICLTensor *>(tensors.get_tensor(TensorType::ACL_DST));
// Set srcs
unsigned int idx = 0;
add_3D_tensor_argument(idx, src, window_collapsed);
add_3D_tensor_argument(idx, dst, window_collapsed);
enqueue(queue, *this, slice, lws_hint());
}
} // namespace kernels
} // namespace opencl
} // namespace arm_compute

The run will call the function defined in the .cl file.

For the Arm® Neon™ backend case:

namespace arm_compute
{
namespace cpu
{
namespace kernels
{
namespace
{
Status validate_arguments(const ITensorInfo *src, const ITensorInfo *dst)
{
// Note: ARM_COMPUTE_RETURN_ERROR_ON_CPU_F16_UNSUPPORTED(src) is not needed here as this kernel doesn't use CPU FP16 instructions.
if (dst->tensor_shape().total_size() != 0)
{
ARM_COMPUTE_RETURN_ERROR_ON(src->tensor_shape().total_size() != dst->tensor_shape().total_size());
}
return Status{};
}
template <typename T>
void reshape_tensor_per_element(const Window &window, const ITensor *src, ITensor *dst)
{
const TensorShape &src_shape = src->info()->tensor_shape();
const TensorShape &dst_shape = dst->info()->tensor_shape();
Iterator dst_it(dst, window);
window,
[&](const Coordinates &dst_coord)
{
Coordinates src_coord = index2coords(src_shape, coords2index(dst_shape, dst_coord));
const auto output_ptr = dst->ptr_to_element(dst_coord);
const auto input_ptr = src->ptr_to_element(src_coord);
*reinterpret_cast<T *>(output_ptr) = *reinterpret_cast<T *>(input_ptr);
},
dst_it);
}
void reshape_tensor_per_element_selector(const Window &window, const ITensor *src, ITensor *dst)
{
switch (src->info()->data_type())
{
reshape_tensor_per_element<uint8_t>(window, src, dst);
break;
reshape_tensor_per_element<uint16_t>(window, src, dst);
break;
reshape_tensor_per_element<uint32_t>(window, src, dst);
break;
reshape_tensor_per_element<uint64_t>(window, src, dst);
break;
default:
ARM_COMPUTE_ERROR("Unsupported data type!");
}
}
void reshape_tensor_per_row(const Window &window, const ITensor *src, ITensor *dst)
{
const TensorShape &src_shape = src->info()->tensor_shape();
const TensorShape &dst_shape = dst->info()->tensor_shape();
Coordinates src_coord{};
Coordinates dst_coord{};
const auto element_size = dst->info()->element_size();
const auto window_start_x = static_cast<int>(window.x().start());
const auto window_end_x = static_cast<int>(window.x().end());
const auto src_row_size = static_cast<int>(src_shape[0]);
const auto row_size_in_bytes = src_row_size * element_size;
auto output_ptr = dst->ptr_to_element(dst_coord);
auto input_ptr = src->ptr_to_element(src_coord);
Window win = window;
win.set(Window::DimX, Window::Dimension(0, 1, 1));
Iterator dst_it(dst, win);
win,
[&](Coordinates &id)
{
dst_coord = id;
for (int x = window_start_x; x < window_end_x; x += src_row_size)
{
src_coord = index2coords(src_shape, coords2index(dst_shape, dst_coord));
output_ptr = dst->ptr_to_element(dst_coord);
input_ptr = src->ptr_to_element(src_coord);
std::memcpy(output_ptr, input_ptr, row_size_in_bytes);
dst_coord.increment(Window::DimX, src_row_size);
}
},
dst_it);
}
void reshape_tensor_per_window(const Window &window, const ITensor *src, ITensor *dst)
{
Iterator src_it(src, window);
Iterator dst_it(dst, window);
const size_t element_size = dst->info()->element_size();
const auto window_size = window.x().end() - window.x().start();
const auto window_size_in_bytes = window_size * element_size;
const auto input_ptr = src_it.ptr();
const auto output_ptr = dst_it.ptr();
std::memcpy(output_ptr, input_ptr, window_size_in_bytes);
}
} // namespace
void CpuReshapeKernel::configure(const ITensorInfo *src, ITensorInfo *dst)
{
_reshape_tensor_fn = reshape_tensor_per_element_selector;
// Configure kernel window
Window win = calculate_max_window(*dst);
ICpuKernel::configure(win);
}
Status CpuReshapeKernel::validate(const ITensorInfo *src, const ITensorInfo *dst)
{
return Status{};
}
void CpuReshapeKernel::run_op(ITensorPack &tensors, const Window &window, const ThreadInfo &info)
{
const auto src = tensors.get_const_tensor(TensorType::ACL_SRC);
auto dst = tensors.get_tensor(TensorType::ACL_DST);
_reshape_tensor_fn(window, src, dst);
}
const char *CpuReshapeKernel::name() const
{
return "CpuReshapeKernel";
}
size_t CpuReshapeKernel::get_mws(const CPUInfo &platform, size_t thread_count) const
{
ARM_COMPUTE_UNUSED(thread_count);
ARM_COMPUTE_UNUSED(platform);
}
void CpuReshapeKernel::prepare(ITensorPack &tensors)
{
const auto src = tensors.get_const_tensor(TensorType::ACL_SRC);
auto dst = tensors.get_tensor(TensorType::ACL_DST);
const ITensorInfo *src_info = src->info();
const ITensorInfo *dst_info = dst->info();
// Calculate kernel window based on the padding info
Window win;
const bool src_has_holes = has_holes(*src_info, src_info->num_dimensions() - 1);
const bool dst_has_holes = has_holes(*dst_info, dst_info->num_dimensions() - 1);
const bool src_has_holes_in_x = has_holes(*src_info, Window::DimX);
const bool dst_has_holes_in_x = has_holes(*dst_info, Window::DimX);
const auto src_row_size = static_cast<int>(src_info->tensor_shape()[0]);
const auto dst_row_size = static_cast<int>(dst_info->tensor_shape()[0]);
if (!src_has_holes && !dst_has_holes)
{
std::tie(win, _split_dimension) = calculate_squashed_or_max_window(*dst_info);
/*
Copy the tensor per window. If the src and dst tensors
are contiguous memory allocations without any holes or
padding, then the tensor is squashed to 1D window and
we can use use a single memcopy call to copy the whole
window in reshape_tensor_per_window fn
*/
_reshape_tensor_fn = reshape_tensor_per_window;
}
else
{
win = calculate_max_window(*dst_info);
/*
Copy tensor row by row if src and dst have no holes in X
dim and they have the same number of elements in their rows
*/
if (!src_has_holes_in_x && !dst_has_holes_in_x && (src_row_size == dst_row_size))
{
_reshape_tensor_fn = reshape_tensor_per_row;
}
else
{
/*
Fall back to the element wise copy
*/
_reshape_tensor_fn = reshape_tensor_per_element_selector;
}
}
ICPPKernel::configure(win);
}
} // namespace kernels
} // namespace cpu
} // namespace arm_compute

In the Arm® Neon™ case, there is no need to add an extra file and we implement the kernel in the same NEReshapeLayerKernel.cpp file. If the tests are already in place, the new kernel can be tested using the existing tests by adding the configure and run of the kernel to the compute_target() in the fixture.

Add a function

Memory management and scheduling the underlying kernel(s) must be handled by the function implementation. A kernel class must support window() API which return the execute window for the configuration that the kernel is configured for. A window specifies the dimensions of a workload. It has a start and end on each of the dimension. A maximum of Coordinates::num_max_dimensions is supported. The run time layer is expected to query the kernel for the window size and schedule the window as it sees fit. It could choose to split the window into sub windows so that it could be run in parallel. The split must adhere to the following rules

  • max[n].start() <= sub[n].start() < max[n].end()
  • sub[n].start() < sub[n].end() <= max[n].end()
  • max[n].step() == sub[n].step()
  • (sub[n].start() - max[n].start()) % max[n].step() == 0
  • (sub[n].end() - sub[n].start()) % max[n].step() == 0

CPPScheduler::schedule provides a sample implementation that is used for Arm® Neon™ kernels. Memory management is the other aspect that the runtime layer is supposed to handle. Memory management of the tensors is abstracted using TensorAllocator. Each tensor holds a pointer to a TensorAllocator object, which is used to allocate and free the memory at runtime. The implementation that is currently supported in Compute Library allows memory blocks, required to be fulfilled for a given operator, to be grouped together under a MemoryGroup. Each group can be acquired and released. The underlying implementation of memory groups vary depending on whether Arm® Neon™ or CL is used. The memory group class uses memory pool to provide the required memory. It also uses the memory manager to manage the lifetime and a IPoolManager to manage the memory pools registered with the memory manager.

We have seen the various interfaces for a kernel in the core library, the same structure the same file structure design exists in the runtime module. IFunction is the base class for all the functions, it has two child interfaces: ICLSimpleFunction and INESimpleFunction that are used as base class for functions which call a single kernel.

The new operator has to implement validate(), configure() and run(), these methods will call the respective function in the kernel considering that the multi-threading is used for the kernels which are parallelizable, by default std::thread::hardware_concurrency() threads are used. For Arm® Neon™ function can be used CPPScheduler::set_num_threads() to manually set the number of threads, whereas for OpenCL kernels all the kernels are enqueued on the queue associated with CLScheduler and the queue is then flushed. For the runtime functions, there is an extra method implemented: prepare(), this method prepares the function for the run, it does all the heavy operations that are done only once (reshape the weight, release the memory not necessary after the reshape, etc). The prepare method can be called standalone or in the first run, if not called before, after then the function will be marked as prepared. The files we add are:

OpenCL function

Neon™ function

As we did in the kernel we have to edit the runtime libraries to register the new operator modifying the relative library file:

For the special case where the new function calls only one kernel, we could use as base class ICLSimpleFunction or INESimpleFunction. The configure and the validate methods will simply call the corresponding functions. The structure will be:

namespace arm_compute
{
struct CLReshapeLayer::Impl
{
const ICLTensor *src{nullptr};
ICLTensor *dst{nullptr};
std::unique_ptr<opencl::ClReshape> op{nullptr};
};
CLReshapeLayer::CLReshapeLayer() : _impl(std::make_unique<Impl>())
{
}
CLReshapeLayer::CLReshapeLayer(CLReshapeLayer &&) = default;
CLReshapeLayer &CLReshapeLayer::operator=(CLReshapeLayer &&) = default;
void CLReshapeLayer::configure(const ICLTensor *input, ICLTensor *output)
{
configure(CLKernelLibrary::get().get_compile_context(), input, output);
}
void CLReshapeLayer::configure(const CLCompileContext &compile_context, const ICLTensor *input, ICLTensor *output)
{
_impl->src = input;
_impl->dst = output;
_impl->op = std::make_unique<opencl::ClReshape>();
_impl->op->configure(compile_context, input->info(), output->info());
}
Status CLReshapeLayer::validate(const ITensorInfo *input, const ITensorInfo *output)
{
return Status{};
}
{
ITensorPack pack;
_impl->op->run(pack);
}
} // namespace arm_compute

If the function is more complicated and calls more than one kernel we have to use the memory manager to manage the intermediate tensors; in the configure() method we call the manage() function passing the tensor to keep track, in the run method we will have to acquire all the buffer managed and released at the end. For OpenCL if we want to add two tensor input and reshape the result:

using namespace arm_compute;
CLAddReshapeLayer:: CLAddReshapeLayer(std::shared_ptr<IMemoryManager> memory_manager)
: _memory_group(std::move(memory_manager))
{
}
void CLAddReshapeLayer::configure(const ICLTensor *input1, const ICLTensor *input2, ICLTensor *output)
{
// Allocate memory
add_output.allocator()->init(info);
// Manage intermediate buffers
memory_group.manage(&_addOutput);
// Initialise kernel
_add_kernel.configure(input1, input2, &add_output);
_reshape_kernel.configure(&add_output, output);
// Allocate intermediate tensors
add_output.allocator()->allocate();
}
Status CLAddReshapeLayer::validate(const ITensorInfo *input1, const ITensorInfo *input2, const ITensorInfo *output)
{
TensorInfo add_output();
return Status{};
}
{
memory_group.acquire();
// Run Add
add_kernel.run();
// Run Reshape
CLScheduler::get().enqueue(reshape_kernel);
memory_group.release();
}

For Neon™:

using namespace arm_compute;
NEAddReshapeLayer:: NEAddReshapeLayer (std::shared_ptr<IMemoryManager> memory_manager)
: _memory_group(std::move(memory_manager))
{
}
void NEAddReshapeLayer::configure(const ITensor *input1, const ITensor *input2, ITensor *output)
{
// Allocate memory
add_output.allocator()->init(info);
// Manage intermediate buffers
memory_group.manage(&_addOutput);
// Initialise kernel
add_kernel.configure(input1, input2, &addOutput);
reshape_kernel.configure(&addOutput, output);
// Allocate intermediate tensors
add_output.allocator()->allocate();
}
{
memory_group.acquire();
// Run Add
add_kernel.run();
// Run Reshape
NEScheduler::get().schedule(_reshape_kernel.get(), Window::DimY);
memory_group.release();
}

At this point, everything is in place at the library level. If you are following an tests driven implementation and all the tests are already in place, we can call the function configuration in the fixture and remove any redundant code like the allocation of the intermediate tensors since it's done in the function. Run the final tests to check the results match with the expected results from the reference implementation.

Add validation artifacts

Add the reference implementation and the tests

As mentioned in the introduction, the reference implementation is a pure C++ implementation without any optimization or backend specific instruction. The reference implementation consist of two files into the folder tests/validation/reference:

where we will put respectively the declaration and definition of the new operator. All the utility functions that are used ONLY in the tests are in test/validation/helpers.h, for all the others, as mentioned before, there are helpers in the library. Compute Library and the tests do use templates, the reference implementation is a generic implementation independent from the datatype and we use the templates to generalize the datatype concept. Following the example, let's have a look at the ReshapeLayer operator:

template <typename T>
SimpleTensor<T> reshape_layer(const SimpleTensor<T> &src, const TensorShape &output_shape);
template <typename T>
SimpleTensor<T> reshape_layer(const SimpleTensor<T> &src, const TensorShape &output_shape)
{
ARM_COMPUTE_ERROR_ON(src.shape().total_size() != output_shape.total_size());
SimpleTensor<T> dst(output_shape, src.data_type());
std::copy_n(src.data(), src.num_elements(), dst.data());
return dst;
}
template SimpleTensor<uint8_t> reshape_layer(const SimpleTensor<uint8_t> &src, const TensorShape &output_shape);
template SimpleTensor<int8_t> reshape_layer(const SimpleTensor<int8_t> &src, const TensorShape &output_shape);
template SimpleTensor<uint16_t> reshape_layer(const SimpleTensor<uint16_t> &src, const TensorShape &output_shape);
template SimpleTensor<int16_t> reshape_layer(const SimpleTensor<int16_t> &src, const TensorShape &output_shape);
template SimpleTensor<uint32_t> reshape_layer(const SimpleTensor<uint32_t> &src, const TensorShape &output_shape);
template SimpleTensor<int32_t> reshape_layer(const SimpleTensor<int32_t> &src, const TensorShape &output_shape);
template SimpleTensor<half> reshape_layer(const SimpleTensor<half> &src, const TensorShape &output_shape);
template SimpleTensor<float> reshape_layer(const SimpleTensor<float> &src, const TensorShape &output_shape);
template SimpleTensor<bfloat16> reshape_layer(const SimpleTensor<bfloat16> &src, const TensorShape &output_shape);

An explicit instantiation of the template for the required datatypes must be added in the .cpp file.

Add dataset

One of the parameters of the tests is the dataset, it will be used to generate versions of the test case with different inputs. To pass the dataset at the fixture data test case we have three cases

  • the operator dataset is simple so it can be added directly in the test case data declaration
  • we can create a class that return tuples at the test framework
class PoolingTypes final : public framework::dataset::ContainerDataset<std::vector<PoolingType>>
{
public:
PoolingTypes()
: ContainerDataset("PoolType",
{
})
{
}
};
  • if we want to create dynamically the dataset combining different parameter, we can create the dataset using iterators. For example, dataset for ReshapeLayer:
class ReshapeLayerDataset
{
public:
using type = std::tuple<TensorShape, TensorShape>;
struct iterator
{
iterator(std::vector<TensorShape>::const_iterator in_it, std::vector<TensorShape>::const_iterator out_it)
: _in_it{ std::move(in_it) }, _out_it{ std::move(out_it) }
{
}
std::string description() const
{
std::stringstream description;
description << "In=" << *_in_it << ":";
description << "Out=" << *_out_it;
return description.str();
}
{
return std::make_tuple(*_in_it, *_out_it);
}
iterator &operator++()
{
++_in_it;
++_out_it;
return *this;
}
private:
std::vector<TensorShape>::const_iterator _in_it;
std::vector<TensorShape>::const_iterator _out_it;
};
iterator begin() const
{
return iterator(_in_shapes.begin(), _out_shapes.begin());
}
int size() const
{
return std::min(_in_shapes.size(), _out_shapes.size());
}
void add_config(TensorShape in, TensorShape out)
{
_in_shapes.emplace_back(std::move(in));
_out_shapes.emplace_back(std::move(out));
}
protected:
ReshapeLayerDataset() = default;
ReshapeLayerDataset(ReshapeLayerDataset &&) = default;
private:
std::vector<TensorShape> _in_shapes{};
std::vector<TensorShape> _out_shapes{};
};

Add a fixture and a data test case

Benchmark and validation tests are based on the same framework to setup and run the tests. In addition to running simple, self-contained test functions the framework supports fixtures and data test cases. Fixtures can be used to share common setup, teardown or even run tasks among multiple test cases, for that purpose a fixture can define a "setup", "teardown" and "run" method. Adding tests for the new operator in the runtime library we need to implement at least the setup method, that is used to call two methods for configure, run and return the output respectively of the target (CL or Neon™) and the reference (C++ implementation).

For example let's have a look at Reshape Layer Fixture :

template <typename TensorType, typename AccessorType, typename FunctionType, typename T>
class ReshapeLayerGenericValidationFixture : public framework::Fixture
{
public:
void setup(TensorShape input_shape, TensorShape output_shape, DataType data_type, bool add_x_padding = false)
{
_target = compute_target(input_shape, output_shape, data_type, add_x_padding);
_reference = compute_reference(input_shape, output_shape, data_type);
}
protected:
template <typename U>
void fill(U &&tensor, int i)
{
library->fill_tensor_uniform(tensor, i);
}
TensorType compute_target(const TensorShape &input_shape, const TensorShape &output_shape, DataType data_type, bool add_x_padding = false)
{
// Check if indeed the input shape can be reshape to the output one
// Create tensors
TensorType src = create_tensor<TensorType>(input_shape, data_type);
TensorType dst = create_tensor<TensorType>(output_shape, data_type);
// Create and configure function
FunctionType reshape;
reshape.configure(&src, &dst);
ARM_COMPUTE_ASSERT(src.info()->is_resizable());
ARM_COMPUTE_ASSERT(dst.info()->is_resizable());
if(add_x_padding)
{
// Add random padding in x dimension
}
// Allocate tensors
src.allocator()->allocate();
dst.allocator()->allocate();
ARM_COMPUTE_ASSERT(!src.info()->is_resizable());
ARM_COMPUTE_ASSERT(!dst.info()->is_resizable());
// Fill tensors
fill(AccessorType(src), 0);
// Compute function
reshape.run();
return dst;
}
SimpleTensor<T> compute_reference(const TensorShape &input_shape, const TensorShape &output_shape, DataType data_type)
{
// Create reference
SimpleTensor<T> src{ input_shape, data_type };
// Fill reference
fill(src, 0);
return reference::reshape_layer<T>(src, output_shape);
}
TensorType _target{};
SimpleTensor<T> _reference{};
};
template <typename TensorType, typename AccessorType, typename FunctionType, typename T>
class ReshapeLayerValidationFixture : public ReshapeLayerGenericValidationFixture<TensorType, AccessorType, FunctionType, T>
{
public:
void setup(TensorShape input_shape, TensorShape output_shape, DataType data_type)
{
ReshapeLayerGenericValidationFixture<TensorType, AccessorType, FunctionType, T>::setup(input_shape, output_shape, data_type);
}
};
template <typename TensorType, typename AccessorType, typename FunctionType, typename T>
class ReshapeLayerPaddedValidationFixture : public ReshapeLayerGenericValidationFixture<TensorType, AccessorType, FunctionType, T>
{
public:
void setup(TensorShape input_shape, TensorShape output_shape, DataType data_type)
{
ReshapeLayerGenericValidationFixture<TensorType, AccessorType, FunctionType, T>::setup(input_shape, output_shape, data_type, true /* add_x_padding */);
}
};

In the fixture class above we can see that the setup method computes the target and reference and store them in the two members _target and _reference which will be used later to check for correctness. The compute_target method reflects the exact behavior expected when we call a function. The input and output tensor must be declared, function configured, tensors allocated, the input tensor filled with required data, and finally, the function must be run and the results returned. This fixture is used in the test case, that is a parameterized test case that inherits from a fixture. The test case will have access to all public and protected members of the fixture. Only the setup and teardown methods of the fixture will be used. The setup method of the fixture needs to be a template and must accept inputs from the dataset as arguments. The body of this function will be used as a test function. For the fixture test case the first argument is the name of the test case (has to be unique within the enclosing test suite), the second argument is the class name of the fixture, the third argument is the dataset mode in which the test will be active (PRECOMMIT or NIGHTLY) and the fourth argument is the dataset. For example:

template <typename T>
using CLActivationLayerFixture = ActivationValidationFixture<CLTensor, CLAccessor, CLActivationLayer, T>;
TEST_SUITE(ActivationLayer)
TEST_SUITE(Float)
FIXTURE_DATA_TEST_CASE(RunSmall, CLActivationLayerFixture<half>, framework::DatasetMode::ALL, combine(combine(datasets::SmallShapes(), ActivationDataset),
{
// Validate output
validate(CLAccessor(_target), _reference, tolerance(_function, _data_type));
}

This will produce a set of tests that can be filtered with "CL/ReshapeLayer/Float/FP16/RunSmall". Each test produced from the cartesian product of the dataset is associated to a number and can be filtered specifying all the parameters.

arm_compute::DataLayout::NCHW
@ NCHW
Num samples, channels, height, width.
arm_compute::test::validation::dst_shape
TensorShape dst_shape
Definition: DFT.cpp:164
arm_compute::DataType::QSYMM8_PER_CHANNEL
@ QSYMM8_PER_CHANNEL
quantized, symmetric per channel fixed-point 8-bit number
arm_compute::opencl::kernels::ClReshapeKernel::run_op
void run_op(ITensorPack &tensors, const Window &window, cl::CommandQueue &queue) override
Enqueue the OpenCL kernel to process the given window on the passed OpenCL command queue.
Definition: ClReshapeKernel.cpp:105
arm_compute::test::validation::configure
im2col_func configure(src_target.info(), dst_target.info(), spatial_kernel, conv_info, has_bias)
arm_compute::DataType::U64
@ U64
unsigned 64-bit number
arm_compute::test::validation::run
lstmq run()
arm_compute::test::validation::src
SimpleTensor< float > src
Definition: DFT.cpp:155
type
decltype(strategy::transforms) typedef type
Definition: gemm_interleaved.hpp:347
arm_compute::PoolingType::L2
@ L2
L2 Pooling.
arm_compute::CLReshapeLayer::operator=
CLReshapeLayer & operator=(const CLReshapeLayer &)=delete
Prevent instances of this class from being copied (As this class contains pointers)
arm_compute::test::validation::add_padding_x
void add_padding_x(std::initializer_list< ITensor * > tensors, const DataLayout &data_layout, bool only_right_pad)
Add random padding along the X axis (between 1 and 16 columns per side) to all the input tensors.
Definition: Helpers.cpp:331
arm_compute::DataLayout
DataLayout
[DataLayout enum definition]
Definition: CoreTypes.h:110
arm_compute::DataType::F64
@ F64
64-bit floating-point number
arm_compute::calculate_max_window
Window calculate_max_window(const ValidRegion &valid_region, const Steps &steps, bool skip_border, BorderSize border_size)
Definition: WindowHelpers.cpp:29
arm_compute::cpu::kernels::CpuReshapeKernel::get_mws
size_t get_mws(const CPUInfo &platform, size_t thread_count) const override
Return minimum workload size of the relevant kernel.
Definition: CpuReshapeKernel.cpp:208
arm_compute::test::validation::output_shape
TensorShape output_shape
Definition: LSTMLayerQuantized.cpp:469
arm_compute::DataLayout::NCDHW
@ NCDHW
Num samples, channels, depth, height, width.
arm_compute::CLScheduler::enqueue
void enqueue(ICLKernel &kernel, bool flush=true)
Schedule the execution of the passed kernel if possible.
Definition: CLScheduler.cpp:232
arm_compute::DataLayout::NHWC
@ NHWC
Num samples, height, width, channels.
arm_compute::opencl::kernels::ClReshapeKernel::ClReshapeKernel
ClReshapeKernel()
Definition: ClReshapeKernel.cpp:66
arm_compute::DataType::QASYMM8
@ QASYMM8
quantized, asymmetric fixed-point 8-bit number unsigned
arm_compute::DataType::U16
@ U16
unsigned 16-bit number
arm_compute::test::validation::dst
auto dst
Definition: DFT.cpp:170
arm_compute::cpu::kernels::validate_arguments
Status validate_arguments(const ITensorInfo *src, const ITensorInfo *weights, const ITensorInfo *dst, const PadStrideInfo &conv_info)
Definition: CpuDirectConv2dKernel.cpp:57
ARM_COMPUTE_ERROR_ON_UNCONFIGURED_KERNEL
#define ARM_COMPUTE_ERROR_ON_UNCONFIGURED_KERNEL(k)
Definition: Validate.h:1079
arm_compute::DataType::QSYMM8
@ QSYMM8
quantized, symmetric fixed-point 8-bit number
arm_compute::Window::DimX
static constexpr size_t DimX
Alias for dimension 0 also known as X dimension.
Definition: Window.h:43
arm_compute::cpu::kernels::CpuReshapeKernel::validate
static Status validate(const ITensorInfo *src, const ITensorInfo *dst)
Static function to check if given info will lead to a valid configuration.
Definition: CpuReshapeKernel.cpp:186
arm_compute::Window::collapse_if_possible
Window collapse_if_possible(const Window &full_window, size_t first, size_t last, bool *has_collapsed=nullptr) const
Collapse the dimensions between first and last if possible.
Definition: Window.inl:72
arm_compute::ICLTensor
Interface for OpenCL tensor.
Definition: ICLTensor.h:41
ARM_COMPUTE_ERROR
#define ARM_COMPUTE_ERROR(msg)
Print the given message then throw an std::runtime_error.
Definition: Error.h:354
arm_compute::TensorType
TensorType
Memory type.
Definition: Types.h:38
arm_compute::index2coords
Coordinates index2coords(const TensorShape &shape, int index)
Convert a linear index into n-dimensional coordinates.
Definition: Helpers.inl:164
arm_compute::ITensor
Interface for CPU tensor.
Definition: ITensor.h:36
arm_compute::ITensorPack::add_tensor
void add_tensor(int id, ITensor *tensor)
Add tensor to the pack.
Definition: ITensorPack.cpp:38
caffe_mnist_image_extractor.str
str
Definition: caffe_mnist_image_extractor.py:21
arm_compute::test::validation::CLActivationLayerFixture
ActivationValidationFixture< CLTensor, CLAccessor, CLActivationLayer, T > CLActivationLayerFixture
[CLActivationLayerFixture snippet]
Definition: ActivationLayer.cpp:142
arm_compute::CLReshapeLayer::run
void run() override
Run the kernels contained in the function.
Definition: CLReshapeLayer.cpp:73
arm_compute::DataType::S8
@ S8
signed 8-bit number
arm_compute::CLKernelLibrary::get
static CLKernelLibrary & get()
Access the KernelLibrary singleton.
Definition: CLKernelLibrary.cpp:41
ARM_COMPUTE_RETURN_ERROR_ON_MISMATCHING_DATA_TYPES
#define ARM_COMPUTE_RETURN_ERROR_ON_MISMATCHING_DATA_TYPES(...)
Definition: Validate.h:677
arm_compute::CPUInfo
Definition: CPPTypes.h:66
arm_compute::cpu::data_layout
constexpr auto data_layout
Definition: impl.h:36
FIXTURE_DATA_TEST_CASE
#define FIXTURE_DATA_TEST_CASE(TEST_NAME, FIXTURE, MODE, DATASET)
Definition: Macros.h:258
arm_compute::CLReshapeLayer::CLReshapeLayer
CLReshapeLayer()
Default Constructor.
Definition: CLReshapeLayer.cpp:44
reshape_layer
__kernel void reshape_layer(__global uchar *input_ptr, uint input_stride_x, uint input_step_x, uint input_stride_y, uint input_step_y, uint input_stride_z, uint input_step_z, uint input_offset_first_element_in_bytes, __global uchar *output_ptr, uint output_stride_x, uint output_step_x, uint output_stride_y, uint output_step_y, uint output_stride_z, uint output_step_z, uint output_offset_first_element_in_bytes, int2 input_shape, int2 output_shape)
Perform tensor reshape.
Definition: reshape_layer.cl:49
ARM_COMPUTE_RETURN_ON_ERROR
#define ARM_COMPUTE_RETURN_ON_ERROR(status)
Checks if a status contains an error and returns it.
Definition: Error.h:205
arm_compute::TensorInfo::num_dimensions
size_t num_dimensions() const override
The number of dimensions of the tensor (rank)
Definition: TensorInfo.h:237
ARM_COMPUTE_ERROR_ON_NULLPTR
#define ARM_COMPUTE_ERROR_ON_NULLPTR(...)
Definition: Validate.h:159
arm_compute::cpu::kernels::CpuReshapeKernel::name
const char * name() const override
Name of the kernel.
Definition: CpuReshapeKernel.cpp:203
ARM_COMPUTE_ERROR_ON
#define ARM_COMPUTE_ERROR_ON(cond)
If the condition is true then an error message is printed and an exception thrown.
Definition: Error.h:466
ARM_COMPUTE_ERROR_THROW_ON
#define ARM_COMPUTE_ERROR_THROW_ON(status)
Definition: Error.h:455
arm_compute::DataType::U32
@ U32
unsigned 32-bit number
arm_compute::execute_window_loop
void execute_window_loop(const Window &w, L &&lambda_function, Ts &&...iterators)
Iterate through the passed window, automatically adjusting the iterators and calling the lambda_funct...
Definition: Helpers.inl:74
arm_compute::opencl::ClReshape::validate
static Status validate(const ITensorInfo *input, const ITensorInfo *output)
Static function to check if given info will lead to a valid configuration.
Definition: ClReshape.cpp:42
ARM_COMPUTE_RETURN_ERROR_ON
#define ARM_COMPUTE_RETURN_ERROR_ON(cond)
If the condition is true, an error is returned.
Definition: Error.h:298
arm_compute::PoolingType::AVG
@ AVG
Average Pooling.
arm_compute::ACL_DST
@ ACL_DST
Definition: Types.h:55
arm_compute::cpu::kernels::CpuReshapeKernel::prepare
void prepare(ITensorPack &tensors)
Prepare the reshape kernel for execution (Only executed once) by calculating max or squashed window a...
Definition: CpuReshapeKernel.cpp:216
arm_compute::TensorShape::total_size
size_t total_size() const
Collapses all dimensions to a single linear total size.
Definition: TensorShape.h:175
arm_compute::DataType::U8
@ U8
unsigned 8-bit number
ARM_COMPUTE_RETURN_ERROR_ON_F16_UNSUPPORTED
#define ARM_COMPUTE_RETURN_ERROR_ON_F16_UNSUPPORTED(tensor)
Definition: CLValidate.h:36
arm_compute::DataType::S16
@ S16
signed 16-bit number
arm_compute::Scheduler::get
static IScheduler & get()
Access the scheduler singleton.
Definition: Scheduler.cpp:94
arm_compute::create_kernel
cl::Kernel create_kernel(const CLCompileContext &ctx, const std::string &kernel_name, const std::set< std::string > &build_opts=std::set< std::string >())
Creates an opencl kernel using a compile context.
Definition: CLHelpers.cpp:409
arm_compute::test::validation::fill
library fill(src, distribution, 0)
arm_compute::Status
Status class.
Definition: Error.h:52
arm_compute::DataType::QASYMM8_SIGNED
@ QASYMM8_SIGNED
quantized, asymmetric fixed-point 8-bit number signed
check_header_guards.include
string include
Definition: check_header_guards.py:179
ARM_COMPUTE_ERROR_ON_INVALID_SUBWINDOW
#define ARM_COMPUTE_ERROR_ON_INVALID_SUBWINDOW(f, s)
Definition: Validate.h:203
ARM_COMPUTE_ASSERT
#define ARM_COMPUTE_ASSERT(cond)
Definition: Validate.h:37
arm_compute::opencl::kernels::ClReshapeKernel::validate
static Status validate(const ITensorInfo *src, const ITensorInfo *dst)
Static function to check if given info will lead to a valid configuration.
Definition: ClReshapeKernel.cpp:98
arm_compute::test::validation::pack
ITensorPack pack
Definition: Im2Col.cpp:204
ARM_COMPUTE_UNUSED
#define ARM_COMPUTE_UNUSED(...)
To avoid unused variables warnings.
Definition: Error.h:151
tensor
CLTensor * tensor
Pointer to the auxiliary tensor.
Definition: ClWorkloadRuntime.cpp:67
arm_compute::CLReshapeLayer::~CLReshapeLayer
~CLReshapeLayer()
Default Destructor.
arm_compute::Window::DimY
static constexpr size_t DimY
Alias for dimension 1 also known as Y dimension.
Definition: Window.h:45
arm_compute::CLScheduler::get
static CLScheduler & get()
Access the scheduler singleton.
Definition: CLScheduler.cpp:112
arm_compute::test::framework::dataset::make
std::enable_if< is_container< T >::value, ContainerDataset< T > >::type make(std::string name, T &&values)
Helper function to create a ContainerDataset.
Definition: ContainerDataset.h:160
arm_compute::opencl::kernels::ClReshapeKernel::configure
void configure(const CLCompileContext &compile_context, const ITensorInfo *src, ITensorInfo *dst)
Set the src and dst of the kernel.
Definition: ClReshapeKernel.cpp:71
arm_compute::test::validation::data_type
data_type
Definition: Cast.cpp:230
arm_compute::test::validation::input_shape
TensorShape input_shape
Validate test suite is to test ARM_COMPUTE_RETURN_ON_* macros we use to check the validity of given a...
Definition: LSTMLayerQuantized.cpp:466
arm_compute::test::library
std::unique_ptr< AssetsLibrary > library
Definition: main.cpp:77
arm_compute::IKernel::window
const Window & window() const
The maximum window the kernel can be executed on.
Definition: IKernel.cpp:28
arm_compute::cpu::kernels::CpuReshapeKernel::run_op
void run_op(ITensorPack &tensors, const Window &window, const ThreadInfo &info) override
Execute the kernel on the passed window.
Definition: CpuReshapeKernel.cpp:192
ARM_COMPUTE_RETURN_ERROR_ON_MISMATCHING_QUANTIZATION_INFO
#define ARM_COMPUTE_RETURN_ERROR_ON_MISMATCHING_QUANTIZATION_INFO(...)
Definition: Validate.h:753
arm_compute::IScheduler::schedule
virtual void schedule(ICPPKernel *kernel, const Hints &hints)=0
Runs the kernel in the same thread as the caller synchronously.
arm_compute::ICLKernel::add_3D_tensor_argument
void add_3D_tensor_argument(unsigned int &idx, const ICLTensor *tensor, const Window &window)
Add the passed 3D tensor's parameters to the object's kernel's arguments starting from the index idx.
Definition: ICLKernel.h:234
arm_compute::TensorInfo
Store the tensor's metadata.
Definition: TensorInfo.h:41
arm_compute::coords2index
int coords2index(const TensorShape &shape, const Coordinates &coord)
Convert n-dimensional coordinates into a linear index.
Definition: Helpers.inl:183
arm_compute::PoolingType::MAX
@ MAX
Max Pooling.
arm_compute::get_cl_unsigned_type_from_element_size
std::string get_cl_unsigned_type_from_element_size(size_t element_size)
Translates the element size to an unsigned integer data type.
Definition: CLHelpers.cpp:107
TEST_SUITE_END
#define TEST_SUITE_END()
Definition: Macros.h:39
arm_compute::cpu::kernels::CpuReshapeKernel::configure
void configure(const ITensorInfo *src, ITensorInfo *dst)
Configure kernel for a given list of arguments.
Definition: CpuReshapeKernel.cpp:173
arm_compute::DataType::S64
@ S64
signed 64-bit number
arm_compute::ELEMENTWISE
@ ELEMENTWISE
Elementwise CL kernel type.
Definition: CLTypes.h:83
arm_compute::ICLKernel::num_arguments_per_3D_tensor
constexpr static unsigned int num_arguments_per_3D_tensor()
Returns the number of arguments enqueued per 3D tensor object.
Definition: ICLKernel.h:321
arm_compute::calculate_squashed_or_max_window
std::pair< Window, size_t > calculate_squashed_or_max_window(const ITensorInfo &src0, const ITensorInfo &src1)
Definition: WindowHelpers.cpp:256
arm_compute
Copyright (c) 2017-2024 Arm Limited.
Definition: introduction.dox:24
arm_compute::to_string
std::string to_string(const ClComponentElementwiseBinary::Attributes::ElementwiseOp &op)
Formatted output of the arm_compute::experimental::dynamic_fusion::ClComponentElementwiseBinary::Attr...
Definition: ElementwiseBinary.h:68
arm_compute::DataType::F16
@ F16
16-bit floating-point number
arm_compute::test::framework::dataset::combine
CartesianProductDataset< T, U > combine(T &&dataset1, U &&dataset2)
Helper function to create a CartesianProductDataset.
Definition: CartesianProductDataset.h:180
arm_compute::has_padding_changed
bool has_padding_changed(const std::unordered_map< const ITensorInfo *, PaddingSize > &padding_map)
Check if the previously stored padding info has changed after configuring a kernel.
Definition: Utils.cpp:492
arm_compute::Window::DimZ
static constexpr size_t DimZ
Alias for dimension 2 also known as Z dimension.
Definition: Window.h:47
TEST_SUITE
#define TEST_SUITE(SUITE_NAME)
Definition: Macros.h:34
arm_compute::DataType::S32
@ S32
signed 32-bit number
arm_compute::ICLKernel::lws_hint
cl::NDRange lws_hint() const
Return the Local-Workgroup-Size hint.
Definition: ICLKernel.h:383
arm_compute::test::validation::src_info
TensorInfo src_info(src_shape, 1, data_type)
ARM_COMPUTE_RETURN_ERROR_ON_NULLPTR
#define ARM_COMPUTE_RETURN_ERROR_ON_NULLPTR(...)
Definition: Validate.h:161
arm_compute::validate
Status validate(const ITensorInfo *scores_in, const ITensorInfo *boxes_in, const ITensorInfo *batch_splits_in, const ITensorInfo *scores_out, const ITensorInfo *boxes_out, const ITensorInfo *classes, const ITensorInfo *batch_splits_out, const ITensorInfo *keeps, const ITensorInfo *keeps_size, const BoxNMSLimitInfo info)
Definition: CPPBoxWithNonMaximaSuppressionLimit.cpp:243
arm_compute::DataLayout::UNKNOWN
@ UNKNOWN
Unknown data layout.
arm_compute::ACL_SRC
@ ACL_SRC
Definition: Types.h:44
arm_compute::ITensorInfo
Store the tensor's metadata.
Definition: ITensorInfo.h:44
arm_compute::DataType::F32
@ F32
32-bit floating-point number
arm_compute::test::validation::info
ScaleKernelInfo info(interpolation_policy, default_border_mode, PixelValue(), sampling_policy, false)
arm_compute::CLReshapeLayer::configure
void configure(const ICLTensor *input, ICLTensor *output)
Initialise the kernel's inputs and outputs.
Definition: CLReshapeLayer.cpp:52
arm_compute::DataType::UNKNOWN
@ UNKNOWN
Unknown data type.
arm_compute::has_holes
bool has_holes(const ITensorInfo &info)
Check if the tensor has any holes.
Definition: Utils.cpp:28
arm_compute::DataType
DataType
Available data types.
Definition: CoreTypes.h:83
arm_compute::get_padding_info
std::unordered_map< const ITensorInfo *, PaddingSize > get_padding_info(std::initializer_list< const ITensorInfo * > infos)
Stores padding information before configuring a kernel.
Definition: Utils.cpp:477
arm_compute::validate_arguments
Status validate_arguments(const ITensorInfo *input, const ITensorInfo *output, unsigned int axis, ReductionOperation op)
Definition: NEReductionOperationKernel.cpp:252
arm_compute::test::framework::dataset::operator*
CartesianProductDataset< T, U > operator*(T &&dataset1, U &&dataset2)
Helper function to create a CartesianProductDataset.
Definition: CartesianProductDataset.h:208
arm_compute::test::validation::reference::slice
SimpleTensor< T > slice(const SimpleTensor< T > &src, Coordinates starts, Coordinates ends)
Definition: SliceOperations.cpp:38
arm_compute::ICPPKernel::default_mws
static constexpr size_t default_mws
Definition: ICPPKernel.h:41
arm_compute::TensorInfo::tensor_shape
const TensorShape & tensor_shape() const override
Size for each dimension of the tensor.
Definition: TensorInfo.h:245
arm_compute::test::validation::input
auto input
Definition: LSTMLayerQuantized.cpp:486
arm_compute::DataLayout::NDHWC
@ NDHWC
Num samples, depth, height, width, channels.
arm_compute::CLReshapeLayer::validate
static Status validate(const ITensorInfo *input, const ITensorInfo *output)
Static function to check if given info will lead to a valid configuration of CLReshapeLayer.
Definition: CLReshapeLayer.cpp:65
arm_compute::operator<<
inline ::std::ostream & operator<<(::std::ostream &os, const ClComponentElementwiseBinary::Attributes::ElementwiseOp &op)
Formatted output of the pute::experimental::dynamic_fusion::ClComponentElementwiseBinary::Attributes:...
Definition: ElementwiseBinary.h:47
arm_compute::enqueue
void enqueue(cl::CommandQueue &queue, ICLKernel &kernel, const Window &window, const cl::NDRange &lws_hint=CLKernelLibrary::get().default_ndrange(), bool use_dummy_work_items=false)
Add the kernel to the command queue with the given window.
Definition: ICLKernel.cpp:33