├── 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 | [![Build Status](https://travis-ci.org/hortonworks-spark/spark-schema-registry.svg?branch=master)](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 | --------------------------------------------------------------------------------