├── README.md ├── example-core ├── pom.xml └── src │ ├── main │ └── scala │ │ └── avrotest │ │ ├── ParquetAvroSparkExample.scala │ │ ├── SparkAvroKryoRegistrator.scala │ │ └── UserOperations.scala │ └── test │ ├── resources │ └── log4j.properties │ └── scala │ └── avrotest │ └── UserTestSuite.scala ├── example-format ├── pom.xml └── src │ └── main │ └── resources │ └── avro │ └── user.avdl └── pom.xml /README.md: -------------------------------------------------------------------------------- 1 | Spark SQL, Avro and Parquet 2 | =========================== 3 | 4 | This tutorial shows how to query data stored as Avro objects stored 5 | inside a columnar format (Parquet) via the Spark SQL query 6 | interface. The main intention of the tutorial is to show the seamless 7 | integration of the functional RDD operators that come with Spark and 8 | its SQL interface. For users who are unfamiliar with Avro we show how 9 | to make use of Avro interface description language (IDL) inside a 10 | Spark Maven project. 11 | 12 | _Acknowledgments: Thanks to Matt Massie (@massie) and the ADAM 13 | project for showing how to integrate Avro and Parquet with Spark._ 14 | 15 | Building the example 16 | -------------------- 17 | 18 | ``` 19 | $ git clone https://github.com/apache/spark.git 20 | $ cd spark 21 | $ sbt/sbt clean publish-local 22 | ``` 23 | 24 | Then in a different directory 25 | 26 | ``` 27 | $ git clone https://github.com/AndreSchumacher/avro-parquet-spark-example.git 28 | $ cd avro-parquet-spark-example 29 | $ mvn package 30 | ``` 31 | 32 | Project setup 33 | ------------- 34 | 35 | Here we are using Maven to build the project due to the available Avro 36 | IDL compiler plugin. Obviously one could have achieved the same goal 37 | using sbt. 38 | 39 | There are two subprojects: 40 | 41 | * `example-format`, which contains the Avro description of the primary 42 | data record we are using (`User`) 43 | * `example-code`, which contains the actual code that executes the 44 | queries 45 | 46 | There are two ways to specify a schema for Avro records: via a 47 | description in JSON format or via the IDL. We chose the latter since 48 | it is easier to comprehend. 49 | 50 | Our example models the user database of a social network, where users 51 | are asked to complete a personal profile which contains among other 52 | things their name and favorite color. Users can also send text 53 | messages to other users. The schemas of the resulting `User` and 54 | `Message` records then look as follows. 55 | 56 | ```xml 57 | @namespace("avrotest.avro") 58 | protocol AvroSparkSQL { 59 | record User { 60 | // The name of the user 61 | string name = "unknown"; 62 | // The age of the user 63 | int age = 0; 64 | // The favorite color of the user 65 | string favorite_color = "unknown"; 66 | } 67 | record Message { 68 | // The ID of the message 69 | long ID = 0; 70 | // The sender of this message 71 | string sender = "unknown"; 72 | // The recipient of this message 73 | string recipient = "unknown"; 74 | // The content of the message 75 | string content = ""; 76 | } 77 | } 78 | ``` 79 | 80 | This file is stored as part of the `example-format` project and is 81 | eventually compiled into a Java implementation of the class that 82 | represents these two types of records. Note that the different 83 | attributes are defined via their name, their type and an optional 84 | default value. For more information on how to specify Avro records see 85 | [the Avro documentation](http://avro.apache.org/docs/current/idl.html). 86 | 87 | Part of the description is also the _namespace_ of the protocol, which 88 | will result in the package name of the classes that will be generated 89 | from the description. We use the Avro maven plugin to do this 90 | transformation. It is added to `example-format/pom.xml` as follows: 91 | 92 | ```xml 93 | 94 | org.apache.avro 95 | avro-maven-plugin 96 | 97 | ``` 98 | 99 | Data generation 100 | --------------- 101 | 102 | Once the code generation has completed, objects of type `User` can be 103 | created via the `Builder` that was generated. For example: 104 | 105 | ```Scala 106 | import avrotest.avro.User 107 | 108 | User.newBuilder() 109 | .setName("User1") 110 | .setAge(10) 111 | .setFavoriteColor("blue") 112 | .build() 113 | ``` 114 | 115 | We can create a set of users and store these inside an Avro file as 116 | follows. 117 | 118 | ```Scala 119 | import org.apache.avro.file.DataFileWriter 120 | import org.apache.avro.specific.SpecificDatumWriter 121 | import avrotest.avro.User 122 | 123 | val userDatumWriter = new SpecificDatumWriter[User](classOf[User]) 124 | val dataFileWriter = new DataFileWriter[User](userDatumWriter) 125 | dataFileWriter.create(User.getClassSchema, file) 126 | 127 | for(i <- 1 to 10) { 128 | dataFileWriter.append(createUser(i)) 129 | } 130 | 131 | dataFileWriter.close() 132 | ``` 133 | 134 | Note that `createUser` in the above example is a factory method that 135 | uses the `Builder` to create `User` objects as described above. 136 | Similarly a set of messages can be created by using the class 137 | `Message` instead of `User` and a corresponding factory method. It is 138 | generally also possible to skip the step of code generation (for 139 | example, when the schema is generated dynamically). In this case there 140 | is similar but different approach using generic writers to write data 141 | to Avro files. 142 | 143 | Data stored in Avro format has the advantage of being accessible from 144 | a large number of programming languages and frameworks for which there 145 | exist Avro code generators. In order to process it via columnar 146 | frameworks such as Parquet we need to convert the data first (or store 147 | it in Parquet format right away). This tutorial assumes that you want 148 | to convert Avro files to Parquet files stored inside, say, HDFS. The 149 | conversion can be achieved as follows. 150 | 151 | ```Scala 152 | import org.apache.avro.file.DataFileReader 153 | import org.apache.avro.generic.{GenericDatumReader, IndexedRecord} 154 | import org.apache.avro.mapred.FsInput 155 | import parquet.avro.AvroParquetWriter 156 | import avrotest.avro.User 157 | 158 | val schema = User.getClassSchema 159 | val fsInput = new FsInput(input, conf) 160 | val reader = new GenericDatumReader[IndexedRecord](schema) 161 | val dataFileReader = DataFileReader.openReader(fsInput, reader) 162 | val parquetWriter = new AvroParquetWriter[IndexedRecord](output, schema) 163 | 164 | while(dataFileReader.hasNext) { 165 | parquetWriter.write(dataFileReader.next()) 166 | } 167 | 168 | dataFileReader.close() 169 | parquetWriter.close() 170 | ``` 171 | 172 | Here `input` is the (Hadoop) path under which the Avro file is stored, 173 | and `output` is the destination (Hadoop) path for the result Parquet 174 | file. Note that different from above we are using the generic variant 175 | of the Avro readers. That means that we only changing the `schema = ...` 176 | line we can actually convert any Avro file adhering to that schema 177 | to a corresponding Parquet file. 178 | 179 | Import into Spark SQL 180 | --------------------- 181 | 182 | The data written in the last step can be directly imported as a table 183 | inside Spark SQL and then queried. This can be done as follows. 184 | 185 | ```Scala 186 | import org.apache.spark.SparkConf 187 | import org.apache.spark.SparkContext 188 | import org.apache.spark.sql.SQLContext 189 | 190 | val conf = new SparkConf(true) 191 | .setMaster("local") 192 | .setAppName("ParquetAvroExample") 193 | val sqc = new SQLContext(new SparkContext(conf)) 194 | 195 | val schemaUserRdd = sqc.parquetFile(parquetUserFile.getParent.toString) 196 | schemaUserRdd.registerAsTable("UserTable") 197 | 198 | val schemaMessageRdd = sqc.parquetFile(parquetMessageFile.getParent.toString) 199 | schemaMessageRdd.registerAsTable("MessageTable") 200 | ``` 201 | 202 | Here `parquetUserFile` is the path under which the previously 203 | generated Parquet file containing the user data was stored. It is 204 | important to note that we do not need to specify a schema when we 205 | import the data, since the schema is preserved all the way from the 206 | initial specification in Avro IDL to the registration as a table 207 | inside Spark SQL. 208 | 209 | Querying the user and message databases 210 | --------------------------------------- 211 | 212 | After the tables have been registered, they can queried via SQL 213 | syntax, for example: 214 | 215 | ```Scala 216 | sqc.sql("SELECT favorite_color FROM UserTable WHERE name = \"User5\"") 217 | .collect() 218 | ``` 219 | 220 | The result will be returned as a sequence of `Row` objects, whose 221 | fields can be accessed via `apply()` functions. Also more complicated 222 | operations can be performed, for example: 223 | 224 | ```Scala 225 | sql("""SELECT name, COUNT(recipient) FROM 226 | UserTable JOIN MessageTable ON UserTable.name = MessageTable.sender 227 | GROUP BY name ORDER BY name""") 228 | .collect() 229 | ``` 230 | 231 | The last example will generate a list of pairs of usernames and 232 | counts, corresponding to the number of messages that user has sent. 233 | 234 | Mixing SQL and other Spark operations 235 | ------------------------------------- 236 | 237 | Since SQL data is stored as RDDs that have a schema attached to them 238 | (hence, `SchemaRDD`), SQL and other operations on RDDs can be mixed 239 | freely, for example: 240 | 241 | ```Scala 242 | sqc.sql("SELECT content from MessageTable") 243 | .flatMap(row => row.getString(0).replace(",", "").split(" ")) 244 | .map(word => (word, 1)) 245 | .reduceByKey(_ + _) 246 | .collect() 247 | .toMap 248 | ``` 249 | 250 | The previous example counts the number of times each word appears in 251 | any of the messages in the MessageTable. 252 | 253 | Importing Avro objects directly as RDD 254 | -------------------------------------- 255 | 256 | It is also possible to make direct use of the code-generated Avro 257 | classes in Spark. This requires registering a special Kryo serializer 258 | for each of the generated classes. Look at the example code for how 259 | this is done. The data can then be directly manupulated via Spark's 260 | Scala API. For example: 261 | 262 | ```Scala 263 | def myMapFunc(user: User): String = user.toString 264 | 265 | val userRDD: RDD[User] = readParquetRDD[User](sc, parquetFileName) 266 | userRDD.map(myMapFunc).collect().foreach(println(_)) 267 | ``` 268 | -------------------------------------------------------------------------------- /example-core/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4.0.0 4 | 5 | avrotest 6 | example-parent 7 | 0.0.1-SNAPSHOT 8 | ../pom.xml 9 | 10 | example-core 11 | jar 12 | Example of Avro and Spark-SQL: Core 13 | 14 | 15 | 16 | org.apache.maven.plugins 17 | maven-jar-plugin 18 | 2.4 19 | 20 | 21 | 22 | test-jar 23 | 24 | 25 | 26 | 27 | 28 | 29 | org.apache.maven.plugins 30 | maven-surefire-plugin 31 | 32 | true 33 | 34 | 35 | 36 | 37 | org.scalatest 38 | scalatest-maven-plugin 39 | 40 | ${project.build.directory}/surefire-reports 41 | . 42 | ParquetAvroExampleTest.txt 43 | 47 | -Xmx1024m -Dsun.io.serialization.extendedDebugInfo=true 48 | 49 | 50 | 51 | test 52 | 53 | test 54 | 55 | 56 | 57 | 58 | 59 | 60 | 61 | 62 | 63 | org.scala-lang 64 | scala-library 65 | 66 | 67 | org.apache.hadoop 68 | hadoop-client 69 | 70 | 71 | avrotest 72 | example-format 73 | ${project.version} 74 | 75 | 76 | org.apache.spark 77 | spark-core_${scala.artifact.suffix} 78 | 79 | 80 | org.apache.spark 81 | spark-sql_${scala.artifact.suffix} 82 | 83 | 84 | org.apache.avro 85 | avro 86 | 87 | 88 | org.apache.avro 89 | avro-mapred 90 | hadoop2 91 | 92 | 93 | com.twitter 94 | parquet-avro 95 | 96 | 97 | org.scalatest 98 | scalatest_${scala.artifact.suffix} 99 | test 100 | 101 | 102 | it.unimi.dsi 103 | fastutil 104 | 6.5.15 105 | 106 | 107 | 108 | -------------------------------------------------------------------------------- /example-core/src/main/scala/avrotest/ParquetAvroSparkExample.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 avrotest 19 | 20 | import com.google.common.io.Files 21 | 22 | import org.apache.hadoop.fs.Path 23 | 24 | import org.apache.spark.SparkConf 25 | import org.apache.spark.SparkContext 26 | import org.apache.spark.rdd.RDD 27 | import org.apache.spark.sql.SQLContext 28 | import org.apache.spark.sql.catalyst.util.getTempFilePath 29 | 30 | // our own classes generated from user.avdl by Avro tools 31 | import avrotest.avro.{Message, User} 32 | import avrotest.UserOperations._ 33 | 34 | object ParquetAvroSparkExample { 35 | 36 | private var sqc: SQLContext = _ 37 | 38 | def main(args: Array[String]) { 39 | // Create a Spark Configuration for local cluster mode 40 | val conf = new SparkConf(true) 41 | .setMaster("local") 42 | .setAppName("ParquetAvroExample") 43 | 44 | // Create a Spark Context and wrap it inside a SQLContext 45 | sqc = new SQLContext(new SparkContext(conf)) 46 | 47 | // Prepare some input data 48 | val avroFiles = (getTempFilePath("users", ".avro"), getTempFilePath("messages", ".avro")) 49 | val parquetFiles = ( 50 | new Path(Files.createTempDir().toString, "users.parquet"), 51 | new Path(Files.createTempDir().toString, "messages.parquet")) 52 | 53 | // Generate some input (100 users, 1000 messages) and write then as Avro files to the local 54 | // file system 55 | writeAvroFile(avroFiles._1, createUser, 100) 56 | writeAvroFile(avroFiles._2, createMessage(100)_, 1000) 57 | // Now convert the Avro file to a Parquet file (we could have generated one right away) 58 | convertAvroToParquetAvroFile( 59 | new Path(avroFiles._1.toString), 60 | new Path(parquetFiles._1.toString), 61 | User.getClassSchema, 62 | sqc.sparkContext.hadoopConfiguration) 63 | convertAvroToParquetAvroFile( 64 | new Path(avroFiles._2.toString), 65 | new Path(parquetFiles._2.toString), 66 | Message.getClassSchema, 67 | sqc.sparkContext.hadoopConfiguration) 68 | 69 | // Import the Parquet files we just generated and register them as tables 70 | sqc.parquetFile(parquetFiles._1.getParent.toString) 71 | .registerAsTable("UserTable") 72 | sqc.parquetFile(parquetFiles._2.getParent.toString) 73 | .registerAsTable("MessageTable") 74 | 75 | // Now let's do some queries 76 | println("The age of User3:") 77 | println(findAgeOfUser("User3", sqc)) 78 | println("The favorite color of User4:") 79 | println(findFavoriteColorOfUser("User4", sqc)) 80 | println("Favorite color distribution:") 81 | val result = findColorDistribution(sqc) 82 | for (color <- result.keys) { 83 | println(s"color: $color count: ${result.apply(color)}") 84 | } 85 | findNumberOfMessagesSent(sqc).foreach { 86 | case (sender, messages) => println(s"$sender sent $messages messages") 87 | } 88 | findMutualMessageExchanges(sqc).foreach { 89 | case (user_a, user_b) => println(s"$user_a and $user_b mutually exchanged messages") 90 | } 91 | println("Count words in messages:") 92 | countWordsInMessages(sqc).toTraversable.foreach { 93 | case (word, count) => println(s"word: $word count: $count") 94 | } 95 | 96 | // Stop the SparkContext 97 | sqc.sparkContext.stop() 98 | 99 | // What follows is an example of how to use Avro objects inside Spark directly. For that we 100 | // need to register a few Kryo serializers. Note: this is only required if we would like to 101 | // use User objects inside Spark's MapReduce operations. For Spark SQL this is not required 102 | // and in fact it seems to mess up the Parquet Row serialization(?). 103 | setKryoProperties(conf) 104 | val sc = new SparkContext(conf) 105 | 106 | def myMapFunc(user: User): String = user.toString 107 | 108 | println("Let's load the User file as a RDD[User], call toString() on each and collect the result") 109 | val userRDD: RDD[User] = readParquetRDD[User](sc, parquetFiles._1.toString) 110 | userRDD.map(myMapFunc).collect().foreach(println(_)) 111 | sc.stop() 112 | } 113 | 114 | /** 115 | * Note1: Spark uses Kryo for serializing and deserializing Objects contained in RDD's 116 | * and processed by its functional operators. In order to use Avro objects as part 117 | * of those operations we need to register them and specify and appropriate (De)Serializer. 118 | * Note2: This step is not neccesary if we one rely on relation operations of Spark SQL, 119 | * since these use Row objects that are always serializable. 120 | */ 121 | def setKryoProperties(conf: SparkConf) { 122 | conf.set("spark.kryo.registrator", classOf[SparkAvroKryoRegistrator].getName) 123 | conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") 124 | conf.set("spark.kryoserializer.buffer.mb", 4.toString) 125 | conf.set("spark.kryo.referenceTracking", "false") 126 | } 127 | } 128 | -------------------------------------------------------------------------------- /example-core/src/main/scala/avrotest/SparkAvroKryoRegistrator.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 avrotest 19 | 20 | import scala.reflect.ClassTag 21 | 22 | import org.apache.avro.io.{BinaryEncoder, EncoderFactory, DecoderFactory, BinaryDecoder} 23 | import org.apache.avro.specific.{SpecificDatumWriter, SpecificDatumReader, SpecificRecord} 24 | 25 | import com.esotericsoftware.kryo.{Serializer, Kryo} 26 | import com.esotericsoftware.kryo.io.{Output, Input} 27 | 28 | import it.unimi.dsi.fastutil.io.{FastByteArrayOutputStream, FastByteArrayInputStream} 29 | 30 | import org.apache.spark.serializer.KryoRegistrator 31 | 32 | import avrotest.avro.{Message, User} 33 | 34 | // This file is based (and mostly copied from): 35 | // https://github.com/bigdatagenomics/adam/blob/master/adam-core/src/main/scala/org/bdgenomics/adam/serialization/ADAMKryoRegistrator.scala 36 | // Thanks to Matt Massie (@massie) and the ADAM project 37 | 38 | case class InputStreamWithDecoder(size: Int) { 39 | val buffer = new Array[Byte](size) 40 | val stream = new FastByteArrayInputStream(buffer) 41 | val decoder = DecoderFactory.get().directBinaryDecoder(stream, null.asInstanceOf[BinaryDecoder]) 42 | } 43 | 44 | // NOTE: This class is not thread-safe; however, Spark guarantees that only a single thread 45 | // will access it. 46 | class AvroSerializer[T <: SpecificRecord](implicit tag: ClassTag[T]) extends Serializer[T] { 47 | val reader = new SpecificDatumReader[T](tag.runtimeClass.asInstanceOf[Class[T]]) 48 | val writer = new SpecificDatumWriter[T](tag.runtimeClass.asInstanceOf[Class[T]]) 49 | var in = InputStreamWithDecoder(1024) 50 | val outstream = new FastByteArrayOutputStream() 51 | val encoder = EncoderFactory.get().directBinaryEncoder(outstream, null.asInstanceOf[BinaryEncoder]) 52 | 53 | setAcceptsNull(false) 54 | 55 | def write(kryo: Kryo, kryoOut: Output, record: T) = { 56 | outstream.reset() 57 | writer.write(record, encoder) 58 | kryoOut.writeInt(outstream.array.length, true) 59 | kryoOut.write(outstream.array) 60 | } 61 | 62 | def read(kryo: Kryo, kryoIn: Input, klazz: Class[T]): T = this.synchronized { 63 | val len = kryoIn.readInt(true) 64 | if (len > in.size) { 65 | in = InputStreamWithDecoder(len + 1024) 66 | } 67 | in.stream.reset() 68 | // Read Kryo bytes into input buffer 69 | kryoIn.readBytes(in.buffer, 0, len) 70 | // Read the Avro object from the buffer 71 | reader.read(null.asInstanceOf[T], in.decoder) 72 | } 73 | } 74 | 75 | class SparkAvroKryoRegistrator extends KryoRegistrator { 76 | override def registerClasses(kryo: Kryo) { 77 | kryo.register(classOf[User], new AvroSerializer[User]()) 78 | kryo.register(classOf[Message], new AvroSerializer[Message]()) 79 | } 80 | } 81 | -------------------------------------------------------------------------------- /example-core/src/main/scala/avrotest/UserOperations.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 avrotest 19 | 20 | import java.io.File 21 | 22 | import scala.util.Random 23 | 24 | import parquet.avro.{AvroReadSupport, AvroParquetWriter} 25 | import parquet.hadoop.ParquetInputFormat 26 | 27 | import org.apache.hadoop.conf.Configuration 28 | import org.apache.hadoop.fs.Path 29 | import org.apache.hadoop.mapred.JobConf 30 | 31 | import org.apache.avro.Schema 32 | import org.apache.avro.file.{DataFileReader, DataFileWriter} 33 | import org.apache.avro.generic.{GenericDatumReader, IndexedRecord} 34 | import org.apache.avro.mapred.FsInput 35 | import org.apache.avro.specific.{SpecificRecord, SpecificDatumWriter} 36 | 37 | import org.apache.spark.SparkContext 38 | import org.apache.spark.rdd.RDD 39 | import org.apache.spark.sql.SQLContext 40 | import scala.reflect.ClassTag 41 | 42 | // our own class generated from user.avdl by Avro tools 43 | import avrotest.avro.{Message, User} 44 | 45 | // Implicits 46 | import org.apache.spark.SparkContext._ 47 | import collection.JavaConversions._ 48 | 49 | object UserOperations { 50 | 51 | var random: Random = new Random(15485863l) 52 | 53 | /** 54 | * For the given username, find the favorite number of the user. 55 | * 56 | * @param name User name 57 | * @param sqc The SQLContext to use 58 | * @return The user's age 59 | */ 60 | def findAgeOfUser(name: String, sqc: SQLContext): Int = { 61 | sqc.sql("SELECT age FROM UserTable WHERE name = \"" + name + "\"") 62 | .collect() 63 | .apply(0) 64 | .getInt(0) 65 | } 66 | 67 | /** 68 | * For the given username, find the favorite color of the user. 69 | * 70 | * @param name User name 71 | * @param sqc The SQLContext to use 72 | * @return The favorite color 73 | */ 74 | def findFavoriteColorOfUser(name: String, sqc: SQLContext): String = { 75 | sqc.sql("SELECT favorite_color FROM UserTable WHERE name = \"" + name + "\"") 76 | .collect() 77 | .apply(0) 78 | .getString(0) 79 | } 80 | 81 | /** 82 | * For every color that is contained in the table, find the number of 83 | * users whose favorite color matches that color. 84 | * 85 | * @param sqc The SQLContext to use 86 | * @return A map of colors to the number of its occurrences 87 | */ 88 | def findColorDistribution(sqc: SQLContext): Map[String, Long] = { 89 | val result = new collection.mutable.HashMap[String, Long]() 90 | val colorCounts = sqc.sql("SELECT favorite_color, COUNT(name) FROM UserTable GROUP BY favorite_color") 91 | .collect() 92 | for(row <- colorCounts) { 93 | result += row.getString(0) -> row.getLong(1) 94 | } 95 | result.toMap 96 | } 97 | 98 | /** 99 | * For each user in the UserTable find the number of messages sent in the 100 | * MessageTable. 101 | * 102 | * @param sqc The SQLContext to use 103 | * @return A list of pairs (user name, number of messages sent by that user) 104 | */ 105 | def findNumberOfMessagesSent(sqc: SQLContext): Seq[(String, Long)] = { 106 | sqc.sql(""" 107 | SELECT name, COUNT(recipient) FROM 108 | UserTable JOIN MessageTable ON UserTable.name = MessageTable.sender 109 | GROUP BY name ORDER BY name""") 110 | .collect() 111 | .map(row => (row.getString(0), row.getLong(1))) 112 | } 113 | 114 | /** 115 | * Find all pairs of users from the MessageTable that have mutually exchanged 116 | * messages. Note: this may report duplicate pairs (User1, User2) and (User2, User1). 117 | * 118 | * @param sqc The SQLContext to use 119 | * @return A list of pairs (user name, user name) 120 | */ 121 | def findMutualMessageExchanges(sqc: SQLContext): Seq[(String, String)] = { 122 | sqc.sql(""" 123 | SELECT DISTINCT A.sender, B.sender FROM 124 | (SELECT sender, recipient FROM MessageTable) A 125 | JOIN 126 | (SELECT sender, recipient FROM MessageTable) B 127 | ON A.recipient = B.sender AND A.sender = B.recipient""") 128 | .collect() 129 | .map(row => (row.getString(0), row.getString(1))) 130 | } 131 | 132 | /** 133 | * Counter the number of occurrences of each word contained in a message in 134 | * the MessageTable and returns the result as a word->count Map. 135 | * 136 | * @param sqc he SQLContext to use 137 | * @return A Map that has the words as key and the count as value 138 | */ 139 | def countWordsInMessages(sqc: SQLContext): Map[String, Int] = { 140 | sqc.sql("SELECT content from MessageTable") 141 | .flatMap(row => 142 | row.getString(0).replace(",", "").split(" ")) 143 | .map(word => (word, 1)) 144 | .reduceByKey(_ + _) 145 | .collect() 146 | .toMap 147 | } 148 | 149 | /** 150 | * Read in a parquet file containing Avro data and return the result as an RDD. 151 | * 152 | * @param sc The SparkContext to use 153 | * @param parquetFile The Parquet input file assumed to contain Avro objects 154 | * @return An RDD that contains the data of the file 155 | */ 156 | def readParquetRDD[T <% SpecificRecord](sc: SparkContext, parquetFile: String)(implicit tag: ClassTag[T]): RDD[T] = { 157 | val jobConf= new JobConf(sc.hadoopConfiguration) 158 | ParquetInputFormat.setReadSupportClass(jobConf, classOf[AvroReadSupport[T]]) 159 | sc.newAPIHadoopFile( 160 | parquetFile, 161 | classOf[ParquetInputFormat[T]], 162 | classOf[Void], 163 | tag.runtimeClass.asInstanceOf[Class[T]], 164 | jobConf) 165 | .map(_._2.asInstanceOf[T]) 166 | } 167 | 168 | /** 169 | * Generates a of Avro objects and stores them inside an Avro file. 170 | * 171 | * @param file The output file 172 | * @param factoryMethod The function to call to actually create the objects 173 | * @param count The number of objects to generate 174 | */ 175 | def writeAvroFile[T <: SpecificRecord]( 176 | file: File, 177 | factoryMethod: Int => T, 178 | count: Int): Unit = { 179 | val prototype = factoryMethod(0) 180 | val datumWriter = new SpecificDatumWriter[T]( 181 | prototype.getClass.asInstanceOf[java.lang.Class[T]]) 182 | val dataFileWriter = new DataFileWriter[T](datumWriter) 183 | 184 | dataFileWriter.create(prototype.getSchema, file) 185 | for(i <- 1 to count) { 186 | dataFileWriter.append(factoryMethod(i)) 187 | } 188 | dataFileWriter.close() 189 | } 190 | 191 | /** 192 | * Converts an Avro file that contains a set of Avro objects to a Parquet file. 193 | * 194 | * @param input The Avro file to convert 195 | * @param output The output file (possibly on HDFS) 196 | * @param schema The Avro schema of the input file 197 | * @param conf A Hadoop `Configuration` to use 198 | */ 199 | def convertAvroToParquetAvroFile( 200 | input: Path, 201 | output: Path, 202 | schema: Schema, 203 | conf: Configuration): Unit = { 204 | val fsInput = new FsInput(input, conf) 205 | val reader = new GenericDatumReader[IndexedRecord](schema) 206 | val dataFileReader = DataFileReader.openReader(fsInput, reader) 207 | val parquetWriter = new AvroParquetWriter[IndexedRecord](output, schema) 208 | 209 | while(dataFileReader.hasNext) { 210 | // Mote: the writer does not copy the passed object and buffers 211 | // writers. Therefore we need to pass a new User object every time, 212 | // although the reader allows us to re-use the same object. 213 | parquetWriter.write(dataFileReader.next()) 214 | } 215 | 216 | dataFileReader.close() 217 | parquetWriter.close() 218 | } 219 | 220 | /** 221 | * Creates a User Avro object. 222 | * 223 | * @param id The ID of the user to generate 224 | * @return An Avro object that represents the user 225 | */ 226 | def createUser(id: Int): User = { 227 | val builder = User.newBuilder() 228 | .setName(s"User$id") 229 | .setAge(id / 10) 230 | if (id >= 5) { 231 | builder 232 | .setFavoriteColor("blue") 233 | .build() 234 | } else { 235 | builder 236 | .setFavoriteColor("red") 237 | .build() 238 | } 239 | } 240 | 241 | /** 242 | * Creates a Message Avro object. 243 | * 244 | * @param id The ID of the message to generate 245 | * @return an Avro object that represents the mssage 246 | */ 247 | def createMessage(maxUserId: Int)(id: Int): Message = { 248 | val sender = random.nextInt(maxUserId) 249 | var recipient = random.nextInt(maxUserId) 250 | while (recipient == sender) recipient = random.nextInt(maxUserId) 251 | 252 | Message.newBuilder() 253 | .setID(id) 254 | .setSender(s"User$sender") 255 | .setRecipient(s"User$recipient") 256 | .setContent(s"Hey there, User$recipient, this is me, User$sender") 257 | .build() 258 | } 259 | } 260 | -------------------------------------------------------------------------------- /example-core/src/test/resources/log4j.properties: -------------------------------------------------------------------------------- 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 | # Set everything to be logged to the file core/target/unit-tests.log 19 | log4j.rootLogger=DEBUG, CA, FA 20 | 21 | #Console Appender 22 | log4j.appender.CA=org.apache.log4j.ConsoleAppender 23 | log4j.appender.CA.layout=org.apache.log4j.PatternLayout 24 | log4j.appender.CA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %p %c: %m%n 25 | log4j.appender.CA.Threshold = WARN 26 | 27 | 28 | #File Appender 29 | log4j.appender.FA=org.apache.log4j.FileAppender 30 | log4j.appender.FA.append=false 31 | log4j.appender.FA.file=target/unit-tests.log 32 | log4j.appender.FA.layout=org.apache.log4j.PatternLayout 33 | log4j.appender.FA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %p %c{1}: %m%n 34 | 35 | # Set the logger level of File Appender to WARN 36 | log4j.appender.FA.Threshold = INFO 37 | 38 | # Some packages are noisy for no good reason. 39 | log4j.additivity.org.apache.hadoop.hive.serde2.lazy.LazyStruct=false 40 | log4j.logger.org.apache.hadoop.hive.serde2.lazy.LazyStruct=OFF 41 | 42 | log4j.additivity.org.apache.hadoop.hive.metastore.RetryingHMSHandler=false 43 | log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=OFF 44 | 45 | log4j.additivity.hive.ql.metadata.Hive=false 46 | log4j.logger.hive.ql.metadata.Hive=OFF 47 | 48 | # Parquet related logging 49 | log4j.logger.parquet.hadoop=WARN 50 | log4j.logger.parquet.hadoop.ParquetRecordReader=ERROR 51 | log4j.logger.org.apache.spark.sql.parquet=INFO 52 | -------------------------------------------------------------------------------- /example-core/src/test/scala/avrotest/UserTestSuite.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 avrotest 19 | 20 | import scala.util.Random 21 | 22 | import org.scalatest.FunSuite 23 | import org.scalatest.BeforeAndAfterEach 24 | 25 | import com.google.common.io.Files 26 | 27 | import org.apache.hadoop.fs.Path 28 | 29 | import org.apache.spark.{SparkConf, SparkContext} 30 | import org.apache.spark.sql.SQLContext 31 | import org.apache.spark.sql.catalyst.util._ 32 | 33 | import avrotest.UserOperations._ 34 | import avrotest.avro.{Message, User} 35 | 36 | // inspired by [[org.apache.spark.LocalSparkContext]] 37 | class UserTestSuite extends FunSuite with BeforeAndAfterEach { 38 | 39 | @transient var sqc: SQLContext = _ 40 | 41 | override def beforeEach() { 42 | println("creating spark context") 43 | val conf = 44 | new SparkConf(false) 45 | .setMaster("local") 46 | .setAppName("test") 47 | sqc = new SQLContext(new SparkContext(conf)) 48 | 49 | UserOperations.random = new Random(15485863l) 50 | 51 | val avroFiles = (getTempFilePath("users", ".avro"), getTempFilePath("messages", ".avro")) 52 | val parquetFiles = ( 53 | new Path(Files.createTempDir().toString, "users.parquet"), 54 | new Path(Files.createTempDir().toString, "messages.parquet")) 55 | 56 | // Generate some input (100 users, 1000 messages) and write it as an Avro file to the local 57 | // file system 58 | writeAvroFile(avroFiles._1, createUser, 100) 59 | writeAvroFile(avroFiles._2, createMessage(100)_, 1000) 60 | // Now convert the Avro file to a Parquet file (we could have generated one right away) 61 | convertAvroToParquetAvroFile( 62 | new Path(avroFiles._1.toString), 63 | new Path(parquetFiles._1.toString), 64 | User.getClassSchema, 65 | sqc.sparkContext.hadoopConfiguration) 66 | convertAvroToParquetAvroFile( 67 | new Path(avroFiles._2.toString), 68 | new Path(parquetFiles._2.toString), 69 | Message.getClassSchema, 70 | sqc.sparkContext.hadoopConfiguration) 71 | 72 | // Import the Parquet files we just generated and register them as tables 73 | sqc.parquetFile(parquetFiles._1.getParent.toString) 74 | .registerAsTable("UserTable") 75 | sqc.parquetFile(parquetFiles._2.getParent.toString) 76 | .registerAsTable("MessageTable") 77 | } 78 | 79 | override def afterEach() { 80 | resetSparkContext() 81 | } 82 | 83 | private def resetSparkContext() = { 84 | if (sqc != null) { 85 | println("stopping Spark Context") 86 | sqc.sparkContext.stop() 87 | } 88 | sqc = null 89 | } 90 | 91 | test("Favorite color") { 92 | assert(findFavoriteColorOfUser("User1", sqc) === "red") 93 | } 94 | 95 | test("Age") { 96 | assert(findAgeOfUser("User20", sqc) === 2) 97 | } 98 | 99 | test("Color distribution") { 100 | assert(findColorDistribution(sqc).apply("red") === 4) 101 | } 102 | 103 | // Note: this test could fail because of a different random generator implementation 104 | // although the seed is fixed 105 | test("Number of messages") { 106 | val tmp = findNumberOfMessagesSent(sqc) 107 | assert(tmp(0)._1 === "User1") 108 | assert(tmp(0)._2.toInt === 12) 109 | } 110 | 111 | // Note: this test could fail because of a different random generator implementation 112 | // although the seed is fixed 113 | test("Mutual message exchange") { 114 | val finder = findMutualMessageExchanges(sqc) 115 | .find { 116 | case (user1: String, user2: String) => user1 == "User0" && user2 == "User43" 117 | } 118 | assert(finder.isDefined) 119 | } 120 | 121 | test("Count words in messages") { 122 | val tmp = countWordsInMessages(sqc) 123 | assert(tmp("Hey") === 1000) 124 | } 125 | } 126 | -------------------------------------------------------------------------------- /example-format/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4.0.0 4 | 5 | avrotest 6 | example-parent 7 | 0.0.1-SNAPSHOT 8 | ../pom.xml 9 | 10 | 11 | example-format 12 | jar 13 | Example of Avro and Spark-SQL: Format 14 | 15 | 16 | 17 | 18 | org.apache.avro 19 | avro-maven-plugin 20 | 21 | 22 | schemas 23 | generate-sources 24 | 25 | schema 26 | protocol 27 | idl-protocol 28 | 29 | 30 | ${project.basedir}/src/main/resources/avro 31 | 32 | 33 | 34 | 35 | 36 | 37 | 38 | 39 | org.apache.avro 40 | avro 41 | ${avro.version} 42 | 43 | 44 | 45 | -------------------------------------------------------------------------------- /example-format/src/main/resources/avro/user.avdl: -------------------------------------------------------------------------------- 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 | @namespace("avrotest.avro") 19 | protocol AvroSparkSQL { 20 | record User { 21 | // The name of the user 22 | string name = "unknown"; 23 | // The age of the user 24 | int age = 0; 25 | // The favorite color of the user 26 | string favorite_color = "unknown"; 27 | } 28 | record Message { 29 | // The ID of the message 30 | long ID = 0; 31 | // The sender of this message 32 | string sender = "unknown"; 33 | // The recipient of this message 34 | string recipient = "unknown"; 35 | // The content of the message 36 | string content = ""; 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 4.0.0 5 | avrotest 6 | example-parent 7 | 0.0.1-SNAPSHOT 8 | pom 9 | Example of Avro and Spark-SQL 10 | 11 | 12 | 1.7 13 | 2.10.4 14 | 2.10 15 | 1.7.6 16 | 1.0.0 17 | 1.4.3 18 | 2.2.0 19 | 20 | 21 | 22 | example-format 23 | example-core 24 | 25 | 26 | 27 | target/scala-${scala.version}/classes 28 | target/scala-${scala.version}/test-classes 29 | 30 | 31 | 32 | net.alchim31.maven 33 | scala-maven-plugin 34 | 3.1.5 35 | 36 | 37 | org.apache.avro 38 | avro-maven-plugin 39 | ${avro.version} 40 | 41 | 42 | org.apache.maven.plugins 43 | maven-clean-plugin 44 | 2.5 45 | 46 | 47 | org.apache.maven.plugins 48 | maven-compiler-plugin 49 | 3.1 50 | 51 | ${java.version} 52 | ${java.version} 53 | 54 | 55 | 56 | org.apache.maven.plugins 57 | maven-jar-plugin 58 | 2.4 59 | 60 | 61 | org.apache.maven.plugins 62 | maven-resources-plugin 63 | 2.6 64 | 65 | 66 | org.apache.maven.plugins 67 | maven-surefire-plugin 68 | 2.16 69 | 70 | 71 | org.scalatest 72 | scalatest-maven-plugin 73 | 1.0-RC2 74 | 75 | 76 | 77 | 78 | 79 | net.alchim31.maven 80 | scala-maven-plugin 81 | 82 | 83 | scala-compile-first 84 | process-resources 85 | 86 | compile 87 | 88 | 89 | 90 | scala-test-compile-first 91 | process-test-resources 92 | 93 | testCompile 94 | 95 | 96 | 97 | attach-scaladocs 98 | verify 99 | 100 | doc-jar 101 | 102 | 103 | 104 | 105 | ${scala.version} 106 | incremental 107 | true 108 | 109 | -unchecked 110 | -optimise 111 | -deprecation 112 | 113 | 114 | -Xms64m 115 | -Xms1024m 116 | -Xmx1024m 117 | 118 | 119 | -source 120 | ${java.version} 121 | -target 122 | ${java.version} 123 | 124 | 125 | 126 | 127 | 128 | 129 | 130 | 131 | 132 | org.apache.hadoop 133 | hadoop-client 134 | ${hadoop.version} 135 | 136 | 137 | asm 138 | asm 139 | 140 | 141 | org.jboss.netty 142 | netty 143 | 144 | 145 | org.codehaus.jackson 146 | * 147 | 148 | 149 | org.sonatype.sisu.inject 150 | * 151 | 152 | 153 | 154 | 155 | org.scala-lang 156 | scala-library 157 | ${scala.version} 158 | 159 | 160 | org.apache.spark 161 | spark-core_${scala.artifact.suffix} 162 | ${spark.version} 163 | 164 | 165 | org.apache.spark 166 | spark-sql_${scala.artifact.suffix} 167 | ${spark.version} 168 | 169 | 170 | org.apache.avro 171 | avro 172 | ${avro.version} 173 | 174 | 175 | org.apache.avro 176 | avro-mapred 177 | ${avro.version} 178 | hadoop2 179 | 180 | 181 | com.twitter 182 | parquet-avro 183 | ${parquet.version} 184 | 185 | 186 | org.scalatest 187 | scalatest_${scala.artifact.suffix} 188 | 1.9.2 189 | test 190 | 191 | 192 | 193 | 194 | --------------------------------------------------------------------------------