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 2011/11/16 23:07:35 UTC

svn commit: r1202904 - in /cassandra/branches/cassandra-1.0: ./ src/java/org/apache/cassandra/db/ src/java/org/apache/cassandra/io/sstable/

Author: jbellis
Date: Wed Nov 16 22:07:35 2011
New Revision: 1202904

URL: http://svn.apache.org/viewvc?rev=1202904&view=rev
Log:
record partitioner in sstable metadata component
patch by Yuki Morishita and jbellis for CASSANDRA-3393

Modified:
    cassandra/branches/cassandra-1.0/CHANGES.txt
    cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/SystemTable.java
    cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/Descriptor.java
    cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/SSTableMetadata.java
    cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/SSTableReader.java

Modified: cassandra/branches/cassandra-1.0/CHANGES.txt
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/CHANGES.txt?rev=1202904&r1=1202903&r2=1202904&view=diff
==============================================================================
--- cassandra/branches/cassandra-1.0/CHANGES.txt (original)
+++ cassandra/branches/cassandra-1.0/CHANGES.txt Wed Nov 16 22:07:35 2011
@@ -1,6 +1,7 @@
 1.0.4
  * fix "liveSize" stat when sstables are removed (CASSANDRA-3496)
  * add bloom filter FP rates to nodetool cfstats (CASSANDRA-3347)
+ * record partitioner in sstable metadata component (CASSANDRA-3393)
 
 
 1.0.3

Modified: cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/SystemTable.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/SystemTable.java?rev=1202904&r1=1202903&r2=1202904&view=diff
==============================================================================
--- cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/SystemTable.java (original)
+++ cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/SystemTable.java Wed Nov 16 22:07:35 2011
@@ -62,7 +62,6 @@ public class SystemTable
     private static final ByteBuffer TOKEN = ByteBufferUtil.bytes("Token");
     private static final ByteBuffer GENERATION = ByteBufferUtil.bytes("Generation");
     private static final ByteBuffer CLUSTERNAME = ByteBufferUtil.bytes("ClusterName");
-    private static final ByteBuffer PARTITIONER = ByteBufferUtil.bytes("Partioner");
     private static final ByteBuffer CURRENT_LOCAL_NODE_ID_KEY = ByteBufferUtil.bytes("CurrentLocal");
     private static final ByteBuffer ALL_LOCAL_NODE_ID_KEY = ByteBufferUtil.bytes("Local");
 
@@ -247,7 +246,7 @@ public class SystemTable
      * One of three things will happen if you try to read the system table:
      * 1. files are present and you can read them: great
      * 2. no files are there: great (new node is assumed)
-     * 3. files are present but you can't read them: bad (suspect that the partitioner was changed).
+     * 3. files are present but you can't read them: bad
      * @throws ConfigurationException
      */
     public static void checkHealth() throws ConfigurationException, IOException
@@ -260,28 +259,26 @@ public class SystemTable
         catch (AssertionError err)
         {
             // this happens when a user switches from OPP to RP.
-            ConfigurationException ex = new ConfigurationException("Could not read system table. Did you change partitioners?");
+            ConfigurationException ex = new ConfigurationException("Could not read system table!");
             ex.initCause(err);
             throw ex;
         }
         
         SortedSet<ByteBuffer> cols = new TreeSet<ByteBuffer>(BytesType.instance);
-        cols.add(PARTITIONER);
         cols.add(CLUSTERNAME);
         QueryFilter filter = QueryFilter.getNamesFilter(decorate(LOCATION_KEY), new QueryPath(STATUS_CF), cols);
         ColumnFamily cf = table.getColumnFamilyStore(STATUS_CF).getColumnFamily(filter);
         
         if (cf == null)
         {
-            // this is either a brand new node (there will be no files), or the partitioner was changed from RP to OPP.
+            // this is a brand new node
             ColumnFamilyStore cfs = table.getColumnFamilyStore(STATUS_CF);
             if (!cfs.getSSTables().isEmpty())
-                throw new ConfigurationException("Found system table files, but they couldn't be loaded. Did you change the partitioner?");
+                throw new ConfigurationException("Found system table files, but they couldn't be loaded!");
 
             // no system files.  this is a new node.
             RowMutation rm = new RowMutation(Table.SYSTEM_TABLE, LOCATION_KEY);
             cf = ColumnFamily.create(Table.SYSTEM_TABLE, SystemTable.STATUS_CF);
-            cf.addColumn(new Column(PARTITIONER, ByteBufferUtil.bytes(DatabaseDescriptor.getPartitioner().getClass().getName()), FBUtilities.timestampMicros()));
             cf.addColumn(new Column(CLUSTERNAME, ByteBufferUtil.bytes(DatabaseDescriptor.getClusterName()), FBUtilities.timestampMicros()));
             rm.add(cf);
             rm.apply();
@@ -290,12 +287,8 @@ public class SystemTable
         }
         
         
-        IColumn partitionerCol = cf.getColumn(PARTITIONER);
         IColumn clusterCol = cf.getColumn(CLUSTERNAME);
-        assert partitionerCol != null;
         assert clusterCol != null;
-        if (!DatabaseDescriptor.getPartitioner().getClass().getName().equals(ByteBufferUtil.string(partitionerCol.value())))
-            throw new ConfigurationException("Detected partitioner mismatch! Did you change the partitioner?");
         String savedClusterName = ByteBufferUtil.string(clusterCol.value());
         if (!DatabaseDescriptor.getClusterName().equals(savedClusterName))
             throw new ConfigurationException("Saved cluster name " + savedClusterName + " != configured name " + DatabaseDescriptor.getClusterName());

Modified: cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/Descriptor.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/Descriptor.java?rev=1202904&r1=1202903&r2=1202904&view=diff
==============================================================================
--- cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/Descriptor.java (original)
+++ cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/Descriptor.java Wed Nov 16 22:07:35 2011
@@ -56,7 +56,9 @@ public class Descriptor
     // f (0.7.0): switched bloom filter implementations in data component
     // g (0.8): tracks flushed-at context in metadata component
     // h (1.0): tracks max client timestamp in metadata component
-    public static final String CURRENT_VERSION = "hb";
+    // hb (1.0.3): records compression ration in metadata component
+    // hc (1.0.4): records partitioner in metadata component
+    public static final String CURRENT_VERSION = "hc";
 
     public final File directory;
     /** version has the following format: <code>[a-z]+</code> */
@@ -75,6 +77,7 @@ public class Descriptor
     public final boolean metadataIncludesReplayPosition;
     public final boolean tracksMaxTimestamp;
     public final boolean hasCompressionRatio;
+    public final boolean hasPartitioner;
 
     public enum TempState
     {
@@ -117,6 +120,7 @@ public class Descriptor
         metadataIncludesReplayPosition = version.compareTo("g") >= 0;
         tracksMaxTimestamp = version.compareTo("h") >= 0;
         hasCompressionRatio = version.compareTo("hb") >= 0;
+        hasPartitioner = version.compareTo("hc") >= 0;
         isLatestVersion = version.compareTo(CURRENT_VERSION) == 0;
     }
 

Modified: cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/SSTableMetadata.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/SSTableMetadata.java?rev=1202904&r1=1202903&r2=1202904&view=diff
==============================================================================
--- cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/SSTableMetadata.java (original)
+++ cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/SSTableMetadata.java Wed Nov 16 22:07:35 2011
@@ -29,6 +29,7 @@ import java.io.IOException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.utils.EstimatedHistogram;
@@ -40,6 +41,8 @@ import org.apache.cassandra.utils.Estima
  *  - estimated column count histogram
  *  - replay position
  *  - max column timestamp
+ *  - compression ratio
+ *  - partitioner
  *
  * An SSTableMetadata should be instantiated via the Collector, openFromDescriptor()
  * or createDefaultInstance()
@@ -52,20 +55,27 @@ public class SSTableMetadata
     protected final ReplayPosition replayPosition;
     protected final long maxTimestamp;
     protected final double compressionRatio;
+    protected final String partitioner;
     public static final SSTableMetadataSerializer serializer = new SSTableMetadataSerializer();
 
     private SSTableMetadata()
     {
-        this(defaultRowSizeHistogram(), defaultColumnCountHistogram(), ReplayPosition.NONE, Long.MIN_VALUE, Double.MIN_VALUE);
+        this(defaultRowSizeHistogram(),
+             defaultColumnCountHistogram(),
+             ReplayPosition.NONE,
+             Long.MIN_VALUE,
+             Double.MIN_VALUE,
+             DatabaseDescriptor.getPartitioner().getClass().getCanonicalName());
     }
 
-    private SSTableMetadata(EstimatedHistogram rowSizes, EstimatedHistogram columnCounts, ReplayPosition replayPosition, long maxTimestamp, double cr)
+    private SSTableMetadata(EstimatedHistogram rowSizes, EstimatedHistogram columnCounts, ReplayPosition replayPosition, long maxTimestamp, double cr, String partitioner)
     {
         this.estimatedRowSize = rowSizes;
         this.estimatedColumnCount = columnCounts;
         this.replayPosition = replayPosition;
         this.maxTimestamp = maxTimestamp;
         this.compressionRatio = cr;
+        this.partitioner = partitioner;
     }
 
     public static SSTableMetadata createDefaultInstance()
@@ -103,6 +113,11 @@ public class SSTableMetadata
         return compressionRatio;
     }
 
+    public String getPartitioner()
+    {
+        return partitioner;
+    }
+
     static EstimatedHistogram defaultColumnCountHistogram()
     {
         // EH of 114 can track a max value of 2395318855, i.e., > 2B columns
@@ -158,7 +173,12 @@ public class SSTableMetadata
 
         public SSTableMetadata finalizeMetadata()
         {
-            return new SSTableMetadata(estimatedRowSize, estimatedColumnCount, replayPosition, maxTimestamp, compressionRatio);
+            return new SSTableMetadata(estimatedRowSize,
+                                       estimatedColumnCount,
+                                       replayPosition,
+                                       maxTimestamp,
+                                       compressionRatio,
+                                       DatabaseDescriptor.getPartitioner().getClass().getCanonicalName());
         }
 
         public Collector estimatedRowSize(EstimatedHistogram estimatedRowSize)
@@ -191,6 +211,7 @@ public class SSTableMetadata
             ReplayPosition.serializer.serialize(sstableStats.getReplayPosition(), dos);
             dos.writeLong(sstableStats.getMaxTimestamp());
             dos.writeDouble(sstableStats.getCompressionRatio());
+            dos.writeUTF(sstableStats.getPartitioner());
         }
 
         public SSTableMetadata deserialize(Descriptor descriptor) throws IOException
@@ -223,9 +244,12 @@ public class SSTableMetadata
                                           : ReplayPosition.NONE;
             long maxTimestamp = desc.tracksMaxTimestamp ? dis.readLong() : Long.MIN_VALUE;
             double compressionRatio = desc.hasCompressionRatio
-                                        ? dis.readDouble()
-                                        : Double.MIN_VALUE;
-            return new SSTableMetadata(rowSizes, columnCounts, replayPosition, maxTimestamp, compressionRatio);
+                                    ? dis.readDouble()
+                                    : Double.MIN_VALUE;
+            String partitioner = desc.hasPartitioner
+                               ? dis.readUTF()
+                               : DatabaseDescriptor.getPartitioner().getClass().getCanonicalName();
+            return new SSTableMetadata(rowSizes, columnCounts, replayPosition, maxTimestamp, compressionRatio, partitioner);
         }
     }
 }

Modified: cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/SSTableReader.java?rev=1202904&r1=1202903&r2=1202904&view=diff
==============================================================================
--- cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/SSTableReader.java (original)
+++ cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/SSTableReader.java Wed Nov 16 22:07:35 2011
@@ -137,6 +137,12 @@ public class SSTableReader extends SSTab
                                         ? SSTableMetadata.serializer.deserialize(descriptor)
                                         : SSTableMetadata.createDefaultInstance();
 
+        // check if sstable is created using same partitioner as this node
+        String partitionerName = partitioner.getClass().getCanonicalName();
+        if (!partitionerName.equals(sstableMetadata.getPartitioner()))
+            throw new RuntimeException(String.format("Cannot open %s because partitioner does not match %s",
+                                                     descriptor, partitionerName));
+
         SSTableReader sstable = new SSTableReader(descriptor,
                                                   components,
                                                   metadata,