-
Notifications
You must be signed in to change notification settings - Fork 3.8k
CASSANDRA-20752: Add SSTableFlushObserver#onSSTableWriterSwitched #4380
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
base: trunk
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change | ||||
---|---|---|---|---|---|---|
|
@@ -41,6 +41,12 @@ public interface PerColumnIndexWriter | |||||
*/ | ||||||
void complete(Stopwatch stopwatch) throws IOException; | ||||||
|
||||||
/** | ||||||
* Called when current sstable writer is switched during sharded compaction to free any in-memory resources associated | ||||||
* with the sstable for current index without waiting for full transaction to complete | ||||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Suggested change
|
||||||
*/ | ||||||
void onSSTableWriterSwitched(Stopwatch stopwatch) throws IOException; | ||||||
|
||||||
/** | ||||||
* Aborts accumulating data. Allows to clean up resources on error. | ||||||
* <p> | ||||||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -164,6 +164,27 @@ public void staticRow(Row staticRow) | |
} | ||
} | ||
|
||
@Override | ||
public void onSSTableWriterSwitched() | ||
{ | ||
if (aborted) return; | ||
|
||
try | ||
{ | ||
for (PerColumnIndexWriter w : perIndexWriters) | ||
{ | ||
w.onSSTableWriterSwitched(stopwatch); | ||
} | ||
} | ||
catch (Throwable t) | ||
{ | ||
logger.error(indexDescriptor.logMessage("Failed to flush segment on sstable writer switched"), t); | ||
abort(t, true); | ||
// fail compaction task or index build task if SAI failed | ||
throw Throwables.unchecked(t); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is unnecessary. When |
||
} | ||
} | ||
|
||
@Override | ||
public void complete() | ||
{ | ||
|
Original file line number | Diff line number | Diff line change | ||||
---|---|---|---|---|---|---|
|
@@ -100,6 +100,18 @@ public void addRow(PrimaryKey key, Row row, long sstableRowId) throws IOExceptio | |||||
} | ||||||
} | ||||||
|
||||||
@Override | ||||||
public void onSSTableWriterSwitched(Stopwatch stopwatch) throws IOException | ||||||
{ | ||||||
if (maybeAbort()) | ||||||
return; | ||||||
|
||||||
boolean emptySegment = currentBuilder == null || currentBuilder.isEmpty(); | ||||||
logger.debug("Flushing index {} with {}buffered data on sstable writer switched...", index.identifier().indexName, emptySegment ? "no " : ""); | ||||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This should use the standard format...
Suggested change
|
||||||
if (!emptySegment) | ||||||
flushSegment(); | ||||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Note that we don't have to aggressively create a new builder here because |
||||||
} | ||||||
|
||||||
@Override | ||||||
public void complete(Stopwatch stopwatch) throws IOException | ||||||
{ | ||||||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -47,7 +47,7 @@ public abstract class SegmentBuilder | |
private static long testLastValidSegmentRowId = -1; | ||
|
||
/** The number of column indexes being built globally. (Starts at one to avoid divide by zero.) */ | ||
private static final AtomicInteger ACTIVE_BUILDER_COUNT = new AtomicInteger(0); | ||
private static final AtomicInteger ACTIVE_BUILDER_COUNT = new AtomicInteger(1); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. In the DS fork, we actually do set this to 1, but here in OSS, we start at zero and ensure safety by making sure the access and increment/decrement operations happen in the correct order below. I would revert this and remove the misleading field comment. However, that does mean you'll probably get a failure in |
||
|
||
/** Minimum flush size, dynamically updated as segment builds are started and completed/aborted. */ | ||
private static volatile long minimumFlushBytes; | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -22,6 +22,10 @@ | |
|
||
import java.util.Collection; | ||
import java.util.Collections; | ||
import java.util.concurrent.ExecutorService; | ||
import java.util.concurrent.Executors; | ||
import java.util.concurrent.Future; | ||
import java.util.concurrent.TimeUnit; | ||
|
||
import com.google.common.collect.Lists; | ||
import org.junit.Assert; | ||
|
@@ -42,6 +46,7 @@ | |
import org.apache.cassandra.index.sai.SAITester; | ||
import org.apache.cassandra.index.sai.utils.IndexIdentifier; | ||
import org.apache.cassandra.index.sai.disk.v1.SSTableIndexWriter; | ||
import org.apache.cassandra.index.sai.disk.v1.segment.SegmentBuilder; | ||
import org.apache.cassandra.index.sai.utils.IndexTermType; | ||
import org.apache.cassandra.inject.ActionBuilder; | ||
import org.apache.cassandra.inject.Expression; | ||
|
@@ -57,10 +62,12 @@ | |
import org.apache.cassandra.service.ActiveRepairService; | ||
import org.apache.cassandra.streaming.PreviewKind; | ||
import org.apache.cassandra.utils.ByteBufferUtil; | ||
import org.apache.cassandra.utils.FBUtilities; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. unused import |
||
import org.apache.cassandra.utils.Throwables; | ||
import org.apache.cassandra.utils.TimeUUID; | ||
import org.apache.cassandra.utils.concurrent.Refs; | ||
|
||
import static org.assertj.core.api.Assertions.assertThat; | ||
import static org.assertj.core.api.Assertions.assertThatThrownBy; | ||
import static org.junit.Assert.assertNotEquals; | ||
import static org.junit.Assert.assertNull; | ||
|
@@ -352,4 +359,53 @@ public void testConcurrentIndexDropWithCompaction() throws Throwable | |
.isInstanceOf(InvalidQueryException.class) | ||
.hasMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE); | ||
} | ||
|
||
@Test | ||
public void testSegmentBuilderFlushWithShardedCompaction() throws Throwable | ||
{ | ||
int shards = 64; | ||
String createTable = "CREATE TABLE %s (id1 TEXT PRIMARY KEY, v1 INT, v2 TEXT) WITH compaction = " + | ||
"{'class' : 'UnifiedCompactionStrategy', 'enabled' : false, 'base_shard_count': " + shards + ", 'min_sstable_size': '1KiB' }"; | ||
createTable(createTable); | ||
createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")); | ||
createIndex(String.format(CREATE_INDEX_TEMPLATE, "v2")); | ||
disableCompaction(keyspace(), currentTable()); | ||
|
||
int rowsPerSSTable = 2000; | ||
int numSSTables = 4; | ||
int key = 0; | ||
for (int s = 0; s < numSSTables; s++) | ||
{ | ||
for (int i = 0; i < rowsPerSSTable; i++) | ||
{ | ||
execute("INSERT INTO %s (id1, v1, v2) VALUES (?, 0, '01e2wefnewirui32e21e21wre')", Integer.toString(key++)); | ||
} | ||
flush(); | ||
} | ||
|
||
ExecutorService executor = Executors.newSingleThreadExecutor(); | ||
try | ||
{ | ||
Future<?> future = executor.submit(() -> { | ||
getCurrentColumnFamilyStore().forceMajorCompaction(false, 1); | ||
waitForCompactions(); | ||
}); | ||
|
||
// verify that it's not accumulating segment builders | ||
while (!future.isDone()) | ||
{ | ||
// ACTIVE_BUILDER_COUNT starts from 1. There are 2 segments for 2 indexes | ||
assertThat(SegmentBuilder.getActiveBuilderCount()).isGreaterThanOrEqualTo(1).isLessThanOrEqualTo(3); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If we revert the
(Suggested by Claude, but I think it's correct.) |
||
} | ||
future.get(30, TimeUnit.SECONDS); | ||
|
||
// verify results are sharded | ||
assertThat(getCurrentColumnFamilyStore().getLiveSSTables()).hasSize(shards); | ||
} | ||
finally | ||
{ | ||
executor.shutdown(); | ||
executor.awaitTermination(30, TimeUnit.SECONDS); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: If this times out, we might want to fail the test? |
||
} | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -32,10 +32,12 @@ | |
import org.apache.commons.lang3.tuple.ImmutableTriple; | ||
import org.apache.commons.lang3.tuple.Triple; | ||
import org.junit.Assert; | ||
import static org.junit.Assert.assertFalse; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. unused import |
||
import org.junit.Before; | ||
import org.junit.BeforeClass; | ||
import org.junit.Test; | ||
|
||
|
||
import org.apache.cassandra.config.DatabaseDescriptor; | ||
import org.apache.cassandra.db.Clustering; | ||
import org.apache.cassandra.db.DecoratedKey; | ||
|
@@ -73,6 +75,7 @@ | |
import static org.mockito.Mockito.mock; | ||
import static org.mockito.Mockito.when; | ||
|
||
|
||
public class SSTableFlushObserverTest | ||
{ | ||
|
||
|
@@ -167,14 +170,15 @@ public void testFlushObserver() throws Exception | |
BufferCell.live(getColumn(cfm, "height"), now, LongType.instance.decompose(178L)))); | ||
|
||
writer.append(new RowIterator(cfm, key, Collections.singletonList(buildRow(expected.get(key))))); | ||
|
||
writer.onSSTableWriterSwitched(); | ||
reader = writer.finish(true); | ||
} | ||
finally | ||
{ | ||
FileUtils.closeQuietly(writer); | ||
} | ||
|
||
|
||
Assert.assertTrue(observer.isWriterSwitched); | ||
Assert.assertTrue(observer.isComplete); | ||
Assert.assertEquals(expected.size(), observer.rows.size()); | ||
|
||
|
@@ -265,6 +269,7 @@ private static class FlushObserver implements SSTableFlushObserver | |
private boolean beginCalled; | ||
private boolean failOnBegin; | ||
private boolean abortCalled; | ||
private boolean isWriterSwitched; | ||
|
||
@Override | ||
public void begin() | ||
|
@@ -274,6 +279,12 @@ public void begin() | |
throw new RuntimeException("Failed to initialize"); | ||
} | ||
|
||
@Override | ||
public void onSSTableWriterSwitched() | ||
{ | ||
isWriterSwitched = true; | ||
} | ||
|
||
@Override | ||
public void startPartition(DecoratedKey key, long dataPosition, long indexPosition) | ||
{ | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.