You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by cm...@apache.org on 2013/12/04 00:13:02 UTC

svn commit: r1547637 - in /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs: CHANGES.txt src/contrib/libwebhdfs/src/hdfs_json_parser.c src/main/native/libhdfs/exception.c src/main/native/libhdfs/test_libhdfs_threaded.c

Author: cmccabe
Date: Tue Dec  3 23:13:02 2013
New Revision: 1547637

URL: http://svn.apache.org/r1547637
Log:
HDFS-4997. libhdfs does not return correct error code in most cases (cmccabe)

Modified:
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_json_parser.c
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/exception.c
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/test_libhdfs_threaded.c

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt?rev=1547637&r1=1547636&r2=1547637&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt Tue Dec  3 23:13:02 2013
@@ -767,6 +767,8 @@ Release 2.3.0 - UNRELEASED
     HDFS-5563. NFS gateway should commit the buffered data when read request comes
     after write to the same file (brandonli)
 
+    HDFS-4997. libhdfs doesn't return correct error codes in most cases (cmccabe)
+
 Release 2.2.0 - 2013-10-13
 
   INCOMPATIBLE CHANGES

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_json_parser.c
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_json_parser.c?rev=1547637&r1=1547636&r2=1547637&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_json_parser.c (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_json_parser.c Tue Dec  3 23:13:02 2013
@@ -39,14 +39,6 @@ struct jsonException {
   const char *message;
 };
 
-static void dotsToSlashes(char *str)
-{
-    for (; *str != '\0'; str++) {
-        if (*str == '.')
-            *str = '/';
-    }
-}
-
 /** Print out the JSON exception information */
 static int printJsonExceptionV(struct jsonException *exc, int noPrintFlags,
                                const char *fmt, va_list ap)
@@ -62,7 +54,6 @@ static int printJsonExceptionV(struct js
         fprintf(stderr, "printJsonExceptionV: internal out of memory error\n");
         return EINTERNAL;
     }
-    dotsToSlashes(javaClassName);
     getExceptionInfo(javaClassName, noPrintFlags, &excErrno, &shouldPrint);
     free(javaClassName);
     

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/exception.c
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/exception.c?rev=1547637&r1=1547636&r2=1547637&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/exception.c (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/exception.c Tue Dec  3 23:13:02 2013
@@ -35,36 +35,55 @@ struct ExceptionInfo {
 
 static const struct ExceptionInfo gExceptionInfo[] = {
     {
-        .name = "java/io/FileNotFoundException",
+        .name = "java.io.FileNotFoundException",
         .noPrintFlag = NOPRINT_EXC_FILE_NOT_FOUND,
         .excErrno = ENOENT,
     },
     {
-        .name = "org/apache/hadoop/security/AccessControlException",
+        .name = "org.apache.hadoop.security.AccessControlException",
         .noPrintFlag = NOPRINT_EXC_ACCESS_CONTROL,
         .excErrno = EACCES,
     },
     {
-        .name = "org/apache/hadoop/fs/UnresolvedLinkException",
+        .name = "org.apache.hadoop.fs.UnresolvedLinkException",
         .noPrintFlag = NOPRINT_EXC_UNRESOLVED_LINK,
         .excErrno = ENOLINK,
     },
     {
-        .name = "org/apache/hadoop/fs/ParentNotDirectoryException",
+        .name = "org.apache.hadoop.fs.ParentNotDirectoryException",
         .noPrintFlag = NOPRINT_EXC_PARENT_NOT_DIRECTORY,
         .excErrno = ENOTDIR,
     },
     {
-        .name = "java/lang/IllegalArgumentException",
+        .name = "java.lang.IllegalArgumentException",
         .noPrintFlag = NOPRINT_EXC_ILLEGAL_ARGUMENT,
         .excErrno = EINVAL,
     },
     {
-        .name = "java/lang/OutOfMemoryError",
+        .name = "java.lang.OutOfMemoryError",
         .noPrintFlag = 0,
         .excErrno = ENOMEM,
     },
-    
+    {
+        .name = "org.apache.hadoop.hdfs.server.namenode.SafeModeException",
+        .noPrintFlag = 0,
+        .excErrno = EROFS,
+    },
+    {
+        .name = "org.apache.hadoop.fs.FileAlreadyExistsException",
+        .noPrintFlag = 0,
+        .excErrno = EEXIST,
+    },
+    {
+        .name = "org.apache.hadoop.hdfs.protocol.QuotaExceededException",
+        .noPrintFlag = 0,
+        .excErrno = EDQUOT,
+    },
+    {
+        .name = "org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException",
+        .noPrintFlag = 0,
+        .excErrno = ESTALE,
+    },
 };
 
 void getExceptionInfo(const char *excName, int noPrintFlags,

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/test_libhdfs_threaded.c
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/test_libhdfs_threaded.c?rev=1547637&r1=1547636&r2=1547637&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/test_libhdfs_threaded.c (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/test_libhdfs_threaded.c Tue Dec  3 23:13:02 2013
@@ -48,7 +48,8 @@ struct tlhThreadInfo {
     pthread_t thread;
 };
 
-static int hdfsSingleNameNodeConnect(struct NativeMiniDfsCluster *cl, hdfsFS *fs)
+static int hdfsSingleNameNodeConnect(struct NativeMiniDfsCluster *cl, hdfsFS *fs,
+                                     const char *username)
 {
     int ret, port;
     hdfsFS hdfs;
@@ -70,6 +71,9 @@ static int hdfsSingleNameNodeConnect(str
                           TO_STR(TLH_DEFAULT_BLOCK_SIZE));
     hdfsBuilderConfSetStr(bld, "dfs.blocksize",
                           TO_STR(TLH_DEFAULT_BLOCK_SIZE));
+    if (username) {
+        hdfsBuilderSetUserName(bld, username);
+    }
     hdfs = hdfsBuilderConnect(bld);
     if (!hdfs) {
         ret = -errno;
@@ -110,36 +114,58 @@ static int doTestGetDefaultBlockSize(hdf
     return 0;
 }
 
-static int doTestHdfsOperations(struct tlhThreadInfo *ti, hdfsFS fs)
+struct tlhPaths {
+    char prefix[256];
+    char file1[256];
+    char file2[256];
+};
+
+static int setupPaths(const struct tlhThreadInfo *ti, struct tlhPaths *paths)
 {
-    char prefix[256], tmp[256];
+    memset(paths, sizeof(*paths), 0);
+    if (snprintf(paths->prefix, sizeof(paths->prefix), "/tlhData%04d",
+                 ti->threadIdx) >= sizeof(paths->prefix)) {
+        return ENAMETOOLONG;
+    }
+    if (snprintf(paths->file1, sizeof(paths->file1), "%s/file1",
+                 paths->prefix) >= sizeof(paths->file1)) {
+        return ENAMETOOLONG;
+    }
+    if (snprintf(paths->file2, sizeof(paths->file2), "%s/file2",
+                 paths->prefix) >= sizeof(paths->file2)) {
+        return ENAMETOOLONG;
+    }
+    return 0;
+}
+
+static int doTestHdfsOperations(struct tlhThreadInfo *ti, hdfsFS fs,
+                                const struct tlhPaths *paths)
+{
+    char tmp[4096];
     hdfsFile file;
     int ret, expected;
     hdfsFileInfo *fileInfo;
     struct hdfsReadStatistics *readStats = NULL;
 
-    snprintf(prefix, sizeof(prefix), "/tlhData%04d", ti->threadIdx);
-
-    if (hdfsExists(fs, prefix) == 0) {
-        EXPECT_ZERO(hdfsDelete(fs, prefix, 1));
+    if (hdfsExists(fs, paths->prefix) == 0) {
+        EXPECT_ZERO(hdfsDelete(fs, paths->prefix, 1));
     }
-    EXPECT_ZERO(hdfsCreateDirectory(fs, prefix));
-    snprintf(tmp, sizeof(tmp), "%s/file", prefix);
+    EXPECT_ZERO(hdfsCreateDirectory(fs, paths->prefix));
 
-    EXPECT_ZERO(doTestGetDefaultBlockSize(fs, prefix));
+    EXPECT_ZERO(doTestGetDefaultBlockSize(fs, paths->prefix));
 
     /* There should not be any file to open for reading. */
-    EXPECT_NULL(hdfsOpenFile(fs, tmp, O_RDONLY, 0, 0, 0));
+    EXPECT_NULL(hdfsOpenFile(fs, paths->file1, O_RDONLY, 0, 0, 0));
 
     /* hdfsOpenFile should not accept mode = 3 */
-    EXPECT_NULL(hdfsOpenFile(fs, tmp, 3, 0, 0, 0));
+    EXPECT_NULL(hdfsOpenFile(fs, paths->file1, 3, 0, 0, 0));
 
-    file = hdfsOpenFile(fs, tmp, O_WRONLY, 0, 0, 0);
+    file = hdfsOpenFile(fs, paths->file1, O_WRONLY, 0, 0, 0);
     EXPECT_NONNULL(file);
 
     /* TODO: implement writeFully and use it here */
-    expected = strlen(prefix);
-    ret = hdfsWrite(fs, file, prefix, expected);
+    expected = strlen(paths->prefix);
+    ret = hdfsWrite(fs, file, paths->prefix, expected);
     if (ret < 0) {
         ret = errno;
         fprintf(stderr, "hdfsWrite failed and set errno %d\n", ret);
@@ -155,7 +181,7 @@ static int doTestHdfsOperations(struct t
     EXPECT_ZERO(hdfsCloseFile(fs, file));
 
     /* Let's re-open the file for reading */
-    file = hdfsOpenFile(fs, tmp, O_RDONLY, 0, 0, 0);
+    file = hdfsOpenFile(fs, paths->file1, O_RDONLY, 0, 0, 0);
     EXPECT_NONNULL(file);
 
     EXPECT_ZERO(hdfsFileGetReadStatistics(file, &readStats));
@@ -180,60 +206,67 @@ static int doTestHdfsOperations(struct t
     errno = 0;
     EXPECT_INT_EQ(expected, readStats->totalBytesRead);
     hdfsFileFreeReadStatistics(readStats);
-    EXPECT_ZERO(memcmp(prefix, tmp, expected));
+    EXPECT_ZERO(memcmp(paths->prefix, tmp, expected));
     EXPECT_ZERO(hdfsCloseFile(fs, file));
 
     // TODO: Non-recursive delete should fail?
     //EXPECT_NONZERO(hdfsDelete(fs, prefix, 0));
+    EXPECT_ZERO(hdfsCopy(fs, paths->file1, fs, paths->file2));
 
-    snprintf(tmp, sizeof(tmp), "%s/file", prefix);
-    EXPECT_ZERO(hdfsChown(fs, tmp, NULL, NULL));
-    EXPECT_ZERO(hdfsChown(fs, tmp, NULL, "doop"));
-    fileInfo = hdfsGetPathInfo(fs, tmp);
+    EXPECT_ZERO(hdfsChown(fs, paths->file2, NULL, NULL));
+    EXPECT_ZERO(hdfsChown(fs, paths->file2, NULL, "doop"));
+    fileInfo = hdfsGetPathInfo(fs, paths->file2);
     EXPECT_NONNULL(fileInfo);
     EXPECT_ZERO(strcmp("doop", fileInfo->mGroup));
     hdfsFreeFileInfo(fileInfo, 1);
 
-    EXPECT_ZERO(hdfsChown(fs, tmp, "ha", "doop2"));
-    fileInfo = hdfsGetPathInfo(fs, tmp);
+    EXPECT_ZERO(hdfsChown(fs, paths->file2, "ha", "doop2"));
+    fileInfo = hdfsGetPathInfo(fs, paths->file2);
     EXPECT_NONNULL(fileInfo);
     EXPECT_ZERO(strcmp("ha", fileInfo->mOwner));
     EXPECT_ZERO(strcmp("doop2", fileInfo->mGroup));
     hdfsFreeFileInfo(fileInfo, 1);
 
-    EXPECT_ZERO(hdfsChown(fs, tmp, "ha2", NULL));
-    fileInfo = hdfsGetPathInfo(fs, tmp);
+    EXPECT_ZERO(hdfsChown(fs, paths->file2, "ha2", NULL));
+    fileInfo = hdfsGetPathInfo(fs, paths->file2);
     EXPECT_NONNULL(fileInfo);
     EXPECT_ZERO(strcmp("ha2", fileInfo->mOwner));
     EXPECT_ZERO(strcmp("doop2", fileInfo->mGroup));
     hdfsFreeFileInfo(fileInfo, 1);
 
-    EXPECT_ZERO(hdfsDelete(fs, prefix, 1));
+    snprintf(tmp, sizeof(tmp), "%s/nonexistent-file-name", paths->prefix);
+    EXPECT_NEGATIVE_ONE_WITH_ERRNO(hdfsChown(fs, tmp, "ha3", NULL), ENOENT);
     return 0;
 }
 
-static void *testHdfsOperations(void *v)
+static int testHdfsOperationsImpl(struct tlhThreadInfo *ti)
 {
-    struct tlhThreadInfo *ti = (struct tlhThreadInfo*)v;
     hdfsFS fs = NULL;
-    int ret;
+    struct tlhPaths paths;
 
     fprintf(stderr, "testHdfsOperations(threadIdx=%d): starting\n",
         ti->threadIdx);
-    ret = hdfsSingleNameNodeConnect(tlhCluster, &fs);
-    if (ret) {
-        fprintf(stderr, "testHdfsOperations(threadIdx=%d): "
-            "hdfsSingleNameNodeConnect failed with error %d.\n",
-            ti->threadIdx, ret);
-        ti->success = EIO;
-        return NULL;
-    }
-    ti->success = doTestHdfsOperations(ti, fs);
-    if (hdfsDisconnect(fs)) {
-        ret = errno;
-        fprintf(stderr, "hdfsDisconnect error %d\n", ret);
-        ti->success = ret;
-    }
+    EXPECT_ZERO(hdfsSingleNameNodeConnect(tlhCluster, &fs, NULL));
+    EXPECT_ZERO(setupPaths(ti, &paths));
+    // test some operations
+    EXPECT_ZERO(doTestHdfsOperations(ti, fs, &paths));
+    EXPECT_ZERO(hdfsDisconnect(fs));
+    // reconnect as user "foo" and verify that we get permission errors
+    EXPECT_ZERO(hdfsSingleNameNodeConnect(tlhCluster, &fs, "foo"));
+    EXPECT_NEGATIVE_ONE_WITH_ERRNO(hdfsChown(fs, paths.file1, "ha3", NULL), EACCES);
+    EXPECT_ZERO(hdfsDisconnect(fs));
+    // reconnect to do the final delete.
+    EXPECT_ZERO(hdfsSingleNameNodeConnect(tlhCluster, &fs, NULL));
+    EXPECT_ZERO(hdfsDelete(fs, paths.prefix, 1));
+    EXPECT_ZERO(hdfsDisconnect(fs));
+    return 0;
+}
+
+static void *testHdfsOperations(void *v)
+{
+    struct tlhThreadInfo *ti = (struct tlhThreadInfo*)v;
+    int ret = testHdfsOperationsImpl(ti);
+    ti->success = ret;
     return NULL;
 }