瀏覽代碼

HDFS-3539. libhdfs code cleanups. Contributed by Colin Patrick McCabe

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1361461 13f79535-47bb-0310-9956-ffa450edef68
Eli Collins 13 年之前
父節點
當前提交
2e63754888

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -143,6 +143,8 @@ Release 2.0.1-alpha - UNRELEASED
     HDFS-3606. libhdfs: create self-contained unit test.
     HDFS-3606. libhdfs: create self-contained unit test.
     (Colin Patrick McCabe via eli)
     (Colin Patrick McCabe via eli)
 
 
+    HDFS-3539. libhdfs code cleanups. (Colin Patrick McCabe via eli)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     HDFS-2982. Startup performance suffers when there are many edit log
     HDFS-2982. Startup performance suffers when there are many edit log

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/fuse-dfs/src/fuse_connect.c

@@ -23,6 +23,8 @@
 
 
 #include <limits.h>
 #include <limits.h>
 #include <search.h>
 #include <search.h>
+#include <stdio.h>
+#include <stdlib.h>
 
 
 #define HADOOP_SECURITY_AUTHENTICATION "hadoop.security.authentication"
 #define HADOOP_SECURITY_AUTHENTICATION "hadoop.security.authentication"
 
 

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/fuse-dfs/src/fuse_dfs.c

@@ -22,6 +22,9 @@
 #include "fuse_init.h"
 #include "fuse_init.h"
 #include "fuse_connect.h"
 #include "fuse_connect.h"
 
 
+#include <string.h>
+#include <stdlib.h>
+
 int is_protected(const char *path) {
 int is_protected(const char *path) {
 
 
   dfs_context *dfs = (dfs_context*)fuse_get_context()->private_data;
   dfs_context *dfs = (dfs_context*)fuse_get_context()->private_data;

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/contrib/fuse-dfs/src/fuse_impls_chown.c

@@ -21,7 +21,9 @@
 #include "fuse_impls.h"
 #include "fuse_impls.h"
 #include "fuse_connect.h"
 #include "fuse_connect.h"
 
 
- int dfs_chown(const char *path, uid_t uid, gid_t gid)
+#include <stdlib.h>
+
+int dfs_chown(const char *path, uid_t uid, gid_t gid)
 {
 {
   TRACE1("chown", path)
   TRACE1("chown", path)
 
 

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/fuse-dfs/src/fuse_impls_release.c

@@ -21,6 +21,8 @@
 #include "fuse_file_handle.h"
 #include "fuse_file_handle.h"
 #include "fuse_connect.h"
 #include "fuse_connect.h"
 
 
+#include <stdlib.h>
+
 /**
 /**
  * release a fuse_file_info structure.
  * release a fuse_file_info structure.
  *
  *

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/contrib/fuse-dfs/src/fuse_init.c

@@ -16,14 +16,16 @@
  * limitations under the License.
  * limitations under the License.
  */
  */
 
 
-#include <strings.h>
-
 #include "fuse_dfs.h"
 #include "fuse_dfs.h"
 #include "fuse_init.h"
 #include "fuse_init.h"
 #include "fuse_options.h"
 #include "fuse_options.h"
 #include "fuse_context_handle.h"
 #include "fuse_context_handle.h"
 #include "fuse_connect.h"
 #include "fuse_connect.h"
 
 
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
 // Hacked up function to basically do:
 // Hacked up function to basically do:
 //  protectedpaths = split(options.protected,':');
 //  protectedpaths = split(options.protected,':');
 
 

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/contrib/fuse-dfs/src/fuse_options.c

@@ -16,11 +16,12 @@
  * limitations under the License.
  * limitations under the License.
  */
  */
 
 
+#include "fuse_context_handle.h"
 #include "fuse_dfs.h"
 #include "fuse_dfs.h"
 #include "fuse_options.h"
 #include "fuse_options.h"
-#include <getopt.h>
 
 
-#include "fuse_context_handle.h"
+#include <getopt.h>
+#include <stdlib.h>
 
 
 void print_options() {
 void print_options() {
   printf("options:\n"
   printf("options:\n"

+ 47 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/native/hdfs.c

@@ -19,6 +19,8 @@
 #include "hdfs.h"
 #include "hdfs.h"
 #include "hdfsJniHelper.h"
 #include "hdfsJniHelper.h"
 
 
+#include <stdio.h>
+#include <string.h>
 
 
 /* Some frequently used Java paths */
 /* Some frequently used Java paths */
 #define HADOOP_CONF     "org/apache/hadoop/conf/Configuration"
 #define HADOOP_CONF     "org/apache/hadoop/conf/Configuration"
@@ -47,6 +49,50 @@
 
 
 #define KERBEROS_TICKET_CACHE_PATH "hadoop.security.kerberos.ticket.cache.path"
 #define KERBEROS_TICKET_CACHE_PATH "hadoop.security.kerberos.ticket.cache.path"
 
 
+// Bit fields for hdfsFile_internal flags
+#define HDFS_FILE_SUPPORTS_DIRECT_READ (1<<0)
+
+tSize readDirect(hdfsFS fs, hdfsFile f, void* buffer, tSize length);
+
+/**
+ * The C equivalent of org.apache.org.hadoop.FSData(Input|Output)Stream .
+ */
+enum hdfsStreamType
+{
+    UNINITIALIZED = 0,
+    INPUT = 1,
+    OUTPUT = 2,
+};
+
+/**
+ * The 'file-handle' to a file in hdfs.
+ */
+struct hdfsFile_internal {
+    void* file;
+    enum hdfsStreamType type;
+    int flags;
+};
+    
+int hdfsFileIsOpenForRead(hdfsFile file)
+{
+    return (file->type == INPUT);
+}
+
+int hdfsFileIsOpenForWrite(hdfsFile file)
+{
+    return (file->type == OUTPUT);
+}
+
+int hdfsFileUsesDirectRead(hdfsFile file)
+{
+    return !!(file->flags & HDFS_FILE_SUPPORTS_DIRECT_READ);
+}
+
+void hdfsFileDisableDirectRead(hdfsFile file)
+{
+    file->flags &= ~HDFS_FILE_SUPPORTS_DIRECT_READ;
+}
+
 /**
 /**
  * hdfsJniEnv: A wrapper struct to be used as 'value'
  * hdfsJniEnv: A wrapper struct to be used as 'value'
  * while saving thread -> JNIEnv* mappings
  * while saving thread -> JNIEnv* mappings
@@ -2182,7 +2228,7 @@ getFileInfoFromStat(JNIEnv *env, jobject jStat, hdfsFileInfo *fileInfo)
                                    "FileStatus::getModificationTime");
                                    "FileStatus::getModificationTime");
         return -1;
         return -1;
     }
     }
-    fileInfo->mLastMod = (tTime) (jVal.j / 1000);
+    fileInfo->mLastMod = jVal.j / 1000;
 
 
     if (invokeMethod(env, &jVal, &jExc, INSTANCE, jStat,
     if (invokeMethod(env, &jVal, &jExc, INSTANCE, jStat,
                      HADOOP_STAT, "getAccessTime", "()J") != 0) {
                      HADOOP_STAT, "getAccessTime", "()J") != 0) {

+ 29 - 33
hadoop-hdfs-project/hadoop-hdfs/src/main/native/hdfs.h

@@ -19,18 +19,10 @@
 #ifndef LIBHDFS_HDFS_H
 #ifndef LIBHDFS_HDFS_H
 #define LIBHDFS_HDFS_H
 #define LIBHDFS_HDFS_H
 
 
-#include <sys/types.h>
-#include <sys/stat.h>
-
-#include <fcntl.h>
-#include <stdio.h>
-#include <stdint.h>
-#include <string.h>
-#include <stdlib.h>
-#include <time.h>
-#include <errno.h>
-
-#include <jni.h>
+#include <errno.h> /* for EINTERNAL, etc. */
+#include <fcntl.h> /* for O_RDONLY, O_WRONLY */
+#include <stdint.h> /* for uint64_t, etc. */
+#include <time.h> /* for time_t */
 
 
 #ifndef O_RDONLY
 #ifndef O_RDONLY
 #define O_RDONLY 1
 #define O_RDONLY 1
@@ -46,10 +38,10 @@
 
 
 
 
 /** All APIs set errno to meaningful values */
 /** All APIs set errno to meaningful values */
+
 #ifdef __cplusplus
 #ifdef __cplusplus
 extern  "C" {
 extern  "C" {
 #endif
 #endif
-
     /**
     /**
      * Some utility decls used in libhdfs.
      * Some utility decls used in libhdfs.
      */
      */
@@ -67,33 +59,37 @@ extern  "C" {
     /**
     /**
      * The C reflection of org.apache.org.hadoop.FileSystem .
      * The C reflection of org.apache.org.hadoop.FileSystem .
      */
      */
-    typedef void* hdfsFS;
-
+    struct hdfs_internal;
+    typedef struct hdfs_internal* hdfsFS;
     
     
+    struct hdfsFile_internal;
+    typedef struct hdfsFile_internal* hdfsFile;
+
     /**
     /**
-     * The C equivalent of org.apache.org.hadoop.FSData(Input|Output)Stream .
+     * Determine if a file is open for read.
+     *
+     * @param file     The HDFS file
+     * @return         1 if the file is open for read; 0 otherwise
      */
      */
-    enum hdfsStreamType
-    {
-        UNINITIALIZED = 0,
-        INPUT = 1,
-        OUTPUT = 2,
-    };
+    int hdfsFileIsOpenForRead(hdfsFile file);
 
 
-    
-    // Bit fields for hdfsFile_internal flags
-    #define HDFS_FILE_SUPPORTS_DIRECT_READ (1<<0)
+    /**
+     * Determine if a file is open for write.
+     *
+     * @param file     The HDFS file
+     * @return         1 if the file is open for write; 0 otherwise
+     */
+    int hdfsFileIsOpenForWrite(hdfsFile file);
 
 
     /**
     /**
-     * The 'file-handle' to a file in hdfs.
+     * Disable the direct read optimization for a file.
+     *
+     * This is mainly provided for unit testing purposes.
+     *
+     * @param file     The HDFS file
      */
      */
-    struct hdfsFile_internal {
-        void* file;
-        enum hdfsStreamType type;
-        uint32_t flags;
-    };
-    typedef struct hdfsFile_internal* hdfsFile;
-      
+    void hdfsFileDisableDirectRead(hdfsFile file);
+
     /** 
     /** 
      * hdfsConnectAsUser - Connect to a hdfs file system as a specific user
      * hdfsConnectAsUser - Connect to a hdfs file system as a specific user
      * Connect to the hdfs.
      * Connect to the hdfs.

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/native/hdfs_read.c

@@ -18,6 +18,9 @@
 
 
 #include "hdfs.h" 
 #include "hdfs.h" 
 
 
+#include <stdio.h>
+#include <stdlib.h>
+
 int main(int argc, char **argv) {
 int main(int argc, char **argv) {
 
 
     if (argc != 4) {
     if (argc != 4) {

+ 47 - 44
hadoop-hdfs-project/hadoop-hdfs/src/main/native/hdfs_test.c

@@ -17,8 +17,15 @@
  */
  */
 
 
 #include "hdfs.h" 
 #include "hdfs.h" 
+#include "hdfs_test.h" 
 
 
-tSize readDirect(hdfsFS fs, hdfsFile f, void* buffer, tSize length);
+#include <inttypes.h>
+#include <jni.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+#include <time.h>
+#include <unistd.h>
 
 
 void permission_disp(short permissions, char *rtr) {
 void permission_disp(short permissions, char *rtr) {
   rtr[9] = '\0';
   rtr[9] = '\0';
@@ -53,6 +60,9 @@ void permission_disp(short permissions, char *rtr) {
 } 
 } 
 
 
 int main(int argc, char **argv) {
 int main(int argc, char **argv) {
+    char buffer[32];
+    tSize num_written_bytes;
+
     hdfsFS fs = hdfsConnectNewInstance("default", 0);
     hdfsFS fs = hdfsConnectNewInstance("default", 0);
     if(!fs) {
     if(!fs) {
         fprintf(stderr, "Oops! Failed to connect to hdfs!\n");
         fprintf(stderr, "Oops! Failed to connect to hdfs!\n");
@@ -77,7 +87,7 @@ int main(int argc, char **argv) {
             exit(-1);
             exit(-1);
         }
         }
         fprintf(stderr, "Opened %s for writing successfully...\n", writePath);
         fprintf(stderr, "Opened %s for writing successfully...\n", writePath);
-        tSize num_written_bytes =
+        num_written_bytes =
           hdfsWrite(fs, writeFile, (void*)fileContents, strlen(fileContents)+1);
           hdfsWrite(fs, writeFile, (void*)fileContents, strlen(fileContents)+1);
         if (num_written_bytes != strlen(fileContents) + 1) {
         if (num_written_bytes != strlen(fileContents) + 1) {
           fprintf(stderr, "Failed to write correct number of bytes - expected %d, got %d\n",
           fprintf(stderr, "Failed to write correct number of bytes - expected %d, got %d\n",
@@ -127,6 +137,13 @@ int main(int argc, char **argv) {
             exit(-1);
             exit(-1);
         }
         }
 
 
+        if (!hdfsFileIsOpenForRead(readFile)) {
+            fprintf(stderr, "hdfsFileIsOpenForRead: we just opened a file "
+                    "with O_RDONLY, and it did not show up as 'open for "
+                    "read'\n");
+            exit(-1);
+        }
+
         fprintf(stderr, "hdfsAvailable: %d\n", hdfsAvailable(fs, readFile));
         fprintf(stderr, "hdfsAvailable: %d\n", hdfsAvailable(fs, readFile));
 
 
         tOffset seekPos = 1;
         tOffset seekPos = 1;
@@ -144,7 +161,7 @@ int main(int argc, char **argv) {
         }
         }
         fprintf(stderr, "Current position: %ld\n", currentPos);
         fprintf(stderr, "Current position: %ld\n", currentPos);
 
 
-        if ((readFile->flags & HDFS_FILE_SUPPORTS_DIRECT_READ) == 0) {
+        if (!hdfsFileUsesDirectRead(readFile)) {
           fprintf(stderr, "Direct read support incorrectly not detected "
           fprintf(stderr, "Direct read support incorrectly not detected "
                   "for HDFS filesystem\n");
                   "for HDFS filesystem\n");
           exit(-1);
           exit(-1);
@@ -152,11 +169,31 @@ int main(int argc, char **argv) {
 
 
         fprintf(stderr, "Direct read support detected for HDFS\n");
         fprintf(stderr, "Direct read support detected for HDFS\n");
 
 
-        // Clear flags so that we really go through slow read path
-        readFile->flags &= ~HDFS_FILE_SUPPORTS_DIRECT_READ;
+        // Test the direct read path
+        if(hdfsSeek(fs, readFile, 0)) {
+            fprintf(stderr, "Failed to seek %s for reading!\n", readPath);
+            exit(-1);
+        }
+        memset(buffer, 0, sizeof(buffer));
+        tSize num_read_bytes = hdfsRead(fs, readFile, (void*)buffer,
+                sizeof(buffer));
+        if (strncmp(fileContents, buffer, strlen(fileContents)) != 0) {
+            fprintf(stderr, "Failed to read (direct). Expected %s but got %s (%d bytes)\n",
+                    fileContents, buffer, num_read_bytes);
+            exit(-1);
+        }
+        fprintf(stderr, "Read (direct) following %d bytes:\n%s\n",
+                num_read_bytes, buffer);
+        if (hdfsSeek(fs, readFile, 0L)) {
+            fprintf(stderr, "Failed to seek to file start!\n");
+            exit(-1);
+        }
+
+        // Disable the direct read path so that we really go through the slow
+        // read path
+        hdfsFileDisableDirectRead(readFile);
 
 
-        static char buffer[32];
-        tSize num_read_bytes = hdfsRead(fs, readFile, (void*)buffer, 
+        num_read_bytes = hdfsRead(fs, readFile, (void*)buffer, 
                 sizeof(buffer));
                 sizeof(buffer));
         fprintf(stderr, "Read following %d bytes:\n%s\n", 
         fprintf(stderr, "Read following %d bytes:\n%s\n", 
                 num_read_bytes, buffer);
                 num_read_bytes, buffer);
@@ -168,24 +205,6 @@ int main(int argc, char **argv) {
         fprintf(stderr, "Read following %d bytes:\n%s\n", 
         fprintf(stderr, "Read following %d bytes:\n%s\n", 
                 num_read_bytes, buffer);
                 num_read_bytes, buffer);
 
 
-        if (hdfsSeek(fs, readFile, 0L)) {
-            fprintf(stderr,
-                    "Failed to seek to file start for direct read test!\n");
-            exit(-1);
-        }
-
-        readFile->flags |= HDFS_FILE_SUPPORTS_DIRECT_READ;
-
-        memset(buffer, 0, strlen(fileContents + 1));
-        num_read_bytes = hdfsRead(fs, readFile, (void*)buffer,
-                sizeof(buffer));
-        if (strncmp(fileContents, buffer, strlen(fileContents)) != 0) {
-            fprintf(stderr, "Failed to read (direct). Expected %s but got %s (%d bytes)\n",
-                    fileContents, buffer, num_read_bytes);
-            exit(-1);
-        }
-        fprintf(stderr, "Read (direct) following %d bytes:\n%s\n",
-                num_read_bytes, buffer);
         hdfsCloseFile(fs, readFile);
         hdfsCloseFile(fs, readFile);
 
 
         // Test correct behaviour for unsupported filesystems
         // Test correct behaviour for unsupported filesystems
@@ -195,34 +214,18 @@ int main(int argc, char **argv) {
             exit(-1);
             exit(-1);
         }
         }
 
 
-        tSize num_written_bytes = hdfsWrite(lfs, localFile,
-                                            (void*)fileContents,
-                                            strlen(fileContents) + 1);
+        num_written_bytes = hdfsWrite(lfs, localFile, (void*)fileContents,
+                                      strlen(fileContents) + 1);
 
 
         hdfsCloseFile(lfs, localFile);
         hdfsCloseFile(lfs, localFile);
         localFile = hdfsOpenFile(lfs, writePath, O_RDONLY, 0, 0, 0);
         localFile = hdfsOpenFile(lfs, writePath, O_RDONLY, 0, 0, 0);
 
 
-        if (localFile->flags & HDFS_FILE_SUPPORTS_DIRECT_READ) {
+        if (hdfsFileUsesDirectRead(localFile)) {
           fprintf(stderr, "Direct read support incorrectly detected for local "
           fprintf(stderr, "Direct read support incorrectly detected for local "
                   "filesystem\n");
                   "filesystem\n");
           exit(-1);
           exit(-1);
         }
         }
 
 
-        memset(buffer, 0, strlen(fileContents + 1));
-        int result = readDirect(lfs, localFile, (void*)buffer, sizeof(buffer));
-        if (result != -1) {
-          fprintf(stderr, "Expected error from local direct read not seen!\n");
-          exit(-1);
-        }
-
-        if (errno != ENOTSUP) {
-          fprintf(stderr, "Error code not correctly set to ENOTSUP, was %d!\n",
-                  errno);
-          exit(-1);
-        }
-
-        fprintf(stderr, "Expected exception thrown for unsupported direct read\n");
-
         hdfsCloseFile(lfs, localFile);
         hdfsCloseFile(lfs, localFile);
     }
     }
 
 

+ 46 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/native/hdfs_test.h

@@ -0,0 +1,46 @@
+/**
+ * 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 LIBHDFS_HDFS_TEST_H
+#define LIBHDFS_HDFS_TEST_H
+
+struct hdfs_internal;
+
+/**
+ * Some functions that are visible only for testing.
+ *
+ * This header is not meant to be exported or used outside of the libhdfs unit
+ * tests.
+ */
+
+#ifdef __cplusplus
+extern  "C" {
+#endif
+    /**
+     * Determine if a file is using the "direct read" optimization.
+     *
+     * @param file     The HDFS file
+     * @return         1 if the file is using the direct read optimization,
+     *                 0 otherwise.
+     */
+    int hdfsFileUsesDirectRead(struct hdfs_internal *file);
+#ifdef __cplusplus
+}
+#endif
+
+#endif

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/native/hdfs_write.c

@@ -16,10 +16,12 @@
  * limitations under the License.
  * limitations under the License.
  */
  */
 
 
-#include <limits.h>
-
 #include "hdfs.h" 
 #include "hdfs.h" 
 
 
+#include <limits.h>
+#include <stdio.h>
+#include <stdlib.h>
+
 int main(int argc, char **argv) {
 int main(int argc, char **argv) {
 
 
     if (argc != 4) {
     if (argc != 4) {