-
Notifications
You must be signed in to change notification settings - Fork 178
Add Schema Registry Avro functions for column deserialization #573
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
arerlend
wants to merge
10
commits into
master
Choose a base branch
from
schemaregistry-avro
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.
Open
Changes from 7 commits
Commits
Show all changes
10 commits
Select commit
Hold shift + click to select a range
b2d3202
initial add, build issues
arerlend 113e62e
build issues
arerlend b80c214
scala/spark versions
arerlend 1423463
cleanup pom
arerlend 2bcf9da
remove scalatest plugin
arerlend f33217c
switch to scala 2.11 spark 2.4.7
arerlend e7e91b5
Merge branch 'master' into arerlend.add-sr-avro
arerlend 429a95e
clean up comments/signatures
arerlend fa97942
add arg/config tests
arerlend 83291be
add parse mode
arerlend 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
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,88 @@ | ||
# azure-schemaregistry-spark (WIP) | ||
|
||
## Overview | ||
|
||
Schema Registry support in Java is provided by the official Schema Registry SDK in the Azure Java SDK repository. | ||
|
||
Schema Registry serializer craft payloads that contain a schema ID and an encoded payload. The ID references a registry-stored schema that can be used to decode the user-specified payload. | ||
|
||
However, consuming Schema Registry-backed payloads in Spark is particularly difficult, since - | ||
- Spark Kafka does not support plug-in with KafkaSerializer and KafkaDeserializer objects, and | ||
- Object management is non-trivial given Spark's driver-executor model. | ||
|
||
For these reasons, Spark functions are required to simplify SR UX in Spark. This repository contains packages that will provide Spark support in Scala for serialization and deserialization of registry-backed payloads. Code is work in progress. | ||
|
||
Currently, only Avro encodings are supported by Azure Schema Registry clients. `from_avro` and `to_avro` found in the `functions.scala` files will be usable for converting Spark SQL columns from registry-backed payloads to columns of the correct Spark SQL datatype (e.g. `StringType`, `StructType`, etc.). | ||
|
||
## Usage | ||
|
||
Compile the JAR and build with dependencies using the following Maven commmand: | ||
```bash | ||
mvn clean compile assembly:single | ||
``` | ||
|
||
The JAR can then be uploaded without additional required dependencies in your Databricks environment. If using `spark-submit`, use the `--jars` option to submit the path of the custom JAR. | ||
|
||
Spark/Databricks usage is the following: | ||
|
||
```scala | ||
import com.microsoft.azure.schemaregistry.spark.avro.functions._; | ||
|
||
val props: HashMap[String, String] = new HashMap() | ||
props.put("schema.registry.url", SCHEMA_REGISTRY_URL) | ||
props.put("schema.registry.tenant.id", SCHEMA_REGISTRY_TENANT_ID) | ||
props.put("schema.registry.client.id", SCHEMA_REGISTRY_CLIENT_ID) | ||
props.put("schema.registry.client.secret", SCHEMA_REGISTRY_CLIENT_SECRET) | ||
|
||
|
||
val df = spark.readStream | ||
.format("kafka") | ||
.option("subscribe", TOPIC) | ||
.option("kafka.bootstrap.servers", BOOTSTRAP_SERVERS) | ||
.option("kafka.sasl.mechanism", "PLAIN") | ||
.option("kafka.security.protocol", "SASL_SSL") | ||
.option("kafka.sasl.jaas.config", EH_SASL) | ||
.option("kafka.request.timeout.ms", "60000") | ||
.option("kafka.session.timeout.ms", "60000") | ||
.option("failOnDataLoss", "false") | ||
.option("startingOffsets", "earliest") | ||
.option("kafka.group.id", "kafka-group") | ||
.load() | ||
|
||
// from_avro() arguments: | ||
// Spark SQL Column | ||
// schema GUID | ||
// properties for communicating with SR service (see props above) | ||
df.select(from_avro($"value", "[schema guid]", props)) | ||
.writeStream | ||
.outputMode("append") | ||
.format("console") | ||
.start() | ||
.awaitTermination() | ||
``` | ||
|
||
## Schema Evolution | ||
|
||
In the context of stream processing, the primary use case is where the schema GUID references a schema matching in the stream. | ||
|
||
However, there are two edge cases that will be common in streaming scenarios in which we are concerned with schema evolution - | ||
- Stream jobs reading old data with new schemas - only backwards compatible data will be readable, meaning that fields may be null. | ||
- Stream jobs reading new data with old schemas - even if the Spark job schema is forwards compatible with the new schema, projecting data written with the new schema to the old one will result in data loss in the case of additional fields being added. | ||
|
||
To handle the more dangerous second case, Spark functions will throw if incoming data contains fields that cannot be captured by the existing schema. This behavior is based on the assumption that perceived data loss is prohibited. | ||
|
||
To handle the first first case, a parameter will be introduced called `requireExactSchemaMatch`: | ||
- If true, if the schema in the payload is not an exact match to the Spark-specified schema, then the job will throw. This allows users to specify that their pipeline contain one schema only. (q: add list of permitted guids?) | ||
- If false, the job will attempt to read the data incoming in the stream. In the case of upgraded consumers reading backwards compatible schemas, the job will be able to properly read the schemas (nullable deleted fields, adding new optional fields). | ||
|
||
## Failure Modes | ||
|
||
Two modes will be supported as dictated by Spark SQL - | ||
- `FailFastMode` - fail on catching any exception | ||
- `PermissiveMode` - continue processing if parsing exceptions are caught | ||
|
||
Customers will be able to configure the stream with specific failure models, but the default failure model will be `FailFastMode` to prevent perceived data loss with `PermissiveMode`. | ||
|
||
See also: | ||
- aka.ms/schemaregistry | ||
- https://github.com/Azure/azure-schema-registry-for-kafka |
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 |
---|---|---|
@@ -0,0 +1,57 @@ | ||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd"> | ||
<modelVersion>4.0.0</modelVersion> | ||
<groupId>com.microsoft.azure</groupId> | ||
<artifactId>azure-schemaregistry-spark-avro_${scala.binary.version}</artifactId> | ||
<version>1.0.0-beta.4</version> | ||
<name>${project.artifactId}</name> | ||
|
||
<parent> | ||
<groupId>com.microsoft.azure</groupId> | ||
<artifactId>azure-eventhubs-spark-parent_${scala.binary.version}</artifactId> | ||
<version>2.3.18</version> | ||
<relativePath>../pom.xml</relativePath> | ||
</parent> | ||
<packaging>jar</packaging> | ||
|
||
<properties> | ||
<maven.compiler.source>1.8</maven.compiler.source> | ||
<maven.compiler.target>1.8</maven.compiler.target> | ||
<encoding>UTF-8</encoding> | ||
</properties> | ||
|
||
<dependencies> | ||
<dependency> | ||
<groupId>org.apache.spark</groupId> | ||
<artifactId>spark-catalyst_${scala.binary.version}</artifactId> | ||
<version>${spark.version}</version> | ||
<type>test-jar</type> | ||
<scope>test</scope> | ||
</dependency> | ||
<dependency> | ||
<groupId>com.azure</groupId> | ||
<artifactId>azure-data-schemaregistry</artifactId> | ||
<version>1.0.0-beta.4</version> | ||
</dependency> | ||
<dependency> | ||
<groupId>com.azure</groupId> | ||
<artifactId>azure-data-schemaregistry-avro</artifactId> | ||
<version>1.0.0-beta.4</version> | ||
</dependency> | ||
<dependency> | ||
<groupId>com.azure</groupId> | ||
<artifactId>azure-identity</artifactId> | ||
<version>1.1.3</version> | ||
</dependency> | ||
</dependencies> | ||
|
||
<build> | ||
<plugins> | ||
<plugin> | ||
<artifactId>maven-assembly-plugin</artifactId> | ||
<configuration> | ||
<skipAssembly>false</skipAssembly> | ||
</configuration> | ||
</plugin> | ||
</plugins> | ||
</build> | ||
</project> |
129 changes: 129 additions & 0 deletions
129
...vro/src/main/scala/com/microsoft/azure/schemaregistry/spark/avro/AvroDataToCatalyst.scala
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 |
---|---|---|
@@ -0,0 +1,129 @@ | ||
/* | ||
* 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 com.microsoft.azure.schemaregistry.spark.avro | ||
|
||
import java.io.ByteArrayInputStream | ||
|
||
import com.azure.core.util.serializer.TypeReference | ||
import com.azure.data.schemaregistry.SchemaRegistryClientBuilder | ||
import com.azure.data.schemaregistry.avro.{SchemaRegistryAvroSerializer, SchemaRegistryAvroSerializerBuilder} | ||
import com.azure.identity.ClientSecretCredentialBuilder | ||
import org.apache.avro.Schema | ||
import org.apache.avro.generic.GenericRecord | ||
import org.apache.spark.sql.AnalysisException | ||
import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, SpecificInternalRow, UnaryExpression} | ||
import org.apache.spark.sql.catalyst.expressions.codegen.{CodeGenerator, CodegenContext, ExprCode} | ||
import org.apache.spark.sql.catalyst.util.{FailFastMode, ParseMode, PermissiveMode} | ||
import org.apache.spark.sql.types._ | ||
|
||
import scala.util.control.NonFatal | ||
|
||
case class AvroDataToCatalyst( | ||
child: Expression, | ||
schemaId: String, | ||
options: Map[java.lang.String, java.lang.String], | ||
requireExactSchemaMatch: Boolean) | ||
extends UnaryExpression with ExpectsInputTypes { | ||
|
||
override def inputTypes: Seq[BinaryType] = Seq(BinaryType) | ||
|
||
override lazy val dataType: DataType = { | ||
val dt = SchemaConverters.toSqlType(new Schema.Parser().parse(expectedSchemaString)).dataType; | ||
dt | ||
// todo: compare stream compat to group compat and verify equal | ||
arerlend marked this conversation as resolved.
Show resolved
Hide resolved
|
||
} | ||
|
||
override def nullable: Boolean = true | ||
|
||
private val expectedSchemaString : String = { | ||
new String(schemaRegistryAsyncClient.getSchema(schemaId).block().getSchema) | ||
} | ||
|
||
@transient private lazy val schemaRegistryCredential = new ClientSecretCredentialBuilder() | ||
.tenantId(options.getOrElse("schema.registry.tenant.id", null)) | ||
.clientId(options.getOrElse("schema.registry.client.id", null)) | ||
.clientSecret(options.getOrElse("schema.registry.client.secret", null)) | ||
.build() | ||
|
||
@transient private lazy val schemaRegistryAsyncClient = new SchemaRegistryClientBuilder() | ||
.endpoint(options.getOrElse("schema.registry.url", null)) | ||
.credential(schemaRegistryCredential) | ||
.buildAsyncClient() | ||
|
||
@transient private lazy val deserializer = new SchemaRegistryAvroSerializerBuilder() | ||
.schemaRegistryAsyncClient(schemaRegistryAsyncClient) | ||
.schemaGroup(options.getOrElse("schema.group", null)) | ||
.autoRegisterSchema(options.getOrElse("specific.avro.reader", false).asInstanceOf[Boolean]) | ||
.buildSerializer() | ||
|
||
@transient private lazy val avroConverter = { | ||
new AvroDeserializer(new Schema.Parser().parse(expectedSchemaString), dataType) | ||
} | ||
|
||
@transient private lazy val expectedSchema = new Schema.Parser().parse(expectedSchemaString) | ||
|
||
@transient private lazy val parseMode: ParseMode = { | ||
FailFastMode // permissive mode | ||
} | ||
|
||
@transient private lazy val nullResultRow: Any = dataType match { | ||
case st: StructType => | ||
val resultRow = new SpecificInternalRow(st.map(_.dataType)) | ||
for(i <- 0 until st.length) { | ||
resultRow.setNullAt(i) | ||
} | ||
resultRow | ||
|
||
case _ => | ||
null | ||
} | ||
|
||
override def nullSafeEval(input: Any): Any = { | ||
try { | ||
val binary = new ByteArrayInputStream(input.asInstanceOf[Array[Byte]]) | ||
// compare schema version and datatype version | ||
val genericRecord = deserializer.deserialize(binary, TypeReference.createInstance(classOf[GenericRecord])) | ||
|
||
if (requireExactSchemaMatch) { | ||
if (!expectedSchema.equals(genericRecord.getSchema)) { | ||
throw new IncompatibleSchemaException(s"Schema not exact match, payload schema did not match expected schema. Payload schema: ${genericRecord.getSchema}") | ||
} | ||
} | ||
|
||
avroConverter.deserialize(genericRecord) | ||
} catch { | ||
case NonFatal(e) => parseMode match { | ||
case PermissiveMode => nullResultRow | ||
case FailFastMode => | ||
throw new Exception("Malformed records are detected in record parsing. " + | ||
s"Current parse Mode: ${FailFastMode.name}. To process malformed records as null " + | ||
"result, try setting the option 'mode' as 'PERMISSIVE'.", e) | ||
case _ => | ||
throw new Exception(s"Unknown parse mode: ${parseMode.name}") | ||
} | ||
} | ||
} | ||
|
||
override def prettyName: String = "from_avro" | ||
|
||
override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { | ||
val expr = ctx.addReferenceObj("this", this) | ||
defineCodeGen(ctx, ev, input => | ||
s"(${CodeGenerator.boxedType(dataType)})$expr.nullSafeEval($input)") | ||
} | ||
} |
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.
Uh oh!
There was an error while loading. Please reload this page.