Skip to content

Commit 5e94c3c

Browse files
authored
CNDB-15526: Load TOC component before performing component discovery (#2028)
### What is the issue TOCComponent performs component discovery before loading the TOC. This can be detrimental to custom component loading and can be optimized to try loading the TOC directly first before resorting to discovery. ### What does this PR fix and why was it fixed Load the TOC component without discovery if possible Also don't check for compression file existence when not necessary since it will open the file
1 parent dd5da49 commit 5e94c3c

File tree

2 files changed

+24
-13
lines changed

2 files changed

+24
-13
lines changed

src/java/org/apache/cassandra/io/sstable/format/TOCComponent.java

Lines changed: 19 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -114,18 +114,29 @@ public static Set<Component> loadOrCreate(Descriptor descriptor)
114114
{
115115
try
116116
{
117-
SSTableWatcher.instance.discoverComponents(descriptor);
118-
return TOCComponent.loadTOC(descriptor);
117+
// Try loading TOC first without discovering components or checking file existence.
118+
return TOCComponent.loadTOC(descriptor, false);
119119
}
120120
catch (FileNotFoundException | NoSuchFileException e)
121121
{
122-
Set<Component> components = descriptor.discoverComponents();
123-
if (components.isEmpty())
124-
return components; // sstable doesn't exist yet
122+
SSTableWatcher.instance.discoverComponents(descriptor);
123+
124+
// Try loading TOC again after discovering components, still without existence checks
125+
try
126+
{
127+
return TOCComponent.loadTOC(descriptor, false);
128+
}
129+
catch (FileNotFoundException | NoSuchFileException e2)
130+
{
131+
// Still no TOC, create it from discovered components
132+
Set<Component> components = descriptor.discoverComponents();
133+
if (components.isEmpty())
134+
return components; // sstable doesn't exist yet
125135

126-
components.add(Components.TOC);
127-
TOCComponent.appendTOC(descriptor, components);
128-
return components;
136+
components.add(Components.TOC);
137+
TOCComponent.appendTOC(descriptor, components);
138+
return components;
139+
}
129140
}
130141
}
131142
catch (IOException e)

src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -348,8 +348,6 @@ private ICompressor getEncryptor(Descriptor desc, boolean writeTime)
348348
}
349349

350350
File compressionFile = desc.fileFor(Components.COMPRESSION_INFO);
351-
if (!compressionFile.exists())
352-
return null;
353351

354352
try
355353
{
@@ -364,10 +362,12 @@ private ICompressor getEncryptor(Descriptor desc, boolean writeTime)
364362
return compressor.encryptionOnly();
365363
return null;
366364
}
367-
catch (Exception e)
365+
catch (Throwable t)
368366
{
369-
// If we can't read the compression metadata, assume no encryption
370-
logger.debug("Could not read compression metadata for {}: {}", desc, e.getMessage());
367+
// If we can't read the compression metadata, assume no encryption.
368+
// During flush, the compression file may not be accessible yet in some implementations
369+
// causing FSReadError. Catch Throwable to handle both Exception and Error.
370+
logger.debug("Could not read compression metadata for {}: {}", desc, t.getMessage());
371371
return null;
372372
}
373373
}

0 commit comments

Comments
 (0)