Skip to content

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
wants to merge 10 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
3 changes: 2 additions & 1 deletion pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -50,14 +50,15 @@
<properties>
<sbt.project.name>azure-eventhubs-spark-parent</sbt.project.name>
<scala.binary.version>2.11</scala.binary.version>
<spark.version>2.3.3</spark.version>
<spark.version>2.4.7</spark.version>
<github.global.server>github</github.global.server>
<java.version>1.8</java.version>
<CodeCacheSize>1g</CodeCacheSize>
</properties>

<modules>
<module>core</module>
<module>schemaregistry-avro</module>
</modules>

<profiles>
Expand Down
88 changes: 88 additions & 0 deletions schemaregistry-avro/README.md
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.
- 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 (currently unsupported)

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
57 changes: 57 additions & 0 deletions schemaregistry-avro/pom.xml
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>
Original file line number Diff line number Diff line change
@@ -0,0 +1,131 @@
/*
* 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.{SchemaRegistryAvroSerializerBuilder}
import com.azure.identity.ClientSecretCredentialBuilder
import org.apache.avro.Schema
import org.apache.avro.generic.GenericRecord
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
}

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 = {
val mode = options.get("mode").map(ParseMode.fromString).getOrElse(FailFastMode)
if (mode != PermissiveMode && mode != FailFastMode) {
throw new IllegalArgumentException(mode + "parse mode not supported.")
}
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)")
}
}
Loading