You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@rocketmq.apache.org by li...@apache.org on 2022/07/28 08:55:12 UTC

[rocketmq-clients] branch master updated: Retry ack/nack/forward-to-dlq of messages unless the lifecycle completes: success or receipt-handle becomes invalid (#82)

This is an automated email from the ASF dual-hosted git repository.

lizhanhui pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/rocketmq-clients.git


The following commit(s) were added to refs/heads/master by this push:
     new a4718c8  Retry ack/nack/forward-to-dlq of messages unless the lifecycle completes: success or receipt-handle becomes invalid (#82)
a4718c8 is described below

commit a4718c895cddbc985eea9387e97cc622a3fc3759
Author: Zhanhui Li <li...@gmail.com>
AuthorDate: Thu Jul 28 16:55:08 2022 +0800

    Retry ack/nack/forward-to-dlq of messages unless the lifecycle completes: success or receipt-handle becomes invalid (#82)
    
    * Make client-id unique in case container clients retart
    
    * Delete unused files from opencensus
    
    * Retry ack/nack/forward-to-dlq of messages unless the lifecycle completes: success or receipt-handle becomes invalid
---
 cpp/source/rocketmq/ClientImpl.cpp                 |   1 +
 cpp/source/rocketmq/ConsumeTask.cpp                |  62 ++-
 cpp/source/rocketmq/include/ClientImpl.h           |   8 +
 cpp/source/rocketmq/include/ConsumeTask.h          |   2 +
 cpp/third_party/opencensus/0.5.0-alpha/README.md   |   5 +-
 .../opencensus/0.5.0-alpha/opencensus/BUILD        |  37 --
 .../opencensus/0.5.0-alpha/opencensus/common/BUILD |  28 -
 .../0.5.0-alpha/opencensus/common/internal/BUILD   | 163 ------
 .../opencensus/common/internal/grpc/BUILD          |  57 --
 .../0.5.0-alpha/opencensus/context/BUILD           |  86 ---
 .../opencensus/0.5.0-alpha/opencensus/copts.bzl    |  47 --
 .../opencensus/0.5.0-alpha/opencensus/curl.bzl     | 201 -------
 .../opencensus/exporters/stats/stdout/BUILD        |  45 --
 .../opencensus/exporters/trace/ocagent/BUILD       |  56 --
 .../opencensus/exporters/trace/stackdriver/BUILD   |  44 --
 .../opencensus/exporters/trace/stdout/BUILD        |  49 --
 .../opencensus/exporters/trace/zipkin/BUILD        |  56 --
 .../opencensus/0.5.0-alpha/opencensus/stats/BUILD  | 264 ---------
 .../0.5.0-alpha/opencensus/stats/examples/BUILD    |  52 --
 .../opencensus/0.5.0-alpha/opencensus/tags/BUILD   | 199 -------
 .../opencensus/0.5.0-alpha/opencensus/trace/BUILD  | 601 ---------------------
 .../0.5.0-alpha/opencensus/trace/examples/BUILD    |  39 --
 .../opencensus/proto/agent/common/v1/BUILD.bazel   |  34 --
 .../opencensus/proto/agent/metrics/v1/BUILD.bazel  |  55 --
 .../opencensus/proto/agent/trace/v1/BUILD.bazel    |  56 --
 .../proto/opencensus/proto/metrics/v1/BUILD.bazel  |  48 --
 .../proto/opencensus/proto/resource/v1/BUILD.bazel |  38 --
 .../proto/opencensus/proto/stats/v1/BUILD.bazel    |  45 --
 .../proto/opencensus/proto/trace/v1/BUILD.bazel    |  83 ---
 29 files changed, 57 insertions(+), 2404 deletions(-)

diff --git a/cpp/source/rocketmq/ClientImpl.cpp b/cpp/source/rocketmq/ClientImpl.cpp
index 0532d73..bccfe37 100644
--- a/cpp/source/rocketmq/ClientImpl.cpp
+++ b/cpp/source/rocketmq/ClientImpl.cpp
@@ -609,6 +609,7 @@ std::string ClientImpl::clientId() {
   std::string processID = std::to_string(getpid());
   ss << processID << "#";
   ss << sequence.fetch_add(1, std::memory_order_relaxed);
+  ss << "_" << MixAll::millisecondsOf(std::chrono::system_clock::now().time_since_epoch());
   return ss.str();
 }
 
diff --git a/cpp/source/rocketmq/ConsumeTask.cpp b/cpp/source/rocketmq/ConsumeTask.cpp
index 9e87828..5981d07 100644
--- a/cpp/source/rocketmq/ConsumeTask.cpp
+++ b/cpp/source/rocketmq/ConsumeTask.cpp
@@ -18,11 +18,12 @@
 #include "ConsumeTask.h"
 
 #include "ConsumeStats.h"
-#include "rocketmq/Logger.h"
-#include "spdlog/spdlog.h"
 #include "PushConsumerImpl.h"
 #include "Tag.h"
 #include "rocketmq/ConsumeResult.h"
+#include "rocketmq/ErrorCode.h"
+#include "rocketmq/Logger.h"
+#include "spdlog/spdlog.h"
 
 ROCKETMQ_NAMESPACE_BEGIN
 
@@ -70,42 +71,56 @@ void ConsumeTask::schedule() {
 }
 
 void ConsumeTask::onAck(std::shared_ptr<ConsumeTask> task, const std::error_code& ec) {
-  if (task->fifo_ && ec) {
-    auto service = task->service_.lock();
-    task->next_step_ = NextStep::Ack;
-    task->schedule();
-  } else {
-    // If it is not FIFO or ack operation succeeded
+  // Treat both success and invalid-receipt-handle as completion
+  if (!ec || ec == ErrorCode::InvalidReceiptHandle) {
     task->pop();
     task->next_step_ = NextStep::Consume;
+    task->submit();
+    return;
   }
-  task->submit();
+
+  // Try to ack again later
+  SPDLOG_WARN("Failed to ack message[message-id={}]. Cause: {}. Action: retry after 1s.", task->messages_[0]->id(),
+              ec.message());
+  task->next_step_ = NextStep::Ack;
+  task->schedule();
 }
 
 void ConsumeTask::onNack(std::shared_ptr<ConsumeTask> task, const std::error_code& ec) {
   assert(!task->fifo_);
   assert(!task->messages_.empty());
-  if (ec) {
-    SPDLOG_WARN("Failed to nack message[message-id={}]. Cause: {}", task->messages_[0]->id(), ec.message());
+
+  // Treat both success and invalid-receipt-handle as completion
+  if (!ec || ErrorCode::InvalidReceiptHandle == ec) {
+    task->pop();
+    task->next_step_ = NextStep::Consume;
+    task->submit();
+    return;
   }
-  task->pop();
-  task->next_step_ = NextStep::Consume;
-  task->submit();
+
+  SPDLOG_WARN("Failed to nack message[message-id={}]. Cause: {}. Action: retry after 1s.", task->messages_[0]->id(),
+              ec.message());
+  task->next_step_ = NextStep::Nack;
+  task->schedule();
 }
 
 void ConsumeTask::onForward(std::shared_ptr<ConsumeTask> task, const std::error_code& ec) {
   assert(task->fifo_);
   assert(!task->messages_.empty());
-  if (ec) {
-    SPDLOG_DEBUG("Failed to forward Message[message-id={}] to DLQ", task->messages_[0]->id());
-    task->next_step_ = NextStep::Forward;
-    task->schedule();
-  } else {
-    SPDLOG_DEBUG("Message[message-id={}] forwarded to DLQ", task->messages_[0]->id());
+
+  // Treat both success and invalid-receipt-handle as completion
+  if (!ec || ErrorCode::InvalidReceiptHandle == ec) {
+    SPDLOG_DEBUG("Message[message-id={}] is forwarded to DLQ", task->messages_[0]->id());
     task->pop();
     task->next_step_ = NextStep::Consume;
     task->submit();
+    return;
   }
+
+  SPDLOG_DEBUG("Failed to forward Message[message-id={}] to DLQ. Cause: {}.  Action: retry after 1s.",
+               task->messages_[0]->id(), ec.message());
+  task->next_step_ = NextStep::Forward;
+  task->schedule();
 }
 
 void ConsumeTask::process() {
@@ -199,6 +214,13 @@ void ConsumeTask::process() {
       svc->ack(*messages_[0], callback);
       break;
     }
+
+    case NextStep::Nack: {
+      auto callback = std::bind(&ConsumeTask::onNack, self, std::placeholders::_1);
+      svc->nack(*messages_[0], callback);
+      break;
+    }
+
     case NextStep::Forward: {
       assert(!messages_.empty());
       auto callback = std::bind(&ConsumeTask::onForward, self, std::placeholders::_1);
diff --git a/cpp/source/rocketmq/include/ClientImpl.h b/cpp/source/rocketmq/include/ClientImpl.h
index f0b29f7..78d2759 100644
--- a/cpp/source/rocketmq/include/ClientImpl.h
+++ b/cpp/source/rocketmq/include/ClientImpl.h
@@ -228,6 +228,14 @@ private:
 
   void doVerify(std::string target, std::string command_id, MessageConstPtr message);
 
+  /**
+   * ClientID is required to remain unique in the following scenarios:
+   *
+   * 1. Create multiple clients;
+   * 2. Restart the client program in container deployments;
+   *
+   * @return Unique Client-ID
+   */
   static std::string clientId();
 
   std::string metricServiceEndpoint() const;
diff --git a/cpp/source/rocketmq/include/ConsumeTask.h b/cpp/source/rocketmq/include/ConsumeTask.h
index 80ca971..38acfc3 100644
--- a/cpp/source/rocketmq/include/ConsumeTask.h
+++ b/cpp/source/rocketmq/include/ConsumeTask.h
@@ -41,6 +41,8 @@ enum class NextStep : std::uint8_t
    */
   Ack,
 
+  Nack,
+
   /**
    * @brief Forward the head, aka, messages_[0], to dead-letter-queue.
    */
diff --git a/cpp/third_party/opencensus/0.5.0-alpha/README.md b/cpp/third_party/opencensus/0.5.0-alpha/README.md
index 0eeb8d4..16dc0a1 100644
--- a/cpp/third_party/opencensus/0.5.0-alpha/README.md
+++ b/cpp/third_party/opencensus/0.5.0-alpha/README.md
@@ -1 +1,4 @@
-Based on commit-id: 3e6aa4c0fb31d2f39a2d38365483599ab50bef6d
\ No newline at end of file
+Based on commit-id: 3e6aa4c0fb31d2f39a2d38365483599ab50bef6d
+
+find . -name 'BUILD.bazel' -delete
+find . -name '*.bzl' -delete
\ No newline at end of file
diff --git a/cpp/third_party/opencensus/0.5.0-alpha/opencensus/BUILD b/cpp/third_party/opencensus/0.5.0-alpha/opencensus/BUILD
deleted file mode 100644
index 2c914a3..0000000
--- a/cpp/third_party/opencensus/0.5.0-alpha/opencensus/BUILD
+++ /dev/null
@@ -1,37 +0,0 @@
-# Copyright 2018, OpenCensus Authors
-#
-# 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.
-
-licenses(["notice"])  # Apache 2.0
-
-config_setting(
-    name = "llvm_compiler",
-    values = {
-        "compiler": "llvm",
-    },
-    visibility = [
-        ":__subpackages__",
-        "//examples:__subpackages__",
-    ],
-)
-
-config_setting(
-    name = "windows",
-    values = {
-        "cpu": "x64_windows",
-    },
-    visibility = [
-        ":__subpackages__",
-        "//examples:__subpackages__",
-    ],
-)
diff --git a/cpp/third_party/opencensus/0.5.0-alpha/opencensus/common/BUILD b/cpp/third_party/opencensus/0.5.0-alpha/opencensus/common/BUILD
deleted file mode 100644
index 64bed6c..0000000
--- a/cpp/third_party/opencensus/0.5.0-alpha/opencensus/common/BUILD
+++ /dev/null
@@ -1,28 +0,0 @@
-# OpenCensus C++ Common library.
-# See ../../README.md for details.
-#
-# Copyright 2018, OpenCensus Authors
-#
-# 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.
-
-load("//opencensus:copts.bzl", "DEFAULT_COPTS", "TEST_COPTS")
-
-licenses(["notice"])  # Apache 2.0
-
-package(default_visibility = ["//opencensus:__subpackages__"])
-
-cc_library(
-    name = "version",
-    hdrs = ["version.h"],
-    copts = DEFAULT_COPTS,
-)
diff --git a/cpp/third_party/opencensus/0.5.0-alpha/opencensus/common/internal/BUILD b/cpp/third_party/opencensus/0.5.0-alpha/opencensus/common/internal/BUILD
deleted file mode 100644
index 8afef74..0000000
--- a/cpp/third_party/opencensus/0.5.0-alpha/opencensus/common/internal/BUILD
+++ /dev/null
@@ -1,163 +0,0 @@
-# OpenCensus C++ internal libraries.
-#
-# Copyright 2017, OpenCensus Authors
-#
-# 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.
-
-load("//opencensus:copts.bzl", "DEFAULT_COPTS", "TEST_COPTS")
-
-licenses(["notice"])  # Apache 2.0
-
-package(default_visibility = ["//opencensus:__subpackages__"])
-
-cc_library(
-    name = "hostname",
-    srcs = ["hostname.cc"],
-    hdrs = ["hostname.h"],
-    copts = DEFAULT_COPTS,
-    deps = ["@com_google_absl//absl/strings"],
-)
-
-cc_library(
-    name = "random_lib",
-    srcs = ["random.cc"],
-    hdrs = ["random.h"],
-    copts = DEFAULT_COPTS,
-    deps = [
-        "@com_google_absl//absl/base:core_headers",
-        "@com_google_absl//absl/synchronization",
-        "@com_google_absl//absl/time",
-    ],
-)
-
-cc_library(
-    name = "stats_object",
-    hdrs = ["stats_object.h"],
-    copts = DEFAULT_COPTS,
-    deps = [
-        "@com_google_absl//absl/base:core_headers",
-        "@com_google_absl//absl/strings",
-        "@com_google_absl//absl/time",
-        "@com_google_absl//absl/types:span",
-    ],
-)
-
-cc_library(
-    name = "string_vector_hash",
-    hdrs = ["string_vector_hash.h"],
-    copts = DEFAULT_COPTS,
-    deps = ["@com_google_absl//absl/hash"],
-)
-
-cc_library(
-    name = "timestamp",
-    srcs = ["timestamp.cc"],
-    hdrs = ["timestamp.h"],
-    copts = DEFAULT_COPTS,
-    deps = [
-        "@com_google_absl//absl/time",
-        "@com_google_protobuf//:protobuf",
-    ],
-)
-
-cc_library(
-    name = "varint",
-    srcs = ["varint.cc"],
-    hdrs = ["varint.h"],
-    copts = DEFAULT_COPTS,
-    deps = ["@com_google_absl//absl/strings"],
-)
-
-# Tests
-# ========================================================================= #
-
-cc_test(
-    name = "hostname_test",
-    srcs = ["hostname_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":hostname",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_test(
-    name = "random_test",
-    srcs = ["random_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":random_lib",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_binary(
-    name = "random_benchmark",
-    testonly = 1,
-    srcs = ["random_benchmark.cc"],
-    copts = TEST_COPTS,
-    linkstatic = 1,
-    deps = [
-        ":random_lib",
-        "@com_github_google_benchmark//:benchmark",
-    ],
-)
-
-cc_test(
-    name = "stats_object_test",
-    srcs = ["stats_object_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":stats_object",
-        "@com_google_absl//absl/strings",
-        "@com_google_absl//absl/types:span",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_binary(
-    name = "timestamp_benchmark",
-    testonly = 1,
-    srcs = ["timestamp_benchmark.cc"],
-    copts = TEST_COPTS,
-    linkstatic = 1,
-    deps = [
-        ":timestamp",
-        "@com_github_google_benchmark//:benchmark",
-        "@com_google_absl//absl/time",
-        "@com_google_protobuf//:protobuf",
-    ],
-)
-
-cc_test(
-    name = "timestamp_test",
-    srcs = ["timestamp_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":timestamp",
-        "@com_google_absl//absl/time",
-        "@com_google_googletest//:gtest_main",
-        "@com_google_protobuf//:protobuf",
-    ],
-)
-
-cc_test(
-    name = "varint_test",
-    srcs = ["varint_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":varint",
-        "@com_google_absl//absl/strings",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
diff --git a/cpp/third_party/opencensus/0.5.0-alpha/opencensus/common/internal/grpc/BUILD b/cpp/third_party/opencensus/0.5.0-alpha/opencensus/common/internal/grpc/BUILD
deleted file mode 100644
index 51f43c8..0000000
--- a/cpp/third_party/opencensus/0.5.0-alpha/opencensus/common/internal/grpc/BUILD
+++ /dev/null
@@ -1,57 +0,0 @@
-# OpenCensus C++ internal libraries.
-#
-# Copyright 2018, OpenCensus Authors
-#
-# 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.
-
-load("//opencensus:copts.bzl", "DEFAULT_COPTS", "TEST_COPTS")
-
-licenses(["notice"])  # Apache 2.0
-
-package(default_visibility = ["//opencensus:__subpackages__"])
-
-cc_library(
-    name = "status",
-    srcs = ["status.cc"],
-    hdrs = ["status.h"],
-    copts = DEFAULT_COPTS,
-    deps = [
-        "@com_github_grpc_grpc//:grpc++",
-        "@com_google_absl//absl/strings",
-    ],
-)
-
-cc_library(
-    name = "with_user_agent",
-    srcs = ["with_user_agent.cc"],
-    hdrs = ["with_user_agent.h"],
-    copts = DEFAULT_COPTS,
-    deps = [
-        "//opencensus/common:version",
-        "@com_github_grpc_grpc//:grpc++",
-    ],
-)
-
-# Tests
-# ========================================================================= #
-
-cc_test(
-    name = "status_test",
-    srcs = ["status_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":status",
-        "@com_github_grpc_grpc//:grpc++",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
diff --git a/cpp/third_party/opencensus/0.5.0-alpha/opencensus/context/BUILD b/cpp/third_party/opencensus/0.5.0-alpha/opencensus/context/BUILD
deleted file mode 100644
index e60ec92..0000000
--- a/cpp/third_party/opencensus/0.5.0-alpha/opencensus/context/BUILD
+++ /dev/null
@@ -1,86 +0,0 @@
-# OpenCensus C++ Context library.
-# See context.h for details.
-#
-# Copyright 2018, OpenCensus Authors
-#
-# 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.
-
-load("//opencensus:copts.bzl", "DEFAULT_COPTS", "TEST_COPTS")
-
-licenses(["notice"])  # Apache 2.0
-
-package(default_visibility = ["//visibility:private"])
-
-cc_library(
-    name = "context",
-    srcs = [
-        "internal/context.cc",
-        "internal/with_context.cc",
-    ],
-    hdrs = [
-        "context.h",
-        "with_context.h",
-    ],
-    copts = DEFAULT_COPTS,
-    visibility = ["//visibility:public"],
-    deps = [
-        "//opencensus/tags",
-        "//opencensus/trace",
-        "@com_google_absl//absl/strings",
-    ],
-)
-
-# Tests
-# ========================================================================= #
-
-cc_test(
-    name = "context_test",
-    srcs = ["internal/context_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":context",
-        "//opencensus/tags",
-        "//opencensus/tags:context_util",
-        "//opencensus/tags:with_tag_map",
-        "//opencensus/trace",
-        "//opencensus/trace:context_util",
-        "//opencensus/trace:span_context",
-        "//opencensus/trace:with_span",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_test(
-    name = "with_context_test",
-    srcs = ["internal/with_context_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":context",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-# Benchmarks
-# ========================================================================= #
-
-cc_binary(
-    name = "context_benchmark",
-    testonly = 1,
-    srcs = ["internal/context_benchmark.cc"],
-    copts = TEST_COPTS,
-    linkstatic = 1,
-    deps = [
-        ":context",
-        "@com_github_google_benchmark//:benchmark",
-    ],
-)
diff --git a/cpp/third_party/opencensus/0.5.0-alpha/opencensus/copts.bzl b/cpp/third_party/opencensus/0.5.0-alpha/opencensus/copts.bzl
deleted file mode 100644
index 1947e1d..0000000
--- a/cpp/third_party/opencensus/0.5.0-alpha/opencensus/copts.bzl
+++ /dev/null
@@ -1,47 +0,0 @@
-# Copyright 2018, OpenCensus Authors
-#
-# 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.
-
-"""Compiler options for OpenCensus.
-
-Flags specified here must not impact ABI. Code compiled with and without these
-opts will be linked together, and in some cases headers compiled with and
-without these options will be part of the same program.
-
-We use the same flags as absl, plus turn some warnings into errors.
-"""
-
-load(
-    "@com_google_absl//absl:copts/GENERATED_copts.bzl",
-    "ABSL_GCC_FLAGS",
-    "ABSL_GCC_TEST_FLAGS",
-    "ABSL_LLVM_FLAGS",
-    "ABSL_LLVM_TEST_FLAGS",
-    "ABSL_MSVC_FLAGS",
-    "ABSL_MSVC_TEST_FLAGS",
-)
-
-WERROR = ["-Werror=return-type", "-Werror=switch"]
-WARN_FLAGS = ["-Wno-sign-compare"]
-
-DEFAULT_COPTS = select({
-    "//opencensus:llvm_compiler": ABSL_LLVM_FLAGS + WERROR + WARN_FLAGS,
-    "//opencensus:windows": ABSL_MSVC_FLAGS,
-    "//conditions:default": ABSL_GCC_FLAGS + WERROR + WARN_FLAGS,
-})
-
-TEST_COPTS = DEFAULT_COPTS + select({
-    "//opencensus:llvm_compiler": ABSL_LLVM_TEST_FLAGS + WERROR + WARN_FLAGS,
-    "//opencensus:windows": ABSL_MSVC_TEST_FLAGS,
-    "//conditions:default": ABSL_GCC_TEST_FLAGS + WERROR + WARN_FLAGS,
-})
diff --git a/cpp/third_party/opencensus/0.5.0-alpha/opencensus/curl.bzl b/cpp/third_party/opencensus/0.5.0-alpha/opencensus/curl.bzl
deleted file mode 100644
index b83d054..0000000
--- a/cpp/third_party/opencensus/0.5.0-alpha/opencensus/curl.bzl
+++ /dev/null
@@ -1,201 +0,0 @@
-# Copyright 2018, OpenCensus Authors
-#
-# 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.
-
-# Compiler options for building libcurl.
-
-BASE_CURL_COPTS = [
-    # Disable everything else except HTTP protocol.
-    "-DHTTP_ONLY=1",
-    "-DENABLE_IPV6=1",
-    "-DGETHOSTNAME_TYPE_ARG2=size_t",
-    "-DGETSERVBYPORT_R_ARGS=6",
-    "-DGETSERVBYPORT_R_BUFSIZE=4096",
-    "-DHAVE_ALARM=1",
-    "-DHAVE_ALLOCA_H=1",
-    "-DHAVE_ARPA_INET_H=1",
-    "-DHAVE_ARPA_TFTP_H=1",
-    "-DHAVE_ASSERT_H=1",
-    "-DHAVE_BASENAME=1",
-    "-DHAVE_BOOL_T=1",
-    "-DHAVE_CLOCK_GETTIME_MONOTONIC=1",
-    "-DHAVE_CONNECT=1",
-    "-DHAVE_DLFCN_H=1",
-    "-DHAVE_ENGINE_LOAD_BUILTIN_ENGINES=1",
-    "-DHAVE_ERRNO_H=1",
-    "-DHAVE_FCNTL=1",
-    "-DHAVE_FCNTL_H=1",
-    "-DHAVE_FCNTL_O_NONBLOCK=1",
-    "-DHAVE_FDOPEN=1",
-    "-DHAVE_FREEADDRINFO=1",
-    "-DHAVE_FREEIFADDRS=1",
-    "-DHAVE_FSETXATTR=1",
-    "-DHAVE_FSETXATTR_5=1",
-    "-DHAVE_FTRUNCATE=1",
-    "-DHAVE_GAI_STRERROR=1",
-    "-DHAVE_GETADDRINFO=1",
-    "-DHAVE_GETADDRINFO_THREADSAFE=1",
-    "-DHAVE_GETEUID=1",
-    "-DHAVE_GETHOSTBYADDR=1",
-    "-DHAVE_GETHOSTBYADDR_R=1",
-    "-DHAVE_GETHOSTBYADDR_R_8=1",
-    "-DHAVE_GETHOSTBYNAME=1",
-    "-DHAVE_GETHOSTBYNAME_R=1",
-    "-DHAVE_GETHOSTBYNAME_R_6=1",
-    "-DHAVE_GETHOSTNAME=1",
-    "-DHAVE_GETIFADDRS=1",
-    "-DHAVE_GETNAMEINFO=1",
-    "-DHAVE_GETPPID=1",
-    "-DHAVE_GETPWUID=1",
-    "-DHAVE_GETPWUID_R=1",
-    "-DHAVE_GETRLIMIT=1",
-    "-DHAVE_GETSERVBYPORT_R=1",
-    "-DHAVE_GETTIMEOFDAY=1",
-    "-DHAVE_GMTIME_R=1",
-    "-DHAVE_IFADDRS_H=1",
-    "-DHAVE_IF_NAMETOINDEX=1",
-    "-DHAVE_INET_NTOP=1",
-    "-DHAVE_INET_PTON=1",
-    "-DHAVE_INTTYPES_H=1",
-    "-DHAVE_IOCTL=1",
-    "-DHAVE_IOCTL_FIONBIO=1",
-    "-DHAVE_IOCTL_SIOCGIFADDR=1",
-    "-DHAVE_LIBGEN_H=1",
-    "-DHAVE_LL=1",
-    "-DHAVE_LOCALE_H=1",
-    "-DHAVE_LOCALTIME_R=1",
-    "-DHAVE_LONGLONG=1",
-    "-DHAVE_MALLOC_H=1",
-    "-DHAVE_MEMORY_H=1",
-    "-DHAVE_NETDB_H=1",
-    "-DHAVE_NETINET_IN_H=1",
-    "-DHAVE_NETINET_TCP_H=1",
-    "-DHAVE_NET_IF_H=1",
-    "-DHAVE_PIPE=1",
-    "-DHAVE_POLL=1",
-    "-DHAVE_POLL_FINE=1",
-    "-DHAVE_POLL_H=1",
-    "-DHAVE_POSIX_STRERROR_R=1",
-    "-DHAVE_PTHREAD_H=1",
-    "-DHAVE_PWD_H=1",
-    "-DHAVE_RECV=1",
-    "-DHAVE_SELECT=1",
-    "-DHAVE_SEND=1",
-    "-DHAVE_SETJMP_H=1",
-    "-DHAVE_SETLOCALE=1",
-    "-DHAVE_SETRLIMIT=1",
-    "-DHAVE_SETSOCKOPT=1",
-    "-DHAVE_SGTTY_H=1",
-    "-DHAVE_SIGACTION=1",
-    "-DHAVE_SIGINTERRUPT=1",
-    "-DHAVE_SIGNAL=1",
-    "-DHAVE_SIGNAL_H=1",
-    "-DHAVE_SIGSETJMP=1",
-    "-DHAVE_SIG_ATOMIC_T=1",
-    "-DHAVE_SOCKADDR_IN6_SIN6_SCOPE_ID=1",
-    "-DHAVE_SOCKET=1",
-    "-DHAVE_SOCKETPAIR=1",
-    "-DHAVE_STDBOOL_H=1",
-    "-DHAVE_STDINT_H=1",
-    "-DHAVE_STDIO_H=1",
-    "-DHAVE_STDLIB_H=1",
-    "-DHAVE_STRCASECMP=1",
-    "-DHAVE_STRDUP=1",
-    "-DHAVE_STRERROR_R=1",
-    "-DHAVE_STRINGS_H=1",
-    "-DHAVE_STRING_H=1",
-    "-DHAVE_STRNCASECMP=1",
-    "-DHAVE_STRSTR=1",
-    "-DHAVE_STRTOK_R=1",
-    "-DHAVE_STRTOLL=1",
-    "-DHAVE_STRUCT_SOCKADDR_STORAGE=1",
-    "-DHAVE_STRUCT_TIMEVAL=1",
-    "-DHAVE_SYS_IOCTL_H=1",
-    "-DHAVE_SYS_PARAM_H=1",
-    "-DHAVE_SYS_POLL_H=1",
-    "-DHAVE_SYS_RESOURCE_H=1",
-    "-DHAVE_SYS_SELECT_H=1",
-    "-DHAVE_SYS_SOCKET_H=1",
-    "-DHAVE_SYS_STAT_H=1",
-    "-DHAVE_SYS_TIME_H=1",
-    "-DHAVE_SYS_TYPES_H=1",
-    "-DHAVE_SYS_UIO_H=1",
-    "-DHAVE_SYS_UN_H=1",
-    "-DHAVE_SYS_WAIT_H=1",
-    "-DHAVE_SYS_XATTR_H=1",
-    "-DHAVE_TERMIOS_H=1",
-    "-DHAVE_TERMIO_H=1",
-    "-DHAVE_TIME_H=1",
-    "-DHAVE_UNISTD_H=1",
-    "-DHAVE_UTIME=1",
-    "-DHAVE_UTIMES=1",
-    "-DHAVE_UTIME_H=1",
-    "-DHAVE_VARIADIC_MACROS_C99=1",
-    "-DHAVE_VARIADIC_MACROS_GCC=1",
-    "-DHAVE_WRITABLE_ARGV=1",
-    "-DHAVE_WRITEV=1",
-    "-DRECV_TYPE_ARG1=int",
-    "-DRECV_TYPE_ARG2=void*",
-    "-DRECV_TYPE_ARG3=size_t",
-    "-DRECV_TYPE_ARG4=int",
-    "-DRECV_TYPE_RETV=ssize_t",
-    "-DRETSIGTYPE=void",
-    "-DSELECT_QUAL_ARG5=",
-    "-DSELECT_TYPE_ARG1=int",
-    "-DSELECT_TYPE_ARG234=fd_set*",
-    "-DSELECT_TYPE_RETV=int",
-    "-DSEND_QUAL_ARG2=const",
-    "-DSEND_TYPE_ARG1=int",
-    "-DSEND_TYPE_ARG2=void*",
-    "-DSEND_TYPE_ARG3=size_t",
-    "-DSEND_TYPE_ARG4=int",
-    "-DSEND_TYPE_RETV=ssize_t",
-    "-DSIZEOF_CURL_OFF_T=8",
-    "-DSIZEOF_INT=4",
-    "-DSIZEOF_LONG=8",
-    "-DSIZEOF_OFF_T=8",
-    "-DSIZEOF_SHORT=2",
-    "-DSIZEOF_SIZE_T=8",
-    "-DSIZEOF_TIME_T=8",
-    "-DSTDC_HEADERS=1",
-    "-DSTRERROR_R_TYPE_ARG3=size_t",
-    "-DTIME_WITH_SYS_TIME=1",
-    "-DUSE_THREADS_POSIX=1",
-    "-DUSE_UNIX_SOCKETS=1",
-
-    # Extra defines needed by curl
-    "-DBUILDING_LIBCURL",
-    "-DCURL_HIDDEN_SYMBOLS",
-]
-
-LINUX_CURL_COPTS = [
-    "-DHAVE_LINUX_TCP_H=1",
-    "-DHAVE_MSG_NOSIGNAL=1",
-]
-
-CURL_COPTS = select({
-    "//:windows": [
-        # Disable everything else except HTTP protocol.
-        "/DHTTP_ONLY=1",
-        "/DCURL_STATICLIB",
-        "/DWIN32",
-        "/DBUILDING_LIBCURL",
-        "/DUSE_WIN32_IDN",
-        "/DWANT_IDN_PROTOTYPES",
-        "/DUSE_IPV6",
-        "/DUSE_WINDOWS_SSPI",
-        "/DUSE_SCHANNEL",
-    ],
-    "//:osx": BASE_CURL_COPTS,
-    "//conditions:default": BASE_CURL_COPTS + LINUX_CURL_COPTS,
-})
diff --git a/cpp/third_party/opencensus/0.5.0-alpha/opencensus/exporters/stats/stdout/BUILD b/cpp/third_party/opencensus/0.5.0-alpha/opencensus/exporters/stats/stdout/BUILD
deleted file mode 100644
index 4c41c8b..0000000
--- a/cpp/third_party/opencensus/0.5.0-alpha/opencensus/exporters/stats/stdout/BUILD
+++ /dev/null
@@ -1,45 +0,0 @@
-# Copyright 2018, OpenCensus Authors
-#
-# 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.
-
-load("//opencensus:copts.bzl", "DEFAULT_COPTS", "TEST_COPTS")
-
-licenses(["notice"])  # Apache License 2.0
-
-package(default_visibility = ["//visibility:private"])
-
-cc_library(
-    name = "stdout_exporter",
-    srcs = ["internal/stdout_exporter.cc"],
-    hdrs = ["stdout_exporter.h"],
-    copts = DEFAULT_COPTS,
-    visibility = ["//visibility:public"],
-    deps = [
-        "//opencensus/stats",
-        "@com_google_absl//absl/memory",
-        "@com_google_absl//absl/strings",
-        "@com_google_absl//absl/time",
-    ],
-)
-
-cc_test(
-    name = "stdout_exporter_test",
-    srcs = ["internal/stdout_exporter_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":stdout_exporter",
-        "//opencensus/stats",
-        "//opencensus/stats:test_utils",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
diff --git a/cpp/third_party/opencensus/0.5.0-alpha/opencensus/exporters/trace/ocagent/BUILD b/cpp/third_party/opencensus/0.5.0-alpha/opencensus/exporters/trace/ocagent/BUILD
deleted file mode 100644
index 837f64a..0000000
--- a/cpp/third_party/opencensus/0.5.0-alpha/opencensus/exporters/trace/ocagent/BUILD
+++ /dev/null
@@ -1,56 +0,0 @@
-# Copyright 2019, OpenCensus Authors
-#
-# 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.
-
-load("//opencensus:copts.bzl", "DEFAULT_COPTS", "TEST_COPTS")
-
-licenses(["notice"])  # Apache License 2.0
-
-package(default_visibility = ["//visibility:private"])
-
-# Libraries
-# ========================================================================= #
-
-cc_library(
-    name = "ocagent_exporter",
-    srcs = ["internal/ocagent_exporter.cc"],
-    hdrs = ["ocagent_exporter.h"],
-    copts = DEFAULT_COPTS,
-    visibility = ["//visibility:public"],
-    deps = [
-        "//opencensus/common:version",
-        "//opencensus/common/internal:hostname",
-        "//opencensus/common/internal:timestamp",
-        "//opencensus/common/internal/grpc:status",
-        "//opencensus/common/internal/grpc:with_user_agent",
-        "//opencensus/trace",
-        "@com_github_grpc_grpc//:grpc++",
-        "@com_google_absl//absl/memory",
-        "@com_google_absl//absl/strings",
-        "@com_google_absl//absl/time",
-        "@opencensus_proto//opencensus/proto/agent/trace/v1:trace_service_grpc_cc",
-        "@opencensus_proto//opencensus/proto/agent/trace/v1:trace_service_proto_cc",
-    ],
-)
-
-cc_test(
-    name = "ocagent_exporter_test",
-    srcs = ["internal/ocagent_exporter_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":ocagent_exporter",
-        "//opencensus/trace",
-        "@com_google_absl//absl/time",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
diff --git a/cpp/third_party/opencensus/0.5.0-alpha/opencensus/exporters/trace/stackdriver/BUILD b/cpp/third_party/opencensus/0.5.0-alpha/opencensus/exporters/trace/stackdriver/BUILD
deleted file mode 100644
index 297fb7d..0000000
--- a/cpp/third_party/opencensus/0.5.0-alpha/opencensus/exporters/trace/stackdriver/BUILD
+++ /dev/null
@@ -1,44 +0,0 @@
-# Copyright 2018, OpenCensus Authors
-#
-# 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.
-
-load("//opencensus:copts.bzl", "DEFAULT_COPTS")
-
-licenses(["notice"])  # Apache License 2.0
-
-package(default_visibility = ["//visibility:private"])
-
-# Libraries
-# ========================================================================= #
-
-cc_library(
-    name = "stackdriver_exporter",
-    srcs = ["internal/stackdriver_exporter.cc"],
-    hdrs = ["stackdriver_exporter.h"],
-    copts = DEFAULT_COPTS,
-    visibility = ["//visibility:public"],
-    deps = [
-        "//opencensus/common:version",
-        "//opencensus/common/internal:timestamp",
-        "//opencensus/common/internal/grpc:status",
-        "//opencensus/common/internal/grpc:with_user_agent",
-        "//opencensus/trace",
-        "@com_github_grpc_grpc//:grpc++",
-        "@com_google_absl//absl/base:core_headers",
-        "@com_google_absl//absl/memory",
-        "@com_google_absl//absl/strings",
-        "@com_google_absl//absl/time",
-        "@com_google_googleapis//google/devtools/cloudtrace/v2:cloudtrace_cc_grpc",
-        "@com_google_googleapis//google/devtools/cloudtrace/v2:cloudtrace_cc_proto",
-    ],
-)
diff --git a/cpp/third_party/opencensus/0.5.0-alpha/opencensus/exporters/trace/stdout/BUILD b/cpp/third_party/opencensus/0.5.0-alpha/opencensus/exporters/trace/stdout/BUILD
deleted file mode 100644
index 6c44d13..0000000
--- a/cpp/third_party/opencensus/0.5.0-alpha/opencensus/exporters/trace/stdout/BUILD
+++ /dev/null
@@ -1,49 +0,0 @@
-# Copyright 2018, OpenCensus Authors
-#
-# 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.
-
-load("//opencensus:copts.bzl", "DEFAULT_COPTS", "TEST_COPTS")
-
-licenses(["notice"])  # Apache License 2.0
-
-package(default_visibility = ["//visibility:public"])
-
-# Libraries
-# ========================================================================= #
-
-cc_library(
-    name = "stdout_exporter",
-    srcs = ["internal/stdout_exporter.cc"],
-    hdrs = ["stdout_exporter.h"],
-    copts = DEFAULT_COPTS,
-    deps = [
-        "//opencensus/trace",
-        "@com_google_absl//absl/base:core_headers",
-        "@com_google_absl//absl/memory",
-    ],
-)
-
-# Tests
-# ========================================================================= #
-
-cc_test(
-    name = "stdout_exporter_test",
-    srcs = ["internal/stdout_exporter_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":stdout_exporter",
-        "//opencensus/trace",
-        "@com_google_absl//absl/time",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
diff --git a/cpp/third_party/opencensus/0.5.0-alpha/opencensus/exporters/trace/zipkin/BUILD b/cpp/third_party/opencensus/0.5.0-alpha/opencensus/exporters/trace/zipkin/BUILD
deleted file mode 100644
index c92c2ae..0000000
--- a/cpp/third_party/opencensus/0.5.0-alpha/opencensus/exporters/trace/zipkin/BUILD
+++ /dev/null
@@ -1,56 +0,0 @@
-# Copyright 2018, OpenCensus Authors
-#
-# 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.
-
-load("//opencensus:copts.bzl", "DEFAULT_COPTS", "TEST_COPTS")
-
-licenses(["notice"])  # Apache License 2.0
-
-package(default_visibility = ["//visibility:public"])
-
-# Libraries
-# ========================================================================= #
-
-cc_library(
-    name = "zipkin_exporter",
-    srcs = [
-        "internal/zipkin_exporter.cc",
-    ],
-    hdrs = [
-        "zipkin_exporter.h",
-    ],
-    copts = DEFAULT_COPTS,
-    deps = [
-        "//opencensus/trace",
-        "@com_github_curl//:curl",
-        "@com_github_tencent_rapidjson//:rapidjson",
-        "@com_google_absl//absl/memory",
-        "@com_google_absl//absl/strings",
-        "@com_google_absl//absl/time",
-    ],
-)
-
-# Tests
-# ========================================================================= #
-
-cc_test(
-    name = "zipkin_exporter_test",
-    srcs = ["internal/zipkin_exporter_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":zipkin_exporter",
-        "//opencensus/trace",
-        "@com_google_absl//absl/time",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
diff --git a/cpp/third_party/opencensus/0.5.0-alpha/opencensus/stats/BUILD b/cpp/third_party/opencensus/0.5.0-alpha/opencensus/stats/BUILD
deleted file mode 100644
index b6f0fd0..0000000
--- a/cpp/third_party/opencensus/0.5.0-alpha/opencensus/stats/BUILD
+++ /dev/null
@@ -1,264 +0,0 @@
-# OpenCensus C++ Stats library.
-#
-# Copyright 2017, OpenCensus Authors
-#
-# 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.
-
-load("//opencensus:copts.bzl", "DEFAULT_COPTS", "TEST_COPTS")
-
-licenses(["notice"])  # Apache License 2.0
-
-package(default_visibility = ["//visibility:private"])
-
-# The public stats API.
-cc_library(
-    name = "stats",
-    hdrs = [
-        "aggregation.h",
-        "bucket_boundaries.h",
-        "distribution.h",
-        "measure.h",
-        "measure_descriptor.h",
-        "measure_registry.h",
-        "recording.h",
-        "stats.h",
-        "stats_exporter.h",
-        "tag_key.h",
-        "tag_set.h",
-        "view.h",
-        "view_data.h",
-        "view_descriptor.h",
-    ],
-    copts = DEFAULT_COPTS,
-    visibility = ["//visibility:public"],
-    deps = [
-        ":core",
-        ":recording",
-        "//opencensus/common/internal:stats_object",
-        "//opencensus/common/internal:string_vector_hash",
-        "//opencensus/tags",
-        "@com_google_absl//absl/base:core_headers",
-        "@com_google_absl//absl/strings",
-        "@com_google_absl//absl/time",
-    ],
-)
-
-# Test-only utilities.
-cc_library(
-    name = "test_utils",
-    testonly = 1,
-    srcs = ["testing/test_utils.cc"],
-    hdrs = ["testing/test_utils.h"],
-    copts = DEFAULT_COPTS,
-    visibility = ["//visibility:public"],
-    deps = [
-        ":core",
-        "@com_google_absl//absl/memory",
-        "@com_google_absl//absl/time",
-    ],
-)
-
-cc_library(
-    name = "core",
-    srcs = [
-        "internal/aggregation.cc",
-        "internal/aggregation_window.cc",
-        "internal/bucket_boundaries.cc",
-        "internal/delta_producer.cc",
-        "internal/distribution.cc",
-        "internal/measure.cc",
-        "internal/measure_data.cc",
-        "internal/measure_descriptor.cc",
-        "internal/measure_registry.cc",
-        "internal/measure_registry_impl.cc",
-        "internal/set_aggregation_window.cc",
-        "internal/stats_exporter.cc",
-        "internal/stats_manager.cc",
-        "internal/view.cc",
-        "internal/view_data.cc",
-        "internal/view_data_impl.cc",
-        "internal/view_descriptor.cc",
-    ],
-    hdrs = [
-        "aggregation.h",
-        "bucket_boundaries.h",
-        "distribution.h",
-        "internal/aggregation_window.h",
-        "internal/delta_producer.h",
-        "internal/measure_data.h",
-        "internal/measure_registry_impl.h",
-        "internal/set_aggregation_window.h",
-        "internal/stats_exporter_impl.h",
-        "internal/stats_manager.h",
-        "internal/view_data_impl.h",
-        "measure.h",
-        "measure_descriptor.h",
-        "measure_registry.h",
-        "stats_exporter.h",
-        "tag_key.h",
-        "tag_set.h",
-        "view.h",
-        "view_data.h",
-        "view_descriptor.h",
-    ],
-    copts = DEFAULT_COPTS,
-    deps = [
-        "//opencensus/common/internal:stats_object",
-        "//opencensus/common/internal:string_vector_hash",
-        "//opencensus/tags",
-        "@com_google_absl//absl/base:core_headers",
-        "@com_google_absl//absl/memory",
-        "@com_google_absl//absl/strings",
-        "@com_google_absl//absl/synchronization",
-        "@com_google_absl//absl/time",
-        "@com_google_absl//absl/types:span",
-    ],
-)
-
-cc_library(
-    name = "recording",
-    srcs = ["internal/recording.cc"],
-    hdrs = ["recording.h"],
-    copts = DEFAULT_COPTS,
-    deps = [
-        ":core",
-        "//opencensus/tags",
-        "//opencensus/tags:context_util",
-    ],
-)
-
-# Tests
-# ========================================================================= #
-
-cc_test(
-    name = "debug_string_test",
-    srcs = ["internal/debug_string_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":core",
-        "@com_google_absl//absl/time",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_test(
-    name = "distribution_test",
-    srcs = ["internal/distribution_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":core",
-        ":test_utils",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_test(
-    name = "bucket_boundaries_test",
-    srcs = ["internal/bucket_boundaries_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":core",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_test(
-    name = "measure_data_test",
-    size = "small",
-    srcs = ["internal/measure_data_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":core",
-        ":test_utils",
-        "@com_google_absl//absl/types:span",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_test(
-    name = "measure_registry_test",
-    srcs = ["internal/measure_registry_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":core",
-        "@com_google_absl//absl/strings",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_test(
-    name = "stats_exporter_test",
-    srcs = ["internal/stats_exporter_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":core",
-        "@com_google_absl//absl/memory",
-        "@com_google_absl//absl/time",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_test(
-    name = "stats_manager_test",
-    srcs = ["internal/stats_manager_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":core",
-        ":recording",
-        ":test_utils",
-        "//opencensus/tags",
-        "//opencensus/tags:with_tag_map",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_test(
-    name = "view_data_impl_test",
-    srcs = ["internal/view_data_impl_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":core",
-        "@com_google_absl//absl/time",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_test(
-    name = "view_data_test",
-    srcs = ["internal/view_data_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":core",
-        ":test_utils",
-        "@com_google_absl//absl/time",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-# Benchmarks
-# ========================================================================= #
-cc_binary(
-    name = "stats_manager_benchmark",
-    testonly = 1,
-    srcs = ["internal/stats_manager_benchmark.cc"],
-    copts = TEST_COPTS,
-    linkstatic = 1,
-    deps = [
-        ":core",
-        ":recording",
-        "@com_github_google_benchmark//:benchmark",
-        "@com_google_absl//absl/memory",
-        "@com_google_absl//absl/strings",
-        "@com_google_absl//absl/time",
-    ],
-)
diff --git a/cpp/third_party/opencensus/0.5.0-alpha/opencensus/stats/examples/BUILD b/cpp/third_party/opencensus/0.5.0-alpha/opencensus/stats/examples/BUILD
deleted file mode 100644
index c8e918e..0000000
--- a/cpp/third_party/opencensus/0.5.0-alpha/opencensus/stats/examples/BUILD
+++ /dev/null
@@ -1,52 +0,0 @@
-# OpenCensus C++ Stats library: Example usage.
-#
-# Copyright 2017, OpenCensus Authors
-#
-# 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.
-
-load("//opencensus:copts.bzl", "TEST_COPTS")
-
-licenses(["notice"])  # Apache 2.0
-
-package(
-    default_testonly = 1,
-    default_visibility = ["//visibility:private"],
-)
-
-# Examples are defined as tests in order to make sure the code compiles and
-# doesn't crash.
-
-cc_test(
-    name = "view_and_recording_example",
-    srcs = ["view_and_recording_example.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        "//opencensus/stats",
-        "@com_google_absl//absl/time",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_test(
-    name = "exporter_example",
-    srcs = ["exporter_example.cc"],
-    copts = TEST_COPTS,
-    tags = ["noci"],  # This runs slowly and is not very useful as a test.
-    deps = [
-        "//opencensus/stats",
-        "@com_google_absl//absl/memory",
-        "@com_google_absl//absl/strings",
-        "@com_google_absl//absl/time",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
diff --git a/cpp/third_party/opencensus/0.5.0-alpha/opencensus/tags/BUILD b/cpp/third_party/opencensus/0.5.0-alpha/opencensus/tags/BUILD
deleted file mode 100644
index 8506029..0000000
--- a/cpp/third_party/opencensus/0.5.0-alpha/opencensus/tags/BUILD
+++ /dev/null
@@ -1,199 +0,0 @@
-# OpenCensus C++ Tags library.
-#
-# Copyright 2018, OpenCensus Authors
-#
-# 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.
-
-load("//opencensus:copts.bzl", "DEFAULT_COPTS", "TEST_COPTS")
-load("//bazel:cc_fuzz_target.bzl", "cc_fuzz_target")
-
-licenses(["notice"])  # Apache License 2.0
-
-package(default_visibility = ["//visibility:private"])
-
-# The public tags API.
-cc_library(
-    name = "tags",
-    srcs = [
-        "internal/tag_key.cc",
-        "internal/tag_map.cc",
-    ],
-    hdrs = [
-        "tag_key.h",
-        "tag_map.h",
-    ],
-    copts = DEFAULT_COPTS,
-    visibility = ["//visibility:public"],
-    deps = [
-        "@com_google_absl//absl/base:core_headers",
-        "@com_google_absl//absl/hash",
-        "@com_google_absl//absl/memory",
-        "@com_google_absl//absl/strings",
-        "@com_google_absl//absl/synchronization",
-    ],
-)
-
-cc_library(
-    name = "context_util",
-    srcs = ["internal/context_util.cc"],
-    hdrs = ["context_util.h"],
-    copts = DEFAULT_COPTS,
-    visibility = ["//visibility:public"],
-    deps = [
-        ":tags",
-        "//opencensus/context",
-    ],
-)
-
-cc_library(
-    name = "grpc_tags_bin",
-    srcs = ["internal/grpc_tags_bin.cc"],
-    hdrs = ["propagation/grpc_tags_bin.h"],
-    copts = DEFAULT_COPTS,
-    visibility = ["//visibility:public"],
-    deps = [
-        ":tags",
-        "//opencensus/common/internal:varint",
-        "@com_google_absl//absl/strings",
-    ],
-)
-
-cc_library(
-    name = "with_tag_map",
-    srcs = ["internal/with_tag_map.cc"],
-    hdrs = ["with_tag_map.h"],
-    copts = DEFAULT_COPTS,
-    visibility = ["//visibility:public"],
-    deps = [
-        ":tags",
-        "//opencensus/context",
-    ],
-)
-
-# Tests
-# ========================================================================= #
-
-cc_test(
-    name = "context_util_test",
-    srcs = ["internal/context_util_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":context_util",
-        ":tags",
-        ":with_tag_map",
-        "//opencensus/context",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_test(
-    name = "grpc_tags_bin_test",
-    srcs = ["internal/grpc_tags_bin_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":grpc_tags_bin",
-        ":tags",
-        "@com_google_absl//absl/strings",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_test(
-    name = "tag_key_test",
-    srcs = ["internal/tag_key_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":tags",
-        "@com_google_absl//absl/hash",
-        "@com_google_absl//absl/strings",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_test(
-    name = "tag_map_test",
-    srcs = ["internal/tag_map_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":tags",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_test(
-    name = "with_tag_map_test",
-    srcs = ["internal/with_tag_map_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":tags",
-        ":with_tag_map",
-        "//opencensus/context",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-# Benchmarks
-# ========================================================================= #
-
-cc_binary(
-    name = "grpc_tags_bin_benchmark",
-    testonly = 1,
-    srcs = ["internal/grpc_tags_bin_benchmark.cc"],
-    copts = TEST_COPTS,
-    linkstatic = 1,
-    deps = [
-        ":grpc_tags_bin",
-        ":tags",
-        "@com_github_google_benchmark//:benchmark",
-        "@com_google_absl//absl/strings",
-    ],
-)
-
-cc_binary(
-    name = "tag_map_benchmark",
-    testonly = 1,
-    srcs = ["internal/tag_map_benchmark.cc"],
-    copts = TEST_COPTS,
-    linkstatic = 1,
-    deps = [
-        ":tags",
-        "@com_github_google_benchmark//:benchmark",
-        "@com_google_absl//absl/strings",
-    ],
-)
-
-cc_binary(
-    name = "with_tag_map_benchmark",
-    testonly = 1,
-    srcs = ["internal/with_tag_map_benchmark.cc"],
-    copts = TEST_COPTS,
-    linkstatic = 1,
-    deps = [
-        ":tags",
-        ":with_tag_map",
-        "@com_github_google_benchmark//:benchmark",
-    ],
-)
-
-# Fuzzers
-# ========================================================================= #
-
-cc_fuzz_target(
-    name = "grpc_tags_bin_fuzzer",
-    srcs = ["internal/grpc_tags_bin_fuzzer.cc"],
-    corpus = glob(["internal/grpc_tags_bin_corpus/*"]),
-    deps = [
-        ":grpc_tags_bin",
-        "@com_google_absl//absl/strings",
-    ],
-)
diff --git a/cpp/third_party/opencensus/0.5.0-alpha/opencensus/trace/BUILD b/cpp/third_party/opencensus/0.5.0-alpha/opencensus/trace/BUILD
deleted file mode 100644
index 5215dfc..0000000
--- a/cpp/third_party/opencensus/0.5.0-alpha/opencensus/trace/BUILD
+++ /dev/null
@@ -1,601 +0,0 @@
-# OpenCensus C++ Trace library.
-# See ../../README.md for details.
-#
-# Copyright 2017, OpenCensus Authors
-#
-# 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.
-
-load("//opencensus:copts.bzl", "DEFAULT_COPTS", "TEST_COPTS")
-load("//bazel:cc_fuzz_target.bzl", "cc_fuzz_target")
-
-licenses(["notice"])  # Apache 2.0
-
-package(default_visibility = ["//visibility:private"])
-
-cc_library(
-    name = "trace",
-    srcs = [
-        "internal/annotation.cc",
-        "internal/attribute_list.cc",
-        "internal/attribute_value.cc",
-        "internal/attribute_value_ref.cc",
-        "internal/event_with_time.h",
-        "internal/link.cc",
-        "internal/local_span_store.cc",
-        "internal/local_span_store_impl.cc",
-        "internal/message_event.cc",
-        "internal/running_span_store.cc",
-        "internal/running_span_store_impl.cc",
-        "internal/sampler.cc",
-        "internal/span.cc",
-        "internal/span_data.cc",
-        "internal/span_exporter.cc",
-        "internal/span_exporter_impl.cc",
-        "internal/span_impl.cc",
-        "internal/status.cc",
-        "internal/trace_config.cc",
-        "internal/trace_config_impl.cc",
-    ],
-    hdrs = [
-        "attribute_value_ref.h",
-        "exporter/annotation.h",
-        "exporter/attribute_value.h",
-        "exporter/link.h",
-        "exporter/message_event.h",
-        "exporter/span_data.h",
-        "exporter/span_exporter.h",
-        "exporter/status.h",
-        "internal/attribute_list.h",
-        "internal/local_span_store.h",
-        "internal/local_span_store_impl.h",
-        "internal/running_span_store.h",
-        "internal/running_span_store_impl.h",
-        "internal/span_exporter_impl.h",
-        "internal/span_impl.h",
-        "internal/trace_config_impl.h",
-        "internal/trace_events.h",
-        "internal/trace_params_impl.h",
-        "sampler.h",
-        "span.h",
-        "status_code.h",
-        "trace_config.h",
-        "trace_params.h",
-    ],
-    copts = DEFAULT_COPTS,
-    visibility = ["//visibility:public"],
-    deps = [
-        ":cloud_trace_context",
-        ":span_context",
-        ":trace_context",
-        "//opencensus/common/internal:random_lib",
-        "@com_google_absl//absl/base:core_headers",
-        "@com_google_absl//absl/base:endian",
-        "@com_google_absl//absl/strings",
-        "@com_google_absl//absl/synchronization",
-        "@com_google_absl//absl/time",
-        "@com_google_absl//absl/types:span",
-    ],
-)
-
-cc_library(
-    name = "b3",
-    srcs = [
-        "internal/b3.cc",
-    ],
-    hdrs = [
-        "propagation/b3.h",
-    ],
-    copts = DEFAULT_COPTS,
-    visibility = ["//visibility:public"],
-    deps = [
-        ":span_context",
-        "@com_google_absl//absl/base:endian",
-        "@com_google_absl//absl/strings",
-    ],
-)
-
-cc_library(
-    name = "cloud_trace_context",
-    srcs = [
-        "internal/cloud_trace_context.cc",
-    ],
-    hdrs = [
-        "propagation/cloud_trace_context.h",
-    ],
-    copts = DEFAULT_COPTS,
-    visibility = ["//visibility:public"],
-    deps = [
-        ":span_context",
-        "@com_google_absl//absl/base:endian",
-        "@com_google_absl//absl/strings",
-    ],
-)
-
-cc_library(
-    name = "context_util",
-    srcs = ["internal/context_util.cc"],
-    hdrs = ["context_util.h"],
-    copts = DEFAULT_COPTS,
-    visibility = ["//visibility:public"],
-    deps = [
-        ":trace",
-        "//opencensus/context",
-    ],
-)
-
-cc_library(
-    name = "grpc_trace_bin",
-    srcs = [
-        "internal/grpc_trace_bin.cc",
-    ],
-    hdrs = [
-        "propagation/grpc_trace_bin.h",
-    ],
-    copts = DEFAULT_COPTS,
-    visibility = ["//visibility:public"],
-    deps = [
-        ":span_context",
-        "@com_google_absl//absl/strings",
-    ],
-)
-
-cc_library(
-    name = "span_context",
-    srcs = [
-        "internal/span_context.cc",
-        "internal/span_id.cc",
-        "internal/trace_id.cc",
-        "internal/trace_options.cc",
-    ],
-    hdrs = [
-        "span_context.h",
-        "span_id.h",
-        "trace_id.h",
-        "trace_options.h",
-    ],
-    copts = DEFAULT_COPTS,
-    visibility = ["//visibility:public"],
-    deps = [
-        "@com_google_absl//absl/strings",
-    ],
-)
-
-cc_library(
-    name = "trace_context",
-    srcs = [
-        "internal/trace_context.cc",
-    ],
-    hdrs = [
-        "propagation/trace_context.h",
-    ],
-    copts = DEFAULT_COPTS,
-    visibility = ["//visibility:public"],
-    deps = [
-        ":span_context",
-        "@com_google_absl//absl/base:endian",
-        "@com_google_absl//absl/strings",
-    ],
-)
-
-cc_library(
-    name = "with_span",
-    srcs = ["internal/with_span.cc"],
-    hdrs = ["with_span.h"],
-    copts = DEFAULT_COPTS,
-    visibility = ["//visibility:public"],
-    deps = [
-        ":trace",
-        "//opencensus/context",
-    ],
-)
-
-# Tests
-# ========================================================================= #
-
-cc_test(
-    name = "annotation_test",
-    srcs = ["internal/annotation_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":trace",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_test(
-    name = "attribute_value_ref_test",
-    srcs = ["internal/attribute_value_ref_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":trace",
-        "@com_google_absl//absl/strings",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_test(
-    name = "attribute_value_test",
-    srcs = ["internal/attribute_value_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":trace",
-        "@com_google_absl//absl/strings",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_test(
-    name = "b3_test",
-    srcs = ["internal/b3_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":b3",
-        ":span_context",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_test(
-    name = "cloud_trace_context_test",
-    srcs = ["internal/cloud_trace_context_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":cloud_trace_context",
-        ":span_context",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_test(
-    name = "context_util_test",
-    srcs = ["internal/context_util_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":context_util",
-        ":span_context",
-        ":trace",
-        ":with_span",
-        "//opencensus/context",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_test(
-    name = "grpc_trace_bin_test",
-    srcs = ["internal/grpc_trace_bin_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":grpc_trace_bin",
-        ":span_context",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_test(
-    name = "link_test",
-    srcs = ["internal/link_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":span_context",
-        ":trace",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_test(
-    name = "local_span_store_test",
-    srcs = ["internal/local_span_store_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":trace",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_test(
-    name = "running_span_store_test",
-    srcs = ["internal/running_span_store_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":span_context",
-        ":trace",
-        "@com_google_absl//absl/base:core_headers",
-        "@com_google_absl//absl/memory",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_test(
-    name = "sampler_test",
-    srcs = ["internal/sampler_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":trace",
-        "@com_google_absl//absl/strings",
-        "@com_google_absl//absl/time",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_test(
-    name = "span_test",
-    srcs = ["internal/span_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":span_context",
-        ":trace",
-        "@com_google_absl//absl/strings",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_test(
-    name = "span_id_test",
-    srcs = ["internal/span_id_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":span_context",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_test(
-    name = "span_options_test",
-    srcs = ["internal/span_options_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":span_context",
-        ":trace",
-        "@com_google_absl//absl/strings",
-        "@com_google_absl//absl/synchronization",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_test(
-    name = "span_context_test",
-    srcs = ["internal/span_context_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":span_context",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_test(
-    name = "span_exporter_test",
-    srcs = ["internal/span_exporter_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":trace",
-        "@com_google_absl//absl/memory",
-        "@com_google_absl//absl/synchronization",
-        "@com_google_absl//absl/time",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_test(
-    name = "status_test",
-    srcs = ["internal/status_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":trace",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_test(
-    name = "trace_config_test",
-    srcs = ["internal/trace_config_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":trace",
-        "@com_google_absl//absl/time",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_test(
-    name = "trace_context_test",
-    srcs = ["internal/trace_context_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":span_context",
-        ":trace_context",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_test(
-    name = "trace_options_test",
-    srcs = ["internal/trace_options_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":span_context",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-cc_test(
-    name = "with_span_test",
-    srcs = ["internal/with_span_test.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        ":span_context",
-        ":trace",
-        ":with_span",
-        "//opencensus/context",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
-
-# Benchmarks
-# ========================================================================= #
-
-cc_binary(
-    name = "attribute_value_ref_benchmark",
-    testonly = 1,
-    srcs = ["internal/attribute_value_ref_benchmark.cc"],
-    copts = TEST_COPTS,
-    linkstatic = 1,
-    deps = [
-        ":trace",
-        "@com_github_google_benchmark//:benchmark",
-    ],
-)
-
-cc_binary(
-    name = "b3_benchmark",
-    testonly = 1,
-    srcs = ["internal/b3_benchmark.cc"],
-    copts = TEST_COPTS,
-    linkstatic = 1,
-    deps = [
-        ":b3",
-        "@com_github_google_benchmark//:benchmark",
-    ],
-)
-
-cc_binary(
-    name = "cloud_trace_context_benchmark",
-    testonly = 1,
-    srcs = ["internal/cloud_trace_context_benchmark.cc"],
-    copts = TEST_COPTS,
-    linkstatic = 1,
-    deps = [
-        ":cloud_trace_context",
-        "@com_github_google_benchmark//:benchmark",
-    ],
-)
-
-cc_binary(
-    name = "grpc_trace_bin_benchmark",
-    testonly = 1,
-    srcs = ["internal/grpc_trace_bin_benchmark.cc"],
-    copts = TEST_COPTS,
-    linkstatic = 1,
-    deps = [
-        ":grpc_trace_bin",
-        "@com_github_google_benchmark//:benchmark",
-    ],
-)
-
-cc_binary(
-    name = "sampler_benchmark",
-    testonly = 1,
-    srcs = ["internal/sampler_benchmark.cc"],
-    copts = TEST_COPTS,
-    linkstatic = 1,
-    deps = [
-        ":span_context",
-        ":trace",
-        "@com_github_google_benchmark//:benchmark",
-    ],
-)
-
-cc_binary(
-    name = "span_benchmark",
-    testonly = 1,
-    srcs = ["internal/span_benchmark.cc"],
-    copts = TEST_COPTS,
-    linkstatic = 1,
-    deps = [
-        ":span_context",
-        ":trace",
-        "@com_github_google_benchmark//:benchmark",
-    ],
-)
-
-cc_binary(
-    name = "span_id_benchmark",
-    testonly = 1,
-    srcs = ["internal/span_id_benchmark.cc"],
-    copts = TEST_COPTS,
-    linkstatic = 1,
-    deps = [
-        ":span_context",
-        "@com_github_google_benchmark//:benchmark",
-    ],
-)
-
-cc_binary(
-    name = "trace_context_benchmark",
-    testonly = 1,
-    srcs = ["internal/trace_context_benchmark.cc"],
-    copts = TEST_COPTS,
-    linkstatic = 1,
-    deps = [
-        ":trace_context",
-        "@com_github_google_benchmark//:benchmark",
-    ],
-)
-
-cc_binary(
-    name = "with_span_benchmark",
-    testonly = 1,
-    srcs = ["internal/with_span_benchmark.cc"],
-    copts = TEST_COPTS,
-    linkstatic = 1,
-    deps = [
-        ":trace",
-        ":with_span",
-        "@com_github_google_benchmark//:benchmark",
-    ],
-)
-
-# Fuzzers
-# ========================================================================= #
-
-cc_fuzz_target(
-    name = "b3_fuzzer",
-    srcs = ["internal/b3_fuzzer.cc"],
-    corpus = glob(["internal/b3_corpus/*"]),
-    deps = [
-        ":b3",
-        "@com_google_absl//absl/strings",
-    ],
-)
-
-cc_fuzz_target(
-    name = "cloud_trace_context_fuzzer",
-    srcs = ["internal/cloud_trace_context_fuzzer.cc"],
-    corpus = glob(["internal/cloud_trace_context_corpus/*"]),
-    deps = [
-        ":cloud_trace_context",
-        ":span_context",
-        "@com_google_absl//absl/strings",
-    ],
-)
-
-cc_fuzz_target(
-    name = "grpc_trace_bin_fuzzer",
-    srcs = ["internal/grpc_trace_bin_fuzzer.cc"],
-    corpus = glob(["internal/grpc_trace_bin_corpus/*"]),
-    deps = [
-        ":grpc_trace_bin",
-        ":span_context",
-        "@com_google_absl//absl/strings",
-    ],
-)
-
-cc_fuzz_target(
-    name = "trace_context_fuzzer",
-    srcs = ["internal/trace_context_fuzzer.cc"],
-    corpus = glob(["internal/trace_context_corpus/*"]),
-    deps = [
-        ":span_context",
-        ":trace_context",
-        "@com_google_absl//absl/strings",
-    ],
-)
diff --git a/cpp/third_party/opencensus/0.5.0-alpha/opencensus/trace/examples/BUILD b/cpp/third_party/opencensus/0.5.0-alpha/opencensus/trace/examples/BUILD
deleted file mode 100644
index d8f91a4..0000000
--- a/cpp/third_party/opencensus/0.5.0-alpha/opencensus/trace/examples/BUILD
+++ /dev/null
@@ -1,39 +0,0 @@
-# OpenCensus C++ Trace library: Example usage.
-#
-# Copyright 2017, OpenCensus Authors
-#
-# 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.
-
-load("//opencensus:copts.bzl", "TEST_COPTS")
-
-licenses(["notice"])  # Apache 2.0
-
-package(
-    default_testonly = 1,
-    default_visibility = ["//visibility:private"],
-)
-
-# Examples are defined as tests in order to make sure the code compiles and
-# doesn't crash.
-
-cc_test(
-    name = "span_example",
-    srcs = ["span_example.cc"],
-    copts = TEST_COPTS,
-    deps = [
-        "//opencensus/trace",
-        "@com_google_absl//absl/memory",
-        "@com_google_absl//absl/strings",
-        "@com_google_googletest//:gtest_main",
-    ],
-)
diff --git a/cpp/third_party/opencensus/proto/opencensus/proto/agent/common/v1/BUILD.bazel b/cpp/third_party/opencensus/proto/opencensus/proto/agent/common/v1/BUILD.bazel
deleted file mode 100644
index a078e61..0000000
--- a/cpp/third_party/opencensus/proto/opencensus/proto/agent/common/v1/BUILD.bazel
+++ /dev/null
@@ -1,34 +0,0 @@
-# Copyright 2018, OpenCensus Authors
-#
-# 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.
-
-package(default_visibility = ["//visibility:public"])
-
-load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library")
-
-proto_library(
-    name = "common_proto",
-    srcs = ["common.proto"],
-    deps = [
-        "@com_google_protobuf//:timestamp_proto",
-    ],
-)
-
-go_proto_library(
-    name = "common_proto_go",
-    proto = ":common_proto",
-    importpath = "github.com/census-instrumentation/opencensus-proto/gen-go/agent/common/v1",
-    deps = [
-        "@com_github_golang_protobuf//ptypes/timestamp:go_default_library",
-    ],
-)
diff --git a/cpp/third_party/opencensus/proto/opencensus/proto/agent/metrics/v1/BUILD.bazel b/cpp/third_party/opencensus/proto/opencensus/proto/agent/metrics/v1/BUILD.bazel
deleted file mode 100644
index 4983e5b..0000000
--- a/cpp/third_party/opencensus/proto/opencensus/proto/agent/metrics/v1/BUILD.bazel
+++ /dev/null
@@ -1,55 +0,0 @@
-# Copyright 2018, OpenCensus Authors
-#
-# 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.
-
-package(default_visibility = ["//visibility:public"])
-
-load("@grpc_java//:java_grpc_library.bzl", "java_grpc_library")
-load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library")
-
-proto_library(
-    name = "metrics_service_proto",
-    srcs = ["metrics_service.proto"],
-    deps = [
-        "//opencensus/proto/agent/common/v1:common_proto",
-        "//opencensus/proto/metrics/v1:metrics_proto",
-        "//opencensus/proto/resource/v1:resource_proto",
-    ],
-)
-
-cc_proto_library(
-    name = "metrics_service_proto_cc",
-    deps = [":metrics_service_proto"],
-)
-
-java_proto_library(
-    name = "metrics_service_proto_java",
-    deps = [":metrics_service_proto"],
-)
-
-java_grpc_library(
-    name = "metrics_service_grpc_java",
-    srcs = [":metrics_service_proto"],
-    deps = [":metrics_service_proto_java"],
-)
-
-go_proto_library(
-    name = "metrics_service_proto_go",
-    proto = ":metrics_service_proto",
-    importpath = "github.com/census-instrumentation/opencensus-proto/gen-go/agent/metrics/v1",
-    deps = [
-        "//opencensus/proto/agent/common/v1:common_proto_go",
-        "//opencensus/proto/metrics/v1:metrics_proto_go",
-        "//opencensus/proto/resource/v1:resource_proto_go",
-    ],
-)
diff --git a/cpp/third_party/opencensus/proto/opencensus/proto/agent/trace/v1/BUILD.bazel b/cpp/third_party/opencensus/proto/opencensus/proto/agent/trace/v1/BUILD.bazel
deleted file mode 100644
index 819c5e4..0000000
--- a/cpp/third_party/opencensus/proto/opencensus/proto/agent/trace/v1/BUILD.bazel
+++ /dev/null
@@ -1,56 +0,0 @@
-# Copyright 2018, OpenCensus Authors
-#
-# 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.
-
-package(default_visibility = ["//visibility:public"])
-
-load("@grpc_java//:java_grpc_library.bzl", "java_grpc_library")
-load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library")
-
-proto_library(
-    name = "trace_service_proto",
-    srcs = ["trace_service.proto"],
-    deps = [
-        "//opencensus/proto/agent/common/v1:common_proto",
-        "//opencensus/proto/resource/v1:resource_proto",
-        "//opencensus/proto/trace/v1:trace_proto",
-        "//opencensus/proto/trace/v1:trace_config_proto",
-    ],
-)
-
-cc_proto_library(
-    name = "trace_service_proto_cc",
-    deps = [":trace_service_proto"],
-)
-
-java_proto_library(
-    name = "trace_service_proto_java",
-    deps = [":trace_service_proto"],
-)
-
-java_grpc_library(
-    name = "trace_service_grpc_java",
-    srcs = [":trace_service_proto"],
-    deps = [":trace_service_proto_java"],
-)
-
-go_proto_library(
-    name = "trace_service_proto_go",
-    proto = ":trace_service_proto",
-    importpath = "github.com/census-instrumentation/opencensus-proto/gen-go/agent/trace/v1",
-    deps = [
-        "//opencensus/proto/agent/common/v1:common_proto_go",
-        "//opencensus/proto/resource/v1:resource_proto_go",
-        "//opencensus/proto/trace/v1:trace_and_config_proto_go",
-    ],
-)
diff --git a/cpp/third_party/opencensus/proto/opencensus/proto/metrics/v1/BUILD.bazel b/cpp/third_party/opencensus/proto/opencensus/proto/metrics/v1/BUILD.bazel
deleted file mode 100644
index 333c027..0000000
--- a/cpp/third_party/opencensus/proto/opencensus/proto/metrics/v1/BUILD.bazel
+++ /dev/null
@@ -1,48 +0,0 @@
-# Copyright 2018, OpenCensus Authors
-#
-# 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.
-
-package(default_visibility = ["//visibility:public"])
-
-load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library")
-
-proto_library(
-    name = "metrics_proto",
-    srcs = ["metrics.proto"],
-    deps = [
-        "@com_google_protobuf//:timestamp_proto",
-        "@com_google_protobuf//:wrappers_proto",
-        "//opencensus/proto/resource/v1:resource_proto",
-    ],
-)
-
-cc_proto_library(
-    name = "metrics_proto_cc",
-    deps = [":metrics_proto"],
-)
-
-java_proto_library(
-    name = "metrics_proto_java",
-    deps = [":metrics_proto"],
-)
-
-go_proto_library(
-    name = "metrics_proto_go",
-    proto = ":metrics_proto",
-    importpath = "github.com/census-instrumentation/opencensus-proto/gen-go/metrics/v1",
-    deps = [
-        "//opencensus/proto/resource/v1:resource_proto_go",
-        "@com_github_golang_protobuf//ptypes/timestamp:go_default_library",
-        "@com_github_golang_protobuf//ptypes/wrappers:go_default_library",
-    ],
-)
diff --git a/cpp/third_party/opencensus/proto/opencensus/proto/resource/v1/BUILD.bazel b/cpp/third_party/opencensus/proto/opencensus/proto/resource/v1/BUILD.bazel
deleted file mode 100644
index d17f43a..0000000
--- a/cpp/third_party/opencensus/proto/opencensus/proto/resource/v1/BUILD.bazel
+++ /dev/null
@@ -1,38 +0,0 @@
-# Copyright 2018, OpenCensus Authors
-#
-# 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.
-
-package(default_visibility = ["//visibility:public"])
-
-load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library")
-
-proto_library(
-    name = "resource_proto",
-    srcs = ["resource.proto"],
-)
-
-cc_proto_library(
-    name = "resource_proto_cc",
-    deps = [":resource_proto"],
-)
-
-java_proto_library(
-    name = "resource_proto_java",
-    deps = [":resource_proto"],
-)
-
-go_proto_library(
-    name = "resource_proto_go",
-    proto = ":resource_proto",
-    importpath = "github.com/census-instrumentation/opencensus-proto/gen-go/resource/v1",
-)
diff --git a/cpp/third_party/opencensus/proto/opencensus/proto/stats/v1/BUILD.bazel b/cpp/third_party/opencensus/proto/opencensus/proto/stats/v1/BUILD.bazel
deleted file mode 100644
index 10c5ee5..0000000
--- a/cpp/third_party/opencensus/proto/opencensus/proto/stats/v1/BUILD.bazel
+++ /dev/null
@@ -1,45 +0,0 @@
-# Copyright 2017, OpenCensus Authors
-#
-# 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.
-
-package(default_visibility = ["//visibility:public"])
-
-load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library")
-
-proto_library(
-    name = "stats_proto",
-    srcs = ["stats.proto"],
-    deps = [
-        "@com_google_protobuf//:duration_proto",
-        "@com_google_protobuf//:timestamp_proto",
-    ],
-)
-
-cc_proto_library(
-    name = "stats_proto_cc",
-    deps = [":stats_proto"],
-)
-
-java_proto_library(
-    name = "stats_proto_java",
-    deps = [":stats_proto"],
-)
-
-go_proto_library(
-    name = "stats_proto_go",
-    proto = ":stats_proto",
-    importpath = "github.com/census-instrumentation/opencensus-proto/gen-go/stats/v1",
-    deps = [
-        "@com_github_golang_protobuf//ptypes/timestamp:go_default_library",
-    ],
-)
diff --git a/cpp/third_party/opencensus/proto/opencensus/proto/trace/v1/BUILD.bazel b/cpp/third_party/opencensus/proto/opencensus/proto/trace/v1/BUILD.bazel
deleted file mode 100644
index 589d880..0000000
--- a/cpp/third_party/opencensus/proto/opencensus/proto/trace/v1/BUILD.bazel
+++ /dev/null
@@ -1,83 +0,0 @@
-# Copyright 2017, OpenCensus Authors
-#
-# 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.
-
-package(default_visibility = ["//visibility:public"])
-
-load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library")
-load("@io_bazel_rules_go//go:def.bzl", "go_library")
-
-proto_library(
-    name = "trace_proto",
-    srcs = ["trace.proto"],
-    deps = [
-        "@com_google_protobuf//:timestamp_proto",
-        "@com_google_protobuf//:wrappers_proto",
-        "//opencensus/proto/resource/v1:resource_proto",
-    ],
-)
-
-proto_library(
-    name = "trace_config_proto",
-    srcs = ["trace_config.proto"],
-)
-
-cc_proto_library(
-    name = "trace_proto_cc",
-    deps = [":trace_proto"],
-)
-
-cc_proto_library(
-    name = "trace_config_proto_cc",
-    deps = [":trace_config_proto"],
-)
-
-java_proto_library(
-    name = "trace_proto_java",
-    deps = [":trace_proto"],
-)
-
-java_proto_library(
-    name = "trace_config_proto_java",
-    deps = [":trace_config_proto"],
-)
-
-go_proto_library(
-    name = "trace_proto_go",
-    proto = ":trace_proto",
-    importpath = "github.com/census-instrumentation/opencensus-proto/gen-go/trace/v1",
-    deps = [
-        "@com_github_golang_protobuf//ptypes/timestamp:go_default_library",
-        "@com_github_golang_protobuf//ptypes/wrappers:go_default_library",
-        "//opencensus/proto/resource/v1:resource_proto_go",
-    ],
-)
-
-go_proto_library(
-    name = "trace_config_proto_go",
-    proto = ":trace_config_proto",
-    importpath = "github.com/census-instrumentation/opencensus-proto/gen-go/trace/v1",
-)
-
-# This a workaround because `trace_proto_go` and `trace_config_proto_go` have
-# the same importpath, and so cause a compile issue if both are depended on
-# directly by another `go_proto_library` (such as `trace_service_proto_go`).
-# See: https://github.com/bazelbuild/rules_go/issues/1841
-go_library(
-    name = "trace_and_config_proto_go",
-    srcs = [],
-    embed = [
-        ":trace_proto_go",
-        ":trace_config_proto_go",
-    ]
-)