You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by xk...@apache.org on 2018/05/04 19:28:00 UTC

[32/50] [abbrv] hadoop git commit: HDFS-11807. libhdfs++: Get minidfscluster tests running under valgrind. Contributed by Anatoli Shein.

HDFS-11807. libhdfs++: Get minidfscluster tests running under valgrind.  Contributed by Anatoli Shein.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/19ae588f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/19ae588f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/19ae588f

Branch: refs/heads/HDFS-12943
Commit: 19ae588fde9930c042cdb2848b8a1a0ff514b575
Parents: fe649bb
Author: James Clampffer <jh...@apache.org>
Authored: Wed May 2 11:49:12 2018 -0400
Committer: James Clampffer <jh...@apache.org>
Committed: Wed May 2 11:49:12 2018 -0400

----------------------------------------------------------------------
 .../src/main/native/libhdfs-tests/expect.h      |  60 +++++
 .../libhdfs-tests/test_libhdfs_mini_stress.c    | 253 ++++++++++++++-----
 .../src/main/native/libhdfspp/CMakeLists.txt    |   2 +-
 .../main/native/libhdfspp/tests/CMakeLists.txt  |   6 +
 .../main/native/libhdfspp/tests/memcheck.supp   |  27 ++
 5 files changed, 279 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/19ae588f/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/expect.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/expect.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/expect.h
index 528c96f..d843b67 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/expect.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/expect.h
@@ -132,6 +132,54 @@ struct hdfsFile_internal;
         } \
     } while (0);
 
+#define EXPECT_INT_LT(x, y) \
+    do { \
+        int __my_ret__ = x; \
+        int __my_errno__ = errno; \
+        if (__my_ret__ >= (y)) { \
+            fprintf(stderr, "TEST_ERROR: failed on %s:%d with return " \
+              "code %d (errno: %d): expected less than %d\n", \
+               __FILE__, __LINE__, __my_ret__, __my_errno__, (y)); \
+            return -1; \
+        } \
+    } while (0);
+
+#define EXPECT_INT_LE(x, y) \
+    do { \
+        int __my_ret__ = x; \
+        int __my_errno__ = errno; \
+        if (__my_ret__ > (y)) { \
+            fprintf(stderr, "TEST_ERROR: failed on %s:%d with return " \
+              "code %d (errno: %d): expected less than or equal %d\n", \
+               __FILE__, __LINE__, __my_ret__, __my_errno__, (y)); \
+            return -1; \
+        } \
+    } while (0);
+
+#define EXPECT_INT_GT(x, y) \
+    do { \
+        int __my_ret__ = x; \
+        int __my_errno__ = errno; \
+        if (__my_ret__ <= (y)) { \
+            fprintf(stderr, "TEST_ERROR: failed on %s:%d with return " \
+              "code %d (errno: %d): expected greater than %d\n", \
+               __FILE__, __LINE__, __my_ret__, __my_errno__, (y)); \
+            return -1; \
+        } \
+    } while (0);
+
+#define EXPECT_INT_GE(x, y) \
+    do { \
+        int __my_ret__ = x; \
+        int __my_errno__ = errno; \
+        if (__my_ret__ < (y)) { \
+            fprintf(stderr, "TEST_ERROR: failed on %s:%d with return " \
+              "code %d (errno: %d): expected greater than or equal %d\n", \
+               __FILE__, __LINE__, __my_ret__, __my_errno__, (y)); \
+            return -1; \
+        } \
+    } while (0);
+
 #define EXPECT_INT64_EQ(x, y) \
     do { \
         int64_t __my_ret__ = y; \
@@ -144,6 +192,18 @@ struct hdfsFile_internal;
         } \
     } while (0);
 
+#define ASSERT_INT64_EQ(x, y) \
+    do { \
+        int64_t __my_ret__ = y; \
+        int __my_errno__ = errno; \
+        if (__my_ret__ != (x)) { \
+            fprintf(stderr, "TEST_ERROR: failed on %s:%d with return " \
+              "value %"PRId64" (errno: %d): expected %"PRId64"\n", \
+               __FILE__, __LINE__, __my_ret__, __my_errno__, (x)); \
+            exit(EXIT_FAILURE); \
+        } \
+    } while (0);
+
 #define EXPECT_UINT64_EQ(x, y) \
     do { \
         uint64_t __my_ret__ = y; \

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19ae588f/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/test_libhdfs_mini_stress.c
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/test_libhdfs_mini_stress.c b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/test_libhdfs_mini_stress.c
index dca4782..9054287 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/test_libhdfs_mini_stress.c
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/test_libhdfs_mini_stress.c
@@ -16,6 +16,7 @@
  * limitations under the License.
  */
 
+#include "common/util_c.h"
 #include "expect.h"
 #include "hdfs/hdfs.h"
 #include "hdfspp/hdfs_ext.h"
@@ -24,10 +25,15 @@
 
 #include <errno.h>
 #include <inttypes.h>
+#include <pwd.h>
 #include <stdint.h>
 #include <stdio.h>
 #include <stdlib.h>
 #include <string.h>
+#include <sys/socket.h>
+#include <sys/types.h>
+#include <sys/wait.h>
+#include <unistd.h>
 
 #define TO_STR_HELPER(X) #X
 #define TO_STR(X) TO_STR_HELPER(X)
@@ -44,6 +50,8 @@
 
 #define TLH_DEFAULT_IPC_CLIENT_CONNECT_RETRY_INTERVAL_MS 5
 
+#define MAX_DIGITS_IN_INT 10
+
 #ifndef RANDOM_ERROR_RATIO
 #define RANDOM_ERROR_RATIO 1000000000
 #endif
@@ -62,15 +70,13 @@ struct tlhThreadInfo {
 
 };
 
-static int hdfsNameNodeConnect(struct NativeMiniDfsCluster *cl, hdfsFS *fs,
+static int hdfsNameNodeConnect(tPort port, hdfsFS *fs,
                                const char *username)
 {
   int ret;
-  tPort port;
   hdfsFS hdfs;
   struct hdfsBuilder *bld;
 
-  port = (tPort)nmdGetNameNodePort(cl);
   if (port < 0) {
     fprintf(stderr, "hdfsNameNodeConnect: nmdGetNameNodePort "
             "returned error %d\n", port);
@@ -104,6 +110,44 @@ static int hdfsNameNodeConnect(struct NativeMiniDfsCluster *cl, hdfsFS *fs,
   return 0;
 }
 
+#ifdef VALGRIND
+static int hdfsCurlData(const char *host, const tPort port, const char *dirNm,
+                         const char *fileNm, tSize fileSz)
+{
+  int ret;
+  const char *content;
+  content = fileNm;
+  char tmpFile[14] = "stress_XXXXXX";
+
+  // Retrieve user id (always successful)
+  uid_t uid = geteuid();
+  // Retrieve password struct entry
+  struct passwd *pw;
+  EXPECT_NONNULL(pw = getpwuid(uid));
+
+  int fd = -1;
+  EXPECT_NONNEGATIVE(fd = mkstemp(tmpFile));
+
+  tSize sz = 0;
+  while (sz < fileSz) {
+    EXPECT_NONNEGATIVE(ret = write(fd, content, strlen(content)));
+    sz += ret;
+  }
+
+  int curlSize = 200;
+  char curlStr[curlSize];
+  ret = snprintf(curlStr,curlSize,"curl -L -i -X PUT -T %s \"http://%s:%d/webhdfs/v1%s?op=CREATE&overwrite=true&user.name=%s\"",tmpFile,host,(int)port,fileNm,pw->pw_name);
+  //Check for errors during snprintf
+  EXPECT_NONNEGATIVE(ret);
+  //Check for truncation during snprintf
+  EXPECT_INT_LT(ret, curlSize);
+
+  EXPECT_ZERO(system(curlStr));
+  EXPECT_ZERO(unlink(tmpFile));
+
+  return 0;
+}
+#else
 static int hdfsWriteData(hdfsFS hdfs, const char *dirNm,
                          const char *fileNm, tSize fileSz)
 {
@@ -142,6 +186,7 @@ static int hdfsWriteData(hdfsFS hdfs, const char *dirNm,
   EXPECT_ZERO(hdfsCloseFile(hdfs, file));
   return 0;
 }
+#endif
 
 static int fileEventCallback1(const char * event, const char * cluster, const char * file, int64_t value, int64_t cookie)
 {
@@ -223,6 +268,7 @@ static int doTestHdfsMiniStress(struct tlhThreadInfo *ti, int randomErr)
   fprintf(stderr, "testHdfsMiniStress(threadIdx=%d): finished read loop\n",
           ti->threadIdx);
   EXPECT_ZERO(nErrs);
+  hdfsFreeFileInfo(fileInfo, 1);
   return 0;
 }
 
@@ -275,76 +321,147 @@ static int checkFailures(struct tlhThreadInfo *ti, int tlhNumThreads)
 
 /**
  * Test intended to stress libhdfs client with concurrent requests. Currently focused
- * on concurrent reads.
+ * on concurrent reads. In order to run this test under valgrind and avoid JVM issues
+ * we fork a child process that runs a mini dfs cluster, and the parent process
+ * communicates with it using a socket pair.
  */
-int main(void)
+int main(int argc, char *argv[])
 {
-  int i, tlhNumThreads;
-  char *dirNm, *fileNm;
-  tSize fileSz;
-  const char *tlhNumThreadsStr, *tlhNumDNsStr;
-  hdfsFS hdfs = NULL;
-  struct NativeMiniDfsCluster* tlhCluster;
-  struct tlhThreadInfo ti[TLH_MAX_THREADS];
-  struct NativeMiniDfsConf conf = {
+  tPort port;
+#ifdef VALGRIND
+  int httpPort;
+  char * httpHost;
+  size_t hostSize;
+  int fds[2];
+  static const int parentsocket = 0;
+  static const int childsocket = 1;
+  int status;
+  // If there is an argument, the child code executes and starts a mini dfs cluster
+  if (argc > 1) {
+    // The argument contains child socket
+    fds[childsocket] = (int) strtol(argv[1],NULL,10);
+#endif
+    const char *tlhNumDNsStr;
+    struct NativeMiniDfsCluster* tlhCluster;
+    struct NativeMiniDfsConf conf = {
       1, /* doFormat */
-  };
-
-  dirNm = "/tlhMiniStressData";
-  fileNm = "/tlhMiniStressData/file";
-  fileSz = 2*1024*1024;
-
-  tlhNumDNsStr = getenv("TLH_NUM_DNS");
-  if (!tlhNumDNsStr) {
-    tlhNumDNsStr = "1";
-  }
-  conf.numDataNodes = atoi(tlhNumDNsStr);
-  if ((conf.numDataNodes <= 0) || (conf.numDataNodes > TLH_MAX_DNS)) {
-    fprintf(stderr, "testLibHdfsMiniStress: must have a number of datanodes "
-            "between 1 and %d inclusive, not %d\n",
-            TLH_MAX_DNS, conf.numDataNodes);
-    return EXIT_FAILURE;
-  }
-
-  tlhNumThreadsStr = getenv("TLH_NUM_THREADS");
-  if (!tlhNumThreadsStr) {
-    tlhNumThreadsStr = "8";
-  }
-  tlhNumThreads = atoi(tlhNumThreadsStr);
-  if ((tlhNumThreads <= 0) || (tlhNumThreads > TLH_MAX_THREADS)) {
-    fprintf(stderr, "testLibHdfsMiniStress: must have a number of threads "
-            "between 1 and %d inclusive, not %d\n",
-            TLH_MAX_THREADS, tlhNumThreads);
-    return EXIT_FAILURE;
-  }
-  memset(&ti[0], 0, sizeof(ti));
-  for (i = 0; i < tlhNumThreads; i++) {
-    ti[i].threadIdx = i;
-  }
-
-  tlhCluster = nmdCreate(&conf);
-  EXPECT_NONNULL(tlhCluster);
-  EXPECT_ZERO(nmdWaitClusterUp(tlhCluster));
+      1, /* webhdfs */
+      0, /* webhdfs port */
+      1  /* shortcircuit */
+    };
+    tlhNumDNsStr = getenv("TLH_NUM_DNS");
+    if (!tlhNumDNsStr) {
+      tlhNumDNsStr = "1";
+    }
+    conf.numDataNodes = atoi(tlhNumDNsStr);
+    if ((conf.numDataNodes <= 0) || (conf.numDataNodes > TLH_MAX_DNS)) {
+      fprintf(stderr, "testLibHdfsMiniStress: must have a number of datanodes "
+              "between 1 and %d inclusive, not %d\n",
+              TLH_MAX_DNS, conf.numDataNodes);
+      return EXIT_FAILURE;
+    }
+    tlhCluster = nmdCreate(&conf);
+    EXPECT_NONNULL(tlhCluster);
+    EXPECT_ZERO(nmdWaitClusterUp(tlhCluster));
+    port = (tPort)nmdGetNameNodePort(tlhCluster);
+#ifdef VALGRIND
+    EXPECT_ZERO(nmdGetNameNodeHttpAddress(tlhCluster, &httpPort, (const char **) &httpHost));
+    hostSize = strlen(httpHost) + 1;
+    // The child is sending hdfs port, webhdfs port, hostname size, and hostname to the parent
+    ASSERT_INT64_EQ(write(fds[childsocket], &port, sizeof(tPort)), sizeof(tPort));
+    ASSERT_INT64_EQ(write(fds[childsocket], &httpPort, sizeof(int)), sizeof(int));
+    ASSERT_INT64_EQ(write(fds[childsocket], &hostSize, sizeof(size_t)), sizeof(size_t));
+    ASSERT_INT64_EQ(write(fds[childsocket], httpHost, hostSize), hostSize);
+    free(httpHost);
+    // The child is waiting for the parent to finish and send a message
+    ASSERT_INT64_EQ(read(fds[childsocket], &port, sizeof(tPort)), sizeof(tPort));
+    EXPECT_ZERO(nmdShutdown(tlhCluster));
+    nmdFree(tlhCluster);
+  } else { // If there is no argument, the parent code executes
+#endif
+    hdfsFS hdfs = NULL;
+    int i, tlhNumThreads;
+    char *dirNm, *fileNm;
+    tSize fileSz;
+    const char *tlhNumThreadsStr;
+    struct tlhThreadInfo ti[TLH_MAX_THREADS];
+
+    dirNm = "/tlhMiniStressData";
+    fileNm = "/tlhMiniStressData/file";
+    fileSz = 2*1024*1024;
+
+    tlhNumThreadsStr = getenv("TLH_NUM_THREADS");
+    if (!tlhNumThreadsStr) {
+      tlhNumThreadsStr = "8";
+    }
+    tlhNumThreads = atoi(tlhNumThreadsStr);
+    if ((tlhNumThreads <= 0) || (tlhNumThreads > TLH_MAX_THREADS)) {
+      fprintf(stderr, "testLibHdfsMiniStress: must have a number of threads "
+              "between 1 and %d inclusive, not %d\n",
+              TLH_MAX_THREADS, tlhNumThreads);
+      return EXIT_FAILURE;
+    }
+    memset(&ti[0], 0, sizeof(ti));
+    for (i = 0; i < tlhNumThreads; i++) {
+      ti[i].threadIdx = i;
+    }
 
-  EXPECT_ZERO(hdfsNameNodeConnect(tlhCluster, &hdfs, NULL));
+#ifdef VALGRIND
+    EXPECT_ZERO(socketpair(PF_LOCAL, SOCK_STREAM, 0, fds));
+    // Forking off a child to execute JVM stuff
+    pid_t pid = fork();
+    if(pid == 0){
+      // The child execs this program from the beginning and passes
+      // its file descriptor as a command line argument.
+      char ch_fd[MAX_DIGITS_IN_INT + 1];
+      sprintf(ch_fd, "%d", fds[childsocket]);
+      // This has to be done with exec() to prevent valgrind from tracing the child
+      execl(argv[0], argv[0], ch_fd, NULL);
+      // This code should never execute
+      fprintf(stderr, "execl() failed.\n");
+      return EXIT_FAILURE;
+    }
+    close(fds[childsocket]);
+    // The parent is receiving hdfs port, webhdfs port, hostname size, and hostname from the child
+    ASSERT_INT64_EQ(read(fds[parentsocket], &port, sizeof(tPort)), sizeof(tPort));
+    ASSERT_INT64_EQ(read(fds[parentsocket], &httpPort, sizeof(int)), sizeof(int));
+    ASSERT_INT64_EQ(read(fds[parentsocket], &hostSize, sizeof(size_t)), sizeof(size_t));
+    httpHost = malloc(hostSize);
+    ASSERT_INT64_EQ(read(fds[parentsocket], httpHost, hostSize), hostSize);
+    EXPECT_ZERO(hdfsNameNodeConnect(port, &hdfs, NULL));
+    EXPECT_ZERO(hdfsCurlData(httpHost, httpPort, dirNm, fileNm, fileSz));
+    free(httpHost);
+#else
+    EXPECT_ZERO(hdfsNameNodeConnect(port, &hdfs, NULL));
+    EXPECT_ZERO(hdfsWriteData(hdfs, dirNm, fileNm, fileSz));
+#endif
 
-  // Single threaded writes for now.
-  EXPECT_ZERO(hdfsWriteData(hdfs, dirNm, fileNm, fileSz));
+    // Multi-threaded reads.
+    for (i = 0; i < tlhNumThreads; i++) {
+      ti[i].theThread.start = testHdfsMiniStress;
+      ti[i].theThread.arg = &ti[i];
+      ti[i].hdfs = hdfs;
+      ti[i].fileNm = fileNm;
+      EXPECT_ZERO(threadCreate(&ti[i].theThread));
+    }
+    for (i = 0; i < tlhNumThreads; i++) {
+      EXPECT_ZERO(threadJoin(&ti[i].theThread));
+    }
 
-  // Multi-threaded reads.
-  for (i = 0; i < tlhNumThreads; i++) {
-    ti[i].theThread.start = testHdfsMiniStress;
-    ti[i].theThread.arg = &ti[i];
-    ti[i].hdfs = hdfs;
-    ti[i].fileNm = fileNm;
-    EXPECT_ZERO(threadCreate(&ti[i].theThread));
-  }
-  for (i = 0; i < tlhNumThreads; i++) {
-    EXPECT_ZERO(threadJoin(&ti[i].theThread));
+    EXPECT_ZERO(hdfsDisconnect(hdfs));
+    EXPECT_ZERO(checkFailures(ti, tlhNumThreads));
+#ifdef VALGRIND
+    //Send this message to the child to notify it that it can now shut down
+    ASSERT_INT64_EQ(write(fds[parentsocket], &port, sizeof(tPort)), sizeof(tPort));
+    // Wait for the child to exit and verify it returned EXIT_SUCCESS
+    waitpid(pid, &status, 0);
+    EXPECT_ZERO(status);
   }
-
-  EXPECT_ZERO(hdfsDisconnect(hdfs));
-  EXPECT_ZERO(nmdShutdown(tlhCluster));
-  nmdFree(tlhCluster);
-  return checkFailures(ti, tlhNumThreads);
+#else
+    EXPECT_ZERO(nmdShutdown(tlhCluster));
+    nmdFree(tlhCluster);
+#endif
+  // Clean up static data and prevent valgrind memory leaks
+  ShutdownProtobufLibrary_C();
+  return EXIT_SUCCESS;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19ae588f/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/CMakeLists.txt
index 94b1b56..63fa80d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/CMakeLists.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/CMakeLists.txt
@@ -90,7 +90,7 @@ if (NOT PROTOC_IS_COMPATIBLE)
 endif (NOT PROTOC_IS_COMPATIBLE)
 
 find_program(MEMORYCHECK_COMMAND valgrind HINTS ${VALGRIND_DIR} )
-set(MEMORYCHECK_COMMAND_OPTIONS "--trace-children=yes --leak-check=full --error-exitcode=1")
+set(MEMORYCHECK_COMMAND_OPTIONS "--trace-children=no --leak-check=full --error-exitcode=1 --suppressions=${PROJECT_SOURCE_DIR}/tests/memcheck.supp")
 message(STATUS "valgrind location: ${MEMORYCHECK_COMMAND}")
 
 if (REQUIRE_VALGRIND AND MEMORYCHECK_COMMAND MATCHES "MEMORYCHECK_COMMAND-NOTFOUND" )

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19ae588f/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/CMakeLists.txt
index 3331935..6157902 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/CMakeLists.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/CMakeLists.txt
@@ -141,6 +141,7 @@ include_directories (
 )
 
 add_library(hdfspp_test_shim_static STATIC hdfs_shim.c libhdfs_wrapper.c libhdfspp_wrapper.cc ${LIBHDFSPP_BINDING_C}/hdfs.cc)
+add_library(hdfspp_test_static STATIC ${LIBHDFSPP_BINDING_C}/hdfs.cc)
 
 # TODO: get all of the mini dfs library bits here in one place
 # add_library(hdfspp_mini_cluster     native_mini_dfs ${JAVA_JVM_LIBRARY} )
@@ -155,6 +156,11 @@ build_libhdfs_test(hdfspp_mini_dfs_smoke hdfspp_test_shim_static ${CMAKE_CURRENT
 link_libhdfs_test (hdfspp_mini_dfs_smoke hdfspp_test_shim_static fs reader rpc proto common connection gmock_main ${PROTOBUF_LIBRARIES} ${OPENSSL_LIBRARIES} native_mini_dfs ${JAVA_JVM_LIBRARY} ${SASL_LIBRARIES})
 add_libhdfs_test  (hdfspp_mini_dfs_smoke hdfspp_test_shim_static)
 
+build_libhdfs_test(libhdfs_mini_stress_valgrind hdfspp_test_static expect.c test_libhdfs_mini_stress.c ${OS_DIR}/thread.c)
+link_libhdfs_test(libhdfs_mini_stress_valgrind hdfspp_test_static fs reader rpc proto common connection ${PROTOBUF_LIBRARIES} ${OPENSSL_LIBRARIES} native_mini_dfs ${JAVA_JVM_LIBRARY} ${SASL_LIBRARIES})
+add_memcheck_test(libhdfs_mini_stress_valgrind_hdfspp_test_static libhdfs_mini_stress_valgrind_hdfspp_test_static)
+set_target_properties(libhdfs_mini_stress_valgrind_hdfspp_test_static PROPERTIES COMPILE_DEFINITIONS "VALGRIND")
+
 build_libhdfs_test(libhdfs_mini_stress hdfspp_test_shim_static expect.c test_libhdfs_mini_stress.c ${OS_DIR}/thread.c)
 link_libhdfs_test(libhdfs_mini_stress hdfspp_test_shim_static fs reader rpc proto common connection ${PROTOBUF_LIBRARIES} ${OPENSSL_LIBRARIES} native_mini_dfs ${JAVA_JVM_LIBRARY} ${SASL_LIBRARIES})
 add_libhdfs_test(libhdfs_mini_stress hdfspp_test_shim_static)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19ae588f/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/memcheck.supp
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/memcheck.supp b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/memcheck.supp
new file mode 100644
index 0000000..cf80d07
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/memcheck.supp
@@ -0,0 +1,27 @@
+#
+# 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.
+
+{
+   noai6ai_cached suppression
+   Memcheck:Free
+   fun:free
+   fun:__libc_freeres
+   fun:_vgnU_freeres
+   fun:__run_exit_handlers
+   fun:exit
+   ...
+}
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org