[ 
https://issues.apache.org/jira/browse/CASSANDRA-19495?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17832856#comment-17832856
 ] 

Stefan Miklosovic edited comment on CASSANDRA-19495 at 4/1/24 3:13 PM:
-----------------------------------------------------------------------

BTW this ticket ... while it does what it should, if one starts to dig deeper, 
there are some not-so-obvious consequences of how hints delivery / deletion is 
done. For example, if one calls HintsServiceMBean.deleteAllHints (or 
deleteAllHintsForEndpoint), it will indeed delete the hints, but only these 
which are already on disk and for which writer is closed. That means that it 
will not delete hints which are in buffers or for which the writer is not 
closed yet. I consider this to be a bug. If a user invokes "delete all hints", 
it should indeed delete them _ALL_, in buffers included.

The behavior described above, when it comes to this ticket, might have a 
consequence that if buffers are not cleared on deleting all hints, then once it 
goes to check if we should hint, it will look into "what is the oldest hint" in 
order to decide if we violated the threshold or not, and there will be no 
descriptors, indeed, that is fine, but buffers will not be cleared so it will 
resolve the oldest hint in buffers. So by deletion of hints, it wont be 100% 
"reset".  On the other hand, when a node comes up online, all hints will be 
delivered and earliest hint record will be removed from the underlying map. 

edit: deletion / cleanup of all hints is done in HintsService.excise(uuid), it 
does close the writer hence flushes the buffers etc ... I think this should be 
applied to normal deletion too.


was (Author: smiklosovic):
BTW this ticket ... while it does what it should, if one starts to dig deeper, 
there are some not-so-obvious consequences of how hints delivery / deletion is 
done. For example, if one calls HintsServiceMBean.deleteAllHints (or 
deleteAllHintsForEndpoint), it will indeed delete the hints, but only these 
which are already on disk and for which writer is closed. That means that it 
will not delete hints which are in buffers or for which the writer is not 
closed yet. I consider this to be a bug. If a user invokes "delete all hints", 
it should indeed delete them _ALL_, in buffers included.

The behavior described above, when it comes to this ticket, might have a 
consequence that if buffers are not cleared on deleting all hints, then once it 
goes to check if we should hint, it will look into "what is the oldest hint" in 
order to decide if we violated the threshold or not, and there will be no 
descriptors, indeed, that is fine, but buffers will not be cleared so it will 
resolve the oldest hint in buffers. So by deletion of hints, it wont be 100% 
"reset".  On the other hand, when a node comes up online, all hints will be 
delivered and earliest hint record will be removed from the underlying map. 

> Hints not stored after node goes down for the second time
> ---------------------------------------------------------
>
>                 Key: CASSANDRA-19495
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-19495
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Consistency/Hints
>            Reporter: Paul Chandler
>            Assignee: Stefan Miklosovic
>            Priority: Urgent
>             Fix For: 4.1.x, 5.0-rc, 5.x
>
>          Time Spent: 50m
>  Remaining Estimate: 0h
>
> I have a scenario where a node goes down, hints are recorded on the second 
> node and replayed, as expected. If the first node goes down for a second time 
> and time span between the first time it stopped and the second time it 
> stopped is more than the max_hint_window then the hint is not recorded, no 
> hint file is created, and the mutation never arrives at the node after it 
> comes up again.
> I have debugged this and it appears to due to the way hint window is 
> persisted after https://issues.apache.org/jira/browse/CASSANDRA-14309
> The code here: 
> [https://github.com/apache/cassandra/blame/cassandra-4.1/src/java/org/apache/cassandra/service/StorageProxy.java#L2402]
>  uses the time stored in the HintsBuffer.earliestHintByHost map.  This map is 
> based on the UUID of the host, but this does not seem to be cleared when the 
> node is back up, and I think this is what is causing the problem.
>  
> This is in cassandra 4.1.5



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org

Reply via email to