Explorar o código

HDFS-10188. libhdfs++: Implement debug allocators. Contributed by Xiaowei Zhu

James %!s(int64=9) %!d(string=hai) anos
pai
achega
d1ab7d0eeb

+ 52 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/new_delete.h

@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef COMMON_HDFS_NEW_DELETE_H_
+#define COMMON_HDFS_NEW_DELETE_H_
+
+#include <cstring>
+
+struct mem_struct {
+  size_t mem_size;
+};
+
+#ifndef NDEBUG
+#define MEMCHECKED_CLASS(clazz) \
+static void* operator new(size_t size) { \
+  void* p = ::malloc(size); \
+  return p; \
+} \
+static void* operator new[](size_t size) { \
+  mem_struct* p = (mem_struct*)::malloc(sizeof(mem_struct) + size); \
+  p->mem_size = size; \
+  return (void*)++p; \
+} \
+static void operator delete(void* p) { \
+  ::memset(p, 0, sizeof(clazz)); \
+  ::free(p); \
+} \
+static void operator delete[](void* p) { \
+  mem_struct* header = (mem_struct*)p; \
+  size_t size = (--header)->mem_size; \
+  ::memset(p, 0, size); \
+  ::free(header); \
+}
+#else
+#define MEMCHECKED_CLASS(clazz)
+#endif
+#endif

+ 7 - 5
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.h

@@ -24,6 +24,7 @@
 #include "common/libhdfs_events_impl.h"
 #include "common/logging.h"
 #include "common/util.h"
+#include "common/new_delete.h"
 
 #include "asio.hpp"
 
@@ -31,12 +32,13 @@ namespace hdfs {
 
 class DataNodeConnection : public AsyncStream {
 public:
-    std::string uuid_;
-    std::unique_ptr<hadoop::common::TokenProto> token_;
+  MEMCHECKED_CLASS(DataNodeConnection)
+  std::string uuid_;
+  std::unique_ptr<hadoop::common::TokenProto> token_;
 
-    virtual ~DataNodeConnection();
-    virtual void Connect(std::function<void(Status status, std::shared_ptr<DataNodeConnection> dn)> handler) = 0;
-    virtual void Cancel() = 0;
+  virtual ~DataNodeConnection();
+  virtual void Connect(std::function<void(Status status, std::shared_ptr<DataNodeConnection> dn)> handler) = 0;
+  virtual void Cancel() = 0;
 };
 
 

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.h

@@ -22,6 +22,7 @@
 #include "common/async_stream.h"
 #include "common/cancel_tracker.h"
 #include "common/libhdfs_events_impl.h"
+#include "common/new_delete.h"
 #include "reader/fileinfo.h"
 #include "reader/readergroup.h"
 
@@ -49,6 +50,7 @@ class DataNodeConnection;
  */
 class FileHandleImpl : public FileHandle {
 public:
+  MEMCHECKED_CLASS(FileHandleImpl)
   FileHandleImpl(const std::string & cluster_name,
                  const std::string & path,
                  ::asio::io_service *io_service, const std::string &client_name,

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h

@@ -22,6 +22,7 @@
 #include "common/libhdfs_events_impl.h"
 #include "common/hdfs_public_api.h"
 #include "common/async_stream.h"
+#include "common/new_delete.h"
 #include "hdfspp/hdfspp.h"
 #include "fs/bad_datanode_tracker.h"
 #include "rpc/rpc_engine.h"
@@ -47,6 +48,7 @@ namespace hdfs {
  */
 class NameNodeOperations {
 public:
+  MEMCHECKED_CLASS(NameNodeOperations);
   NameNodeOperations(::asio::io_service *io_service, const Options &options,
             const std::string &client_name, const std::string &user_name,
             const char *protocol_name, int protocol_version) :
@@ -83,6 +85,7 @@ private:
  */
 class FileSystemImpl : public FileSystem {
 public:
+  MEMCHECKED_CLASS(FileSystemImpl)
   FileSystemImpl(IoService *&io_service, const std::string& user_name, const Options &options);
   ~FileSystemImpl() override;
 

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h

@@ -21,6 +21,7 @@
 #include "hdfspp/status.h"
 #include "common/async_stream.h"
 #include "common/cancel_tracker.h"
+#include "common/new_delete.h"
 #include "datatransfer.pb.h"
 #include "connection/datanodeconnection.h"
 
@@ -67,6 +68,7 @@ struct BlockReaderOptions {
  */
 class BlockReader {
 public:
+  MEMCHECKED_CLASS(BlockReader)
   virtual void AsyncReadBlock(
     const std::string & client_name,
     const hadoop::hdfs::LocatedBlockProto &block, size_t offset,

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_engine.h

@@ -23,6 +23,7 @@
 
 #include "common/retry_policy.h"
 #include "common/libhdfs_events_impl.h"
+#include "common/new_delete.h"
 
 #include <google/protobuf/message_lite.h>
 #include <google/protobuf/io/coded_stream.h>
@@ -64,6 +65,7 @@ class RpcConnection;
  */
 class Request {
  public:
+  MEMCHECKED_CLASS(Request)
   typedef std::function<void(::google::protobuf::io::CodedInputStream *is,
                              const Status &status)> Handler;
 
@@ -108,6 +110,7 @@ class Request {
  */
 class RpcConnection : public std::enable_shared_from_this<RpcConnection> {
  public:
+  MEMCHECKED_CLASS(RpcConnection)
   RpcConnection(LockFreeRpcEngine *engine);
   virtual ~RpcConnection();
 
@@ -212,6 +215,7 @@ class RpcConnection : public std::enable_shared_from_this<RpcConnection> {
  */
 class LockFreeRpcEngine {
 public:
+  MEMCHECKED_CLASS(LockFreeRpcEngine)
   /* Enqueues a CommsError without acquiring a lock*/
   virtual void AsyncRpcCommsError(const Status &status,
                       std::shared_ptr<RpcConnection> failedConnection,
@@ -238,6 +242,7 @@ public:
  */
 class RpcEngine : public LockFreeRpcEngine {
  public:
+  MEMCHECKED_CLASS(RpcEngine)
   enum { kRpcVersion = 9 };
   enum {
     kCallIdAuthorizationFailed = -1,