20
20
*/
21
21
package org .apache .cassandra .db .lifecycle ;
22
22
23
-
24
23
import java .nio .file .Path ;
25
24
import java .util .ArrayList ;
26
25
import java .util .Arrays ;
38
37
import java .util .stream .Collectors ;
39
38
import java .util .zip .CRC32 ;
40
39
40
+ import com .google .common .annotations .VisibleForTesting ;
41
+ import org .slf4j .Logger ;
42
+ import org .slf4j .LoggerFactory ;
43
+
41
44
import org .apache .cassandra .io .sstable .Component ;
45
+ import org .apache .cassandra .io .sstable .Descriptor ;
42
46
import org .apache .cassandra .io .sstable .SSTable ;
47
+ import org .apache .cassandra .io .sstable .format .SSTableFormat ;
43
48
import org .apache .cassandra .io .sstable .format .SSTableReader ;
44
49
import org .apache .cassandra .io .util .File ;
45
50
import org .apache .cassandra .io .util .FileUtils ;
46
51
import org .apache .cassandra .io .util .PathUtils ;
47
52
import org .apache .cassandra .utils .FBUtilities ;
48
53
54
+ import static org .apache .cassandra .io .sstable .Descriptor .TMP_EXT ;
49
55
import static org .apache .cassandra .utils .LocalizeString .toUpperCaseLocalized ;
50
56
51
57
/**
55
61
*/
56
62
final class LogRecord
57
63
{
64
+ private static final Logger logger = LoggerFactory .getLogger (LogRecord .class );
65
+ @ VisibleForTesting
66
+ static boolean INCLUDE_STATS_FOR_TESTS = false ;
67
+
58
68
public enum Type
59
69
{
60
70
UNKNOWN , // a record that cannot be parsed
@@ -78,7 +88,10 @@ public boolean matches(LogRecord record)
78
88
return this == record .type ;
79
89
}
80
90
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
+ }
82
95
}
83
96
84
97
/**
@@ -194,17 +207,65 @@ private static String absolutePath(File baseFile)
194
207
195
208
public LogRecord withExistingFiles (List <File > existingFiles )
196
209
{
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
+
197
213
return make (type , existingFiles , 0 , absolutePath .get ());
198
214
}
199
215
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
+ */
200
222
public static LogRecord make (Type type , List <File > files , int minFiles , String absolutePath )
201
223
{
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
+ }
202
257
// CASSANDRA-11889: File.lastModified() returns a positive value only if the file exists, therefore
203
258
// we filter by positive values to only consider the files that still exists right now, in case things
204
259
// 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 ());
206
261
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 ));
208
269
}
209
270
210
271
private LogRecord (Type type , long updateTime )
0 commit comments