Uploaded image for project: 'Ignite'
  1. Ignite
  2. IGNITE-16893

Implement HLC and clock syncrhonization logic

Attach filesAttach ScreenshotAdd voteVotersWatch issueWatchersCreate sub-taskLinkCloneUpdate Comment AuthorReplace String in CommentUpdate Comment VisibilityDelete Comments
    XMLWordPrintableJSON

Details

    • Improvement
    • Status: Open
    • Major
    • Resolution: Unresolved
    • None
    • None
    • None

    Description

      HLC is similar to LC, but has a physical meaning and accurately represents physical time within a bound error.

      The rules for updating HLC:

      Initially l.j = 0; c.j = 0 
      
      Send or local event 
      l’.j = l.j; 
      l.j = max(l’.j, pt.j); 
      If (l.j = l’.j) then c.j = c.j + 1
      Else c.j = 0;
      Timestamp with l.j, c.j 
      

      Receive event of message m

      l’.j = l.j; 
      l.j = max(l’.j, l.m, pt.j); 
      If (l.j = l’.j =l.m) then c.j = max(c.j, c.m)+1 
      Elseif (l.j =l’.j) then c.j = c.j + 1 
      Elseif (l.j =l.m) then c.j = c.m + 1 
      Else c.j = 0 
      Timestamp with l.j, c.j

      !Screenshot from 2022-04-22 16-14-27.png!The following statements hold true for HLC, represented by (l,c):

      1. For any two events e and f, e <- f => (l.e; c.e) < (l.f; c.f) // Lexicographic comparison
      2. For any event f, l.f >= pt.f
      3. l.f > pt.f => (∃g : g <- f && pt.g = l.f)
      4. For any event f, |l.f - pt.f| < ϶,
        where ϶ represents clock sync uncertainty. For NTP e ~= 2 * ntp_offset
      5. For any event f, c.f = k && k > 0 => (
        ∃g1; g2; … ; gk : (∀j: 1 <= j < k : gj <- gj+1) && (∀j : 1 <= j < k : l.(gj) = l.f) && (gk hb f)
        )
      6. For any event f, c.f <= |{g : g <- f && l.g = l.f}|
      7. For any event f, c.f <=  N * (϶ + 1), if a physical clock of a node is incremented by at least one between any two events on that node

      HLC may overflow if physical time is not catching up with logical time. In this case only a causal counter can be used to move the timestamp forward. The bounded time staleness requirement is aimed to fix this.

      HLC is not limited by NTP and can be used with other time sync protocols, like PTP.

      Consistent Cut

      HLC can also be used to take a consistent snapshot at a logical time t.

      The consistent cut should capture all causal relationships. It can be written down as follows:

      (e ∈ C) && (e’ <- e) => e’ ∈ C

      https://www.cs.cornell.edu/courses/cs5414/2010fa/publications/BM93.pdf

      In the picture are shown two cuts: C - is the consistent cut, C’ - is the inconsistent cut.

      HLC allows taking a consistent cut at logical time l=t, c=K

      A global time, corresponding to the cut, lies in [t-϶,t]

      HLC Update Rules

      We assume two major event sources for updating HLC for enlisted nodes:

      RAFT events

      RAFT events help to synchronize HLC between RAFT replicas. All RAFT communications are initiated by a leader, and only one leader can exist at a time. This enforces monotonic growth of HLC on raft group replicas. RequestVote and AppendEntries RPC calls are enriched with sender’s HLC. The HLC update rules are applied on receiving messages. RAFT lease intervals are bound to the HLC range.

      Transaction protocol events

      Another source of HLC sync is a transaction protocol. Each message involved in the execution of the transaction carries the sender’s HLC and updates receiver HLC according to rules.

       

      Tx.Phase1

      Attachments

        Issue Links

        Activity

          This comment will be Viewable by All Users Viewable by All Users
          Cancel

          People

            Sergey Uttsel Sergey Uttsel
            alapin Alexander Lapin

            Dates

              Created:
              Updated:

              Slack

                Issue deployment