You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ma...@apache.org on 2020/04/03 10:55:23 UTC

[cassandra] branch trunk updated (6ae6596 -> 49abedc)

This is an automated email from the ASF dual-hosted git repository.

marcuse pushed a change to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git.


    from 6ae6596  Fix tests expecting exceptions wrapped in RuntimeException.
     new dcf6fe8  Emit metric on preview repair failure
     new 6a302f7  Set now in seconds in the future for validation repairs
     new caa3bd8  Expose repair streaming metrics
     new 49abedc  Fix force compaction of wrapping ranges

The 4 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 CHANGES.txt                                        |  4 ++
 src/java/org/apache/cassandra/config/Config.java   |  7 +++
 .../cassandra/config/DatabaseDescriptor.java       |  7 +++
 .../cassandra/db/compaction/CompactionManager.java | 18 +++++++-
 .../{MetricNameFactory.java => RepairMetrics.java} | 20 +++++----
 .../apache/cassandra/metrics/StreamingMetrics.java |  2 +
 .../org/apache/cassandra/repair/RepairJob.java     | 19 ++++++--
 .../apache/cassandra/repair/RepairRunnable.java    |  2 +
 .../repair/consistent/SyncStatSummary.java         | 12 +++++
 .../cassandra/service/ActiveRepairService.java     |  2 +
 .../apache/cassandra/streaming/StreamSession.java  | 13 ++++++
 .../compaction/LeveledCompactionStrategyTest.java  | 52 ++++++++++++++++++++++
 12 files changed, 145 insertions(+), 13 deletions(-)
 copy src/java/org/apache/cassandra/metrics/{MetricNameFactory.java => RepairMetrics.java} (68%)


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


[cassandra] 03/04: Expose repair streaming metrics

Posted by ma...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

marcuse pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git

commit caa3bd83dcb92a3145b7ef0fd73bbd3708b255bf
Author: Sankalp Kohli <sa...@apple.com>
AuthorDate: Mon Mar 23 10:59:27 2020 +0100

    Expose repair streaming metrics
    
    Patch by Sankalp Kohli; reviewed by Ekaterina Dimitrova for CASSANDRA-15656
---
 CHANGES.txt                                                 |  1 +
 src/java/org/apache/cassandra/metrics/StreamingMetrics.java |  2 ++
 src/java/org/apache/cassandra/streaming/StreamSession.java  | 13 +++++++++++++
 3 files changed, 16 insertions(+)

diff --git a/CHANGES.txt b/CHANGES.txt
index 77d69ca..55243b8 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.0-alpha4
+ * Expose repair streaming metrics (CASSANDRA-15656)
  * Set now in seconds in the future for validation repairs (CASSANDRA-15655)
  * Emit metric on preview repair failure (CASSANDRA-15654)
  * Use more appropriate logging levels (CASSANDRA-15661)
diff --git a/src/java/org/apache/cassandra/metrics/StreamingMetrics.java b/src/java/org/apache/cassandra/metrics/StreamingMetrics.java
index 793a8c0..80a5e13 100644
--- a/src/java/org/apache/cassandra/metrics/StreamingMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/StreamingMetrics.java
@@ -39,6 +39,8 @@ public class StreamingMetrics
     public static final Counter activeStreamsOutbound = Metrics.counter(DefaultNameFactory.createMetricName(TYPE_NAME, "ActiveOutboundStreams", null));
     public static final Counter totalIncomingBytes = Metrics.counter(DefaultNameFactory.createMetricName(TYPE_NAME, "TotalIncomingBytes", null));
     public static final Counter totalOutgoingBytes = Metrics.counter(DefaultNameFactory.createMetricName(TYPE_NAME, "TotalOutgoingBytes", null));
+    public static final Counter totalOutgoingRepairBytes = Metrics.counter(DefaultNameFactory.createMetricName(TYPE_NAME, "TotalOutgoingRepairBytes", null));
+    public static final Counter totalOutgoingRepairSSTables = Metrics.counter(DefaultNameFactory.createMetricName(TYPE_NAME, "TotalOutgoingRepairSSTables", null));
     public final Counter incomingBytes;
     public final Counter outgoingBytes;
 
diff --git a/src/java/org/apache/cassandra/streaming/StreamSession.java b/src/java/org/apache/cassandra/streaming/StreamSession.java
index 95d3755..05bb5ff 100644
--- a/src/java/org/apache/cassandra/streaming/StreamSession.java
+++ b/src/java/org/apache/cassandra/streaming/StreamSession.java
@@ -510,8 +510,21 @@ public class StreamSession implements IEndpointStateChangeSubscriber
         state(State.PREPARING);
         PrepareSynMessage prepare = new PrepareSynMessage();
         prepare.requests.addAll(requests);
+        long totalBytesToStream = 0;
+        long totalSSTablesStreamed = 0;
         for (StreamTransferTask task : transfers.values())
+        {
+            totalBytesToStream += task.getTotalSize();
+            totalSSTablesStreamed += task.getTotalNumberOfFiles();
             prepare.summaries.add(task.getSummary());
+        }
+
+        if(StreamOperation.REPAIR == getStreamOperation())
+        {
+            StreamingMetrics.totalOutgoingRepairBytes.inc(totalBytesToStream);
+            StreamingMetrics.totalOutgoingRepairSSTables.inc(totalSSTablesStreamed);
+        }
+
         messageSender.sendMessage(prepare);
     }
 


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


[cassandra] 04/04: Fix force compaction of wrapping ranges

Posted by ma...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

marcuse pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git

commit 49abedc2c30c6274339bc203e0ddd10f128dae58
Author: Jeff Jirsa <jj...@apple.com>
AuthorDate: Mon Mar 23 10:16:26 2020 +0100

    Fix force compaction of wrapping ranges
    
    Patch by Jeff Jirsa; reviewed by Benjamin Lerer for CASSANDRA-15664
---
 CHANGES.txt                                        |  1 +
 .../cassandra/db/compaction/CompactionManager.java | 18 +++++++-
 .../compaction/LeveledCompactionStrategyTest.java  | 52 ++++++++++++++++++++++
 3 files changed, 69 insertions(+), 2 deletions(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index 55243b8..65111d0 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.0-alpha4
+ * Fix force compaction of wrapping ranges (CASSANDRA-15664)
  * Expose repair streaming metrics (CASSANDRA-15656)
  * Set now in seconds in the future for validation repairs (CASSANDRA-15655)
  * Emit metric on preview repair failure (CASSANDRA-15654)
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index 28db027..7924a1f 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -33,6 +33,7 @@ import com.google.common.base.Preconditions;
 import com.google.common.collect.*;
 import com.google.common.util.concurrent.*;
 
+import org.apache.cassandra.dht.AbstractBounds;
 import org.apache.cassandra.locator.RangesAtEndpoint;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -929,8 +930,21 @@ public class CompactionManager implements CompactionManagerMBean
 
         for (Range<Token> tokenRange : tokenRangeCollection)
         {
-            Iterable<SSTableReader> ssTableReaders = View.sstablesInBounds(tokenRange.left.minKeyBound(), tokenRange.right.maxKeyBound(), tree);
-            Iterables.addAll(sstables, ssTableReaders);
+            if (!AbstractBounds.strictlyWrapsAround(tokenRange.left, tokenRange.right))
+            {
+                Iterable<SSTableReader> ssTableReaders = View.sstablesInBounds(tokenRange.left.minKeyBound(), tokenRange.right.maxKeyBound(), tree);
+                Iterables.addAll(sstables, ssTableReaders);
+            }
+            else
+            {
+                // Searching an interval tree will not return the correct results for a wrapping range
+                // so we have to unwrap it first
+                for (Range<Token> unwrappedRange : tokenRange.unwrap())
+                {
+                    Iterable<SSTableReader> ssTableReaders = View.sstablesInBounds(unwrappedRange.left.minKeyBound(), unwrappedRange.right.maxKeyBound(), tree);
+                    Iterables.addAll(sstables, ssTableReaders);
+                }
+            }
         }
         return sstables;
     }
diff --git a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
index b925bab..8a8ed13 100644
--- a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
@@ -460,6 +460,58 @@ public class LeveledCompactionStrategyTest
 
         // the 11 tables containing key1 should all compact to 1 table
         assertEquals(1, cfs.getLiveSSTables().size());
+        // Set it up again
+        cfs.truncateBlocking();
+
+        // create 10 sstables that contain data for both key1 and key2
+        for (int i = 0; i < numIterations; i++)
+        {
+            for (DecoratedKey key : keys)
+            {
+                UpdateBuilder update = UpdateBuilder.create(cfs.metadata(), key);
+                for (int c = 0; c < columns; c++)
+                    update.newRow("column" + c).add("val", value);
+                update.applyUnsafe();
+            }
+            cfs.forceBlockingFlush();
+        }
+
+        // create 20 more sstables with 10 containing data for key1 and other 10 containing data for key2
+        for (int i = 0; i < numIterations; i++)
+        {
+            for (DecoratedKey key : keys)
+            {
+                UpdateBuilder update = UpdateBuilder.create(cfs.metadata(), key);
+                for (int c = 0; c < columns; c++)
+                    update.newRow("column" + c).add("val", value);
+                update.applyUnsafe();
+                cfs.forceBlockingFlush();
+            }
+        }
+
+        // We should have a total of 30 sstables again
+        assertEquals(30, cfs.getLiveSSTables().size());
+
+        // This time, we're going to make sure the token range wraps around, to cover the full range
+        Range<Token> wrappingRange;
+        if (key1.getToken().compareTo(key2.getToken()) < 0)
+        {
+            wrappingRange = new Range<>(key2.getToken(), key1.getToken());
+        }
+        else
+        {
+            wrappingRange = new Range<>(key1.getToken(), key2.getToken());
+        }
+        Collection<Range<Token>> wrappingRanges = new ArrayList<>(Arrays.asList(wrappingRange));
+        cfs.forceCompactionForTokenRange(wrappingRanges);
+
+        while(CompactionManager.instance.isCompacting(Arrays.asList(cfs), (sstable) -> true))
+        {
+            Thread.sleep(100);
+        }
+
+        // should all compact to 1 table
+        assertEquals(1, cfs.getLiveSSTables().size());
     }
 
     @Test


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


[cassandra] 01/04: Emit metric on preview repair failure

Posted by ma...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

marcuse pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git

commit dcf6fe8162f00353dae497a6adcfec2ac88f8e0a
Author: Blake Eggleston <bd...@gmail.com>
AuthorDate: Tue Mar 17 15:51:36 2020 +0100

    Emit metric on preview repair failure
    
    Patch by Blake Eggleston; reviewed by Ekaterina Dimitrova for CASSANDRA-15654
---
 CHANGES.txt                                        |  1 +
 .../apache/cassandra/metrics/RepairMetrics.java    | 34 ++++++++++++++++++++++
 .../apache/cassandra/repair/RepairRunnable.java    |  2 ++
 .../repair/consistent/SyncStatSummary.java         | 12 ++++++++
 .../cassandra/service/ActiveRepairService.java     |  2 ++
 5 files changed, 51 insertions(+)

diff --git a/CHANGES.txt b/CHANGES.txt
index b71d8da..9896272 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.0-alpha4
+ * Emit metric on preview repair failure (CASSANDRA-15654)
  * Use more appropriate logging levels (CASSANDRA-15661)
  * Added production recommendations and improved compaction doc organization
  * Document usage of EC2Snitch with intra-region VPC peering (CASSANDRA-15337)
diff --git a/src/java/org/apache/cassandra/metrics/RepairMetrics.java b/src/java/org/apache/cassandra/metrics/RepairMetrics.java
new file mode 100644
index 0000000..5b4f67e
--- /dev/null
+++ b/src/java/org/apache/cassandra/metrics/RepairMetrics.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.metrics;
+
+import com.codahale.metrics.Counter;
+
+import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
+
+public class RepairMetrics
+{
+    public static final String TYPE_NAME = "Repair";
+    public static final Counter previewFailures = Metrics.counter(DefaultNameFactory.createMetricName(TYPE_NAME, "PreviewFailures", null));
+
+    public static void init()
+    {
+        // noop
+    }
+}
diff --git a/src/java/org/apache/cassandra/repair/RepairRunnable.java b/src/java/org/apache/cassandra/repair/RepairRunnable.java
index c673a6c..0ac34a3 100644
--- a/src/java/org/apache/cassandra/repair/RepairRunnable.java
+++ b/src/java/org/apache/cassandra/repair/RepairRunnable.java
@@ -53,6 +53,7 @@ import com.codahale.metrics.Timer;
 import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.metrics.RepairMetrics;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
@@ -527,6 +528,7 @@ public class RepairRunnable implements Runnable, ProgressEventNotifier
                     else
                     {
                         message = (previewKind == PreviewKind.REPAIRED ? "Repaired data is inconsistent\n" : "Preview complete\n") + summary.toString();
+                        RepairMetrics.previewFailures.inc();
                     }
                     notification(message);
 
diff --git a/src/java/org/apache/cassandra/repair/consistent/SyncStatSummary.java b/src/java/org/apache/cassandra/repair/consistent/SyncStatSummary.java
index 156fde7..f8e1bfb 100644
--- a/src/java/org/apache/cassandra/repair/consistent/SyncStatSummary.java
+++ b/src/java/org/apache/cassandra/repair/consistent/SyncStatSummary.java
@@ -30,6 +30,8 @@ import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.repair.RepairResult;
 import org.apache.cassandra.repair.RepairSessionResult;
 import org.apache.cassandra.repair.SyncStat;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.streaming.SessionSummary;
 import org.apache.cassandra.streaming.StreamSummary;
 import org.apache.cassandra.utils.FBUtilities;
@@ -130,6 +132,12 @@ public class SyncStatSummary
             totalsCalculated = true;
         }
 
+        boolean isCounter()
+        {
+            TableMetadata tmd = Schema.instance.getTableMetadata(keyspace, table);
+            return tmd != null && tmd.isCounter();
+        }
+
         public String toString()
         {
             if (!totalsCalculated)
@@ -192,6 +200,10 @@ public class SyncStatSummary
         summaries.values().forEach(Table::calculateTotals);
         for (Table table: summaries.values())
         {
+            if (table.isCounter())
+            {
+                continue;
+            }
             table.calculateTotals();
             files += table.files;
             bytes += table.bytes;
diff --git a/src/java/org/apache/cassandra/service/ActiveRepairService.java b/src/java/org/apache/cassandra/service/ActiveRepairService.java
index 467d2bc..c679987 100644
--- a/src/java/org/apache/cassandra/service/ActiveRepairService.java
+++ b/src/java/org/apache/cassandra/service/ActiveRepairService.java
@@ -58,6 +58,7 @@ import org.apache.cassandra.gms.IFailureDetectionEventListener;
 import org.apache.cassandra.gms.VersionedValue;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.TokenMetadata;
+import org.apache.cassandra.metrics.RepairMetrics;
 import org.apache.cassandra.net.RequestCallback;
 import org.apache.cassandra.net.Message;
 import org.apache.cassandra.net.MessagingService;
@@ -142,6 +143,7 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
                                                                  new NamedThreadFactory("Repair-Task"),
                                                                  "internal",
                                                                  new ThreadPoolExecutor.AbortPolicy());
+        RepairMetrics.init();
     }
 
     private final IFailureDetector failureDetector;


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


[cassandra] 02/04: Set now in seconds in the future for validation repairs

Posted by ma...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

marcuse pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git

commit 6a302f734a1dede4b4bdee5e5f67c5ada6f0115d
Author: Blake Eggleston <bd...@gmail.com>
AuthorDate: Tue Mar 17 16:47:50 2020 +0100

    Set now in seconds in the future for validation repairs
    
    Patch by Blake Eggleston; reviewed by Ekaterina Dimitrova for CASSANDRA-15655
---
 CHANGES.txt                                           |  1 +
 src/java/org/apache/cassandra/config/Config.java      |  7 +++++++
 .../apache/cassandra/config/DatabaseDescriptor.java   |  7 +++++++
 src/java/org/apache/cassandra/repair/RepairJob.java   | 19 ++++++++++++++++---
 4 files changed, 31 insertions(+), 3 deletions(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index 9896272..77d69ca 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.0-alpha4
+ * Set now in seconds in the future for validation repairs (CASSANDRA-15655)
  * Emit metric on preview repair failure (CASSANDRA-15654)
  * Use more appropriate logging levels (CASSANDRA-15661)
  * Added production recommendations and improved compaction doc organization
diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java
index 5a24410..3fc314f 100644
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@ -471,6 +471,13 @@ public class Config
     public volatile boolean report_unconfirmed_repaired_data_mismatches = false;
 
     /**
+     * number of seconds to set nowInSec into the future when performing validation previews against repaired data
+     * this (attempts) to prevent a race where validations on different machines are started on different sides of
+     * a tombstone being compacted away
+     */
+    public volatile int validation_preview_purge_head_start_in_sec = 60 * 60;
+
+    /**
      * @deprecated migrate to {@link DatabaseDescriptor#isClientInitialized()}
      */
     @Deprecated
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 14db023..7af310e 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -3061,4 +3061,11 @@ public class DatabaseDescriptor
             throw new ConfigurationException(String.format("%s must be positive value < %d, but was %d",
                                                            name, unit.overflowThreshold(), val), false);
     }
+
+    public static int getValidationPreviewPurgeHeadStartInSec()
+    {
+        int seconds = conf.validation_preview_purge_head_start_in_sec;
+        return Math.max(seconds, 0);
+    }
+
 }
diff --git a/src/java/org/apache/cassandra/repair/RepairJob.java b/src/java/org/apache/cassandra/repair/RepairJob.java
index 3740070..e609f0d 100644
--- a/src/java/org/apache/cassandra/repair/RepairJob.java
+++ b/src/java/org/apache/cassandra/repair/RepairJob.java
@@ -71,6 +71,19 @@ public class RepairJob extends AbstractFuture<RepairResult> implements Runnable
         this.parallelismDegree = session.parallelismDegree;
     }
 
+    public int getNowInSeconds()
+    {
+        int nowInSeconds = FBUtilities.nowInSeconds();
+        if (session.previewKind == PreviewKind.REPAIRED)
+        {
+            return nowInSeconds + DatabaseDescriptor.getValidationPreviewPurgeHeadStartInSec();
+        }
+        else
+        {
+            return nowInSeconds;
+        }
+    }
+
     /**
      * Runs repair job.
      *
@@ -345,7 +358,7 @@ public class RepairJob extends AbstractFuture<RepairResult> implements Runnable
         String message = String.format("Requesting merkle trees for %s (to %s)", desc.columnFamily, endpoints);
         logger.info("{} {}", session.previewKind.logPrefix(desc.sessionId), message);
         Tracing.traceRepair(message);
-        int nowInSec = FBUtilities.nowInSeconds();
+        int nowInSec = getNowInSeconds();
         List<ListenableFuture<TreeResponse>> tasks = new ArrayList<>(endpoints.size());
         for (InetAddressAndPort endpoint : endpoints)
         {
@@ -365,7 +378,7 @@ public class RepairJob extends AbstractFuture<RepairResult> implements Runnable
         String message = String.format("Requesting merkle trees for %s (to %s)", desc.columnFamily, endpoints);
         logger.info("{} {}", session.previewKind.logPrefix(desc.sessionId), message);
         Tracing.traceRepair(message);
-        int nowInSec = FBUtilities.nowInSeconds();
+        int nowInSec = getNowInSeconds();
         List<ListenableFuture<TreeResponse>> tasks = new ArrayList<>(endpoints.size());
 
         Queue<InetAddressAndPort> requests = new LinkedList<>(endpoints);
@@ -407,7 +420,7 @@ public class RepairJob extends AbstractFuture<RepairResult> implements Runnable
         String message = String.format("Requesting merkle trees for %s (to %s)", desc.columnFamily, endpoints);
         logger.info("{} {}", session.previewKind.logPrefix(desc.sessionId), message);
         Tracing.traceRepair(message);
-        int nowInSec = FBUtilities.nowInSeconds();
+        int nowInSec = getNowInSeconds();
         List<ListenableFuture<TreeResponse>> tasks = new ArrayList<>(endpoints.size());
 
         Map<String, Queue<InetAddressAndPort>> requestsByDatacenter = new HashMap<>();


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