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/10/20 19:45:22 UTC

svn commit: r827741 - /incubator/cassandra/trunk/src/java/org/apache/cassandra/db/HintedHandOffManager.java

Author: jbellis
Date: Tue Oct 20 17:45:22 2009
New Revision: 827741

URL: http://svn.apache.org/viewvc?rev=827741&view=rev
Log:
fix HH of tombstones.  patch by jbellis; reviewed by junrao for CASSANDRA-491

Modified:
    incubator/cassandra/trunk/src/java/org/apache/cassandra/db/HintedHandOffManager.java

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/db/HintedHandOffManager.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/HintedHandOffManager.java?rev=827741&r1=827740&r2=827741&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/db/HintedHandOffManager.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/db/HintedHandOffManager.java Tue Oct 20 17:45:22 2009
@@ -41,6 +41,27 @@
 
 
 /**
+ * For each table (keyspace), there is a row in the system hints CF.
+ * SuperColumns in that row are keys for which we have hinted data.
+ * Subcolumns names within that supercolumn are host IPs. Subcolumn values are always empty.
+ * Instead, we store the row data "normally" in the application table it belongs in.
+ *
+ * So when we deliver hints we look up endpoints that need data delivered
+ * on a per-key basis, then read that entire row out and send it over.
+ * (TODO handle rows that have incrementally grown too large for a single message.)
+ *
+ * HHM never deletes the row from Application tables; there is no way to distinguish that
+ * from hinted tombstones!  instead, rely on cleanup compactions to remove data
+ * that doesn't belong on this node.  (Cleanup compactions may be started manually
+ * -- on a per node basis -- with "nodeprobe cleanup.")
+ *
+ * TODO this avoids our hint rows from growing excessively large by offloading the
+ * message data into application tables.  But, this means that cleanup compactions
+ * will nuke HH data.  Probably better would be to store the RowMutation messages
+ * in a HHData (non-super) CF, modifying the above to store a UUID value in the
+ * HH subcolumn value, which we use as a key to a [standard] HHData system CF
+ * that would contain the message bytes.
+ *
  * There are two ways hinted data gets delivered to the intended nodes.
  *
  * runHints() runs periodically and pushes the hinted data on this node to
@@ -49,6 +70,7 @@
  * runDelieverHints() is called when some other node starts up (potentially
  * from a failure) and delivers the hinted data just to that node.
  */
+
 public class HintedHandOffManager
 {
     private static HintedHandOffManager instance_;
@@ -88,12 +110,7 @@
 
         Table table = Table.open(tableName);
         Row row = table.get(key);
-        Row purgedRow = new Row(tableName,key);
-        for (ColumnFamily cf : row.getColumnFamilies())
-        {
-            purgedRow.addColumnFamily(ColumnFamilyStore.removeDeleted(cf));
-        }
-        RowMutation rm = new RowMutation(tableName, purgedRow);
+        RowMutation rm = new RowMutation(tableName, row);
         Message message = rm.makeRowMutationMessage();
         QuorumResponseHandler<Boolean> quorumResponseHandler = new QuorumResponseHandler<Boolean>(1, new WriteResponseResolver());
         MessagingService.instance().sendRR(message, new EndPoint[]{ endPoint }, quorumResponseHandler);
@@ -108,40 +125,10 @@
         rm.apply();
     }
 
-    private static void deleteHintedData(String tableName, String key) throws IOException
+    private static void deleteHintKey(String tableName, byte[] key) throws IOException
     {
-        // delete the row from Application CFs: find the largest timestamp in any of
-        // the data columns, and delete the entire CF with that value for the tombstone.
-
-        // Note that we delete all data associated with the key: this may be more than
-        // we sent earlier in sendMessage, since HH is not serialized with writes.
-        // This is sub-optimal but okay, since HH is just an effort to make a recovering
-        // node more consistent than it would have been; we can rely on the other
-        // consistency mechanisms to finish the job in this corner case.
-        RowMutation rm = new RowMutation(tableName, key);
-        Table table = Table.open(tableName);
-        Row row = table.get(key); // not necessary to do removeDeleted here
-        Collection<ColumnFamily> cfs = row.getColumnFamilies();
-        for (ColumnFamily cf : cfs)
-        {
-            long maxTS = Long.MIN_VALUE;
-            if (!cf.isSuper())
-            {
-                for (IColumn col : cf.getSortedColumns())
-                    maxTS = Math.max(maxTS, col.timestamp());
-            }
-            else
-            {
-                for (IColumn col : cf.getSortedColumns())
-                {
-                    maxTS = Math.max(maxTS, col.timestamp());
-                    Collection<IColumn> subColumns = col.getSubColumns();
-                    for (IColumn subCol : subColumns)
-                        maxTS = Math.max(maxTS, subCol.timestamp());
-                }
-            }
-            rm.delete(new QueryPath(cf.name()), maxTS);
-        }
+        RowMutation rm = new RowMutation(Table.SYSTEM_TABLE, tableName);
+        rm.delete(new QueryPath(HINTS_CF, key, null), System.currentTimeMillis());
         rm.apply();
     }
 
@@ -183,7 +170,7 @@
                 }
                 if (deleted == endpoints.size())
                 {
-                    deleteHintedData(tableName, keyStr);
+                    deleteHintKey(tableName, keyColumn.name());
                 }
             }
         }
@@ -219,13 +206,17 @@
                 Collection<IColumn> endpoints = keyColumn.getSubColumns();
                 for (IColumn hintEndPoint : endpoints)
                 {
-                    if (hintEndPoint.name().equals(targetEPBytes) && sendMessage(endPoint.getHost(), null, keyStr))
+                    if (new String(hintEndPoint.name(), "UTF-8").equals(targetEPBytes) && sendMessage(endPoint.getHost(), null, keyStr))
                     {
-                        deleteEndPoint(hintEndPoint.name(), tableName, keyColumn.name(), keyColumn.timestamp());
                         if (endpoints.size() == 1)
                         {
-                            deleteHintedData(tableName, keyStr);
+                            deleteHintKey(tableName, keyColumn.name());
+                        }
+                        else
+                        {
+                            deleteEndPoint(hintEndPoint.name(), tableName, keyColumn.name(), keyColumn.timestamp());
                         }
+                        break;
                     }
                 }
             }