Skip to content

Commit

Permalink
Address review comments
Browse files Browse the repository at this point in the history
  • Loading branch information
adelapena committed Oct 2, 2024
1 parent 63131d1 commit b28eb42
Show file tree
Hide file tree
Showing 9 changed files with 27 additions and 43 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -624,7 +624,7 @@ public FileHandle createIndexBuildTimeFileHandle()
@Override
public IndexInput openInput()
{
return IndexFileUtils.instance().openBlockingInput(createFileHandle());
return IndexFileUtils.instance.openBlockingInput(createFileHandle());
}

@Override
Expand Down Expand Up @@ -662,7 +662,7 @@ public IndexOutputWriter openOutput(boolean append) throws IOException
component,
file);

return IndexFileUtils.instance().openOutput(file, byteOrder(), append);
return IndexFileUtils.instance.openOutput(file, byteOrder(), append);
}

@Override
Expand Down
24 changes: 12 additions & 12 deletions src/java/org/apache/cassandra/index/sai/disk/v1/TermsReader.java
Original file line number Diff line number Diff line change
Expand Up @@ -92,7 +92,7 @@ public TermsReader(IndexContext indexContext,
termDictionaryRoot = root;
this.termDictionaryFileEncodingVersion = termsDataEncodingVersion;

try (final IndexInput indexInput = IndexFileUtils.instance().openInput(termDictionaryFile))
try (final IndexInput indexInput = IndexFileUtils.instance.openInput(termDictionaryFile))
{
// if the pointer is -1 then this is a previous version of the index
// use the old way to validate the footer
Expand All @@ -107,7 +107,7 @@ public TermsReader(IndexContext indexContext,
}
}

try (final IndexInput indexInput = IndexFileUtils.instance().openInput(postingsFile))
try (final IndexInput indexInput = IndexFileUtils.instance.openInput(postingsFile))
{
validate(indexInput);
}
Expand Down Expand Up @@ -168,8 +168,8 @@ public class TermQuery
TermQuery(ByteComparable term, QueryEventListener.TrieIndexEventListener listener, QueryContext context)
{
this.listener = listener;
postingsInput = IndexFileUtils.instance().openInput(postingsFile);
postingsSummaryInput = IndexFileUtils.instance().openInput(postingsFile);
postingsInput = IndexFileUtils.instance.openInput(postingsFile);
postingsSummaryInput = IndexFileUtils.instance.openInput(postingsFile);
this.term = term;
lookupStartTime = System.nanoTime();
this.context = context;
Expand Down Expand Up @@ -299,8 +299,8 @@ private PostingList readAndMergePostings(TrieTermsDictionaryReader reader) throw
ArrayList<PostingList> postingLists = new ArrayList<>();

// index inputs will be closed with the onClose method of the returned merged posting list
IndexInput postingsInput = IndexFileUtils.instance().openInput(postingsFile);
IndexInput postingsSummaryInput = IndexFileUtils.instance().openInput(postingsFile);
IndexInput postingsInput = IndexFileUtils.instance.openInput(postingsFile);
IndexInput postingsSummaryInput = IndexFileUtils.instance.openInput(postingsFile);

do
{
Expand Down Expand Up @@ -329,8 +329,8 @@ private PostingList readFilterAndMergePosting(TrieTermsDictionaryReader reader)
ArrayList<PostingList> postingLists = new ArrayList<>();

// index inputs will be closed with the onClose method of the returned merged posting list
IndexInput postingsInput = IndexFileUtils.instance().openInput(postingsFile);
IndexInput postingsSummaryInput = IndexFileUtils.instance().openInput(postingsFile);
IndexInput postingsInput = IndexFileUtils.instance.openInput(postingsFile);
IndexInput postingsSummaryInput = IndexFileUtils.instance.openInput(postingsFile);

do
{
Expand Down Expand Up @@ -379,8 +379,8 @@ private class TermsScanner implements TermsIterator
private TermsScanner(Version version, AbstractType<?> type)
{
this.termsDictionaryReader = new TrieTermsDictionaryReader(termDictionaryFile.instantiateRebufferer(), termDictionaryRoot, termDictionaryFileEncodingVersion);
this.postingsInput = IndexFileUtils.instance().openInput(postingsFile);
this.postingsSummaryInput = IndexFileUtils.instance().openInput(postingsFile);
this.postingsInput = IndexFileUtils.instance.openInput(postingsFile);
this.postingsSummaryInput = IndexFileUtils.instance.openInput(postingsFile);
// We decode based on the logic used to encode the min and max terms in the trie.
if (version.onOrAfter(Version.DB) && TypeUtil.isComposite(type))
{
Expand Down Expand Up @@ -451,8 +451,8 @@ private class ReverseTermsScanner implements TermsIterator
private ReverseTermsScanner()
{
this.iterator = new ReverseTrieTermsDictionaryReader(termDictionaryFile.instantiateRebufferer(), termDictionaryRoot);
this.postingsInput = IndexFileUtils.instance().openInput(postingsFile);
this.postingsSummaryInput = IndexFileUtils.instance().openInput(postingsFile);
this.postingsInput = IndexFileUtils.instance.openInput(postingsFile);
this.postingsSummaryInput = IndexFileUtils.instance.openInput(postingsFile);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -101,7 +101,7 @@ public BlockPackedReader(FileHandle file, NumericValuesMeta meta) throws IOExcep
@Override
public LongArray open()
{
var indexInput = IndexFileUtils.instance().openInput(file);
var indexInput = IndexFileUtils.instance.openInput(file);
return new AbstractBlockPackedReader(indexInput, blockBitsPerValue, blockShift, blockMask, 0, valueCount)
{
@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -86,7 +86,7 @@ public MonotonicBlockPackedReader(FileHandle file, NumericValuesMeta meta) throw
@SuppressWarnings("resource")
public LongArray open()
{
var indexInput = IndexFileUtils.instance().openInput(file);
var indexInput = IndexFileUtils.instance.openInput(file);
return new AbstractBlockPackedReader(indexInput, blockBitsPerValue, blockShift, blockMask, 0, valueCount)
{
@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -147,8 +147,8 @@ public IteratorState(DocMapper docMapper, boolean isAscending, BKDReader.Interse
scratch = new byte[packedBytesLength];

final long firstLeafFilePointer = getMinLeafBlockFP();
bkdInput = IndexFileUtils.instance().openInput(kdtreeFile);
bkdPostingsInput = IndexFileUtils.instance().openInput(postingsFile);
bkdInput = IndexFileUtils.instance.openInput(kdtreeFile);
bkdPostingsInput = IndexFileUtils.instance.openInput(postingsFile);
bkdInput.seek(firstLeafFilePointer);

NavigableMap<Long, Integer> leafNodeToLeafFP = getLeafOffsets();
Expand Down Expand Up @@ -420,9 +420,9 @@ public PostingList intersect(IntersectVisitor visitor, QueryEventListener.BKDInd
}

listener.onSegmentHit();
IndexInput bkdInput = IndexFileUtils.instance().openInput(indexFile);
IndexInput postingsInput = IndexFileUtils.instance().openInput(postingsFile);
IndexInput postingsSummaryInput = IndexFileUtils.instance().openInput(postingsFile);
IndexInput bkdInput = IndexFileUtils.instance.openInput(indexFile);
IndexInput postingsInput = IndexFileUtils.instance.openInput(postingsFile);
IndexInput postingsSummaryInput = IndexFileUtils.instance.openInput(postingsFile);
PackedIndexTree index = new PackedIndexTree();

Intersection completable =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.nio.channels.FileChannel;
import java.nio.file.StandardOpenOption;
import java.util.zip.CRC32;

import com.google.common.annotations.VisibleForTesting;
Expand All @@ -32,7 +31,6 @@
import io.github.jbellis.jvector.disk.BufferedRandomAccessWriter;
import net.nicoulaj.compilecommand.annotations.DontInline;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.index.sai.disk.io.IndexInput;
import org.apache.cassandra.index.sai.disk.io.IndexInputReader;
import org.apache.cassandra.index.sai.disk.io.IndexOutputWriter;
import org.apache.cassandra.io.compress.BufferType;
Expand All @@ -55,24 +53,10 @@ public class IndexFileUtils
.finishOnClose(true)
.build();

private static final IndexFileUtils instance = new IndexFileUtils(defaultWriterOption);
private static IndexFileUtils overrideInstance = null;
public static volatile IndexFileUtils instance = new IndexFileUtils(defaultWriterOption);

private final SequentialWriterOption writerOption;

public static synchronized void setOverrideInstance(IndexFileUtils overrideInstance)
{
IndexFileUtils.overrideInstance = overrideInstance;
}

public static IndexFileUtils instance()
{
if (overrideInstance == null)
return instance;
else
return overrideInstance;
}

@VisibleForTesting
protected IndexFileUtils(SequentialWriterOption writerOption)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -174,8 +174,8 @@ private long writePostings(int rows) throws IOException

protected final PostingsReader openPostingsReader() throws IOException
{
IndexInput input = IndexFileUtils.instance().openInput(postings);
IndexInput summaryInput = IndexFileUtils.instance().openInput(postings);
IndexInput input = IndexFileUtils.instance.openInput(postings);
IndexInput summaryInput = IndexFileUtils.instance.openInput(postings);

PostingsReader.BlocksSummary summary = new PostingsReader.BlocksSummary(summaryInput, summaryPosition);
return new PostingsReader(input, summary, QueryEventListener.PostingListEventListener.NO_OP);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@
public class TrackingIndexFileUtils extends IndexFileUtils
{
private final Map<TrackingIndexInput, String> openInputs = Collections.synchronizedMap(new HashMap<>());
private final Set<TrackingIndexInput> closedInputs = new HashSet<>();
private final Set<TrackingIndexInput> closedInputs = Collections.synchronizedSet(new HashSet<>());

public TrackingIndexFileUtils(SequentialWriterOption writerOption)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ public IndexDescriptor newIndexDescriptor(Descriptor descriptor, SequentialWrite
{
TrackingIndexFileUtils trackingIndexFileUtils = new TrackingIndexFileUtils(sequentialWriterOption);
trackedIndexFileUtils.add(trackingIndexFileUtils);
IndexFileUtils.setOverrideInstance(trackingIndexFileUtils);
IndexFileUtils.instance = trackingIndexFileUtils;
return IndexDescriptor.empty(descriptor);
}

Expand All @@ -58,6 +58,6 @@ protected void afterIfSuccessful()
protected void afterAlways(List<Throwable> errors)
{
trackedIndexFileUtils.clear();
IndexFileUtils.setOverrideInstance(null);
IndexFileUtils.instance = null;
}
}

1 comment on commit b28eb42

@cassci-bot
Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Build rejected: 7 NEW test failure(s) in 2 builds., Build 2: ran 17705 tests with 17 failures and 128 skipped.
Butler analysis done on ds-cassandra-pr-gate/CNDB-11092-main vs last 16 runs of ds-cassandra-build-nightly/main.
junit.framework.TestSuite.org.apache.cassandra.index.sai.disk.v2.sortedterms.SortedTermsTest-compression: test is constantly failing. No failures on upstream;
branch story: [FF] vs upstream: [++++++++++++++]; [NEW]
org.apache.cassandra.index.sai.cql.TinySegmentQueryWriteLifecycleTest.testWriteLifecycle[aa_BaseDataModel{primaryKey=p}]: test is constantly failing. No failures on upstream;
branch story: [FF] vs upstream: [++++++++++++++++++++++++++++++++++++++++++++++++++++++++]; [NEW]
org.apache.cassandra.index.sai.cql.QueryWriteLifecycleTest.testWriteLifecycle[ca_CompositePartitionKeyDataModel{primaryKey=p1, p2}]: test is constantly failing. No failures on upstream;
branch story: [F] vs upstream: [++++++++++++++++++++++++++++++++++++++++++++++++++++++++]; [NEW]
org.apache.cassandra.index.sai.disk.vector.VectorCompressionTest.testOpenAiV3Small: test failed in the recent build. No failures on upstream;
branch story: [F+] vs upstream: [++++++++++++++++++++++++++++++++++++++++++++++++++++++++]; [NEW]
org.apache.cassandra.index.sai.disk.vector.VectorCompressionTest.testOpenAiV3Large: test failed in the recent build. No failures on upstream;
branch story: [F+] vs upstream: [++++++++++++++++++++++++++++++++++++++++++++++++++++++++]; [NEW]
org.apache.cassandra.index.sai.cql.types.collections.sets.SetFloatTest.test[version=ca,dataset=set,wide=false,scenario=SSTABLE_QUERY]: test failed in the recent build. No failures on upstream;
branch story: [F+] vs upstream: [++++++++++++++++++++++++++++++++++++++++++++++++++++++++]; [NEW]
org.apache.cassandra.index.sai.cql.VectorSiftSmallTest.testMultiSegmentBuild: test is constantly failing. No failures on upstream;
branch story: [FF] vs upstream: [++++++++++++++++++++++++++++++++++++++++++++++++++++++++]; [NEW]
butler comparison

Please sign in to comment.