Skip to content

Commit

Permalink
Test fixes
Browse files Browse the repository at this point in the history
  • Loading branch information
blambov committed Oct 10, 2024
1 parent 3d9b815 commit 1da1be1
Show file tree
Hide file tree
Showing 2 changed files with 10 additions and 3 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -1042,10 +1042,11 @@ private void debugLogCompactionSummaryInfo(UUID taskId,
StringBuilder newSSTableNames = new StringBuilder(newSStables.size() * 100);
for (SSTableReader reader : newSStables)
newSSTableNames.append(reader.descriptor.baseFileUri()).append(",");
logger.debug("Compacted ({}) {} sstables to [{}] to level={}. {} to {} (~{}% of original) in {}ms. " +
logger.debug("Compacted ({}{}) {} sstables to [{}] to level={}. {} to {} (~{}% of original) in {}ms. " +
"Read Throughput = {}, Write Throughput = {}, Row Throughput = ~{}/s, Partition Throughput = ~{}/s." +
" {} total partitions merged to {}. Partition merge counts were {}.",
taskId,
tokenRange() != null ? " range " + tokenRange() : "",
transaction.originals().size(),
newSSTableNames.toString(),
getLevel(),
Expand All @@ -1066,7 +1067,10 @@ private void debugLogCompactingMessage(UUID taskId)
{
Set<SSTableReader> originals = transaction.originals();
StringBuilder ssTableLoggerMsg = new StringBuilder(originals.size() * 100);
ssTableLoggerMsg.append("Compacting (").append(taskId).append(')').append(" [");
ssTableLoggerMsg.append("Compacting (").append(taskId);
if (tokenRange() != null)
ssTableLoggerMsg.append(" range ").append(tokenRange());
ssTableLoggerMsg.append(") [");
for (SSTableReader sstr : originals)
{
ssTableLoggerMsg.append(sstr.getFilename())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,10 @@ private RuntimeException earlyOpenUnsupported()

public void update(SSTableReader reader, boolean original)
{
earlyOpenUnsupported();
if (original)
throw earlyOpenUnsupported();

mainTransaction.update(reader, original);
}

public void update(Collection<SSTableReader> readers, boolean original)
Expand Down

1 comment on commit 1da1be1

@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: 9 NEW test failure(s) in 3 builds., Build 3: ran 17697 tests with 14 failures and 128 skipped.
Butler analysis done on ds-cassandra-pr-gate/STAR-1872 vs last 16 runs of ds-cassandra-build-nightly/main.
org.apache.cassandra.index.sai.cql.QueryWriteLifecycleTest.testWriteLifecycle[aa_CompoundKeyDataModel{primaryKey=p, c}]: test is constantly failing. No failures on upstream;
branch story: [FF] vs upstream: [++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++]; [NEW]
org.apache.cassandra.index.sai.cql.TinySegmentQueryWriteLifecycleTest.testWriteLifecycle[ca_CompositePartitionKeyDataModel{primaryKey=p1, p2}]: test is constantly failing. No failures on upstream;
branch story: [F] vs upstream: [++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++]; [NEW]
org.apache.cassandra.db.compaction.UnifiedCompactionStrategyTest.testMaximalSelectionReshard[useDiskBoundaries true]: test is constantly failing. No results on upstream;
branch story: [FF] vs upstream: []; [NEW]
org.apache.cassandra.db.compaction.UnifiedCompactionStrategyTest.testGetNextBackgroundTasksParallelizeOutputShards[useDiskBoundaries true]: test is constantly failing. No results on upstream;
branch story: [FF] vs upstream: []; [NEW]
org.apache.cassandra.index.sai.disk.vector.VectorCompressionTest.testNV_QA_4: test looks flaky. No failures on upstream;
branch story: [+F+] vs upstream: [++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++]; [NEW]
org.apache.cassandra.index.sai.disk.vector.VectorCompressionTest.testAda002: test looks flaky. No failures on upstream;
branch story: [+F+] vs upstream: [++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++]; [NEW]
org.apache.cassandra.index.sai.disk.vector.VectorCompressionTest.testOpenAiV3Large: test looks flaky. No failures on upstream;
branch story: [+F+] vs upstream: [++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++]; [NEW]
org.apache.cassandra.index.sai.disk.vector.VectorCompressionTest.testBert: test failed in the recent build. No failures on upstream;
branch story: [FF+] vs upstream: [++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++]; [NEW]
org.apache.cassandra.index.sai.cql.VectorSiftSmallTest.testMultiSegmentBuild: test is constantly failing. No failures on upstream;
branch story: [FFF] vs upstream: [++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++]; [NEW]
butler comparison

Please sign in to comment.