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 2021/04/21 20:49:14 UTC

[GitHub] [cassandra] jacek-lewandowski opened a new pull request #978: CASSANDRA-16619: Add originating host ID to sstable metadata (trunk)

jacek-lewandowski opened a new pull request #978:
URL: https://github.com/apache/cassandra/pull/978


   


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


[GitHub] [cassandra] jacek-lewandowski commented on a change in pull request #978: CASSANDRA-16619: Add originating host ID to sstable metadata (trunk)

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on a change in pull request #978:
URL: https://github.com/apache/cassandra/pull/978#discussion_r626725625



##########
File path: test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
##########
@@ -612,7 +612,7 @@ private static void loadLegacyTable(String tablePattern, String legacyVersion) t
      * during development. I.e. remove the {@code @Ignore} annotation temporarily.
      * </p>
      */
-    @Ignore
+//    @Ignore

Review comment:
       ah yes, indeed




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


[GitHub] [cassandra] blerer commented on pull request #978: CASSANDRA-16619: Add originating host ID to sstable metadata (trunk)

Posted by GitBox <gi...@apache.org>.
blerer commented on pull request #978:
URL: https://github.com/apache/cassandra/pull/978#issuecomment-851957840


   Merged manually


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


[GitHub] [cassandra] blerer closed pull request #978: CASSANDRA-16619: Add originating host ID to sstable metadata (trunk)

Posted by GitBox <gi...@apache.org>.
blerer closed pull request #978:
URL: https://github.com/apache/cassandra/pull/978


   


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


[GitHub] [cassandra] jacek-lewandowski commented on a change in pull request #978: CASSANDRA-16619: Add originating host ID to sstable metadata (trunk)

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on a change in pull request #978:
URL: https://github.com/apache/cassandra/pull/978#discussion_r627299107



##########
File path: src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
##########
@@ -285,11 +286,25 @@ public void runMayThrow()
      * A set of known safe-to-discard commit log replay positions, based on
      * the range covered by on disk sstables and those prior to the most recent truncation record
      */
-    public static IntervalSet<CommitLogPosition> persistedIntervals(Iterable<SSTableReader> onDisk, CommitLogPosition truncatedAt)
+    public static IntervalSet<CommitLogPosition> persistedIntervals(Iterable<SSTableReader> onDisk,
+                                                                    CommitLogPosition truncatedAt,
+                                                                    UUID localhostId)
     {
         IntervalSet.Builder<CommitLogPosition> builder = new IntervalSet.Builder<>();
+        List<String> skippedSSTables = new ArrayList<>();
         for (SSTableReader reader : onDisk)
-            builder.addAll(reader.getSSTableMetadata().commitLogIntervals);
+        {
+            UUID originatingHostId = reader.getSSTableMetadata().originatingHostId;
+            if (originatingHostId != null && originatingHostId.equals(localhostId))
+                builder.addAll(reader.getSSTableMetadata().commitLogIntervals);
+            else
+                skippedSSTables.add(reader.getFilename());
+        }
+
+        if (!skippedSSTables.isEmpty()) {
+            logger.warn("Origin of {} sstables is unknown or doesn't match the local node; commitLogIntervals for them were ignored", skippedSSTables.size());

Review comment:
       hmm... I think I can add more information




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


[GitHub] [cassandra] jakubzytka commented on a change in pull request #978: CASSANDRA-16619: Add originating host ID to sstable metadata (trunk)

Posted by GitBox <gi...@apache.org>.
jakubzytka commented on a change in pull request #978:
URL: https://github.com/apache/cassandra/pull/978#discussion_r626677035



##########
File path: src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
##########
@@ -151,6 +154,7 @@ public SSTableReader open(SSTableReaderBuilder builder)
             hasCommitLogLowerBound = version.compareTo("mb") >= 0;
             hasCommitLogIntervals = version.compareTo("mc") >= 0;
             hasAccurateMinMax = version.compareTo("md") >= 0;
+            hasOriginatingHostId = version.matches("(m[e-z])|(n[b-z])");

Review comment:
       `o*` versions are good too ;)
   Noone's gonna remember to change this if `o` is ever introduced/
   I think we need to make some comparisons with `version.compareTo("nb") >= 0` being the last




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


[GitHub] [cassandra] jacek-lewandowski commented on a change in pull request #978: CASSANDRA-16619: Add originating host ID to sstable metadata (trunk)

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on a change in pull request #978:
URL: https://github.com/apache/cassandra/pull/978#discussion_r626725988



##########
File path: test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
##########
@@ -122,53 +124,46 @@ public File serialize(Map<MetadataType, MetadataComponent> metadata, MetadataSer
 
         TableMetadata cfm = SchemaLoader.standardCFMD("ks1", "cf1").build();
         MetadataCollector collector = new MetadataCollector(cfm.comparator)
-                                          .commitLogIntervals(new IntervalSet<>(cllb, club));
+                                      .commitLogIntervals(new IntervalSet<>(cllb, club));
 
         String partitioner = RandomPartitioner.class.getCanonicalName();
         double bfFpChance = 0.1;
         return collector.finalizeMetadata(partitioner, bfFpChance, 0, null, false, SerializationHeader.make(cfm, Collections.emptyList()));
     }
 
-    @Test
-    public void testMaReadMa() throws IOException
-    {
-        testOldReadsNew("ma", "ma");
-    }
-
-    @Test
-    public void testMaReadMb() throws IOException
-    {
-        testOldReadsNew("ma", "mb");
-    }
-
-    @Test
-    public void testMaReadMc() throws IOException
+    private void testVersions(String... versions) throws Throwable

Review comment:
       ok




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


[GitHub] [cassandra] jakubzytka commented on a change in pull request #978: CASSANDRA-16619: Add originating host ID to sstable metadata (trunk)

Posted by GitBox <gi...@apache.org>.
jakubzytka commented on a change in pull request #978:
URL: https://github.com/apache/cassandra/pull/978#discussion_r626671879



##########
File path: src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
##########
@@ -285,11 +286,25 @@ public void runMayThrow()
      * A set of known safe-to-discard commit log replay positions, based on
      * the range covered by on disk sstables and those prior to the most recent truncation record
      */
-    public static IntervalSet<CommitLogPosition> persistedIntervals(Iterable<SSTableReader> onDisk, CommitLogPosition truncatedAt)
+    public static IntervalSet<CommitLogPosition> persistedIntervals(Iterable<SSTableReader> onDisk,
+                                                                    CommitLogPosition truncatedAt,
+                                                                    UUID localhostId)
     {
         IntervalSet.Builder<CommitLogPosition> builder = new IntervalSet.Builder<>();
+        List<String> skippedSSTables = new ArrayList<>();
         for (SSTableReader reader : onDisk)
-            builder.addAll(reader.getSSTableMetadata().commitLogIntervals);
+        {
+            UUID originatingHostId = reader.getSSTableMetadata().originatingHostId;
+            if (originatingHostId != null && originatingHostId.equals(localhostId))
+                builder.addAll(reader.getSSTableMetadata().commitLogIntervals);
+            else
+                skippedSSTables.add(reader.getFilename());
+        }
+
+        if (!skippedSSTables.isEmpty()) {
+            logger.warn("Origin of {} sstables is unknown or doesn't match the local node; commitLogIntervals for them were ignored", skippedSSTables.size());

Review comment:
       We can get a big number here after upgrade. Perhaps it makes sense to add some info about that - that this is nothing to worry about?




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


[GitHub] [cassandra] jacek-lewandowski commented on a change in pull request #978: CASSANDRA-16619: Add originating host ID to sstable metadata (trunk)

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on a change in pull request #978:
URL: https://github.com/apache/cassandra/pull/978#discussion_r627298450



##########
File path: src/java/org/apache/cassandra/db/SystemKeyspace.java
##########
@@ -1101,28 +1101,40 @@ public static void setIndexRemoved(String keyspaceName, String indexName)
     }
 
     /**
-     * Read the host ID from the system keyspace, creating (and storing) one if
-     * none exists.
+     * Read the host ID from the system keyspace.
      */
     public static UUID getLocalHostId()
     {
         String req = "SELECT host_id FROM system.%s WHERE key='%s'";
         UntypedResultSet result = executeInternal(format(req, LOCAL, LOCAL));
 
         // Look up the Host UUID (return it if found)
-        if (!result.isEmpty() && result.one().has("host_id"))
+        if (result != null && !result.isEmpty() && result.one().has("host_id"))

Review comment:
       just to make static analysis happy




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


[GitHub] [cassandra] jakubzytka commented on a change in pull request #978: CASSANDRA-16619: Add originating host ID to sstable metadata (trunk)

Posted by GitBox <gi...@apache.org>.
jakubzytka commented on a change in pull request #978:
URL: https://github.com/apache/cassandra/pull/978#discussion_r626705748



##########
File path: test/distributed/org/apache/cassandra/distributed/test/metrics/StreamingMetricsTest.java
##########
@@ -140,15 +144,20 @@ public void testMetricsWithRepairAndStreamingToTwoNodes() throws Exception
         testMetricsWithStreamingToTwoNodes(true);
     }
 
+    private int getNumberOfSSTables(Cluster cluster, int node) {

Review comment:
       could you please elaborate what do these changes fix?




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


[GitHub] [cassandra] jakubzytka commented on a change in pull request #978: CASSANDRA-16619: Add originating host ID to sstable metadata (trunk)

Posted by GitBox <gi...@apache.org>.
jakubzytka commented on a change in pull request #978:
URL: https://github.com/apache/cassandra/pull/978#discussion_r626643671



##########
File path: src/java/org/apache/cassandra/db/SystemKeyspace.java
##########
@@ -1101,28 +1101,40 @@ public static void setIndexRemoved(String keyspaceName, String indexName)
     }
 
     /**
-     * Read the host ID from the system keyspace, creating (and storing) one if
-     * none exists.
+     * Read the host ID from the system keyspace.
      */
     public static UUID getLocalHostId()
     {
         String req = "SELECT host_id FROM system.%s WHERE key='%s'";
         UntypedResultSet result = executeInternal(format(req, LOCAL, LOCAL));
 
         // Look up the Host UUID (return it if found)
-        if (!result.isEmpty() && result.one().has("host_id"))
+        if (result != null && !result.isEmpty() && result.one().has("host_id"))

Review comment:
       why is `result != null` necessary now?




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


[GitHub] [cassandra] jacek-lewandowski commented on a change in pull request #978: CASSANDRA-16619: Add originating host ID to sstable metadata (trunk)

Posted by GitBox <gi...@apache.org>.
jacek-lewandowski commented on a change in pull request #978:
URL: https://github.com/apache/cassandra/pull/978#discussion_r627301541



##########
File path: test/distributed/org/apache/cassandra/distributed/test/metrics/StreamingMetricsTest.java
##########
@@ -140,15 +144,20 @@ public void testMetricsWithRepairAndStreamingToTwoNodes() throws Exception
         testMetricsWithStreamingToTwoNodes(true);
     }
 
+    private int getNumberOfSSTables(Cluster cluster, int node) {

Review comment:
       hehe... this is in-jvm distributed test. I assumed exact number of sstables in various stages of the test case. After adding that sentinel to commit log replayer, there is one extra sstable created because local host id is unknown. And it is unknown because the server is running in a kind of degraded mode (for speeding up tests, etc).




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


[GitHub] [cassandra] jakubzytka commented on a change in pull request #978: CASSANDRA-16619: Add originating host ID to sstable metadata (trunk)

Posted by GitBox <gi...@apache.org>.
jakubzytka commented on a change in pull request #978:
URL: https://github.com/apache/cassandra/pull/978#discussion_r626715015



##########
File path: test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
##########
@@ -122,53 +124,46 @@ public File serialize(Map<MetadataType, MetadataComponent> metadata, MetadataSer
 
         TableMetadata cfm = SchemaLoader.standardCFMD("ks1", "cf1").build();
         MetadataCollector collector = new MetadataCollector(cfm.comparator)
-                                          .commitLogIntervals(new IntervalSet<>(cllb, club));
+                                      .commitLogIntervals(new IntervalSet<>(cllb, club));
 
         String partitioner = RandomPartitioner.class.getCanonicalName();
         double bfFpChance = 0.1;
         return collector.finalizeMetadata(partitioner, bfFpChance, 0, null, false, SerializationHeader.make(cfm, Collections.emptyList()));
     }
 
-    @Test
-    public void testMaReadMa() throws IOException
-    {
-        testOldReadsNew("ma", "ma");
-    }
-
-    @Test
-    public void testMaReadMb() throws IOException
-    {
-        testOldReadsNew("ma", "mb");
-    }
-
-    @Test
-    public void testMaReadMc() throws IOException
+    private void testVersions(String... versions) throws Throwable

Review comment:
       I propose `testOldReadsNew` name (and similarly for `testOldReadsNewMVersions` and `testOldReadsNewNVersions`). This is the crux of the test, but with the current name it's hidden in the very details of the implementation.




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


[GitHub] [cassandra] jakubzytka commented on a change in pull request #978: CASSANDRA-16619: Add originating host ID to sstable metadata (trunk)

Posted by GitBox <gi...@apache.org>.
jakubzytka commented on a change in pull request #978:
URL: https://github.com/apache/cassandra/pull/978#discussion_r626707882



##########
File path: test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
##########
@@ -612,7 +612,7 @@ private static void loadLegacyTable(String tablePattern, String legacyVersion) t
      * during development. I.e. remove the {@code @Ignore} annotation temporarily.
      * </p>
      */
-    @Ignore
+//    @Ignore

Review comment:
       I believe we want this `@Ignore` back, once the serialization samples have been generated




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