Skip to content
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
wants to merge 5 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
import org.apache.hudi.utilities.schema.SchemaProvider;
import org.apache.hudi.utilities.sources.helpers.AvroConvertor;
import org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen;
import org.apache.hudi.utilities.sources.helpers.KafkaSourceUtil;
import org.apache.hudi.utilities.streamer.DefaultStreamContext;
import org.apache.hudi.utilities.streamer.StreamContext;

Expand All @@ -48,7 +49,6 @@
import static org.apache.hudi.common.util.ConfigUtils.STREAMER_CONFIG_PREFIX;
import static org.apache.hudi.common.util.ConfigUtils.getStringWithAltKeys;
import static org.apache.hudi.utilities.config.KafkaSourceConfig.KAFKA_AVRO_VALUE_DESERIALIZER_CLASS;
import static org.apache.hudi.utilities.config.KafkaSourceConfig.KAFKA_VALUE_DESERIALIZER_SCHEMA;

/**
* Reads avro serialized Kafka data, based on the confluent schema-registry.
Expand Down Expand Up @@ -92,15 +92,15 @@ public AvroKafkaSource(TypedProperties properties, JavaSparkContext sparkContext
}

if (deserializerClassName.equals(KafkaAvroSchemaDeserializer.class.getName())) {
configureSchemaDeserializer();
KafkaSourceUtil.configureSchemaDeserializer(schemaProvider, props);
}
offsetGen = new KafkaOffsetGen(props);
}

@Override
protected InputBatch<JavaRDD<GenericRecord>> readFromCheckpoint(Option<Checkpoint> lastCheckpoint, long sourceLimit) {
if (deserializerClassName.equals(KafkaAvroSchemaDeserializer.class.getName())) {
configureSchemaDeserializer();
KafkaSourceUtil.configureSchemaDeserializer(schemaProvider, props);
offsetGen = new KafkaOffsetGen(props);
}
return super.readFromCheckpoint(lastCheckpoint, sourceLimit);
Expand Down Expand Up @@ -135,11 +135,4 @@ protected JavaRDD<GenericRecord> maybeAppendKafkaOffsets(JavaRDD<ConsumerRecord<
return kafkaRDD.map(consumerRecord -> (GenericRecord) consumerRecord.value());
}
}

private void configureSchemaDeserializer() {
if (schemaProvider == null) {
throw new HoodieReadFromSourceException("SchemaProvider has to be set to use KafkaAvroSchemaDeserializer");
}
props.put(KAFKA_VALUE_DESERIALIZER_SCHEMA.key(), schemaProvider.getSourceSchema().toString());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.utilities.config.HoodieSchemaProviderConfig;
import org.apache.hudi.utilities.config.KafkaSourceConfig;
import org.apache.hudi.utilities.deser.KafkaAvroSchemaDeserializer;
import org.apache.hudi.utilities.exception.HoodieReadFromSourceException;
import org.apache.hudi.utilities.ingestion.HoodieIngestionMetrics;
import org.apache.hudi.utilities.schema.SchemaProvider;
Expand All @@ -34,6 +35,7 @@
import org.apache.hudi.utilities.sources.helpers.AvroConvertor;
import org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen;
import org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen.CheckpointUtils;
import org.apache.hudi.utilities.sources.helpers.KafkaSourceUtil;

import org.apache.avro.Schema;
import org.apache.avro.Schema.Field;
Expand Down Expand Up @@ -105,6 +107,10 @@ public DebeziumSource(TypedProperties props, JavaSparkContext sparkContext,
schemaRegistryProvider = (SchemaRegistryProvider) schemaProvider;
}

if (deserializerClassName.equals(KafkaAvroSchemaDeserializer.class.getName())) {
KafkaSourceUtil.configureSchemaDeserializer(schemaRegistryProvider, props);
}

offsetGen = new KafkaOffsetGen(props);
this.metrics = metrics;
}
Expand Down
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, "");
Copy link
Contributor

@danny0405 danny0405 Dec 17, 2024

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.

Copy link
Contributor Author

@vamsikarnika vamsikarnika Dec 18, 2024

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.

Copy link
Contributor

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?

Copy link
Contributor Author

@vamsikarnika vamsikarnika Dec 23, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Copy link
Contributor

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?

Copy link
Contributor Author

@vamsikarnika vamsikarnika Dec 23, 2024

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.

Copy link
Contributor

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.

Copy link
Contributor Author

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.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Caching the KafkaConsumer happens as part of the RDD creation

Does it make sense if we hard code the cache as false first then create/reuse the consumer based on the schema change?

Copy link
Contributor Author

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.

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);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import org.apache.hudi.utilities.config.HoodieStreamerConfig;
import org.apache.hudi.utilities.config.KafkaSourceConfig;
import org.apache.hudi.utilities.exception.HoodieReadFromSourceException;
import org.apache.hudi.utilities.deser.KafkaAvroSchemaDeserializer;
import org.apache.hudi.utilities.ingestion.HoodieIngestionMetrics;
import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
import org.apache.hudi.utilities.schema.SchemaProvider;
Expand Down Expand Up @@ -63,9 +64,13 @@
import static org.apache.hudi.utilities.schema.KafkaOffsetPostProcessor.KAFKA_SOURCE_OFFSET_COLUMN;
import static org.apache.hudi.utilities.schema.KafkaOffsetPostProcessor.KAFKA_SOURCE_PARTITION_COLUMN;
import static org.apache.hudi.utilities.schema.KafkaOffsetPostProcessor.KAFKA_SOURCE_TIMESTAMP_COLUMN;
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.assertNotEquals;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.mockito.Mockito.mock;

public class TestAvroKafkaSource extends SparkClientFunctionalTestHarness {
Expand Down Expand Up @@ -274,4 +279,28 @@ public void testAppendKafkaOffsetsSourceFormatAdapter() throws IOException {
.getBatch().get();
assertEquals(numMessages, nullKafkaKeyDataset.toDF().filter("_hoodie_kafka_source_key is null").count());
}

@Test
void testConfigureSchemaDeserializer() throws IOException {
final String topic = TEST_TOPIC_PREFIX + "testAvroSchemaDeserializer";
TypedProperties props = createPropsForKafkaSource(topic, null, "earliest");

props.put("hoodie.deltastreamer.source.kafka.value.deserializer.class", KafkaAvroSchemaDeserializer.class.getName());
assertThrows(HoodieReadFromSourceException.class, () -> new AvroKafkaSource(props, jsc(), spark(), schemaProvider, metrics));

String schemaFilePath = TestAvroKafkaSource.class.getClassLoader().getResource("schema/simple-test-with-default-value.avsc").getPath();
props.put("hoodie.deltastreamer.schemaprovider.source.schema.file", schemaFilePath);
SchemaProvider schemaProvider = UtilHelpers.wrapSchemaProviderWithPostProcessor(
UtilHelpers.createSchemaProvider(FilebasedSchemaProvider.class.getName(), props, jsc()), props, jsc(), new ArrayList<>());
AvroKafkaSource avroKafkaSource = new AvroKafkaSource(props, jsc(), spark(), schemaProvider, metrics);
assertTrue(avroKafkaSource.props.containsKey(NATIVE_KAFKA_CONSUMER_GROUP_ID));
String groupId = avroKafkaSource.props.getString(NATIVE_KAFKA_CONSUMER_GROUP_ID, "");
assertTrue(groupId.length() <= GROUP_ID_MAX_BYTES_LENGTH);

schemaFilePath = TestAvroKafkaSource.class.getClassLoader().getResource("schema/evolved-test-with-default-value.avsc").getPath();
props.put("hoodie.deltastreamer.schemaprovider.source.schema.file", schemaFilePath);
avroKafkaSource = new AvroKafkaSource(props, jsc(), spark(), schemaProvider, metrics);
String newGroupId = avroKafkaSource.props.getString(NATIVE_KAFKA_CONSUMER_GROUP_ID, "");
assertNotEquals(groupId, newGroupId);
}
}
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);
}
}
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
}
]
}
Loading