You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by "sashapolo (via GitHub)" <gi...@apache.org> on 2023/04/27 08:52:16 UTC

[GitHub] [ignite-3] sashapolo commented on a diff in pull request #1693: IGNITE-18839 HybridTimestamp and its usages refactored to use 8-bytes representation

sashapolo commented on code in PR #1693:
URL: https://github.com/apache/ignite-3/pull/1693#discussion_r1178796930


##########
modules/core/src/test/java/org/apache/ignite/internal/util/PendingComparableValuesTrackerTest.java:
##########
@@ -97,8 +97,7 @@ public void testMultithreadedWaitFor() throws Exception {
 
                 tracker.update(now);
 
-                HybridTimestamp timestampToWait =
-                        new HybridTimestamp(now.getPhysical() + 1, now.getLogical() + random.nextInt(1000));
+                HybridTimestamp timestampToWait = now.addTicks((1 << HybridTimestamp.LOGICAL_TIME_BITS_SIZE) + random.nextInt(1000));

Review Comment:
   What is this change for? The previous version was much more readable



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/snapshot/incoming/IncomingSnapshotCopier.java:
##########
@@ -393,7 +393,7 @@ private String createPartitionInfo() {
     private void writeVersion(ResponseEntry entry, int i) {
         RowId rowId = new RowId(partId(), entry.rowId());
 
-        HybridTimestamp timestamp = i < entry.timestamps().size() ? entry.timestamps().get(i) : null;
+        Long timestamp = i < entry.timestamps().length ? entry.timestamps()[i] : null;

Review Comment:
   Can we move `i < entry.timestamps().length` check to the `if` below? What's the point of creating a `Long` here?



##########
modules/transactions/src/main/java/org/apache/ignite/internal/tx/TransactionIds.java:
##########
@@ -32,7 +34,7 @@ public class TransactionIds {
      * @return Transaction ID corresponding to the provided values.
      */
     public static UUID transactionId(HybridTimestamp beginTimestamp, int nodeId) {
-        return new UUID(beginTimestamp.getPhysical(), (((long) beginTimestamp.getLogical()) << Integer.SIZE) | (nodeId & 0xFFFFFFFFL));
+        return new UUID(beginTimestamp.longValue(), (nodeId & 0xFFFFFFFFL));

Review Comment:
   Let's use `Integer.toUnsignedLong` here



##########
modules/transactions/src/main/java/org/apache/ignite/internal/tx/TransactionIds.java:
##########
@@ -42,6 +44,6 @@ public static UUID transactionId(HybridTimestamp beginTimestamp, int nodeId) {
      * @return Begin timestamp of the transaction.
      */
     public static HybridTimestamp beginTimestamp(UUID transactionId) {
-        return new HybridTimestamp(transactionId.getMostSignificantBits(), (int) (transactionId.getLeastSignificantBits() >> Integer.SIZE));
+        return requireNonNull(HybridTimestamp.of(transactionId.getMostSignificantBits()));

Review Comment:
   Can `HybridTimestamp.of` return null? 



##########
modules/core/src/main/java/org/apache/ignite/internal/hlc/HybridTimestamp.java:
##########
@@ -55,13 +66,35 @@ public final class HybridTimestamp implements Comparable<HybridTimestamp>, Seria
      * @param logical The logical time.
      */
     public HybridTimestamp(long physical, int logical) {
-        assert physical > 0 : physical;
-        // Value -1 is used in "org.apache.ignite.internal.hlc.HybridClock.update" to produce "0" after the increment.
-        // Real usable value cannot be negative.
-        assert logical >= -1 : logical;
+        if (physical < 0 || physical >= (1L << PHYSICAL_TIME_BITS_SIZE)) {
+            throw new IllegalArgumentException("physical time is out of bounds: " + physical);
+        }
 
-        this.physical = physical;
-        this.logical = logical;
+        if (logical < 0 || logical >= (1L << LOGICAL_TIME_BITS_SIZE)) {
+            throw new IllegalArgumentException("logical time is out of bounds: " + logical);
+        }
+
+        time = (physical << LOGICAL_TIME_BITS_SIZE) | logical;
+
+        if (time <= 0) {
+            throw new IllegalArgumentException("time is out of bounds: " + time);
+        }
+    }
+
+    private HybridTimestamp(long time) {
+        if (time <= 0) {
+            throw new IllegalArgumentException("time is out of bounds: " + time);
+        }
+
+        this.time = time;
+    }
+
+    public static @Nullable HybridTimestamp of(long time) {

Review Comment:
   Looks like we need to add a bunch of `@Nullable` annotations to methods that use this method's result as their result



##########
modules/core/src/main/java/org/apache/ignite/internal/hlc/HybridTimestamp.java:
##########
@@ -190,6 +220,6 @@ public String toString() {
      * Returns a new hybrid timestamp with incremented physical component.
      */
     public HybridTimestamp addPhysicalTime(long mills) {
-        return new HybridTimestamp(physical + mills, logical);
+        return new HybridTimestamp(addExact(getPhysical(), mills), getLogical());

Review Comment:
   Wouldn't it be more effective to shift `mills` and add it to the whole long value?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org