You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jb...@apache.org on 2009/07/28 04:45:40 UTC

svn commit: r798368 - in /incubator/cassandra/trunk/src/java/org/apache/cassandra: db/CommitLog.java gms/ApplicationState.java gms/EndPointState.java io/AbstractWriter.java io/IFileWriter.java io/SSTableWriter.java io/SequenceFile.java

Author: jbellis
Date: Tue Jul 28 02:45:40 2009
New Revision: 798368

URL: http://svn.apache.org/viewvc?rev=798368&view=rev
Log:
mv AbstractWriter to its own top-level class and remove redundant IFileWriter
patch by jbellis; reviewed by Jun Rao for CASSANDRA-182

Added:
    incubator/cassandra/trunk/src/java/org/apache/cassandra/io/AbstractWriter.java
Removed:
    incubator/cassandra/trunk/src/java/org/apache/cassandra/io/IFileWriter.java
Modified:
    incubator/cassandra/trunk/src/java/org/apache/cassandra/db/CommitLog.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/gms/ApplicationState.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/gms/EndPointState.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableWriter.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SequenceFile.java

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/db/CommitLog.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/CommitLog.java?rev=798368&r1=798367&r2=798368&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/db/CommitLog.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/db/CommitLog.java Tue Jul 28 02:45:40 2009
@@ -22,11 +22,7 @@
 import java.util.*;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.io.DataInputBuffer;
-import org.apache.cassandra.io.DataOutputBuffer;
-import org.apache.cassandra.io.IFileReader;
-import org.apache.cassandra.io.IFileWriter;
-import org.apache.cassandra.io.SequenceFile;
+import org.apache.cassandra.io.*;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.FileUtils;
 
@@ -123,7 +119,7 @@
         return Long.parseLong(entries[entries.length - 2]);
     }
 
-    private static IFileWriter createWriter(String file) throws IOException
+    private static AbstractWriter createWriter(String file) throws IOException
     {        
         return SequenceFile.writer(file);
     }
@@ -153,7 +149,7 @@
     private String logFile_;
     /* header for current commit log */
     private CommitLogHeader clHeader_;
-    private IFileWriter logWriter_;
+    private AbstractWriter logWriter_;
 
     /*
      * Generates a file name of the format CommitLog-<table>-<timestamp>.log in the
@@ -213,7 +209,7 @@
     */
     private static void writeCommitLogHeader(String commitLogFileName, byte[] bytes) throws IOException
     {
-        IFileWriter logWriter = CommitLog.createWriter(commitLogFileName);
+        AbstractWriter logWriter = CommitLog.createWriter(commitLogFileName);
         writeCommitLogHeader(logWriter, bytes);
         logWriter.close();
     }
@@ -240,7 +236,7 @@
         logWriter_.seek(currentPos);
     }
 
-    private static void writeCommitLogHeader(IFileWriter logWriter, byte[] bytes) throws IOException
+    private static void writeCommitLogHeader(AbstractWriter logWriter, byte[] bytes) throws IOException
     {
         logWriter.writeLong(bytes.length);
         logWriter.writeDirect(bytes);

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/gms/ApplicationState.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/gms/ApplicationState.java?rev=798368&r1=798367&r2=798368&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/gms/ApplicationState.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/gms/ApplicationState.java Tue Jul 28 02:45:40 2009
@@ -22,10 +22,7 @@
 import java.io.DataOutputStream;
 import java.io.IOException;
 
-import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.io.ICompactSerializer;
-import org.apache.cassandra.io.IFileReader;
-import org.apache.cassandra.io.IFileWriter;
 
 
 /**

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/gms/EndPointState.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/gms/EndPointState.java?rev=798368&r1=798367&r2=798368&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/gms/EndPointState.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/gms/EndPointState.java Tue Jul 28 02:45:40 2009
@@ -23,10 +23,8 @@
 import java.io.IOException;
 import java.util.*;
 import org.apache.cassandra.io.ICompactSerializer;
-import org.apache.cassandra.io.IFileReader;
-import org.apache.cassandra.io.IFileWriter;
+
 import org.apache.log4j.Logger;
-import org.apache.cassandra.utils.*;
 
 /**
  * This abstraction represents both the HeartBeatState and the ApplicationState in an EndPointState

Added: incubator/cassandra/trunk/src/java/org/apache/cassandra/io/AbstractWriter.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/io/AbstractWriter.java?rev=798368&view=auto
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/io/AbstractWriter.java (added)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/io/AbstractWriter.java Tue Jul 28 02:45:40 2009
@@ -0,0 +1,278 @@
+package org.apache.cassandra.io;
+
+import java.io.File;
+import java.io.RandomAccessFile;
+import java.io.IOException;
+
+public abstract class AbstractWriter
+{
+    protected String filename_;
+
+    AbstractWriter(String filename)
+    {
+        filename_ = filename;
+    }
+
+    public String getFileName()
+    {
+        return filename_;
+    }
+
+    public long lastModified()
+    {
+        File file = new File(filename_);
+        return file.lastModified();
+    }
+
+    /**
+     * Get the current position of the file pointer.
+     * @return current file pointer position
+     * @throws java.io.IOException
+     */
+    public abstract long getCurrentPosition() throws IOException;
+
+    /**
+     * Seeks the file pointer to the specified position.
+     * @param position position within the file to seek to.
+     * @throws java.io.IOException
+     */
+    public abstract void seek(long position) throws IOException;
+
+    /**
+     * Appends the buffer to the the underlying SequenceFile.
+     * @param buffer buffer which contains the serialized data.
+     * @throws java.io.IOException
+     */
+    public abstract void append(DataOutputBuffer buffer) throws IOException;
+
+    /**
+     * Appends the key and the value to the the underlying SequenceFile.
+     * @param keyBuffer buffer which contains the serialized key.
+     * @param buffer buffer which contains the serialized data.
+     * @throws java.io.IOException
+     */
+    public abstract void append(DataOutputBuffer keyBuffer, DataOutputBuffer buffer) throws IOException;
+
+    /**
+     * Appends the key and the value to the the underlying SequenceFile.
+     * @param key key associated with this peice of data.
+     * @param buffer buffer containing the serialized data.
+     * @throws java.io.IOException
+     */
+    public abstract void append(String key, DataOutputBuffer buffer) throws IOException;
+
+    /**
+     * Appends the key and the value to the the underlying SequenceFile.
+     * @param key key associated with this peice of data.
+     * @param value byte array containing the serialized data.
+     * @throws java.io.IOException
+     */
+    public abstract void append(String key, byte[] value) throws IOException;
+
+    /**
+     * Appends the key and the long value to the the underlying SequenceFile.
+     * This is used in the contruction of the index file associated with a
+     * SSTable.
+     * @param key key associated with this peice of data.
+     * @param value value associated with this key.
+     * @throws java.io.IOException
+     */
+    public abstract void append(String key, long value) throws IOException;
+
+    /**
+     * Be extremely careful while using this API. This currently
+     * used to write the commit log header in the commit logs.
+     * If not used carefully it could completely screw up reads
+     * of other key/value pairs that are written.
+     *
+     * @param bytes serialized version of the commit log header.
+     * @throws java.io.IOException
+    */
+    public abstract long writeDirect(byte[] bytes) throws IOException;
+
+    /**
+     * Write a long into the underlying sub system.
+     * @param value long to be written
+     * @throws java.io.IOException
+     */
+    public abstract void writeLong(long value) throws IOException;
+
+    /**
+     * Close the file which is being used for the write.
+     * @throws java.io.IOException
+     */
+    public abstract void close() throws IOException;
+
+    /**
+     * Close the file after appending the passed in footer information.
+     * @param footer footer information.
+     * @param size size of the footer.
+     * @throws java.io.IOException
+     */
+    public abstract void close(byte[] footer, int size) throws IOException;
+
+    /**
+     * @return the size of the file.
+     * @throws java.io.IOException
+     */
+    public abstract long getFileSize() throws IOException;
+
+
+    public static class Writer extends AbstractWriter
+    {
+        protected RandomAccessFile file_;
+
+        Writer(String filename) throws IOException
+        {
+            super(filename);
+            init(filename);
+        }
+
+        Writer(String filename, int size) throws IOException
+        {
+            super(filename);
+            init(filename, size);
+        }
+
+        protected void init(String filename) throws IOException
+        {
+            File file = new File(filename);
+            if (!file.exists())
+            {
+                file.createNewFile();
+            }
+            file_ = new RandomAccessFile(file, "rw");
+        }
+
+        protected void init(String filename, int size) throws IOException
+        {
+            init(filename);
+        }
+
+        public long getCurrentPosition() throws IOException
+        {
+            return file_.getFilePointer();
+        }
+
+        public void seek(long position) throws IOException
+        {
+            file_.seek(position);
+        }
+
+        public void append(DataOutputBuffer buffer) throws IOException
+        {
+            file_.write(buffer.getData(), 0, buffer.getLength());
+        }
+
+        public void append(DataOutputBuffer keyBuffer, DataOutputBuffer buffer) throws IOException
+        {
+            int keyBufLength = keyBuffer.getLength();
+            if (keyBuffer == null || keyBufLength == 0)
+                throw new IllegalArgumentException("Key cannot be NULL or of zero length.");
+
+            file_.writeInt(keyBufLength);
+            file_.write(keyBuffer.getData(), 0, keyBufLength);
+
+            int length = buffer.getLength();
+            file_.writeInt(length);
+            file_.write(buffer.getData(), 0, length);
+        }
+
+        public void append(String key, DataOutputBuffer buffer) throws IOException
+        {
+            if (key == null)
+                throw new IllegalArgumentException("Key cannot be NULL.");
+
+            file_.writeUTF(key);
+            int length = buffer.getLength();
+            file_.writeInt(length);
+            file_.write(buffer.getData(), 0, length);
+        }
+
+        public void append(String key, byte[] value) throws IOException
+        {
+            if (key == null)
+                throw new IllegalArgumentException("Key cannot be NULL.");
+
+            file_.writeUTF(key);
+            file_.writeInt(value.length);
+            file_.write(value);
+        }
+
+        public void append(String key, long value) throws IOException
+        {
+            if (key == null)
+                throw new IllegalArgumentException("Key cannot be NULL.");
+
+            file_.writeUTF(key);
+            file_.writeLong(value);
+        }
+
+        /**
+         * Be extremely careful while using this API. This currently
+         * used to write the commit log header in the commit logs.
+         * If not used carefully it could completely screw up reads
+         * of other key/value pairs that are written.
+         *
+         * @param bytes the bytes to write
+         */
+        public long writeDirect(byte[] bytes) throws IOException
+        {
+            file_.write(bytes);
+            return file_.getFilePointer();
+        }
+
+        public void writeLong(long value) throws IOException
+        {
+            file_.writeLong(value);
+        }
+
+        public void close() throws IOException
+        {
+            file_.getChannel().force(true);
+            file_.close();
+        }
+
+        public void close(byte[] footer, int size) throws IOException
+        {
+            file_.writeInt(size);
+            file_.write(footer, 0, size);
+        }
+
+        public String getFileName()
+        {
+            return filename_;
+        }
+
+        public long getFileSize() throws IOException
+        {
+            return file_.length();
+        }
+    }
+
+    public static class BufferWriter extends Writer
+    {
+
+        BufferWriter(String filename, int size) throws IOException
+        {
+            super(filename, size);
+        }
+
+        @Override
+        protected void init(String filename) throws IOException
+        {
+            init(filename, 0);
+        }
+
+        @Override
+        protected void init(String filename, int size) throws IOException
+        {
+            File file = new File(filename);
+            file_ = new BufferedRandomAccessFile(file, "rw", size);
+            if (!file.exists())
+            {
+                file.createNewFile();
+            }
+        }
+    }
+}

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableWriter.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableWriter.java?rev=798368&r1=798367&r2=798368&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableWriter.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableWriter.java Tue Jul 28 02:45:40 2009
@@ -18,7 +18,7 @@
     private static Logger logger = Logger.getLogger(SSTableWriter.class);
 
     private long keysWritten;
-    private IFileWriter dataWriter;
+    private AbstractWriter dataWriter;
     private BufferedRandomAccessFile indexRAF;
     private String lastWrittenKey;
     private BloomFilter bf;

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SequenceFile.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SequenceFile.java?rev=798368&r1=798367&r2=798368&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SequenceFile.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SequenceFile.java Tue Jul 28 02:45:40 2009
@@ -38,186 +38,6 @@
 
 public class SequenceFile
 {
-    public static abstract class AbstractWriter implements IFileWriter
-    {
-        protected String filename_;
-
-        AbstractWriter(String filename)
-        {
-            filename_ = filename;
-        }
-
-        public String getFileName()
-        {
-            return filename_;
-        }
-
-        public long lastModified()
-        {
-            File file = new File(filename_);
-            return file.lastModified();
-        }
-    }
-
-    public static class Writer extends AbstractWriter
-    {
-        protected RandomAccessFile file_;
-
-        Writer(String filename) throws IOException
-        {
-            super(filename);
-            init(filename);
-        }
-
-        Writer(String filename, int size) throws IOException
-        {
-            super(filename);
-            init(filename, size);
-        }
-
-        protected void init(String filename) throws IOException
-        {
-            File file = new File(filename);
-            if (!file.exists())
-            {
-                file.createNewFile();
-            }
-            file_ = new RandomAccessFile(file, "rw");
-        }
-
-        protected void init(String filename, int size) throws IOException
-        {
-            init(filename);
-        }
-
-        public long getCurrentPosition() throws IOException
-        {
-            return file_.getFilePointer();
-        }
-
-        public void seek(long position) throws IOException
-        {
-            file_.seek(position);
-        }
-
-        public void append(DataOutputBuffer buffer) throws IOException
-        {
-            file_.write(buffer.getData(), 0, buffer.getLength());
-        }
-
-        public void append(DataOutputBuffer keyBuffer, DataOutputBuffer buffer) throws IOException
-        {
-            int keyBufLength = keyBuffer.getLength();
-            if (keyBuffer == null || keyBufLength == 0)
-                throw new IllegalArgumentException("Key cannot be NULL or of zero length.");
-
-            file_.writeInt(keyBufLength);
-            file_.write(keyBuffer.getData(), 0, keyBufLength);
-
-            int length = buffer.getLength();
-            file_.writeInt(length);
-            file_.write(buffer.getData(), 0, length);
-        }
-
-        public void append(String key, DataOutputBuffer buffer) throws IOException
-        {
-            if (key == null)
-                throw new IllegalArgumentException("Key cannot be NULL.");
-
-            file_.writeUTF(key);
-            int length = buffer.getLength();
-            file_.writeInt(length);
-            file_.write(buffer.getData(), 0, length);
-        }
-
-        public void append(String key, byte[] value) throws IOException
-        {
-            if (key == null)
-                throw new IllegalArgumentException("Key cannot be NULL.");
-
-            file_.writeUTF(key);
-            file_.writeInt(value.length);
-            file_.write(value);
-        }
-
-        public void append(String key, long value) throws IOException
-        {
-            if (key == null)
-                throw new IllegalArgumentException("Key cannot be NULL.");
-
-            file_.writeUTF(key);
-            file_.writeLong(value);
-        }
-
-        /**
-         * Be extremely careful while using this API. This currently
-         * used to write the commit log header in the commit logs.
-         * If not used carefully it could completely screw up reads
-         * of other key/value pairs that are written.
-         *
-         * @param bytes the bytes to write
-         */
-        public long writeDirect(byte[] bytes) throws IOException
-        {
-            file_.write(bytes);
-            return file_.getFilePointer();
-        }
-
-        public void writeLong(long value) throws IOException
-        {
-            file_.writeLong(value);
-        }
-
-        public void close() throws IOException
-        {
-            file_.getChannel().force(true);
-            file_.close();
-        }
-
-        public void close(byte[] footer, int size) throws IOException
-        {
-            file_.writeInt(size);
-            file_.write(footer, 0, size);
-        }
-
-        public String getFileName()
-        {
-            return filename_;
-        }
-
-        public long getFileSize() throws IOException
-        {
-            return file_.length();
-        }
-    }
-
-    public static class BufferWriter extends Writer
-    {
-
-        BufferWriter(String filename, int size) throws IOException
-        {
-            super(filename, size);
-        }
-
-        @Override
-        protected void init(String filename) throws IOException
-        {
-            init(filename, 0);
-        }
-
-        @Override
-        protected void init(String filename, int size) throws IOException
-        {
-            File file = new File(filename);
-            file_ = new BufferedRandomAccessFile(file, "rw", size);
-            if (!file.exists())
-            {
-                file.createNewFile();
-            }
-        }
-    }
-
-
     /**
      *  This is a reader that finds the block for a starting column and returns
      *  blocks before/after it for each next call. This function assumes that
@@ -703,14 +523,14 @@
     public static final short utfPrefix_ = 2;
     public static final String marker_ = "Bloom-Filter";
 
-    public static IFileWriter writer(String filename) throws IOException
+    public static AbstractWriter writer(String filename) throws IOException
     {
-        return new Writer(filename);
+        return new AbstractWriter.Writer(filename);
     }
 
-    public static IFileWriter bufferedWriter(String filename, int size) throws IOException
+    public static AbstractWriter bufferedWriter(String filename, int size) throws IOException
     {
-        return new BufferWriter(filename, size);
+        return new AbstractWriter.BufferWriter(filename, size);
     }
 
     public static IFileReader reader(String filename) throws IOException