hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From st...@apache.org
Subject svn commit: r823790 - in /hadoop/hbase/trunk/src: java/org/apache/hadoop/hbase/regionserver/HLog.java test/org/apache/hadoop/hbase/regionserver/TestHLog.java
Date Sat, 10 Oct 2009 04:58:37 GMT
Author: stack
Date: Sat Oct 10 04:58:37 2009
New Revision: 823790

URL: http://svn.apache.org/viewvc?rev=823790&view=rev
Log:
HBASE-1887 Update hbase trunk to latests on hadoop 0.21 branch so we can all test sync/append;
add fix for non-pathological case

Modified:
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HLog.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHLog.java

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HLog.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HLog.java?rev=823790&r1=823789&r2=823790&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HLog.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HLog.java Sat Oct 10
04:58:37 2009
@@ -370,16 +370,20 @@
    *         this.end = in.getPos() + length;
    */
   private static class WALReader extends SequenceFile.Reader {
+    private long length;
+    
     WALReader(final FileSystem fs, final Path p, final Configuration c)
     throws IOException {
       super(fs, p, c);
+      
     }
 
     @Override
     protected FSDataInputStream openFile(FileSystem fs, Path file,
       int bufferSize, long length)
     throws IOException {
-      return new WALReaderFSDataInputStream(super.openFile(fs, file, bufferSize, length));
+      return new WALReaderFSDataInputStream(super.openFile(fs, file, bufferSize,
+        length), length);
     }
 
     /**
@@ -387,10 +391,12 @@
      */
     static class WALReaderFSDataInputStream extends FSDataInputStream {
       private boolean firstGetPosInvocation = true;
+      private long length;
 
-      WALReaderFSDataInputStream(final FSDataInputStream is)
+      WALReaderFSDataInputStream(final FSDataInputStream is, final long l)
       throws IOException {
         super(is);
+        this.length = l;
       }
 
       @Override
@@ -401,7 +407,12 @@
           // SequenceFile.Reader constructor comes out with the correct length
           // on the file:
           //         this.end = in.getPos() + length;
-          return this.in.available();
+          // 
+          long available = this.in.available();
+          // Length gets added up in the SF.Reader constructor so subtract the
+          // difference.  If available < this.length, then return this.length.
+          // I ain't sure what else to do.
+          return available >= this.length? available - this.length: this.length;
         }
         return super.getPos();
       }
@@ -988,7 +999,8 @@
           SequenceFile.Reader in = null;
           int count = 0;
           try {
-            in = new SequenceFile.Reader(fs, logfiles[i].getPath(), conf);
+            long len = fs.getFileStatus(logfiles[i].getPath()).getLen();
+            in = HLog.getReader(fs, logfiles[i].getPath(), conf);
             try {
               HLogKey key = newKey(conf);
               KeyValue val = new KeyValue();

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHLog.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHLog.java?rev=823790&r1=823789&r2=823790&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHLog.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHLog.java Sat Oct
10 04:58:37 2009
@@ -65,6 +65,45 @@
   }
 
   /**
+   * Just write multiple logs then split.  Before fix for HADOOP-2283, this
+   * would fail.
+   * @throws IOException
+   */
+  public void testSplit() throws IOException {
+    final byte [] tableName = Bytes.toBytes(getName());
+    final byte [] rowName = tableName;
+    HLog log = new HLog(this.fs, this.dir, this.conf, null);
+    final int howmany = 3;
+    // Add edits for three regions.
+    try {
+      for (int ii = 0; ii < howmany; ii++) {
+        for (int i = 0; i < howmany; i++) {
+          for (int j = 0; j < howmany; j++) {
+            List<KeyValue> edit = new ArrayList<KeyValue>();
+            byte [] family = Bytes.toBytes("column");
+            byte [] qualifier = Bytes.toBytes(Integer.toString(j));
+            byte [] column = Bytes.toBytes("column:" + Integer.toString(j));
+            edit.add(new KeyValue(rowName, family, qualifier, 
+                System.currentTimeMillis(), column));
+            System.out.println("Region " + i + ": " + edit);
+            log.append(Bytes.toBytes("" + i), tableName, edit,
+              false, System.currentTimeMillis());
+          }
+        }
+        log.rollWriter();
+      }
+      List<Path> splits =
+        HLog.splitLog(this.testDir, this.dir, this.fs, this.conf);
+      verifySplits(splits, howmany);
+      log = null;
+    } finally {
+      if (log != null) {
+        log.closeAndDelete();
+      }
+    }
+  }
+
+  /**
    * Test new HDFS-265 sync.
    * @throws Exception
    */
@@ -123,45 +162,6 @@
     assertEquals(total * 2, count);
   }
  
-  /**
-   * Just write multiple logs then split.  Before fix for HADOOP-2283, this
-   * would fail.
-   * @throws IOException
-   */
-  public void testSplit() throws IOException {
-    final byte [] tableName = Bytes.toBytes(getName());
-    final byte [] rowName = tableName;
-    HLog log = new HLog(this.fs, this.dir, this.conf, null);
-    final int howmany = 3;
-    // Add edits for three regions.
-    try {
-      for (int ii = 0; ii < howmany; ii++) {
-        for (int i = 0; i < howmany; i++) {
-          for (int j = 0; j < howmany; j++) {
-            List<KeyValue> edit = new ArrayList<KeyValue>();
-            byte [] family = Bytes.toBytes("column");
-            byte [] qualifier = Bytes.toBytes(Integer.toString(j));
-            byte [] column = Bytes.toBytes("column:" + Integer.toString(j));
-            edit.add(new KeyValue(rowName, family, qualifier, 
-                System.currentTimeMillis(), column));
-            System.out.println("Region " + i + ": " + edit);
-            log.append(Bytes.toBytes("" + i), tableName, edit,
-              false, System.currentTimeMillis());
-          }
-        }
-        log.rollWriter();
-      }
-      List<Path> splits =
-        HLog.splitLog(this.testDir, this.dir, this.fs, this.conf);
-      verifySplits(splits, howmany);
-      log = null;
-    } finally {
-      if (log != null) {
-        log.closeAndDelete();
-      }
-    }
-  }
-
   private void verifySplits(List<Path> splits, final int howmany)
   throws IOException {
     assertEquals(howmany, splits.size());



Mime
View raw message