KAFKA-10173: Fix suppress changelog binary schema compatibility#8905
KAFKA-10173: Fix suppress changelog binary schema compatibility#8905vvcephei merged 23 commits intoapache:trunkfrom vvcephei:kafka-10173-improve-header-comparison
Conversation
| if (record.partition() != partition) { | ||
| throw new IllegalStateException( | ||
| String.format( | ||
| "record partition [%d] is being restored by the wrong suppress partition [%d]", | ||
| record.partition(), | ||
| partition | ||
| ) | ||
| ); | ||
| } |
There was a problem hiding this comment.
On the side, I realized we can consolidate this check and perform it first, rather than after we're already written bad data into the buffer.
| ) | ||
| ); | ||
| } else if (V_1_CHANGELOG_HEADERS.lastHeader("v").equals(record.headers().lastHeader("v"))) { | ||
| } else if (Arrays.equals(record.headers().lastHeader("v").value(), V_1_CHANGELOG_HEADER_VALUE)) { |
There was a problem hiding this comment.
This is the fix (although it was probably fine before). The implementation of Header.equals is not specified by any contract, so it's safer to perform a direct comparison on the header values. Just as before, I'm comparing byte arrays to avoid deserializing the value.
| private static final BytesSerializer KEY_SERIALIZER = new BytesSerializer(); | ||
| private static final ByteArraySerializer VALUE_SERIALIZER = new ByteArraySerializer(); | ||
| private static final byte[] V_1_CHANGELOG_HEADER_VALUE = {(byte) 1}; | ||
| private static final RecordHeaders V_1_CHANGELOG_HEADERS = |
There was a problem hiding this comment.
my IDEA says this variable is never used.
There was a problem hiding this comment.
I saw it is used in line 342.
There was a problem hiding this comment.
my bad. The unused variable is V_1_CHANGELOG_HEADERS rather than V_1_CHANGELOG_HEADER_VALUE
There was a problem hiding this comment.
Ah, right. My mistake. Thanks for pointing it out.
| ); | ||
| } | ||
| } else { | ||
| if (record.headers().lastHeader("v") == null) { |
There was a problem hiding this comment.
nit:
We seek the last header many times. Could we reuse the return value?
guozhangwang
left a comment
There was a problem hiding this comment.
The PR lgtm, please feel free to merge after addressed @chia7712 's comments.
| private static final BytesSerializer KEY_SERIALIZER = new BytesSerializer(); | ||
| private static final ByteArraySerializer VALUE_SERIALIZER = new ByteArraySerializer(); | ||
| private static final byte[] V_1_CHANGELOG_HEADER_VALUE = {(byte) 1}; | ||
| private static final RecordHeaders V_1_CHANGELOG_HEADERS = |
There was a problem hiding this comment.
I saw it is used in line 342.
|
I'm still cleaning up this PR. I'll call for reviews when it's ready. |
I was getting this exception, and somehow, the parallel GC parameter was the culprit
java.lang.OutOfMemoryError: Java heap space
at org.apache.kafka.streams.kstream.internals.FullChangeSerde.decomposeLegacyFormattedArrayIntoChangeArrays(FullChangeSerde.java:82)
at org.apache.kafka.streams.state.internals.TimeOrderedKeyValueBufferChangelogDeserializationHelper.deserializeV2(TimeOrderedKeyValueBufferChangelogDeserializationHelper.java:90)
at org.apache.kafka.streams.state.internals.TimeOrderedKeyValueBufferChangelogDeserializationHelper.duckTypeV2(TimeOrderedKeyValueBufferChangelogDeserializationHelper.java:61)
at org.apache.kafka.streams.state.internals.InMemoryTimeOrderedKeyValueBuffer.restoreBatch(InMemoryTimeOrderedKeyValueBuffer.java:369)
at org.apache.kafka.streams.state.internals.InMemoryTimeOrderedKeyValueBuffer$$Lambda$284/0x00000001002cb440.restoreBatch(Unknown Source)
at org.apache.kafka.streams.state.internals.TimeOrderedKeyValueBufferTest.shouldRestoreV3FormatWithV2Header(TimeOrderedKeyValueBufferTest.java:742)
vvcephei
left a comment
There was a problem hiding this comment.
Whew! Can you take another look @guozhangwang and @chia7712 ?
After finding the root cause, I was able to fixed several related problems. The diff size is unfortunate but it's almost all the result of copy/pasting the smoke test into the upgrade-system-tests modules.
|
|
||
| defaultMaxHeapSize = "2g" | ||
| defaultJvmArgs = ["-Xss4m", "-XX:+UseParallelGC"] | ||
| defaultJvmArgs = ["-Xss4m"] |
There was a problem hiding this comment.
@ijuma , you'll probably want to know about this.
I have no idea why, but one of the new tests in this PR was failing with:
java.lang.OutOfMemoryError: Java heap space
at org.apache.kafka.streams.kstream.internals.FullChangeSerde.decomposeLegacyFormattedArrayIntoChangeArrays(FullChangeSerde.java:82)
at org.apache.kafka.streams.state.internals.TimeOrderedKeyValueBufferChangelogDeserializationHelper.deserializeV2(TimeOrderedKeyValueBufferChangelogDeserializationHelper.java:90)
at org.apache.kafka.streams.state.internals.TimeOrderedKeyValueBufferChangelogDeserializationHelper.duckTypeV2(TimeOrderedKeyValueBufferChangelogDeserializationHelper.java:61)
at org.apache.kafka.streams.state.internals.InMemoryTimeOrderedKeyValueBuffer.restoreBatch(InMemoryTimeOrderedKeyValueBuffer.java:369)
at org.apache.kafka.streams.state.internals.InMemoryTimeOrderedKeyValueBuffer$$Lambda$284/0x00000001002cb440.restoreBatch(Unknown Source)
at org.apache.kafka.streams.state.internals.TimeOrderedKeyValueBufferTest.shouldRestoreV3FormatWithV2Header(TimeOrderedKeyValueBufferTest.java:742)
I captured a flight recording and a heap dump on exit, but everything looked fine, and the heap was only a few megs at the time of the crash. I noticed first that if I just overrode all the jvm args, the test would pass, and through trial and error, I identified this one as the "cause".
I get an OOMe every time with -XX:+UseParallelGC and I've never gotten it without the flag. WDYT about dropping it?
There was a problem hiding this comment.
Aha! I figured it out. There actually was a bug in the test. While duck-typing, the code was trying to allocate an array of 1.8GB. It's funny that disabling this flag made this test pass on java 11 and 14. Maybe the flag partitions the heap on those versions or something, so the test didn't actually have the full 2GB available. Anyway, I'm about to push a fix and put the flag back.
| * We used to serialize a Change into a single byte[]. Now, we don't anymore, but we still keep this logic here | ||
| * so that we can produce the legacy format to test that we can still deserialize it. | ||
| */ | ||
| public static byte[] mergeChangeArraysIntoSingleLegacyFormattedArray(final Change<byte[]> serialChange) { |
There was a problem hiding this comment.
Only used in the test now, so I moved it.
| if (oldValue == null) { | ||
| buffer.putInt(NULL_VALUE_SENTINEL); | ||
| } else if (priorValue == oldValue) { | ||
| } else if (Arrays.equals(priorValue, oldValue)) { |
There was a problem hiding this comment.
This was correct before, since we check equality and enforce identity in the constructor, but Arrays.equals is extremely cheap when the arrays are identical, so explicitly doing an identity check instead of equality was a micro-optimization.
| private static final byte[] V_1_CHANGELOG_HEADER_VALUE = {(byte) 1}; | ||
| private static final byte[] V_2_CHANGELOG_HEADER_VALUE = {(byte) 2}; | ||
| private static final byte[] V_3_CHANGELOG_HEADER_VALUE = {(byte) 3}; | ||
| static final RecordHeaders CHANGELOG_HEADERS = | ||
| new RecordHeaders(new Header[] {new RecordHeader("v", V_3_CHANGELOG_HEADER_VALUE)}); |
There was a problem hiding this comment.
We don't need to store the whole RecordHeaders for the old versions, just the actual version flag.
| // in this case, the changelog value is a serialized BufferValue | ||
| } else if (Arrays.equals(versionHeader.value(), V_2_CHANGELOG_HEADER_VALUE)) { | ||
|
|
||
| final DeserializationResult deserializationResult = duckTypeV2(record, key); |
There was a problem hiding this comment.
See the comment on this method for why we need to duck-type version 2. I pulled these deserializations into a helper class because all the extra branches pushed our cyclomatic complexity over the limit.
But I kept the first two branches here because they aren't pure functions. They perform a lookup in the buffer itself as part of converting the old format.
There was a problem hiding this comment.
Could you clarify which comment are you referring to? I did not see any comments for the "restoreBatch" method..
There was a problem hiding this comment.
Sorry, the comments in duckTypeV2.
Basically, because we released three versions that would write data in the "v3" format, but with the "v2" flag, when we see the v2 flag, the data might be in v2 format or v3 format. The only way to tell is to just try to deserialize it in v2 format, and if we get an exception, then to try with v3 format.
| "replication.factor": self.REPLICATION_FACTOR, | ||
| "num.standby.replicas": 2, | ||
| "buffered.records.per.partition": 100, | ||
| "commit.interval.ms": 1000, | ||
| "auto.offset.reset": "earliest", | ||
| "acks": "all"} |
There was a problem hiding this comment.
Moved from the java code so that all the configs can be defined together.
| # can be replaced with metadata_2_versions | ||
| backward_compatible_metadata_2_versions = [str(LATEST_0_10_2), str(LATEST_0_11_0), str(LATEST_1_0), str(LATEST_1_1)] | ||
| metadata_3_or_higher_versions = [str(LATEST_2_0), str(LATEST_2_1), str(LATEST_2_2), str(LATEST_2_3), str(LATEST_2_4), str(LATEST_2_5), str(DEV_VERSION)] | ||
| smoke_test_versions = [str(LATEST_2_2), str(LATEST_2_3), str(LATEST_2_4), str(LATEST_2_5)] |
There was a problem hiding this comment.
See KAFKA-10203 for why I couldn't go past 2.2
|
|
||
| @matrix(from_version=metadata_2_versions, to_version=metadata_2_versions) | ||
| def test_simple_upgrade_downgrade(self, from_version, to_version): | ||
| @matrix(from_version=smoke_test_versions, to_version=dev_version) |
There was a problem hiding this comment.
We were previously not testing 2.0+ at all. After rewriting this as a smoke test, it only applies to 2.2+. I also figured it makes more sense just to test upgrades to the current branch, rather than testing cross-upgrades between every pair of versions.
There was a problem hiding this comment.
+1, I think this is a great find.
| self.zk.start() | ||
|
|
||
| self.kafka = KafkaService(self.test_context, num_nodes=1, zk=self.zk, topics=self.topics) | ||
| self.kafka = KafkaService(self.test_context, num_nodes=1, zk=self.zk, topics={ |
There was a problem hiding this comment.
A lot of these changes are part of adapting the test to the smoke test app.
| @@ -349,56 +370,42 @@ def get_version_string(self, version): | |||
| def start_all_nodes_with(self, version): | |||
There was a problem hiding this comment.
I refactored this method to start all the nodes concurrently, rather than one at a time. We still do a rolling upgrade, but there's no need to do a rolling startup.
| // in this case, the changelog value is a serialized BufferValue | ||
| } else if (Arrays.equals(versionHeader.value(), V_2_CHANGELOG_HEADER_VALUE)) { | ||
|
|
||
| final DeserializationResult deserializationResult = duckTypeV2(record, key); |
There was a problem hiding this comment.
Could you clarify which comment are you referring to? I did not see any comments for the "restoreBatch" method..
| import static org.apache.kafka.streams.tests.SmokeTestDriver.generate; | ||
| import static org.apache.kafka.streams.tests.SmokeTestDriver.generatePerpetually; | ||
|
|
||
| public class StreamsSmokeTest { |
There was a problem hiding this comment.
I'm assuming 22..25 client / drive code are all copy-pastes here so I skipped reviewing them. LMK if they aren't.
|
|
||
| @matrix(from_version=metadata_2_versions, to_version=metadata_2_versions) | ||
| def test_simple_upgrade_downgrade(self, from_version, to_version): | ||
| @matrix(from_version=smoke_test_versions, to_version=dev_version) |
There was a problem hiding this comment.
+1, I think this is a great find.
| return deserializationResult; | ||
| } | ||
|
|
||
| private static DeserializationResult deserializeV2(final ConsumerRecord<byte[], byte[]> record, |
There was a problem hiding this comment.
Some docs, either here or directly inside InMemoryTimeOrderedKeyValueBuffer.java explaining the format difference would help a lot. You can see some examples like object GroupMetadataManager
| changelogTopic, | ||
| key, | ||
| null, | ||
| null, |
There was a problem hiding this comment.
I'm just thinking, maybe we should encode headers to tombstones too in case in the future we changed the semantics of tombstones?
There was a problem hiding this comment.
I remember considering this when I added the first version header. The reason I didn't is that, since the initial version didn't have any headers, even if we change the tombstone format in the future, we'll always have to interpret a "no header, null value" record as being a "legacy format" tombstone, just like we have to interpret a "no header, non-null value" as being a "legacy format" data record.
You can think of "no header" as indicating "version 0". Since we haven't changed the format of tombstones yet, there's no value in adding a "version 1" flag. We should just wait until we do need to make such a change (if ever).
|
test this |
|
Hmm. Still saw the OOME in https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/3134/ |
|
Retest this please |
|
Ah, that heap space thing was legit. Fix coming... |
|
Hey @guozhangwang , you might want to take a look at that last fix. The duck-typing code was producing an OOME some times, when it would just interpret a random integer out of the buffer as a "size" (integer) and blindly allocate an array of that size. I added a Util (with tests) that has a guard to prevent this. |
|
I will follow up shortly to extract the system tests to a separate PR, since we're having trouble running the tests at all right now, and we wouldn't know if they are even more broken. |
|
Ok, @guozhangwang , This is my "final" iteration. I pulled the system tests out, and I'll follow up with another PR later. This PR should be sufficient for the basic purpose, thanks to the new "binary" compatibility unit tests. |
guozhangwang
left a comment
There was a problem hiding this comment.
LGTM. We can merge after green build.
Let's trigger system test builds on the follow-up PR
|
Failures were unrelated: |
We inadvertently changed the binary schema of the suppress buffer changelog in 2.4.0 without bumping the schema version number. As a result, it is impossible to upgrade from 2.3.x to 2.4+ if you are using suppression. * Refactor the schema compatibility test to use serialized data from older versions as a more foolproof compatibility test. * Refactor the upgrade system test to use the smoke test application so that we actually exercise a significant portion of the Streams API during upgrade testing * Add more recent versions to the upgrade system test matrix * Fix the compatibility bug by bumping the schema version to 3 Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Guozhang Wang <wangguoz@gmail.com>
We inadvertently changed the binary schema of the suppress buffer changelog in 2.4.0 without bumping the schema version number. As a result, it is impossible to upgrade from 2.3.x to 2.4+ if you are using suppression. * Refactor the schema compatibility test to use serialized data from older versions as a more foolproof compatibility test. * Refactor the upgrade system test to use the smoke test application so that we actually exercise a significant portion of the Streams API during upgrade testing * Add more recent versions to the upgrade system test matrix * Fix the compatibility bug by bumping the schema version to 3 Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Guozhang Wang <wangguoz@gmail.com>
We inadvertently changed the binary schema of the suppress buffer changelog in 2.4.0 without bumping the schema version number. As a result, it is impossible to upgrade from 2.3.x to 2.4+ if you are using suppression. * Refactor the schema compatibility test to use serialized data from older versions as a more foolproof compatibility test. * Refactor the upgrade system test to use the smoke test application so that we actually exercise a significant portion of the Streams API during upgrade testing * Add more recent versions to the upgrade system test matrix * Fix the compatibility bug by bumping the schema version to 3 Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Guozhang Wang <wangguoz@gmail.com>
|
backported to 2.6, 2.5, and 2.4. I ran the streams and client tests each time, as well as systemTestLibs. |
* 'trunk' of github.com:apache/kafka: KAFKA-10180: Fix security_config caching in system tests (apache#8917) KAFKA-10173: Fix suppress changelog binary schema compatibility (apache#8905) KAFKA-10166: always write checkpoint before closing an (initialized) task (apache#8926) MINOR: Rename SslTransportLayer.State."NOT_INITALIZED" enum value to "NOT_INITIALIZED" MINOR: Update Scala to 2.13.3 (apache#8931) KAFKA-9076: support consumer sync across clusters in MM 2.0 (apache#7577) MINOR: Remove Diamond and code code Alignment (apache#8107) KAFKA-10198: guard against recycling dirty state (apache#8924)
We inadvertently changed the binary schema of the suppress buffer changelog
in 2.4.0 without bumping the schema version number. As a result, it is impossible
to upgrade from 2.3.x to 2.4+ if you are using suppression.
as a more foolproof compatibility test.
actually exercise a significant portion of the Streams API during upgrade testing
Committer Checklist (excluded from commit message)