Skip to content

Commit c5301ee

Browse files
Make schema converter configurable (#268)
* Make schema converter configurable * Use SPI to load custom schema converter * Add Readme * Optimize imports
1 parent 5e068d9 commit c5301ee

File tree

11 files changed

+238
-14
lines changed

11 files changed

+238
-14
lines changed

README.md

Lines changed: 34 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -153,6 +153,40 @@ val sqlSchema = new StructType(new StructField ....
153153
val avroSchema = SparkAvroConversions.toAvroSchema(sqlSchema, avro_schema_name, avro_schema_namespace)
154154
```
155155

156+
#### Custom data conversions
157+
If you would like to use custom logic to convert from Avro to Spark, you can implement the `SchemaConverter` trait.
158+
The custom class is loaded in ABRiS using the service provider interface (SPI), so you need to register your class in your
159+
`META-INF/services` resource directory. You can then configure the custom class with its short name or the fully qualified name.
160+
161+
**Example**
162+
163+
Custom schema converter implementation
164+
```scala
165+
package za.co.absa.abris.avro.sql
166+
import org.apache.avro.Schema
167+
import org.apache.spark.sql.types.DataType
168+
169+
class CustomSchemaConverter extends SchemaConverter {
170+
override val shortName: String = "custom"
171+
override def toSqlType(avroSchema: Schema): DataType = ???
172+
}
173+
```
174+
175+
Provider configuration file `META-INF/services/za.co.absa.abris.avro.sql.SchemaConverter`:
176+
```
177+
za.co.absa.abris.avro.sql.CustomSchemaConverter
178+
```
179+
180+
Abris configuration
181+
```scala
182+
val abrisConfig = AbrisConfig
183+
.fromConfluentAvro
184+
.downloadReaderSchemaByLatestVersion
185+
.andTopicNameStrategy("topic123")
186+
.usingSchemaRegistry(registryConfig)
187+
.withSchemaConverter("custom")
188+
```
189+
156190
## Multiple schemas in one topic
157191
The naming strategies RecordName and TopicRecordName allow for a one topic to receive different payloads,
158192
i.e. payloads containing different schemas that do not have to be compatible,
Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,16 @@
1+
#
2+
# Copyright 2022 ABSA Group Limited
3+
#
4+
# Licensed under the Apache License, Version 2.0 (the "License");
5+
# you may not use this file except in compliance with the License.
6+
# You may obtain a copy of the License at
7+
#
8+
# http://www.apache.org/licenses/LICENSE-2.0
9+
#
10+
# Unless required by applicable law or agreed to in writing, software
11+
# distributed under the License is distributed on an "AS IS" BASIS,
12+
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13+
# See the License for the specific language governing permissions and
14+
# limitations under the License.
15+
#
16+
za.co.absa.abris.avro.sql.DefaultSchemaConverter

src/main/scala/za/co/absa/abris/avro/sql/AvroDataToCatalyst.scala

Lines changed: 13 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,7 @@ import za.co.absa.abris.avro.read.confluent.{ConfluentConstants, SchemaManagerFa
2929
import za.co.absa.abris.config.InternalFromAvroConfig
3030

3131
import java.nio.ByteBuffer
32+
import java.util.ServiceLoader
3233
import scala.collection.mutable
3334
import scala.util.control.NonFatal
3435
import scala.util.{Failure, Success, Try}
@@ -39,9 +40,11 @@ private[abris] case class AvroDataToCatalyst(
3940
schemaRegistryConf: Option[Map[String,String]]
4041
) extends UnaryExpression with ExpectsInputTypes {
4142

43+
private val schemaConverter = loadSchemaConverter(config.schemaConverter)
44+
4245
override def inputTypes: Seq[BinaryType.type] = Seq(BinaryType)
4346

44-
override lazy val dataType: DataType = SchemaConverters.toSqlType(readerSchema).dataType
47+
override lazy val dataType: DataType = schemaConverter.toSqlType(readerSchema)
4548

4649
override def nullable: Boolean = true
4750

@@ -168,4 +171,13 @@ private[abris] case class AvroDataToCatalyst(
168171
override protected def withNewChildInternal(newChild: Expression): Expression =
169172
copy(child = newChild)
170173

174+
private def loadSchemaConverter(nameOpt: Option[String]) = {
175+
import scala.collection.JavaConverters._
176+
nameOpt match {
177+
case Some(name) => ServiceLoader.load(classOf[SchemaConverter]).asScala
178+
.find(c => c.shortName == name || c.getClass.getName == name)
179+
.getOrElse(throw new ClassNotFoundException(s"Could not find schema converter $name"))
180+
case None => new DefaultSchemaConverter()
181+
}
182+
}
171183
}
Lines changed: 25 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,25 @@
1+
/*
2+
* Copyright 2022 ABSA Group Limited
3+
*
4+
* Licensed under the Apache License, Version 2.0 (the "License");
5+
* you may not use this file except in compliance with the License.
6+
* You may obtain a copy of the License at
7+
*
8+
* http://www.apache.org/licenses/LICENSE-2.0
9+
*
10+
* Unless required by applicable law or agreed to in writing, software
11+
* distributed under the License is distributed on an "AS IS" BASIS,
12+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13+
* See the License for the specific language governing permissions and
14+
* limitations under the License.
15+
*/
16+
17+
package za.co.absa.abris.avro.sql
18+
import org.apache.avro.Schema
19+
import org.apache.spark.sql.avro.SchemaConverters
20+
import org.apache.spark.sql.types.DataType
21+
22+
class DefaultSchemaConverter extends SchemaConverter {
23+
override val shortName: String = "default"
24+
override def toSqlType(avroSchema: Schema): DataType = SchemaConverters.toSqlType(avroSchema).dataType
25+
}
Lines changed: 25 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,25 @@
1+
/*
2+
* Copyright 2022 ABSA Group Limited
3+
*
4+
* Licensed under the Apache License, Version 2.0 (the "License");
5+
* you may not use this file except in compliance with the License.
6+
* You may obtain a copy of the License at
7+
*
8+
* http://www.apache.org/licenses/LICENSE-2.0
9+
*
10+
* Unless required by applicable law or agreed to in writing, software
11+
* distributed under the License is distributed on an "AS IS" BASIS,
12+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13+
* See the License for the specific language governing permissions and
14+
* limitations under the License.
15+
*/
16+
17+
package za.co.absa.abris.avro.sql
18+
19+
import org.apache.avro.Schema
20+
import org.apache.spark.sql.types.DataType
21+
22+
trait SchemaConverter {
23+
val shortName: String
24+
def toSqlType(avroSchema: Schema): DataType
25+
}

src/main/scala/za/co/absa/abris/config/Config.scala

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -330,6 +330,12 @@ class FromAvroConfig private(
330330
schemaRegistryConf
331331
)
332332

333+
def withSchemaConverter(schemaConverter: String): FromAvroConfig =
334+
new FromAvroConfig(
335+
abrisConfig + (Key.SchemaConverter -> schemaConverter),
336+
schemaRegistryConf
337+
)
338+
333339
def validate(): Unit = {
334340
if(!abrisConfig.contains(Key.ReaderSchema)) {
335341
throw new IllegalArgumentException(s"Missing mandatory config property ${Key.ReaderSchema}")
@@ -346,5 +352,6 @@ object FromAvroConfig {
346352
private[abris] object Key {
347353
val ReaderSchema = "readerSchema"
348354
val WriterSchema = "writerSchema"
355+
val SchemaConverter = "schemaConverter"
349356
}
350357
}

src/main/scala/za/co/absa/abris/config/InternalFromAvroConfig.scala

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -27,4 +27,8 @@ private[abris] class InternalFromAvroConfig(map: Map[String, Any]) {
2727
val writerSchema: Option[Schema] = map
2828
.get(Key.WriterSchema)
2929
.map(s => AvroSchemaUtils.parse(s.asInstanceOf[String]))
30+
31+
val schemaConverter: Option[String] = map
32+
.get(Key.SchemaConverter)
33+
.map(_.asInstanceOf[String])
3034
}
Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,16 @@
1+
#
2+
# Copyright 2022 ABSA Group Limited
3+
#
4+
# Licensed under the Apache License, Version 2.0 (the "License");
5+
# you may not use this file except in compliance with the License.
6+
# You may obtain a copy of the License at
7+
#
8+
# http://www.apache.org/licenses/LICENSE-2.0
9+
#
10+
# Unless required by applicable law or agreed to in writing, software
11+
# distributed under the License is distributed on an "AS IS" BASIS,
12+
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13+
# See the License for the specific language governing permissions and
14+
# limitations under the License.
15+
#
16+
za.co.absa.abris.avro.sql.DummySchemaConverter

src/test/scala/za/co/absa/abris/avro/sql/AvroDataToCatalystSpec.scala

Lines changed: 64 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -16,27 +16,16 @@
1616

1717
package za.co.absa.abris.avro.sql
1818

19-
import org.apache.spark.sql.SparkSession
2019
import org.apache.spark.sql.functions.col
21-
import org.scalatest.flatspec.AnyFlatSpec
20+
import org.apache.spark.sql.types.{IntegerType, LongType, StructField, StructType}
2221
import org.scalatest.BeforeAndAfterEach
22+
import org.scalatest.flatspec.AnyFlatSpec
2323
import org.scalatest.matchers.should.Matchers
2424
import za.co.absa.abris.avro.functions._
2525
import za.co.absa.abris.config.{AbrisConfig, FromAvroConfig}
2626
import za.co.absa.abris.examples.data.generation.TestSchemas
2727

2828
class AvroDataToCatalystSpec extends AnyFlatSpec with Matchers with BeforeAndAfterEach {
29-
private val spark = SparkSession
30-
.builder()
31-
.appName("unitTest")
32-
.master("local[2]")
33-
.config("spark.driver.bindAddress", "localhost")
34-
.config("spark.ui.enabled", "false")
35-
.getOrCreate()
36-
37-
import spark.implicits._
38-
39-
4029
it should "not print schema registry configs in the spark plan" in {
4130
val sensitiveData = "username:password"
4231
val schemaString = TestSchemas.NATIVE_SIMPLE_NESTED_SCHEMA
@@ -53,4 +42,66 @@ class AvroDataToCatalystSpec extends AnyFlatSpec with Matchers with BeforeAndAft
5342
column.expr.toString() should not include sensitiveData
5443
}
5544

45+
it should "use the default schema converter by default" in {
46+
val schemaString = TestSchemas.NATIVE_SIMPLE_NESTED_SCHEMA
47+
val dummyUrl = "dummyUrl"
48+
val expectedDataType = StructType(Seq(
49+
StructField("int", IntegerType, nullable = false),
50+
StructField("long", LongType, nullable = false)
51+
))
52+
53+
val fromAvroConfig = FromAvroConfig()
54+
.withReaderSchema(schemaString)
55+
.withSchemaRegistryConfig(Map(
56+
AbrisConfig.SCHEMA_REGISTRY_URL -> dummyUrl
57+
))
58+
59+
val column = from_avro(col("avroBytes"), fromAvroConfig)
60+
column.expr.dataType shouldBe expectedDataType
61+
}
62+
63+
it should "use a custom schema converter identified by the short name" in {
64+
val schemaString = TestSchemas.NATIVE_SIMPLE_NESTED_SCHEMA
65+
val dummyUrl = "dummyUrl"
66+
67+
val fromAvroConfig = FromAvroConfig()
68+
.withReaderSchema(schemaString)
69+
.withSchemaRegistryConfig(Map(
70+
AbrisConfig.SCHEMA_REGISTRY_URL -> dummyUrl
71+
))
72+
.withSchemaConverter(DummySchemaConverter.name)
73+
74+
val column = from_avro(col("avroBytes"), fromAvroConfig)
75+
column.expr.dataType shouldBe DummySchemaConverter.dataType
76+
}
77+
78+
it should "use a custom schema converter identified by the fully qualified name" in {
79+
val schemaString = TestSchemas.NATIVE_SIMPLE_NESTED_SCHEMA
80+
val dummyUrl = "dummyUrl"
81+
82+
val fromAvroConfig = FromAvroConfig()
83+
.withReaderSchema(schemaString)
84+
.withSchemaRegistryConfig(Map(
85+
AbrisConfig.SCHEMA_REGISTRY_URL -> dummyUrl
86+
))
87+
.withSchemaConverter("za.co.absa.abris.avro.sql.DummySchemaConverter")
88+
89+
val column = from_avro(col("avroBytes"), fromAvroConfig)
90+
column.expr.dataType shouldBe DummySchemaConverter.dataType
91+
}
92+
93+
it should "throw an error if the specified custom schema converter does not exist" in {
94+
val schemaString = TestSchemas.NATIVE_SIMPLE_NESTED_SCHEMA
95+
val dummyUrl = "dummyUrl"
96+
97+
val fromAvroConfig = FromAvroConfig()
98+
.withReaderSchema(schemaString)
99+
.withSchemaRegistryConfig(Map(
100+
AbrisConfig.SCHEMA_REGISTRY_URL -> dummyUrl
101+
))
102+
.withSchemaConverter("nonexistent")
103+
104+
val ex = intercept[ClassNotFoundException](from_avro(col("avroBytes"), fromAvroConfig))
105+
ex.getMessage should include ("nonexistent")
106+
}
56107
}
Lines changed: 31 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,31 @@
1+
/*
2+
* Copyright 2022 ABSA Group Limited
3+
*
4+
* Licensed under the Apache License, Version 2.0 (the "License");
5+
* you may not use this file except in compliance with the License.
6+
* You may obtain a copy of the License at
7+
*
8+
* http://www.apache.org/licenses/LICENSE-2.0
9+
*
10+
* Unless required by applicable law or agreed to in writing, software
11+
* distributed under the License is distributed on an "AS IS" BASIS,
12+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13+
* See the License for the specific language governing permissions and
14+
* limitations under the License.
15+
*/
16+
17+
package za.co.absa.abris.avro.sql
18+
19+
import org.apache.avro.Schema
20+
import org.apache.spark.sql.types.{DataType, LongType, StructField, StructType}
21+
import za.co.absa.abris.avro.sql.DummySchemaConverter._
22+
23+
class DummySchemaConverter extends SchemaConverter {
24+
override val shortName: String = name
25+
override def toSqlType(avroSchema: Schema): DataType = dataType
26+
}
27+
28+
object DummySchemaConverter {
29+
val name: String = "dummy"
30+
val dataType: DataType = StructType(Seq(StructField("long", LongType)))
31+
}

0 commit comments

Comments
 (0)