diff --git a/.gitmodules b/.gitmodules index ced5dcf94..ec484eb61 100644 --- a/.gitmodules +++ b/.gitmodules @@ -17,3 +17,7 @@ [submodule "third_party/json"] path = third_party/json url = https://github.com/nlohmann/json + +[submodule "third_party/dlpack"] + path = third_party/dlpack + url = https://github.com/dmlc/dlpack diff --git a/.vscode/settings.json b/.vscode/settings.json index 640196a66..00260f078 100644 --- a/.vscode/settings.json +++ b/.vscode/settings.json @@ -3,8 +3,6 @@ "cmake.environment": { "ARK_ROOT": "${workspaceFolder}/build", "ARK_IGNORE_BINARY_CACHE": "1", - "ARK_DISABLE_GRAPH_OPT": "0", - "ARK_IPC_LISTEN_PORT_BASE": "42000", // "ARK_LOG_LEVEL": "DEBUG" }, "cmake.ctestArgs": [ diff --git a/CMakeLists.txt b/CMakeLists.txt index 2e80ea1e8..c3b09b0e6 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -17,6 +17,7 @@ option(ARK_USE_CUDA "Use NVIDIA/CUDA." OFF) option(ARK_USE_ROCM "Use AMD/ROCm." OFF) option(ARK_BYPASS_GPU_CHECK "Bypass GPU check." OFF) option(ARK_BUILD_TESTS "Build unit tests." ON) +option(ARK_BUILD_PYTHON "Build Python module." ON) if(ARK_BYPASS_GPU_CHECK) if(ARK_USE_CUDA) diff --git a/ark/CMakeLists.txt b/ark/CMakeLists.txt index 208d9f9cb..9616ea875 100644 --- a/ark/CMakeLists.txt +++ b/ark/CMakeLists.txt @@ -17,6 +17,7 @@ set(COMMON_LIBS ARK::numa ARK::ibverbs pthread rt) target_include_directories(ark_obj PUBLIC ${CMAKE_CURRENT_SOURCE_DIR}/include) target_include_directories(ark_obj PRIVATE ${CMAKE_CURRENT_SOURCE_DIR}) target_include_directories(ark_obj SYSTEM PRIVATE + ${DLPACK_INCLUDE_DIRS} ${JSON_INCLUDE_DIRS} ${MSCCLPP_INCLUDE_DIRS} ${IBVERBS_INCLUDE_DIRS} diff --git a/ark/api/executor.cpp b/ark/api/executor.cpp index d9fc9217b..17d579763 100644 --- a/ark/api/executor.cpp +++ b/ark/api/executor.cpp @@ -3,24 +3,29 @@ #include "ark/executor.hpp" +#include + #include #include #include #include #include +#include +#include "ark/data_type.hpp" #include "ark/model.hpp" #include "ark/planner.hpp" #include "codegen.hpp" #include "env.h" #include "file_io.h" -#include "gpu/gpu.h" -#include "gpu/gpu_event.h" -#include "gpu/gpu_kernel.h" -#include "gpu/gpu_logging.h" -#include "gpu/gpu_manager.h" +#include "gpu/gpu.hpp" +#include "gpu/gpu_event.hpp" +#include "gpu/gpu_kernel.hpp" +#include "gpu/gpu_logging.hpp" +#include "gpu/gpu_manager.hpp" #include "logging.hpp" #include "model/model_buffer.hpp" +#include "model_buffer_manager.hpp" #include "model/model_data_type.hpp" #include "model/model_tensor.hpp" #include "utils/utils_net.hpp" @@ -140,20 +145,30 @@ static size_t tensor_stride_bytes(const Json &tensor) { class Executor::Impl { public: - Impl(int rank, int world_size, int gpu_id, const std::string &name, - const std::string &plan); - ~Impl() = default; + Impl(int device_id, Stream stream, const std::string &name, bool loop_mode); + ~Impl(); + + void init(const PlanJson& plan); + + int device_id() const { return device_id_; } + + Stream stream() const { return reinterpret_cast(stream_raw_); } + + std::string plan() const { return plan_json_.dump_pretty(); } void compile(); - void launch(int64_t max_spin_count); + void launch(); void run(int iter); void wait(int64_t max_spin_count); float stop(int64_t max_spin_count); void barrier(); - void tensor_read(const Tensor tensor, void *data, size_t bytes) const; - void tensor_write(const Tensor tensor, const void *data, - size_t bytes) const; + uintptr_t tensor_address(const Tensor tensor) const; + + void tensor_read(const Tensor tensor, void *data, size_t bytes, + Stream stream, bool is_d2d) const; + void tensor_write(const Tensor tensor, const void *data, size_t bytes, + Stream stream, bool is_d2d) const; private: void init_communicator(); @@ -162,14 +177,20 @@ class Executor::Impl { void init_channels(const std::set &remote_ranks); protected: - const int rank_; - const int world_size_; - int gpu_id_; + int device_id_; + std::string name_; + bool loop_mode_; + + gpuStream stream_raw_; + + int rank_; + int world_size_; bool is_launched_ = false; bool is_recording_ = false; float elapsed_msec_ = -1; + PlanJson plan_json_; std::map buffer_id_to_offset_; size_t total_bytes_; std::shared_ptr codegen_; @@ -177,8 +198,7 @@ class Executor::Impl { std::shared_ptr timer_end_; std::shared_ptr buffer_; std::shared_ptr flag_; - std::shared_ptr main_stream_; - std::shared_ptr copy_stream_; + std::shared_ptr stream_; std::shared_ptr kernel_; // For communication @@ -190,30 +210,38 @@ class Executor::Impl { rank_to_sm_channels_; }; -Executor::Impl::Impl(int rank, int world_size, int gpu_id, - const std::string &name, const std::string &plan) - : rank_(rank), world_size_(world_size), gpu_id_(gpu_id) { - if (rank < 0 || rank >= world_size) { - ERR(InvalidUsageError, "Invalid rank ", rank, " with world size ", - world_size); +Executor::Impl::Impl(int device_id, Stream stream, const std::string &name, + bool loop_mode) + : device_id_(device_id), name_(name), loop_mode_(loop_mode) { + if (device_id < 0) { + ERR(InvalidUsageError, "Invalid device ID ", device_id); } - if (gpu_id < 0) { - ERR(InvalidUsageError, "Invalid GPU ID ", gpu_id); + if (stream) { + stream_raw_ = reinterpret_cast(stream); + } else { + stream_ = GpuManager::get_instance(device_id_)->create_stream(); + stream_raw_ = stream_->get(); + } +} + +Executor::Impl::~Impl() { + if (is_launched_) stop(-1); +} + +void Executor::Impl::init(const PlanJson &plan_json) { + plan_json_ = plan_json; + rank_ = plan_json_["Rank"].get(); + world_size_ = plan_json_["WorldSize"].get(); + + if (rank_ < 0 || rank_ >= world_size_) { + ERR(InvalidUsageError, "Invalid rank ", rank_, " with world size ", + world_size_); } if (world_size_ > 1) { init_communicator(); } - Json plan_json; - auto &plan_path = get_env().enforce_plan_path; - if (!plan_path.empty()) { - LOG(INFO, "Enforce executor plan path: ", plan_path); - plan_json = Json::parse(read_file(plan_path)); - } else { - plan_json = Json::parse(plan); - } - - auto gpu_manager = GpuManager::get_instance(gpu_id_); + auto gpu_manager = GpuManager::get_instance(device_id_); if (!gpu_manager->info().arch->belongs_to( Arch::from_name(plan_json.at("Architecture")))) { LOG(WARN, "Architecture name of the plan `", @@ -222,7 +250,7 @@ Executor::Impl::Impl(int rank, int world_size, int gpu_id, gpu_manager->info().arch->name(), "`."); } - buffer_id_to_offset_ = init_buffers(plan_json); + buffer_id_to_offset_ = init_buffers(plan_json_); std::string buffer_id_to_offset_str; for (const auto &kv : buffer_id_to_offset_) { @@ -230,34 +258,39 @@ Executor::Impl::Impl(int rank, int world_size, int gpu_id, std::to_string(kv.first) + ": " + std::to_string(kv.second) + ", "; } - codegen_ = - std::make_shared(plan_json, buffer_id_to_offset_, name); + codegen_ = std::make_shared(plan_json_, buffer_id_to_offset_, + name_); timer_begin_ = gpu_manager->create_event(); timer_end_ = gpu_manager->create_event(); buffer_ = gpu_manager->malloc(total_bytes_, 65536); flag_ = gpu_manager->malloc_host( sizeof(int), gpuHostAllocMapped | gpuHostAllocWriteCombined); - main_stream_ = gpu_manager->create_stream(); - copy_stream_ = gpu_manager->create_stream(); int threads_per_block = static_cast( codegen_->num_warps_per_proc() * gpu_manager->info().threads_per_warp); int num_sm = static_cast(codegen_->num_procs()); - int *flag = flag_->ref(); size_t smem_block_total = static_cast(gpu_manager->info().smem_block_total); if (world_size_ > 1) { - auto remote_ranks = init_remote_ranks(plan_json); + auto remote_ranks = init_remote_ranks(plan_json_); init_channels(remote_ranks); } + std::string kernel_name; + if (loop_mode_) { + kernel_name = "ark_loop_kernel"; + } else { + kernel_name = "ark_kernel"; + } + if (!name_.empty()) { + kernel_name += "_" + name_; + } + kernel_ = std::shared_ptr(new GpuKernel( - gpu_id_, codegen_->code(), {threads_per_block, 1, 1}, {num_sm, 1, 1}, - std::max(smem_block_total, size_t(4)), name, - {std::pair{buffer_->ref(), sizeof(buffer_->ref())}, - std::pair{flag, sizeof(flag)}})); + device_id_, codegen_->code(), {threads_per_block, 1, 1}, {num_sm, 1, 1}, + std::max(smem_block_total, size_t(4)), kernel_name)); } void Executor::Impl::init_communicator() { @@ -362,7 +395,16 @@ std::map Executor::Impl::init_buffers(const Json &plan_json) { } continue; } - buffer_id_to_offset[buf_info->buffer->id()] = offset; + if (buf_info->buffer->is_external()) { + if (buf_info->buffer->device_id() != device_id_) { + ERR(InvalidUsageError, + "PyTorch tensor and model execution are on different GPUs"); + } + continue; + } else { + buffer_id_to_offset[buf_info->buffer->id()] = offset; + offset += buf_info->bytes; + } for (const auto &tag_info : buf_info->buffer->send_tags()) { remote_rank_to_send_tags_and_offsets[tag_info.first] .first.push_back(tag_info.second); @@ -375,7 +417,6 @@ std::map Executor::Impl::init_buffers(const Json &plan_json) { remote_rank_to_recv_tags_and_offsets[tag_info.first] .second.push_back(offset); } - offset += buf_info->bytes; } total_bytes_ = offset; @@ -451,7 +492,11 @@ std::map Executor::Impl::init_buffers(const Json &plan_json) { bootstrap->recv(tags.data(), len * sizeof(int), remote_rank, 1); bootstrap->recv(offsets.data(), len * sizeof(size_t), remote_rank, 2); for (int i = 0; i < len; ++i) { - buffer_id_to_offset[send_tag_to_buffer_id[tags[i]]] = offsets[i]; + if (!buffer_id_to_info[send_tag_to_buffer_id[tags[i]]] + ->buffer->is_external()) { + buffer_id_to_offset[send_tag_to_buffer_id[tags[i]]] = + offsets[i]; + } } } for (auto &kv : remote_rank_to_recv_tag_to_buffer_id) { @@ -467,10 +512,13 @@ std::map Executor::Impl::init_buffers(const Json &plan_json) { bootstrap->recv(tags.data(), len * sizeof(int), remote_rank, 4); bootstrap->recv(offsets.data(), len * sizeof(size_t), remote_rank, 5); for (int i = 0; i < len; ++i) { - buffer_id_to_offset[recv_tag_to_buffer_id[tags[i]]] = offsets[i]; + if (!buffer_id_to_info[recv_tag_to_buffer_id[tags[i]]] + ->buffer->is_external()) { + buffer_id_to_offset[recv_tag_to_buffer_id[tags[i]]] = + offsets[i]; + } } } - return buffer_id_to_offset; } @@ -517,7 +565,7 @@ void Executor::Impl::init_channels(const std::set &remote_ranks) { mscclpp::TransportFlags all_transports = mscclpp::Transport::CudaIpc | mscclpp::Transport::Ethernet; if (!get_env().disable_ib) { - all_transports |= IBs[gpu_id_]; + all_transports |= IBs[device_id_]; } mscclpp::RegisteredMemory regmem = comm_->registerMemory(buffer_->ref(), buffer_->bytes(), all_transports); @@ -538,12 +586,12 @@ void Executor::Impl::init_channels(const std::set &remote_ranks) { if (remote_node == this_node) { add_connection(remote_rank, mscclpp::Transport::CudaIpc); if (!get_env().disable_ib) { - add_connection(remote_rank, IBs[gpu_id_]); + add_connection(remote_rank, IBs[device_id_]); } } else { add_connection(remote_rank, get_env().disable_ib ? mscclpp::Transport::Ethernet - : IBs[gpu_id_]); + : IBs[device_id_]); } comm_->sendMemoryOnSetup(regmem, remote_rank, 0); rank_to_remote_regmem_future[remote_rank] = @@ -596,13 +644,12 @@ void Executor::Impl::init_channels(const std::set &remote_ranks) { void Executor::Impl::compile() { kernel_->compile(); } -void Executor::Impl::launch(int64_t max_spin_count) { +void Executor::Impl::launch() { if (!kernel_->is_compiled()) { ERR(InvalidUsageError, "Need to compile first before initialization."); } if (is_launched_) { - // Wait until previous works finish. - this->wait(max_spin_count); + LOG(WARN, "Ignore launching twice."); return; } auto get_global_rt = [&](const std::string &symbol) { @@ -631,83 +678,102 @@ void Executor::Impl::launch(int64_t max_spin_count) { sm_handles[i] = it2->second[0]->deviceHandle(); } } - GLOG(gpuSetDevice(gpu_id_)); + GLOG(gpuSetDevice(device_id_)); GLOG(gpuMemcpyAsync( proxy_chan_addr, proxy_handles.data(), proxy_handles.size() * sizeof(mscclpp::SimpleProxyChannel::DeviceHandle), - gpuMemcpyHostToDevice, copy_stream_->get())); + gpuMemcpyHostToDevice, stream_raw_)); GLOG(gpuMemcpyAsync( proxy_secondary_chan_addr, proxy_secondary_handles.data(), proxy_secondary_handles.size() * sizeof(mscclpp::SimpleProxyChannel::DeviceHandle), - gpuMemcpyHostToDevice, copy_stream_->get())); + gpuMemcpyHostToDevice, stream_raw_)); GLOG(gpuMemcpyAsync( sm_chan_addr, sm_handles.data(), sm_handles.size() * sizeof(mscclpp::SmChannel::DeviceHandle), - gpuMemcpyHostToDevice, copy_stream_->get())); - copy_stream_->sync(); + gpuMemcpyHostToDevice, stream_raw_)); + GLOG(gpuStreamSynchronize(stream_raw_)); } elapsed_msec_ = -1; - if (!kernel_->is_compiled()) { - ERR(InvalidUsageError, "Need to compile first before initialization."); - } else if (is_launched_) { - LOG(WARN, "Ignore launching twice."); - return; - } - timer_begin_->record(main_stream_); + timer_begin_->record(stream_raw_); if (world_size_ > 1) { proxy_service_->startProxy(); } - // Initialize loop flags. - atomicStoreRelaxed(flag_->ref(), 0); - kernel_->launch(main_stream_); - timer_end_->record(main_stream_); + if (loop_mode_) { + // Initialize loop flags. + atomicStoreRelaxed(flag_->ref(), 0); + void *buf_ptr = buffer_->ref(); + void *flag_ptr = flag_->ref(); + std::vector args = {&buf_ptr, &flag_ptr}; + kernel_->launch(stream_raw_, args); + } is_recording_ = true; is_launched_ = true; } void Executor::Impl::run(int iter) { - if (iter > 0) { + if (iter <= 0) return; + if (loop_mode_) { while (atomicLoadRelaxed(flag_->ref()) > 0) { } atomicStoreRelaxed(flag_->ref(), iter); + } else { + void *buf_ptr = buffer_->ref(); + int i = 0; + std::vector args = {&buf_ptr, reinterpret_cast(&i)}; + for (; i < iter; i++) { + kernel_->launch(stream_raw_, args); + } } } void Executor::Impl::wait(int64_t max_spin_count) { int64_t cnt = max_spin_count; - while (atomicLoadRelaxed(flag_->ref()) > 0) { - if (cnt-- > 0) { - continue; - } - // Check if the kernel encountered an error. - gpuError res = main_stream_->query(); - if (res == gpuSuccess) { - if (atomicLoadRelaxed(flag_->ref()) > 0) { - LOG(WARN, "Stream is finished but the loop flag is still set."); - break; + if (loop_mode_) { + while (atomicLoadRelaxed(flag_->ref()) > 0) { + if (cnt-- > 0) { + continue; + } + // Check if the kernel encountered an error. + gpuError res = gpuStreamQuery(stream_raw_); + if (res == gpuSuccess) { + if (atomicLoadRelaxed(flag_->ref()) > 0) { + LOG(WARN, + "Stream is finished but the loop flag is still set."); + break; + } else { + LOG(WARN, + "wait() is delayed by a stream query. Regarding " + "timing measurements may be inaccurate."); + break; + } + } else if (res == gpuErrorNotReady) { + cnt = max_spin_count; } else { - LOG(WARN, - "wait() is delayed by a stream query. Regarding " - "timing measurements may be inaccurate."); - break; + GLOG(res); } - } else if (res == gpuErrorNotReady) { - cnt = max_spin_count; - } else { - GLOG(res); } + } else { + if (max_spin_count >= 0) { + LOG(WARN, "max_spin_count is ignored in non-loop mode."); + } + GLOG(gpuStreamSynchronize(stream_raw_)); } } float Executor::Impl::stop(int64_t max_spin_count) { this->wait(max_spin_count); - atomicStoreRelaxed(flag_->ref(), -1); - main_stream_->sync(); + if (is_recording_) { + timer_end_->record(stream_raw_); + } + if (loop_mode_) { + atomicStoreRelaxed(flag_->ref(), -1); + } + GLOG(gpuStreamSynchronize(stream_raw_)); if (is_recording_) { elapsed_msec_ = timer_end_->elapsed_msec(*timer_begin_); is_recording_ = false; @@ -725,74 +791,154 @@ void Executor::Impl::barrier() { } } -void Executor::Impl::tensor_read(const Tensor tensor, void *data, - size_t bytes) const { - GLOG(gpuSetDevice(gpu_id_)); +uintptr_t Executor::Impl::tensor_address(const Tensor tensor) const { + size_t buffer_id = tensor.ref()->buffer()->id(); + if (buffer_id_to_offset_.find(buffer_id) == buffer_id_to_offset_.end()) { + ERR(InternalError, "Invalid buffer ID: ", buffer_id); + } + size_t offset = buffer_id_to_offset_.at(buffer_id); + return reinterpret_cast(buffer_->ref(offset)); +} + +void Executor::Impl::tensor_read(const Tensor tensor, void *data, size_t bytes, + Stream stream, bool is_d2d) const { + GLOG(gpuSetDevice(device_id_)); + if (tensor.ref()->buffer()->is_external()) { + ERR(InvalidUsageError, + "Reading data from a tensor preallocated by PyTorch is not " + "supported. Use PyTorch's native methods."); + } + std::shared_ptr copy_stream; + gpuStream copy_stream_raw; + if (stream) { + copy_stream_raw = reinterpret_cast(stream); + if ((stream == stream_raw_) && is_launched_) { + LOG(WARN, + "Reading from a tensor in the same stream of the kernel " + "may cause a deadlock."); + } + } else { + copy_stream = GpuManager::get_instance(device_id_)->create_stream(); + copy_stream_raw = copy_stream->get(); + } size_t tensor_data_bytes = tensor.shape().nelems() * tensor.data_type().bytes(); - if (bytes < tensor_data_bytes) { - ERR(InvalidUsageError, "Data buffer (", bytes, - ") is smaller than the tensor data (", tensor_data_bytes, ")."); + if (bytes != tensor_data_bytes) { + ERR(InvalidUsageError, "Destination bytes (", bytes, + ") mismatches the tensor data bytes (", tensor_data_bytes, ")."); } - size_t tensor_bytes = - tensor.strides().nelems() * tensor.data_type().bytes(); - void *src = - buffer_->ref(buffer_id_to_offset_.at(tensor.ref()->buffer()->id())); + auto kind = (is_d2d) ? gpuMemcpyDeviceToDevice : gpuMemcpyDeviceToHost; + void *src = reinterpret_cast(tensor_address(tensor)); if (tensor.strides() == tensor.shape()) { - GLOG(gpuMemcpyAsync(data, src, bytes, gpuMemcpyDeviceToHost, - copy_stream_->get())); - copy_stream_->sync(); + GLOG(gpuMemcpyAsync(data, src, bytes, kind, copy_stream_raw)); } else { + size_t tensor_bytes = + tensor.strides().nelems() * tensor.data_type().bytes(); std::vector tensor_host(tensor_bytes); GLOG(gpuMemcpyAsync(tensor_host.data(), src, tensor_bytes, - gpuMemcpyDeviceToHost, copy_stream_->get())); - copy_stream_->sync(); - tensor_to_data(tensor_host.data(), static_cast(data), - tensor.shape(), tensor.strides(), tensor.offsets(), + gpuMemcpyDeviceToHost, copy_stream_raw)); + GLOG(gpuStreamSynchronize(copy_stream_raw)); + if (!is_d2d) { + tensor_to_data(tensor_host.data(), static_cast(data), + tensor.shape(), tensor.strides(), tensor.offsets(), + tensor.data_type().bytes()); + return; + } + // TODO: convert data layout on the device directly + std::vector data_host(bytes); + tensor_to_data(tensor_host.data(), data_host.data(), tensor.shape(), + tensor.strides(), tensor.offsets(), tensor.data_type().bytes()); + GLOG(gpuMemcpyAsync(data, data_host.data(), bytes, + gpuMemcpyHostToDevice, copy_stream_raw)); } + GLOG(gpuStreamSynchronize(copy_stream_raw)); } void Executor::Impl::tensor_write(const Tensor tensor, const void *data, - size_t bytes) const { - GLOG(gpuSetDevice(gpu_id_)); + size_t bytes, Stream stream, + bool is_d2d) const { + GLOG(gpuSetDevice(device_id_)); + if (tensor.ref()->buffer()->is_external()) { + ERR(InvalidUsageError, + "Writing data to a tensor preallocated by PyTorch is not " + "supported. Use PyTorch's native methods."); + } + std::shared_ptr copy_stream; + gpuStream copy_stream_raw; + if (stream) { + copy_stream_raw = reinterpret_cast(stream); + if ((stream == stream_raw_) && is_launched_) { + LOG(WARN, + "Writing to a tensor in the same stream of the kernel " + "may cause a deadlock."); + } + } else { + copy_stream = GpuManager::get_instance(device_id_)->create_stream(); + copy_stream_raw = copy_stream->get(); + } size_t tensor_data_bytes = tensor.shape().nelems() * tensor.data_type().bytes(); - if (bytes < tensor_data_bytes) { - ERR(InvalidUsageError, "Data buffer (", bytes, - ") is smaller than the tensor data (", tensor_data_bytes, ")."); + if (bytes != tensor_data_bytes) { + ERR(InvalidUsageError, "Source bytes (", bytes, + ") mismatches the tensor data bytes (", tensor_data_bytes, ")."); } size_t tensor_bytes = tensor.strides().nelems() * tensor.data_type().bytes(); - void *dst = - buffer_->ref(buffer_id_to_offset_.at(tensor.ref()->buffer()->id())); + auto kind = (is_d2d) ? gpuMemcpyDeviceToDevice : gpuMemcpyHostToDevice; + void *dst = reinterpret_cast(tensor_address(tensor)); if (tensor.strides() == tensor.shape()) { - GLOG(gpuMemcpyAsync(dst, data, tensor_bytes, gpuMemcpyHostToDevice, - copy_stream_->get())); + GLOG(gpuMemcpyAsync(dst, data, tensor_bytes, kind, copy_stream_raw)); } else { std::vector tensor_host(tensor_bytes); - GLOG(gpuMemcpyAsync(tensor_host.data(), dst, tensor_bytes, - gpuMemcpyDeviceToHost, copy_stream_->get())); - copy_stream_->sync(); - data_to_tensor(tensor_host.data(), static_cast(data), - tensor.shape(), tensor.strides(), tensor.offsets(), - tensor.data_type().bytes()); + if (!is_d2d) { + GLOG(gpuMemcpyAsync(tensor_host.data(), dst, tensor_bytes, + gpuMemcpyDeviceToHost, copy_stream_raw)); + GLOG(gpuStreamSynchronize(copy_stream_raw)); + data_to_tensor(tensor_host.data(), + static_cast(data), tensor.shape(), + tensor.strides(), tensor.offsets(), + tensor.data_type().bytes()); + } else { + // TODO: convert data layout on the device directly + std::vector tmp(bytes); + GLOG(gpuMemcpyAsync(tmp.data(), data, bytes, gpuMemcpyDeviceToHost, + copy_stream_raw)); + GLOG(gpuStreamSynchronize(copy_stream_raw)); + data_to_tensor(tensor_host.data(), tmp.data(), tensor.shape(), + tensor.strides(), tensor.offsets(), + tensor.data_type().bytes()); + } GLOG(gpuMemcpyAsync(dst, tensor_host.data(), tensor_bytes, - gpuMemcpyHostToDevice, copy_stream_->get())); + gpuMemcpyHostToDevice, copy_stream_raw)); } - copy_stream_->sync(); + GLOG(gpuStreamSynchronize(copy_stream_raw)); } -Executor::Executor(int rank, int world_size, int gpu_id, - const std::string &name, const std::string &plan) - : impl_(std::make_unique(rank, world_size, gpu_id, name, - plan)) {} +Executor::Executor(int device_id, Stream stream, const std::string &name, + const std::string &plan, bool loop_mode) + : impl_(std::make_unique(device_id, stream, name, + loop_mode)) { + auto &plan_path = get_env().enforce_plan_path; + if (!plan_path.empty()) { + LOG(INFO, "Enforce executor plan path: ", plan_path); + impl_->init(Json::parse(read_file(plan_path))); + } else if (!plan.empty()) { + impl_->init(Json::parse(plan)); + } +} Executor::~Executor() = default; +int Executor::device_id() const { return impl_->device_id(); } + +Stream Executor::stream() const { return impl_->stream(); } + +std::string Executor::plan() const { return impl_->plan(); } + void Executor::compile() { impl_->compile(); } -void Executor::launch(int64_t max_spin_count) { impl_->launch(max_spin_count); } +void Executor::launch() { impl_->launch(); } void Executor::run(int iter) { impl_->run(iter); } @@ -804,29 +950,39 @@ float Executor::stop(int64_t max_spin_count) { void Executor::barrier() { impl_->barrier(); } -void Executor::destroy() { impl_.reset(nullptr); } +void Executor::destroy() { + ModelBufferManager::get_instance().clear_buffers(); + impl_.reset(nullptr); +} bool Executor::destroyed() const { return impl_.get() == nullptr; } -void Executor::tensor_read(const Tensor tensor, void *data, - size_t bytes) const { - impl_->tensor_read(tensor, data, bytes); +uintptr_t Executor::tensor_address(const Tensor tensor) const { + return impl_->tensor_address(tensor); } -void Executor::tensor_write(const Tensor tensor, const void *data, - size_t bytes) const { - impl_->tensor_write(tensor, data, bytes); +void Executor::tensor_read(const Tensor tensor, void *data, size_t bytes, + Stream stream, bool is_d2d) const { + impl_->tensor_read(tensor, data, bytes, stream, is_d2d); } -DefaultExecutor::DefaultExecutor(const Model &model, int gpu_id, - const std::string &name) - : Executor( - model.rank(), model.world_size(), - (gpu_id < 0) ? (model.rank() % get_env().num_ranks_per_host) : gpu_id, - name, - Planner(model, (gpu_id < 0) - ? (model.rank() % get_env().num_ranks_per_host) - : gpu_id) - .plan()) {} +void Executor::tensor_write(const Tensor tensor, const void *data, size_t bytes, + Stream stream, bool is_d2d) const { + impl_->tensor_write(tensor, data, bytes, stream, is_d2d); +} + +DefaultExecutor::DefaultExecutor( + const Model &model, int device_id, Stream stream, + const std::vector &config_rules, + const std::string &name, bool loop_mode) + : Executor((device_id < 0) ? (model.rank() % get_env().num_ranks_per_host) + : device_id, + stream, name, "", loop_mode) { + Planner planner(model, impl_->device_id()); + for (const auto &rule : config_rules) { + planner.install_config_rule(rule); + } + impl_->init(Json::parse(planner.plan())); +} } // namespace ark diff --git a/ark/api/executor_test.cpp b/ark/api/executor_test.cpp new file mode 100644 index 000000000..b0b398ac9 --- /dev/null +++ b/ark/api/executor_test.cpp @@ -0,0 +1,150 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +#include "ark/executor.hpp" + +#include "gpu/gpu.hpp" +#include "model/model_json.hpp" +#include "unittest/unittest_utils.h" + +template +ark::unittest::State test_executor() { + ark::gpuStream stream; + UNITTEST_EQ( + ark::gpuStreamCreateWithFlags(&stream, ark::gpuStreamNonBlocking), + ark::gpuSuccess); + + ark::Model empty; + { + ark::DefaultExecutor executor(empty, 0, stream, {}, "test", LoopMode); + UNITTEST_EQ(executor.device_id(), 0); + UNITTEST_EQ(executor.stream(), stream); + + executor.compile(); + executor.launch(); + executor.run(1); + executor.wait(); + executor.stop(); + executor.destroy(); + } + { + ark::DefaultExecutor executor(empty, 0, stream, {}, "test", LoopMode); + executor.compile(); + executor.launch(); + executor.run(1); + executor.wait(); + executor.stop(); + + executor.launch(); + executor.run(1); + executor.wait(); + executor.stop(); + + executor.destroy(); + } + { + ark::DefaultExecutor executor(empty, 0, stream, {}, "test", LoopMode); + UNITTEST_THROW(executor.launch(), ark::InvalidUsageError); + + executor.compile(); + executor.launch(); + executor.launch(); // Will be ignored with a warning. + executor.run(1); + executor.wait(); + executor.wait(); // nothing to do + + // Stop & destroy automatically. + } + + UNITTEST_EQ(ark::gpuStreamDestroy(stream), ark::gpuSuccess); + return ark::unittest::SUCCESS; +} + +ark::unittest::State test_executor_loop() { return test_executor(); } + +ark::unittest::State test_executor_no_loop() { return test_executor(); } + +ark::unittest::State test_executor_tensor_read_write() { + // Alloc CPU array + std::vector host_data(1024); + void *host_ptr = host_data.data(); + for (size_t i = 0; i < host_data.size(); ++i) { + host_data[i] = static_cast(i); + } + + // Alloc GPU array + void *dev_ptr; + UNITTEST_EQ(ark::gpuMalloc(&dev_ptr, 1024 * sizeof(float)), + ark::gpuSuccess); + + // Create an ARK tensor + ark::Model m; + auto tensor = m.tensor({1024}, ark::FP32); + m.noop(tensor); + + ark::DefaultExecutor executor(m, 0); + executor.compile(); + executor.launch(); + + // Copy data from CPU array to ARK tensor + executor.tensor_write(tensor, host_ptr, 1024 * sizeof(float)); + + // Copy data from ARK tensor to GPU array + executor.tensor_read(tensor, dev_ptr, 1024 * sizeof(float), nullptr, true); + + // Check the data + std::vector dev_data(1024); + executor.tensor_read(tensor, dev_data.data(), 1024 * sizeof(float)); + for (size_t i = 0; i < dev_data.size(); ++i) { + UNITTEST_EQ(dev_data[i], static_cast(i)); + dev_data[i] = -1; + } + + UNITTEST_EQ(ark::gpuMemcpy(dev_data.data(), dev_ptr, 1024 * sizeof(float), + ark::gpuMemcpyDeviceToHost), + ark::gpuSuccess); + for (size_t i = 0; i < dev_data.size(); ++i) { + UNITTEST_EQ(dev_data[i], static_cast(i)); + dev_data[i] = -1; + } + + // Copy -1s back to GPU array + UNITTEST_EQ(ark::gpuMemcpy(dev_ptr, dev_data.data(), 1024 * sizeof(float), + ark::gpuMemcpyHostToDevice), + ark::gpuSuccess); + + // Copy data from GPU array to ARK tensor + executor.tensor_write(tensor, dev_ptr, 1024 * sizeof(float), nullptr, true); + + // Copy data from ARK tensor to CPU array + executor.tensor_read(tensor, host_ptr, 1024 * sizeof(float)); + + // Check the data + for (size_t i = 0; i < host_data.size(); ++i) { + UNITTEST_EQ(host_data[i], -1); + } + + return ark::unittest::SUCCESS; +} + +ark::unittest::State test_executor_invalid() { + // Invalid device ID. + UNITTEST_THROW(ark::Executor(-1, nullptr, "test", ""), + ark::InvalidUsageError); + + // Invalid rank. + ark::PlanJson plan; + plan["Rank"] = 1; + UNITTEST_THROW(ark::Executor(0, nullptr, "test", plan.dump(), true), + ark::InvalidUsageError); + + return ark::unittest::SUCCESS; +} + +int main() { + UNITTEST(test_executor_loop); + UNITTEST(test_executor_no_loop); + UNITTEST(test_executor_tensor_read_write); + UNITTEST(test_executor_invalid); + return 0; +} diff --git a/ark/api/planner.cpp b/ark/api/planner.cpp index 22b9b680e..d36f33cbe 100644 --- a/ark/api/planner.cpp +++ b/ark/api/planner.cpp @@ -7,7 +7,7 @@ #include "context_impl.hpp" #include "env.h" #include "file_io.h" -#include "gpu/gpu_manager.h" +#include "gpu/gpu_manager.hpp" #include "model/model_json.hpp" #include "model/model_node.hpp" #include "model/model_op.hpp" diff --git a/ark/api/tensor.cpp b/ark/api/tensor.cpp index 4b03c3ac8..4d33bd9f1 100644 --- a/ark/api/tensor.cpp +++ b/ark/api/tensor.cpp @@ -3,11 +3,25 @@ #include "ark/tensor.hpp" +#include "model/model_buffer.hpp" #include "model/model_data_type.hpp" #include "model/model_tensor.hpp" namespace ark { +Tensor::Tensor(void* data_ptr, int32_t device_id, + const std::vector& shape, + const DataType& dtype) { + size_t external_data_size = std::accumulate(shape.begin(), shape.end(), 1, + std::multiplies()) * + dtype.bytes(); + auto buffer = + std::make_shared(data_ptr, external_data_size, device_id); + auto tensor = std::make_shared(dtype.ref(), buffer, Dims(shape), + Dims(shape), Dims(), Dims()); + ref_ = tensor; +} + size_t Tensor::id() const { if (ref_) { return ref_->id(); @@ -43,14 +57,14 @@ Dims Tensor::padded_shape() const { return Dims(); } -const DataType &Tensor::data_type() const { +const DataType& Tensor::data_type() const { if (ref_) { return DataType::from_name(ref_->data_type()->type_name()); } return NONE; } -std::ostream &operator<<(std::ostream &os, const Tensor &tensor) { +std::ostream& operator<<(std::ostream& os, const Tensor& tensor) { if (tensor.is_null()) { os << "null"; } else { diff --git a/ark/codegen.cpp b/ark/codegen.cpp index 6d5d5fc84..1619b863f 100644 --- a/ark/codegen.cpp +++ b/ark/codegen.cpp @@ -10,6 +10,7 @@ #include "file_io.h" #include "logging.hpp" #include "model/model_buffer.hpp" +#include "model_buffer_manager.hpp" #include "model/model_data_type.hpp" #include "model/model_op.hpp" #include "model/model_tensor.hpp" @@ -174,7 +175,7 @@ CodeGenerator::Impl::Impl(const PlanJson &plan, {"@NUM_WARPS_PER_BLOCK@", std::to_string(num_warps_per_proc_)}, {"@DEFINITIONS@", definitions_ss.str()}, {"@BODY@", body_ss.str()}, - {"@NAME@", name_}, + {"@NAME@", (name_.empty() ? "" : "_" + name_)}, }; code_ = replace(template_code, replacements); } @@ -224,11 +225,19 @@ std::string CodeGenerator::Impl::def_task(const Json &task_json) { auto &arg = impl_args[i]; if (arg.type_name() == "TENSOR") { auto tns = arg.value(); - size_t buffer_offset = - buffer_id_to_offset_.at(tns->buffer()->id()); - size_t offset = buffer_offset + ModelOffset(tns).value(); - ss << "(" << tns->data_type()->type_str() << "*)&_buf[" - << offset << "]"; + if (tns->buffer()->is_external()) { + void *buf_addr = + ModelBufferManager::get_instance().get_buffer( + tns->buffer()->id()); + ss << "(" << tns->data_type()->type_str() << "*)" + << buf_addr; + } else { + size_t buffer_offset = + buffer_id_to_offset_.at(tns->buffer()->id()); + size_t offset = buffer_offset + ModelOffset(tns).value(); + ss << "(" << tns->data_type()->type_str() << "*)&_buf[" + << offset << "]"; + } } else if (arg.type_name() == "OFFSET") { auto moff = arg.value(); size_t buffer_offset = @@ -288,10 +297,14 @@ std::string CodeGenerator::Impl::resource_group( size_t proc_b = *rg_proc_range.begin(); size_t proc_e = *rg_proc_range.end(); size_t proc_s = rg_proc_range.step(); + std::map task_infos_map; + for (auto &task_info : task_infos) { + task_infos_map[task_info.at("Id").get()] = task_info; + } std::stringstream ss; for (auto &tg : rg_json["TaskGroups"]) { size_t task_id = tg["TaskId"]; - auto &task_info = task_infos[task_id]; + auto &task_info = task_infos_map.at(task_id); Range task_range(tg["TaskRange"][0], tg["TaskRange"][1]); size_t task_gran = tg["Granularity"]; size_t num_warps_per_task = task_info["NumWarps"]; @@ -305,7 +318,8 @@ std::string CodeGenerator::Impl::resource_group( n_slots = total_warps / num_warps_per_task; } if (n_slots == 0) { - ERR(PlanError, "not enough resources for task group"); + ERR(PlanError, "not enough resources for task group: ", + tg.dump()); } size_t task_b = *task_range.begin(); diff --git a/ark/codegen.hpp b/ark/codegen.hpp index 4f8307e7e..1ed8ec9f2 100644 --- a/ark/codegen.hpp +++ b/ark/codegen.hpp @@ -8,6 +8,7 @@ #include #include +#include "model_buffer_manager.hpp" #include "model/model_json.hpp" namespace ark { diff --git a/ark/gpu/gpu.h b/ark/gpu/gpu.hpp similarity index 98% rename from ark/gpu/gpu.h rename to ark/gpu/gpu.hpp index 2f1eba3ba..531d6c7ee 100644 --- a/ark/gpu/gpu.h +++ b/ark/gpu/gpu.hpp @@ -1,8 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. -#ifndef ARK_GPU_H_ -#define ARK_GPU_H_ +#ifndef ARK_GPU_HPP_ +#define ARK_GPU_HPP_ #include @@ -125,6 +125,7 @@ ARK_GPU_DEFINE_CONSTANT_ALIAS(gpuPointerAttributeSyncMemops, // runtime API ARK_GPU_DEFINE_FUNC_ALIAS(gpuGetErrorString, cudaGetErrorString, hipGetErrorString); +ARK_GPU_DEFINE_FUNC_ALIAS(gpuGetLastError, cudaGetLastError, hipGetLastError); ARK_GPU_DEFINE_FUNC_ALIAS(gpuDeviceGetAttribute, cudaDeviceGetAttribute, hipDeviceGetAttribute); ARK_GPU_DEFINE_FUNC_ALIAS(gpuDeviceSynchronize, cudaDeviceSynchronize, @@ -183,4 +184,4 @@ ARK_GPU_DEFINE_FUNC_ALIAS(gpuPointerSetAttribute, cuPointerSetAttribute, } // namespace ark -#endif // ARK_GPU_H_ +#endif // ARK_GPU_HPP_ diff --git a/ark/gpu/gpu_compile.cpp b/ark/gpu/gpu_compile.cpp index 4571a9413..21908fa4d 100644 --- a/ark/gpu/gpu_compile.cpp +++ b/ark/gpu/gpu_compile.cpp @@ -1,7 +1,7 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. -#include "gpu/gpu_compile.h" +#include "gpu/gpu_compile.hpp" #include #include @@ -22,7 +22,7 @@ #include "cpu_timer.h" #include "env.h" #include "file_io.h" -#include "gpu/gpu_logging.h" +#include "gpu/gpu_logging.hpp" #include "utils/utils_string.hpp" #define ARK_DEBUG_KERNEL 0 diff --git a/ark/gpu/gpu_compile.h b/ark/gpu/gpu_compile.hpp similarity index 78% rename from ark/gpu/gpu_compile.h rename to ark/gpu/gpu_compile.hpp index 58048e78c..8b9e1a9fd 100644 --- a/ark/gpu/gpu_compile.h +++ b/ark/gpu/gpu_compile.hpp @@ -1,8 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. -#ifndef ARK_GPU_COMPILE_H_ -#define ARK_GPU_COMPILE_H_ +#ifndef ARK_GPU_COMPILE_HPP_ +#define ARK_GPU_COMPILE_HPP_ #include #include @@ -16,4 +16,4 @@ const std::string gpu_compile(const std::vector &codes, } // namespace ark -#endif // ARK_GPU_COMPILE_H_ +#endif // ARK_GPU_COMPILE_HPP_ diff --git a/ark/gpu/gpu_event.cpp b/ark/gpu/gpu_event.cpp index 93ec3fd52..06779b91a 100644 --- a/ark/gpu/gpu_event.cpp +++ b/ark/gpu/gpu_event.cpp @@ -1,11 +1,10 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. -#include "gpu/gpu_event.h" +#include "gpu/gpu_event.hpp" -#include "gpu/gpu.h" -#include "gpu/gpu_logging.h" -#include "gpu/gpu_manager.h" +#include "gpu/gpu_logging.hpp" +#include "gpu/gpu_manager.hpp" namespace ark { class GpuEvent::Impl { @@ -15,7 +14,7 @@ class GpuEvent::Impl { Impl(const Impl&) = delete; Impl& operator=(const Impl&) = delete; - void record(std::shared_ptr stream); + void record(gpuStream stream); float elapsed_msec(const GpuEvent& other) const; private: @@ -32,8 +31,8 @@ GpuEvent::Impl::Impl(bool disable_timing) { GpuEvent::Impl::~Impl() { GLOG(gpuEventDestroy(event_)); } -void GpuEvent::Impl::record(std::shared_ptr stream) { - GLOG(gpuEventRecord(event_, stream->get())); +void GpuEvent::Impl::record(gpuStream stream) { + GLOG(gpuEventRecord(event_, stream)); } float GpuEvent::Impl::elapsed_msec(const GpuEvent& other) const { @@ -45,9 +44,7 @@ float GpuEvent::Impl::elapsed_msec(const GpuEvent& other) const { GpuEvent::GpuEvent(bool disable_timing) : pimpl_(std::make_shared(disable_timing)) {} -void GpuEvent::record(std::shared_ptr stream) { - pimpl_->record(stream); -} +void GpuEvent::record(gpuStream stream) { pimpl_->record(stream); } float GpuEvent::elapsed_msec(const GpuEvent& other) const { return pimpl_->elapsed_msec(other); diff --git a/ark/gpu/gpu_event.h b/ark/gpu/gpu_event.hpp similarity index 78% rename from ark/gpu/gpu_event.h rename to ark/gpu/gpu_event.hpp index 4599ecaa4..bd2a7c952 100644 --- a/ark/gpu/gpu_event.h +++ b/ark/gpu/gpu_event.hpp @@ -1,11 +1,13 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. -#ifndef ARK_GPU_EVENT_H_ -#define ARK_GPU_EVENT_H_ +#ifndef ARK_GPU_EVENT_HPP_ +#define ARK_GPU_EVENT_HPP_ #include +#include "gpu/gpu.hpp" + namespace ark { class GpuStream; @@ -17,7 +19,7 @@ class GpuEvent { GpuEvent(const GpuEvent &) = delete; GpuEvent &operator=(const GpuEvent &) = delete; - void record(std::shared_ptr stream); + void record(gpuStream stream); float elapsed_msec(const GpuEvent &other) const; protected: @@ -31,4 +33,4 @@ class GpuEvent { }; } // namespace ark -#endif // ARK_GPU_EVENT_H_ +#endif // ARK_GPU_EVENT_HPP_ diff --git a/ark/gpu/gpu_kernel.cpp b/ark/gpu/gpu_kernel.cpp index 44ff43a1d..d4412f80e 100644 --- a/ark/gpu/gpu_kernel.cpp +++ b/ark/gpu/gpu_kernel.cpp @@ -1,50 +1,38 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. -#include "gpu_kernel.h" +#include "gpu_kernel.hpp" #include #include -#include "gpu.h" -#include "gpu_compile.h" -#include "gpu_logging.h" -#include "gpu_manager.h" +#include "gpu.hpp" +#include "gpu_compile.hpp" +#include "gpu_logging.hpp" +#include "gpu_manager.hpp" namespace ark { GpuKernel::GpuKernel(int gpu_id, const std::string& code, const std::array& block_dim, const std::array& grid_dim, size_t smem_bytes, - const std::string& kernel_name, - std::initializer_list> args) { - this->init(gpu_id, code, block_dim, grid_dim, smem_bytes, kernel_name, - args); + const std::string& kernel_name) { + this->init(gpu_id, code, block_dim, grid_dim, smem_bytes, kernel_name); } void GpuKernel::init(int gpu_id, const std::string& code, const std::array& block_dim, const std::array& grid_dim, size_t smem_bytes, - const std::string& kernel_name, - std::initializer_list> args) { + const std::string& kernel_name) { gpu_manager_ = GpuManager::get_instance(gpu_id); code_ = code; block_dim_ = block_dim; grid_dim_ = grid_dim; smem_bytes_ = smem_bytes; kernel_name_ = kernel_name; - params_ptr_.resize(args.size()); - args_.resize(args.size()); if (kernel_name_.size() == 0) { ERR(InvalidUsageError, "Invalid kernel name: ", kernel_name_); } - size_t idx = 0; - for (auto& pair : args) { - args_[idx].reset(new uint8_t[pair.second]); - std::memcpy(args_[idx].get(), &(pair.first), pair.second); - params_ptr_[idx] = static_cast(args_[idx].get()); - idx++; - } } void GpuKernel::compile() { @@ -68,12 +56,13 @@ void GpuKernel::compile() { dynamic_smem_size_bytes)); } -void GpuKernel::launch(std::shared_ptr stream) { +void GpuKernel::launch(gpuStream stream, std::vector& args) { if (!this->is_compiled()) { ERR(InvalidUsageError, "Kernel is not compiled yet."); } gpu_manager_->launch(function_, grid_dim_, block_dim_, smem_bytes_, stream, - params_ptr_.data(), nullptr); + args.data(), nullptr); + GLOG(gpuGetLastError()); } gpuDeviceptr GpuKernel::get_global(const std::string& name, diff --git a/ark/gpu/gpu_kernel.h b/ark/gpu/gpu_kernel.hpp similarity index 68% rename from ark/gpu/gpu_kernel.h rename to ark/gpu/gpu_kernel.hpp index c3b60aec4..5308cfead 100644 --- a/ark/gpu/gpu_kernel.h +++ b/ark/gpu/gpu_kernel.hpp @@ -1,13 +1,14 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. -#ifndef ARK_GPU_KERNEL_H_ -#define ARK_GPU_KERNEL_H_ +#ifndef ARK_GPU_KERNEL_HPP_ +#define ARK_GPU_KERNEL_HPP_ #include #include +#include -#include "gpu_stream.h" +#include "gpu_stream.hpp" namespace ark { @@ -18,16 +19,14 @@ class GpuKernel { GpuKernel(int gpu_id, const std::string& codes, const std::array& block_dim, const std::array& grid_dim, size_t smem_bytes, - const std::string& kernel_name, - std::initializer_list> args = {}); + const std::string& kernel_name); void init(int gpu_id, const std::string& codes, const std::array& block_dim, const std::array& grid_dim, size_t smem_bytes, - const std::string& kernel_name, - std::initializer_list> args = {}); + const std::string& kernel_name); void compile(); - void launch(std::shared_ptr stream); + void launch(gpuStream stream, std::vector& args); gpuDeviceptr get_global(const std::string& name, bool ignore_not_found = false) const; @@ -43,10 +42,8 @@ class GpuKernel { std::string bin_; gpuModule module_; gpuFunction function_ = nullptr; - std::vector params_ptr_; - std::vector> args_; }; } // namespace ark -#endif // ARK_GPU_KERNEL_H_ +#endif // ARK_GPU_KERNEL_HPP_ diff --git a/ark/gpu/gpu_kernel_test.cpp b/ark/gpu/gpu_kernel_test.cpp index 870ad7ab9..342ef9656 100644 --- a/ark/gpu/gpu_kernel_test.cpp +++ b/ark/gpu/gpu_kernel_test.cpp @@ -1,7 +1,7 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. -#include "gpu/gpu_kernel.h" +#include "gpu/gpu_kernel.hpp" #include "unittest/unittest_utils.h" @@ -9,7 +9,13 @@ const std::string void_kernel = "extern \"C\" __global__ void kernel() {}"; ark::unittest::State test_gpu_kernel() { ark::GpuKernel kernel(0, void_kernel, {1, 1, 1}, {1, 1, 1}, 0, "kernel"); + UNITTEST_TRUE(!kernel.is_compiled()); kernel.compile(); + UNITTEST_TRUE(kernel.is_compiled()); + std::vector args; + for (int i = 0; i < 10; i++) { + kernel.launch(nullptr, args); + } return ark::unittest::SUCCESS; } diff --git a/ark/gpu/gpu_logging.h b/ark/gpu/gpu_logging.hpp similarity index 92% rename from ark/gpu/gpu_logging.h rename to ark/gpu/gpu_logging.hpp index e67894b9e..9be8f41c4 100644 --- a/ark/gpu/gpu_logging.h +++ b/ark/gpu/gpu_logging.hpp @@ -1,10 +1,10 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. -#ifndef ARK_GPU_LOGGING_H_ -#define ARK_GPU_LOGGING_H_ +#ifndef ARK_GPU_LOGGING_HPP_ +#define ARK_GPU_LOGGING_HPP_ -#include "gpu/gpu.h" +#include "gpu/gpu.hpp" #include "logging.hpp" #define GLOG(cmd) \ @@ -29,4 +29,4 @@ } \ } while (0) -#endif // ARK_GPU_LOGGING_H_ +#endif // ARK_GPU_LOGGING_HPP_ diff --git a/ark/gpu/gpu_manager.cpp b/ark/gpu/gpu_manager.cpp index 1aaa365cd..2b5be490b 100644 --- a/ark/gpu/gpu_manager.cpp +++ b/ark/gpu/gpu_manager.cpp @@ -1,11 +1,11 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. -#include "gpu/gpu_manager.h" +#include "gpu/gpu_manager.hpp" #include -#include "gpu/gpu_logging.h" +#include "gpu/gpu_logging.hpp" #include "utils/utils_string.hpp" namespace ark { @@ -20,11 +20,10 @@ class GpuManager::Impl { int gpu_id_; GpuManager::Info info_; - std::shared_ptr main_stream_; void launch(gpuFunction kernel, const std::array &grid_dim, const std::array &block_dim, int smem_bytes, - std::shared_ptr stream, void **params, void **extra); + gpuStream stream, void **params, void **extra); }; GpuManager::Impl::Impl(int gpu_id) : gpu_id_(gpu_id) { @@ -78,11 +77,11 @@ GpuManager::Impl::Impl(int gpu_id) : gpu_id_(gpu_id) { void GpuManager::Impl::launch(gpuFunction kernel, const std::array &grid_dim, const std::array &block_dim, - int smem_bytes, std::shared_ptr stream, - void **params, void **extra) { + int smem_bytes, gpuStream stream, void **params, + void **extra) { GLOG_DRV(gpuModuleLaunchKernel( kernel, grid_dim[0], grid_dim[1], grid_dim[2], block_dim[0], - block_dim[1], block_dim[2], smem_bytes, stream->get(), params, extra)); + block_dim[1], block_dim[2], smem_bytes, stream, params, extra)); } std::shared_ptr GpuManager::get_instance(int gpu_id) { @@ -104,9 +103,7 @@ std::shared_ptr GpuManager::get_instance(int gpu_id) { } } -GpuManager::GpuManager(int gpu_id) : pimpl_(std::make_shared(gpu_id)) { - this->pimpl_->main_stream_ = std::shared_ptr(new GpuStream()); -} +GpuManager::GpuManager(int gpu_id) : pimpl_(std::make_shared(gpu_id)) {} std::shared_ptr GpuManager::malloc(size_t bytes, size_t align, bool expose) { @@ -128,8 +125,6 @@ std::shared_ptr GpuManager::create_stream() const { return std::shared_ptr(new GpuStream()); } -int GpuManager::get_gpu_id() const { return pimpl_->gpu_id_; } - const GpuManager::Info &GpuManager::info() const { return pimpl_->info_; } void GpuManager::set_current() const { GLOG(gpuSetDevice(pimpl_->gpu_id_)); } @@ -137,8 +132,7 @@ void GpuManager::set_current() const { GLOG(gpuSetDevice(pimpl_->gpu_id_)); } void GpuManager::launch(gpuFunction function, const std::array &grid_dim, const std::array &block_dim, int smem_bytes, - std::shared_ptr stream, void **params, - void **extra) const { + gpuStream stream, void **params, void **extra) const { this->set_current(); pimpl_->launch(function, grid_dim, block_dim, smem_bytes, stream, params, extra); diff --git a/ark/gpu/gpu_manager.h b/ark/gpu/gpu_manager.hpp similarity index 82% rename from ark/gpu/gpu_manager.h rename to ark/gpu/gpu_manager.hpp index 05014ac47..eeeda4d94 100644 --- a/ark/gpu/gpu_manager.h +++ b/ark/gpu/gpu_manager.hpp @@ -1,16 +1,16 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. -#ifndef ARK_GPU_MANAGER_H_ -#define ARK_GPU_MANAGER_H_ +#ifndef ARK_GPU_MANAGER_HPP_ +#define ARK_GPU_MANAGER_HPP_ #include #include "arch.hpp" -#include "gpu/gpu.h" -#include "gpu/gpu_event.h" -#include "gpu/gpu_memory.h" -#include "gpu/gpu_stream.h" +#include "gpu/gpu.hpp" +#include "gpu/gpu_event.hpp" +#include "gpu/gpu_memory.hpp" +#include "gpu/gpu_stream.hpp" namespace ark { @@ -30,11 +30,9 @@ class GpuManager { std::shared_ptr create_event(bool disable_timing = false) const; std::shared_ptr create_stream() const; - int get_gpu_id() const; void launch(gpuFunction function, const std::array &grid_dim, const std::array &block_dim, int smem_bytes, - std::shared_ptr stream, void **params, - void **extra) const; + gpuStream stream, void **params, void **extra) const; struct Info; const Info &info() const; @@ -64,4 +62,4 @@ class GpuManager { } // namespace ark -#endif // ARK_GPU_MANAGER_H_ +#endif // ARK_GPU_MANAGER_HPP_ diff --git a/ark/gpu/gpu_memory.cpp b/ark/gpu/gpu_memory.cpp index 446222e24..fac8d3672 100644 --- a/ark/gpu/gpu_memory.cpp +++ b/ark/gpu/gpu_memory.cpp @@ -1,11 +1,11 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. -#include "gpu/gpu_memory.h" +#include "gpu/gpu_memory.hpp" -#include "gpu/gpu.h" -#include "gpu/gpu_logging.h" -#include "gpu/gpu_manager.h" +#include "gpu/gpu.hpp" +#include "gpu/gpu_logging.hpp" +#include "gpu/gpu_manager.hpp" namespace ark { diff --git a/ark/gpu/gpu_memory.h b/ark/gpu/gpu_memory.hpp similarity index 87% rename from ark/gpu/gpu_memory.h rename to ark/gpu/gpu_memory.hpp index cd7a6f04f..6b277d40b 100644 --- a/ark/gpu/gpu_memory.h +++ b/ark/gpu/gpu_memory.hpp @@ -1,13 +1,13 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. -#ifndef ARK_GPU_MEMORY_H_ -#define ARK_GPU_MEMORY_H_ +#ifndef ARK_GPU_MEMORY_HPP_ +#define ARK_GPU_MEMORY_HPP_ #include #include -#include "gpu/gpu.h" +#include "gpu/gpu.hpp" namespace ark { @@ -40,7 +40,7 @@ class GpuHostMemory { GpuHostMemory(const GpuHostMemory&) = delete; GpuHostMemory& operator=(const GpuHostMemory&) = delete; - template + template T* ref() const { return reinterpret_cast(ptr_); } @@ -54,4 +54,4 @@ class GpuHostMemory { } // namespace ark -#endif // ARK_GPU_MEMORY_H_ +#endif // ARK_GPU_MEMORY_HPP_ diff --git a/ark/gpu/gpu_stream.cpp b/ark/gpu/gpu_stream.cpp index 52502365a..17d4e21f5 100644 --- a/ark/gpu/gpu_stream.cpp +++ b/ark/gpu/gpu_stream.cpp @@ -1,10 +1,10 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. -#include "gpu/gpu_stream.h" +#include "gpu/gpu_stream.hpp" -#include "gpu/gpu_logging.h" -#include "gpu/gpu_manager.h" +#include "gpu/gpu_logging.hpp" +#include "gpu/gpu_manager.hpp" namespace ark { class GpuStream::Impl { diff --git a/ark/gpu/gpu_stream.h b/ark/gpu/gpu_stream.hpp similarity index 79% rename from ark/gpu/gpu_stream.h rename to ark/gpu/gpu_stream.hpp index e76f01827..9d8775f95 100644 --- a/ark/gpu/gpu_stream.h +++ b/ark/gpu/gpu_stream.hpp @@ -1,12 +1,12 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. -#ifndef ARK_GPU_STREAM_H_ -#define ARK_GPU_STREAM_H_ +#ifndef ARK_GPU_STREAM_HPP_ +#define ARK_GPU_STREAM_HPP_ #include -#include "gpu/gpu.h" +#include "gpu/gpu.hpp" namespace ark { @@ -30,4 +30,4 @@ class GpuStream { }; } // namespace ark -#endif // ARK_GPU_STREAM_H_ +#endif // ARK_GPU_STREAM_HPP_ diff --git a/ark/include/ark/executor.hpp b/ark/include/ark/executor.hpp index 4682af7d0..7f30f39ed 100644 --- a/ark/include/ark/executor.hpp +++ b/ark/include/ark/executor.hpp @@ -5,6 +5,7 @@ #define ARK_EXECUTOR_HPP #include +#include #include #include #include @@ -12,21 +13,33 @@ namespace ark { +using Stream = void *; + /// Convenience class for executing a model. class Executor { public: /// Constructor. - Executor(int rank, int world_size, int gpu_id, const std::string &name, - const std::string &plan); + Executor(int device_id, Stream stream, const std::string &name, + const std::string &plan, bool loop_mode = true); + /// Destructor. ~Executor(); + /// Return the device ID. + int device_id() const; + + /// Return the stream of the executor. + Stream stream() const; + + /// Return the plan string. + std::string plan() const; + /// Compile the model. This must be called before `launch()`. void compile(); /// Launch the model (not running yet). This must be called after /// `compile()`. - void launch(int64_t max_spin_count = -1); + void launch(); /// Run the model for `iter` iterations. void run(int iter); @@ -39,30 +52,39 @@ class Executor { /// again. float stop(int64_t max_spin_count = -1); + /// Barrier for all rank executors. void barrier(); + /// Destroy the executor. void destroy(); + /// Return whether the executor is destroyed. bool destroyed() const; + /// Return the raw virtual address of the tensor. + uintptr_t tensor_address(const Tensor tensor) const; + template - void tensor_read(const Tensor tensor, std::vector &data) const { + void tensor_read(const Tensor tensor, std::vector &data, + Stream stream = nullptr) const { tensor_read(tensor, reinterpret_cast(data.data()), - data.size() * sizeof(T)); + data.size() * sizeof(T), stream); } template - void tensor_write(const Tensor tensor, const std::vector &data) const { + void tensor_write(const Tensor tensor, const std::vector &data, + Stream stream = nullptr) const { tensor_write(tensor, reinterpret_cast(data.data()), - data.size() * sizeof(T)); + data.size() * sizeof(T), stream); } - void tensor_read(const Tensor tensor, void *data, size_t bytes) const; + void tensor_read(const Tensor tensor, void *data, size_t bytes, + Stream stream = nullptr, bool is_d2d = false) const; - void tensor_write(const Tensor tensor, const void *data, - size_t bytes) const; + void tensor_write(const Tensor tensor, const void *data, size_t bytes, + Stream stream = nullptr, bool is_d2d = false) const; - private: + protected: class Impl; std::unique_ptr impl_; }; @@ -71,8 +93,10 @@ class Model; class DefaultExecutor : public Executor { public: - DefaultExecutor(const Model &model, int gpu_id = -1, - const std::string &name = "DefaultExecutor"); + DefaultExecutor( + const Model &model, int device_id = -1, Stream stream = nullptr, + const std::vector &config_rules = {}, + const std::string &name = "DefaultExecutor", bool loop_mode = true); }; } // namespace ark diff --git a/ark/include/ark/tensor.hpp b/ark/include/ark/tensor.hpp index 747ce5fea..d13748175 100644 --- a/ark/include/ark/tensor.hpp +++ b/ark/include/ark/tensor.hpp @@ -31,6 +31,8 @@ class Tensor { Tensor(ModelTensorRef ref) : ref_(ref) {} Tensor(const Tensor &other) = default; Tensor &operator=(const Tensor &other) = default; + Tensor(void *data_ptr, int32_t device_id, const std::vector &shape, + const DataType &dtype); bool operator==(const Tensor &other) const { return ref_ == other.ref_; } bool operator!=(const Tensor &other) const { return ref_ != other.ref_; } diff --git a/ark/include/kernels/kernel_template.in b/ark/include/kernels/kernel_template.in index ea1862920..a8a56f141 100644 --- a/ark/include/kernels/kernel_template.in +++ b/ark/include/kernels/kernel_template.in @@ -33,12 +33,12 @@ __device__ sync::State ARK_LOOP_SYNC_STATE; @DEFINITIONS@ -__device__ void ark_loop_body(char *_buf, int _iter) { +__device__ void ark_body(char *_buf, int _iter) { @BODY@ } extern "C" __global__ __launch_bounds__(ARK_WARPS_PER_BLOCK * Arch::ThreadsPerWarp, 1) -void @NAME@(char *_buf, int *_iter) { +void ark_loop_kernel@NAME@(char *_buf, int *_iter) { int *shared_mem = (int *)_ARK_SMEM; for (int i = threadIdx.x; i < ARK_SMEM_RESERVED_BYTES / sizeof(int); i += blockDim.x) { shared_mem[i] = 0; @@ -52,10 +52,10 @@ void @NAME@(char *_buf, int *_iter) { sync_gpu<@NUM_BLOCKS@>(ARK_LOOP_SYNC_STATE); if (ARK_ITER < 0) return; - ark_loop_body(_buf, 0); + ark_body(_buf, 0); for (int _i = 1; _i < ARK_ITER; ++_i) { sync_gpu<@NUM_BLOCKS@>(ARK_LOOP_SYNC_STATE); - ark_loop_body(_buf, _i); + ark_body(_buf, _i); } if (threadIdx.x == 0) { __threadfence_system(); @@ -67,3 +67,12 @@ void @NAME@(char *_buf, int *_iter) { sync_gpu<@NUM_BLOCKS@>(ARK_LOOP_SYNC_STATE); } } + +extern "C" __global__ __launch_bounds__(ARK_WARPS_PER_BLOCK * Arch::ThreadsPerWarp, 1) +void ark_kernel@NAME@(char *_buf, int _iter) { + int *shared_mem = (int *)_ARK_SMEM; + for (int i = threadIdx.x; i < ARK_SMEM_RESERVED_BYTES / sizeof(int); i += blockDim.x) { + shared_mem[i] = 0; + } + ark_body(_buf, _iter); +} diff --git a/ark/model/model_buffer.cpp b/ark/model/model_buffer.cpp index e637307fd..5ce255ce5 100644 --- a/ark/model/model_buffer.cpp +++ b/ark/model/model_buffer.cpp @@ -4,13 +4,13 @@ #include "model_buffer.hpp" #include "logging.hpp" +#include "model_buffer_manager.hpp" namespace ark { -ModelBuffer::ModelBuffer(int rank) : rank_(rank) { - static size_t id = 0; - id_ = id++; -} +size_t ModelBuffer::curr_id = 0; + +ModelBuffer::ModelBuffer(int rank) : rank_(rank) { id_ = curr_id++; } ModelBuffer::ModelBuffer(size_t id, int rank, const std::vector &send_tags, @@ -24,6 +24,23 @@ ModelBuffer::ModelBuffer(size_t id, int rank, } } +ModelBuffer::ModelBuffer(void *data, size_t size, int32_t device_id) + : rank_(-1), + external_data_(data), + external_data_size_(size), + device_id_(device_id), + is_external_(true) { + id_ = curr_id++; +} + +ModelBuffer::ModelBuffer(size_t id, void *data, size_t size, int32_t device_id) + : id_(id), + rank_(-1), + external_data_(data), + external_data_size_(size), + device_id_(device_id), + is_external_(true) {} + void ModelBuffer::tag_send(int remote_rank, int tag) { send_tags_.insert(TagInfo{remote_rank, tag}); } @@ -46,6 +63,14 @@ Json ModelBuffer::serialize() const { } j["SendTags"] = send_tags; j["RecvTags"] = recv_tags; + j["IsExternal"] = is_external_; + if (is_external_) { + ModelBufferManager::get_instance().register_buffer(id_, external_data_, + external_data_size_); + j["ExternalDataSize"] = external_data_size_; + j["DeviceId"] = device_id_; + } + // external_data_ptr_ is not included in JSON return j; } @@ -57,7 +82,30 @@ std::shared_ptr ModelBuffer::deserialize(const Json &serialized) { } else if (!serialized.contains("SendTags")) { ERR(ModelError, "ModelBuffer deserialization failed: missing SendTags"); } else if (!serialized.contains("RecvTags")) { - ERR(ModelError, "ModelBuffer deserialization failed: missing RecvTags"); + ERR(ModelError, + "ModelBuffer deserialization failed: missing RecvTags"); + } else if (!serialized.contains("IsExternal")) { + ERR(ModelError, + "ModelBuffer deserialization failed: missing IsExternal"); + } + if (serialized["IsExternal"]) { + if (!serialized.contains("ExternalDataSize")) { + ERR(ModelError, + "ModelBuffer deserialization failed: missing ExternalDataSize"); + } else if (!serialized.contains("DeviceId")) { + ERR(ModelError, + "ModelBuffer deserialization failed: missing DeviceId"); + } + void *data_ptr = + ModelBufferManager::get_instance().get_buffer(serialized["Id"]); + if (!data_ptr) { + ERR(ModelError, + "ModelBuffer deserialization failed: external buffer not found " + "in BufferManager"); + } + return std::make_shared(serialized["Id"], data_ptr, + serialized["ExternalDataSize"], + serialized["DeviceId"]); } return std::make_shared(serialized["Id"], serialized["Rank"], serialized["SendTags"], diff --git a/ark/model/model_buffer.hpp b/ark/model/model_buffer.hpp index 7ad3db206..e7f1045b2 100644 --- a/ark/model/model_buffer.hpp +++ b/ark/model/model_buffer.hpp @@ -22,6 +22,10 @@ class ModelBuffer { ModelBuffer(size_t id, int rank, const std::vector &send_tags, const std::vector &recv_tags); + // externally managed buffer + ModelBuffer(void *data, size_t size, int32_t device_id); + ModelBuffer(size_t id, void *data, size_t size, int32_t device_id); + size_t id() const { return id_; } int rank() const { return rank_; } @@ -44,11 +48,22 @@ class ModelBuffer { static std::shared_ptr deserialize(const Json &serialized); + // external buffer management + size_t external_data_size() const { return external_data_size_; } + void *external_data() const { return external_data_; } + int32_t device_id() const { return device_id_; } + bool is_external() const { return is_external_; } + private: + static size_t curr_id; size_t id_; int rank_; std::set send_tags_; std::set recv_tags_; + void *external_data_ = nullptr; + size_t external_data_size_ = 0; + int32_t device_id_; + bool is_external_ = false; }; } // namespace ark diff --git a/ark/model/model_json.cpp b/ark/model/model_json.cpp index b82f9e484..c2099e2c9 100644 --- a/ark/model/model_json.cpp +++ b/ark/model/model_json.cpp @@ -287,6 +287,7 @@ PlanJson::PlanJson(const Json &json) : Json((json != nullptr) ? json : Json{{"Rank", 0}, {"WorldSize", 1}, + {"Architecture", "ANY"}, {"NumProcessors", 1}, {"NumWarpsPerProcessor", 1}, {"TaskInfos", Json::array()}, diff --git a/ark/model_buffer_manager.hpp b/ark/model_buffer_manager.hpp new file mode 100644 index 000000000..4baaec7fe --- /dev/null +++ b/ark/model_buffer_manager.hpp @@ -0,0 +1,57 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +#ifndef ARK_MODEL_BUFFER_MANAGER_HPP_ +#define ARK_MODEL_BUFFER_MANAGER_HPP_ + +#include +#include + +namespace ark { +// Manages externally allocated buffers not in the ARK memory space. +class ModelBufferManager { + public: + static ModelBufferManager& get_instance() { + static ModelBufferManager instance; + return instance; + } + + void register_buffer(size_t id, void* data, size_t size) { + buffers_[id] = std::make_tuple(data, size); + } + + void* get_buffer(size_t id) { + auto it = buffers_.find(id); + if (it != buffers_.end()) { + return std::get<0>(it->second); + } + return nullptr; + } + + size_t get_buffer_size(size_t id) { + auto it = buffers_.find(id); + if (it != buffers_.end()) { + return std::get<1>(it->second); + } + return 0; + } + + const std::unordered_map>& get_buffers() + const { + return buffers_; + } + + void clear_buffers() { buffers_.clear(); } + + bool is_empty() const { return buffers_.empty(); } + + private: + // Maps buffer IDs to pointers and sizes. + std::unordered_map> buffers_; + ModelBufferManager() {} + ModelBufferManager(const ModelBufferManager&) = delete; + ModelBufferManager& operator=(const ModelBufferManager&) = delete; +}; +} // namespace ark + +#endif // ARK_MODEL_BUFFER_MANAGER_HPP_ diff --git a/ark/ops/ops_all_reduce_test.cpp b/ark/ops/ops_all_reduce_test.cpp index a37d211f7..8cf68b085 100644 --- a/ark/ops/ops_all_reduce_test.cpp +++ b/ark/ops/ops_all_reduce_test.cpp @@ -35,10 +35,9 @@ void test_all_reduce_internal(ark::DimType nelem) { std::vector ones_vec(ones.shape().nelems(), ark::half_t(1.0f)); - auto result = - ark::op_test("all_reduce", m, {ones}, {output}, - baseline_all_reduce, - {ones_vec.data()}, false, gpu_id, NumGpus); + auto result = ark::op_test( + "all_reduce", m, {ones}, {output}, + baseline_all_reduce, {ones_vec.data()}); UNITTEST_LOG(result); UNITTEST_EQ(result.max_diff[0], 0.0f); return ark::unittest::SUCCESS; @@ -126,10 +125,9 @@ void test_all_reduce_packet_internal(ark::DimType nelem) { std::vector ones_vec(ones.shape().nelems(), ark::half_t(1.0f)); - auto result = - ark::op_test("all_reduce_packet", m, {ones}, {output}, - baseline_all_reduce, - {ones_vec.data()}, false, gpu_id, NumGpus); + auto result = ark::op_test( + "all_reduce_packet", m, {ones}, {output}, + baseline_all_reduce, {ones_vec.data()}); UNITTEST_LOG(result); UNITTEST_EQ(result.max_diff[0], 0.0f); return ark::unittest::SUCCESS; @@ -233,10 +231,10 @@ void test_all_reduce_sm_internal(ark::DimType nelem) { std::vector ones_vec(ones.shape().nelems(), ark::half_t(1.0f)); - auto result = ark::op_test( - "all_reduce_sm", m, {ones}, {output}, - baseline_all_reduce, {ones_vec.data()}, - false, gpu_id, NumGpus, config_rule); + auto result = + ark::op_test("all_reduce_sm", m, {ones}, {output}, + baseline_all_reduce, + {ones_vec.data()}, {config_rule}); UNITTEST_LOG(result); UNITTEST_EQ(result.max_diff[0], 0.0f); return ark::unittest::SUCCESS; diff --git a/ark/ops/ops_communication_test.cpp b/ark/ops/ops_communication_test.cpp index dec310331..8cdad41b2 100644 --- a/ark/ops/ops_communication_test.cpp +++ b/ark/ops/ops_communication_test.cpp @@ -231,9 +231,7 @@ ark::unittest::State test_communication_send_recv_bidir_sm() { ark::Tensor tns2 = model.identity(tns2_data, {tns}); tns2 = model.recv(tns2_data, remote_gpu_id, tag); - ark::Planner planner(model, gpu_id); - planner.install_config_rule(config_rule); - ark::Executor exe(gpu_id, 2, gpu_id, "Executor", planner.plan()); + ark::DefaultExecutor exe(model, gpu_id, nullptr, {config_rule}); exe.compile(); std::vector data(1024); @@ -277,9 +275,7 @@ ark::unittest::State test_communication_send_recv_bidir_sm() { ark::Tensor sum = model.add(tns2, tns_data); - ark::Planner planner(model, gpu_id); - planner.install_config_rule(config_rule); - ark::Executor exe(gpu_id, 2, gpu_id, "Executor", planner.plan()); + ark::DefaultExecutor exe(model, gpu_id, nullptr, {config_rule}); exe.compile(); std::vector data(1024); @@ -437,7 +433,7 @@ ark::unittest::State test_communication_send_recv_reduce() { ark::Planner planner(model, gpu_id); planner.install_config_rule(config_rule); - ark::Executor exe(gpu_id, 2, gpu_id, "Executor", planner.plan()); + ark::Executor exe(gpu_id, nullptr, "Executor", planner.plan()); exe.compile(); std::vector data(1024); diff --git a/ark/ops/ops_embedding_test.cpp b/ark/ops/ops_embedding_test.cpp index a458ae7b2..222605296 100644 --- a/ark/ops/ops_embedding_test.cpp +++ b/ark/ops/ops_embedding_test.cpp @@ -78,9 +78,9 @@ ark::unittest::State test_embedding() { } else if (std::is_same::value) { type_str = "bf16"; } - auto result = ark::op_test("embedding_" + type_str, m, {ti, tw}, {to}, - baseline_embedding, - {ti_data.data(), tw_data.data()}, true); + auto result = + ark::op_test("embedding_" + type_str, m, {ti, tw}, {to}, + baseline_embedding, {ti_data.data(), tw_data.data()}); UNITTEST_LOG(result); UNITTEST_EQ(result.max_diff[0], 0.0f); return ark::unittest::SUCCESS; diff --git a/ark/ops/ops_matmul_test.cpp b/ark/ops/ops_matmul_test.cpp index b86a4bc3e..11682ca49 100644 --- a/ark/ops/ops_matmul_test.cpp +++ b/ark/ops/ops_matmul_test.cpp @@ -3,7 +3,7 @@ #include -#include "gpu/gpu.h" +#include "gpu/gpu.hpp" #include "logging.hpp" #include "model/model_node.hpp" #include "model/model_op.hpp" diff --git a/ark/ops/ops_test_common.cpp b/ark/ops/ops_test_common.cpp index 0e8f215ae..4e94d06a7 100644 --- a/ark/ops/ops_test_common.cpp +++ b/ark/ops/ops_test_common.cpp @@ -10,7 +10,7 @@ #include "ark/planner.hpp" #include "ark/random.hpp" #include "env.h" -#include "gpu/gpu_logging.h" +#include "gpu/gpu_logging.hpp" #include "logging.hpp" #include "model/model_data_type.hpp" #include "model/model_tensor.hpp" @@ -31,16 +31,13 @@ std::ostream &operator<<(std::ostream &os, const OpsTestResult &result) { return os; } -OpsTestResult op_test(const std::string &test_name_prefix, const Model &model, - const std::vector &inputs, - const std::vector &outputs, - OpsTestBaseline baseline, - const std::vector &inputs_data, - bool print_on_error, int rank, int world_size, - Planner::ConfigRule config_rule) { - Planner planner(model, rank); - planner.install_config_rule(config_rule); - Executor exe(rank, world_size, rank, "Executor", planner.plan()); +OpsTestResult op_test( + const std::string &test_name_prefix, const Model &model, + const std::vector &inputs, const std::vector &outputs, + OpsTestBaseline baseline, const std::vector &inputs_data, + const std::vector &config_rules, + bool print_on_error) { + DefaultExecutor exe(model, -1, nullptr, config_rules); exe.compile(); std::vector>> inputs_data_storages; @@ -136,7 +133,8 @@ OpsTestResult op_test(const std::string &test_name_prefix, const Model &model, for (auto t : gt) { gt_ptrs.push_back(t->data()); } - baseline(gt_ptrs, output_shapes, inputs_data_refs, input_shapes, rank); + baseline(gt_ptrs, output_shapes, inputs_data_refs, input_shapes, + model.rank()); std::stringstream test_name; test_name << test_name_prefix; @@ -150,6 +148,7 @@ OpsTestResult op_test(const std::string &test_name_prefix, const Model &model, OpsTestResult result; result.test_name = test_name.str(); + result.plan = exe.plan(); // Compare results with the ground truth. for (size_t i = 0; i < outputs.size(); i++) { @@ -190,7 +189,7 @@ OpsTestResult op_test(const std::string &test_name_prefix, const Model &model, GLOG(gpuDeviceSynchronize()); // Throughput test. - if (world_size > 1) { + if (model.world_size() > 1) { // For multi-GPU, we need to make sure that all GPUs run the same // number of iterations. Rather than doing allgather, we just // use a magic number here. diff --git a/ark/ops/ops_test_common.hpp b/ark/ops/ops_test_common.hpp index e5b6c4f8e..3848773e6 100644 --- a/ark/ops/ops_test_common.hpp +++ b/ark/ops/ops_test_common.hpp @@ -134,6 +134,7 @@ TensorCompareResult tensor_compare(T *ground_truth, T *res, Dims shape, struct OpsTestResult { std::string test_name; + std::string plan; int iter; float msec_per_iter; std::vector mse; @@ -166,14 +167,12 @@ using OpsTestBaseline = std::function &inputs, - const std::vector &outputs, - OpsTestBaseline baseline, - const std::vector &inputs_data = {}, - bool print_on_error = false, int rank = 0, - int world_size = 1, - Planner::ConfigRule config_rule = nullptr); +OpsTestResult op_test( + const std::string &test_name_prefix, const Model &model, + const std::vector &inputs, const std::vector &outputs, + OpsTestBaseline baseline, const std::vector &inputs_data = {}, + const std::vector &config_rules = {}, + bool print_on_error = false); OpsTestGpuMem to_gpu(void *host_ptr, size_t size); diff --git a/arkprof.py b/arkprof.py new file mode 100644 index 000000000..5fb62e118 --- /dev/null +++ b/arkprof.py @@ -0,0 +1,7 @@ +import ark +import sys + +ark.init() +ark.Profiler(ark.Plan.from_file(sys.argv[1])).run( + iter=1000, profile_processor_groups=False +) diff --git a/docs/plan_file.md b/docs/plan_file.md index 90a4537a2..c06ccc35d 100644 --- a/docs/plan_file.md +++ b/docs/plan_file.md @@ -6,6 +6,7 @@ See an example plan file: [Example 1](../examples/tutorial/default_plan.json) - Rank (Int) - WorldSize (Int) + - Architecture (String) - NumProcessors (Int) - NumWarpsPerProcessor (Int) - TaskInfos (Array of TaskInfo) @@ -42,6 +43,23 @@ See an example plan file: [Example 1](../examples/tutorial/default_plan.json) `ProcessorRange`, `WarpRange`, `SramRange`, and `TaskRange` are in the "range" format, i.e., `[Begin, End, Step]` that indicates an arithmetic integer sequence with a common difference of `Step`, starting from `Begin` and ends before `End` (does not include `End`). They alternatively can be in the format `[Begin, End]` that assumes `Step` is 1. +## Architecture + +A name that refers to the hardware architecture where the plan is supposed to run over. The following names are currently supported. + +- `ANY`: compatible with all architectures. + +- NVIDIA Family + - `CUDA`: compatible with all supported NVIDIA architectures. + - `CUDA_70`: compatible with NVIDIA Volta architecture. + - `CUDA_80`: compatible with NVIDIA Ampere architecture. + - `CUDA_90`: compatible with NVIDIA Hopper architecture. + +- AMD Family + - `ROCM`: compatible with all supported AMD architectures. + - `ROCM_90A`: compatible with AMD CDNA 2 (GFX90A) architecture. + - `ROCM_942`: compatible with AMD CDNA 3 (GFX942) architecture. + ## TaskInfo A `TaskInfo` object describes a sequential set of operators. The followings describe each field of `TaskInfo`. diff --git a/examples/llama/README.md b/examples/llama/README.md index 090dd1de3..1fe040ae0 100644 --- a/examples/llama/README.md +++ b/examples/llama/README.md @@ -29,10 +29,10 @@ Llama2 examples over ARK. 4. Download Llama2 model weights and tokenizer weights. * The model and tokenizer should be compatible with the [official PyTorch implementation](https://github.com/facebookresearch/llama/blob/main/llama). -5. Run the model accuracy test. `--pth_path` is the path to the model weights file (`consolidated.00.pth`). +5. Run the model accuracy test. `--ckpt_dir` is the directory where the model weight files are at (e.g., `consolidated.00.pth`). ```bash - python3 model_test.py --pth_path=/path/to/model/weights.pth + python3 model_test.py --ckpt_dir=/directory/of/model/weights ``` 6. Test text generation. `--pth_path` is the path to the model weights file (`consolidated.00.pth`), `--tok_path` is the path to the tokenizer weights file (`tokenizer.model`), and `--params_path` is the path to the model parameters (`params.json`). diff --git a/examples/llama/model_7b_b1_s2048.py b/examples/llama/model_7b_b1_s2048.py new file mode 100644 index 000000000..d4a080c84 --- /dev/null +++ b/examples/llama/model_7b_b1_s2048.py @@ -0,0 +1,704 @@ +# Copyright (c) Microsoft Corporation. +# Licensed under the MIT license. + +"""LLaMA 2 Transformer model. + Correspond to https://github.com/facebookresearch/llama/blob/main/llama/model.py +""" + +import ark +import math +from dataclasses import dataclass +from typing import Optional +import os + + +@dataclass +class ModelArgs: + dim: int = 4096 + n_layers: int = 32 + n_heads: int = 32 + n_kv_heads: Optional[int] = None + vocab_size: int = -1 # defined later by tokenizer + multiple_of: int = ( + 256 # make SwiGLU hidden layer size multiple of large power of 2 + ) + ffn_dim_multiplier: Optional[float] = None + norm_eps: float = 1e-5 + max_batch_size: int = 32 + max_seq_len: int = 2048 + + +@dataclass +class ModelArgs7B(ModelArgs): + dim: int = 4096 + n_layers: int = 32 + n_heads: int = 32 + n_kv_heads: Optional[int] = None + vocab_size: int = -1 # defined later by tokenizer + multiple_of: int = ( + 256 # make SwiGLU hidden layer size multiple of large power of 2 + ) + ffn_dim_multiplier: Optional[float] = None + norm_eps: float = 1e-5 + max_batch_size: int = 32 + max_seq_len: int = 2048 + + +@dataclass +class ModelArgs13B(ModelArgs): + dim: int = 5120 + n_layers: int = 40 + n_heads: int = 40 + n_kv_heads: Optional[int] = None + vocab_size: int = -1 # defined later by tokenizer + multiple_of: int = ( + 256 # make SwiGLU hidden layer size multiple of large power of 2 + ) + ffn_dim_multiplier: Optional[float] = None + norm_eps: float = 1e-5 + max_batch_size: int = 32 + max_seq_len: int = 2048 + + +@dataclass +class ModelArgs70B(ModelArgs): + dim: int = 8192 + n_layers: int = 80 + n_heads: int = 64 + n_kv_heads: Optional[int] = 8 + vocab_size: int = -1 + multiple_of: int = ( + 4096 # make SwiGLU hidden layer size multiple of large power of 2 + ) + ffn_dim_multiplier: Optional[float] = 1.3 + norm_eps: float = 1e-5 + max_batch_size: int = 32 + max_seq_len: int = 4096 + + +class RMSNorm(ark.Module): + """ + Root mean square layer normalization (RMSNorm). + """ + + def __init__( + self, dim: int, eps: float = 1e-6, dtype: ark.DataType = ark.fp16 + ): + super().__init__() + self.eps = eps + self.dtype = dtype + self.weight = ark.parameter([1, 1, dim], ark.fp32) + + def forward(self, x): + with ark.PlannerContext( + warp_range=[0, 8], + sync=False, + config={ + "NumWarps": 1, + "SramBytes": 0, + "NumTasks": 2048, + "Granularity": 7, + }, + ): + with ark.PlannerContext(config={"Tile": [1, 4096]}): + x = ark.cast(x, ark.fp32) + x2 = ark.mul(x, x) + with ark.PlannerContext(config={"ImplType": "WarpWise"}): + mean = ark.reduce_mean(x2, axis=-1) + with ark.PlannerContext( + config={ + "NumWarps": 1, + "SramBytes": 0, + "Tile": [64, 1], + "NumTasks": 32, + } + ): + rrms = ark.rsqrt(mean) + with ark.PlannerContext( + warp_range=[0, 8], + sync=False, + config={ + "NumWarps": 1, + "SramBytes": 0, + "NumTasks": 2048, + "Tile": [1, 4096], + "Granularity": 7, + }, + ): + x = ark.mul(x, rrms) + x = ark.mul(x, self.weight, x) + return ark.cast(x, self.dtype) + + +class ColumnParallelLinear(ark.Module): + """Linear layer with column parallelism. + + The linear layer is defined as Y = XA + b. A is parallelized along + its second dimension as A = [A_1, ..., A_p]. + Here the weight = A^T, so we need to partition the weight matrix along + its first dimension. + + """ + + def __init__( + self, + in_dim: int, + out_dim: int, + dtype: ark.DataType = ark.fp16, + gather_output: bool = True, + local_rank: int = 0, + world_size: int = 1, + ): + super().__init__() + self.in_dim = in_dim + self.out_dim = out_dim + self.dtype = dtype + self.local_rank = local_rank + self.world_size = world_size + self.gather_output = gather_output + + self.weight = ark.parameter([out_dim // world_size, in_dim], dtype) + + def forward(self, x): + if self.world_size == 1 or self.gather_output == False: + return ark.matmul(x, self.weight, transpose_other=True) + # We need to concat the output_tensor_shards along the last dimension + output_tensor = ark.tensor( + [x.shape()[0], x.shape()[1], self.out_dim], self.dtype + ) + output_tensor_shards = ark.sharding( + output_tensor, + axis=2, + dim_per_shard=self.out_dim // self.world_size, + ) + local_result = ark.identity( + output_tensor_shards[self.local_rank], deps=output_tensor_shards + ) + # (batch_size, seq_len, out_dim // world_size) + local_result = ark.matmul( + x, self.weight, local_result, transpose_other=True + ) + gather_input = ark.identity(output_tensor, deps=[local_result]) + # return gather_input + gather_reshape = ark.reshape( + gather_input, [x.shape()[0] * x.shape()[1], self.out_dim] + ) + gather_out = ark.local_all_gather( + gather_reshape, self.local_rank, self.world_size, 1 + ) + return ark.reshape( + gather_out, [x.shape()[0], x.shape()[1], self.out_dim] + ) + + +class RowParallelLinear(ark.Module): + """Linear layer with row parallelism. + + The linear layer is defined as Y = XA + b. A is parallelized along + its first dimension and X along its second dimension as: + - - + | A_1 | + | . | + A = | . | X = [X_1, ..., X_p] + | . | + | A_p | + - - + + Here the weight = A^T, so we need to partition the weight matrix along + its second dimension. + """ + + def __init__( + self, + in_dim: int, + out_dim: int, + dtype: ark.DataType = ark.fp16, + input_is_parallel: bool = False, + local_rank: int = 0, + world_size: int = 1, + ): + super().__init__() + self.in_dim = in_dim + self.out_dim = out_dim + self.dtype = dtype + self.local_rank = local_rank + self.world_size = world_size + self.input_is_parallel = input_is_parallel + + self.weight = ark.parameter([out_dim, in_dim // world_size], dtype) + + def forward(self, x): + if self.world_size == 1: + return ark.matmul(x, self.weight, transpose_other=True) + x_ndims = len(x.shape()) + if self.input_is_parallel: + input_parallel = x + else: + x_shards = ark.sharding( + x, x_ndims - 1, self.in_dim // self.world_size + ) + input_parallel = x_shards[self.local_rank] + local_result = ark.matmul( + input_parallel, self.weight, transpose_other=True + ) + reduced_result = ark.local_all_reduce( + local_result, self.local_rank, self.world_size + ) + return reduced_result + + +class ParallelEmbedding(ark.Module): + """Embedding layer.""" + + def __init__( + self, + vocab_size: int, + dim: int, + dtype: ark.DataType, + local_rank: int = 0, + world_size: int = 1, + ): + super().__init__() + self.vocab_size = vocab_size + self.dim = dim + self.weight = ark.parameter([vocab_size, dim // world_size], dtype) + self.out_dim = dim + self.dtype = dtype + self.world_size = world_size + self.local_rank = local_rank + + def forward(self, x): + if self.world_size == 1: + return ark.embedding(x, self.weight) + + output_tensor = ark.tensor( + [x.shape()[0], x.shape()[1], self.out_dim], self.dtype + ) + output_tensor_shards = ark.sharding( + output_tensor, axis=2, dim_per_shard=self.out_dim // self.world_size + ) + local_result = ark.identity( + output_tensor_shards[self.local_rank], deps=output_tensor_shards + ) + local_result = ark.embedding(x, self.weight, local_result) + gather_input = ark.identity(output_tensor, deps=[local_result]) + gather_reshape = ark.reshape( + gather_input, [x.shape()[0] * x.shape()[1], self.out_dim] + ) + gather_out = ark.local_all_gather( + gather_reshape, self.local_rank, self.world_size, 1 + ) + return ark.reshape( + gather_out, [x.shape()[0], x.shape()[1], self.out_dim] + ) + + +class Linear(ark.Module): + """ + Linear layer module with weights and no bias. + """ + + def __init__( + self, in_dim: int, out_dim: int, dtype: ark.DataType = ark.fp16 + ): + super().__init__() + self.dtype = dtype + self.weight = ark.parameter([out_dim, in_dim], dtype) + + def forward(self, x): + return ark.matmul(x, self.weight, transpose_other=True) + + +class Silu(ark.Module): + """ + Silu activation function, silu(x) = x * sigmoid(x) + """ + + def __init__(self): + super().__init__() + + def forward(self, x: ark.Tensor): + # We need to specify output tensor so that the sigmoid op will not be an in-place operator + output = ark.tensor(x.shape(), x.dtype()) + x1 = ark.sigmoid(x, output) + return ark.mul(x, x1) + + +class FeedForward(ark.Module): + def __init__( + self, + dim: int, + hidden_dim: int, + multiple_of: int, + ffn_dim_multiplier: Optional[float], + dtype: ark.DataType = ark.fp16, + local_rank: int = 0, + world_size: int = 1, + ): + super().__init__() + hidden_dim = int(2 * hidden_dim / 3) + # custom dim factor multiplier + if ffn_dim_multiplier is not None: + hidden_dim = int(ffn_dim_multiplier * hidden_dim) + hidden_dim = multiple_of * ( + (hidden_dim + multiple_of - 1) // multiple_of + ) + + self.w1 = ColumnParallelLinear( + dim, hidden_dim, dtype, False, local_rank, world_size + ) + self.w2 = RowParallelLinear( + hidden_dim, dim, dtype, True, local_rank, world_size + ) + self.w3 = ColumnParallelLinear( + dim, hidden_dim, dtype, False, local_rank, world_size + ) + + def forward(self, x): + # self.w2(F.silu(self.w1(x)) * self.w3(x)) + with ark.PlannerContext( + warp_range=[0, 8], + sram_range=[0, 49344], + sync=False, + config={ + "NumWarps": 4, + "NumTasks": 688, + }, + ): + with ark.PlannerContext( + config={"SramBytes": 24672, "TileShapeMNK": [256, 128, 32]} + ): + x1 = self.w1(x) + with ark.PlannerContext(config={"SramBytes": 0, "Tile": [256, 128]}): + x1 = Silu()(x1) + with ark.PlannerContext( + warp_range=[0, 8], + sram_range=[0, 49344], + sync=False, + config={ + "NumWarps": 4, + "NumTasks": 688, + }, + ): + with ark.PlannerContext( + config={"SramBytes": 24672, "TileShapeMNK": [256, 128, 32]} + ): + x2 = self.w3(x) + with ark.PlannerContext(config={"SramBytes": 0, "Tile": [256, 128]}): + x3 = ark.mul(x1, x2) + x4 = self.w2(x3) + return x4 + + +def apply_rotary_emb(xq, xk, freqs_cis): + """ + Apply rotary embeddings to xq and xk. + """ + xq_out = ark.rope(xq, freqs_cis) + xk_out = ark.rope(xk, freqs_cis) + return xq_out, xk_out + + +class Softmax(ark.Module): + def __init__(self): + super(Softmax, self).__init__() + + def forward(self, input): + with ark.PlannerContext( + warp_range=[0, 8], + sram_range=[0, 0], + sync=False, + config={ + "NumWarps": 1, + "SramBytes": 0, + "NumTasks": 65536, + }, + ): + with ark.PlannerContext(config={"ImplType": "WarpWise"}): + max = ark.reduce_max(input, axis=-1) + with ark.PlannerContext(config={"Tile": [1, 2048]}): + output = ark.sub(input, max) + output = ark.exp(output) + with ark.PlannerContext(config={"ImplType": "WarpWise"}): + sum = ark.reduce_sum(output, axis=-1) + with ark.PlannerContext(config={"Tile": [1, 2048]}): + output = ark.div(output, sum) + return output + + +class Attention(ark.Module): + def __init__( + self, + args: ModelArgs, + dtype: ark.DataType = ark.fp16, + local_rank: int = 0, + world_size: int = 1, + ): + super().__init__() + self.n_kv_heads = ( + args.n_heads if args.n_kv_heads is None else args.n_kv_heads + ) + model_parallel_size = world_size + self.dtype = dtype + self.n_local_heads = args.n_heads // model_parallel_size + self.n_local_kv_heads = self.n_kv_heads // model_parallel_size + self.n_rep = self.n_local_heads // self.n_local_kv_heads + self.head_dim = args.dim // args.n_heads + self.wq = ColumnParallelLinear( + args.dim, + args.n_heads * self.head_dim, + dtype, + False, + local_rank, + world_size, + ) + self.wk = ColumnParallelLinear( + args.dim, + self.n_kv_heads * self.head_dim, + dtype, + False, + local_rank, + world_size, + ) + self.wv = ColumnParallelLinear( + args.dim, + self.n_kv_heads * self.head_dim, + dtype, + False, + local_rank, + world_size, + ) + self.wo = RowParallelLinear( + args.n_heads * self.head_dim, + args.dim, + dtype, + True, + local_rank, + world_size, + ) + + def forward( + self, + x: ark.Tensor, + start_pos: int, + freqs_cis: ark.Tensor, + mask: Optional[ark.Tensor], + ): + bsz, seqlen, _ = x.shape() + + with ark.PlannerContext( + warp_range=[0, 4], + sram_range=[0, 24672], + sync=False, + config={"NumWarps": 4, "NumTasks": 256}, + ): + with ark.PlannerContext( + config={"SramBytes": 24672, "TileShapeMNK": [256, 128, 32]} + ): + xq = self.wq(x) + xq = ark.reshape( + xq, [bsz, seqlen, self.n_local_heads, self.head_dim] + ) + with ark.PlannerContext( + config={"SramBytes": 0, "Tile": [256, 1, 128]} + ): + if freqs_cis is not None: + xq = ark.rope(xq, freqs_cis) + with ark.PlannerContext(config={"SramBytes": 0, "Tile": [256, 128]}): + xq = ark.transpose(xq, [0, 2, 1, 3]) + + with ark.PlannerContext( + warp_range=[0, 4], + sram_range=[0, 24672], + sync=False, + config={"NumWarps": 4, "NumTasks": 256}, + ): + with ark.PlannerContext( + config={"SramBytes": 24672, "TileShapeMNK": [256, 128, 32]} + ): + xk = self.wk(x) + xk = ark.reshape( + xk, [bsz, seqlen, self.n_local_kv_heads, self.head_dim] + ) + with ark.PlannerContext( + config={"SramBytes": 0, "Tile": [256, 1, 128]} + ): + if freqs_cis is not None: + xk = ark.rope(xk, freqs_cis) + keys = xk + with ark.PlannerContext(config={"SramBytes": 0, "Tile": [256, 128]}): + keys = ark.transpose(keys, [0, 2, 1, 3]) + + with ark.PlannerContext( + warp_range=[0, 4], + sram_range=[0, 24672], + sync=False, + config={ + "NumWarps": 4, + "NumTasks": 256, + "SramBytes": 24672, + "TileShapeMNK": [256, 128, 32], + }, + ): + with ark.PlannerContext( + config={"SramBytes": 24672, "TileShapeMNK": [256, 128, 32]} + ): + xv = self.wv(x) + xv = ark.reshape( + xv, [bsz, seqlen, self.n_local_kv_heads, self.head_dim] + ) + values = xv + with ark.PlannerContext( + config={"SramBytes": 0, "Tile": [256, 1, 128]} + ): + values = ark.transpose(values, [0, 2, 1, 3]) + + with ark.PlannerContext( + warp_range=[0, 8], + sram_range=[0, 49344], + sync=False, + config={ + "NumWarps": 4, + "NumTasks": 4096, + "Granularity": 2, + }, + ): + with ark.PlannerContext( + config={"SramBytes": 24672, "TileShapeMNK": [256, 128, 32]} + ): + scores = ark.matmul(xq, keys, transpose_other=True) + with ark.PlannerContext(config={"SramBytes": 0, "Tile": [256, 128]}): + scores = ark.mul(scores, 1.0 / math.sqrt(self.head_dim)) + + if mask is not None: + scores = ark.add(scores, mask) + + scores = Softmax()(scores) + + with ark.PlannerContext( + warp_range=[0, 4], + sram_range=[0, 24672], + sync=False, + config={ + "NumWarps": 4, + "NumTasks": 256, + }, + ): + with ark.PlannerContext( + config={"SramBytes": 24672, "TileShapeMNK": [256, 128, 32]} + ): + output = ark.matmul(scores, values) + with ark.PlannerContext( + config={"SramBytes": 0, "Tile": [256, 1, 128]} + ): + output = ark.transpose(output, [0, 2, 1, 3]) + output = ark.reshape( + output, [bsz, seqlen, self.head_dim * self.n_local_heads] + ) + return self.wo(output) + + +class TransformerBlock(ark.Module): + def __init__( + self, + layer_id: int, + args: ModelArgs, + dtype: ark.DataType = ark.fp16, + local_rank: int = 0, + world_size: int = 1, + ): + super().__init__() + self.n_heads = args.n_heads + self.dim = args.dim + self.head_dim = args.dim // args.n_heads + self.attention = Attention(args, dtype, local_rank, world_size) + self.feed_forward = FeedForward( + dim=args.dim, + hidden_dim=4 * args.dim, + multiple_of=args.multiple_of, + ffn_dim_multiplier=args.ffn_dim_multiplier, + dtype=dtype, + local_rank=local_rank, + world_size=world_size, + ) + self.layer_id = layer_id + self.attention_norm = RMSNorm(args.dim, eps=args.norm_eps, dtype=dtype) + self.ffn_norm = RMSNorm(args.dim, eps=args.norm_eps, dtype=dtype) + + def forward( + self, + x: ark.Tensor, + start_pos: int, + freqs_cis: ark.Tensor, + mask: Optional[ark.Tensor], + ): + attention_norm_x = self.attention_norm(x) + h = self.attention.forward(attention_norm_x, start_pos, freqs_cis, mask) + with ark.PlannerContext( + warp_range=[0, 4], + config={ + "NumWarps": 4, + "Tile": [256, 128], + "NumTasks": 256, + "SramBytes": 0, + }, + ): + h = ark.add(x, h) + ff = self.feed_forward(self.ffn_norm(h)) + with ark.PlannerContext( + warp_range=[0, 4], + config={ + "NumWarps": 4, + "Tile": [256, 128], + "NumTasks": 256, + "SramBytes": 0, + }, + ): + out = ark.add(h, ff) + return out + + +class Transformer(ark.Module): + def __init__( + self, + params: ModelArgs, + dtype: ark.DataType = ark.fp16, + local_rank: int = 0, + world_size: int = 1, + ): + super().__init__() + self.params = params + self.vocab_size = params.vocab_size + self.n_layers = params.n_layers + + self.tok_embeddings = ParallelEmbedding( + params.vocab_size, params.dim, dtype, local_rank, world_size + ) + + self.layers = [] + for layer_id in range(self.n_layers): + self.layers.append( + TransformerBlock( + layer_id, params, dtype, local_rank, world_size + ) + ) + self.register_module(f"layers.{layer_id}", self.layers[layer_id]) + self.norm = RMSNorm(params.dim, eps=params.norm_eps, dtype=dtype) + self.output = ColumnParallelLinear( + params.dim, params.vocab_size, dtype, True, local_rank, world_size + ) + + def forward( + self, + tokens: ark.Tensor, + start_pos: int, + freqs_cis: ark.Tensor, + mask: Optional[ark.Tensor], + ): + h = self.tok_embeddings(tokens) + + for layer in self.layers: + h = layer(h, start_pos, freqs_cis, mask) + h = self.norm(h) + output = self.output(h) + return output diff --git a/examples/llama/model_test.py b/examples/llama/model_test.py index 737d3ec8b..f559a826b 100644 --- a/examples/llama/model_test.py +++ b/examples/llama/model_test.py @@ -58,30 +58,37 @@ def run_ark( ] output = module(*module_inputs) - runtime = ark.Runtime() - # Prefer num_warps_per_sm = 16 for nvidia and 8 for amd - runtime.launch(num_warps_per_sm=8) + with ark.Runtime() as rt: + plan = ark.DefaultPlanner().plan() + with open("plan.json", "w") as f: + f.write(str(plan)) + rt.launch(plan=plan) - # Load model parameters - if state_dict: - module.load_state_dict(state_dict) + # Load model parameters + if state_dict: + print("Loading state_dict") + module.load_state_dict(state_dict) + print("Loading state_dict done") - # Load input data into tensors - tensors = [i for i in module_inputs if isinstance(i, ark.Tensor)] - tensor_data = [i for i in inputs if isinstance(i, np.ndarray)] - for tensor, ndarray in zip(tensors, tensor_data): - tensor.from_numpy(ndarray) + # Load input data into tensors + tensors = [i for i in module_inputs if isinstance(i, ark.Tensor)] + tensor_data = [i for i in inputs if isinstance(i, np.ndarray)] + for tensor, ndarray in zip(tensors, tensor_data): + tensor.from_numpy(ndarray) - start_time = time.time() + start_time = time.time() - # Run the model - runtime.run(iter=iterations) + # Run the model + print("Run:", iterations) - end_time = time.time() + rt.run(iter=iterations) + print("Run done") - if isinstance(output, list) or isinstance(output, tuple): - outputs = [o.to_numpy() for o in output] - outputs = [output.to_numpy()] + end_time = time.time() + + if isinstance(output, list) or isinstance(output, tuple): + outputs = [o.to_numpy() for o in output] + outputs = [output.to_numpy()] return RunResults(outputs=outputs, runtime=end_time - start_time) @@ -160,7 +167,9 @@ def test_module( else: prefix = module_name_prefix + "." if module_name_prefix else "" # Load the state_dict from the given path + print("Loading ckpt:", ckpt_path) state_dict_pt = torch.load(ckpt_path) + print("Loading ckpt done") state_dict_pt = { k[len(prefix) :]: v for k, v in state_dict_pt.items() @@ -182,6 +191,7 @@ def test_module( rank=rank, world_size=world_size, ) + print("Run ARK done") if not test_thru_ark_only: # PyTorch module @@ -195,6 +205,7 @@ def test_module( inputs_pt, iterations=test_thru_iterations if test_thru else 1, ) + print("Run PyTorch done") if test_thru: print( @@ -430,43 +441,43 @@ def test_transformer_block( low=-1, high=1, size=(batch_size, seq_len, args.dim) ).astype(dtype) - module = model_ark.Attention( - args, ark.DataType.from_numpy(dtype), rank, world_size - ) + # module = model_ark.Attention( + # args, ark.DataType.from_numpy(dtype), rank, world_size + # ) # module_inputs = [ # ark.tensor(list(i.shape), ark.DataType.from_numpy(i.dtype)) # if isinstance(i, np.ndarray) # else i # for i in inputs # ] - feature_tensor = ark.tensor( - list(feature.shape), ark.DataType.from_numpy(feature.dtype) - ) - freqs_cis_ark_tensor = ark.tensor( - list(freqs_cis_ark.shape), ark.DataType.from_numpy(freqs_cis_ark.dtype) - ) - output = module(feature_tensor, 0, freqs_cis_ark_tensor, None) - - ark.Model.get_model().create_nodes() - print(ark.Model.get_model().serialize()) - - # test_module( - # module_class_ark=model_ark.TransformerBlock, - # module_args_ark=[ - # 0, - # args, - # ark.DataType.from_numpy(dtype), - # rank, - # world_size, - # ], - # inputs_ark=[feature, 0, freqs_cis_ark, None], - # module_class_pt=model_pt.TransformerBlock, - # module_args_pt=[0, args], - # inputs_pt=[feature.astype(dtype), 0, freqs_cis, None], - # module_name_prefix="layers.0", - # rank=rank, - # world_size=world_size, + # feature_tensor = ark.tensor( + # list(feature.shape), ark.DataType.from_numpy(feature.dtype) # ) + # freqs_cis_ark_tensor = ark.tensor( + # list(freqs_cis_ark.shape), ark.DataType.from_numpy(freqs_cis_ark.dtype) + # ) + # output = module(feature_tensor, 0, freqs_cis_ark_tensor, None) + + # print(ark.Model.get_model().serialize()) + + test_module( + module_class_ark=model_ark.TransformerBlock, + module_args_ark=[ + 0, + args, + ark.DataType.from_numpy(dtype), + rank, + world_size, + ], + inputs_ark=[feature, 0, freqs_cis_ark, None], + module_class_pt=model_pt.TransformerBlock, + module_args_pt=[0, args], + inputs_pt=[feature.astype(dtype), 0, freqs_cis, None], + module_name_prefix="layers.0", + rank=rank, + world_size=world_size, + test_thru=False, + ) def test_transformer( @@ -570,7 +581,7 @@ def worker( # Configurations args = ModelArgs7B() batch_size = 1 - seq_len = 512 + seq_len = 2048 dtype = np.float16 world_size = ngpus @@ -578,7 +589,7 @@ def worker( args.vocab_size = 32000 # Reduce max_seq_len due to OOM from the PyTorch model - args.max_seq_len = 512 + args.max_seq_len = 2048 # Verify the configurations assert batch_size <= args.max_batch_size diff --git a/examples/llama/plan_llama2_7b_b1_s2048.json b/examples/llama/plan_llama2_7b_b1_s2048.json new file mode 100644 index 000000000..b0bc757dc --- /dev/null +++ b/examples/llama/plan_llama2_7b_b1_s2048.json @@ -0,0 +1,1206 @@ +{ + "Rank": 0, + "WorldSize": 1, + "Architecture": "ROCM_942", + "NumProcessors": 304, + "NumWarpsPerProcessor": 8, + "TaskInfos": [ + { + "Id": 0, + "NumWarps": 1, + "SramBytes": 0, + "Ops": [ + { + "Type": "Cast", + "Name": "cast", + "IsVirtual": false, + "ReadTensors": [ + {"Id":9,"DataType":"FP16","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":11,"DataType":"FP32","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":11,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":12,"DataType":"FP32","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":11,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": {}, + "Config": { + "NumWarps": 1, + "SramBytes": 0, + "Tile": [1,4096], + "NumTasks": 2048 + } + }, + { + "Type": "Mul", + "Name": "mul", + "IsVirtual": false, + "ReadTensors": [ + {"Id":12,"DataType":"FP32","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":11,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}}, + {"Id":12,"DataType":"FP32","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":11,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":13,"DataType":"FP32","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":12,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":14,"DataType":"FP32","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":12,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": {}, + "Config": { + "NumWarps": 1, + "SramBytes": 0, + "Tile": [1,4096], + "NumTasks": 2048 + } + }, + { + "Type": "ReduceMean", + "Name": "reduce_mean", + "IsVirtual": false, + "ReadTensors": [ + {"Id":14,"DataType":"FP32","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":12,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":15,"DataType":"FP32","Shape":[1,2048,1],"Strides":[1,2048,1],"Offsets":[0,0,0],"PaddedShape":[1,2048,1],"Buffer":{"Id":13,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":16,"DataType":"FP32","Shape":[1,2048,1],"Strides":[1,2048,1],"Offsets":[0,0,0],"PaddedShape":[1,2048,1],"Buffer":{"Id":13,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": { + "Axis": {"INT":2}, + "KeepDim": {"BOOL":true} + }, + "Config": { + "NumWarps": 1, + "ImplType": "WarpWise", + "SramBytes": 0, + "NumTasks": 2048 + } + } + ] + }, + { + "Id": 3, + "NumWarps": 1, + "SramBytes": 0, + "Ops": [ + { + "Type": "Rsqrt", + "Name": "rsqrt", + "IsVirtual": false, + "ReadTensors": [ + {"Id":16,"DataType":"FP32","Shape":[1,2048,1],"Strides":[1,2048,1],"Offsets":[0,0,0],"PaddedShape":[1,2048,1],"Buffer":{"Id":13,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":17,"DataType":"FP32","Shape":[1,2048,1],"Strides":[1,2048,1],"Offsets":[0,0,0],"PaddedShape":[1,2048,1],"Buffer":{"Id":14,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":18,"DataType":"FP32","Shape":[1,2048,1],"Strides":[1,2048,1],"Offsets":[0,0,0],"PaddedShape":[1,2048,1],"Buffer":{"Id":14,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": {}, + "Config": { + "NumWarps": 1, + "SramBytes": 0, + "Tile": [64,1], + "NumTasks": 32 + } + } + ] + }, + { + "Id": 4, + "NumWarps": 1, + "SramBytes": 0, + "Ops": [ + { + "Type": "Mul", + "Name": "mul_1", + "IsVirtual": false, + "ReadTensors": [ + {"Id":12,"DataType":"FP32","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":11,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}}, + {"Id":18,"DataType":"FP32","Shape":[1,2048,1],"Strides":[1,2048,1],"Offsets":[0,0,0],"PaddedShape":[1,2048,1],"Buffer":{"Id":14,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":19,"DataType":"FP32","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":15,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":20,"DataType":"FP32","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":15,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": {}, + "Config": { + "NumWarps": 1, + "SramBytes": 0, + "Tile": [1,4096], + "NumTasks": 2048 + } + }, + { + "Type": "Mul", + "Name": "mul_2", + "IsVirtual": false, + "ReadTensors": [ + {"Id":20,"DataType":"FP32","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":15,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}}, + {"Id":7,"DataType":"FP32","Shape":[1,1,4096],"Strides":[1,1,4096],"Offsets":[0,0,0],"PaddedShape":[1,1,4096],"Buffer":{"Id":7,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":20,"DataType":"FP32","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":15,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":21,"DataType":"FP32","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":15,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": {}, + "Config": { + "NumWarps": 1, + "SramBytes": 0, + "Tile": [1,4096], + "NumTasks": 2048 + } + }, + { + "Type": "Cast", + "Name": "cast_1", + "IsVirtual": false, + "ReadTensors": [ + {"Id":21,"DataType":"FP32","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":15,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":22,"DataType":"FP16","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":16,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":23,"DataType":"FP16","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":16,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": {}, + "Config": { + "NumWarps": 1, + "SramBytes": 0, + "Tile": [1,4096], + "NumTasks": 2048 + } + } + ] + }, + { + "Id": 7, + "NumWarps": 4, + "SramBytes": 24672, + "Ops": [ + { + "Type": "Matmul", + "Name": "matmul", + "IsVirtual": false, + "ReadTensors": [ + {"Id":23,"DataType":"FP16","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":16,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}}, + {"Id":0,"DataType":"FP16","Shape":[4096,4096],"Strides":[4096,4096],"Offsets":[0,0],"PaddedShape":[4096,4096],"Buffer":{"Id":0,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":24,"DataType":"FP16","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":17,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":25,"DataType":"FP16","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":17,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": { + "TransposeInput": {"BOOL":false}, + "TransposeOther": {"BOOL":true} + }, + "Config": { + "NumWarps": 4, + "SramBytes": 24672, + "TileShapeMNK": [256,128,32], + "NumTasks": 256 + } + }, + { + "Type": "Rope", + "Name": "rope", + "IsVirtual": false, + "ReadTensors": [ + {"Id":30,"DataType":"FP16","Shape":[1,2048,32,128],"Strides":[1,2048,32,128],"Offsets":[0,0,0,0],"PaddedShape":[1,2048,32,128],"Buffer":{"Id":17,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}}, + {"Id":10,"DataType":"FP16","Shape":[1,2048,1,128],"Strides":[1,2048,1,128],"Offsets":[0,0,0,0],"PaddedShape":[1,2048,1,128],"Buffer":{"Id":10,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":33,"DataType":"FP16","Shape":[1,2048,32,128],"Strides":[1,2048,32,128],"Offsets":[0,0,0,0],"PaddedShape":[1,2048,32,128],"Buffer":{"Id":20,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":34,"DataType":"FP16","Shape":[1,2048,32,128],"Strides":[1,2048,32,128],"Offsets":[0,0,0,0],"PaddedShape":[1,2048,32,128],"Buffer":{"Id":20,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": {}, + "Config": { + "NumWarps": 4, + "SramBytes": 0, + "Tile": [256,1,128], + "NumTasks": 256 + } + } + ] + }, + { + "Id": 10, + "NumWarps": 4, + "SramBytes": 24672, + "Ops": [ + { + "Type": "Matmul", + "Name": "matmul_1", + "IsVirtual": false, + "ReadTensors": [ + {"Id":23,"DataType":"FP16","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":16,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}}, + {"Id":1,"DataType":"FP16","Shape":[4096,4096],"Strides":[4096,4096],"Offsets":[0,0],"PaddedShape":[4096,4096],"Buffer":{"Id":1,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":26,"DataType":"FP16","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":18,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":27,"DataType":"FP16","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":18,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": { + "TransposeInput": {"BOOL":false}, + "TransposeOther": {"BOOL":true} + }, + "Config": { + "NumWarps": 4, + "SramBytes": 24672, + "TileShapeMNK": [256,128,32], + "NumTasks": 256 + } + }, + { + "Type": "Rope", + "Name": "rope_1", + "IsVirtual": false, + "ReadTensors": [ + {"Id":31,"DataType":"FP16","Shape":[1,2048,32,128],"Strides":[1,2048,32,128],"Offsets":[0,0,0,0],"PaddedShape":[1,2048,32,128],"Buffer":{"Id":18,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}}, + {"Id":10,"DataType":"FP16","Shape":[1,2048,1,128],"Strides":[1,2048,1,128],"Offsets":[0,0,0,0],"PaddedShape":[1,2048,1,128],"Buffer":{"Id":10,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":35,"DataType":"FP16","Shape":[1,2048,32,128],"Strides":[1,2048,32,128],"Offsets":[0,0,0,0],"PaddedShape":[1,2048,32,128],"Buffer":{"Id":21,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":36,"DataType":"FP16","Shape":[1,2048,32,128],"Strides":[1,2048,32,128],"Offsets":[0,0,0,0],"PaddedShape":[1,2048,32,128],"Buffer":{"Id":21,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": {}, + "Config": { + "NumWarps": 4, + "SramBytes": 0, + "Tile": [256,128], + "NumTasks": 256 + } + } + ] + }, + { + "Id": 13, + "NumWarps": 4, + "SramBytes": 24672, + "Ops": [ + { + "Type": "Matmul", + "Name": "matmul_2", + "IsVirtual": false, + "ReadTensors": [ + {"Id":23,"DataType":"FP16","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":16,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}}, + {"Id":2,"DataType":"FP16","Shape":[4096,4096],"Strides":[4096,4096],"Offsets":[0,0],"PaddedShape":[4096,4096],"Buffer":{"Id":2,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":28,"DataType":"FP16","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":19,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":29,"DataType":"FP16","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":19,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": { + "TransposeInput": {"BOOL":false}, + "TransposeOther": {"BOOL":true} + }, + "Config": { + "NumWarps": 4, + "SramBytes": 24672, + "TileShapeMNK": [256,128,32], + "NumTasks": 256 + } + } + ] + }, + { + "Id": 15, + "NumWarps": 4, + "SramBytes": 24672, + "Ops": [ + { + "Type": "Matmul", + "Name": "matmul_3", + "IsVirtual": false, + "ReadTensors": [ + {"Id":38,"DataType":"FP16","Shape":[1,32,2048,128],"Strides":[1,32,2048,128],"Offsets":[0,0,0,0],"PaddedShape":[1,32,2048,128],"Buffer":{"Id":22,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}}, + {"Id":42,"DataType":"FP16","Shape":[1,32,2048,128],"Strides":[1,32,2048,128],"Offsets":[0,0,0,0],"PaddedShape":[1,32,2048,128],"Buffer":{"Id":24,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":43,"DataType":"FP16","Shape":[1,32,2048,2048],"Strides":[1,32,2048,2048],"Offsets":[0,0,0,0],"PaddedShape":[1,32,2048,2048],"Buffer":{"Id":25,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":44,"DataType":"FP16","Shape":[1,32,2048,2048],"Strides":[1,32,2048,2048],"Offsets":[0,0,0,0],"PaddedShape":[1,32,2048,2048],"Buffer":{"Id":25,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": { + "TransposeInput": {"BOOL":false}, + "TransposeOther": {"BOOL":true} + }, + "Config": { + "NumWarps": 4, + "SramBytes": 24672, + "TileShapeMNK": [256,128,32], + "NumTasks": 4096 + } + }, + { + "Type": "ScalarMul", + "Name": "mul_3", + "IsVirtual": false, + "ReadTensors": [ + {"Id":44,"DataType":"FP16","Shape":[1,32,2048,2048],"Strides":[1,32,2048,2048],"Offsets":[0,0,0,0],"PaddedShape":[1,32,2048,2048],"Buffer":{"Id":25,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":45,"DataType":"FP16","Shape":[1,32,2048,2048],"Strides":[1,32,2048,2048],"Offsets":[0,0,0,0],"PaddedShape":[1,32,2048,2048],"Buffer":{"Id":26,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":46,"DataType":"FP16","Shape":[1,32,2048,2048],"Strides":[1,32,2048,2048],"Offsets":[0,0,0,0],"PaddedShape":[1,32,2048,2048],"Buffer":{"Id":26,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": { + "Factor": {"FLOAT":0.0883883461356163} + }, + "Config": { + "NumWarps": 4, + "SramBytes": 0, + "Tile": [256,128], + "NumTasks": 4096 + } + } + ] + }, + { + "Id": 17, + "NumWarps": 1, + "SramBytes": 0, + "Ops": [ + { + "Type": "ReduceMax", + "Name": "reduce_max", + "IsVirtual": false, + "ReadTensors": [ + {"Id":46,"DataType":"FP16","Shape":[1,32,2048,2048],"Strides":[1,32,2048,2048],"Offsets":[0,0,0,0],"PaddedShape":[1,32,2048,2048],"Buffer":{"Id":26,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":47,"DataType":"FP16","Shape":[1,32,2048,1],"Strides":[1,32,2048,1],"Offsets":[0,0,0,0],"PaddedShape":[1,32,2048,1],"Buffer":{"Id":27,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":48,"DataType":"FP16","Shape":[1,32,2048,1],"Strides":[1,32,2048,1],"Offsets":[0,0,0,0],"PaddedShape":[1,32,2048,1],"Buffer":{"Id":27,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": { + "Axis": {"INT":3}, + "KeepDim": {"BOOL":true} + }, + "Config": { + "NumWarps": 1, + "ImplType": "WarpWise", + "SramBytes": 0, + "NumTasks": 65536 + } + }, + { + "Type": "Sub", + "Name": "sub", + "IsVirtual": false, + "ReadTensors": [ + {"Id":46,"DataType":"FP16","Shape":[1,32,2048,2048],"Strides":[1,32,2048,2048],"Offsets":[0,0,0,0],"PaddedShape":[1,32,2048,2048],"Buffer":{"Id":26,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}}, + {"Id":48,"DataType":"FP16","Shape":[1,32,2048,1],"Strides":[1,32,2048,1],"Offsets":[0,0,0,0],"PaddedShape":[1,32,2048,1],"Buffer":{"Id":27,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":46,"DataType":"FP16","Shape":[1,32,2048,2048],"Strides":[1,32,2048,2048],"Offsets":[0,0,0,0],"PaddedShape":[1,32,2048,2048],"Buffer":{"Id":26,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":49,"DataType":"FP16","Shape":[1,32,2048,2048],"Strides":[1,32,2048,2048],"Offsets":[0,0,0,0],"PaddedShape":[1,32,2048,2048],"Buffer":{"Id":26,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": {}, + "Config": { + "NumWarps": 1, + "SramBytes": 0, + "Tile": [1,2048], + "NumTasks": 65536 + } + }, + { + "Type": "Exp", + "Name": "exp", + "IsVirtual": false, + "ReadTensors": [ + {"Id":49,"DataType":"FP16","Shape":[1,32,2048,2048],"Strides":[1,32,2048,2048],"Offsets":[0,0,0,0],"PaddedShape":[1,32,2048,2048],"Buffer":{"Id":26,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":49,"DataType":"FP16","Shape":[1,32,2048,2048],"Strides":[1,32,2048,2048],"Offsets":[0,0,0,0],"PaddedShape":[1,32,2048,2048],"Buffer":{"Id":26,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":50,"DataType":"FP16","Shape":[1,32,2048,2048],"Strides":[1,32,2048,2048],"Offsets":[0,0,0,0],"PaddedShape":[1,32,2048,2048],"Buffer":{"Id":26,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": {}, + "Config": { + "NumWarps": 1, + "SramBytes": 0, + "Tile": [1,2048], + "NumTasks": 65536 + } + }, + { + "Type": "ReduceSum", + "Name": "reduce_sum", + "IsVirtual": false, + "ReadTensors": [ + {"Id":50,"DataType":"FP16","Shape":[1,32,2048,2048],"Strides":[1,32,2048,2048],"Offsets":[0,0,0,0],"PaddedShape":[1,32,2048,2048],"Buffer":{"Id":26,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":51,"DataType":"FP16","Shape":[1,32,2048,1],"Strides":[1,32,2048,1],"Offsets":[0,0,0,0],"PaddedShape":[1,32,2048,1],"Buffer":{"Id":28,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":52,"DataType":"FP16","Shape":[1,32,2048,1],"Strides":[1,32,2048,1],"Offsets":[0,0,0,0],"PaddedShape":[1,32,2048,1],"Buffer":{"Id":28,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": { + "Axis": {"INT":3}, + "KeepDim": {"BOOL":true} + }, + "Config": { + "NumWarps": 1, + "ImplType": "WarpWise", + "SramBytes": 0, + "NumTasks": 65536 + } + }, + { + "Type": "Div", + "Name": "div", + "IsVirtual": false, + "ReadTensors": [ + {"Id":50,"DataType":"FP16","Shape":[1,32,2048,2048],"Strides":[1,32,2048,2048],"Offsets":[0,0,0,0],"PaddedShape":[1,32,2048,2048],"Buffer":{"Id":26,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}}, + {"Id":52,"DataType":"FP16","Shape":[1,32,2048,1],"Strides":[1,32,2048,1],"Offsets":[0,0,0,0],"PaddedShape":[1,32,2048,1],"Buffer":{"Id":28,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":50,"DataType":"FP16","Shape":[1,32,2048,2048],"Strides":[1,32,2048,2048],"Offsets":[0,0,0,0],"PaddedShape":[1,32,2048,2048],"Buffer":{"Id":26,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":53,"DataType":"FP16","Shape":[1,32,2048,2048],"Strides":[1,32,2048,2048],"Offsets":[0,0,0,0],"PaddedShape":[1,32,2048,2048],"Buffer":{"Id":26,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": {}, + "Config": { + "NumWarps": 1, + "SramBytes": 0, + "Tile": [1,2048], + "NumTasks": 65536 + } + } + ] + }, + { + "Id": 22, + "NumWarps": 4, + "SramBytes": 24672, + "Ops": [ + { + "Type": "Matmul", + "Name": "matmul_4", + "IsVirtual": false, + "ReadTensors": [ + {"Id":53,"DataType":"FP16","Shape":[1,32,2048,2048],"Strides":[1,32,2048,2048],"Offsets":[0,0,0,0],"PaddedShape":[1,32,2048,2048],"Buffer":{"Id":26,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}}, + {"Id":40,"DataType":"FP16","Shape":[1,32,2048,128],"Strides":[1,32,2048,128],"Offsets":[0,0,0,0],"PaddedShape":[1,32,2048,128],"Buffer":{"Id":23,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":54,"DataType":"FP16","Shape":[1,32,2048,128],"Strides":[1,32,2048,128],"Offsets":[0,0,0,0],"PaddedShape":[1,32,2048,128],"Buffer":{"Id":29,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":55,"DataType":"FP16","Shape":[1,32,2048,128],"Strides":[1,32,2048,128],"Offsets":[0,0,0,0],"PaddedShape":[1,32,2048,128],"Buffer":{"Id":29,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": { + "TransposeInput": {"BOOL":false}, + "TransposeOther": {"BOOL":false} + }, + "Config": { + "NumWarps": 4, + "SramBytes": 24672, + "TileShapeMNK": [256,128,32], + "NumTasks": 256 + } + } + ] + }, + { + "Id": 24, + "NumWarps": 4, + "SramBytes": 24672, + "Ops": [ + { + "Type": "Matmul", + "Name": "matmul_5", + "IsVirtual": false, + "ReadTensors": [ + {"Id":58,"DataType":"FP16","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":30,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}}, + {"Id":3,"DataType":"FP16","Shape":[4096,4096],"Strides":[4096,4096],"Offsets":[0,0],"PaddedShape":[4096,4096],"Buffer":{"Id":3,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":59,"DataType":"FP16","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":31,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":60,"DataType":"FP16","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":31,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": { + "TransposeInput": {"BOOL":false}, + "TransposeOther": {"BOOL":true} + }, + "Config": { + "NumWarps": 4, + "SramBytes": 24672, + "TileShapeMNK": [256,128,32], + "NumTasks": 256 + } + }, + { + "Type": "Add", + "Name": "add", + "IsVirtual": false, + "ReadTensors": [ + {"Id":9,"DataType":"FP16","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}}, + {"Id":60,"DataType":"FP16","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":31,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":61,"DataType":"FP16","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":32,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":62,"DataType":"FP16","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":32,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": {}, + "Config": { + "NumWarps": 4, + "SramBytes": 0, + "Tile": [256,128], + "NumTasks": 256 + } + }, + { + "Type": "Cast", + "Name": "cast_2", + "IsVirtual": false, + "ReadTensors": [ + {"Id":62,"DataType":"FP16","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":32,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":63,"DataType":"FP32","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":33,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":64,"DataType":"FP32","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":33,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": {}, + "Config": { + "NumWarps": 4, + "SramBytes": 0, + "Tile": [256,128], + "NumTasks": 256 + } + }, + { + "Type": "Mul", + "Name": "mul_4", + "IsVirtual": false, + "ReadTensors": [ + {"Id":64,"DataType":"FP32","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":33,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}}, + {"Id":64,"DataType":"FP32","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":33,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":65,"DataType":"FP32","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":34,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":66,"DataType":"FP32","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":34,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": {}, + "Config": { + "NumWarps": 4, + "SramBytes": 0, + "Tile": [256,128], + "NumTasks": 256 + } + } + ] + }, + { + "Id": 28, + "NumWarps": 1, + "SramBytes": 0, + "Ops": [ + { + "Type": "ReduceMean", + "Name": "reduce_mean_1", + "IsVirtual": false, + "ReadTensors": [ + {"Id":66,"DataType":"FP32","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":34,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":67,"DataType":"FP32","Shape":[1,2048,1],"Strides":[1,2048,1],"Offsets":[0,0,0],"PaddedShape":[1,2048,1],"Buffer":{"Id":35,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":68,"DataType":"FP32","Shape":[1,2048,1],"Strides":[1,2048,1],"Offsets":[0,0,0],"PaddedShape":[1,2048,1],"Buffer":{"Id":35,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": { + "Axis": {"INT":2}, + "KeepDim": {"BOOL":true} + }, + "Config": { + "NumWarps": 1, + "ImplType": "WarpWise", + "SramBytes": 0, + "NumTasks": 2048 + } + } + ] + }, + { + "Id": 29, + "NumWarps": 1, + "SramBytes": 0, + "Ops": [ + { + "Type": "Rsqrt", + "Name": "rsqrt_1", + "IsVirtual": false, + "ReadTensors": [ + {"Id":68,"DataType":"FP32","Shape":[1,2048,1],"Strides":[1,2048,1],"Offsets":[0,0,0],"PaddedShape":[1,2048,1],"Buffer":{"Id":35,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":69,"DataType":"FP32","Shape":[1,2048,1],"Strides":[1,2048,1],"Offsets":[0,0,0],"PaddedShape":[1,2048,1],"Buffer":{"Id":36,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":70,"DataType":"FP32","Shape":[1,2048,1],"Strides":[1,2048,1],"Offsets":[0,0,0],"PaddedShape":[1,2048,1],"Buffer":{"Id":36,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": {}, + "Config": { + "NumWarps": 1, + "SramBytes": 0, + "Tile": [64,1], + "NumTasks": 32 + } + } + ] + }, + { + "Id": 30, + "NumWarps": 1, + "SramBytes": 0, + "Ops": [ + { + "Type": "Mul", + "Name": "mul_5", + "IsVirtual": false, + "ReadTensors": [ + {"Id":64,"DataType":"FP32","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":33,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}}, + {"Id":70,"DataType":"FP32","Shape":[1,2048,1],"Strides":[1,2048,1],"Offsets":[0,0,0],"PaddedShape":[1,2048,1],"Buffer":{"Id":36,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":71,"DataType":"FP32","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":37,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":72,"DataType":"FP32","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":37,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": {}, + "Config": { + "NumWarps": 1, + "SramBytes": 0, + "Tile": [1,4096], + "NumTasks": 2048 + } + }, + { + "Type": "Mul", + "Name": "mul_6", + "IsVirtual": false, + "ReadTensors": [ + {"Id":72,"DataType":"FP32","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":37,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}}, + {"Id":8,"DataType":"FP32","Shape":[1,1,4096],"Strides":[1,1,4096],"Offsets":[0,0,0],"PaddedShape":[1,1,4096],"Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":72,"DataType":"FP32","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":37,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":73,"DataType":"FP32","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":37,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": {}, + "Config": { + "NumWarps": 1, + "SramBytes": 0, + "Tile": [1,4096], + "NumTasks": 2048 + } + }, + { + "Type": "Cast", + "Name": "cast_3", + "IsVirtual": false, + "ReadTensors": [ + {"Id":73,"DataType":"FP32","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":37,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":74,"DataType":"FP16","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":38,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":75,"DataType":"FP16","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":38,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": {}, + "Config": { + "NumWarps": 1, + "SramBytes": 0, + "Tile": [1,4096], + "NumTasks": 2048 + } + } + ] + }, + { + "Id": 33, + "NumWarps": 4, + "SramBytes": 24672, + "Ops": [ + { + "Type": "Matmul", + "Name": "matmul_6", + "IsVirtual": false, + "ReadTensors": [ + {"Id":75,"DataType":"FP16","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":38,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}}, + {"Id":4,"DataType":"FP16","Shape":[11008,4096],"Strides":[11008,4096],"Offsets":[0,0],"PaddedShape":[11008,4096],"Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":76,"DataType":"FP16","Shape":[1,2048,11008],"Strides":[1,2048,11008],"Offsets":[0,0,0],"PaddedShape":[1,2048,11008],"Buffer":{"Id":39,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":77,"DataType":"FP16","Shape":[1,2048,11008],"Strides":[1,2048,11008],"Offsets":[0,0,0],"PaddedShape":[1,2048,11008],"Buffer":{"Id":39,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": { + "TransposeInput": {"BOOL":false}, + "TransposeOther": {"BOOL":true} + }, + "Config": { + "NumWarps": 4, + "SramBytes": 24672, + "TileShapeMNK": [256,128,32], + "NumTasks": 688 + } + }, + { + "Type": "Sigmoid", + "Name": "sigmoid", + "IsVirtual": false, + "ReadTensors": [ + {"Id":77,"DataType":"FP16","Shape":[1,2048,11008],"Strides":[1,2048,11008],"Offsets":[0,0,0],"PaddedShape":[1,2048,11008],"Buffer":{"Id":39,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":78,"DataType":"FP16","Shape":[1,2048,11008],"Strides":[1,2048,11008],"Offsets":[0,0,0],"PaddedShape":[1,2048,11008],"Buffer":{"Id":40,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":79,"DataType":"FP16","Shape":[1,2048,11008],"Strides":[1,2048,11008],"Offsets":[0,0,0],"PaddedShape":[1,2048,11008],"Buffer":{"Id":40,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": {}, + "Config": { + "NumWarps": 4, + "SramBytes": 0, + "Tile": [256,128], + "NumTasks": 688 + } + }, + { + "Type": "Mul", + "Name": "mul_7", + "IsVirtual": false, + "ReadTensors": [ + {"Id":77,"DataType":"FP16","Shape":[1,2048,11008],"Strides":[1,2048,11008],"Offsets":[0,0,0],"PaddedShape":[1,2048,11008],"Buffer":{"Id":39,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}}, + {"Id":79,"DataType":"FP16","Shape":[1,2048,11008],"Strides":[1,2048,11008],"Offsets":[0,0,0],"PaddedShape":[1,2048,11008],"Buffer":{"Id":40,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":80,"DataType":"FP16","Shape":[1,2048,11008],"Strides":[1,2048,11008],"Offsets":[0,0,0],"PaddedShape":[1,2048,11008],"Buffer":{"Id":41,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":81,"DataType":"FP16","Shape":[1,2048,11008],"Strides":[1,2048,11008],"Offsets":[0,0,0],"PaddedShape":[1,2048,11008],"Buffer":{"Id":41,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": {}, + "Config": { + "NumWarps": 4, + "SramBytes": 0, + "Tile": [256,128], + "NumTasks": 688 + } + } + ] + }, + { + "Id": 36, + "NumWarps": 4, + "SramBytes": 24672, + "Ops": [ + { + "Type": "Matmul", + "Name": "matmul_7", + "IsVirtual": false, + "ReadTensors": [ + {"Id":75,"DataType":"FP16","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":38,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}}, + {"Id":6,"DataType":"FP16","Shape":[11008,4096],"Strides":[11008,4096],"Offsets":[0,0],"PaddedShape":[11008,4096],"Buffer":{"Id":6,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":82,"DataType":"FP16","Shape":[1,2048,11008],"Strides":[1,2048,11008],"Offsets":[0,0,0],"PaddedShape":[1,2048,11008],"Buffer":{"Id":42,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":83,"DataType":"FP16","Shape":[1,2048,11008],"Strides":[1,2048,11008],"Offsets":[0,0,0],"PaddedShape":[1,2048,11008],"Buffer":{"Id":42,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": { + "TransposeInput": {"BOOL":false}, + "TransposeOther": {"BOOL":true} + }, + "Config": { + "NumWarps": 4, + "SramBytes": 24672, + "TileShapeMNK": [256,128,32], + "NumTasks": 602 + } + }, + { + "Type": "Mul", + "Name": "mul_8", + "IsVirtual": false, + "ReadTensors": [ + {"Id":81,"DataType":"FP16","Shape":[1,2048,11008],"Strides":[1,2048,11008],"Offsets":[0,0,0],"PaddedShape":[1,2048,11008],"Buffer":{"Id":41,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}}, + {"Id":83,"DataType":"FP16","Shape":[1,2048,11008],"Strides":[1,2048,11008],"Offsets":[0,0,0],"PaddedShape":[1,2048,11008],"Buffer":{"Id":42,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":84,"DataType":"FP16","Shape":[1,2048,11008],"Strides":[1,2048,11008],"Offsets":[0,0,0],"PaddedShape":[1,2048,11008],"Buffer":{"Id":43,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":85,"DataType":"FP16","Shape":[1,2048,11008],"Strides":[1,2048,11008],"Offsets":[0,0,0],"PaddedShape":[1,2048,11008],"Buffer":{"Id":43,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": {}, + "Config": { + "NumWarps": 4, + "SramBytes": 0, + "Tile": [256,128], + "NumTasks": 602 + } + } + ] + }, + { + "Id": 37, + "NumWarps": 4, + "SramBytes": 16480, + "Ops": [ + { + "Type": "Matmul", + "Name": "matmul_7", + "IsVirtual": false, + "ReadTensors": [ + {"Id":102,"DataType":"FP16","Shape":[1,1792,4096],"Strides":[1,2048,4096],"Offsets":[0,256,0],"PaddedShape":[1,1792,4096],"Buffer":{"Id":38,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}}, + {"Id":6,"DataType":"FP16","Shape":[11008,4096],"Strides":[11008,4096],"Offsets":[0,0],"PaddedShape":[11008,4096],"Buffer":{"Id":6,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":101,"DataType":"FP16","Shape":[1,1792,11008],"Strides":[1,2048,11008],"Offsets":[0,256,0],"PaddedShape":[1,1792,11008],"Buffer":{"Id":42,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":100,"DataType":"FP16","Shape":[1,1792,11008],"Strides":[1,2048,11008],"Offsets":[0,256,0],"PaddedShape":[1,1792,11008],"Buffer":{"Id":42,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": { + "TransposeInput": {"BOOL":false}, + "TransposeOther": {"BOOL":true} + }, + "Config": { + "NumWarps": 4, + "SramBytes": 16480, + "TileShapeMNK": [128,128,32], + "NumTasks": 172 + } + }, + { + "Type": "Mul", + "Name": "mul_8", + "IsVirtual": false, + "ReadTensors": [ + {"Id":81,"DataType":"FP16","Shape":[1,1792,11008],"Strides":[1,2048,11008],"Offsets":[0,256,0],"PaddedShape":[1,1792,11008],"Buffer":{"Id":41,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}}, + {"Id":83,"DataType":"FP16","Shape":[1,1792,11008],"Strides":[1,2048,11008],"Offsets":[0,256,0],"PaddedShape":[1,1792,11008],"Buffer":{"Id":42,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":84,"DataType":"FP16","Shape":[1,1792,11008],"Strides":[1,2048,11008],"Offsets":[0,256,0],"PaddedShape":[1,1792,11008],"Buffer":{"Id":43,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":85,"DataType":"FP16","Shape":[1,1792,11008],"Strides":[1,2048,11008],"Offsets":[0,256,0],"PaddedShape":[1,1792,11008],"Buffer":{"Id":43,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": {}, + "Config": { + "NumWarps": 4, + "SramBytes": 0, + "Tile": [128,128], + "NumTasks": 172 + } + } + ] + }, + { + "Id": 38, + "NumWarps": 4, + "SramBytes": 24672, + "Ops": [ + { + "Type": "Matmul", + "Name": "matmul_8", + "IsVirtual": false, + "ReadTensors": [ + {"Id":85,"DataType":"FP16","Shape":[1,2048,11008],"Strides":[1,2048,11008],"Offsets":[0,0,0],"PaddedShape":[1,2048,11008],"Buffer":{"Id":43,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}}, + {"Id":5,"DataType":"FP16","Shape":[4096,11008],"Strides":[4096,11008],"Offsets":[0,0],"PaddedShape":[4096,11008],"Buffer":{"Id":5,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":86,"DataType":"FP16","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":44,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":87,"DataType":"FP16","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":44,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": { + "TransposeInput": {"BOOL":false}, + "TransposeOther": {"BOOL":true} + }, + "Config": { + "NumWarps": 4, + "SramBytes": 24672, + "TileShapeMNK": [256,128,32], + "NumTasks": 256 + } + }, + { + "Type": "Add", + "Name": "add_1", + "IsVirtual": false, + "ReadTensors": [ + {"Id":62,"DataType":"FP16","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":32,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}}, + {"Id":87,"DataType":"FP16","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":44,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "WriteTensors": [ + {"Id":88,"DataType":"FP16","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":45,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "ResultTensors": [ + {"Id":89,"DataType":"FP16","Shape":[1,2048,4096],"Strides":[1,2048,4096],"Offsets":[0,0,0],"PaddedShape":[1,2048,4096],"Buffer":{"Id":45,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} + ], + "Args": {}, + "Config": { + "NumWarps": 4, + "SramBytes": 0, + "Tile": [256,128], + "NumTasks": 256 + } + } + ] + } + ], + "ProcessorGroups": [ + { + "ProcessorRange": [0,304], + "ResourceGroups": [ + { + "ProcessorRange": [0,304], + "WarpRange": [0,8], + "SramRange": [0,0], + "TaskGroups": [ + {"TaskId":0,"TaskRange":[0,2048],"Granularity":7} + ] + } + ] + }, + { + "ProcessorRange": [0,32], + "ResourceGroups": [ + { + "ProcessorRange": [0,32], + "WarpRange": [0,1], + "SramRange": [0,0], + "TaskGroups": [ + {"TaskId":3,"TaskRange":[0,32],"Granularity":1} + ] + } + ] + }, + { + "ProcessorRange": [0,304], + "ResourceGroups": [ + { + "ProcessorRange": [0,304], + "WarpRange": [0,8], + "SramRange": [0,0], + "TaskGroups": [ + {"TaskId":4,"TaskRange":[0,2048],"Granularity":7} + ] + } + ] + }, + { + "ProcessorRange": [0,256], + "ResourceGroups": [ + { + "ProcessorRange": [0,256], + "WarpRange": [0,4], + "SramRange": [0,24672], + "TaskGroups": [ + {"TaskId":7,"TaskRange":[0,256],"Granularity":1} + ] + } + ] + }, + { + "ProcessorRange": [0,256], + "ResourceGroups": [ + { + "ProcessorRange": [0,256], + "WarpRange": [0,4], + "SramRange": [0,24672], + "TaskGroups": [ + {"TaskId":10,"TaskRange":[0,256],"Granularity":1} + ] + } + ] + }, + { + "ProcessorRange": [0,256], + "ResourceGroups": [ + { + "ProcessorRange": [0,256], + "WarpRange": [0,4], + "SramRange": [0,24672], + "TaskGroups": [ + {"TaskId":13,"TaskRange":[0,256],"Granularity":1} + ] + } + ] + }, + { + "ProcessorRange": [0,304], + "ResourceGroups": [ + { + "ProcessorRange": [0,304], + "WarpRange": [0,8], + "SramRange": [0,49344], + "TaskGroups": [ + {"TaskId":15,"TaskRange":[0,4096],"Granularity":2} + ] + } + ] + }, + { + "ProcessorRange": [0,304], + "ResourceGroups": [ + { + "ProcessorRange": [0,304], + "WarpRange": [0,8], + "SramRange": [0,0], + "TaskGroups": [ + {"TaskId":17,"TaskRange":[0,65536],"Granularity":1} + ] + } + ] + }, + { + "ProcessorRange": [0,256], + "ResourceGroups": [ + { + "ProcessorRange": [0,256], + "WarpRange": [0,4], + "SramRange": [0,24672], + "TaskGroups": [ + {"TaskId":22,"TaskRange":[0,256],"Granularity":1} + ] + } + ] + }, + { + "ProcessorRange": [0,256], + "ResourceGroups": [ + { + "ProcessorRange": [0,256], + "WarpRange": [0,4], + "SramRange": [0,24672], + "TaskGroups": [ + {"TaskId":24,"TaskRange":[0,256],"Granularity":1} + ] + } + ] + }, + { + "ProcessorRange": [0,304], + "ResourceGroups": [ + { + "ProcessorRange": [0,304], + "WarpRange": [0,8], + "SramRange": [0,0], + "TaskGroups": [ + {"TaskId":28,"TaskRange":[0,2048],"Granularity":7} + ] + } + ] + }, + { + "ProcessorRange": [0,32], + "ResourceGroups": [ + { + "ProcessorRange": [0,32], + "WarpRange": [0,1], + "SramRange": [0,0], + "TaskGroups": [ + {"TaskId":29,"TaskRange":[0,32],"Granularity":1} + ] + } + ] + }, + { + "ProcessorRange": [0,304], + "ResourceGroups": [ + { + "ProcessorRange": [0,304], + "WarpRange": [0,8], + "SramRange": [0,0], + "TaskGroups": [ + {"TaskId":30,"TaskRange":[0,2048],"Granularity":7} + ] + } + ] + }, + { + "ProcessorRange": [0,304], + "ResourceGroups": [ + { + "ProcessorRange": [0,304], + "WarpRange": [0,8], + "SramRange": [0,49344], + "TaskGroups": [ + {"TaskId":33,"TaskRange":[0,688],"Granularity":1} + ] + } + ] + }, + { + "ProcessorRange": [0,304], + "ResourceGroups": [ + { + "ProcessorRange": [0,304], + "WarpRange": [0,8], + "SramRange": [0,49344], + "TaskGroups": [ + {"TaskId":36,"TaskRange":[0,602],"Granularity":2}, + {"TaskId":37,"TaskRange":[0,172],"Granularity":1} + ] + } + ] + }, + { + "ProcessorRange": [0,256], + "ResourceGroups": [ + { + "ProcessorRange": [0,256], + "WarpRange": [0,4], + "SramRange": [0,24672], + "TaskGroups": [ + {"TaskId":38,"TaskRange":[0,256],"Granularity":1} + ] + } + ] + } + ] +} \ No newline at end of file diff --git a/examples/tutorial/default_plan.json b/examples/tutorial/default_plan.json index c6b4be243..bb774a5b8 100644 --- a/examples/tutorial/default_plan.json +++ b/examples/tutorial/default_plan.json @@ -1,36 +1,37 @@ { "Rank": 0, "WorldSize": 1, - "NumProcessors": 108, - "NumWarpsPerProcessor": 8, + "Architecture": "ROCM_942", + "NumProcessors": 304, + "NumWarpsPerProcessor": 4, "TaskInfos": [ { "Id": 0, - "NumWarps": 8, - "SramBytes": 147456, + "NumWarps": 4, + "SramBytes": 24672, "Ops": [ { "Type": "Matmul", "Name": "matmul", "IsVirtual": false, "ReadTensors": [ - {"Id":0,"DataType":"FP16","Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096],"Buffer":{"Id":0,"Rank":-1,"SendTags":[],"RecvTags":[]}}, - {"Id":1,"DataType":"FP16","Shape":[11008,4096],"Strides":[11008,4096],"Offsets":[0,0],"PaddedShape":[11008,4096],"Buffer":{"Id":1,"Rank":-1,"SendTags":[],"RecvTags":[]}} + {"Id":0,"DataType":"FP16","Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096],"Buffer":{"Id":0,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}}, + {"Id":1,"DataType":"FP16","Shape":[11008,4096],"Strides":[11008,4096],"Offsets":[0,0],"PaddedShape":[11008,4096],"Buffer":{"Id":1,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} ], "WriteTensors": [ - {"Id":4,"DataType":"FP16","Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008],"Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[]}} + {"Id":4,"DataType":"FP16","Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008],"Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} ], "ResultTensors": [ - {"Id":5,"DataType":"FP16","Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008],"Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[]}} + {"Id":5,"DataType":"FP16","Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008],"Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} ], "Args": { "TransposeInput": {"BOOL":false}, "TransposeOther": {"BOOL":true} }, "Config": { - "NumWarps": 8, - "SramBytes": 147456, - "TileShapeMNK": [128,256,64], + "NumWarps": 4, + "SramBytes": 24672, + "TileShapeMNK": [128,256,32], "NumTasks": 172 } } @@ -46,13 +47,13 @@ "Name": "sigmoid", "IsVirtual": false, "ReadTensors": [ - {"Id":5,"DataType":"FP16","Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008],"Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[]}} + {"Id":5,"DataType":"FP16","Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008],"Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} ], "WriteTensors": [ - {"Id":6,"DataType":"FP16","Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008],"Buffer":{"Id":5,"Rank":-1,"SendTags":[],"RecvTags":[]}} + {"Id":6,"DataType":"FP16","Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008],"Buffer":{"Id":5,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} ], "ResultTensors": [ - {"Id":7,"DataType":"FP16","Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008],"Buffer":{"Id":5,"Rank":-1,"SendTags":[],"RecvTags":[]}} + {"Id":7,"DataType":"FP16","Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008],"Buffer":{"Id":5,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} ], "Args": {}, "Config": { @@ -74,14 +75,14 @@ "Name": "mul", "IsVirtual": false, "ReadTensors": [ - {"Id":5,"DataType":"FP16","Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008],"Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[]}}, - {"Id":7,"DataType":"FP16","Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008],"Buffer":{"Id":5,"Rank":-1,"SendTags":[],"RecvTags":[]}} + {"Id":5,"DataType":"FP16","Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008],"Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}}, + {"Id":7,"DataType":"FP16","Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008],"Buffer":{"Id":5,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} ], "WriteTensors": [ - {"Id":8,"DataType":"FP16","Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008],"Buffer":{"Id":6,"Rank":-1,"SendTags":[],"RecvTags":[]}} + {"Id":8,"DataType":"FP16","Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008],"Buffer":{"Id":6,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} ], "ResultTensors": [ - {"Id":9,"DataType":"FP16","Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008],"Buffer":{"Id":6,"Rank":-1,"SendTags":[],"RecvTags":[]}} + {"Id":9,"DataType":"FP16","Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008],"Buffer":{"Id":6,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} ], "Args": {}, "Config": { @@ -95,31 +96,31 @@ }, { "Id": 3, - "NumWarps": 8, - "SramBytes": 147456, + "NumWarps": 4, + "SramBytes": 24672, "Ops": [ { "Type": "Matmul", "Name": "matmul_1", "IsVirtual": false, "ReadTensors": [ - {"Id":0,"DataType":"FP16","Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096],"Buffer":{"Id":0,"Rank":-1,"SendTags":[],"RecvTags":[]}}, - {"Id":3,"DataType":"FP16","Shape":[11008,4096],"Strides":[11008,4096],"Offsets":[0,0],"PaddedShape":[11008,4096],"Buffer":{"Id":3,"Rank":-1,"SendTags":[],"RecvTags":[]}} + {"Id":0,"DataType":"FP16","Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096],"Buffer":{"Id":0,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}}, + {"Id":3,"DataType":"FP16","Shape":[11008,4096],"Strides":[11008,4096],"Offsets":[0,0],"PaddedShape":[11008,4096],"Buffer":{"Id":3,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} ], "WriteTensors": [ - {"Id":10,"DataType":"FP16","Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008],"Buffer":{"Id":7,"Rank":-1,"SendTags":[],"RecvTags":[]}} + {"Id":10,"DataType":"FP16","Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008],"Buffer":{"Id":7,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} ], "ResultTensors": [ - {"Id":11,"DataType":"FP16","Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008],"Buffer":{"Id":7,"Rank":-1,"SendTags":[],"RecvTags":[]}} + {"Id":11,"DataType":"FP16","Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008],"Buffer":{"Id":7,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} ], "Args": { "TransposeInput": {"BOOL":false}, "TransposeOther": {"BOOL":true} }, "Config": { - "NumWarps": 8, - "SramBytes": 147456, - "TileShapeMNK": [128,256,64], + "NumWarps": 4, + "SramBytes": 24672, + "TileShapeMNK": [128,256,32], "NumTasks": 172 } } @@ -135,14 +136,14 @@ "Name": "mul_1", "IsVirtual": false, "ReadTensors": [ - {"Id":9,"DataType":"FP16","Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008],"Buffer":{"Id":6,"Rank":-1,"SendTags":[],"RecvTags":[]}}, - {"Id":11,"DataType":"FP16","Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008],"Buffer":{"Id":7,"Rank":-1,"SendTags":[],"RecvTags":[]}} + {"Id":9,"DataType":"FP16","Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008],"Buffer":{"Id":6,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}}, + {"Id":11,"DataType":"FP16","Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008],"Buffer":{"Id":7,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} ], "WriteTensors": [ - {"Id":12,"DataType":"FP16","Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008],"Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[]}} + {"Id":12,"DataType":"FP16","Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008],"Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} ], "ResultTensors": [ - {"Id":13,"DataType":"FP16","Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008],"Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[]}} + {"Id":13,"DataType":"FP16","Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008],"Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} ], "Args": {}, "Config": { @@ -156,31 +157,31 @@ }, { "Id": 5, - "NumWarps": 8, - "SramBytes": 147456, + "NumWarps": 4, + "SramBytes": 24672, "Ops": [ { "Type": "Matmul", "Name": "matmul_2", "IsVirtual": false, "ReadTensors": [ - {"Id":13,"DataType":"FP16","Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008],"Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[]}}, - {"Id":2,"DataType":"FP16","Shape":[4096,11008],"Strides":[4096,11008],"Offsets":[0,0],"PaddedShape":[4096,11008],"Buffer":{"Id":2,"Rank":-1,"SendTags":[],"RecvTags":[]}} + {"Id":13,"DataType":"FP16","Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008],"Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}}, + {"Id":2,"DataType":"FP16","Shape":[4096,11008],"Strides":[4096,11008],"Offsets":[0,0],"PaddedShape":[4096,11008],"Buffer":{"Id":2,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} ], "WriteTensors": [ - {"Id":14,"DataType":"FP16","Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096],"Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[]}} + {"Id":14,"DataType":"FP16","Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096],"Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} ], "ResultTensors": [ - {"Id":15,"DataType":"FP16","Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096],"Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[]}} + {"Id":15,"DataType":"FP16","Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096],"Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false}} ], "Args": { "TransposeInput": {"BOOL":false}, "TransposeOther": {"BOOL":true} }, "Config": { - "NumWarps": 8, - "SramBytes": 147456, - "TileShapeMNK": [128,256,64], + "NumWarps": 4, + "SramBytes": 24672, + "TileShapeMNK": [128,256,32], "NumTasks": 64 } } @@ -189,12 +190,12 @@ ], "ProcessorGroups": [ { - "ProcessorRange": [0,108], + "ProcessorRange": [0,172], "ResourceGroups": [ { - "ProcessorRange": [0,108], - "WarpRange": [0,8], - "SramRange": [0,147456], + "ProcessorRange": [0,172], + "WarpRange": [0,4], + "SramRange": [0,24672], "TaskGroups": [ {"TaskId":0,"TaskRange":[0,172],"Granularity":1} ] @@ -202,10 +203,10 @@ ] }, { - "ProcessorRange": [0,108], + "ProcessorRange": [0,304], "ResourceGroups": [ { - "ProcessorRange": [0,108], + "ProcessorRange": [0,304], "WarpRange": [0,1], "SramRange": [0,0], "TaskGroups": [ @@ -215,10 +216,10 @@ ] }, { - "ProcessorRange": [0,108], + "ProcessorRange": [0,304], "ResourceGroups": [ { - "ProcessorRange": [0,108], + "ProcessorRange": [0,304], "WarpRange": [0,1], "SramRange": [0,0], "TaskGroups": [ @@ -228,12 +229,12 @@ ] }, { - "ProcessorRange": [0,108], + "ProcessorRange": [0,172], "ResourceGroups": [ { - "ProcessorRange": [0,108], - "WarpRange": [0,8], - "SramRange": [0,147456], + "ProcessorRange": [0,172], + "WarpRange": [0,4], + "SramRange": [0,24672], "TaskGroups": [ {"TaskId":3,"TaskRange":[0,172],"Granularity":1} ] @@ -241,10 +242,10 @@ ] }, { - "ProcessorRange": [0,108], + "ProcessorRange": [0,304], "ResourceGroups": [ { - "ProcessorRange": [0,108], + "ProcessorRange": [0,304], "WarpRange": [0,1], "SramRange": [0,0], "TaskGroups": [ @@ -258,8 +259,8 @@ "ResourceGroups": [ { "ProcessorRange": [0,64], - "WarpRange": [0,8], - "SramRange": [0,147456], + "WarpRange": [0,4], + "SramRange": [0,24672], "TaskGroups": [ {"TaskId":5,"TaskRange":[0,64],"Granularity":1} ] @@ -267,4 +268,4 @@ ] } ] -} +} \ No newline at end of file diff --git a/examples/tutorial/plan.json b/examples/tutorial/plan.json index c0854e505..335c27549 100644 --- a/examples/tutorial/plan.json +++ b/examples/tutorial/plan.json @@ -1,6 +1,7 @@ { "Rank": 0, "WorldSize": 1, + "Architecture": "CUDA_80", "NumProcessors": 108, "NumWarpsPerProcessor": 8, "TaskInfos": [ @@ -14,14 +15,14 @@ "Name": "matmul", "IsVirtual": false, "ReadTensors": [ - {"Id":0,"DataType":"FP16","Buffer":{"Id":0,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]}, - {"Id":1,"DataType":"FP16","Buffer":{"Id":1,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[11008,4096],"Strides":[11008,4096],"Offsets":[0,0],"PaddedShape":[11008,4096]} + {"Id":0,"DataType":"FP16","Buffer":{"Id":0,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]}, + {"Id":1,"DataType":"FP16","Buffer":{"Id":1,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[11008,4096],"Strides":[11008,4096],"Offsets":[0,0],"PaddedShape":[11008,4096]} ], "WriteTensors": [ - {"Id":4,"DataType":"FP16","Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":4,"DataType":"FP16","Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "ResultTensors": [ - {"Id":5,"DataType":"FP16","Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":5,"DataType":"FP16","Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "Args": { "TransposeInput": {"BOOL":false}, @@ -46,13 +47,13 @@ "Name": "sigmoid", "IsVirtual": false, "ReadTensors": [ - {"Id":5,"DataType":"FP16","Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":5,"DataType":"FP16","Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "WriteTensors": [ - {"Id":6,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":6,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "ResultTensors": [ - {"Id":7,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":7,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "Args": {}, "Config": { @@ -74,14 +75,14 @@ "Name": "mul", "IsVirtual": false, "ReadTensors": [ - {"Id":5,"DataType":"FP16","Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]}, - {"Id":7,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":5,"DataType":"FP16","Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]}, + {"Id":7,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "WriteTensors": [ - {"Id":8,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":8,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "ResultTensors": [ - {"Id":9,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":9,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "Args": {}, "Config": { @@ -103,14 +104,14 @@ "Name": "matmul_1", "IsVirtual": false, "ReadTensors": [ - {"Id":0,"DataType":"FP16","Buffer":{"Id":0,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]}, - {"Id":3,"DataType":"FP16","Buffer":{"Id":3,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[11008,4096],"Strides":[11008,4096],"Offsets":[0,0],"PaddedShape":[11008,4096]} + {"Id":0,"DataType":"FP16","Buffer":{"Id":0,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]}, + {"Id":3,"DataType":"FP16","Buffer":{"Id":3,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[11008,4096],"Strides":[11008,4096],"Offsets":[0,0],"PaddedShape":[11008,4096]} ], "WriteTensors": [ - {"Id":10,"DataType":"FP16","Buffer":{"Id":7,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":10,"DataType":"FP16","Buffer":{"Id":7,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "ResultTensors": [ - {"Id":11,"DataType":"FP16","Buffer":{"Id":7,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":11,"DataType":"FP16","Buffer":{"Id":7,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "Args": { "TransposeInput": {"BOOL":false}, @@ -135,14 +136,14 @@ "Name": "mul_1", "IsVirtual": false, "ReadTensors": [ - {"Id":9,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]}, - {"Id":11,"DataType":"FP16","Buffer":{"Id":7,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":9,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]}, + {"Id":11,"DataType":"FP16","Buffer":{"Id":7,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "WriteTensors": [ - {"Id":12,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":12,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "ResultTensors": [ - {"Id":13,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":13,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "Args": {}, "Config": { @@ -164,14 +165,14 @@ "Name": "matmul_1", "IsVirtual": false, "ReadTensors": [ - {"Id":16,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,8320],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,8320]}, - {"Id":17,"DataType":"FP16","Buffer":{"Id":2,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[4096,8320],"Strides":[4096,11008],"Offsets":[0,0],"PaddedShape":[4096,8320]} + {"Id":16,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,8320],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,8320]}, + {"Id":17,"DataType":"FP16","Buffer":{"Id":2,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[4096,8320],"Strides":[4096,11008],"Offsets":[0,0],"PaddedShape":[4096,8320]} ], "WriteTensors": [ - {"Id":14,"DataType":"FP16","Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} + {"Id":14,"DataType":"FP16","Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} ], "ResultTensors": [ - {"Id":22,"DataType":"FP16","Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} + {"Id":22,"DataType":"FP16","Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} ], "Args": { "TransposeInput": {"BOOL":false}, @@ -196,14 +197,14 @@ "Name": "matmul_1", "IsVirtual": false, "ReadTensors": [ - {"Id":18,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,2688],"Strides":[1,512,11008],"Offsets":[0,0,8320],"PaddedShape":[1,512,2688]}, - {"Id":19,"DataType":"FP16","Buffer":{"Id":2,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[4096,2688],"Strides":[4096,11008],"Offsets":[0,8320],"PaddedShape":[4096,2688]} + {"Id":18,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,2688],"Strides":[1,512,11008],"Offsets":[0,0,8320],"PaddedShape":[1,512,2688]}, + {"Id":19,"DataType":"FP16","Buffer":{"Id":2,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[4096,2688],"Strides":[4096,11008],"Offsets":[0,8320],"PaddedShape":[4096,2688]} ], "WriteTensors": [ - {"Id":20,"DataType":"FP16","Buffer":{"Id":10,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} + {"Id":20,"DataType":"FP16","Buffer":{"Id":10,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} ], "ResultTensors": [ - {"Id":21,"DataType":"FP16","Buffer":{"Id":10,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} + {"Id":21,"DataType":"FP16","Buffer":{"Id":10,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} ], "Args": { "TransposeInput": {"BOOL":false}, @@ -228,14 +229,14 @@ "Name": "add_1", "IsVirtual": false, "ReadTensors": [ - {"Id":22,"DataType":"FP16","Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]}, - {"Id":21,"DataType":"FP16","Buffer":{"Id":10,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} + {"Id":22,"DataType":"FP16","Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]}, + {"Id":21,"DataType":"FP16","Buffer":{"Id":10,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} ], "WriteTensors": [ - {"Id":23,"DataType":"FP16","Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} + {"Id":23,"DataType":"FP16","Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} ], "ResultTensors": [ - {"Id":15,"DataType":"FP16","Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} + {"Id":15,"DataType":"FP16","Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} ], "Args": {}, "Config": { diff --git a/examples/tutorial/plan_1_larger_tile.json b/examples/tutorial/plan_1_larger_tile.json index 3a3f66530..04d2e9d60 100644 --- a/examples/tutorial/plan_1_larger_tile.json +++ b/examples/tutorial/plan_1_larger_tile.json @@ -1,6 +1,7 @@ { "Rank": 0, "WorldSize": 1, + "Architecture": "CUDA_80", "NumProcessors": 108, "NumWarpsPerProcessor": 8, "TaskInfos": [ @@ -14,14 +15,14 @@ "Name": "matmul", "IsVirtual": false, "ReadTensors": [ - {"Id":0,"DataType":"FP16","Buffer":{"Id":0,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]}, - {"Id":1,"DataType":"FP16","Buffer":{"Id":1,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[11008,4096],"Strides":[11008,4096],"Offsets":[0,0],"PaddedShape":[11008,4096]} + {"Id":0,"DataType":"FP16","Buffer":{"Id":0,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]}, + {"Id":1,"DataType":"FP16","Buffer":{"Id":1,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[11008,4096],"Strides":[11008,4096],"Offsets":[0,0],"PaddedShape":[11008,4096]} ], "WriteTensors": [ - {"Id":4,"DataType":"FP16","Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":4,"DataType":"FP16","Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "ResultTensors": [ - {"Id":5,"DataType":"FP16","Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":5,"DataType":"FP16","Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "Args": { "TransposeInput": {"BOOL":false}, @@ -46,13 +47,13 @@ "Name": "sigmoid", "IsVirtual": false, "ReadTensors": [ - {"Id":5,"DataType":"FP16","Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":5,"DataType":"FP16","Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "WriteTensors": [ - {"Id":6,"DataType":"FP16","Buffer":{"Id":5,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":6,"DataType":"FP16","Buffer":{"Id":5,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "ResultTensors": [ - {"Id":7,"DataType":"FP16","Buffer":{"Id":5,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":7,"DataType":"FP16","Buffer":{"Id":5,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "Args": {}, "Config": { @@ -74,14 +75,14 @@ "Name": "mul", "IsVirtual": false, "ReadTensors": [ - {"Id":5,"DataType":"FP16","Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]}, - {"Id":7,"DataType":"FP16","Buffer":{"Id":5,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":5,"DataType":"FP16","Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]}, + {"Id":7,"DataType":"FP16","Buffer":{"Id":5,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "WriteTensors": [ - {"Id":8,"DataType":"FP16","Buffer":{"Id":6,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":8,"DataType":"FP16","Buffer":{"Id":6,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "ResultTensors": [ - {"Id":9,"DataType":"FP16","Buffer":{"Id":6,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":9,"DataType":"FP16","Buffer":{"Id":6,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "Args": {}, "Config": { @@ -103,14 +104,14 @@ "Name": "matmul_1", "IsVirtual": false, "ReadTensors": [ - {"Id":0,"DataType":"FP16","Buffer":{"Id":0,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]}, - {"Id":3,"DataType":"FP16","Buffer":{"Id":3,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[11008,4096],"Strides":[11008,4096],"Offsets":[0,0],"PaddedShape":[11008,4096]} + {"Id":0,"DataType":"FP16","Buffer":{"Id":0,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]}, + {"Id":3,"DataType":"FP16","Buffer":{"Id":3,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[11008,4096],"Strides":[11008,4096],"Offsets":[0,0],"PaddedShape":[11008,4096]} ], "WriteTensors": [ - {"Id":10,"DataType":"FP16","Buffer":{"Id":7,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":10,"DataType":"FP16","Buffer":{"Id":7,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "ResultTensors": [ - {"Id":11,"DataType":"FP16","Buffer":{"Id":7,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":11,"DataType":"FP16","Buffer":{"Id":7,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "Args": { "TransposeInput": {"BOOL":false}, @@ -135,14 +136,14 @@ "Name": "mul_1", "IsVirtual": false, "ReadTensors": [ - {"Id":9,"DataType":"FP16","Buffer":{"Id":6,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]}, - {"Id":11,"DataType":"FP16","Buffer":{"Id":7,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":9,"DataType":"FP16","Buffer":{"Id":6,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]}, + {"Id":11,"DataType":"FP16","Buffer":{"Id":7,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "WriteTensors": [ - {"Id":12,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":12,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "ResultTensors": [ - {"Id":13,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":13,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "Args": {}, "Config": { @@ -164,14 +165,14 @@ "Name": "matmul_1", "IsVirtual": false, "ReadTensors": [ - {"Id":13,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]}, - {"Id":2,"DataType":"FP16","Buffer":{"Id":2,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[4096,11008],"Strides":[4096,11008],"Offsets":[0,0],"PaddedShape":[4096,11008]} + {"Id":13,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]}, + {"Id":2,"DataType":"FP16","Buffer":{"Id":2,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[4096,11008],"Strides":[4096,11008],"Offsets":[0,0],"PaddedShape":[4096,11008]} ], "WriteTensors": [ - {"Id":14,"DataType":"FP16","Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} + {"Id":14,"DataType":"FP16","Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} ], "ResultTensors": [ - {"Id":15,"DataType":"FP16","Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} + {"Id":15,"DataType":"FP16","Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} ], "Args": { "TransposeInput": {"BOOL":false}, diff --git a/examples/tutorial/plan_2_split_k.json b/examples/tutorial/plan_2_split_k.json index 493515d8c..837944171 100644 --- a/examples/tutorial/plan_2_split_k.json +++ b/examples/tutorial/plan_2_split_k.json @@ -1,6 +1,7 @@ { "Rank": 0, "WorldSize": 1, + "Architecture": "CUDA_80", "NumProcessors": 108, "NumWarpsPerProcessor": 8, "TaskInfos": [ @@ -14,14 +15,14 @@ "Name": "matmul", "IsVirtual": false, "ReadTensors": [ - {"Id":0,"DataType":"FP16","Buffer":{"Id":0,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]}, - {"Id":1,"DataType":"FP16","Buffer":{"Id":1,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[11008,4096],"Strides":[11008,4096],"Offsets":[0,0],"PaddedShape":[11008,4096]} + {"Id":0,"DataType":"FP16","Buffer":{"Id":0,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]}, + {"Id":1,"DataType":"FP16","Buffer":{"Id":1,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[11008,4096],"Strides":[11008,4096],"Offsets":[0,0],"PaddedShape":[11008,4096]} ], "WriteTensors": [ - {"Id":4,"DataType":"FP16","Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":4,"DataType":"FP16","Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "ResultTensors": [ - {"Id":5,"DataType":"FP16","Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":5,"DataType":"FP16","Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "Args": { "TransposeInput": {"BOOL":false}, @@ -46,13 +47,13 @@ "Name": "sigmoid", "IsVirtual": false, "ReadTensors": [ - {"Id":5,"DataType":"FP16","Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":5,"DataType":"FP16","Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "WriteTensors": [ - {"Id":6,"DataType":"FP16","Buffer":{"Id":5,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":6,"DataType":"FP16","Buffer":{"Id":5,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "ResultTensors": [ - {"Id":7,"DataType":"FP16","Buffer":{"Id":5,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":7,"DataType":"FP16","Buffer":{"Id":5,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "Args": {}, "Config": { @@ -74,14 +75,14 @@ "Name": "mul", "IsVirtual": false, "ReadTensors": [ - {"Id":5,"DataType":"FP16","Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]}, - {"Id":7,"DataType":"FP16","Buffer":{"Id":5,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":5,"DataType":"FP16","Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]}, + {"Id":7,"DataType":"FP16","Buffer":{"Id":5,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "WriteTensors": [ - {"Id":8,"DataType":"FP16","Buffer":{"Id":6,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":8,"DataType":"FP16","Buffer":{"Id":6,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "ResultTensors": [ - {"Id":9,"DataType":"FP16","Buffer":{"Id":6,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":9,"DataType":"FP16","Buffer":{"Id":6,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "Args": {}, "Config": { @@ -103,14 +104,14 @@ "Name": "matmul_1", "IsVirtual": false, "ReadTensors": [ - {"Id":0,"DataType":"FP16","Buffer":{"Id":0,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]}, - {"Id":3,"DataType":"FP16","Buffer":{"Id":3,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[11008,4096],"Strides":[11008,4096],"Offsets":[0,0],"PaddedShape":[11008,4096]} + {"Id":0,"DataType":"FP16","Buffer":{"Id":0,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]}, + {"Id":3,"DataType":"FP16","Buffer":{"Id":3,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[11008,4096],"Strides":[11008,4096],"Offsets":[0,0],"PaddedShape":[11008,4096]} ], "WriteTensors": [ - {"Id":10,"DataType":"FP16","Buffer":{"Id":7,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":10,"DataType":"FP16","Buffer":{"Id":7,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "ResultTensors": [ - {"Id":11,"DataType":"FP16","Buffer":{"Id":7,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":11,"DataType":"FP16","Buffer":{"Id":7,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "Args": { "TransposeInput": {"BOOL":false}, @@ -135,14 +136,14 @@ "Name": "mul_1", "IsVirtual": false, "ReadTensors": [ - {"Id":9,"DataType":"FP16","Buffer":{"Id":6,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]}, - {"Id":11,"DataType":"FP16","Buffer":{"Id":7,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":9,"DataType":"FP16","Buffer":{"Id":6,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]}, + {"Id":11,"DataType":"FP16","Buffer":{"Id":7,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "WriteTensors": [ - {"Id":12,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":12,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "ResultTensors": [ - {"Id":13,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":13,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "Args": {}, "Config": { @@ -164,14 +165,14 @@ "Name": "matmul_1", "IsVirtual": false, "ReadTensors": [ - {"Id":16,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,8320],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,8320]}, - {"Id":17,"DataType":"FP16","Buffer":{"Id":2,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[4096,8320],"Strides":[4096,11008],"Offsets":[0,0],"PaddedShape":[4096,8320]} + {"Id":16,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,8320],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,8320]}, + {"Id":17,"DataType":"FP16","Buffer":{"Id":2,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[4096,8320],"Strides":[4096,11008],"Offsets":[0,0],"PaddedShape":[4096,8320]} ], "WriteTensors": [ - {"Id":14,"DataType":"FP16","Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} + {"Id":14,"DataType":"FP16","Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} ], "ResultTensors": [ - {"Id":22,"DataType":"FP16","Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} + {"Id":22,"DataType":"FP16","Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} ], "Args": { "TransposeInput": {"BOOL":false}, @@ -196,14 +197,14 @@ "Name": "matmul_1", "IsVirtual": false, "ReadTensors": [ - {"Id":18,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,2688],"Strides":[1,512,11008],"Offsets":[0,0,8320],"PaddedShape":[1,512,2688]}, - {"Id":19,"DataType":"FP16","Buffer":{"Id":2,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[4096,2688],"Strides":[4096,11008],"Offsets":[0,8320],"PaddedShape":[4096,2688]} + {"Id":18,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,2688],"Strides":[1,512,11008],"Offsets":[0,0,8320],"PaddedShape":[1,512,2688]}, + {"Id":19,"DataType":"FP16","Buffer":{"Id":2,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[4096,2688],"Strides":[4096,11008],"Offsets":[0,8320],"PaddedShape":[4096,2688]} ], "WriteTensors": [ - {"Id":20,"DataType":"FP16","Buffer":{"Id":10,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} + {"Id":20,"DataType":"FP16","Buffer":{"Id":10,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} ], "ResultTensors": [ - {"Id":21,"DataType":"FP16","Buffer":{"Id":10,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} + {"Id":21,"DataType":"FP16","Buffer":{"Id":10,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} ], "Args": { "TransposeInput": {"BOOL":false}, @@ -228,14 +229,14 @@ "Name": "add_1", "IsVirtual": false, "ReadTensors": [ - {"Id":22,"DataType":"FP16","Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]}, - {"Id":21,"DataType":"FP16","Buffer":{"Id":10,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} + {"Id":22,"DataType":"FP16","Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]}, + {"Id":21,"DataType":"FP16","Buffer":{"Id":10,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} ], "WriteTensors": [ - {"Id":23,"DataType":"FP16","Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} + {"Id":23,"DataType":"FP16","Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} ], "ResultTensors": [ - {"Id":15,"DataType":"FP16","Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} + {"Id":15,"DataType":"FP16","Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} ], "Args": {}, "Config": { diff --git a/examples/tutorial/plan_3_overwrite.json b/examples/tutorial/plan_3_overwrite.json index c0854e505..335c27549 100644 --- a/examples/tutorial/plan_3_overwrite.json +++ b/examples/tutorial/plan_3_overwrite.json @@ -1,6 +1,7 @@ { "Rank": 0, "WorldSize": 1, + "Architecture": "CUDA_80", "NumProcessors": 108, "NumWarpsPerProcessor": 8, "TaskInfos": [ @@ -14,14 +15,14 @@ "Name": "matmul", "IsVirtual": false, "ReadTensors": [ - {"Id":0,"DataType":"FP16","Buffer":{"Id":0,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]}, - {"Id":1,"DataType":"FP16","Buffer":{"Id":1,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[11008,4096],"Strides":[11008,4096],"Offsets":[0,0],"PaddedShape":[11008,4096]} + {"Id":0,"DataType":"FP16","Buffer":{"Id":0,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]}, + {"Id":1,"DataType":"FP16","Buffer":{"Id":1,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[11008,4096],"Strides":[11008,4096],"Offsets":[0,0],"PaddedShape":[11008,4096]} ], "WriteTensors": [ - {"Id":4,"DataType":"FP16","Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":4,"DataType":"FP16","Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "ResultTensors": [ - {"Id":5,"DataType":"FP16","Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":5,"DataType":"FP16","Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "Args": { "TransposeInput": {"BOOL":false}, @@ -46,13 +47,13 @@ "Name": "sigmoid", "IsVirtual": false, "ReadTensors": [ - {"Id":5,"DataType":"FP16","Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":5,"DataType":"FP16","Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "WriteTensors": [ - {"Id":6,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":6,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "ResultTensors": [ - {"Id":7,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":7,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "Args": {}, "Config": { @@ -74,14 +75,14 @@ "Name": "mul", "IsVirtual": false, "ReadTensors": [ - {"Id":5,"DataType":"FP16","Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]}, - {"Id":7,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":5,"DataType":"FP16","Buffer":{"Id":4,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]}, + {"Id":7,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "WriteTensors": [ - {"Id":8,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":8,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "ResultTensors": [ - {"Id":9,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":9,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "Args": {}, "Config": { @@ -103,14 +104,14 @@ "Name": "matmul_1", "IsVirtual": false, "ReadTensors": [ - {"Id":0,"DataType":"FP16","Buffer":{"Id":0,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]}, - {"Id":3,"DataType":"FP16","Buffer":{"Id":3,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[11008,4096],"Strides":[11008,4096],"Offsets":[0,0],"PaddedShape":[11008,4096]} + {"Id":0,"DataType":"FP16","Buffer":{"Id":0,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]}, + {"Id":3,"DataType":"FP16","Buffer":{"Id":3,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[11008,4096],"Strides":[11008,4096],"Offsets":[0,0],"PaddedShape":[11008,4096]} ], "WriteTensors": [ - {"Id":10,"DataType":"FP16","Buffer":{"Id":7,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":10,"DataType":"FP16","Buffer":{"Id":7,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "ResultTensors": [ - {"Id":11,"DataType":"FP16","Buffer":{"Id":7,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":11,"DataType":"FP16","Buffer":{"Id":7,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "Args": { "TransposeInput": {"BOOL":false}, @@ -135,14 +136,14 @@ "Name": "mul_1", "IsVirtual": false, "ReadTensors": [ - {"Id":9,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]}, - {"Id":11,"DataType":"FP16","Buffer":{"Id":7,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":9,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]}, + {"Id":11,"DataType":"FP16","Buffer":{"Id":7,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "WriteTensors": [ - {"Id":12,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":12,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "ResultTensors": [ - {"Id":13,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} + {"Id":13,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,11008],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,11008]} ], "Args": {}, "Config": { @@ -164,14 +165,14 @@ "Name": "matmul_1", "IsVirtual": false, "ReadTensors": [ - {"Id":16,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,8320],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,8320]}, - {"Id":17,"DataType":"FP16","Buffer":{"Id":2,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[4096,8320],"Strides":[4096,11008],"Offsets":[0,0],"PaddedShape":[4096,8320]} + {"Id":16,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,8320],"Strides":[1,512,11008],"Offsets":[0,0,0],"PaddedShape":[1,512,8320]}, + {"Id":17,"DataType":"FP16","Buffer":{"Id":2,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[4096,8320],"Strides":[4096,11008],"Offsets":[0,0],"PaddedShape":[4096,8320]} ], "WriteTensors": [ - {"Id":14,"DataType":"FP16","Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} + {"Id":14,"DataType":"FP16","Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} ], "ResultTensors": [ - {"Id":22,"DataType":"FP16","Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} + {"Id":22,"DataType":"FP16","Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} ], "Args": { "TransposeInput": {"BOOL":false}, @@ -196,14 +197,14 @@ "Name": "matmul_1", "IsVirtual": false, "ReadTensors": [ - {"Id":18,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,2688],"Strides":[1,512,11008],"Offsets":[0,0,8320],"PaddedShape":[1,512,2688]}, - {"Id":19,"DataType":"FP16","Buffer":{"Id":2,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[4096,2688],"Strides":[4096,11008],"Offsets":[0,8320],"PaddedShape":[4096,2688]} + {"Id":18,"DataType":"FP16","Buffer":{"Id":8,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,2688],"Strides":[1,512,11008],"Offsets":[0,0,8320],"PaddedShape":[1,512,2688]}, + {"Id":19,"DataType":"FP16","Buffer":{"Id":2,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[4096,2688],"Strides":[4096,11008],"Offsets":[0,8320],"PaddedShape":[4096,2688]} ], "WriteTensors": [ - {"Id":20,"DataType":"FP16","Buffer":{"Id":10,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} + {"Id":20,"DataType":"FP16","Buffer":{"Id":10,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} ], "ResultTensors": [ - {"Id":21,"DataType":"FP16","Buffer":{"Id":10,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} + {"Id":21,"DataType":"FP16","Buffer":{"Id":10,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} ], "Args": { "TransposeInput": {"BOOL":false}, @@ -228,14 +229,14 @@ "Name": "add_1", "IsVirtual": false, "ReadTensors": [ - {"Id":22,"DataType":"FP16","Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]}, - {"Id":21,"DataType":"FP16","Buffer":{"Id":10,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} + {"Id":22,"DataType":"FP16","Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]}, + {"Id":21,"DataType":"FP16","Buffer":{"Id":10,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} ], "WriteTensors": [ - {"Id":23,"DataType":"FP16","Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} + {"Id":23,"DataType":"FP16","Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} ], "ResultTensors": [ - {"Id":15,"DataType":"FP16","Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[]},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} + {"Id":15,"DataType":"FP16","Buffer":{"Id":9,"Rank":-1,"SendTags":[],"RecvTags":[],"IsExternal":false},"Shape":[1,512,4096],"Strides":[1,512,4096],"Offsets":[0,0,0],"PaddedShape":[1,512,4096]} ], "Args": {}, "Config": { diff --git a/examples/tutorial/plan_tutorial.py b/examples/tutorial/plan_tutorial.py index 560021522..a2c5e3e57 100644 --- a/examples/tutorial/plan_tutorial.py +++ b/examples/tutorial/plan_tutorial.py @@ -339,7 +339,7 @@ def main(plan_path: str): plan = planner.plan() with open("default_plan.json", "w") as f: - f.write(plan) + f.write(str(plan)) rt.launch(plan=plan) # Initialize @@ -364,7 +364,7 @@ def main(plan_path: str): print(f"File {plan_path} does not exist. Exiting...") return with ark.Runtime.get_runtime() as rt: - rt.launch(plan_path=plan_path) + rt.launch(plan=ark.Plan.from_file(plan_path)) # Initialize InputModule.initialize() diff --git a/examples/tutorial/torch_tutorial.py b/examples/tutorial/torch_tutorial.py new file mode 100644 index 000000000..e9482a7cc --- /dev/null +++ b/examples/tutorial/torch_tutorial.py @@ -0,0 +1,27 @@ +# Copyright (c) Microsoft Corporation. +# Licensed under the MIT license. + +import ark +import torch + + +class ArkAddModule(ark.RuntimeModule): + def build_forward(self, x: ark.Tensor, y: ark.Tensor) -> ark.Tensor: + return ark.add(x, y) + + +# ARK module for addition +module = ArkAddModule() + +# Define two torch arrays +x = torch.ones(64) * 2 +y = torch.ones(64) * 3 + +# Run the ARK module +z = module(x, y) + +w = module(x, z) + +# Print the result +print(z) # 5 +print(w) # 7 diff --git a/python/CMakeLists.txt b/python/CMakeLists.txt index efb9aea3e..bd25d01e6 100644 --- a/python/CMakeLists.txt +++ b/python/CMakeLists.txt @@ -20,3 +20,4 @@ file(GLOB_RECURSE BIND_SOURCES CONFIGURE_DEPENDS ${CMAKE_CURRENT_SOURCE_DIR}/*.c pybind11_add_module(ark_py ${BIND_SOURCES}) set_target_properties(ark_py PROPERTIES OUTPUT_NAME _ark_core) target_link_libraries(ark_py PRIVATE ark_static) +target_include_directories(ark_py SYSTEM PRIVATE ${DLPACK_INCLUDE_DIRS}) diff --git a/python/ark/__init__.py b/python/ark/__init__.py index b1d0f7873..c20b50b8c 100644 --- a/python/ark/__init__.py +++ b/python/ark/__init__.py @@ -1,15 +1,12 @@ # Copyright (c) Microsoft Corporation. # Licensed under the MIT license. -import sys import os if os.environ.get("ARK_ROOT", None) is None: os.environ["ARK_ROOT"] = os.path.abspath(os.path.dirname(__file__)) -sys.path.insert(0, os.path.dirname(os.path.abspath(__file__))) - -import _ark_core +from . import _ark_core from .model import Model @@ -38,7 +35,7 @@ def set_world_size(world_size): from .init import init from .tensor import Dims, Tensor, Parameter -from .module import Module +from .module import Module, RuntimeModule from .runtime import Runtime from .serialize import save, load from .data_type import ( @@ -51,6 +48,7 @@ def set_world_size(world_size): uint8, byte, ) +from .profiler import Profiler from .ops import * from .planner import * from .error import * diff --git a/python/ark/data_type.py b/python/ark/data_type.py index fe95d0d88..8ab982106 100644 --- a/python/ark/data_type.py +++ b/python/ark/data_type.py @@ -4,16 +4,20 @@ import numpy from . import _ark_core +try: + import torch +except ImportError: + from . import torch_mock as torch _REGISTRY_DATA_TYPE = { - "fp32": {"np": numpy.float32}, - "fp16": {"np": numpy.float16}, - "bf16": {"np": None}, - "int32": {"np": numpy.int32}, - "uint32": {"np": numpy.uint32}, - "int8": {"np": numpy.int8}, - "uint8": {"np": numpy.uint8}, - "byte": {"np": numpy.ubyte}, + "fp32": {"np": numpy.float32, "torch": torch.float32}, + "fp16": {"np": numpy.float16, "torch": torch.float16}, + "bf16": {"np": None, "torch": torch.bfloat16}, + "int32": {"np": numpy.int32, "torch": torch.int32}, + "uint32": {"np": numpy.uint32, "torch": None}, + "int8": {"np": numpy.int8, "torch": torch.int8}, + "uint8": {"np": numpy.uint8, "torch": torch.uint8}, + "byte": {"np": numpy.ubyte, "torch": torch.uint8}, } @@ -23,6 +27,7 @@ def __new__(cls, name, bases, attrs): if name in _REGISTRY_DATA_TYPE: reg = _REGISTRY_DATA_TYPE[name] new_class.to_numpy = staticmethod(lambda: reg["np"]) + new_class.to_torch = staticmethod(lambda: reg["torch"]) new_class.ctype = staticmethod( lambda: getattr(_ark_core, name.upper()) ) @@ -59,6 +64,28 @@ def from_numpy(np_type: numpy.dtype) -> "DataType": f" to ark data type." ) + @staticmethod + def from_torch(torch_type: torch.dtype) -> "DataType": + """ + Return the corresponding ark data type. + + Parameters: + torch_type (torch.dtype): The torch data type. + + Returns: + DataType: The corresponding ark data type. + + Raises: + ValueError: If there is no defined conversion from torch data type to ark data type. + """ + for type_name, reg in _REGISTRY_DATA_TYPE.items(): + if reg["torch"] == torch_type: + return DataType.from_name(type_name) + raise ValueError( + f"Undefined conversion from torch data type {torch_type}" + f" to ark data type." + ) + @staticmethod def from_name(type_name: str) -> "DataType": """ @@ -104,6 +131,16 @@ def to_numpy() -> numpy.dtype: """ ... + @staticmethod + def to_torch() -> torch.dtype: + """ + Return the corresponding torch data type. + + Returns: + torch.dtype: The corresponding torch data type. + """ + ... + @staticmethod def ctype() -> _ark_core._DataType: """ diff --git a/python/ark/init.py b/python/ark/init.py index be71e8e02..32f530791 100644 --- a/python/ark/init.py +++ b/python/ark/init.py @@ -1,7 +1,7 @@ # Copyright (c) Microsoft Corporation. # Licensed under the MIT license. -import _ark_core +from . import _ark_core from .model import Model from .runtime import _RuntimeState @@ -9,7 +9,6 @@ def init(): """Initializes ARK.""" Model.reset() - if _RuntimeState.executor is not None: - if not _RuntimeState.executor.destroyed(): - _RuntimeState.executor.destroy() + if _RuntimeState.runtime: + _RuntimeState.delete_all() _ark_core.init() diff --git a/python/ark/model.py b/python/ark/model.py index e6208fc16..87af88f49 100644 --- a/python/ark/model.py +++ b/python/ark/model.py @@ -2,7 +2,7 @@ # Licensed under the MIT license. from typing import NewType -from _ark_core import _Model +from ._ark_core import _Model _ModelState = NewType("_ModelState", None) diff --git a/python/ark/module.py b/python/ark/module.py index 62b941281..faeeea40d 100644 --- a/python/ark/module.py +++ b/python/ark/module.py @@ -3,8 +3,20 @@ import logging import numpy as np -from typing import Any, Dict -from .tensor import Parameter +from typing import Any, Dict, List, Union +from .tensor import Tensor, Parameter +from .runtime import Runtime, DefaultPlanner +from .ops import tensor +from .data_type import DataType + +try: + import torch + + _no_torch = False +except ImportError: + from . import torch_mock as torch + + _no_torch = True class Module: @@ -57,7 +69,9 @@ def params_dict(self, prefix="") -> Dict[str, Parameter]: return params_dict def load_state_dict( - self, state_dict: Dict[str, np.ndarray], prefix: str = "" + self, + state_dict: Dict[str, Union[np.ndarray, torch.Tensor]], + prefix: str = "", ): """ Loads a model from a state_dict and copy the parameters to the device GPU. @@ -68,21 +82,112 @@ def load_state_dict( all_keys = set(state_dict.keys()) pd = self.params_dict(prefix) for name, param in pd.items(): - param.from_numpy(state_dict[name]) + data = state_dict.get(name, None) + if data is None: + continue + param.copy(data) all_keys.remove(name) if all_keys: logging.warning( f"{len(all_keys)} unused parameter(s) in state_dict" ) - def state_dict(self, prefix: str = "") -> Dict[str, np.ndarray]: + def state_dict( + self, prefix: str = "", mode: str = "numpy" + ) -> Dict[str, Union[np.ndarray, torch.Tensor]]: """ Copies the parameters from the device GPU to the host and saves the model to a state_dict. Must be called after the executor is launched. """ - return {k: v.to_numpy() for k, v in self.params_dict(prefix).items()} + if mode == "numpy": + return { + k: v.to_numpy() for k, v in self.params_dict(prefix).items() + } + elif mode == "torch": + return { + k: v.to_torch() for k, v in self.params_dict(prefix).items() + } + raise ValueError(f"Unsupported mode: {mode}") def forward(self, *args: Any, **kwargs: Any) -> Any: ... def backward(self, *args: Any, **kwargs: Any) -> Any: ... + + def initialize(self): + for param in self.parameters.values(): + param.initialize() + for module in self.sub_modules.values(): + module.initialize() + + +def _recursive_ark_to_torch(object): + if isinstance(object, Tensor): + return object.to_torch() + if isinstance(object, dict): + return {k: _recursive_ark_to_torch(v) for k, v in object.items()} + if isinstance(object, list): + return [_recursive_ark_to_torch(v) for v in object] + return object + + +class RuntimeModule(Module): + def __init__(self): + if _no_torch: + raise ImportError("torch is not available") + super().__init__() + self.built_forward = False + self.built_backward = False + self.forward_input_tensor_args: List[Tensor] = [] + self.forward_input_tensor_kwargs: Dict[str, Tensor] = {} + self.forward_input_args = [] + self.forward_input_kwargs = {} + self.forward_output = None + self.backward_tensor_args = [] + self.backward_tensor_kwargs = {} + + def build_forward(self, *args: Any, **kwargs: Any) -> Any: ... + + def build_backward(self, *args: Any, **kwargs: Any) -> Any: ... + + def forward(self, *args: Any, **kwargs: Any) -> Any: + if not self.built_forward: + for arg in args: + if isinstance(arg, torch.Tensor): + self.forward_input_tensor_args.append( + tensor( + list(arg.shape), + DataType.from_torch(arg.dtype), + ) + ) + self.forward_input_args.append( + self.forward_input_tensor_args[-1] + ) + else: + self.forward_input_args.append(arg) + for key, value in kwargs.items(): + if isinstance(value, torch.Tensor): + self.forward_input_tensor_kwargs[key] = tensor( + list(value.shape), + DataType.from_torch(value.dtype), + ) + self.forward_input_kwargs[key] = ( + self.forward_input_tensor_kwargs[key] + ) + else: + self.forward_input_kwargs[key] = value + self.forward_output = self.build_forward( + *self.forward_input_args, + **self.forward_input_kwargs, + ) + self.built_forward = True + + with Runtime.get_runtime() as rt: + rt.launch(plan=DefaultPlanner().plan()) + for tns, arg in zip(self.forward_input_tensor_args, args): + tns.copy(arg) + for key, value in self.forward_input_tensor_kwargs.items(): + value.copy(kwargs[key]) + + rt.run() + return _recursive_ark_to_torch(self.forward_output) diff --git a/python/ark/ops.py b/python/ark/ops.py index 484e248ca..f890e5d1b 100644 --- a/python/ark/ops.py +++ b/python/ark/ops.py @@ -1,7 +1,8 @@ # Copyright (c) Microsoft Corporation. # Licensed under the MIT license. -from typing import List, Iterable, Union +import json +from typing import Any, Dict, List, Iterable, Union from .tensor import Dims, Tensor, Parameter, NullTensor from .data_type import DataType, fp32 @@ -12,6 +13,12 @@ def _is_list_or_tuple(obj): return isinstance(obj, list) or isinstance(obj, tuple) +def _config_to_str(config: Union[str, Dict[str, Any]]) -> str: + if isinstance(config, str): + return config + return json.dumps(config) + + def _tensor( shape: Iterable[int], dtype: DataType = fp32, @@ -52,6 +59,7 @@ def add( input: Union[Tensor, float], other: Union[Tensor, float], output: Tensor = NullTensor, + config: Union[str, Dict[str, Any]] = "", name: str = "add", ) -> Union[Tensor, float]: """ @@ -61,6 +69,8 @@ def add( tensor_add = ark.add(tensor1, tensor2) """ if isinstance(input, Tensor) and isinstance(other, Tensor): + if input.runtime_id != other.runtime_id: + raise ValueError("Tensors must be on the same runtime") a = input._tensor b = other._tensor elif isinstance(input, Tensor): @@ -73,24 +83,33 @@ def add( return input + other else: return Tensor( - Model.get_model().copy(input + other, output._tensor, name) + Model.get_model().copy( + input + other, output._tensor, _config_to_str(config), name + ) ) if output is not NullTensor: output = output._tensor - return Tensor(Model.get_model().add(a, b, output, name)) + return Tensor( + Model.get_model().add(a, b, output, _config_to_str(config), name), + runtime_id=input.runtime_id, + ) def cast( input: Tensor, dtype: DataType, output: Tensor = NullTensor, + config: Union[str, Dict[str, Any]] = "", name: str = "cast", ) -> Tensor: """Type casting.""" if output is not NullTensor: output = output._tensor return Tensor( - Model.get_model().cast(input._tensor, dtype.ctype(), output, name) + Model.get_model().cast( + input._tensor, dtype.ctype(), output, _config_to_str(config), name + ), + runtime_id=input.runtime_id, ) @@ -99,28 +118,37 @@ def constant( shape: Iterable[int], dtype: DataType = fp32, name: str = "constant", + runtime_id: int = -1, ) -> Tensor: """Constant.""" return Tensor( - Model.get_model().constant(value, Dims(shape), dtype.ctype(), name) + Model.get_model().constant(value, Dims(shape), dtype.ctype(), name), + runtime_id=runtime_id, ) def copy( - input: Union[Tensor, float], output: Tensor = NullTensor, name: str = "copy" + input: Union[Tensor, float], + output: Tensor = NullTensor, + config: Union[str, Dict[str, Any]] = "", + name: str = "copy", ) -> Tensor: """Data caopy.""" if output is not NullTensor: output = output._tensor if isinstance(input, Tensor): intput = intput._tensor - return Tensor(Model.get_model().copy(intput, output, name)) + return Tensor( + Model.get_model().copy(intput, output, _config_to_str(config), name), + runtime_id=input.runtime_id, + ) def div( input: Tensor, other: Union[Tensor, float], output: Tensor = NullTensor, + config: Union[str, Dict[str, Any]] = "", name: str = "div", ) -> Tensor: """ @@ -132,26 +160,42 @@ def div( if output is not NullTensor: output = output._tensor if isinstance(other, Tensor): + if input.runtime_id != other.runtime_id: + raise ValueError("Tensors must be on the same runtime") other = other._tensor - return Tensor(Model.get_model().div(input._tensor, other, output, name)) + return Tensor( + Model.get_model().div( + input._tensor, other, output, _config_to_str(config), name + ), + runtime_id=input.runtime_id, + ) def embedding( input: Tensor, weight: Tensor, output: Tensor = NullTensor, + config: Union[str, Dict[str, Any]] = "", name: str = "embedding", ) -> Tensor: """Embedding layer.""" + if input.runtime_id != weight.runtime_id: + raise ValueError("Tensors must be on the same runtime") if output is not NullTensor: output = output._tensor return Tensor( - Model.get_model().embedding(input._tensor, weight._tensor, output, name) + Model.get_model().embedding( + input._tensor, weight._tensor, output, _config_to_str(config), name + ), + runtime_id=input.runtime_id, ) def exp( - input: Tensor, output: Tensor = NullTensor, name: str = "exp" + input: Tensor, + output: Tensor = NullTensor, + config: Union[str, Dict[str, Any]] = "", + name: str = "exp", ) -> Tensor: """ Calculates the exponential of the `input` tensor, element-wise. @@ -160,11 +204,19 @@ def exp( """ if output is not NullTensor: output = output._tensor - return Tensor(Model.get_model().exp(input._tensor, output, name)) + return Tensor( + Model.get_model().exp( + input._tensor, output, _config_to_str(config), name + ), + runtime_id=input.runtime_id, + ) def gelu( - input: Tensor, output: Tensor = NullTensor, name: str = "gelu" + input: Tensor, + output: Tensor = NullTensor, + config: Union[str, Dict[str, Any]] = "", + name: str = "gelu", ) -> Tensor: """ Applies the Gaussian Error Linear Unit (GELU) activation @@ -176,7 +228,12 @@ def gelu( """ if output is not NullTensor: output = output._tensor - return Tensor(Model.get_model().gelu(input._tensor, output, name)) + return Tensor( + Model.get_model().gelu( + input._tensor, output, _config_to_str(config), name + ), + runtime_id=input.runtime_id, + ) def identity( @@ -191,8 +248,13 @@ def identity( for dep in deps: if not isinstance(dep, Tensor): raise TypeError("All dependencies should be a tensor") + if input.runtime_id != dep.runtime_id: + raise ValueError("All tensors must be on the same runtime") dep_tensors.append(dep._tensor) - return Tensor(Model.get_model().identity(input._tensor, dep_tensors, name)) + return Tensor( + Model.get_model().identity(input._tensor, dep_tensors, name), + runtime_id=input.runtime_id, + ) def matmul( @@ -201,6 +263,7 @@ def matmul( output: Tensor = NullTensor, transpose_input: bool = False, transpose_other: bool = False, + config: Union[str, Dict[str, Any]] = "", name: str = "matmul", ) -> Tensor: """ @@ -212,6 +275,8 @@ def matmul( Usage: tensor_matmul = ark.matmul(tensor1, tensor2) """ + if input.runtime_id != other.runtime_id: + raise ValueError("Tensors must be on the same runtime") if output is not NullTensor: output = output._tensor return Tensor( @@ -221,8 +286,10 @@ def matmul( output, transpose_input, transpose_other, + _config_to_str(config), name, - ) + ), + runtime_id=input.runtime_id, ) @@ -230,6 +297,7 @@ def mul( input: Tensor, other: Union[Tensor, float], output: Tensor = NullTensor, + config: Union[str, Dict[str, Any]] = "", name: str = "mul", ) -> Tensor: """ @@ -241,8 +309,15 @@ def mul( if output is not NullTensor: output = output._tensor if isinstance(other, Tensor): + if input.runtime_id != other.runtime_id: + raise ValueError("Tensors must be on the same runtime") other = other._tensor - return Tensor(Model.get_model().mul(input._tensor, other, output, name)) + return Tensor( + Model.get_model().mul( + input._tensor, other, output, _config_to_str(config), name + ), + runtime_id=input.runtime_id, + ) def noop(input: Tensor, name: str = "noop"): @@ -257,6 +332,7 @@ def reduce_max( axis: int, keepdims: bool = True, output: Tensor = NullTensor, + config: Union[str, Dict[str, Any]] = "", name: str = "reduce_max", ) -> Tensor: """ @@ -269,8 +345,9 @@ def reduce_max( output = output._tensor return Tensor( Model.get_model().reduce_max( - input._tensor, axis, keepdims, output, name - ) + input._tensor, axis, keepdims, output, _config_to_str(config), name + ), + runtime_id=input.runtime_id, ) @@ -279,6 +356,7 @@ def reduce_mean( axis: int, keepdims: bool = True, output: Tensor = NullTensor, + config: Union[str, Dict[str, Any]] = "", name: str = "reduce_mean", ) -> Tensor: """ @@ -291,8 +369,9 @@ def reduce_mean( output = output._tensor return Tensor( Model.get_model().reduce_mean( - input._tensor, axis, keepdims, output, name - ) + input._tensor, axis, keepdims, output, _config_to_str(config), name + ), + runtime_id=input.runtime_id, ) @@ -301,6 +380,7 @@ def reduce_sum( axis: int, keepdims: bool = True, output: Tensor = NullTensor, + config: Union[str, Dict[str, Any]] = "", name: str = "reduce_sum", ) -> Tensor: """ @@ -315,13 +395,17 @@ def reduce_sum( output = output._tensor return Tensor( Model.get_model().reduce_sum( - input._tensor, axis, keepdims, output, name - ) + input._tensor, axis, keepdims, output, _config_to_str(config), name + ), + runtime_id=input.runtime_id, ) def relu( - input: Tensor, output: Tensor = NullTensor, name: str = "relu" + input: Tensor, + output: Tensor = NullTensor, + config: Union[str, Dict[str, Any]] = "", + name: str = "relu", ) -> Tensor: """ Applies the ReLU activation function to the `input` tensor, @@ -331,7 +415,12 @@ def relu( """ if output is not NullTensor: output = output._tensor - return Tensor(Model.get_model().relu(input._tensor, output, name)) + return Tensor( + Model.get_model().relu( + input._tensor, output, _config_to_str(config), name + ), + runtime_id=input.runtime_id, + ) def reshape( @@ -359,7 +448,8 @@ def reshape( if len(shape) > 4: raise ValueError("Only support tensors with up to 4 dimensions") return Tensor( - Model.get_model().reshape(input._tensor, Dims(shape), allowzero, name) + Model.get_model().reshape(input._tensor, Dims(shape), allowzero, name), + runtime_id=input.runtime_id, ) @@ -367,6 +457,7 @@ def rope( input: Tensor, other: Tensor, output: Tensor = NullTensor, + config: Union[str, Dict[str, Any]] = "", name: str = "rope", ) -> Tensor: """ @@ -376,13 +467,21 @@ def rope( """ if output is not NullTensor: output = output._tensor + if input.runtime_id != other.runtime_id: + raise ValueError("Tensors must be on the same runtime") return Tensor( - Model.get_model().rope(input._tensor, other._tensor, output, name) + Model.get_model().rope( + input._tensor, other._tensor, output, _config_to_str(config), name + ), + runtime_id=input.runtime_id, ) def rsqrt( - input: Tensor, output: Tensor = NullTensor, name: str = "rsqrt" + input: Tensor, + output: Tensor = NullTensor, + config: Union[str, Dict[str, Any]] = "", + name: str = "rsqrt", ) -> Tensor: """ Calculates the square root of the `input` tensor, element-wise. @@ -391,7 +490,12 @@ def rsqrt( """ if output is not NullTensor: output = output._tensor - return Tensor(Model.get_model().rsqrt(input._tensor, output, name)) + return Tensor( + Model.get_model().rsqrt( + input._tensor, output, _config_to_str(config), name + ), + runtime_id=input.runtime_id, + ) def sharding( @@ -409,11 +513,16 @@ def sharding( _tensor_list = Model.get_model().sharding( input._tensor, axis, dim_per_shard, name ) - return [Tensor(_tensor) for _tensor in _tensor_list] + return [ + Tensor(_tensor, runtime_id=input.runtime_id) for _tensor in _tensor_list + ] def sigmoid( - input: Tensor, output: Tensor = NullTensor, name: str = "sigmoid" + input: Tensor, + output: Tensor = NullTensor, + config: Union[str, Dict[str, Any]] = "", + name: str = "sigmoid", ) -> Tensor: """ Applies the Sigmoid activation function to the `input` tensor, @@ -423,11 +532,19 @@ def sigmoid( """ if output is not NullTensor: output = output._tensor - return Tensor(Model.get_model().sigmoid(input._tensor, output, name)) + return Tensor( + Model.get_model().sigmoid( + input._tensor, output, _config_to_str(config), name + ), + runtime_id=input.runtime_id, + ) def sqrt( - input: Tensor, output: Tensor = NullTensor, name: str = "sqrt" + input: Tensor, + output: Tensor = NullTensor, + config: Union[str, Dict[str, Any]] = "", + name: str = "sqrt", ) -> Tensor: """ Calculates the square root of the `input` tensor, element-wise. @@ -436,13 +553,19 @@ def sqrt( """ if output is not NullTensor: output = output._tensor - return Tensor(Model.get_model().sqrt(input._tensor, output, name)) + return Tensor( + Model.get_model().sqrt( + input._tensor, output, _config_to_str(config), name + ), + runtime_id=input.runtime_id, + ) def sub( input: Tensor, other: Union[Tensor, float], output: Tensor = NullTensor, + config: Union[str, Dict[str, Any]] = "", name: str = "sub", ) -> Tensor: """ @@ -454,8 +577,15 @@ def sub( if output is not NullTensor: output = output._tensor if isinstance(other, Tensor): + if input.runtime_id != other.runtime_id: + raise ValueError("Tensors must be on the same runtime") other = other._tensor - return Tensor(Model.get_model().sub(input._tensor, other, output, name)) + return Tensor( + Model.get_model().sub( + input._tensor, other, output, _config_to_str(config), name + ), + runtime_id=input.runtime_id, + ) def tensor( @@ -466,6 +596,7 @@ def tensor( padded_shape: Iterable[int] = [], rank: int = -1, name: str = "", + runtime_id: int = -1, ) -> Tensor: """ Construct a tensor with given shape and data type. @@ -482,6 +613,7 @@ def transpose( input: Tensor, perm: Iterable[int], output: Tensor = NullTensor, + config: Union[str, Dict[str, Any]] = "", name: str = "transpose", ) -> Tensor: """ @@ -501,7 +633,10 @@ def transpose( if len(perm) > 4: raise ValueError("Only support perm up to 4 dimensions") return Tensor( - Model.get_model().transpose(input._tensor, perm, output, name) + Model.get_model().transpose( + input._tensor, perm, output, _config_to_str(config), name + ), + runtime_id=input.runtime_id, ) @@ -513,18 +648,23 @@ def mean( axis: int, keepdims: bool = True, output: Tensor = NullTensor, + config: Union[str, Dict[str, Any]] = "", name: str = "mean", ) -> Tensor: """Alias of reduce_mean.""" - return reduce_mean(input, axis, keepdims, output, name) + return reduce_mean(input, axis, keepdims, output, config, name) def ones( - shape: Iterable[int], dtype: DataType = fp32, name: str = "ones" + shape: Iterable[int], + dtype: DataType = fp32, + name: str = "ones", + runtime_id: int = -1, ) -> Tensor: """Ones.""" return Tensor( - Model.get_model().constant(1, Dims(shape), dtype.ctype(), name) + Model.get_model().constant(1, Dims(shape), dtype.ctype(), name), + runtime_id=runtime_id, ) @@ -535,12 +675,14 @@ def parameter( offsets: Iterable[int] = [], padded_shape: Iterable[int] = [], name: str = "", + runtime_id: int = -1, ) -> Parameter: """ Construct a parameter with given shape and data type. """ return Parameter( - _tensor(shape, dtype, strides, offsets, padded_shape, name) + _tensor(shape, dtype, strides, offsets, padded_shape, name), + runtime_id=runtime_id, ) @@ -574,11 +716,15 @@ def layernorm( def zeros( - shape: Iterable[int], dtype: DataType = fp32, name: str = "zeros" + shape: Iterable[int], + dtype: DataType = fp32, + name: str = "zeros", + runtime_id: int = -1, ) -> Tensor: """Zeros.""" return Tensor( - Model.get_model().constant(0, Dims(shape), dtype.ctype(), name) + Model.get_model().constant(0, Dims(shape), dtype.ctype(), name), + runtime_id=runtime_id, ) diff --git a/python/ark/profiler.py b/python/ark/profiler.py new file mode 100644 index 000000000..e47f5b7aa --- /dev/null +++ b/python/ark/profiler.py @@ -0,0 +1,51 @@ +# Copyright (c) Microsoft Corporation. +# Licensed under the MIT license. + +import sys +import time + +from .runtime import Runtime +from .planner import Plan + + +def timeit(plan: Plan, iter: int, loop_mode: bool): + with Runtime() as rt: + rt.launch(plan=plan, loop_mode=loop_mode) + start_time = time.time() + rt.run(iter=iter) + end_time = time.time() + return (end_time - start_time) / iter + + +class Profiler: + def __init__(self, plan: Plan): + self.plan = plan + + def run( + self, + iter: int = 1000, + loop_mode: bool = True, + profile_processor_groups: bool = False, + ): + sys.stderr.write( + f"End-to-end: {timeit(self.plan, iter, loop_mode):.6f} seconds/iter\n" + ) + + if not profile_processor_groups: + return + num_processor_groups = len(self.plan.processor_groups) + new_plan = { + "Rank": self.plan.rank, + "WorldSize": self.plan.world_size, + "Architecture": self.plan.architecture, + "NumProcessors": self.plan.num_processors, + "NumWarpsPerProcessor": self.plan.num_warps_per_processor, + "TaskInfos": self.plan.task_infos, + "ProcessorGroups": [None], + } + for i in range(num_processor_groups): + new_plan["ProcessorGroups"][0] = self.plan.processor_groups[i] + lat_per_iter = timeit(Plan(new_plan), iter, loop_mode) + sys.stderr.write( + f"Processor group {i}: {lat_per_iter:.6f} seconds/iter\n" + ) diff --git a/python/ark/runtime.py b/python/ark/runtime.py index ab844708e..c2e507bca 100644 --- a/python/ark/runtime.py +++ b/python/ark/runtime.py @@ -3,6 +3,7 @@ import logging from enum import Enum +from typing import Dict, List from _ark_core import _Executor from .planner import Planner, Plan @@ -13,8 +14,36 @@ class _RuntimeState: The _RuntimeState class is used to store the state of the model. """ - runtime = None - executor = None + runtime: Dict[int, "Runtime"] = {} + + @staticmethod + def reset_all(): + """ + Resets all runtimes. + """ + runtime_ids = list(_RuntimeState.runtime.keys()) + for runtime_id in runtime_ids: + _RuntimeState.runtime[runtime_id].reset() + + @staticmethod + def delete_all(): + """ + Deletes all runtimes. + """ + runtime_ids = list(_RuntimeState.runtime.keys()) + for runtime_id in runtime_ids: + _RuntimeState.runtime[runtime_id].reset(delete=True) + + @staticmethod + def print_runtime_states(): + """ + Print runtimes and their corresponding states. + """ + print(f"{'Runtime ID':<12} | {'Status':<20}") + print(f"{'-'*12} | {'-'*20}") + for runtime_id, runtime in _RuntimeState.runtime.items(): + runtime_id = "-1(Default)" if runtime_id == -1 else runtime_id + print(f"{runtime_id:<12} | {runtime.state:<20}") class Executor(_Executor): @@ -35,22 +64,48 @@ class State(Enum): LaunchedNotRunning = 1 Running = 2 + def __init__(self, runtime_id: int = -1): + self.runtime_id = runtime_id + self.executor: Executor = None + self.state: Runtime.State = Runtime.State.Init + _RuntimeState.runtime[runtime_id] = self + + def get_state(self) -> "Runtime.State": + """ + Get the runtime state. + """ + return self.state + @staticmethod - def get_runtime() -> "Runtime": + def exists(runtime_id: int) -> bool: """ - Get the runtime. + Check if a runtime exists with the given ID. """ - if _RuntimeState.runtime is None: - _RuntimeState.runtime = Runtime() - return _RuntimeState.runtime + return runtime_id in _RuntimeState.runtime - def __init__(self): - self.executor: Executor = None - self.state: Runtime.State = Runtime.State.Init - _RuntimeState.runtime = self + @staticmethod + def get_all_ids() -> List[int]: + """ + Get a list of all existing runtime IDs. + """ + return list(_RuntimeState.runtime.keys()) - def __del__(self): - self.reset() + @staticmethod + def get_runtime(runtime_id=-1) -> "Runtime": + """ + Get the runtime by ID. If runtime_id is not provided, use a default ID of -1. + If the runtime does not exist, create a new runtime with the given ID. + """ + if runtime_id not in _RuntimeState.runtime: + _RuntimeState.runtime[runtime_id] = Runtime(runtime_id) + return _RuntimeState.runtime[runtime_id] + + @staticmethod + def see_runtime_statuses() -> "Dict[int, Runtime]": + """ + Returns the runtime dictionary containing all of the runtimes. + """ + return _RuntimeState.runtime def __enter__(self): return self @@ -77,6 +132,8 @@ def launch( self, plan: Plan = None, device_id: int = 0, + stream: int = 0, + loop_mode: bool = True, ): """ Create an executor and schedule the ARK model. The scheduler will generate @@ -84,25 +141,27 @@ def launch( initialized. The executor will compile the cuda kernels and launch the ARK runtime. """ if self.launched(): - logging.warn("Runtime is already launched, skip launching") + logging.warning( + f"Runtime {self.runtime_id} is already launched, skip launching" + ) return plan = Planner(device_id).plan() if plan is None else plan # If the RuntimeState is init, we need to create a new executor and # compile the kernels if self.state == Runtime.State.Init: - if _RuntimeState.executor is not None: - if not _RuntimeState.executor.destroyed(): - logging.warn("Destroying an old executor") - _RuntimeState.executor.destroy() - - _RuntimeState.executor = Executor( - plan.rank, - plan.world_size, + if self.executor is not None: + if not self.executor.destroyed(): + logging.warning( + f"Runtime {self.runtime_id}, has already been launched. Destroying the old executor" + ) + self.executor.destroy() + self.executor = Executor( device_id, + stream, "ArkRuntime", str(plan), + loop_mode, ) - self.executor = _RuntimeState.executor self.executor.compile() self.executor.launch() self.state = Runtime.State.LaunchedNotRunning @@ -112,8 +171,8 @@ def run(self, iter=1, non_blocking=False): Run the ARK program for iter iterations and wait for the kernel to finish. """ if self.state != Runtime.State.LaunchedNotRunning: - logging.error("ARK runtime is not launched") - raise RuntimeError("ARK runtime is not launched") + logging.error(f"ARK runtime {self.runtime_id} is not launched") + raise RuntimeError(f"ARK runtime {self.runtime_id} is not launched") self.state = Runtime.State.Running self.executor.run(iter) if not non_blocking: @@ -133,7 +192,9 @@ def wait(self): Wait for the kernel to finish. """ if self.state != Runtime.State.Running: - logging.warn("ARK runtime is not running, skip waiting") + logging.warning( + f"ARK runtime {self.runtime_id} is not running, skip waiting" + ) return self.executor.wait() self.state = Runtime.State.LaunchedNotRunning @@ -144,15 +205,17 @@ def stop(self) -> float: Once this is called, we need to call `launch()` again to run the model again. """ if not self.launched(): - logging.warn("ARK runtime is never launched, skip stopping") + logging.warning( + f"ARK runtime {self.runtime_id} is never launched, skip stopping" + ) return elapsed = self.executor.stop() self.state = Runtime.State.LaunchedNotRunning return elapsed - def reset(self): + def reset(self, delete=False): """ - Reset the runtime. + Reset the runtime. If delete is True, delete the runtime associated with the runtime_id. """ if self.launched(): self.stop() @@ -161,3 +224,26 @@ def reset(self): self.executor.destroy() self.executor = None self.state = Runtime.State.Init + if delete: + del _RuntimeState.runtime[self.runtime_id] + + @staticmethod + def reset_all_runtimes(): + """ + Reset all runtimes. + """ + _RuntimeState.reset_all() + + @staticmethod + def delete_all_runtimes(): + """ + Delete all runtimes. + """ + _RuntimeState.delete_all() + + @staticmethod + def print_runtime_states(): + """ + Print runtimes and their corresponding states. + """ + _RuntimeState.print_runtime_states() diff --git a/python/ark/tensor.py b/python/ark/tensor.py index 316d18566..eed7a4259 100644 --- a/python/ark/tensor.py +++ b/python/ark/tensor.py @@ -2,11 +2,21 @@ # Licensed under the MIT license. import numpy as np -from typing import List +from typing import Callable, List, Union, Type -from _ark_core import _Dims, _Tensor, _NullTensor +from ._ark_core import _Dims, _Tensor, _NullTensor from .data_type import DataType from .runtime import Runtime +from .model import Model + +try: + import torch + + _no_torch = False +except ImportError: + from . import torch_mock as torch + + _no_torch = True NullTensor = _NullTensor @@ -15,14 +25,26 @@ class Dims(_Dims): pass +Initializer = Type[Callable[[], Union[torch.Tensor, np.ndarray]]] + + class Tensor: - def __init__(self, _tensor: _Tensor): + def __init__( + self, + _tensor: _Tensor, + initializer: Initializer = None, + runtime_id: int = -1, + ): """ Initializes a new instance of the Tensor class. Args: _tensor (_ark_core._Tensor): The underlying _Tensor object. + intializer (Initializer): The initializer for the Tensor. + runtime_id (int): The ID of the Runtime to use. Defaults to -1, which is the default Runtime. """ self._tensor = _tensor + self.initializer: Initializer = initializer + self.runtime_id = runtime_id def shape(self) -> List[int]: """ @@ -48,17 +70,26 @@ def dtype(self) -> DataType: """ return DataType.from_ctype(self._tensor.data_type()) - def to_numpy(self, ndarray: np.ndarray = None) -> np.ndarray: + def to_numpy( + self, ndarray: np.ndarray = None, stream: int = 0 + ) -> np.ndarray: """ Copy a tensor from device to host. If `ndarray` is None, a new numpy array will be created. If the tensor is not allocated, an empty numpy array without the data buffer will be returned. """ np_type = self.dtype().to_numpy() - rt = Runtime.get_runtime() + if np_type is None: + raise ValueError( + f"Tensor data type {self.dtype().__name__} is not supported by numpy." + ) + rt = Runtime.get_runtime(self.runtime_id) if not rt.launched(): - return np.ndarray(self.shape(), dtype=np_type, buffer=None) - if ndarray is None: + raise RuntimeError( + "Tensor is not allocated yet. `Tensor.to_numpy()` is " + "usable only after you call `Runtime.launch()`." + ) + elif ndarray is None: ndarray = np.zeros(self.shape(), dtype=np_type) elif not ndarray.flags["C_CONTIGUOUS"]: raise ValueError("ndarray is not contiguous in memory") @@ -68,14 +99,71 @@ def to_numpy(self, ndarray: np.ndarray = None) -> np.ndarray: raise ValueError("ndarray dtype does not match the tensor") elif ndarray.nbytes != self.nelems() * self.dtype().element_size(): raise ValueError("ndarray size does not match the tensor") - rt.executor.tensor_read(self._tensor, ndarray) + rt.executor.tensor_read(self._tensor, ndarray, stream) return ndarray - def from_numpy(self, ndarray: np.ndarray) -> "Tensor": + def to_torch( + self, tensor: torch.Tensor = None, stream: int = 0 + ) -> torch.Tensor: + """ """ + if _no_torch: + raise ImportError("torch is not available") + rt = Runtime.get_runtime(self.runtime_id) + if not rt.launched(): + raise RuntimeError( + "Tensor is not allocated yet. `Tensor.to_torch()` is " + "usable only after you call `Runtime.launch()`." + ) + torch_type = self.dtype().to_torch() + if tensor is None: + dev_name = f"cuda:{rt.executor.device_id()}" + tensor = torch.zeros( + self.shape(), dtype=torch_type, device=torch.device(dev_name) + ) + elif list(tensor.shape) != self.shape(): + raise ValueError( + f"torch tensor shape {list(tensor.shape)} " + f"does not match the tensor {self.shape()}" + ) + elif tensor.dtype != torch_type: + raise ValueError( + f"torch tensor dtype {tensor.dtype} " + f"does not match the tensor {torch_type}" + ) + elif not tensor.is_contiguous(): + raise ValueError("torch tensor is not contiguous in memory") + elif tensor.numel() != self.nelems(): + raise ValueError( + f"torch tensor size {tensor.numel()} " + f"does not match the tensor {self.nelems()}" + ) + tensor_bytes = self.nelems() * self.dtype().element_size() + rt.executor.tensor_read( + self._tensor, tensor.data_ptr(), tensor_bytes, stream, True + ) + return tensor + + def get_torch_view(self) -> torch.Tensor: + """ + Returns a torch tensor that shares the same memory with the device tensor. + """ + if _no_torch: + raise ImportError("torch is not available") + rt = Runtime.get_runtime(self.runtime_id) + if not rt.launched(): + raise RuntimeError( + "Tensor is not allocated yet. `Tensor.get_torch_view()` is " + "usable only after you call `Runtime.launch()`." + ) + dl_tensor = rt.executor.get_dl_tensor(self._tensor) + torch_view = torch.utils.dlpack.from_dlpack(dl_tensor) + return torch_view + + def from_numpy(self, ndarray: np.ndarray, stream: int = 0) -> "Tensor": """ Copies the tensor from a host numpy array to the device. """ - rt = Runtime.get_runtime() + rt = Runtime.get_runtime(self.runtime_id) if not rt.launched(): raise RuntimeError( "Tensor is not allocated yet. `Tensor.from_numpy()` is " @@ -86,7 +174,68 @@ def from_numpy(self, ndarray: np.ndarray) -> "Tensor": ndarray = np.ascontiguousarray(ndarray) if ndarray.nbytes != self.nelems() * self.dtype().element_size(): raise ValueError("ndarray size does not match the tensor") - rt.executor.tensor_write(self._tensor, ndarray) + rt.executor.tensor_write(self._tensor, ndarray, stream) + return self + + @staticmethod + def from_torch(tensor: torch.Tensor, runtime_id: int = -1) -> "Tensor": + """ + Returns an ARK tensor that shares the same memory with the torch tensor. + """ + if _no_torch: + raise ImportError("torch is not available") + elif not tensor.is_contiguous(): + raise ValueError("Torch tensor must be contiguous.") + elif tensor.device.type == "cpu": + raise ValueError("Torch tensor must be on a device.") + ark_dtype = DataType.from_torch(tensor.dtype) + dl_capsule = torch.utils.dlpack.to_dlpack(tensor) + ark_tensor = _Tensor(dl_capsule, ark_dtype.ctype()) + return Tensor(ark_tensor, runtime_id=runtime_id) + + def copy( + self, data: Union[np.ndarray, torch.Tensor], stream: int = 0 + ) -> "Tensor": + """ + Copies data into this tensor. The data type may differ, + but the size must match. + """ + rt = Runtime.get_runtime(self.runtime_id) + if not rt.launched(): + raise RuntimeError( + "Tensor is not allocated yet. `Tensor.from_numpy()` is " + "usable only after you call `Runtime.launch()`." + ) + tensor_bytes = self.nelems() * self.dtype().element_size() + if isinstance(data, torch.Tensor): + if not data.is_contiguous(): + data = data.contiguous() + if data.numel() * data.element_size() != tensor_bytes: + raise ValueError("data size does not match the tensor") + rt.executor.tensor_write( + self._tensor, + data.data_ptr(), + tensor_bytes, + stream, + data.device.type == "cuda", + ) + elif isinstance(data, np.ndarray): + if not data.flags["C_CONTIGUOUS"]: + data = np.ascontiguousarray(data) + if data.nbytes != tensor_bytes: + raise ValueError("data size does not match the tensor") + rt.executor.tensor_write(self._tensor, data, stream) + else: + raise ValueError("data must be a numpy array or a torch tensor") + return self + + def initialize(self) -> "Tensor": + """ + Initializes the tensor. + """ + if self.initializer is not None: + data = self.initializer() + self.copy(data) return self @@ -95,8 +244,9 @@ class Parameter(Tensor): A tensor as a parameter. """ - def __init__(self, _tensor: _Tensor): + def __init__(self, _tensor: _Tensor, runtime_id: int = -1): """ Initializes a new instance of the Parameter class. """ super().__init__(_tensor) + self.runtime_id = runtime_id diff --git a/python/ark/torch_mock.py b/python/ark/torch_mock.py new file mode 100644 index 000000000..68333e431 --- /dev/null +++ b/python/ark/torch_mock.py @@ -0,0 +1,29 @@ +# Copyright (c) Microsoft Corporation. +# Licensed under the MIT license. + + +class dtype: ... + + +class float32: ... + + +class float16: ... + + +class bfloat16: ... + + +class int32: ... + + +class int8: ... + + +class uint8: ... + + +class ubyte: ... + + +class Tensor: ... diff --git a/python/ark_py.cpp b/python/ark_py.cpp index 1bc4255d6..75788ba55 100644 --- a/python/ark_py.cpp +++ b/python/ark_py.cpp @@ -7,6 +7,7 @@ namespace py = pybind11; +extern void register_plan_manager(py::module &m); extern void register_data_type(py::module &m); extern void register_dims(py::module &m); extern void register_error(py::module &m); @@ -22,6 +23,7 @@ extern void register_version(py::module &m); PYBIND11_MODULE(_ark_core, m) { m.doc() = "Bind ARK C++ APIs to Python"; + register_plan_manager(m); register_data_type(m); register_dims(m); register_error(m); diff --git a/python/executor_py.cpp b/python/executor_py.cpp index dc2840329..c2a8cc8ab 100644 --- a/python/executor_py.cpp +++ b/python/executor_py.cpp @@ -1,43 +1,194 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#include #include #include #include #include #include - +#include +#include namespace py = pybind11; static void tensor_write(ark::Executor *exe, const ark::Tensor &tensor, - py::buffer host_buffer) { + py::buffer host_buffer, uintptr_t stream) { py::buffer_info info = host_buffer.request(); exe->tensor_write(tensor, reinterpret_cast(info.ptr), - info.size * info.itemsize); + info.size * info.itemsize, + reinterpret_cast(stream), false); +} + +static void tensor_write(ark::Executor *exe, const ark::Tensor &tensor, + size_t address, size_t bytes, uintptr_t stream, + bool is_d2d) { + exe->tensor_write(tensor, reinterpret_cast(address), bytes, + reinterpret_cast(stream), is_d2d); } static void tensor_read(ark::Executor *exe, const ark::Tensor &tensor, - py::buffer host_buffer) { + py::buffer host_buffer, uintptr_t stream) { py::buffer_info info = host_buffer.request(); exe->tensor_read(tensor, reinterpret_cast(info.ptr), - info.size * info.itemsize); + info.size * info.itemsize, + reinterpret_cast(stream), false); +} + +static void tensor_read(ark::Executor *exe, const ark::Tensor &tensor, + size_t address, size_t bytes, uintptr_t stream, + bool is_d2d) { + exe->tensor_read(tensor, reinterpret_cast(address), bytes, + reinterpret_cast(stream), is_d2d); +} + +static DLDataType get_dl_dtype(const ark::DataType &ark_data_type) { + DLDataType dl_data_type; + dl_data_type.lanes = 1; + if (ark_data_type == ark::FP32) { + dl_data_type.code = kDLFloat; + dl_data_type.bits = 32; + } else if (ark_data_type == ark::FP16) { + dl_data_type.code = kDLFloat; + dl_data_type.bits = 16; + } else if (ark_data_type == ark::BF16) { + dl_data_type.code = kDLBfloat; + dl_data_type.bits = 16; + } else if (ark_data_type == ark::INT32) { + dl_data_type.code = kDLInt; + dl_data_type.bits = 32; + } else if (ark_data_type == ark::UINT32) { + dl_data_type.code = kDLUInt; + dl_data_type.bits = 32; + } else if (ark_data_type == ark::INT8) { + dl_data_type.code = kDLInt; + dl_data_type.bits = 8; + } else if (ark_data_type == ark::UINT8) { + dl_data_type.code = kDLUInt; + dl_data_type.bits = 8; + } else if (ark_data_type == ark::BYTE) { + dl_data_type.code = kDLUInt; + dl_data_type.bits = 8; + } else { + throw std::runtime_error("unexpected error"); + } + return dl_data_type; +} + +static DLDeviceType get_device_type() { +#if defined(ARK_CUDA) + return kDLCUDA; +#elif defined(ARK_ROCM) + return kDLROCM; +#else + return kDLCPU; +#endif +} + +static DLManagedTensor *to_dlpack(ark::Executor &exe, + const ark::Tensor &tensor) { + DLTensor dl_tensor; + dl_tensor.data = reinterpret_cast(exe.tensor_address(tensor)); + size_t offset_in_elements = + tensor.offsets().is_no_dim() ? 0 : tensor.offsets().vector()[0]; + dl_tensor.byte_offset = offset_in_elements * tensor.data_type().bytes(); + dl_tensor.device.device_type = get_device_type(); + dl_tensor.device.device_id = static_cast(exe.device_id()); + dl_tensor.ndim = static_cast(tensor.shape().ndims()); + dl_tensor.dtype = get_dl_dtype(tensor.data_type()); + + dl_tensor.shape = + tensor.shape().is_no_dim() ? nullptr : new int64_t[dl_tensor.ndim]; + dl_tensor.strides = + tensor.strides().is_no_dim() ? nullptr : new int64_t[dl_tensor.ndim]; + auto shape = tensor.shape(); + if (dl_tensor.shape) { + for (int i = 0; i < dl_tensor.ndim; ++i) { + dl_tensor.shape[i] = shape[i]; + } + } + if (dl_tensor.strides) { + dl_tensor.strides[dl_tensor.ndim - 1] = 1; + for (int i = dl_tensor.ndim - 2; i >= 0; --i) { + dl_tensor.strides[i] = + dl_tensor.shape[i + 1] * dl_tensor.strides[i + 1]; + } + } + DLManagedTensor *dl_managed_tensor = new DLManagedTensor(); + dl_managed_tensor->dl_tensor = dl_tensor; + dl_managed_tensor->manager_ctx = nullptr; + dl_managed_tensor->deleter = [](DLManagedTensor *self) { + if (self->dl_tensor.shape) { + delete[] self->dl_tensor.shape; + self->dl_tensor.shape = nullptr; + } + if (self->dl_tensor.strides) { + delete[] self->dl_tensor.strides; + self->dl_tensor.strides = nullptr; + } + }; + return dl_managed_tensor; +} + +void free_capsule(PyObject *capsule) { + const char *name = PyCapsule_GetName(capsule); + auto *dl_managed_tensor = + static_cast(PyCapsule_GetPointer(capsule, name)); + if (dl_managed_tensor) { + dl_managed_tensor->deleter(dl_managed_tensor); + dl_managed_tensor = nullptr; + } +} + +py::capsule to_dlpack_capsule(ark::Executor &self, const ark::Tensor &tensor) { + DLManagedTensor *dl_managed_tensor = to_dlpack(self, tensor); + const char *capsule_name = "dltensor"; + PyObject *dl_capsule = PyCapsule_New(static_cast(dl_managed_tensor), + capsule_name, free_capsule); + return py::reinterpret_steal(dl_capsule); } void register_executor(py::module &m) { py::class_(m, "_Executor") - .def( - py::init(), - py::arg("rank"), py::arg("world_size"), py::arg("gpu_id"), - py::arg("name"), py::arg("plan")) + .def(py::init([](int device_id, uintptr_t stream, + const std::string &name, const std::string &plan, + bool loop_mode) { + return new ark::Executor(device_id, + reinterpret_cast(stream), + name, plan, loop_mode); + })) + .def("device_id", &ark::Executor::device_id) + .def("stream", + [](ark::Executor *self) { + return reinterpret_cast(self->stream()); + }) + .def("plan", &ark::Executor::plan) .def("compile", &ark::Executor::compile) - .def("launch", &ark::Executor::launch, py::arg("max_spin_count") = -1) + .def("launch", &ark::Executor::launch) .def("run", &ark::Executor::run, py::arg("iter")) .def("wait", &ark::Executor::wait, py::arg("max_spin_count") = -1) .def("stop", &ark::Executor::stop, py::arg("max_spin_count") = -1) .def("barrier", &ark::Executor::barrier) .def("destroy", &ark::Executor::destroy) .def("destroyed", &ark::Executor::destroyed) - .def("tensor_read", &tensor_read, py::arg("tensor"), py::arg("data")) - .def("tensor_write", &tensor_write, py::arg("tensor"), py::arg("data")); + .def("tensor_address", &ark::Executor::tensor_address) + .def("tensor_read", + py::overload_cast(&tensor_read), + py::arg("tensor"), py::arg("data"), py::arg("stream")) + .def("tensor_read", + py::overload_cast(&tensor_read), + py::arg("tensor"), py::arg("address"), py::arg("bytes"), + py::arg("stream"), py::arg("is_d2d")) + .def("tensor_write", + py::overload_cast(&tensor_write), + py::arg("tensor"), py::arg("data"), py::arg("stream")) + .def("tensor_write", + py::overload_cast(&tensor_write), + py::arg("tensor"), py::arg("address"), py::arg("bytes"), + py::arg("stream"), py::arg("is_d2d")) + .def("get_dl_tensor", &to_dlpack_capsule); } diff --git a/python/tensor_py.cpp b/python/tensor_py.cpp index fbd909d3d..16eb03421 100644 --- a/python/tensor_py.cpp +++ b/python/tensor_py.cpp @@ -1,6 +1,7 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#include #include #include #include @@ -9,8 +10,51 @@ namespace py = pybind11; -void register_tensor(py::module &m) { +struct DLTensorMetadata { + void* data_ptr; + int32_t device_id; + DLDeviceType device_type; + int32_t ndim; + DLDataType dtype; + std::vector shape; + std::vector strides; + uint64_t byte_offset; +}; + +static DLTensorMetadata extractDLTensorMetadata(DLManagedTensor* dl_tensor) { + DLTensorMetadata metadata; + metadata.data_ptr = dl_tensor->dl_tensor.data; + metadata.device_id = dl_tensor->dl_tensor.device.device_id; + metadata.device_type = dl_tensor->dl_tensor.device.device_type; + metadata.ndim = dl_tensor->dl_tensor.ndim; + metadata.dtype = dl_tensor->dl_tensor.dtype; + metadata.shape.assign( + dl_tensor->dl_tensor.shape, + dl_tensor->dl_tensor.shape + dl_tensor->dl_tensor.ndim); + if (dl_tensor->dl_tensor.strides != nullptr) { + metadata.strides.assign( + dl_tensor->dl_tensor.strides, + dl_tensor->dl_tensor.strides + dl_tensor->dl_tensor.ndim); + } + metadata.byte_offset = dl_tensor->dl_tensor.byte_offset; + return metadata; +} + +void register_tensor(py::module& m) { py::class_(m, "_Tensor") + .def(py::init([](py::capsule capsule, const ark::DataType& dtype) { + DLManagedTensor* dl_tensor = (DLManagedTensor*)capsule; + if (!dl_tensor) { + throw std::runtime_error( + "Capsule does not contain a DLManagedTensor"); + } + DLTensorMetadata metadata = extractDLTensorMetadata(dl_tensor); + int32_t device_id = metadata.device_id; + void* data_ptr = metadata.data_ptr; + auto shape = metadata.shape; + + return new ark::Tensor(data_ptr, device_id, shape, dtype); + })) .def("id", &ark::Tensor::id) .def("shape", &ark::Tensor::shape, py::return_value_policy::reference) .def("strides", &ark::Tensor::strides, diff --git a/python/unittest/test.py b/python/unittest/test.py index 2d9647e3a..238b16fb6 100644 --- a/python/unittest/test.py +++ b/python/unittest/test.py @@ -10,3 +10,4 @@ from test_error import * from test_model import * from test_runtime import * +from test_conversion import * diff --git a/python/unittest/test_conversion.py b/python/unittest/test_conversion.py new file mode 100644 index 000000000..833b88662 --- /dev/null +++ b/python/unittest/test_conversion.py @@ -0,0 +1,207 @@ +import pytest +import numpy as np +import ark +from typing import Callable + +try: + import torch + + _no_torch = False +except ImportError: + _no_torch = True + +# ARK to Torch tests + + +def initialize_tensor(dimensions, dtype): + tensor = ark.tensor(dimensions, dtype) + tensor_host = np.random.rand(*dimensions).astype(dtype.to_numpy()) + return tensor, tensor_host + + +# Test function to validate the integrity of the PyTorch view of the ARK tensor, +# including its data and attributes such as shape and data type. +@pytest.mark.parametrize("num_dims,size", [(1, 5), (1, 1024), (2, 5), (2, 32)]) +@pytest.mark.parametrize("dtype", [ark.fp16, ark.fp32]) +def test_values_fixed_dims(num_dims: int, size: int, dtype: ark.DataType): + ark.init() + dimensions = [size] * num_dims + + input_tensor, input_tensor_host = initialize_tensor(dimensions, dtype) + other_tensor, other_tensor_host = initialize_tensor(dimensions, dtype) + output_tensor = ark.add(input_tensor, other_tensor) + + runtime = ark.Runtime() + runtime.launch() + + input_tensor.from_numpy(input_tensor_host) + other_tensor.from_numpy(other_tensor_host) + + input_view = input_tensor.get_torch_view() + other_view = other_tensor.get_torch_view() + output_view = output_tensor.get_torch_view() + + runtime.run() + + input_view_numpy = input_view.cpu().numpy() + other_view_numpy = other_view.cpu().numpy() + output_view_numpy = output_view.cpu().numpy() + + output_tensor_host = output_tensor.to_numpy() + + runtime.stop() + runtime.delete_all_runtimes() + + assert np.allclose(input_tensor_host, input_view_numpy) + assert np.allclose(other_tensor_host, other_view_numpy) + assert np.allclose(output_tensor_host, output_view_numpy) + + +# Function to check if there is a difference between two arrays at a specific index +def check_diff(input_tensor_host, input_view_numpy, value, index): + mask = np.ones(input_tensor_host.shape, dtype=bool) + mask[index] = False + if not np.allclose(input_tensor_host[mask], input_view_numpy[mask]): + print("Difference found at index: ", index) + return False + if input_view_numpy[index] != value: + print(input_view_numpy[index], value) + return False + return True + + +# Test function to check if changes to the torch views are reflected in the original tensors +@pytest.mark.parametrize("dtype", [ark.fp16, ark.fp32]) +def test_ark_to_torch_aliasing(dtype: ark.DataType): + ark.init() + dimensions = [4, 4] + input_tensor, input_tensor_host = initialize_tensor(dimensions, dtype) + other_tensor, other_tensor_host = initialize_tensor(dimensions, dtype) + output_tensor = ark.mul(input_tensor, other_tensor) + runtime = ark.Runtime() + runtime.launch() + input_tensor.from_numpy(input_tensor_host) + other_tensor.from_numpy(other_tensor_host) + + input_view = input_tensor.get_torch_view() + other_view = other_tensor.get_torch_view() + output_view = output_tensor.get_torch_view() + # make changes to the views + input_view[1, 1] = 20 + other_view[0, 0] = 30 + runtime.run() + output_view[3, 0] = 40 + + output_tensor_host = output_tensor.to_numpy() + input_view_numpy = input_view.cpu().numpy() + other_view_numpy = other_view.cpu().numpy() + output_view_numpy = output_view.cpu().numpy() + # Check if changes to the views are reflected in the original tensors + print(input_view_numpy) + assert check_diff(input_tensor_host, input_view_numpy, 20, (1, 1)) + assert check_diff(other_tensor_host, other_view_numpy, 30, (0, 0)) + assert check_diff(output_tensor_host, output_view_numpy, 40, (3, 0)) + + runtime.stop() + runtime.reset() + + +def test_conversion_torch(): + if _no_torch: + pytest.skip("PyTorch not available") + + dimensions = [4, 4] + + ark.init() + t = ark.constant(7, dimensions) + + with ark.Runtime() as rt: + rt.launch() + + torch_tensor = t.to_torch() + + assert torch_tensor.shape == (4, 4) + assert torch_tensor.dtype == torch.float32 + assert torch_tensor.device.type == "cuda" + assert torch.all(torch_tensor == 0) + + rt.run() + + torch_tensor = t.to_torch() + assert torch.all(torch_tensor == 7) + + +# Torch to ARK tests + +ArkBinOp = Callable[[ark.Tensor, ark.Tensor], ark.Tensor] +TorchBinOp = Callable[[torch.Tensor, torch.Tensor], torch.Tensor] +ArkUnOp = Callable[[ark.Tensor], ark.Tensor] +TorchUnOp = Callable[[torch.Tensor], torch.Tensor] + + +# Verify the accuracy of binary operations involving ARK view tensors +@pytest.mark.parametrize( + "dtype, ark_op, torch_op, tensor_dims", + [(torch.float16, ark.add, torch.add, (2, 3))], +) +def test_bin_op(dtype, ark_op: ArkBinOp, torch_op: TorchBinOp, tensor_dims): + ark.init() + input_tensor = torch.randn(tensor_dims, dtype=dtype, device="cuda:0") + other_tensor = torch.randn(tensor_dims, dtype=dtype, device="cuda:0") + expected_output = torch_op(input_tensor, other_tensor).cpu().numpy() + input_ark_view = ark.Tensor.from_torch(input_tensor) + other_ark_view = ark.Tensor.from_torch(other_tensor) + output = ark_op(input_ark_view, other_ark_view) + runtime = ark.Runtime() + runtime.launch() + runtime.run() + output_host = output.to_numpy() + runtime.stop() + runtime.reset() + assert np.allclose(output_host, expected_output) + + +# Verify the accuracy of unary operations involving ARK view tensors +@pytest.mark.parametrize( + "dtype, ark_op, torch_op, tensor_dims", + [(torch.float16, ark.exp, torch.exp, (3, 3))], +) +def test_unary_op(dtype, ark_op: ArkUnOp, torch_op: TorchUnOp, tensor_dims): + ark.init() + input_tensor = torch.randn(tensor_dims, dtype=dtype, device="cuda:0") + expected_output = torch_op(input_tensor).cpu().numpy() + input_ark_view = ark.Tensor.from_torch(input_tensor) + output = ark_op(input_ark_view) + runtime = ark.Runtime() + runtime.launch() + runtime.run() + output_host = output.to_numpy() + runtime.stop() + runtime.reset() + assert np.allclose(output_host, expected_output) + + +# Test function to check if changes in torch tensors are reflected in ARK views +@pytest.mark.parametrize("dtype, tensor_dims", [(torch.float16, (64, 64))]) +def test_torch_to_ark_aliasing(dtype, tensor_dims): + ark.init() + # Initialize a PyTorch tensor + input_tensor = torch.randn(tensor_dims, dtype=dtype, device="cuda:0") + other_tensor = torch.randn(tensor_dims, dtype=dtype, device="cuda:0") + + input_ark_view = ark.Tensor.from_torch(input_tensor) + other_ark_view = ark.Tensor.from_torch(other_tensor) + + output = ark.add(input_ark_view, other_ark_view) + # Perform in place operations + input_tensor += other_tensor + other_tensor += input_tensor + expected_output = (input_tensor + other_tensor).cpu().numpy() + + runtime = ark.Runtime() + runtime.launch() + runtime.run() + output_host = output.to_numpy() + runtime.stop() + runtime.reset() + assert np.allclose(output_host, expected_output) diff --git a/python/unittest/test_runtime.py b/python/unittest/test_runtime.py index d91fd85c5..8c00b51f8 100644 --- a/python/unittest/test_runtime.py +++ b/python/unittest/test_runtime.py @@ -4,9 +4,11 @@ import ark +empty_plan = ark.Plan(None) + + def test_runtime_relaunch(): ark.init() - with ark.Runtime.get_runtime() as rt: assert rt.launched() == False rt.launch() @@ -16,3 +18,101 @@ def test_runtime_relaunch(): assert rt.launched() == False rt.launch() assert rt.launched() == True + + +def test_multiple_runtime_launch(): + ark.init() + num_runtimes = 5 + for i in range(num_runtimes): + rt = ark.Runtime.get_runtime(i) + assert rt.launched() == False + rt.launch(plan=empty_plan, device_id=i) + assert rt.launched() == True + for i in range(num_runtimes): + rt = ark.Runtime.get_runtime(i) + assert rt.launched() == True + ark.Runtime.delete_all_runtimes() + + +def test_stop_runtime(): + ark.init() + rt1 = ark.Runtime.get_runtime(1) + rt1.launch(plan=empty_plan, device_id=1) + rt2 = ark.Runtime.get_runtime(2) + rt2.launch(plan=empty_plan, device_id=2) + rt1.stop() + rt1.reset() + assert rt1.state == ark.Runtime.State.Init + assert rt2.state == ark.Runtime.State.LaunchedNotRunning + ark.Runtime.delete_all_runtimes() + + +def test_reset_runtime(): + ark.init() + rt1 = ark.Runtime.get_runtime(0) + rt1.launch(plan=empty_plan, device_id=1) + rt2 = ark.Runtime.get_runtime(1) + rt2.launch(plan=empty_plan, device_id=2) + rt1.reset() + assert rt1.launched() == False + assert rt2.launched() == True + rt1.launch(plan=empty_plan) + assert rt1.launched() == True + ark.Runtime.delete_all_runtimes() + + +def test_multiple_runtimes_complex(): + ark.init() + num_runtimes = 3 + runtime_list = [ark.Runtime.get_runtime(i) for i in range(num_runtimes)] + default_runtime = ark.Runtime.get_runtime() + runtime_list.append(default_runtime) + for i, rt in enumerate(runtime_list): + rt.launch(plan=empty_plan, device_id=i) + assert rt.launched() == True + runtime_list[0].stop() + assert runtime_list[0].state == ark.Runtime.State.LaunchedNotRunning + for rt in runtime_list[1:]: + assert rt.launched() == True + runtime_list[1].reset() + assert runtime_list[1].state == ark.Runtime.State.Init + assert runtime_list[0].state == ark.Runtime.State.LaunchedNotRunning + assert runtime_list[2].state == ark.Runtime.State.LaunchedNotRunning + runtime_list[1].launch(plan=empty_plan, device_id=1) + for rt in runtime_list: + assert rt.launched() == True + ark.Runtime.delete_all_runtimes() + + +def test_runtime_state_after_reset(): + ark.init() + rt = ark.Runtime.get_runtime() + rt.launch(plan=empty_plan) + rt.reset() + assert rt.launched() == False + assert rt.running() == False + ark.Runtime.delete_all_runtimes() + + +def test_see_runtime_statuses(): + ark.init() + num_runtimes = 3 + runtimes = [ark.Runtime.get_runtime(i) for i in range(num_runtimes)] + runtime_statuses = ark.Runtime.see_runtime_statuses() + assert len(runtime_statuses) == num_runtimes + for i in range(num_runtimes): + assert i in runtime_statuses + for i, rt in enumerate(runtimes): + assert runtime_statuses[i] == rt + ark.Runtime.delete_all_runtimes() + + +def test_multiple_runtimes_init(): + ark.init() + runtimes = [ark.Runtime.get_runtime(i) for i in range(3)] + for rt in runtimes: + assert rt.state == ark.Runtime.State.Init + ark.init() + runtimes = ark.Runtime.see_runtime_statuses() + assert len(runtimes) == 0 + ark.Runtime.delete_all_runtimes() diff --git a/python/unittest/unittest_common.py b/python/unittest/unittest_common.py new file mode 100644 index 000000000..0c385e89a --- /dev/null +++ b/python/unittest/unittest_common.py @@ -0,0 +1,28 @@ +# Copyright (c) Microsoft Corporation. +# Licensed under the MIT license. + +import pytest +import ark + + +def pytest_ark(need_torch: bool = False): + """ + Decorator for ARK unit tests. + """ + + def decorator(test_func): + if need_torch: + try: + import torch + except ImportError: + return pytest.mark.skip(reason="torch is not installed")( + test_func + ) + + def wrapper(*args, **kwargs): + ark.init() + test_func(*args, **kwargs) + + return wrapper + + return decorator diff --git a/third_party/CMakeLists.txt b/third_party/CMakeLists.txt index 96e442289..49251be74 100644 --- a/third_party/CMakeLists.txt +++ b/third_party/CMakeLists.txt @@ -40,6 +40,19 @@ if (NOT json_POPULATED) endif() set(JSON_INCLUDE_DIRS ${json_SOURCE_DIR}/include PARENT_SCOPE) +# DLPack +FetchContent_Declare( + dlpack + GIT_REPOSITORY https://github.com/dmlc/dlpack + GIT_TAG v0.8 + SOURCE_DIR ${CMAKE_CURRENT_SOURCE_DIR}/dlpack +) +FetchContent_GetProperties(dlpack) +if (NOT dlpack_POPULATED) + FetchContent_Populate(dlpack) +endif() +set(DLPACK_INCLUDE_DIRS ${dlpack_SOURCE_DIR}/include PARENT_SCOPE) + if(ARK_USE_CUDA) # Configure CUTLASS FetchContent_Declare( diff --git a/third_party/dlpack b/third_party/dlpack new file mode 160000 index 000000000..365b823ce --- /dev/null +++ b/third_party/dlpack @@ -0,0 +1 @@ +Subproject commit 365b823cedb281cd0240ca601aba9b78771f91a3