-
Notifications
You must be signed in to change notification settings - Fork 2.5k
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
[HUDI-8378] Fix Avro schema deserializer failing with schema evolution #12111
Open
vamsikarnika
wants to merge
5
commits into
apache:master
Choose a base branch
from
vamsikarnika:fix_avro_deserializer
base: master
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.
+192
−10
Open
Changes from all commits
Commits
Show all changes
5 commits
Select commit
Hold shift + click to select a range
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 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 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
50 changes: 50 additions & 0 deletions
50
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/KafkaSourceUtil.java
This file contains 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 |
---|---|---|
@@ -0,0 +1,50 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.hudi.utilities.sources.helpers; | ||
|
||
import org.apache.hudi.common.config.TypedProperties; | ||
import org.apache.hudi.common.util.StringUtils; | ||
import org.apache.hudi.common.util.hash.HashID; | ||
import org.apache.hudi.utilities.exception.HoodieReadFromSourceException; | ||
import org.apache.hudi.utilities.schema.SchemaProvider; | ||
|
||
import com.google.crypto.tink.subtle.Base64; | ||
|
||
import java.util.Objects; | ||
|
||
import static org.apache.hudi.utilities.config.KafkaSourceConfig.KAFKA_VALUE_DESERIALIZER_SCHEMA; | ||
|
||
public class KafkaSourceUtil { | ||
|
||
public static final String NATIVE_KAFKA_CONSUMER_GROUP_ID = "group.id"; | ||
public static final int GROUP_ID_MAX_BYTES_LENGTH = 255; | ||
|
||
public static void configureSchemaDeserializer(SchemaProvider schemaProvider, TypedProperties props) { | ||
if (schemaProvider == null || Objects.isNull(schemaProvider.getSourceSchema())) { | ||
throw new HoodieReadFromSourceException("SchemaProvider has to be set to use KafkaAvroSchemaDeserializer"); | ||
} | ||
props.put(KAFKA_VALUE_DESERIALIZER_SCHEMA.key(), schemaProvider.getSourceSchema().toString()); | ||
// assign consumer group id based on the schema, since if there's a change in the schema we ensure KafkaRDDIterator doesn't use cached Kafka Consumer | ||
String groupId = props.getString(NATIVE_KAFKA_CONSUMER_GROUP_ID, ""); | ||
String schemaHash = Base64.encode(HashID.hash(schemaProvider.getSourceSchema().toString(), HashID.Size.BITS_128)); | ||
String updatedConsumerGroup = groupId.isEmpty() ? schemaHash | ||
: StringUtils.concatenateWithThreshold(String.format("%s_", groupId), schemaHash, GROUP_ID_MAX_BYTES_LENGTH); | ||
props.put(NATIVE_KAFKA_CONSUMER_GROUP_ID, updatedConsumerGroup); | ||
} | ||
} |
This file contains 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
62 changes: 62 additions & 0 deletions
62
...tilities/src/test/java/org/apache/hudi/utilities/sources/helpers/TestKafkaSourceUtil.java
This file contains 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 |
---|---|---|
@@ -0,0 +1,62 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.hudi.utilities.sources.helpers; | ||
|
||
import org.apache.hudi.common.config.TypedProperties; | ||
import org.apache.hudi.utilities.exception.HoodieReadFromSourceException; | ||
import org.apache.hudi.utilities.schema.SchemaProvider; | ||
|
||
import org.apache.avro.Schema; | ||
import org.junit.jupiter.api.Test; | ||
import org.junit.jupiter.api.extension.ExtendWith; | ||
import org.mockito.Mock; | ||
import org.mockito.junit.jupiter.MockitoExtension; | ||
|
||
import static org.apache.hudi.utilities.config.KafkaSourceConfig.KAFKA_VALUE_DESERIALIZER_SCHEMA; | ||
import static org.apache.hudi.utilities.sources.helpers.KafkaSourceUtil.GROUP_ID_MAX_BYTES_LENGTH; | ||
import static org.apache.hudi.utilities.sources.helpers.KafkaSourceUtil.NATIVE_KAFKA_CONSUMER_GROUP_ID; | ||
import static org.junit.jupiter.api.Assertions.assertEquals; | ||
import static org.junit.jupiter.api.Assertions.assertThrows; | ||
import static org.junit.jupiter.api.Assertions.assertTrue; | ||
import static org.mockito.Mockito.when; | ||
|
||
@ExtendWith(MockitoExtension.class) | ||
public class TestKafkaSourceUtil { | ||
|
||
@Mock | ||
SchemaProvider schemaProvider; | ||
|
||
@Test | ||
void testConfigureSchemaDeserializer() { | ||
TypedProperties props = new TypedProperties(); | ||
// should throw exception when schema provider is null. | ||
assertThrows(HoodieReadFromSourceException.class, () -> KafkaSourceUtil.configureSchemaDeserializer(schemaProvider, props)); | ||
|
||
String avroSchemaJson = | ||
"{\"type\":\"record\",\"name\":\"Person\",\"fields\":[{\"name\":\"name\",\"type\":\"string\"}," | ||
+ "{\"name\":\"age\",\"type\":\"int\"},{\"name\":\"email\",\"type\":[\"null\",\"string\"],\"default\":null}," | ||
+ "{\"name\":\"isEmployed\",\"type\":\"boolean\"}]}"; | ||
Schema schema = new Schema.Parser().parse(avroSchemaJson); | ||
when(schemaProvider.getSourceSchema()).thenReturn(schema); | ||
KafkaSourceUtil.configureSchemaDeserializer(schemaProvider, props); | ||
assertTrue(props.containsKey(NATIVE_KAFKA_CONSUMER_GROUP_ID)); | ||
assertTrue(props.getString(NATIVE_KAFKA_CONSUMER_GROUP_ID, "").length() <= GROUP_ID_MAX_BYTES_LENGTH); | ||
assertEquals(props.getString(KAFKA_VALUE_DESERIALIZER_SCHEMA.key()), avroSchemaJson); | ||
} | ||
} |
42 changes: 42 additions & 0 deletions
42
hudi-utilities/src/test/resources/schema/evolved-test-with-default-value.avsc
This file contains 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 |
---|---|---|
@@ -0,0 +1,42 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
{ | ||
"namespace": "example.avro", | ||
"type": "record", | ||
"name": "User", | ||
"fields": [ | ||
{"name": "name", "type": "string"}, | ||
{"name": "favorite_number", "type": "int"}, | ||
{ | ||
"name": "age", | ||
"type": "int", | ||
"default": 30 | ||
}, | ||
{"name": "favorite_color", "type": "string"}, | ||
{ | ||
"name": "email", | ||
"type": ["null", "string"], | ||
"default": null | ||
}, | ||
{ | ||
"name": "phone", | ||
"type": ["null", "string"], | ||
"default": null | ||
} | ||
] | ||
} |
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.
Instead of setting up the group id, why not just fetch the latest schema and set up it again if the schema evolved.
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 KafkaRDD class from the Kafka client package is caching the KafkaConsumer for a given group id and partition. As a result, we are using KafkaAvroSchemaDeserializer with the previous schema to deserialize the records. This leads to issues during schema evolution.
This changes makes sure that new KafkaConsumer is created when there's a schema change present.
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.
Does Kafka client itself has an option to control the cache TTL?
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.
No, while creating the Kafka RDD, it's always hardcoded to true
https://github.com/apache/spark/blob/876450c4130062a80f4ecb7b7afd232b5481a99f/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaUtils.scala#L70
Since useConsumerCache is set to true. We always get the existing consumer with outdated schema
https://github.com/apache/spark/blob/876450c4130062a80f4ecb7b7afd232b5481a99f/connector/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala#L337
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.
Can we copy this utility into hudi and override the flag as false then?
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.
But if we override the flag to false, we might end up creating new KafkaConsumer every time.
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.
Can we create based on the schema change then? The delta-streamer is a micro-batch for each run, maybe we can add some smart strategy here.
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.
This change will do exactly the same. Caching the KafkaConsumer happens as part of the RDD creation, I'm not sure we can extract that functionality of the kafka utils libr.
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.
Does it make sense if we hard code the cache as false first then create/reuse the consumer based on the schema change?
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.
We can't control how the KafkaConsumer is created, It's done during the RDD creation. I don't think we can control that.