aboutsummaryrefslogtreecommitdiffhomepage
path: root/tensorflow/stream_executor/host
diff options
context:
space:
mode:
authorGravatar Peter Hawkins <phawkins@google.com>2016-12-05 06:27:11 -0800
committerGravatar TensorFlower Gardener <gardener@tensorflow.org>2016-12-05 06:44:09 -0800
commite4a4e922ed4be890f4273a7a276768dfefe80a4c (patch)
treeb724c791eb601946961899a621ea4335babfd1e0 /tensorflow/stream_executor/host
parentb00e490c3834d72aecce445f72b3a717b068d9db (diff)
Add a Host platform to the stream executor library that runs code on the CPU.
Avoid using a linker-initialized mutex in multi_platform_manager to work around a initialization/destruction ordering problem on Mac. Include the CUDA platform ID module even in non-CUDA builds. Change: 141045549
Diffstat (limited to 'tensorflow/stream_executor/host')
-rw-r--r--tensorflow/stream_executor/host/host_gpu_executor.cc263
-rw-r--r--tensorflow/stream_executor/host/host_gpu_executor.h215
-rw-r--r--tensorflow/stream_executor/host/host_platform.cc120
-rw-r--r--tensorflow/stream_executor/host/host_platform.h88
-rw-r--r--tensorflow/stream_executor/host/host_platform_id.cc26
-rw-r--r--tensorflow/stream_executor/host/host_platform_id.h36
-rw-r--r--tensorflow/stream_executor/host/host_stream.cc57
-rw-r--r--tensorflow/stream_executor/host/host_stream.h58
-rw-r--r--tensorflow/stream_executor/host/host_timer.cc52
-rw-r--r--tensorflow/stream_executor/host/host_timer.h63
10 files changed, 978 insertions, 0 deletions
diff --git a/tensorflow/stream_executor/host/host_gpu_executor.cc b/tensorflow/stream_executor/host/host_gpu_executor.cc
new file mode 100644
index 0000000000..ff07432bb7
--- /dev/null
+++ b/tensorflow/stream_executor/host/host_gpu_executor.cc
@@ -0,0 +1,263 @@
+/* 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.
+==============================================================================*/
+
+// Implementation of HostExecutor class [of those methods not defined in the
+// class declaration].
+#include "tensorflow/stream_executor/host/host_gpu_executor.h"
+
+#include <string.h>
+
+#include "tensorflow/stream_executor/host/host_platform_id.h"
+#include "tensorflow/stream_executor/host/host_stream.h"
+#include "tensorflow/stream_executor/host/host_timer.h"
+#include "tensorflow/stream_executor/lib/statusor.h"
+#include "tensorflow/stream_executor/plugin_registry.h"
+
+namespace gpu = ::perftools::gputools;
+
+namespace perftools {
+namespace gputools {
+namespace host {
+
+HostStream *AsHostStream(Stream *stream) {
+ DCHECK(stream != nullptr);
+ return dynamic_cast<HostStream *>(stream->implementation());
+}
+
+HostExecutor::HostExecutor(const PluginConfig &plugin_config)
+ : plugin_config_(plugin_config) {}
+
+HostExecutor::~HostExecutor() {}
+
+void *HostExecutor::Allocate(uint64 size) { return new char[size]; }
+
+void *HostExecutor::AllocateSubBuffer(DeviceMemoryBase *parent,
+ uint64 offset_bytes, uint64 size_bytes) {
+ return reinterpret_cast<char *>(parent->opaque()) + offset_bytes;
+}
+
+void HostExecutor::Deallocate(DeviceMemoryBase *mem) {
+ if (!mem->is_sub_buffer()) {
+ delete[] static_cast<char *>(mem->opaque());
+ }
+}
+
+bool HostExecutor::SynchronousMemZero(DeviceMemoryBase *location, uint64 size) {
+ memset(location->opaque(), 0, size);
+ return true;
+}
+
+bool HostExecutor::SynchronousMemSet(DeviceMemoryBase *location, int value,
+ uint64 size) {
+ memset(location->opaque(), value, size);
+ return true;
+}
+
+bool HostExecutor::Memcpy(Stream *stream, void *host_dst,
+ const DeviceMemoryBase &gpu_src, uint64 size) {
+ // Enqueue the [asynchronous] memcpy on the stream (HostStream) associated
+ // with the HostExecutor.
+ void *src_mem = const_cast<void *>(gpu_src.opaque());
+ AsHostStream(stream)->EnqueueTask(
+ [host_dst, src_mem, size]() { memcpy(host_dst, src_mem, size); });
+ return true;
+}
+
+bool HostExecutor::Memcpy(Stream *stream, DeviceMemoryBase *gpu_dst,
+ const void *host_src, uint64 size) {
+ void *dst_mem = gpu_dst->opaque();
+ // Enqueue the [asynchronous] memcpy on the stream (HostStream) associated
+ // with the HostExecutor.
+ AsHostStream(stream)->EnqueueTask(
+ [dst_mem, host_src, size]() { memcpy(dst_mem, host_src, size); });
+ return true;
+}
+
+bool HostExecutor::MemcpyDeviceToDevice(Stream *stream,
+ DeviceMemoryBase *gpu_dst,
+ const DeviceMemoryBase &gpu_src,
+ uint64 size) {
+ void *dst_mem = gpu_dst->opaque();
+ void *src_mem = const_cast<void *>(gpu_src.opaque());
+ // Enqueue this [asynchronous] "device-to-device" (i.e., host-to-host, given
+ // the nature of the HostExecutor) memcpy on the stream (HostStream)
+ // associated with the HostExecutor.
+ AsHostStream(stream)->EnqueueTask(
+ [src_mem, dst_mem, size]() { memcpy(src_mem, dst_mem, size); });
+ return true;
+}
+
+bool HostExecutor::MemZero(Stream *stream, DeviceMemoryBase *location,
+ uint64 size) {
+ void *gpu_mem = location->opaque();
+ // Enqueue the [asynchronous] memzero on the stream (HostStream) associated
+ // with the HostExecutor.
+ AsHostStream(stream)->EnqueueTask(
+ [gpu_mem, size]() { memset(gpu_mem, 0, size); });
+ return true;
+}
+
+bool HostExecutor::Memset(Stream *stream, DeviceMemoryBase *location,
+ uint8 pattern, uint64 size) {
+ void *gpu_mem = location->opaque();
+ // Enqueue the [asynchronous] memzero on the stream (HostStream) associated
+ // with the HostExecutor.
+ AsHostStream(stream)->EnqueueTask(
+ [gpu_mem, size, pattern]() { memset(gpu_mem, pattern, size); });
+ return true;
+}
+
+bool HostExecutor::Memset32(Stream *stream, DeviceMemoryBase *location,
+ uint32 pattern, uint64 size) {
+ void *gpu_mem = location->opaque();
+ // Enqueue the [asynchronous] memzero on the stream (HostStream) associated
+ // with the HostExecutor.
+ AsHostStream(stream)->EnqueueTask(
+ [gpu_mem, size, pattern]() { memset(gpu_mem, pattern, size); });
+ return true;
+}
+
+bool HostExecutor::SynchronousMemcpy(DeviceMemoryBase *gpu_dst,
+ const void *host_src, uint64 size) {
+ memcpy(gpu_dst->opaque(), host_src, size);
+ return true;
+}
+
+bool HostExecutor::SynchronousMemcpy(void *host_dst,
+ const DeviceMemoryBase &gpu_src,
+ uint64 size) {
+ memcpy(host_dst, gpu_src.opaque(), size);
+ return true;
+}
+
+bool HostExecutor::SynchronousMemcpyDeviceToDevice(
+ DeviceMemoryBase *gpu_dst, const DeviceMemoryBase &gpu_src, uint64 size) {
+ memcpy(gpu_dst->opaque(), gpu_src.opaque(), size);
+ return true;
+}
+
+bool HostExecutor::HostCallback(Stream *stream,
+ std::function<void()> callback) {
+ AsHostStream(stream)->EnqueueTask(callback);
+ return true;
+}
+
+bool HostExecutor::AllocateStream(Stream *stream) { return true; }
+
+void HostExecutor::DeallocateStream(Stream *stream) {}
+
+bool HostExecutor::CreateStreamDependency(Stream *dependent, Stream *other) {
+ AsHostStream(dependent)->EnqueueTask(
+ [other]() { other->BlockHostUntilDone(); });
+ AsHostStream(dependent)->BlockUntilDone();
+ return true;
+}
+
+bool HostExecutor::StartTimer(Stream *stream, Timer *timer) {
+ dynamic_cast<HostTimer *>(timer->implementation())->Start(stream);
+ return true;
+}
+
+bool HostExecutor::StopTimer(Stream *stream, Timer *timer) {
+ dynamic_cast<HostTimer *>(timer->implementation())->Stop(stream);
+ return true;
+}
+
+bool HostExecutor::BlockHostUntilDone(Stream *stream) {
+ AsHostStream(stream)->BlockUntilDone();
+ return true;
+}
+
+DeviceDescription *HostExecutor::PopulateDeviceDescription() const {
+ internal::DeviceDescriptionBuilder builder;
+
+ builder.set_device_address_bits(64);
+
+ // TODO(rspringer): How to report a value that's based in reality but that
+ // doesn't result in thrashing or other badness? 4GiB chosen arbitrarily.
+ builder.set_device_memory_size(static_cast<uint64>(4) * 1024 * 1024 * 1024);
+
+ builder.set_clock_rate_ghz(static_cast<float>(CLOCKS_PER_SEC) / 1e9);
+
+ auto built = builder.Build();
+ return built.release();
+}
+
+bool HostExecutor::SupportsBlas() const {
+ return PluginRegistry::Instance()
+ ->GetFactory<PluginRegistry::BlasFactory>(kHostPlatformId,
+ plugin_config_.blas())
+ .ok();
+}
+
+blas::BlasSupport *HostExecutor::CreateBlas() {
+ PluginRegistry *registry = PluginRegistry::Instance();
+ port::StatusOr<PluginRegistry::BlasFactory> status =
+ registry->GetFactory<PluginRegistry::BlasFactory>(kHostPlatformId,
+ plugin_config_.blas());
+ if (!status.ok()) {
+ LOG(ERROR) << "Unable to retrieve BLAS factory: "
+ << status.status().error_message();
+ return nullptr;
+ }
+
+ return status.ValueOrDie()(this);
+}
+
+bool HostExecutor::SupportsFft() const {
+ return PluginRegistry::Instance()
+ ->GetFactory<PluginRegistry::FftFactory>(kHostPlatformId,
+ plugin_config_.fft())
+ .ok();
+}
+
+fft::FftSupport *HostExecutor::CreateFft() {
+ PluginRegistry *registry = PluginRegistry::Instance();
+ port::StatusOr<PluginRegistry::FftFactory> status =
+ registry->GetFactory<PluginRegistry::FftFactory>(kHostPlatformId,
+ plugin_config_.fft());
+ if (!status.ok()) {
+ LOG(ERROR) << "Unable to retrieve FFT factory: "
+ << status.status().error_message();
+ return nullptr;
+ }
+
+ return status.ValueOrDie()(this);
+}
+
+bool HostExecutor::SupportsRng() const {
+ return PluginRegistry::Instance()
+ ->GetFactory<PluginRegistry::RngFactory>(kHostPlatformId,
+ plugin_config_.rng())
+ .ok();
+}
+
+rng::RngSupport *HostExecutor::CreateRng() {
+ PluginRegistry *registry = PluginRegistry::Instance();
+ port::StatusOr<PluginRegistry::RngFactory> status =
+ registry->GetFactory<PluginRegistry::RngFactory>(kHostPlatformId,
+ plugin_config_.rng());
+ if (!status.ok()) {
+ LOG(ERROR) << "Unable to retrieve RNG factory: "
+ << status.status().error_message();
+ return nullptr;
+ }
+
+ return status.ValueOrDie()(this);
+}
+
+} // namespace host
+} // namespace gputools
+} // namespace perftools
diff --git a/tensorflow/stream_executor/host/host_gpu_executor.h b/tensorflow/stream_executor/host/host_gpu_executor.h
new file mode 100644
index 0000000000..f217f7947f
--- /dev/null
+++ b/tensorflow/stream_executor/host/host_gpu_executor.h
@@ -0,0 +1,215 @@
+/* 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.
+==============================================================================*/
+
+// Declares the HostExecutor class, which is a CPU-only implementation of
+// the StreamExecutor interface. For now, this is used for testing and to
+// examine the performance of host-based StreamExecutor code.
+#ifndef TENSORFLOW_STREAM_EXECUTOR_HOST_HOST_GPU_EXECUTOR_H_
+#define TENSORFLOW_STREAM_EXECUTOR_HOST_HOST_GPU_EXECUTOR_H_
+
+#include "tensorflow/stream_executor/blas.h"
+#include "tensorflow/stream_executor/host/host_stream.h"
+#include "tensorflow/stream_executor/host/host_timer.h"
+#include "tensorflow/stream_executor/lib/error.h"
+#include "tensorflow/stream_executor/lib/status.h"
+#include "tensorflow/stream_executor/rng.h"
+#include "tensorflow/stream_executor/stream_executor.h"
+#include "tensorflow/stream_executor/stream_executor_internal.h"
+
+namespace perftools {
+namespace gputools {
+namespace host {
+
+// An implementation of StreamExecutor that does no communication or interaction
+// with a device, but DOES perform memory operations backed by the host.
+// Plugin routines (RNG, BLAS) are also supported and functional.
+// Kernel invocations will fail, but host callbacks may be enqueued on this
+// executor and its associated stream, and should follow standard ordering
+// semantics.
+//
+// This is useful for evaluating the performance of host-based or fallback
+// routines executed under the context of a GPU executor.
+// See stream_executor.h for description of the below operations.
+class HostExecutor : public internal::StreamExecutorInterface {
+ public:
+ explicit HostExecutor(const PluginConfig &plugin_config);
+ ~HostExecutor() override;
+
+ port::Status Init(int device_ordinal, DeviceOptions device_options) override {
+ return port::Status::OK();
+ }
+
+ bool GetKernel(const MultiKernelLoaderSpec &spec,
+ KernelBase *kernel) override {
+ return false;
+ }
+ bool Launch(Stream *stream, const ThreadDim &thread_dims,
+ const BlockDim &block_dims, const KernelBase &kernel,
+ const KernelArgsArrayBase &args) override {
+ return false;
+ }
+
+ void *Allocate(uint64 size) override;
+ void *AllocateSubBuffer(DeviceMemoryBase *mem, uint64 offset_bytes,
+ uint64 size_bytes) override;
+ void Deallocate(DeviceMemoryBase *mem) override;
+
+ void *HostMemoryAllocate(uint64 size) override { return new char[size]; }
+ void HostMemoryDeallocate(void *mem) override {
+ delete[] static_cast<char *>(mem);
+ }
+ bool HostMemoryRegister(void *mem, uint64 size) override { return true; }
+ bool HostMemoryUnregister(void *mem) override { return true; }
+
+ bool Memcpy(Stream *stream, void *host_dst, const DeviceMemoryBase &gpu_src,
+ uint64 size) override;
+ bool Memcpy(Stream *stream, DeviceMemoryBase *gpu_dst, const void *host_src,
+ uint64 size) override;
+ bool MemcpyDeviceToDevice(Stream *stream, DeviceMemoryBase *gpu_dst,
+ const DeviceMemoryBase &host_src,
+ uint64 size) override;
+
+ bool MemZero(Stream *stream, DeviceMemoryBase *location,
+ uint64 size) override;
+ bool Memset(Stream *stream, DeviceMemoryBase *location, uint8 pattern,
+ uint64 size) override;
+ bool Memset32(Stream *stream, DeviceMemoryBase *location, uint32 pattern,
+ uint64 size) override;
+
+ // No "synchronize all activity" implemented for this platform at the moment.
+ bool SynchronizeAllActivity() override { return false; }
+ bool SynchronousMemZero(DeviceMemoryBase *location, uint64 size) override;
+
+ bool SynchronousMemSet(DeviceMemoryBase *location, int value,
+ uint64 size) override;
+
+ bool SynchronousMemcpy(DeviceMemoryBase *gpu_dst, const void *host_src,
+ uint64 size) override;
+ bool SynchronousMemcpy(void *host_dst, const DeviceMemoryBase &gpu_src,
+ uint64 size) override;
+ bool SynchronousMemcpyDeviceToDevice(DeviceMemoryBase *gpu_dst,
+ const DeviceMemoryBase &gpu_src,
+ uint64 size) override;
+
+ bool HostCallback(Stream *stream, std::function<void()> callback) override;
+
+ port::Status AllocateEvent(Event *event) override {
+ return port::Status{port::error::UNIMPLEMENTED, ""};
+ }
+
+ port::Status DeallocateEvent(Event *event) override {
+ return port::Status{port::error::UNIMPLEMENTED, ""};
+ }
+
+ port::Status RecordEvent(Stream *stream, Event *event) override {
+ return port::Status{port::error::UNIMPLEMENTED, ""};
+ }
+
+ port::Status WaitForEvent(Stream *stream, Event *event) override {
+ return port::Status{port::error::UNIMPLEMENTED, ""};
+ }
+
+ Event::Status PollForEventStatus(Event *event) override {
+ return Event::Status::kError;
+ }
+
+ bool AllocateStream(Stream *stream) override;
+ void DeallocateStream(Stream *stream) override;
+ bool CreateStreamDependency(Stream *dependent, Stream *other) override;
+
+ // No special initialization is necessary for host timers.
+ bool AllocateTimer(Timer *timer) override { return true; }
+
+ void DeallocateTimer(Timer *timer) override {}
+
+ bool StartTimer(Stream *stream, Timer *timer) override;
+
+ bool StopTimer(Stream *stream, Timer *timer) override;
+
+ bool BlockHostUntilDone(Stream *stream) override;
+
+ int PlatformDeviceCount() override { return 1; }
+
+ bool DeviceMemoryUsage(int64 *free, int64 *total) const override {
+ return false;
+ }
+
+ DeviceDescription *PopulateDeviceDescription() const override;
+
+ port::Status EnablePeerAccessTo(StreamExecutorInterface *other) override {
+ return port::Status::OK();
+ }
+
+ bool CanEnablePeerAccessTo(StreamExecutorInterface *other) override {
+ return true;
+ }
+
+ SharedMemoryConfig GetDeviceSharedMemoryConfig() override {
+ LOG(INFO) << "Shared memory configuration is unsupported for host "
+ << "executors.";
+ return SharedMemoryConfig::kDefault;
+ }
+
+ port::Status SetDeviceSharedMemoryConfig(SharedMemoryConfig config) override {
+ string error_msg{
+ "Shared memory configuration is unsupported for host "
+ "executors."};
+ LOG(INFO) << error_msg;
+ return port::Status{port::error::UNIMPLEMENTED, error_msg};
+ }
+
+ bool SupportsBlas() const override;
+ blas::BlasSupport *CreateBlas() override;
+
+ bool SupportsDnn() const override { return false; }
+ dnn::DnnSupport *CreateDnn() override { return nullptr; }
+
+ bool SupportsFft() const override;
+ fft::FftSupport *CreateFft() override;
+
+ bool SupportsRng() const override;
+ rng::RngSupport *CreateRng() override;
+
+ std::unique_ptr<internal::EventInterface> CreateEventImplementation()
+ override {
+ LOG(WARNING) << "Events not currently supported by HostExecutor.";
+ return nullptr;
+ }
+
+ std::unique_ptr<internal::KernelInterface> CreateKernelImplementation()
+ override {
+ return nullptr;
+ }
+
+ std::unique_ptr<internal::StreamInterface> GetStreamImplementation()
+ override {
+ return std::unique_ptr<internal::StreamInterface>(new HostStream());
+ }
+
+ std::unique_ptr<internal::TimerInterface> GetTimerImplementation() override {
+ return std::unique_ptr<internal::TimerInterface>(new HostTimer());
+ }
+
+ void *CudaContextHack() override { return nullptr; }
+
+ private:
+ const PluginConfig plugin_config_;
+};
+
+} // namespace host
+} // namespace gputools
+} // namespace perftools
+
+#endif // TENSORFLOW_STREAM_EXECUTOR_HOST_HOST_GPU_EXECUTOR_H_
diff --git a/tensorflow/stream_executor/host/host_platform.cc b/tensorflow/stream_executor/host/host_platform.cc
new file mode 100644
index 0000000000..1fa4dfce84
--- /dev/null
+++ b/tensorflow/stream_executor/host/host_platform.cc
@@ -0,0 +1,120 @@
+/* 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/stream_executor/host/host_platform.h"
+
+#include <thread>
+
+#include "tensorflow/stream_executor/host/host_gpu_executor.h"
+#include "tensorflow/stream_executor/host/host_platform_id.h"
+#include "tensorflow/stream_executor/lib/error.h"
+#include "tensorflow/stream_executor/lib/initialize.h"
+#include "tensorflow/stream_executor/lib/ptr_util.h"
+#include "tensorflow/stream_executor/lib/status.h"
+#include "tensorflow/stream_executor/lib/status_macros.h"
+#include "tensorflow/stream_executor/lib/stringprintf.h"
+
+namespace gpu = ::perftools::gputools;
+
+namespace perftools {
+namespace gputools {
+namespace host {
+
+HostPlatform::HostPlatform() : name_("Host") {}
+
+HostPlatform::~HostPlatform() {}
+
+Platform::Id HostPlatform::id() const { return kHostPlatformId; }
+
+int HostPlatform::VisibleDeviceCount() const {
+ return std::thread::hardware_concurrency();
+}
+
+const string& HostPlatform::Name() const { return name_; }
+
+port::StatusOr<StreamExecutor*> HostPlatform::ExecutorForDevice(int ordinal) {
+ StreamExecutorConfig config;
+ config.ordinal = ordinal;
+ config.plugin_config = PluginConfig();
+ config.device_options = DeviceOptions::Default();
+ return GetExecutor(config);
+}
+
+port::StatusOr<StreamExecutor*> HostPlatform::ExecutorForDeviceWithPluginConfig(
+ int device_ordinal, const PluginConfig& plugin_config) {
+ StreamExecutorConfig config;
+ config.ordinal = device_ordinal;
+ config.plugin_config = plugin_config;
+ config.device_options = DeviceOptions::Default();
+ return GetExecutor(config);
+}
+
+port::StatusOr<StreamExecutor*> HostPlatform::GetExecutor(
+ const StreamExecutorConfig& config) {
+ mutex_lock lock(executors_mutex_);
+
+ port::StatusOr<StreamExecutor*> status = executor_cache_.Get(config);
+ if (status.ok()) {
+ return status.ValueOrDie();
+ }
+
+ port::StatusOr<std::unique_ptr<StreamExecutor>> executor =
+ GetUncachedExecutor(config);
+ if (!executor.ok()) {
+ return executor.status();
+ }
+
+ StreamExecutor* naked_executor = executor.ValueOrDie().get();
+ SE_RETURN_IF_ERROR(
+ executor_cache_.Insert(config, executor.ConsumeValueOrDie()));
+ return naked_executor;
+}
+
+port::StatusOr<std::unique_ptr<StreamExecutor>>
+HostPlatform::GetUncachedExecutor(const StreamExecutorConfig& config) {
+ auto executor = port::MakeUnique<StreamExecutor>(
+ this, new HostExecutor(config.plugin_config));
+ auto init_status = executor->Init(config.ordinal, config.device_options);
+ if (!init_status.ok()) {
+ return port::Status{
+ port::error::INTERNAL,
+ port::Printf(
+ "failed initializing StreamExecutor for device ordinal %d: %s",
+ config.ordinal, init_status.ToString().c_str())};
+ }
+
+ return std::move(executor);
+}
+
+void HostPlatform::RegisterTraceListener(
+ std::unique_ptr<TraceListener> listener) {
+ LOG(FATAL) << "not yet implemented: register host trace listener";
+}
+
+void HostPlatform::UnregisterTraceListener(TraceListener* listener) {
+ LOG(FATAL) << "not yet implemented: unregister host trace listener";
+}
+
+static void InitializeHostPlatform() {
+ std::unique_ptr<gpu::Platform> platform(new gpu::host::HostPlatform);
+ SE_CHECK_OK(gpu::MultiPlatformManager::RegisterPlatform(std::move(platform)));
+}
+
+} // namespace host
+} // namespace gputools
+} // namespace perftools
+
+REGISTER_MODULE_INITIALIZER(
+ host_platform, perftools::gputools::host::InitializeHostPlatform());
diff --git a/tensorflow/stream_executor/host/host_platform.h b/tensorflow/stream_executor/host/host_platform.h
new file mode 100644
index 0000000000..86805ef3e3
--- /dev/null
+++ b/tensorflow/stream_executor/host/host_platform.h
@@ -0,0 +1,88 @@
+/* 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.
+==============================================================================*/
+
+// Declares the "host" platform, which is a CPU-only implementation of the
+// StreamExecutor. The host platform only supports memory operations and plugin
+// routines, and is primarily used for testing.
+#ifndef TENSORFLOW_STREAM_EXECUTOR_HOST_HOST_PLATFORM_H_
+#define TENSORFLOW_STREAM_EXECUTOR_HOST_HOST_PLATFORM_H_
+
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "tensorflow/stream_executor/executor_cache.h"
+#include "tensorflow/stream_executor/lib/statusor.h"
+#include "tensorflow/stream_executor/multi_platform_manager.h"
+#include "tensorflow/stream_executor/platform.h"
+#include "tensorflow/stream_executor/platform/mutex.h"
+#include "tensorflow/stream_executor/platform/port.h"
+#include "tensorflow/stream_executor/platform/thread_annotations.h"
+#include "tensorflow/stream_executor/stream_executor_pimpl.h"
+#include "tensorflow/stream_executor/trace_listener.h"
+
+namespace perftools {
+namespace gputools {
+namespace host {
+
+// Host (CPU) platform plugin, registered as a singleton value via module
+// initializer.
+class HostPlatform : public Platform {
+ public:
+ HostPlatform();
+ ~HostPlatform() override;
+
+ Platform::Id id() const override;
+
+ // Device count is less clear-cut for CPUs than accelerators. This call
+ // currently returns the number of thread units in the host, as reported by
+ // base::NumCPUs().
+ int VisibleDeviceCount() const override;
+
+ const string& Name() const override;
+
+ port::StatusOr<StreamExecutor*> ExecutorForDevice(int ordinal) override;
+
+ port::StatusOr<StreamExecutor*> ExecutorForDeviceWithPluginConfig(
+ int ordinal, const PluginConfig& config) override;
+
+ port::StatusOr<StreamExecutor*> GetExecutor(
+ const StreamExecutorConfig& config) override;
+
+ port::StatusOr<std::unique_ptr<StreamExecutor>> GetUncachedExecutor(
+ const StreamExecutorConfig& config) override;
+
+ void RegisterTraceListener(std::unique_ptr<TraceListener> listener) override;
+
+ void UnregisterTraceListener(TraceListener* listener) override;
+
+ private:
+ // This platform's name.
+ string name_;
+
+ // mutex that guards the ordinal-to-executor map.
+ mutable mutex executors_mutex_;
+
+ // Cache of created StreamExecutors.
+ ExecutorCache executor_cache_;
+
+ SE_DISALLOW_COPY_AND_ASSIGN(HostPlatform);
+};
+
+} // namespace host
+} // namespace gputools
+} // namespace perftools
+
+#endif // TENSORFLOW_STREAM_EXECUTOR_HOST_HOST_PLATFORM_H_
diff --git a/tensorflow/stream_executor/host/host_platform_id.cc b/tensorflow/stream_executor/host/host_platform_id.cc
new file mode 100644
index 0000000000..69a203f298
--- /dev/null
+++ b/tensorflow/stream_executor/host/host_platform_id.cc
@@ -0,0 +1,26 @@
+/* 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/stream_executor/host/host_platform_id.h"
+
+namespace perftools {
+namespace gputools {
+namespace host {
+
+PLATFORM_DEFINE_ID(kHostPlatformId);
+
+} // namespace host
+} // namespace gputools
+} // namespace perftools
diff --git a/tensorflow/stream_executor/host/host_platform_id.h b/tensorflow/stream_executor/host/host_platform_id.h
new file mode 100644
index 0000000000..61d84ea2e2
--- /dev/null
+++ b/tensorflow/stream_executor/host/host_platform_id.h
@@ -0,0 +1,36 @@
+/* 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_STREAM_EXECUTOR_HOST_HOST_PLATFORM_ID_H_
+#define TENSORFLOW_STREAM_EXECUTOR_HOST_HOST_PLATFORM_ID_H_
+
+#include "tensorflow/stream_executor/platform.h"
+
+namespace perftools {
+namespace gputools {
+namespace host {
+
+// Opaque and unique identifier for the host platform.
+// This is needed so that plugins can refer to/identify this platform without
+// instantiating a HostPlatform object.
+// This is broken out here to avoid a circular dependency between HostPlatform
+// and HostStreamExecutor.
+extern const Platform::Id kHostPlatformId;
+
+} // namespace host
+} // namespace gputools
+} // namespace perftools
+
+#endif // TENSORFLOW_STREAM_EXECUTOR_HOST_HOST_PLATFORM_ID_H_
diff --git a/tensorflow/stream_executor/host/host_stream.cc b/tensorflow/stream_executor/host/host_stream.cc
new file mode 100644
index 0000000000..c81ca406b8
--- /dev/null
+++ b/tensorflow/stream_executor/host/host_stream.cc
@@ -0,0 +1,57 @@
+/* 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.
+==============================================================================*/
+
+// Class method definitions for HostStream, the Stream implementation for
+// the HostExecutor implementation.
+#include "tensorflow/stream_executor/host/host_stream.h"
+
+namespace perftools {
+namespace gputools {
+namespace host {
+
+HostStream::HostStream()
+ : host_executor_(new port::ThreadPool(port::Env::Default(),
+ port::ThreadOptions(),
+ "host_executor", kExecutorThreads)) {}
+
+HostStream::~HostStream() {}
+
+bool HostStream::EnqueueTask(std::function<void()> task) {
+ {
+ mutex_lock lock(mu_);
+ ++pending_tasks_;
+ }
+ host_executor_->Schedule([this, task]() {
+ task();
+ {
+ mutex_lock lock(mu_);
+ --pending_tasks_;
+ }
+ completion_condition_.notify_all();
+ });
+ return true;
+}
+
+void HostStream::BlockUntilDone() {
+ mutex_lock lock(mu_);
+ completion_condition_.wait(lock, [this]() {
+ return pending_tasks_ == 0;
+ });
+}
+
+} // namespace host
+
+} // namespace gputools
+} // namespace perftools
diff --git a/tensorflow/stream_executor/host/host_stream.h b/tensorflow/stream_executor/host/host_stream.h
new file mode 100644
index 0000000000..9894d17feb
--- /dev/null
+++ b/tensorflow/stream_executor/host/host_stream.h
@@ -0,0 +1,58 @@
+/* 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.
+==============================================================================*/
+
+// Class declaration for Stream type that enqueues tasks onto a host/CPU-based
+// execution context (as opposed to a GPU device), HostExecutor.
+#ifndef TENSORFLOW_STREAM_EXECUTOR_HOST_HOST_STREAM_H_
+#define TENSORFLOW_STREAM_EXECUTOR_HOST_HOST_STREAM_H_
+
+#include <functional>
+#include <memory>
+
+#include "tensorflow/stream_executor/lib/threadpool.h"
+#include "tensorflow/stream_executor/stream_executor_internal.h"
+
+namespace perftools {
+namespace gputools {
+namespace host {
+
+class HostStream : public internal::StreamInterface {
+ public:
+ HostStream();
+ ~HostStream() override;
+
+ bool EnqueueTask(std::function<void()> task);
+
+ void *CudaStreamHack() override { return nullptr; }
+ void **CudaStreamMemberHack() override { return nullptr; }
+
+ void BlockUntilDone();
+
+ private:
+ // Use only one thread and own task queue to preserve FIFO ordering
+ // for the operations enqueued by any given stream.
+ static const int kExecutorThreads = 1;
+ std::unique_ptr<port::ThreadPool> host_executor_;
+
+ mutex mu_;
+ int pending_tasks_ GUARDED_BY(mu_) = 0;
+ condition_variable completion_condition_;
+};
+
+} // namespace host
+} // namespace gputools
+} // namespace perftools
+
+#endif // TENSORFLOW_STREAM_EXECUTOR_HOST_HOST_STREAM_H_
diff --git a/tensorflow/stream_executor/host/host_timer.cc b/tensorflow/stream_executor/host/host_timer.cc
new file mode 100644
index 0000000000..187db9f0c2
--- /dev/null
+++ b/tensorflow/stream_executor/host/host_timer.cc
@@ -0,0 +1,52 @@
+/* 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/stream_executor/host/host_timer.h"
+
+#include "tensorflow/stream_executor/platform/logging.h"
+#include "tensorflow/stream_executor/stream.h"
+#include "tensorflow/stream_executor/stream_executor_internal.h"
+
+namespace gpu = ::perftools::gputools;
+
+namespace perftools {
+namespace gputools {
+namespace host {
+
+using std::chrono::duration_cast;
+
+bool HostTimer::Start(Stream* stream) {
+ return stream->ThenDoHostCallback([this]() { this->StartNow(); }).ok();
+}
+
+bool HostTimer::Stop(Stream* stream) {
+ return stream->ThenDoHostCallback([this]() { this->StopNow(); }).ok();
+}
+
+uint64 HostTimer::Microseconds() const {
+ return duration_cast<std::chrono::microseconds>(duration_).count();
+}
+
+uint64 HostTimer::Nanoseconds() const {
+ return duration_cast<std::chrono::nanoseconds>(duration_).count();
+}
+
+void HostTimer::StartNow() { start_time_ = clock::now(); }
+
+void HostTimer::StopNow() { duration_ = clock::now() - start_time_; }
+
+} // namespace host
+} // namespace gputools
+} // namespace perftools
diff --git a/tensorflow/stream_executor/host/host_timer.h b/tensorflow/stream_executor/host/host_timer.h
new file mode 100644
index 0000000000..17af7c0521
--- /dev/null
+++ b/tensorflow/stream_executor/host/host_timer.h
@@ -0,0 +1,63 @@
+/* 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_STREAM_EXECUTOR_HOST_HOST_TIMER_H_
+#define TENSORFLOW_STREAM_EXECUTOR_HOST_HOST_TIMER_H_
+
+#include <chrono>
+
+#include "tensorflow/stream_executor/stream_executor_internal.h"
+
+namespace perftools {
+namespace gputools {
+namespace host {
+
+class HostTimer : public internal::TimerInterface {
+ public:
+ HostTimer() {}
+ ~HostTimer() override {}
+
+ // Begins the timer at the present point in the stream.
+ bool Start(Stream *stream);
+
+ // Stops the timer at the present point in the stream.
+ bool Stop(Stream *stream);
+
+ // Returns the most recent value recorded for a start/stopcycle, in
+ // microseconds.
+ uint64 Microseconds() const override;
+
+ // Returns the most recent value recorded for a start/stopcycle, in
+ // nanoseconds.
+ uint64 Nanoseconds() const override;
+
+ private:
+ using clock = std::chrono::high_resolution_clock;
+
+ clock::time_point start_time_;
+ clock::duration duration_;
+
+ // Actually starts (rather than enqueues starting) the timer.
+ void StartNow();
+
+ // Actually stops (rather than enqueues stopping) the timer.
+ void StopNow();
+};
+
+} // namespace host
+} // namespace gputools
+} // namespace perftools
+
+#endif // TENSORFLOW_STREAM_EXECUTOR_HOST_HOST_TIMER_H_