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

Dmitry Konstantinov edited comment on CASSANDRA-20074 at 11/15/24 11:24 AM:
----------------------------------------------------------------------------

yes, I took the same source as we use in new SyncRunnable(preciseTime) to align 
them.
By default: MonotonicClock.Global.preciseTime.now() -> 
org.apache.cassandra.utils.Clock.Default#nanoTime -> System.nanoTime();
org.apache.cassandra.utils.Clock.Global#nanoTime -> 
org.apache.cassandra.utils.Clock.Default#nanoTime -> System.nanoTime();
both ways uses the same source of time. So, to have less questions we can fix 
the code one more time to have only one method but from a correctness point of 
view I think it does not matter..


was (Author: dnk):
yes, I took the same source as we use in new SyncRunnable(preciseTime) to align 
them.
By default: MonotonicClock.Global.preciseTime.now() -> 
org.apache.cassandra.utils.Clock.Default#nanoTime -> System.nanoTime();
org.apache.cassandra.utils.Clock.Global#nanoTime -> 
org.apache.cassandra.utils.Clock.Default#nanoTime -> System.nanoTime();
both ways uses the same source of time.

> AbstractCommitLogService#lastSyncedAt initialized with currentTimeMillis() 
> but later compared and updated with System.nanoTime() 
> ---------------------------------------------------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-20074
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-20074
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Local/Commit Log
>            Reporter: Dmitry Konstantinov
>            Assignee: Dmitry Konstantinov
>            Priority: Low
>             Fix For: 4.0.15, 4.1.8, 5.0.3, 5.1
>
>          Time Spent: 0.5h
>  Remaining Estimate: 0h
>
> Within org.apache.cassandra.db.commitlog.AbstractCommitLogService
> we use System.currentMillis()-based time to initialize lastSyncedAt field:
> {code:java}
> protected volatile long lastSyncedAt = currentTimeMillis();{code}
> but later we use clock.now() = System.nanoTime() to compare and update the 
> field value:
> {code:java}
> long pollStarted = clock.now();
> boolean flushToDisk = lastSyncedAt + syncIntervalNanos <= pollStarted || 
> state != NORMAL || syncRequested; {code}
> As of now we are lucky that in the current JDK implementations 
> System.nanoTime() is much bigger than currentTimeMillis() but it is not 
> guaranteed by a specification of the API. If  nanoTime() < 
> currentTimeMillis() then we can stuck without sync of commit log to disk.
> So, we need to align the time sources and change the init logic to:
> {code:java}
> protected volatile long lastSyncedAt = preciseTime.now();{code}



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

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to