Skip to content

Commit

Permalink
make receivedEntireSSTable volatile to be sure it's visible, and rela…
Browse files Browse the repository at this point in the history
…x the nodetool failure check to be tolerant of alternate repair failure orderings
  • Loading branch information
maedhroz committed Jul 24, 2023
1 parent 22d172d commit 368e404
Show file tree
Hide file tree
Showing 2 changed files with 8 additions and 10 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -68,9 +68,9 @@ public class CassandraStreamReceiver implements StreamReceiver
private final LifecycleTransaction txn;

// holds references to SSTables received
protected Collection<SSTableReader> sstables;
protected final Collection<SSTableReader> sstables;

protected boolean receivedEntireSSTable;
protected volatile boolean receivedEntireSSTable;

private final boolean requiresWritePath;

Expand Down Expand Up @@ -226,7 +226,7 @@ private void sendThroughWritePath(ColumnFamilyStore cfs, Collection<SSTableReade
}
}

public synchronized void finishTransaction()
public synchronized void finishTransaction()
{
txn.finish();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,6 @@
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.distributed.Cluster;
import org.apache.cassandra.distributed.api.IInvokableInstance;
import org.apache.cassandra.distributed.api.NodeToolResult;
import org.apache.cassandra.distributed.test.TestBaseImpl;
import org.apache.cassandra.index.sai.IndexContext;
import org.apache.cassandra.index.sai.disk.format.IndexDescriptor;
Expand All @@ -44,9 +43,10 @@
import org.apache.lucene.store.IndexInput;

import static net.bytebuddy.matcher.ElementMatchers.named;
import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
import static org.assertj.core.api.Assertions.assertThat;
import static org.junit.Assert.assertFalse;

import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
import static org.apache.cassandra.distributed.api.Feature.NETWORK;

public class IndexStreamingFailureTest extends TestBaseImpl
Expand Down Expand Up @@ -107,8 +107,9 @@ private void testAvailabilityAfterStreaming(String table, boolean streamEntireSS
assertThat(rs.length).isEqualTo(0);

// The repair job should fail when index completion fails. This should also fail the streaming transaction.
NodeToolResult result = second.nodetoolResult("repair", KEYSPACE);
result.asserts().failure().errorContains(TEST_ERROR_MESSAGE);
long mark = second.logs().mark();
second.nodetoolResult("repair", KEYSPACE).asserts().failure();
assertFalse("There should be an injected failure in the logs.", second.logs().grep(mark, TEST_ERROR_MESSAGE).getResult().isEmpty());

// The SSTable should not be added to the table view, as the streaming transaction failed...
rs = second.executeInternal(String.format("select pk from %s.%s where pk = ?", KEYSPACE, table), 1);
Expand Down Expand Up @@ -154,10 +155,7 @@ static void installErrors(ClassLoader loader, int node)
.intercept(MethodDelegation.to(ByteBuddyHelper.class))
.make()
.load(loader, ClassLoadingStrategy.Default.INJECTION);
}

if (node == 2)
{
new ByteBuddy().rebase(SAICodecUtils.class)
.method(named("validateChecksum"))
.intercept(MethodDelegation.to(ByteBuddyHelper.class))
Expand Down

0 comments on commit 368e404

Please sign in to comment.