Refactor EagerProfiler: rename it to ProfileSession.

PiperOrigin-RevId: 231845855
This commit is contained in:
A. Unique TensorFlower 2019-01-31 13:44:12 -08:00 committed by TensorFlower Gardener
parent 2cd4c39ff3
commit 376351335d
16 changed files with 399 additions and 95 deletions

View File

@ -70,7 +70,7 @@ tf_cuda_library(
"//tensorflow/core/distributed_runtime:remote_device",
"//tensorflow/core/distributed_runtime:server_lib",
"//tensorflow/core/distributed_runtime:worker_env",
"//tensorflow/core/profiler/lib:eager_profiler",
"//tensorflow/core/profiler/lib:profiler_session",
"//tensorflow/core:gpu_runtime",
],
)
@ -110,7 +110,7 @@ tf_cuda_library(
"//tensorflow/core/distributed_runtime/rpc:grpc_worker_service",
"//tensorflow/core/distributed_runtime/rpc:rpc_rendezvous_mgr",
"//tensorflow/core/distributed_runtime/rpc/eager:grpc_eager_client",
"//tensorflow/core/profiler/lib:eager_profiler",
"//tensorflow/core/profiler/lib:profiler_session",
],
)

View File

@ -52,7 +52,7 @@ limitations under the License.
#include "tensorflow/core/lib/gtl/stl_util.h"
#include "tensorflow/core/platform/mutex.h"
#include "tensorflow/core/platform/thread_annotations.h"
#include "tensorflow/core/profiler/lib/eager_profiler.h"
#include "tensorflow/core/profiler/lib/profiler_session.h"
#include "tensorflow/core/public/version.h"
struct TFE_ContextOptions {
@ -109,9 +109,9 @@ struct TFE_Op {
struct TFE_Profiler {
TFE_Profiler(TFE_Context* ctx)
: profiler(tensorflow::EagerProfiler::Create(&ctx->context)) {}
: profiler(tensorflow::ProfilerSession::Create(&ctx->context)) {}
std::unique_ptr<tensorflow::EagerProfiler> profiler;
std::unique_ptr<tensorflow::ProfilerSession> profiler;
};
namespace tensorflow {

View File

@ -394,3 +394,14 @@ tf_cuda_cc_test(
"@com_google_googletest//:gtest_main",
],
)
tf_cuda_library(
name = "profiler_interface",
hdrs = [
"profiler_interface.h",
],
deps = [
"//tensorflow/core:lib",
"//tensorflow/core:protos_all_cc",
],
)

View File

@ -0,0 +1,26 @@
package(
default_visibility = ["//tensorflow:internal"],
)
licenses(["notice"]) # Apache 2.0
load(
"//tensorflow:tensorflow.bzl",
"tf_cuda_library",
)
tf_cuda_library(
name = "tracer",
srcs = [
"tracer.cc",
],
hdrs = [
"tracer.h",
],
deps = [
"//tensorflow/core:core_cpu_lib",
"//tensorflow/core:device_tracer",
"//tensorflow/core/profiler/internal:profiler_interface",
"//tensorflow/core/profiler/internal/runtime:eager_profiler",
],
)

View File

@ -0,0 +1,59 @@
/* Copyright 2016 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.
==============================================================================*/
#include "tensorflow/core/profiler/internal/gpu/tracer.h"
#include "tensorflow/core/common_runtime/step_stats_collector.h"
namespace tensorflow {
namespace profiler {
namespace gpu {
/* static */ std::unique_ptr<ProfilerInterface> Tracer::Create() {
return absl::WrapUnique(new Tracer());
}
Status Tracer::Start() {
device_tracer_ = CreateDeviceTracer();
if (!device_tracer_) {
return Status(tensorflow::error::Code::FAILED_PRECONDITION,
"Failed to create device tracer.");
}
return device_tracer_->Start();
}
Status Tracer::Stop() {
if (!device_tracer_) {
return Status(tensorflow::error::Code::FAILED_PRECONDITION,
"No running device tracer.");
}
return device_tracer_->Stop();
}
Status Tracer::CollectData(RunMetadata* run_metadata) {
if (!device_tracer_) {
return Status(tensorflow::error::Code::FAILED_PRECONDITION,
"No running device tracer.");
}
auto step_stats_collector =
absl::make_unique<StepStatsCollector>(run_metadata->mutable_step_stats());
Status s = device_tracer_->Collect(step_stats_collector.get());
step_stats_collector->Finalize();
return s;
}
Tracer::Tracer() {}
} // namespace gpu
} // namespace profiler
} // namespace tensorflow

View File

@ -0,0 +1,48 @@
/* Copyright 2016 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.
==============================================================================*/
#ifndef TENSORFLOW_CORE_PROFILER_INTERNAL_GPU_TRACER_H_
#define TENSORFLOW_CORE_PROFILER_INTERNAL_GPU_TRACER_H_
#include "tensorflow/core/platform/device_tracer.h"
#include "tensorflow/core/profiler/internal/profiler_interface.h"
namespace tensorflow {
namespace profiler {
namespace gpu {
class Tracer : public ProfilerInterface {
public:
static std::unique_ptr<ProfilerInterface> Create();
Status Start() override;
Status Stop() override;
Status CollectData(RunMetadata* run_metadata) override;
private:
Tracer();
// Trace is neither copyable nor movable.
Tracer(const Tracer&) = delete;
Tracer& operator=(const Tracer&) = delete;
std::unique_ptr<DeviceTracer> device_tracer_;
};
} // namespace gpu
} // namespace profiler
} // namespace tensorflow
#endif // TENSORFLOW_CORE_PROFILER_INTERNAL_GPU_TRACER_H_

View File

@ -0,0 +1,49 @@
/* Copyright 2016 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.
==============================================================================*/
#ifndef TENSORFLOW_CORE_PROFILER_INTERNAL_PROFILER_INTERFACE_H_
#define TENSORFLOW_CORE_PROFILER_INTERNAL_PROFILER_INTERFACE_H_
#include "tensorflow/core/lib/core/status.h"
#include "tensorflow/core/protobuf/config.pb.h"
namespace tensorflow {
namespace profiler {
// Interface for tensorflow profiler plugins.
//
// ProfileSession calls each of these methods at most once per instance, and
// implementations can rely on that guarantee for simplicity.
//
// Thread-safety: Implementations are only required to be go/thread-compatible.
// ProfileSession is go/thread-safe and synchronizes access to ProfilerInterface
// instances.
class ProfilerInterface {
public:
virtual ~ProfilerInterface() = default;
// Starts profiling.
virtual Status Start() = 0;
// Stops profiling.
virtual Status Stop() = 0;
// Moves collected profile data into run_metadata.
virtual Status CollectData(RunMetadata* run_metadata) = 0;
};
} // namespace profiler
} // namespace tensorflow
#endif // TENSORFLOW_CORE_PROFILER_INTERNAL_PROFILER_INTERFACE_H_

View File

@ -0,0 +1,24 @@
package(
default_visibility = ["//tensorflow:internal"],
)
licenses(["notice"]) # Apache 2.0
load(
"//tensorflow:tensorflow.bzl",
"tf_cuda_library",
)
tf_cuda_library(
name = "eager_profiler",
srcs = [
"eager_profiler.cc",
],
hdrs = [
"eager_profiler.h",
],
deps = [
"//tensorflow/core/common_runtime/eager:context",
"//tensorflow/core/profiler/internal:profiler_interface",
],
)

View File

@ -0,0 +1,61 @@
/* Copyright 2016 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.
==============================================================================*/
#include "tensorflow/core/profiler/internal/runtime/eager_profiler.h"
namespace tensorflow {
namespace profiler {
namespace runtime {
TraceCollector::TraceCollector(EagerContext* const eager_context)
: context_(eager_context) {}
void TraceCollector::BeforeClearRunMetadata() {
run_metadata_.MergeFrom(*context_->RunMetadataProto());
}
Status TraceCollector::CollectData(RunMetadata* run_metadata) {
run_metadata->MergeFrom(run_metadata_);
return Status::OK();
}
/* static */ std::unique_ptr<ProfilerInterface> EagerProfiler::Create(
EagerContext* const eager_context) {
return absl::WrapUnique(new EagerProfiler(eager_context));
}
Status EagerProfiler::Start() {
if (context_ == nullptr) {
return Status(tensorflow::error::Code::FAILED_PRECONDITION,
"No eager context attached.");
}
return context_->RegisterRunMetadataListener(&collector_);
}
Status EagerProfiler::Stop() {
collector_.BeforeClearRunMetadata();
context_->ClearRunMetadataListener();
return Status::OK();
}
Status EagerProfiler::CollectData(RunMetadata* run_metadata) {
return collector_.CollectData(run_metadata);
}
EagerProfiler::EagerProfiler(EagerContext* const eager_context)
: context_(eager_context), collector_(eager_context) {}
} // namespace runtime
} // namespace profiler
} // namespace tensorflow

View File

@ -0,0 +1,64 @@
/* Copyright 2016 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.
==============================================================================*/
#ifndef TENSORFLOW_CORE_PROFILER_INTERNAL_RUNTIME_EAGER_PROFILER_H_
#define TENSORFLOW_CORE_PROFILER_INTERNAL_RUNTIME_EAGER_PROFILER_H_
#include "tensorflow/core/common_runtime/eager/context.h"
#include "tensorflow/core/profiler/internal/profiler_interface.h"
namespace tensorflow {
namespace profiler {
namespace runtime {
class TraceCollector : public RunMetadataListener {
public:
TraceCollector(EagerContext* const eager_context);
void BeforeClearRunMetadata() override;
Status CollectData(RunMetadata* run_metadata);
private:
RunMetadata run_metadata_;
EagerContext* const context_;
};
class EagerProfiler : public ProfilerInterface {
public:
static std::unique_ptr<ProfilerInterface> Create(
EagerContext* const eager_context);
Status Start() override;
Status Stop() override;
Status CollectData(RunMetadata* run_metadata) override;
private:
EagerProfiler(EagerContext* const eager_context);
// Trace is neither copyable nor movable.
EagerProfiler(const EagerProfiler&) = delete;
EagerProfiler& operator=(const EagerProfiler&) = delete;
EagerContext* const context_;
TraceCollector collector_;
};
} // namespace runtime
} // namespace profiler
} // namespace tensorflow
#endif // TENSORFLOW_CORE_PROFILER_INTERNAL_RUNTIME_EAGER_PROFILER_H_

View File

@ -13,17 +13,20 @@ load(
)
tf_cuda_library(
name = "eager_profiler",
name = "profiler_session",
srcs = [
"eager_profiler.cc",
"profiler_session.cc",
],
hdrs = [
"eager_profiler.h",
"profiler_session.h",
],
visibility = ["//tensorflow:internal"],
deps = [
"//tensorflow/core/common_runtime/eager:context",
"//tensorflow/contrib/tpu/profiler:trace_events_proto_cc",
"//tensorflow/core/profiler/internal/gpu:tracer",
"//tensorflow/core/profiler/internal/runtime:eager_profiler",
"//tensorflow/core/profiler/internal:profiler_interface",
] + select({
"//tensorflow:android": [
"//tensorflow/core:android_tensorflow_lib_lite",

View File

@ -13,16 +13,16 @@ See the License for the specific language governing permissions and
limitations under the License.
==============================================================================*/
#include "tensorflow/core/profiler/lib/eager_profiler.h"
#include "tensorflow/core/profiler/lib/profiler_session.h"
#include <string>
#include "tensorflow/contrib/tpu/profiler/trace_events.pb.h"
#include "tensorflow/core/common_runtime/eager/context.h"
#include "tensorflow/core/common_runtime/step_stats_collector.h"
#include "tensorflow/core/framework/graph.pb.h"
#include "tensorflow/core/platform/device_tracer.h"
#include "tensorflow/core/platform/env.h"
#include "tensorflow/core/platform/mutex.h"
#include "tensorflow/core/platform/types.h"
#include "tensorflow/core/profiler/internal/gpu/tracer.h"
#include "tensorflow/core/profiler/internal/runtime/eager_profiler.h"
#include "tensorflow/core/protobuf/config.pb.h"
namespace tensorflow {
@ -72,90 +72,55 @@ void ConvertRunMetadataToTraceEvent(RunMetadata* run_metadata,
} // namespace
/*static*/ std::unique_ptr<EagerProfiler> EagerProfiler::Create(
/*static*/ std::unique_ptr<ProfilerSession> ProfilerSession::Create(
EagerContext* const context) {
return absl::WrapUnique(new EagerProfiler(context));
return absl::WrapUnique(new ProfilerSession(context));
}
void EagerProfiler::BeforeClearRunMetadata() {
mutex_lock l(mutex_);
run_metadata_.MergeFrom(*context_->RunMetadataProto());
}
Status EagerProfiler::Status() {
Status ProfilerSession::Status() {
mutex_lock l(mutex_);
return status_;
}
Status EagerProfiler::SerializeToString(string* content) {
Status ProfilerSession::SerializeToString(string* content) {
mutex_lock l(mutex_);
if (!status_.ok()) return status_;
Stop();
// Get profiling data from device tracer
if (device_tracer_ != nullptr) {
std::unique_ptr<StepStatsCollector> step_stats_collector(
new StepStatsCollector(run_metadata_.mutable_step_stats()));
tensorflow::Status s = device_tracer_->Collect(step_stats_collector.get());
if (!s.ok()) {
device_tracer_.reset(nullptr);
LOG(WARNING) << "Failed to collect data from device tracer. "
<< s.error_message();
}
step_stats_collector->Finalize();
for (auto& profiler : profilers_) {
profiler->Stop().IgnoreError();
}
RunMetadata run_metadata;
for (auto& profiler : profilers_) {
profiler->CollectData(&run_metadata).IgnoreError();
}
tpu::Trace trace;
ConvertRunMetadataToTraceEvent(&run_metadata_, &trace, start_time_micros_);
ConvertRunMetadataToTraceEvent(&run_metadata, &trace, start_time_micros_);
trace.SerializeToString(content);
return Status::OK();
}
EagerProfiler::EagerProfiler(EagerContext* const context)
: context_(context),
start_time_micros_(Env::Default()->NowNanos() / EnvTime::kMicrosToNanos) {
LOG(INFO) << "Eager Profiler started.";
ProfilerSession::ProfilerSession(EagerContext* const context)
: start_time_micros_(Env::Default()->NowNanos() / EnvTime::kMicrosToNanos) {
LOG(INFO) << "Profile Session started.";
status_ = context_->RegisterRunMetadataListener(this);
if (!status_.ok()) {
context_ = nullptr;
LOG(WARNING)
<< "Eager Profiler failed to start. Another profiler is running.";
return;
if (context != nullptr) {
profilers_.push_back(
tensorflow::profiler::runtime::EagerProfiler::Create(context));
}
profilers_.push_back(tensorflow::profiler::gpu::Tracer::Create());
// TODO(fishx): Allow user disable device tracer.
device_tracer_ = CreateDeviceTracer();
if (!device_tracer_) {
LOG(WARNING) << "Continue profiling without device tracer. "
<< "Failed to create device tracer.";
return;
}
class Status s = device_tracer_->Start();
if (!s.ok()) {
device_tracer_.reset(nullptr);
LOG(WARNING) << "Continue profiling without device tracer. "
<< s.error_message();
status_ = Status::OK();
for (auto& profiler : profilers_) {
profiler->Start().IgnoreError();
}
}
EagerProfiler::~EagerProfiler() { Stop(); }
void EagerProfiler::Stop() {
if (context_ != nullptr) {
context_->ClearRunMetadataListener();
run_metadata_.MergeFrom(*context_->RunMetadataProto());
context_ = nullptr;
if (device_tracer_ != nullptr) {
tensorflow::Status s = device_tracer_->Stop();
if (!s.ok()) {
device_tracer_.reset(nullptr);
LOG(WARNING) << "Failed to stop device tracer. " << s.error_message();
}
}
LOG(INFO) << "Eager Profiler ended with status:" << status_;
ProfilerSession::~ProfilerSession() {
for (auto& profiler : profilers_) {
profiler->Stop().IgnoreError();
}
}

View File

@ -12,14 +12,13 @@ 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.
==============================================================================*/
#ifndef TENSORFLOW_CORE_COMMON_RUNTIME_EAGER_PROFILER_H_
#define TENSORFLOW_CORE_COMMON_RUNTIME_EAGER_PROFILER_H_
#ifndef TENSORFLOW_CORE_PROFILER_LIB_PROFILER_SESSION_H_
#define TENSORFLOW_CORE_PROFILER_LIB_PROFILER_SESSION_H_
#include "tensorflow/core/common_runtime/eager/context.h"
#include "tensorflow/core/lib/core/status.h"
#include "tensorflow/core/platform/device_tracer.h"
#include "tensorflow/core/platform/mutex.h"
#include "tensorflow/core/protobuf/config.pb.h"
#include "tensorflow/core/profiler/internal/profiler_interface.h"
namespace tensorflow {
@ -29,38 +28,34 @@ namespace tensorflow {
// Multiple instances of it can be created, but at most one of them will profile
// for each EagerContext. Status() will return OK only for the instance that is
// profiling.
// Thread-safety: TFE_Profiler is thread-safe.
class EagerProfiler : RunMetadataListener {
// Thread-safety: ProfilerSession is thread-safe.
class ProfilerSession {
public:
// Creates and EagerProfiler and starts profiling.
static std::unique_ptr<EagerProfiler> Create(EagerContext* const context);
// Creates and ProfilerSession and starts profiling.
static std::unique_ptr<ProfilerSession> Create(EagerContext* const context);
// Deletes an exsiting Profiler and enables starting a new one.
~EagerProfiler() override;
~ProfilerSession();
void BeforeClearRunMetadata() override LOCKS_EXCLUDED(mutex_)
EXCLUSIVE_LOCKS_REQUIRED(context_->MetadataMu());
tensorflow::Status Status() LOCKS_EXCLUDED(mutex_);
tensorflow::Status SerializeToString(string* content) LOCKS_EXCLUDED(mutex_);
private:
// Constructs an instance of the class and starts profiling
explicit EagerProfiler(EagerContext* const context);
explicit ProfilerSession(EagerContext* const context);
// Profiler is neither copyable or movable.
EagerProfiler(const EagerProfiler&) = delete;
EagerProfiler& operator=(const EagerProfiler&) = delete;
ProfilerSession(const ProfilerSession&) = delete;
ProfilerSession& operator=(const ProfilerSession&) = delete;
void Stop() EXCLUSIVE_LOCKS_REQUIRED(mutex_);
std::vector<std::unique_ptr<tensorflow::profiler::ProfilerInterface>>
profilers_ GUARDED_BY(mutex_);
RunMetadata run_metadata_ GUARDED_BY(mutex_);
tensorflow::Status status_ GUARDED_BY(mutex_);
std::unique_ptr<DeviceTracer> device_tracer_ GUARDED_BY(mutex_);
EagerContext* context_ GUARDED_BY(mutex_);
const uint64 start_time_micros_;
mutex mutex_;
};
} // namespace tensorflow
#endif // TENSORFLOW_CORE_COMMON_RUNTIME_EAGER_PROFILER_H_
#endif // TENSORFLOW_CORE_PROFILER_LIB_PROFILER_SESSION_H_

View File

@ -14,7 +14,7 @@ tf_cuda_library(
"//tensorflow/contrib/tpu/profiler:tpu_profiler_proto_cc",
"//tensorflow/core:framework",
"//tensorflow/core/common_runtime/eager:context",
"//tensorflow/core/profiler/lib:eager_profiler",
"//tensorflow/core/profiler/lib:profiler_session",
],
alwayslink = 1,
)

View File

@ -17,7 +17,7 @@ limitations under the License.
#include "grpcpp/support/status.h"
#include "tensorflow/contrib/tpu/profiler/tpu_profiler.grpc.pb.h"
#include "tensorflow/core/common_runtime/eager/context.h"
#include "tensorflow/core/profiler/lib/eager_profiler.h"
#include "tensorflow/core/profiler/lib/profiler_session.h"
#include "tensorflow/core/util/ptr_util.h"
namespace tensorflow {
@ -38,8 +38,8 @@ class ProfilerServiceImpl : public TPUProfiler::Service {
::grpc::Status Profile(::grpc::ServerContext* ctx, const ProfileRequest* req,
ProfileResponse* response) override {
LOG(INFO) << "Received a profile request.";
std::unique_ptr<EagerProfiler> profiler =
EagerProfiler::Create(eager_context_);
std::unique_ptr<ProfilerSession> profiler =
ProfilerSession::Create(eager_context_);
if (!profiler->Status().ok()) {
return ::grpc::Status(::grpc::StatusCode::INTERNAL,
profiler->Status().error_message());

View File

@ -20,7 +20,6 @@ limitations under the License.
#include "grpcpp/support/status.h"
#include "tensorflow/contrib/tpu/profiler/tpu_profiler.grpc.pb.h"
#include "tensorflow/core/common_runtime/eager/context.h"
#include "tensorflow/core/profiler/lib/eager_profiler.h"
namespace tensorflow {
std::unique_ptr<TPUProfiler::Service> CreateProfilerService(