Refactor to extract XPlane to tool converters.

PiperOrigin-RevId: 321865397
Change-Id: Ifa496f60df649e68a4bc647c1417a065cdcf141a
This commit is contained in:
A. Unique TensorFlower 2020-07-17 15:19:04 -07:00 committed by TensorFlower Gardener
parent ec25e31817
commit a44a11793c
17 changed files with 199 additions and 92 deletions

View File

@ -102,6 +102,7 @@ cc_library(
"//tensorflow/core/profiler/protobuf:steps_db_proto_cc",
"//tensorflow/core/profiler/protobuf:tf_function_proto_cc",
"//tensorflow/core/profiler/utils:diagnostics",
"//tensorflow/core/profiler/utils:hardware_type_utils",
"//tensorflow/core/profiler/utils:html_utils",
"//tensorflow/core/profiler/utils:math_utils",
"//tensorflow/core/profiler/utils:op_metrics_db_utils",
@ -128,6 +129,7 @@ cc_library(
"//tensorflow/core/profiler/protobuf:steps_db_proto_cc",
"//tensorflow/core/profiler/utils:diagnostics",
"//tensorflow/core/profiler/utils:event_span",
"//tensorflow/core/profiler/utils:hardware_type_utils",
"//tensorflow/core/profiler/utils:html_utils",
"//tensorflow/core/profiler/utils:math_utils",
"//tensorflow/core/profiler/utils:tf_op_utils",
@ -303,7 +305,6 @@ cc_library(
"//tensorflow/core/profiler/protobuf:xplane_proto_cc",
"//tensorflow/core/profiler/rpc/client:save_profile",
"//tensorflow/core/profiler/utils:xplane_schema",
"//tensorflow/core/profiler/utils:xplane_utils",
"@com_google_absl//absl/container:flat_hash_set",
"@com_google_absl//absl/strings",
],
@ -505,6 +506,7 @@ cc_library(
"//tensorflow/core/profiler/protobuf:xplane_proto_cc",
"//tensorflow/core/profiler/utils:tf_xplane_visitor",
"//tensorflow/core/profiler/utils:xplane_schema",
"//tensorflow/core/profiler/utils:xplane_utils",
"//tensorflow/core/profiler/utils:xplane_visitor",
"@com_google_absl//absl/algorithm:container",
"@com_google_absl//absl/container:flat_hash_map",

View File

@ -38,6 +38,7 @@ limitations under the License.
#include "tensorflow/core/profiler/protobuf/steps_db.pb.h"
#include "tensorflow/core/profiler/utils/diagnostics.h"
#include "tensorflow/core/profiler/utils/event_span.h"
#include "tensorflow/core/profiler/utils/hardware_type_utils.h"
#include "tensorflow/core/profiler/utils/html_utils.h"
#include "tensorflow/core/profiler/utils/math_utils.h"
#include "tensorflow/core/profiler/utils/tf_op_utils.h"
@ -553,12 +554,13 @@ StepSummary ComputeStepTimeSummaryInMs(
}
InputPipelineAnalysisResult ConvertOpStatsToInputPipelineAnalysis(
const OpStats& op_stats, const HardwareType& hardware_type) {
const OpStats& op_stats) {
InputPipelineAnalysisResult result =
ComputeGenericInputPipelineAnalysisResult(
op_stats.step_db().step_sequence());
PopulateStepDiagnostics(op_stats, result.mutable_diagnostics());
result.set_hardware_type(HardwareType_Name(hardware_type));
result.set_hardware_type(HardwareType_Name(
ParseHardwareType(op_stats.run_environment().device_type())));
GenerateHostResult(op_stats.host_op_metrics_db(), &result);
InputPipelineAnalysisRecommendation recommendation = GenerateRecommendation();

View File

@ -59,7 +59,7 @@ BottleneckAnalysis ComputeBottleneckAnalysis(
any_step_details);
InputPipelineAnalysisResult ConvertOpStatsToInputPipelineAnalysis(
const OpStats& op_stats, const HardwareType& hardware_type);
const OpStats& op_stats);
// Returns true if explanation for "All Others" time is also included in
// input_statement.

View File

@ -31,6 +31,7 @@ limitations under the License.
#include "tensorflow/core/profiler/protobuf/steps_db.pb.h"
#include "tensorflow/core/profiler/protobuf/tf_function.pb.h"
#include "tensorflow/core/profiler/utils/diagnostics.h"
#include "tensorflow/core/profiler/utils/hardware_type_utils.h"
#include "tensorflow/core/profiler/utils/html_utils.h"
#include "tensorflow/core/profiler/utils/math_utils.h"
#include "tensorflow/core/profiler/utils/op_metrics_db_utils.h"
@ -316,14 +317,13 @@ std::string EagerRecommendationHtml(double host_op_time_eager_percent,
return recommendation;
}
OverviewPage ConvertOpStatsToOverviewPage(const OpStats& op_stats,
HardwareType hardware_type) {
OverviewPage ConvertOpStatsToOverviewPage(const OpStats& op_stats) {
OverviewPage overview_page;
*overview_page.mutable_run_environment() =
ComputeRunEnvironment(op_stats.run_environment());
*overview_page.mutable_analysis() = ComputeAnalysisResult(op_stats);
*overview_page.mutable_input_analysis() =
ConvertOpStatsToInputPipelineAnalysis(op_stats, hardware_type);
ConvertOpStatsToInputPipelineAnalysis(op_stats);
BottleneckAnalysis bottleneck = ComputeBottleneckAnalysis(
overview_page.input_analysis().input_time_breakdown(),
overview_page.input_analysis().step_details());
@ -331,7 +331,8 @@ OverviewPage ConvertOpStatsToOverviewPage(const OpStats& op_stats,
bottleneck, op_stats.device_op_metrics_db().precision_stats());
SetCommonRecommendation(
bottleneck.input_classification(), bottleneck.input_statement(), "",
hardware_type, TfFunctionRecommendationHtml(op_stats.tf_function_db()),
ParseHardwareType(op_stats.run_environment().device_type()),
TfFunctionRecommendationHtml(op_stats.tf_function_db()),
EagerRecommendationHtml(
overview_page.analysis().host_op_time_eager_percent(),
overview_page.analysis().device_op_time_eager_percent()),

View File

@ -54,8 +54,7 @@ OverviewPageAnalysis ComputeAnalysisResult(const OpStats& op_stats);
OverviewPageRunEnvironment ComputeRunEnvironment(
const RunEnvironment& run_environment);
OverviewPage ConvertOpStatsToOverviewPage(const OpStats& op_stats,
HardwareType hardware_type);
OverviewPage ConvertOpStatsToOverviewPage(const OpStats& op_stats);
// Returns a html which provides tf-function related recommendation.
std::string TfFunctionRecommendationHtml(const TfFunctionDb& tf_function_db);

View File

@ -80,7 +80,7 @@ TEST(OpStatsToTfStats, GpuTfStats) {
kKernel3DurationNs, /*on_device=*/true, kKernel3,
&device_plane, &stream2);
const OpStats op_stats = ConvertXSpaceToOpStats(space);
const OpStats op_stats = ConvertXSpaceToOpStats(space, {OP_METRICS_DB});
const TfStatsDatabase tf_stats = ConvertOpStatsToTfStats(op_stats);
// TfOp1, TfOp2, Idle

View File

@ -35,6 +35,7 @@ limitations under the License.
#include "tensorflow/core/profiler/protobuf/xplane.pb.h"
#include "tensorflow/core/profiler/utils/tf_xplane_visitor.h"
#include "tensorflow/core/profiler/utils/xplane_schema.h"
#include "tensorflow/core/profiler/utils/xplane_utils.h"
#include "tensorflow/core/profiler/utils/xplane_visitor.h"
namespace tensorflow {
@ -481,6 +482,23 @@ void ProcessMemoryProfileProto(int64 max_num_snapshots,
}
}
template <typename Proto>
Status ConvertProtoToJson(const Proto& proto_output, std::string* json_output) {
protobuf::util::JsonPrintOptions json_options;
json_options.always_print_primitive_fields = true;
auto status = protobuf::util::MessageToJsonString(proto_output, json_output,
json_options);
if (!status.ok()) {
// Convert error_msg google::protobuf::StringPiece (or absl::string_view) to
// tensorflow::StringPiece.
auto error_msg = status.message();
return errors::Internal(
"Could not convert proto to JSON string: ",
absl::string_view(error_msg.data(), error_msg.length()));
}
return Status::OK();
}
} // namespace
MemoryProfile ConvertXPlaneToMemoryProfile(const XPlane& host_plane,
@ -490,5 +508,15 @@ MemoryProfile ConvertXPlaneToMemoryProfile(const XPlane& host_plane,
return memory_profile;
}
Status ConvertXSpaceToMemoryProfileJson(const XSpace& xspace,
std::string* json_output) {
if (const XPlane* host_plane =
FindPlaneWithName(xspace, kHostThreadsPlaneName)) {
MemoryProfile memory_profile = ConvertXPlaneToMemoryProfile(*host_plane);
TF_RETURN_IF_ERROR(ConvertProtoToJson(memory_profile, json_output));
}
return Status::OK();
}
} // namespace profiler
} // namespace tensorflow

View File

@ -16,6 +16,7 @@ limitations under the License.
#ifndef TENSORFLOW_CORE_PROFILER_CONVERT_XPLANE_TO_MEMORY_PROFILE_H_
#define TENSORFLOW_CORE_PROFILER_CONVERT_XPLANE_TO_MEMORY_PROFILE_H_
#include "tensorflow/core/platform/status.h"
#include "tensorflow/core/platform/types.h"
#include "tensorflow/core/profiler/protobuf/memory_profile.pb.h"
#include "tensorflow/core/profiler/protobuf/xplane.pb.h"
@ -29,6 +30,8 @@ namespace profiler {
MemoryProfile ConvertXPlaneToMemoryProfile(const XPlane& host_plane,
int64 max_num_snapshots = 1000);
Status ConvertXSpaceToMemoryProfileJson(const XSpace& xspace,
std::string* json_output);
} // namespace profiler
} // namespace tensorflow

View File

@ -106,7 +106,8 @@ void SetRunEnvironment(int32 accelerator_count, RunEnvironment* env) {
}
void ProcessHostPlane(const XPlane* host_plane, bool use_device_step_events,
OpMetricsDb* op_metrics_db, StepEvents* step_events) {
const OpStatsConfig& config, OpMetricsDb* op_metrics_db,
StepEvents* step_events) {
absl::flat_hash_map<int64, TfOp> tf_ops =
CollectTfOpsFromHostThreadsXPlane(*host_plane);
OpMetricsDbCombiner combiner(op_metrics_db);
@ -114,9 +115,11 @@ void ProcessHostPlane(const XPlane* host_plane, bool use_device_step_events,
plane.ForEachLine([&](const XLineVisitor& line) {
ConsumeTfMetricsDbData(
ConvertHostThreadsXLineToTfMetricsDbData(line, tf_ops), &combiner);
CombineStepEvents(ConvertHostThreadsXLineToStepEvents(
line, use_device_step_events, *step_events),
step_events);
if (config.contains(STEP_DB)) {
CombineStepEvents(ConvertHostThreadsXLineToStepEvents(
line, use_device_step_events, *step_events),
step_events);
}
});
}
@ -138,7 +141,8 @@ void PropagateXSpaceDiagnosticsToOpStats(const XSpace& space,
}
}
OpStats ConvertXSpaceToOpStats(const XSpace& space) {
OpStats ConvertXSpaceToOpStats(const XSpace& space,
const OpStatsConfig& config) {
const XPlane* host_plane = FindPlaneWithName(space, kHostThreadsPlaneName);
std::vector<const XPlane*> device_planes =
FindPlanesWithPrefix(space, kGpuPlanePrefix);
@ -152,34 +156,45 @@ OpStats ConvertXSpaceToOpStats(const XSpace& space) {
std::vector<KernelReport> reports;
for (const XPlane* device_trace : device_planes) {
if (!op_stats.has_perf_env()) {
*op_stats.mutable_perf_env() = GetPerfEnvFromXPlane(*device_trace);
if (config.contains(OP_METRICS_DB)) {
if (!op_stats.has_perf_env()) {
*op_stats.mutable_perf_env() = GetPerfEnvFromXPlane(*device_trace);
}
const PerfEnv& perf_env = op_stats.perf_env();
OpMetricsDb device_op_metrics_db = ConvertDeviceTraceXPlaneToOpMetricsDb(
*device_trace, perf_env.peak_tera_flops_per_second(),
perf_env.peak_hbm_bw_giga_bytes_per_second());
op_metrics_db_combiner.Combine(device_op_metrics_db);
}
if (config.contains(STEP_DB)) {
CombineStepEvents(ConvertDeviceTraceXPlaneToStepEvents(*device_trace),
&step_events);
}
if (config.contains(KERNEL_STATS_DB)) {
KernelStatsDb kernel_stats_db = ConvertDeviceTraceXPlaneToKernelStatsDb(
*device_trace, /*on_kernel_fn=*/{});
reports.insert(reports.begin(), kernel_stats_db.reports().begin(),
kernel_stats_db.reports().end());
}
const PerfEnv& perf_env = op_stats.perf_env();
OpMetricsDb device_op_metrics_db = ConvertDeviceTraceXPlaneToOpMetricsDb(
*device_trace, perf_env.peak_tera_flops_per_second(),
perf_env.peak_hbm_bw_giga_bytes_per_second());
op_metrics_db_combiner.Combine(device_op_metrics_db);
CombineStepEvents(ConvertDeviceTraceXPlaneToStepEvents(*device_trace),
&step_events);
KernelStatsDb kernel_stats_db = ConvertDeviceTraceXPlaneToKernelStatsDb(
*device_trace, /*on_kernel_fn=*/{});
reports.insert(reports.begin(), kernel_stats_db.reports().begin(),
kernel_stats_db.reports().end());
}
GroupKernelReports(&reports, op_stats.mutable_kernel_stats_db());
SortKernelsByTotalDurationDesc(op_stats.mutable_kernel_stats_db());
// Convert a host plane.
if (config.contains(KERNEL_STATS_DB)) {
GroupKernelReports(&reports, op_stats.mutable_kernel_stats_db());
SortKernelsByTotalDurationDesc(op_stats.mutable_kernel_stats_db());
}
bool has_device = !device_planes.empty();
if (host_plane) {
ProcessHostPlane(host_plane, has_device,
// Convert a host plane.
if (host_plane && config.contains(OP_METRICS_DB)) {
ProcessHostPlane(host_plane, has_device, config,
op_stats.mutable_host_op_metrics_db(), &step_events);
}
StepEvents nonoverlapped_step_events = ToNonOverlappedStepEvents(step_events);
*op_stats.mutable_step_db() =
ConvertStepEventsToStepDb(has_device, nonoverlapped_step_events);
*op_stats.mutable_device_op_metrics_db()->mutable_precision_stats() =
ComputePrecisionStats(nonoverlapped_step_events);
if (config.contains(STEP_DB)) {
StepEvents nonoverlapped_step_events =
ToNonOverlappedStepEvents(step_events);
*op_stats.mutable_step_db() =
ConvertStepEventsToStepDb(has_device, nonoverlapped_step_events);
*op_stats.mutable_device_op_metrics_db()->mutable_precision_stats() =
ComputePrecisionStats(nonoverlapped_step_events);
}
return op_stats;
}

View File

@ -16,14 +16,24 @@ limitations under the License.
#ifndef TENSORFLOW_CORE_PROFILER_CONVERT_XPLANE_TO_OP_STATS_H_
#define TENSORFLOW_CORE_PROFILER_CONVERT_XPLANE_TO_OP_STATS_H_
#include "absl/container/flat_hash_set.h"
#include "tensorflow/core/profiler/protobuf/op_stats.pb.h"
#include "tensorflow/core/profiler/protobuf/xplane.pb.h"
namespace tensorflow {
namespace profiler {
enum OpStatsKind {
OP_METRICS_DB,
STEP_DB,
KERNEL_STATS_DB,
};
using OpStatsConfig = absl::flat_hash_set<OpStatsKind>;
// NOTE: call GroupTfEvents before if OpStats.step_db needs to be generated.
OpStats ConvertXSpaceToOpStats(const XSpace& space);
OpStats ConvertXSpaceToOpStats(const XSpace& space,
const OpStatsConfig& config);
// Propagate and dedup the diagnostics in XSpace and add to OpStats.
void PropagateXSpaceDiagnosticsToOpStats(const XSpace& space,

View File

@ -61,8 +61,8 @@ TEST(ConvertXPlaneToOpStats, PerfEnv) {
*device_plane.GetOrCreateStatMetadata("compute_cap_minor"),
absl::StrCat(kComputeCapMinor));
GroupTfEvents(&space, /*event_group_name_map=*/nullptr);
OpStats op_stats = ConvertXSpaceToOpStats(space);
GroupTfEvents(&space, /*group_metadata_map=*/nullptr);
OpStats op_stats = ConvertXSpaceToOpStats(space, {OP_METRICS_DB});
const PerfEnv& perf_env = op_stats.perf_env();
EXPECT_NEAR(141, perf_env.peak_tera_flops_per_second(), kMaxError);
EXPECT_NEAR(900, perf_env.peak_hbm_bw_giga_bytes_per_second(), kMaxError);
@ -76,8 +76,8 @@ TEST(ConvertXPlaneToOpStats, RunEnvironment) {
XPlaneBuilder device_plane2(
GetOrCreateGpuXPlane(&space, /*device_ordinal=*/1));
GroupTfEvents(&space, /*event_group_name_map=*/nullptr);
OpStats op_stats = ConvertXSpaceToOpStats(space);
GroupTfEvents(&space, /*group_metadata_map=*/nullptr);
OpStats op_stats = ConvertXSpaceToOpStats(space, {});
const RunEnvironment& run_env = op_stats.run_environment();
EXPECT_EQ("GPU", run_env.device_type());
@ -106,8 +106,8 @@ TEST(ConvertXPlaneToOpStats, CpuOnlyStepDbTest) {
{{StatType::kStepId, kStepId}});
CreateXEvent(&host_plane_builder, &tf_executor_thread, "matmul", 30, 70);
GroupTfEvents(&space, /*event_group_name_map=*/nullptr);
OpStats op_stats = ConvertXSpaceToOpStats(space);
GroupTfEvents(&space, /*group_metadata_map=*/nullptr);
OpStats op_stats = ConvertXSpaceToOpStats(space, {OP_METRICS_DB, STEP_DB});
const StepDatabaseResult& step_db = op_stats.step_db();
EXPECT_EQ(step_db.step_sequence_size(), 1);
@ -143,8 +143,8 @@ TEST(ConvertXPlaneToOpStats, GpuStepDbTest) {
CreateXEvent(&device_plane_builder, &stream, "matmul", 50, 40,
{{StatType::kCorrelationId, kCorrelationId}});
GroupTfEvents(&space, /*event_group_name_map=*/nullptr);
OpStats op_stats = ConvertXSpaceToOpStats(space);
GroupTfEvents(&space, /*group_metadata_map=*/nullptr);
OpStats op_stats = ConvertXSpaceToOpStats(space, {OP_METRICS_DB, STEP_DB});
const StepDatabaseResult& step_db = op_stats.step_db();
EXPECT_EQ(step_db.step_sequence_size(), 1);
@ -161,7 +161,7 @@ TEST(ConvertXPlaneToOpStats, PropagateAndDedupErrors) {
*space.add_errors() = kError;
*space.add_errors() = kError;
OpStats op_stats = ConvertXSpaceToOpStats(space);
OpStats op_stats = ConvertXSpaceToOpStats(space, {});
EXPECT_EQ(1, op_stats.diagnostics().errors_size());
EXPECT_EQ(kError, op_stats.diagnostics().errors(/*index=*/0));

View File

@ -30,18 +30,15 @@ limitations under the License.
#include "tensorflow/core/profiler/convert/xplane_to_op_stats.h"
#include "tensorflow/core/profiler/convert/xplane_to_trace_events.h"
#include "tensorflow/core/profiler/profiler_service.pb.h"
#include "tensorflow/core/profiler/protobuf/hardware_types.pb.h"
#include "tensorflow/core/profiler/protobuf/input_pipeline.pb.h"
#include "tensorflow/core/profiler/protobuf/kernel_stats.pb.h"
#include "tensorflow/core/profiler/protobuf/memory_profile.pb.h"
#include "tensorflow/core/profiler/protobuf/op_stats.pb.h"
#include "tensorflow/core/profiler/protobuf/overview_page.pb.h"
#include "tensorflow/core/profiler/protobuf/tf_stats.pb.h"
#include "tensorflow/core/profiler/protobuf/trace_events.pb.h"
#include "tensorflow/core/profiler/protobuf/xplane.pb.h"
#include "tensorflow/core/profiler/rpc/client/save_profile.h"
#include "tensorflow/core/profiler/utils/xplane_schema.h"
#include "tensorflow/core/profiler/utils/xplane_utils.h"
namespace tensorflow {
namespace profiler {
@ -55,12 +52,6 @@ const absl::string_view kKernelStats = "kernel_stats";
const absl::string_view kMemoryProfile = "memory_profile";
const absl::string_view kXPlane = "xplane";
HardwareType HardwareTypeFromRunEnvironment(const RunEnvironment& run_env) {
if (run_env.device_type() == "GPU") return HardwareType::GPU;
if (run_env.device_type() == "CPU") return HardwareType::CPU_ONLY;
return HardwareType::UNKNOWN_HARDWARE;
}
template <typename Proto>
void AddToolData(absl::string_view tool_name, const Proto& tool_output,
ProfileResponse* response) {
@ -69,23 +60,6 @@ void AddToolData(absl::string_view tool_name, const Proto& tool_output,
tool_output.SerializeToString(tool_data->mutable_data());
}
template <typename Proto>
Status ConvertProtoToJson(const Proto& proto_output, std::string* json_output) {
protobuf::util::JsonPrintOptions json_options;
json_options.always_print_primitive_fields = true;
auto status = protobuf::util::MessageToJsonString(proto_output, json_output,
json_options);
if (!status.ok()) {
// Convert error_msg google::protobuf::StringPiece (or absl::string_view) to
// tensorflow::StringPiece.
auto error_msg = status.message();
return errors::Internal(
"Could not convert proto to JSON string: ",
absl::string_view(error_msg.data(), error_msg.length()));
}
return Status::OK();
}
// Returns the tool name with extension.
std::string ToolName(absl::string_view tool) {
if (tool == kTraceViewer) return "trace.json.gz";
@ -115,21 +89,18 @@ Status ConvertXSpaceToProfileResponse(const XSpace& xspace,
// Trace viewer is the only tool, skip OpStats conversion.
if (tools.size() == 1) return Status::OK();
}
OpStats op_stats = ConvertXSpaceToOpStats(xspace);
HardwareType hw_type =
HardwareTypeFromRunEnvironment(op_stats.run_environment());
OpStats op_stats =
ConvertXSpaceToOpStats(xspace, {OP_METRICS_DB, STEP_DB, KERNEL_STATS_DB});
if (tools.contains(kOverviewPage)) {
OverviewPage overview_page_db =
ConvertOpStatsToOverviewPage(op_stats, hw_type);
OverviewPage overview_page_db = ConvertOpStatsToOverviewPage(op_stats);
AddToolData(ToolName(kOverviewPage), overview_page_db, response);
if (tools.contains(kInputPipeline)) {
AddToolData(ToolName(kInputPipeline), overview_page_db.input_analysis(),
response);
}
} else if (tools.contains(kInputPipeline)) {
InputPipelineAnalysisResult input_pipeline_analysis =
ConvertOpStatsToInputPipelineAnalysis(op_stats, hw_type);
AddToolData(ToolName(kInputPipeline), input_pipeline_analysis, response);
AddToolData(ToolName(kInputPipeline),
ConvertOpStatsToInputPipelineAnalysis(op_stats), response);
}
if (tools.contains(kTensorflowStats)) {
TfStatsDatabase tf_stats_db = ConvertOpStatsToTfStats(op_stats);
@ -139,15 +110,11 @@ Status ConvertXSpaceToProfileResponse(const XSpace& xspace,
AddToolData(ToolName(kKernelStats), op_stats.kernel_stats_db(), response);
}
if (tools.contains(kMemoryProfile)) {
if (const XPlane* host_plane =
FindPlaneWithName(xspace, kHostThreadsPlaneName)) {
MemoryProfile memory_profile = ConvertXPlaneToMemoryProfile(*host_plane);
std::string json_output;
TF_RETURN_IF_ERROR(ConvertProtoToJson(memory_profile, &json_output));
TF_RETURN_IF_ERROR(SaveGzippedToolDataToTensorboardProfile(
req.repository_root(), req.session_id(), req.host_name(),
ToolName(kMemoryProfile), json_output));
}
std::string json_output;
TF_RETURN_IF_ERROR(ConvertXSpaceToMemoryProfileJson(xspace, &json_output));
TF_RETURN_IF_ERROR(SaveGzippedToolDataToTensorboardProfile(
req.repository_root(), req.session_id(), req.host_name(),
ToolName(kMemoryProfile), json_output));
}
return Status::OK();
}

View File

@ -47,6 +47,7 @@ cc_library(
deps = [
"//tensorflow/core:lib",
"//tensorflow/core/profiler/protobuf:hardware_types_proto_cc",
"@com_google_absl//absl/strings",
],
)

View File

@ -74,5 +74,13 @@ double GetFlopMaxThroughputPerSM(const DeviceCapabilities& device_cap) {
device_cap.clock_rate_in_ghz();
}
HardwareType ParseHardwareType(absl::string_view device_type) {
if (device_type == "GPU" || device_type == "Nvidia GPU")
return HardwareType::GPU;
if (device_type == "CPU") return HardwareType::CPU_ONLY;
if (device_type == "TPU") return HardwareType::TPU;
return HardwareType::UNKNOWN_HARDWARE;
}
} // namespace profiler
} // namespace tensorflow

View File

@ -16,6 +16,7 @@ limitations under the License.
#ifndef TENSORFLOW_CORE_PROFILER_UTILS_HARDWARE_TYPE_UTILS_H_
#define TENSORFLOW_CORE_PROFILER_UTILS_HARDWARE_TYPE_UTILS_H_
#include "absl/strings/string_view.h"
#include "tensorflow/core/profiler/protobuf/hardware_types.pb.h"
namespace tensorflow {
@ -25,6 +26,8 @@ namespace profiler {
// streaming multiprocessor.
double GetFlopMaxThroughputPerSM(const DeviceCapabilities& device_cap);
HardwareType ParseHardwareType(absl::string_view device_type);
} // namespace profiler
} // namespace tensorflow

View File

@ -119,6 +119,8 @@ tf_python_pybind_extension(
deps = [
"//tensorflow/core:lib",
"//tensorflow/core/profiler:profiler_service_proto_cc",
"//tensorflow/core/profiler/convert:op_stats_to_tf_stats",
"//tensorflow/core/profiler/convert:xplane_to_op_stats",
"//tensorflow/core/profiler/convert:xplane_to_profile_response",
"//tensorflow/core/profiler/convert:xplane_to_trace_events",
"//tensorflow/core/profiler/lib:profiler_session_headers",

View File

@ -20,9 +20,16 @@ limitations under the License.
#include "pybind11/pytypes.h"
#include "tensorflow/core/platform/host_info.h"
#include "tensorflow/core/platform/types.h"
#include "tensorflow/core/profiler/convert/op_stats_to_input_pipeline_analysis.h"
#include "tensorflow/core/profiler/convert/op_stats_to_overview_page.h"
#include "tensorflow/core/profiler/convert/op_stats_to_tf_stats.h"
#include "tensorflow/core/profiler/convert/xplane_to_memory_profile.h"
#include "tensorflow/core/profiler/convert/xplane_to_op_stats.h"
#include "tensorflow/core/profiler/convert/xplane_to_profile_response.h"
#include "tensorflow/core/profiler/convert/xplane_to_trace_events.h"
#include "tensorflow/core/profiler/lib/profiler_session.h"
#include "tensorflow/core/profiler/protobuf/input_pipeline.pb.h"
#include "tensorflow/core/profiler/protobuf/kernel_stats.pb.h"
#include "tensorflow/core/profiler/rpc/client/capture_profile.h"
#include "tensorflow/core/profiler/rpc/client/save_profile.h"
#include "tensorflow/core/profiler/rpc/profiler_server.h"
@ -32,6 +39,10 @@ namespace py = ::pybind11;
namespace {
using ::tensorflow::profiler::KERNEL_STATS_DB;
using ::tensorflow::profiler::OP_METRICS_DB;
using ::tensorflow::profiler::STEP_DB;
tensorflow::ProfileRequest MakeProfileRequest(
const tensorflow::string& logdir, const tensorflow::string& session_id,
const tensorflow::string& host) {
@ -162,4 +173,59 @@ PYBIND11_MODULE(_pywrap_profiler, m) {
tensorflow::MaybeRaiseRegisteredFromStatus(status);
return content;
});
m.def("xspace_to_trace_events", [](const py::bytes& serialized_xspace_proto) {
tensorflow::string content;
tensorflow::profiler::XSpace xspace;
xspace.ParseFromString(serialized_xspace_proto);
tensorflow::profiler::ConvertXSpaceToTraceEventsString(xspace, &content);
return py::bytes(content);
});
m.def("xspace_to_overview_page",
[](const py::bytes& serialized_xspace_proto) {
tensorflow::profiler::XSpace xspace;
xspace.ParseFromString(serialized_xspace_proto);
tensorflow::profiler::OverviewPage overview_page =
tensorflow::profiler::ConvertOpStatsToOverviewPage(
ConvertXSpaceToOpStats(xspace, {OP_METRICS_DB, STEP_DB}));
return py::bytes(overview_page.SerializeAsString());
});
m.def("xspace_to_input_pipeline",
[](const py::bytes& serialized_xspace_proto) {
tensorflow::profiler::XSpace xspace;
xspace.ParseFromString(serialized_xspace_proto);
tensorflow::profiler::InputPipelineAnalysisResult input_pipeline =
tensorflow::profiler::ConvertOpStatsToInputPipelineAnalysis(
ConvertXSpaceToOpStats(xspace, {OP_METRICS_DB, STEP_DB}));
return py::bytes(input_pipeline.SerializeAsString());
});
m.def("xspace_to_tf_stats", [](const py::bytes& serialized_xspace_proto) {
tensorflow::profiler::XSpace xspace;
xspace.ParseFromString(serialized_xspace_proto);
tensorflow::profiler::TfStatsDatabase tf_stats_db =
tensorflow::profiler::ConvertOpStatsToTfStats(
ConvertXSpaceToOpStats(xspace, {OP_METRICS_DB}));
return py::bytes(tf_stats_db.SerializeAsString());
});
m.def("xspace_to_kernel_stats", [](const py::bytes& serialized_xspace_proto) {
tensorflow::profiler::XSpace xspace;
xspace.ParseFromString(serialized_xspace_proto);
tensorflow::profiler::OpStats op_stats =
ConvertXSpaceToOpStats(xspace, {KERNEL_STATS_DB});
return py::bytes(op_stats.kernel_stats_db().SerializeAsString());
});
m.def("xspace_to_memory_profile",
[](const py::bytes& serialized_xspace_proto) {
tensorflow::profiler::XSpace xspace;
xspace.ParseFromString(serialized_xspace_proto);
std::string json_output;
tensorflow::profiler::ConvertXSpaceToMemoryProfileJson(xspace,
&json_output);
return py::bytes(json_output);
});
};