Author: gdusbabek
Date: Wed May 12 18:56:52 2010
New Revision: 943636

URL: http://svn.apache.org/viewvc?rev=943636&view=rev
Log:
remove unused cfIdMap from CLH. Patch by gdusbabek, reviewed by jbellis. 
CASSANDRA-1069

Modified:
    cassandra/trunk/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
    
cassandra/trunk/src/java/org/apache/cassandra/db/commitlog/CommitLogHeader.java
    
cassandra/trunk/test/unit/org/apache/cassandra/db/commitlog/CommitLogHeaderTest.java

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/commitlog/CommitLog.java?rev=943636&r1=943635&r2=943636&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/commitlog/CommitLog.java 
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/commitlog/CommitLog.java 
Wed May 12 18:56:52 2010
@@ -189,7 +189,6 @@ public class CommitLog
             int bufferSize = (int)Math.min(file.length(), 32 * 1024 * 1024);
             BufferedRandomAccessFile reader = new 
BufferedRandomAccessFile(file.getAbsolutePath(), "r", bufferSize);
             final CommitLogHeader clHeader = 
CommitLogHeader.readCommitLogHeader(reader);
-            final Map<Pair<String, String>, Integer> cfIdMap = 
clHeader.getCfIdMap();
             /* seek to the lowest position where any CF has non-flushed data */
             int lowPos = CommitLogHeader.getLowestPosition(clHeader);
             if (lowPos == 0)

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/db/commitlog/CommitLogHeader.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/commitlog/CommitLogHeader.java?rev=943636&r1=943635&r2=943636&view=diff
==============================================================================
--- 
cassandra/trunk/src/java/org/apache/cassandra/db/commitlog/CommitLogHeader.java 
(original)
+++ 
cassandra/trunk/src/java/org/apache/cassandra/db/commitlog/CommitLogHeader.java 
Wed May 12 18:56:52 2010
@@ -52,14 +52,11 @@ class CommitLogHeader
     }
 
     private Map<Integer, Integer> lastFlushedAt; // position at which each CF 
was last flushed
-    private final byte[] serializedCfMap; // serialized. only needed during 
commit log recovery.
     private final int cfCount; // we keep this in case cfcount changes in the 
interim (size of lastFlushedAt is not a good indication).
     
-    private transient Map<Pair<String, String>, Integer> cfIdMap; // only 
needed during recovery. created from this.serializedCfMap.
-    
     CommitLogHeader()
     {
-        this(new HashMap<Integer, Integer>(), 
serializeCfIdMap(CFMetaData.getCfIdMap()), CFMetaData.getCfIdMap().size());
+        this(new HashMap<Integer, Integer>(), CFMetaData.getCfIdMap().size());
     }
     
     /*
@@ -67,11 +64,10 @@ class CommitLogHeader
      * also builds an index of position to column family
      * Id.
     */
-    private CommitLogHeader(Map<Integer, Integer> lastFlushedAt, byte[] 
serializedCfMap, int cfCount)
+    private CommitLogHeader(Map<Integer, Integer> lastFlushedAt, int cfCount)
     {
         this.cfCount = cfCount;
         this.lastFlushedAt = lastFlushedAt;
-        this.serializedCfMap = serializedCfMap;
         assert lastFlushedAt.size() <= cfCount;
     }
         
@@ -100,53 +96,6 @@ class CommitLogHeader
     {
         return lastFlushedAt.isEmpty();
     }
-    
-    synchronized Map<Pair<String, String>, Integer> getCfIdMap()
-    {
-        if (cfIdMap != null)
-            return cfIdMap;
-        DataInputStream in = new DataInputStream(new 
ByteArrayInputStream(serializedCfMap));
-        cfIdMap = new HashMap<Pair<String, String>, Integer>();
-        try
-        {
-            int sz = in.readInt();
-            for (int i = 0; i < sz; i++)
-            {
-                Pair<String, String> key = new Pair<String, 
String>(in.readUTF(), in.readUTF());
-                cfIdMap.put(key, in.readInt());
-            }
-        }
-        catch (IOException ex)
-        {
-            throw new IOError(ex);
-        }
-        return cfIdMap;
-    }
-    
-    private static byte[] serializeCfIdMap(Map<Pair<String, String>, Integer> 
map)
-    {
-        ByteArrayOutputStream out = new ByteArrayOutputStream();
-        DataOutputStream dos = new DataOutputStream(out);
-        
-        try
-        {
-            dos.writeInt(map.size());
-            for (Map.Entry<Pair<String, String>, Integer> entry : 
map.entrySet())
-            {
-                Pair<String, String> p = entry.getKey();
-                dos.writeUTF(p.left);
-                dos.writeUTF(p.right);
-                dos.writeInt(entry.getValue());
-            }
-            dos.close();
-        }
-        catch (IOException ex)
-        {
-            throw new IOError(ex);
-        }
-            
-        return out.toByteArray();
-    }
 
     byte[] toByteArray() throws IOException
     {
@@ -194,18 +143,14 @@ class CommitLogHeader
             assert clHeader.lastFlushedAt.size() <= clHeader.cfCount;
             Checksum checksum = new CRC32();
 
-            // write the first checksum after the fixed-size part, so we won't 
OOM allocating a bogus cfmap buffer
+            // write the first checksum after the fixed-size part, so we won't 
read garbage lastFlushedAt data.
             dos.writeInt(clHeader.cfCount); // 4
-            dos.writeInt(clHeader.serializedCfMap.length); // 4
             dos.writeInt(clHeader.lastFlushedAt.size()); // 4
             checksum.update(clHeader.cfCount);
-            checksum.update(clHeader.serializedCfMap.length);
             checksum.update(clHeader.lastFlushedAt.size());
             dos.writeLong(checksum.getValue());
 
-            // write the 2nd checksum after the cfmap and lastflushedat map
-            dos.write(clHeader.serializedCfMap); // colMap.length
-            checksum.update(clHeader.serializedCfMap, 0, 
clHeader.serializedCfMap.length);
+            // write the 2nd checksum after the lastflushedat map
             for (Map.Entry<Integer, Integer> entry : 
clHeader.lastFlushedAt.entrySet())
             {
                 dos.writeInt(entry.getKey()); // 4
@@ -229,18 +174,12 @@ class CommitLogHeader
 
             int cfCount = dis.readInt();
             checksum.update(cfCount);
-            int cfMapLength = dis.readInt();
-            checksum.update(cfMapLength);
             int lastFlushedAtSize = dis.readInt();
             checksum.update(lastFlushedAtSize);
             if (checksum.getValue() != dis.readLong())
             {
                 throw new IOException("Invalid or corrupt commitlog header");
             }
-
-            byte[] cfMap = new byte[cfMapLength];
-            dis.readFully(cfMap);
-            checksum.update(cfMap, 0, cfMap.length);
             Map<Integer, Integer> lastFlushedAt = new HashMap<Integer, 
Integer>();
             for (int i = 0; i < lastFlushedAtSize; i++)
             {
@@ -255,7 +194,7 @@ class CommitLogHeader
                 throw new IOException("Invalid or corrupt commitlog header");
             }
 
-            return new CommitLogHeader(lastFlushedAt, cfMap, cfCount);
+            return new CommitLogHeader(lastFlushedAt, cfCount);
         }
     }
 }

Modified: 
cassandra/trunk/test/unit/org/apache/cassandra/db/commitlog/CommitLogHeaderTest.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/db/commitlog/CommitLogHeaderTest.java?rev=943636&r1=943635&r2=943636&view=diff
==============================================================================
--- 
cassandra/trunk/test/unit/org/apache/cassandra/db/commitlog/CommitLogHeaderTest.java
 (original)
+++ 
cassandra/trunk/test/unit/org/apache/cassandra/db/commitlog/CommitLogHeaderTest.java
 Wed May 12 18:56:52 2010
@@ -80,21 +80,4 @@ public class CommitLogHeaderTest extends
         
         assert one.length == two.length;
     }
-
-    @Test
-    public void byteArray() throws IOException
-    {
-        CommitLogHeader clh = new CommitLogHeader();
-        assert clh.getCfIdMap().size() > 0;
-        CommitLogHeader clh2 = CommitLogHeader.serializer.deserialize(new 
DataInputStream(new ByteArrayInputStream(clh.toByteArray())));
-        assert clh.getCfIdMap().equals(clh2.getCfIdMap());
-    }
-    
-    @Test
-    public void cfMapSerialization() throws IOException
-    {
-        Map<Pair<String, String>, Integer> map = CFMetaData.getCfIdMap();
-        CommitLogHeader clh = new CommitLogHeader();
-        assert clh.getCfIdMap().equals(map);
-    }
 }


Reply via email to