hadoop-common-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From dhr...@apache.org
Subject svn commit: r644798 - in /hadoop/core/trunk: ./ src/java/org/apache/hadoop/dfs/ src/test/org/apache/hadoop/dfs/
Date Fri, 04 Apr 2008 18:00:52 GMT
Author: dhruba
Date: Fri Apr  4 11:00:49 2008
New Revision: 644798

URL: http://svn.apache.org/viewvc?rev=644798&view=rev
Log:
HADOOP-3146. A DFSOutputStream.flush method is renamed as
DFSOutputStream.fsync.  (dhruba)


Modified:
    hadoop/core/trunk/CHANGES.txt
    hadoop/core/trunk/src/java/org/apache/hadoop/dfs/DFSClient.java
    hadoop/core/trunk/src/test/org/apache/hadoop/dfs/TestAbandonBlock.java
    hadoop/core/trunk/src/test/org/apache/hadoop/dfs/TestFileAppend.java
    hadoop/core/trunk/src/test/org/apache/hadoop/dfs/TestFileCreation.java

Modified: hadoop/core/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/CHANGES.txt?rev=644798&r1=644797&r2=644798&view=diff
==============================================================================
--- hadoop/core/trunk/CHANGES.txt (original)
+++ hadoop/core/trunk/CHANGES.txt Fri Apr  4 11:00:49 2008
@@ -479,6 +479,9 @@
     HADOOP-3089.  Streaming should accept stderr from task before
     first key arrives.  (Rick Cox via tomwhite)
 
+    HADOOP-3146. A DFSOutputStream.flush method is renamed as
+    DFSOutputStream.fsync.  (dhruba)
+
 Release 0.16.2 - 2008-04-02
 
   BUG FIXES

Modified: hadoop/core/trunk/src/java/org/apache/hadoop/dfs/DFSClient.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/java/org/apache/hadoop/dfs/DFSClient.java?rev=644798&r1=644797&r2=644798&view=diff
==============================================================================
--- hadoop/core/trunk/src/java/org/apache/hadoop/dfs/DFSClient.java (original)
+++ hadoop/core/trunk/src/java/org/apache/hadoop/dfs/DFSClient.java Fri Apr  4 11:00:49 2008
@@ -2359,8 +2359,7 @@
      * that data has been flushed to persistent store on the 
      * datanode. Block allocations are persisted on namenode.
      */
-    @Override
-    public synchronized void flush() throws IOException {
+    public synchronized void fsync() throws IOException {
       Packet savePacket = null;
       int position = 0;
       long saveOffset = 0;

Modified: hadoop/core/trunk/src/test/org/apache/hadoop/dfs/TestAbandonBlock.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/test/org/apache/hadoop/dfs/TestAbandonBlock.java?rev=644798&r1=644797&r2=644798&view=diff
==============================================================================
--- hadoop/core/trunk/src/test/org/apache/hadoop/dfs/TestAbandonBlock.java (original)
+++ hadoop/core/trunk/src/test/org/apache/hadoop/dfs/TestAbandonBlock.java Fri Apr  4 11:00:49
2008
@@ -33,7 +33,13 @@
   static final String FILE_NAME_PREFIX
       = "/" + TestAbandonBlock.class.getSimpleName() + "_"; 
 
-  public void testAbondanBlock() throws IOException {
+  private void flushFile(FSDataOutputStream stm) throws IOException {
+    DFSClient.DFSOutputStream dfstream = (DFSClient.DFSOutputStream)
+                                            (stm.getWrappedStream());
+    dfstream.fsync();
+  }
+
+  public void testAbandonBlock() throws IOException {
     MiniDFSCluster cluster = new MiniDFSCluster(CONF, 2, true, null);
     FileSystem fs = cluster.getFileSystem();
 
@@ -45,7 +51,7 @@
       for(int i = 0; i < 1024; i++) {
         fout.write(123);
       }
-      fout.flush();
+      flushFile(fout);
   
       //try reading the block by someone
       DFSClient dfsclient = new DFSClient(CONF);
@@ -66,4 +72,4 @@
       try{cluster.shutdown();} catch(Exception e) {}
     }
   }
-}
\ No newline at end of file
+}

Modified: hadoop/core/trunk/src/test/org/apache/hadoop/dfs/TestFileAppend.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/test/org/apache/hadoop/dfs/TestFileAppend.java?rev=644798&r1=644797&r2=644798&view=diff
==============================================================================
--- hadoop/core/trunk/src/test/org/apache/hadoop/dfs/TestFileAppend.java (original)
+++ hadoop/core/trunk/src/test/org/apache/hadoop/dfs/TestFileAppend.java Fri Apr  4 11:00:49
2008
@@ -62,6 +62,12 @@
     return stm;
   }
 
+  private void flushFile(FSDataOutputStream stm) throws IOException {
+    DFSClient.DFSOutputStream dfstream = (DFSClient.DFSOutputStream)
+                                            (stm.getWrappedStream());
+    dfstream.fsync();
+  }
+
   //
   // writes to file but does not close it
   //
@@ -226,14 +232,14 @@
       // write to file
       int mid = fileSize/2;
       stm.write(fileContents, 0, mid);
-      stm.flush();
+      flushFile(stm);
       System.out.println("Wrote and Flushed first part of file.");
 
       // write the remainder of the file
       stm.write(fileContents, mid, fileSize - mid);
       System.out.println("Written second part of file");
-      stm.flush();
-      stm.flush(); // two consecutive flushes is being tested here.
+      flushFile(stm);
+      flushFile(stm); // two consecutive flushes is being tested here.
       System.out.println("Wrote and Flushed second part of file.");
 
       // verify that full blocks are sane
@@ -279,7 +285,7 @@
       int start = 0;
       for (start = 0; (start + 29) < fileSize; ) {
         stm.write(fileContents, start, 29);
-        stm.flush();
+        flushFile(stm);
         start += 29;
       }
       stm.write(fileContents, start, fileSize-start);

Modified: hadoop/core/trunk/src/test/org/apache/hadoop/dfs/TestFileCreation.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/test/org/apache/hadoop/dfs/TestFileCreation.java?rev=644798&r1=644797&r2=644798&view=diff
==============================================================================
--- hadoop/core/trunk/src/test/org/apache/hadoop/dfs/TestFileCreation.java (original)
+++ hadoop/core/trunk/src/test/org/apache/hadoop/dfs/TestFileCreation.java Fri Apr  4 11:00:49
2008
@@ -57,6 +57,12 @@
     return stm;
   }
 
+  private void flushFile(FSDataOutputStream stm) throws IOException {
+    DFSClient.DFSOutputStream dfstream = (DFSClient.DFSOutputStream)
+                                            (stm.getWrappedStream());
+    dfstream.fsync();
+  }
+
   //
   // writes to file but does not close it
   //
@@ -377,7 +383,7 @@
 
       // write two full blocks.
       writeFile(stm, numBlocks * blockSize);
-      stm.flush();
+      flushFile(stm);
 
       // create another new file.
       //



Mime
View raw message