-
Notifications
You must be signed in to change notification settings - Fork 5
Fix off-by-one error in written offset callback #464
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
Open
hachikuji
wants to merge
8
commits into
main
Choose a base branch
from
fix-commit-off-by-one
base: main
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Open
Changes from all commits
Commits
Show all changes
8 commits
Select commit
Hold shift + click to select a range
a220517
Fix off-by-one error in written offset callback
hachikuji 58702a0
Fix failing integration tests
hachikuji 7a4651a
Remove debug logging
hachikuji dcc34f2
Revert test parameterization
hachikuji a483b83
unused imports
hachikuji 4de68fc
Use exclusive offset when restoring batches
hachikuji 4f0fc47
Remove FACT parameterization which is broken
hachikuji 8005462
checkstyle
hachikuji File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -12,7 +12,6 @@ | |
|
|
||
| package dev.responsive.kafka.integration; | ||
|
|
||
| import static dev.responsive.kafka.api.config.ResponsiveConfig.MONGO_CONNECTION_STRING_CONFIG; | ||
| import static dev.responsive.kafka.testutils.IntegrationTestUtils.getCassandraValidName; | ||
| import static dev.responsive.kafka.testutils.IntegrationTestUtils.pipeInput; | ||
| import static dev.responsive.kafka.testutils.IntegrationTestUtils.slurpPartition; | ||
|
|
@@ -57,14 +56,12 @@ | |
| import dev.responsive.kafka.internal.db.CassandraClientFactory; | ||
| import dev.responsive.kafka.internal.db.DefaultCassandraClientFactory; | ||
| import dev.responsive.kafka.internal.db.RemoteKVTable; | ||
| import dev.responsive.kafka.internal.db.mongo.CollectionCreationOptions; | ||
| import dev.responsive.kafka.internal.db.mongo.MongoKVTable; | ||
| import dev.responsive.kafka.internal.db.partitioning.SubPartitioner; | ||
| import dev.responsive.kafka.internal.db.partitioning.TablePartitioner; | ||
| import dev.responsive.kafka.internal.db.spec.DefaultTableSpec; | ||
| import dev.responsive.kafka.internal.metrics.ResponsiveMetrics; | ||
| import dev.responsive.kafka.internal.stores.SchemaTypes.KVSchema; | ||
| import dev.responsive.kafka.internal.stores.TtlResolver; | ||
| import dev.responsive.kafka.internal.utils.SessionUtil; | ||
| import dev.responsive.kafka.testutils.IntegrationTestUtils; | ||
| import dev.responsive.kafka.testutils.IntegrationTestUtils.MockResponsiveKafkaStreams; | ||
| import dev.responsive.kafka.testutils.ResponsiveConfigParam; | ||
|
|
@@ -75,6 +72,7 @@ | |
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.Optional; | ||
| import java.util.OptionalInt; | ||
| import java.util.Properties; | ||
| import java.util.concurrent.ConcurrentHashMap; | ||
| import java.util.concurrent.ConcurrentLinkedQueue; | ||
|
|
@@ -107,6 +105,7 @@ | |
| import org.apache.kafka.streams.processor.internals.DefaultKafkaClientSupplier; | ||
| import org.junit.jupiter.api.AfterEach; | ||
| import org.junit.jupiter.api.BeforeEach; | ||
| import org.junit.jupiter.api.Test; | ||
| import org.junit.jupiter.api.TestInfo; | ||
| import org.junit.jupiter.api.extension.RegisterExtension; | ||
| import org.junit.jupiter.params.ParameterizedTest; | ||
|
|
@@ -185,7 +184,8 @@ public void shouldFlushStoresBeforeClose(final KVSchema type) throws Exception { | |
| final List<ConsumerRecord<Long, Long>> changelogRecords | ||
| = slurpPartition(changelog, properties); | ||
| final long last = changelogRecords.get(changelogRecords.size() - 1).offset(); | ||
| assertThat(cassandraOffset, equalTo(last)); | ||
| // Written offset is exclusive | ||
| assertThat(cassandraOffset, equalTo(last + 1)); | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -259,9 +259,9 @@ public void shouldRepairOffsetsIfOutOfRangeAndConfigured(final KVSchema type) th | |
| } | ||
| } | ||
|
|
||
| @ParameterizedTest | ||
| @EnumSource(KVSchema.class) | ||
| public void shouldRestoreUnflushedChangelog(final KVSchema type) throws Exception { | ||
| @Test | ||
| public void shouldRestoreUnflushedChangelog() throws Exception { | ||
| final KVSchema type = KVSchema.KEY_VALUE; | ||
| final Map<String, Object> properties = getMutableProperties(type); | ||
| final KafkaProducer<Long, Long> producer = new KafkaProducer<>(properties); | ||
| final KafkaClientSupplier defaultClientSupplier = new DefaultKafkaClientSupplier(); | ||
|
|
@@ -280,6 +280,12 @@ public void shouldRestoreUnflushedChangelog(final KVSchema type) throws Exceptio | |
| waitTillFullyConsumed(admin, input, name, Duration.ofSeconds(120)); | ||
| } | ||
|
|
||
| final TopicPartition changelog = new TopicPartition(name + "-" + aggName() + "-changelog", 0); | ||
| final ResponsiveConfig config = ResponsiveConfig.responsiveConfig(properties); | ||
|
|
||
| final RemoteKVTable<?> changelogTable = remoteKVTable(type, defaultFactory, config, changelog); | ||
| assertThat(changelogTable.lastWrittenOffset(0), greaterThan(0L)); | ||
|
|
||
| // restart with fault injecting cassandra client | ||
| final FaultInjectingCassandraClientSupplier cassandraFaultInjector | ||
| = new FaultInjectingCassandraClientSupplier(); | ||
|
|
@@ -299,22 +305,12 @@ public void shouldRestoreUnflushedChangelog(final KVSchema type) throws Exceptio | |
| IntegrationTestUtils | ||
| .waitTillConsumedPast(admin, input, name, endInput + 1, Duration.ofSeconds(30)); | ||
| } | ||
| final TopicPartition changelog = new TopicPartition(name + "-" + aggName() + "-changelog", 0); | ||
|
|
||
| // Make sure changelog is ahead of remote | ||
| final ResponsiveConfig config = ResponsiveConfig.responsiveConfig(properties); | ||
| final RemoteKVTable<?> table; | ||
|
|
||
| table = remoteKVTable(type, defaultFactory, config, changelog); | ||
|
|
||
| final long remoteOffset = table.lastWrittenOffset(0); | ||
| final long remoteOffset = changelogTable.lastWrittenOffset(0); | ||
| assertThat(remoteOffset, greaterThan(0L)); | ||
|
|
||
| final long changelogOffset = admin.listOffsets(Map.of(changelog, OffsetSpec.latest())).all() | ||
| .get() | ||
| .get(changelog) | ||
| .offset(); | ||
| assertThat(remoteOffset, lessThan(changelogOffset)); | ||
| final long changelogEndOffset = IntegrationTestUtils.endOffset(admin, changelog); | ||
| assertThat(remoteOffset, lessThan(changelogEndOffset)); | ||
|
|
||
| // Restart with restore recorder | ||
| final TestKafkaClientSupplier recordingClientSupplier = new TestKafkaClientSupplier(); | ||
|
|
@@ -351,30 +347,33 @@ private RemoteKVTable<?> remoteKVTable( | |
| ) throws InterruptedException, TimeoutException { | ||
| final RemoteKVTable<?> table; | ||
|
|
||
| if (type == KVSchema.FACT) { | ||
| final CassandraClient cassandraClient = defaultFactory.createClient( | ||
| defaultFactory.createCqlSession(config, null), | ||
| config); | ||
| final CassandraClient cassandraClient = defaultFactory.createClient( | ||
| defaultFactory.createCqlSession(config, null), | ||
| config); | ||
|
|
||
| if (type == KVSchema.FACT) { | ||
| table = cassandraClient.factFactory() | ||
| .create(new DefaultTableSpec( | ||
| aggName(), | ||
| TablePartitioner.defaultPartitioner(), | ||
| TtlResolver.NO_TTL, | ||
| config | ||
| )); | ||
|
|
||
| } else if (type == KVSchema.KEY_VALUE) { | ||
| final var connectionString = config.getPassword(MONGO_CONNECTION_STRING_CONFIG).value(); | ||
| final var mongoClient = SessionUtil.connect(connectionString, "", null); | ||
| table = new MongoKVTable( | ||
| mongoClient, | ||
| final SubPartitioner partitioner = SubPartitioner.create( | ||
| OptionalInt.empty(), | ||
| NUM_PARTITIONS, | ||
| aggName(), | ||
| CollectionCreationOptions.fromConfig(config), | ||
| TtlResolver.NO_TTL, | ||
| config | ||
| config, | ||
| changelog.topic() | ||
| ); | ||
| table.init(0); | ||
| table = cassandraClient.kvFactory() | ||
|
Contributor
Author
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. @ableegoldman I had to partially revert #383 here. One of the tests |
||
| .create(new DefaultTableSpec( | ||
| aggName(), | ||
| partitioner, | ||
| TtlResolver.NO_TTL, | ||
| config | ||
| )); | ||
| } else { | ||
| throw new IllegalArgumentException("Unsupported type: " + type); | ||
| } | ||
|
|
@@ -482,7 +481,7 @@ public CqlSession createCqlSession( | |
| final var spy = spy(wrapped); | ||
| doAnswer(a -> { | ||
| final Fault fault = this.fault.get(); | ||
| if (fault != null && a.getArgument(0) instanceof BatchStatement) { | ||
| if (fault != null && (a.getArgument(0) instanceof BatchStatement)) { | ||
| fault.fire(); | ||
| } | ||
| return wrapped.execute((Statement<?>) a.getArgument(0)); | ||
|
|
@@ -537,14 +536,7 @@ public ConsumerRecords<byte[], byte[]> record(final ConsumerRecords<byte[], byte | |
| private Map<String, Object> getMutableProperties(final KVSchema type) { | ||
| final Map<String, Object> properties = new HashMap<>(responsiveProps); | ||
|
|
||
| if (type == KVSchema.FACT) { | ||
| properties.put(ResponsiveConfig.STORAGE_BACKEND_TYPE_CONFIG, StorageBackend.CASSANDRA.name()); | ||
| } else if (type == KVSchema.KEY_VALUE) { | ||
| properties.put(ResponsiveConfig.STORAGE_BACKEND_TYPE_CONFIG, StorageBackend.MONGO_DB.name()); | ||
| } else { | ||
| throw new IllegalArgumentException("Unexpected schema type: " + type.name()); | ||
| } | ||
|
|
||
| properties.put(ResponsiveConfig.STORAGE_BACKEND_TYPE_CONFIG, StorageBackend.CASSANDRA.name()); | ||
| properties.put(KEY_SERIALIZER_CLASS_CONFIG, LongSerializer.class); | ||
| properties.put(VALUE_SERIALIZER_CLASS_CONFIG, LongSerializer.class); | ||
| properties.put(KEY_DESERIALIZER_CLASS_CONFIG, LongDeserializer.class); | ||
|
|
||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
The fault injector doesn't actually work with
KVSchema.FACTbecause it uses a different statement type. I think the off-by-one error was allowing the assertions to pass because we always reconsumed one extra record.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.
I haven't found a good way to target the update from
flushwhen using theFACTtable. It uses aDefaultBoundStatementwhich is used by several other queries prior toflush. Without the targeted fault, the task crashes before it reads any input.