├── spark-schema-registry
├── src
│ ├── test
│ │ ├── resources
│ │ │ ├── numbers.avsc
│ │ │ ├── trucks-v1.avsc
│ │ │ ├── trucks-v2.avsc
│ │ │ └── schema-registry-test.yaml
│ │ └── scala
│ │ │ └── com
│ │ │ └── hortonworks
│ │ │ └── spark
│ │ │ └── registry
│ │ │ └── util
│ │ │ └── SchemaRegistryUtilSuite.scala
│ └── main
│ │ └── scala
│ │ └── com
│ │ └── hortonworks
│ │ └── spark
│ │ └── registry
│ │ ├── avro
│ │ ├── AvroDataToCatalyst.scala
│ │ ├── CatalystDataToAvro.scala
│ │ ├── SchemaConverters.scala
│ │ ├── AvroSerializer.scala
│ │ └── AvroDeserializer.scala
│ │ └── util
│ │ └── util.scala
└── pom.xml
├── .gitignore
├── .travis.yml
├── assembly
├── src
│ └── main
│ │ └── assembly
│ │ └── assembly.xml
└── pom.xml
├── examples
├── src
│ └── main
│ │ └── scala
│ │ └── com
│ │ └── hortonworks
│ │ └── spark
│ │ └── registry
│ │ └── examples
│ │ ├── SchemaRegistryAvroReader.scala
│ │ ├── SchemaJsonExample.scala
│ │ ├── SchemaRegistryJsonExample.scala
│ │ └── SchemaRegistryAvroExample.scala
└── pom.xml
├── pom.xml
├── LICENSE
└── README.md
/spark-schema-registry/src/test/resources/numbers.avsc:
--------------------------------------------------------------------------------
1 | {
2 | "type": "int"
3 | }
--------------------------------------------------------------------------------
/spark-schema-registry/src/test/resources/trucks-v1.avsc:
--------------------------------------------------------------------------------
1 | {
2 | "type": "record",
3 | "namespace": "com.hortonworks.registries",
4 | "name": "trucks",
5 | "fields": [
6 | {
7 | "name": "driverId",
8 | "type": "long"
9 | },
10 | {
11 | "name": "driverName",
12 | "type": ["string", "null"]
13 | },
14 | {
15 | "name": "miles",
16 | "type": "long"
17 | }
18 | ]
19 | }
--------------------------------------------------------------------------------
/.gitignore:
--------------------------------------------------------------------------------
1 | # Maven
2 | target/
3 | pom.xml.tag
4 | pom.xml.releaseBackup
5 | pom.xml.versionsBackup
6 | pom.xml.next
7 | release.properties
8 | dependency-reduced-pom.xml
9 | buildNumber.properties
10 | .mvn/timing.properties
11 |
12 | # Java template
13 | *.class
14 |
15 | # logs
16 | logs
17 | *.log
18 |
19 | # Eclipse
20 | .settings/
21 | .project
22 | .classpath
23 |
24 | # Intellij
25 | *.iml
26 | *.ipr
27 | *.iws
28 | .idea/
29 |
30 | # Package Files
31 | *.jar
32 | *.war
33 | *.ear
34 |
35 | # mac dir files
36 | .DS_Store
37 |
--------------------------------------------------------------------------------
/spark-schema-registry/src/test/resources/trucks-v2.avsc:
--------------------------------------------------------------------------------
1 | {
2 | "type": "record",
3 | "namespace": "com.hortonworks.registries",
4 | "name": "trucks",
5 | "fields": [
6 | {
7 | "name": "driverId",
8 | "type": "long"
9 | },
10 | {
11 | "name": "driverName",
12 | "type": ["string", "null"]
13 | },
14 | {
15 | "name": "miles",
16 | "type": "long"
17 | },
18 | {
19 | "name": "latitude",
20 | "type": "double",
21 | "default": 0.0
22 | },
23 | {
24 | "name": "longitude",
25 | "type": "double",
26 | "default": 0.0
27 | }
28 | ]
29 | }
--------------------------------------------------------------------------------
/.travis.yml:
--------------------------------------------------------------------------------
1 | # Licensed to the Apache Software Foundation (ASF) under one or more
2 | # contributor license agreements. See the NOTICE file distributed with
3 | # this work for additional information regarding copyright ownership.
4 | # The ASF licenses this file to You under the Apache License, Version 2.0
5 | # (the "License"); you may not use this file except in compliance with
6 | # the License. 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 | sudo: false
17 | dist: trusty
18 |
19 | language: java
20 | jdk:
21 | - oraclejdk8
22 |
23 | cache:
24 | directories:
25 | - $HOME/.m2
26 |
27 | notifications:
28 | email: false
29 |
30 | install:
31 | - mvn clean package
32 |
--------------------------------------------------------------------------------
/assembly/src/main/assembly/assembly.xml:
--------------------------------------------------------------------------------
1 |
17 |
18 | dist
19 |
20 | tar.gz
21 |
22 | false
23 |
24 |
25 |
26 | false
27 | libs
28 | false
29 |
30 | com.hortonworks:spark-schema-registry
31 |
32 |
33 |
34 | false
35 | examples
36 | false
37 |
38 | com.hortonworks:spark-schema-registry-examples
39 |
40 |
41 |
42 |
43 |
44 |
45 | ${project.basedir}./spark-schema-registry/target
46 | /libs
47 |
48 | spark-schema-registry*with-dependencies.jar
49 |
50 |
51 |
52 |
53 |
--------------------------------------------------------------------------------
/assembly/pom.xml:
--------------------------------------------------------------------------------
1 |
2 | 4.0.0
3 |
4 | spark-schema-registry-parent
5 | com.hortonworks
6 | 0.1-SNAPSHOT
7 |
8 |
9 | hortonworks-spark-schema-registry-bin
10 | pom
11 | Spark - Schema Registry Assembly
12 | https://github.com/hortonworks-spark/spark-schema-registry
13 |
14 |
15 |
16 | com.hortonworks
17 | spark-schema-registry
18 | ${project.version}
19 |
20 |
21 | com.hortonworks
22 | spark-schema-registry-examples
23 | ${project.version}
24 |
25 |
26 |
27 |
28 | hortonworks-spark-schema-registry
29 |
30 |
31 |
32 | ${final.Name}-${project.version}
33 |
34 |
35 | org.apache.maven.plugins
36 | maven-assembly-plugin
37 |
38 |
39 | package
40 |
41 | single
42 |
43 |
44 |
45 |
46 | true
47 | false
48 |
49 | ${project.basedir}/src/main/assembly/assembly.xml
50 |
51 | false
52 |
53 |
54 |
55 |
56 |
--------------------------------------------------------------------------------
/examples/src/main/scala/com/hortonworks/spark/registry/examples/SchemaRegistryAvroReader.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.hortonworks.spark.registry.examples
19 |
20 | import java.util.UUID
21 |
22 | import com.hortonworks.spark.registry.util._
23 | import org.apache.spark.sql.SparkSession
24 | import org.apache.spark.sql.streaming.{OutputMode, Trigger}
25 |
26 | /**
27 | * This example de-serializes the ouput produced by [[SchemaRegistryAvroExample]] and
28 | * prints the output to console. The schema is automatically infered by querying the schema
29 | * registry.
30 | *
31 | * Usage:
32 | * SchemaRegistryAvroReader [security.protocol]
33 | */
34 | object SchemaRegistryAvroReader {
35 |
36 | def main(args: Array[String]): Unit = {
37 |
38 | val schemaRegistryUrl = if (args.length > 0) args(0) else "http://localhost:9090/api/v1/"
39 | val bootstrapServers = if (args.length > 1) args(1) else "localhost:9092"
40 | val topic = if (args.length > 2) args(2) else "topic1-out"
41 | val checkpointLocation =
42 | if (args.length > 3) args(3) else "/tmp/temporary-" + UUID.randomUUID.toString
43 | val securityProtocol =
44 | if (args.length > 4) Option(args(4)) else None
45 |
46 | val spark = SparkSession
47 | .builder
48 | .appName("SchemaRegistryAvroReader")
49 | .getOrCreate()
50 |
51 | val reader = spark
52 | .readStream
53 | .format("kafka")
54 | .option("kafka.bootstrap.servers", bootstrapServers)
55 | .option("subscribe", topic)
56 |
57 | val messages = securityProtocol
58 | .map(p => reader.option("kafka.security.protocol", p).load())
59 | .getOrElse(reader.load())
60 |
61 | import spark.implicits._
62 |
63 | // the schema registry client config
64 | val config = Map[String, Object]("schema.registry.url" -> schemaRegistryUrl)
65 |
66 | // the schema registry config that will be implicitly passed
67 | implicit val srConfig: SchemaRegistryConfig = SchemaRegistryConfig(config)
68 |
69 | // Read messages from kafka and deserialize.
70 | // This uses the schema registry schema associated with the topic.
71 | val df = messages
72 | .select(from_sr($"value", topic).alias("message"))
73 |
74 | // write the output to console
75 | // should produce events like {"driverId":14,"truckId":25,"miles":373}
76 | val query = df
77 | .writeStream
78 | .format("console")
79 | .trigger(Trigger.ProcessingTime(10000))
80 | .outputMode(OutputMode.Append())
81 | .start()
82 |
83 | query.awaitTermination()
84 | }
85 |
86 | }
87 |
--------------------------------------------------------------------------------
/examples/pom.xml:
--------------------------------------------------------------------------------
1 |
2 |
18 |
19 |
20 | 4.0.0
21 |
22 |
23 | com.hortonworks
24 | spark-schema-registry-parent
25 | 0.1-SNAPSHOT
26 | ../pom.xml
27 |
28 |
29 | spark-schema-registry-examples
30 | jar
31 | Spark Schema Registry Examples
32 |
33 |
34 |
35 | com.hortonworks
36 | spark-schema-registry
37 | ${project.version}
38 |
39 |
40 | org.scala-lang
41 | scala-library
42 |
43 |
44 | org.apache.spark
45 | spark-sql_${scala.binary.version}
46 |
47 |
48 | org.apache.spark
49 | spark-sql-kafka-0-10_${scala.binary.version}
50 |
51 |
52 | org.apache.avro
53 | avro
54 |
55 |
56 |
57 |
58 |
59 |
60 |
61 | net.alchim31.maven
62 | scala-maven-plugin
63 | 3.3.2
64 |
65 |
66 |
67 | compile
68 | testCompile
69 |
70 |
71 |
72 | -dependencyfile
73 | ${project.build.directory}/.scala_dependencies
74 |
75 |
76 |
77 |
78 |
79 |
80 | org.apache.maven.plugins
81 | maven-surefire-plugin
82 | 2.21.0
83 |
84 |
85 | true
86 |
87 |
88 |
89 |
90 |
91 |
--------------------------------------------------------------------------------
/spark-schema-registry/src/main/scala/com/hortonworks/spark/registry/avro/AvroDataToCatalyst.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 | package com.hortonworks.spark.registry.avro
18 |
19 | import java.io.ByteArrayInputStream
20 |
21 | import com.hortonworks.registries.schemaregistry.{SchemaVersionInfo, SchemaVersionKey}
22 | import com.hortonworks.registries.schemaregistry.client.SchemaRegistryClient
23 | import com.hortonworks.registries.schemaregistry.serdes.avro.AvroSnapshotDeserializer
24 | import org.apache.avro.Schema
25 | import org.apache.spark.sql.catalyst.InternalRow
26 | import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, UnaryExpression}
27 | import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode}
28 | import org.apache.spark.sql.types.{BinaryType, DataType}
29 |
30 | import scala.collection.JavaConverters._
31 |
32 | /**
33 | * Note: This is a modified version of
34 | * https://github.com/apache/spark/blob/master/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala
35 | * that users schema registry de-serialization.
36 | */
37 | case class AvroDataToCatalyst(child: Expression, schemaName: String, version: Option[Int], config: Map[String, Object])
38 | extends UnaryExpression with ExpectsInputTypes {
39 |
40 | override def inputTypes = Seq(BinaryType)
41 |
42 | @transient private lazy val srDeser: AvroSnapshotDeserializer = {
43 | val obj = new AvroSnapshotDeserializer()
44 | obj.init(config.asJava)
45 | obj
46 | }
47 |
48 | @transient private lazy val srSchema = fetchSchemaVersionInfo(schemaName, version)
49 |
50 | @transient private lazy val avroSchema = new Schema.Parser().parse(srSchema.getSchemaText)
51 |
52 | override lazy val dataType: DataType = SchemaConverters.toSqlType(avroSchema).dataType
53 |
54 | @transient private lazy val avroDeser= new AvroDeserializer(avroSchema, dataType)
55 |
56 | override def nullable: Boolean = true
57 |
58 | override def nullSafeEval(input: Any): Any = {
59 | val binary = input.asInstanceOf[Array[Byte]]
60 | val row = avroDeser.deserialize(srDeser.deserialize(new ByteArrayInputStream(binary), srSchema.getVersion))
61 | val result = row match {
62 | case r: InternalRow => r.copy()
63 | case _ => row
64 | }
65 | result
66 | }
67 |
68 | override def simpleString: String = {
69 | s"from_sr(${child.sql}, ${dataType.simpleString})"
70 | }
71 |
72 | override def sql: String = {
73 | s"from_sr(${child.sql}, ${dataType.catalogString})"
74 | }
75 |
76 | override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
77 | val expr = ctx.addReferenceObj("this", this)
78 | defineCodeGen(ctx, ev, input =>
79 | s"(${ctx.boxedType(dataType)})$expr.nullSafeEval($input)")
80 | }
81 |
82 | private def fetchSchemaVersionInfo(schemaName: String, version: Option[Int]): SchemaVersionInfo = {
83 | val srClient = new SchemaRegistryClient(config.asJava)
84 | version.map(v => srClient.getSchemaVersionInfo(new SchemaVersionKey(schemaName, v)))
85 | .getOrElse(srClient.getLatestSchemaVersionInfo(schemaName))
86 | }
87 |
88 | }
--------------------------------------------------------------------------------
/spark-schema-registry/src/main/scala/com/hortonworks/spark/registry/avro/CatalystDataToAvro.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 | package com.hortonworks.spark.registry.avro
18 |
19 | import com.hortonworks.registries.schemaregistry.{SchemaCompatibility, SchemaMetadata}
20 | import com.hortonworks.registries.schemaregistry.avro.AvroSchemaProvider
21 | import com.hortonworks.registries.schemaregistry.client.SchemaRegistryClient
22 | import com.hortonworks.registries.schemaregistry.serdes.avro.AvroSnapshotSerializer
23 | import org.apache.spark.sql.catalyst.expressions.{Expression, UnaryExpression}
24 | import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode}
25 | import org.apache.spark.sql.types.{BinaryType, DataType}
26 |
27 | import scala.collection.JavaConverters._
28 |
29 | /**
30 | * Note: This is a modified version of
31 | * https://github.com/apache/spark/blob/master/external/avro/src/main/scala/org/apache/spark/sql/avro/CatalystDataToAvro.scala
32 | * that uses schema registry serialization.
33 | */
34 | case class CatalystDataToAvro(
35 | child: Expression,
36 | schemaName: String,
37 | recordName: String,
38 | nameSpace: String,
39 | config: Map[String, Object]
40 | ) extends UnaryExpression {
41 |
42 | override def dataType: DataType = BinaryType
43 |
44 | private val topLevelRecordName = if (recordName == "") schemaName else recordName
45 |
46 | @transient private lazy val avroType =
47 | SchemaConverters.toAvroType(child.dataType, child.nullable, topLevelRecordName, nameSpace)
48 |
49 | @transient private lazy val avroSer =
50 | new AvroSerializer(child.dataType, avroType, child.nullable)
51 |
52 | @transient private lazy val srSer: AvroSnapshotSerializer = {
53 | val obj = new AvroSnapshotSerializer()
54 | obj.init(config.asJava)
55 | obj
56 | }
57 |
58 | @transient private lazy val srClient = new SchemaRegistryClient(config.asJava)
59 |
60 | @transient private lazy val schemaMetadata = {
61 | var schemaMetadataInfo = srClient.getSchemaMetadataInfo(schemaName)
62 | if (schemaMetadataInfo == null) {
63 | val generatedSchemaMetadata = new SchemaMetadata.Builder(schemaName).
64 | `type`(AvroSchemaProvider.TYPE)
65 | .schemaGroup("Autogenerated group")
66 | .description("Autogenerated schema")
67 | .compatibility(SchemaCompatibility.BACKWARD).build
68 | srClient.addSchemaMetadata(generatedSchemaMetadata)
69 | generatedSchemaMetadata
70 | } else {
71 | schemaMetadataInfo.getSchemaMetadata
72 | }
73 | }
74 |
75 | override def nullSafeEval(input: Any): Any = {
76 | val avroData = avroSer.serialize(input)
77 | srSer.serialize(avroData.asInstanceOf[Object], schemaMetadata)
78 | }
79 |
80 | override def simpleString: String = {
81 | s"to_sr(${child.sql}, ${child.dataType.simpleString})"
82 | }
83 |
84 | override def sql: String = {
85 | s"to_sr(${child.sql}, ${child.dataType.catalogString})"
86 | }
87 |
88 | override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
89 | val expr = ctx.addReferenceObj("this", this)
90 | defineCodeGen(ctx, ev, input =>
91 | s"(byte[]) $expr.nullSafeEval($input)")
92 | }
93 | }
94 |
--------------------------------------------------------------------------------
/examples/src/main/scala/com/hortonworks/spark/registry/examples/SchemaJsonExample.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.hortonworks.spark.registry.examples
19 |
20 | import java.util.UUID
21 |
22 | import org.apache.spark.sql.SparkSession
23 | import org.apache.spark.sql.functions.{from_json, struct, to_json}
24 | import org.apache.spark.sql.streaming.{OutputMode, Trigger}
25 | import org.apache.spark.sql.types._
26 |
27 | /**
28 | * This example illustrates the usage of Spark StructType to manually define the schema
29 | * and process messages from kafka. [[SchemaRegistryJsonExample]]
30 | * and [[SchemaRegistryAvroExample]], illustrates how this can be simplified by
31 | * integrating with Schema registry.
32 | *
33 | * This example depends on the resources available at
34 | * https://github.com/hortonworks/registry/tree/master/examples/schema-registry
35 | *
36 | * To run the example:
37 | * 1. Run the Spark application
38 | * SchemaJsonExample
39 | * 2. Ingest the sample data (json) into input topic
40 | * $ cat examples/schema-registry/avro/data/truck_events_json | kafka-console-producer.sh
41 | * --broker-list host:port --topic topic1
42 | * 3. Monitor the output topic
43 | * $ kafka-console-consumer.sh --bootstrap-server host:port --new-consumer --topic topic1-out
44 | *
45 | */
46 | object SchemaJsonExample {
47 |
48 | def main(args: Array[String]): Unit = {
49 |
50 | val bootstrapServers = if (args.length > 0) args(0) else "localhost:9092"
51 | val topic = if (args.length > 1) args(1) else "topic1"
52 | val outTopic = if (args.length > 2) args(2) else "topic1-out"
53 | val checkpointLocation =
54 | if (args.length > 3) args(3) else "/tmp/temporary-" + UUID.randomUUID.toString
55 |
56 | val spark = SparkSession
57 | .builder
58 | .appName("SchemaExample")
59 | .getOrCreate()
60 |
61 | val messages = spark
62 | .readStream
63 | .format("kafka")
64 | .option("kafka.bootstrap.servers", bootstrapServers)
65 | .option("subscribe", topic)
66 | .load()
67 |
68 | import spark.implicits._
69 |
70 | // the schema for truck events
71 | val schema = StructType(Seq(
72 | StructField("driverId", IntegerType, nullable = false),
73 | StructField("truckId", IntegerType, nullable = false),
74 | StructField("eventTime", StringType, nullable = false),
75 | StructField("eventType", StringType, nullable = false),
76 | StructField("longitude", DoubleType, nullable = false),
77 | StructField("latitude", DoubleType, nullable = false),
78 | StructField("eventKey", StringType, nullable = false),
79 | StructField("correlationId", StringType, nullable = false),
80 | StructField("driverName", StringType, nullable = false),
81 | StructField("routeId", IntegerType, nullable = false),
82 | StructField("routeName", StringType, nullable = false),
83 | StructField("eventDate", StringType, nullable = false),
84 | StructField("miles", IntegerType, nullable = false)
85 | ))
86 |
87 | // read messages from kafka and parse it using the above schema
88 | val df = messages
89 | .select(from_json($"value".cast("string"), schema).alias("value"))
90 |
91 | // project (driverId, truckId, miles) for the events where miles > 300
92 | val filtered = df.select($"value.driverId", $"value.truckId", $"value.miles")
93 | .where("value.miles > 300")
94 |
95 | // write the output to a kafka topic serialized as a JSON string.
96 | // should produce events like {"driverId":14,"truckId":25,"miles":373}
97 | val query = filtered
98 | .select(to_json(struct($"*")).alias("value"))
99 | .writeStream
100 | .format("kafka")
101 | .option("kafka.bootstrap.servers", bootstrapServers)
102 | .option("topic", outTopic)
103 | .option("checkpointLocation", checkpointLocation)
104 | .trigger(Trigger.ProcessingTime(10000))
105 | .outputMode(OutputMode.Append())
106 | .start()
107 |
108 | query.awaitTermination()
109 | }
110 |
111 | }
112 |
--------------------------------------------------------------------------------
/examples/src/main/scala/com/hortonworks/spark/registry/examples/SchemaRegistryJsonExample.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.hortonworks.spark.registry.examples
19 |
20 | import java.util.UUID
21 |
22 | import com.hortonworks.spark.registry.util._
23 | import org.apache.spark.sql.SparkSession
24 | import org.apache.spark.sql.functions.{from_json, struct, to_json}
25 | import org.apache.spark.sql.streaming.{OutputMode, Trigger}
26 |
27 | /**
28 | * This example illustrates the usage of Schema registry to infer the spark schema
29 | * based on the kafka topic. See [[SchemaJsonExample]] for the manual approach of
30 | * specifying the schema as a Spark StructType.
31 | *
32 | * This example depends on the resources available at
33 | * https://github.com/hortonworks/registry/tree/master/examples/schema-registry
34 | *
35 | * To run the example:
36 | * 1. Start a schema registry instance and register the truck events schema with the input kafka topic
37 | * name (E.g. topic1) as the schema name.
38 | * https://github.com/hortonworks/registry/blob/master/examples/schema-registry/avro/src/main/resources/truck_events.avsc
39 | * 2. Run the Spark application
40 | * SchemaRegistryJsonExample [security.protocol]
41 | * 3. Ingest the sample data (json) into input topic
42 | * E.g. cat examples/schema-registry/avro/data/truck_events_json | kafka-console-producer.sh
43 | * --broker-list host:port --topic topic1
44 | * 4. Monitor the output topic
45 | * E.g. kafka-console-consumer.sh --bootstrap-server host:port --new-consumer --topic topic1-out
46 | *
47 | */
48 | object SchemaRegistryJsonExample {
49 |
50 | def main(args: Array[String]): Unit = {
51 |
52 | val schemaRegistryUrl = if (args.length > 0) args(0) else "http://localhost:9090/api/v1/"
53 | val bootstrapServers = if (args.length > 1) args(1) else "localhost:9092"
54 | val topic = if (args.length > 2) args(2) else "topic1"
55 | val outTopic = if (args.length > 3) args(3) else "topic1-out"
56 | val checkpointLocation =
57 | if (args.length > 4) args(4) else "/tmp/temporary-" + UUID.randomUUID.toString
58 | val securityProtocol =
59 | if (args.length > 5) Option(args(5)) else None
60 |
61 | val spark = SparkSession
62 | .builder
63 | .appName("SchemaRegistryJsonExample")
64 | .getOrCreate()
65 |
66 | val reader = spark
67 | .readStream
68 | .format("kafka")
69 | .option("kafka.bootstrap.servers", bootstrapServers)
70 | .option("subscribe", topic)
71 |
72 | val messages = securityProtocol
73 | .map(p => reader.option("kafka.security.protocol", p).load())
74 | .getOrElse(reader.load())
75 |
76 |
77 | import spark.implicits._
78 |
79 | // the schema registry client config
80 | val config = Map[String, Object]("schema.registry.url" -> schemaRegistryUrl)
81 |
82 | // the schema registry config that will be implicitly passed
83 | implicit val srConfig:SchemaRegistryConfig = SchemaRegistryConfig(config)
84 |
85 | // retrieve the translated "Spark schema" corresponding to the schema associated
86 | // with the topic in the schema registry.
87 | val schema = sparkSchema(topic)
88 |
89 | // read messages from kafka and parse it using the above schema
90 | val df = messages
91 | .select(from_json($"value".cast("string"), schema).alias("value"))
92 |
93 | // project (driverId, truckId, miles) for the events where miles > 300
94 | val filtered = df.select($"value.driverId", $"value.truckId", $"value.miles")
95 | .where("value.miles > 300")
96 |
97 | // write the output to a kafka topic
98 | // should produce events like {"driverId":14,"truckId":25,"miles":373}
99 | val writer = filtered
100 | .select(to_json(struct($"*")).alias("value"))
101 | .writeStream
102 | .format("kafka")
103 | .option("kafka.bootstrap.servers", bootstrapServers)
104 | .option("topic", outTopic)
105 | .option("checkpointLocation", checkpointLocation)
106 | .trigger(Trigger.ProcessingTime(10000))
107 | .outputMode(OutputMode.Append())
108 |
109 | val query = securityProtocol
110 | .map(p => writer.option("kafka.security.protocol", p).start())
111 | .getOrElse(writer.start())
112 |
113 | query.awaitTermination()
114 | }
115 |
116 | }
117 |
--------------------------------------------------------------------------------
/examples/src/main/scala/com/hortonworks/spark/registry/examples/SchemaRegistryAvroExample.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.hortonworks.spark.registry.examples
19 |
20 | import java.util.UUID
21 |
22 | import com.hortonworks.spark.registry.util._
23 | import org.apache.spark.sql.SparkSession
24 | import org.apache.spark.sql.functions.struct
25 | import org.apache.spark.sql.streaming.{OutputMode, Trigger}
26 |
27 | /**
28 | * This example illustrates the usage of Schema registry to de-serialize and serialize
29 | * messages from and to kafka. See [[SchemaJsonExample]] for the manual approach of
30 | * specifying the schema as a Spark StructType and [[SchemaRegistryJsonExample]] for
31 | * using the schema to decode plain JSON messages.
32 | *
33 | * This example depends on the resources available at
34 | * https://github.com/hortonworks/registry/tree/master/examples/schema-registry
35 | *
36 | * To run the example:
37 | * 1. Start a schema registry instance and register the input schema.
38 | * E.g. 'topic1' as input schema from below:
39 | * https://github.com/hortonworks/registry/blob/master/examples/schema-registry/avro/src/main/resources/truck_events.avsc
40 | * 2. Run the Spark application
41 | * SchemaRegistryAvroExample [kafka.security.protocol]
42 | * 3. Ingest sample data using the schema registry example app into input topic
43 | * E.g. java -jar avro-examples-*.jar -d data/truck_events_json -p data/kafka-producer.props -sm -s data/truck_events.avsc
44 | * (more details - https://github.com/hortonworks/registry/tree/master/examples/schema-registry/avro)
45 | * 4. Monitor the output topic using [[SchemaRegistryAvroReader]]
46 | */
47 | object SchemaRegistryAvroExample {
48 |
49 | def main(args: Array[String]): Unit = {
50 |
51 | val schemaRegistryUrl = if (args.length > 0) args(0) else "http://localhost:9090/api/v1/"
52 | val bootstrapServers = if (args.length > 1) args(1) else "localhost:9092"
53 | val topic = if (args.length > 2) args(2) else "topic1"
54 | val outTopic = if (args.length > 3) args(3) else "topic1-out"
55 | val checkpointLocation =
56 | if (args.length > 4) args(4) else "/tmp/temporary-" + UUID.randomUUID.toString
57 | val securityProtocol =
58 | if (args.length > 5) Option(args(5)) else None
59 |
60 | val spark = SparkSession
61 | .builder
62 | .appName("SchemaRegistryAvroExample")
63 | .getOrCreate()
64 |
65 | val reader = spark
66 | .readStream
67 | .format("kafka")
68 | .option("kafka.bootstrap.servers", bootstrapServers)
69 | .option("subscribe", topic)
70 |
71 | val messages = securityProtocol
72 | .map(p => reader.option("kafka.security.protocol", p).load())
73 | .getOrElse(reader.load())
74 |
75 | import spark.implicits._
76 |
77 | // the schema registry client config
78 | val config = Map[String, Object]("schema.registry.url" -> schemaRegistryUrl)
79 |
80 | // the schema registry config that will be implicitly passed
81 | implicit val srConfig: SchemaRegistryConfig = SchemaRegistryConfig(config)
82 |
83 | // Read messages from kafka and deserialize.
84 | // This uses the schema registry schema
85 | // associated with the topic and maps it to spark schema.
86 | val df = messages
87 | .select(from_sr($"value", topic).alias("message"))
88 |
89 | // project (driverId, truckId, miles) for the events where miles > 300
90 | val filtered = df.select($"message.driverId", $"message.truckId", $"message.miles")
91 | .where("message.miles > 300")
92 |
93 | // write the output as schema registry serialized avro records to a kafka topic
94 | // should produce events like {"driverId":14,"truckId":25,"miles":373}
95 | val writer = filtered
96 | .select(to_sr(struct($"*"), outTopic).alias("value"))
97 | .writeStream
98 | .format("kafka")
99 | .option("kafka.bootstrap.servers", bootstrapServers)
100 | .option("topic", outTopic)
101 | .option("checkpointLocation", checkpointLocation)
102 | .trigger(Trigger.ProcessingTime(10000))
103 | .outputMode(OutputMode.Append())
104 |
105 | val query = securityProtocol
106 | .map(p => writer.option("kafka.security.protocol", p).start())
107 | .getOrElse(writer.start())
108 |
109 | query.awaitTermination()
110 | }
111 |
112 | }
113 |
--------------------------------------------------------------------------------
/spark-schema-registry/src/main/scala/com/hortonworks/spark/registry/util/util.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.hortonworks.spark.registry
19 |
20 | import com.hortonworks.registries.schemaregistry.{SchemaVersionInfo, SchemaVersionKey}
21 | import com.hortonworks.registries.schemaregistry.client.SchemaRegistryClient
22 | import com.hortonworks.spark.registry.avro._
23 | import org.apache.avro.Schema
24 | import org.apache.spark.sql.Column
25 | import org.apache.spark.sql.types.DataType
26 |
27 | import scala.collection.JavaConverters._
28 |
29 | /**
30 | * Spark schemas and deserialization based on
31 | * schema registry (https://github.com/hortonworks/registry) schemas.
32 | */
33 | package object util {
34 |
35 | // wrapper class for Schema registry config
36 | case class SchemaRegistryConfig(config: Map[String, Object])
37 |
38 | /**
39 | * Returns the spark schema (Datatype) corresponding to the latest version of
40 | * schema defined in the schema registry.
41 | *
42 | * @param schemaName the schema name
43 | * @return the spark Datatype corresponding to the schema
44 | */
45 | def sparkSchema(schemaName: String)(implicit srConfig: SchemaRegistryConfig): DataType = {
46 | sparkDataType(fetchSchemaVersionInfo(schemaName, Option.empty, srConfig.config))
47 | }
48 |
49 | /**
50 | * Returns the spark schema (Datatype) corresponding to the given version of
51 | * schema defined in the schema registry.
52 | *
53 | * @param schemaName the schema name
54 | * @param version the schema version
55 | * @return the spark Datatype corresponding to the schema
56 | */
57 |
58 | def sparkSchema(schemaName: String, version: Int)(implicit srConfig: SchemaRegistryConfig): DataType = {
59 | sparkDataType(fetchSchemaVersionInfo(schemaName, Option(version), srConfig.config))
60 | }
61 |
62 | /**
63 | * Returns a schema (String) for the latest version of the schema
64 | * from schema registry. E.g. If the schema type stored in
65 | * schema registry is Avro, this would return the schema as an Avro string.
66 | *
67 | * @param schemaName the schema name
68 | * @return the schema string
69 | */
70 | def schemaText(schemaName: String)(implicit srConfig: SchemaRegistryConfig): String = {
71 | fetchSchemaVersionInfo(schemaName, Option.empty, srConfig.config).getSchemaText
72 | }
73 |
74 | /**
75 | * Returns a schema (String) for the given (schema, version)
76 | * from schema registry. E.g. If the schema type stored in
77 | * schema registry is Avro, this would return the schema as an Avro string.
78 | *
79 | * @param schemaName the schema name
80 | * @param version the schema version
81 | * @return the schema string
82 | */
83 | def schemaText(schemaName: String, version: Int)(implicit srConfig: SchemaRegistryConfig): String = {
84 | fetchSchemaVersionInfo(schemaName, Option(version), srConfig.config).getSchemaText
85 | }
86 |
87 | /**
88 | * Converts schema registry binary format to spark column. This uses the
89 | * latest version of the schema.
90 | *
91 | * @param data schema registry serialized bytes
92 | * @param schemaName schema name
93 | */
94 | def from_sr(data: Column, schemaName: String)(implicit srConfig: SchemaRegistryConfig): Column = {
95 | new Column(AvroDataToCatalyst(data.expr, schemaName, Option.empty, srConfig.config))
96 | }
97 |
98 | /**
99 | * Converts schema registry binary format to spark column.
100 | *
101 | * @param data schema registry serialized bytes
102 | * @param schemaName schema name
103 | * @param version schema version
104 | */
105 | def from_sr(data: Column, schemaName: String, version: Int)(implicit srConfig: SchemaRegistryConfig): Column = {
106 | new Column(AvroDataToCatalyst(data.expr, schemaName, Option(version), srConfig.config))
107 | }
108 |
109 | /**
110 | * Converts a spark column data to binary format of schema registry.
111 | * This looks up a schema registry schema for the `schemaName` that matches the input
112 | * and automatically registers a new schema if not found.
113 | *
114 | * @param data the data column
115 | * @param schemaName the schema registry schema name
116 | */
117 | def to_sr(
118 | data: Column,
119 | schemaName: String,
120 | topLevelRecordName: String = "",
121 | namespace: String = "com.hortonworks.registries")
122 | (implicit srConfig: SchemaRegistryConfig): Column = {
123 | new Column(CatalystDataToAvro(data.expr, schemaName, topLevelRecordName, namespace, srConfig.config))
124 | }
125 |
126 | private def sparkDataType(srSchema: SchemaVersionInfo): DataType = {
127 | val avroSchema = new Schema.Parser().parse(srSchema.getSchemaText)
128 | SchemaConverters.toSqlType(avroSchema).dataType
129 | }
130 |
131 | private def fetchSchemaVersionInfo(
132 | schemaName: String,
133 | version: Option[Integer],
134 | config: Map[String, Object]): SchemaVersionInfo = {
135 | val srClient = new SchemaRegistryClient(config.asJava)
136 | version.map(v => srClient.getSchemaVersionInfo(new SchemaVersionKey(schemaName, v)))
137 | .getOrElse(srClient.getLatestSchemaVersionInfo(schemaName))
138 | }
139 | }
140 |
141 |
--------------------------------------------------------------------------------
/spark-schema-registry/src/test/resources/schema-registry-test.yaml:
--------------------------------------------------------------------------------
1 | # registries configuration
2 | modules:
3 | # - name: tag-registry
4 | # className: com.hortonworks.iotas.registries.tag.service.TagRegistryModule
5 | - name: schema-registry
6 | className: com.hortonworks.registries.schemaregistry.webservice.SchemaRegistryModule
7 | config:
8 | schemaProviders:
9 | - providerClass: "com.hortonworks.registries.schemaregistry.avro.AvroSchemaProvider"
10 | defaultSerializerClass: "com.hortonworks.registries.schemaregistry.serdes.avro.AvroSnapshotSerializer"
11 | defaultDeserializerClass: "com.hortonworks.registries.schemaregistry.serdes.avro.AvroSnapshotDeserializer"
12 | # schema cache properties
13 | # inmemory schema versions cache size
14 | schemaCacheSize: 10000
15 | # inmemory schema version cache entry expiry interval after access
16 | schemaCacheExpiryInterval: 3600
17 |
18 |
19 | servletFilters:
20 | # - className: "com.hortonworks.registries.auth.server.AuthenticationFilter"
21 | # params:
22 | # type: "kerberos"
23 | # kerberos.principal: "HTTP/streamline-ui-host.com"
24 | # kerberos.keytab: "/vagrant/keytabs/http.keytab"
25 | # kerberos.name.rules: "RULE:[2:$1@$0]([jt]t@.*EXAMPLE.COM)s/.*/$MAPRED_USER/ RULE:[2:$1@$0]([nd]n@.*EXAMPLE.COM)s/.*/$HDFS_USER/DEFAULT"
26 | - className: "com.hortonworks.registries.schemaregistry.webservice.RewriteUriFilter"
27 | params:
28 | # value format is [,,*|]*
29 | # below /subjects and /schemas/ids are forwarded to /api/v1/confluent
30 | forwardPaths: "/api/v1/confluent,/subjects/*,/schemas/ids/*"
31 | redirectPaths: "/ui/,/"
32 |
33 | # HA configuration
34 | #haConfig:
35 | # className: com.hortonworks.registries.ha.zk.ZKLeadershipParticipant
36 | # config:
37 | # # This url is a list of ZK servers separated by ,
38 | # connect.url: "localhost:2181"
39 | # # root node prefix in ZK for this instance
40 | # root: "/registry"
41 | # session.timeout.ms: 30000
42 | # connection.timeout.ms: 20000
43 | # retry.limit: 5
44 | # retry.base.sleep.time.ms: 1000
45 | # retry.max.sleep.time.ms: 5000
46 |
47 | fileStorageConfiguration:
48 | className: "com.hortonworks.registries.common.util.LocalFileSystemStorage"
49 | properties:
50 | directory: "/tmp/storage"
51 |
52 | # storage provider configuration
53 | # providerClass can be inmemory and jdbc.
54 | #
55 | # Example configuration for inmemory is:
56 | #storageProviderConfiguration:
57 | # providerClass: "com.hortonworks.registries.storage.impl.memory.InMemoryStorageManager"
58 | #
59 | # Example configuration for phoenix based JDBC driver
60 | #storageProviderConfiguration:
61 | # providerClass: "com.hortonworks.registries.storage.impl.jdbc.JdbcStorageManager"
62 | # properties:
63 | # db.type: "phoenix"
64 | # queryTimeoutInSecs: 30
65 | # db.properties:
66 | # jdbcDriverClass: "com.hortonworks.phoenix.jdbc.PhoenixDriver"
67 | # jdbcUrl: "jdbc:phoenix:localhost:2181"
68 | #
69 | # MySQL based jdbc provider configuration is:
70 | #storageProviderConfiguration:
71 | # providerClass: "com.hortonworks.registries.storage.impl.jdbc.JdbcStorageManager"
72 | # properties:
73 | # db.type: "mysql"
74 | # queryTimeoutInSecs: 30
75 | # db.properties:
76 | # dataSourceClassName: "com.mysql.cj.jdbc.MysqlDataSource"
77 | # dataSource.url: "jdbc:mysql://localhost:3307/test"
78 | #storageProviderConfiguration:
79 | # providerClass: "com.hortonworks.registries.storage.impl.jdbc.JdbcStorageManager"
80 | # properties:
81 | # db.type: "mysql"
82 | # queryTimeoutInSecs: 30
83 | # db.properties:
84 | # dataSourceClassName: "org.mariadb.jdbc.MariaDbDataSource"
85 | # dataSource.url: "jdbc:mysql://localhost/schema_registry"
86 | # dataSource.user: "root"
87 | # dataSource.password: "mysql17"
88 |
89 | storageProviderConfiguration:
90 | providerClass: "com.hortonworks.registries.storage.impl.memory.InMemoryStorageManager"
91 |
92 | #enable CORS, may want to disable in production
93 | enableCors: true
94 |
95 | ## swagger configuration
96 | swagger:
97 | resourcePackage: com.hortonworks.registries.schemaregistry.webservice
98 |
99 | # use the simple server factory if you only want to run on a single port
100 | #server:
101 | # type: simple
102 | # connector:
103 | # type: http
104 | # port: 8080
105 |
106 | server:
107 | # softNofileLimit: 1000
108 | # hardNofileLimit: 1000
109 | applicationConnectors:
110 | - type: http
111 | port: 0
112 | # - type: https
113 | # port: 8443
114 | # keyStorePath: example.keystore
115 | # keyStorePassword: example
116 | # validateCerts: false
117 | # this requires the alpn-boot library on the JVM's boot classpath
118 | # - type: spdy3
119 | # port: 8445
120 | # keyStorePath: example.keystore
121 | # keyStorePassword: example
122 | # validateCerts: false
123 | adminConnectors:
124 | - type: http
125 | port: 0
126 | # - type: https
127 | # port: 8444
128 | # keyStorePath: example.keystore
129 | # keyStorePassword: example
130 | # validateCerts: false
131 |
132 | # Logging settings.
133 | logging:
134 |
135 | # The default level of all loggers. Can be OFF, ERROR, WARN, INFO, DEBUG, TRACE, or ALL.
136 | level: INFO
137 |
138 | # Logger-specific levels.
139 | loggers:
140 |
141 | # Sets the level for 'com.example.app' to DEBUG.
142 | com.hortonworks.registries: DEBUG
143 |
144 |
145 | appenders:
146 | - type: console
147 | # - type: file
148 | # threshold: DEBUG
149 | # logFormat: "%-6level [%d{HH:mm:ss.SSS}] [%t] %logger{5} - %X{code} %msg %n"
150 | # currentLogFilename: /tmp/application.log
151 | # archivedLogFilenamePattern: /tmp/application-%d{yyyy-MM-dd}-%i.log.gz
152 | # archivedFileCount: 7
153 | # timeZone: UTC
154 | # maxFileSize: 10MB
155 |
156 | #jerseyClient:
157 | # minThreads: 1
158 | # maxThreads: 16
159 | # workQueueSize: 1000
160 | # gzipEnabled: true
161 | # gzipEnabledForRequests: true
162 | # chunkedEncodingEnabled: true
163 | # timeout: 1000ms
164 | # connectionTimeout: 1000ms
165 | # timeToLive: 1h
166 | # cookiesEnabled: false
167 | # maxConnections: 10
168 | # maxConnectionsPerRoute: 1024
169 | # keepAlive: 0ms
170 | # retries: 10
171 | # userAgent: Storm-Client
172 |
--------------------------------------------------------------------------------
/pom.xml:
--------------------------------------------------------------------------------
1 |
2 |
18 |
19 |
20 | 4.0.0
21 |
22 | com.hortonworks
23 | spark-schema-registry-parent
24 | 0.1-SNAPSHOT
25 | pom
26 | Spark - Schema Registry integration
27 |
28 | Leverage Schema registry for managing Spark schemas and to
29 | serialize/de-serialize messages in spark data sources and sinks.
30 |
31 | https://github.com/hortonworks-spark/spark-schema-registry
32 |
33 |
34 | spark-schema-registry
35 | examples
36 | assembly
37 |
38 |
39 |
40 | 2.3.2
41 | 2.11.8
42 | 2.11
43 | 0.5.2
44 | 4.12
45 | 1.8.2
46 | 2.9.5
47 | 3.2.5
48 | 3.0.5
49 |
50 |
51 |
52 |
53 |
54 | com.hortonworks.registries
55 | schema-registry-serdes
56 | ${schema.registry.version}
57 |
58 |
59 | org.slf4j
60 | log4j-over-slf4j
61 |
62 |
63 |
64 |
65 | org.scala-lang
66 | scala-library
67 | ${scala.version}
68 | provided
69 |
70 |
71 | org.apache.spark
72 | spark-sql_${scala.binary.version}
73 | ${spark.version}
74 | provided
75 |
76 |
77 | org.slf4j
78 | slf4j-log4j12
79 |
80 |
81 | org.eclipse.jetty
82 | jetty-servlet
83 |
84 |
85 |
86 |
87 | org.apache.spark
88 | spark-sql-kafka-0-10_${scala.binary.version}
89 | ${spark.version}
90 | provided
91 |
92 |
93 | org.apache.avro
94 | avro
95 | ${avro.version}
96 |
97 |
98 |
99 | junit
100 | junit
101 | ${junit.version}
102 | test
103 |
104 |
105 | org.scalatest
106 | scalatest_${scala.binary.version}
107 | ${scalatest.version}
108 | test
109 |
110 |
111 | com.fasterxml.jackson.core
112 | jackson-annotations
113 | ${jackson.version}
114 | test
115 |
116 |
117 | com.fasterxml.jackson.core
118 | jackson-databind
119 | ${jackson.version}
120 | test
121 |
122 |
123 | io.dropwizard.metrics
124 | metrics-core
125 | ${dropwizard.version}
126 | test
127 |
128 |
129 | com.hortonworks.registries
130 | schema-registry-webservice
131 | ${schema.registry.version}
132 | test
133 |
134 |
135 | org.slf4j
136 | log4j-over-slf4j
137 |
138 |
139 | org.slf4j
140 | slf4j-log4j12
141 |
142 |
143 | log4j
144 | log4j
145 |
146 |
147 |
148 |
149 |
150 |
151 |
152 |
153 |
154 |
155 |
156 | net.alchim31.maven
157 | scala-maven-plugin
158 | 3.3.2
159 |
160 |
161 |
162 | compile
163 | testCompile
164 |
165 |
166 |
167 | -dependencyfile
168 | ${project.build.directory}/.scala_dependencies
169 |
170 |
171 |
172 |
173 |
174 |
175 | org.apache.maven.plugins
176 | maven-surefire-plugin
177 | 2.21.0
178 |
179 |
180 | true
181 |
182 |
183 |
184 |
185 |
186 |
187 |
--------------------------------------------------------------------------------
/spark-schema-registry/pom.xml:
--------------------------------------------------------------------------------
1 |
2 |
18 |
19 |
20 | 4.0.0
21 |
22 |
23 | com.hortonworks
24 | spark-schema-registry-parent
25 | 0.1-SNAPSHOT
26 | ../pom.xml
27 |
28 |
29 | spark-schema-registry
30 | jar
31 | Spark Schema Registry
32 |
33 |
34 |
35 | com.hortonworks.registries
36 | schema-registry-serdes
37 |
38 |
39 | org.scala-lang
40 | scala-library
41 |
42 |
43 | org.apache.spark
44 | spark-sql_${scala.binary.version}
45 |
46 |
47 | org.apache.avro
48 | avro
49 |
50 |
51 |
52 | org.scalatest
53 | scalatest_${scala.binary.version}
54 | test
55 |
56 |
57 | com.fasterxml.jackson.core
58 | jackson-annotations
59 | test
60 |
61 |
62 | com.fasterxml.jackson.core
63 | jackson-databind
64 | test
65 |
66 |
67 | io.dropwizard.metrics
68 | metrics-core
69 | test
70 |
71 |
72 | com.hortonworks.registries
73 | schema-registry-webservice
74 |
75 |
76 | org.slf4j
77 | log4j-over-slf4j
78 |
79 |
80 | org.slf4j
81 | slf4j-log4j12
82 |
83 |
84 | log4j
85 | log4j
86 |
87 |
88 | test
89 |
90 |
91 |
92 |
93 |
94 |
95 |
96 | net.alchim31.maven
97 | scala-maven-plugin
98 | 3.3.2
99 |
100 |
101 |
102 | compile
103 | testCompile
104 |
105 |
106 |
107 | -dependencyfile
108 | ${project.build.directory}/.scala_dependencies
109 |
110 |
111 |
112 |
113 |
114 |
115 | org.apache.maven.plugins
116 | maven-surefire-plugin
117 | 2.21.0
118 |
119 |
120 | true
121 |
122 |
123 |
124 | org.scalatest
125 | scalatest-maven-plugin
126 | 1.0
127 |
128 | ${project.build.directory}/surefire-reports
129 | .
130 | SparkTestSuite.txt
131 |
132 |
133 |
134 | test
135 |
136 | test
137 |
138 |
139 |
140 |
141 |
142 |
143 | org.apache.maven.plugins
144 | maven-shade-plugin
145 |
146 | true
147 |
148 |
149 | *:*
150 |
151 | META-INF/*.SF
152 | META-INF/*.sf
153 | META-INF/*.DSA
154 | META-INF/*.dsa
155 | META-INF/*.RSA
156 | META-INF/*.rsa
157 | META-INF/*.EC
158 | META-INF/*.ec
159 | META-INF/MSFTSIG.SF
160 | META-INF/MSFTSIG.RSA
161 |
162 |
163 |
164 |
165 |
166 | org.apache.avro
167 | com.hortonworks.shaded.org.apache.avro
168 |
169 |
170 | true
171 | jar-with-dependencies
172 |
173 |
174 |
175 | package
176 |
177 | shade
178 |
179 |
180 |
181 |
182 |
183 |
184 |
185 |
186 |
187 |
188 |
189 |
190 |
191 |
192 |
--------------------------------------------------------------------------------
/spark-schema-registry/src/main/scala/com/hortonworks/spark/registry/avro/SchemaConverters.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.hortonworks.spark.registry.avro
19 |
20 | import org.apache.avro.{LogicalTypes, Schema, SchemaBuilder}
21 | import org.apache.avro.LogicalTypes.{Date, Decimal, TimestampMicros, TimestampMillis}
22 | import org.apache.avro.Schema.Type._
23 | import org.apache.spark.sql.catalyst.util.RandomUUIDGenerator
24 | import org.apache.spark.sql.types._
25 |
26 | import scala.collection.JavaConverters._
27 | import scala.util.Random
28 |
29 | /**
30 | * This object contains method that are used to convert sparkSQL schemas to avro schemas and vice
31 | * versa.
32 | * NOTE:
33 | * This is taken from Apache spark master since spark versions <= 2.3
34 | * does not have built in Avro support.
35 | * https://github.com/apache/spark/tree/master/external/avro/src/main/scala/org/apache/spark/sql/avro
36 | * This has some fixes for handling the avro namespaces correctly.
37 | */
38 | object SchemaConverters {
39 | object AvroOutputTimestampType extends Enumeration {
40 | val TIMESTAMP_MICROS, TIMESTAMP_MILLIS = Value
41 | }
42 |
43 | // Max precision of a decimal value stored in `numBytes` bytes
44 | private def maxPrecisionForBytes(numBytes: Int): Int = {
45 | Math.round( // convert double to long
46 | Math.floor(Math.log10( // number of base-10 digits
47 | Math.pow(2, 8 * numBytes - 1) - 1))) // max value stored in numBytes
48 | .asInstanceOf[Int]
49 | }
50 |
51 | // Returns the minimum number of bytes needed to store a decimal with a given `precision`.
52 | private lazy val minBytesForPrecision = Array.tabulate[Int](39)(computeMinBytesForPrecision)
53 |
54 | private def computeMinBytesForPrecision(precision : Int) : Int = {
55 | var numBytes = 1
56 | while (math.pow(2.0, 8 * numBytes - 1) < math.pow(10.0, precision)) {
57 | numBytes += 1
58 | }
59 | numBytes
60 | }
61 |
62 |
63 | private lazy val uuidGenerator = RandomUUIDGenerator(new Random().nextLong())
64 |
65 | private lazy val nullSchema = Schema.create(Schema.Type.NULL)
66 |
67 | case class SchemaType(dataType: DataType, nullable: Boolean)
68 |
69 | /**
70 | * This function takes an avro schema and returns a sql schema.
71 | */
72 | def toSqlType(avroSchema: Schema): SchemaType = {
73 | avroSchema.getType match {
74 | case INT => avroSchema.getLogicalType match {
75 | case _: Date => SchemaType(DateType, nullable = false)
76 | case _ => SchemaType(IntegerType, nullable = false)
77 | }
78 | case STRING => SchemaType(StringType, nullable = false)
79 | case BOOLEAN => SchemaType(BooleanType, nullable = false)
80 | case BYTES | FIXED => avroSchema.getLogicalType match {
81 | // For FIXED type, if the precision requires more bytes than fixed size, the logical
82 | // type will be null, which is handled by Avro library.
83 | case d: Decimal => SchemaType(DecimalType(d.getPrecision, d.getScale), nullable = false)
84 | case _ => SchemaType(BinaryType, nullable = false)
85 | }
86 |
87 | case DOUBLE => SchemaType(DoubleType, nullable = false)
88 | case FLOAT => SchemaType(FloatType, nullable = false)
89 | case LONG => avroSchema.getLogicalType match {
90 | case _: TimestampMillis | _: TimestampMicros => SchemaType(TimestampType, nullable = false)
91 | case _ => SchemaType(LongType, nullable = false)
92 | }
93 |
94 | case ENUM => SchemaType(StringType, nullable = false)
95 |
96 | case RECORD =>
97 | val fields = avroSchema.getFields.asScala.map { f =>
98 | val schemaType = toSqlType(f.schema())
99 | StructField(f.name, schemaType.dataType, schemaType.nullable)
100 | }
101 |
102 | SchemaType(StructType(fields), nullable = false)
103 |
104 | case ARRAY =>
105 | val schemaType = toSqlType(avroSchema.getElementType)
106 | SchemaType(
107 | ArrayType(schemaType.dataType, containsNull = schemaType.nullable),
108 | nullable = false)
109 |
110 | case MAP =>
111 | val schemaType = toSqlType(avroSchema.getValueType)
112 | SchemaType(
113 | MapType(StringType, schemaType.dataType, valueContainsNull = schemaType.nullable),
114 | nullable = false)
115 |
116 | case UNION =>
117 | if (avroSchema.getTypes.asScala.exists(_.getType == NULL)) {
118 | // In case of a union with null, eliminate it and make a recursive call
119 | val remainingUnionTypes = avroSchema.getTypes.asScala.filterNot(_.getType == NULL)
120 | if (remainingUnionTypes.size == 1) {
121 | toSqlType(remainingUnionTypes.head).copy(nullable = true)
122 | } else {
123 | toSqlType(Schema.createUnion(remainingUnionTypes.asJava)).copy(nullable = true)
124 | }
125 | } else avroSchema.getTypes.asScala.map(_.getType) match {
126 | case Seq(t1) =>
127 | toSqlType(avroSchema.getTypes.get(0))
128 | case Seq(t1, t2) if Set(t1, t2) == Set(INT, LONG) =>
129 | SchemaType(LongType, nullable = false)
130 | case Seq(t1, t2) if Set(t1, t2) == Set(FLOAT, DOUBLE) =>
131 | SchemaType(DoubleType, nullable = false)
132 | case _ =>
133 | // Convert complex unions to struct types where field names are member0, member1, etc.
134 | // This is consistent with the behavior when converting between Avro and Parquet.
135 | val fields = avroSchema.getTypes.asScala.zipWithIndex.map {
136 | case (s, i) =>
137 | val schemaType = toSqlType(s)
138 | // All fields are nullable because only one of them is set at a time
139 | StructField(s"member$i", schemaType.dataType, nullable = true)
140 | }
141 |
142 | SchemaType(StructType(fields), nullable = false)
143 | }
144 |
145 | case other => throw new IncompatibleSchemaException(s"Unsupported type $other")
146 | }
147 | }
148 |
149 | def toAvroType(
150 | catalystType: DataType,
151 | nullable: Boolean = false,
152 | recordName: String = "topLevelRecord",
153 | nameSpace: String = "")
154 | : Schema = {
155 | val builder = SchemaBuilder.builder()
156 |
157 | val schema = catalystType match {
158 | case BooleanType => builder.booleanType()
159 | case ByteType | ShortType | IntegerType => builder.intType()
160 | case LongType => builder.longType()
161 | case DateType =>
162 | LogicalTypes.date().addToSchema(builder.intType())
163 | case TimestampType =>
164 | LogicalTypes.timestampMicros().addToSchema(builder.longType())
165 |
166 | case FloatType => builder.floatType()
167 | case DoubleType => builder.doubleType()
168 | case StringType => builder.stringType()
169 | case d: DecimalType =>
170 | val avroType = LogicalTypes.decimal(d.precision, d.scale)
171 | val fixedSize = minBytesForPrecision(d.precision)
172 | // Need to avoid naming conflict for the fixed fields
173 | val name = nameSpace match {
174 | case "" => s"$recordName.fixed"
175 | case _ => s"$nameSpace.$recordName.fixed"
176 | }
177 | avroType.addToSchema(SchemaBuilder.fixed(name).size(fixedSize))
178 |
179 | case BinaryType => builder.bytesType()
180 | case ArrayType(et, containsNull) =>
181 | builder.array()
182 | .items(toAvroType(et, containsNull, recordName, nameSpace))
183 | case MapType(StringType, vt, valueContainsNull) =>
184 | builder.map()
185 | .values(toAvroType(vt, valueContainsNull, recordName, nameSpace))
186 | case st: StructType =>
187 | val childNameSpace = if (nameSpace != "") s"$nameSpace.$recordName" else recordName
188 | val fieldsAssembler = builder.record(recordName).namespace(nameSpace).fields()
189 | st.foreach { f =>
190 | val fieldAvroType =
191 | toAvroType(f.dataType, f.nullable, f.name, childNameSpace)
192 | fieldsAssembler.name(f.name).`type`(fieldAvroType).noDefault()
193 | }
194 | fieldsAssembler.endRecord()
195 |
196 | // This should never happen.
197 | case other => throw new IncompatibleSchemaException(s"Unexpected type $other.")
198 | }
199 | if (nullable) {
200 | Schema.createUnion(schema, nullSchema)
201 | } else {
202 | schema
203 | }
204 | }
205 | }
206 |
207 | class IncompatibleSchemaException(msg: String, ex: Throwable = null) extends Exception(msg, ex)
208 |
--------------------------------------------------------------------------------
/spark-schema-registry/src/main/scala/com/hortonworks/spark/registry/avro/AvroSerializer.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.hortonworks.spark.registry.avro
19 |
20 | import java.nio.ByteBuffer
21 |
22 | import scala.collection.JavaConverters._
23 |
24 | import org.apache.avro.{LogicalTypes, Schema}
25 | import org.apache.avro.Conversions.DecimalConversion
26 | import org.apache.avro.LogicalTypes.{TimestampMicros, TimestampMillis}
27 | import org.apache.avro.Schema
28 | import org.apache.avro.Schema.Type
29 | import org.apache.avro.Schema.Type._
30 | import org.apache.avro.generic.GenericData.{EnumSymbol, Fixed, Record}
31 | import org.apache.avro.generic.GenericData.Record
32 | import org.apache.avro.util.Utf8
33 |
34 | import org.apache.spark.sql.catalyst.InternalRow
35 | import org.apache.spark.sql.catalyst.expressions.{SpecializedGetters, SpecificInternalRow}
36 | import org.apache.spark.sql.types._
37 |
38 |
39 | /**
40 | * A serializer to serialize data in catalyst format to data in avro format.
41 | * NOTE:
42 | * This is taken from Apache spark master since spark versions 2.3
43 | * and below does not have built in Avro support.
44 | * https://github.com/apache/spark/tree/master/external/avro/src/main/scala/org/apache/spark/sql/avro
45 | */
46 | class AvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: Boolean) {
47 |
48 | def serialize(catalystData: Any): Any = {
49 | converter.apply(catalystData)
50 | }
51 |
52 | private val converter: Any => Any = {
53 | val actualAvroType = resolveNullableType(rootAvroType, nullable)
54 | val baseConverter = rootCatalystType match {
55 | case st: StructType =>
56 | newStructConverter(st, actualAvroType).asInstanceOf[Any => Any]
57 | case _ =>
58 | val tmpRow = new SpecificInternalRow(Seq(rootCatalystType))
59 | val converter = newConverter(rootCatalystType, actualAvroType)
60 | (data: Any) =>
61 | tmpRow.update(0, data)
62 | converter.apply(tmpRow, 0)
63 | }
64 | if (nullable) {
65 | (data: Any) =>
66 | if (data == null) {
67 | null
68 | } else {
69 | baseConverter.apply(data)
70 | }
71 | } else {
72 | baseConverter
73 | }
74 | }
75 |
76 | private type Converter = (SpecializedGetters, Int) => Any
77 |
78 | private lazy val decimalConversions = new DecimalConversion()
79 |
80 | private def newConverter(catalystType: DataType, avroType: Schema): Converter = {
81 | (catalystType, avroType.getType) match {
82 | case (NullType, NULL) =>
83 | (getter, ordinal) => null
84 | case (BooleanType, BOOLEAN) =>
85 | (getter, ordinal) => getter.getBoolean(ordinal)
86 | case (ByteType, INT) =>
87 | (getter, ordinal) => getter.getByte(ordinal).toInt
88 | case (ShortType, INT) =>
89 | (getter, ordinal) => getter.getShort(ordinal).toInt
90 | case (IntegerType, INT) =>
91 | (getter, ordinal) => getter.getInt(ordinal)
92 | case (LongType, LONG) =>
93 | (getter, ordinal) => getter.getLong(ordinal)
94 | case (FloatType, FLOAT) =>
95 | (getter, ordinal) => getter.getFloat(ordinal)
96 | case (DoubleType, DOUBLE) =>
97 | (getter, ordinal) => getter.getDouble(ordinal)
98 | case (d: DecimalType, FIXED)
99 | if avroType.getLogicalType == LogicalTypes.decimal(d.precision, d.scale) =>
100 | (getter, ordinal) =>
101 | val decimal = getter.getDecimal(ordinal, d.precision, d.scale)
102 | decimalConversions.toFixed(decimal.toJavaBigDecimal, avroType,
103 | LogicalTypes.decimal(d.precision, d.scale))
104 |
105 | case (d: DecimalType, BYTES)
106 | if avroType.getLogicalType == LogicalTypes.decimal(d.precision, d.scale) =>
107 | (getter, ordinal) =>
108 | val decimal = getter.getDecimal(ordinal, d.precision, d.scale)
109 | decimalConversions.toBytes(decimal.toJavaBigDecimal, avroType,
110 | LogicalTypes.decimal(d.precision, d.scale))
111 |
112 | case (StringType, ENUM) =>
113 | val enumSymbols: Set[String] = avroType.getEnumSymbols.asScala.toSet
114 | (getter, ordinal) =>
115 | val data = getter.getUTF8String(ordinal).toString
116 | if (!enumSymbols.contains(data)) {
117 | throw new IncompatibleSchemaException(
118 | "Cannot write \"" + data + "\" since it's not defined in enum \"" +
119 | enumSymbols.mkString("\", \"") + "\"")
120 | }
121 | new EnumSymbol(avroType, data)
122 |
123 | case (StringType, STRING) =>
124 | (getter, ordinal) => new Utf8(getter.getUTF8String(ordinal).getBytes)
125 |
126 | case (BinaryType, FIXED) =>
127 | val size = avroType.getFixedSize()
128 | (getter, ordinal) =>
129 | val data: Array[Byte] = getter.getBinary(ordinal)
130 | if (data.length != size) {
131 | throw new IncompatibleSchemaException(
132 | s"Cannot write ${data.length} ${if (data.length > 1) "bytes" else "byte"} of " +
133 | "binary data into FIXED Type with size of " +
134 | s"$size ${if (size > 1) "bytes" else "byte"}")
135 | }
136 | new Fixed(avroType, data)
137 |
138 | case (BinaryType, BYTES) =>
139 | (getter, ordinal) => ByteBuffer.wrap(getter.getBinary(ordinal))
140 |
141 | case (DateType, INT) =>
142 | (getter, ordinal) => getter.getInt(ordinal)
143 |
144 | case (TimestampType, LONG) => avroType.getLogicalType match {
145 | case _: TimestampMillis => (getter, ordinal) => getter.getLong(ordinal) / 1000
146 | case _: TimestampMicros => (getter, ordinal) => getter.getLong(ordinal)
147 | // For backward compatibility, if the Avro type is Long and it is not logical type,
148 | // output the timestamp value as with millisecond precision.
149 | case null => (getter, ordinal) => getter.getLong(ordinal) / 1000
150 | case other => throw new IncompatibleSchemaException(
151 | s"Cannot convert Catalyst Timestamp type to Avro logical type ${other}")
152 | }
153 |
154 | case (ArrayType(et, containsNull), ARRAY) =>
155 | val elementConverter = newConverter(
156 | et, resolveNullableType(avroType.getElementType, containsNull))
157 | (getter, ordinal) => {
158 | val arrayData = getter.getArray(ordinal)
159 | val len = arrayData.numElements()
160 | val result = new Array[Any](len)
161 | var i = 0
162 | while (i < len) {
163 | if (containsNull && arrayData.isNullAt(i)) {
164 | result(i) = null
165 | } else {
166 | result(i) = elementConverter(arrayData, i)
167 | }
168 | i += 1
169 | }
170 | // avro writer is expecting a Java Collection, so we convert it into
171 | // `ArrayList` backed by the specified array without data copying.
172 | java.util.Arrays.asList(result: _*)
173 | }
174 |
175 | case (st: StructType, RECORD) =>
176 | val structConverter = newStructConverter(st, avroType)
177 | val numFields = st.length
178 | (getter, ordinal) => structConverter(getter.getStruct(ordinal, numFields))
179 |
180 | case (MapType(kt, vt, valueContainsNull), MAP) if kt == StringType =>
181 | val valueConverter = newConverter(
182 | vt, resolveNullableType(avroType.getValueType, valueContainsNull))
183 | (getter, ordinal) =>
184 | val mapData = getter.getMap(ordinal)
185 | val len = mapData.numElements()
186 | val result = new java.util.HashMap[String, Any](len)
187 | val keyArray = mapData.keyArray()
188 | val valueArray = mapData.valueArray()
189 | var i = 0
190 | while (i < len) {
191 | val key = keyArray.getUTF8String(i).toString
192 | if (valueContainsNull && valueArray.isNullAt(i)) {
193 | result.put(key, null)
194 | } else {
195 | result.put(key, valueConverter(valueArray, i))
196 | }
197 | i += 1
198 | }
199 | result
200 |
201 | case other =>
202 | throw new IncompatibleSchemaException(s"Cannot convert Catalyst type $catalystType to " +
203 | s"Avro type $avroType.")
204 | }
205 | }
206 |
207 | private def newStructConverter(
208 | catalystStruct: StructType, avroStruct: Schema): InternalRow => Record = {
209 | if (avroStruct.getType != RECORD || avroStruct.getFields.size() != catalystStruct.length) {
210 | throw new IncompatibleSchemaException(s"Cannot convert Catalyst type $catalystStruct to " +
211 | s"Avro type $avroStruct.")
212 | }
213 | val fieldConverters = catalystStruct.zip(avroStruct.getFields.asScala).map {
214 | case (f1, f2) => newConverter(f1.dataType, resolveNullableType(f2.schema(), f1.nullable))
215 | }
216 | val numFields = catalystStruct.length
217 | (row: InternalRow) =>
218 | val result = new Record(avroStruct)
219 | var i = 0
220 | while (i < numFields) {
221 | if (row.isNullAt(i)) {
222 | result.put(i, null)
223 | } else {
224 | result.put(i, fieldConverters(i).apply(row, i))
225 | }
226 | i += 1
227 | }
228 | result
229 | }
230 |
231 | private def resolveNullableType(avroType: Schema, nullable: Boolean): Schema = {
232 | if (nullable && avroType.getType != NULL) {
233 | // avro uses union to represent nullable type.
234 | val fields = avroType.getTypes.asScala
235 | assert(fields.length == 2)
236 | val actualType = fields.filter(_.getType != Type.NULL)
237 | assert(actualType.length == 1)
238 | actualType.head
239 | } else {
240 | avroType
241 | }
242 | }
243 | }
244 |
--------------------------------------------------------------------------------
/LICENSE:
--------------------------------------------------------------------------------
1 |
2 | Apache License
3 | Version 2.0, January 2004
4 | http://www.apache.org/licenses/
5 |
6 | TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
7 |
8 | 1. Definitions.
9 |
10 | "License" shall mean the terms and conditions for use, reproduction,
11 | and distribution as defined by Sections 1 through 9 of this document.
12 |
13 | "Licensor" shall mean the copyright owner or entity authorized by
14 | the copyright owner that is granting the License.
15 |
16 | "Legal Entity" shall mean the union of the acting entity and all
17 | other entities that control, are controlled by, or are under common
18 | control with that entity. For the purposes of this definition,
19 | "control" means (i) the power, direct or indirect, to cause the
20 | direction or management of such entity, whether by contract or
21 | otherwise, or (ii) ownership of fifty percent (50%) or more of the
22 | outstanding shares, or (iii) beneficial ownership of such entity.
23 |
24 | "You" (or "Your") shall mean an individual or Legal Entity
25 | exercising permissions granted by this License.
26 |
27 | "Source" form shall mean the preferred form for making modifications,
28 | including but not limited to software source code, documentation
29 | source, and configuration files.
30 |
31 | "Object" form shall mean any form resulting from mechanical
32 | transformation or translation of a Source form, including but
33 | not limited to compiled object code, generated documentation,
34 | and conversions to other media types.
35 |
36 | "Work" shall mean the work of authorship, whether in Source or
37 | Object form, made available under the License, as indicated by a
38 | copyright notice that is included in or attached to the work
39 | (an example is provided in the Appendix below).
40 |
41 | "Derivative Works" shall mean any work, whether in Source or Object
42 | form, that is based on (or derived from) the Work and for which the
43 | editorial revisions, annotations, elaborations, or other modifications
44 | represent, as a whole, an original work of authorship. For the purposes
45 | of this License, Derivative Works shall not include works that remain
46 | separable from, or merely link (or bind by name) to the interfaces of,
47 | the Work and Derivative Works thereof.
48 |
49 | "Contribution" shall mean any work of authorship, including
50 | the original version of the Work and any modifications or additions
51 | to that Work or Derivative Works thereof, that is intentionally
52 | submitted to Licensor for inclusion in the Work by the copyright owner
53 | or by an individual or Legal Entity authorized to submit on behalf of
54 | the copyright owner. For the purposes of this definition, "submitted"
55 | means any form of electronic, verbal, or written communication sent
56 | to the Licensor or its representatives, including but not limited to
57 | communication on electronic mailing lists, source code control systems,
58 | and issue tracking systems that are managed by, or on behalf of, the
59 | Licensor for the purpose of discussing and improving the Work, but
60 | excluding communication that is conspicuously marked or otherwise
61 | designated in writing by the copyright owner as "Not a Contribution."
62 |
63 | "Contributor" shall mean Licensor and any individual or Legal Entity
64 | on behalf of whom a Contribution has been received by Licensor and
65 | subsequently incorporated within the Work.
66 |
67 | 2. Grant of Copyright License. Subject to the terms and conditions of
68 | this License, each Contributor hereby grants to You a perpetual,
69 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable
70 | copyright license to reproduce, prepare Derivative Works of,
71 | publicly display, publicly perform, sublicense, and distribute the
72 | Work and such Derivative Works in Source or Object form.
73 |
74 | 3. Grant of Patent License. Subject to the terms and conditions of
75 | this License, each Contributor hereby grants to You a perpetual,
76 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable
77 | (except as stated in this section) patent license to make, have made,
78 | use, offer to sell, sell, import, and otherwise transfer the Work,
79 | where such license applies only to those patent claims licensable
80 | by such Contributor that are necessarily infringed by their
81 | Contribution(s) alone or by combination of their Contribution(s)
82 | with the Work to which such Contribution(s) was submitted. If You
83 | institute patent litigation against any entity (including a
84 | cross-claim or counterclaim in a lawsuit) alleging that the Work
85 | or a Contribution incorporated within the Work constitutes direct
86 | or contributory patent infringement, then any patent licenses
87 | granted to You under this License for that Work shall terminate
88 | as of the date such litigation is filed.
89 |
90 | 4. Redistribution. You may reproduce and distribute copies of the
91 | Work or Derivative Works thereof in any medium, with or without
92 | modifications, and in Source or Object form, provided that You
93 | meet the following conditions:
94 |
95 | (a) You must give any other recipients of the Work or
96 | Derivative Works a copy of this License; and
97 |
98 | (b) You must cause any modified files to carry prominent notices
99 | stating that You changed the files; and
100 |
101 | (c) You must retain, in the Source form of any Derivative Works
102 | that You distribute, all copyright, patent, trademark, and
103 | attribution notices from the Source form of the Work,
104 | excluding those notices that do not pertain to any part of
105 | the Derivative Works; and
106 |
107 | (d) If the Work includes a "NOTICE" text file as part of its
108 | distribution, then any Derivative Works that You distribute must
109 | include a readable copy of the attribution notices contained
110 | within such NOTICE file, excluding those notices that do not
111 | pertain to any part of the Derivative Works, in at least one
112 | of the following places: within a NOTICE text file distributed
113 | as part of the Derivative Works; within the Source form or
114 | documentation, if provided along with the Derivative Works; or,
115 | within a display generated by the Derivative Works, if and
116 | wherever such third-party notices normally appear. The contents
117 | of the NOTICE file are for informational purposes only and
118 | do not modify the License. You may add Your own attribution
119 | notices within Derivative Works that You distribute, alongside
120 | or as an addendum to the NOTICE text from the Work, provided
121 | that such additional attribution notices cannot be construed
122 | as modifying the License.
123 |
124 | You may add Your own copyright statement to Your modifications and
125 | may provide additional or different license terms and conditions
126 | for use, reproduction, or distribution of Your modifications, or
127 | for any such Derivative Works as a whole, provided Your use,
128 | reproduction, and distribution of the Work otherwise complies with
129 | the conditions stated in this License.
130 |
131 | 5. Submission of Contributions. Unless You explicitly state otherwise,
132 | any Contribution intentionally submitted for inclusion in the Work
133 | by You to the Licensor shall be under the terms and conditions of
134 | this License, without any additional terms or conditions.
135 | Notwithstanding the above, nothing herein shall supersede or modify
136 | the terms of any separate license agreement you may have executed
137 | with Licensor regarding such Contributions.
138 |
139 | 6. Trademarks. This License does not grant permission to use the trade
140 | names, trademarks, service marks, or product names of the Licensor,
141 | except as required for reasonable and customary use in describing the
142 | origin of the Work and reproducing the content of the NOTICE file.
143 |
144 | 7. Disclaimer of Warranty. Unless required by applicable law or
145 | agreed to in writing, Licensor provides the Work (and each
146 | Contributor provides its Contributions) on an "AS IS" BASIS,
147 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
148 | implied, including, without limitation, any warranties or conditions
149 | of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
150 | PARTICULAR PURPOSE. You are solely responsible for determining the
151 | appropriateness of using or redistributing the Work and assume any
152 | risks associated with Your exercise of permissions under this License.
153 |
154 | 8. Limitation of Liability. In no event and under no legal theory,
155 | whether in tort (including negligence), contract, or otherwise,
156 | unless required by applicable law (such as deliberate and grossly
157 | negligent acts) or agreed to in writing, shall any Contributor be
158 | liable to You for damages, including any direct, indirect, special,
159 | incidental, or consequential damages of any character arising as a
160 | result of this License or out of the use or inability to use the
161 | Work (including but not limited to damages for loss of goodwill,
162 | work stoppage, computer failure or malfunction, or any and all
163 | other commercial damages or losses), even if such Contributor
164 | has been advised of the possibility of such damages.
165 |
166 | 9. Accepting Warranty or Additional Liability. While redistributing
167 | the Work or Derivative Works thereof, You may choose to offer,
168 | and charge a fee for, acceptance of support, warranty, indemnity,
169 | or other liability obligations and/or rights consistent with this
170 | License. However, in accepting such obligations, You may act only
171 | on Your own behalf and on Your sole responsibility, not on behalf
172 | of any other Contributor, and only if You agree to indemnify,
173 | defend, and hold each Contributor harmless for any liability
174 | incurred by, or claims asserted against, such Contributor by reason
175 | of your accepting any such warranty or additional liability.
176 |
177 | END OF TERMS AND CONDITIONS
178 |
--------------------------------------------------------------------------------
/README.md:
--------------------------------------------------------------------------------
1 | [](https://travis-ci.org/hortonworks-spark/spark-schema-registry)
2 |
3 | # Apache Spark - Schema Registry integration
4 |
5 | The [Apache Spark](https://spark.apache.org/) - [Schema Registry](https://github.com/hortonworks/registry) integration is a library to
6 | leverage Schema registry for managing Spark schemas and to serialize/de-serialize messages in spark data sources and sinks.
7 |
8 | ### Compiling
9 |
10 | mvn clean package
11 |
12 | ### Running the example programs
13 |
14 | The [examples](examples/src/main/scala/com/hortonworks/spark/registry/examples/) illustrates the API usage and how to integrate with schema registry.
15 | The examples can be run from IDE (e.g. Intellij) by specifying a master URL or via spark-submit.
16 |
17 | spark-submit --master \
18 | --jars spark-schema-registry-0.1-SNAPSHOT-jar-with-dependencies.jar \
19 | --packages org.apache.spark:spark-sql-kafka-0-10_2.11:2.3.1 \
20 | --class com.hortonworks.spark.registry.examples. \
21 | spark-schema-registry-examples-0.1-SNAPSHOT.jar \
22 |
23 |
24 | ### Using the APIs
25 |
26 | Typically in a spark application you need to define the spark schema for the data you are going to process.
27 |
28 | ```scala
29 | // the schema for truck events
30 | val schema = StructType(Seq(
31 | StructField("driverId", IntegerType, nullable = false),
32 | StructField("truckId", IntegerType, nullable = false),
33 | StructField("miles", LongType, nullable = false),
34 | StructField("eventType", StringType, nullable = false),
35 | ...
36 | ...
37 | )
38 |
39 | // read Json string messages from the data source
40 | val messages = spark
41 | .readStream
42 | .format(...)
43 | .option(...)
44 | .load()
45 |
46 | // parse the messages using the above schema and do further operations
47 | val df = messages
48 | .select(from_json($"value".cast("string"), schema).alias("value"))
49 | ...
50 |
51 | // project (driverId, truckId, miles) for the events where miles > 300
52 | val filtered = df.select($"value.driverId", $"value.truckId", $"value.miles")
53 | .where("value.miles > 300")
54 | ```
55 |
56 | The above approach is brittle since the schema information is tightly coupled with the code. The code needs to be changed
57 | when the schema changes. Also there is no sharing or re-use of schema between the message producers and the applications
58 | that wishes to consume the messages. Schema registry also allows you to manage different versions of the schema and define
59 | compatibility policies.
60 |
61 | #### Configuration
62 |
63 | The Schema registry integration comes as utility methods which can be imported into the scope.
64 |
65 | ```scala
66 | import com.hortonworks.spark.registry.util._
67 | ```
68 |
69 | Before invoking the APIs, you need to define an implicit `SchemaRegistryConfig` which will be passed to the APIs. The main
70 | configuration here is the schema registry URL.
71 |
72 | ```scala
73 | // the schema registry client config
74 | val config = Map[String, Object]("schema.registry.url" -> schemaRegistryUrl)
75 |
76 | // the schema registry config that will be implicitly passed
77 | implicit val srConfig:SchemaRegistryConfig = SchemaRegistryConfig(config)
78 | ```
79 | #### Fetching spark schema by name
80 |
81 | The API supports fetching the Schema Registry schema as a Spark schema.
82 |
83 |
84 | - `sparkSchema(schemaName: String)`
85 |
86 | Returns the spark schema corresponding to the latest version of schema defined in the schema registry.
87 |
88 | - `sparkSchema(schemaName: String, version: Int)`
89 |
90 | Returns the spark schema corresponding to the given version of schema defined in the schema registry.
91 |
92 | The example discussed above can by simplified as follows without having to explicitly specify the spark schema
93 | in the code.
94 |
95 | ```scala
96 | // retrieve the translated "Spark schema" by specifying the schema registry schema name
97 | val schema = sparkSchema(name)
98 |
99 | // parse the messages using the above schema and do further operations
100 | val df = messages
101 | .select(from_json($"value".cast("string"), schema).alias("value"))
102 | ...
103 |
104 | // project (driverId, truckId, miles) for the events where miles > 300
105 | val filtered = df.select($"value.driverId", $"value.truckId", $"value.miles")
106 | .where("value.miles > 300")
107 |
108 | ```
109 |
110 | #### Serializing messages using schema registry
111 |
112 | The following method can be used to serialize the messages from spark to schema registry binary format
113 | using schema registry serializers.
114 |
115 | - `to_sr(data: Column, schemaName: String, topLevelRecordName: String, namespace: String)`
116 |
117 | Converts a spark column data to binary format of schema registry. This looks up a schema registry schema
118 | for the `schemaName` that matches the input and automatically registers a new schema if not found.
119 | The `topoLevelRecordName` and `namespace` are optional and will be mapped to avro top level record name
120 | and record namespace.
121 |
122 | #### De-serializing messages using schema registry
123 |
124 | The following methods can be used to de-serialize schema registry serialized messages into spark columns.
125 |
126 | - `from_sr(data: Column, schemaName: String)`
127 |
128 | Converts schema registry binary format to spark column. This uses the latest version of the schema.
129 |
130 | - `from_sr(data: Column, schemaName: String, version: Int)`
131 |
132 | Converts schema registry binary format to spark column using the given schema registry schema name and version.
133 |
134 | #### Serialization - deserialization example
135 |
136 | Here is an example that uses the `from_sr` to de-serialize schema registry formatted messages into spark,
137 | transforms and serializes it back to schema registry format using `to_sr` and writes to a data sink.
138 |
139 | This example assumes Spark structured streaming, but this should work well for the non-streaming use cases as well (read and write).
140 |
141 | ```scala
142 | // Read schema registry formatted messages and deserialize to spark columns.
143 | val df = messages
144 | .select(from_sr($"value", topic).alias("message"))
145 |
146 | // project (driverId, truckId, miles) for the events where miles > 300
147 | val filtered = df.select($"message.driverId", $"message.truckId", $"message.miles")
148 | .where("message.miles > 300")
149 |
150 | // write the output as schema registry serialized bytes to a sink
151 | // should produce events like {"driverId":14,"truckId":25,"miles":373}
152 | val query = filtered
153 | .select(to_sr(struct($"*"), outSchemaName).alias("value"))
154 | .writeStream
155 | .format(..)
156 | .start()
157 | ```
158 |
159 | The output schema `outSchemaName` would be automatically published to schema registry if it does not exist.
160 |
161 | ### Building and deploying your app
162 |
163 | Add a maven dependency in your project to make use of the library and build your application jar.
164 |
165 |
166 | com.hortonworks
167 | spark-schema-registry
168 | 0.1-SNAPSHOT
169 |
170 |
171 | Once the application jar is built, you can deploy it by adding the dependency in spark-submit via `--packages`
172 |
173 | spark-submit --master \
174 | --packages com.hortonworks:spark-schema-registry:0.1-SNAPSHOT \
175 | --class YourApp \
176 | your-application-jar \
177 | args ...
178 |
179 | To make it work, you should make sure this package is published in some repositories or exists in your local repository.
180 |
181 | If this package is not published to repository or your Spark application cannot access external network, you could use uber jar
182 | instead, like:
183 |
184 | spark-submit --master \
185 | --jars spark-schema-registry-0.1-SNAPSHOT-jar-with-dependencies.jar \
186 | --class YourApp \
187 | your-application-jar \
188 | args ...
189 |
190 | ### Running in a Kerberos enabled cluster
191 |
192 | The library works in a Kerberos set up where Spark and Schema registry has been deployed on a Kerberos
193 | enabled cluster.
194 |
195 | The main thing is to setup the appropriate `JAAS` config for the `RegistryClient` (and `KafkaClient` if the spark
196 | data source or sink is Kafka). As an example, to run the `SchemaRegistryAvroExample` in a Kerberos set up,
197 |
198 | 1. Create a keytab (say app.keytab) with the login user and principal you want to run the Application.
199 | 2. Create an app_jaas.conf and specify the keytab and principal created in step 1.
200 |
201 | (if deploying to YARN, the keytab and conf files will be distributed as YARN local resources. They will end up
202 | in the current directory of the Spark YARN container and the location should be specified as ./app.keytab)
203 |
204 | ```
205 | RegistryClient {
206 | com.sun.security.auth.module.Krb5LoginModule required
207 | useKeyTab=true
208 | keyTab="./app.keytab"
209 | storeKey=true
210 | useTicketCache=false
211 | principal="";
212 | };
213 |
214 | KafkaClient {
215 | com.sun.security.auth.module.Krb5LoginModule required
216 | useKeyTab=true
217 | keyTab="./app.keytab"
218 | storeKey=true
219 | useTicketCache=false
220 | serviceName="kafka"
221 | principal="";
222 | };
223 |
224 | ```
225 |
226 | 4. Provide the required ACLs for the kafka topics (in-topic, out-topic) for the principal.
227 |
228 | 5. In `spark-submit` pass the jaas configuration file via `extraJavaOptions` (and also as local resource files in YARN cluster mode)
229 |
230 | ```
231 | spark-submit --master yarn --deploy-mode cluster \
232 | --keytab app.keytab --principal \
233 | --files app_jaas.conf#app_jaas.conf,app.keytab#app.keytab \
234 | --jars spark-schema-registry-0.1-SNAPSHOT-jar-with-dependencies.jar \
235 | --packages org.apache.spark:spark-sql-kafka-0-10_2.11:2.3.1 \
236 | --conf "spark.executor.extraJavaOptions=-Djava.security.auth.login.config=./app_jaas.conf" \
237 | --conf "spark.driver.extraJavaOptions=-Djava.security.auth.login.config=./app_jaas.conf" \
238 | --class com.hortonworks.spark.registry.examples.SchemaRegistryAvroExample
239 | spark-schema-registry-examples-0.1-SNAPSHOT.jar \
240 | SASL_PLAINTEXT
241 | ```
242 |
--------------------------------------------------------------------------------
/spark-schema-registry/src/test/scala/com/hortonworks/spark/registry/util/SchemaRegistryUtilSuite.scala:
--------------------------------------------------------------------------------
1 | package com.hortonworks.spark.registry.util
2 |
3 | import java.io.{ByteArrayInputStream, File}
4 |
5 | import com.google.common.base.Charsets
6 | import com.google.common.io.Resources
7 | import com.hortonworks.registries.schemaregistry.{SchemaCompatibility, SchemaMetadata, SchemaVersion}
8 | import com.hortonworks.registries.schemaregistry.avro.AvroSchemaProvider
9 | import com.hortonworks.registries.schemaregistry.client.SchemaRegistryClient
10 | import com.hortonworks.registries.schemaregistry.serdes.avro.{AvroSnapshotDeserializer, AvroSnapshotSerializer}
11 | import com.hortonworks.registries.schemaregistry.webservice.LocalSchemaRegistryServer
12 | import org.apache.avro.Schema
13 | import org.apache.avro.generic.GenericData
14 | import org.apache.avro.util.Utf8
15 | import org.apache.spark.sql.SparkSession
16 | import org.apache.spark.sql.functions._
17 | import org.apache.spark.sql.types.{StructField, _}
18 | import org.scalatest.{BeforeAndAfterAll, FunSuite}
19 |
20 | import scala.collection.JavaConverters._
21 |
22 | case class Truck(driverId: Long, driverName: String, miles: Long)
23 | case class TruckV2(driverId: Long, driverName: String, miles: Long, latitude: Double, longitude: Double)
24 |
25 | class SchemaRegistryUtilSuite extends FunSuite with BeforeAndAfterAll {
26 |
27 | var localSchemaRegistryServer: LocalSchemaRegistryServer = _
28 | implicit var registryConfig: SchemaRegistryConfig = _
29 | var trucksV1Schema: String = _
30 | var trucksV2Schema: String = _
31 | var numbersSchema: String = _
32 | var srClient: SchemaRegistryClient = _
33 | var trucksSchemaMetadata: SchemaMetadata = _
34 | var numbersSchemaMetadata: SchemaMetadata = _
35 |
36 | override protected def beforeAll(): Unit = {
37 | val configPath = new File(Resources.getResource("schema-registry-test.yaml").toURI).getAbsolutePath
38 | val localSchemaRegistryServer = new LocalSchemaRegistryServer(configPath)
39 | localSchemaRegistryServer.start()
40 | val srUrl = localSchemaRegistryServer.getLocalURL + "api/v1"
41 | registryConfig = SchemaRegistryConfig(Map(SchemaRegistryClient.Configuration.SCHEMA_REGISTRY_URL.name -> srUrl))
42 | srClient = new SchemaRegistryClient(registryConfig.config.asJava)
43 | trucksSchemaMetadata = new SchemaMetadata.Builder("trucks").
44 | `type`(AvroSchemaProvider.TYPE)
45 | .schemaGroup("trucks-group")
46 | .description("Trucks schema")
47 | .compatibility(SchemaCompatibility.BACKWARD).build
48 | numbersSchemaMetadata = new SchemaMetadata.Builder("numbers").
49 | `type`(AvroSchemaProvider.TYPE)
50 | .schemaGroup("numbers-group")
51 | .description("Numbers schema")
52 | .compatibility(SchemaCompatibility.BACKWARD).build
53 | srClient.addSchemaMetadata(trucksSchemaMetadata)
54 | trucksV1Schema = Resources.toString(Resources.getResource("trucks-v1.avsc"), Charsets.UTF_8)
55 | srClient.addSchemaVersion(trucksSchemaMetadata, new SchemaVersion(trucksV1Schema, "v1"))
56 | trucksV2Schema = Resources.toString(Resources.getResource("trucks-v2.avsc"), Charsets.UTF_8)
57 | srClient.addSchemaVersion(trucksSchemaMetadata, new SchemaVersion(trucksV2Schema, "v2"))
58 | numbersSchema = Resources.toString(Resources.getResource("numbers.avsc"), Charsets.UTF_8)
59 | srClient.addSchemaVersion(numbersSchemaMetadata, new SchemaVersion(numbersSchema, "v1"))
60 | }
61 |
62 |
63 | test("test schema text - latest version") {
64 | val spark = SparkSession
65 | .builder
66 | .master("local[2]")
67 | .getOrCreate()
68 | val schema = schemaText("trucks")
69 | assert(trucksV2Schema == schema)
70 | }
71 |
72 | test("test schema text - specific version") {
73 | val spark = SparkSession
74 | .builder
75 | .master("local[2]")
76 | .getOrCreate()
77 | val schema = schemaText("trucks", 1)
78 | assert(trucksV1Schema == schema)
79 | }
80 |
81 |
82 | test("test spark schema - latest version") {
83 | val spark = SparkSession
84 | .builder
85 | .master("local[2]")
86 | .getOrCreate()
87 | val schema = sparkSchema("trucks")
88 | assert(sparkSchemaV2 == schema)
89 | }
90 |
91 | test("test spark schema - specific version") {
92 | val spark = SparkSession
93 | .builder
94 | .master("local[2]")
95 | .getOrCreate()
96 | val schema = sparkSchema("trucks", 1)
97 | assert(sparkSchemaV1 == schema)
98 | }
99 |
100 | test("test to_sr") {
101 | val spark = SparkSession
102 | .builder
103 | .master("local[2]")
104 | .getOrCreate()
105 |
106 | val trucks = Seq(
107 | Truck(1, "driver_1", 100),
108 | Truck(2, "driver_2", 200),
109 | Truck(3, "driver_3", 300),
110 | Truck(4, "driver_4", 400),
111 | Truck(5, "driver_5", 500)
112 | )
113 |
114 | // serialize the rows using spark
115 | import spark.implicits._
116 | val rows = trucks.toDF()
117 | .select(to_sr(struct($"*"), "events").alias("events"))
118 | .collect().map(r => r.get(0).asInstanceOf[Array[Byte]]).toList
119 |
120 | // SR should register the 'events' schema automatically
121 | val versions = srClient.getAllVersions("events")
122 | assert(versions.size() == 1)
123 |
124 | // de-serialize the rows using SR deserializer
125 | val srDeser = new AvroSnapshotDeserializer()
126 | srDeser.init(registryConfig.config.asJava)
127 | val result = rows.map(row => srDeser.deserialize(new ByteArrayInputStream(row), 1))
128 | .map(r => r.asInstanceOf[GenericData.Record])
129 | .map(t => Truck(
130 | t.get("driverId").asInstanceOf[Long],
131 | t.get("driverName").asInstanceOf[Utf8].toString,
132 | t.get("miles").asInstanceOf[Long]))
133 |
134 | assert(trucks == result)
135 |
136 | }
137 |
138 | test("test to_sr with registered schema") {
139 | val spark = SparkSession
140 | .builder
141 | .master("local[2]")
142 | .getOrCreate()
143 |
144 | val trucks = Seq(
145 | Truck(1, "driver_1", 100),
146 | Truck(2, "driver_2", 200),
147 | Truck(3, "driver_3", 300),
148 | Truck(4, "driver_4", 400),
149 | Truck(5, "driver_5", 500)
150 | )
151 |
152 | // serialize the rows using spark
153 | import spark.implicits._
154 | val rows = trucks.toDF()
155 | .select(to_sr(struct($"*"), "trucks", "trucks").alias("events"))
156 | .collect().map(r => r.get(0).asInstanceOf[Array[Byte]]).toList
157 |
158 | // de-serialize the rows using SR deserializer
159 | val srDeser = new AvroSnapshotDeserializer()
160 | srDeser.init(registryConfig.config.asJava)
161 | val result = rows.map(row => srDeser.deserialize(new ByteArrayInputStream(row), 1))
162 | .map(r => r.asInstanceOf[GenericData.Record])
163 | .map(t => Truck(
164 | t.get("driverId").asInstanceOf[Long],
165 | "driver_" + t.get("driverId").asInstanceOf[Long],
166 | t.get("miles").asInstanceOf[Long]))
167 |
168 | assert(trucks == result)
169 |
170 | val versions = srClient.getAllVersions("trucks")
171 |
172 | assert(versions.size() == 2)
173 | }
174 |
175 |
176 | test("test from_sr specific version") {
177 | val spark = SparkSession
178 | .builder
179 | .master("local[2]")
180 | .getOrCreate()
181 |
182 | val trucks = Seq(
183 | Truck(1, "driver_1", 100),
184 | Truck(2, "driver_2", 200),
185 | Truck(3, "driver_3", 300),
186 | Truck(4, "driver_4", 400),
187 | Truck(5, "driver_5", 500)
188 | )
189 |
190 | // serialize the rows using SR serializer
191 | val srSer = new AvroSnapshotSerializer()
192 | srSer.init(registryConfig.config.asJava)
193 | val serialized = trucks.map(t => {
194 | val avroRec = new GenericData.Record(new Schema.Parser().parse(trucksV1Schema))
195 | avroRec.put("driverId", t.driverId)
196 | avroRec.put("driverName", t.driverName)
197 | avroRec.put("miles", t.miles)
198 | srSer.serialize(avroRec, trucksSchemaMetadata)
199 | }).asInstanceOf[List[Array[Byte]]]
200 |
201 | // de-serialize using spark
202 | import spark.implicits._
203 | val result = serialized.toDF("value")
204 | .select(from_sr($"value", "trucks", 1).alias("t"))
205 | .select($"t.driverId", $"t.driverName", $"t.miles")
206 | .as[Truck].collect().toList
207 |
208 | assert(trucks == result)
209 | }
210 |
211 |
212 | test("test from_sr latest version") {
213 | val spark = SparkSession
214 | .builder
215 | .master("local[2]")
216 | .getOrCreate()
217 |
218 | val trucks = Seq(
219 | TruckV2(1, "driver_1", 100, 1.0, 1.0),
220 | TruckV2(2, "driver_2", 200, 2.0, 2.0),
221 | TruckV2(3, "driver_3", 300, 3.0, 3.0),
222 | TruckV2(4, "driver_4", 400, 4.0, 4.0),
223 | TruckV2(5, "driver_5", 500, 5.0, 5.0)
224 | )
225 |
226 | // serialize the rows using SR serializer
227 | val srSer = new AvroSnapshotSerializer()
228 | srSer.init(registryConfig.config.asJava)
229 | val serialized = trucks.map(t => {
230 | val avroRec = new GenericData.Record(new Schema.Parser().parse(trucksV2Schema))
231 | avroRec.put("driverId", t.driverId)
232 | avroRec.put("driverName", t.driverName)
233 | avroRec.put("miles", t.miles)
234 | avroRec.put("latitude", t.latitude)
235 | avroRec.put("longitude", t.longitude)
236 | srSer.serialize(avroRec, trucksSchemaMetadata)
237 | }).asInstanceOf[List[Array[Byte]]]
238 |
239 | // de-serialize using spark
240 | import spark.implicits._
241 | val result = serialized.toDF("value")
242 | .select(from_sr($"value", "trucks").alias("t"))
243 | .select($"t.driverId", $"t.driverName", $"t.miles", $"t.latitude", $"t.longitude")
244 | .as[TruckV2].collect().toList
245 |
246 | assert(trucks == result)
247 | }
248 |
249 | test("test primitive ser") {
250 | val spark = SparkSession
251 | .builder
252 | .master("local[2]")
253 | .getOrCreate()
254 |
255 | // serialize the rows using spark
256 | import spark.implicits._
257 | val numbers = List(1, 2, 3, 4, 5)
258 | val rows = numbers.toDF("value")
259 | .select(to_sr($"value", "numbers").alias("numbers"))
260 | .collect().map(r => r.get(0).asInstanceOf[Array[Byte]]).toList
261 |
262 | // SR should register the 'numbers' schema automatically
263 | val versions = srClient.getAllVersions("numbers")
264 | assert(versions.size() == 1)
265 |
266 | // de-serialize the rows using SR deserializer
267 | val srDeser = new AvroSnapshotDeserializer()
268 | srDeser.init(registryConfig.config.asJava)
269 | val result = rows.map(row => srDeser.deserialize(new ByteArrayInputStream(row), 1))
270 | .map(r => r.asInstanceOf[Int])
271 | assert(numbers == result)
272 | }
273 |
274 | test("test primitive deser") {
275 | val spark = SparkSession
276 | .builder
277 | .master("local[2]")
278 | .getOrCreate()
279 |
280 | val numbers = List(1, 2, 3, 4, 5)
281 | // serialize the rows using SR serializer
282 | val srSer = new AvroSnapshotSerializer()
283 | srSer.init(registryConfig.config.asJava)
284 | val serialized = numbers.map(n => srSer.serialize(Integer.valueOf(n), numbersSchemaMetadata))
285 | .asInstanceOf[List[Array[Byte]]]
286 |
287 | // de-serialize using spark
288 | import spark.implicits._
289 | val result = serialized.toDF("value")
290 | .select(from_sr($"value", "numbers").alias("n"))
291 | .as[Int].collect().toList
292 |
293 | assert(numbers == result)
294 | }
295 |
296 | private val sparkSchemaV1 = StructType(Seq(
297 | StructField("driverId", LongType, nullable = false),
298 | StructField("driverName", StringType, nullable = true),
299 | StructField("miles", LongType, nullable = false)
300 | ))
301 |
302 | private val sparkSchemaV2 = StructType(Seq(
303 | StructField("driverId", LongType, nullable = false),
304 | StructField("driverName", StringType, nullable = true),
305 | StructField("miles", LongType, nullable = false),
306 | StructField("latitude", DoubleType, nullable = false),
307 | StructField("longitude", DoubleType, nullable = false)
308 | ))
309 |
310 | }
311 |
--------------------------------------------------------------------------------
/spark-schema-registry/src/main/scala/com/hortonworks/spark/registry/avro/AvroDeserializer.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.hortonworks.spark.registry.avro
19 |
20 | import java.math.BigDecimal
21 | import java.nio.ByteBuffer
22 |
23 | import scala.collection.JavaConverters._
24 | import scala.collection.mutable.ArrayBuffer
25 |
26 | import org.apache.avro.{LogicalTypes, Schema, SchemaBuilder}
27 | import org.apache.avro.Conversions.DecimalConversion
28 | import org.apache.avro.LogicalTypes.{TimestampMicros, TimestampMillis}
29 | import org.apache.avro.Schema.Type._
30 | import org.apache.avro.generic._
31 | import org.apache.avro.util.Utf8
32 |
33 | import org.apache.spark.sql.catalyst.InternalRow
34 | import org.apache.spark.sql.catalyst.expressions.{SpecificInternalRow, UnsafeArrayData}
35 | import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, DateTimeUtils, GenericArrayData}
36 | import org.apache.spark.sql.types._
37 | import org.apache.spark.unsafe.types.UTF8String
38 |
39 | /**
40 | * A deserializer to deserialize data in avro format to data in catalyst format.
41 | * NOTE:
42 | * This is taken from Apache spark master since spark versions 2.3
43 | * and below does not have built in Avro support.
44 | * https://github.com/apache/spark/tree/master/external/avro/src/main/scala/org/apache/spark/sql/avro
45 | */
46 | class AvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType) {
47 | private lazy val decimalConversions = new DecimalConversion()
48 |
49 | private val converter: Any => Any = rootCatalystType match {
50 | // A shortcut for empty schema.
51 | case st: StructType if st.isEmpty =>
52 | (data: Any) => InternalRow.empty
53 |
54 | case st: StructType =>
55 | val resultRow = new SpecificInternalRow(st.map(_.dataType))
56 | val fieldUpdater = new RowUpdater(resultRow)
57 | val writer = getRecordWriter(rootAvroType, st, Nil)
58 | (data: Any) => {
59 | val record = data.asInstanceOf[GenericRecord]
60 | writer(fieldUpdater, record)
61 | resultRow
62 | }
63 |
64 | case _ =>
65 | val tmpRow = new SpecificInternalRow(Seq(rootCatalystType))
66 | val fieldUpdater = new RowUpdater(tmpRow)
67 | val writer = newWriter(rootAvroType, rootCatalystType, Nil)
68 | (data: Any) => {
69 | writer(fieldUpdater, 0, data)
70 | tmpRow.get(0, rootCatalystType)
71 | }
72 | }
73 |
74 | def deserialize(data: Any): Any = converter(data)
75 |
76 | /**
77 | * Creates a writer to write avro values to Catalyst values at the given ordinal with the given
78 | * updater.
79 | */
80 | private def newWriter(
81 | avroType: Schema,
82 | catalystType: DataType,
83 | path: List[String]): (CatalystDataUpdater, Int, Any) => Unit =
84 | (avroType.getType, catalystType) match {
85 | case (NULL, NullType) => (updater, ordinal, _) =>
86 | updater.setNullAt(ordinal)
87 |
88 | // TODO: we can avoid boxing if future version of avro provide primitive accessors.
89 | case (BOOLEAN, BooleanType) => (updater, ordinal, value) =>
90 | updater.setBoolean(ordinal, value.asInstanceOf[Boolean])
91 |
92 | case (INT, IntegerType) => (updater, ordinal, value) =>
93 | updater.setInt(ordinal, value.asInstanceOf[Int])
94 |
95 | case (INT, DateType) => (updater, ordinal, value) =>
96 | updater.setInt(ordinal, value.asInstanceOf[Int])
97 |
98 | case (LONG, LongType) => (updater, ordinal, value) =>
99 | updater.setLong(ordinal, value.asInstanceOf[Long])
100 |
101 | case (LONG, TimestampType) => avroType.getLogicalType match {
102 | case _: TimestampMillis => (updater, ordinal, value) =>
103 | updater.setLong(ordinal, value.asInstanceOf[Long] * 1000)
104 | case _: TimestampMicros => (updater, ordinal, value) =>
105 | updater.setLong(ordinal, value.asInstanceOf[Long])
106 | case null => (updater, ordinal, value) =>
107 | // For backward compatibility, if the Avro type is Long and it is not logical type,
108 | // the value is processed as timestamp type with millisecond precision.
109 | updater.setLong(ordinal, value.asInstanceOf[Long] * 1000)
110 | case other => throw new IncompatibleSchemaException(
111 | s"Cannot convert Avro logical type ${other} to Catalyst Timestamp type.")
112 | }
113 |
114 | // Before we upgrade Avro to 1.8 for logical type support, spark-avro converts Long to Date.
115 | // For backward compatibility, we still keep this conversion.
116 | case (LONG, DateType) => (updater, ordinal, value) =>
117 | updater.setInt(ordinal, (value.asInstanceOf[Long] / DateTimeUtils.MILLIS_PER_DAY).toInt)
118 |
119 | case (FLOAT, FloatType) => (updater, ordinal, value) =>
120 | updater.setFloat(ordinal, value.asInstanceOf[Float])
121 |
122 | case (DOUBLE, DoubleType) => (updater, ordinal, value) =>
123 | updater.setDouble(ordinal, value.asInstanceOf[Double])
124 |
125 | case (STRING, StringType) => (updater, ordinal, value) =>
126 | val str = value match {
127 | case s: String => UTF8String.fromString(s)
128 | case s: Utf8 =>
129 | val bytes = new Array[Byte](s.getByteLength)
130 | System.arraycopy(s.getBytes, 0, bytes, 0, s.getByteLength)
131 | UTF8String.fromBytes(bytes)
132 | }
133 | updater.set(ordinal, str)
134 |
135 | case (ENUM, StringType) => (updater, ordinal, value) =>
136 | updater.set(ordinal, UTF8String.fromString(value.toString))
137 |
138 | case (FIXED, BinaryType) => (updater, ordinal, value) =>
139 | updater.set(ordinal, value.asInstanceOf[GenericFixed].bytes().clone())
140 |
141 | case (BYTES, BinaryType) => (updater, ordinal, value) =>
142 | val bytes = value match {
143 | case b: ByteBuffer =>
144 | val bytes = new Array[Byte](b.remaining)
145 | b.get(bytes)
146 | bytes
147 | case b: Array[Byte] => b
148 | case other => throw new RuntimeException(s"$other is not a valid avro binary.")
149 | }
150 | updater.set(ordinal, bytes)
151 |
152 | case (FIXED, d: DecimalType) => (updater, ordinal, value) =>
153 | val bigDecimal = decimalConversions.fromFixed(value.asInstanceOf[GenericFixed], avroType,
154 | LogicalTypes.decimal(d.precision, d.scale))
155 | val decimal = createDecimal(bigDecimal, d.precision, d.scale)
156 | updater.setDecimal(ordinal, decimal)
157 |
158 | case (BYTES, d: DecimalType) => (updater, ordinal, value) =>
159 | val bigDecimal = decimalConversions.fromBytes(value.asInstanceOf[ByteBuffer], avroType,
160 | LogicalTypes.decimal(d.precision, d.scale))
161 | val decimal = createDecimal(bigDecimal, d.precision, d.scale)
162 | updater.setDecimal(ordinal, decimal)
163 |
164 | case (RECORD, st: StructType) =>
165 | val writeRecord = getRecordWriter(avroType, st, path)
166 | (updater, ordinal, value) =>
167 | val row = new SpecificInternalRow(st)
168 | writeRecord(new RowUpdater(row), value.asInstanceOf[GenericRecord])
169 | updater.set(ordinal, row)
170 |
171 | case (ARRAY, ArrayType(elementType, containsNull)) =>
172 | val elementWriter = newWriter(avroType.getElementType, elementType, path)
173 | (updater, ordinal, value) =>
174 | val array = value.asInstanceOf[GenericData.Array[Any]]
175 | val len = array.size()
176 | val result = createArrayData(elementType, len)
177 | val elementUpdater = new ArrayDataUpdater(result)
178 |
179 | var i = 0
180 | while (i < len) {
181 | val element = array.get(i)
182 | if (element == null) {
183 | if (!containsNull) {
184 | throw new RuntimeException(s"Array value at path ${path.mkString(".")} is not " +
185 | "allowed to be null")
186 | } else {
187 | elementUpdater.setNullAt(i)
188 | }
189 | } else {
190 | elementWriter(elementUpdater, i, element)
191 | }
192 | i += 1
193 | }
194 |
195 | updater.set(ordinal, result)
196 |
197 | case (MAP, MapType(keyType, valueType, valueContainsNull)) if keyType == StringType =>
198 | val keyWriter = newWriter(SchemaBuilder.builder().stringType(), StringType, path)
199 | val valueWriter = newWriter(avroType.getValueType, valueType, path)
200 | (updater, ordinal, value) =>
201 | val map = value.asInstanceOf[java.util.Map[AnyRef, AnyRef]]
202 | val keyArray = createArrayData(keyType, map.size())
203 | val keyUpdater = new ArrayDataUpdater(keyArray)
204 | val valueArray = createArrayData(valueType, map.size())
205 | val valueUpdater = new ArrayDataUpdater(valueArray)
206 | val iter = map.entrySet().iterator()
207 | var i = 0
208 | while (iter.hasNext) {
209 | val entry = iter.next()
210 | assert(entry.getKey != null)
211 | keyWriter(keyUpdater, i, entry.getKey)
212 | if (entry.getValue == null) {
213 | if (!valueContainsNull) {
214 | throw new RuntimeException(s"Map value at path ${path.mkString(".")} is not " +
215 | "allowed to be null")
216 | } else {
217 | valueUpdater.setNullAt(i)
218 | }
219 | } else {
220 | valueWriter(valueUpdater, i, entry.getValue)
221 | }
222 | i += 1
223 | }
224 |
225 | updater.set(ordinal, new ArrayBasedMapData(keyArray, valueArray))
226 |
227 | case (UNION, _) =>
228 | val allTypes = avroType.getTypes.asScala
229 | val nonNullTypes = allTypes.filter(_.getType != NULL)
230 | if (nonNullTypes.nonEmpty) {
231 | if (nonNullTypes.length == 1) {
232 | newWriter(nonNullTypes.head, catalystType, path)
233 | } else {
234 | nonNullTypes.map(_.getType) match {
235 | case Seq(a, b) if Set(a, b) == Set(INT, LONG) && catalystType == LongType =>
236 | (updater, ordinal, value) => value match {
237 | case null => updater.setNullAt(ordinal)
238 | case l: java.lang.Long => updater.setLong(ordinal, l)
239 | case i: java.lang.Integer => updater.setLong(ordinal, i.longValue())
240 | }
241 |
242 | case Seq(a, b) if Set(a, b) == Set(FLOAT, DOUBLE) && catalystType == DoubleType =>
243 | (updater, ordinal, value) => value match {
244 | case null => updater.setNullAt(ordinal)
245 | case d: java.lang.Double => updater.setDouble(ordinal, d)
246 | case f: java.lang.Float => updater.setDouble(ordinal, f.doubleValue())
247 | }
248 |
249 | case _ =>
250 | catalystType match {
251 | case st: StructType if st.length == nonNullTypes.size =>
252 | val fieldWriters = nonNullTypes.zip(st.fields).map {
253 | case (schema, field) => newWriter(schema, field.dataType, path :+ field.name)
254 | }.toArray
255 | (updater, ordinal, value) => {
256 | val row = new SpecificInternalRow(st)
257 | val fieldUpdater = new RowUpdater(row)
258 | val i = GenericData.get().resolveUnion(avroType, value)
259 | fieldWriters(i)(fieldUpdater, i, value)
260 | updater.set(ordinal, row)
261 | }
262 |
263 | case _ =>
264 | throw new IncompatibleSchemaException(
265 | s"Cannot convert Avro to catalyst because schema at path " +
266 | s"${path.mkString(".")} is not compatible " +
267 | s"(avroType = $avroType, sqlType = $catalystType).\n" +
268 | s"Source Avro schema: $rootAvroType.\n" +
269 | s"Target Catalyst type: $rootCatalystType")
270 | }
271 | }
272 | }
273 | } else {
274 | (updater, ordinal, value) => updater.setNullAt(ordinal)
275 | }
276 |
277 | case _ =>
278 | throw new IncompatibleSchemaException(
279 | s"Cannot convert Avro to catalyst because schema at path ${path.mkString(".")} " +
280 | s"is not compatible (avroType = $avroType, sqlType = $catalystType).\n" +
281 | s"Source Avro schema: $rootAvroType.\n" +
282 | s"Target Catalyst type: $rootCatalystType")
283 | }
284 |
285 | // TODO: move the following method in Decimal object on creating Decimal from BigDecimal?
286 | private def createDecimal(decimal: BigDecimal, precision: Int, scale: Int): Decimal = {
287 | if (precision <= Decimal.MAX_LONG_DIGITS) {
288 | // Constructs a `Decimal` with an unscaled `Long` value if possible.
289 | Decimal(decimal.unscaledValue().longValue(), precision, scale)
290 | } else {
291 | // Otherwise, resorts to an unscaled `BigInteger` instead.
292 | Decimal(decimal, precision, scale)
293 | }
294 | }
295 |
296 | private def getRecordWriter(
297 | avroType: Schema,
298 | sqlType: StructType,
299 | path: List[String]): (CatalystDataUpdater, GenericRecord) => Unit = {
300 | val validFieldIndexes = ArrayBuffer.empty[Int]
301 | val fieldWriters = ArrayBuffer.empty[(CatalystDataUpdater, Any) => Unit]
302 |
303 | val length = sqlType.length
304 | var i = 0
305 | while (i < length) {
306 | val sqlField = sqlType.fields(i)
307 | val avroField = avroType.getField(sqlField.name)
308 | if (avroField != null) {
309 | validFieldIndexes += avroField.pos()
310 |
311 | val baseWriter = newWriter(avroField.schema(), sqlField.dataType, path :+ sqlField.name)
312 | val ordinal = i
313 | val fieldWriter = (fieldUpdater: CatalystDataUpdater, value: Any) => {
314 | if (value == null) {
315 | fieldUpdater.setNullAt(ordinal)
316 | } else {
317 | baseWriter(fieldUpdater, ordinal, value)
318 | }
319 | }
320 | fieldWriters += fieldWriter
321 | } else if (!sqlField.nullable) {
322 | throw new IncompatibleSchemaException(
323 | s"""
324 | |Cannot find non-nullable field ${path.mkString(".")}.${sqlField.name} in Avro schema.
325 | |Source Avro schema: $rootAvroType.
326 | |Target Catalyst type: $rootCatalystType.
327 | """.stripMargin)
328 | }
329 | i += 1
330 | }
331 |
332 | (fieldUpdater, record) => {
333 | var i = 0
334 | while (i < validFieldIndexes.length) {
335 | fieldWriters(i)(fieldUpdater, record.get(validFieldIndexes(i)))
336 | i += 1
337 | }
338 | }
339 | }
340 |
341 | private def createArrayData(elementType: DataType, length: Int): ArrayData = elementType match {
342 | case BooleanType => UnsafeArrayData.fromPrimitiveArray(new Array[Boolean](length))
343 | case ByteType => UnsafeArrayData.fromPrimitiveArray(new Array[Byte](length))
344 | case ShortType => UnsafeArrayData.fromPrimitiveArray(new Array[Short](length))
345 | case IntegerType => UnsafeArrayData.fromPrimitiveArray(new Array[Int](length))
346 | case LongType => UnsafeArrayData.fromPrimitiveArray(new Array[Long](length))
347 | case FloatType => UnsafeArrayData.fromPrimitiveArray(new Array[Float](length))
348 | case DoubleType => UnsafeArrayData.fromPrimitiveArray(new Array[Double](length))
349 | case _ => new GenericArrayData(new Array[Any](length))
350 | }
351 |
352 | /**
353 | * A base interface for updating values inside catalyst data structure like `InternalRow` and
354 | * `ArrayData`.
355 | */
356 | sealed trait CatalystDataUpdater {
357 | def set(ordinal: Int, value: Any): Unit
358 |
359 | def setNullAt(ordinal: Int): Unit = set(ordinal, null)
360 | def setBoolean(ordinal: Int, value: Boolean): Unit = set(ordinal, value)
361 | def setByte(ordinal: Int, value: Byte): Unit = set(ordinal, value)
362 | def setShort(ordinal: Int, value: Short): Unit = set(ordinal, value)
363 | def setInt(ordinal: Int, value: Int): Unit = set(ordinal, value)
364 | def setLong(ordinal: Int, value: Long): Unit = set(ordinal, value)
365 | def setDouble(ordinal: Int, value: Double): Unit = set(ordinal, value)
366 | def setFloat(ordinal: Int, value: Float): Unit = set(ordinal, value)
367 | def setDecimal(ordinal: Int, value: Decimal): Unit = set(ordinal, value)
368 | }
369 |
370 | final class RowUpdater(row: InternalRow) extends CatalystDataUpdater {
371 | override def set(ordinal: Int, value: Any): Unit = row.update(ordinal, value)
372 |
373 | override def setNullAt(ordinal: Int): Unit = row.setNullAt(ordinal)
374 | override def setBoolean(ordinal: Int, value: Boolean): Unit = row.setBoolean(ordinal, value)
375 | override def setByte(ordinal: Int, value: Byte): Unit = row.setByte(ordinal, value)
376 | override def setShort(ordinal: Int, value: Short): Unit = row.setShort(ordinal, value)
377 | override def setInt(ordinal: Int, value: Int): Unit = row.setInt(ordinal, value)
378 | override def setLong(ordinal: Int, value: Long): Unit = row.setLong(ordinal, value)
379 | override def setDouble(ordinal: Int, value: Double): Unit = row.setDouble(ordinal, value)
380 | override def setFloat(ordinal: Int, value: Float): Unit = row.setFloat(ordinal, value)
381 | override def setDecimal(ordinal: Int, value: Decimal): Unit =
382 | row.setDecimal(ordinal, value, value.precision)
383 | }
384 |
385 | final class ArrayDataUpdater(array: ArrayData) extends CatalystDataUpdater {
386 | override def set(ordinal: Int, value: Any): Unit = array.update(ordinal, value)
387 |
388 | override def setNullAt(ordinal: Int): Unit = array.setNullAt(ordinal)
389 | override def setBoolean(ordinal: Int, value: Boolean): Unit = array.setBoolean(ordinal, value)
390 | override def setByte(ordinal: Int, value: Byte): Unit = array.setByte(ordinal, value)
391 | override def setShort(ordinal: Int, value: Short): Unit = array.setShort(ordinal, value)
392 | override def setInt(ordinal: Int, value: Int): Unit = array.setInt(ordinal, value)
393 | override def setLong(ordinal: Int, value: Long): Unit = array.setLong(ordinal, value)
394 | override def setDouble(ordinal: Int, value: Double): Unit = array.setDouble(ordinal, value)
395 | override def setFloat(ordinal: Int, value: Float): Unit = array.setFloat(ordinal, value)
396 | override def setDecimal(ordinal: Int, value: Decimal): Unit = array.update(ordinal, value)
397 | }
398 | }
399 |
--------------------------------------------------------------------------------