You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by GitBox <gi...@apache.org> on 2020/10/23 17:02:35 UTC

[GitHub] [cassandra] dcapwell commented on a change in pull request #786: CASSANDRA-16103: Invalid serialized size for responses

dcapwell commented on a change in pull request #786:
URL: https://github.com/apache/cassandra/pull/786#discussion_r511011914



##########
File path: src/java/org/apache/cassandra/net/Message.java
##########
@@ -367,8 +368,13 @@ private Header(long id, Verb verb, InetAddressAndPort from, long createdAtNanos,
             this.id = id;
             this.verb = verb;
             this.from = from;
-            this.createdAtNanos = createdAtNanos;
             this.expiresAtNanos = expiresAtNanos;
+            if (verb.isResponse())
+                // Correct createdAtNanos to enforce the contraint, createdAtNanos <= expiresAtNanos
+                this.createdAtNanos = Math.min(createdAtNanos, expiresAtNanos);
+            else
+                this.createdAtNanos = createdAtNanos;
+            Preconditions.checkArgument(this.createdAtNanos <= this.expiresAtNanos, "createdAtNanos cannot be more recent than expiresAtNanos");

Review comment:
       if this fails, would be good to also print the time.  To keep the string create lazy, may be best to do
   
   ```
   if (this.createdAtNanos > this.expiresAtNanos)
     throw new IllegalArgumentException("createdAtNanos ("+this.createdAtNanos+") cannot be more recent than expiresAtNanos ("+this.expiresAtNanos+")");
   ```

##########
File path: src/java/org/apache/cassandra/net/Message.java
##########
@@ -367,8 +368,13 @@ private Header(long id, Verb verb, InetAddressAndPort from, long createdAtNanos,
             this.id = id;
             this.verb = verb;
             this.from = from;
-            this.createdAtNanos = createdAtNanos;
             this.expiresAtNanos = expiresAtNanos;
+            if (verb.isResponse())
+                // Correct createdAtNanos to enforce the contraint, createdAtNanos <= expiresAtNanos
+                this.createdAtNanos = Math.min(createdAtNanos, expiresAtNanos);

Review comment:
       not sure how I feel about this... its no longer created at if we do this...  will need to think about this more.

##########
File path: test/unit/org/apache/cassandra/net/MessageTest.java
##########
@@ -39,6 +39,8 @@
 import org.apache.cassandra.tracing.Tracing.TraceType;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.UUIDGen;
+import org.assertj.core.api.Assertions;
+import org.openjdk.jmh.annotations.TearDown;

Review comment:
       can you remove this import?  not used.

##########
File path: test/unit/org/apache/cassandra/net/FramingTest.java
##########
@@ -213,6 +217,37 @@ public void burnRandomLegacy()
         burnRandomLegacy(1000);
     }
 
+    @Test
+    public void testSerializeSizeMatchesEdgeCases() // See CASSANDRA-16103
+    {
+        int v40 = MessagingService.Version.VERSION_40.value;
+        Consumer<Long> subTest = timeGap ->
+        {
+            long createdAt = 0;
+            long expiresAt = createdAt + timeGap;
+            Message<NoPayload> message = Message.builder(Verb.READ_REPAIR_RSP, NoPayload.noPayload)
+                                                .from(FBUtilities.getBroadcastAddressAndPort())
+                                                .withCreatedAt(createdAt)
+                                                .withExpiresAt(TimeUnit.MILLISECONDS.toNanos(expiresAt))

Review comment:
       withCreatedAt takes nanos, so `createdAt` has to be nanos, so timeGap must also be in nanos... which means `expiresAt` is also nanos.... so shouldn't have this conversion?

##########
File path: test/unit/org/apache/cassandra/utils/CassandraGenerators.java
##########
@@ -105,8 +114,26 @@
                                                                                        .<Message<? extends ReadCommand>>map(c -> Message.builder(Verb.READ_REQ, c).build())
                                                                                        .describedAs(CassandraGenerators::toStringRecursive);
 
+    private static Gen<Message<NoPayload>> responseGen(Verb verb)
+    {
+        return gen(rnd -> {
+            long timeSpan = SMALL_TIME_SPAN_GEN.generate(rnd);

Review comment:
       I kinda feel its more clear if we do the following (assuming SMALL_TIME_SPAN_GEN doesn't return negative values, see other comment)
   
   ```
   long realCreatedAt = TIMESTAMP_NANOS.generate(rnd);
   long networkDelay = SMALL_TIME_SPAN_GEN.generate(rnd);
   long timeSpan = SMALL_TIME_SPAN_GEN.generate(rnd);
   
   long createdAtNanos = realCreatedAt + networkDelay;
   long expiresAtNanos = realCreatedAt + timeSpan;
   ```
   
   my thinking is, we keep track of the request create timestamp, and account for network delay.  This will get us into cases where createdAtNanos is > expiresAtNanos.

##########
File path: test/unit/org/apache/cassandra/utils/Generators.java
##########
@@ -177,21 +178,28 @@
     // all time is boxed in the future around 50 years from today: Aug 20th, 2020 UTC
     public static final Gen<Timestamp> TIMESTAMP_GEN;
     public static final Gen<Date> DATE_GEN;
+    public static final Gen<Long> TIMESTAMP_NANOS;
+    public static final Gen<Long> SMALL_TIME_SPAN_GEN;
 
     static
     {
+        long secondInNanos = 1_000_000_000L;
         ZonedDateTime now = ZonedDateTime.of(2020, 8, 20,
                                              0, 0, 0, 0, ZoneOffset.UTC);
         ZonedDateTime startOfTime = now.minusYears(50);
         ZonedDateTime endOfDays = now.plusYears(50);
         Constraint millisConstraint = Constraint.between(startOfTime.toInstant().toEpochMilli(), endOfDays.toInstant().toEpochMilli());
-        Constraint nanosInSecondConstraint = Constraint.between(0, 999999999);
+        Constraint nanosInSecondConstraint = Constraint.between(0, secondInNanos - 1);
+        // Represents the timespan based on the most of the default request timeouts. See DatabaseDescriptor
+        Constraint smallTimeSpanNanosConstraint = Constraint.between(-1 * secondInNanos, 10 * secondInNanos);

Review comment:
       I feel weird having negative time here, though in your test case it makes sense; should we move this generator there?




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org