Hmm, that dates back to early 0.7...  I think the intent is to skip
deleted hints, even if the tombstoned hints haven't expired yet (and
thus would not be elided by the getCF filter).  Sounds like that's
breaking pagination.  Normally, all hints get delivered at once and
then we compact away the tombstones -- that's my guess as to why we
haven't hit this before.

Can you create a ticket?

On Mon, Feb 20, 2012 at 4:52 PM, Todd Burruss <bburr...@expedia.com> wrote:
> I'm testing hinted handoff in 1.1 beta1 and cannot seem to get a hint 
> delivered.  3 node cluster, RF = 3, writing with CL = ONE.  killed a host 
> then did the write using the CLI on another node.  I can see hint waiting 
> using CLI and I see the log messages at the end of this email.  this suggests 
> the hints exist bu are not being delivered (and I'll see the log messages 
> over and over.)
>
> I did see tracing with debugger and see that in 
> HintedHandoffManager.deliverHintsToEndpointInternal, this line will remove 
> the hint because of the Integer.MAX_VALUE
>
>
>            ColumnFamily hintsPage = 
> ColumnFamilyStore.removeDeleted(hintStore.getColumnFamily(filter), 
> Integer.MAX_VALUE);
>
> I'm not sure I quite understand while MAX is used when the same "remove" is 
> done in getColumnFamily(filter).  regardless if it is useful or not, it 
> prevents the hints from delivery.
>
> any thoughts?
>
>
> [default@unknown] use system;
> Authenticated to keyspace: system
> [default@system] list hintscolumnfamily;
> Using default limit of 100
> -------------------
> RowKey: 00
> => (super_column=493ecfa05c1411e100000da23097c7ff,
>     (column=6b6579, value=6b35, timestamp=1329777799580, ttl=86400)
>     (column=6d75746174696f6e, 
> value=000662746f64646200026b3500000001000003e801000003e87fffffff80000000000000000000000100026335000004b96d055fd1300000000c68696e746564207772697465,
>  timestamp=1329777799579, ttl=86400)
>     (column=7461626c65, value=62746f646462, timestamp=1329777799580, 
> ttl=86400)
>     (column=76657273696f6e, value=00000004, timestamp=1329777799580, 
> ttl=86400))
>
> 1 Row Returned.
> Elapsed time: 58 msec(s).
>
>
>  INFO [HintedHandoff:1] 2012-02-20 14:44:53,811 HintedHandOffManager.java 
> (line 296) Started hinted handoff for token: 0 with IP: /192.168.56.1
>  INFO [HintedHandoff:1] 2012-02-20 14:44:53,815 HintedHandOffManager.java 
> (line 373) Finished hinted handoff of 0 rows to endpoint /192.168.56.1
>
>
> I have a hint waiting to be deli



-- 
Jonathan Ellis
Project Chair, Apache Cassandra
co-founder of DataStax, the source for professional Cassandra support
http://www.datastax.com

Reply via email to