You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by ab...@apache.org on 2018/08/15 20:43:58 UTC

kudu git commit: make_shared: fix build for newer libc++

Repository: kudu
Updated Branches:
  refs/heads/branch-1.7.x b24f166e3 -> 3fa3480be


make_shared: fix build for newer libc++

Previously, we used friendship with various internal classes in the 'std'
namespace to allow make_shared to work against classes with private
constructors. With the version of libc++ that comes with LLVM 6, these
tricks no longer work and I was unable to find a suitable friend definition.

This patch switches to using a different approach based on constructing
a locally-scoped subclass of the target class.

I also noticed that TypeEncodingInfo and TypeInfo were only using shared_ptr
due to the pre-C++11 prohibition of scoped_ptrs in containers, so switched
them to unique_ptr.

Change-Id: Ib0dd0338ee531ab3578ba7291637860b56ba6230
Reviewed-on: http://gerrit.cloudera.org:8080/9847
Tested-by: Kudu Jenkins
Reviewed-by: Adar Dembo <ad...@cloudera.com>
(cherry picked from commit 61d3fff2fd93358b6c71aaec14189bf7f55de99a)
Reviewed-on: http://gerrit.cloudera.org:8080/10549
Reviewed-by: Todd Lipcon <to...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/kudu/repo
Commit: http://git-wip-us.apache.org/repos/asf/kudu/commit/3fa3480b
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/3fa3480b
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/3fa3480b

Branch: refs/heads/branch-1.7.x
Commit: 3fa3480bec5a23aea3f00c87e487fba4654c96f3
Parents: b24f166
Author: Todd Lipcon <to...@apache.org>
Authored: Mon Mar 26 12:17:30 2018 -0700
Committer: Attila Bukor <ab...@apache.org>
Committed: Wed Aug 15 19:39:04 2018 +0000

----------------------------------------------------------------------
 src/kudu/cfile/type_encodings.cc     | 12 ++---
 src/kudu/cfile/type_encodings.h      |  7 ++-
 src/kudu/common/types.cc             |  7 ++-
 src/kudu/common/types.h              |  4 +-
 src/kudu/consensus/log_reader.cc     |  3 +-
 src/kudu/consensus/log_reader.h      | 10 ++--
 src/kudu/consensus/raft_consensus.cc |  8 +--
 src/kudu/consensus/raft_consensus.h  | 13 ++---
 src/kudu/master/ts_descriptor.cc     |  2 +-
 src/kudu/master/ts_descriptor.h      |  8 +--
 src/kudu/rpc/periodic.cc             |  2 +-
 src/kudu/rpc/periodic.h              | 11 ++---
 src/kudu/util/make_shared.h          | 81 ++++++++++++++++---------------
 13 files changed, 82 insertions(+), 86 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/3fa3480b/src/kudu/cfile/type_encodings.cc
----------------------------------------------------------------------
diff --git a/src/kudu/cfile/type_encodings.cc b/src/kudu/cfile/type_encodings.cc
index d184ea6..444dea1 100644
--- a/src/kudu/cfile/type_encodings.cc
+++ b/src/kudu/cfile/type_encodings.cc
@@ -34,10 +34,9 @@
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/util/slice.h"
 
-
 using std::make_pair;
 using std::pair;
-using std::shared_ptr;
+using std::unique_ptr;
 using std::unordered_map;
 
 namespace kudu {
@@ -284,15 +283,14 @@ class TypeEncodingResolver {
     TypeEncodingTraits<type, encoding> traits;
     pair<DataType, EncodingType> encoding_for_type = make_pair(type, encoding);
     if (mapping_.find(encoding_for_type) == mapping_.end()) {
-      default_mapping_.insert(make_pair(type, encoding));
+      default_mapping_.emplace(type, encoding);
     }
-    mapping_.insert(
-        make_pair(make_pair(type, encoding),
-                  std::make_shared<TypeEncodingInfo>(traits)));
+    mapping_.emplace(make_pair(type, encoding),
+                     unique_ptr<TypeEncodingInfo>(new TypeEncodingInfo(traits)));
   }
 
   unordered_map<pair<DataType, EncodingType>,
-      shared_ptr<const TypeEncodingInfo>,
+      unique_ptr<const TypeEncodingInfo>,
       EncodingMapHash > mapping_;
 
   unordered_map<DataType, EncodingType, std::hash<size_t> > default_mapping_;

http://git-wip-us.apache.org/repos/asf/kudu/blob/3fa3480b/src/kudu/cfile/type_encodings.h
----------------------------------------------------------------------
diff --git a/src/kudu/cfile/type_encodings.h b/src/kudu/cfile/type_encodings.h
index bbcaf87..2094ac8 100644
--- a/src/kudu/cfile/type_encodings.h
+++ b/src/kudu/cfile/type_encodings.h
@@ -19,7 +19,6 @@
 
 #include "kudu/common/common.pb.h"
 #include "kudu/gutil/macros.h"
-#include "kudu/util/make_shared.h"
 #include "kudu/util/status.h"
 
 namespace kudu {
@@ -38,7 +37,6 @@ struct WriterOptions;
 // Mimicked after common::TypeInfo et al
 class TypeEncodingInfo {
  public:
-
   static Status Get(const TypeInfo* typeinfo, EncodingType encoding, const TypeEncodingInfo** out);
 
   static const EncodingType GetDefaultEncoding(const TypeInfo* typeinfo);
@@ -54,9 +52,10 @@ class TypeEncodingInfo {
   Status CreateBlockDecoder(BlockDecoder **bd, const Slice &slice,
                             CFileIterator *iter) const;
  private:
-  ALLOW_MAKE_SHARED(TypeEncodingInfo);
   friend class TypeEncodingResolver;
-  template<typename TypeEncodingTraitsClass> TypeEncodingInfo(TypeEncodingTraitsClass t);
+
+  template<typename TypeEncodingTraitsClass>
+  explicit TypeEncodingInfo(TypeEncodingTraitsClass t);
 
   EncodingType encoding_type_;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/3fa3480b/src/kudu/common/types.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/types.cc b/src/kudu/common/types.cc
index 12a09d2..13ae93a 100644
--- a/src/kudu/common/types.cc
+++ b/src/kudu/common/types.cc
@@ -19,13 +19,12 @@
 
 #include <memory>
 #include <unordered_map>
-#include <utility>
 
 #include "kudu/gutil/singleton.h"
 #include "kudu/util/logging.h"
 
-using std::shared_ptr;
 using std::string;
+using std::unique_ptr;
 using std::unordered_map;
 
 namespace kudu {
@@ -92,11 +91,11 @@ class TypeInfoResolver {
 
   template<DataType type> void AddMapping() {
     TypeTraits<type> traits;
-    mapping_.insert(make_pair(type, std::make_shared<TypeInfo>(traits)));
+    mapping_.emplace(type, unique_ptr<TypeInfo>(new TypeInfo(traits)));
   }
 
   unordered_map<DataType,
-                shared_ptr<const TypeInfo>,
+                unique_ptr<const TypeInfo>,
                 std::hash<size_t> > mapping_;
 
   friend class Singleton<TypeInfoResolver>;

http://git-wip-us.apache.org/repos/asf/kudu/blob/3fa3480b/src/kudu/common/types.h
----------------------------------------------------------------------
diff --git a/src/kudu/common/types.h b/src/kudu/common/types.h
index 5896341..7b7aca3 100644
--- a/src/kudu/common/types.h
+++ b/src/kudu/common/types.h
@@ -37,7 +37,6 @@
 #include "kudu/gutil/strings/escaping.h"
 #include "kudu/gutil/strings/numbers.h"
 #include "kudu/util/int128.h"
-#include "kudu/util/make_shared.h"
 #include "kudu/util/slice.h"
 // IWYU pragma: no_include "kudu/util/status.h"
 
@@ -78,9 +77,8 @@ class TypeInfo {
   }
 
  private:
-  ALLOW_MAKE_SHARED(TypeInfo);
   friend class TypeInfoResolver;
-  template<typename Type> TypeInfo(Type t);
+  template<typename Type> explicit TypeInfo(Type t);
 
   const DataType type_;
   const DataType physical_type_;

http://git-wip-us.apache.org/repos/asf/kudu/blob/3fa3480b/src/kudu/consensus/log_reader.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/log_reader.cc b/src/kudu/consensus/log_reader.cc
index d357f23..581f3ca 100644
--- a/src/kudu/consensus/log_reader.cc
+++ b/src/kudu/consensus/log_reader.cc
@@ -80,8 +80,7 @@ Status LogReader::Open(Env* env,
                        const string& tablet_id,
                        const scoped_refptr<MetricEntity>& metric_entity,
                        shared_ptr<LogReader>* reader) {
-  auto log_reader = std::make_shared<LogReader>(
-      env, index, tablet_id, metric_entity);
+  auto log_reader = LogReader::make_shared(env, index, tablet_id, metric_entity);
 
   RETURN_NOT_OK_PREPEND(log_reader->Init(tablet_wal_dir),
                         "Unable to initialize log reader")

http://git-wip-us.apache.org/repos/asf/kudu/blob/3fa3480b/src/kudu/consensus/log_reader.h
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/log_reader.h b/src/kudu/consensus/log_reader.h
index bc88cb3..e9c5e84 100644
--- a/src/kudu/consensus/log_reader.h
+++ b/src/kudu/consensus/log_reader.h
@@ -54,7 +54,7 @@ struct LogIndexEntry;
 // Reads a set of segments from a given path. Segment headers and footers
 // are read and parsed, but entries are not.
 // This class is thread safe.
-class LogReader {
+class LogReader : public enable_make_shared<LogReader> {
  public:
   ~LogReader();
 
@@ -114,6 +114,10 @@ class LogReader {
 
   std::string ToString() const;
 
+ protected:
+  LogReader(Env* env, scoped_refptr<LogIndex> index, std::string tablet_id,
+            const scoped_refptr<MetricEntity>& metric_entity);
+
  private:
   FRIEND_TEST(LogTestOptionalCompression, TestLogReader);
   FRIEND_TEST(LogTestOptionalCompression, TestReadLogWithReplacedReplicates);
@@ -168,9 +172,6 @@ class LogReader {
                                   faststring* tmp_buf,
                                   gscoped_ptr<LogEntryBatchPB>* batch) const;
 
-  LogReader(Env* env, scoped_refptr<LogIndex> index, std::string tablet_id,
-            const scoped_refptr<MetricEntity>& metric_entity);
-
   // Reads the headers of all segments in 'tablet_wal_path'.
   Status Init(const std::string& tablet_wal_path);
 
@@ -194,7 +195,6 @@ class LogReader {
 
   State state_;
 
-  ALLOW_MAKE_SHARED(LogReader);
   DISALLOW_COPY_AND_ASSIGN(LogReader);
 };
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/3fa3480b/src/kudu/consensus/raft_consensus.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/raft_consensus.cc b/src/kudu/consensus/raft_consensus.cc
index dc59d39..09cbe12 100644
--- a/src/kudu/consensus/raft_consensus.cc
+++ b/src/kudu/consensus/raft_consensus.cc
@@ -201,10 +201,10 @@ Status RaftConsensus::Create(ConsensusOptions options,
                              scoped_refptr<ConsensusMetadataManager> cmeta_manager,
                              ThreadPool* raft_pool,
                              shared_ptr<RaftConsensus>* consensus_out) {
-  shared_ptr<RaftConsensus> consensus(std::make_shared<RaftConsensus>(std::move(options),
-                                                                      std::move(local_peer_pb),
-                                                                      std::move(cmeta_manager),
-                                                                      raft_pool));
+  shared_ptr<RaftConsensus> consensus(RaftConsensus::make_shared(std::move(options),
+                                                                 std::move(local_peer_pb),
+                                                                 std::move(cmeta_manager),
+                                                                 raft_pool));
   RETURN_NOT_OK_PREPEND(consensus->Init(), "Unable to initialize Raft consensus");
   *consensus_out = std::move(consensus);
   return Status::OK();

http://git-wip-us.apache.org/repos/asf/kudu/blob/3fa3480b/src/kudu/consensus/raft_consensus.h
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/raft_consensus.h b/src/kudu/consensus/raft_consensus.h
index cfc38c5..d80630c 100644
--- a/src/kudu/consensus/raft_consensus.h
+++ b/src/kudu/consensus/raft_consensus.h
@@ -87,6 +87,7 @@ typedef int64_t ConsensusTerm;
 typedef StdStatusCallback ConsensusReplicatedCallback;
 
 class RaftConsensus : public std::enable_shared_from_this<RaftConsensus>,
+                      public enable_make_shared<RaftConsensus>,
                       public PeerMessageQueueObserver {
  public:
 
@@ -353,8 +354,13 @@ class RaftConsensus : public std::enable_shared_from_this<RaftConsensus>,
   // Return the on-disk size of the consensus metadata, in bytes.
   int64_t MetadataOnDiskSize() const;
 
+ protected:
+  RaftConsensus(ConsensusOptions options,
+                RaftPeerPB local_peer_pb,
+                scoped_refptr<ConsensusMetadataManager> cmeta_manager,
+                ThreadPool* raft_pool);
+
  private:
-  ALLOW_MAKE_SHARED(RaftConsensus);
   friend class RaftConsensusQuorumTest;
   FRIEND_TEST(RaftConsensusQuorumTest, TestConsensusContinuesIfAMinorityFallsBehind);
   FRIEND_TEST(RaftConsensusQuorumTest, TestConsensusStopsIfAMajorityFallsBehind);
@@ -420,11 +426,6 @@ class RaftConsensus : public std::enable_shared_from_this<RaftConsensus>,
   using LockGuard = std::lock_guard<simple_spinlock>;
   using UniqueLock = std::unique_lock<simple_spinlock>;
 
-  RaftConsensus(ConsensusOptions options,
-                RaftPeerPB local_peer_pb,
-                scoped_refptr<ConsensusMetadataManager> cmeta_manager,
-                ThreadPool* raft_pool);
-
   // Initializes the RaftConsensus object, including loading the consensus
   // metadata.
   Status Init();

http://git-wip-us.apache.org/repos/asf/kudu/blob/3fa3480b/src/kudu/master/ts_descriptor.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/ts_descriptor.cc b/src/kudu/master/ts_descriptor.cc
index 4072b33..27b60b8 100644
--- a/src/kudu/master/ts_descriptor.cc
+++ b/src/kudu/master/ts_descriptor.cc
@@ -56,7 +56,7 @@ namespace master {
 Status TSDescriptor::RegisterNew(const NodeInstancePB& instance,
                                  const ServerRegistrationPB& registration,
                                  shared_ptr<TSDescriptor>* desc) {
-  shared_ptr<TSDescriptor> ret(make_shared<TSDescriptor>(instance.permanent_uuid()));
+  shared_ptr<TSDescriptor> ret(TSDescriptor::make_shared(instance.permanent_uuid()));
   RETURN_NOT_OK(ret->Register(instance, registration));
   desc->swap(ret);
   return Status::OK();

http://git-wip-us.apache.org/repos/asf/kudu/blob/3fa3480b/src/kudu/master/ts_descriptor.h
----------------------------------------------------------------------
diff --git a/src/kudu/master/ts_descriptor.h b/src/kudu/master/ts_descriptor.h
index 55cb275..a26f68a 100644
--- a/src/kudu/master/ts_descriptor.h
+++ b/src/kudu/master/ts_descriptor.h
@@ -56,7 +56,7 @@ namespace master {
 //
 // Tracks the last heartbeat, status, instance identifier, etc.
 // This class is thread-safe.
-class TSDescriptor {
+class TSDescriptor : public enable_make_shared<TSDescriptor> {
  public:
   static Status RegisterNew(const NodeInstancePB& instance,
                             const ServerRegistrationPB& registration,
@@ -123,11 +123,12 @@ class TSDescriptor {
   // Includes the UUID as well as last known host/port.
   std::string ToString() const;
 
+ protected:
+  explicit TSDescriptor(std::string perm_id);
+
  private:
   FRIEND_TEST(TestTSDescriptor, TestReplicaCreationsDecay);
 
-  explicit TSDescriptor(std::string perm_id);
-
   // Uses DNS to resolve registered hosts to a single Sockaddr.
   // Returns the resolved address as well as the hostname associated with it
   // in 'addr' and 'host'.
@@ -156,7 +157,6 @@ class TSDescriptor {
   std::shared_ptr<tserver::TabletServerAdminServiceProxy> ts_admin_proxy_;
   std::shared_ptr<consensus::ConsensusServiceProxy> consensus_proxy_;
 
-  ALLOW_MAKE_SHARED(TSDescriptor);
   DISALLOW_COPY_AND_ASSIGN(TSDescriptor);
 };
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/3fa3480b/src/kudu/rpc/periodic.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/periodic.cc b/src/kudu/rpc/periodic.cc
index 3b9412d..2ca066c 100644
--- a/src/kudu/rpc/periodic.cc
+++ b/src/kudu/rpc/periodic.cc
@@ -46,7 +46,7 @@ shared_ptr<PeriodicTimer> PeriodicTimer::Create(
     RunTaskFunctor functor,
     MonoDelta period,
     Options options) {
-  return std::make_shared<PeriodicTimer>(
+  return PeriodicTimer::make_shared(
       std::move(messenger), std::move(functor), period, options);
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/3fa3480b/src/kudu/rpc/periodic.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/periodic.h b/src/kudu/rpc/periodic.h
index 075d900..592cbd5 100644
--- a/src/kudu/rpc/periodic.h
+++ b/src/kudu/rpc/periodic.h
@@ -62,7 +62,8 @@ class Messenger;
 // timer cancelation, which would allow us to implement synchronous Stop(), use
 // exclusive ownership, and remove the restriction that the delta passed
 // into Snooze() be greater than GetMinimumPeriod().
-class PeriodicTimer : public std::enable_shared_from_this<PeriodicTimer> {
+class PeriodicTimer : public std::enable_shared_from_this<PeriodicTimer>,
+                      public enable_make_shared<PeriodicTimer> {
  public:
   typedef std::function<void(void)> RunTaskFunctor;
 
@@ -144,14 +145,14 @@ class PeriodicTimer : public std::enable_shared_from_this<PeriodicTimer> {
   // Returns true iff the timer has been started.
   bool started() const;
 
- private:
-  FRIEND_TEST(PeriodicTimerTest, TestCallbackRestartsTimer);
-
+ protected:
   PeriodicTimer(std::shared_ptr<Messenger> messenger,
                 RunTaskFunctor functor,
                 MonoDelta period,
                 Options options);
 
+ private:
+  FRIEND_TEST(PeriodicTimerTest, TestCallbackRestartsTimer);
   // Calculate the minimum period for the timer, which varies depending on
   // 'jitter_pct_' and the output of the PRNG.
   MonoDelta GetMinimumPeriod();
@@ -207,8 +208,6 @@ class PeriodicTimer : public std::enable_shared_from_this<PeriodicTimer> {
   // Whether the timer is running or not.
   bool started_;
 
-  ALLOW_MAKE_SHARED(PeriodicTimer);
-
   DISALLOW_COPY_AND_ASSIGN(PeriodicTimer);
 };
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/3fa3480b/src/kudu/util/make_shared.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/make_shared.h b/src/kudu/util/make_shared.h
index 5e0cb6c..649cae7 100644
--- a/src/kudu/util/make_shared.h
+++ b/src/kudu/util/make_shared.h
@@ -18,44 +18,47 @@
 #pragma once
 
 #include <memory>
+#include <utility>
 
-// It isn't possible to use std::make_shared() with a class that has private
-// constructors. Moreover, the standard workarounds are inelegant when said
-// class has non-default constructors. As such, we employ a simple solution:
-// declare the class as a friend to std::make_shared()'s internal allocator.
-// This approach is non-portable and must be implemented separately for each
-// supported STL implementation.
-//
-// Note: due to friendship restrictions on partial template specialization,
-// it isn't possible to befriend just the allocation function; the entire
-// allocator class must be befriended.
-//
-// See http://stackoverflow.com/q/8147027 for a longer discussion.
+// It isn't possible to use 'std::make_shared' on a class with private or protected
+// constructors. Using friends as a workaround worked in some earlier libc++/libstdcxx
+// versions, but in the latest versions there are some static_asserts that seem to defeat
+// this trickery. So, instead, we rely on the "curiously recurring template pattern" (CRTP)
+// to inject a static 'make_shared' function inside the class.
+//
+// See https://stackoverflow.com/questions/8147027/how-do-i-call-stdmake-shared-on-a-class-with-only-protected-or-private-const
+// for some details.
+//
+// Usage:
+//
+//  class MyClass : public enable_make_shared<MyClass> {
+//   public:
+//     ...
+//
+//   protected:
+//    // The constructor must be protected rather than private.
+//    MyClass(Foo arg1, Bar arg2) {
+//    }
+//
+//  }
+//
+//    shared_ptr<MyClass> foo = MyClass::make_shared(arg1, arg2);
+template<class T>
+class enable_make_shared { // NOLINT
+ public:
+
+  // Define a static make_shared member which constructs the public subclass
+  // and casts it back to the desired class.
+  template<typename... Arg>
+  static std::shared_ptr<T> make_shared(Arg&&... args) {
+    // Define a struct subclass with a public constructor which will be accessible
+    // from make_shared.
+    struct make_shared_enabler : public T { // NOLINT
+      explicit make_shared_enabler(Arg&&... args) : T(std::forward<Arg>(args)...) {
+      }
+    };
 
-#ifdef __GLIBCXX__
-  // In libstdc++, new_allocator is defined as a class (ext/new_allocator.h)
-  // but forward declared as a struct (ext/alloc_traits.h). Clang complains
-  // about this when -Wmismatched-tags is set, which gcc doesn't support
-  // (which probably explains why the discrepancy exists in the first place).
-  // We can temporarily disable this warning via pragmas [1], but we must
-  // not expose them to gcc due to its poor handling of the _Pragma() C99
-  // operator [2].
-  //
-  // 1. http://clang.llvm.org/docs/UsersManual.html#controlling-diagnostics-via-pragmas
-  // 2. https://gcc.gnu.org/bugzilla/show_bug.cgi?id=60875
-  #ifdef __clang__
-    #define ALLOW_MAKE_SHARED(T)                                \
-      _Pragma("clang diagnostic push")                          \
-      _Pragma("clang diagnostic ignored \"-Wmismatched-tags\"") \
-      friend class __gnu_cxx::new_allocator<T>                  \
-      _Pragma("clang diagnostic pop")
-  #else
-    #define ALLOW_MAKE_SHARED(T) \
-      friend class __gnu_cxx::new_allocator<T>
-  #endif
-#elif defined(_LIBCPP_VERSION)
-  #define ALLOW_MAKE_SHARED(T) \
-    friend class std::__1::__libcpp_compressed_pair_imp<std::__1::allocator<T>, T, 1>
-#else
-  #error "Need to implement ALLOW_MAKE_SHARED for your platform!"
-#endif
+    return ::std::make_shared<make_shared_enabler>(
+        ::std::forward<Arg>(args)...);
+  }
+};