未验证 提交 0790f949 编写于 作者: L liutiexing 提交者: GitHub

Add cuda tracer (#39488)

* add align for WorkQueue

* add spinlock

* merge develop

* merge

* Add EventsWaiter

* Revert "Add EventsWaiter"

This reverts commit e206173aa9be7401b83a53581627bfaf557c8fb2.

* add log for Executor

* Add CudaTracer to trace CUDA events
Co-authored-by: Nliutiexing <liutiexing@google.com>
上级 765a2ada
cc_library(host_tracer SRCS host_tracer.cc DEPS enforce)
cc_library(new_profiler SRCS profiler.cc DEPS host_tracer)
cc_library(cuda_tracer SRCS cuda_tracer.cc cupti_data_process.cc DEPS workqueue_utils enforce glog)
cc_library(new_profiler SRCS profiler.cc DEPS host_tracer cuda_tracer)
cc_library(event_node SRCS event_node.cc DEPS enforce)
cc_library(chrometracinglogger SRCS chrometracing_logger.cc DEPS event_node)
cc_test(test_event_node SRCS test_event_node.cc DEPS event_node chrometracinglogger)
......
// Copyright (c) 2022 PaddlePaddle Authors. All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#include "paddle/fluid/platform/profiler/cuda_tracer.h"
#include <string>
#include <unordered_map>
#include "glog/logging.h"
#include "paddle/fluid/framework/new_executor/workqueue/workqueue_utils.h"
#include "paddle/fluid/platform/os_info.h"
#include "paddle/fluid/platform/profiler/cupti_data_process.h"
#define CUPTI_CALL(call) \
do { \
CUptiResult _status = call; \
if (_status != CUPTI_SUCCESS) { \
const char* errstr; \
dynload::cuptiGetResultString(_status, &errstr); \
LOG(ERROR) << "Function " << #call << " failed with error " << errstr; \
exit(-1); \
} \
} while (0)
namespace paddle {
namespace platform {
namespace details {
std::unordered_map<uint32_t, uint64_t> CreateThreadIdMapping() {
std::unordered_map<uint32_t, uint64_t> mapping;
std::unordered_map<uint64_t, ThreadId> ids = GetAllThreadIds();
for (const auto& id : ids) {
mapping[id.second.cupti_tid] = id.second.sys_tid;
}
return mapping;
}
} // namespace details
CudaTracer::CudaTracer() {}
void CudaTracer::PrepareTracing() {
PADDLE_ENFORCE_EQ(
state_ == TracerState::UNINITED || state_ == TracerState::STOPED, true,
platform::errors::PreconditionNotMet("Tracer must be UNINITED"));
EnableCuptiActivity();
state_ = TracerState::READY;
}
void CudaTracer::StartTracing() {
PADDLE_ENFORCE_EQ(
state_ == TracerState::READY, true,
platform::errors::PreconditionNotMet("Tracer must be READY or STOPPED"));
ConsumeBuffers();
tracing_start_ns_ = PosixInNsec();
state_ = TracerState::STARTED;
}
void CudaTracer::StopTracing() {
PADDLE_ENFORCE_EQ(
state_, TracerState::STARTED,
platform::errors::PreconditionNotMet("Tracer must be STARTED"));
DisableCuptiActivity();
state_ = TracerState::STOPED;
}
void CudaTracer::CollectTraceData(TraceEventCollector* collector) {
PADDLE_ENFORCE_EQ(
state_, TracerState::STOPED,
platform::errors::PreconditionNotMet("Tracer must be STOPED"));
ProcessCuptiActivity(collector);
}
int CudaTracer::ProcessCuptiActivity(TraceEventCollector* collector) {
int record_cnt = 0;
#ifdef PADDLE_WITH_CUPTI
CUPTI_CALL(dynload::cuptiActivityFlushAll(CUPTI_ACTIVITY_FLAG_FLUSH_FORCED));
auto mapping = details::CreateThreadIdMapping();
std::vector<ActivityBuffer> buffers = ConsumeBuffers();
for (auto& buffer : buffers) {
if (buffer.addr == nullptr || buffer.valid_size == 0) {
continue;
}
CUpti_Activity* record = nullptr;
while (true) {
CUptiResult status = dynload::cuptiActivityGetNextRecord(
buffer.addr, buffer.valid_size, &record);
if (status == CUPTI_SUCCESS) {
details::ProcessCuptiActivityRecord(record, tracing_start_ns_, mapping,
collector);
++record_cnt;
} else if (status == CUPTI_ERROR_MAX_LIMIT_REACHED) {
break;
} else {
CUPTI_CALL(status);
}
}
ReleaseBuffer(buffer.addr);
}
#endif
return record_cnt;
}
void CudaTracer::EnableCuptiActivity() {
#ifdef PADDLE_WITH_CUPTI
CUPTI_CALL(dynload::cuptiActivityRegisterCallbacks(BufferRequestedCallback,
BufferCompletedCallback));
CUPTI_CALL(dynload::cuptiActivityEnable(CUPTI_ACTIVITY_KIND_MEMCPY));
CUPTI_CALL(
dynload::cuptiActivityEnable(CUPTI_ACTIVITY_KIND_CONCURRENT_KERNEL));
CUPTI_CALL(dynload::cuptiActivityEnable(CUPTI_ACTIVITY_KIND_DRIVER));
CUPTI_CALL(dynload::cuptiActivityEnable(CUPTI_ACTIVITY_KIND_RUNTIME));
CUPTI_CALL(dynload::cuptiActivityEnable(CUPTI_ACTIVITY_KIND_MEMSET));
VLOG(3) << "enable cupti activity";
#endif
}
void CudaTracer::DisableCuptiActivity() {
#ifdef PADDLE_WITH_CUPTI
CUPTI_CALL(dynload::cuptiActivityDisable(CUPTI_ACTIVITY_KIND_MEMCPY));
CUPTI_CALL(
dynload::cuptiActivityDisable(CUPTI_ACTIVITY_KIND_CONCURRENT_KERNEL));
CUPTI_CALL(dynload::cuptiActivityDisable(CUPTI_ACTIVITY_KIND_DRIVER));
CUPTI_CALL(dynload::cuptiActivityDisable(CUPTI_ACTIVITY_KIND_RUNTIME));
CUPTI_CALL(dynload::cuptiActivityDisable(CUPTI_ACTIVITY_KIND_MEMSET));
VLOG(3) << "disable cupti activity";
#endif
}
#ifdef PADDLE_WITH_CUPTI
void CUPTIAPI CudaTracer::BufferRequestedCallback(uint8_t** buffer,
size_t* size,
size_t* max_num_records) {
GetInstance().AllocateBuffer(buffer, size);
*max_num_records = 0;
}
void CUPTIAPI CudaTracer::BufferCompletedCallback(CUcontext ctx,
uint32_t stream_id,
uint8_t* buffer, size_t size,
size_t valid_size) {
GetInstance().ProduceBuffer(buffer, valid_size);
size_t dropped = 0;
CUPTI_CALL(
dynload::cuptiActivityGetNumDroppedRecords(ctx, stream_id, &dropped));
if (dropped != 0) {
LOG(WARNING) << "Stream " << stream_id << " Dropped " << dropped
<< " activity records";
}
}
#endif
void CudaTracer::AllocateBuffer(uint8_t** buffer, size_t* size) {
constexpr size_t kBufSize = 1 << 23; // 8 MB
constexpr size_t kBufAlign = 8; // 8 B
*buffer = reinterpret_cast<uint8_t*>(
paddle::framework::AlignedMalloc(kBufSize, kBufAlign));
*size = kBufSize;
}
void CudaTracer::ProduceBuffer(uint8_t* buffer, size_t valid_size) {
std::lock_guard<std::mutex> guard(activity_buffer_lock_);
activity_buffers_.emplace_back(buffer, valid_size);
}
std::vector<CudaTracer::ActivityBuffer> CudaTracer::ConsumeBuffers() {
std::vector<ActivityBuffer> buffers;
{
std::lock_guard<std::mutex> guard(activity_buffer_lock_);
buffers.swap(activity_buffers_);
}
return buffers;
}
void CudaTracer::ReleaseBuffer(uint8_t* buffer) {
paddle::framework::AlignedFree(buffer);
}
} // namespace platform
} // namespace paddle
// Copyright (c) 2022 PaddlePaddle Authors. All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#pragma once
#include <cstdint>
#include <mutex>
#include <vector>
#include "paddle/fluid/platform/dynload/cupti.h"
#include "paddle/fluid/platform/macros.h"
#include "paddle/fluid/platform/profiler/tracer_base.h"
namespace paddle {
namespace platform {
// Based on CUDA CUPTI
class CudaTracer : public TracerBase {
public:
// Singleton. CUPTI imposes this restriction.
static CudaTracer& GetInstance() {
static CudaTracer instance;
return instance;
}
void PrepareTracing() override;
void StartTracing() override;
void StopTracing() override;
void CollectTraceData(TraceEventCollector* collector) override;
private:
struct ActivityBuffer {
ActivityBuffer(uint8_t* addr, size_t size) : addr(addr), valid_size(size) {}
uint8_t* addr;
size_t valid_size;
};
CudaTracer();
DISABLE_COPY_AND_ASSIGN(CudaTracer);
void EnableCuptiActivity();
void DisableCuptiActivity();
int ProcessCuptiActivity(TraceEventCollector* collector);
#ifdef PADDLE_WITH_CUPTI
// Used by CUPTI Activity API to request buffer
static void CUPTIAPI BufferRequestedCallback(uint8_t** buffer, size_t* size,
size_t* max_num_records);
// Used by CUPTI Activity API to commit a completed buffer
static void CUPTIAPI BufferCompletedCallback(CUcontext ctx,
uint32_t stream_id,
uint8_t* buffer, size_t size,
size_t valid_size);
#endif
void AllocateBuffer(uint8_t** buffer, size_t* size);
void ProduceBuffer(uint8_t* buffer, size_t valid_size);
std::vector<ActivityBuffer> ConsumeBuffers();
void ReleaseBuffer(uint8_t* buffer);
uint64_t tracing_start_ns_ = UINT64_MAX;
std::mutex activity_buffer_lock_;
std::vector<ActivityBuffer> activity_buffers_;
};
} // namespace platform
} // namespace paddle
// Copyright (c) 2022 PaddlePaddle Authors. All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#include "paddle/fluid/platform/profiler/cupti_data_process.h"
#include <cstdio>
#include "paddle/fluid/platform/os_info.h"
namespace paddle {
namespace platform {
namespace details {
#ifdef PADDLE_WITH_CUPTI
void AddKernelRecord(const CUpti_ActivityKernel4* kernel, uint64_t start_ns,
TraceEventCollector* collector) {
if (kernel->start < start_ns) {
return;
}
DeviceTraceEvent event;
event.name = kernel->name;
event.type = TracerEventType::Kernel;
event.start_ns = kernel->start;
event.end_ns = kernel->end;
event.device_id = kernel->deviceId;
event.context_id = kernel->contextId;
event.stream_id = kernel->streamId;
event.correlation_id = kernel->correlationId;
event.kernel_info.block_x = kernel->blockX;
event.kernel_info.block_y = kernel->blockY;
event.kernel_info.block_z = kernel->blockZ;
event.kernel_info.grid_x = kernel->gridX;
event.kernel_info.grid_y = kernel->gridY;
event.kernel_info.grid_z = kernel->gridZ;
event.kernel_info.dynamic_shared_memory = kernel->dynamicSharedMemory;
event.kernel_info.static_shared_memory = kernel->staticSharedMemory;
event.kernel_info.registers_per_thread = kernel->registersPerThread;
event.kernel_info.local_memory_per_thread = kernel->localMemoryPerThread;
event.kernel_info.local_memory_total = kernel->localMemoryTotal;
event.kernel_info.queued = kernel->queued;
event.kernel_info.submitted = kernel->submitted;
event.kernel_info.completed = kernel->completed;
collector->AddDeviceEvent(std::move(event));
}
const char* MemcpyKind(uint8_t kind) {
switch (kind) {
case CUPTI_ACTIVITY_MEMCPY_KIND_HTOD:
return "MEMCPY_HtoD";
case CUPTI_ACTIVITY_MEMCPY_KIND_DTOH:
return "MEMCPY_DtoH";
case CUPTI_ACTIVITY_MEMCPY_KIND_HTOA:
return "MEMCPY_HtoA";
case CUPTI_ACTIVITY_MEMCPY_KIND_ATOH:
return "MEMCPY_AtoH";
case CUPTI_ACTIVITY_MEMCPY_KIND_ATOA:
return "MEMCPY_AtoA";
case CUPTI_ACTIVITY_MEMCPY_KIND_ATOD:
return "MEMCPY_AtoD";
case CUPTI_ACTIVITY_MEMCPY_KIND_DTOA:
return "MEMCPY_DtoA";
case CUPTI_ACTIVITY_MEMCPY_KIND_DTOD:
return "MEMCPY_DtoD";
case CUPTI_ACTIVITY_MEMCPY_KIND_HTOH:
return "MEMCPY_HtoH";
case CUPTI_ACTIVITY_MEMCPY_KIND_PTOP:
return "MEMCPY_PtoP";
default:
return "MEMCPY";
}
}
const char* MemoryKind(uint16_t kind) {
switch (kind) {
case CUPTI_ACTIVITY_MEMORY_KIND_UNKNOWN:
return "Unknown";
case CUPTI_ACTIVITY_MEMORY_KIND_PAGEABLE:
return "Pageable";
case CUPTI_ACTIVITY_MEMORY_KIND_PINNED:
return "Pinned";
case CUPTI_ACTIVITY_MEMORY_KIND_DEVICE:
return "Device";
case CUPTI_ACTIVITY_MEMORY_KIND_ARRAY:
return "Array";
case CUPTI_ACTIVITY_MEMORY_KIND_MANAGED:
return "Managed";
case CUPTI_ACTIVITY_MEMORY_KIND_DEVICE_STATIC:
return "Device Static";
case CUPTI_ACTIVITY_MEMORY_KIND_MANAGED_STATIC:
return "Managed Static";
default:
return "Unknown";
}
}
void AddMemcpyRecord(const CUpti_ActivityMemcpy* memcpy, uint64_t start_ns,
TraceEventCollector* collector) {
if (memcpy->start < start_ns) {
return;
}
DeviceTraceEvent event;
event.name = MemcpyKind(memcpy->copyKind);
event.type = TracerEventType::Memcpy;
event.start_ns = memcpy->start;
event.end_ns = memcpy->end;
event.device_id = memcpy->deviceId;
event.context_id = memcpy->contextId;
event.stream_id = memcpy->streamId;
event.correlation_id = memcpy->correlationId;
event.memcpy_info.num_bytes = memcpy->bytes;
// snprintf(event.memcpy_info.copy_kind, kMemKindMaxLen, "%s",
// MemcpyKind(memcpy->copyKind));
snprintf(event.memcpy_info.src_kind, kMemKindMaxLen, "%s",
MemcpyKind(memcpy->srcKind));
snprintf(event.memcpy_info.dst_kind, kMemKindMaxLen, "%s",
MemcpyKind(memcpy->dstKind));
collector->AddDeviceEvent(std::move(event));
}
void AddMemcpy2Record(const CUpti_ActivityMemcpy2* memcpy2, uint64_t start_ns,
TraceEventCollector* collector) {
if (memcpy2->start < start_ns) {
return;
}
DeviceTraceEvent event;
event.name = MemcpyKind(memcpy2->copyKind);
event.type = TracerEventType::Memcpy;
event.start_ns = memcpy2->start;
event.end_ns = memcpy2->end;
event.device_id = memcpy2->deviceId;
event.context_id = memcpy2->contextId;
event.stream_id = memcpy2->streamId;
event.correlation_id = memcpy2->correlationId;
event.memcpy_info.num_bytes = memcpy2->bytes;
// snprintf(event.memcpy_info.copy_kind, kMemKindMaxLen, "%s",
// MemcpyKind(memcpy2->copyKind));
snprintf(event.memcpy_info.src_kind, kMemKindMaxLen, "%s",
MemcpyKind(memcpy2->srcKind));
snprintf(event.memcpy_info.dst_kind, kMemKindMaxLen, "%s",
MemcpyKind(memcpy2->dstKind));
collector->AddDeviceEvent(std::move(event));
}
void AddMemsetRecord(const CUpti_ActivityMemset* memset, uint64_t start_ns,
TraceEventCollector* collector) {
if (memset->start < start_ns) {
return;
}
DeviceTraceEvent event;
event.name = "MEMSET";
event.type = TracerEventType::Memset;
event.start_ns = memset->start;
event.end_ns = memset->end;
event.device_id = memset->deviceId;
event.context_id = memset->contextId;
event.stream_id = memset->streamId;
event.correlation_id = memset->correlationId;
event.memset_info.num_bytes = memset->bytes;
snprintf(event.memset_info.memory_kind, kMemKindMaxLen, "%s",
MemoryKind(memset->memoryKind));
event.memset_info.value = memset->value;
collector->AddDeviceEvent(std::move(event));
}
class CuptiRuntimeCbidStr {
public:
static const CuptiRuntimeCbidStr& GetInstance() {
static CuptiRuntimeCbidStr inst;
return inst;
}
std::string RuntimeKind(CUpti_CallbackId cbid) const {
auto iter = cbid_str_.find(cbid);
if (iter == cbid_str_.end()) {
return "Runtime API " + std::to_string(cbid);
}
return iter->second;
}
private:
CuptiRuntimeCbidStr();
std::unordered_map<CUpti_CallbackId, std::string> cbid_str_;
};
CuptiRuntimeCbidStr::CuptiRuntimeCbidStr() {
#define REGISTER_RUNTIME_CBID_STR(cbid) \
cbid_str_[CUPTI_RUNTIME_TRACE_CBID_##cbid] = #cbid
REGISTER_RUNTIME_CBID_STR(cudaBindTexture_v3020);
REGISTER_RUNTIME_CBID_STR(cudaConfigureCall_v3020);
REGISTER_RUNTIME_CBID_STR(cudaDeviceGetAttribute_v5000);
REGISTER_RUNTIME_CBID_STR(cudaDeviceGetStreamPriorityRange_v5050);
REGISTER_RUNTIME_CBID_STR(cudaDeviceSynchronize_v3020);
REGISTER_RUNTIME_CBID_STR(cudaDriverGetVersion_v3020);
REGISTER_RUNTIME_CBID_STR(cudaEventCreateWithFlags_v3020);
REGISTER_RUNTIME_CBID_STR(cudaEventDestroy_v3020);
REGISTER_RUNTIME_CBID_STR(cudaEventDestroy_v3020);
REGISTER_RUNTIME_CBID_STR(cudaEventQuery_v3020);
REGISTER_RUNTIME_CBID_STR(cudaEventRecord_v3020);
REGISTER_RUNTIME_CBID_STR(cudaFreeHost_v3020);
REGISTER_RUNTIME_CBID_STR(cudaFree_v3020);
REGISTER_RUNTIME_CBID_STR(cudaFuncGetAttributes_v3020);
REGISTER_RUNTIME_CBID_STR(cudaGetDeviceCount_v3020);
REGISTER_RUNTIME_CBID_STR(cudaGetDeviceProperties_v3020);
REGISTER_RUNTIME_CBID_STR(cudaGetDevice_v3020);
REGISTER_RUNTIME_CBID_STR(cudaGetErrorString_v3020);
REGISTER_RUNTIME_CBID_STR(cudaGetLastError_v3020);
REGISTER_RUNTIME_CBID_STR(cudaHostAlloc_v3020);
REGISTER_RUNTIME_CBID_STR(cudaHostGetDevicePointer_v3020);
REGISTER_RUNTIME_CBID_STR(cudaLaunchKernel_v7000);
REGISTER_RUNTIME_CBID_STR(cudaMallocHost_v3020);
REGISTER_RUNTIME_CBID_STR(cudaMalloc_v3020);
REGISTER_RUNTIME_CBID_STR(cudaMemcpyAsync_v3020);
REGISTER_RUNTIME_CBID_STR(cudaMemcpy_v3020);
REGISTER_RUNTIME_CBID_STR(cudaMemsetAsync_v3020);
REGISTER_RUNTIME_CBID_STR(cudaMemset_v3020);
REGISTER_RUNTIME_CBID_STR(
cudaOccupancyMaxActiveBlocksPerMultiprocessorWithFlags_v7000);
REGISTER_RUNTIME_CBID_STR(cudaPeekAtLastError_v3020);
REGISTER_RUNTIME_CBID_STR(cudaRuntimeGetVersion_v3020);
REGISTER_RUNTIME_CBID_STR(cudaSetDevice_v3020);
REGISTER_RUNTIME_CBID_STR(cudaStreamCreate_v3020);
REGISTER_RUNTIME_CBID_STR(cudaStreamCreateWithFlags_v5000);
REGISTER_RUNTIME_CBID_STR(cudaStreamCreateWithPriority_v5050);
REGISTER_RUNTIME_CBID_STR(cudaStreamDestroy_v5050);
REGISTER_RUNTIME_CBID_STR(cudaStreamSynchronize_v3020);
REGISTER_RUNTIME_CBID_STR(cudaStreamWaitEvent_v3020);
REGISTER_RUNTIME_CBID_STR(cudaUnbindTexture_v3020);
REGISTER_RUNTIME_CBID_STR(cudaSetupArgument_v3020);
REGISTER_RUNTIME_CBID_STR(cudaLaunch_v3020);
REGISTER_RUNTIME_CBID_STR(cudaDeviceGetPCIBusId_v4010);
#if CUDA_VERSION >= 9000
REGISTER_RUNTIME_CBID_STR(cudaLaunchCooperativeKernel_v9000);
REGISTER_RUNTIME_CBID_STR(cudaLaunchCooperativeKernelMultiDevice_v9000);
#endif
#undef REGISTER_RUNTIME_CBID_STR
}
void AddApiRecord(const CUpti_ActivityAPI* api, uint64_t start_ns,
const std::unordered_map<uint32_t, uint64_t> tid_mapping,
TraceEventCollector* collector) {
if (api->start < start_ns) {
return;
}
RuntimeTraceEvent event;
event.name = CuptiRuntimeCbidStr::GetInstance().RuntimeKind(api->cbid);
event.start_ns = api->start;
event.end_ns = api->end;
event.process_id = GetProcessId();
uint64_t tid = 0;
auto iter = tid_mapping.find(api->threadId);
if (iter == tid_mapping.end()) {
} else {
tid = iter->second;
}
event.thread_id = tid;
event.correlation_id = api->correlationId;
event.callback_id = api->cbid;
collector->AddRuntimeEvent(std::move(event));
}
void ProcessCuptiActivityRecord(
const CUpti_Activity* record, uint64_t start_ns,
const std::unordered_map<uint32_t, uint64_t> tid_mapping,
TraceEventCollector* collector) {
switch (record->kind) {
case CUPTI_ACTIVITY_KIND_KERNEL:
case CUPTI_ACTIVITY_KIND_CONCURRENT_KERNEL:
AddKernelRecord(reinterpret_cast<const CUpti_ActivityKernel4*>(record),
start_ns, collector);
break;
case CUPTI_ACTIVITY_KIND_MEMCPY:
AddMemcpyRecord(reinterpret_cast<const CUpti_ActivityMemcpy*>(record),
start_ns, collector);
break;
case CUPTI_ACTIVITY_KIND_MEMCPY2:
AddMemcpy2Record(reinterpret_cast<const CUpti_ActivityMemcpy2*>(record),
start_ns, collector);
break;
case CUPTI_ACTIVITY_KIND_MEMSET:
AddMemsetRecord(reinterpret_cast<const CUpti_ActivityMemset*>(record),
start_ns, collector);
break;
case CUPTI_ACTIVITY_KIND_DRIVER:
case CUPTI_ACTIVITY_KIND_RUNTIME:
AddApiRecord(reinterpret_cast<const CUpti_ActivityAPI*>(record), start_ns,
tid_mapping, collector);
break;
default:
break;
}
}
#endif
} // namespace details
} // namespace platform
} // namespace paddle
// Copyright (c) 2022 PaddlePaddle Authors. All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#pragma once
#include <unordered_map>
#include "paddle/fluid/platform/dynload/cupti.h"
#include "paddle/fluid/platform/profiler/trace_event_collector.h"
namespace paddle {
namespace platform {
namespace details {
#ifdef PADDLE_WITH_CUPTI
void ProcessCuptiActivityRecord(
const CUpti_Activity* record, uint64_t start_ns,
const std::unordered_map<uint32_t, uint64_t> tid_mapping,
TraceEventCollector* collector);
#endif
} // namespace details
} // namespace platform
} // namespace paddle
/* Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved.
licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License. */
// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#pragma once
......
/* Copyright (c) 2022 PaddlePaddle Authors. All Rights Reserved.
licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License. */
// Copyright (c) 2022 PaddlePaddle Authors. All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#include "paddle/fluid/platform/profiler/host_tracer.h"
#include "glog/logging.h"
......
/* Copyright (c) 2022 PaddlePaddle Authors. All Rights Reserved.
licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License. */
// Copyright (c) 2022 PaddlePaddle Authors. All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#pragma once
......
/* Copyright (c) 2022 PaddlePaddle Authors. All Rights Reserved.
licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License. */
// Copyright (c) 2022 PaddlePaddle Authors. All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#include "paddle/fluid/platform/profiler/profiler.h"
#include "glog/logging.h"
......@@ -24,6 +24,7 @@ limitations under the License. */
#include "paddle/fluid/platform/device/gpu/gpu_info.h"
#endif
#include "paddle/fluid/platform/enforce.h"
#include "paddle/fluid/platform/profiler/cuda_tracer.h"
#include "paddle/fluid/platform/profiler/host_tracer.h"
#include "paddle/fluid/platform/profiler/trace_event_collector.h"
......@@ -46,6 +47,7 @@ Profiler::Profiler(const ProfilerOptions& options) {
HostTracerOptions host_tracer_options;
host_tracer_options.trace_level = options.trace_level;
tracers_.emplace_back(new HostTracer(host_tracer_options), true);
tracers_.emplace_back(&CudaTracer::GetInstance(), false);
}
Profiler::~Profiler() { alive_.store(false); }
......
/* Copyright (c) 2022 PaddlePaddle Authors. All Rights Reserved.
licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License. */
// Copyright (c) 2022 PaddlePaddle Authors. All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#pragma once
......
/* Copyright (c) 2022 PaddlePaddle Authors. All Rights Reserved.
licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License. */
// Copyright (c) 2022 PaddlePaddle Authors. All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#include <set>
#include <string>
......@@ -44,10 +44,44 @@ TEST(ProfilerTest, TestHostTracer) {
}
auto nodetree = profiler->Stop();
std::set<std::string> host_events;
for (const auto pair : nodetree->Traverse(true))
for (const auto pair : nodetree->Traverse(true)) {
for (const auto evt : pair.second) {
host_events.insert(evt->Name());
}
}
EXPECT_EQ(host_events.count("TestTraceLevel_record1"), 1u);
EXPECT_EQ(host_events.count("TestTraceLevel_record2"), 0u);
}
TEST(ProfilerTest, TestCudaTracer) {
using paddle::platform::ProfilerOptions;
using paddle::platform::Profiler;
ProfilerOptions options;
options.trace_level = 0;
auto profiler = Profiler::Create(options);
EXPECT_TRUE(profiler);
profiler->Prepare();
profiler->Start();
#ifdef PADDLE_WITH_CUDA
cudaStream_t stream;
cudaStreamCreate(&stream);
cudaStreamSynchronize(stream);
#endif
#ifdef PADDLE_WITH_HIP
hipStream_t stream;
hipStreamCreate(&stream);
hipStreamSynchronize(stream);
#endif
auto nodetree = profiler->Stop();
std::vector<std::string> runtime_events;
for (const auto pair : nodetree->Traverse(true)) {
for (const auto host_node : pair.second) {
for (auto runtime_node : host_node->GetRuntimeTraceEventNodes()) {
runtime_events.push_back(runtime_node->Name());
}
}
}
#ifdef PADDLE_WITH_CUPTI
EXPECT_GT(runtime_events.size(), 0u);
#endif
}
/* Copyright (c) 2022 PaddlePaddle Authors. All Rights Reserved.
licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License. */
// Copyright (c) 2022 PaddlePaddle Authors. All Rights Reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#pragma once
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册