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 2014/02/08 23:34:36 UTC

git commit: Remove pre-2.0 supercolumn compatibility code

Updated Branches:
  refs/heads/trunk b75ed1c34 -> cca65d7c1


Remove pre-2.0 supercolumn compatibility code


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/cca65d7c
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/cca65d7c
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/cca65d7c

Branch: refs/heads/trunk
Commit: cca65d7c1638dcd9370b080f08fd55faefc2733e
Parents: b75ed1c
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Sun Feb 9 01:34:37 2014 +0300
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Sun Feb 9 01:34:37 2014 +0300

----------------------------------------------------------------------
 .../cassandra/db/ColumnFamilySerializer.java    |  26 +--
 .../org/apache/cassandra/db/SuperColumns.java   | 225 +------------------
 2 files changed, 6 insertions(+), 245 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/cca65d7c/src/java/org/apache/cassandra/db/ColumnFamilySerializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilySerializer.java b/src/java/org/apache/cassandra/db/ColumnFamilySerializer.java
index 92aa955..7a1d16d 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilySerializer.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilySerializer.java
@@ -60,13 +60,6 @@ public class ColumnFamilySerializer implements IVersionedSerializer<ColumnFamily
 
             out.writeBoolean(true);
             serializeCfId(cf.id(), out, version);
-
-            if (cf.metadata().isSuper() && version < MessagingService.VERSION_20)
-            {
-                SuperColumns.serializeSuperColumnFamily(cf, out, version);
-                return;
-            }
-
             cf.getComparator().deletionInfoSerializer().serialize(cf.deletionInfo(), out, version);
             ColumnSerializer columnSerializer = cf.getComparator().columnSerializer();
             int count = cf.getColumnCount();
@@ -120,20 +113,11 @@ public class ColumnFamilySerializer implements IVersionedSerializer<ColumnFamily
 
     public long contentSerializedSize(ColumnFamily cf, TypeSizes typeSizes, int version)
     {
-        long size = 0L;
-
-        if (cf.metadata().isSuper() && version < MessagingService.VERSION_20)
-        {
-            size += SuperColumns.serializedSize(cf, typeSizes, version);
-        }
-        else
-        {
-            size += cf.getComparator().deletionInfoSerializer().serializedSize(cf.deletionInfo(), typeSizes, version);
-            size += typeSizes.sizeof(cf.getColumnCount());
-            ColumnSerializer columnSerializer = cf.getComparator().columnSerializer();
-            for (Cell cell : cf)
-                size += columnSerializer.serializedSize(cell, typeSizes);
-        }
+        long size = cf.getComparator().deletionInfoSerializer().serializedSize(cf.deletionInfo(), typeSizes, version);
+        size += typeSizes.sizeof(cf.getColumnCount());
+        ColumnSerializer columnSerializer = cf.getComparator().columnSerializer();
+        for (Cell cell : cf)
+            size += columnSerializer.serializedSize(cell, typeSizes);
         return size;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cca65d7c/src/java/org/apache/cassandra/db/SuperColumns.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SuperColumns.java b/src/java/org/apache/cassandra/db/SuperColumns.java
index a7d44dc..b854856 100644
--- a/src/java/org/apache/cassandra/db/SuperColumns.java
+++ b/src/java/org/apache/cassandra/db/SuperColumns.java
@@ -18,15 +18,12 @@
 package org.apache.cassandra.db;
 
 import java.io.DataInput;
-import java.io.DataOutput;
 import java.io.IOError;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Iterator;
-import java.util.LinkedHashMap;
 import java.util.List;
-import java.util.Map;
 import java.util.SortedSet;
 import java.util.TreeSet;
 
@@ -43,68 +40,6 @@ public class SuperColumns
         return new SCIterator(in, superColumnCount, flag, expireBefore, type);
     }
 
-    public static void serializeSuperColumnFamily(ColumnFamily scf, DataOutput out, int version) throws IOException
-    {
-        /*
-         * There is 2 complications:
-         *   1) We need to know the number of super columns in the column
-         *   family to write in the header (so we do a first pass to group
-         *   columns before serializing).
-         *   2) For deletion infos, we need to figure out which are top-level
-         *   deletions and which are super columns deletions (i.e. the
-         *   subcolumns range deletions).
-         */
-        DeletionInfo delInfo = scf.deletionInfo();
-        Map<CellName, List<Cell>> scMap = groupSuperColumns(scf);
-
-        // Actually Serialize
-        scf.getComparator().deletionInfoSerializer().serialize(new DeletionInfo(delInfo.getTopLevelDeletion()), out, version);
-        out.writeInt(scMap.size());
-
-        CellNameType subComparator = subType(scf.getComparator());
-        for (Map.Entry<CellName, List<Cell>> entry : scMap.entrySet())
-        {
-            scf.getComparator().cellSerializer().serialize(entry.getKey(), out);
-
-            DeletionTime delTime = delInfo.rangeCovering(entry.getKey());
-            DeletionInfo scDelInfo = delTime == null ? DeletionInfo.live() : new DeletionInfo(delTime);
-            DeletionTime.serializer.serialize(scDelInfo.getTopLevelDeletion(), out);
-
-            out.writeInt(entry.getValue().size());
-            ColumnSerializer serializer = subComparator.columnSerializer();
-            for (Cell subCell : entry.getValue())
-                serializer.serialize(subCell, out);
-        }
-    }
-
-    private static Map<CellName, List<Cell>> groupSuperColumns(ColumnFamily scf)
-    {
-        CellNameType type = scf.getComparator();
-        // The order of insertion matters!
-        Map<CellName, List<Cell>> scMap = new LinkedHashMap<>();
-
-        CellName scName = null;
-        List<Cell> subCells = null;
-        CellNameType scType = scType(type);
-        CellNameType subType = subType(type);
-        for (Cell cell : scf)
-        {
-            CellName newScName = scType.makeCellName(scName(cell.name()));
-            CellName newSubName = subType.makeCellName(subName(cell.name()));
-
-            if (scName == null || scType.compare(scName, newScName) != 0)
-            {
-                // new super cell
-                scName = newScName;
-                subCells = new ArrayList<>();
-                scMap.put(scName, subCells);
-            }
-
-            subCells.add(((Cell) cell).withUpdatedName(newSubName));
-        }
-        return scMap;
-    }
-
     public static void deserializerSuperColumnFamily(DataInput in, ColumnFamily cf, ColumnSerializer.Flag flag, int version) throws IOException
     {
         // Note that there was no way to insert a range tombstone in a SCF in 1.2
@@ -116,32 +51,6 @@ public class SuperColumns
             cf.addAtom(iter.next());
     }
 
-    public static long serializedSize(ColumnFamily scf, TypeSizes typeSizes, int version)
-    {
-        Map<CellName, List<Cell>> scMap = groupSuperColumns(scf);
-        DeletionInfo delInfo = scf.deletionInfo();
-
-        // Actually Serialize
-        long size = scType(scf.getComparator()).deletionInfoSerializer().serializedSize(new DeletionInfo(delInfo.getTopLevelDeletion()), version);
-
-        CellNameType scType = scType(scf.getComparator());
-        CellNameType subType = subType(scf.getComparator());
-        ColumnSerializer colSer = subType.columnSerializer();
-        for (Map.Entry<CellName, List<Cell>> entry : scMap.entrySet())
-        {
-            size += scType.cellSerializer().serializedSize(entry.getKey(), typeSizes);
-
-            DeletionTime delTime = delInfo.rangeCovering(entry.getKey());
-            DeletionInfo scDelInfo = delTime == null ? DeletionInfo.live() : new DeletionInfo(delTime);
-            size += DeletionTime.serializer.serializedSize(scDelInfo.getTopLevelDeletion(), TypeSizes.NATIVE);
-
-            size += typeSizes.sizeof(entry.getValue().size());
-            for (Cell subCell : entry.getValue())
-                size += colSer.serializedSize(subCell, typeSizes);
-        }
-        return size;
-    }
-
     private static class SCIterator implements Iterator<OnDiskAtom>
     {
         private final DataInput in;
@@ -215,11 +124,6 @@ public class SuperColumns
         }
     }
 
-    private static CellNameType scType(CellNameType type)
-    {
-        return new SimpleDenseCellNameType(type.subtype(0));
-    }
-
     private static CellNameType subType(CellNameType type)
     {
         return new SimpleDenseCellNameType(type.subtype(1));
@@ -259,121 +163,6 @@ public class SuperColumns
         return CellNames.compositeDense(scName).end();
     }
 
-    public static SCFilter filterToSC(CellNameType type, IDiskAtomFilter filter)
-    {
-        if (filter instanceof NamesQueryFilter)
-            return namesFilterToSC(type, (NamesQueryFilter)filter);
-        else
-            return sliceFilterToSC(type, (SliceQueryFilter)filter);
-    }
-
-    public static SCFilter namesFilterToSC(CellNameType type, NamesQueryFilter filter)
-    {
-        ByteBuffer scName = null;
-        CellNameType subComparator = subType(type);
-        SortedSet<CellName> newColumns = new TreeSet<CellName>(subComparator);
-        for (CellName name : filter.columns)
-        {
-            ByteBuffer newScName = scName(name);
-
-            if (scName == null)
-            {
-                scName = newScName;
-            }
-            else if (type.subtype(0).compare(scName, newScName) != 0)
-            {
-                // If we're selecting column across multiple SC, it's not something we can translate for an old node
-                throw new RuntimeException("Cannot convert filter to old super column format. Update all nodes to Cassandra 2.0 first.");
-            }
-
-            newColumns.add(subComparator.makeCellName(name));
-        }
-        return new SCFilter(scName, new NamesQueryFilter(newColumns));
-    }
-
-    private static boolean isEndOfRange(Composite c)
-    {
-        return c.eoc() == Composite.EOC.END;
-    }
-
-    public static SCFilter sliceFilterToSC(CellNameType type, SliceQueryFilter filter)
-    {
-        /*
-         * There is 3 main cases that we can translate back into super column
-         * queries:
-         *   1) We have only one slice where the first component of start and
-         *   finish is the same, we translate as a slice query on one SC.
-         *   2) We have only one slice, neither the start and finish have a 2nd
-         *   component, and end has the 'end of component' set, we translate
-         *   as a slice of SCs.
-         *   3) Each slice has the same first component for start and finish, no
-         *   2nd component and each finish has the 'end of component' set, we
-         *   translate as a names query of SCs (the filter must then not be reversed).
-         * Otherwise, we can't do much.
-         */
-
-        boolean reversed = filter.reversed;
-        if (filter.slices.length == 1)
-        {
-            Composite start = filter.slices[0].start;
-            Composite finish = filter.slices[0].start;
-
-            if (filter.compositesToGroup == 1)
-            {
-                // Note: all the resulting filter must have compositeToGroup == 0 because this
-                // make no sense for super column on the destination node otherwise
-                if (start.isEmpty())
-                {
-                    if (finish.isEmpty())
-                        // An 'IdentityFilter', keep as is (except for the compositeToGroup)
-                        return new SCFilter(null, new SliceQueryFilter(filter.start(), filter.finish(), reversed, filter.count));
-
-                    if (subName(finish) == null
-                            && ((!reversed && !isEndOfRange(finish)) || (reversed && isEndOfRange(finish))))
-                        return new SCFilter(null, new SliceQueryFilter(Composites.EMPTY, CellNames.simpleDense(scName(finish)), reversed, filter.count));
-                }
-                else if (finish.isEmpty())
-                {
-                    if (subName(start) == null
-                            && ((!reversed && isEndOfRange(start)) || (reversed && !isEndOfRange(start))))
-                        return new SCFilter(null, new SliceQueryFilter(CellNames.simpleDense(scName(start)), Composites.EMPTY, reversed, filter.count));
-                }
-                else if (subName(start) == null && subName(finish) == null
-                        && ((   reversed && !isEndOfRange(start) && isEndOfRange(finish))
-                            || (!reversed &&  isEndOfRange(start) && !isEndOfRange(finish))))
-                {
-                    // A slice of supercolumns
-                    return new SCFilter(null, new SliceQueryFilter(CellNames.simpleDense(scName(start)),
-                                                                   CellNames.simpleDense(scName(finish)),
-                                                                   reversed,
-                                                                   filter.count));
-                }
-            }
-            else if (filter.compositesToGroup == 0 && type.subtype(0).compare(scName(start), scName(finish)) == 0)
-            {
-                // A slice of subcolumns
-                return new SCFilter(scName(start), filter.withUpdatedSlice(CellNames.simpleDense(subName(start)), CellNames.simpleDense(subName(finish))));
-            }
-        }
-        else if (!reversed)
-        {
-            SortedSet<CellName> columns = new TreeSet<CellName>(scType(type));
-            for (int i = 0; i < filter.slices.length; ++i)
-            {
-                Composite start = filter.slices[i].start;
-                Composite finish = filter.slices[i].finish;
-
-                if (subName(start) != null || subName(finish) != null
-                  || type.subtype(0).compare(scName(start), scName(finish)) != 0
-                  || isEndOfRange(start) || !isEndOfRange(finish))
-                    throw new RuntimeException("Cannot convert filter to old super column format. Update all nodes to Cassandra 2.0 first.");
-
-                columns.add(CellNames.simpleDense(scName(start)));
-            }
-            return new SCFilter(null, new NamesQueryFilter(columns));
-        }
-        throw new RuntimeException("Cannot convert filter to old super column format. Update all nodes to Cassandra 2.0 first.");
-    }
 
     public static IDiskAtomFilter fromSCFilter(CellNameType type, ByteBuffer scName, IDiskAtomFilter filter)
     {
@@ -397,7 +186,7 @@ public class SuperColumns
         }
         else
         {
-            SortedSet<CellName> newColumns = new TreeSet<CellName>(type);
+            SortedSet<CellName> newColumns = new TreeSet<>(type);
             for (CellName c : filter.columns)
                 newColumns.add(type.makeCellName(scName, c.toByteBuffer()));
             return filter.withUpdatedColumns(newColumns);
@@ -431,16 +220,4 @@ public class SuperColumns
             return new SliceQueryFilter(start, end, filter.reversed, filter.count);
         }
     }
-
-    public static class SCFilter
-    {
-        public final ByteBuffer scName;
-        public final IDiskAtomFilter updatedFilter;
-
-        public SCFilter(ByteBuffer scName, IDiskAtomFilter updatedFilter)
-        {
-            this.scName = scName;
-            this.updatedFilter = updatedFilter;
-        }
-    }
 }