merge open source device tracer with google one and move it to core/profiler/internal/gpu.

PiperOrigin-RevId: 268153734
This commit is contained in:
A. Unique TensorFlower 2019-09-09 22:37:51 -07:00 committed by TensorFlower Gardener
parent c9c46b589e
commit 191f36cb17
11 changed files with 466 additions and 777 deletions

View File

@ -9,7 +9,6 @@ load(
)
load(
"//tensorflow/core/platform:default/build_config.bzl",
"tf_additional_device_tracer_test_flags",
"tf_kernel_tests_linkstatic",
)
load(
@ -237,8 +236,7 @@ tf_cuda_cc_test(
srcs = [
"c_api_experimental_test.cc",
],
args =
["--heap_check=local"] + tf_additional_device_tracer_test_flags(),
args = ["--heap_check=local"],
extra_copts = tfe_xla_copts(),
linkstatic = tf_kernel_tests_linkstatic(),
tags = tf_cuda_tests_tags() + ["nomac"],

View File

@ -107,8 +107,6 @@ load(
"//tensorflow/core/platform:default/build_config.bzl",
"tf_additional_all_protos",
"tf_additional_core_deps",
"tf_additional_device_tracer_cuda_deps",
"tf_additional_device_tracer_test_flags",
"tf_additional_human_readable_json_deps",
"tf_additional_lib_defines",
"tf_additional_lib_deps",
@ -3398,28 +3396,6 @@ cc_library(
alwayslink = 1,
)
tf_cuda_library(
name = "device_tracer",
srcs = [
"//tensorflow/core/platform:legacy_device_tracer_srcs",
],
copts = tf_copts(),
cuda_deps = tf_additional_device_tracer_cuda_deps(),
visibility = [
"//tensorflow:internal",
],
deps = [
":core_cpu_internal",
":lib",
":protos_all_cc",
"//tensorflow/core/profiler/internal:parse_annotation",
"//tensorflow/core/profiler/internal:profiler_interface",
"//tensorflow/core/profiler/lib:traceme",
"@com_google_absl//absl/flags:flag",
],
alwayslink = True,
)
tf_proto_library_cc(
name = "replay_log_proto",
srcs = ["protobuf/replay_log.proto"],
@ -5235,36 +5211,6 @@ tf_cc_test(
],
)
tf_cc_test_gpu(
name = "device_tracer_test",
size = "small",
srcs = ["//tensorflow/core/platform:device_tracer_test.cc"],
args =
["--heap_check=local"] + tf_additional_device_tracer_test_flags(),
linkstatic = tf_kernel_tests_linkstatic(),
tags = tf_cuda_tests_tags() + ["nomac"],
deps = [
":all_kernels",
":core_cpu",
":core_cpu_internal",
":device_tracer",
":direct_session",
":direct_session_internal",
":framework",
":framework_internal",
":gpu_runtime",
":lib",
":lib_internal",
":protos_all_cc",
":test",
":test_main",
":testlib",
"//tensorflow/cc:cc_ops",
"//tensorflow/core/kernels:ops_util",
"//tensorflow/core/profiler/internal:profiler_interface",
],
)
tf_cc_tests(
name = "common_runtime_input_colocation_exemption_registry_test",
size = "small",

View File

@ -10,7 +10,6 @@
load(
"//tensorflow/core/platform:default/build_config.bzl",
"tf_additional_device_tracer_srcs",
"tf_additional_lib_hdrs",
"tf_additional_lib_srcs",
"tf_additional_libdevice_srcs",
@ -474,8 +473,6 @@ filegroup(
"**/monitoring.cc",
"**/stream_executor.h",
"**/env_time.cc",
"**/device_tracer.cc",
"**/tpu_tracer.cc",
"**/logger.cc",
"**/logging.cc",
"**/human_readable_json.cc",
@ -573,8 +570,6 @@ filegroup(
"**/env_time.cc",
"**/monitoring.cc",
"**/cuda_libdevice_path.cc",
"**/device_tracer.cc",
"**/tpu_tracer.cc",
"**/logger.cc",
"**/logging.cc",
"**/human_readable_json.cc",
@ -600,12 +595,6 @@ filegroup(
visibility = ["//tensorflow/core:__pkg__"],
)
filegroup(
name = "legacy_device_tracer_srcs",
srcs = tf_additional_device_tracer_srcs(),
visibility = ["//tensorflow/core:__pkg__"],
)
filegroup(
name = "legacy_minimal_lib_srcs",
srcs = tf_additional_minimal_lib_srcs(),

View File

@ -573,24 +573,11 @@ def tf_protos_grappler():
)
def tf_additional_device_tracer_srcs():
return ["default/device_tracer.cc"]
return ["device_tracer.cc"]
def tf_additional_cupti_utils_cuda_deps():
return []
def tf_additional_device_tracer_cuda_deps():
return [
"//tensorflow/stream_executor/cuda:cupti_stub",
"@com_google_absl//absl/base",
"@com_google_absl//absl/strings",
"@com_google_absl//absl/strings:str_format",
"@com_google_absl//absl/container:node_hash_map",
"@com_google_absl//absl/container:flat_hash_map",
]
def tf_additional_device_tracer_test_flags():
return []
def tf_additional_cupti_test_flags():
return []

View File

@ -1,681 +0,0 @@
/* Copyright 2015 The TensorFlow 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.
==============================================================================*/
#if GOOGLE_CUDA
#include <stdlib.h>
#include <memory>
#include "absl/base/casts.h"
#include "absl/container/flat_hash_map.h"
#include "absl/container/node_hash_map.h"
#include "absl/strings/ascii.h"
#include "absl/strings/str_cat.h"
#include "absl/strings/str_format.h"
#include "third_party/gpus/cuda/extras/CUPTI/include/cupti.h"
#include "tensorflow/core/common_runtime/step_stats_collector.h"
#include "tensorflow/core/framework/step_stats.pb.h"
#include "tensorflow/core/lib/core/errors.h"
#include "tensorflow/core/lib/core/status.h"
#include "tensorflow/core/lib/hash/hash.h"
#include "tensorflow/core/lib/strings/strcat.h"
#include "tensorflow/core/lib/strings/stringprintf.h"
#include "tensorflow/core/platform/abi.h"
#include "tensorflow/core/platform/annotation.h"
#include "tensorflow/core/platform/env.h"
#include "tensorflow/core/platform/macros.h"
#include "tensorflow/core/platform/mem.h"
#include "tensorflow/core/platform/mutex.h"
#include "tensorflow/core/platform/tracing.h"
#include "tensorflow/core/profiler/internal/profiler_interface.h"
#include "tensorflow/core/util/env_var.h"
namespace tensorflow {
namespace {
Status ToStatus(CUptiResult result) {
if (result == CUPTI_SUCCESS) {
return Status::OK();
}
const char* str = nullptr;
cuptiGetResultString(result, &str);
return errors::Unavailable("CUPTI error: ", str ? str : "<unknown>");
}
Status ToStatus(CUresult result) {
if (result == CUDA_SUCCESS) {
return Status::OK();
}
const char* str = nullptr;
cuGetErrorName(result, &str);
return errors::Unavailable("CUDA error: ", str ? str : "<unknown>");
}
void LogIfError(const Status& status) {
if (status.ok()) {
return;
}
LOG(ERROR) << status.error_message();
}
bool IsAscii(string& str) {
for (auto& ch : str) {
if (!absl::ascii_isascii(ch)) {
return false;
}
}
return true;
}
struct KernelRecord {
const char* kernel_name;
// TODO(csigg): cuStreamGetCtx introduced in CUDA 9.2 would allow us to only
// record the stream and infer the context during collection.
CUcontext context;
CUstream stream;
CUevent start_event;
CUevent stop_event;
const std::string* annotation;
};
struct MemcpyRecord {
CUmemorytype src_type;
CUmemorytype dst_type;
size_t size_bytes;
CUcontext context;
CUstream stream;
CUevent start_event;
CUevent stop_event;
const std::string* annotation;
};
Status CreateAndRecordEvent(CUevent* event, CUstream stream) {
TF_RETURN_IF_ERROR(ToStatus(cuEventCreate(event, CU_EVENT_DEFAULT)));
return ToStatus(cuEventRecord(*event, stream));
}
// Stores a series of kernel and memcpy records.
class CudaEventRecorder {
public:
// Registers the start of a kernel launch. The returned index should be passed
// to StopKernel() after the kernel launch has completed.
size_t StartKernel(const char* kernel_name, CUcontext context,
CUstream stream) {
KernelRecord record = {kernel_name, context, stream};
LogIfError(CreateAndRecordEvent(&record.start_event, stream));
mutex_lock lock(mutex_);
if (tracing::ScopedAnnotation::IsEnabled()) {
record.annotation =
&*annotations_.emplace(Annotation::CurrentAnnotation()).first;
}
kernel_records_.push_back(record);
return kernel_records_.size() - 1;
}
void StopKernel(size_t index) {
mutex_lock lock(mutex_);
auto& record = kernel_records_[index];
LogIfError(CreateAndRecordEvent(&record.stop_event, record.stream));
}
// Registers the start of a copy operation. The returned index should be
// passed to StopMemcpy() after the kernel launch has completed.
size_t StartMemcpy(CUmemorytype src_type, CUmemorytype dst_type,
size_t size_bytes, CUcontext context, CUstream stream) {
MemcpyRecord record = {src_type, dst_type, size_bytes, context, stream};
LogIfError(CreateAndRecordEvent(&record.start_event, stream));
mutex_lock lock(mutex_);
if (tracing::ScopedAnnotation::IsEnabled()) {
record.annotation =
&*annotations_.emplace(Annotation::CurrentAnnotation()).first;
}
memcpy_records_.push_back(record);
return memcpy_records_.size() - 1;
}
void StopMemcpy(size_t index) {
mutex_lock lock(mutex_);
auto& record = memcpy_records_[index];
LogIfError(CreateAndRecordEvent(&record.stop_event, record.stream));
}
std::vector<KernelRecord> ConsumeKernelRecords() {
mutex_lock lock(mutex_);
return std::move(kernel_records_);
}
std::vector<MemcpyRecord> ConsumeMemcpyRecords() {
mutex_lock lock(mutex_);
return std::move(memcpy_records_);
}
private:
mutex mutex_;
std::unordered_set<std::string> annotations_ GUARDED_BY(mutex_);
std::vector<KernelRecord> kernel_records_ GUARDED_BY(mutex_);
std::vector<MemcpyRecord> memcpy_records_ GUARDED_BY(mutex_);
};
// Instances register callbacks with CUPTI to notify the event recorder before
// and after kernel launches and memory copies.
class CuptiCallbackHook {
public:
CuptiCallbackHook() : subscriber_(nullptr) {}
Status Enable(CudaEventRecorder* recorder) {
TF_RETURN_IF_ERROR(
ToStatus(cuptiSubscribe(&subscriber_, &CuptiCallback, recorder)));
for (auto cbid : {CUPTI_DRIVER_TRACE_CBID_cuLaunchKernel,
CUPTI_DRIVER_TRACE_CBID_cuMemcpy,
CUPTI_DRIVER_TRACE_CBID_cuMemcpyAsync,
CUPTI_DRIVER_TRACE_CBID_cuMemcpyHtoD_v2,
CUPTI_DRIVER_TRACE_CBID_cuMemcpyHtoDAsync_v2,
CUPTI_DRIVER_TRACE_CBID_cuMemcpyDtoH_v2,
CUPTI_DRIVER_TRACE_CBID_cuMemcpyDtoHAsync_v2,
CUPTI_DRIVER_TRACE_CBID_cuMemcpyDtoD_v2,
CUPTI_DRIVER_TRACE_CBID_cuMemcpyDtoDAsync_v2}) {
TF_RETURN_IF_ERROR(ToStatus(cuptiEnableCallback(
/*enable=*/1, subscriber_, CUPTI_CB_DOMAIN_DRIVER_API, cbid)));
}
return Status::OK();
}
~CuptiCallbackHook() { LogIfError(ToStatus(cuptiUnsubscribe(subscriber_))); }
private:
static void CUPTIAPI CuptiCallback(void* userdata,
CUpti_CallbackDomain domain,
CUpti_CallbackId cbid,
const void* cbdata) {
auto recorder = static_cast<CudaEventRecorder*>(userdata);
auto data = static_cast<const CUpti_CallbackData*>(cbdata);
DCHECK_EQ(domain, CUPTI_CB_DOMAIN_DRIVER_API);
if (data->callbackSite == CUPTI_API_ENTER) {
DriverApiEnterCallback(cbid, *data, recorder);
} else {
DriverApiExitCallback(cbid, *data, recorder);
}
}
static CUmemorytype GetMemoryType(CUdeviceptr ptr) {
CUmemorytype mem_type;
auto status =
cuPointerGetAttribute(&mem_type, CU_POINTER_ATTRIBUTE_MEMORY_TYPE, ptr);
if (status == CUDA_ERROR_INVALID_VALUE) {
// Pointer not registered with CUDA, must be host memory.
return CU_MEMORYTYPE_HOST;
}
LogIfError(ToStatus(status));
return mem_type;
}
template <typename T>
static void StartMemcpy(CUmemorytype src_type, CUmemorytype dst_type,
const CUpti_CallbackData& cbdata,
CudaEventRecorder* recorder) {
auto params = static_cast<const T*>(cbdata.functionParams);
*cbdata.correlationData = recorder->StartMemcpy(
src_type, dst_type, params->ByteCount, cbdata.context, nullptr);
}
template <typename T>
static void StartMemcpyAsync(CUmemorytype src_type, CUmemorytype dst_type,
const CUpti_CallbackData& cbdata,
CudaEventRecorder* recorder) {
auto params = static_cast<const T*>(cbdata.functionParams);
*cbdata.correlationData = recorder->StartMemcpy(
src_type, dst_type, params->ByteCount, cbdata.context, params->hStream);
}
static void DriverApiEnterCallback(CUpti_CallbackId cbid,
const CUpti_CallbackData& cbdata,
CudaEventRecorder* recorder) {
switch (cbid) {
case CUPTI_DRIVER_TRACE_CBID_cuLaunchKernel: {
DCHECK_NE(cbdata.symbolName, nullptr);
auto params =
static_cast<const cuLaunchKernel_params*>(cbdata.functionParams);
*cbdata.correlationData = recorder->StartKernel(
cbdata.symbolName, cbdata.context, params->hStream);
return;
}
case CUPTI_DRIVER_TRACE_CBID_cuMemcpy: {
auto params =
static_cast<const cuMemcpy_params*>(cbdata.functionParams);
return StartMemcpy<cuMemcpy_params>(GetMemoryType(params->src),
GetMemoryType(params->dst), cbdata,
recorder);
}
case CUPTI_DRIVER_TRACE_CBID_cuMemcpyAsync: {
auto params =
static_cast<const cuMemcpyAsync_params*>(cbdata.functionParams);
return StartMemcpyAsync<cuMemcpyAsync_params>(
GetMemoryType(params->src), GetMemoryType(params->dst), cbdata,
recorder);
}
case CUPTI_DRIVER_TRACE_CBID_cuMemcpyHtoD_v2:
return StartMemcpy<cuMemcpyHtoD_v2_params>(
CU_MEMORYTYPE_HOST, CU_MEMORYTYPE_DEVICE, cbdata, recorder);
case CUPTI_DRIVER_TRACE_CBID_cuMemcpyHtoDAsync_v2:
return StartMemcpyAsync<cuMemcpyHtoDAsync_v2_params>(
CU_MEMORYTYPE_HOST, CU_MEMORYTYPE_DEVICE, cbdata, recorder);
case CUPTI_DRIVER_TRACE_CBID_cuMemcpyDtoH_v2:
return StartMemcpy<cuMemcpyDtoH_v2_params>(
CU_MEMORYTYPE_DEVICE, CU_MEMORYTYPE_HOST, cbdata, recorder);
case CUPTI_DRIVER_TRACE_CBID_cuMemcpyDtoHAsync_v2:
return StartMemcpyAsync<cuMemcpyDtoHAsync_v2_params>(
CU_MEMORYTYPE_DEVICE, CU_MEMORYTYPE_HOST, cbdata, recorder);
case CUPTI_DRIVER_TRACE_CBID_cuMemcpyDtoD_v2:
return StartMemcpy<cuMemcpyDtoD_v2_params>(
CU_MEMORYTYPE_DEVICE, CU_MEMORYTYPE_DEVICE, cbdata, recorder);
case CUPTI_DRIVER_TRACE_CBID_cuMemcpyDtoDAsync_v2:
return StartMemcpyAsync<cuMemcpyDtoDAsync_v2_params>(
CU_MEMORYTYPE_DEVICE, CU_MEMORYTYPE_DEVICE, cbdata, recorder);
default:
LOG(ERROR) << "Unexpected callback id: " << cbid;
}
}
static void DriverApiExitCallback(CUpti_CallbackId cbid,
const CUpti_CallbackData& cbdata,
CudaEventRecorder* recorder) {
switch (cbid) {
case CUPTI_DRIVER_TRACE_CBID_cuLaunchKernel:
recorder->StopKernel(*cbdata.correlationData);
break;
case CUPTI_DRIVER_TRACE_CBID_cuMemcpy:
case CUPTI_DRIVER_TRACE_CBID_cuMemcpyAsync:
case CUPTI_DRIVER_TRACE_CBID_cuMemcpyHtoD_v2:
case CUPTI_DRIVER_TRACE_CBID_cuMemcpyHtoDAsync_v2:
case CUPTI_DRIVER_TRACE_CBID_cuMemcpyDtoH_v2:
case CUPTI_DRIVER_TRACE_CBID_cuMemcpyDtoHAsync_v2:
case CUPTI_DRIVER_TRACE_CBID_cuMemcpyDtoD_v2:
case CUPTI_DRIVER_TRACE_CBID_cuMemcpyDtoDAsync_v2:
recorder->StopMemcpy(*cbdata.correlationData);
break;
default:
LOG(ERROR) << "Unexpected callback id: " << cbid;
}
}
CUpti_SubscriberHandle subscriber_;
};
// 'GpuTracer' is an interface for collecting low-level execution timings
// of hardware accelerator (e.g. GPU) computation and DMA transfers.
class GpuTracer : public profiler::ProfilerInterface {
public:
GpuTracer();
~GpuTracer() override;
// ProfilerInterface interface:
Status Start() override;
Status Stop() override;
// Collect trace results. Results are added to the specified
// StepStatsCollector. Does not clear any existing stats.
// It is an error to call 'Collect' while a trace is running.
Status CollectData(RunMetadata* run_metadata) override;
profiler::DeviceType GetDeviceType() override {
return profiler::DeviceType::kGpu;
}
private:
std::unique_ptr<CudaEventRecorder> recorder_;
std::unique_ptr<CuptiCallbackHook> cupti_hook_;
mutex mu_;
bool enabled_ GUARDED_BY(mu_);
};
GpuTracer::GpuTracer() : recorder_(new CudaEventRecorder()), enabled_(false) {
VLOG(1) << "GpuTracer created.";
}
GpuTracer::~GpuTracer() {
// Unregister the CUPTI callbacks if needed to prevent them from accessing
// freed memory.
Stop().IgnoreError();
}
Status GpuTracer::Start() {
VLOG(1) << "GpuTracer::Start";
mutex_lock l(mu_);
if (enabled_) {
return errors::FailedPrecondition("GpuTracer is already enabled.");
}
cupti_hook_.reset(new CuptiCallbackHook());
TF_RETURN_IF_ERROR(cupti_hook_->Enable(recorder_.get()));
tracing::ScopedAnnotation::Enable(true);
enabled_ = true;
return Status::OK();
}
Status GpuTracer::Stop() {
VLOG(1) << "GpuTracer::Stop";
mutex_lock l(mu_);
if (!enabled_) {
return Status::OK();
}
cupti_hook_.reset();
tracing::ScopedAnnotation::Enable(false);
enabled_ = false;
return Status::OK();
}
class CudaEventCollector {
struct DeviceInfo {
int ordinal;
std::string name;
int num_contexts;
};
struct ContextInfo {
int index;
const DeviceInfo* dev_info;
int num_streams;
CUevent end_event;
};
struct StreamInfo {
std::string name;
int index; // 0 is reserved for null stream.
const ContextInfo* ctx_info;
};
// Include context in key to distinguish null streams.
using StreamKey = std::pair<CUcontext, CUstream>;
CudaEventCollector(CudaEventRecorder* recorder, StepStatsCollector* collector)
: recorder_(recorder), collector_(collector) {
DCHECK(recorder != nullptr);
DCHECK(collector != nullptr);
}
// Populates device_infos_ from all devices.
Status InitializeDeviceInfos() {
int count;
TF_RETURN_IF_ERROR(ToStatus(cuDeviceGetCount(&count)));
for (int ordinal = 0; ordinal < count; ++ordinal) {
CUdevice device;
TF_RETURN_IF_ERROR(ToStatus(cuDeviceGet(&device, ordinal)));
char name[100];
TF_RETURN_IF_ERROR(ToStatus(cuDeviceGetName(name, sizeof(name), device)));
device_infos_[device] = {ordinal, name};
}
return Status::OK();
}
// Returns element from context_infos_, adding it if not yet present.
Status GetContextInfo(CUcontext context, ContextInfo** ctx_info_ptr) {
auto it = context_infos_.find(context);
if (it == context_infos_.end()) {
TF_RETURN_IF_ERROR(ToStatus(cuCtxSetCurrent(context)));
CUdevice device;
TF_RETURN_IF_ERROR(ToStatus(cuCtxGetDevice(&device)));
auto& dev_info = device_infos_[device];
ContextInfo ctx_info = {dev_info.num_contexts++, &dev_info};
it = context_infos_.emplace(context, ctx_info).first;
}
*ctx_info_ptr = &it->second;
return Status::OK();
}
// Adds element to stream_infos_ if not yet present. If present, clear name
// if it doesn't match parameter.
Status AddStreamInfo(CUcontext context, CUstream stream,
absl::string_view name) {
StreamKey key(context, stream);
auto it = stream_infos_.find(key);
if (it != stream_infos_.end()) {
if (it->second.name != name) {
it->second.name.clear(); // Stream with inconsistent names, clear it.
}
return Status::OK();
}
ContextInfo* ctx_info;
TF_RETURN_IF_ERROR(GetContextInfo(context, &ctx_info));
int index = stream ? ++ctx_info->num_streams : 0;
StreamInfo stream_info = {static_cast<std::string>(name), index, ctx_info};
stream_infos_.emplace(key, stream_info);
return Status::OK();
}
// Returns string describing source and destination memory types.
static std::string GetMemcpyName(const MemcpyRecord& record) {
auto get_memory_type = [](CUmemorytype mem_type) {
switch (mem_type) {
case CU_MEMORYTYPE_HOST:
return 'H';
case CU_MEMORYTYPE_DEVICE:
return 'D';
case CU_MEMORYTYPE_ARRAY:
return 'A';
case CU_MEMORYTYPE_UNIFIED:
return 'U';
default:
LOG(ERROR) << "Unknown memory type: " << mem_type;
return '?';
}
};
return absl::StrFormat("Memcpy%cto%c", get_memory_type(record.src_type),
get_memory_type(record.dst_type));
}
// Returns time in microseconds between events recorded on the GPU.
static uint64_t GetElapsedTimeUs(CUevent start, CUevent stop) {
float elapsed_ms = 0.0f;
LogIfError(ToStatus(cuEventElapsedTime(&elapsed_ms, start, stop)));
return static_cast<uint64>(
std::llroundf(1000 * std::max(elapsed_ms, 0.0f)));
}
// Synchronizes all contexts.
Status Synchronize() const {
for (const auto& pair : context_infos_) {
TF_RETURN_IF_ERROR(ToStatus(cuCtxSetCurrent(pair.first)));
TF_RETURN_IF_ERROR(ToStatus(cuCtxSynchronize()));
}
return Status::OK();
}
// Save stats to collector;
Status SaveStats(std::unique_ptr<NodeExecStats> stats,
const StreamInfo& stream_info) const {
auto ctx_info = stream_info.ctx_info;
auto dev_info = ctx_info->dev_info;
// TODO(csigg): tfprof_node.cc, run_metadata_test.py, and timeline_test.py
// currently require this particular formatting.
collector_->Save(
absl::StrFormat("/device:GPU:%d/stream:all", dev_info->ordinal),
new NodeExecStats(*stats));
auto name = absl::StrFormat("/gpu:%d (%s)/context#%d/", dev_info->ordinal,
dev_info->name, ctx_info->index);
if (stream_info.index) {
absl::StrAppend(&name, "stream#", std::to_string(stream_info.index));
} else {
absl::StrAppend(&name, "null stream");
}
if (!stream_info.name.empty()) {
absl::StrAppend(&name, ":", stream_info.name);
}
collector_->Save(name, stats.release());
return Status::OK();
}
Status SaveRecord(const KernelRecord& record) const {
if (!record.start_event || !record.stop_event) {
return Status::OK();
}
const auto& stream_info =
stream_infos_.at(StreamKey(record.context, record.stream));
auto start_us =
GetElapsedTimeUs(record.start_event, stream_info.ctx_info->end_event);
auto elapsed_us = GetElapsedTimeUs(record.start_event, record.stop_event);
auto stats = absl::make_unique<NodeExecStats>();
std::string node_name = port::MaybeAbiDemangle(record.kernel_name);
// Sometimes CUPTI returns invalid characters. See b/129892466.
if (!IsAscii(node_name)) {
node_name = "<invalid_name>";
}
if (record.annotation) {
node_name = absl::StrCat(*record.annotation, "@@", node_name);
}
stats->set_node_name(node_name);
// TODO(csigg): Report grid size?
std::string node_label;
stats->set_timeline_label(node_label);
stats->set_all_start_micros(end_walltime_us_ - start_us);
stats->set_op_end_rel_micros(elapsed_us);
stats->set_all_end_rel_micros(elapsed_us);
return SaveStats(std::move(stats), stream_info);
}
Status SaveRecord(const MemcpyRecord& record) const {
if (!record.start_event || !record.stop_event) {
return Status::OK();
}
const auto& stream_info =
stream_infos_.at(StreamKey(record.context, record.stream));
auto start_us =
GetElapsedTimeUs(record.start_event, stream_info.ctx_info->end_event);
auto elapsed_us = GetElapsedTimeUs(record.start_event, record.stop_event);
auto stats = absl::make_unique<NodeExecStats>();
std::string node_name = GetMemcpyName(record);
// Sometimes CUPTI returns invalid characters. See b/129892466.
if (!IsAscii(node_name)) {
node_name = "<invalid_name>";
}
if (record.annotation) {
node_name = absl::StrCat(*record.annotation, "@@", node_name);
}
stats->set_node_name(node_name);
// TODO(csigg): Show label in Chrome trace viewer.
std::string node_label = absl::StrFormat("%d bytes", record.size_bytes);
stats->set_timeline_label(node_label);
stats->set_all_start_micros(end_walltime_us_ - start_us);
stats->set_op_end_rel_micros(elapsed_us);
stats->set_all_end_rel_micros(elapsed_us);
return SaveStats(std::move(stats), stream_info);
}
Status Collect() {
TF_RETURN_IF_ERROR(InitializeDeviceInfos());
auto kernel_records = recorder_->ConsumeKernelRecords();
auto memcpy_records = recorder_->ConsumeMemcpyRecords();
LOG(INFO) << "Collecting " << kernel_records.size() << " kernel records, "
<< memcpy_records.size() << " memcpy records.";
// Gather all profiled streams and contexts.
for (const auto& record : kernel_records) {
TF_RETURN_IF_ERROR(
AddStreamInfo(record.context, record.stream, "Kernel"));
}
for (const auto& record : memcpy_records) {
TF_RETURN_IF_ERROR(
AddStreamInfo(record.context, record.stream, GetMemcpyName(record)));
}
// Synchronize all contexts, record end events, synchronize again.
TF_RETURN_IF_ERROR(Synchronize());
for (auto& pair : context_infos_) {
TF_RETURN_IF_ERROR(ToStatus(cuCtxSetCurrent(pair.first)));
TF_RETURN_IF_ERROR(CreateAndRecordEvent(&pair.second.end_event, nullptr));
}
TF_RETURN_IF_ERROR(Synchronize());
end_walltime_us_ = Env::Default()->NowMicros();
for (const auto& record : kernel_records) {
TF_RETURN_IF_ERROR(SaveRecord(record));
}
for (const auto& record : memcpy_records) {
TF_RETURN_IF_ERROR(SaveRecord(record));
}
return Status::OK();
}
public:
// Consumes the records in recorder and saves them to the collector.
static Status Collect(CudaEventRecorder* recorder,
StepStatsCollector* collector) {
CUcontext context;
TF_RETURN_IF_ERROR(ToStatus(cuCtxGetCurrent(&context)));
auto status = CudaEventCollector(recorder, collector).Collect();
TF_RETURN_IF_ERROR(ToStatus(cuCtxSetCurrent(context)));
return status;
}
private:
CudaEventRecorder* recorder_;
StepStatsCollector* collector_;
absl::node_hash_map<CUdevice, DeviceInfo> device_infos_;
absl::node_hash_map<CUcontext, ContextInfo> context_infos_;
absl::flat_hash_map<StreamKey, StreamInfo, hash<StreamKey>> stream_infos_;
int64 end_walltime_us_;
};
Status GpuTracer::CollectData(RunMetadata* run_metadata) {
mutex_lock l(mu_);
if (enabled_) {
return errors::FailedPrecondition("GpuTracer is still enabled.");
}
StepStatsCollector step_stats_collector(run_metadata->mutable_step_stats());
TF_RETURN_IF_ERROR(
CudaEventCollector::Collect(recorder_.get(), &step_stats_collector));
step_stats_collector.Finalize();
return Status::OK();
}
} // namespace
// Not in anonymous namespace for testing purposes.
std::unique_ptr<profiler::ProfilerInterface> CreateGpuTracer(
const profiler::ProfilerOptions& options) {
auto status = cuInit(0);
if (status != CUDA_SUCCESS) {
LogIfError(ToStatus(status));
return nullptr;
}
if (options.device_type != profiler::DeviceType::kGpu &&
options.device_type != profiler::DeviceType::kUnspecified)
return nullptr;
return absl::make_unique<GpuTracer>();
}
auto register_device_tracer_factory = [] {
bool enable;
TF_CHECK_OK(ReadBoolFromEnvVar("TF_ENABLE_OSS_GPU_PROFILER", true, &enable));
if (enable) {
RegisterProfilerFactory(&CreateGpuTracer);
}
return 0;
}();
} // namespace tensorflow
#endif // GOOGLE_CUDA

View File

@ -426,7 +426,6 @@ tf_cc_test(
deps = [
"//tensorflow/core:core_cpu",
"//tensorflow/core:core_cpu_internal",
"//tensorflow/core:device_tracer",
"//tensorflow/core:framework",
"//tensorflow/core:lib",
"//tensorflow/core:lib_internal",

View File

@ -1,6 +1,19 @@
load(
"//tensorflow:tensorflow.bzl",
"tf_copts",
"tf_cuda_library",
"if_cuda_is_configured_compat",
)
load("//tensorflow:tensorflow.bzl", "tf_cc_test_gpu")
load(
"//tensorflow/core/platform:default/build_config.bzl",
"tf_additional_cupti_utils_cuda_deps",
"tf_additional_device_tracer_srcs",
"tf_kernel_tests_linkstatic",
)
load(
"//tensorflow/core/platform:default/build_config_root.bzl",
"tf_cuda_tests_tags",
)
package(
@ -8,16 +21,54 @@ package(
licenses = ["notice"], # Apache 2.0
)
alias(
tf_cuda_library(
name = "device_tracer",
actual = "//tensorflow/core:device_tracer",
srcs = tf_additional_device_tracer_srcs(),
copts = tf_copts(),
cuda_deps = [
"//tensorflow/core/profiler/internal/gpu:cupti_tracer",
"//tensorflow/core/profiler/internal/gpu:cupti_wrapper",
],
deps = [
":cupti_utils",
"//tensorflow/core:core_cpu_internal",
"//tensorflow/core:lib",
"//tensorflow/core:protos_all_cc",
"//tensorflow/core/profiler/internal:parse_annotation",
"//tensorflow/core/profiler/internal:profiler_interface",
"//tensorflow/core/profiler/lib:traceme",
"@com_google_absl//absl/flags:flag",
],
alwayslink = 1,
)
load(
"//tensorflow:tensorflow.bzl",
"tf_copts",
"tf_cuda_library",
"if_cuda_is_configured_compat",
tf_cc_test_gpu(
name = "device_tracer_test",
size = "small",
srcs = ["device_tracer_test.cc"],
args = ["--heap_check=local"],
linkstatic = tf_kernel_tests_linkstatic(),
tags = tf_cuda_tests_tags() + ["nomac"],
deps = [
":device_tracer",
"//tensorflow/cc:cc_ops",
"//tensorflow/core:all_kernels",
"//tensorflow/core:core_cpu",
"//tensorflow/core:core_cpu_internal",
"//tensorflow/core:direct_session",
"//tensorflow/core:direct_session_internal",
"//tensorflow/core:framework",
"//tensorflow/core:framework_internal",
"//tensorflow/core:gpu_runtime",
"//tensorflow/core:lib",
"//tensorflow/core:lib_internal",
"//tensorflow/core:protos_all_cc",
"//tensorflow/core:test",
"//tensorflow/core:test_main",
"//tensorflow/core:testlib",
"//tensorflow/core/kernels:ops_util",
"//tensorflow/core/profiler/internal:profiler_interface",
],
)
tf_cuda_library(
@ -52,10 +103,12 @@ tf_cuda_library(
visibility = ["//visibility:public"],
deps = [
":cupti_interface",
":cupti_utils",
"//tensorflow/core:lib",
"//tensorflow/core/platform:annotation",
"@com_google_absl//absl/container:fixed_array",
"@com_google_absl//absl/container:flat_hash_map",
"@com_google_absl//absl/container:node_hash_map",
"@com_google_absl//absl/container:node_hash_set",
"@com_google_absl//absl/types:optional",
],

View File

@ -19,6 +19,7 @@ limitations under the License.
#include "absl/container/node_hash_map.h"
#include "tensorflow/core/lib/core/errors.h"
#include "tensorflow/core/lib/gtl/cleanup.h"
#include "tensorflow/core/lib/hash/hash.h"
#include "tensorflow/core/platform/abi.h"
#include "tensorflow/core/platform/annotation.h"
#include "tensorflow/core/platform/env.h"
@ -904,7 +905,7 @@ class CudaEventRecorder {
CuptiInterface *cupti_interface_;
CuptiTraceCollector *collector_;
const int ordinal_;
string device_name_;
std::string device_name_;
uint64 end_walltime_us_;
// Include context in key to distinguish null streams.
using StreamKey = std::pair<CUcontext, CUstream>;
@ -1172,7 +1173,7 @@ const char *GetTraceEventTypeName(const CuptiTracerEventType &type) {
}
void AnnotationMap::Add(uint32 device_id, uint32 correlation_id,
const string &annotation) {
const std::string &annotation) {
if (annotation.empty()) return;
VLOG(3) << "Add annotation: device_id: " << device_id
<< " correlation_id: " << correlation_id
@ -1386,7 +1387,7 @@ Status CuptiTracer::HandleCallback(CUpti_CallbackDomain domain,
device_id, domain, cbid, cbdata));
} else if (cbdata->callbackSite == CUPTI_API_EXIT) {
// Set up the map from correlation id to annotation string.
const string &annotation = tensorflow::Annotation::CurrentAnnotation();
const std::string &annotation = tensorflow::Annotation::CurrentAnnotation();
if (!annotation.empty()) {
annotation_map_->Add(device_id, cbdata->correlationId, annotation);
}

View File

@ -159,7 +159,8 @@ class CuptiTraceCollector {
virtual ~CuptiTraceCollector() {}
virtual void AddEvent(CuptiTracerEvent&& event) = 0;
virtual void OnEventsDropped(const string& reason, uint32 num_events) = 0;
virtual void OnEventsDropped(const std::string& reason,
uint32 num_events) = 0;
virtual void Flush() = 0;
protected:
@ -173,7 +174,8 @@ class AnnotationMap {
public:
explicit AnnotationMap(uint64 max_size, uint32 num_gpus)
: max_size_(max_size), per_device_map_(num_gpus) {}
void Add(uint32 device_id, uint32 correlation_id, const string& annotation);
void Add(uint32 device_id, uint32 correlation_id,
const std::string& annotation);
absl::string_view LookUp(uint32 device_id, uint32 correlation_id);
private:
@ -183,7 +185,7 @@ class AnnotationMap {
absl::Mutex mutex;
// Annotation tends to be repetitive, use a hash_set to store the strings,
// an use the reference to the string in the map.
absl::node_hash_set<string> annotations;
absl::node_hash_set<std::string> annotations;
absl::flat_hash_map<uint32, absl::string_view> correlation_map;
};
const uint64 max_size_;

View File

@ -0,0 +1,395 @@
/* Copyright 2019 The TensorFlow 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.
==============================================================================*/
#if GOOGLE_CUDA
#include <stdlib.h>
#include <memory>
#include "absl/container/fixed_array.h"
#include "absl/strings/str_cat.h"
#include "tensorflow/core/common_runtime/step_stats_collector.h"
#include "tensorflow/core/lib/core/errors.h"
#include "tensorflow/core/lib/strings/str_util.h"
#include "tensorflow/core/platform/annotation.h"
#include "tensorflow/core/platform/macros.h"
#include "tensorflow/core/platform/stringprintf.h"
#include "tensorflow/core/profiler/internal/gpu/cupti_tracer.h"
#include "tensorflow/core/profiler/internal/gpu/cupti_wrapper.h"
#include "tensorflow/core/profiler/internal/parse_annotation.h"
#include "tensorflow/core/profiler/internal/profiler_interface.h"
#include "tensorflow/core/util/env_var.h"
namespace tensorflow {
namespace profiler {
// Adapter from CuptiTraceCollector to StepStatsCollector: This class convert
// and filter from CuptiTracerEvent to tensorflow::NodeExecStats.
// We can not just forward event on the fly because StepStatsCollector have
// a single mutex for all devices, Therefore we will cache events and forward
// only when Flush().
class StepStatsCuptiTracerAdaptor : public CuptiTraceCollector {
public:
StepStatsCuptiTracerAdaptor(const CuptiTracerCollectorOptions& option,
const std::string prefix, int num_gpus,
uint64 start_walltime_ns, uint64 start_gpu_ns,
StepStatsCollector* trace_collector)
: CuptiTraceCollector(option),
trace_collector_(trace_collector),
num_callback_events_(0),
num_activity_events_(0),
start_walltime_ns_(start_walltime_ns),
start_gpu_ns_(start_gpu_ns),
num_gpus_(num_gpus),
per_device_adaptor_(num_gpus) {
for (int i = 0; i < num_gpus; ++i) { // for each device id.
per_device_adaptor_[i].stream_device =
strings::StrCat(prefix, "/device:GPU:", i, "/stream:");
per_device_adaptor_[i].memcpy_device =
strings::StrCat(prefix, "/device:GPU:", i, "/memcpy");
per_device_adaptor_[i].sync_device =
strings::StrCat(prefix, "/device:GPU:", i, "/sync");
}
}
void AddEvent(CuptiTracerEvent&& event) override {
if (event.device_id >= num_gpus_) return;
if (event.source == CuptiTracerEventSource::DriverCallback) {
if (num_callback_events_ > options_.max_callback_api_events) {
OnEventsDropped("trace collector", 1);
return;
}
num_callback_events_++;
} else {
if (num_activity_events_ > options_.max_activity_api_events) {
OnEventsDropped("trace collector", 1);
return;
}
num_activity_events_++;
}
per_device_adaptor_[event.device_id].AddEvent(std::move(event));
}
void OnEventsDropped(const string& reason, uint32 num_events) override {}
void Flush() override {
LOG(INFO) << " GpuTracer has collected " << num_callback_events_
<< " callback api events and " << num_activity_events_
<< " activity events.";
for (int i = 0; i < num_gpus_; ++i) {
per_device_adaptor_[i].Flush(trace_collector_, start_walltime_ns_,
start_gpu_ns_);
}
}
private:
StepStatsCollector* trace_collector_;
std::atomic<int> num_callback_events_;
std::atomic<int> num_activity_events_;
uint64 start_walltime_ns_;
uint64 start_gpu_ns_;
int num_gpus_;
struct CorrelationInfo {
CorrelationInfo(uint32 t, uint32 e) : thread_id(t), enqueue_time_ns(e) {}
uint32 thread_id;
uint64 enqueue_time_ns;
};
struct PerDeviceAdaptor {
void AddEvent(CuptiTracerEvent&& event) {
absl::MutexLock lock(&mutex);
if (event.source == CuptiTracerEventSource::DriverCallback) {
// Cupti api callcack events were used to populate launch times etc.
if (event.name == "cuStreamSynchronize") {
events.emplace_back(std::move(event));
}
if (event.correlation_id != CuptiTracerEvent::kInvalidCorrelationId) {
correlation_info.insert(
{event.correlation_id,
CorrelationInfo(event.thread_id, event.start_time_ns)});
}
} else {
// Cupti activity events measure device times etc.
events.emplace_back(std::move(event));
}
}
void Flush(StepStatsCollector* collector, uint64 start_walltime_ns,
uint64 start_gpu_ns) {
absl::MutexLock lock(&mutex);
for (auto& event : events) {
NodeExecStats* ns = new NodeExecStats;
ns->set_all_start_micros(
(start_walltime_ns + (event.start_time_ns - start_gpu_ns)) / 1000);
ns->set_op_start_rel_micros(0);
auto elapsed_ns = event.end_time_ns - event.start_time_ns;
ns->set_op_end_rel_micros(elapsed_ns / 1000);
ns->set_all_end_rel_micros(elapsed_ns / 1000);
if (event.source == CuptiTracerEventSource::DriverCallback) {
DCHECK_EQ(event.name, "cuStreamSynchronize");
ns->set_node_name(string(event.name));
ns->set_timeline_label(absl::StrCat("ThreadId ", event.thread_id));
ns->set_thread_id(event.thread_id);
collector->Save(sync_device, ns);
} else { // CuptiTracerEventSource::Activity
// Get launch information if available.
if (event.correlation_id != CuptiTracerEvent::kInvalidCorrelationId) {
auto it = correlation_info.find(event.correlation_id);
if (it != correlation_info.end()) {
ns->set_scheduled_micros(it->second.enqueue_time_ns / 1000);
ns->set_thread_id(it->second.thread_id);
}
}
auto annotation_stack = ParseAnnotationStack(event.annotation);
absl::string_view activity_name = !annotation_stack.empty()
? annotation_stack.back().name
: event.name;
ns->set_node_name(string(activity_name));
switch (event.type) {
case CuptiTracerEventType::Kernel: {
const string details = strings::Printf(
"regs:%llu shm:%llu grid:%llu,%llu,%llu block:%llu,%llu,%llu",
event.kernel_info.registers_per_thread,
event.kernel_info.static_shared_memory_usage,
event.kernel_info.grid_x, event.kernel_info.grid_y,
event.kernel_info.grid_z, event.kernel_info.block_x,
event.kernel_info.block_y, event.kernel_info.block_z);
ns->set_timeline_label(absl::StrCat(activity_name, " ", details));
auto nscopy = new NodeExecStats(*ns);
collector->Save(absl::StrCat(stream_device, "all"), ns);
collector->Save(absl::StrCat(stream_device, event.stream_id),
nscopy);
break;
}
case CuptiTracerEventType::MemcpyH2D:
case CuptiTracerEventType::MemcpyD2H:
case CuptiTracerEventType::MemcpyD2D:
case CuptiTracerEventType::MemcpyP2P: {
string details = absl::StrCat(
activity_name, " bytes:", event.memcpy_info.num_bytes);
if (event.memcpy_info.async) {
absl::StrAppend(&details, " aync");
}
if (event.memcpy_info.destination != event.device_id) {
absl::StrAppend(&details,
" to device:", event.memcpy_info.destination);
}
ns->set_timeline_label(std::move(details));
auto nscopy = new NodeExecStats(*ns);
collector->Save(memcpy_device, ns);
collector->Save(
absl::StrCat(stream_device, event.stream_id, "<",
GetTraceEventTypeName(event.type), ">"),
nscopy);
break;
}
default:
ns->set_timeline_label(string(activity_name));
collector->Save(stream_device, ns);
}
}
}
}
absl::Mutex mutex;
std::string stream_device GUARDED_BY(mutex);
std::string memcpy_device GUARDED_BY(mutex);
std::string sync_device GUARDED_BY(mutex);
std::vector<CuptiTracerEvent> events GUARDED_BY(mutex);
absl::flat_hash_map<uint32, CorrelationInfo> correlation_info
GUARDED_BY(mutex);
};
absl::FixedArray<PerDeviceAdaptor> per_device_adaptor_;
TF_DISALLOW_COPY_AND_ASSIGN(StepStatsCuptiTracerAdaptor);
};
// GpuTracer for GPU.
class GpuTracer : public profiler::ProfilerInterface {
public:
GpuTracer(CuptiTracer* cupti_tracer, CuptiInterface* cupti_interface)
: cupti_tracer_(cupti_tracer),
cupti_interface_(cupti_interface),
trace_collector_(&step_stats_) {
VLOG(1) << "GpuTracer created.";
}
~GpuTracer() override {}
// GpuTracer interface:
Status Start() override;
Status Stop() override;
Status CollectData(RunMetadata* run_metadata) override;
profiler::DeviceType GetDeviceType() override {
return profiler::DeviceType::kGpu;
}
private:
Status DoStart();
Status DoStop();
enum State {
kNotStarted,
kStartedOk,
kStartedError,
kStoppedOk,
kStoppedError
};
State profiling_state_ = State::kNotStarted;
CuptiTracer* cupti_tracer_;
CuptiTracerOptions options_;
CuptiInterface* cupti_interface_;
StepStats step_stats_;
StepStatsCollector trace_collector_;
std::unique_ptr<StepStatsCuptiTracerAdaptor> step_stats_cupti_adaptor_;
};
Status GpuTracer::DoStart() {
if (!cupti_tracer_->IsAvailable()) {
return errors::Unavailable("Another profile session running.");
}
options_.cbids_selected = {
// KERNEL
CUPTI_DRIVER_TRACE_CBID_cuLaunchKernel,
// MEMCPY
CUPTI_DRIVER_TRACE_CBID_cuMemcpy,
CUPTI_DRIVER_TRACE_CBID_cuMemcpyAsync,
CUPTI_DRIVER_TRACE_CBID_cuMemcpyHtoD_v2,
CUPTI_DRIVER_TRACE_CBID_cuMemcpyHtoDAsync_v2,
CUPTI_DRIVER_TRACE_CBID_cuMemcpyDtoH_v2,
CUPTI_DRIVER_TRACE_CBID_cuMemcpyDtoHAsync_v2,
CUPTI_DRIVER_TRACE_CBID_cuMemcpyDtoD_v2,
CUPTI_DRIVER_TRACE_CBID_cuMemcpyDtoDAsync_v2,
CUPTI_DRIVER_TRACE_CBID_cuMemcpyAtoH_v2,
CUPTI_DRIVER_TRACE_CBID_cuMemcpyAtoHAsync_v2,
CUPTI_DRIVER_TRACE_CBID_cuMemcpyAtoD_v2,
CUPTI_DRIVER_TRACE_CBID_cuMemcpyDtoA_v2,
CUPTI_DRIVER_TRACE_CBID_cuMemcpyAtoA_v2,
CUPTI_DRIVER_TRACE_CBID_cuMemcpy2D_v2,
CUPTI_DRIVER_TRACE_CBID_cuMemcpy2DUnaligned_v2,
CUPTI_DRIVER_TRACE_CBID_cuMemcpy2DAsync_v2,
CUPTI_DRIVER_TRACE_CBID_cuMemcpy3D_v2,
CUPTI_DRIVER_TRACE_CBID_cuMemcpy3DAsync_v2,
CUPTI_DRIVER_TRACE_CBID_cuMemcpyHtoA_v2,
CUPTI_DRIVER_TRACE_CBID_cuMemcpyHtoAAsync_v2,
// GENERIC
CUPTI_DRIVER_TRACE_CBID_cuStreamSynchronize,
};
bool trace_concurrent_kernels = false;
ReadBoolFromEnvVar("TF_GPU_CUPTI_FORCE_CONCURRENT_KERNEL", false,
&trace_concurrent_kernels)
.IgnoreError();
options_.activities_selected.push_back(
trace_concurrent_kernels ? CUPTI_ACTIVITY_KIND_CONCURRENT_KERNEL
: CUPTI_ACTIVITY_KIND_KERNEL);
options_.activities_selected.push_back(CUPTI_ACTIVITY_KIND_MEMCPY);
options_.activities_selected.push_back(CUPTI_ACTIVITY_KIND_MEMCPY2);
options_.activities_selected.push_back(CUPTI_ACTIVITY_KIND_OVERHEAD);
#if CUDA_VERSION < 10000
if (!trace_concurrent_kernels) options_.cupti_finalize = true;
#endif
CuptiTracerCollectorOptions collector_options;
uint64 start_gputime_ns = CuptiTracer::GetTimestamp();
uint64 start_walltime_ns = tensorflow::EnvTime::Default()->NowNanos();
int num_gpus = cupti_tracer_->NumGpus();
step_stats_cupti_adaptor_ = absl::make_unique<StepStatsCuptiTracerAdaptor>(
collector_options, "", num_gpus, start_walltime_ns, start_gputime_ns,
&trace_collector_);
tensorflow::tracing::ScopedAnnotation::Enable(true);
cupti_tracer_->Enable(options_, cupti_interface_,
step_stats_cupti_adaptor_.get());
return Status::OK();
}
Status GpuTracer::Start() {
Status status = DoStart();
if (status.ok()) {
profiling_state_ = State::kStartedOk;
return Status::OK();
} else {
profiling_state_ = State::kStartedError;
return status;
}
}
Status GpuTracer::DoStop() {
cupti_tracer_->Disable();
tensorflow::tracing::ScopedAnnotation::Enable(false);
return Status::OK();
}
Status GpuTracer::Stop() {
if (profiling_state_ == State::kStartedOk) {
Status status = DoStop();
profiling_state_ = status.ok() ? State::kStoppedOk : State::kStoppedError;
}
return Status::OK();
}
Status GpuTracer::CollectData(RunMetadata* run_metadata) {
switch (profiling_state_) {
case State::kNotStarted:
VLOG(1) << "No trace data collected, session wasn't started";
return Status::OK();
case State::kStartedOk:
return errors::FailedPrecondition("Cannot collect trace before stopping");
case State::kStartedError:
LOG(ERROR) << "Cannot collect, xprof failed to start";
return Status::OK();
case State::kStoppedError:
VLOG(1) << "No trace data collected";
return Status::OK();
case State::kStoppedOk: {
// Input run_metadata is shared by profiler interfaces, we need append.
trace_collector_.Finalize();
for (auto& dev_stats : *step_stats_.mutable_dev_stats()) {
run_metadata->mutable_step_stats()->add_dev_stats()->Swap(&dev_stats);
}
return Status::OK();
}
}
return errors::Internal("Invalid profiling state: ", profiling_state_);
}
} // namespace profiler
// Not in anonymous namespace for testing purposes.
std::unique_ptr<profiler::ProfilerInterface> CreateGpuTracer(
const profiler::ProfilerOptions& options) {
if (options.device_type != profiler::DeviceType::kGpu &&
options.device_type != profiler::DeviceType::kUnspecified)
return nullptr;
profiler::CuptiTracer* cupti_tracer =
profiler::CuptiTracer::GetCuptiTracerSingleton();
if (!cupti_tracer->IsAvailable()) {
return nullptr;
}
profiler::CuptiInterface* cupti_interface = profiler::GetCuptiInterface();
return absl::make_unique<profiler::GpuTracer>(cupti_tracer, cupti_interface);
}
auto register_gpu_tracer_factory = [] {
RegisterProfilerFactory(&CreateGpuTracer);
return 0;
}();
} // namespace tensorflow
#endif // GOOGLE_CUDA