From 9da965bfa3f136196f10ebe512ed03c13a5c2f50 Mon Sep 17 00:00:00 2001 From: Christian Sarofeen Date: Tue, 1 Oct 2024 08:08:49 -0700 Subject: [PATCH 01/11] Duplicate much of FusionExecutor related to compilation into a new file. --- CMakeLists.txt | 1 + csrc/fusion_executor/compiled_kernel.cpp | 765 +++++++++++++++++++++++ csrc/fusion_executor/compiled_kernel.h | 360 +++++++++++ csrc/fusion_executor/executor.h | 8 +- csrc/fusion_executor/executor_params.h | 5 +- 5 files changed, 1133 insertions(+), 6 deletions(-) create mode 100644 csrc/fusion_executor/compiled_kernel.cpp create mode 100644 csrc/fusion_executor/compiled_kernel.h diff --git a/CMakeLists.txt b/CMakeLists.txt index 0611cfecb38..dac8f916887 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -126,6 +126,7 @@ list(APPEND NVFUSER_SRCS ${NVFUSER_SRCS_DIR}/expr_simplifier.cpp ${NVFUSER_SRCS_DIR}/fusion.cpp ${NVFUSER_SRCS_DIR}/fusion_executor/allocations.cpp + ${NVFUSER_SRCS_DIR}/fusion_executor/compiled_kernel.cpp ${NVFUSER_SRCS_DIR}/fusion_executor/executor.cpp ${NVFUSER_SRCS_DIR}/fusion_executor/executor_kernel_arg.cpp ${NVFUSER_SRCS_DIR}/fusion_executor/executor_params.cpp diff --git a/csrc/fusion_executor/compiled_kernel.cpp b/csrc/fusion_executor/compiled_kernel.cpp new file mode 100644 index 00000000000..2e7e445a044 --- /dev/null +++ b/csrc/fusion_executor/compiled_kernel.cpp @@ -0,0 +1,765 @@ +// clang-format off +/* + * SPDX-FileCopyrightText: Copyright (c) 2023-present NVIDIA CORPORATION & AFFILIATES. + * All rights reserved. + * SPDX-License-Identifier: BSD-3-Clause + */ +// clang-format on + +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include + +namespace nvfuser { + +namespace { + +static const char* defineIndexType(PrimDataType index_type) { + if (index_type == DataType::Int32) { + return "typedef int nvfuser_index_t;\n"; + } else if (index_type == DataType::Int) { + return "typedef int64_t nvfuser_index_t;\n"; + } else { + NVF_THROW("invalid indexing type: ", index_type); + } +} + +static const char* defineTypes() { + return R"( +using int8_t = signed char; +using uint8_t = unsigned char; +using int16_t = short int; +using uint16_t = unsigned short int; +using int32_t = int; +using uint32_t = unsigned int; +using int64_t = long long int; +using uint64_t = unsigned long long int; + +// Modified from cuda.h +struct TensorMap { + alignas(64) + uint64_t opaque[16]; +}; +)"; +} + +static const std::string& includeStdComplex() { + static std::string result = std::string(R"ESCAPE( +#ifdef __NVCC__ +#include +#endif // __NVCC__ +)ESCAPE"); + return result; +} + +// When executing nvFuser with: NVFUSER_EXTERNAL_SRC=file1.cu,file2.cu +// This function retrieves structured code from the specified files. +// The files should be comma-separated, and their order corresponds to the +// fusion_id order. If the provided number of files is fewer than the fusion +// segments, the function will resort to the available files in sequence +// and issue a warning. +std::string getStructuredCodeFromExternalFiles(const int64_t fusion_id) { + auto external_code_path = getNvFuserEnv("EXTERNAL_SRC"); + if (!external_code_path) { + return ""; + } + std::string all_external_code_paths(external_code_path); + if (all_external_code_paths.empty() || fusion_id < 1) { + return ""; + } + auto getExternalCodeFile = + [fusion_id](const std::string& input) -> std::string { + std::stringstream ss(input); + std::string token; + int64_t count = 0; + while (std::getline(ss, token, ',')) { + if (++count == fusion_id) { + return token; + } + } + debug() + << "Didn't find requested external source code. Will use generated code!\n" + << "Number of source code files should equal the number of fusion segments.\n" + << "External source code filenames should be delineated with commas, e.g.: file1.cu,file2.cu.\n"; + return ""; + }; + + std::string single_code_path = getExternalCodeFile(all_external_code_paths); + if (single_code_path.empty()) { + return ""; + } + std::ifstream cuda_src(single_code_path); + if (!cuda_src.is_open()) { + debug() << "Failed to open external source file: " << single_code_path + << std::endl; + return ""; + } + debug() << "--------> Compiling external CUDA code: " << single_code_path + << std::endl; + + std::stringstream buffer; + buffer << cuda_src.rdbuf(); + return buffer.str(); +} +} // namespace + +void CompiledKernel::compileFusion( + Fusion* fusion, + const KernelArgumentHolder& args, + const LaunchParams& launch_params, + CompileParams compile_params, + SchedulerType scheduler_type, + int64_t fusion_id, + int64_t concrete_id, + int64_t runtime_id, + int64_t group_id) { + FUSER_PERF_SCOPE("CompiledKernel::compileFusion"); + + NVF_ERROR( + !fusion->outputs().empty(), "No output found for this kernel, aborting."); + + // TODO: refactor the options_ passed through + options_.device = c10::Device(c10::DeviceType::CUDA, args.getDeviceIndex()); + + // NOTE: Profiling needs to be started below the isExpressionEvaluated query + // given the conditional can exit early from compilation. + if (isProfilerEnabled()) { + NVF_CHECK( + group_id >= 0, + "An invalid segment id is passed to FusionProfiler!:", + group_id); + FusionProfiler::segment(group_id).startCompile(args.getDeviceIndex()); + } + + for (auto out : fusion->outputs()) { + const auto logical_domain = out->as()->getLogicalDomain(); + // walking through outputs to see if output shapes are dependent on + // non-tensor inputs. For which case, we should have disabled output + // allocation, since the caching id only looks at tensor shapes. + // See issue https://github.com/csarofeen/pytorch/issues/2002 + std::vector output_extents; + for (const auto id : logical_domain) { + Val* extent = nullptr; + if (id->isReduction() || id->isStride() || id->isDeviceDim()) { + continue; + } else if (id->isBroadcast() && id->hasExpandedExtent()) { + extent = id->expandedExtent(); + } else { + extent = id->extent(); + } + output_extents.emplace_back(extent); + } + auto dependencies = InputsOf::outputs(output_extents); + if (std::any_of(dependencies.begin(), dependencies.end(), [](Val* val) { + return val->isFusionInput(); + })) { + // TODO: parameter cache is too big a hammer here. We should consider + // separate the caching logic of output sizes & launch params. Since + // output size dependency should only invalidate the output sizes + disable_parameter_cache_ = true; + break; + } + } + + if (isDebugDumpEnabled(DebugDumpOption::FusionIr)) { + fusion->print(); + } else if (isDebugDumpEnabled(DebugDumpOption::FusionIrMath)) { + fusion->printMath(); + } + + //! Force index_type to int and disable magic zero if we detect that the + //! kernel contains any TMA memory operations. + const std::vector& exprs = fusion->exprs(); + bool has_cp_async_bulk = std::any_of(exprs.begin(), exprs.end(), [](Expr* e) { + return ir_utils::isCpAsyncBulk(e); + }); + + // Disable magic zero if there are any TMA operations in Fusion + if (has_cp_async_bulk) { + compile_params.enable_magic_zero = false; + } + + // Set the index type of compile params if not already set. If set, + // make sure the compile param type is valid with the given kernel + // arguments. + auto arg_index_type = args.getSmallestIndexTypeOfArguments(); + if (compile_params.index_type.has_value()) { + // If the int32 compilation is requested, but the arguments demand + // int64, that's an error + NVF_ERROR( + !(compile_params.index_type.value() == PrimDataType::Int32 && + arg_index_type == PrimDataType::Int), + "Compilation with int32 is requested but int64 is required for the arguments"); + NVF_ERROR( + !has_cp_async_bulk || + (compile_params.index_type.value() == PrimDataType::Int32), + "Compilation with int64 is requested but int32 is required because ", + "of TMA operations."); + + } else if (arg_index_type == PrimDataType::Int) { + // If the given compile option doesn't specify the index type, and + // the arguments require 64-bit indexing, we need to use 64-bit + // indexing. Note that if the arg type is 32-bit, it doesn't mean + // it's safe to use 32-bit for the whole kernel, so unless it's + // specified through CompileParams, we do not use 32-bit indexing. + compile_params.index_type = arg_index_type; + NVF_ERROR( + !has_cp_async_bulk, + "Compilation with int64 is required based on input arguments, but ", + "int32 is required because of TMA operations."); + } else if (has_cp_async_bulk) { + // TMA operations require 32-bit indexing. + compile_params.index_type = PrimDataType::Int32; + } + + c10::DeviceGuard dg(options_.device); + + NVF_ERROR( + options_.device.is_cuda(), "Provided device to CUDA fuser is the CPU."); + auto properties = at::cuda::getDeviceProperties(options_.device.index()); + // TODO: These properties should be set as part of the constructor so that it + // can be const + warp_size_ = properties->warpSize; + + lowered_ = std::make_unique(fusion, compile_params); + for (const auto& hook : lowering_hooks_) { + hook(lowered_.get()); + } + lowered_->run(); + + kir::Kernel* kernel = lowered_->kernel(); + + for (const auto& hook : post_lowering_hooks_) { + hook(kernel); + } + createKernelId(scheduler_type, fusion_id, concrete_id, runtime_id, group_id); + setUsedTVs(); + + if (isDebugDumpEnabled(DebugDumpOption::KernelIr)) { + kernel->print(); + } + + if (isDebugDumpEnabled(DebugDumpOption::BankConflictInfo)) { + auto bank_conflict_info = getBankConflictInfo(kernel); + if (bank_conflict_info.empty()) { + debug() << "===== No bank confliction =====" << std::endl; + } else { + debug() << "======= Bank confliction =======" << std::endl; + for (auto info : bank_conflict_info) { + debug() << "Expr: " << info.first->toString() << std::endl; + auto conflict = info.second; + if (conflict.first > 1) { + debug() << "input conflict: " << conflict.first << " way, "; + } + if (conflict.second > 1) { + debug() << "output conflict: " << conflict.second << " way"; + } + debug() << std::endl; + } + debug() << "================================" << std::endl; + } + } + + kernel_code_ = codegen::generateCudaKernel(kernel, kernelName()); + + // If NVFUSER_EXTERNAL_SRC is set, utilize the external source code. + // If the loaded external source code is empty, revert to the default codegen. + // The external_structured_code is moved to structured_code and explicitly + // cleared to avoid use-after-move scenarios. + // Note: we index these with getGlobalFusionCount() instead of fusion_id_ in + // order to match the numbering of files output with + // NVFUSER_DUMP=cuda_to_file + auto structured_code = + getStructuredCodeFromExternalFiles(getGlobalFusionCount()); + if (structured_code.empty()) { + structured_code = getStructuredCode(); + } + + const kir::KernelSummary& kernel_summary = kernel->summary(); + + // TODO: this replicates the target GPU version computation from + // executor_utils. + std::pair target_arch; + bool compile_to_sass = false; + executor_utils::queryTargetGPUVersion( + properties, + std::ref(target_arch.first), + std::ref(target_arch.second), + compile_to_sass); + + NVF_CHECK( + target_arch >= kernel_summary.min_device_version, + "Target compute capability is ", + target_arch.first, + ".", + target_arch.second, + " but this fusion requires at least ", + kernel_summary.min_device_version.first, + ".", + kernel_summary.min_device_version.second, + ". Reason: ", + kernel_summary.min_device_version_reason); + + // We currently shouldn't allocate any more shared mem + // tensors statically but could keep this path if + // needed in later development. + if (!kernel_summary.static_smem_allocations.empty()) { + ExpressionEvaluator static_evaluator; + const auto static_smem_size = computeSharedMemory( + static_evaluator, + kernel_summary.static_smem_allocations, + kernel->indexType()); + NVF_ERROR( + static_smem_size < max_static_smem_, + "The static shared memory allocation is larger than available memory."); + } + + if (kernel_summary.has_dynamic_local_memory_allocations) { + std::stringstream ss; + ss << "Allocations must be based on constant integers for local memory. However, found: "; + for (auto alloc : kernel_summary.dynamic_lmem_allocations) { + ss << alloc->buffer()->toString() << ", "; + } + ss << " have dynamic allocations but are placed in local memory."; + NVF_THROW(ss.str()); + } + + NVF_ERROR( + launch_params.nThreads() > 0, "launch param inferred block size < 0"); + + // TODO: high water mark should be computed via occupancy API after + // compilation. + + // Basically setting high water mark as 1 when we don't provide args for + // compilation, it will just generate a kernel that gets ditched at the first + // run - not great. We should have better heuristics. + block_size_high_water_mark_ = + std::max(launch_params.nThreads(), block_size_high_water_mark_); + maxrregcount_high_water_mark_ = compile_params.maxrregcount; + compiled_kernel_ = executor_utils::getCompiledKernel( + kernel_code_, + structured_code, + kernelName(), + kernel_id_, + compile_params, + launch_params.nThreads()); + + NVF_ERROR(validKernelId(), "Invalid kernel id for CompiledKernel."); + + if (isDebugDumpEnabled(DebugDumpOption::Sass)) { + debug() << disassembledKernelSASS() << std::endl; + } + if (isProfilerEnabled()) { + FusionProfiler::segment(group_id).stopCompile(); + } +} + +std::string CompiledKernel::getStructuredCode( + const std::string& kernel_str, + PrimDataType index_type) const { + // generating cuda code; + std::string code = ""; + code += includeStdComplex(); + code += std::string("namespace {\n") + defineTypes() + + defineIndexType(index_type) + executor_utils::kernelPreamble() + + kernel_str + "}\n"; + + if (isDebugDumpEnabled(DebugDumpOption::CudaKernel)) { + debug() << "\n======= Codegen output for kernel: " << kernelName() + << " =======\n\n" + << kernel_str << "\n======================================\n\n"; + } else if (isDebugDumpEnabled(DebugDumpOption::CudaFull)) { + debug() << "\n======= Codegen output for kernel: " << kernelName() + << " =======\n\n" + << code << "\n======================================\n\n"; + } + if (isDebugDumpEnabled(DebugDumpOption::CudaToFile)) { + std::stringstream file_name; + file_name << "__tmp_kernel_" << kernel_id_ << ".cu"; + debug() << "PRINTING: " << file_name.str() << std::endl; + std::ofstream out(file_name.str()); + out << code << std::endl; + out.close(); + } + + return code; +} + +std::string CompiledKernel::getStructuredCode() const { + return getStructuredCode(kernelString(), kernel()->indexType()); +} + +void CompiledKernel::compileRtc( + const std::string& code, + const std::string& name, + bool structured, + PrimDataType index_type) { + FUSER_PERF_SCOPE("CompiledKernel::compileRtc"); + NVF_ERROR( + index_type == PrimDataType::Int || index_type == PrimDataType::Int32 || + "Invalid index type: ", + index_type); + + createKernelId(); + + std::string scode; + if (!structured) { + scode = getStructuredCode(code, index_type); + } else { + scode = code; + } + compiled_kernel_ = + executor_utils::getCompiledKernel(std::nullopt, scode, name, kernel_id_); +} + +float CompiledKernel::runRtc( + const LaunchParams& launch_params, + const std::vector& args, + PrimDataType index_type) { + FUSER_PERF_SCOPE("CompiledKernel::runRtc"); + + c10::DeviceGuard dg(options_.device); + auto stream = at::cuda::getCurrentCUDAStream(); + + cudaEvent_t start_event = {}; + cudaEvent_t finish_event = {}; + + NVFUSER_CUDA_RT_SAFE_CALL(cudaEventCreate(&start_event)); + NVFUSER_CUDA_RT_SAFE_CALL(cudaEventCreate(&finish_event)); + + NVFUSER_CUDA_RT_SAFE_CALL(cudaEventRecord(start_event, stream)); + + std::vector> data; + std::vector pointers; + + for (const auto& input : args) { + auto dtype = + std::get(aten_to_data_type(input.scalar_type()).type); + DataType metadata_type = globalTensorMetaData(dtype, input.dim()); + + std::shared_ptr struct_ = std::make_shared(); + TensorMetaData* metadata = (TensorMetaData*)struct_.get(); + metadata->dtype = dtype; + metadata->data = input.data_ptr(); + metadata->logical_size = input.sizes(); + metadata->logical_stride = input.strides(); + metadata->alloc_size = input.sizes(); + metadata->alloc_stride = input.strides(); + + data.emplace_back(polymorphicValueToBytes( + PolymorphicValue(std::move(struct_)), metadata_type, index_type)); + pointers.emplace_back(data.back().data()); + } + + NVFUSER_CUDA_SAFE_CALL(cuLaunchKernel( + compiled_kernel_->function, + launch_params.gdimx(), + launch_params.gdimy(), + launch_params.gdimz(), + launch_params.bdimx(), + launch_params.bdimy(), + launch_params.bdimz(), + launch_params.smem(), + stream, + pointers.data(), + nullptr)); + + NVFUSER_CUDA_RT_SAFE_CALL(cudaEventRecord(finish_event, stream)); + NVFUSER_CUDA_RT_SAFE_CALL(cudaEventSynchronize(start_event)); + NVFUSER_CUDA_RT_SAFE_CALL(cudaEventSynchronize(finish_event)); + + float kernel_time_ms = 0; + NVFUSER_CUDA_RT_SAFE_CALL( + cudaEventElapsedTime(&kernel_time_ms, start_event, finish_event)); + NVFUSER_CUDA_RT_SAFE_CALL(cudaEventDestroy(start_event)); + NVFUSER_CUDA_RT_SAFE_CALL(cudaEventDestroy(finish_event)); + + return kernel_time_ms; +} + +// flatbuffers::Offset CompiledKernel::serialize( +// flatbuffers::FlatBufferBuilder& builder) const { +// // See table definition for CompiledKernel in serde/fusion_cache.fbs +// using fb_executor_entry = flatbuffers::Offset; + +// // Separate unordered_map for executor_entry_lookup into key and value +// // vectors. The key value is the cache_id value in the +// KernelArgumentHolder. std::vector executor_entry_lookup_keys_fb; +// std::vector executor_entry_lookup_values_fb; +// for (const auto& [key, value] : executor_entry_lookup_) { +// executor_entry_lookup_keys_fb.push_back(key); +// executor_entry_lookup_values_fb.push_back(serialize(builder, value)); +// } + +// // When compilation is skipped, avoid serializing cubin because it doesn't +// // exist. The remaining fields are also not necessary in this case. +// if (!hasCompiledKernel()) { +// return serde::CreateCompiledKernelDirect(builder); +// } + +// return serde::CreateCompiledKernelDirect( +// builder, +// device_smem_limit_, +// block_size_high_water_mark_, +// maxrregcount_high_water_mark_, +// warp_size_, +// toUnderlying(scheduler_type_), +// fusion_id_, +// concrete_id_, +// runtime_id_, +// group_id_, +// kernel_code_.c_str(), +// &executor_entry_lookup_keys_fb, +// &executor_entry_lookup_values_fb, +// toUnderlying(kernel()->indexType()), +// serialize(builder, compiled_kernel_.get())); +// } + +// flatbuffers::Offset CompiledKernel::serialize( +// flatbuffers::FlatBufferBuilder& builder, +// const executor_utils::CompiledKernel* compiled_kernel) const { +// NVF_ERROR( +// compiled_kernel_ != nullptr && +// (!compiled_kernel->cubin.empty() || !compiled_kernel->ptx.empty()), +// "Expected compiled cuda kernel before serializing CompiledKernel."); + +// auto fb_kernel_name = builder.CreateString(compiled_kernel->kernel_name); +// auto fb_compile_args = builder.CreateString(compiled_kernel->compile_args); + +// flatbuffers::Offset> fb_cubin = 0; +// flatbuffers::Offset fb_cubin_filename = 0; +// if (!compiled_kernel->cubin.empty()) { +// uint8_t* cubin_ptr = nullptr; +// fb_cubin = builder.CreateUninitializedVector( +// compiled_kernel->cubin.size(), &cubin_ptr); +// std::copy( +// compiled_kernel->cubin.begin(), +// compiled_kernel->cubin.end(), +// cubin_ptr); +// fb_cubin_filename = +// builder.CreateString(compiled_kernel->cubin_filename); +// } + +// flatbuffers::Offset> fb_ptx = 0; +// flatbuffers::Offset fb_ptx_filename = 0; +// if (!compiled_kernel->ptx.empty()) { +// uint8_t* ptx_ptr = nullptr; +// fb_ptx = builder.CreateUninitializedVector( +// compiled_kernel->ptx.size(), &ptx_ptr); +// std::copy( +// compiled_kernel->ptx.begin(), compiled_kernel->ptx.end(), ptx_ptr); +// fb_ptx_filename = builder.CreateString(compiled_kernel->ptx_filename); +// } + +// serde::CudaKernelBuilder ckb(builder); +// ckb.add_cubin(fb_cubin); +// ckb.add_cubin_filename(fb_cubin_filename); +// ckb.add_ptx(fb_ptx); +// ckb.add_ptx_filename(fb_ptx_filename); +// ckb.add_kernel_name(fb_kernel_name); +// ckb.add_compile_args(fb_compile_args); +// ckb.add_block_size(compiled_kernel->block_size); +// return ckb.Finish(); +// } + +// std::unique_ptr& CompiledKernel:: +// evaluatorPrecomputedValues() { +// if (!evaluator_precomputed_values_) { +// evaluator_precomputed_values_ = +// std::make_unique(lowered_->kernel()); +// } +// return evaluator_precomputed_values_; +// } + +// void CompiledKernel::deserialize( +// const serde::CompiledKernel* buffer, +// Fusion* fusion, +// int8_t device_index, +// CompileParams compile_params, +// SchedulerType heuristic, +// int64_t fusion_id, +// int64_t concrete_id, +// int64_t runtime_id, +// int64_t group_id) { +// // See table definition for CompiledKernel in serde/fusion_cache.fbs + +// NVF_ERROR(buffer != nullptr, "serde::CompiledKernel is nullptr."); + +// // TODO Should we set fusion_id, concrete_id, runtime_id, and group_id when +// we +// // skip compilation? +// if (isExpressionEvaluated(fusion)) { +// fusion_ = std::make_unique(*fusion); +// NVF_ERROR(!hasCompiledKernel(), "Failed to deserialize CompiledKernel"); +// return; +// } + +// NVF_ERROR( +// fusion_id == buffer->fusion_id(), +// "Expected given fusion_id to match serde fusion_id."); +// NVF_ERROR( +// concrete_id == buffer->concrete_id(), +// "Expected given concrete_id to match serde concrete_id."); +// NVF_ERROR( +// runtime_id == buffer->runtime_id(), +// "Expected given runtime_id to match serde runtime_id."); +// NVF_ERROR( +// group_id == buffer->group_id(), +// "Expected given group_id to match serde group_id."); +// NVF_ERROR( +// toUnderlying(heuristic) == buffer->heuristic(), +// ": ", +// toUnderlying(heuristic), +// " vs ", +// buffer->heuristic()); + +// // Initialize CompileOptions +// options_.device = c10::Device(c10::DeviceType::CUDA, device_index); +// c10::DeviceGuard dg(options_.device); + +// // Initialize internal fields +// device_smem_limit_ = buffer->device_smem_limit(); +// block_size_high_water_mark_ = buffer->block_size_high_water_mark(); +// maxrregcount_high_water_mark_ = buffer->maxrregcount_high_water_mark(); +// warp_size_ = buffer->warp_size(); +// kernel_code_ = buffer->kernel_code()->str(); + +// // KernelDB query checks kernel_code string and compile_params before +// // copying cubin. +// compile_params.index_type = serde::mapToNvfuserDtype(buffer->index_type()); +// compile_params.maxrregcount = maxrregcount_high_water_mark_; + +// // Get lowered fusion +// lowered_ = std::make_unique(fusion, compile_params); +// lowered_->run(); + +// // Replace integers that are tensor sizes by named scalars like +// "T0.size[0]" createKernelId( +// heuristic, +// buffer->fusion_id(), +// buffer->concrete_id(), +// buffer->runtime_id(), +// buffer->group_id()); +// setUsedTVs(); + +// // GlobalBufferInfo requires lowered kernel before deserialization +// for (auto idx : c10::irange(buffer->executor_entry_lookup_keys()->size())) +// { +// executor_entry_lookup_.emplace( +// buffer->executor_entry_lookup_keys()->Get(idx), +// deserialize(buffer->executor_entry_lookup_values()->Get(idx))); +// } + +// compiled_kernel_ = executor_utils::getCompiledKernel( +// buffer->compiled_kernel(), compile_params); + +// NVF_ERROR(hasCompiledKernel(), "Failed to deserialize CompiledKernel"); +// } + +void CompiledKernel::setUsedTVs() { + auto used_vals = fusion()->usedMathVals(); + auto used_tvs = ir_utils::filterByType(used_vals); + used_tvs_.clear(); + used_tvs_.insert(used_tvs_.begin(), used_tvs.begin(), used_tvs.end()); +} + +namespace { +void validateCooperativeLaunch( + CUfunction kernel, + const LaunchParams& launch_params, + int64_t device_index) { + int num_blocks_per_SM = -1; + auto block_size = + launch_params.bdimx() * launch_params.bdimy() * launch_params.bdimz(); + NVFUSER_CUDA_SAFE_CALL(cuOccupancyMaxActiveBlocksPerMultiprocessor( + &num_blocks_per_SM, + kernel, + (int)block_size, + (size_t)launch_params.smem())); + + auto grid_size = + launch_params.gdimx() * launch_params.gdimy() * launch_params.gdimz(); + auto max_active_blocks = num_blocks_per_SM * + at::cuda::getDeviceProperties((c10::DeviceIndex)device_index) + ->multiProcessorCount; + NVF_ERROR( + (int64_t)(max_active_blocks) >= grid_size, + "Wanted to launch a cooperative kernel, however the number of blocks is greater than ", + "what can be resident on the GPU at once. Need: ", + grid_size, + " (", + launch_params.gdimx(), + " * ", + launch_params.gdimy(), + " * ", + launch_params.gdimz(), + ") but limited to ", + num_blocks_per_SM, + " * ", + at::cuda::getDeviceProperties(device_index)->multiProcessorCount); +} +} // namespace + +void CompiledKernel::recompileKernel( + const LaunchParams& new_launch_params, + const CompileParams& new_compile_params) { + FUSER_PERF_SCOPE("CompiledKernel::runFusion::recompileKernel"); + + const auto structured_code = getStructuredCode(); + block_size_high_water_mark_ = new_launch_params.nThreads(); + maxrregcount_high_water_mark_ = new_compile_params.maxrregcount; + + compiled_kernel_ = executor_utils::getCompiledKernel( + kernel_code_, + structured_code, + kernelName(), + kernel_id_, + new_compile_params, + block_size_high_water_mark_); + + if (kernel()->summary().has_cooperative_grid_reduction) { + // We need to increase shared memory before kernel launch, but also before + // calling into `validateCooperativeLaunch`! + // So we need to do it there before calling into the validation, to avoid + // false positives + validateCooperativeLaunch( + compiled_kernel_->function, new_launch_params, options_.device.index()); + } +} + +} // namespace nvfuser diff --git a/csrc/fusion_executor/compiled_kernel.h b/csrc/fusion_executor/compiled_kernel.h new file mode 100644 index 00000000000..3959049b9c4 --- /dev/null +++ b/csrc/fusion_executor/compiled_kernel.h @@ -0,0 +1,360 @@ +// clang-format off +/* + * SPDX-FileCopyrightText: Copyright (c) 2023-present NVIDIA CORPORATION & AFFILIATES. + * All rights reserved. + * SPDX-License-Identifier: BSD-3-Clause + */ +// clang-format on +#pragma once +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +// #include +#include +#include + +#include + +#include + +namespace nvfuser { + +// TODO: Should this actually be in launch params? +struct CompileOptions { + c10::Device device = c10::Device(c10::DeviceType::CUDA, 0); +}; + +class CompiledKernel : public NonCopyable { + public: + // NVF_API was added for nvfuser_extension. See examples/sinh_extension. + NVF_API CompiledKernel() = default; + + //! To compile a fusion with the 32-bit index type, CompileParams + //! must be passed in. There used to be an index type associated + //! with KernelArgumentHolder, but it is no longer the case. + NVF_API void compileFusion( + Fusion* fusion, + const KernelArgumentHolder& args, + const LaunchParams& launch_constraints, + CompileParams compile_params, + SchedulerType sceduler_type = SchedulerType::None, + int64_t fusion_id = 0, + int64_t concrete_id = 0, + int64_t runtime_id = 0, + int64_t group_id = 0); + + // TODO: merge it with the overload above. + //! This API is merely here so we don't have to go back and update all cpp + //! tests. + void compileFusion( + Fusion* fusion, + const at::ArrayRef& inputs = {}, + const LaunchParams& launch_constraints = LaunchParams(), + CompileParams compile_params = CompileParams()) { + KernelArgumentHolder args = + KernelArgumentHolder::createKernelArgumentHolder(inputs); + compileFusion(fusion, args, launch_constraints, compile_params); + } + + //! Used by user defined schedules in python frontend + void compileFusion( + Fusion* fusion, + const at::ArrayRef& inputs, + int64_t fusion_id, + int64_t concrete_id) { + KernelArgumentHolder args = + KernelArgumentHolder::createKernelArgumentHolder(inputs); + compileFusion( + fusion, + args, + LaunchParams(), + CompileParams(), + SchedulerType::None, + fusion_id, + concrete_id); + } + + // Register a lowering hooks that are called to modify the GpuLower object + // before running lowering passes. The main use case is for unit tests to + // modify the lowering process. + void registerLoweringHook(std::function hook) { + lowering_hooks_.push_back(std::move(hook)); + } + + // Register a post-lowering hooks that are called to modify the kernel after + // lowering. The main use case is for unit tests to modify the kernel. + void registerPostLoweringHook(std::function hook) { + post_lowering_hooks_.push_back(std::move(hook)); + } + + // Function to query whether compilation was attempted for a `CompiledKernel` + bool isCompiled() const { + int num_compiled_artifacts = (fusion_ != nullptr) + (lowered_ != nullptr); + NVF_ERROR(num_compiled_artifacts <= 1); + return num_compiled_artifacts == 1; + }; + + // function to query whether a `CompiledKernel` has a compiled kernel to + // execute + bool hasCompiledKernel() const { + if (compiled_kernel_ != nullptr) { + NVF_ERROR(compiled_kernel_->function != nullptr); + NVF_ERROR( + fusion_ == nullptr, + "fusion_ should only be initialized when using expression evaluator."); + } + return validKernelId() && lowered_ && compiled_kernel_ != nullptr; + }; + + using ExecutorCompileTimeInfoCache = + executor_utils::caching::ExecutorCompileTimeInfoCache; + + kir::Kernel* kernel() const { + NVF_ERROR(lowered_); + return lowered_->kernel(); + } + + Fusion* fusion() const { + NVF_ERROR(isCompiled()); + if (fusion_ != nullptr) { + return fusion_.get(); + } + if (lowered_ != nullptr) { + return lowered_->kernel()->as(); + } + NVF_THROW("unreachable because of the isCompiled check"); + } + + const ThreadPredicateMap& threadPredMap() const { + return lowered_->threadPredMap(); + } + + //! get register spills (load + store) of the compiled kernel + int getKernelRegisterSpills() const { + return compiled_kernel_->register_spills; + } + + //! Returns the string of the compiled kernel + NVF_API std::string kernelString() const { + NVF_ERROR(!kernel_code_.empty(), "Kernel code not generated"); + return kernel_code_; + } + + // Add preamble and wrap in namespace + NVF_API std::string getStructuredCode( + const std::string& kernel, + PrimDataType index_type) const; + + NVF_API std::string getStructuredCode() const; + + //! Returns a const reference to the latest compiled kernel. + const executor_utils::CompiledKernel& compiledKernel() const { + return *compiled_kernel_; + } + + //! Returns the disassembled latest compiled binary + NVF_API std::string disassembledBinary( + const std::string& nvdisasm_args = "") const { + return executor_utils::disassembleBinary( + compiled_kernel_->cubin, nvdisasm_args); + } + + //! Returns the disassembled latest compiled binary + NVF_API std::string disassembledKernelSASS() const { + return executor_utils::disassembleBinary( + compiled_kernel_->cubin, "-fun 1 -c"); + } + + static void setGlobalFusionCount(int64_t new_fusion_count) { + global_fusion_count_.store(new_fusion_count); + } + + static int64_t getGlobalFusionCount() { + return global_fusion_count_.load(); + } + + int64_t groupId() const { + return group_id_; + } + void setGroupId(int64_t gid) { + group_id_ = gid; + } + + bool validKernelId() const { + return !kernel_id_.empty(); + } + + void createKernelId( + SchedulerType scheduler_type = SchedulerType::None, + int64_t fusion_id = 0, + int64_t concrete_id = 0, + int64_t runtime_id = 0, + int64_t group_id = 0) { + NVF_ERROR(fusion_id > -1, "Invalid fusion_id."); + NVF_ERROR(concrete_id > -1, "Invalid concrete_id."); + NVF_ERROR(runtime_id > -1, "Invalid runtime_id."); + NVF_ERROR(group_id > -1, "Invalid group_id"); + + scheduler_type_ = scheduler_type; + fusion_id_ = fusion_id; + concrete_id_ = concrete_id; + runtime_id_ = runtime_id; + group_id_ = group_id; + ++global_fusion_count_; + + std::stringstream ss; + if (isOptionEnabled(EnableOption::StaticFusionCount)) { + ss << global_fusion_count_.load(); + } else { + ss << toString(scheduler_type_); + ss << "_f" << fusion_id_; + ss << "_c" << concrete_id_; + ss << "_r" << runtime_id_; + ss << "_g" << group_id_; + } + kernel_id_ = ss.str(); + } + + std::string kernelName() const { + NVF_ERROR(!kernel_id_.empty(), "Invalid kernel name for fusion executor."); + std::stringstream ss; + ss << "nvfuser_" << kernel_id_; + return ss.str(); + } + + //! Internal tests only. Compiles CUDA code with NVRTC directly from + //! string. This util provides a path to test runtime code, i.e. the resource + //! strings. + // TODO: Consider split out compileRtc and runRtc to a different + //! class. Not much code is shared with the normal path. + NVF_API void compileRtc( + const std::string& code, + const std::string& name, + bool structured, + PrimDataType index_type); + + //! Internal tests only. Runs the compiled CUDA kernel from + //! compileRtc. Return the elapsed milliseconds. + NVF_API float runRtc( + const LaunchParams& launch_params, + const std::vector& args, + PrimDataType indextype); + + //! Internal knob used for debugging/profiling only + void disableLaunchParamCache() { + disable_parameter_cache_ = true; + } + + // //! Serialize Fusion Executor using flatbuffers + // flatbuffers::Offset serialize( + // flatbuffers::FlatBufferBuilder& builder) const; + + // //! Serialize CompiledKernel using flatbuffers + // flatbuffers::Offset serialize( + // flatbuffers::FlatBufferBuilder& builder, + // const executor_utils::CompiledKernel* kernel) const; + + // //! Deserialize Fusion Executor using flatbuffers + // void deserialize( + // const serde::CompiledKernel* buffer, + // Fusion* fusion, + // int8_t device_index, + // CompileParams compile_params, + // SchedulerType scheduler_type, + // int64_t fusion_id, + // int64_t concrete_id, + // int64_t runtime_id, + // int64_t group_id); + + private: + void setUsedTVs(); + + const std::vector& getUsedTVs() const { + return used_tvs_; + }; + + // Recompile the kernel if the number of threads in the block has increased + // or maxrregcount has changed + void recompileKernel( + const LaunchParams& new_launch_params, + const CompileParams& new_compile_params); + + private: + CompileOptions options_; + + // Assuming sm70 or above: + // limit of statically allocated smem is 48 KB: + // See: + // https://docs.nvidia.com/cuda/cuda-c-programming-guide/index.html#shared-memory-7-x + // https://docs.nvidia.com/cuda/cuda-c-programming-guide/index.html#shared-memory-8-x + const int64_t max_static_smem_ = 48 << 10; + + int64_t warp_size_ = 0; + std::unique_ptr compiled_kernel_; + + // TensorViews actually used in the kernel. + std::vector used_tvs_; + + // ID of fusion in python frontend fusion cache, which maps to a single + // CompiledKernelCache. + int64_t fusion_id_ = -1; + + // ID of (device, concrete_info) key in CompiledKernelCache + int64_t concrete_id_ = -1; + + // ID of FusionKernelRuntime given (device, concrete_info) key + int64_t runtime_id_ = -1; + + // ID of segment in FusionKernelRuntime + int64_t group_id_ = -1; + + inline static std::atomic global_fusion_count_; + + // Scheduling Heuristic for this Fusion + SchedulerType scheduler_type_ = SchedulerType::None; + + // Kernel name for fusion executor + std::string kernel_id_; + + std::unique_ptr lowered_; + + // Initialized for non-compiled fusions + std::unique_ptr fusion_; + + // Track the block size this kernel was compiled with. If the block size + // increases, recompile to adjust maxregister count. + int64_t block_size_high_water_mark_ = 1; + int64_t maxrregcount_high_water_mark_ = 255; + + // Profiling support: disable caching of launch params and output allocation + // output allocation is also disable when output sizes are dependent on + // runtime scalar inputs, such as for the case of tensor factory. see + // https://github.com/csarofeen/pytorch/issues/2002 + bool disable_parameter_cache_ = false; + + // Profiling support: kept copy of the cuda kernel + std::string kernel_code_; + + // Lowering hooks that are called after the GpuLower instance is created + // before running lowering passes. + // The main use case is for unit tests to modify the lowering process. + std::vector> lowering_hooks_; + + // Post-lowering hooks that are called to modify the kernel after lowering. + // The main use case is for unit tests to modify the kernel. + std::vector> post_lowering_hooks_; + + Communicator* communicator_; +}; + +} // namespace nvfuser diff --git a/csrc/fusion_executor/executor.h b/csrc/fusion_executor/executor.h index 8ab4500f06d..788041f3289 100644 --- a/csrc/fusion_executor/executor.h +++ b/csrc/fusion_executor/executor.h @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include @@ -29,11 +30,6 @@ namespace nvfuser { -// TODO: Should this actually be in launch params? -struct CompileOptions { - c10::Device device = c10::Device(c10::DeviceType::CUDA, 0); -}; - class FusionExecutor : public NonCopyable { public: // NVF_API was added for nvfuser_extension. See examples/sinh_extension. @@ -469,6 +465,8 @@ class FusionExecutor : public NonCopyable { void resetCompiledKernelProperties(); private: + CompiledKernel compiled_kernel_2_; + CompileOptions options_; //! Absolute limit of all available shared mem space from cudaDeviceProp diff --git a/csrc/fusion_executor/executor_params.h b/csrc/fusion_executor/executor_params.h index b8fcc4f7277..5b2d3a01e02 100644 --- a/csrc/fusion_executor/executor_params.h +++ b/csrc/fusion_executor/executor_params.h @@ -11,6 +11,8 @@ #include #include +#include + #include namespace nvfuser { @@ -21,6 +23,7 @@ struct CompileParams { bool enable_magic_zero = true; // if true, save ptxas info to compile log and check for register spilling bool enable_ptxas_verbose = false; + std::optional device = std::nullopt; bool operator==(const CompileParams& other) const { // Disallow comparison if the index type is nullopt @@ -32,7 +35,7 @@ struct CompileParams { "cannot compare as the other index type is not defined"); return index_type == other.index_type && maxrregcount == other.maxrregcount && - enable_magic_zero == other.enable_magic_zero; + enable_magic_zero == other.enable_magic_zero && device == other.device; } bool operator!=(const CompileParams& other) const { From 1009f403a9131e21588040b99ac97aefd763437b Mon Sep 17 00:00:00 2001 From: Christian Sarofeen Date: Tue, 1 Oct 2024 18:03:36 -0700 Subject: [PATCH 02/11] Duplicating much of FusionExecutor now building. --- csrc/fusion_executor/compiled_kernel.cpp | 53 +--- csrc/fusion_executor/compiled_kernel.h | 157 +++++++--- csrc/fusion_executor/executor.cpp | 350 +++++++---------------- csrc/fusion_executor/executor.h | 306 +++++++++++++++++--- tests/cpp/test_memory.cpp | 4 +- 5 files changed, 494 insertions(+), 376 deletions(-) diff --git a/csrc/fusion_executor/compiled_kernel.cpp b/csrc/fusion_executor/compiled_kernel.cpp index 2e7e445a044..fc99e7ac42b 100644 --- a/csrc/fusion_executor/compiled_kernel.cpp +++ b/csrc/fusion_executor/compiled_kernel.cpp @@ -140,7 +140,7 @@ std::string getStructuredCodeFromExternalFiles(const int64_t fusion_id) { void CompiledKernel::compileFusion( Fusion* fusion, - const KernelArgumentHolder& args, + c10::Device device, const LaunchParams& launch_params, CompileParams compile_params, SchedulerType scheduler_type, @@ -153,8 +153,7 @@ void CompiledKernel::compileFusion( NVF_ERROR( !fusion->outputs().empty(), "No output found for this kernel, aborting."); - // TODO: refactor the options_ passed through - options_.device = c10::Device(c10::DeviceType::CUDA, args.getDeviceIndex()); + options_.device = device; // NOTE: Profiling needs to be started below the isExpressionEvaluated query // given the conditional can exit early from compilation. @@ -163,7 +162,7 @@ void CompiledKernel::compileFusion( group_id >= 0, "An invalid segment id is passed to FusionProfiler!:", group_id); - FusionProfiler::segment(group_id).startCompile(args.getDeviceIndex()); + FusionProfiler::segment(group_id).startCompile(device.index()); } for (auto out : fusion->outputs()) { @@ -202,51 +201,6 @@ void CompiledKernel::compileFusion( fusion->printMath(); } - //! Force index_type to int and disable magic zero if we detect that the - //! kernel contains any TMA memory operations. - const std::vector& exprs = fusion->exprs(); - bool has_cp_async_bulk = std::any_of(exprs.begin(), exprs.end(), [](Expr* e) { - return ir_utils::isCpAsyncBulk(e); - }); - - // Disable magic zero if there are any TMA operations in Fusion - if (has_cp_async_bulk) { - compile_params.enable_magic_zero = false; - } - - // Set the index type of compile params if not already set. If set, - // make sure the compile param type is valid with the given kernel - // arguments. - auto arg_index_type = args.getSmallestIndexTypeOfArguments(); - if (compile_params.index_type.has_value()) { - // If the int32 compilation is requested, but the arguments demand - // int64, that's an error - NVF_ERROR( - !(compile_params.index_type.value() == PrimDataType::Int32 && - arg_index_type == PrimDataType::Int), - "Compilation with int32 is requested but int64 is required for the arguments"); - NVF_ERROR( - !has_cp_async_bulk || - (compile_params.index_type.value() == PrimDataType::Int32), - "Compilation with int64 is requested but int32 is required because ", - "of TMA operations."); - - } else if (arg_index_type == PrimDataType::Int) { - // If the given compile option doesn't specify the index type, and - // the arguments require 64-bit indexing, we need to use 64-bit - // indexing. Note that if the arg type is 32-bit, it doesn't mean - // it's safe to use 32-bit for the whole kernel, so unless it's - // specified through CompileParams, we do not use 32-bit indexing. - compile_params.index_type = arg_index_type; - NVF_ERROR( - !has_cp_async_bulk, - "Compilation with int64 is required based on input arguments, but ", - "int32 is required because of TMA operations."); - } else if (has_cp_async_bulk) { - // TMA operations require 32-bit indexing. - compile_params.index_type = PrimDataType::Int32; - } - c10::DeviceGuard dg(options_.device); NVF_ERROR( @@ -295,6 +249,7 @@ void CompiledKernel::compileFusion( } } + // TODO: pass in kernel name? kernel_code_ = codegen::generateCudaKernel(kernel, kernelName()); // If NVFUSER_EXTERNAL_SRC is set, utilize the external source code. diff --git a/csrc/fusion_executor/compiled_kernel.h b/csrc/fusion_executor/compiled_kernel.h index 3959049b9c4..6ec5a642259 100644 --- a/csrc/fusion_executor/compiled_kernel.h +++ b/csrc/fusion_executor/compiled_kernel.h @@ -44,45 +44,45 @@ class CompiledKernel : public NonCopyable { //! with KernelArgumentHolder, but it is no longer the case. NVF_API void compileFusion( Fusion* fusion, - const KernelArgumentHolder& args, - const LaunchParams& launch_constraints, + c10::Device device, + const LaunchParams& launch_params, CompileParams compile_params, - SchedulerType sceduler_type = SchedulerType::None, + SchedulerType scheduler_type = SchedulerType::None, int64_t fusion_id = 0, int64_t concrete_id = 0, int64_t runtime_id = 0, int64_t group_id = 0); - // TODO: merge it with the overload above. - //! This API is merely here so we don't have to go back and update all cpp - //! tests. - void compileFusion( - Fusion* fusion, - const at::ArrayRef& inputs = {}, - const LaunchParams& launch_constraints = LaunchParams(), - CompileParams compile_params = CompileParams()) { - KernelArgumentHolder args = - KernelArgumentHolder::createKernelArgumentHolder(inputs); - compileFusion(fusion, args, launch_constraints, compile_params); - } - - //! Used by user defined schedules in python frontend - void compileFusion( - Fusion* fusion, - const at::ArrayRef& inputs, - int64_t fusion_id, - int64_t concrete_id) { - KernelArgumentHolder args = - KernelArgumentHolder::createKernelArgumentHolder(inputs); - compileFusion( - fusion, - args, - LaunchParams(), - CompileParams(), - SchedulerType::None, - fusion_id, - concrete_id); - } + // // TODO: merge it with the overload above. + // //! This API is merely here so we don't have to go back and update all cpp + // //! tests. + // void compileFusion( + // Fusion* fusion, + // const at::ArrayRef& inputs = {}, + // const LaunchParams& launch_constraints = LaunchParams(), + // CompileParams compile_params = CompileParams()) { + // KernelArgumentHolder args = + // KernelArgumentHolder::createKernelArgumentHolder(inputs); + // compileFusion(fusion, args, launch_constraints, compile_params); + // } + + // //! Used by user defined schedules in python frontend + // void compileFusion( + // Fusion* fusion, + // const at::ArrayRef& inputs, + // int64_t fusion_id, + // int64_t concrete_id) { + // KernelArgumentHolder args = + // KernelArgumentHolder::createKernelArgumentHolder(inputs); + // compileFusion( + // fusion, + // args, + // LaunchParams(), + // CompileParams(), + // SchedulerType::None, + // fusion_id, + // concrete_id); + // } // Register a lowering hooks that are called to modify the GpuLower object // before running lowering passes. The main use case is for unit tests to @@ -158,8 +158,12 @@ class CompiledKernel : public NonCopyable { NVF_API std::string getStructuredCode() const; //! Returns a const reference to the latest compiled kernel. - const executor_utils::CompiledKernel& compiledKernel() const { - return *compiled_kernel_; + const std::unique_ptr& compiledKernel() + const { + return compiled_kernel_; + } + std::unique_ptr& compiledKernel() { + return compiled_kernel_; } //! Returns the disassembled latest compiled binary @@ -183,7 +187,7 @@ class CompiledKernel : public NonCopyable { return global_fusion_count_.load(); } - int64_t groupId() const { + const int64_t& groupId() const { return group_id_; } void setGroupId(int64_t gid) { @@ -255,6 +259,83 @@ class CompiledKernel : public NonCopyable { disable_parameter_cache_ = true; } + // Temporary accessors for refactor: + CompileOptions& options() { + return options_; + } + int64_t& fusionId() { + return fusion_id_; + } + const int64_t& fusionId() const { + return fusion_id_; + } + int64_t& concreteId() { + return concrete_id_; + } + int64_t& runtimeId() { + return runtime_id_; + } + const int64_t& concreteId() const { + return concrete_id_; + } + const int64_t& runtimeId() const { + return runtime_id_; + } + int64_t& groupId() { + return group_id_; + } + static std::atomic& globalFusionCount() { + return global_fusion_count_; + } + SchedulerType& schedulerType() { + return scheduler_type_; + } + const SchedulerType& schedulerType() const { + return scheduler_type_; + } + std::string& kernelId() { + return kernel_id_; + } + const std::string& kernelId() const { + return kernel_id_; + } + std::unique_ptr& lowered() { + return lowered_; + } + std::unique_ptr& fusion() { + return fusion_; + } + const std::unique_ptr& lowered() const { + return lowered_; + } + int64_t& blockSizeHighWaterMark() { + return block_size_high_water_mark_; + } + int64_t& maxrregcountHighWaterMark() { + return maxrregcount_high_water_mark_; + } + const int64_t& blockSizeHighWaterMark() const { + return block_size_high_water_mark_; + } + const int64_t& maxrregcountHighWaterMark() const { + return maxrregcount_high_water_mark_; + } + bool& disablePaarameterCache() { + return disable_parameter_cache_; + } + std::string& kernelCode() { + return kernel_code_; + } + const std::string& kernelCode() const { + return kernel_code_; + } + std::vector>& loweringHooks() { + return lowering_hooks_; + } + std::vector>& postLoweringHooks() { + return post_lowering_hooks_; + } + // //! Serialize Fusion Executor using flatbuffers // flatbuffers::Offset serialize( // flatbuffers::FlatBufferBuilder& builder) const; @@ -276,7 +357,7 @@ class CompiledKernel : public NonCopyable { // int64_t runtime_id, // int64_t group_id); - private: + // private: void setUsedTVs(); const std::vector& getUsedTVs() const { @@ -353,8 +434,6 @@ class CompiledKernel : public NonCopyable { // Post-lowering hooks that are called to modify the kernel after lowering. // The main use case is for unit tests to modify the kernel. std::vector> post_lowering_hooks_; - - Communicator* communicator_; }; } // namespace nvfuser diff --git a/csrc/fusion_executor/executor.cpp b/csrc/fusion_executor/executor.cpp index b322342b598..03df9b90d31 100644 --- a/csrc/fusion_executor/executor.cpp +++ b/csrc/fusion_executor/executor.cpp @@ -145,7 +145,7 @@ std::unique_ptr& FusionExecutor:: evaluatorPrecomputedValues() { if (!evaluator_precomputed_values_) { evaluator_precomputed_values_ = - std::make_unique(lowered_->kernel()); + std::make_unique(lowered()->kernel()); } return evaluator_precomputed_values_; } @@ -153,6 +153,9 @@ std::unique_ptr& FusionExecutor:: std::string FusionExecutor::getStructuredCode( const std::string& kernel_str, PrimDataType index_type) const { + if (use_external_compiler_) { + return compiled_kernel_2_.getStructuredCode(kernel_str, index_type); + } // generating cuda code; std::string code = ""; code += includeStdComplex(); @@ -171,7 +174,7 @@ std::string FusionExecutor::getStructuredCode( } if (isDebugDumpEnabled(DebugDumpOption::CudaToFile)) { std::stringstream file_name; - file_name << "__tmp_kernel_" << kernel_id_ << ".cu"; + file_name << "__tmp_kernel_" << kernelId() << ".cu"; debug() << "PRINTING: " << file_name.str() << std::endl; std::ofstream out(file_name.str()); out << code << std::endl; @@ -182,11 +185,14 @@ std::string FusionExecutor::getStructuredCode( } std::string FusionExecutor::getStructuredCode() const { + if (use_external_compiler_) { + return compiled_kernel_2_.getStructuredCode(); + } return getStructuredCode(kernelString(), kernel()->indexType()); } void FusionExecutor::compileFusion( - Fusion* fusion, + Fusion* _fusion, const KernelArgumentHolder& args, const LaunchParams& launch_constraints, CompileParams compile_params, @@ -198,22 +204,20 @@ void FusionExecutor::compileFusion( FUSER_PERF_SCOPE("FusionExecutor::compileFusion"); NVF_ERROR( - !fusion->outputs().empty(), "No output found for this kernel, aborting."); + !_fusion->outputs().empty(), + "No output found for this kernel, aborting."); - // TODO: refactor the options_ passed through - options_.device = c10::Device(c10::DeviceType::CUDA, args.getDeviceIndex()); - - if (isExpressionEvaluated(fusion)) { - fusion_ = std::make_unique(*fusion); + if (isExpressionEvaluated(_fusion)) { + fusion() = std::make_unique(*_fusion); return; } - const std::vector& exprs = fusion->exprs(); + const std::vector& exprs = _fusion->exprs(); if (std::all_of(exprs.begin(), exprs.end(), [](Expr* e) { return isResharding(e) && isLowerableToCommunication(e); })) { host_ir_container_ = std::make_unique(); - IrCloner cloner = Fusion::copy(fusion, host_ir_container_.get()); + IrCloner cloner = Fusion::copy(_fusion, host_ir_container_.get()); for (Expr* e : exprs) { std::vector communications = lowerCommunication(cloner.clone(e)); @@ -224,52 +228,6 @@ void FusionExecutor::compileFusion( return; } - // NOTE: Profiling needs to be started below the isExpressionEvaluated query - // given the conditional can exit early from compilation. - if (isProfilerEnabled()) { - NVF_CHECK( - group_id >= 0, - "An invalid segment id is passed to FusionProfiler!:", - group_id); - FusionProfiler::segment(group_id).startCompile(args.getDeviceIndex()); - } - - for (auto out : fusion->outputs()) { - const auto logical_domain = out->as()->getLogicalDomain(); - // walking through outputs to see if output shapes are dependent on - // non-tensor inputs. For which case, we should have disabled output - // allocation, since the caching id only looks at tensor shapes. - // See issue https://github.com/csarofeen/pytorch/issues/2002 - std::vector output_extents; - for (const auto id : logical_domain) { - Val* extent = nullptr; - if (id->isReduction() || id->isStride() || id->isDeviceDim()) { - continue; - } else if (id->isBroadcast() && id->hasExpandedExtent()) { - extent = id->expandedExtent(); - } else { - extent = id->extent(); - } - output_extents.emplace_back(extent); - } - auto dependencies = InputsOf::outputs(output_extents); - if (std::any_of(dependencies.begin(), dependencies.end(), [](Val* val) { - return val->isFusionInput(); - })) { - // TODO: parameter cache is too big a hammer here. We should consider - // separate the caching logic of output sizes & launch params. Since - // output size dependency should only invalidate the output sizes - disable_parameter_cache_ = true; - break; - } - } - - if (isDebugDumpEnabled(DebugDumpOption::FusionIr)) { - fusion->print(); - } else if (isDebugDumpEnabled(DebugDumpOption::FusionIrMath)) { - fusion->printMath(); - } - //! Force index_type to int and disable magic zero if we detect that the //! kernel contains any TMA memory operations. bool has_cp_async_bulk = std::any_of(exprs.begin(), exprs.end(), [](Expr* e) { @@ -314,149 +272,43 @@ void FusionExecutor::compileFusion( compile_params.index_type = PrimDataType::Int32; } - c10::DeviceGuard dg(options_.device); + c10::DeviceGuard dg(options().device); NVF_ERROR( - options_.device.is_cuda(), "Provided device to CUDA fuser is the CPU."); - auto properties = at::cuda::getDeviceProperties(options_.device.index()); + options().device.is_cuda(), "Provided device to CUDA fuser is the CPU."); + auto properties = at::cuda::getDeviceProperties(options().device.index()); // TODO: These properties should be set as part of the constructor so that it // can be const device_smem_limit_ = static_cast(properties->sharedMemPerBlockOptin); warp_size_ = properties->warpSize; - lowered_ = std::make_unique(fusion, compile_params); - for (const auto& hook : lowering_hooks_) { - hook(lowered_.get()); - } - lowered_->run(); - - kir::Kernel* kernel = lowered_->kernel(); - - for (const auto& hook : post_lowering_hooks_) { - hook(kernel); - } - createKernelId(scheduler_type, fusion_id, concrete_id, runtime_id, group_id); - setUsedTVs(); - - if (isDebugDumpEnabled(DebugDumpOption::KernelIr)) { - kernel->print(); - } - - if (isDebugDumpEnabled(DebugDumpOption::BankConflictInfo)) { - auto bank_conflict_info = getBankConflictInfo(kernel); - if (bank_conflict_info.empty()) { - debug() << "===== No bank confliction =====" << std::endl; - } else { - debug() << "======= Bank confliction =======" << std::endl; - for (auto info : bank_conflict_info) { - debug() << "Expr: " << info.first->toString() << std::endl; - auto conflict = info.second; - if (conflict.first > 1) { - debug() << "input conflict: " << conflict.first << " way, "; - } - if (conflict.second > 1) { - debug() << "output conflict: " << conflict.second << " way"; - } - debug() << std::endl; - } - debug() << "================================" << std::endl; - } - } - - kernel_code_ = codegen::generateCudaKernel(kernel, kernelName()); - - // If NVFUSER_EXTERNAL_SRC is set, utilize the external source code. - // If the loaded external source code is empty, revert to the default codegen. - // The external_structured_code is moved to structured_code and explicitly - // cleared to avoid use-after-move scenarios. - // Note: we index these with getGlobalFusionCount() instead of fusion_id_ in - // order to match the numbering of files output with - // NVFUSER_DUMP=cuda_to_file - auto structured_code = - getStructuredCodeFromExternalFiles(getGlobalFusionCount()); - if (structured_code.empty()) { - structured_code = getStructuredCode(); - } - - const kir::KernelSummary& kernel_summary = kernel->summary(); - - // TODO: this replicates the target GPU version computation from - // executor_utils. - std::pair target_arch; - bool compile_to_sass = false; - executor_utils::queryTargetGPUVersion( - properties, - std::ref(target_arch.first), - std::ref(target_arch.second), - compile_to_sass); - - NVF_CHECK( - target_arch >= kernel_summary.min_device_version, - "Target compute capability is ", - target_arch.first, - ".", - target_arch.second, - " but this fusion requires at least ", - kernel_summary.min_device_version.first, - ".", - kernel_summary.min_device_version.second, - ". Reason: ", - kernel_summary.min_device_version_reason); - - // We currently shouldn't allocate any more shared mem - // tensors statically but could keep this path if - // needed in later development. - if (!kernel_summary.static_smem_allocations.empty()) { - ExpressionEvaluator static_evaluator; - const auto static_smem_size = computeSharedMemory( - static_evaluator, - kernel_summary.static_smem_allocations, - kernel->indexType()); - NVF_ERROR( - static_smem_size < max_static_smem_, - "The static shared memory allocation is larger than available memory."); - } - - if (kernel_summary.has_dynamic_local_memory_allocations) { - std::stringstream ss; - ss << "Allocations must be based on constant integers for local memory. However, found: "; - for (auto alloc : kernel_summary.dynamic_lmem_allocations) { - ss << alloc->buffer()->toString() << ", "; - } - ss << " have dynamic allocations but are placed in local memory."; - NVF_THROW(ss.str()); - } - // TODO: pass block_size here; std::optional dynamic_smem = std::nullopt; std::optional block_size = std::nullopt; + auto launch_params = launch_constraints; if (!args.empty()) { - auto expr_eval = executor_utils::bindInputs(args, kernel); - auto launch_params = computeLaunchParams( - launch_constraints, expr_eval, warp_size_, kernel->indexType()); + auto expr_eval = executor_utils::bindInputs(args, fusion().get()); + launch_params = computeLaunchParams( + launch_constraints, + expr_eval, + warp_size_, + compile_params.index_type.value()); block_size = launch_params.nThreads(); dynamic_smem = launch_params.smem(); NVF_ERROR(block_size > 0, "launch param inferred block size < 0"); } - // TODO: high water mark should be computed via occupancy API after - // compilation. - - // Basically setting high water mark as 1 when we don't provide args for - // compilation, it will just generate a kernel that gets ditched at the first - // run - not great. We should have better heuristics. - block_size_high_water_mark_ = std::max( - (block_size.has_value() ? block_size.value() : 1), - block_size_high_water_mark_); - maxrregcount_high_water_mark_ = compile_params.maxrregcount; - compiled_kernel_ = executor_utils::getCompiledKernel( - kernel_code_, - structured_code, - kernelName(), - kernel_id_, + use_external_compiler_ = true; + compiled_kernel_2_.compileFusion( + _fusion, + options().device, + launch_params, compile_params, - block_size); - NVF_ERROR(validKernelId(), "Invalid kernel id for FusionExecutor."); + scheduler_type, + fusion_id, + concrete_id, + runtime_id, + group_id); // These should be nullopt at this point, but reset just in case resetCompiledKernelProperties(); @@ -466,13 +318,6 @@ void FusionExecutor::compileFusion( if (dynamic_smem.has_value()) { ensureAvailableDynamicSmemSize(dynamic_smem.value()); } - - if (isDebugDumpEnabled(DebugDumpOption::Sass)) { - debug() << disassembledKernelSASS() << std::endl; - } - if (isProfilerEnabled()) { - FusionProfiler::segment(group_id).stopCompile(); - } } LaunchParams FusionExecutor::computeLaunchParams( @@ -487,7 +332,7 @@ LaunchParams FusionExecutor::computeLaunchParams( auto data_cache = compileTimeDataCache(); - auto lower = lowered_.get(); + auto lower = lowered().get(); auto& used_tvs = getUsedTVs(); auto parallel_binding_ids_entry = executor_utils::caching::ExecutorCompileTimeEntry< @@ -577,7 +422,7 @@ LaunchParams FusionExecutor::computeLaunchParams( expr_eval.precomputedValues()->evaluate(); } - const auto kernel = lowered_->kernel(); + const auto kernel = lowered()->kernel(); const auto& kernel_summary = kernel->summary(); // Calculate Dynamic Shared Memory Size @@ -641,7 +486,7 @@ std::vector FusionExecutor::getIntermediateBufferInfo( FUSER_PERF_SCOPE("FusionExecutor::getIntermediateBufferInfo"); std::vector global_buffers; - const auto kernel = lowered_->kernel(); + const auto kernel = lowered()->kernel(); const auto& kernel_summary = kernel->summary(); for (auto alloc : kernel_summary.global_allocations) { @@ -686,6 +531,10 @@ std::vector FusionExecutor::getIntermediateBufferInfo( } void FusionExecutor::setUsedTVs() { + if (use_external_compiler_) { + compiled_kernel_2_.setUsedTVs(); + return; + } auto used_vals = fusion()->usedMathVals(); auto used_tvs = ir_utils::filterByType(used_vals); used_tvs_.clear(); @@ -842,7 +691,7 @@ void FusionExecutor::initializeExecutorEntry( if (outputs.empty()) { output_info = - getBufferInfos(expr_eval, index_type, lowered_->kernel()->outputs()); + getBufferInfos(expr_eval, index_type, lowered()->kernel()->outputs()); } else { // Need to save the information necessary for allocations as // future uses of this ExecutorEntry may not be provided with @@ -999,19 +848,23 @@ void FusionExecutor::recomputeArgs( void FusionExecutor::recompileKernel( const LaunchParams& new_launch_params, const CompileParams& new_compile_params) { + if (use_external_compiler_) { + return compiled_kernel_2_.recompileKernel( + new_launch_params, new_compile_params); + } FUSER_PERF_SCOPE("FusionExecutor::runFusion::recompileKernel"); const auto structured_code = getStructuredCode(); - block_size_high_water_mark_ = new_launch_params.nThreads(); - maxrregcount_high_water_mark_ = new_compile_params.maxrregcount; + blockSizeHighWaterMark() = new_launch_params.nThreads(); + maxrregcountHighWaterMark() = new_compile_params.maxrregcount; compiled_kernel_ = executor_utils::getCompiledKernel( - kernel_code_, + kernelCode(), structured_code, kernelName(), - kernel_id_, + kernelId(), new_compile_params, - block_size_high_water_mark_); + blockSizeHighWaterMark()); resetCompiledKernelProperties(); @@ -1022,7 +875,9 @@ void FusionExecutor::recompileKernel( // false positives ensureAvailableDynamicSmemSize(new_launch_params.smem()); validateCooperativeLaunch( - compiled_kernel_->function, new_launch_params, options_.device.index()); + compiled_kernel_->function, + new_launch_params, + options().device.index()); } } @@ -1146,12 +1001,12 @@ std::vector FusionExecutor::runFusion( if (isProfilerEnabled()) { NVF_CHECK( - group_id_ >= 0, + groupId() >= 0, "An invalid segment id is passed to FusionProfiler!:", - group_id_); - SegmentProfiler& sprof = FusionProfiler::segment(group_id_); + groupId()); + SegmentProfiler& sprof = FusionProfiler::segment(groupId()); sprof.inputBytesAccessed(inputBytesProcessed(args)); - sprof.scheduler(toString(scheduler_type_)); + sprof.scheduler(toString(schedulerType())); sprof.startKernel(args.getDeviceIndex()); } @@ -1162,13 +1017,13 @@ std::vector FusionExecutor::runFusion( " provided number of outputs does not match fusion output"); // Bind fusion inputs - auto expr_eval = executor_utils::bindInputs(args, fusion()); + auto expr_eval = executor_utils::bindInputs(args, fusion().get()); - if (isExpressionEvaluated(fusion())) { + if (isExpressionEvaluated(fusion().get())) { FUSER_PERF_SCOPE("FusionExecutor::runFusion::evaluate_with_ExprEval"); outputs = evaluateFusionOutputs(outputs, expr_eval); if (isProfilerEnabled()) { - auto& sprof = FusionProfiler::segment(group_id_); + auto& sprof = FusionProfiler::segment(groupId()); sprof.stopKernel(); sprof.outputBytesAccessed(outputBytesProcessed(outputs)); } @@ -1181,7 +1036,7 @@ std::vector FusionExecutor::runFusion( std::vector output_info = getBufferInfos( expr_eval, PrimDataType::Int, host_ir_container_->outputs()); outputs = allocateOutputs( - host_ir_container_.get(), output_info, options_.device, expr_eval); + host_ir_container_.get(), output_info, options().device, expr_eval); } for (Expr* e : host_ir_container_->topLevelExprs()) { NVF_ERROR(e->isA()); @@ -1215,19 +1070,19 @@ std::vector FusionExecutor::runFusion( if (isDebugDumpEnabled(DebugDumpOption::FusionArgs)) { dumpFusionArgs( - fusion_id_, args, launch_constraints, compile_params, outputs); + fusionId(), args, launch_constraints, compile_params, outputs); } - c10::DeviceGuard dg(options_.device); + c10::DeviceGuard dg(options().device); auto stream = at::cuda::getCurrentCUDAStream(); at::cuda::jit::initializeCudaContext(); - NVF_ERROR(lowered_); + NVF_ERROR(lowered()); // Placeholder for the case where parameter cache is not used ExecutorEntry temporary_executor_entry; ExecutorEntry* executor_entry = - args.getCacheId().has_value() && !disable_parameter_cache_ + args.getCacheId().has_value() && !disablePaarameterCache() ? &executor_entry_lookup_[*args.getCacheId()] : &temporary_executor_entry; @@ -1242,9 +1097,8 @@ std::vector FusionExecutor::runFusion( kernel()->indexType()); } - if (!(executor_entry->launch_params.nThreads() <= - block_size_high_water_mark_ && - compile_params.maxrregcount == maxrregcount_high_water_mark_)) { + if (!(executor_entry->launch_params.nThreads() <= blockSizeHighWaterMark() && + compile_params.maxrregcount == maxrregcountHighWaterMark())) { recompileKernel(executor_entry->launch_params, compile_params); } @@ -1257,7 +1111,7 @@ std::vector FusionExecutor::runFusion( // only allocate outputs when not given if (outputs.empty()) { outputs = allocateOutputs( - fusion(), executor_entry->outputs, options_.device, expr_eval); + fusion().get(), executor_entry->outputs, options().device, expr_eval); } args.push(outputs); @@ -1299,18 +1153,20 @@ std::vector FusionExecutor::runFusion( // to reset to zero upon completion of the kernel, or if we have // enabled the option (unsafe) intermediate_buffer = contigZeroedTensor( - unexpanded_sizes, buf_info.type, options_.device); + unexpanded_sizes, buf_info.type, options().device); } else { intermediate_buffer = at::zeros( unexpanded_sizes, - at::TensorOptions().dtype(buf_info.type).device(options_.device)); + at::TensorOptions() + .dtype(buf_info.type) + .device(options().device)); } } else { intermediate_buffer = at::native::empty_cuda( unexpanded_sizes, buf_info.type, c10::nullopt, - options_.device, + options().device, c10::nullopt); if (shouldFillAllocationWithNan()) { fillTensorWithNan(intermediate_buffer); @@ -1341,7 +1197,7 @@ std::vector FusionExecutor::runFusion( if (isDebugDumpEnabled(DebugDumpOption::KernelArgs)) { dumpKernelArgs( - fusion_id_, + fusionId(), args, num_inputs, outputs, @@ -1371,7 +1227,7 @@ std::vector FusionExecutor::runFusion( launch_params_.smem())); const int64_t device_id = - static_cast(options_.device.index()); + static_cast(options().device.index()); const auto prop = at::cuda::getDeviceProperties((c10::DeviceIndex)device_id); const int64_t warps_per_sm = @@ -1427,7 +1283,7 @@ std::vector FusionExecutor::runFusion( } if (isProfilerEnabled()) { - auto& sprof = FusionProfiler::segment(group_id_); + auto& sprof = FusionProfiler::segment(groupId()); sprof.stopKernel(); sprof.outputBytesAccessed(outputBytesProcessed(outputs)); } @@ -1464,6 +1320,9 @@ void FusionExecutor::compileRtc( const std::string& name, bool structured, PrimDataType index_type) { + if (use_external_compiler_) { + return compiled_kernel_2_.compileRtc(code, name, structured, index_type); + } FUSER_PERF_SCOPE("FusionExecutor::compileRtc"); NVF_ERROR( index_type == PrimDataType::Int || index_type == PrimDataType::Int32 || @@ -1479,16 +1338,19 @@ void FusionExecutor::compileRtc( scode = code; } compiled_kernel_ = - executor_utils::getCompiledKernel(std::nullopt, scode, name, kernel_id_); + executor_utils::getCompiledKernel(std::nullopt, scode, name, kernelId()); } float FusionExecutor::runRtc( const LaunchParams& launch_params, const std::vector& args, PrimDataType index_type) { + if (use_external_compiler_) { + return compiled_kernel_2_.runRtc(launch_params, args, index_type); + } FUSER_PERF_SCOPE("FusionExecutor::runRtc"); - c10::DeviceGuard dg(options_.device); + c10::DeviceGuard dg(options().device); auto stream = at::cuda::getCurrentCUDAStream(); cudaEvent_t start_event = {}; @@ -1570,15 +1432,15 @@ flatbuffers::Offset FusionExecutor::serialize( return serde::CreateFusionExecutorDirect( builder, device_smem_limit_, - block_size_high_water_mark_, - maxrregcount_high_water_mark_, + blockSizeHighWaterMark(), + maxrregcountHighWaterMark(), warp_size_, - toUnderlying(scheduler_type_), - fusion_id_, - concrete_id_, - runtime_id_, - group_id_, - kernel_code_.c_str(), + toUnderlying(schedulerType()), + fusionId(), + concreteId(), + runtimeId(), + groupId(), + kernelCode().c_str(), &executor_entry_lookup_keys_fb, &executor_entry_lookup_values_fb, toUnderlying(kernel()->indexType()), @@ -1703,7 +1565,7 @@ flatbuffers::Offset FusionExecutor::serialize( void FusionExecutor::deserialize( const serde::FusionExecutor* buffer, - Fusion* fusion, + Fusion* _fusion, int8_t device_index, CompileParams compile_params, SchedulerType heuristic, @@ -1717,8 +1579,8 @@ void FusionExecutor::deserialize( // TODO Should we set fusion_id, concrete_id, runtime_id, and group_id when we // skip compilation? - if (isExpressionEvaluated(fusion)) { - fusion_ = std::make_unique(*fusion); + if (isExpressionEvaluated(_fusion)) { + fusion() = std::make_unique(*_fusion); NVF_ERROR(!hasCompiledKernel(), "Failed to deserialize FusionExecutor"); return; } @@ -1743,24 +1605,24 @@ void FusionExecutor::deserialize( buffer->heuristic()); // Initialize CompileOptions - options_.device = c10::Device(c10::DeviceType::CUDA, device_index); - c10::DeviceGuard dg(options_.device); + options().device = c10::Device(c10::DeviceType::CUDA, device_index); + c10::DeviceGuard dg(options().device); // Initialize internal fields device_smem_limit_ = buffer->device_smem_limit(); - block_size_high_water_mark_ = buffer->block_size_high_water_mark(); - maxrregcount_high_water_mark_ = buffer->maxrregcount_high_water_mark(); + blockSizeHighWaterMark() = buffer->block_size_high_water_mark(); + maxrregcountHighWaterMark() = buffer->maxrregcount_high_water_mark(); warp_size_ = buffer->warp_size(); - kernel_code_ = buffer->kernel_code()->str(); + kernelCode() = buffer->kernel_code()->str(); // KernelDB query checks kernel_code string and compile_params before // copying cubin. compile_params.index_type = serde::mapToNvfuserDtype(buffer->index_type()); - compile_params.maxrregcount = maxrregcount_high_water_mark_; + compile_params.maxrregcount = maxrregcountHighWaterMark(); // Get lowered fusion - lowered_ = std::make_unique(fusion, compile_params); - lowered_->run(); + lowered() = std::make_unique(_fusion, compile_params); + lowered()->run(); // Replace integers that are tensor sizes by named scalars like "T0.size[0]" createKernelId( @@ -1816,7 +1678,7 @@ GlobalBufferInfo FusionExecutor::deserialize( NVF_ERROR( buffer->tv() != -1, "Serialization failed to encode buffer tv position."); - NVF_ERROR(lowered_ != nullptr, "Lowered kernel is not initialized."); + NVF_ERROR(lowered() != nullptr, "Lowered kernel is not initialized."); GlobalBufferInfo info; if (buffer->is_fusion_output()) { diff --git a/csrc/fusion_executor/executor.h b/csrc/fusion_executor/executor.h index 788041f3289..12e965583e3 100644 --- a/csrc/fusion_executor/executor.h +++ b/csrc/fusion_executor/executor.h @@ -122,19 +122,30 @@ class FusionExecutor : public NonCopyable { // before running lowering passes. The main use case is for unit tests to // modify the lowering process. void registerLoweringHook(std::function hook) { - lowering_hooks_.push_back(std::move(hook)); + if (use_external_compiler_) { + compiled_kernel_2_.registerLoweringHook(hook); + } else { + lowering_hooks_.push_back(std::move(hook)); + } } // Register a post-lowering hooks that are called to modify the kernel after // lowering. The main use case is for unit tests to modify the kernel. void registerPostLoweringHook(std::function hook) { - post_lowering_hooks_.push_back(std::move(hook)); + if (use_external_compiler_) { + compiled_kernel_2_.registerPostLoweringHook(std::move(hook)); + } else { + post_lowering_hooks_.push_back(std::move(hook)); + } } // Function to query whether compilation was attempted for a `FusionExecutor` bool isCompiled() const { - int num_compiled_artifacts = (fusion_ != nullptr) + (lowered_ != nullptr) + - (host_ir_container_ != nullptr); + if (use_external_compiler_) { + return compiled_kernel_2_.isCompiled(); + } + int num_compiled_artifacts = (fusion() != nullptr) + + (lowered() != nullptr) + (host_ir_container_ != nullptr); NVF_ERROR(num_compiled_artifacts <= 1); return num_compiled_artifacts == 1; }; @@ -142,13 +153,16 @@ class FusionExecutor : public NonCopyable { // function to query whether a `FusionExecutor` has a compiled kernel to // execute bool hasCompiledKernel() const { + if (use_external_compiler_) { + return compiled_kernel_2_.hasCompiledKernel(); + } if (compiled_kernel_ != nullptr) { NVF_ERROR(compiled_kernel_->function != nullptr); NVF_ERROR( - fusion_ == nullptr, - "fusion_ should only be initialized when using expression evaluator."); + fusion() == nullptr, + "fusion() should only be initialized when using expression evaluator."); } - return validKernelId() && lowered_ && compiled_kernel_ != nullptr; + return validKernelId() && lowered() && compiled_kernel_ != nullptr; }; void evictCache(size_t cache_id) { @@ -184,17 +198,26 @@ class FusionExecutor : public NonCopyable { executor_utils::caching::ExecutorCompileTimeInfoCache; kir::Kernel* kernel() const { - NVF_ERROR(lowered_); - return lowered_->kernel(); + if (use_external_compiler_) { + return compiled_kernel_2_.kernel(); + } + NVF_ERROR(lowered()); + return lowered()->kernel(); } Fusion* fusion() const { NVF_ERROR(isCompiled()); if (fusion_ != nullptr) { + if (use_external_compiler_) { + return compiled_kernel_2_.fusion(); + } return fusion_.get(); } - if (lowered_ != nullptr) { - return lowered_->kernel()->as(); + if (lowered() != nullptr) { + if (use_external_compiler_) { + return compiled_kernel_2_.fusion(); + } + return lowered()->kernel()->as(); } if (host_ir_container_ != nullptr) { return host_ir_container_->as(); @@ -203,7 +226,10 @@ class FusionExecutor : public NonCopyable { } const ThreadPredicateMap& threadPredMap() const { - return lowered_->threadPredMap(); + if (use_external_compiler_) { + return compiled_kernel_2_.threadPredMap(); + } + return lowered()->threadPredMap(); } //! Internal knob used for debugging/profiling only @@ -225,6 +251,9 @@ class FusionExecutor : public NonCopyable { //! get register spills (load + store) of the compiled kernel int getKernelRegisterSpills() const { + if (use_external_compiler_) { + return compiled_kernel_2_.getKernelRegisterSpills(); + } return compiled_kernel_->register_spills; } //! Returns the input bytes accessed for a kernel @@ -241,8 +270,11 @@ class FusionExecutor : public NonCopyable { //! Returns the string of the compiled kernel NVF_API std::string kernelString() const { - NVF_ERROR(!kernel_code_.empty(), "Kernel code not generated"); - return kernel_code_; + if (use_external_compiler_) { + return compiled_kernel_2_.kernelString(); + } + NVF_ERROR(!kernelCode().empty(), "Kernel code not generated"); + return kernelCode(); } // Add preamble and wrap in namespace @@ -253,40 +285,72 @@ class FusionExecutor : public NonCopyable { NVF_API std::string getStructuredCode() const; //! Returns a const reference to the latest compiled kernel. - const executor_utils::CompiledKernel& compiledKernel() const { - return *compiled_kernel_; + const std::unique_ptr& compiledKernel() + const { + if (use_external_compiler_) { + return compiled_kernel_2_.compiledKernel(); + } + return compiled_kernel_; + } + std::unique_ptr& compiledKernel() { + if (use_external_compiler_) { + return compiled_kernel_2_.compiledKernel(); + } + return compiled_kernel_; } //! Returns the disassembled latest compiled binary NVF_API std::string disassembledBinary( const std::string& nvdisasm_args = "") const { + if (use_external_compiler_) { + return compiled_kernel_2_.disassembledBinary(nvdisasm_args); + } return executor_utils::disassembleBinary( compiled_kernel_->cubin, nvdisasm_args); } //! Returns the disassembled latest compiled binary NVF_API std::string disassembledKernelSASS() const { + if (use_external_compiler_) { + return compiled_kernel_2_.disassembledKernelSASS(); + } return executor_utils::disassembleBinary( compiled_kernel_->cubin, "-fun 1 -c"); } static void setGlobalFusionCount(int64_t new_fusion_count) { - global_fusion_count_.store(new_fusion_count); + CompiledKernel::setGlobalFusionCount(new_fusion_count); } static int64_t getGlobalFusionCount() { - return global_fusion_count_.load(); + return CompiledKernel::getGlobalFusionCount(); } - int64_t groupId() const { + const int64_t& groupId() const { + if (use_external_compiler_) { + return compiled_kernel_2_.groupId(); + } + return group_id_; + } + int64_t& groupId() { + if (use_external_compiler_) { + return compiled_kernel_2_.groupId(); + } return group_id_; } + void setGroupId(int64_t gid) { + if (use_external_compiler_) { + return compiled_kernel_2_.setGroupId(gid); + } group_id_ = gid; } bool validKernelId() const { - return !kernel_id_.empty(); + if (use_external_compiler_) { + return compiled_kernel_2_.validKernelId(); + } + return !kernelId().empty(); } void createKernelId( @@ -295,41 +359,48 @@ class FusionExecutor : public NonCopyable { int64_t concrete_id = 0, int64_t runtime_id = 0, int64_t group_id = 0) { + if (use_external_compiler_) { + return compiled_kernel_2_.createKernelId( + scheduler_type, fusion_id, concrete_id, runtime_id, group_id); + } NVF_ERROR(fusion_id > -1, "Invalid fusion_id."); NVF_ERROR(concrete_id > -1, "Invalid concrete_id."); NVF_ERROR(runtime_id > -1, "Invalid runtime_id."); NVF_ERROR(group_id > -1, "Invalid group_id"); - scheduler_type_ = scheduler_type; - fusion_id_ = fusion_id; - concrete_id_ = concrete_id; - runtime_id_ = runtime_id; - group_id_ = group_id; - ++global_fusion_count_; + schedulerType() = scheduler_type; + fusionId() = fusion_id; + concreteId() = concrete_id; + runtimeId() = runtime_id; + groupId() = group_id; + ++globalFusionCount(); std::stringstream ss; if (isOptionEnabled(EnableOption::StaticFusionCount)) { - ss << global_fusion_count_.load(); + ss << globalFusionCount().load(); } else { - ss << toString(scheduler_type_); - ss << "_f" << fusion_id_; - ss << "_c" << concrete_id_; - ss << "_r" << runtime_id_; - ss << "_g" << group_id_; + ss << toString(schedulerType()); + ss << "_f" << fusionId(); + ss << "_c" << concreteId(); + ss << "_r" << runtimeId(); + ss << "_g" << groupId(); } - kernel_id_ = ss.str(); + kernelId() = ss.str(); } std::string kernelName() const { - NVF_ERROR(!kernel_id_.empty(), "Invalid kernel name for fusion executor."); + if (use_external_compiler_) { + return compiled_kernel_2_.kernelName(); + } + NVF_ERROR(!kernelId().empty(), "Invalid kernel name for fusion executor."); std::stringstream ss; - ss << "nvfuser_" << kernel_id_; + ss << "nvfuser_" << kernelId(); return ss.str(); } //! Internal tests only. Compiles CUDA code with NVRTC directly from - //! string. This util provides a path to test runtime code, i.e. the resource - //! strings. + //! string. This util provides a path to test runtime code, i.e. the + //! resource strings. // TODO: Consider split out compileRtc and runRtc to a different //! class. Not much code is shared with the normal path. NVF_API void compileRtc( @@ -347,7 +418,11 @@ class FusionExecutor : public NonCopyable { //! Internal knob used for debugging/profiling only void disableLaunchParamCache() { - disable_parameter_cache_ = true; + if (use_external_compiler_) { + compiled_kernel_2_.disableLaunchParamCache(); + } else { + disablePaarameterCache() = true; + } } //! Serialize Fusion Executor using flatbuffers @@ -384,6 +459,9 @@ class FusionExecutor : public NonCopyable { void setUsedTVs(); const std::vector& getUsedTVs() const { + if (use_external_compiler_) { + return compiled_kernel_2_.getUsedTVs(); + } return used_tvs_; }; @@ -411,8 +489,8 @@ class FusionExecutor : public NonCopyable { // to we have now. void computeArgs(ExecutorEntry&, ExpressionEvaluator&, const kir::Kernel*) const; - // Updates an existing set of arguments based on the current arguments. It is - // is an error to call this before `computeArgs` has been invoked. + // Updates an existing set of arguments based on the current arguments. It + // is is an error to call this before `computeArgs` has been invoked. // recomputeArgs will fail if the arity of the function changes, or the rank // of any tensor changes (as these are compiled-in to the generated kernel // and therefore would require us to do a larger recompilation). @@ -464,8 +542,152 @@ class FusionExecutor : public NonCopyable { //! Clear the cached properties of the compiled kernel void resetCompiledKernelProperties(); + // Temporary accessors for refactor: + CompileOptions& options() { + if (use_external_compiler_) { + return compiled_kernel_2_.options(); + } + return options_; + } + int64_t& fusionId() { + if (use_external_compiler_) { + return compiled_kernel_2_.fusionId(); + } + return fusion_id_; + } + const int64_t& fusionId() const { + if (use_external_compiler_) { + return compiled_kernel_2_.fusionId(); + } + return fusion_id_; + } + int64_t& concreteId() { + if (use_external_compiler_) { + return compiled_kernel_2_.concreteId(); + } + return concrete_id_; + } + int64_t& runtimeId() { + if (use_external_compiler_) { + return compiled_kernel_2_.runtimeId(); + } + return runtime_id_; + } + const int64_t& concreteId() const { + if (use_external_compiler_) { + return compiled_kernel_2_.concreteId(); + } + return concrete_id_; + } + const int64_t& runtimeId() const { + if (use_external_compiler_) { + return compiled_kernel_2_.runtimeId(); + } + return runtime_id_; + } + static std::atomic& globalFusionCount() { + return CompiledKernel::globalFusionCount(); + } + SchedulerType& schedulerType() { + if (use_external_compiler_) { + return compiled_kernel_2_.schedulerType(); + } + return scheduler_type_; + } + const SchedulerType& schedulerType() const { + if (use_external_compiler_) { + return compiled_kernel_2_.schedulerType(); + } + return scheduler_type_; + } + std::string& kernelId() { + if (use_external_compiler_) { + return compiled_kernel_2_.kernelId(); + } + return kernel_id_; + } + const std::string& kernelId() const { + if (use_external_compiler_) { + return compiled_kernel_2_.kernelId(); + } + return kernel_id_; + } + std::unique_ptr& lowered() { + if (use_external_compiler_) { + return compiled_kernel_2_.lowered(); + } + return lowered_; + } + const std::unique_ptr& lowered() const { + if (use_external_compiler_) { + return compiled_kernel_2_.lowered(); + } + return lowered_; + } + std::unique_ptr& fusion() { + if (use_external_compiler_) { + return compiled_kernel_2_.fusion(); + } + return fusion_; + } + int64_t& blockSizeHighWaterMark() { + if (use_external_compiler_) { + return compiled_kernel_2_.blockSizeHighWaterMark(); + } + return block_size_high_water_mark_; + } + int64_t& maxrregcountHighWaterMark() { + if (use_external_compiler_) { + return compiled_kernel_2_.maxrregcountHighWaterMark(); + } + return maxrregcount_high_water_mark_; + } + const int64_t& blockSizeHighWaterMark() const { + if (use_external_compiler_) { + return compiled_kernel_2_.blockSizeHighWaterMark(); + } + return block_size_high_water_mark_; + } + const int64_t& maxrregcountHighWaterMark() const { + if (use_external_compiler_) { + return compiled_kernel_2_.maxrregcountHighWaterMark(); + } + return maxrregcount_high_water_mark_; + } + bool& disablePaarameterCache() { + if (use_external_compiler_) { + return compiled_kernel_2_.disablePaarameterCache(); + } + return disable_parameter_cache_; + } + std::string& kernelCode() { + if (use_external_compiler_) { + return compiled_kernel_2_.kernelCode(); + } + return kernel_code_; + } + const std::string& kernelCode() const { + if (use_external_compiler_) { + return compiled_kernel_2_.kernelCode(); + } + return kernel_code_; + } + std::vector>& loweringHooks() { + if (use_external_compiler_) { + return compiled_kernel_2_.loweringHooks(); + } + return lowering_hooks_; + } + std::vector>& postLoweringHooks() { + if (use_external_compiler_) { + return compiled_kernel_2_.postLoweringHooks(); + } + return post_lowering_hooks_; + } + private: CompiledKernel compiled_kernel_2_; + bool use_external_compiler_ = false; CompileOptions options_; @@ -525,8 +747,8 @@ class FusionExecutor : public NonCopyable { int64_t block_size_high_water_mark_ = 1; int64_t maxrregcount_high_water_mark_ = 255; - // lookup table to take short cut to retrieve recorded information in order to - // launch kernels without re-inference parameters. + // lookup table to take short cut to retrieve recorded information in order + // to launch kernels without re-inference parameters. std::unordered_map executor_entry_lookup_; // Compile time information caching. This is used for shape inference diff --git a/tests/cpp/test_memory.cpp b/tests/cpp/test_memory.cpp index 29698033f15..6e9d074760a 100644 --- a/tests/cpp/test_memory.cpp +++ b/tests/cpp/test_memory.cpp @@ -84,7 +84,7 @@ TEST_P(MemoryTest, LoadCache) { } // Verify PTX. - const executor_utils::CompiledKernel& compiled_kernel = fe.compiledKernel(); + const executor_utils::CompiledKernel& compiled_kernel = *fe.compiledKernel(); std::string ptx(compiled_kernel.ptx.begin(), compiled_kernel.ptx.end()); std::regex regex(R"(ld\.global\.)" + cache_op_str + R"(\.\S+)"); std::smatch match; @@ -159,7 +159,7 @@ TEST_F(MemoryTest, RefineCachePolicy) { } // Verify PTX. - const executor_utils::CompiledKernel& compiled_kernel = fe.compiledKernel(); + const executor_utils::CompiledKernel& compiled_kernel = *fe.compiledKernel(); std::string ptx(compiled_kernel.ptx.begin(), compiled_kernel.ptx.end()); expectMatchCount(ptx, R"(ld\.global\.ca\.v4\.\S+)", 1); expectMatchCount(ptx, R"(ld\.global\.cs\.v4\.\S+)", 1); From 9ae5fb95986d0818bbfcf08e2efe7bca5c3f1002 Mon Sep 17 00:00:00 2001 From: Christian Sarofeen Date: Wed, 2 Oct 2024 12:10:36 -0700 Subject: [PATCH 03/11] Resolve most test failures. --- csrc/fusion_executor/compiled_kernel.cpp | 34 ++++---- csrc/fusion_executor/compiled_kernel.h | 33 +++----- csrc/fusion_executor/executor.cpp | 71 +++++++++------- csrc/fusion_executor/executor.h | 100 +++++++++++------------ 4 files changed, 122 insertions(+), 116 deletions(-) diff --git a/csrc/fusion_executor/compiled_kernel.cpp b/csrc/fusion_executor/compiled_kernel.cpp index fc99e7ac42b..f38b6f82251 100644 --- a/csrc/fusion_executor/compiled_kernel.cpp +++ b/csrc/fusion_executor/compiled_kernel.cpp @@ -138,11 +138,21 @@ std::string getStructuredCodeFromExternalFiles(const int64_t fusion_id) { } } // namespace -void CompiledKernel::compileFusion( +NVF_API CompiledKernel::CompiledKernel( Fusion* fusion, + CompileParams compile_params) + : compile_params_(compile_params), + lowered_(std::make_unique(fusion, compile_params)) { + FUSER_PERF_SCOPE("CompiledKernel::CompiledKernel"); + for (const auto& hook : lowering_hooks_) { + hook(lowered_.get()); + } + lowered_->run(); +} + +void CompiledKernel::compileFusion( c10::Device device, const LaunchParams& launch_params, - CompileParams compile_params, SchedulerType scheduler_type, int64_t fusion_id, int64_t concrete_id, @@ -151,7 +161,8 @@ void CompiledKernel::compileFusion( FUSER_PERF_SCOPE("CompiledKernel::compileFusion"); NVF_ERROR( - !fusion->outputs().empty(), "No output found for this kernel, aborting."); + !fusion()->outputs().empty(), + "No output found for this kernel, aborting."); options_.device = device; @@ -165,7 +176,7 @@ void CompiledKernel::compileFusion( FusionProfiler::segment(group_id).startCompile(device.index()); } - for (auto out : fusion->outputs()) { + for (auto out : fusion()->outputs()) { const auto logical_domain = out->as()->getLogicalDomain(); // walking through outputs to see if output shapes are dependent on // non-tensor inputs. For which case, we should have disabled output @@ -196,9 +207,9 @@ void CompiledKernel::compileFusion( } if (isDebugDumpEnabled(DebugDumpOption::FusionIr)) { - fusion->print(); + fusion()->print(); } else if (isDebugDumpEnabled(DebugDumpOption::FusionIrMath)) { - fusion->printMath(); + fusion()->printMath(); } c10::DeviceGuard dg(options_.device); @@ -209,13 +220,6 @@ void CompiledKernel::compileFusion( // TODO: These properties should be set as part of the constructor so that it // can be const warp_size_ = properties->warpSize; - - lowered_ = std::make_unique(fusion, compile_params); - for (const auto& hook : lowering_hooks_) { - hook(lowered_.get()); - } - lowered_->run(); - kir::Kernel* kernel = lowered_->kernel(); for (const auto& hook : post_lowering_hooks_) { @@ -325,13 +329,13 @@ void CompiledKernel::compileFusion( // run - not great. We should have better heuristics. block_size_high_water_mark_ = std::max(launch_params.nThreads(), block_size_high_water_mark_); - maxrregcount_high_water_mark_ = compile_params.maxrregcount; + maxrregcount_high_water_mark_ = compile_params_.maxrregcount; compiled_kernel_ = executor_utils::getCompiledKernel( kernel_code_, structured_code, kernelName(), kernel_id_, - compile_params, + compile_params_, launch_params.nThreads()); NVF_ERROR(validKernelId(), "Invalid kernel id for CompiledKernel."); diff --git a/csrc/fusion_executor/compiled_kernel.h b/csrc/fusion_executor/compiled_kernel.h index 6ec5a642259..043dae4522e 100644 --- a/csrc/fusion_executor/compiled_kernel.h +++ b/csrc/fusion_executor/compiled_kernel.h @@ -38,15 +38,14 @@ class CompiledKernel : public NonCopyable { public: // NVF_API was added for nvfuser_extension. See examples/sinh_extension. NVF_API CompiledKernel() = default; + NVF_API CompiledKernel(Fusion* fusion, CompileParams compile_params); //! To compile a fusion with the 32-bit index type, CompileParams //! must be passed in. There used to be an index type associated //! with KernelArgumentHolder, but it is no longer the case. NVF_API void compileFusion( - Fusion* fusion, c10::Device device, const LaunchParams& launch_params, - CompileParams compile_params, SchedulerType scheduler_type = SchedulerType::None, int64_t fusion_id = 0, int64_t concrete_id = 0, @@ -99,9 +98,7 @@ class CompiledKernel : public NonCopyable { // Function to query whether compilation was attempted for a `CompiledKernel` bool isCompiled() const { - int num_compiled_artifacts = (fusion_ != nullptr) + (lowered_ != nullptr); - NVF_ERROR(num_compiled_artifacts <= 1); - return num_compiled_artifacts == 1; + return lowered_ != nullptr; }; // function to query whether a `CompiledKernel` has a compiled kernel to @@ -109,9 +106,6 @@ class CompiledKernel : public NonCopyable { bool hasCompiledKernel() const { if (compiled_kernel_ != nullptr) { NVF_ERROR(compiled_kernel_->function != nullptr); - NVF_ERROR( - fusion_ == nullptr, - "fusion_ should only be initialized when using expression evaluator."); } return validKernelId() && lowered_ && compiled_kernel_ != nullptr; }; @@ -125,14 +119,8 @@ class CompiledKernel : public NonCopyable { } Fusion* fusion() const { - NVF_ERROR(isCompiled()); - if (fusion_ != nullptr) { - return fusion_.get(); - } - if (lowered_ != nullptr) { - return lowered_->kernel()->as(); - } - NVF_THROW("unreachable because of the isCompiled check"); + NVF_ERROR(lowered_ != nullptr); + return lowered_->kernel()->as(); } const ThreadPredicateMap& threadPredMap() const { @@ -302,9 +290,11 @@ class CompiledKernel : public NonCopyable { std::unique_ptr& lowered() { return lowered_; } - std::unique_ptr& fusion() { - return fusion_; + Fusion* fusion() { + NVF_ERROR(lowered_ != nullptr); + return lowered_->kernel()->as(); } + const std::unique_ptr& lowered() const { return lowered_; } @@ -372,7 +362,7 @@ class CompiledKernel : public NonCopyable { private: CompileOptions options_; - + CompileParams compile_params_; // Assuming sm70 or above: // limit of statically allocated smem is 48 KB: // See: @@ -407,10 +397,7 @@ class CompiledKernel : public NonCopyable { // Kernel name for fusion executor std::string kernel_id_; - std::unique_ptr lowered_; - - // Initialized for non-compiled fusions - std::unique_ptr fusion_; + std::unique_ptr lowered_ = nullptr; // Track the block size this kernel was compiled with. If the block size // increases, recompile to adjust maxregister count. diff --git a/csrc/fusion_executor/executor.cpp b/csrc/fusion_executor/executor.cpp index 03df9b90d31..4446bbd44da 100644 --- a/csrc/fusion_executor/executor.cpp +++ b/csrc/fusion_executor/executor.cpp @@ -154,7 +154,7 @@ std::string FusionExecutor::getStructuredCode( const std::string& kernel_str, PrimDataType index_type) const { if (use_external_compiler_) { - return compiled_kernel_2_.getStructuredCode(kernel_str, index_type); + return compiled_kernel_2_->getStructuredCode(kernel_str, index_type); } // generating cuda code; std::string code = ""; @@ -186,7 +186,7 @@ std::string FusionExecutor::getStructuredCode( std::string FusionExecutor::getStructuredCode() const { if (use_external_compiler_) { - return compiled_kernel_2_.getStructuredCode(); + return compiled_kernel_2_->getStructuredCode(); } return getStructuredCode(kernelString(), kernel()->indexType()); } @@ -208,7 +208,7 @@ void FusionExecutor::compileFusion( "No output found for this kernel, aborting."); if (isExpressionEvaluated(_fusion)) { - fusion() = std::make_unique(*_fusion); + fusion_ = std::make_unique(*_fusion); return; } @@ -271,6 +271,9 @@ void FusionExecutor::compileFusion( // TMA operations require 32-bit indexing. compile_params.index_type = PrimDataType::Int32; } + if (!compile_params.index_type.has_value()) { + compile_params.index_type = arg_index_type; + } c10::DeviceGuard dg(options().device); @@ -282,12 +285,20 @@ void FusionExecutor::compileFusion( device_smem_limit_ = static_cast(properties->sharedMemPerBlockOptin); warp_size_ = properties->warpSize; + // Lowered is needed to compute launch parameters as it uses the CA map. We + // could modify that, but simply generating that part first. + use_external_compiler_ = true; + compiled_kernel_2_ = + std::make_unique(_fusion, compile_params); + // TODO: pass block_size here; std::optional dynamic_smem = std::nullopt; std::optional block_size = std::nullopt; auto launch_params = launch_constraints; if (!args.empty()) { - auto expr_eval = executor_utils::bindInputs(args, fusion().get()); + auto expr_eval = executor_utils::bindInputs( + args, compiled_kernel_2_->lowered()->kernel()->as()); + NVF_ERROR(compile_params.index_type.has_value()); launch_params = computeLaunchParams( launch_constraints, expr_eval, @@ -298,12 +309,12 @@ void FusionExecutor::compileFusion( NVF_ERROR(block_size > 0, "launch param inferred block size < 0"); } - use_external_compiler_ = true; - compiled_kernel_2_.compileFusion( - _fusion, + // Now that we have launch parameters we can compile the kernel. It's a bit + // odd we need launch parameters for compilation, need to go back and check + // why this is the case. + compiled_kernel_2_->compileFusion( options().device, launch_params, - compile_params, scheduler_type, fusion_id, concrete_id, @@ -331,8 +342,10 @@ LaunchParams FusionExecutor::computeLaunchParams( LaunchParams launch_params; auto data_cache = compileTimeDataCache(); - auto lower = lowered().get(); + if (getUsedTVs().empty()) { + setUsedTVs(); + } auto& used_tvs = getUsedTVs(); auto parallel_binding_ids_entry = executor_utils::caching::ExecutorCompileTimeEntry< @@ -532,7 +545,7 @@ std::vector FusionExecutor::getIntermediateBufferInfo( void FusionExecutor::setUsedTVs() { if (use_external_compiler_) { - compiled_kernel_2_.setUsedTVs(); + compiled_kernel_2_->setUsedTVs(); return; } auto used_vals = fusion()->usedMathVals(); @@ -849,7 +862,7 @@ void FusionExecutor::recompileKernel( const LaunchParams& new_launch_params, const CompileParams& new_compile_params) { if (use_external_compiler_) { - return compiled_kernel_2_.recompileKernel( + return compiled_kernel_2_->recompileKernel( new_launch_params, new_compile_params); } FUSER_PERF_SCOPE("FusionExecutor::runFusion::recompileKernel"); @@ -890,7 +903,7 @@ int64_t FusionExecutor::getAvailableDynamicSmemSize() { NVFUSER_CUDA_SAFE_CALL(cuFuncGetAttribute( &size, CU_FUNC_ATTRIBUTE_MAX_DYNAMIC_SHARED_SIZE_BYTES, - compiled_kernel_->function)); + compiledKernel()->function)); available_dynamic_smem_size_ = size; } return available_dynamic_smem_size_.value(); @@ -906,7 +919,7 @@ int64_t FusionExecutor::getStaticSmemSize() { NVFUSER_CUDA_SAFE_CALL(cuFuncGetAttribute( &size, CU_FUNC_ATTRIBUTE_SHARED_SIZE_BYTES, - compiled_kernel_->function)); + compiledKernel()->function)); static_smem_size_ = size; } return static_smem_size_.value(); @@ -945,7 +958,7 @@ int64_t FusionExecutor::ensureAvailableDynamicSmemSize( if (dynamic_smem_size > getAvailableDynamicSmemSize()) { validateDynamicSmemSize(dynamic_smem_size); NVFUSER_CUDA_SAFE_CALL(cuFuncSetAttribute( - compiled_kernel_->function, + compiledKernel()->function, CU_FUNC_ATTRIBUTE_MAX_DYNAMIC_SHARED_SIZE_BYTES, dynamic_smem_size)); available_dynamic_smem_size_ = dynamic_smem_size; @@ -1017,9 +1030,11 @@ std::vector FusionExecutor::runFusion( " provided number of outputs does not match fusion output"); // Bind fusion inputs - auto expr_eval = executor_utils::bindInputs(args, fusion().get()); + std::cout << "A" << std::endl; + auto expr_eval = executor_utils::bindInputs(args, fusion()); + std::cout << "B" << std::endl; - if (isExpressionEvaluated(fusion().get())) { + if (isExpressionEvaluated(fusion())) { FUSER_PERF_SCOPE("FusionExecutor::runFusion::evaluate_with_ExprEval"); outputs = evaluateFusionOutputs(outputs, expr_eval); if (isProfilerEnabled()) { @@ -1111,7 +1126,7 @@ std::vector FusionExecutor::runFusion( // only allocate outputs when not given if (outputs.empty()) { outputs = allocateOutputs( - fusion().get(), executor_entry->outputs, options().device, expr_eval); + fusion(), executor_entry->outputs, options().device, expr_eval); } args.push(outputs); @@ -1222,7 +1237,7 @@ std::vector FusionExecutor::runFusion( int blocks_per_sm = -1; NVFUSER_CUDA_SAFE_CALL(cuOccupancyMaxActiveBlocksPerMultiprocessor( &blocks_per_sm, - compiled_kernel_->function, + compiledKernel()->function, launch_params_.nThreads(), launch_params_.smem())); @@ -1249,7 +1264,7 @@ std::vector FusionExecutor::runFusion( if (!kernel()->summary().has_cooperative_grid_reduction) { FUSER_PERF_SCOPE("ExecutorRunFusion::cuLaunchKernel"); NVFUSER_CUDA_SAFE_CALL(cuLaunchKernel( - compiled_kernel_->function, + compiledKernel()->function, launch_params_.gdimx(), launch_params_.gdimy(), launch_params_.gdimz(), @@ -1263,7 +1278,7 @@ std::vector FusionExecutor::runFusion( } else { FUSER_PERF_SCOPE("ExecutorRunFusion::cuLaunchCooperativeKernel"); NVFUSER_CUDA_SAFE_CALL(cuLaunchCooperativeKernel( - compiled_kernel_->function, + compiledKernel()->function, launch_params_.gdimx(), launch_params_.gdimy(), launch_params_.gdimz(), @@ -1321,7 +1336,7 @@ void FusionExecutor::compileRtc( bool structured, PrimDataType index_type) { if (use_external_compiler_) { - return compiled_kernel_2_.compileRtc(code, name, structured, index_type); + return compiled_kernel_2_->compileRtc(code, name, structured, index_type); } FUSER_PERF_SCOPE("FusionExecutor::compileRtc"); NVF_ERROR( @@ -1337,7 +1352,7 @@ void FusionExecutor::compileRtc( } else { scode = code; } - compiled_kernel_ = + compiledKernel() = executor_utils::getCompiledKernel(std::nullopt, scode, name, kernelId()); } @@ -1346,7 +1361,7 @@ float FusionExecutor::runRtc( const std::vector& args, PrimDataType index_type) { if (use_external_compiler_) { - return compiled_kernel_2_.runRtc(launch_params, args, index_type); + return compiled_kernel_2_->runRtc(launch_params, args, index_type); } FUSER_PERF_SCOPE("FusionExecutor::runRtc"); @@ -1384,7 +1399,7 @@ float FusionExecutor::runRtc( } NVFUSER_CUDA_SAFE_CALL(cuLaunchKernel( - compiled_kernel_->function, + compiledKernel()->function, launch_params.gdimx(), launch_params.gdimy(), launch_params.gdimz(), @@ -1444,14 +1459,14 @@ flatbuffers::Offset FusionExecutor::serialize( &executor_entry_lookup_keys_fb, &executor_entry_lookup_values_fb, toUnderlying(kernel()->indexType()), - serialize(builder, compiled_kernel_.get())); + serialize(builder, compiledKernel().get())); } flatbuffers::Offset FusionExecutor::serialize( flatbuffers::FlatBufferBuilder& builder, const executor_utils::CompiledKernel* compiled_kernel) const { NVF_ERROR( - compiled_kernel_ != nullptr && + compiledKernel() != nullptr && (!compiled_kernel->cubin.empty() || !compiled_kernel->ptx.empty()), "Expected compiled cuda kernel before serializing FusionExecutor."); @@ -1580,7 +1595,7 @@ void FusionExecutor::deserialize( // TODO Should we set fusion_id, concrete_id, runtime_id, and group_id when we // skip compilation? if (isExpressionEvaluated(_fusion)) { - fusion() = std::make_unique(*_fusion); + fusion_ = std::make_unique(*_fusion); NVF_ERROR(!hasCompiledKernel(), "Failed to deserialize FusionExecutor"); return; } @@ -1640,7 +1655,7 @@ void FusionExecutor::deserialize( deserialize(buffer->executor_entry_lookup_values()->Get(idx))); } - compiled_kernel_ = executor_utils::getCompiledKernel( + compiledKernel() = executor_utils::getCompiledKernel( buffer->compiled_kernel(), compile_params); NVF_ERROR(hasCompiledKernel(), "Failed to deserialize FusionExecutor"); diff --git a/csrc/fusion_executor/executor.h b/csrc/fusion_executor/executor.h index 12e965583e3..3240d81573c 100644 --- a/csrc/fusion_executor/executor.h +++ b/csrc/fusion_executor/executor.h @@ -123,7 +123,7 @@ class FusionExecutor : public NonCopyable { // modify the lowering process. void registerLoweringHook(std::function hook) { if (use_external_compiler_) { - compiled_kernel_2_.registerLoweringHook(hook); + compiled_kernel_2_->registerLoweringHook(hook); } else { lowering_hooks_.push_back(std::move(hook)); } @@ -133,7 +133,7 @@ class FusionExecutor : public NonCopyable { // lowering. The main use case is for unit tests to modify the kernel. void registerPostLoweringHook(std::function hook) { if (use_external_compiler_) { - compiled_kernel_2_.registerPostLoweringHook(std::move(hook)); + compiled_kernel_2_->registerPostLoweringHook(std::move(hook)); } else { post_lowering_hooks_.push_back(std::move(hook)); } @@ -142,10 +142,10 @@ class FusionExecutor : public NonCopyable { // Function to query whether compilation was attempted for a `FusionExecutor` bool isCompiled() const { if (use_external_compiler_) { - return compiled_kernel_2_.isCompiled(); + return compiled_kernel_2_->isCompiled(); } - int num_compiled_artifacts = (fusion() != nullptr) + - (lowered() != nullptr) + (host_ir_container_ != nullptr); + int num_compiled_artifacts = (fusion_ != nullptr) + (lowered_ != nullptr) + + (host_ir_container_ != nullptr); NVF_ERROR(num_compiled_artifacts <= 1); return num_compiled_artifacts == 1; }; @@ -154,7 +154,7 @@ class FusionExecutor : public NonCopyable { // execute bool hasCompiledKernel() const { if (use_external_compiler_) { - return compiled_kernel_2_.hasCompiledKernel(); + return compiled_kernel_2_->hasCompiledKernel(); } if (compiled_kernel_ != nullptr) { NVF_ERROR(compiled_kernel_->function != nullptr); @@ -199,7 +199,7 @@ class FusionExecutor : public NonCopyable { kir::Kernel* kernel() const { if (use_external_compiler_) { - return compiled_kernel_2_.kernel(); + return compiled_kernel_2_->kernel(); } NVF_ERROR(lowered()); return lowered()->kernel(); @@ -209,13 +209,13 @@ class FusionExecutor : public NonCopyable { NVF_ERROR(isCompiled()); if (fusion_ != nullptr) { if (use_external_compiler_) { - return compiled_kernel_2_.fusion(); + return compiled_kernel_2_->fusion(); } return fusion_.get(); } if (lowered() != nullptr) { if (use_external_compiler_) { - return compiled_kernel_2_.fusion(); + return compiled_kernel_2_->lowered()->kernel()->as(); } return lowered()->kernel()->as(); } @@ -227,7 +227,7 @@ class FusionExecutor : public NonCopyable { const ThreadPredicateMap& threadPredMap() const { if (use_external_compiler_) { - return compiled_kernel_2_.threadPredMap(); + return compiled_kernel_2_->threadPredMap(); } return lowered()->threadPredMap(); } @@ -252,7 +252,7 @@ class FusionExecutor : public NonCopyable { //! get register spills (load + store) of the compiled kernel int getKernelRegisterSpills() const { if (use_external_compiler_) { - return compiled_kernel_2_.getKernelRegisterSpills(); + return compiled_kernel_2_->getKernelRegisterSpills(); } return compiled_kernel_->register_spills; } @@ -271,7 +271,7 @@ class FusionExecutor : public NonCopyable { //! Returns the string of the compiled kernel NVF_API std::string kernelString() const { if (use_external_compiler_) { - return compiled_kernel_2_.kernelString(); + return compiled_kernel_2_->kernelString(); } NVF_ERROR(!kernelCode().empty(), "Kernel code not generated"); return kernelCode(); @@ -288,13 +288,13 @@ class FusionExecutor : public NonCopyable { const std::unique_ptr& compiledKernel() const { if (use_external_compiler_) { - return compiled_kernel_2_.compiledKernel(); + return compiled_kernel_2_->compiledKernel(); } return compiled_kernel_; } std::unique_ptr& compiledKernel() { if (use_external_compiler_) { - return compiled_kernel_2_.compiledKernel(); + return compiled_kernel_2_->compiledKernel(); } return compiled_kernel_; } @@ -303,7 +303,7 @@ class FusionExecutor : public NonCopyable { NVF_API std::string disassembledBinary( const std::string& nvdisasm_args = "") const { if (use_external_compiler_) { - return compiled_kernel_2_.disassembledBinary(nvdisasm_args); + return compiled_kernel_2_->disassembledBinary(nvdisasm_args); } return executor_utils::disassembleBinary( compiled_kernel_->cubin, nvdisasm_args); @@ -312,7 +312,7 @@ class FusionExecutor : public NonCopyable { //! Returns the disassembled latest compiled binary NVF_API std::string disassembledKernelSASS() const { if (use_external_compiler_) { - return compiled_kernel_2_.disassembledKernelSASS(); + return compiled_kernel_2_->disassembledKernelSASS(); } return executor_utils::disassembleBinary( compiled_kernel_->cubin, "-fun 1 -c"); @@ -328,27 +328,27 @@ class FusionExecutor : public NonCopyable { const int64_t& groupId() const { if (use_external_compiler_) { - return compiled_kernel_2_.groupId(); + return compiled_kernel_2_->groupId(); } return group_id_; } int64_t& groupId() { if (use_external_compiler_) { - return compiled_kernel_2_.groupId(); + return compiled_kernel_2_->groupId(); } return group_id_; } void setGroupId(int64_t gid) { if (use_external_compiler_) { - return compiled_kernel_2_.setGroupId(gid); + return compiled_kernel_2_->setGroupId(gid); } group_id_ = gid; } bool validKernelId() const { if (use_external_compiler_) { - return compiled_kernel_2_.validKernelId(); + return compiled_kernel_2_->validKernelId(); } return !kernelId().empty(); } @@ -360,7 +360,7 @@ class FusionExecutor : public NonCopyable { int64_t runtime_id = 0, int64_t group_id = 0) { if (use_external_compiler_) { - return compiled_kernel_2_.createKernelId( + return compiled_kernel_2_->createKernelId( scheduler_type, fusion_id, concrete_id, runtime_id, group_id); } NVF_ERROR(fusion_id > -1, "Invalid fusion_id."); @@ -390,7 +390,7 @@ class FusionExecutor : public NonCopyable { std::string kernelName() const { if (use_external_compiler_) { - return compiled_kernel_2_.kernelName(); + return compiled_kernel_2_->kernelName(); } NVF_ERROR(!kernelId().empty(), "Invalid kernel name for fusion executor."); std::stringstream ss; @@ -419,7 +419,7 @@ class FusionExecutor : public NonCopyable { //! Internal knob used for debugging/profiling only void disableLaunchParamCache() { if (use_external_compiler_) { - compiled_kernel_2_.disableLaunchParamCache(); + compiled_kernel_2_->disableLaunchParamCache(); } else { disablePaarameterCache() = true; } @@ -460,7 +460,7 @@ class FusionExecutor : public NonCopyable { const std::vector& getUsedTVs() const { if (use_external_compiler_) { - return compiled_kernel_2_.getUsedTVs(); + return compiled_kernel_2_->getUsedTVs(); } return used_tvs_; }; @@ -545,43 +545,43 @@ class FusionExecutor : public NonCopyable { // Temporary accessors for refactor: CompileOptions& options() { if (use_external_compiler_) { - return compiled_kernel_2_.options(); + return compiled_kernel_2_->options(); } return options_; } int64_t& fusionId() { if (use_external_compiler_) { - return compiled_kernel_2_.fusionId(); + return compiled_kernel_2_->fusionId(); } return fusion_id_; } const int64_t& fusionId() const { if (use_external_compiler_) { - return compiled_kernel_2_.fusionId(); + return compiled_kernel_2_->fusionId(); } return fusion_id_; } int64_t& concreteId() { if (use_external_compiler_) { - return compiled_kernel_2_.concreteId(); + return compiled_kernel_2_->concreteId(); } return concrete_id_; } int64_t& runtimeId() { if (use_external_compiler_) { - return compiled_kernel_2_.runtimeId(); + return compiled_kernel_2_->runtimeId(); } return runtime_id_; } const int64_t& concreteId() const { if (use_external_compiler_) { - return compiled_kernel_2_.concreteId(); + return compiled_kernel_2_->concreteId(); } return concrete_id_; } const int64_t& runtimeId() const { if (use_external_compiler_) { - return compiled_kernel_2_.runtimeId(); + return compiled_kernel_2_->runtimeId(); } return runtime_id_; } @@ -590,103 +590,103 @@ class FusionExecutor : public NonCopyable { } SchedulerType& schedulerType() { if (use_external_compiler_) { - return compiled_kernel_2_.schedulerType(); + return compiled_kernel_2_->schedulerType(); } return scheduler_type_; } const SchedulerType& schedulerType() const { if (use_external_compiler_) { - return compiled_kernel_2_.schedulerType(); + return compiled_kernel_2_->schedulerType(); } return scheduler_type_; } std::string& kernelId() { if (use_external_compiler_) { - return compiled_kernel_2_.kernelId(); + return compiled_kernel_2_->kernelId(); } return kernel_id_; } const std::string& kernelId() const { if (use_external_compiler_) { - return compiled_kernel_2_.kernelId(); + return compiled_kernel_2_->kernelId(); } return kernel_id_; } std::unique_ptr& lowered() { if (use_external_compiler_) { - return compiled_kernel_2_.lowered(); + return compiled_kernel_2_->lowered(); } return lowered_; } const std::unique_ptr& lowered() const { if (use_external_compiler_) { - return compiled_kernel_2_.lowered(); + return compiled_kernel_2_->lowered(); } return lowered_; } - std::unique_ptr& fusion() { + Fusion* fusion() { if (use_external_compiler_) { - return compiled_kernel_2_.fusion(); + return compiled_kernel_2_->fusion(); } - return fusion_; + return fusion_.get(); } int64_t& blockSizeHighWaterMark() { if (use_external_compiler_) { - return compiled_kernel_2_.blockSizeHighWaterMark(); + return compiled_kernel_2_->blockSizeHighWaterMark(); } return block_size_high_water_mark_; } int64_t& maxrregcountHighWaterMark() { if (use_external_compiler_) { - return compiled_kernel_2_.maxrregcountHighWaterMark(); + return compiled_kernel_2_->maxrregcountHighWaterMark(); } return maxrregcount_high_water_mark_; } const int64_t& blockSizeHighWaterMark() const { if (use_external_compiler_) { - return compiled_kernel_2_.blockSizeHighWaterMark(); + return compiled_kernel_2_->blockSizeHighWaterMark(); } return block_size_high_water_mark_; } const int64_t& maxrregcountHighWaterMark() const { if (use_external_compiler_) { - return compiled_kernel_2_.maxrregcountHighWaterMark(); + return compiled_kernel_2_->maxrregcountHighWaterMark(); } return maxrregcount_high_water_mark_; } bool& disablePaarameterCache() { if (use_external_compiler_) { - return compiled_kernel_2_.disablePaarameterCache(); + return compiled_kernel_2_->disablePaarameterCache(); } return disable_parameter_cache_; } std::string& kernelCode() { if (use_external_compiler_) { - return compiled_kernel_2_.kernelCode(); + return compiled_kernel_2_->kernelCode(); } return kernel_code_; } const std::string& kernelCode() const { if (use_external_compiler_) { - return compiled_kernel_2_.kernelCode(); + return compiled_kernel_2_->kernelCode(); } return kernel_code_; } std::vector>& loweringHooks() { if (use_external_compiler_) { - return compiled_kernel_2_.loweringHooks(); + return compiled_kernel_2_->loweringHooks(); } return lowering_hooks_; } std::vector>& postLoweringHooks() { if (use_external_compiler_) { - return compiled_kernel_2_.postLoweringHooks(); + return compiled_kernel_2_->postLoweringHooks(); } return post_lowering_hooks_; } private: - CompiledKernel compiled_kernel_2_; + std::unique_ptr compiled_kernel_2_; bool use_external_compiler_ = false; CompileOptions options_; From bde8e46f488c26a5ee0e5ac65b6923b80109d859 Mon Sep 17 00:00:00 2001 From: Christian Sarofeen Date: Fri, 4 Oct 2024 06:41:11 -0700 Subject: [PATCH 04/11] Fix almost all tests. --- csrc/fusion_executor/executor.cpp | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/csrc/fusion_executor/executor.cpp b/csrc/fusion_executor/executor.cpp index 4446bbd44da..3b5e0aae094 100644 --- a/csrc/fusion_executor/executor.cpp +++ b/csrc/fusion_executor/executor.cpp @@ -212,10 +212,18 @@ void FusionExecutor::compileFusion( return; } - const std::vector& exprs = _fusion->exprs(); - if (std::all_of(exprs.begin(), exprs.end(), [](Expr* e) { + std::vector exprs = _fusion->exprs(); + if (std::any_of(exprs.begin(), exprs.end(), [](Expr* e) { return isResharding(e) && isLowerableToCommunication(e); })) { + NVF_ERROR( + std::all_of( + exprs.begin(), + exprs.end(), + [](Expr* e) { + return isResharding(e) && isLowerableToCommunication(e); + }), + "Could not execute fusion as all expressions in a host IR container must be communication based at this point."); host_ir_container_ = std::make_unique(); IrCloner cloner = Fusion::copy(_fusion, host_ir_container_.get()); for (Expr* e : exprs) { @@ -1030,9 +1038,7 @@ std::vector FusionExecutor::runFusion( " provided number of outputs does not match fusion output"); // Bind fusion inputs - std::cout << "A" << std::endl; auto expr_eval = executor_utils::bindInputs(args, fusion()); - std::cout << "B" << std::endl; if (isExpressionEvaluated(fusion())) { FUSER_PERF_SCOPE("FusionExecutor::runFusion::evaluate_with_ExprEval"); From 3cfe0bdf7d43bd3a98681daba37dd9a6c80f4477 Mon Sep 17 00:00:00 2001 From: Christian Sarofeen Date: Fri, 4 Oct 2024 08:42:04 -0700 Subject: [PATCH 05/11] Fix the last test. --- csrc/fusion_executor/executor.cpp | 8 ++++++++ csrc/fusion_executor/executor.h | 12 ++---------- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/csrc/fusion_executor/executor.cpp b/csrc/fusion_executor/executor.cpp index 3b5e0aae094..b95f557c4b6 100644 --- a/csrc/fusion_executor/executor.cpp +++ b/csrc/fusion_executor/executor.cpp @@ -317,6 +317,14 @@ void FusionExecutor::compileFusion( NVF_ERROR(block_size > 0, "launch param inferred block size < 0"); } + for (const auto& hook : lowering_hooks_) { + compiled_kernel_2_->registerLoweringHook(hook); + } + + for (const auto& hook : post_lowering_hooks_) { + compiled_kernel_2_->registerPostLoweringHook(hook); + } + // Now that we have launch parameters we can compile the kernel. It's a bit // odd we need launch parameters for compilation, need to go back and check // why this is the case. diff --git a/csrc/fusion_executor/executor.h b/csrc/fusion_executor/executor.h index 3240d81573c..16ececc016c 100644 --- a/csrc/fusion_executor/executor.h +++ b/csrc/fusion_executor/executor.h @@ -122,21 +122,13 @@ class FusionExecutor : public NonCopyable { // before running lowering passes. The main use case is for unit tests to // modify the lowering process. void registerLoweringHook(std::function hook) { - if (use_external_compiler_) { - compiled_kernel_2_->registerLoweringHook(hook); - } else { - lowering_hooks_.push_back(std::move(hook)); - } + lowering_hooks_.push_back(std::move(hook)); } // Register a post-lowering hooks that are called to modify the kernel after // lowering. The main use case is for unit tests to modify the kernel. void registerPostLoweringHook(std::function hook) { - if (use_external_compiler_) { - compiled_kernel_2_->registerPostLoweringHook(std::move(hook)); - } else { - post_lowering_hooks_.push_back(std::move(hook)); - } + post_lowering_hooks_.push_back(std::move(hook)); } // Function to query whether compilation was attempted for a `FusionExecutor` From f309bfb9765a422bef0433d3df76985b8c0e2727 Mon Sep 17 00:00:00 2001 From: Christian Sarofeen Date: Fri, 4 Oct 2024 12:26:47 -0700 Subject: [PATCH 06/11] Fix multi-gpu tests. --- csrc/runtime/executor.cpp | 1 - csrc/runtime/executor.h | 18 +++++++++++++++++- 2 files changed, 17 insertions(+), 2 deletions(-) diff --git a/csrc/runtime/executor.cpp b/csrc/runtime/executor.cpp index b9cb59d171d..4805c808df9 100644 --- a/csrc/runtime/executor.cpp +++ b/csrc/runtime/executor.cpp @@ -1047,7 +1047,6 @@ std::vector FusionExecutor::runFusion( // Bind fusion inputs auto expr_eval = executor_utils::bindInputs(args, fusion()); - if (isExpressionEvaluated(fusion())) { FUSER_PERF_SCOPE("FusionExecutor::runFusion::evaluate_with_ExprEval"); outputs = evaluateFusionOutputs(outputs, expr_eval); diff --git a/csrc/runtime/executor.h b/csrc/runtime/executor.h index 55560886f0d..eedd3d651fb 100644 --- a/csrc/runtime/executor.h +++ b/csrc/runtime/executor.h @@ -620,7 +620,23 @@ class FusionExecutor : public NonCopyable { if (use_external_compiler_) { return compiled_kernel_2_->fusion(); } - return fusion_.get(); + NVF_ERROR(isCompiled()); + if (fusion_ != nullptr) { + if (use_external_compiler_) { + return compiled_kernel_2_->fusion(); + } + return fusion_.get(); + } + if (lowered() != nullptr) { + if (use_external_compiler_) { + return compiled_kernel_2_->lowered()->kernel()->as(); + } + return lowered()->kernel()->as(); + } + if (host_ir_container_ != nullptr) { + return host_ir_container_->as(); + } + NVF_THROW("unreachable because of the isCompiled check"); } int64_t& blockSizeHighWaterMark() { if (use_external_compiler_) { From 484a1edbd4e84a667ca8594722028702111d8b43 Mon Sep 17 00:00:00 2001 From: Christian Sarofeen Date: Sat, 5 Oct 2024 20:09:24 +0000 Subject: [PATCH 07/11] Fix device setting. --- csrc/runtime/executor.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/csrc/runtime/executor.cpp b/csrc/runtime/executor.cpp index 4805c808df9..fbed1fbeed7 100644 --- a/csrc/runtime/executor.cpp +++ b/csrc/runtime/executor.cpp @@ -207,6 +207,9 @@ void FusionExecutor::compileFusion( !_fusion->outputs().empty(), "No output found for this kernel, aborting."); + // TODO: refactor the options_ passed through + options_.device = c10::Device(c10::DeviceType::CUDA, args.getDeviceIndex()); + if (isExpressionEvaluated(_fusion)) { fusion_ = std::make_unique(*_fusion); return; From 5330e6eb714958723abf5455947b22cb52bf9138 Mon Sep 17 00:00:00 2001 From: Christian Sarofeen Date: Sat, 5 Oct 2024 18:51:06 -0700 Subject: [PATCH 08/11] Start to cleanup duplicate code. --- benchmarks/cpp/matmul.cpp | 9 +- csrc/evaluator_common.cpp | 6 +- csrc/python_frontend/fusion_definition.cpp | 14 +- csrc/runtime/compiled_kernel.cpp | 10 +- csrc/runtime/compiled_kernel.h | 8 +- csrc/runtime/executor.cpp | 559 +++++------------- csrc/runtime/executor.h | 402 +------------ csrc/runtime/fusion_executor_cache.cpp | 20 +- csrc/runtime/fusion_kernel_runtime.cpp | 9 +- tests/cpp/test_alias.cpp | 13 +- tests/cpp/test_circular_buffering.cpp | 3 +- tests/cpp/test_external_src.cpp | 12 +- tests/cpp/test_gpu1.cpp | 28 +- tests/cpp/test_gpu2.cpp | 30 +- tests/cpp/test_gpu3.cpp | 88 ++- tests/cpp/test_gpu_fused_reduction.cpp | 92 ++- tests/cpp/test_gpu_outer_reduction.cpp | 25 +- tests/cpp/test_gpu_utils.cpp | 5 +- tests/cpp/test_indexing_advanced.cpp | 8 +- tests/cpp/test_matmul.cpp | 61 +- tests/cpp/test_matmul_aten_evaluation.cpp | 4 +- tests/cpp/test_matmul_sass.cpp | 4 +- tests/cpp/test_mbarrier.cpp | 2 +- tests/cpp/test_memory.cpp | 130 ++-- .../test_multidevice_lower_communication.cpp | 8 +- tests/cpp/test_no_op.cpp | 4 +- tests/cpp/test_predicate_elimination.cpp | 3 +- tests/cpp/test_resize.cpp | 14 +- tests/cpp/test_swizzle.cpp | 2 +- tests/cpp/test_translate_mma.cpp | 17 +- 30 files changed, 560 insertions(+), 1030 deletions(-) diff --git a/benchmarks/cpp/matmul.cpp b/benchmarks/cpp/matmul.cpp index 4f93dfbaf62..15bb9b23a97 100644 --- a/benchmarks/cpp/matmul.cpp +++ b/benchmarks/cpp/matmul.cpp @@ -178,7 +178,8 @@ static void SingleMatmulBase( FusionExecutor fe; fe.compileFusion(fusion, args, launch_constraints, cparams); NVF_CHECK( - getBankConflictInfo(fe.kernel(), launch_constraints).empty(), + getBankConflictInfo(fe.compiledKernel()->kernel(), launch_constraints) + .empty(), "Shared memory bank conflict not removed."); std::vector aten_inputs({inputs.first, inputs.second}); @@ -359,7 +360,7 @@ static void SingleMatmulPartitionedK( auto lparams = LaunchParams(); fe.compileFusion(fusion, args, lparams, cparams); NVF_CHECK( - getBankConflictInfo(fe.kernel(), lparams).empty(), + getBankConflictInfo(fe.compiledKernel()->kernel(), lparams).empty(), "Shared memory bank conflict not removed."); // Warm up run @@ -466,7 +467,9 @@ static void NvFuserScheduler_MatmulSplitKReduction( fusion, args, heuristic_params->lparams, heuristic_params->cparams); NVF_CHECK( - getBankConflictInfo(fe.kernel(), heuristic_params->lparams).empty(), + getBankConflictInfo( + fe.compiledKernel()->kernel(), heuristic_params->lparams) + .empty(), "Shared memory bank conflict not removed."); // Warm up run diff --git a/csrc/evaluator_common.cpp b/csrc/evaluator_common.cpp index 34852d7d279..ce44ce01be3 100644 --- a/csrc/evaluator_common.cpp +++ b/csrc/evaluator_common.cpp @@ -184,7 +184,11 @@ void PrecomputedValues::bindValues( const std::vector& inputs, const KernelArgumentHolder& args) { NVF_ERROR( - args.size() == inputs.size(), "kernel inputs size does not match args"); + args.size() == inputs.size(), + "kernel inputs size does not match args: ", + args.size(), + " vs ", + inputs.size()); for (const auto i : c10::irange((int64_t)inputs.size())) { const auto input = inputs[i]; diff --git a/csrc/python_frontend/fusion_definition.cpp b/csrc/python_frontend/fusion_definition.cpp index 717315967c7..eb045548e45 100644 --- a/csrc/python_frontend/fusion_definition.cpp +++ b/csrc/python_frontend/fusion_definition.cpp @@ -416,10 +416,11 @@ std::string FusionDefinition::lastCudaCode( if (!override_user_schedule && (user_exec != nullptr)) { if (intrinsic_code) { - result = user_exec->getStructuredCode( - user_exec->kernelString(), user_exec->kernel()->indexType()); + result = user_exec->compiledKernel()->getStructuredCode( + user_exec->compiledKernel()->kernelString(), + user_exec->compiledKernel()->kernel()->indexType()); } else { - result = user_exec->kernelString(); + result = user_exec->compiledKernel()->kernelString(); } } else { result = scheds->auto_gen_schedules->getMostRecentCode(intrinsic_code); @@ -445,10 +446,11 @@ std::string FusionDefinition::cudaCodeFor( scheds, user_sched_id.value(), device); auto user_exec = user_sched.executor.get(); if (intrinsic_code) { - return user_exec->getStructuredCode( - user_exec->kernelString(), user_exec->kernel()->indexType()); + return user_exec->compiledKernel()->getStructuredCode( + user_exec->compiledKernel()->kernelString(), + user_exec->compiledKernel()->kernel()->indexType()); } else { - return user_exec->kernelString(); + return user_exec->compiledKernel()->kernelString(); } } } diff --git a/csrc/runtime/compiled_kernel.cpp b/csrc/runtime/compiled_kernel.cpp index bf94ba8a8a3..228c2bf2fde 100644 --- a/csrc/runtime/compiled_kernel.cpp +++ b/csrc/runtime/compiled_kernel.cpp @@ -152,7 +152,7 @@ NVF_API CompiledKernel::CompiledKernel( void CompiledKernel::compileFusion( c10::Device device, - const LaunchParams& launch_params, + int64_t block_size, SchedulerType scheduler_type, int64_t fusion_id, int64_t concrete_id, @@ -318,8 +318,7 @@ void CompiledKernel::compileFusion( NVF_THROW(ss.str()); } - NVF_ERROR( - launch_params.nThreads() > 0, "launch param inferred block size < 0"); + NVF_ERROR(block_size > 0, "launch param inferred block size < 0"); // TODO: high water mark should be computed via occupancy API after // compilation. @@ -328,7 +327,7 @@ void CompiledKernel::compileFusion( // compilation, it will just generate a kernel that gets ditched at the first // run - not great. We should have better heuristics. block_size_high_water_mark_ = - std::max(launch_params.nThreads(), block_size_high_water_mark_); + std::max(block_size, block_size_high_water_mark_); maxrregcount_high_water_mark_ = compile_params_.maxrregcount; compiled_kernel_ = executor_utils::getCompiledKernel( kernel_code_, @@ -336,7 +335,7 @@ void CompiledKernel::compileFusion( kernelName(), kernel_id_, compile_params_, - launch_params.nThreads()); + block_size); NVF_ERROR(validKernelId(), "Invalid kernel id for CompiledKernel."); @@ -698,7 +697,6 @@ void CompiledKernel::recompileKernel( const LaunchParams& new_launch_params, const CompileParams& new_compile_params) { FUSER_PERF_SCOPE("CompiledKernel::runFusion::recompileKernel"); - const auto structured_code = getStructuredCode(); block_size_high_water_mark_ = new_launch_params.nThreads(); maxrregcount_high_water_mark_ = new_compile_params.maxrregcount; diff --git a/csrc/runtime/compiled_kernel.h b/csrc/runtime/compiled_kernel.h index d72e625d1f2..eca5ca7221e 100644 --- a/csrc/runtime/compiled_kernel.h +++ b/csrc/runtime/compiled_kernel.h @@ -29,7 +29,7 @@ namespace nvfuser { -// TODO: Should this actually be in launch params? +// TODO: Bury this someplace more useful. struct CompileOptions { c10::Device device = c10::Device(c10::DeviceType::CUDA, 0); }; @@ -45,7 +45,7 @@ class CompiledKernel : public NonCopyable { //! with KernelArgumentHolder, but it is no longer the case. NVF_API void compileFusion( c10::Device device, - const LaunchParams& launch_params, + int64_t block_size, SchedulerType scheduler_type = SchedulerType::None, int64_t fusion_id = 0, int64_t concrete_id = 0, @@ -123,10 +123,6 @@ class CompiledKernel : public NonCopyable { return lowered_->kernel()->as(); } - const ThreadPredicateMap& threadPredMap() const { - return lowered_->threadPredMap(); - } - //! get register spills (load + store) of the compiled kernel int getKernelRegisterSpills() const { return compiled_kernel_->register_spills; diff --git a/csrc/runtime/executor.cpp b/csrc/runtime/executor.cpp index fbed1fbeed7..25ba5b2254c 100644 --- a/csrc/runtime/executor.cpp +++ b/csrc/runtime/executor.cpp @@ -87,55 +87,6 @@ static const std::string& includeStdComplex() { return result; } -// When executing nvFuser with: NVFUSER_EXTERNAL_SRC=file1.cu,file2.cu -// This function retrieves structured code from the specified files. -// The files should be comma-separated, and their order corresponds to the -// fusion_id order. If the provided number of files is fewer than the fusion -// segments, the function will resort to the available files in sequence -// and issue a warning. -std::string getStructuredCodeFromExternalFiles(const int64_t fusion_id) { - auto external_code_path = getNvFuserEnv("EXTERNAL_SRC"); - if (!external_code_path) { - return ""; - } - std::string all_external_code_paths(external_code_path); - if (all_external_code_paths.empty() || fusion_id < 1) { - return ""; - } - auto getExternalCodeFile = - [fusion_id](const std::string& input) -> std::string { - std::stringstream ss(input); - std::string token; - int64_t count = 0; - while (std::getline(ss, token, ',')) { - if (++count == fusion_id) { - return token; - } - } - debug() - << "Didn't find requested external source code. Will use generated code!\n" - << "Number of source code files should equal the number of fusion segments.\n" - << "External source code filenames should be delineated with commas, e.g.: file1.cu,file2.cu.\n"; - return ""; - }; - - std::string single_code_path = getExternalCodeFile(all_external_code_paths); - if (single_code_path.empty()) { - return ""; - } - std::ifstream cuda_src(single_code_path); - if (!cuda_src.is_open()) { - debug() << "Failed to open external source file: " << single_code_path - << std::endl; - return ""; - } - debug() << "--------> Compiling external CUDA code: " << single_code_path - << std::endl; - - std::stringstream buffer; - buffer << cuda_src.rdbuf(); - return buffer.str(); -} } // namespace FusionExecutor::FusionExecutor() @@ -144,53 +95,12 @@ FusionExecutor::FusionExecutor() std::unique_ptr& FusionExecutor:: evaluatorPrecomputedValues() { if (!evaluator_precomputed_values_) { - evaluator_precomputed_values_ = - std::make_unique(lowered()->kernel()); + evaluator_precomputed_values_ = std::make_unique( + compiledKernel_()->lowered()->kernel()); } return evaluator_precomputed_values_; } -std::string FusionExecutor::getStructuredCode( - const std::string& kernel_str, - PrimDataType index_type) const { - if (use_external_compiler_) { - return compiled_kernel_2_->getStructuredCode(kernel_str, index_type); - } - // generating cuda code; - std::string code = ""; - code += includeStdComplex(); - code += std::string("namespace {\n") + defineTypes() + - defineIndexType(index_type) + executor_utils::kernelPreamble() + - kernel_str + "}\n"; - - if (isDebugDumpEnabled(DebugDumpOption::CudaKernel)) { - debug() << "\n======= Codegen output for kernel: " << kernelName() - << " =======\n\n" - << kernel_str << "\n======================================\n\n"; - } else if (isDebugDumpEnabled(DebugDumpOption::CudaFull)) { - debug() << "\n======= Codegen output for kernel: " << kernelName() - << " =======\n\n" - << code << "\n======================================\n\n"; - } - if (isDebugDumpEnabled(DebugDumpOption::CudaToFile)) { - std::stringstream file_name; - file_name << "__tmp_kernel_" << kernelId() << ".cu"; - debug() << "PRINTING: " << file_name.str() << std::endl; - std::ofstream out(file_name.str()); - out << code << std::endl; - out.close(); - } - - return code; -} - -std::string FusionExecutor::getStructuredCode() const { - if (use_external_compiler_) { - return compiled_kernel_2_->getStructuredCode(); - } - return getStructuredCode(kernelString(), kernel()->indexType()); -} - void FusionExecutor::compileFusion( Fusion* _fusion, const KernelArgumentHolder& args, @@ -202,13 +112,11 @@ void FusionExecutor::compileFusion( int64_t runtime_id, int64_t group_id) { FUSER_PERF_SCOPE("FusionExecutor::compileFusion"); - NVF_ERROR( !_fusion->outputs().empty(), "No output found for this kernel, aborting."); - // TODO: refactor the options_ passed through - options_.device = c10::Device(c10::DeviceType::CUDA, args.getDeviceIndex()); + auto device = c10::Device(c10::DeviceType::CUDA, args.getDeviceIndex()); if (isExpressionEvaluated(_fusion)) { fusion_ = std::make_unique(*_fusion); @@ -286,11 +194,10 @@ void FusionExecutor::compileFusion( compile_params.index_type = arg_index_type; } - c10::DeviceGuard dg(options().device); + c10::DeviceGuard dg(device); - NVF_ERROR( - options().device.is_cuda(), "Provided device to CUDA fuser is the CPU."); - auto properties = at::cuda::getDeviceProperties(options().device.index()); + NVF_ERROR(device.is_cuda(), "Provided device to CUDA fuser is the CPU."); + auto properties = at::cuda::getDeviceProperties(device.index()); // TODO: These properties should be set as part of the constructor so that it // can be const device_smem_limit_ = static_cast(properties->sharedMemPerBlockOptin); @@ -298,9 +205,7 @@ void FusionExecutor::compileFusion( // Lowered is needed to compute launch parameters as it uses the CA map. We // could modify that, but simply generating that part first. - use_external_compiler_ = true; - compiled_kernel_2_ = - std::make_unique(_fusion, compile_params); + compiledKernel_() = std::make_unique(_fusion, compile_params); // TODO: pass block_size here; std::optional dynamic_smem = std::nullopt; @@ -308,7 +213,7 @@ void FusionExecutor::compileFusion( auto launch_params = launch_constraints; if (!args.empty()) { auto expr_eval = executor_utils::bindInputs( - args, compiled_kernel_2_->lowered()->kernel()->as()); + args, compiledKernel_()->lowered()->kernel()->as()); NVF_ERROR(compile_params.index_type.has_value()); launch_params = computeLaunchParams( launch_constraints, @@ -321,19 +226,19 @@ void FusionExecutor::compileFusion( } for (const auto& hook : lowering_hooks_) { - compiled_kernel_2_->registerLoweringHook(hook); + compiledKernel_()->registerLoweringHook(hook); } for (const auto& hook : post_lowering_hooks_) { - compiled_kernel_2_->registerPostLoweringHook(hook); + compiledKernel_()->registerPostLoweringHook(hook); } // Now that we have launch parameters we can compile the kernel. It's a bit // odd we need launch parameters for compilation, need to go back and check // why this is the case. - compiled_kernel_2_->compileFusion( - options().device, - launch_params, + compiledKernel_()->compileFusion( + compiledKernel_()->options().device, + launch_params.nThreads(), scheduler_type, fusion_id, concrete_id, @@ -361,11 +266,11 @@ LaunchParams FusionExecutor::computeLaunchParams( LaunchParams launch_params; auto data_cache = compileTimeDataCache(); - auto lower = lowered().get(); - if (getUsedTVs().empty()) { - setUsedTVs(); + auto lower = compiledKernel_()->lowered().get(); + if (compiledKernel_()->getUsedTVs().empty()) { + compiledKernel_()->setUsedTVs(); } - auto& used_tvs = getUsedTVs(); + auto& used_tvs = compiledKernel_()->getUsedTVs(); auto parallel_binding_ids_entry = executor_utils::caching::ExecutorCompileTimeEntry< executor_utils::caching::ParallelBindingIterDomains>( @@ -454,7 +359,7 @@ LaunchParams FusionExecutor::computeLaunchParams( expr_eval.precomputedValues()->evaluate(); } - const auto kernel = lowered()->kernel(); + const auto kernel = compiledKernel_()->lowered()->kernel(); const auto& kernel_summary = kernel->summary(); // Calculate Dynamic Shared Memory Size @@ -503,7 +408,7 @@ LaunchParams FusionExecutor::computeLaunchParams( // This check is only done once a kernel has been compiled, since // maybe_available_dynamic_smem_ needs to be evaluated on // a compiled kernel. - if (hasCompiledKernel()) { + if (compiledKernel_()->hasCompiledKernel()) { validateDynamicSmemSize(dynamic_smem_size); } @@ -518,7 +423,7 @@ std::vector FusionExecutor::getIntermediateBufferInfo( FUSER_PERF_SCOPE("FusionExecutor::getIntermediateBufferInfo"); std::vector global_buffers; - const auto kernel = lowered()->kernel(); + const auto kernel = compiledKernel_()->lowered()->kernel(); const auto& kernel_summary = kernel->summary(); for (auto alloc : kernel_summary.global_allocations) { @@ -562,17 +467,6 @@ std::vector FusionExecutor::getIntermediateBufferInfo( return global_buffers; } -void FusionExecutor::setUsedTVs() { - if (use_external_compiler_) { - compiled_kernel_2_->setUsedTVs(); - return; - } - auto used_vals = fusion()->usedMathVals(); - auto used_tvs = ir_utils::filterByType(used_vals); - used_tvs_.clear(); - used_tvs_.insert(used_tvs_.begin(), used_tvs.begin(), used_tvs.end()); -} - namespace { // Make sure the index type of Kernel is valid @@ -700,20 +594,25 @@ void FusionExecutor::initializeExecutorEntry( auto launch_params = computeLaunchParams( launch_constraints, expr_eval, warp_size_, index_type); - for (const auto& entry : kernel()->summary().validations) { + for (const auto& entry : compiledKernel_()->kernel()->summary().validations) { NVF_CHECK(expr_eval.evaluate(entry.first).as(), entry.second); } executor_utils::validateVectorizedTensors( - kernel(), args, outputs, compileTimeDataCache(), expr_eval); + compiledKernel_()->kernel(), + args, + outputs, + compileTimeDataCache(), + expr_eval); - executor_utils::validateCircularBuffering(kernel(), expr_eval); + executor_utils::validateCircularBuffering( + compiledKernel_()->kernel(), expr_eval); // Check that a full warp exists in blockDim.x if the kernel contains // ElectSync predicate. constexpr int64_t warp_size = 32; NVF_ERROR( - !kernel()->summary().has_elect_sync_predicate || + !compiledKernel_()->kernel()->summary().has_elect_sync_predicate || launch_params.bdimx() >= warp_size, "This cuda kernel contains electSync predicate. " "Expected blockDim.x >= 32 but found ", @@ -722,8 +621,8 @@ void FusionExecutor::initializeExecutorEntry( std::vector output_info; if (outputs.empty()) { - output_info = - getBufferInfos(expr_eval, index_type, lowered()->kernel()->outputs()); + output_info = getBufferInfos( + expr_eval, index_type, compiledKernel_()->kernel()->outputs()); } else { // Need to save the information necessary for allocations as // future uses of this ExecutorEntry may not be provided with @@ -877,77 +776,39 @@ void FusionExecutor::recomputeArgs( } } -void FusionExecutor::recompileKernel( - const LaunchParams& new_launch_params, - const CompileParams& new_compile_params) { - if (use_external_compiler_) { - return compiled_kernel_2_->recompileKernel( - new_launch_params, new_compile_params); - } - FUSER_PERF_SCOPE("FusionExecutor::runFusion::recompileKernel"); - - const auto structured_code = getStructuredCode(); - blockSizeHighWaterMark() = new_launch_params.nThreads(); - maxrregcountHighWaterMark() = new_compile_params.maxrregcount; - - compiled_kernel_ = executor_utils::getCompiledKernel( - kernelCode(), - structured_code, - kernelName(), - kernelId(), - new_compile_params, - blockSizeHighWaterMark()); - - resetCompiledKernelProperties(); - - if (kernel()->summary().has_cooperative_grid_reduction) { - // We need to increase shared memory before kernel launch, but also before - // calling into `validateCooperativeLaunch`! - // So we need to do it there before calling into the validation, to avoid - // false positives - ensureAvailableDynamicSmemSize(new_launch_params.smem()); - validateCooperativeLaunch( - compiled_kernel_->function, - new_launch_params, - options().device.index()); - } -} - +// TODO: Move to CompiledKernel int64_t FusionExecutor::getAvailableDynamicSmemSize() { - NVF_ERROR( - hasCompiledKernel(), - "Cannot get dynamic smem size unless kernel is compiled"); if (!available_dynamic_smem_size_.has_value()) { int size = 0; NVFUSER_CUDA_SAFE_CALL(cuFuncGetAttribute( &size, CU_FUNC_ATTRIBUTE_MAX_DYNAMIC_SHARED_SIZE_BYTES, - compiledKernel()->function)); + compiledKernel_()->compiledKernel()->function)); available_dynamic_smem_size_ = size; } return available_dynamic_smem_size_.value(); } +// TODO: Move to CompiledKernel int64_t FusionExecutor::getStaticSmemSize() { - NVF_ERROR( - hasCompiledKernel(), - "Cannot get static smem size unless kernel is compiled"); if (!static_smem_size_.has_value()) { int size = 0; // Is this really a costly operation worth caching? NVFUSER_CUDA_SAFE_CALL(cuFuncGetAttribute( &size, CU_FUNC_ATTRIBUTE_SHARED_SIZE_BYTES, - compiledKernel()->function)); + compiledKernel_()->compiledKernel()->function)); static_smem_size_ = size; } return static_smem_size_.value(); } +// TODO: Move to CompiledKernel void FusionExecutor::validateDynamicSmemSize(int64_t dynamic_smem_size) { // If specified, check that dynamic smem size matches what the scheduler // expects - int64_t expected_dynamic_smem_size = fusion()->expectedDynamicSmemBytes(); + int64_t expected_dynamic_smem_size = + compiledKernel_()->fusion()->expectedDynamicSmemBytes(); if (expected_dynamic_smem_size >= 0) { NVF_ERROR( dynamic_smem_size == expected_dynamic_smem_size, @@ -969,15 +830,16 @@ void FusionExecutor::validateDynamicSmemSize(int64_t dynamic_smem_size) { device_smem_limit_); } +// TODO: Move to CompiledKernel int64_t FusionExecutor::ensureAvailableDynamicSmemSize( int64_t dynamic_smem_size) { NVF_ERROR( - hasCompiledKernel(), + compiledKernel_()->hasCompiledKernel(), "Cannot set dynamic smem size unless kernel is compiled"); if (dynamic_smem_size > getAvailableDynamicSmemSize()) { validateDynamicSmemSize(dynamic_smem_size); NVFUSER_CUDA_SAFE_CALL(cuFuncSetAttribute( - compiledKernel()->function, + compiledKernel_()->compiledKernel()->function, CU_FUNC_ATTRIBUTE_MAX_DYNAMIC_SHARED_SIZE_BYTES, dynamic_smem_size)); available_dynamic_smem_size_ = dynamic_smem_size; @@ -985,11 +847,13 @@ int64_t FusionExecutor::ensureAvailableDynamicSmemSize( return getAvailableDynamicSmemSize(); } +// TODO: Move to CompiledKernel void FusionExecutor::resetCompiledKernelProperties() { available_dynamic_smem_size_.reset(); static_smem_size_.reset(); } +// Used for ExprEval scheduler std::vector FusionExecutor::evaluateFusionOutputs( std::vector outputs, ExpressionEvaluator& expr_eval) { @@ -999,7 +863,7 @@ std::vector FusionExecutor::evaluateFusionOutputs( "Fusion executor is using expression evaluator,", " and expects that the outputs are not populated, which they were."); if (outputs.empty()) { - for (const auto& out_val : fusion()->outputs()) { + for (const auto& out_val : fusion_->outputs()) { auto out_tensor = expr_eval.evaluate(out_val->as()).as(); expr_eval.bind(out_val, out_tensor); @@ -1033,28 +897,28 @@ std::vector FusionExecutor::runFusion( if (isProfilerEnabled()) { NVF_CHECK( - groupId() >= 0, + compiledKernel_()->groupId() >= 0, "An invalid segment id is passed to FusionProfiler!:", - groupId()); - SegmentProfiler& sprof = FusionProfiler::segment(groupId()); + compiledKernel_()->groupId()); + SegmentProfiler& sprof = + FusionProfiler::segment(compiledKernel_()->groupId()); sprof.inputBytesAccessed(inputBytesProcessed(args)); - sprof.scheduler(toString(schedulerType())); + sprof.scheduler(toString(compiledKernel_()->schedulerType())); sprof.startKernel(args.getDeviceIndex()); } - NVF_ERROR(isCompiled()); - NVF_ERROR( - outputs.empty() || (outputs.size() == fusion()->outputs().size()), - __func__, - " provided number of outputs does not match fusion output"); - // Bind fusion inputs - auto expr_eval = executor_utils::bindInputs(args, fusion()); - if (isExpressionEvaluated(fusion())) { + + if (fusion_ != nullptr && isExpressionEvaluated(fusion_.get())) { + auto expr_eval = executor_utils::bindInputs(args, fusion_.get()); + NVF_ERROR( + outputs.empty() || (outputs.size() == fusion_->outputs().size()), + __func__, + " provided number of outputs does not match fusion output"); FUSER_PERF_SCOPE("FusionExecutor::runFusion::evaluate_with_ExprEval"); outputs = evaluateFusionOutputs(outputs, expr_eval); if (isProfilerEnabled()) { - auto& sprof = FusionProfiler::segment(groupId()); + auto& sprof = FusionProfiler::segment(compiledKernel_()->groupId()); sprof.stopKernel(); sprof.outputBytesAccessed(outputBytesProcessed(outputs)); } @@ -1063,11 +927,19 @@ std::vector FusionExecutor::runFusion( if (host_ir_container_ != nullptr) { FUSER_PERF_SCOPE("FusionExecutor::runFusion::host_ir_evaluate"); + auto expr_eval = + executor_utils::bindInputs(args, host_ir_container_->as()); + auto device = c10::Device(c10::DeviceType::CUDA, args.getDeviceIndex()); + NVF_ERROR( + outputs.empty() || + (outputs.size() == host_ir_container_->outputs().size()), + __func__, + " provided number of outputs does not match fusion output"); if (outputs.empty()) { std::vector output_info = getBufferInfos( expr_eval, PrimDataType::Int, host_ir_container_->outputs()); outputs = allocateOutputs( - host_ir_container_.get(), output_info, options().device, expr_eval); + host_ir_container_.get(), output_info, device, expr_eval); } for (Expr* e : host_ir_container_->topLevelExprs()) { NVF_ERROR(e->isA()); @@ -1089,31 +961,44 @@ std::vector FusionExecutor::runFusion( } return outputs; } - - NVF_ERROR(validKernelId(), "Invalid kernel id for FusionExecutor."); + auto expr_eval = + executor_utils::bindInputs(args, compiledKernel_()->fusion()); + // TODO: Make a run function in compiledKernel and move related logic + NVF_ERROR( + outputs.empty() || + (outputs.size() == compiledKernel_()->fusion()->outputs().size()), + __func__, + " provided number of outputs does not match fusion output"); + NVF_ERROR( + compiledKernel_()->validKernelId(), + "Invalid kernel id for FusionExecutor."); NVF_ERROR( !args.getCacheId().has_value() || outputs.empty(), "short cut input cache is not compatible with pre-allocated output"); - validateIndexType(kernel(), compile_params); + validateIndexType(compiledKernel_()->kernel(), compile_params); const auto num_inputs = args.size(); if (isDebugDumpEnabled(DebugDumpOption::FusionArgs)) { dumpFusionArgs( - fusionId(), args, launch_constraints, compile_params, outputs); + compiledKernel_()->fusionId(), + args, + launch_constraints, + compile_params, + outputs); } - c10::DeviceGuard dg(options().device); + c10::DeviceGuard dg(compiledKernel_()->options().device); auto stream = at::cuda::getCurrentCUDAStream(); at::cuda::jit::initializeCudaContext(); - NVF_ERROR(lowered()); + NVF_ERROR(compiledKernel_()->lowered()); // Placeholder for the case where parameter cache is not used ExecutorEntry temporary_executor_entry; - ExecutorEntry* executor_entry = - args.getCacheId().has_value() && !disablePaarameterCache() + ExecutorEntry* executor_entry = args.getCacheId().has_value() && + !compiledKernel_()->disablePaarameterCache() ? &executor_entry_lookup_[*args.getCacheId()] : &temporary_executor_entry; @@ -1125,12 +1010,15 @@ std::vector FusionExecutor::runFusion( launch_constraints, compile_params, outputs, - kernel()->indexType()); + compiledKernel_()->kernel()->indexType()); } - if (!(executor_entry->launch_params.nThreads() <= blockSizeHighWaterMark() && - compile_params.maxrregcount == maxrregcountHighWaterMark())) { - recompileKernel(executor_entry->launch_params, compile_params); + if (!(executor_entry->launch_params.nThreads() <= + compiledKernel_()->blockSizeHighWaterMark() && + compile_params.maxrregcount == + compiledKernel_()->maxrregcountHighWaterMark())) { + compiledKernel_()->recompileKernel( + executor_entry->launch_params, compile_params); } // TODO: Why does this need to be stored in the class? @@ -1142,20 +1030,24 @@ std::vector FusionExecutor::runFusion( // only allocate outputs when not given if (outputs.empty()) { outputs = allocateOutputs( - fusion(), executor_entry->outputs, options().device, expr_eval); + compiledKernel_()->fusion(), + executor_entry->outputs, + compiledKernel_()->options().device, + expr_eval); } args.push(outputs); for (const auto i : c10::irange(outputs.size())) { - auto output = kernel()->outputs()[i]; + auto output = compiledKernel_()->kernel()->outputs()[i]; if (std::any_of( - kernel()->inputs().begin(), - kernel()->inputs().end(), + compiledKernel_()->kernel()->inputs().begin(), + compiledKernel_()->kernel()->inputs().end(), [&](const auto& in) { return in == output; })) { // Skip trivially forwarded outputs because they are just placeholders continue; } - expr_eval.bind(output, *args[kernel()->inputs().size() + i]); + expr_eval.bind( + output, *args[compiledKernel_()->kernel()->inputs().size() + i]); } std::vector intermediates; @@ -1184,20 +1076,22 @@ std::vector FusionExecutor::runFusion( // to reset to zero upon completion of the kernel, or if we have // enabled the option (unsafe) intermediate_buffer = contigZeroedTensor( - unexpanded_sizes, buf_info.type, options().device); + unexpanded_sizes, + buf_info.type, + compiledKernel_()->options().device); } else { intermediate_buffer = at::zeros( unexpanded_sizes, at::TensorOptions() .dtype(buf_info.type) - .device(options().device)); + .device(compiledKernel_()->options().device)); } } else { intermediate_buffer = at::native::empty_cuda( unexpanded_sizes, buf_info.type, c10::nullopt, - options().device, + compiledKernel_()->options().device, c10::nullopt); if (shouldFillAllocationWithNan()) { fillTensorWithNan(intermediate_buffer); @@ -1210,8 +1104,14 @@ std::vector FusionExecutor::runFusion( args.push(intermediate_buffer); intermediates.push_back(intermediate_buffer); expr_eval.bind( - kernel()->summary().global_allocations.at(i)->buffer(), - *args[kernel()->inputs().size() + outputs.size() + i]); + compiledKernel_() + ->kernel() + ->summary() + .global_allocations.at(i) + ->buffer(), + *args + [compiledKernel_()->kernel()->inputs().size() + outputs.size() + + i]); if (buf_info.is_profile_buffer) { profile_buffer = intermediate_buffer; } @@ -1219,7 +1119,7 @@ std::vector FusionExecutor::runFusion( } if (executor_entry->args.empty()) { - computeArgs(*executor_entry, expr_eval, kernel()); + computeArgs(*executor_entry, expr_eval, compiledKernel_()->kernel()); } if (isDebugDumpEnabled(DebugDumpOption::LaunchParam)) { @@ -1228,7 +1128,7 @@ std::vector FusionExecutor::runFusion( if (isDebugDumpEnabled(DebugDumpOption::KernelArgs)) { dumpKernelArgs( - fusionId(), + compiledKernel_()->fusionId(), args, num_inputs, outputs, @@ -1237,28 +1137,30 @@ std::vector FusionExecutor::runFusion( } if (isDebugDumpEnabled(DebugDumpOption::IndexType)) { - debug() << "Index type: " << kernel()->indexType() << std::endl; + debug() << "Index type: " << compiledKernel_()->kernel()->indexType() + << std::endl; } executor_utils::CudaKernelTimer timer(stream); - if (execute_kernel_ && !kernel()->topLevelExprs().empty()) { + if (execute_kernel_ && + !compiledKernel_()->kernel()->topLevelExprs().empty()) { FUSER_PERF_SCOPE("FusionExecutor::runFusion::execute_kernel"); ensureAvailableDynamicSmemSize(executor_entry->launch_params.smem()); - recomputeArgs(*executor_entry, expr_eval, kernel()); + recomputeArgs(*executor_entry, expr_eval, compiledKernel_()->kernel()); if (isDebugDumpEnabled(DebugDumpOption::Occupancy) || isDebugDumpEnabled(DebugDumpOption::PerfDebugVerbose)) { int blocks_per_sm = -1; NVFUSER_CUDA_SAFE_CALL(cuOccupancyMaxActiveBlocksPerMultiprocessor( &blocks_per_sm, - compiledKernel()->function, + compiledKernel_()->compiledKernel()->function, launch_params_.nThreads(), launch_params_.smem())); - const int64_t device_id = - static_cast(options().device.index()); + const int64_t device_id = static_cast( + compiledKernel_()->options().device.index()); const auto prop = at::cuda::getDeviceProperties((c10::DeviceIndex)device_id); const int64_t warps_per_sm = @@ -1277,10 +1179,13 @@ std::vector FusionExecutor::runFusion( << ", occupancy=" << oss.str() << std::endl; } - if (!kernel()->summary().has_cooperative_grid_reduction) { + if (!compiledKernel_() + ->kernel() + ->summary() + .has_cooperative_grid_reduction) { FUSER_PERF_SCOPE("ExecutorRunFusion::cuLaunchKernel"); NVFUSER_CUDA_SAFE_CALL(cuLaunchKernel( - compiledKernel()->function, + compiledKernel_()->compiledKernel()->function, launch_params_.gdimx(), launch_params_.gdimy(), launch_params_.gdimz(), @@ -1294,7 +1199,7 @@ std::vector FusionExecutor::runFusion( } else { FUSER_PERF_SCOPE("ExecutorRunFusion::cuLaunchCooperativeKernel"); NVFUSER_CUDA_SAFE_CALL(cuLaunchCooperativeKernel( - compiledKernel()->function, + compiledKernel_()->compiledKernel()->function, launch_params_.gdimx(), launch_params_.gdimy(), launch_params_.gdimz(), @@ -1310,11 +1215,11 @@ std::vector FusionExecutor::runFusion( releaseZeroedMemory(); if (isOptionEnabled(EnableOption::KernelProfile)) { - debug() << kernel()->profile().toString(profile_buffer); + debug() << compiledKernel_()->kernel()->profile().toString(profile_buffer); } if (isProfilerEnabled()) { - auto& sprof = FusionProfiler::segment(groupId()); + auto& sprof = FusionProfiler::segment(compiledKernel_()->groupId()); sprof.stopKernel(); sprof.outputBytesAccessed(outputBytesProcessed(outputs)); } @@ -1346,100 +1251,6 @@ int64_t FusionExecutor::outputBytesProcessed( return num_bytes; } -void FusionExecutor::compileRtc( - const std::string& code, - const std::string& name, - bool structured, - PrimDataType index_type) { - if (use_external_compiler_) { - return compiled_kernel_2_->compileRtc(code, name, structured, index_type); - } - FUSER_PERF_SCOPE("FusionExecutor::compileRtc"); - NVF_ERROR( - index_type == PrimDataType::Int || index_type == PrimDataType::Int32 || - "Invalid index type: ", - index_type); - - createKernelId(); - - std::string scode; - if (!structured) { - scode = getStructuredCode(code, index_type); - } else { - scode = code; - } - compiledKernel() = - executor_utils::getCompiledKernel(std::nullopt, scode, name, kernelId()); -} - -float FusionExecutor::runRtc( - const LaunchParams& launch_params, - const std::vector& args, - PrimDataType index_type) { - if (use_external_compiler_) { - return compiled_kernel_2_->runRtc(launch_params, args, index_type); - } - FUSER_PERF_SCOPE("FusionExecutor::runRtc"); - - c10::DeviceGuard dg(options().device); - auto stream = at::cuda::getCurrentCUDAStream(); - - cudaEvent_t start_event = {}; - cudaEvent_t finish_event = {}; - - NVFUSER_CUDA_RT_SAFE_CALL(cudaEventCreate(&start_event)); - NVFUSER_CUDA_RT_SAFE_CALL(cudaEventCreate(&finish_event)); - - NVFUSER_CUDA_RT_SAFE_CALL(cudaEventRecord(start_event, stream)); - - std::vector> data; - std::vector pointers; - - for (const auto& input : args) { - auto dtype = - std::get(aten_to_data_type(input.scalar_type()).type); - DataType metadata_type = globalTensorMetaData(dtype, input.dim()); - - std::shared_ptr struct_ = std::make_shared(); - TensorMetaData* metadata = (TensorMetaData*)struct_.get(); - metadata->dtype = dtype; - metadata->data = input.data_ptr(); - metadata->logical_size = input.sizes(); - metadata->logical_stride = input.strides(); - metadata->alloc_size = input.sizes(); - metadata->alloc_stride = input.strides(); - - data.emplace_back(polymorphicValueToBytes( - PolymorphicValue(std::move(struct_)), metadata_type, index_type)); - pointers.emplace_back(data.back().data()); - } - - NVFUSER_CUDA_SAFE_CALL(cuLaunchKernel( - compiledKernel()->function, - launch_params.gdimx(), - launch_params.gdimy(), - launch_params.gdimz(), - launch_params.bdimx(), - launch_params.bdimy(), - launch_params.bdimz(), - launch_params.smem(), - stream, - pointers.data(), - nullptr)); - - NVFUSER_CUDA_RT_SAFE_CALL(cudaEventRecord(finish_event, stream)); - NVFUSER_CUDA_RT_SAFE_CALL(cudaEventSynchronize(start_event)); - NVFUSER_CUDA_RT_SAFE_CALL(cudaEventSynchronize(finish_event)); - - float kernel_time_ms = 0; - NVFUSER_CUDA_RT_SAFE_CALL( - cudaEventElapsedTime(&kernel_time_ms, start_event, finish_event)); - NVFUSER_CUDA_RT_SAFE_CALL(cudaEventDestroy(start_event)); - NVFUSER_CUDA_RT_SAFE_CALL(cudaEventDestroy(finish_event)); - - return kernel_time_ms; -} - flatbuffers::Offset FusionExecutor::serialize( flatbuffers::FlatBufferBuilder& builder) const { // See table definition for FusionExecutor in serde/fusion_cache.fbs @@ -1456,33 +1267,33 @@ flatbuffers::Offset FusionExecutor::serialize( // When compilation is skipped, avoid serializing cubin because it doesn't // exist. The remaining fields are also not necessary in this case. - if (!hasCompiledKernel()) { + if (!compiledKernel()->hasCompiledKernel()) { return serde::CreateFusionExecutorDirect(builder); } return serde::CreateFusionExecutorDirect( builder, device_smem_limit_, - blockSizeHighWaterMark(), - maxrregcountHighWaterMark(), + compiledKernel()->blockSizeHighWaterMark(), + compiledKernel()->maxrregcountHighWaterMark(), warp_size_, - toUnderlying(schedulerType()), - fusionId(), - concreteId(), - runtimeId(), - groupId(), - kernelCode().c_str(), + toUnderlying(compiledKernel()->schedulerType()), + compiledKernel()->fusionId(), + compiledKernel()->concreteId(), + compiledKernel()->runtimeId(), + compiledKernel()->groupId(), + compiledKernel()->kernelCode().c_str(), &executor_entry_lookup_keys_fb, &executor_entry_lookup_values_fb, - toUnderlying(kernel()->indexType()), - serialize(builder, compiledKernel().get())); + toUnderlying(compiledKernel()->kernel()->indexType()), + serialize(builder, compiledKernel()->compiledKernel().get())); } flatbuffers::Offset FusionExecutor::serialize( flatbuffers::FlatBufferBuilder& builder, const executor_utils::CompiledKernel* compiled_kernel) const { NVF_ERROR( - compiledKernel() != nullptr && + compiledKernel()->compiledKernel() != nullptr && (!compiled_kernel->cubin.empty() || !compiled_kernel->ptx.empty()), "Expected compiled cuda kernel before serializing FusionExecutor."); @@ -1537,10 +1348,13 @@ flatbuffers::Offset FusionExecutor::serialize( outputs_fb.reserve(data.outputs.size()); for (const auto& buffer : data.outputs) { auto tv_iter = std::find( - kernel()->outputs().cbegin(), kernel()->outputs().cend(), buffer.tv); - auto tv_position = (tv_iter == kernel()->outputs().cend()) + compiledKernel()->kernel()->outputs().cbegin(), + compiledKernel()->kernel()->outputs().cend(), + buffer.tv); + auto tv_position = (tv_iter == compiledKernel()->kernel()->outputs().cend()) ? -1 - : std::distance(kernel()->outputs().cbegin(), tv_iter); + : std::distance( + compiledKernel()->kernel()->outputs().cbegin(), tv_iter); outputs_fb.push_back( serialize(builder, buffer, tv_position, true /* is_fusion_output */)); } @@ -1556,14 +1370,16 @@ flatbuffers::Offset FusionExecutor::serialize( return a->buffer() == buffer_tv; }; auto tv_iter = std::find_if( - kernel()->summary().global_allocations.cbegin(), - kernel()->summary().global_allocations.cend(), + compiledKernel()->kernel()->summary().global_allocations.cbegin(), + compiledKernel()->kernel()->summary().global_allocations.cend(), match_tv_predicate); auto tv_position = - (tv_iter == kernel()->summary().global_allocations.cend()) + (tv_iter == + compiledKernel()->kernel()->summary().global_allocations.cend()) ? -1 : std::distance( - kernel()->summary().global_allocations.cbegin(), tv_iter); + compiledKernel()->kernel()->summary().global_allocations.cbegin(), + tv_iter); intermediates_fb.push_back( serialize(builder, buffer, tv_position, false /* is_fusion_output */)); } @@ -1612,57 +1428,12 @@ void FusionExecutor::deserialize( // skip compilation? if (isExpressionEvaluated(_fusion)) { fusion_ = std::make_unique(*_fusion); - NVF_ERROR(!hasCompiledKernel(), "Failed to deserialize FusionExecutor"); return; } - NVF_ERROR( - fusion_id == buffer->fusion_id(), - "Expected given fusion_id to match serde fusion_id."); - NVF_ERROR( - concrete_id == buffer->concrete_id(), - "Expected given concrete_id to match serde concrete_id."); - NVF_ERROR( - runtime_id == buffer->runtime_id(), - "Expected given runtime_id to match serde runtime_id."); - NVF_ERROR( - group_id == buffer->group_id(), - "Expected given group_id to match serde group_id."); - NVF_ERROR( - toUnderlying(heuristic) == buffer->heuristic(), - ": ", - toUnderlying(heuristic), - " vs ", - buffer->heuristic()); - - // Initialize CompileOptions - options().device = c10::Device(c10::DeviceType::CUDA, device_index); - c10::DeviceGuard dg(options().device); - // Initialize internal fields device_smem_limit_ = buffer->device_smem_limit(); - blockSizeHighWaterMark() = buffer->block_size_high_water_mark(); - maxrregcountHighWaterMark() = buffer->maxrregcount_high_water_mark(); warp_size_ = buffer->warp_size(); - kernelCode() = buffer->kernel_code()->str(); - - // KernelDB query checks kernel_code string and compile_params before - // copying cubin. - compile_params.index_type = serde::mapToNvfuserDtype(buffer->index_type()); - compile_params.maxrregcount = maxrregcountHighWaterMark(); - - // Get lowered fusion - lowered() = std::make_unique(_fusion, compile_params); - lowered()->run(); - - // Replace integers that are tensor sizes by named scalars like "T0.size[0]" - createKernelId( - heuristic, - buffer->fusion_id(), - buffer->concrete_id(), - buffer->runtime_id(), - buffer->group_id()); - setUsedTVs(); // GlobalBufferInfo requires lowered kernel before deserialization for (auto idx : c10::irange(buffer->executor_entry_lookup_keys()->size())) { @@ -1670,11 +1441,6 @@ void FusionExecutor::deserialize( buffer->executor_entry_lookup_keys()->Get(idx), deserialize(buffer->executor_entry_lookup_values()->Get(idx))); } - - compiledKernel() = executor_utils::getCompiledKernel( - buffer->compiled_kernel(), compile_params); - - NVF_ERROR(hasCompiledKernel(), "Failed to deserialize FusionExecutor"); } FusionExecutor::ExecutorEntry FusionExecutor::deserialize( @@ -1709,15 +1475,18 @@ GlobalBufferInfo FusionExecutor::deserialize( NVF_ERROR( buffer->tv() != -1, "Serialization failed to encode buffer tv position."); - NVF_ERROR(lowered() != nullptr, "Lowered kernel is not initialized."); + NVF_ERROR( + compiledKernel_()->lowered() != nullptr, + "Lowered kernel is not initialized."); GlobalBufferInfo info; if (buffer->is_fusion_output()) { - auto out_val = kernel()->outputs().at(buffer->tv()); + auto out_val = compiledKernel_()->kernel()->outputs().at(buffer->tv()); NVF_ERROR(out_val != nullptr); info.tv = dynamic_cast(out_val); } else { - auto out_val = kernel()->summary().global_allocations.at(buffer->tv()); + auto out_val = compiledKernel_()->kernel()->summary().global_allocations.at( + buffer->tv()); NVF_ERROR(out_val != nullptr); info.tv = dynamic_cast(out_val->buffer()); } diff --git a/csrc/runtime/executor.h b/csrc/runtime/executor.h index eedd3d651fb..7e7ad3f59d4 100644 --- a/csrc/runtime/executor.h +++ b/csrc/runtime/executor.h @@ -133,8 +133,8 @@ class FusionExecutor : public NonCopyable { // Function to query whether compilation was attempted for a `FusionExecutor` bool isCompiled() const { - if (use_external_compiler_) { - return compiled_kernel_2_->isCompiled(); + if (compiledKernel()) { + return true; } int num_compiled_artifacts = (fusion_ != nullptr) + (lowered_ != nullptr) + (host_ir_container_ != nullptr); @@ -142,21 +142,6 @@ class FusionExecutor : public NonCopyable { return num_compiled_artifacts == 1; }; - // function to query whether a `FusionExecutor` has a compiled kernel to - // execute - bool hasCompiledKernel() const { - if (use_external_compiler_) { - return compiled_kernel_2_->hasCompiledKernel(); - } - if (compiled_kernel_ != nullptr) { - NVF_ERROR(compiled_kernel_->function != nullptr); - NVF_ERROR( - fusion() == nullptr, - "fusion() should only be initialized when using expression evaluator."); - } - return validKernelId() && lowered() && compiled_kernel_ != nullptr; - }; - void evictCache(size_t cache_id) { executor_entry_lookup_.erase(cache_id); } @@ -189,41 +174,6 @@ class FusionExecutor : public NonCopyable { using ExecutorCompileTimeInfoCache = executor_utils::caching::ExecutorCompileTimeInfoCache; - kir::Kernel* kernel() const { - if (use_external_compiler_) { - return compiled_kernel_2_->kernel(); - } - NVF_ERROR(lowered()); - return lowered()->kernel(); - } - - Fusion* fusion() const { - NVF_ERROR(isCompiled()); - if (fusion_ != nullptr) { - if (use_external_compiler_) { - return compiled_kernel_2_->fusion(); - } - return fusion_.get(); - } - if (lowered() != nullptr) { - if (use_external_compiler_) { - return compiled_kernel_2_->lowered()->kernel()->as(); - } - return lowered()->kernel()->as(); - } - if (host_ir_container_ != nullptr) { - return host_ir_container_->as(); - } - NVF_THROW("unreachable because of the isCompiled check"); - } - - const ThreadPredicateMap& threadPredMap() const { - if (use_external_compiler_) { - return compiled_kernel_2_->threadPredMap(); - } - return lowered()->threadPredMap(); - } - //! Internal knob used for debugging/profiling only void setExecuteKernelFlag(bool execute_kernel) { execute_kernel_ = execute_kernel; @@ -241,13 +191,6 @@ class FusionExecutor : public NonCopyable { kernel_occupancy_ = occupancy; } - //! get register spills (load + store) of the compiled kernel - int getKernelRegisterSpills() const { - if (use_external_compiler_) { - return compiled_kernel_2_->getKernelRegisterSpills(); - } - return compiled_kernel_->register_spills; - } //! Returns the input bytes accessed for a kernel //! \note It is important to sample the args struct prior to adding the // 1 output to the args struct @@ -260,56 +203,6 @@ class FusionExecutor : public NonCopyable { return launch_params_; } - //! Returns the string of the compiled kernel - NVF_API std::string kernelString() const { - if (use_external_compiler_) { - return compiled_kernel_2_->kernelString(); - } - NVF_ERROR(!kernelCode().empty(), "Kernel code not generated"); - return kernelCode(); - } - - // Add preamble and wrap in namespace - NVF_API std::string getStructuredCode( - const std::string& kernel, - PrimDataType index_type) const; - - NVF_API std::string getStructuredCode() const; - - //! Returns a const reference to the latest compiled kernel. - const std::unique_ptr& compiledKernel() - const { - if (use_external_compiler_) { - return compiled_kernel_2_->compiledKernel(); - } - return compiled_kernel_; - } - std::unique_ptr& compiledKernel() { - if (use_external_compiler_) { - return compiled_kernel_2_->compiledKernel(); - } - return compiled_kernel_; - } - - //! Returns the disassembled latest compiled binary - NVF_API std::string disassembledBinary( - const std::string& nvdisasm_args = "") const { - if (use_external_compiler_) { - return compiled_kernel_2_->disassembledBinary(nvdisasm_args); - } - return executor_utils::disassembleBinary( - compiled_kernel_->cubin, nvdisasm_args); - } - - //! Returns the disassembled latest compiled binary - NVF_API std::string disassembledKernelSASS() const { - if (use_external_compiler_) { - return compiled_kernel_2_->disassembledKernelSASS(); - } - return executor_utils::disassembleBinary( - compiled_kernel_->cubin, "-fun 1 -c"); - } - static void setGlobalFusionCount(int64_t new_fusion_count) { CompiledKernel::setGlobalFusionCount(new_fusion_count); } @@ -318,105 +211,6 @@ class FusionExecutor : public NonCopyable { return CompiledKernel::getGlobalFusionCount(); } - const int64_t& groupId() const { - if (use_external_compiler_) { - return compiled_kernel_2_->groupId(); - } - return group_id_; - } - int64_t& groupId() { - if (use_external_compiler_) { - return compiled_kernel_2_->groupId(); - } - return group_id_; - } - - void setGroupId(int64_t gid) { - if (use_external_compiler_) { - return compiled_kernel_2_->setGroupId(gid); - } - group_id_ = gid; - } - - bool validKernelId() const { - if (use_external_compiler_) { - return compiled_kernel_2_->validKernelId(); - } - return !kernelId().empty(); - } - - void createKernelId( - SchedulerType scheduler_type = SchedulerType::None, - int64_t fusion_id = 0, - int64_t concrete_id = 0, - int64_t runtime_id = 0, - int64_t group_id = 0) { - if (use_external_compiler_) { - return compiled_kernel_2_->createKernelId( - scheduler_type, fusion_id, concrete_id, runtime_id, group_id); - } - NVF_ERROR(fusion_id > -1, "Invalid fusion_id."); - NVF_ERROR(concrete_id > -1, "Invalid concrete_id."); - NVF_ERROR(runtime_id > -1, "Invalid runtime_id."); - NVF_ERROR(group_id > -1, "Invalid group_id"); - - schedulerType() = scheduler_type; - fusionId() = fusion_id; - concreteId() = concrete_id; - runtimeId() = runtime_id; - groupId() = group_id; - ++globalFusionCount(); - - std::stringstream ss; - if (isOptionEnabled(EnableOption::StaticFusionCount)) { - ss << globalFusionCount().load(); - } else { - ss << toString(schedulerType()); - ss << "_f" << fusionId(); - ss << "_c" << concreteId(); - ss << "_r" << runtimeId(); - ss << "_g" << groupId(); - } - kernelId() = ss.str(); - } - - std::string kernelName() const { - if (use_external_compiler_) { - return compiled_kernel_2_->kernelName(); - } - NVF_ERROR(!kernelId().empty(), "Invalid kernel name for fusion executor."); - std::stringstream ss; - ss << "nvfuser_" << kernelId(); - return ss.str(); - } - - //! Internal tests only. Compiles CUDA code with NVRTC directly from - //! string. This util provides a path to test runtime code, i.e. the - //! resource strings. - // TODO: Consider split out compileRtc and runRtc to a different - //! class. Not much code is shared with the normal path. - NVF_API void compileRtc( - const std::string& code, - const std::string& name, - bool structured, - PrimDataType index_type); - - //! Internal tests only. Runs the compiled CUDA kernel from - //! compileRtc. Return the elapsed milliseconds. - NVF_API float runRtc( - const LaunchParams& launch_params, - const std::vector& args, - PrimDataType indextype); - - //! Internal knob used for debugging/profiling only - void disableLaunchParamCache() { - if (use_external_compiler_) { - compiled_kernel_2_->disableLaunchParamCache(); - } else { - disablePaarameterCache() = true; - } - } - //! Serialize Fusion Executor using flatbuffers flatbuffers::Offset serialize( flatbuffers::FlatBufferBuilder& builder) const; @@ -433,6 +227,14 @@ class FusionExecutor : public NonCopyable { int64_t runtime_id, int64_t group_id); + const std::unique_ptr& compiledKernel() const { + return compiled_kernel_2_; + } + + const std::unique_ptr& hostIrContainer() const { + return host_ir_container_; + } + private: LaunchParams computeLaunchParams( const LaunchParams& launch_constraints, @@ -448,15 +250,6 @@ class FusionExecutor : public NonCopyable { ExpressionEvaluator& expr_eval, DataType index_dtype); - void setUsedTVs(); - - const std::vector& getUsedTVs() const { - if (use_external_compiler_) { - return compiled_kernel_2_->getUsedTVs(); - } - return used_tvs_; - }; - ExecutorCompileTimeInfoCache* compileTimeDataCache() { return &compile_time_info_cache_; } @@ -472,11 +265,6 @@ class FusionExecutor : public NonCopyable { std::unique_ptr& evaluatorPrecomputedValues(); - // Recompile the kernel if the number of threads in the block has increased - // or maxrregcount has changed - void recompileKernel( - const LaunchParams& new_launch_params, - const CompileParams& new_compile_params); // Creates the initial set of arguments to a kernel, based on the arguments // to we have now. void computeArgs(ExecutorEntry&, ExpressionEvaluator&, const kir::Kernel*) @@ -534,170 +322,18 @@ class FusionExecutor : public NonCopyable { //! Clear the cached properties of the compiled kernel void resetCompiledKernelProperties(); - // Temporary accessors for refactor: - CompileOptions& options() { - if (use_external_compiler_) { - return compiled_kernel_2_->options(); - } - return options_; - } - int64_t& fusionId() { - if (use_external_compiler_) { - return compiled_kernel_2_->fusionId(); - } - return fusion_id_; - } - const int64_t& fusionId() const { - if (use_external_compiler_) { - return compiled_kernel_2_->fusionId(); - } - return fusion_id_; - } - int64_t& concreteId() { - if (use_external_compiler_) { - return compiled_kernel_2_->concreteId(); - } - return concrete_id_; - } - int64_t& runtimeId() { - if (use_external_compiler_) { - return compiled_kernel_2_->runtimeId(); - } - return runtime_id_; - } - const int64_t& concreteId() const { - if (use_external_compiler_) { - return compiled_kernel_2_->concreteId(); - } - return concrete_id_; - } - const int64_t& runtimeId() const { - if (use_external_compiler_) { - return compiled_kernel_2_->runtimeId(); - } - return runtime_id_; - } - static std::atomic& globalFusionCount() { - return CompiledKernel::globalFusionCount(); - } - SchedulerType& schedulerType() { - if (use_external_compiler_) { - return compiled_kernel_2_->schedulerType(); - } - return scheduler_type_; - } - const SchedulerType& schedulerType() const { - if (use_external_compiler_) { - return compiled_kernel_2_->schedulerType(); - } - return scheduler_type_; - } - std::string& kernelId() { - if (use_external_compiler_) { - return compiled_kernel_2_->kernelId(); - } - return kernel_id_; - } - const std::string& kernelId() const { - if (use_external_compiler_) { - return compiled_kernel_2_->kernelId(); - } - return kernel_id_; + std::unique_ptr& compiledKernel_() { + return compiled_kernel_2_; } - std::unique_ptr& lowered() { - if (use_external_compiler_) { - return compiled_kernel_2_->lowered(); - } - return lowered_; - } - const std::unique_ptr& lowered() const { - if (use_external_compiler_) { - return compiled_kernel_2_->lowered(); - } - return lowered_; - } - Fusion* fusion() { - if (use_external_compiler_) { - return compiled_kernel_2_->fusion(); - } - NVF_ERROR(isCompiled()); - if (fusion_ != nullptr) { - if (use_external_compiler_) { - return compiled_kernel_2_->fusion(); - } - return fusion_.get(); - } - if (lowered() != nullptr) { - if (use_external_compiler_) { - return compiled_kernel_2_->lowered()->kernel()->as(); - } - return lowered()->kernel()->as(); - } - if (host_ir_container_ != nullptr) { - return host_ir_container_->as(); - } - NVF_THROW("unreachable because of the isCompiled check"); - } - int64_t& blockSizeHighWaterMark() { - if (use_external_compiler_) { - return compiled_kernel_2_->blockSizeHighWaterMark(); - } - return block_size_high_water_mark_; - } - int64_t& maxrregcountHighWaterMark() { - if (use_external_compiler_) { - return compiled_kernel_2_->maxrregcountHighWaterMark(); - } - return maxrregcount_high_water_mark_; - } - const int64_t& blockSizeHighWaterMark() const { - if (use_external_compiler_) { - return compiled_kernel_2_->blockSizeHighWaterMark(); - } - return block_size_high_water_mark_; - } - const int64_t& maxrregcountHighWaterMark() const { - if (use_external_compiler_) { - return compiled_kernel_2_->maxrregcountHighWaterMark(); - } - return maxrregcount_high_water_mark_; - } - bool& disablePaarameterCache() { - if (use_external_compiler_) { - return compiled_kernel_2_->disablePaarameterCache(); - } - return disable_parameter_cache_; - } - std::string& kernelCode() { - if (use_external_compiler_) { - return compiled_kernel_2_->kernelCode(); - } - return kernel_code_; - } - const std::string& kernelCode() const { - if (use_external_compiler_) { - return compiled_kernel_2_->kernelCode(); - } - return kernel_code_; - } - std::vector>& loweringHooks() { - if (use_external_compiler_) { - return compiled_kernel_2_->loweringHooks(); - } - return lowering_hooks_; - } - std::vector>& postLoweringHooks() { - if (use_external_compiler_) { - return compiled_kernel_2_->postLoweringHooks(); + + void disableLaunchParamCache() { + if (compiledKernel()) { + compiledKernel()->disableLaunchParamCache(); } - return post_lowering_hooks_; } private: std::unique_ptr compiled_kernel_2_; - bool use_external_compiler_ = false; - - CompileOptions options_; //! Absolute limit of all available shared mem space from cudaDeviceProp int64_t device_smem_limit_ = 0; @@ -779,12 +415,6 @@ class FusionExecutor : public NonCopyable { // Profiling support: the last launch param used LaunchParams launch_params_; - // Profiling support: disable caching of launch params and output allocation - // output allocation is also disable when output sizes are dependent on - // runtime scalar inputs, such as for the case of tensor factory. see - // https://github.com/csarofeen/pytorch/issues/2002 - bool disable_parameter_cache_ = false; - // Profiling support: kept copy of the cuda kernel std::string kernel_code_; diff --git a/csrc/runtime/fusion_executor_cache.cpp b/csrc/runtime/fusion_executor_cache.cpp index a3b8de148e2..aedcf8cd593 100644 --- a/csrc/runtime/fusion_executor_cache.cpp +++ b/csrc/runtime/fusion_executor_cache.cpp @@ -183,30 +183,30 @@ std::string FusionExecutorCache::getCode( NVF_CHECK(kernel_runtime->isCompiled(), "Fusion is not compiled!"); bool first_kernel = true; - for (const auto& exec : kernel_runtime->executors()) { + for (const auto& fe : kernel_runtime->executors()) { if (first_kernel) { first_kernel = false; } else { kernel_code += "\n"; } - kernel_code += exec.kernelString(); + kernel_code += fe.compiledKernel()->kernelString(); } if (intrinsic_code) { - const auto& execs = kernel_runtime->executors(); - const FusionExecutor& fe = execs[0]; - auto index_type = fe.kernel()->indexType(); + const auto& fes = kernel_runtime->executors(); + auto index_type = fes[0].compiledKernel()->kernel()->indexType(); // Make sure all the segment index types match. All segments currently // use the same index type but this code change in the future. - for (const auto& exec : execs) { + for (const auto& fe : fes) { NVF_CHECK( - index_type == exec.kernel()->indexType(), + index_type == fe.compiledKernel()->kernel()->indexType(), "Index Type mismatch between Segment Executors: ", index_type, " ", - exec.kernel()->indexType()); + fe.compiledKernel()->kernel()->indexType()); } - std::string full_code = fe.getStructuredCode(kernel_code, index_type); + std::string full_code = + fes[0].compiledKernel()->getStructuredCode(kernel_code, index_type); return full_code; } else { return kernel_code; @@ -239,7 +239,7 @@ std::string FusionExecutorCache::getScheduledIr( ss << fs << "\n"; } for (auto& exec : kernel_runtime->executors()) { - auto sched_ir = exec.kernel()->as(); + auto sched_ir = exec.compiledKernel()->kernel()->as(); sched_ir->print(ss, tensor_transforms); } return ss.str(); diff --git a/csrc/runtime/fusion_kernel_runtime.cpp b/csrc/runtime/fusion_kernel_runtime.cpp index 7a2768e0cbf..c424f049158 100644 --- a/csrc/runtime/fusion_kernel_runtime.cpp +++ b/csrc/runtime/fusion_kernel_runtime.cpp @@ -369,7 +369,7 @@ void FusionKernelRuntime::compileFusionParallel(KernelArgumentHolder args) { void FusionKernelRuntime::disableLaunchParamCache() { for (auto& executor : executors_) { - executor.disableLaunchParamCache(); + executor.compiledKernel()->disableLaunchParamCache(); } } @@ -592,9 +592,10 @@ std::vector FusionKernelRuntime::runKernelWithInput( // TODO: This is a work around for the fallback execution path where a kernel // is not compiled. Perhaps the gorup/segment Id needs to be specified to the // executor at its constructor. Currently, initialization is ad hoc. - if (executor.groupId() < 0) { - executor.setGroupId(group_id); - } + // TODO: Re enable + // if (executor.compiledKernel()->groupId() < 0) { + // executor.compiledKernel()->setGroupId(group_id); + // } auto outputs = executor.runFusion(args, launch_params, compile_params); return outputs; diff --git a/tests/cpp/test_alias.cpp b/tests/cpp/test_alias.cpp index 893fac13050..a90d9a049a4 100644 --- a/tests/cpp/test_alias.cpp +++ b/tests/cpp/test_alias.cpp @@ -346,10 +346,11 @@ const FusionExecutor& onlyExecutorInMostRecentRuntime( bool storesToOutput(const FusionExecutor& executor, const int64_t out_index) { // Get the variable name from the `kir::Kernel` not the input fusion, because // they are not always the same. - std::string var_name = - ir_utils::varName(executor.kernel()->outputs()[out_index]); + std::string var_name = ir_utils::varName( + executor.compiledKernel()->kernel()->outputs()[out_index]); std::regex store_to_output(R"(\b)" + var_name + R"(\[)"); - return std::regex_search(executor.kernelString(), store_to_output); + return std::regex_search( + executor.compiledKernel()->kernelString(), store_to_output); } } // namespace @@ -403,7 +404,7 @@ TEST_F(AliasTest, NotAllOutputsAlias_Pointwise) { EXPECT_EQ(num_stores, 1) << "The generated CUDA kernel is expected to store data to one output:" << std::endl - << fe.kernelString(); + << fe.compiledKernel()->kernelString(); } } } @@ -478,7 +479,7 @@ TEST_F(AliasTest, Issue1452) { EXPECT_EQ(num_stores, 1) << "The generated CUDA kernel is expected to store data to one output:" << std::endl - << fe.kernelString(); + << fe.compiledKernel()->kernelString(); } } } @@ -507,7 +508,7 @@ TEST_F(AliasTest, AliasOutputBeforeNonAliasOutput) { EXPECT_FALSE(storesToOutput(fe, /*out_index=*/0)) << "The generated CUDA kernel shouldn't store data to output 0:" << std::endl - << fe.kernelString(); + << fe.compiledKernel()->kernelString(); } TEST_F(AliasTest, Set_NoAliasForIncompatibleLayout) { diff --git a/tests/cpp/test_circular_buffering.cpp b/tests/cpp/test_circular_buffering.cpp index 31995223c95..88f4ff0aa7a 100644 --- a/tests/cpp/test_circular_buffering.cpp +++ b/tests/cpp/test_circular_buffering.cpp @@ -582,7 +582,8 @@ TEST_P(CircularBufferingTest, SmemBlockGemmCache) { // insertion to ensure ordering of circular buffered tensor access. // The check below makes sure that the sync is inserted so that the // test isn't running on a race condition. - NVF_CHECK(fe.kernel()->summary().war_hazard_syncs_count > 0); + NVF_CHECK( + fe.compiledKernel()->kernel()->summary().war_hazard_syncs_count > 0); } // Vectorized reset test for circular buffered registers diff --git a/tests/cpp/test_external_src.cpp b/tests/cpp/test_external_src.cpp index f0623ade609..ba5fe6c6604 100644 --- a/tests/cpp/test_external_src.cpp +++ b/tests/cpp/test_external_src.cpp @@ -44,7 +44,8 @@ TEST_F(ExternalSrcExample, Reduction_CUDA) { buffer << cuda_src.rdbuf(); std::string cuda_src_str = buffer.str(); - fe.compileRtc(cuda_src_str, "kernel1", true, PrimDataType::Int32); + fe.compiledKernel()->compileRtc( + cuda_src_str, "kernel1", true, PrimDataType::Int32); // The following is a sample launch pattern of the compiled // kernel. It must be adapted for each particular source file. @@ -81,8 +82,8 @@ TEST_F(ExternalSrcExample, Reduction_CUDA) { auto t17 = at::zeros({8}, options_long); clearL2Cache(); std::cout << "Launching the kernel" << std::endl; - float elapsed_time_ms = - fe.runRtc(lp, {t0, t7, t14, t15, t16, t17}, PrimDataType::Int32); + float elapsed_time_ms = fe.compiledKernel()->runRtc( + lp, {t0, t7, t14, t15, t16, t17}, PrimDataType::Int32); std::cout << "kernel run in " << elapsed_time_ms << " ms, achieved " << (read_write_bytes / elapsed_time_ms / 1000.0 / 1000.0) << " GB/s" << std::endl; @@ -115,7 +116,8 @@ TEST_F(ExternalSrcExample, Matmul_CUDA) { buffer << cuda_src.rdbuf(); std::string cuda_src_str = buffer.str(); - fe.compileRtc(cuda_src_str, "kernel1", true, PrimDataType::Int32); + fe.compiledKernel()->compileRtc( + cuda_src_str, "kernel1", true, PrimDataType::Int32); int M = 2048, N = 3456, K = 2048; MmaLayout layout = MmaLayout::TN; @@ -129,7 +131,7 @@ TEST_F(ExternalSrcExample, Matmul_CUDA) { auto output = at::zeros_like(at_output); clearL2Cache(); std::cout << "Launching the kernel" << std::endl; - float elapsed_time_ms = fe.runRtc( + float elapsed_time_ms = fe.compiledKernel()->runRtc( lp, {inputs.first, inputs.second, output}, PrimDataType::Int32); std::cout << "kernel run in " << elapsed_time_ms << " ms." << std::endl; diff --git a/tests/cpp/test_gpu1.cpp b/tests/cpp/test_gpu1.cpp index 05d8e2d16ba..f9cdf1d98dd 100644 --- a/tests/cpp/test_gpu1.cpp +++ b/tests/cpp/test_gpu1.cpp @@ -5814,7 +5814,8 @@ TEST_F(NVFuserTest, FusionSmem_CUDA) { testValidate(&fusion, cg_outputs, aten_inputs, __LINE__, __FILE__); - NVF_CHECK(fe.kernel()->summary().war_hazard_syncs_count == 0); + NVF_CHECK( + fe.compiledKernel()->kernel()->summary().war_hazard_syncs_count == 0); } TEST_F(NVFuserTest, FusionSmemReduce_CUDA) { @@ -5862,7 +5863,8 @@ TEST_F(NVFuserTest, FusionSmemReduce_CUDA) { testValidate( &fusion, cg_outputs, {aten_input}, {aten_output}, __LINE__, __FILE__); - NVF_CHECK(fe.kernel()->summary().war_hazard_syncs_count == 0); + NVF_CHECK( + fe.compiledKernel()->kernel()->summary().war_hazard_syncs_count == 0); } TEST_F(NVFuserTest, FusionSmemBlockGemm_CUDA) { @@ -5933,7 +5935,8 @@ TEST_F(NVFuserTest, FusionSmemBlockGemm_CUDA) { testValidate( &fusion, cg_outputs, aten_inputs, {aten_output}, __LINE__, __FILE__); - NVF_CHECK(fe.kernel()->summary().war_hazard_syncs_count == 0); + NVF_CHECK( + fe.compiledKernel()->kernel()->summary().war_hazard_syncs_count == 0); } TEST_F(NVFuserTest, FusionSmemBlockGemmCache_CUDA) { @@ -6022,7 +6025,8 @@ TEST_F(NVFuserTest, FusionSmemBlockGemmCache_CUDA) { testValidate( &fusion, cg_outputs, aten_inputs, {aten_output}, __LINE__, __FILE__); - NVF_CHECK(fe.kernel()->summary().war_hazard_syncs_count == 0); + NVF_CHECK( + fe.compiledKernel()->kernel()->summary().war_hazard_syncs_count == 0); } TEST_F(NVFuserTest, FusionSmemDynamicPersistentSoftmax2D_CUDA) { @@ -6383,9 +6387,9 @@ TEST_F(NVFuserTest, FusionMagicSchedulerLayerNormalization_CUDA) { // tv11 and tv17 should not be predicated. See issue #496 ASSERT_FALSE(PredicatedChecker::isPredicated( - 11, cg_results.fusion_executor->kernel())); + 11, cg_results.fusion_executor->compiledKernel()->kernel())); ASSERT_FALSE(PredicatedChecker::isPredicated( - 17, cg_results.fusion_executor->kernel())); + 17, cg_results.fusion_executor->compiledKernel()->kernel())); } TEST_F(NVFuserTest, FusionMagicSchedulerRMSNormalization_CUDA) { @@ -7214,7 +7218,8 @@ TEST_F(NVFuserTest, FusionSmemDynamicReductionSymbolic_CUDA) { __FILE__, "", lparams); - NVF_CHECK(fe.kernel()->summary().war_hazard_syncs_count == 0); + NVF_CHECK( + fe.compiledKernel()->kernel()->summary().war_hazard_syncs_count == 0); } TEST_F(NVFuserTest, FusionSmemDynamicReductionSymbolicArg_CUDA) { @@ -7278,7 +7283,8 @@ TEST_F(NVFuserTest, FusionSmemDynamicReductionSymbolicArg_CUDA) { "", lparams); - NVF_CHECK(fe.kernel()->summary().war_hazard_syncs_count == 0); + NVF_CHECK( + fe.compiledKernel()->kernel()->summary().war_hazard_syncs_count == 0); } TEST_F(NVFuserTest, FusionSmemDynamicPwiseMulSymbolicArgWAR_CUDA) { @@ -7335,7 +7341,8 @@ TEST_F(NVFuserTest, FusionSmemDynamicPwiseMulSymbolicArgWAR_CUDA) { testValidate( &fusion, cg_outputs, aten_inputs, __LINE__, __FILE__, "", lparams); - NVF_CHECK(fe.kernel()->summary().war_hazard_syncs_count == 1); + NVF_CHECK( + fe.compiledKernel()->kernel()->summary().war_hazard_syncs_count == 1); } TEST_F(NVFuserTest, FusionSmemDynamicTiledGemm_CUDA) { @@ -7461,7 +7468,8 @@ TEST_F(NVFuserTest, FusionSmemDynamicTiledGemm_CUDA) { testValidate( &fusion, cg_outputs, aten_inputs, {aten_output}, __LINE__, __FILE__); - NVF_CHECK(fe.kernel()->summary().war_hazard_syncs_count == 1); + NVF_CHECK( + fe.compiledKernel()->kernel()->summary().war_hazard_syncs_count == 1); } } // namespace nvfuser diff --git a/tests/cpp/test_gpu2.cpp b/tests/cpp/test_gpu2.cpp index 246a9665a7c..b04b36eae69 100644 --- a/tests/cpp/test_gpu2.cpp +++ b/tests/cpp/test_gpu2.cpp @@ -1974,7 +1974,7 @@ __global__ void kernel1(Tensor T0, Tensor T1) { } } )"; - fe.compileRtc(kernel, "kernel1", false, PrimDataType::Int); + fe.compiledKernel()->compileRtc(kernel, "kernel1", false, PrimDataType::Int); LaunchParams lp( 256, // gdimx 1, // gdimy @@ -1989,7 +1989,7 @@ __global__ void kernel1(Tensor T0, Tensor T1) { const std::vector tensor_dims = {8}; auto in0 = at::randn(tensor_dims, options); auto out0 = at::empty_like(in0); - fe.runRtc(lp, {in0, out0}, PrimDataType::Int); + fe.compiledKernel()->runRtc(lp, {in0, out0}, PrimDataType::Int); auto out_ref = in0 * 2; NVF_CHECK(out_ref.allclose(out0)); @@ -2030,7 +2030,7 @@ __global__ void kernel1( } } )"; - fe.compileRtc(kernel, "kernel1", false, PrimDataType::Int); + fe.compiledKernel()->compileRtc(kernel, "kernel1", false, PrimDataType::Int); LaunchParams lp( 1, // gdimx 1, // gdimy @@ -2046,7 +2046,7 @@ __global__ void kernel1( auto in0 = at::randn(tensor_dims, options); auto out_var = at::empty({x}, options); auto out_avg = at::empty({x}, options); - fe.runRtc(lp, {in0, out_var, out_avg}, PrimDataType::Int); + fe.compiledKernel()->runRtc(lp, {in0, out_var, out_avg}, PrimDataType::Int); NVF_CHECK(in0.var({1, 2}, false).allclose(out_var)); NVF_CHECK(in0.mean({1, 2}).allclose(out_avg, /*rtol*/ 1e-5, /*atol*/ 1e-6)); @@ -2102,7 +2102,7 @@ __global__ void kernel1( } } )"; - fe.compileRtc(kernel, "kernel1", false, PrimDataType::Int); + fe.compiledKernel()->compileRtc(kernel, "kernel1", false, PrimDataType::Int); LaunchParams lp( 1, // gdimx 1, // gdimy @@ -2129,7 +2129,7 @@ __global__ void kernel1( // run kernel auto out_var = at::zeros({x}, options); auto out_avg = at::zeros({x}, options); - fe.runRtc( + fe.compiledKernel()->runRtc( lp, {in0, out_avg, out_var, init_avg, init_var, init_N}, PrimDataType::Int); @@ -2183,7 +2183,7 @@ __global__ void kernel1( } } )"; - fe.compileRtc(kernel, "kernel1", false, PrimDataType::Int); + fe.compiledKernel()->compileRtc(kernel, "kernel1", false, PrimDataType::Int); LaunchParams lp( 1, // gdimx 1, // gdimy @@ -2199,7 +2199,7 @@ __global__ void kernel1( auto in0 = at::randn(tensor_dims, options); auto out_var = at::empty({x}, options); auto out_avg = at::empty({x}, options); - fe.runRtc(lp, {in0, out_avg, out_var}, PrimDataType::Int); + fe.compiledKernel()->runRtc(lp, {in0, out_avg, out_var}, PrimDataType::Int); NVF_CHECK(in0.var({1, 2}, false).allclose(out_var)); NVF_CHECK(in0.mean({1, 2}).allclose(out_avg, /*rtol*/ 1e-5, /*atol*/ 1e-6)); @@ -2258,7 +2258,7 @@ __global__ void kernel1( } } )"; - fe.compileRtc(kernel, "kernel1", false, PrimDataType::Int); + fe.compiledKernel()->compileRtc(kernel, "kernel1", false, PrimDataType::Int); LaunchParams lp( x, // gdimx y, // gdimy @@ -2282,7 +2282,7 @@ __global__ void kernel1( auto work_buf_var = at::empty({x * y * z}, options); auto work_buf_N = at::empty({x * y * z}, options_int); auto sync_flag = at::zeros({1}, options_int); - fe.runRtc( + fe.compiledKernel()->runRtc( lp, {in0, out_avg, @@ -2333,7 +2333,7 @@ TEST_F(NVFuserTest, FusionWelfordOp_CUDA) { outputs[1] /= N; testValidate( - fe.kernel(), + fe.compiledKernel()->kernel(), outputs, {t0}, {t0.mean({1}), t0.var({1}, false), at::ones({M}, options_int) * N}, @@ -2378,7 +2378,7 @@ TEST_F(NVFuserTest, FusionBlockWelfordOp_CUDA) { outputs[1] /= N; testValidate( - fe.kernel(), + fe.compiledKernel()->kernel(), outputs, {t0}, {t0.mean({1}), t0.var({1}, false), at::ones({M}, options_int) * N}, @@ -2423,7 +2423,7 @@ TEST_F(NVFuserTest, FusionGridWelfordOp_CUDA) { outputs[1] /= N; testValidate( - fe.kernel(), + fe.compiledKernel()->kernel(), outputs, {t0}, {t0.mean({1}), t0.var({1}, false), at::ones({M}, options_int) * N}, @@ -2467,7 +2467,7 @@ TEST_F(NVFuserTest, FusionRfactorWelfordOp_CUDA) { outputs[1] /= N; testValidate( - fe.kernel(), + fe.compiledKernel()->kernel(), outputs, {t0}, {t0.mean({1}), t0.var({1}, false), at::ones({M}, options_int) * N}, @@ -2613,7 +2613,7 @@ TEST_P(WelfordReduction, Test) { at_n = at_n.sum({axis}); testValidate( - fe.kernel(), + fe.compiledKernel()->kernel(), outputs, {aten_input}, {at_avg, at_var, at_n}, diff --git a/tests/cpp/test_gpu3.cpp b/tests/cpp/test_gpu3.cpp index f59669b1ade..ca80f003902 100644 --- a/tests/cpp/test_gpu3.cpp +++ b/tests/cpp/test_gpu3.cpp @@ -1466,8 +1466,10 @@ TEST_F(NVFuserTest, FusionVectorizeContigIndexPointwiseSchedule_CUDA) { // vectorization can be done against 2*14=28 rather than 14, so // vector word size should be 4. Broadcasting of tv1 should not // matter. - for (const auto& vec_info : - cg_results.fusion_executor->kernel()->summary().vectorized_set_info) { + for (const auto& vec_info : cg_results.fusion_executor->compiledKernel() + ->kernel() + ->summary() + .vectorized_set_info) { NVF_CHECK( vec_info.word_size == 4, "Invalid vector word size: ", @@ -2010,7 +2012,13 @@ TEST_F(NVFuserTest, FusionPropagateParallelTypesToSiblings_CUDA) { fe.compileFusion(&fusion, {t0}); auto outputs = fe.runFusion({t0}); - testValidate(fe.kernel(), outputs, {t0}, {t0.mean({0})}, __LINE__, __FILE__); + testValidate( + fe.compiledKernel()->kernel(), + outputs, + {t0}, + {t0.mean({0})}, + __LINE__, + __FILE__); } // Test ExactLogicalDomainMap @@ -2600,7 +2608,8 @@ TEST_F(NVFuserTest, FusionContigPredicate_CUDA) { fe.compileFusion(&fusion, {t0}); auto cg_outputs = fe.runFusion({t0}); - testValidate(fe.kernel(), cg_outputs, {t0}, __LINE__, __FILE__); + testValidate( + fe.compiledKernel()->kernel(), cg_outputs, {t0}, __LINE__, __FILE__); } // Repro of https://github.com/csarofeen/pytorch/issues/1777 @@ -5001,7 +5010,13 @@ TEST_F(NVFuserTest, FusionIssue2163ReproInvalidAlias_CUDA) { auto ref_y = ref_x_sub_mean * at_weight.unsqueeze(0); testValidate( - fe.kernel(), {cg_output}, aten_inputs, {ref_y}, __LINE__, __FILE__, ""); + fe.compiledKernel()->kernel(), + {cg_output}, + aten_inputs, + {ref_y}, + __LINE__, + __FILE__, + ""); } // Testing scalar FP types @@ -5218,7 +5233,7 @@ TEST_F(NVFuserTest, FusionVectorizeWelford1_CUDA) { auto ref_N = at::ones({shape[1]}, options_int) * shape[0]; testValidate( - fe.kernel(), + fe.compiledKernel()->kernel(), cg_outputs, {t0}, {ref_avg, ref_var, ref_N}, @@ -5291,7 +5306,7 @@ TEST_F(NVFuserTest, FusionVectorizeWelford2_CUDA) { auto ref_N = at::ones({shape[1]}, options_int) * shape[0]; testValidate( - fe.kernel(), + fe.compiledKernel()->kernel(), cg_outputs, {t0}, {ref_avg, ref_var, ref_N}, @@ -5383,7 +5398,8 @@ TEST_F(NVFuserTest, FusionExprSortMatmulLikeSchedule_CUDA) { fe.compileFusion(&fusion, {t0, t1}); auto cg_outputs = fe.runFusion({t0, t1}); - testValidate(fe.kernel(), cg_outputs, {t0, t1}, __LINE__, __FILE__); + testValidate( + fe.compiledKernel()->kernel(), cg_outputs, {t0, t1}, __LINE__, __FILE__); } TEST_F(NVFuserTest, FusionFloatConstantWhere_CUDA) { @@ -5451,7 +5467,8 @@ TEST_F(NVFuserTest, FusionCpAsyncCommitWait_CUDA) { } auto cg_outputs = fe.runFusion({t0}); - testValidate(fe.kernel(), cg_outputs, {t0}, __LINE__, __FILE__); + testValidate( + fe.compiledKernel()->kernel(), cg_outputs, {t0}, __LINE__, __FILE__); } // Repro of issue #2459 @@ -5521,7 +5538,13 @@ TEST_F(NVFuserTest, FusionClearThreadPredicateByRAWSync_CUDA) { auto t3 = t0.sum({1}).sum({0}); auto t6 = t0.sum({1}); - testValidate(fe.kernel(), cg_outputs, inputs, {t3, t6}, __LINE__, __FILE__); + testValidate( + fe.compiledKernel()->kernel(), + cg_outputs, + inputs, + {t3, t6}, + __LINE__, + __FILE__); } namespace { @@ -5644,7 +5667,12 @@ TEST_F(NVFuserTest, FusionPredicateReductionInitShared_CUDA) { auto ref_t4 = t1.exp(); testValidate( - fe.kernel(), cg_outputs, inputs, {ref_t1, ref_t4}, __LINE__, __FILE__); + fe.compiledKernel()->kernel(), + cg_outputs, + inputs, + {ref_t1, ref_t4}, + __LINE__, + __FILE__); } // Repro of issue #2487 @@ -5698,7 +5726,12 @@ TEST_F(NVFuserTest, FusionPredicateReductionInitGlobal_CUDA) { auto ref_t3 = t1.exp(); testValidate( - fe.kernel(), cg_outputs, inputs, {ref_t1, ref_t3}, __LINE__, __FILE__); + fe.compiledKernel()->kernel(), + cg_outputs, + inputs, + {ref_t1, ref_t3}, + __LINE__, + __FILE__); } TEST_F(NVFuserTest, FusionTypePromotionATenConsistency_CUDA) { @@ -5769,9 +5802,9 @@ TEST_F(NVFuserTest, FusionCompileIndexType_CUDA) { fe.compileFusion(&fusion, large_inputs, LaunchParams(), compile_opts); NVF_CHECK( - fe.kernel()->indexType() == PrimDataType::Int, + fe.compiledKernel()->kernel()->indexType() == PrimDataType::Int, "Unexpected kernel index type: ", - fe.kernel()->indexType()); + fe.compiledKernel()->kernel()->indexType()); // Since the index type is int64, both small and large inputs // should work fine @@ -5786,9 +5819,9 @@ TEST_F(NVFuserTest, FusionCompileIndexType_CUDA) { fe.compileFusion(&fusion, small_inputs, LaunchParams(), compile_opts); NVF_CHECK( - fe.kernel()->indexType() == PrimDataType::Int, + fe.compiledKernel()->kernel()->indexType() == PrimDataType::Int, "Unexpected kernel index type: ", - fe.kernel()->indexType()); + fe.compiledKernel()->kernel()->indexType()); // Since the index type is int64, both small and large inputs // should work fine @@ -5803,9 +5836,9 @@ TEST_F(NVFuserTest, FusionCompileIndexType_CUDA) { fe.compileFusion(&fusion, small_inputs, launch_params, compile_opts); NVF_CHECK( - fe.kernel()->indexType() == PrimDataType::Int32, + fe.compiledKernel()->kernel()->indexType() == PrimDataType::Int32, "Unexpected kernel index type: ", - fe.kernel()->indexType()); + fe.compiledKernel()->kernel()->indexType()); // This should complete successfully as the arguments are small // enough to use the int32 index type @@ -6039,8 +6072,8 @@ TEST_F(NVFuserTest, FusionAvoidRedundantWriteBroadcastedSoftmaxInput_CUDA) { // check thread_pred and write_stride const auto& fe = fec.getMostRecentKernelRuntime()->executors().at(0); - auto kernel = fe.kernel(); - const auto& thread_pred_map = fe.threadPredMap(); + auto kernel = fe.compiledKernel()->kernel(); + const auto& thread_pred_map = fe.compiledKernel()->lowered()->threadPredMap(); for (const auto expr : kernel->exprs()) { auto tv = ir_utils::getTvOutput(expr); if (tv && tv->name() == 15 && tv->getMemoryType() == MemoryType::Global) { @@ -6094,8 +6127,9 @@ TEST_F(NVFuserTest, FusionAvoidRedundantWrite_CUDA) { // check thread_pred and write_stride const auto& fe = fec.getMostRecentKernelRuntime()->executors().at(0); - auto kernel = fe.kernel(); - const auto& thread_pred_map = fe.threadPredMap(); + auto kernel = fe.compiledKernel()->kernel(); + const auto& thread_pred_map = + fe.compiledKernel()->lowered()->threadPredMap(); for (const auto expr : kernel->exprs()) { auto tv = ir_utils::getTvOutput(expr); @@ -6244,8 +6278,8 @@ TEST_F(NVFuserTest, FusionAvoidRedundantWriteNonOutput_CUDA) { auto cg_outputs = fe.runFusion(inputs); // check thread_pred - auto kernel = fe.kernel(); - const auto& thread_pred_map = fe.threadPredMap(); + auto kernel = fe.compiledKernel()->kernel(); + const auto& thread_pred_map = fe.compiledKernel()->lowered()->threadPredMap(); for (const auto expr : kernel->exprs()) { auto tv = ir_utils::getTvOutput(expr); @@ -6308,8 +6342,8 @@ TEST_F(NVFuserTest, FusionAvoidRedundantWriteNonNeighbor_CUDA) { auto cg_outputs = fe.runFusion(inputs); // check thread_pred - auto kernel = fe.kernel(); - const auto& thread_pred_map = fe.threadPredMap(); + auto kernel = fe.compiledKernel()->kernel(); + const auto& thread_pred_map = fe.compiledKernel()->lowered()->threadPredMap(); for (const auto expr : kernel->exprs()) { auto tv = ir_utils::getTvOutput(expr); @@ -7850,7 +7884,7 @@ TEST_F(NVFuserTest, AvoidCachingSliceInput) { const auto num_segments = kernel_runtime->fusionSegments()->groups().size(); NVF_CHECK(num_segments == 3, "Expect 3 segments, got: ", num_segments); for (const auto& fe : kernel_runtime->executors()) { - for (auto expr : fe.fusion()->exprs()) { + for (auto expr : fe.compiledKernel()->fusion()->exprs()) { if (expr->isA()) { auto slice = expr->as(); NVF_CHECK( diff --git a/tests/cpp/test_gpu_fused_reduction.cpp b/tests/cpp/test_gpu_fused_reduction.cpp index 404b73994ba..0054f74cc3f 100644 --- a/tests/cpp/test_gpu_fused_reduction.cpp +++ b/tests/cpp/test_gpu_fused_reduction.cpp @@ -378,7 +378,8 @@ TEST_F(NVFuserTest, FusionGridAllreduce6_CUDA) { auto t0_double = t0.to(at::kDouble); auto ref = t0_double + t0_double.sum({0}).unsqueeze(0); - testValidate(fe.kernel(), outputs, {t0}, {ref}, __LINE__, __FILE__); + testValidate( + fe.compiledKernel()->kernel(), outputs, {t0}, {ref}, __LINE__, __FILE__); } TEST_F(NVFuserTest, FusionGridAllreduceWelford1_CUDA) { @@ -659,7 +660,8 @@ TEST_F(NVFuserTest, FusionGroupedReduction1_CUDA) { auto ref = t0.sum({1}) * 2; - testValidate(fe.kernel(), outputs, {t0}, {ref}, __LINE__, __FILE__); + testValidate( + fe.compiledKernel()->kernel(), outputs, {t0}, {ref}, __LINE__, __FILE__); } // Grouping reductions with different ops @@ -704,7 +706,8 @@ TEST_F(NVFuserTest, FusionGroupedReduction2_CUDA) { auto ref = (t0 + 1).sum({1}) + std::get<0>((t0 + 2).max(1)); - testValidate(fe.kernel(), outputs, {t0}, {ref}, __LINE__, __FILE__); + testValidate( + fe.compiledKernel()->kernel(), outputs, {t0}, {ref}, __LINE__, __FILE__); } // Grouped reduction with different types @@ -747,7 +750,8 @@ TEST_F(NVFuserTest, FusionGroupedReduction3_CUDA) { auto ref = t0.sum({1}) + t0.to(c10::kDouble).sum({1}).to(c10::kFloat); - testValidate(fe.kernel(), outputs, {t0}, {ref}, __LINE__, __FILE__); + testValidate( + fe.compiledKernel()->kernel(), outputs, {t0}, {ref}, __LINE__, __FILE__); } // Testing validation @@ -833,7 +837,8 @@ TEST_F(NVFuserTest, FusionGroupedReduction6_CUDA) { fe.compileFusion(&fusion, {t0}); auto outputs = fe.runFusion({t0}); - testValidate(fe.kernel(), outputs, {t0}, __LINE__, __FILE__); + testValidate( + fe.compiledKernel()->kernel(), outputs, {t0}, __LINE__, __FILE__); } TEST_F(NVFuserTest, FusionGroupedReduction7_CUDA) { @@ -898,7 +903,8 @@ TEST_F(NVFuserTest, FusionGroupedReductionRfactor1_CUDA) { auto ref = t0.sum({0}) * 2; - testValidate(fe.kernel(), outputs, {t0}, {ref}, __LINE__, __FILE__); + testValidate( + fe.compiledKernel()->kernel(), outputs, {t0}, {ref}, __LINE__, __FILE__); } // Rfactoring grouped reductions @@ -943,7 +949,8 @@ TEST_F(NVFuserTest, FusionGroupedReductionRfactor2_CUDA) { auto ref = t0.sum({0}) * 2; - testValidate(fe.kernel(), outputs, {t0}, {ref}, __LINE__, __FILE__); + testValidate( + fe.compiledKernel()->kernel(), outputs, {t0}, {ref}, __LINE__, __FILE__); } // Group reductions of tensors that have computeAt positions set @@ -989,7 +996,8 @@ TEST_F(NVFuserTest, FusionGroupedReductionAfterComputeAt_CUDA) { auto ref = (t0 + 1).sum({1}) * 2; - testValidate(fe.kernel(), outputs, {t0}, {ref}, __LINE__, __FILE__); + testValidate( + fe.compiledKernel()->kernel(), outputs, {t0}, {ref}, __LINE__, __FILE__); } TEST_F(NVFuserTest, FusionGroupAllreduce1_CUDA) { @@ -1030,7 +1038,8 @@ TEST_F(NVFuserTest, FusionGroupAllreduce1_CUDA) { auto t3 = t0.sum({0}).unsqueeze(-1); auto ref = t0 + t3 + t3; - testValidate(fe.kernel(), outputs, {t0}, {ref}, __LINE__, __FILE__); + testValidate( + fe.compiledKernel()->kernel(), outputs, {t0}, {ref}, __LINE__, __FILE__); } // Grid reductionso of different types @@ -1084,7 +1093,8 @@ TEST_F(NVFuserTest, FusionGroupAllreduce2_CUDA) { auto t6 = t0.to(c10::kDouble).sum({1}).unsqueeze(-1).to(c10::kFloat); auto ref = t0 + t2 + t6; - testValidate(fe.kernel(), outputs, {t0}, {ref}, __LINE__, __FILE__); + testValidate( + fe.compiledKernel()->kernel(), outputs, {t0}, {ref}, __LINE__, __FILE__); } // Grouping 3 grid allreduces @@ -1132,7 +1142,13 @@ TEST_F(NVFuserTest, FusionGroupAllreduce3_CUDA) { auto t6 = t0 / std::get<0>(t0.max(0)).unsqueeze(0); auto t9 = t0 - std::get<0>(t0.min(0)).unsqueeze(0); - testValidate(fe.kernel(), outputs, {t0}, {t3, t6, t9}, __LINE__, __FILE__); + testValidate( + fe.compiledKernel()->kernel(), + outputs, + {t0}, + {t3, t6, t9}, + __LINE__, + __FILE__); } // Grouping 8 grid allreduces @@ -1189,7 +1205,8 @@ TEST_F(NVFuserTest, FusionGroupAllreduce4_CUDA) { ref = add(ref, bc); } - testValidate(fe.kernel(), outputs, {t0}, {ref}, __LINE__, __FILE__); + testValidate( + fe.compiledKernel()->kernel(), outputs, {t0}, {ref}, __LINE__, __FILE__); } // Variation of FusionGroupAllreduce5_CUDA but with different @@ -1275,7 +1292,13 @@ TEST_F(NVFuserTest, FusionGroupAllreduce5_CUDA) { auto t15 = t12 / t12.sum({0}).unsqueeze(0).to(at::kComplexDouble); auto t19 = t16 / t16.sum({0}).unsqueeze(0).to(at::kComplexDouble); auto ref = t3 + t7 + t11 + t15 + t19; - testValidate(fe.kernel(), outputs, aten_inputs, {ref}, __LINE__, __FILE__); + testValidate( + fe.compiledKernel()->kernel(), + outputs, + aten_inputs, + {ref}, + __LINE__, + __FILE__); } // Persistent batchnorm backward with grouped allreduce @@ -1483,7 +1506,12 @@ TEST_F(NVFuserTest, FusionPersistentBNBackwardAllreduce_CUDA) { } testValidate( - fe.kernel(), outputs, aten_inputs, {at_grad_input}, __LINE__, __FILE__); + fe.compiledKernel()->kernel(), + outputs, + aten_inputs, + {at_grad_input}, + __LINE__, + __FILE__); } TEST_F(NVFuserTest, FusionGroupedReductionReEntrant1_CUDA) { @@ -1541,7 +1569,8 @@ TEST_F(NVFuserTest, FusionGroupedReductionReEntrant1_CUDA) { auto t0_double = t0.to(at::kDouble); auto ref = (t0_double + 1).sum({0}) + (t0_double + 2).sum({0}); - testValidate(fe.kernel(), outputs, {t0}, {ref}, __LINE__, __FILE__); + testValidate( + fe.compiledKernel()->kernel(), outputs, {t0}, {ref}, __LINE__, __FILE__); } // Channels-last batch norm with vectorization. Relies on re-entrant @@ -1664,7 +1693,13 @@ TEST_F(NVFuserTest, FusionGroupedReductionChannelsLastBatchNormLike_CUDA) { (t1_double - t2_double.unsqueeze(0).unsqueeze(0).unsqueeze(0)); auto t9 = t8.sum(at_reduction_axes); - testValidate(fe.kernel(), outputs, aten_inputs, {t5, t9}, __LINE__, __FILE__); + testValidate( + fe.compiledKernel()->kernel(), + outputs, + aten_inputs, + {t5, t9}, + __LINE__, + __FILE__); } // Test the grouped grid allreduce with BN-like outer reductions @@ -1801,7 +1836,12 @@ TEST_F( auto t13 = t1_double + t12; testValidate( - fe.kernel(), outputs, aten_inputs, {t11, t13}, __LINE__, __FILE__); + fe.compiledKernel()->kernel(), + outputs, + aten_inputs, + {t11, t13}, + __LINE__, + __FILE__); } TEST_F(NVFuserTest, FusionCrossIterationGroupedGridAllreduce1_CUDA) { @@ -1875,7 +1915,8 @@ TEST_F(NVFuserTest, FusionCrossIterationGroupedGridAllreduce1_CUDA) { auto t0_double = t0.to(at::kDouble); auto ref = t0_double + t0_double.sum({0}).unsqueeze(0); - testValidate(fe.kernel(), outputs, {t0}, {ref}, __LINE__, __FILE__); + testValidate( + fe.compiledKernel()->kernel(), outputs, {t0}, {ref}, __LINE__, __FILE__); } // Test grouping of two domains @@ -1953,7 +1994,8 @@ TEST_F(NVFuserTest, FusionCrossIterationGroupedGridAllreduce2_CUDA) { auto t0_double = t0.to(at::kDouble); auto ref = t0_double + t0_double.sum({0}).unsqueeze(0); - testValidate(fe.kernel(), outputs, {t0}, {ref}, __LINE__, __FILE__); + testValidate( + fe.compiledKernel()->kernel(), outputs, {t0}, {ref}, __LINE__, __FILE__); } // Group both expressions and iterations @@ -2039,7 +2081,8 @@ TEST_F(NVFuserTest, FusionCrossIterationGroupedGridAllreduce3_CUDA) { auto t8 = t0_double + 2 + (t0_double + 2).sum({0}).unsqueeze(0); auto ref = t4 + t8; - testValidate(fe.kernel(), outputs, {t0}, {ref}, __LINE__, __FILE__); + testValidate( + fe.compiledKernel()->kernel(), outputs, {t0}, {ref}, __LINE__, __FILE__); } // ParallelType::Group with computeAt @@ -2129,7 +2172,8 @@ TEST_F(NVFuserTest, FusionCrossIterationGroupedGridAllreduce4_CUDA) { auto t0_double = t0.to(at::kDouble); auto ref = t0_double + t0_double.sum({0}).unsqueeze(0); - testValidate(fe.kernel(), outputs, {t0}, {ref}, __LINE__, __FILE__); + testValidate( + fe.compiledKernel()->kernel(), outputs, {t0}, {ref}, __LINE__, __FILE__); } TEST_F(NVFuserTest, FusionCrossIterationGroupedGridAllreduceWelford1_CUDA) { @@ -2190,7 +2234,8 @@ TEST_F(NVFuserTest, FusionCrossIterationGroupedGridAllreduceWelford1_CUDA) { auto t0_double = t0.to(at::kDouble); auto ref = t0_double + t0_double.mean({0}).unsqueeze(0); - testValidate(fe.kernel(), outputs, {t0}, {ref}, __LINE__, __FILE__); + testValidate( + fe.compiledKernel()->kernel(), outputs, {t0}, {ref}, __LINE__, __FILE__); } // Test grouping of two domains @@ -2255,7 +2300,8 @@ TEST_F(NVFuserTest, FusionCrossIterationGroupedGridAllreduceWelford2_CUDA) { auto t0_double = t0.to(at::kDouble); auto ref = t0_double + t0_double.mean({0}).unsqueeze(0); - testValidate(fe.kernel(), outputs, {t0}, {ref}, __LINE__, __FILE__); + testValidate( + fe.compiledKernel()->kernel(), outputs, {t0}, {ref}, __LINE__, __FILE__); } // Follows the pattern of persistent outer grid welford in batchnorm diff --git a/tests/cpp/test_gpu_outer_reduction.cpp b/tests/cpp/test_gpu_outer_reduction.cpp index f8e8f3d7a2f..764abe006ee 100644 --- a/tests/cpp/test_gpu_outer_reduction.cpp +++ b/tests/cpp/test_gpu_outer_reduction.cpp @@ -119,8 +119,10 @@ TEST_F(OuterReductionTest, GroupedGridWelfordOuterOpt) { fe.compileFusion(&fusion, aten_inputs); NVF_CHECK( - fe.kernel()->summary().has_outer_grouped_grid_welford == - params.should_use_opt, + fe.compiledKernel() + ->kernel() + ->summary() + .has_outer_grouped_grid_welford == params.should_use_opt, (params.should_use_opt ? "Failed to use the optimized implementation" : "Should not use the optimized implementation"), ": ", @@ -923,7 +925,7 @@ void grid_persistent_batchnorm_manual( true); testValidate( - fe.kernel(), + fe.compiledKernel()->kernel(), {cg_outputs.at(2)}, aten_inputs, {at_output}, @@ -1251,7 +1253,7 @@ void grid_persistent_batchnorm_bwd_manual( {true, true, true}); testValidate( - fe.kernel(), + fe.compiledKernel()->kernel(), cg_outputs, aten_inputs, {std::get<0>(at_output), std::get<1>(at_output), std::get<2>(at_output)}, @@ -2189,14 +2191,23 @@ TEST_F(OuterReductionTest, IterGroupedBlockReduction) { // lowering & check iteration grouped reductions NVF_CHECK( - fusion_executor.kernel()->summary().has_iter_grouped_reductions, + fusion_executor.compiledKernel() + ->kernel() + ->summary() + .has_iter_grouped_reductions, "There must be iter domain grouped reductions."); NVF_CHECK( - fusion_executor.kernel()->summary().num_grouped_iterations == vect_factor, + fusion_executor.compiledKernel() + ->kernel() + ->summary() + .num_grouped_iterations == vect_factor, "Expected ", vect_factor, " grouped iterations, found ", - fusion_executor.kernel()->summary().num_grouped_iterations); + fusion_executor.compiledKernel() + ->kernel() + ->summary() + .num_grouped_iterations); testValidate( &fusion, diff --git a/tests/cpp/test_gpu_utils.cpp b/tests/cpp/test_gpu_utils.cpp index 053829377d1..249c337f204 100644 --- a/tests/cpp/test_gpu_utils.cpp +++ b/tests/cpp/test_gpu_utils.cpp @@ -1119,12 +1119,13 @@ TEST_F(NVFuserTest, FusionSASSDumpError_CUDA) { fe.compileFusion(&fusion, {t0}); EXPECT_THAT( - [&]() { fe.disassembledKernelSASS(); }, + [&]() { fe.compiledKernel()->disassembledKernelSASS(); }, ::testing::ThrowsMessage( ::testing::HasSubstr("I am fake"))); auto cg_outputs = fe.runFusion({t0}); - testValidate(fe.kernel(), cg_outputs, {t0}, __LINE__, __FILE__); + testValidate( + fe.compiledKernel()->kernel(), cg_outputs, {t0}, __LINE__, __FILE__); } TEST_F(NVFuserTest, ProveLinearAndGetStride) { diff --git a/tests/cpp/test_indexing_advanced.cpp b/tests/cpp/test_indexing_advanced.cpp index 344ac43d342..7caed653d83 100644 --- a/tests/cpp/test_indexing_advanced.cpp +++ b/tests/cpp/test_indexing_advanced.cpp @@ -810,7 +810,13 @@ TEST_P(AdvancedIndexingTest, 18) { auto ref = (t0.unsqueeze(-1) + t1).sum(); - testValidate(fe.kernel(), cg_outputs, inputs, {ref}, __LINE__, __FILE__); + testValidate( + fe.compiledKernel()->kernel(), + cg_outputs, + inputs, + {ref}, + __LINE__, + __FILE__); } TEST_P(AdvancedIndexingTest, 19) { diff --git a/tests/cpp/test_matmul.cpp b/tests/cpp/test_matmul.cpp index 9f4fd14cfa0..f67c602ef99 100644 --- a/tests/cpp/test_matmul.cpp +++ b/tests/cpp/test_matmul.cpp @@ -131,7 +131,7 @@ TEST_P(MatmulTestWithLayout, AmpereMatmul) { {inputs.first, inputs.second}, LaunchParams(), matmul_cparams)); - ASSERT_TRUE(getBankConflictInfo(fe.kernel()).empty()); + ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); auto cg_outputs = fe.runFusion({inputs.first, inputs.second}); auto tref = atMatmul( inputs.first.to(at::kFloat), inputs.second.to(at::kFloat), layout); @@ -183,7 +183,7 @@ TEST_P(MatmulTestWithLayout, AmperePrologueFusionBroadcast) { {inputs.first, inputs.second}, LaunchParams(), matmul_cparams)); - ASSERT_TRUE(getBankConflictInfo(fe.kernel()).empty()); + ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); auto cg_outputs = fe.runFusion({inputs.first, inputs.second}); auto tref = atMatmul( inputs.first.to(at::kFloat), inputs.second.to(at::kFloat), layout); @@ -240,7 +240,7 @@ TEST_P(MatmulTestWithLayout, AmpereProloguePointwise) { {inputs.first, inputs.second}, LaunchParams(), matmul_cparams)); - ASSERT_TRUE(getBankConflictInfo(fe.kernel()).empty()); + ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); auto cg_outputs = fe.runFusion({inputs.first, inputs.second}); auto tref = atMatmul( inputs.first.sin().to(at::kFloat), @@ -297,7 +297,7 @@ TEST_P(MatmulTestWithLayout, AmpereMatmulBFloat16) { {inputs.first, inputs.second}, LaunchParams(), matmul_cparams)); - ASSERT_TRUE(getBankConflictInfo(fe.kernel()).empty()); + ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); auto cg_outputs = fe.runFusion({inputs.first, inputs.second}); auto tref = atMatmul( inputs.first.to(at::kFloat), inputs.second.to(at::kFloat), layout); @@ -356,7 +356,7 @@ TEST_P(MatmulTestWithLayout, AmpereMatmulPipelineGmem) { {inputs.first, inputs.second}, LaunchParams(), matmul_cparams)); - ASSERT_TRUE(getBankConflictInfo(fe.kernel()).empty()); + ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); auto cg_outputs = fe.runFusion({inputs.first, inputs.second}); auto tref = atMatmul( inputs.first.to(at::kFloat), inputs.second.to(at::kFloat), layout); @@ -436,7 +436,7 @@ TEST_P(MatmulTestWithLayout, AmpereSwizzle) { {inputs.first, inputs.second}, LaunchParams(), matmul_cparams)); - ASSERT_TRUE(getBankConflictInfo(fe.kernel()).empty()); + ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); auto cg_outputs = fe.runFusion({inputs.first, inputs.second}); auto tref = atMatmul( inputs.first.to(at::kFloat), inputs.second.to(at::kFloat), layout); @@ -560,7 +560,7 @@ TEST_P(MatmulTestWithLayout, AmpereMatmulRegCircularBuffer) { {inputs.first, inputs.second}, LaunchParams(), matmul_cparams)); - ASSERT_TRUE(getBankConflictInfo(fe.kernel()).empty()); + ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); auto cg_outputs = fe.runFusion({inputs.first, inputs.second}); auto tref = atMatmul( inputs.first.to(at::kFloat), inputs.second.to(at::kFloat), layout); @@ -1275,7 +1275,7 @@ TEST_P(MatmulTestWithLayout, TuringMatmul) { FusionExecutor fe; NVFUSER_TEST_CUDA_ARCH_COMPILE_CHECK( 7, 5, fe.compileFusion(&fusion, {inputs.first, inputs.second})); - ASSERT_TRUE(getBankConflictInfo(fe.kernel()).empty()); + ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); auto cg_outputs = fe.runFusion({inputs.first, inputs.second}); auto tref = atMatmul( inputs.first.to(at::kFloat), inputs.second.to(at::kFloat), layout); @@ -1997,7 +1997,7 @@ TEST_P(MatmulTestWithLayout, AmpereMatmulLargeLoad) { {inputs.first, inputs.second}, LaunchParams(), matmul_cparams)); - ASSERT_TRUE(getBankConflictInfo(fe.kernel()).empty()); + ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); auto cg_outputs = fe.runFusion({inputs.first, inputs.second}); auto tref = atMatmul( inputs.first.to(at::kFloat), inputs.second.to(at::kFloat), layout); @@ -2049,7 +2049,7 @@ TEST_P(MatmulTestWithLayout, TuringMatmulLargeLoad) { {inputs.first, inputs.second}, LaunchParams(), matmul_cparams)); - ASSERT_TRUE(getBankConflictInfo(fe.kernel()).empty()); + ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); auto cg_outputs = fe.runFusion({inputs.first, inputs.second}); auto tref = atMatmul( inputs.first.to(at::kFloat), inputs.second.to(at::kFloat), layout); @@ -2117,7 +2117,7 @@ TEST_P(MatmulTestWithLayout, AmpereMatmulTileCheck4warp) { {inputs.first, inputs.second}, LaunchParams(), matmul_cparams)); - EXPECT_TRUE(getBankConflictInfo(fe.kernel()).empty()); + EXPECT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); auto cg_outputs = fe.runFusion({inputs.first, inputs.second}); auto tref = atMatmul( inputs.first.to(at::kFloat), inputs.second.to(at::kFloat), layout); @@ -2194,7 +2194,7 @@ TEST_P(MatmulTestWithLayout, AmpereMatmulTileCheck8warp) { {inputs.first, inputs.second}, LaunchParams(), matmul_cparams)); - ASSERT_TRUE(getBankConflictInfo(fe.kernel()).empty()); + ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); auto cg_outputs = fe.runFusion({inputs.first, inputs.second}); auto tref = atMatmul( inputs.first.to(at::kFloat), inputs.second.to(at::kFloat), layout); @@ -2261,7 +2261,7 @@ TEST_P(MatmulTestWithLayout, AmpereMatmulTileCheck6warp) { {inputs.first, inputs.second}, LaunchParams(), matmul_cparams)); - ASSERT_TRUE(getBankConflictInfo(fe.kernel()).empty()); + ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); auto cg_outputs = fe.runFusion({inputs.first, inputs.second}); auto tref = atMatmul( inputs.first.to(at::kFloat), inputs.second.to(at::kFloat), layout); @@ -2317,7 +2317,7 @@ TEST_P(MatmulTestWithLayout, AmpereMatmulLargeLoadLargeK) { {inputs.first, inputs.second}, LaunchParams(), matmul_cparams)); - ASSERT_TRUE(getBankConflictInfo(fe.kernel()).empty()); + ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); auto cg_outputs = fe.runFusion({inputs.first, inputs.second}); auto tref = atMatmul( inputs.first.to(at::kFloat), inputs.second.to(at::kFloat), layout); @@ -2367,7 +2367,7 @@ TEST_P(MatmulTestWithLayout, AmpereSplitKLikeStridedBatchedMatmul) { 8, 0, fe.compileFusion(&fusion, {t0, t1}, LaunchParams(), matmul_cparams)); - ASSERT_TRUE(getBankConflictInfo(fe.kernel()).empty()); + ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); auto cg_outputs = fe.runFusion({t0, t1}); auto tref = splitkLikeAtMatmul(t0.to(at::kFloat), t1.to(at::kFloat), layout); NVF_CHECK(cg_outputs[0].allclose(tref, 0.0001, 0.0001)); @@ -2463,7 +2463,7 @@ TEST_P(MatmulTestWithLayout, AmpereMatmulSmemEpilogue) { inputs.first.to(at::kFloat), inputs.second.to(at::kFloat), layout); // check bank conflicts - ASSERT_TRUE(getBankConflictInfo(fe.kernel()).empty()); + ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); // (0.001, 0.001) passed on local A100 but failed on CI A100 NVF_CHECK( cg_outputs[0].allclose(tref, 0.01, 0.01), @@ -2481,7 +2481,8 @@ TEST_P(MatmulTestWithLayout, AmpereMatmulSmemEpilogue) { // - !use_smem_epilogue : A + B (this test is skipped in this case) // - use_smem_epilogue && !promote_prologue_smem_reuse : A + B + C // - use_smem_epilogue && promote_prologue_smem_reuse : max(A + B, C) - auto smem_allocs = fe.kernel()->summary().dynamic_smem_allocations; + auto smem_allocs = + fe.compiledKernel()->kernel()->summary().dynamic_smem_allocations; NVF_CHECK(smem_allocs.size() == 3); if (mparams.promote_prologue_smem_reuse) { // Check prologue shared memory re-use @@ -2601,7 +2602,7 @@ TEST_F(MatmulTest, AmpereMatmulSmemEpiloguePromotionRequiredA100) { inputs.first.to(at::kFloat), inputs.second.to(at::kFloat), layout); // check bank conflicts - ASSERT_TRUE(getBankConflictInfo(fe.kernel()).empty()); + ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); // (0.001, 0.001) passed on local A100 but failed on CI A100 NVF_CHECK( cg_outputs[0].allclose(tref, 0.01, 0.01), @@ -2699,7 +2700,7 @@ TEST_P(MatmulTestWithLayout, AmpereMatmulSmemEpilogueCast) { inputs.first.to(at::kFloat), inputs.second.to(at::kFloat), layout); tref = tref.to(at::kHalf); // check bank conflicts - ASSERT_TRUE(getBankConflictInfo(fe.kernel()).empty()); + ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); // (0.001, 0.001) passed on local A100 but failed on CI A100 NVF_CHECK( cg_outputs[0].allclose(tref, 0.01, 0.01), @@ -2794,7 +2795,7 @@ TEST_P(MatmulTestWithLayout, AmpereMatmulSmemEpilogueRelu) { auto tref = at::relu(t2).to(at::kFloat); // check bank conflicts - ASSERT_TRUE(getBankConflictInfo(fe.kernel()).empty()); + ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); // (0.001, 0.001) passed on local A100 but failed on CI A100 NVF_CHECK( cg_outputs[0].allclose(tref, 0.01, 0.01), @@ -2872,7 +2873,7 @@ TEST_P(MatmulTestWithLayout, FusionAmpereMatmulSplitK_CUDA) { FusionExecutor fe; NVFUSER_TEST_CUDA_ARCH_COMPILE_CHECK( 7, 5, fe.compileFusion(&fusion, {inputs.first, inputs.second})); - EXPECT_TRUE(getBankConflictInfo(fe.kernel()).empty()); + EXPECT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); auto cg_outputs = fe.runFusion({inputs.first, inputs.second}); auto tref = atMatmul( inputs.first.to(at::kFloat), inputs.second.to(at::kFloat), layout); @@ -2938,7 +2939,7 @@ TEST_P(MatmulTestWithLayout, FusionAmpereMatmulSplitKBias_CUDA) { FusionExecutor fe; NVFUSER_TEST_CUDA_ARCH_COMPILE_CHECK( 7, 5, fe.compileFusion(&fusion, inputs)); - EXPECT_TRUE(getBankConflictInfo(fe.kernel()).empty()); + EXPECT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); auto cg_outputs = fe.runFusion(inputs); auto tref = atBiasEpilogue( atMatmul(aten_a.to(at::kFloat), aten_b.to(at::kFloat), layout), @@ -3002,7 +3003,7 @@ TEST_P(MatmulTestWithLayout, AmpereMatmulBatchSplitK) { FusionExecutor fe; NVFUSER_TEST_CUDA_ARCH_COMPILE_CHECK( 7, 5, fe.compileFusion(&fusion, inputs)); - ASSERT_TRUE(getBankConflictInfo(fe.kernel()).empty()); + ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); auto cg_outputs = fe.runFusion(inputs); auto tref = atMatmul(aten_a.to(at::kFloat), aten_b.to(at::kFloat), layout); @@ -3070,7 +3071,7 @@ TEST_P(MatmulTestWithLayout, AmpereMatmulBatchSplitKBias) { FusionExecutor fe; NVFUSER_TEST_CUDA_ARCH_COMPILE_CHECK( 7, 5, fe.compileFusion(&fusion, inputs)); - ASSERT_TRUE(getBankConflictInfo(fe.kernel()).empty()); + ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); auto cg_outputs = fe.runFusion(inputs); auto tref = atBiasEpilogue( atMatmul(aten_a.to(at::kFloat), aten_b.to(at::kFloat), layout), @@ -3133,7 +3134,7 @@ TEST_F(MatmulTest, ReproIssue1808) { {inputs.first, inputs.second}, LaunchParams(), matmul_cparams)); - ASSERT_TRUE(getBankConflictInfo(fe.kernel()).empty()); + ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); auto cg_outputs = fe.runFusion({inputs.first, inputs.second}); auto tref = atMatmul( inputs.first.to(at::kFloat), inputs.second.to(at::kFloat), layout); @@ -3283,7 +3284,7 @@ TEST_P(MatmulTestWithLayout, MisalignedVectorization) { 0, fe.compileFusion( fusion.get(), inputs, LaunchParams(), matmul_cparams)); - ASSERT_TRUE(getBankConflictInfo(fe.kernel()).empty()); + ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); auto outputs = fe.runFusion(inputs); EXPECT_TRUE(outputs[0].allclose(tref, 0.001, 0.001)); @@ -3338,7 +3339,7 @@ TEST_F(MatmulTest, MultipleConsecutiveDims) { FusionExecutor fe; NVFUSER_TEST_CUDA_ARCH_COMPILE_CHECK( 8, 0, fe.compileFusion(&fusion, inputs, LaunchParams(), matmul_cparams)); - ASSERT_TRUE(getBankConflictInfo(fe.kernel()).empty()); + ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); auto cg_outputs = fe.runFusion(inputs); auto tref = at::reshape( at::linear( @@ -3402,7 +3403,7 @@ TEST_F(MatmulTest, DISABLED_MultipleNonConsecutiveMDims) { FusionExecutor fe; NVFUSER_TEST_CUDA_ARCH_COMPILE_CHECK( 8, 0, fe.compileFusion(&fusion, inputs, LaunchParams(), matmul_cparams)); - ASSERT_TRUE(getBankConflictInfo(fe.kernel()).empty()); + ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); auto cg_outputs = fe.runFusion(inputs); auto Apermuted = A.permute({{1, 2}}).reshape({M1 * M2, K}); auto tref = at::linear(Apermuted.to(at::kFloat), B.to(at::kFloat)) @@ -3466,7 +3467,7 @@ TEST_F(MatmulTest, DISABLED_MultipleNonConsecutiveNDims) { FusionExecutor fe; NVFUSER_TEST_CUDA_ARCH_COMPILE_CHECK( 8, 0, fe.compileFusion(&fusion, inputs, LaunchParams(), matmul_cparams)); - ASSERT_TRUE(getBankConflictInfo(fe.kernel()).empty()); + ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); auto cg_outputs = fe.runFusion(inputs); auto Bpermuted = B.permute({{1, 2}}).reshape({N1 * N2, K}); auto tref = at::linear(A.to(at::kFloat), Bpermuted.to(at::kFloat)) @@ -3522,7 +3523,7 @@ TEST_F(MatmulTest, MultipleMDimsBatch) { FusionExecutor fe; NVFUSER_TEST_CUDA_ARCH_COMPILE_CHECK( 8, 0, fe.compileFusion(&fusion, inputs, LaunchParams(), matmul_cparams)); - ASSERT_TRUE(getBankConflictInfo(fe.kernel()).empty()); + ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); auto cg_outputs = fe.runFusion(inputs); auto tref = at::matmul(A.to(at::kFloat), at::permute(B.to(at::kFloat), {0, 2, 1})); diff --git a/tests/cpp/test_matmul_aten_evaluation.cpp b/tests/cpp/test_matmul_aten_evaluation.cpp index 9dd8a927009..52679993469 100644 --- a/tests/cpp/test_matmul_aten_evaluation.cpp +++ b/tests/cpp/test_matmul_aten_evaluation.cpp @@ -240,7 +240,7 @@ TEST_P(LinearNodeParametrizedTest, LinearNodeConcrete) { fec.getMostRecentKernelRuntime()->executors(); EXPECT_EQ(executors.size(), 1); // Verify that fusion compilation was skipped. - EXPECT_FALSE(executors.front().hasCompiledKernel()); + EXPECT_FALSE(executors.front().compiledKernel()->hasCompiledKernel()); EXPECT_TRUE(at::allclose(out[0], out_ref)); } @@ -290,7 +290,7 @@ TEST_P(LinearNodeParametrizedTest, LinearNodeSymbolic) { fec.getMostRecentKernelRuntime()->executors(); EXPECT_EQ(executors.size(), 1); // Verify that fusion compilation was skipped. - EXPECT_FALSE(executors.front().hasCompiledKernel()); + EXPECT_FALSE(executors.front().compiledKernel()->hasCompiledKernel()); EXPECT_TRUE(at::allclose(out[0], out_ref)); } diff --git a/tests/cpp/test_matmul_sass.cpp b/tests/cpp/test_matmul_sass.cpp index be97a6f1498..5b2d4da5c27 100644 --- a/tests/cpp/test_matmul_sass.cpp +++ b/tests/cpp/test_matmul_sass.cpp @@ -107,7 +107,7 @@ sass::Container getSASSFor( NVF_CHECK(cg_outputs[0].allclose(tref, 0.0001, 0.0001)); - return sass::parse(fe.disassembledKernelSASS()); + return sass::parse(fe.compiledKernel()->disassembledKernelSASS()); } // A fusion with epilogue made of binary op (scalar multiplication) @@ -178,7 +178,7 @@ sass::Container getBinaryOpMulEpilogueSASSFor( NVF_CHECK(cg_outputs[0].allclose(tref, 0.0001, 0.0001)); - return sass::parse(fe.disassembledKernelSASS()); + return sass::parse(fe.compiledKernel()->disassembledKernelSASS()); } } // namespace diff --git a/tests/cpp/test_mbarrier.cpp b/tests/cpp/test_mbarrier.cpp index 84c58192271..faa1643bc3e 100644 --- a/tests/cpp/test_mbarrier.cpp +++ b/tests/cpp/test_mbarrier.cpp @@ -131,7 +131,7 @@ TEST_F(MBarrierTest, Simple) { &typeid(kir::MBarrierArrive), &typeid(kir::MBarrierWait), &typeid(kir::MBarrierInvalidate)}; - for (auto expr : fe.kernel()->topLevelExprs()) { + for (auto expr : fe.compiledKernel()->kernel()->topLevelExprs()) { remaining_mbarrier_exprs.erase(&typeid(*expr)); } EXPECT_TRUE(remaining_mbarrier_exprs.empty()); diff --git a/tests/cpp/test_memory.cpp b/tests/cpp/test_memory.cpp index 6e9d074760a..76380e7f772 100644 --- a/tests/cpp/test_memory.cpp +++ b/tests/cpp/test_memory.cpp @@ -84,7 +84,8 @@ TEST_P(MemoryTest, LoadCache) { } // Verify PTX. - const executor_utils::CompiledKernel& compiled_kernel = *fe.compiledKernel(); + const executor_utils::CompiledKernel& compiled_kernel = + *fe.compiledKernel()->compiledKernel(); std::string ptx(compiled_kernel.ptx.begin(), compiled_kernel.ptx.end()); std::regex regex(R"(ld\.global\.)" + cache_op_str + R"(\.\S+)"); std::smatch match; @@ -159,7 +160,8 @@ TEST_F(MemoryTest, RefineCachePolicy) { } // Verify PTX. - const executor_utils::CompiledKernel& compiled_kernel = *fe.compiledKernel(); + const executor_utils::CompiledKernel& compiled_kernel = + *fe.compiledKernel()->compiledKernel(); std::string ptx(compiled_kernel.ptx.begin(), compiled_kernel.ptx.end()); expectMatchCount(ptx, R"(ld\.global\.ca\.v4\.\S+)", 1); expectMatchCount(ptx, R"(ld\.global\.cs\.v4\.\S+)", 1); @@ -466,11 +468,12 @@ TEST_P(TMASimpleLdstTest, Load) { FusionExecutor fe; fe.compileFusion(&fusion, {t0}, {}, matmul_cparams); - EXPECT_EQ(TMADimChecker::getDim(fe.kernel()), dim); - TMAPredicateChecker::checkPredicate(fe.kernel(), 1); + EXPECT_EQ(TMADimChecker::getDim(fe.compiledKernel()->kernel()), dim); + TMAPredicateChecker::checkPredicate(fe.compiledKernel()->kernel(), 1); ASSERT_EQ( - XorFinder::findXor(fe.kernel()), (swizzle != MmaInputSmemSwizzle::None)); - TMADimChecker::getDim(fe.kernel()); + XorFinder::findXor(fe.compiledKernel()->kernel()), + (swizzle != MmaInputSmemSwizzle::None)); + TMADimChecker::getDim(fe.compiledKernel()->kernel()); auto cg_outputs = fe.runFusion({t0}); testValidate(&fusion, cg_outputs, {t0}, {t0}, __LINE__, __FILE__); @@ -586,10 +589,11 @@ TEST_P(TMASimpleLdstTest, Store) { FusionExecutor fe; fe.compileFusion(&fusion, {t0}, {}, matmul_cparams); - EXPECT_EQ(TMADimChecker::getDim(fe.kernel()), dim); - TMAPredicateChecker::checkPredicate(fe.kernel(), 1); + EXPECT_EQ(TMADimChecker::getDim(fe.compiledKernel()->kernel()), dim); + TMAPredicateChecker::checkPredicate(fe.compiledKernel()->kernel(), 1); ASSERT_EQ( - XorFinder::findXor(fe.kernel()), (swizzle != MmaInputSmemSwizzle::None)); + XorFinder::findXor(fe.compiledKernel()->kernel()), + (swizzle != MmaInputSmemSwizzle::None)); auto cg_outputs = fe.runFusion({t0}); testValidate(&fusion, cg_outputs, {t0}, {t0}, __LINE__, __FILE__); @@ -648,8 +652,8 @@ TEST_F(TMAIndexingTest, Load2DTensorWith1DTMA) { FusionExecutor fe; fe.compileFusion(&fusion, {t0}, {}, matmul_cparams); - EXPECT_EQ(TMADimChecker::getDim(fe.kernel()), 1); - TMAPredicateChecker::checkPredicate(fe.kernel(), 1); + EXPECT_EQ(TMADimChecker::getDim(fe.compiledKernel()->kernel()), 1); + TMAPredicateChecker::checkPredicate(fe.compiledKernel()->kernel(), 1); auto cg_outputs = fe.runFusion({t0}); testValidate(&fusion, cg_outputs, {t0}, {t0}, __LINE__, __FILE__); @@ -685,8 +689,8 @@ TEST_F(TMAIndexingTest, Load1DTensorWith2DTMA) { FusionExecutor fe; fe.compileFusion(&fusion, {t0}, {}, matmul_cparams); - EXPECT_EQ(TMADimChecker::getDim(fe.kernel()), 2); - TMAPredicateChecker::checkPredicate(fe.kernel(), 1); + EXPECT_EQ(TMADimChecker::getDim(fe.compiledKernel()->kernel()), 2); + TMAPredicateChecker::checkPredicate(fe.compiledKernel()->kernel(), 1); auto cg_outputs = fe.runFusion({t0}); testValidate(&fusion, cg_outputs, {t0}, {t0}, __LINE__, __FILE__); @@ -722,8 +726,8 @@ TEST_F(TMAIndexingTest, NonOneElementStride) { FusionExecutor fe; fe.compileFusion(&fusion, {t0}, {}, matmul_cparams); - EXPECT_EQ(TMADimChecker::getDim(fe.kernel()), 2); - TMAPredicateChecker::checkPredicate(fe.kernel(), 0); + EXPECT_EQ(TMADimChecker::getDim(fe.compiledKernel()->kernel()), 2); + TMAPredicateChecker::checkPredicate(fe.compiledKernel()->kernel(), 0); auto cg_outputs = fe.runFusion({t0}); testValidate(&fusion, cg_outputs, {t0}, {t0}, __LINE__, __FILE__); @@ -793,8 +797,8 @@ TEST_F(TMAIndexingTest, Advanced) { FusionExecutor fe; fe.compileFusion(&fusion, {t0}, {}, matmul_cparams); - EXPECT_EQ(TMADimChecker::getDim(fe.kernel()), 4); - TMAPredicateChecker::checkPredicate(fe.kernel(), 1); + EXPECT_EQ(TMADimChecker::getDim(fe.compiledKernel()->kernel()), 4); + TMAPredicateChecker::checkPredicate(fe.compiledKernel()->kernel(), 1); auto cg_outputs = fe.runFusion({t0}); testValidate(&fusion, cg_outputs, {t0}, {t0}, __LINE__, __FILE__); @@ -842,8 +846,9 @@ TEST_F(TMAIndexingTest, DefineBoxByCompositing1) { FusionExecutor fe; fe.compileFusion(&fusion, {t0}, {}, matmul_cparams); - EXPECT_EQ(TMADimChecker::getDim(fe.kernel()), 4); - EXPECT_FALSE(PredicatedChecker::isPredicated(tv1, fe.kernel())); + EXPECT_EQ(TMADimChecker::getDim(fe.compiledKernel()->kernel()), 4); + EXPECT_FALSE( + PredicatedChecker::isPredicated(tv1, fe.compiledKernel()->kernel())); auto cg_outputs = fe.runFusion({t0}); testValidate(&fusion, cg_outputs, {t0}, {t0}, __LINE__, __FILE__); @@ -895,8 +900,9 @@ TEST_F(TMAIndexingTest, DefineBoxByCompositing2) { FusionExecutor fe; fe.compileFusion(&fusion, {t0}, {}, matmul_cparams); - EXPECT_EQ(TMADimChecker::getDim(fe.kernel()), 5); - EXPECT_FALSE(PredicatedChecker::isPredicated(tv1, fe.kernel())); + EXPECT_EQ(TMADimChecker::getDim(fe.compiledKernel()->kernel()), 5); + EXPECT_FALSE( + PredicatedChecker::isPredicated(tv1, fe.compiledKernel()->kernel())); auto cg_outputs = fe.runFusion({t0}); testValidate(&fusion, cg_outputs, {t0}, {t0}, __LINE__, __FILE__); @@ -956,8 +962,8 @@ TEST_F(TMAIndexingTest, DefineBoxByRotation1) { FusionExecutor fe; fe.compileFusion(&fusion, {t0}, {}, matmul_cparams); - EXPECT_EQ(TMADimChecker::getDim(fe.kernel()), 3); - TMAPredicateChecker::checkPredicate(fe.kernel(), 1); + EXPECT_EQ(TMADimChecker::getDim(fe.compiledKernel()->kernel()), 3); + TMAPredicateChecker::checkPredicate(fe.compiledKernel()->kernel(), 1); auto cg_outputs = fe.runFusion({t0}); testValidate(&fusion, cg_outputs, {t0}, {t0}, __LINE__, __FILE__); @@ -1008,8 +1014,8 @@ TEST_F(TMAIndexingTest, DefineBoxByRotation2) { // interpreted as viewing then tensor as 2D (M/8, 8) and then applying 2D TMA. // The outer dim of TMA is defined by boxing and striding splits, and the // inner dim is defined as implicit whole. - EXPECT_EQ(TMADimChecker::getDim(fe.kernel()), 2); - TMAPredicateChecker::checkPredicate(fe.kernel(), 1); + EXPECT_EQ(TMADimChecker::getDim(fe.compiledKernel()->kernel()), 2); + TMAPredicateChecker::checkPredicate(fe.compiledKernel()->kernel(), 1); auto cg_outputs = fe.runFusion({t0}); testValidate(&fusion, cg_outputs, {t0}, {t0}, __LINE__, __FILE__); @@ -1071,8 +1077,8 @@ TEST_F(TMAIndexingTest, DefineBoxByRotation3) { // TMA. The dim 0 of TMA is as implicit size-one, and the dim 1 is defined by // a boxing split whose box size is 8, and dim 2 is an implicit whole box with // size N. - EXPECT_EQ(TMADimChecker::getDim(fe.kernel()), 3); - TMAPredicateChecker::checkPredicate(fe.kernel(), 1); + EXPECT_EQ(TMADimChecker::getDim(fe.compiledKernel()->kernel()), 3); + TMAPredicateChecker::checkPredicate(fe.compiledKernel()->kernel(), 1); auto cg_outputs = fe.runFusion({t0}); testValidate(&fusion, cg_outputs, {t0}, {t0}, __LINE__, __FILE__); @@ -1127,9 +1133,9 @@ TEST_F(TMAIndexingTest, NonTrivialGmemAllocationDomain1) { FusionExecutor fe; fe.compileFusion(&fusion, {t0}, {}, matmul_cparams); - EXPECT_EQ(TMADimChecker::getDim(fe.kernel()), 2); - TMAPredicateChecker::checkPredicate(fe.kernel(), 1); - ASSERT_TRUE(XorFinder::findXor(fe.kernel())); + EXPECT_EQ(TMADimChecker::getDim(fe.compiledKernel()->kernel()), 2); + TMAPredicateChecker::checkPredicate(fe.compiledKernel()->kernel(), 1); + ASSERT_TRUE(XorFinder::findXor(fe.compiledKernel()->kernel())); auto cg_outputs = fe.runFusion({t0}); testValidate(&fusion, cg_outputs, {t0}, {t0}, __LINE__, __FILE__); @@ -1182,8 +1188,8 @@ TEST_F(TMAIndexingTest, NonTrivialGmemAllocationDomain2) { FusionExecutor fe; fe.compileFusion(&fusion, {t0}, {}, matmul_cparams); - EXPECT_EQ(TMADimChecker::getDim(fe.kernel()), 3); - TMAPredicateChecker::checkPredicate(fe.kernel(), 1); + EXPECT_EQ(TMADimChecker::getDim(fe.compiledKernel()->kernel()), 3); + TMAPredicateChecker::checkPredicate(fe.compiledKernel()->kernel(), 1); auto cg_outputs = fe.runFusion({t0}); testValidate(&fusion, cg_outputs, {t0}, {t0}, __LINE__, __FILE__); @@ -1230,8 +1236,8 @@ TEST_F(TMAMiscTest, AdvancedThreadParallelizationLoad) { FusionExecutor fe; fe.compileFusion(&fusion, {t0}, {}, matmul_cparams); - EXPECT_EQ(TMADimChecker::getDim(fe.kernel()), 1); - TMAPredicateChecker::checkPredicate(fe.kernel(), 4); + EXPECT_EQ(TMADimChecker::getDim(fe.compiledKernel()->kernel()), 1); + TMAPredicateChecker::checkPredicate(fe.compiledKernel()->kernel(), 4); auto cg_outputs = fe.runFusion({t0}); testValidate(&fusion, cg_outputs, {t0}, {t0}, __LINE__, __FILE__); @@ -1273,8 +1279,8 @@ TEST_F(TMAMiscTest, AdvancedThreadParallelizationStore) { FusionExecutor fe; fe.compileFusion(&fusion, {t0}, {}, matmul_cparams); - EXPECT_EQ(TMADimChecker::getDim(fe.kernel()), 1); - TMAPredicateChecker::checkPredicate(fe.kernel(), 4); + EXPECT_EQ(TMADimChecker::getDim(fe.compiledKernel()->kernel()), 1); + TMAPredicateChecker::checkPredicate(fe.compiledKernel()->kernel(), 4); auto cg_outputs = fe.runFusion({t0}); testValidate(&fusion, cg_outputs, {t0}, {t0}, __LINE__, __FILE__); @@ -1309,8 +1315,8 @@ TEST_F(TMAMiscTest, DisableIndexHoisting) { FusionExecutor fe; fe.compileFusion(&fusion, {t0}, {}, matmul_cparams); - EXPECT_EQ(TMADimChecker::getDim(fe.kernel()), 1); - TMAPredicateChecker::checkPredicate(fe.kernel(), 0); + EXPECT_EQ(TMADimChecker::getDim(fe.compiledKernel()->kernel()), 1); + TMAPredicateChecker::checkPredicate(fe.compiledKernel()->kernel(), 0); auto cg_outputs = fe.runFusion({t0}); testValidate(&fusion, cg_outputs, {t0}, {t0}, __LINE__, __FILE__); @@ -1341,8 +1347,8 @@ TEST_F(TMAMiscTest, Repro1977) { FusionExecutor fe; fe.compileFusion(&fusion, {t0}, {}, matmul_cparams); - EXPECT_EQ(TMADimChecker::getDim(fe.kernel()), 1); - TMAPredicateChecker::checkPredicate(fe.kernel(), 0); + EXPECT_EQ(TMADimChecker::getDim(fe.compiledKernel()->kernel()), 1); + TMAPredicateChecker::checkPredicate(fe.compiledKernel()->kernel(), 0); auto cg_outputs = fe.runFusion({t0}); testValidate(&fusion, cg_outputs, {t0}, {t0}, __LINE__, __FILE__); @@ -1388,7 +1394,7 @@ TEST_F(TMAMiscTest, LoadStrongCorrectness) { FusionExecutor fe; fe.compileFusion(&fusion, {t0}, {}, matmul_cparams); - EXPECT_EQ(TMADimChecker::getDim(fe.kernel()), 1); + EXPECT_EQ(TMADimChecker::getDim(fe.compiledKernel()->kernel()), 1); auto cg_outputs = fe.runFusion({t0}); @@ -1472,8 +1478,8 @@ TEST_F(TMARuntimeInvalidTest, MisalignedGlobalAddress) { FusionExecutor fe; fe.compileFusion(&fusion, {t0_aligned}, {}, matmul_cparams); - EXPECT_EQ(TMADimChecker::getDim(fe.kernel()), 1); - TMAPredicateChecker::checkPredicate(fe.kernel(), 1); + EXPECT_EQ(TMADimChecker::getDim(fe.compiledKernel()->kernel()), 1); + TMAPredicateChecker::checkPredicate(fe.compiledKernel()->kernel(), 1); auto cg_outputs = fe.runFusion({t0_aligned}); testValidate( @@ -1527,8 +1533,8 @@ TEST_F(TMARuntimeInvalidTest, MisalignedGlobalStride) { FusionExecutor fe; fe.compileFusion(&fusion, {t0_aligned}, {}, matmul_cparams); - EXPECT_EQ(TMADimChecker::getDim(fe.kernel()), 2); - TMAPredicateChecker::checkPredicate(fe.kernel(), 1); + EXPECT_EQ(TMADimChecker::getDim(fe.compiledKernel()->kernel()), 2); + TMAPredicateChecker::checkPredicate(fe.compiledKernel()->kernel(), 1); auto cg_outputs = fe.runFusion({t0_aligned}); testValidate( @@ -1621,8 +1627,8 @@ TEST_F(TMARuntimeInvalidTest, SizeOfTransfer) { FusionExecutor fe; fe.compileFusion(&fusion, {t0, items_of_16_bytes}, {}, matmul_cparams); - EXPECT_EQ(TMADimChecker::getDim(fe.kernel()), 1); - TMAPredicateChecker::checkPredicate(fe.kernel(), 1); + EXPECT_EQ(TMADimChecker::getDim(fe.compiledKernel()->kernel()), 1); + TMAPredicateChecker::checkPredicate(fe.compiledKernel()->kernel(), 1); auto cg_outputs = fe.runFusion({t0, items_of_16_bytes}); testValidate( @@ -1674,7 +1680,7 @@ TEST_F(TMARuntimeInvalidTest, InvalidView) { FusionExecutor fe; fe.compileFusion(&fusion, {t0_valid}, {}, matmul_cparams); - EXPECT_EQ(TMADimChecker::getDim(fe.kernel()), 2); + EXPECT_EQ(TMADimChecker::getDim(fe.compiledKernel()->kernel()), 2); auto cg_outputs = fe.runFusion({t0_valid}); testValidate(&fusion, cg_outputs, {t0_valid}, {t0_valid}, __LINE__, __FILE__); @@ -1918,8 +1924,8 @@ TEST_F(TMADocTest, Figure14a) { FusionExecutor fe; fe.compileFusion(&fusion, {t0}, {}, matmul_cparams); - EXPECT_EQ(TMADimChecker::getDim(fe.kernel()), 2); - TMAPredicateChecker::checkPredicate(fe.kernel(), 0); + EXPECT_EQ(TMADimChecker::getDim(fe.compiledKernel()->kernel()), 2); + TMAPredicateChecker::checkPredicate(fe.compiledKernel()->kernel(), 0); auto cg_outputs = fe.runFusion({t0}); testValidate(&fusion, cg_outputs, {t0}, {t0}, __LINE__, __FILE__); @@ -1994,8 +2000,8 @@ TEST_F(TMADocTest, Figure14b) { FusionExecutor fe; fe.compileFusion(&fusion, {t0}, {}, matmul_cparams); - EXPECT_EQ(TMADimChecker::getDim(fe.kernel()), 2); - TMAPredicateChecker::checkPredicate(fe.kernel(), 0); + EXPECT_EQ(TMADimChecker::getDim(fe.compiledKernel()->kernel()), 2); + TMAPredicateChecker::checkPredicate(fe.compiledKernel()->kernel(), 0); auto cg_outputs = fe.runFusion({t0}); testValidate(&fusion, cg_outputs, {t0}, {t0}, __LINE__, __FILE__); @@ -2072,8 +2078,8 @@ TEST_F(TMADocTest, Figure14c) { FusionExecutor fe; fe.compileFusion(&fusion, {t0}, {}, matmul_cparams); - EXPECT_EQ(TMADimChecker::getDim(fe.kernel()), 2); - TMAPredicateChecker::checkPredicate(fe.kernel(), 0); + EXPECT_EQ(TMADimChecker::getDim(fe.compiledKernel()->kernel()), 2); + TMAPredicateChecker::checkPredicate(fe.compiledKernel()->kernel(), 0); auto cg_outputs = fe.runFusion({t0}); testValidate(&fusion, cg_outputs, {t0}, {t0}, __LINE__, __FILE__); @@ -2143,8 +2149,8 @@ TEST_F(TMADocTest, Figure14d) { FusionExecutor fe; fe.compileFusion(&fusion, {t0}, {}, matmul_cparams); - EXPECT_EQ(TMADimChecker::getDim(fe.kernel()), 2); - TMAPredicateChecker::checkPredicate(fe.kernel(), 1); + EXPECT_EQ(TMADimChecker::getDim(fe.compiledKernel()->kernel()), 2); + TMAPredicateChecker::checkPredicate(fe.compiledKernel()->kernel(), 1); auto cg_outputs = fe.runFusion({t0}); testValidate(&fusion, cg_outputs, {t0}, {t0}, __LINE__, __FILE__); @@ -2223,8 +2229,8 @@ TEST_F(TMADocTest, Figure14e) { FusionExecutor fe; fe.compileFusion(&fusion, {t0}, {}, matmul_cparams); - EXPECT_EQ(TMADimChecker::getDim(fe.kernel()), 2); - TMAPredicateChecker::checkPredicate(fe.kernel(), 1); + EXPECT_EQ(TMADimChecker::getDim(fe.compiledKernel()->kernel()), 2); + TMAPredicateChecker::checkPredicate(fe.compiledKernel()->kernel(), 1); auto cg_outputs = fe.runFusion({t0}); testValidate(&fusion, cg_outputs, {t0}, {t0}, __LINE__, __FILE__); @@ -2268,8 +2274,8 @@ TEST_F(TMADocTest, Figure15a) { FusionExecutor fe; fe.compileFusion(&fusion, {t0}, {}, matmul_cparams); - EXPECT_EQ(TMADimChecker::getDim(fe.kernel()), 2); - TMAPredicateChecker::checkPredicate(fe.kernel(), 0); + EXPECT_EQ(TMADimChecker::getDim(fe.compiledKernel()->kernel()), 2); + TMAPredicateChecker::checkPredicate(fe.compiledKernel()->kernel(), 0); auto cg_outputs = fe.runFusion({t0}); testValidate(&fusion, cg_outputs, {t0}, {t0}, __LINE__, __FILE__); @@ -2310,8 +2316,8 @@ TEST_F(TMADocTest, Figure15b) { FusionExecutor fe; fe.compileFusion(&fusion, {t0}, {}, matmul_cparams); - EXPECT_EQ(TMADimChecker::getDim(fe.kernel()), 2); - TMAPredicateChecker::checkPredicate(fe.kernel(), 4); + EXPECT_EQ(TMADimChecker::getDim(fe.compiledKernel()->kernel()), 2); + TMAPredicateChecker::checkPredicate(fe.compiledKernel()->kernel(), 4); auto cg_outputs = fe.runFusion({t0}); testValidate(&fusion, cg_outputs, {t0}, {t0}, __LINE__, __FILE__); diff --git a/tests/cpp/test_multidevice_lower_communication.cpp b/tests/cpp/test_multidevice_lower_communication.cpp index 3c454777f0f..114f5dee4a0 100644 --- a/tests/cpp/test_multidevice_lower_communication.cpp +++ b/tests/cpp/test_multidevice_lower_communication.cpp @@ -19,10 +19,10 @@ namespace nvfuser { namespace { void assertIsCompiledToHostIrContainer(const FusionExecutorCache& fec) { FusionKernelRuntime* runtime = fec.getMostRecentKernelRuntime(); - const std::vector& executors = runtime->executors(); - EXPECT_THAT(executors, testing::SizeIs(1)); - for (const auto& executor : executors) { - EXPECT_TRUE(executor.fusion()->isA()) + const std::vector& fes = runtime->executors(); + EXPECT_THAT(fes, testing::SizeIs(1)); + for (const auto& fe : fes) { + EXPECT_TRUE(fe.hostIrContainer()) << "failed to compile to a HostIrContainer with Communications"; } } diff --git a/tests/cpp/test_no_op.cpp b/tests/cpp/test_no_op.cpp index a6e35e9b9ac..bcf3de4a296 100644 --- a/tests/cpp/test_no_op.cpp +++ b/tests/cpp/test_no_op.cpp @@ -230,7 +230,9 @@ TEST_F(NoOpTest, ExpandedReduction) { runtime->fusionSegments()->groups(), UnorderedElementsAre(HeuristicIs(SchedulerType::NoOp))); const auto& executor = runtime->executors().front(); - EXPECT_THAT(executor.kernel()->summary().global_allocations, IsEmpty()); + EXPECT_THAT( + executor.compiledKernel()->kernel()->summary().global_allocations, + IsEmpty()); } } // namespace nvfuser diff --git a/tests/cpp/test_predicate_elimination.cpp b/tests/cpp/test_predicate_elimination.cpp index 8b941f7e0e6..84c27c5388e 100644 --- a/tests/cpp/test_predicate_elimination.cpp +++ b/tests/cpp/test_predicate_elimination.cpp @@ -390,7 +390,8 @@ TEST_F(PredicateEliminationTest, 8) { fec.getMostRecentKernelRuntime()->executors(); NVF_CHECK(compiled_executors.size() == 1, "Unexpected scheduling"); NVF_CHECK( - !PredicatedChecker::isPredicated(tv6, compiled_executors.at(0).kernel()), + !PredicatedChecker::isPredicated( + tv6, compiled_executors.at(0).compiledKernel()->kernel()), "T6 should not be predicated"); } diff --git a/tests/cpp/test_resize.cpp b/tests/cpp/test_resize.cpp index 7fb13768040..f9ec51be0ca 100644 --- a/tests/cpp/test_resize.cpp +++ b/tests/cpp/test_resize.cpp @@ -1889,8 +1889,11 @@ TEST_F(ResizeTest, FusionSliceForNanoGPT1) { FusionExecutorCache executor_cache(std::move(fusion_ptr)); auto cg_outputs = executor_cache.runFusionWithInputs(aten_inputs); - auto kernel = - executor_cache.getMostRecentKernelRuntime()->executors().at(0).kernel(); + auto kernel = executor_cache.getMostRecentKernelRuntime() + ->executors() + .at(0) + .compiledKernel() + ->kernel(); NVF_CHECK( !kernel->summary().has_cooperative_grid_reduction, "Grid sync should not be used as slicing input should avoid input caching"); @@ -1951,8 +1954,11 @@ TEST_F(ResizeTest, FusionSliceForNanoGPT2) { FusionExecutorCache executor_cache(std::move(fusion_ptr)); auto cg_outputs = executor_cache.runFusionWithInputs(aten_inputs); - auto kernel = - executor_cache.getMostRecentKernelRuntime()->executors().at(0).kernel(); + auto kernel = executor_cache.getMostRecentKernelRuntime() + ->executors() + .at(0) + .compiledKernel() + ->kernel(); // Make sure the slices ops use the same producer TensorView* known_slice_producer = nullptr; diff --git a/tests/cpp/test_swizzle.cpp b/tests/cpp/test_swizzle.cpp index 20c52fe2810..cda00204338 100644 --- a/tests/cpp/test_swizzle.cpp +++ b/tests/cpp/test_swizzle.cpp @@ -737,7 +737,7 @@ TEST_F(SwizzleTest, Transpose1) { auto t = at::randn({10240, 10240}, options); FusionExecutor fe; fe.compileFusion(&fusion, {t}); - EXPECT_TRUE(getBankConflictInfo(fe.kernel()).empty()); + EXPECT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); std::vector outputs = fe.runFusion({t}); EXPECT_TRUE(at::equal(t.t(), outputs[0])); } diff --git a/tests/cpp/test_translate_mma.cpp b/tests/cpp/test_translate_mma.cpp index 2f4cf4f81e3..f951573db67 100644 --- a/tests/cpp/test_translate_mma.cpp +++ b/tests/cpp/test_translate_mma.cpp @@ -232,7 +232,7 @@ TEST_P(CombineMulSumAsMmaTestWithLayout, AmpereMulSumToMatmul_Schedule) { FusionExecutor fe; fe.compileFusion( &fusion, {inputs.first, inputs.second}, LaunchParams(), matmul_cparams); - ASSERT_TRUE(getBankConflictInfo(fe.kernel()).empty()); + ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); auto cg_outputs = fe.runFusion({inputs.first, inputs.second}); auto tref = atMatmul( inputs.first.to(at::kFloat), inputs.second.to(at::kFloat), layout); @@ -273,7 +273,8 @@ TEST_P(CombineMulSumAsMmaTestWithLayout, UseMatmulScheduler) { ir_utils::getOpsOfType(executor_cache.getMostRecentKernelRuntime() ->executors() .at(0) - .kernel()) + .compiledKernel() + ->kernel()) .empty()); // Ensure that the matmul scheduler ran. @@ -392,9 +393,9 @@ TEST_P(MatmulNodeTranslationTest, AutomaticSchedulerMatmulNode) { if (scheduler_type == SchedulerType::Matmul) { // Ensure there's an MmaOp. - EXPECT_FALSE( - ir_utils::getOpsOfType(runtime->executors().at(0).kernel()) - .empty()); + EXPECT_FALSE(ir_utils::getOpsOfType( + runtime->executors().at(0).compiledKernel()->kernel()) + .empty()); } testValidate( @@ -573,9 +574,9 @@ TEST_P(LinearNodeTranslationTest, AutomaticSchedulerLinearNode) { // do if ExprEval accepts the segment. ASSERT_EQ(scheduler_type, SchedulerType::Matmul); // Ensure there's an MmaOp. - EXPECT_FALSE( - ir_utils::getOpsOfType(runtime->executors().at(0).kernel()) - .empty()); + EXPECT_FALSE(ir_utils::getOpsOfType( + runtime->executors().at(0).compiledKernel()->kernel()) + .empty()); } testValidate( From 5d02ca16c141ce769548bdde9f3fda177eda2202 Mon Sep 17 00:00:00 2001 From: Christian Sarofeen Date: Mon, 7 Oct 2024 08:20:05 -0700 Subject: [PATCH 09/11] Fix remaining tests. --- csrc/runtime/executor.cpp | 17 ++++++---- csrc/runtime/executor.h | 60 +++++++++------------------------ tests/cpp/test_external_src.cpp | 24 +++---------- tests/cpp/test_gpu2.cpp | 15 ++++++--- tests/cpp/test_gpu3.cpp | 4 ++- 5 files changed, 45 insertions(+), 75 deletions(-) diff --git a/csrc/runtime/executor.cpp b/csrc/runtime/executor.cpp index 25ba5b2254c..cd4e6b4bba7 100644 --- a/csrc/runtime/executor.cpp +++ b/csrc/runtime/executor.cpp @@ -111,6 +111,8 @@ void FusionExecutor::compileFusion( int64_t concrete_id, int64_t runtime_id, int64_t group_id) { + group_id_ = group_id; + scheduler_type_ = scheduler_type; FUSER_PERF_SCOPE("FusionExecutor::compileFusion"); NVF_ERROR( !_fusion->outputs().empty(), @@ -897,13 +899,16 @@ std::vector FusionExecutor::runFusion( if (isProfilerEnabled()) { NVF_CHECK( - compiledKernel_()->groupId() >= 0, + (compiledKernel_() && compiledKernel_()->groupId() >= 0) || + group_id_ >= 0, "An invalid segment id is passed to FusionProfiler!:", - compiledKernel_()->groupId()); - SegmentProfiler& sprof = - FusionProfiler::segment(compiledKernel_()->groupId()); + compiledKernel_() ? compiledKernel_()->groupId() : group_id_); + SegmentProfiler& sprof = FusionProfiler::segment( + compiledKernel_() ? compiledKernel_()->groupId() : group_id_); sprof.inputBytesAccessed(inputBytesProcessed(args)); - sprof.scheduler(toString(compiledKernel_()->schedulerType())); + sprof.scheduler(toString( + compiledKernel_() ? compiledKernel_()->schedulerType() + : scheduler_type_)); sprof.startKernel(args.getDeviceIndex()); } @@ -918,7 +923,7 @@ std::vector FusionExecutor::runFusion( FUSER_PERF_SCOPE("FusionExecutor::runFusion::evaluate_with_ExprEval"); outputs = evaluateFusionOutputs(outputs, expr_eval); if (isProfilerEnabled()) { - auto& sprof = FusionProfiler::segment(compiledKernel_()->groupId()); + auto& sprof = FusionProfiler::segment(group_id_); sprof.stopKernel(); sprof.outputBytesAccessed(outputBytesProcessed(outputs)); } diff --git a/csrc/runtime/executor.h b/csrc/runtime/executor.h index 7e7ad3f59d4..06634822e88 100644 --- a/csrc/runtime/executor.h +++ b/csrc/runtime/executor.h @@ -136,8 +136,8 @@ class FusionExecutor : public NonCopyable { if (compiledKernel()) { return true; } - int num_compiled_artifacts = (fusion_ != nullptr) + (lowered_ != nullptr) + - (host_ir_container_ != nullptr); + int num_compiled_artifacts = + (fusion_ != nullptr) + (host_ir_container_ != nullptr); NVF_ERROR(num_compiled_artifacts <= 1); return num_compiled_artifacts == 1; }; @@ -231,10 +231,19 @@ class FusionExecutor : public NonCopyable { return compiled_kernel_2_; } + const std::unique_ptr& initCompiledKernel() { + compiledKernel_() = std::make_unique(); + return compiledKernel(); + } + const std::unique_ptr& hostIrContainer() const { return host_ir_container_; } + const std::unique_ptr& fusion() const { + return fusion_; + } + private: LaunchParams computeLaunchParams( const LaunchParams& launch_constraints, @@ -345,52 +354,13 @@ class FusionExecutor : public NonCopyable { //! compiled kernel at the current shared memory/L1 configuration std::optional available_dynamic_smem_size_ = std::nullopt; - // Assuming sm70 or above: - // limit of statically allocated smem is 48 KB: - // See: - // https://docs.nvidia.com/cuda/cuda-c-programming-guide/index.html#shared-memory-7-x - // https://docs.nvidia.com/cuda/cuda-c-programming-guide/index.html#shared-memory-8-x - const int64_t max_static_smem_ = 48 << 10; - int64_t warp_size_ = 0; - std::unique_ptr compiled_kernel_; - - // TensorViews actually used in the kernel. - std::vector used_tvs_; - - // ID of fusion in python frontend fusion cache, which maps to a single - // FusionExecutorCache. - int64_t fusion_id_ = -1; - - // ID of (device, concrete_info) key in FusionExecutorCache - int64_t concrete_id_ = -1; - - // ID of FusionKernelRuntime given (device, concrete_info) key - int64_t runtime_id_ = -1; - - // ID of segment in FusionKernelRuntime - int64_t group_id_ = -1; - - inline static std::atomic global_fusion_count_; - - // Scheduling Heuristic for this Fusion - SchedulerType scheduler_type_ = SchedulerType::None; - - // Kernel name for fusion executor - std::string kernel_id_; - - std::unique_ptr lowered_; // Initialized for non-compiled fusions std::unique_ptr fusion_; std::unique_ptr host_ir_container_; - // Track the block size this kernel was compiled with. If the block size - // increases, recompile to adjust maxregister count. - int64_t block_size_high_water_mark_ = 1; - int64_t maxrregcount_high_water_mark_ = 255; - // lookup table to take short cut to retrieve recorded information in order // to launch kernels without re-inference parameters. std::unordered_map executor_entry_lookup_; @@ -415,9 +385,6 @@ class FusionExecutor : public NonCopyable { // Profiling support: the last launch param used LaunchParams launch_params_; - // Profiling support: kept copy of the cuda kernel - std::string kernel_code_; - // Lowering hooks that are called after the GpuLower instance is created // before running lowering passes. // The main use case is for unit tests to modify the lowering process. @@ -428,6 +395,11 @@ class FusionExecutor : public NonCopyable { std::vector> post_lowering_hooks_; Communicator* communicator_; + + // TODO: Remove once we remove ExprEval and HostIR compilation and execution + // from this class + int64_t group_id_ = -1; + SchedulerType scheduler_type_ = SchedulerType::None; }; } // namespace nvfuser diff --git a/tests/cpp/test_external_src.cpp b/tests/cpp/test_external_src.cpp index ba5fe6c6604..27bcb7c06e9 100644 --- a/tests/cpp/test_external_src.cpp +++ b/tests/cpp/test_external_src.cpp @@ -38,13 +38,12 @@ TEST_F(ExternalSrcExample, Reduction_CUDA) { return; } - std::cout << "Compiling " << path << std::endl; std::ifstream cuda_src(path); std::stringstream buffer; buffer << cuda_src.rdbuf(); std::string cuda_src_str = buffer.str(); - fe.compiledKernel()->compileRtc( + fe.initCompiledKernel()->compileRtc( cuda_src_str, "kernel1", true, PrimDataType::Int32); // The following is a sample launch pattern of the compiled @@ -72,25 +71,16 @@ TEST_F(ExternalSrcExample, Reduction_CUDA) { auto t3 = t2.unsqueeze(0).unsqueeze(0).unsqueeze(0); auto ref = t1 - t3; - float read_write_bytes = - input_shape[0] * input_shape[1] * input_shape[2] * input_shape[3] * 2 * 2; - for (int i = 0; i < 5; ++i) { auto t14 = at::zeros_like(t0, options_float); auto t15 = at::zeros_like(t0, options_float); auto t16 = at::zeros_like(t0, options_int); auto t17 = at::zeros({8}, options_long); clearL2Cache(); - std::cout << "Launching the kernel" << std::endl; - float elapsed_time_ms = fe.compiledKernel()->runRtc( + fe.compiledKernel()->runRtc( lp, {t0, t7, t14, t15, t16, t17}, PrimDataType::Int32); - std::cout << "kernel run in " << elapsed_time_ms << " ms, achieved " - << (read_write_bytes / elapsed_time_ms / 1000.0 / 1000.0) - << " GB/s" << std::endl; auto fusion_out = t7.to(at::kFloat); - std::cout << "Max diff: " << (ref - fusion_out).abs().max().item() - << std::endl; NVF_CHECK(ref.allclose(fusion_out, /*rtol*/ 0.005, /*atol*/ 0.5)); } } @@ -110,13 +100,13 @@ TEST_F(ExternalSrcExample, Matmul_CUDA) { return; } - std::cout << "Compiling " << path << std::endl; + // std::cout << "Compiling " << path << std::endl; std::ifstream cuda_src(path); std::stringstream buffer; buffer << cuda_src.rdbuf(); std::string cuda_src_str = buffer.str(); - fe.compiledKernel()->compileRtc( + fe.initCompiledKernel()->compileRtc( cuda_src_str, "kernel1", true, PrimDataType::Int32); int M = 2048, N = 3456, K = 2048; @@ -130,13 +120,9 @@ TEST_F(ExternalSrcExample, Matmul_CUDA) { for (int i = 0; i < 5; ++i) { auto output = at::zeros_like(at_output); clearL2Cache(); - std::cout << "Launching the kernel" << std::endl; - float elapsed_time_ms = fe.compiledKernel()->runRtc( + fe.compiledKernel()->runRtc( lp, {inputs.first, inputs.second, output}, PrimDataType::Int32); - std::cout << "kernel run in " << elapsed_time_ms << " ms." << std::endl; - std::cout << "Max diff: " << (at_output - output).abs().max().item() - << std::endl; NVF_CHECK(at_output.allclose(output, /*rtol*/ 0.005, /*atol*/ 0.5)); } } diff --git a/tests/cpp/test_gpu2.cpp b/tests/cpp/test_gpu2.cpp index b04b36eae69..97723f743f7 100644 --- a/tests/cpp/test_gpu2.cpp +++ b/tests/cpp/test_gpu2.cpp @@ -1974,7 +1974,8 @@ __global__ void kernel1(Tensor T0, Tensor T1) { } } )"; - fe.compiledKernel()->compileRtc(kernel, "kernel1", false, PrimDataType::Int); + fe.initCompiledKernel()->compileRtc( + kernel, "kernel1", false, PrimDataType::Int); LaunchParams lp( 256, // gdimx 1, // gdimy @@ -2030,7 +2031,8 @@ __global__ void kernel1( } } )"; - fe.compiledKernel()->compileRtc(kernel, "kernel1", false, PrimDataType::Int); + fe.initCompiledKernel()->compileRtc( + kernel, "kernel1", false, PrimDataType::Int); LaunchParams lp( 1, // gdimx 1, // gdimy @@ -2102,7 +2104,8 @@ __global__ void kernel1( } } )"; - fe.compiledKernel()->compileRtc(kernel, "kernel1", false, PrimDataType::Int); + fe.initCompiledKernel()->compileRtc( + kernel, "kernel1", false, PrimDataType::Int); LaunchParams lp( 1, // gdimx 1, // gdimy @@ -2183,7 +2186,8 @@ __global__ void kernel1( } } )"; - fe.compiledKernel()->compileRtc(kernel, "kernel1", false, PrimDataType::Int); + fe.initCompiledKernel()->compileRtc( + kernel, "kernel1", false, PrimDataType::Int); LaunchParams lp( 1, // gdimx 1, // gdimy @@ -2258,7 +2262,8 @@ __global__ void kernel1( } } )"; - fe.compiledKernel()->compileRtc(kernel, "kernel1", false, PrimDataType::Int); + fe.initCompiledKernel()->compileRtc( + kernel, "kernel1", false, PrimDataType::Int); LaunchParams lp( x, // gdimx y, // gdimy diff --git a/tests/cpp/test_gpu3.cpp b/tests/cpp/test_gpu3.cpp index ca80f003902..053c3ca4842 100644 --- a/tests/cpp/test_gpu3.cpp +++ b/tests/cpp/test_gpu3.cpp @@ -7884,7 +7884,9 @@ TEST_F(NVFuserTest, AvoidCachingSliceInput) { const auto num_segments = kernel_runtime->fusionSegments()->groups().size(); NVF_CHECK(num_segments == 3, "Expect 3 segments, got: ", num_segments); for (const auto& fe : kernel_runtime->executors()) { - for (auto expr : fe.compiledKernel()->fusion()->exprs()) { + auto exprs = fe.compiledKernel() ? fe.compiledKernel()->fusion()->exprs() + : fe.fusion()->exprs(); + for (auto expr : exprs) { if (expr->isA()) { auto slice = expr->as(); NVF_CHECK( From acfe78a7ad36bf4cc54b8d08435c5224a7fa74ca Mon Sep 17 00:00:00 2001 From: Christian Sarofeen Date: Mon, 7 Oct 2024 10:42:08 -0700 Subject: [PATCH 10/11] Multi-gpu fixes. --- csrc/runtime/executor.cpp | 2 +- tests/cpp/test_multidevice_matmul.cpp | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/csrc/runtime/executor.cpp b/csrc/runtime/executor.cpp index cd4e6b4bba7..055f7071265 100644 --- a/csrc/runtime/executor.cpp +++ b/csrc/runtime/executor.cpp @@ -239,7 +239,7 @@ void FusionExecutor::compileFusion( // odd we need launch parameters for compilation, need to go back and check // why this is the case. compiledKernel_()->compileFusion( - compiledKernel_()->options().device, + device, launch_params.nThreads(), scheduler_type, fusion_id, diff --git a/tests/cpp/test_multidevice_matmul.cpp b/tests/cpp/test_multidevice_matmul.cpp index 222fc79d4c3..11041b38545 100644 --- a/tests/cpp/test_multidevice_matmul.cpp +++ b/tests/cpp/test_multidevice_matmul.cpp @@ -389,8 +389,8 @@ TEST_F(DistributedMatmulTest, AnnotateWeightOnly) { // x is of shape [2, 3] and replicated. // w is of shape [3, D*5] and column-wise sharded. // y is expected to have shape [2, D*5] and to be also column-wise sharded. - auto x_tensor = at::randn({2, 3}, tensor_options); - auto w_tensor = at::randn({mesh.size(), 3, 5}, tensor_options); + auto x_tensor = at::randn({200, 300}, tensor_options); + auto w_tensor = at::randn({mesh.size(), 300, 500}, tensor_options); auto sharded_w_tensor = shardTensor(w_tensor, w); FusionExecutorCache fec(std::move(fusion)); From ff57be354876fe24eed44bba502c3b7f45c02cbb Mon Sep 17 00:00:00 2001 From: Christian Sarofeen Date: Mon, 7 Oct 2024 12:27:59 -0700 Subject: [PATCH 11/11] More test fixes. --- tests/cpp/test_matmul.cpp | 140 +++++++++++++++++++------------------- tests/cpp/utils.h | 2 +- 2 files changed, 71 insertions(+), 71 deletions(-) diff --git a/tests/cpp/test_matmul.cpp b/tests/cpp/test_matmul.cpp index 8dbcc6cf0e0..04ff8557277 100644 --- a/tests/cpp/test_matmul.cpp +++ b/tests/cpp/test_matmul.cpp @@ -132,8 +132,8 @@ TEST_P(MatmulTestWithLayout, AmpereMatmul) { LaunchParams(), matmul_cparams)); ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); - ASSERT_FALSE( - PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse(fe.kernel())); + ASSERT_FALSE(PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse( + fe.compiledKernel()->kernel())); auto cg_outputs = fe.runFusion({inputs.first, inputs.second}); auto tref = atMatmul( inputs.first.to(at::kFloat), inputs.second.to(at::kFloat), layout); @@ -186,8 +186,8 @@ TEST_P(MatmulTestWithLayout, AmperePrologueFusionBroadcast) { LaunchParams(), matmul_cparams)); ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); - ASSERT_FALSE( - PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse(fe.kernel())); + ASSERT_FALSE(PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse( + fe.compiledKernel()->kernel())); auto cg_outputs = fe.runFusion({inputs.first, inputs.second}); auto tref = atMatmul( inputs.first.to(at::kFloat), inputs.second.to(at::kFloat), layout); @@ -245,8 +245,8 @@ TEST_P(MatmulTestWithLayout, AmpereProloguePointwise) { LaunchParams(), matmul_cparams)); ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); - ASSERT_FALSE( - PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse(fe.kernel())); + ASSERT_FALSE(PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse( + fe.compiledKernel()->kernel())); auto cg_outputs = fe.runFusion({inputs.first, inputs.second}); auto tref = atMatmul( inputs.first.sin().to(at::kFloat), @@ -304,8 +304,8 @@ TEST_P(MatmulTestWithLayout, AmpereMatmulBFloat16) { LaunchParams(), matmul_cparams)); ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); - ASSERT_FALSE( - PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse(fe.kernel())); + ASSERT_FALSE(PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse( + fe.compiledKernel()->kernel())); auto cg_outputs = fe.runFusion({inputs.first, inputs.second}); auto tref = atMatmul( inputs.first.to(at::kFloat), inputs.second.to(at::kFloat), layout); @@ -365,8 +365,8 @@ TEST_P(MatmulTestWithLayout, AmpereMatmulPipelineGmem) { LaunchParams(), matmul_cparams)); ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); - ASSERT_FALSE( - PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse(fe.kernel())); + ASSERT_FALSE(PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse( + fe.compiledKernel()->kernel())); auto cg_outputs = fe.runFusion({inputs.first, inputs.second}); auto tref = atMatmul( inputs.first.to(at::kFloat), inputs.second.to(at::kFloat), layout); @@ -447,8 +447,8 @@ TEST_P(MatmulTestWithLayout, AmpereSwizzle) { LaunchParams(), matmul_cparams)); ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); - ASSERT_FALSE( - PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse(fe.kernel())); + ASSERT_FALSE(PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse( + fe.compiledKernel()->kernel())); auto cg_outputs = fe.runFusion({inputs.first, inputs.second}); auto tref = atMatmul( inputs.first.to(at::kFloat), inputs.second.to(at::kFloat), layout); @@ -573,8 +573,8 @@ TEST_P(MatmulTestWithLayout, AmpereMatmulRegCircularBuffer) { LaunchParams(), matmul_cparams)); ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); - ASSERT_FALSE( - PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse(fe.kernel())); + ASSERT_FALSE(PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse( + fe.compiledKernel()->kernel())); auto cg_outputs = fe.runFusion({inputs.first, inputs.second}); auto tref = atMatmul( inputs.first.to(at::kFloat), inputs.second.to(at::kFloat), layout); @@ -863,8 +863,8 @@ TEST_F(MatmulTest, MatmulMatmulAmpere) { fe.compileFusion(&fusion, {t0, t1, t2}, LaunchParams(), matmul_cparams)); auto cg_outputs = fe.runFusion({t0, t1, t2}); - ASSERT_FALSE( - PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse(fe.kernel())); + ASSERT_FALSE(PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse( + fe.compiledKernel()->kernel())); // relaxed check for now, err accumulation is significant. NVF_CHECK(cg_outputs[0].allclose(tref, 0.1, 0.1)); } @@ -1243,8 +1243,8 @@ TEST_F(MatmulTest, MatmulSoftmaxMatmulAmpere) { fe.compileFusion(&fusion, {t0, t1, t2}, LaunchParams(), matmul_cparams)); auto cg_outputs = fe.runFusion({t0, t1, t2}); - ASSERT_FALSE( - PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse(fe.kernel())); + ASSERT_FALSE(PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse( + fe.compiledKernel()->kernel())); auto g1 = t0.to(at::kFloat).matmul(t1.t().to(at::kFloat)); auto sg1 = at::_softmax(g1, -1, false); auto gsg1 = sg1.matmul(t2.t().to(at::kFloat)); @@ -1292,8 +1292,8 @@ TEST_P(MatmulTestWithLayout, TuringMatmul) { NVFUSER_TEST_CUDA_ARCH_COMPILE_CHECK( 7, 5, fe.compileFusion(&fusion, {inputs.first, inputs.second})); ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); - ASSERT_FALSE( - PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse(fe.kernel())); + ASSERT_FALSE(PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse( + fe.compiledKernel()->kernel())); auto cg_outputs = fe.runFusion({inputs.first, inputs.second}); auto tref = atMatmul( inputs.first.to(at::kFloat), inputs.second.to(at::kFloat), layout); @@ -1439,8 +1439,8 @@ TEST_F(MatmulTest, AmpereMatmulTNCpAsync) { fe.compileFusion(&fusion, {t0, t1}, LaunchParams(), matmul_cparams)); auto cg_outputs = fe.runFusion({t0, t1}); - ASSERT_FALSE( - PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse(fe.kernel())); + ASSERT_FALSE(PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse( + fe.compiledKernel()->kernel())); auto tref = t0.to(at::kFloat).matmul(t1.t().to(at::kFloat)); NVF_CHECK(cg_outputs[0].allclose(tref, 0.0001, 0.0001)); @@ -1608,8 +1608,8 @@ TEST_F(MatmulTest, AmpereStridedBatchedMatmulTN) { fe.compileFusion(&fusion, {t0, t1}, LaunchParams(), matmul_cparams)); auto cg_outputs = fe.runFusion({t0, t1}); - ASSERT_FALSE( - PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse(fe.kernel())); + ASSERT_FALSE(PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse( + fe.compiledKernel()->kernel())); // ref implementation: auto ref_t0 = t0.permute({0, 2, 1, 3}) .contiguous() @@ -1781,8 +1781,8 @@ TEST_F(MatmulTest, AmpereViewMatmulTN) { fe.compileFusion(&fusion, {t0, t1}, LaunchParams(), matmul_cparams)); auto cg_outputs = fe.runFusion({t0, t1}); - ASSERT_FALSE( - PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse(fe.kernel())); + ASSERT_FALSE(PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse( + fe.compiledKernel()->kernel())); auto tref = at::native::view(t0, {M, K}).to(at::kFloat).matmul(t1.t().to(at::kFloat)); @@ -1964,8 +1964,8 @@ TEST_F(MatmulTest, AmpereMatmulTNSwizzled) { FusionExecutor fe; fe.compileFusion(&fusion, {t0, t1}, LaunchParams(), matmul_cparams); auto cg_outputs = fe.runFusion({t0, t1}); - ASSERT_FALSE( - PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse(fe.kernel())); + ASSERT_FALSE(PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse( + fe.compiledKernel()->kernel())); auto tref = t0.to(at::kFloat).matmul(t1.t().to(at::kFloat)); NVF_CHECK(cg_outputs[0].allclose(tref, 0.0001, 0.0001)); @@ -2020,8 +2020,8 @@ TEST_P(MatmulTestWithLayout, AmpereMatmulLargeLoad) { LaunchParams(), matmul_cparams)); ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); - ASSERT_FALSE( - PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse(fe.kernel())); + ASSERT_FALSE(PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse( + fe.compiledKernel()->kernel())); auto cg_outputs = fe.runFusion({inputs.first, inputs.second}); auto tref = atMatmul( inputs.first.to(at::kFloat), inputs.second.to(at::kFloat), layout); @@ -2074,8 +2074,8 @@ TEST_P(MatmulTestWithLayout, TuringMatmulLargeLoad) { LaunchParams(), matmul_cparams)); ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); - ASSERT_FALSE( - PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse(fe.kernel())); + ASSERT_FALSE(PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse( + fe.compiledKernel()->kernel())); auto cg_outputs = fe.runFusion({inputs.first, inputs.second}); auto tref = atMatmul( inputs.first.to(at::kFloat), inputs.second.to(at::kFloat), layout); @@ -2145,8 +2145,8 @@ TEST_P(MatmulTestWithLayout, AmpereMatmulTileCheck4warp) { matmul_cparams)); EXPECT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); auto cg_outputs = fe.runFusion({inputs.first, inputs.second}); - ASSERT_FALSE( - PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse(fe.kernel())); + ASSERT_FALSE(PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse( + fe.compiledKernel()->kernel())); auto tref = atMatmul( inputs.first.to(at::kFloat), inputs.second.to(at::kFloat), layout); NVF_CHECK( @@ -2223,8 +2223,8 @@ TEST_P(MatmulTestWithLayout, AmpereMatmulTileCheck8warp) { LaunchParams(), matmul_cparams)); ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); - ASSERT_FALSE( - PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse(fe.kernel())); + ASSERT_FALSE(PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse( + fe.compiledKernel()->kernel())); auto cg_outputs = fe.runFusion({inputs.first, inputs.second}); auto tref = atMatmul( inputs.first.to(at::kFloat), inputs.second.to(at::kFloat), layout); @@ -2292,8 +2292,8 @@ TEST_P(MatmulTestWithLayout, AmpereMatmulTileCheck6warp) { LaunchParams(), matmul_cparams)); ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); - ASSERT_FALSE( - PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse(fe.kernel())); + ASSERT_FALSE(PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse( + fe.compiledKernel()->kernel())); auto cg_outputs = fe.runFusion({inputs.first, inputs.second}); auto tref = atMatmul( inputs.first.to(at::kFloat), inputs.second.to(at::kFloat), layout); @@ -2350,8 +2350,8 @@ TEST_P(MatmulTestWithLayout, AmpereMatmulLargeLoadLargeK) { LaunchParams(), matmul_cparams)); ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); - ASSERT_FALSE( - PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse(fe.kernel())); + ASSERT_FALSE(PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse( + fe.compiledKernel()->kernel())); auto cg_outputs = fe.runFusion({inputs.first, inputs.second}); auto tref = atMatmul( inputs.first.to(at::kFloat), inputs.second.to(at::kFloat), layout); @@ -2402,8 +2402,8 @@ TEST_P(MatmulTestWithLayout, AmpereSplitKLikeStridedBatchedMatmul) { 0, fe.compileFusion(&fusion, {t0, t1}, LaunchParams(), matmul_cparams)); ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); - ASSERT_FALSE( - PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse(fe.kernel())); + ASSERT_FALSE(PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse( + fe.compiledKernel()->kernel())); auto cg_outputs = fe.runFusion({t0, t1}); auto tref = splitkLikeAtMatmul(t0.to(at::kFloat), t1.to(at::kFloat), layout); NVF_CHECK(cg_outputs[0].allclose(tref, 0.0001, 0.0001)); @@ -2500,8 +2500,8 @@ TEST_P(MatmulTestWithLayout, AmpereMatmulSmemEpilogue) { // check bank conflicts ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); - ASSERT_FALSE( - PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse(fe.kernel())); + ASSERT_FALSE(PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse( + fe.compiledKernel()->kernel())); // (0.001, 0.001) passed on local A100 but failed on CI A100 NVF_CHECK( cg_outputs[0].allclose(tref, 0.01, 0.01), @@ -2641,8 +2641,8 @@ TEST_F(MatmulTest, AmpereMatmulSmemEpiloguePromotionRequiredA100) { // check bank conflicts ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); - ASSERT_FALSE( - PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse(fe.kernel())); + ASSERT_FALSE(PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse( + fe.compiledKernel()->kernel())); // (0.001, 0.001) passed on local A100 but failed on CI A100 NVF_CHECK( cg_outputs[0].allclose(tref, 0.01, 0.01), @@ -2741,8 +2741,8 @@ TEST_P(MatmulTestWithLayout, AmpereMatmulSmemEpilogueCast) { tref = tref.to(at::kHalf); // check bank conflicts ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); - ASSERT_FALSE( - PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse(fe.kernel())); + ASSERT_FALSE(PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse( + fe.compiledKernel()->kernel())); // (0.001, 0.001) passed on local A100 but failed on CI A100 NVF_CHECK( cg_outputs[0].allclose(tref, 0.01, 0.01), @@ -2838,8 +2838,8 @@ TEST_P(MatmulTestWithLayout, AmpereMatmulSmemEpilogueRelu) { // check bank conflicts ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); - ASSERT_FALSE( - PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse(fe.kernel())); + ASSERT_FALSE(PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse( + fe.compiledKernel()->kernel())); // (0.001, 0.001) passed on local A100 but failed on CI A100 NVF_CHECK( cg_outputs[0].allclose(tref, 0.01, 0.01), @@ -2919,8 +2919,8 @@ TEST_P(MatmulTestWithLayout, FusionAmpereMatmulSplitK_CUDA) { 7, 5, fe.compileFusion(&fusion, {inputs.first, inputs.second})); EXPECT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); auto cg_outputs = fe.runFusion({inputs.first, inputs.second}); - ASSERT_FALSE( - PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse(fe.kernel())); + ASSERT_FALSE(PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse( + fe.compiledKernel()->kernel())); auto tref = atMatmul( inputs.first.to(at::kFloat), inputs.second.to(at::kFloat), layout); @@ -2987,8 +2987,8 @@ TEST_P(MatmulTestWithLayout, FusionAmpereMatmulSplitKBias_CUDA) { 7, 5, fe.compileFusion(&fusion, inputs)); EXPECT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); auto cg_outputs = fe.runFusion(inputs); - ASSERT_FALSE( - PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse(fe.kernel())); + ASSERT_FALSE(PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse( + fe.compiledKernel()->kernel())); auto tref = atBiasEpilogue( atMatmul(aten_a.to(at::kFloat), aten_b.to(at::kFloat), layout), aten_bias); @@ -3052,8 +3052,8 @@ TEST_P(MatmulTestWithLayout, AmpereMatmulBatchSplitK) { NVFUSER_TEST_CUDA_ARCH_COMPILE_CHECK( 7, 5, fe.compileFusion(&fusion, inputs)); ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); - ASSERT_FALSE( - PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse(fe.kernel())); + ASSERT_FALSE(PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse( + fe.compiledKernel()->kernel())); auto cg_outputs = fe.runFusion(inputs); auto tref = atMatmul(aten_a.to(at::kFloat), aten_b.to(at::kFloat), layout); @@ -3122,8 +3122,8 @@ TEST_P(MatmulTestWithLayout, AmpereMatmulBatchSplitKBias) { NVFUSER_TEST_CUDA_ARCH_COMPILE_CHECK( 7, 5, fe.compileFusion(&fusion, inputs)); ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); - ASSERT_FALSE( - PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse(fe.kernel())); + ASSERT_FALSE(PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse( + fe.compiledKernel()->kernel())); auto cg_outputs = fe.runFusion(inputs); auto tref = atBiasEpilogue( atMatmul(aten_a.to(at::kFloat), aten_b.to(at::kFloat), layout), @@ -3187,8 +3187,8 @@ TEST_F(MatmulTest, ReproIssue1808) { LaunchParams(), matmul_cparams)); ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); - ASSERT_FALSE( - PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse(fe.kernel())); + ASSERT_FALSE(PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse( + fe.compiledKernel()->kernel())); auto cg_outputs = fe.runFusion({inputs.first, inputs.second}); auto tref = atMatmul( inputs.first.to(at::kFloat), inputs.second.to(at::kFloat), layout); @@ -3339,8 +3339,8 @@ TEST_P(MatmulTestWithLayout, MisalignedVectorization) { fe.compileFusion( fusion.get(), inputs, LaunchParams(), matmul_cparams)); ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); - ASSERT_FALSE( - PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse(fe.kernel())); + ASSERT_FALSE(PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse( + fe.compiledKernel()->kernel())); auto outputs = fe.runFusion(inputs); EXPECT_TRUE(outputs[0].allclose(tref, 0.001, 0.001)); @@ -3396,8 +3396,8 @@ TEST_F(MatmulTest, MultipleConsecutiveDims) { NVFUSER_TEST_CUDA_ARCH_COMPILE_CHECK( 8, 0, fe.compileFusion(&fusion, inputs, LaunchParams(), matmul_cparams)); ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); - ASSERT_FALSE( - PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse(fe.kernel())); + ASSERT_FALSE(PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse( + fe.compiledKernel()->kernel())); auto cg_outputs = fe.runFusion(inputs); auto tref = at::reshape( at::linear( @@ -3462,8 +3462,8 @@ TEST_F(MatmulTest, DISABLED_MultipleNonConsecutiveMDims) { NVFUSER_TEST_CUDA_ARCH_COMPILE_CHECK( 8, 0, fe.compileFusion(&fusion, inputs, LaunchParams(), matmul_cparams)); ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); - ASSERT_FALSE( - PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse(fe.kernel())); + ASSERT_FALSE(PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse( + fe.compiledKernel()->kernel())); auto cg_outputs = fe.runFusion(inputs); auto Apermuted = A.permute({{1, 2}}).reshape({M1 * M2, K}); auto tref = at::linear(Apermuted.to(at::kFloat), B.to(at::kFloat)) @@ -3528,8 +3528,8 @@ TEST_F(MatmulTest, DISABLED_MultipleNonConsecutiveNDims) { NVFUSER_TEST_CUDA_ARCH_COMPILE_CHECK( 8, 0, fe.compileFusion(&fusion, inputs, LaunchParams(), matmul_cparams)); ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); - ASSERT_FALSE( - PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse(fe.kernel())); + ASSERT_FALSE(PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse( + fe.compiledKernel()->kernel())); auto cg_outputs = fe.runFusion(inputs); auto Bpermuted = B.permute({{1, 2}}).reshape({N1 * N2, K}); auto tref = at::linear(A.to(at::kFloat), Bpermuted.to(at::kFloat)) @@ -3586,8 +3586,8 @@ TEST_F(MatmulTest, MultipleMDimsBatch) { NVFUSER_TEST_CUDA_ARCH_COMPILE_CHECK( 8, 0, fe.compileFusion(&fusion, inputs, LaunchParams(), matmul_cparams)); ASSERT_TRUE(getBankConflictInfo(fe.compiledKernel()->kernel()).empty()); - ASSERT_FALSE( - PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse(fe.kernel())); + ASSERT_FALSE(PredicatedChecker::isCpAsyncMmaPredicatedByIfThenElse( + fe.compiledKernel()->kernel())); auto cg_outputs = fe.runFusion(inputs); auto tref = at::matmul(A.to(at::kFloat), at::permute(B.to(at::kFloat), {0, 2, 1})); diff --git a/tests/cpp/utils.h b/tests/cpp/utils.h index a8fcd5d4cd7..bac08ba8c81 100644 --- a/tests/cpp/utils.h +++ b/tests/cpp/utils.h @@ -820,7 +820,7 @@ bool isSchedulerInUse( const SchedulerType& scheduler_type); // Disable magic zero -constexpr CompileParams matmul_cparams{DataType::Int32, 255, false}; +const CompileParams matmul_cparams{DataType::Int32, 255, false}; // Utility to generate tensor with bias applied on the input tensor TensorView* biasEpilogue(TensorView* tensor, TensorView* bias);