You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by al...@apache.org on 2009/03/19 20:48:22 UTC

svn commit: r756155 - in /incubator/cassandra/trunk/src/org/apache/cassandra/io: AIORandomAccessFile.java BufferedRandomAccessFile.java ChecksumRandomAccessFile.java SSTable.java SequenceFile.java

Author: alakshman
Date: Thu Mar 19 19:48:21 2009
New Revision: 756155

URL: http://svn.apache.org/viewvc?rev=756155&view=rev
Log:
Fixed some bugs that resulted from moving sources over.

Modified:
    incubator/cassandra/trunk/src/org/apache/cassandra/io/AIORandomAccessFile.java
    incubator/cassandra/trunk/src/org/apache/cassandra/io/BufferedRandomAccessFile.java
    incubator/cassandra/trunk/src/org/apache/cassandra/io/ChecksumRandomAccessFile.java
    incubator/cassandra/trunk/src/org/apache/cassandra/io/SSTable.java
    incubator/cassandra/trunk/src/org/apache/cassandra/io/SequenceFile.java

Modified: incubator/cassandra/trunk/src/org/apache/cassandra/io/AIORandomAccessFile.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/io/AIORandomAccessFile.java?rev=756155&r1=756154&r2=756155&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/io/AIORandomAccessFile.java (original)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/io/AIORandomAccessFile.java Thu Mar 19 19:48:21 2009
@@ -656,50 +656,6 @@
         this.curr_ += len;
         return len;
     }
-    
-    public static void main(String[] args) throws Throwable
-    {  
-        /*
-        int i = 0;
-        try
-        {
-            RandomAccessFile aRaf2 = new AIORandomAccessFile( new File("/var/cassandra/test.dat"), 64*1024);        
-            aRaf2.seek(0L);
-            while ( i < 10000 )
-            {
-                aRaf2.writeInt(32);
-                aRaf2.writeUTF("Avinash Lakshman");
-                ++i;
-            }
-            aRaf2.close();
-        }
-        catch( IOException ex )
-        {
-            ex.printStackTrace();
-        }
-        */
-        /*
-        int j = 0;
-        try
-        {
-            RandomAccessFile aRaf2 = new AIORandomAccessFile( new File("/var/cassandra/test.dat") );                    
-            while ( j < 10 )
-            {
-                System.out.println( aRaf2.readInt() );
-                System.out.println( aRaf2.readUTF() );
-                ++j;
-            }
-            aRaf2.close();
-        }
-        catch( IOException ex )
-        {
-            ex.printStackTrace();
-        }
-        */
-                
-        ExecutorService es = new ContinuationsExecutor(1, 1, Integer.MAX_VALUE, TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>() );
-        es.execute(new ReadImpl());               
-    }
 }
 
 class ReadImpl implements Runnable

Modified: incubator/cassandra/trunk/src/org/apache/cassandra/io/BufferedRandomAccessFile.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/io/BufferedRandomAccessFile.java?rev=756155&r1=756154&r2=756155&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/io/BufferedRandomAccessFile.java (original)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/io/BufferedRandomAccessFile.java Thu Mar 19 19:48:21 2009
@@ -165,20 +165,20 @@
      * disk. If the file was created read-only, this method is a no-op.
      */
     public void flush() throws IOException
-    {
+    {        
         this.flushBuffer();
     }
     
     /* Flush any dirty bytes in the buffer to disk. */
     private void flushBuffer() throws IOException
-    {
+    {   
         if (this.dirty_)
         {
             if (this.diskPos_ != this.lo_)
                 super.seek(this.lo_);
             int len = (int) (this.curr_ - this.lo_);
             super.write(this.buff_, 0, len);
-            this.diskPos_ = this.curr_;
+            this.diskPos_ = this.curr_;             
             this.dirty_ = false;
         }
     }
@@ -222,7 +222,7 @@
     {
         if (pos >= this.hi_ || pos < this.lo_)
         {
-            // seeking outside of current buffer -- flush and read
+            // seeking outside of current buffer -- flush and read             
             this.flushBuffer();
             this.lo_ = pos & BuffMask_; // start at BuffSz boundary
             this.maxHi_ = this.lo_ + (long) this.buff_.length;
@@ -332,14 +332,14 @@
     }
     
     public void write(byte[] b, int off, int len) throws IOException
-    {
+    {        
         while (len > 0)
-        {
+        {              
             int n = this.writeAtMost(b, off, len);
             off += n;
             len -= n;
-        }
-        this.dirty_ = true;
+            this.dirty_ = true;
+        }        
     }
     
     /*
@@ -347,7 +347,7 @@
      * the number of bytes written.
      */
     private int writeAtMost(byte[] b, int off, int len) throws IOException
-    {
+    {        
         if (this.curr_ >= this.hi_)
         {
             if (this.hitEOF_ && this.hi_ < this.maxHi_)
@@ -356,8 +356,8 @@
                 this.hi_ = this.maxHi_;
             }
             else
-            {
-                // slow path -- write current buffer; read next one
+            {                                
+                // slow path -- write current buffer; read next one                
                 this.seek(this.curr_);
                 if (this.curr_ == this.hi_)
                 {

Modified: incubator/cassandra/trunk/src/org/apache/cassandra/io/ChecksumRandomAccessFile.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/io/ChecksumRandomAccessFile.java?rev=756155&r1=756154&r2=756155&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/io/ChecksumRandomAccessFile.java (original)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/io/ChecksumRandomAccessFile.java Thu Mar 19 19:48:21 2009
@@ -391,8 +391,8 @@
             int n = this.writeAtMost(b, off, len);
             off += n;
             len -= n;
+            this.dirty_ = true;
         }
-        this.dirty_ = true;
     }
     
     /*

Modified: incubator/cassandra/trunk/src/org/apache/cassandra/io/SSTable.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/io/SSTable.java?rev=756155&r1=756154&r2=756155&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/io/SSTable.java (original)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/io/SSTable.java Thu Mar 19 19:48:21 2009
@@ -432,7 +432,8 @@
     */
     public SSTable(String directory, String filename, PartitionerType pType) throws IOException
     {        
-        dataFile_ = directory + System.getProperty("file.separator") + filename + "-Data.db";  
+        dataFile_ = directory + System.getProperty("file.separator") + filename + "-Data.db";
+        // dataWriter_ = SequenceFile.writer(dataFile_);
         dataWriter_ = SequenceFile.bufferedWriter(dataFile_, 4*1024*1024);    
         // dataWriter_ = SequenceFile.chksumWriter(dataFile_, 4*1024*1024);
         SSTable.positionAfterFirstBlockIndex_ = dataWriter_.getCurrentPosition(); 
@@ -747,22 +748,7 @@
         	SSTable.indexMetadataMap_.put(dataFile_, keyPositionInfos);
         }
         
-        keyPositionInfos.add(new KeyPositionInfo(blockIndex.firstKey(), position));
-        /*
-        try
-        {
-            keyPositionInfos.add(new KeyPositionInfo(blockIndex.firstKey(), position));
-        }
-        catch(Exception ex)
-        {
-            Set<String> keysInBlock = blockIndex.keySet();
-            for( String keyInBlock : keysInBlock )
-            {
-                logger_.warn("BLOCK KEY: " + keyInBlock);
-            }
-            logger_.warn(LogUtil.throwableToString(ex));
-        }
-        */
+        keyPositionInfos.add(new KeyPositionInfo(blockIndex.firstKey(), position));        
         blockIndex.clear();        
     }
 
@@ -1088,9 +1074,8 @@
     	/* reset the buffer and serialize the Bloom Filter. */
         DataOutputBuffer bufOut = new DataOutputBuffer();
         BloomFilter.serializer().serialize(bf, bufOut);
-        bufOut.close();
-
         close(bufOut.getData(), bufOut.getLength());
+        bufOut.close();
         // byte[] bytes = new byte[bufOut.getLength()];        
         // System.arraycopy(bufOut.getData(), 0, bytes, 0, bufOut.getLength());
         // close(bytes, bytes.length);             

Modified: incubator/cassandra/trunk/src/org/apache/cassandra/io/SequenceFile.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/io/SequenceFile.java?rev=756155&r1=756154&r2=756155&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/io/SequenceFile.java (original)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/io/SequenceFile.java Thu Mar 19 19:48:21 2009
@@ -974,7 +974,7 @@
 
                         Coordinate coordinate = columnRange.coordinate();
                 		/* seek to the correct offset to the data, and calculate the data size */
-                        file_.skipBytes((int)coordinate.start_);
+                        file_.skipBytes((int)coordinate.start_);                        
                         dataSize = (int)(coordinate.end_ - coordinate.start_);
                         
                         /*