You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tvm.apache.org by GitBox <gi...@apache.org> on 2021/03/19 23:22:03 UTC

[GitHub] [tvm] csullivan opened a new pull request #7711: [Texture support][Part 0] Device API and runtime support

csullivan opened a new pull request #7711:
URL: https://github.com/apache/tvm/pull/7711


   This PR introduces 2d texture memory support to the OpenCL Device API runtime.
   
   **Device runtime**
   - The device runtime supports allocating texture memory both as a temporal workspace and as a runtime data space. In the latter case, special invocation of AllocDataSpace with a memory_scope == "texture(:weight)" is required. Special memory scopes were added to the runtime in https://github.com/apache/tvm/pull/7488. 
   - Workspace allocations are handled via a set of idle texture pools which are grown to match the requested sizes. The strategy employed is to first pick the pool which requires the least amount of extra space beyond, and then to minimize the amount of wasted space that growing a two dimensional pool may incur. A similar approach is taken for the ahead of time graph runtime memory planner for data space allocations (see: #7690).
   - CopyFromTo support is expanded to handle the case of directly reading from / writing to image buffers from host. 
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] ZihengJiang commented on pull request #7711: [Texture support][Part 0] Device API and runtime support

Posted by GitBox <gi...@apache.org>.
ZihengJiang commented on pull request #7711:
URL: https://github.com/apache/tvm/pull/7711#issuecomment-838399646


   LGTM. @tqchen, could you check whether the pr look good to you?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] ZihengJiang edited a comment on pull request #7711: [Texture support][Part 0] Device API and runtime support

Posted by GitBox <gi...@apache.org>.
ZihengJiang edited a comment on pull request #7711:
URL: https://github.com/apache/tvm/pull/7711#issuecomment-855184936


   Merged now. Thanks @csullivan for the hard working.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] tqchen commented on a change in pull request #7711: [Texture support][Part 0] Device API and runtime support

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #7711:
URL: https://github.com/apache/tvm/pull/7711#discussion_r630153926



##########
File path: src/runtime/opencl/opencl_common.h
##########
@@ -289,6 +319,24 @@ class OpenCLThreadEntry {
   // get the global workspace
   static OpenCLThreadEntry* ThreadLocal();
 };
+
+/*! \brief OpenCL runtime buffer structure with tracked memory layout */
+struct BufferDescriptor {
+  enum class MemoryLayout {
+    BUFFER_1D,

Review comment:
       document each entry, per the GoogleC style, use kBuffer1D

##########
File path: src/runtime/opencl/opencl_common.h
##########
@@ -289,6 +319,24 @@ class OpenCLThreadEntry {
   // get the global workspace
   static OpenCLThreadEntry* ThreadLocal();
 };
+
+/*! \brief OpenCL runtime buffer structure with tracked memory layout */
+struct BufferDescriptor {
+  enum class MemoryLayout {
+    BUFFER_1D,
+    IMAGE_2D_ACTIVATION,
+    IMAGE_2D_WEIGHT,
+  };
+  BufferDescriptor() = default;
+  explicit BufferDescriptor(Optional<String> scope) : layout(MemoryLayoutFromScope(scope)) {}
+  static MemoryLayout MemoryLayoutFromScope(Optional<String> mem_scope);
+  static String ScopeFromMemoryLayout(MemoryLayout mem_scope);
+
+  cl_mem buffer{nullptr};
+  MemoryLayout layout{MemoryLayout::BUFFER_1D};
+  std::vector<int64_t> shape;

Review comment:
       Document whether it is the shape of the physical buffer or logical one. Based on my read it is the physical one. Is it necessary given the info is also in the cl_mem?

##########
File path: src/runtime/opencl/opencl_device_api.cc
##########
@@ -138,41 +195,122 @@ void* OpenCLWorkspace::AllocDataSpace(Device dev, size_t size, size_t alignment,
   this->Init();
   ICHECK(context != nullptr) << "No OpenCL device";
   cl_int err_code;
-  cl_mem mptr = clCreateBuffer(this->context, CL_MEM_READ_WRITE, size, nullptr, &err_code);
+  cl::BufferDescriptor* desc = new cl::BufferDescriptor;
+  desc->buffer = clCreateBuffer(this->context, CL_MEM_READ_WRITE, size, nullptr, &err_code);
+  desc->layout = cl::BufferDescriptor::MemoryLayout::BUFFER_1D;
+  desc->shape.push_back(size);
+  desc->dtype = type_hint;
   OPENCL_CHECK_ERROR(err_code);
-  return mptr;
+  return desc;
+}
+
+void* OpenCLWorkspace::AllocDataSpace(Device dev, int ndim, const int64_t* shape, DLDataType dtype,
+                                      Optional<String> mem_scope) {
+  if (!mem_scope.defined() || mem_scope.value() == "global") {
+    return DeviceAPI::AllocDataSpace(dev, ndim, shape, dtype, mem_scope);
+  }
+  ICHECK(IsTextureStorage(std::string(mem_scope.value())))
+      << "Device does not support allocate data space with "
+      << "specified memory scope: " << mem_scope.value();
+
+  ICHECK(ndim > 2) << "Shape for texture allocation must be at least rank 3; "
+                   << "provided shape is rank " << ndim;
+
+  cl::BufferDescriptor* desc = new cl::BufferDescriptor(mem_scope);
+  size_t axis = DefaultTextureLayoutSeparator(ndim, mem_scope.value());
+  auto texture = ApplyTexture2DFlattening<int64_t>(shape, ndim, axis);
+  desc->buffer = AllocTexture(dev, texture.width, texture.height, dtype);
+  desc->shape.insert(desc->shape.end(), &shape[0], &shape[ndim]);

Review comment:
       desc->shape = std::vector<int64_t>(shape, shape + ndim);

##########
File path: src/runtime/opencl/opencl_device_api.cc
##########
@@ -32,6 +32,63 @@ namespace cl {
 std::string GetPlatformInfo(cl_platform_id pid, cl_platform_info param_name);
 std::string GetDeviceInfo(cl_device_id pid, cl_device_info param_name);
 
+struct clImageInfo {
+  size_t origin[3] = {};
+  size_t region[3] = {};
+  size_t row_pitch = 0;
+  size_t slice_pitch = 0;
+};
+
+clImageInfo GetImageInfo(const cl::BufferDescriptor* desc, const DLTensor* tensor) {

Review comment:
       document thie API

##########
File path: src/runtime/texture_pool.cc
##########
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file

Review comment:
       consider move to src/runtime/opencl for now(as texture utils is only needed by OpenCL atm and we can consider add vulkan support later). 
   
   With vulkan support we can move to src/runtime/texture or make it as a header util that both OpenCL and vk can include

##########
File path: src/runtime/opencl/opencl_device_api.cc
##########
@@ -138,41 +195,122 @@ void* OpenCLWorkspace::AllocDataSpace(Device dev, size_t size, size_t alignment,
   this->Init();
   ICHECK(context != nullptr) << "No OpenCL device";
   cl_int err_code;
-  cl_mem mptr = clCreateBuffer(this->context, CL_MEM_READ_WRITE, size, nullptr, &err_code);
+  cl::BufferDescriptor* desc = new cl::BufferDescriptor;
+  desc->buffer = clCreateBuffer(this->context, CL_MEM_READ_WRITE, size, nullptr, &err_code);
+  desc->layout = cl::BufferDescriptor::MemoryLayout::BUFFER_1D;
+  desc->shape.push_back(size);
+  desc->dtype = type_hint;
   OPENCL_CHECK_ERROR(err_code);
-  return mptr;
+  return desc;
+}
+
+void* OpenCLWorkspace::AllocDataSpace(Device dev, int ndim, const int64_t* shape, DLDataType dtype,
+                                      Optional<String> mem_scope) {
+  if (!mem_scope.defined() || mem_scope.value() == "global") {
+    return DeviceAPI::AllocDataSpace(dev, ndim, shape, dtype, mem_scope);
+  }
+  ICHECK(IsTextureStorage(std::string(mem_scope.value())))
+      << "Device does not support allocate data space with "
+      << "specified memory scope: " << mem_scope.value();
+
+  ICHECK(ndim > 2) << "Shape for texture allocation must be at least rank 3; "
+                   << "provided shape is rank " << ndim;
+
+  cl::BufferDescriptor* desc = new cl::BufferDescriptor(mem_scope);
+  size_t axis = DefaultTextureLayoutSeparator(ndim, mem_scope.value());
+  auto texture = ApplyTexture2DFlattening<int64_t>(shape, ndim, axis);
+  desc->buffer = AllocTexture(dev, texture.width, texture.height, dtype);
+  desc->shape.insert(desc->shape.end(), &shape[0], &shape[ndim]);
+  desc->dtype = dtype;
+  return desc;
 }
 
 void OpenCLWorkspace::FreeDataSpace(Device dev, void* ptr) {
   // We have to make sure that the memory object is not in the command queue
   // for some OpenCL platforms.
   OPENCL_CALL(clFinish(this->GetQueue(dev)));
 
-  cl_mem mptr = static_cast<cl_mem>(ptr);
-  OPENCL_CALL(clReleaseMemObject(mptr));
+  cl::BufferDescriptor* desc = static_cast<cl::BufferDescriptor*>(ptr);
+  OPENCL_CALL(clReleaseMemObject(desc->buffer));
+  delete desc;
 }
 
-void OpenCLWorkspace::CopyDataFromTo(const void* from, size_t from_offset, void* to,
-                                     size_t to_offset, size_t size, Device dev_from, Device dev_to,
-                                     DLDataType type_hint, TVMStreamHandle stream) {
+cl_mem OpenCLWorkspace::AllocTexture(Device dev, size_t width, size_t height,
+                                     DLDataType type_hint) {
   this->Init();
-  ICHECK(stream == nullptr);
-  if (IsOpenCLDevice(dev_from) && IsOpenCLDevice(dev_to)) {
-    OPENCL_CALL(clEnqueueCopyBuffer(this->GetQueue(dev_to),
-                                    static_cast<cl_mem>((void*)from),  // NOLINT(*)
-                                    static_cast<cl_mem>(to), from_offset, to_offset, size, 0,
-                                    nullptr, nullptr));
-  } else if (IsOpenCLDevice(dev_from) && dev_to.device_type == kDLCPU) {
-    OPENCL_CALL(clEnqueueReadBuffer(this->GetQueue(dev_from),
-                                    static_cast<cl_mem>((void*)from),  // NOLINT(*)
-                                    CL_FALSE, from_offset, size, static_cast<char*>(to) + to_offset,
-                                    0, nullptr, nullptr));
-    OPENCL_CALL(clFinish(this->GetQueue(dev_from)));
-  } else if (dev_from.device_type == kDLCPU && IsOpenCLDevice(dev_to)) {
-    OPENCL_CALL(clEnqueueWriteBuffer(this->GetQueue(dev_to), static_cast<cl_mem>(to), CL_FALSE,
-                                     to_offset, size, static_cast<const char*>(from) + from_offset,
-                                     0, nullptr, nullptr));
-    OPENCL_CALL(clFinish(this->GetQueue(dev_to)));
+  ICHECK(context != nullptr) << "No OpenCL device";
+  cl_int err_code;
+  cl_channel_type cl_type = DTypeToOpenCLChannelType(type_hint);
+  cl_image_format format = {CL_RGBA, cl_type};
+  cl_image_desc descriptor = {CL_MEM_OBJECT_IMAGE2D, width, height, 0, 0, 0, 0, 0, 0};
+  cl_mem mptr =
+      clCreateImage(this->context, CL_MEM_READ_WRITE, &format, &descriptor, nullptr, &err_code);
+  OPENCL_CHECK_ERROR(err_code);
+  return mptr;
+}
+
+void* OpenCLWorkspace::AllocTextureWorkspace(Device dev, size_t width, size_t height,
+                                             DLDataType type_hint) {
+  return GetThreadEntry()->texture_pool.AllocTexture(dev, width, height, type_hint);
+}
+
+void OpenCLWorkspace::FreeTextureWorkspace(Device dev, void* ptr) {
+  GetThreadEntry()->texture_pool.FreeTexture(dev, ptr);
+}
+
+void OpenCLWorkspace::CopyDataFromTo(DLTensor* from, DLTensor* to, TVMStreamHandle stream) {
+  size_t nbytes = GetDataSize(*from);
+  ICHECK_EQ(nbytes, GetDataSize(*to));
+  ICHECK(IsContiguous(*from) && IsContiguous(*to))
+      << "CopyDataFromTo only support contiguous array for now";
+
+  if (IsOpenCLDevice(from->device) && IsOpenCLDevice(to->device)) {
+    const auto* from_desc = static_cast<const cl::BufferDescriptor*>(from->data);
+    ICHECK(from_desc->layout == cl::BufferDescriptor::MemoryLayout::BUFFER_1D)
+        << "Device to device copying is currently only implemented for OpenCL buffer storage";
+    auto* to_desc = static_cast<cl::BufferDescriptor*>(to->data);
+    OPENCL_CALL(clEnqueueCopyBuffer(this->GetQueue(to->device), from_desc->buffer, to_desc->buffer,
+                                    from->byte_offset, to->byte_offset, nbytes, 0, nullptr,
+                                    nullptr));
+  } else if (IsOpenCLDevice(from->device) && to->device.device_type == kDLCPU) {
+    const auto* from_desc = static_cast<const cl::BufferDescriptor*>(from->data);
+    switch (from_desc->layout) {
+      case cl::BufferDescriptor::MemoryLayout::BUFFER_1D:
+        OPENCL_CALL(clEnqueueReadBuffer(
+            this->GetQueue(from->device), from_desc->buffer, CL_FALSE, from->byte_offset, nbytes,
+            static_cast<char*>(to->data) + to->byte_offset, 0, nullptr, nullptr));
+        break;
+      case cl::BufferDescriptor::MemoryLayout::IMAGE_2D_ACTIVATION:
+      case cl::BufferDescriptor::MemoryLayout::IMAGE_2D_WEIGHT:
+        auto image_info = GetImageInfo(from_desc, from);
+        // TODO(csullivan): Support calculating row_pitch correctly in the case of reuse.

Review comment:
       Would be great to add a few testcases in python that demonstrates the copy into image where image size is bigger than the normal one. Perhaps the easiest way is to construct an NDArray then write a PackedFunc that takes a smaller view from it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] tqchen commented on pull request #7711: [Texture support][Part 0] Device API and runtime support

Posted by GitBox <gi...@apache.org>.
tqchen commented on pull request #7711:
URL: https://github.com/apache/tvm/pull/7711#issuecomment-803204767


   Thanks @csullivan some quick comments, will read more carefully in the incoming week.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] csullivan commented on pull request #7711: [Texture support][Part 0] Device API and runtime support

Posted by GitBox <gi...@apache.org>.
csullivan commented on pull request #7711:
URL: https://github.com/apache/tvm/pull/7711#issuecomment-832296233


   @tqchen, @ZihengJiang, would you kindly consider reviewing once more? 
   
   The main change is to remove texture specific device apis and rely on `tir.tvm_call_packed` (cf #7932) for texture workspace allocations and AllocDataSpace (w/ scope). 
   
   I also introduced an opencl buffer descriptor that tracks the allocation layout. With the layout and the DLTensor CopyDataFromTo overload I've verified that a sub-texture allocation can be correctly copied out of a 2d texture pool of larger extent. This solves an issue I raised in [Part 4](https://github.com/apache/tvm/pull/7689/files#diff-409f906bd8fdca700e76726e204f722ae4b8a8ab39376ffb9bb46434d063678cR63-R71).
   
   I appreciate any additional feedback you have.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] tqchen commented on a change in pull request #7711: [Texture support][Part 0] Device API and runtime support

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #7711:
URL: https://github.com/apache/tvm/pull/7711#discussion_r598031908



##########
File path: include/tvm/runtime/device_api.h
##########
@@ -181,7 +181,38 @@ class TVM_DLL DeviceAPI {
    * \param ptr The pointer to be freed.
    */
   virtual void FreeWorkspace(TVMContext ctx, void* ptr);
-
+  /*!
+   * \brief Allocate a two dimensional texture data space on device
+   * \param ctx The device context to perform operation.
+   * \param width The width of the 2d texture in elements
+   * \param height The height of the 2d texture in elements
+   * \param type_hint The type of elements.
+   */

Review comment:
       It would be great to figure out a way to remove the Texture related functions from the Device API interface, so we can focus on the NDArray related parts.

##########
File path: src/runtime/texture.h
##########
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+/*!
+ * \file texture.h
+ * \brief Texture utilities
+ */
+#ifndef TVM_RUNTIME_TEXTURE_POOL_H_
+#define TVM_RUNTIME_TEXTURE_POOL_H_
+
+#include <tvm/runtime/device_api.h>
+
+#include <memory>
+#include <vector>
+
+namespace tvm {
+namespace runtime {
+
+/*! \brief Structure to represent flattened texture shape */
+template <typename T>
+struct Texture2DShape {
+  T width;
+  T height;
+  T channel;
+};
+
+/*!
+ * \param shape_rank Rank N of the Nd-shape
+ * \param convention Storage scope convention to use for flattening
+ * \return The axis separator that defines the Nd shape partitioning in 2d
+ */
+inline size_t DefaultTextureLayoutSeparator(size_t shape_rank, std::string convention = "texture") {
+  // Texture activation:
+  // e.g. [N,C,H,W,c] -> Texture2d[N*C*H, W, c]
+  // Texture weight:
+  // e.g. [O,I,H,W,c] -> Texture2d[O, I*H*W, c]
+  size_t separator = 0;
+  if (convention == "texture"){
+    separator = shape_rank - 2;
+  } else if (convention == "texture:weight") {
+    separator = 1;
+  } else {
+    LOG(FATAL) << "Encountered unknown texture lowering convention";
+  }
+  return separator;
+}
+
+/*!
+ * \param shape Nd shape
+ * \param rank Number of dimensions N of the Nd shape
+ * \param axis The axis separator that splits the Nd axes into two sets
+ * \return Width and height of the 2d shape
+ */
+template<typename T, typename S>
+Texture2DShape<T> ApplyTexture2DFlattening(const S& shape, size_t rank, size_t axis) {
+  ICHECK(axis < rank) << "Number of axes to flatten into rows must be less than shape rank for 2d flattening";
+  Texture2DShape<T> texture{1, 1, shape[rank - 1]};
+  for (size_t i = 0; i < rank - 1; i++) {
+    if (i < axis) {
+      texture.height *= shape[i];
+    } else {
+      texture.width *= shape[i];
+    }
+  }
+  return texture;
+}
+
+inline bool IsTextureStorage(std::string scope) {
+  return scope.find("texture") != std::string::npos;
+}
+
+class TVM_DLL TexturePool {
+ public:
+  /*!
+   * \brief Create pool with specific device type and device.
+   * \param device_type The device type.
+   * \param device_api The device API.
+   */
+  TexturePool(DLDeviceType device_type, DeviceAPI* device_api);
+  /*! \brief destructor */
+  ~TexturePool();
+  /*!
+   * \brief Allocate temporal texture.
+   * \param ctx The context of allocation.
+   * \param width The width of the 2d texture to be allocated.
+   * \param height The height of the 2d texture to be allocated.
+   */
+  void* AllocTexture(TVMContext ctx, size_t width, size_t height, DLDataType type_hint);
+  /*!
+   * \brief Free temporal texture in backend execution.
+   *
+   * \param ctx The context of allocation.
+   * \param ptr The pointer to be freed.
+   */
+  void FreeTexture(TVMContext ctx, void* ptr);
+
+ private:
+  class Pool;
+  /*! \brief pool of device local array */
+  std::vector<Pool*> array_;

Review comment:
       can we document a bit the motivation behind texture pool? Can we simply move the texture to ND Allocation API in the device api?

##########
File path: src/runtime/c_runtime_api.cc
##########
@@ -450,6 +464,31 @@ int TVMBackendFreeWorkspace(int device_type, int device_id, void* ptr) {
   return 0;
 }
 
+void* TVMBackendAllocTexture(int device_type, int device_id, uint64_t width, uint64_t height,
+                             int dtype_code_hint, int dtype_bits_hint) {
+  TVMContext ctx;
+  ctx.device_type = static_cast<DLDeviceType>(device_type);

Review comment:
       I wonder if it is possible to blend into a PackedFunc now so it can be OpenCL specific




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] ZihengJiang commented on pull request #7711: [Texture support][Part 0] Device API and runtime support

Posted by GitBox <gi...@apache.org>.
ZihengJiang commented on pull request #7711:
URL: https://github.com/apache/tvm/pull/7711#issuecomment-855184936


   LGTM. Thanks @csullivan for the hard working.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] csullivan commented on pull request #7711: [Texture support][Part 0] Device API and runtime support

Posted by GitBox <gi...@apache.org>.
csullivan commented on pull request #7711:
URL: https://github.com/apache/tvm/pull/7711#issuecomment-850905948


   @tqchen Thanks for the great feedback, could you take a look again?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] ZihengJiang merged pull request #7711: [Texture support][Part 0] Device API and runtime support

Posted by GitBox <gi...@apache.org>.
ZihengJiang merged pull request #7711:
URL: https://github.com/apache/tvm/pull/7711


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] tqchen commented on pull request #7711: [Texture support][Part 0] Device API and runtime support

Posted by GitBox <gi...@apache.org>.
tqchen commented on pull request #7711:
URL: https://github.com/apache/tvm/pull/7711#issuecomment-854702400


   Thanks @csullivan . will let @ZihengJiang manage the PR


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [tvm] csullivan commented on a change in pull request #7711: [Texture support][Part 0] Device API and runtime support

Posted by GitBox <gi...@apache.org>.
csullivan commented on a change in pull request #7711:
URL: https://github.com/apache/tvm/pull/7711#discussion_r641988881



##########
File path: src/runtime/opencl/opencl_device_api.cc
##########
@@ -138,41 +195,122 @@ void* OpenCLWorkspace::AllocDataSpace(Device dev, size_t size, size_t alignment,
   this->Init();
   ICHECK(context != nullptr) << "No OpenCL device";
   cl_int err_code;
-  cl_mem mptr = clCreateBuffer(this->context, CL_MEM_READ_WRITE, size, nullptr, &err_code);
+  cl::BufferDescriptor* desc = new cl::BufferDescriptor;
+  desc->buffer = clCreateBuffer(this->context, CL_MEM_READ_WRITE, size, nullptr, &err_code);
+  desc->layout = cl::BufferDescriptor::MemoryLayout::BUFFER_1D;
+  desc->shape.push_back(size);
+  desc->dtype = type_hint;
   OPENCL_CHECK_ERROR(err_code);
-  return mptr;
+  return desc;
+}
+
+void* OpenCLWorkspace::AllocDataSpace(Device dev, int ndim, const int64_t* shape, DLDataType dtype,
+                                      Optional<String> mem_scope) {
+  if (!mem_scope.defined() || mem_scope.value() == "global") {
+    return DeviceAPI::AllocDataSpace(dev, ndim, shape, dtype, mem_scope);
+  }
+  ICHECK(IsTextureStorage(std::string(mem_scope.value())))
+      << "Device does not support allocate data space with "
+      << "specified memory scope: " << mem_scope.value();
+
+  ICHECK(ndim > 2) << "Shape for texture allocation must be at least rank 3; "
+                   << "provided shape is rank " << ndim;
+
+  cl::BufferDescriptor* desc = new cl::BufferDescriptor(mem_scope);
+  size_t axis = DefaultTextureLayoutSeparator(ndim, mem_scope.value());
+  auto texture = ApplyTexture2DFlattening<int64_t>(shape, ndim, axis);
+  desc->buffer = AllocTexture(dev, texture.width, texture.height, dtype);
+  desc->shape.insert(desc->shape.end(), &shape[0], &shape[ndim]);
+  desc->dtype = dtype;
+  return desc;
 }
 
 void OpenCLWorkspace::FreeDataSpace(Device dev, void* ptr) {
   // We have to make sure that the memory object is not in the command queue
   // for some OpenCL platforms.
   OPENCL_CALL(clFinish(this->GetQueue(dev)));
 
-  cl_mem mptr = static_cast<cl_mem>(ptr);
-  OPENCL_CALL(clReleaseMemObject(mptr));
+  cl::BufferDescriptor* desc = static_cast<cl::BufferDescriptor*>(ptr);
+  OPENCL_CALL(clReleaseMemObject(desc->buffer));
+  delete desc;
 }
 
-void OpenCLWorkspace::CopyDataFromTo(const void* from, size_t from_offset, void* to,
-                                     size_t to_offset, size_t size, Device dev_from, Device dev_to,
-                                     DLDataType type_hint, TVMStreamHandle stream) {
+cl_mem OpenCLWorkspace::AllocTexture(Device dev, size_t width, size_t height,
+                                     DLDataType type_hint) {
   this->Init();
-  ICHECK(stream == nullptr);
-  if (IsOpenCLDevice(dev_from) && IsOpenCLDevice(dev_to)) {
-    OPENCL_CALL(clEnqueueCopyBuffer(this->GetQueue(dev_to),
-                                    static_cast<cl_mem>((void*)from),  // NOLINT(*)
-                                    static_cast<cl_mem>(to), from_offset, to_offset, size, 0,
-                                    nullptr, nullptr));
-  } else if (IsOpenCLDevice(dev_from) && dev_to.device_type == kDLCPU) {
-    OPENCL_CALL(clEnqueueReadBuffer(this->GetQueue(dev_from),
-                                    static_cast<cl_mem>((void*)from),  // NOLINT(*)
-                                    CL_FALSE, from_offset, size, static_cast<char*>(to) + to_offset,
-                                    0, nullptr, nullptr));
-    OPENCL_CALL(clFinish(this->GetQueue(dev_from)));
-  } else if (dev_from.device_type == kDLCPU && IsOpenCLDevice(dev_to)) {
-    OPENCL_CALL(clEnqueueWriteBuffer(this->GetQueue(dev_to), static_cast<cl_mem>(to), CL_FALSE,
-                                     to_offset, size, static_cast<const char*>(from) + from_offset,
-                                     0, nullptr, nullptr));
-    OPENCL_CALL(clFinish(this->GetQueue(dev_to)));
+  ICHECK(context != nullptr) << "No OpenCL device";
+  cl_int err_code;
+  cl_channel_type cl_type = DTypeToOpenCLChannelType(type_hint);
+  cl_image_format format = {CL_RGBA, cl_type};
+  cl_image_desc descriptor = {CL_MEM_OBJECT_IMAGE2D, width, height, 0, 0, 0, 0, 0, 0};
+  cl_mem mptr =
+      clCreateImage(this->context, CL_MEM_READ_WRITE, &format, &descriptor, nullptr, &err_code);
+  OPENCL_CHECK_ERROR(err_code);
+  return mptr;
+}
+
+void* OpenCLWorkspace::AllocTextureWorkspace(Device dev, size_t width, size_t height,
+                                             DLDataType type_hint) {
+  return GetThreadEntry()->texture_pool.AllocTexture(dev, width, height, type_hint);
+}
+
+void OpenCLWorkspace::FreeTextureWorkspace(Device dev, void* ptr) {
+  GetThreadEntry()->texture_pool.FreeTexture(dev, ptr);
+}
+
+void OpenCLWorkspace::CopyDataFromTo(DLTensor* from, DLTensor* to, TVMStreamHandle stream) {
+  size_t nbytes = GetDataSize(*from);
+  ICHECK_EQ(nbytes, GetDataSize(*to));
+  ICHECK(IsContiguous(*from) && IsContiguous(*to))
+      << "CopyDataFromTo only support contiguous array for now";
+
+  if (IsOpenCLDevice(from->device) && IsOpenCLDevice(to->device)) {
+    const auto* from_desc = static_cast<const cl::BufferDescriptor*>(from->data);
+    ICHECK(from_desc->layout == cl::BufferDescriptor::MemoryLayout::BUFFER_1D)
+        << "Device to device copying is currently only implemented for OpenCL buffer storage";
+    auto* to_desc = static_cast<cl::BufferDescriptor*>(to->data);
+    OPENCL_CALL(clEnqueueCopyBuffer(this->GetQueue(to->device), from_desc->buffer, to_desc->buffer,
+                                    from->byte_offset, to->byte_offset, nbytes, 0, nullptr,
+                                    nullptr));
+  } else if (IsOpenCLDevice(from->device) && to->device.device_type == kDLCPU) {
+    const auto* from_desc = static_cast<const cl::BufferDescriptor*>(from->data);
+    switch (from_desc->layout) {
+      case cl::BufferDescriptor::MemoryLayout::BUFFER_1D:
+        OPENCL_CALL(clEnqueueReadBuffer(
+            this->GetQueue(from->device), from_desc->buffer, CL_FALSE, from->byte_offset, nbytes,
+            static_cast<char*>(to->data) + to->byte_offset, 0, nullptr, nullptr));
+        break;
+      case cl::BufferDescriptor::MemoryLayout::IMAGE_2D_ACTIVATION:
+      case cl::BufferDescriptor::MemoryLayout::IMAGE_2D_WEIGHT:
+        auto image_info = GetImageInfo(from_desc, from);
+        // TODO(csullivan): Support calculating row_pitch correctly in the case of reuse.

Review comment:
       Thank you for the good suggestion, I added a [test](https://github.com/apache/tvm/pull/7711/files#diff-e9741363006d59a2ce0f9cdac61da8f0b55b79aed6e9cab9a0922113266df46eR65) to demonstrate writing to a subview of a texture and check that the data from larger allocation contains the [expected](https://github.com/apache/tvm/pull/7711/files#diff-e9741363006d59a2ce0f9cdac61da8f0b55b79aed6e9cab9a0922113266df46eR117) two dimensional strides when copied back to host. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org