You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by "JeetKunDoug (via GitHub)" <gi...@apache.org> on 2023/01/23 18:54:30 UTC

[GitHub] [cassandra] JeetKunDoug opened a new pull request, #2109: CASSANDRA-18187 - Add unit tests for per-row TTL and Timestamp usage in CQLSSTableWriter

JeetKunDoug opened a new pull request, #2109:
URL: https://github.com/apache/cassandra/pull/2109

   CQLSSTableWriter supports per-row setting of both timestamp and TTL values, but itโ€™s not tested or documented today. Add tests to cover setting both TTL and Timestamp values for rows using the CQLSSTableWriter.
   
   
   


-- 
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: pr-unsubscribe@cassandra.apache.org

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


[GitHub] [cassandra] smiklosovic closed pull request #2109: CASSANDRA-18187 - Add unit tests for per-row TTL and Timestamp usage in CQLSSTableWriter

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic closed pull request #2109: CASSANDRA-18187 - Add unit tests for per-row TTL and Timestamp usage in CQLSSTableWriter
URL: https://github.com/apache/cassandra/pull/2109


-- 
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: pr-unsubscribe@cassandra.apache.org

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


[GitHub] [cassandra] JeetKunDoug commented on a diff in pull request #2109: CASSANDRA-18187 - Add unit tests for per-row TTL and Timestamp usage in CQLSSTableWriter

Posted by "JeetKunDoug (via GitHub)" <gi...@apache.org>.
JeetKunDoug commented on code in PR #2109:
URL: https://github.com/apache/cassandra/pull/2109#discussion_r1094615527


##########
src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java:
##########
@@ -247,7 +246,7 @@ public CQLSSTableWriter rawAddRow(List<ByteBuffer> values)
         ClientState state = ClientState.forInternalCalls();
         List<ByteBuffer> keys = modificationStatement.buildPartitionKeyNames(options, state);
 
-        long now = currentTimeMillis();
+        long now = Clock.Global.currentTimeMillis();

Review Comment:
   ๐Ÿ‘



-- 
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: pr-unsubscribe@cassandra.apache.org

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


[GitHub] [cassandra] dcapwell commented on a diff in pull request #2109: CASSANDRA-18187 - Add unit tests for per-row TTL and Timestamp usage in CQLSSTableWriter

Posted by "dcapwell (via GitHub)" <gi...@apache.org>.
dcapwell commented on code in PR #2109:
URL: https://github.com/apache/cassandra/pull/2109#discussion_r1093867752


##########
test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java:
##########
@@ -1116,6 +1118,122 @@ public void testFrozenSetTypeCustomOrdered() throws Exception
         assertEquals(0, filtered.size());
     }
 
+    @Test
+    public void testWriteWithTimestamps() throws Exception
+    {
+        long now = currentTimeMillis();
+        long then = now - 1000;
+        final String schema = "CREATE TABLE " + qualifiedTable + " ("
+                              + "  k int,"
+                              + "  c1 int,"
+                              + "  c2 int,"
+                              + "  v text,"
+                              + "  PRIMARY KEY (k)"

Review Comment:
   this is confusing, `c1` and `c2` normally means "clustering column" when you read the code, but this does not include it; can you rename these to v1, v2, and v3?  or change the test to make these clustering columns?



##########
test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java:
##########
@@ -1116,6 +1118,122 @@ public void testFrozenSetTypeCustomOrdered() throws Exception
         assertEquals(0, filtered.size());
     }
 
+    @Test
+    public void testWriteWithTimestamps() throws Exception
+    {
+        long now = currentTimeMillis();
+        long then = now - 1000;
+        final String schema = "CREATE TABLE " + qualifiedTable + " ("
+                              + "  k int,"
+                              + "  c1 int,"
+                              + "  c2 int,"
+                              + "  v text,"
+                              + "  PRIMARY KEY (k)"
+                              + ")";
+
+        CQLSSTableWriter writer = CQLSSTableWriter.builder()
+                                                  .inDirectory(dataDir)
+                                                  .forTable(schema)
+                                                  .using("INSERT INTO " + qualifiedTable +
+                                                         " (k, c1, c2, v) VALUES (?,?,?,?) using timestamp ?" )
+                                                  .build();
+
+        // Note that, all other things being equal, Cassandra will sort these rows lexicographically, so we use "higher" values in the
+        // row we expect to "win" so that we're sure that it isn't just accidentally picked due to the row sorting.
+        writer.addRow( 1, 4, 5, "b", now); // This write should be the one found at the end because it has a higher timestamp
+        writer.addRow( 1, 2, 3, "a", then);
+        writer.close();
+        loadSSTables(dataDir, keyspace);
+
+        UntypedResultSet resultSet = QueryProcessor.executeInternal("SELECT * FROM " + qualifiedTable);
+        assertEquals(1, resultSet.size());
+
+        Iterator<UntypedResultSet.Row> iter = resultSet.iterator();
+        UntypedResultSet.Row r1 = iter.next();
+        assertEquals(1, r1.getInt("k"));
+        assertEquals(4, r1.getInt("c1"));
+        assertEquals(5, r1.getInt("c2"));
+        assertEquals("b", r1.getString("v"));
+        assertFalse(iter.hasNext());
+    }
+    @Test
+    public void testWriteWithTtl() throws Exception
+    {
+        final String schema = "CREATE TABLE " + qualifiedTable + " ("
+                              + "  k int,"
+                              + "  c1 int,"
+                              + "  c2 int,"
+                              + "  v text,"
+                              + "  PRIMARY KEY (k)"

Review Comment:
   this is confusing, `c1` and `c2` normally means "clustering column" when you read the code, but this does not include it; can you rename these to v1, v2, and v3?  or change the test to make these clustering columns?



##########
test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java:
##########
@@ -1116,6 +1118,122 @@ public void testFrozenSetTypeCustomOrdered() throws Exception
         assertEquals(0, filtered.size());
     }
 
+    @Test
+    public void testWriteWithTimestamps() throws Exception
+    {
+        long now = currentTimeMillis();
+        long then = now - 1000;
+        final String schema = "CREATE TABLE " + qualifiedTable + " ("
+                              + "  k int,"
+                              + "  c1 int,"
+                              + "  c2 int,"
+                              + "  v text,"
+                              + "  PRIMARY KEY (k)"
+                              + ")";
+
+        CQLSSTableWriter writer = CQLSSTableWriter.builder()
+                                                  .inDirectory(dataDir)
+                                                  .forTable(schema)
+                                                  .using("INSERT INTO " + qualifiedTable +
+                                                         " (k, c1, c2, v) VALUES (?,?,?,?) using timestamp ?" )
+                                                  .build();
+
+        // Note that, all other things being equal, Cassandra will sort these rows lexicographically, so we use "higher" values in the
+        // row we expect to "win" so that we're sure that it isn't just accidentally picked due to the row sorting.
+        writer.addRow( 1, 4, 5, "b", now); // This write should be the one found at the end because it has a higher timestamp
+        writer.addRow( 1, 2, 3, "a", then);
+        writer.close();
+        loadSSTables(dataDir, keyspace);
+
+        UntypedResultSet resultSet = QueryProcessor.executeInternal("SELECT * FROM " + qualifiedTable);
+        assertEquals(1, resultSet.size());
+
+        Iterator<UntypedResultSet.Row> iter = resultSet.iterator();
+        UntypedResultSet.Row r1 = iter.next();
+        assertEquals(1, r1.getInt("k"));
+        assertEquals(4, r1.getInt("c1"));
+        assertEquals(5, r1.getInt("c2"));
+        assertEquals("b", r1.getString("v"));
+        assertFalse(iter.hasNext());
+    }
+    @Test
+    public void testWriteWithTtl() throws Exception
+    {
+        final String schema = "CREATE TABLE " + qualifiedTable + " ("
+                              + "  k int,"
+                              + "  c1 int,"
+                              + "  c2 int,"
+                              + "  v text,"
+                              + "  PRIMARY KEY (k)"
+                              + ")";
+
+        CQLSSTableWriter.Builder builder = CQLSSTableWriter.builder()
+                                                         .inDirectory(dataDir)
+                                                         .forTable(schema)
+                                                         .using("INSERT INTO " + qualifiedTable +
+                                                                " (k, c1, c2, v) VALUES (?,?,?,?) using TTL ?");
+        CQLSSTableWriter writer = builder.build();
+        // add a row that _should_ show up - 1 hour TTL
+        writer.addRow( 1, 2, 3, "a", 3600);
+        // Insert a row with a TTL of 1 second - should not appear in results once we sleep
+        writer.addRow( 2, 4, 5, "b", 1);
+        writer.close();
+        Thread.sleep(1200); // Slightly over 1 second, just to make sure
+        loadSSTables(dataDir, keyspace);
+
+        UntypedResultSet resultSet = QueryProcessor.executeInternal("SELECT * FROM " + qualifiedTable);
+        assertEquals(1, resultSet.size());
+
+        Iterator<UntypedResultSet.Row> iter = resultSet.iterator();
+        UntypedResultSet.Row r1 = iter.next();
+        assertEquals(1, r1.getInt("k"));
+        assertEquals(2, r1.getInt("c1"));
+        assertEquals(3, r1.getInt("c2"));
+        assertEquals("a", r1.getString("v"));
+        assertFalse(iter.hasNext());
+    }
+    @Test
+    public void testWriteWithTimestampsAndTtl() throws Exception
+    {
+        final String schema = "CREATE TABLE " + qualifiedTable + " ("
+                              + "  k int,"
+                              + "  c1 int,"
+                              + "  c2 int,"
+                              + "  v text,"
+                              + "  PRIMARY KEY (k)"

Review Comment:
   this is confusing, `c1` and `c2` normally means "clustering column" when you read the code, but this does not include it; can you rename these to v1, v2, and v3?  or change the test to make these clustering columns?



-- 
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: pr-unsubscribe@cassandra.apache.org

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


[GitHub] [cassandra] dcapwell commented on a diff in pull request #2109: CASSANDRA-18187 - Add unit tests for per-row TTL and Timestamp usage in CQLSSTableWriter

Posted by "dcapwell (via GitHub)" <gi...@apache.org>.
dcapwell commented on code in PR #2109:
URL: https://github.com/apache/cassandra/pull/2109#discussion_r1093739692


##########
src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java:
##########
@@ -247,7 +246,7 @@ public CQLSSTableWriter rawAddRow(List<ByteBuffer> values)
         ClientState state = ClientState.forInternalCalls();
         List<ByteBuffer> keys = modificationStatement.buildPartitionKeyNames(options, state);
 
-        long now = currentTimeMillis();
+        long now = Clock.Global.currentTimeMillis();

Review Comment:
   please revert this change, the common style is to static import `Global`, so should remove this style change



-- 
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: pr-unsubscribe@cassandra.apache.org

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