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 2022/09/06 10:58:07 UTC

[GitHub] [tvm] srkreddy1238 opened a new pull request, #12711: [OpenCLML] CLML Profiling fixes corresponding to OpenCL Timer recent …

srkreddy1238 opened a new pull request, #12711:
URL: https://github.com/apache/tvm/pull/12711

   Dynamic creation of command queue (#11180) breaks OpenCLML profiling as OpenCLML stores ref. for command queue and reuse later.
   
   Now, we access the command queue from work space always rather storing internally.
   
   Thanks for contributing to TVM!   Please refer to guideline https://tvm.apache.org/docs/contribute/ for useful information and tips. After the pull request is submitted, please request code reviews from [Reviewers](https://github.com/apache/incubator-tvm/blob/master/CONTRIBUTORS.md#reviewers) by @ them in the pull request thread.
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] srkreddy1238 commented on a diff in pull request #12711: [OpenCLML] CLML Profiling fixes corresponding to OpenCL Timer recent …

Posted by GitBox <gi...@apache.org>.
srkreddy1238 commented on code in PR #12711:
URL: https://github.com/apache/tvm/pull/12711#discussion_r964572049


##########
src/runtime/contrib/clml/clml_runtime.cc:
##########
@@ -499,8 +484,17 @@ class CLMLRuntime : public JSONRuntimeBase {
     uint32_t n, c, h, w;
   };
 
-  bool ExtensionStringPresent(cl_device_id device_id) {
+  bool ExtensionStringPresent(void) {
     cl_int result = 0;
+    cl_platform_id platform;
+    cl_device_id device_id;
+    result = clGetPlatformIDs(1, &platform, NULL);
+    ICHECK(result == CL_SUCCESS) << "clGetPlatformIDs:" << result;
+    uint32_t num_devices = 0;
+    result = clGetDeviceIDs(platform, CL_DEVICE_TYPE_GPU, 0, NULL, &num_devices);
+    ICHECK(result == CL_SUCCESS && num_devices == 1) << "clGetDeviceIDs:" << result;
+    result = clGetDeviceIDs(platform, CL_DEVICE_TYPE_GPU, 1, &device_id, NULL);
+    ICHECK(device_id && result == CL_SUCCESS) << "clGetDeviceIDs:" << result;

Review Comment:
   Yep, workspace has real id. Earlier I looked into ThreadEntry->Device, which is DLDevice and it's always zero.
   We always use the first OpenCL device available and a new API ```GetClDeviceId``` may not be required at the moment.
   
   



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] srkreddy1238 commented on a diff in pull request #12711: [OpenCLML] CLML Profiling fixes corresponding to OpenCL Timer recent …

Posted by GitBox <gi...@apache.org>.
srkreddy1238 commented on code in PR #12711:
URL: https://github.com/apache/tvm/pull/12711#discussion_r963862865


##########
src/runtime/contrib/clml/clml_runtime.cc:
##########
@@ -146,22 +169,7 @@ class CLMLRuntime : public JSONRuntimeBase {
       LOG(WARNING) << "CLML Runtime Init: Qualcomm extn not present.\n";
       return;
     }
-
-    // Reuse the OpenCl work space from TVM Device API.
-    auto func = tvm::runtime::Registry::Get("device_api.opencl");
-    ICHECK(func != nullptr) << "Cannot find OpenCL device_api in registry";
-    auto device_api = static_cast<cl::OpenCLWorkspace*>(((*func)()).operator void*());
-    this->context = device_api->context;
-    bool queue_found = false;
-    for (size_t i = 0; i < device_api->devices.size(); ++i) {
-      if (device_api->devices[i] == device_id) {
-        this->queue = device_api->queues[i];
-        this->evts = &(device_api->events[i]);
-        queue_found = true;
-      }
-    }
-    ICHECK(queue_found != false) << "Device queue not found in OpenCL Workspace";
-
+    this->queue = GetCommadQueue();

Review Comment:
   Agreed. Let me cleanup a bit by holding ref. for workspace instead of context and queues. This makes CLML unaffected by any changes to workspace internals.



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] echuraev commented on a diff in pull request #12711: [OpenCLML] CLML Profiling fixes corresponding to OpenCL Timer recent …

Posted by GitBox <gi...@apache.org>.
echuraev commented on code in PR #12711:
URL: https://github.com/apache/tvm/pull/12711#discussion_r964623519


##########
src/runtime/contrib/clml/clml_runtime.cc:
##########
@@ -499,15 +483,18 @@ class CLMLRuntime : public JSONRuntimeBase {
     uint32_t n, c, h, w;
   };
 
-  bool ExtensionStringPresent(cl_device_id device_id) {
+  bool ExtensionStringPresent(void) {
     cl_int result = 0;
-
+    if (workspace->platform_id == nullptr) {
+      return 0;
+    }
     size_t reqd_size = 0;
-    result = clGetDeviceInfo(device_id, CL_DEVICE_EXTENSIONS, 0, NULL, &reqd_size);
+    result = clGetDeviceInfo(workspace->devices[0], CL_DEVICE_EXTENSIONS, 0, NULL, &reqd_size);

Review Comment:
   I agree with you that in current implementation, `device_id` is always zero. So, we can just get the first device in the vector. 
   
   But I have one point for a discussion. I'm afraid that if we change something in the `OpenCLWorkspace` related to the list of the devices in the future, then we can break this piece of code. So, this is why I prefer to get the device in this way: `workspace->devices[workspace->GetThreadEntry()->device.device_id]`. To make it more safe and readable, I proposed to introduce a new method. What do you think about 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.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] echuraev commented on a diff in pull request #12711: [OpenCLML] CLML Profiling fixes corresponding to OpenCL Timer recent …

Posted by GitBox <gi...@apache.org>.
echuraev commented on code in PR #12711:
URL: https://github.com/apache/tvm/pull/12711#discussion_r964625171


##########
src/runtime/contrib/clml/clml_runtime.cc:
##########
@@ -499,8 +484,17 @@ class CLMLRuntime : public JSONRuntimeBase {
     uint32_t n, c, h, w;
   };
 
-  bool ExtensionStringPresent(cl_device_id device_id) {
+  bool ExtensionStringPresent(void) {
     cl_int result = 0;
+    cl_platform_id platform;
+    cl_device_id device_id;
+    result = clGetPlatformIDs(1, &platform, NULL);
+    ICHECK(result == CL_SUCCESS) << "clGetPlatformIDs:" << result;
+    uint32_t num_devices = 0;
+    result = clGetDeviceIDs(platform, CL_DEVICE_TYPE_GPU, 0, NULL, &num_devices);
+    ICHECK(result == CL_SUCCESS && num_devices == 1) << "clGetDeviceIDs:" << result;
+    result = clGetDeviceIDs(platform, CL_DEVICE_TYPE_GPU, 1, &device_id, NULL);
+    ICHECK(device_id && result == CL_SUCCESS) << "clGetDeviceIDs:" << result;

Review Comment:
   I agree with you. Added new comment below. Let's discuss this moment there.



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] echuraev commented on a diff in pull request #12711: [OpenCLML] CLML Profiling fixes corresponding to OpenCL Timer recent …

Posted by GitBox <gi...@apache.org>.
echuraev commented on code in PR #12711:
URL: https://github.com/apache/tvm/pull/12711#discussion_r964381348


##########
src/runtime/contrib/clml/clml_runtime.cc:
##########
@@ -1056,11 +1055,8 @@ class CLMLRuntime : public JSONRuntimeBase {
   CachedLayer layer_;
   // CLML Context
   CLMLInterfaceV2QCOM* h_ClmlIntf = NULL;
-  cl_platform_id platform = NULL;
-  cl_context context = NULL;
-  cl_device_id device_id = NULL;
-  cl_command_queue queue = NULL;
-  std::vector<cl_event>* evts;
+  cl::OpenCLWorkspace* workspace = NULL;
+  cl::OpenCLThreadEntry* tentry = NULL;

Review Comment:
   nit: Do you really need a separate variable for ThreadEntry? You can get it anytime from `OpenCLWorkspace`.



##########
src/runtime/contrib/clml/clml_runtime.cc:
##########
@@ -286,21 +265,26 @@ class CLMLRuntime : public JSONRuntimeBase {
     }
 
     for (size_t i = 0; i < this->layer_.function.size(); ++i) {
-      this->evts->resize(this->evts->size() + 1);
-      cl_event* evt = &(this->evts->back());
-      result = h_ClmlIntf->clEnqueueMLOpQCOM(queue, this->layer_.function[i],
-                                             this->layer_.descriptorSet, 0, NULL, evt);
+      if (getenv("CLML_PROFILING")) {

Review Comment:
   What if we will use method [IsProfiling](https://github.com/apache/tvm/blob/main/src/runtime/opencl/opencl_common.h#L278-L286) instead of this variable? Or it is a possible situation that we have an OpenCL queue created with profiling enable option, but we don't want to profile CLML?



##########
src/runtime/contrib/clml/clml_runtime.cc:
##########
@@ -253,6 +230,8 @@ class CLMLRuntime : public JSONRuntimeBase {
    */
   void Run() override {
     cl_int result = 0;
+    cl_command_queue queue = workspace->GetQueue(tentry->device);
+    std::vector<cl_event>* evts = &(workspace->GetEventQueue(tentry->device));

Review Comment:
   Why do you need pointer on the vector with events instead of reference?



##########
src/runtime/contrib/clml/clml_runtime.cc:
##########
@@ -499,8 +484,17 @@ class CLMLRuntime : public JSONRuntimeBase {
     uint32_t n, c, h, w;
   };
 
-  bool ExtensionStringPresent(cl_device_id device_id) {
+  bool ExtensionStringPresent(void) {
     cl_int result = 0;
+    cl_platform_id platform;
+    cl_device_id device_id;
+    result = clGetPlatformIDs(1, &platform, NULL);
+    ICHECK(result == CL_SUCCESS) << "clGetPlatformIDs:" << result;
+    uint32_t num_devices = 0;
+    result = clGetDeviceIDs(platform, CL_DEVICE_TYPE_GPU, 0, NULL, &num_devices);
+    ICHECK(result == CL_SUCCESS && num_devices == 1) << "clGetDeviceIDs:" << result;
+    result = clGetDeviceIDs(platform, CL_DEVICE_TYPE_GPU, 1, &device_id, NULL);
+    ICHECK(device_id && result == CL_SUCCESS) << "clGetDeviceIDs:" << result;

Review Comment:
   Why do you need this code? You can get `device_id` from the `workspace`:
   ```
   cl_device_id did = workspace->devices[workspace->GetThreadEntry()->device.device_id];
   ```
   
   I think that the best solution is to extend class `OpenCLWorkspace` with method `cl_device_id GetClDeviceId(Device dev)`. This method also will do all necessary checks. I think, implementation of this method will be similar with implementation of [GetQueue](https://github.com/apache/tvm/blob/main/src/runtime/opencl/opencl_common.h#L262-L268).
   
   And after that you'll be able to get `cl_device_id` by calling this new method.



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] echuraev commented on a diff in pull request #12711: [OpenCLML] CLML Profiling fixes corresponding to OpenCL Timer recent …

Posted by GitBox <gi...@apache.org>.
echuraev commented on code in PR #12711:
URL: https://github.com/apache/tvm/pull/12711#discussion_r964492908


##########
src/runtime/contrib/clml/clml_runtime.cc:
##########
@@ -499,8 +484,17 @@ class CLMLRuntime : public JSONRuntimeBase {
     uint32_t n, c, h, w;
   };
 
-  bool ExtensionStringPresent(cl_device_id device_id) {
+  bool ExtensionStringPresent(void) {
     cl_int result = 0;
+    cl_platform_id platform;
+    cl_device_id device_id;
+    result = clGetPlatformIDs(1, &platform, NULL);
+    ICHECK(result == CL_SUCCESS) << "clGetPlatformIDs:" << result;
+    uint32_t num_devices = 0;
+    result = clGetDeviceIDs(platform, CL_DEVICE_TYPE_GPU, 0, NULL, &num_devices);
+    ICHECK(result == CL_SUCCESS && num_devices == 1) << "clGetDeviceIDs:" << result;
+    result = clGetDeviceIDs(platform, CL_DEVICE_TYPE_GPU, 1, &device_id, NULL);
+    ICHECK(device_id && result == CL_SUCCESS) << "clGetDeviceIDs:" << result;

Review Comment:
   > The device_id from workspace is a sequence number in our case it's always 0.
   
   I'm not agreed with you. You can also get real id from driver. In your code, you get the number of available GPU devices and then get id of one GPU device. 
   
   Please, take a look into the method [Init](https://github.com/apache/tvm/blob/main/src/runtime/opencl/opencl_device_api.cc#L412-L459). We do absolutely the same things and [here](https://github.com/apache/tvm/blob/main/src/runtime/opencl/opencl_device_api.cc#L430) we create a vector of real device ids from driver. And next we store these values in a member `devices`. So you can extract the real device id from this vector. 
   
   When you call `workspace->Init();` it will also create an OpenCL workspace for GPU device. In case if no GPU devices found, it will print a warning message. If you need only a GPU device, then you can add a check/assert for it in the CLML specific code.
   
   Only one thing why I suggested adding a new method `GetClDeviceId` because in this method we can do all necessary boundary and other checks.



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] masahi merged pull request #12711: [OpenCLML] CLML Profiling fixes corresponding to OpenCL Timer recent …

Posted by GitBox <gi...@apache.org>.
masahi merged PR #12711:
URL: https://github.com/apache/tvm/pull/12711


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] srkreddy1238 commented on a diff in pull request #12711: [OpenCLML] CLML Profiling fixes corresponding to OpenCL Timer recent …

Posted by GitBox <gi...@apache.org>.
srkreddy1238 commented on code in PR #12711:
URL: https://github.com/apache/tvm/pull/12711#discussion_r964440891


##########
src/runtime/contrib/clml/clml_runtime.cc:
##########
@@ -499,8 +484,17 @@ class CLMLRuntime : public JSONRuntimeBase {
     uint32_t n, c, h, w;
   };
 
-  bool ExtensionStringPresent(cl_device_id device_id) {
+  bool ExtensionStringPresent(void) {
     cl_int result = 0;
+    cl_platform_id platform;
+    cl_device_id device_id;
+    result = clGetPlatformIDs(1, &platform, NULL);
+    ICHECK(result == CL_SUCCESS) << "clGetPlatformIDs:" << result;
+    uint32_t num_devices = 0;
+    result = clGetDeviceIDs(platform, CL_DEVICE_TYPE_GPU, 0, NULL, &num_devices);
+    ICHECK(result == CL_SUCCESS && num_devices == 1) << "clGetDeviceIDs:" << result;
+    result = clGetDeviceIDs(platform, CL_DEVICE_TYPE_GPU, 1, &device_id, NULL);
+    ICHECK(device_id && result == CL_SUCCESS) << "clGetDeviceIDs:" << result;

Review Comment:
   The device_id from workspace is a sequence number in our case it's always ```0```. What we need here is the real id from driver.



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] echuraev commented on a diff in pull request #12711: [OpenCLML] CLML Profiling fixes corresponding to OpenCL Timer recent …

Posted by GitBox <gi...@apache.org>.
echuraev commented on code in PR #12711:
URL: https://github.com/apache/tvm/pull/12711#discussion_r964495712


##########
src/runtime/contrib/clml/clml_runtime.cc:
##########
@@ -286,21 +265,26 @@ class CLMLRuntime : public JSONRuntimeBase {
     }
 
     for (size_t i = 0; i < this->layer_.function.size(); ++i) {
-      this->evts->resize(this->evts->size() + 1);
-      cl_event* evt = &(this->evts->back());
-      result = h_ClmlIntf->clEnqueueMLOpQCOM(queue, this->layer_.function[i],
-                                             this->layer_.descriptorSet, 0, NULL, evt);
+      if (getenv("CLML_PROFILING")) {

Review Comment:
   Thank you for clarification. In this case, I agree with you. It makes no sense to use `IsProfiling` here.



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] srkreddy1238 commented on a diff in pull request #12711: [OpenCLML] CLML Profiling fixes corresponding to OpenCL Timer recent …

Posted by GitBox <gi...@apache.org>.
srkreddy1238 commented on code in PR #12711:
URL: https://github.com/apache/tvm/pull/12711#discussion_r964483099


##########
src/runtime/contrib/clml/clml_runtime.cc:
##########
@@ -1056,11 +1055,8 @@ class CLMLRuntime : public JSONRuntimeBase {
   CachedLayer layer_;
   // CLML Context
   CLMLInterfaceV2QCOM* h_ClmlIntf = NULL;
-  cl_platform_id platform = NULL;
-  cl_context context = NULL;
-  cl_device_id device_id = NULL;
-  cl_command_queue queue = NULL;
-  std::vector<cl_event>* evts;
+  cl::OpenCLWorkspace* workspace = NULL;
+  cl::OpenCLThreadEntry* tentry = NULL;

Review Comment:
   Just for convenience (also it wont change anyway) to minimize long path of accessing queue.



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] srkreddy1238 commented on a diff in pull request #12711: [OpenCLML] CLML Profiling fixes corresponding to OpenCL Timer recent …

Posted by GitBox <gi...@apache.org>.
srkreddy1238 commented on code in PR #12711:
URL: https://github.com/apache/tvm/pull/12711#discussion_r963810992


##########
src/runtime/contrib/clml/clml_runtime.cc:
##########
@@ -146,22 +169,7 @@ class CLMLRuntime : public JSONRuntimeBase {
       LOG(WARNING) << "CLML Runtime Init: Qualcomm extn not present.\n";
       return;
     }
-
-    // Reuse the OpenCl work space from TVM Device API.
-    auto func = tvm::runtime::Registry::Get("device_api.opencl");
-    ICHECK(func != nullptr) << "Cannot find OpenCL device_api in registry";
-    auto device_api = static_cast<cl::OpenCLWorkspace*>(((*func)()).operator void*());
-    this->context = device_api->context;
-    bool queue_found = false;
-    for (size_t i = 0; i < device_api->devices.size(); ++i) {
-      if (device_api->devices[i] == device_id) {
-        this->queue = device_api->queues[i];
-        this->evts = &(device_api->events[i]);
-        queue_found = true;
-      }
-    }
-    ICHECK(queue_found != false) << "Device queue not found in OpenCL Workspace";
-
+    this->queue = GetCommadQueue();

Review Comment:
   Both approaches leads to the same issue.
   
   ```this->queue = workspace->GetQueue(t->device);``` 
   The queue referenced here is getting recreated at https://github.com/apache/tvm/pull/11180/files#diff-783a8419915dfdfe242ef786bce4252f60f9d0decebea66d72af63d9cf9d0058R472
   
   and OpenCLML runtime is unaware of this recreation and continues to use a released command queue.



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] echuraev commented on a diff in pull request #12711: [OpenCLML] CLML Profiling fixes corresponding to OpenCL Timer recent …

Posted by GitBox <gi...@apache.org>.
echuraev commented on code in PR #12711:
URL: https://github.com/apache/tvm/pull/12711#discussion_r964623519


##########
src/runtime/contrib/clml/clml_runtime.cc:
##########
@@ -499,15 +483,18 @@ class CLMLRuntime : public JSONRuntimeBase {
     uint32_t n, c, h, w;
   };
 
-  bool ExtensionStringPresent(cl_device_id device_id) {
+  bool ExtensionStringPresent(void) {
     cl_int result = 0;
-
+    if (workspace->platform_id == nullptr) {
+      return 0;
+    }
     size_t reqd_size = 0;
-    result = clGetDeviceInfo(device_id, CL_DEVICE_EXTENSIONS, 0, NULL, &reqd_size);
+    result = clGetDeviceInfo(workspace->devices[0], CL_DEVICE_EXTENSIONS, 0, NULL, &reqd_size);

Review Comment:
   I agree with you that in current implementation, `device_id` is always zero. So, we can just get the first device in the vector. 
   
   But I have one point for a discussion. I'm afraid that if we change something in the `OpenCLWorkspace` related to the list of the devices in the future, then we can break this piece of code. So, this is why I prefer to get the device in this way: `workspace->devices[workspace->GetThreadEntry()->device.device_id]`. To make it more safe and readable, I proposed to introduce a new method.



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] srkreddy1238 commented on a diff in pull request #12711: [OpenCLML] CLML Profiling fixes corresponding to OpenCL Timer recent …

Posted by GitBox <gi...@apache.org>.
srkreddy1238 commented on code in PR #12711:
URL: https://github.com/apache/tvm/pull/12711#discussion_r964641590


##########
src/runtime/contrib/clml/clml_runtime.cc:
##########
@@ -499,15 +483,18 @@ class CLMLRuntime : public JSONRuntimeBase {
     uint32_t n, c, h, w;
   };
 
-  bool ExtensionStringPresent(cl_device_id device_id) {
+  bool ExtensionStringPresent(void) {
     cl_int result = 0;
-
+    if (workspace->platform_id == nullptr) {
+      return 0;
+    }
     size_t reqd_size = 0;
-    result = clGetDeviceInfo(device_id, CL_DEVICE_EXTENSIONS, 0, NULL, &reqd_size);
+    result = clGetDeviceInfo(workspace->devices[0], CL_DEVICE_EXTENSIONS, 0, NULL, &reqd_size);

Review Comment:
   Its very unlikely for Snapdragon to have multi GPU (at least near future). Anyway, I modified it to be generic.



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] masahi commented on pull request #12711: [OpenCLML] CLML Profiling fixes corresponding to OpenCL Timer recent …

Posted by GitBox <gi...@apache.org>.
masahi commented on PR #12711:
URL: https://github.com/apache/tvm/pull/12711#issuecomment-1238009369

   cc @echuraev 


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] echuraev commented on a diff in pull request #12711: [OpenCLML] CLML Profiling fixes corresponding to OpenCL Timer recent …

Posted by GitBox <gi...@apache.org>.
echuraev commented on code in PR #12711:
URL: https://github.com/apache/tvm/pull/12711#discussion_r963695735


##########
src/runtime/contrib/clml/clml_runtime.cc:
##########
@@ -146,22 +169,7 @@ class CLMLRuntime : public JSONRuntimeBase {
       LOG(WARNING) << "CLML Runtime Init: Qualcomm extn not present.\n";
       return;
     }
-
-    // Reuse the OpenCl work space from TVM Device API.
-    auto func = tvm::runtime::Registry::Get("device_api.opencl");
-    ICHECK(func != nullptr) << "Cannot find OpenCL device_api in registry";
-    auto device_api = static_cast<cl::OpenCLWorkspace*>(((*func)()).operator void*());
-    this->context = device_api->context;
-    bool queue_found = false;
-    for (size_t i = 0; i < device_api->devices.size(); ++i) {
-      if (device_api->devices[i] == device_id) {
-        this->queue = device_api->queues[i];
-        this->evts = &(device_api->events[i]);
-        queue_found = true;
-      }
-    }
-    ICHECK(queue_found != false) << "Device queue not found in OpenCL Workspace";
-
+    this->queue = GetCommadQueue();

Review Comment:
   Sorry, probably I missed something. I have never looked at OpenCLML part of TVM before. Could you please explain why you didn't reuse code from OpenCL runtime?
   Won't this pseudocode work in the same way?
   ```suggestion
     void InitCLML() {
       // Setup CLML Context
       cl_int result = 0;
       // Initialize Context and Command Queue
       OpenCLWorkspace* workspace = OpenCLWorkspace::Global();
       workspace->Init();
       OpenCLThreadEntry* t = workspace->GetThreadEntry();
       cl_device_id did = workspace->devices[t->device.device_id];
       
       if (!ExtensionStringPresent(did)) {
         LOG(WARNING) << "CLML Runtime Init: Qualcomm extn not present.\n";
         return;
       }
       this->queue = workspace->GetQueue(t->device);
   ```
   
   If it works, it won't be necessary to implement function `GetCommadQueue`.



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] srkreddy1238 commented on pull request #12711: [OpenCLML] CLML Profiling fixes corresponding to OpenCL Timer recent …

Posted by GitBox <gi...@apache.org>.
srkreddy1238 commented on PR #12711:
URL: https://github.com/apache/tvm/pull/12711#issuecomment-1240209206

   @masahi can you take a look and merge 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.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] echuraev commented on a diff in pull request #12711: [OpenCLML] CLML Profiling fixes corresponding to OpenCL Timer recent …

Posted by GitBox <gi...@apache.org>.
echuraev commented on code in PR #12711:
URL: https://github.com/apache/tvm/pull/12711#discussion_r963830132


##########
src/runtime/contrib/clml/clml_runtime.cc:
##########
@@ -146,22 +169,7 @@ class CLMLRuntime : public JSONRuntimeBase {
       LOG(WARNING) << "CLML Runtime Init: Qualcomm extn not present.\n";
       return;
     }
-
-    // Reuse the OpenCl work space from TVM Device API.
-    auto func = tvm::runtime::Registry::Get("device_api.opencl");
-    ICHECK(func != nullptr) << "Cannot find OpenCL device_api in registry";
-    auto device_api = static_cast<cl::OpenCLWorkspace*>(((*func)()).operator void*());
-    this->context = device_api->context;
-    bool queue_found = false;
-    for (size_t i = 0; i < device_api->devices.size(); ++i) {
-      if (device_api->devices[i] == device_id) {
-        this->queue = device_api->queues[i];
-        this->evts = &(device_api->events[i]);
-        queue_found = true;
-      }
-    }
-    ICHECK(queue_found != false) << "Device queue not found in OpenCL Workspace";
-
+    this->queue = GetCommadQueue();

Review Comment:
   Yes, I agree with you. One thing that I wanted to say that probably we could unify and reuse existing code instead of writing new similar functions.
   Speaking about recreation function. So, in function `recreateCommandQueue` a new queue is creating and storing in the vector with queues. And when we call  `GetQueue` then a new (recreated) queue will be returned. I don't see any reasons why we cannot call `cl::OpenCLWorkspace::Global()->GetQueue(device)` in all places where we'd like to use command queue and get a new queue.  What do you think about that? 



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] srkreddy1238 commented on a diff in pull request #12711: [OpenCLML] CLML Profiling fixes corresponding to OpenCL Timer recent …

Posted by GitBox <gi...@apache.org>.
srkreddy1238 commented on code in PR #12711:
URL: https://github.com/apache/tvm/pull/12711#discussion_r964461276


##########
src/runtime/contrib/clml/clml_runtime.cc:
##########
@@ -286,21 +265,26 @@ class CLMLRuntime : public JSONRuntimeBase {
     }
 
     for (size_t i = 0; i < this->layer_.function.size(); ++i) {
-      this->evts->resize(this->evts->size() + 1);
-      cl_event* evt = &(this->evts->back());
-      result = h_ClmlIntf->clEnqueueMLOpQCOM(queue, this->layer_.function[i],
-                                             this->layer_.descriptorSet, 0, NULL, evt);
+      if (getenv("CLML_PROFILING")) {

Review Comment:
   CLML profiling is about profiling the ML op's within CLML sub graph (within BYOC).  [isProfiling](https://github.com/apache/tvm/blob/main/src/runtime/opencl/opencl_common.h#L278-L286) is controlled by OpenCLTimer when ever someone want to profile OpenCL kernels (Generated by OpenCL Codegen). CLML doesn't have any kernels (no clEnqueueNDRangeKernel calls here) instead it has extension API.
   
   
   More details, 
   
   Ideally, CLML can have it's own workspace (context & queue) and operate. The only dependency on TVM's OpenCL workspace is to have the buffers allocated on same queue so that we can do hardware level copy while context switching from TVM's OpenCL sub graph to CLML subgraph. Too tight integration here may lead to unexpected functionality break as those who enhance OpenCL runtime may not pay attention to CLML component dependencies.



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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


[GitHub] [tvm] echuraev commented on a diff in pull request #12711: [OpenCLML] CLML Profiling fixes corresponding to OpenCL Timer recent …

Posted by GitBox <gi...@apache.org>.
echuraev commented on code in PR #12711:
URL: https://github.com/apache/tvm/pull/12711#discussion_r963695735


##########
src/runtime/contrib/clml/clml_runtime.cc:
##########
@@ -146,22 +169,7 @@ class CLMLRuntime : public JSONRuntimeBase {
       LOG(WARNING) << "CLML Runtime Init: Qualcomm extn not present.\n";
       return;
     }
-
-    // Reuse the OpenCl work space from TVM Device API.
-    auto func = tvm::runtime::Registry::Get("device_api.opencl");
-    ICHECK(func != nullptr) << "Cannot find OpenCL device_api in registry";
-    auto device_api = static_cast<cl::OpenCLWorkspace*>(((*func)()).operator void*());
-    this->context = device_api->context;
-    bool queue_found = false;
-    for (size_t i = 0; i < device_api->devices.size(); ++i) {
-      if (device_api->devices[i] == device_id) {
-        this->queue = device_api->queues[i];
-        this->evts = &(device_api->events[i]);
-        queue_found = true;
-      }
-    }
-    ICHECK(queue_found != false) << "Device queue not found in OpenCL Workspace";
-
+    this->queue = GetCommadQueue();

Review Comment:
   Sorry, probably I missed something. Could you please explain why you didn't reuse code from OpenCL runtime?
   Won't this pseudocode work in the same way?
   ```suggestion
     void InitCLML() {
       // Setup CLML Context
       cl_int result = 0;
       // Initialize Context and Command Queue
       OpenCLWorkspace* workspace = OpenCLWorkspace::Global();
       workspace->Init();
       OpenCLThreadEntry* t = workspace->GetThreadEntry();
       cl_device_id did = workspace->devices[t->device.device_id];
       
       if (!ExtensionStringPresent(did)) {
         LOG(WARNING) << "CLML Runtime Init: Qualcomm extn not present.\n";
         return;
       }
       this->queue = workspace->GetQueue(t->device);
   ```
   
   If it works, it won't be necessary to implement function `GetCommadQueue`.



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@tvm.apache.org

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