[ https://issues.apache.org/jira/browse/CASSANDRA-19495?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17831309#comment-17831309 ]
Paul Chandler commented on CASSANDRA-19495: ------------------------------------------- I think I get it now, this code {code:java} try (HintsWriter.Session session = writer.newSession(writeBuffer)) { while (iterator.hasNext()) { session.append(iterator.next()); if (session.position() >= maxHintsFileSize) break; } } {code} will iterate over all the ByteBuffers, unless the file is full, if that happens it breaks and falls back out to this part: {code:java} while (true) { if (iterator.hasNext()) flushInternal(iterator, store); if (!iterator.hasNext()) break; {code} This time the iterator still has some bytebuffers, so it does not break, but continues, closes the writer and clears the earliest hint. If the file is not full, all the byteBuffers are processed in the first code block and the second code block will break out of the loop. > 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: Brandon Williams > Priority: Normal > Fix For: 4.1.x, 5.0.x, 5.x > > > 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