6161import org .apache .cassandra .index .sai .utils .SAICodecUtils ;
6262import org .apache .cassandra .io .sstable .SSTableReadsListener ;
6363import org .apache .cassandra .io .sstable .format .SSTableReader ;
64+ import org .apache .cassandra .io .util .FileHandle ;
6465import org .apache .cassandra .io .util .FileUtils ;
6566import org .apache .cassandra .utils .AbstractIterator ;
6667import org .apache .cassandra .utils .CloseableIterator ;
@@ -80,7 +81,8 @@ public class InvertedIndexSearcher extends IndexSearcher
8081 private final Version version ;
8182 private final boolean filterRangeResults ;
8283 private final SSTableReader sstable ;
83- private final DocLengthsReader docLengthsReader ;
84+ private final SegmentMetadata .ComponentMetadata docLengthsMeta ;
85+ private final FileHandle docLengths ;
8486 private final long segmentRowIdOffset ;
8587
8688 protected InvertedIndexSearcher (SSTableContext sstableContext ,
@@ -99,9 +101,9 @@ protected InvertedIndexSearcher(SSTableContext sstableContext,
99101 this .version = version ;
100102 this .filterRangeResults = filterRangeResults ;
101103 perColumnEventListener = (QueryEventListener .TrieIndexEventListener )indexContext .getColumnQueryMetrics ();
102- var docLengthsMeta = segmentMetadata .componentMetadatas .getOptional (IndexComponentType .DOC_LENGTHS );
103104 this .segmentRowIdOffset = segmentMetadata .segmentRowIdOffset ;
104- this .docLengthsReader = docLengthsMeta == null ? null : new DocLengthsReader (indexFiles .docLengths (), docLengthsMeta );
105+ this .docLengthsMeta = segmentMetadata .componentMetadatas .getOptional (IndexComponentType .DOC_LENGTHS );
106+ this .docLengths = docLengthsMeta == null ? null : indexFiles .docLengths ();
105107
106108 Map <String ,String > map = metadata .componentMetadatas .get (IndexComponentType .TERMS_DATA ).attributes ;
107109 String footerPointerString = map .get (SAICodecUtils .FOOTER_POINTER );
@@ -176,7 +178,7 @@ public CloseableIterator<PrimaryKeyWithSortKey> orderBy(Orderer orderer, Express
176178 var iter = new RowIdWithTermsIterator (reader .allTerms (orderer .isAscending ()));
177179 return toMetaSortedIterator (iter , queryContext );
178180 }
179- if (docLengthsReader == null )
181+ if (docLengthsMeta == null )
180182 throw new InvalidRequestException (indexContext .getIndexName () + " does not support BM25 scoring until it is rebuilt" );
181183
182184 // find documents that match each term
@@ -194,11 +196,12 @@ public CloseableIterator<PrimaryKeyWithSortKey> orderBy(Orderer orderer, Express
194196
195197 var pkm = primaryKeyMapFactory .newPerSSTablePrimaryKeyMap ();
196198 var merged = IntersectingPostingList .intersect (postingLists );
197-
199+ var docLengthsReader = new DocLengthsReader (docLengths , docLengthsMeta );
200+
198201 // Wrap the iterator with resource management
199202 var it = new AbstractIterator <DocTF >() { // Anonymous class extends AbstractIterator
200203 private boolean closed ;
201-
204+
202205 @ Override
203206 protected DocTF computeNext ()
204207 {
@@ -222,7 +225,7 @@ public void close()
222225 {
223226 if (closed ) return ;
224227 closed = true ;
225- FileUtils .closeQuietly (pkm , merged );
228+ FileUtils .closeQuietly (pkm , merged , docLengthsReader );
226229 }
227230 };
228231 return bm25Internal (it , queryTerms , documentFrequencies );
@@ -250,7 +253,7 @@ public CloseableIterator<PrimaryKeyWithSortKey> orderResultsBy(SSTableReader rea
250253 {
251254 if (!orderer .isBM25 ())
252255 return super .orderResultsBy (reader , queryContext , keys , orderer , limit );
253- if (docLengthsReader == null )
256+ if (docLengthsMeta == null )
254257 throw new InvalidRequestException (indexContext .getIndexName () + " does not support BM25 scoring until it is rebuilt" );
255258
256259 var queryTerms = orderer .getQueryTerms ();
@@ -279,7 +282,7 @@ public String toString()
279282 @ Override
280283 public void close ()
281284 {
282- FileUtils .closeQuietly (reader , docLengthsReader );
285+ FileUtils .closeQuietly (reader , docLengths );
283286 }
284287
285288 /**
0 commit comments