Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Cassandra 20402 trunk #3955

Open
wants to merge 2 commits into
base: trunk
Choose a base branch
from

Conversation

ekaterinadimitrova2
Copy link
Contributor

Thanks for sending a pull request! Here are some tips if you're new here:

  • Ensure you have added or run the appropriate tests for your PR.
  • Be sure to keep the PR description updated to reflect all changes.
  • Write your PR title to summarize what this PR proposes.
  • If possible, provide a concise example to reproduce the issue for a faster review.
  • Read our contributor guidelines
  • If you're making a documentation change, see our guide to documentation contribution

Commit messages should follow the following format:

<One sentence description, usually Jira title or CHANGES.txt summary>

<Optional lengthier description (context on patch)>

patch by <Authors>; reviewed by <Reviewers> for CASSANDRA-#####

Co-authored-by: Name1 <email1>
Co-authored-by: Name2 <email2>

The Cassandra Jira

@@ -316,7 +317,13 @@ public void checkQueryability(Index.QueryPlan queryPlan)
for (Index index : queryPlan.getIndexes())
{
if (!isIndexQueryable(index))
{
// In Astra index can be queryable during index build, thus we need to check both not queryable and building
if (isIndexBuilding(index))
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Hmmm, we probably need to use instead directly the full rebuild status here instead? Same as in filterForQuery

Copy link
Contributor

Choose a reason for hiding this comment

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

You're right. I don't think we'd want to count the cases where we aren't doing a full/initial build.

Comment on lines +241 to +243
"SELECT pk FROM " + ks2 + '.' + cf1 + " WHERE v1=0 AND v2=0"
))
.hasMessageContaining("Operation failed - received 1 responses and 1 failures: INDEX_BUILD_IN_PROGRESS");
Copy link
Contributor Author

Choose a reason for hiding this comment

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

In 2i ALLOW FILTERING seems to pass here and this query we have here instead complains of lack of ALLOW FILTERING and not about index not being available or building

Copy link
Contributor Author

Choose a reason for hiding this comment

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

exceptionToReasonMap.put(IndexBuildInProgressException.class, INDEX_BUILD_IN_PROGRESS);

if (exceptionToReasonMap.size() != reasons.length-3)
throw new RuntimeException("A new RequestFailureReasons was probably added and you may need to update the exceptionToReasonMap");
Copy link
Contributor

Choose a reason for hiding this comment

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

What if we just add a nullable field to the enum Class<? extends Throwable> exception and do...

if (reason.exception != null)
    exceptionToReasonMap.put(reason.exception, reason);

...in the loop above? Seems like it helps us avoid the need for this size check, which could get brittle.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Hmmm, that makes a lot of sense. I'm not sure why I didn't do it. Honestly, I can't think of a good reason. Thanks


// below code is intentionally higher, coming as a back port from a fork. It was created that high to prevent
// any conflicts with Apache Cassandra code.
INDEX_BUILD_IN_PROGRESS (503);
Copy link
Contributor

Choose a reason for hiding this comment

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

I'm guessing this might not be the only one of these. Thinking of a general comment that takes that into account...maybe something like...

// The following codes have been ported from an external fork, where they were offset explicitly to avoid conflicts.

@@ -316,7 +317,13 @@ public void checkQueryability(Index.QueryPlan queryPlan)
for (Index index : queryPlan.getIndexes())
{
if (!isIndexQueryable(index))
{
// In Astra index can be queryable during index build, thus we need to check both not queryable and building
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: I guess we should eventually remove the specific reference to Astra?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Actually, it seems it applies to any non-SAI index implementation. Looking into isQueryable() (which is used to decide if we add to queryableIndexes) - it returns always true for non-SAI implementations.

Copy link
Contributor

Choose a reason for hiding this comment

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

Yeah, for some historical reason CassandraIndex looks like...

return status == Status.BUILD_SUCCEEDED || status == Status.UNKNOWN || status == Status.FULL_REBUILD_STARTED;

...whereas SAI won't say it's queryable during a full rebuild.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yeah, my bad, CassandraIndex overrides it too since recently, but still, only BUILD_FAILED and DROPPED means non-queryable

Copy link
Contributor Author

@ekaterinadimitrova2 ekaterinadimitrova2 Mar 13, 2025

Choose a reason for hiding this comment

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

So something like that? (with non-destroyed by GH formatting, of course)

public void checkQueryability(Index.QueryPlan queryPlan)
    {
        InetAddressAndPort endpoint = FBUtilities.getBroadcastAddressAndPort();

        for (Index index : queryPlan.getIndexes())
        {
            String indexName = index.getIndexMetadata().name;
            Index.Status indexStatus = getIndexStatus(endpoint, keyspace.getName(), indexName);

            if (!isIndexQueryable(index))
            {
                // Plus isQueryable is always true for non-SAI index implementations,  thus we need to check both not queryable and building
                if (indexStatus == Index.Status.FULL_REBUILD_STARTED)
                    throw new IndexBuildInProgressException(index);

                throw new IndexNotAvailableException(index);
            }
        }
    }

Copy link
Contributor

Choose a reason for hiding this comment

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

Yeah, that looks reasonable

@@ -18,13 +18,17 @@

package org.apache.cassandra.index.sai.cql;

import org.apache.cassandra.index.IndexBuildInProgressException;
import org.apache.cassandra.inject.Injections;
import org.apache.cassandra.inject.InvokePointBuilder;
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: move these C* imports down

String idx = createIndexAsync(String.format("CREATE CUSTOM INDEX ON %%s(v) USING '%s'", StorageAttachedIndex.class.getName()));

assertThatThrownBy(() -> execute("SELECT * FROM %s WHERE v=0"))
.hasMessage("The secondary index '" + idx + "' is not yet available as it is building")
Copy link
Contributor

Choose a reason for hiding this comment

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

I wonder if we could make a little constant out of the message string/pattern in IndexBuildInProgressException? That way we could reuse it here (and in some other tests?) to build error messages without worrying if we'll change them later (assuming they still take just the index name).

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I have it and I forgot to commit it :-) Good call

assertEquals(RequestFailureReason.NOT_CMS, RequestFailureReason.fromCode(8));
assertEquals(RequestFailureReason.INVALID_ROUTING, RequestFailureReason.fromCode(9));
assertEquals(RequestFailureReason.COORDINATOR_BEHIND, RequestFailureReason.fromCode(10));
assertEquals(RequestFailureReason.INDEX_BUILD_IN_PROGRESS, RequestFailureReason.fromCode(503));
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: Can we just use EXPECTED_VALUES to test these in a loop? At least that way we only have to modify EXPECTED_VALUES when we add a new code...

@@ -25,6 +25,8 @@
import java.util.concurrent.CountDownLatch;

import com.google.common.collect.ImmutableSet;

import org.apache.cassandra.index.IndexBuildInProgressException;
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: move import down

@@ -27,6 +27,7 @@
import java.util.concurrent.TimeUnit;

import com.google.common.base.Objects;
import org.apache.cassandra.distributed.api.ConsistencyLevel;
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: move import down

@@ -391,4 +395,30 @@ private void test(String query, boolean requiresAllowFiltering) throws Throwable
assertNotNull(execute(query + " ALLOW FILTERING"));
}

private final Injections.Barrier blockIndexBuild = Injections.newBarrier("block_index_build", 2, false)
.add(InvokePointBuilder.newInvokePoint().onClass(StorageAttachedIndex.class)
.onMethod("startInitialBuild"))
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: formatting?

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

2 participants