Skip to content

Commit 629b025

Browse files
committed
Merge branch 'cassandra-5.0' into trunk
2 parents a39f3b0 + 29746cf commit 629b025

File tree

5 files changed

+254
-11
lines changed

5 files changed

+254
-11
lines changed

CHANGES.txt

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -253,6 +253,7 @@ Merged from 4.1:
253253
* Enforce CQL message size limit on multiframe messages (CASSANDRA-20052)
254254
* Fix race condition in DecayingEstimatedHistogramReservoir during rescale (CASSANDRA-19365)
255255
Merged from 4.0:
256+
* Handle sstable metadata stats file getting a new mtime after compaction has finished (CASSANDRA-18119)
256257
* Honor MAX_PARALLEL_TRANSFERS correctly (CASSANDRA-20532)
257258
* Updating a column with a new TTL but same expiration time is non-deterministic and causes repair mismatches. (CASSANDRA-20561)
258259
* Avoid computing prepared statement size for unprepared batches (CASSANDRA-20556)

src/java/org/apache/cassandra/db/lifecycle/LogFile.java

Lines changed: 18 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -179,6 +179,10 @@ static boolean isLogFile(File file)
179179
this.id = id;
180180
}
181181

182+
/**
183+
* Check a variety of the internals of the LogRecord as well as the state of the LogRecord vs. the files found on disk
184+
* to ensure they remain correct and nothing was changed external to the process.
185+
*/
182186
boolean verify()
183187
{
184188
records.clear();
@@ -245,6 +249,9 @@ LogRecord setErrorInReplicas(LogRecord record)
245249
return record;
246250
}
247251

252+
/**
253+
* Sets the {@link LogRecord.Status#error} if something wrong is found with the record.
254+
*/
248255
static void verifyRecord(LogRecord record, List<File> existingFiles)
249256
{
250257
if (record.checksum != record.computeChecksum())
@@ -256,6 +263,7 @@ static void verifyRecord(LogRecord record, List<File> existingFiles)
256263
return;
257264
}
258265

266+
// If it's not a removal we don't check it since we're not going to take action on it
259267
if (record.type != Type.REMOVE)
260268
return;
261269

@@ -269,14 +277,23 @@ static void verifyRecord(LogRecord record, List<File> existingFiles)
269277
// we can have transaction files with mismatching updateTime resolutions due to switching between jdk8 and jdk11, truncate both to be consistent:
270278
if (truncateMillis(record.updateTime) != truncateMillis(record.status.onDiskRecord.updateTime) && record.status.onDiskRecord.updateTime > 0)
271279
{
280+
// handle the case where we have existing broken transaction file on disk, where the update time is
281+
// based on the stats file. This is just for the first upgrade, patched versions never base the update
282+
// time on the stats file.
283+
LogRecord statsIncluded = LogRecord.make(record.type, existingFiles, existingFiles.size(), record.absolutePath(), true);
284+
if (truncateMillis(statsIncluded.updateTime) == truncateMillis(record.updateTime))
285+
{
286+
logger.warn("Found a legacy log record {} with updateTime based on the stats file, ignoring to allow startup to continue", record);
287+
return;
288+
}
289+
272290
record.setError(String.format("Unexpected files detected for sstable [%s]: " +
273291
"last update time [%tc] (%d) should have been [%tc] (%d)",
274292
record.fileName(),
275293
record.status.onDiskRecord.updateTime,
276294
record.status.onDiskRecord.updateTime,
277295
record.updateTime,
278296
record.updateTime));
279-
280297
}
281298
}
282299

src/java/org/apache/cassandra/db/lifecycle/LogRecord.java

Lines changed: 65 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,6 @@
2020
*/
2121
package org.apache.cassandra.db.lifecycle;
2222

23-
2423
import java.nio.file.Path;
2524
import java.util.ArrayList;
2625
import java.util.Arrays;
@@ -38,14 +37,21 @@
3837
import java.util.stream.Collectors;
3938
import java.util.zip.CRC32;
4039

40+
import com.google.common.annotations.VisibleForTesting;
41+
import org.slf4j.Logger;
42+
import org.slf4j.LoggerFactory;
43+
4144
import org.apache.cassandra.io.sstable.Component;
45+
import org.apache.cassandra.io.sstable.Descriptor;
4246
import org.apache.cassandra.io.sstable.SSTable;
47+
import org.apache.cassandra.io.sstable.format.SSTableFormat;
4348
import org.apache.cassandra.io.sstable.format.SSTableReader;
4449
import org.apache.cassandra.io.util.File;
4550
import org.apache.cassandra.io.util.FileUtils;
4651
import org.apache.cassandra.io.util.PathUtils;
4752
import org.apache.cassandra.utils.FBUtilities;
4853

54+
import static org.apache.cassandra.io.sstable.Descriptor.TMP_EXT;
4955
import static org.apache.cassandra.utils.LocalizeString.toUpperCaseLocalized;
5056

5157
/**
@@ -55,6 +61,10 @@
5561
*/
5662
final class LogRecord
5763
{
64+
private static final Logger logger = LoggerFactory.getLogger(LogRecord.class);
65+
@VisibleForTesting
66+
static boolean INCLUDE_STATS_FOR_TESTS = false;
67+
5868
public enum Type
5969
{
6070
UNKNOWN, // a record that cannot be parsed
@@ -78,7 +88,10 @@ public boolean matches(LogRecord record)
7888
return this == record.type;
7989
}
8090

81-
public boolean isFinal() { return this == Type.COMMIT || this == Type.ABORT; }
91+
public boolean isFinal()
92+
{
93+
return this == Type.COMMIT || this == Type.ABORT;
94+
}
8295
}
8396

8497
/**
@@ -194,17 +207,65 @@ private static String absolutePath(File baseFile)
194207

195208
public LogRecord withExistingFiles(List<File> existingFiles)
196209
{
210+
if (!absolutePath.isPresent())
211+
throw new IllegalStateException(String.format("Cannot create record from existing files for type %s - file is not present", type));
212+
197213
return make(type, existingFiles, 0, absolutePath.get());
198214
}
199215

216+
/**
217+
* We create a LogRecord based on the files on disk; there's some subtlety around how we handle stats files as the
218+
* timestamp can be mutated by the async completion of compaction if things race with node shutdown. To work around this,
219+
* we don't take the stats file timestamp into account when calculating nor using the timestamps for all the components
220+
* as we build the LogRecord.
221+
*/
200222
public static LogRecord make(Type type, List<File> files, int minFiles, String absolutePath)
201223
{
224+
return make(type, files, minFiles, absolutePath, INCLUDE_STATS_FOR_TESTS);
225+
}
226+
227+
/**
228+
* In most cases we skip including the stats file timestamp entirely as it can be mutated during anticompaction
229+
* and thus "invalidate" the LogRecord. There is an edge case where we have a LogRecord that was written w/the wrong
230+
* timestamp (i.e. included a mutated stats file) and we need the node to come up, so we need to expose the selective
231+
* ability to either include the stats file timestamp or not.
232+
*
233+
* See {@link LogFile#verifyRecord}
234+
*/
235+
static LogRecord make(Type type, List<File> files, int minFiles, String absolutePath, boolean includeStatsFile)
236+
{
237+
List<File> toVerify;
238+
File statsFile = null;
239+
if (!includeStatsFile && !files.isEmpty())
240+
{
241+
toVerify = new ArrayList<>(files.size() - 1);
242+
for (File f : files)
243+
{
244+
if (!f.name().endsWith(TMP_EXT))
245+
{
246+
if (Descriptor.componentFromFile(f) == SSTableFormat.Components.STATS)
247+
statsFile = f;
248+
else
249+
toVerify.add(f);
250+
}
251+
}
252+
}
253+
else
254+
{
255+
toVerify = files;
256+
}
202257
// CASSANDRA-11889: File.lastModified() returns a positive value only if the file exists, therefore
203258
// we filter by positive values to only consider the files that still exists right now, in case things
204259
// changed on disk since getExistingFiles() was called
205-
List<Long> positiveModifiedTimes = files.stream().map(File::lastModified).filter(lm -> lm > 0).collect(Collectors.toList());
260+
List<Long> positiveModifiedTimes = toVerify.stream().map(File::lastModified).filter(lm -> lm > 0).collect(Collectors.toList());
206261
long lastModified = positiveModifiedTimes.stream().reduce(0L, Long::max);
207-
return new LogRecord(type, absolutePath, lastModified, Math.max(minFiles, positiveModifiedTimes.size()));
262+
263+
// We need to preserve the file count for the number of existing files found on disk even though we ignored the
264+
// stats file during our timestamp calculation. If the stats file still exists, we add in the count of it as
265+
// a separate validation assumption that it's one of the files considered valid in this LogRecord.
266+
boolean addStatTS = statsFile != null && statsFile.exists();
267+
int positiveTSCount = addStatTS ? positiveModifiedTimes.size() + 1 : positiveModifiedTimes.size();
268+
return new LogRecord(type, absolutePath, lastModified, Math.max(minFiles, positiveTSCount));
208269
}
209270

210271
private LogRecord(Type type, long updateTime)

src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -549,6 +549,8 @@ static boolean removeUnfinishedLeftovers(Map.Entry<String, List<File>> entry)
549549
try(LogFile txn = LogFile.make(entry.getKey(), entry.getValue()))
550550
{
551551
logger.info("Verifying logfile transaction {}", txn);
552+
// We don't check / include the stats file timestamp on LogRecord creation / verification as that might
553+
// be modified by a race in compaction notification and then needlessly fail subsequent node starts.
552554
if (txn.verify())
553555
{
554556
Throwable failure = txn.removeUnfinishedLeftovers(null);

0 commit comments

Comments
 (0)