Working with the Scalding Type -Safe API

Preview:

Citation preview

Copyright © 2014 Criteo

Working with the Scalding Type-Safe API

Without tearing your hair out (too much)Justin Coffey, Sr Staff Devlead, Criteo

Sofian Djamaa, Sr Engineer, Criteo

2016-04-14

Copyright © 2014 Criteo

The Scalding Gods hate you

2

Don’t worry, they hate us too

Hopefully, this presentation will help you understand their whims a bit better

Copyright © 2014 Criteo

3

A Few Words on Scalding

• Scalding is a framework to write Map Reduce jobs in a more functional way

• It’s written in Scala• And uses the Java Map Reduce framework, Cascading• It has a non-typed, “fields” API and a newer typed-API, which is what

we’ll be talking about

Copyright © 2014 Criteo

4

The Promise

TypedPipe.from(TextLine(args("input")))

.flatMap { line => line.split(",") }

.groupBy { word => word }

.size .write(TypedTsv[(String, Long)](args("output")))

Copyright © 2014 Criteo

5

Our Reality

TypedPipe.from(SomeSource(args("some-place"))

.map { TargetType.fromClassWith23Fields }

.groupBy { _.lotsOfDimensions }

.sumByKey

.write(TypedSink[TargetType](args("another-place")))

Copyright © 2014 Criteo

6

Our Reality, continued

java.lang.AssertionError: assertion failed: Arity of (class

com.twitter.scalding.LowPriorityConversions$SingleSetter$) is 1,

which doesn't match: {your class with 23 fields}

Copyright © 2014 Criteo

7

3 hours later, you ask yourself…

Why, oh why, is there a default TupleSetter that doesn’t actually work for anything other than Products?

Copyright © 2014 Criteo

8

What we’re going to talk about

We will humbly* present our work in dealing with pitfalls of using the Type-Safe Scalding API in an existing Hadoop environment.

*and I mean it. I expect we’ve done a fair number of things poorly!

Copyright © 2014 Criteo

9

Your humble hosts for the next 40 minutes

• Justin Coffey: Devlead for the Scalability Analytics team at Criteo, responsible for all analytic systems, data and products.

• Sofian Djamaa: Senior engineer in Scalability Analytics at Criteo, works (hard) on batch and stream processing.

Copyright © 2014 Criteo

10

Criteo Scale

• 2 Hadoop clusters ~2000 nodes total• 28K cores, 300TB RAM, Many Many PBs• 50 billion events logged per day• 25TB data ingested per day• 7 weeks of vacation per year ;)

Copyright © 2014 Criteo

11

Some Context

Before we dig into the nitty-gritty, I think some context is in order.

Copyright © 2014 Criteo

12

A Use Case

• Build a 20TB time-series dataset for fast operational reporting• Source data are application logs in hadoop• Store data in Vertica (a scale-out columnar DB)

Copyright © 2014 Criteo

13

The Scalding Part

• Scalding handles all data transformations• Including simple aggregations of data• Keeps logic in one place, scales out easily• Reduces load on Vertica

Copyright © 2014 Criteo

14

A Note on Scheduling

• We use our very own NIH scheduler, Langoustine• It uses a Scala DSL for code-as-config to describe a job DAG• It is an opinionated scheduler and expects your jobs to be

idempotent• Runs inside a Finatra app (http://twitter.github.io/finatra/)

Copyright © 2014 Criteo

Langoustine Quick UI Tour

Copyright © 2014 Criteo

Langoustine Quick UI Tour

Copyright © 2014 Criteo

Langoustine Quick UI Tour

Red is bad.

Copyright © 2014 Criteo

Langoustine DSL

object HelloWorldWorkflow { val hello = Job( name = "hello", calculationPeriod = 1 minute, primaryTask = { (c: TaskContext) => EchoTask(c, "hello, ") } ).toGraph val world = Job( name = "world", calculationPeriod = 1 minute, primaryTask = { (c: TaskContext) => EchoTask(c, "world!") } ).toGraph

val jobs = world dependsOn hello}

Copyright © 2014 Criteo

Langoustine App

object HelloWorldApp extends DefaultLangoustineLauncher {

override lazy val run = execute(HelloWorldWorkflow.jobs)

}

Copyright © 2014 Criteo

20

File Formats at Criteo

• Our workflow consumes Parquet, Pail/JSON and Text-Delimited data

Copyright © 2014 Criteo

21

Finding the Data

• Consuming data requires knowing where it is• While we do have hive, we don’t use hCatalog outside of it• Each job has to roll its own data location service

Copyright © 2014 Criteo

22

The Type-Safe API

It compiles, it works! (hardy har har)

Copyright © 2014 Criteo

23

23+ Field Logs

23+ fields and Scala 2.10, means writing your own TupleSetters and Converters

Copyright © 2014 Criteo

24

JobTest and Arity

JobTest with non-Products means Arity exceptions.

JobTest just tests your logic, not the nasty stuff at the edges.

Copyright © 2014 Criteo

25

Fine, then

Where’s the beef?

Copyright © 2014 Criteo

26

For a given type, we need

• A TupleSetter and TupleConverter• A Source and a Sink• A partition finder/data locality service

Copyright © 2014 Criteo

27

ScaldingType[T, K]

trait ScaldingType[T, K] { implicit def converter: TupleConverter[T] implicit def setter: TupleSetter[T] def fields: Fields def partitions(key: K): Seq[String] def source(partitionKey: K): Source def sink(partitionKey: K): Source}

Copyright © 2014 Criteo

28

TupleConverter and TupleSetternew TupleConverter[SomeType] { def apply(te: TupleEntry): SomeType = new SomeType(te.getInt(0), …)

def arity: Int = ???}

new TupleSetter[SomeType] { def apply(arg: SomeType): Tuple = { val t = Tuple.size(arity) t.set(0, arg.someInt) … t } def arity: Int = ???}

Copyright © 2014 Criteo

29

Fun, right?

Remind any one of working with JDBC?

Copyright © 2014 Criteo

30

Macros to the rescue!

import com.criteo.scalding.utils._

val setter = SchemaUtils.scaldingTupleSetterFor[SomeType]

val converter = SchemaUtils.scaldingTupleConverterFor[SomeType]

val fields = SchemaUtils.scaldingFieldsFor[SomeType]

Copyright © 2014 Criteo

31

And what about Sources?

trait TsvSources[T, K] { self: ScaldingType[T, K] =>

override def sink(partitionKey: K): Source = Tsv(partitions(partitionKey).head)

override def source(partitionKey: K): Source = Tsv(partitions(partitionKey).head, fields)

}

Copyright © 2014 Criteo

32

Ty(p)ing it all togetherclass Events( val timestamp: DateTime = new DateTime(0), val name: String = “”, val events: Long = 0)

class TimeKey(root: String, time: DateTime)

object Events extends ScaldingType[Events, TimeKey] with TsvSources[Events, TimeKey] { implicit val setter = SchemaUtils.scaldingTupleSetterFor[Events] implicit val converter = SchemaUtils.scaldingTupleConverterFor[Events]

val fields = SchemaUtils.scaldingFieldsFor[Events]

// now, we just have to implement our partition building function! def partitions(key: TimeKey): Seq[String] = ???}

Copyright © 2014 Criteo

33

Let’s write a Job!

We’ll take our Events type and turn it into an EventsSum type, counting the number of events per timestamp as we go.

EventsSum(val time: DateTime, val events: Long)

Copyright © 2014 Criteo

34

EventsSumJobclass EventsSumJob(args: Args) extends Job(args) with TimeJobArgs {

TypedPipe.from[Events]( Events.source(typedArgs).read, Events.fields ).map(EventsSum.fromEvents) .sumByKey .values .write( TypedSink[EventsSum]( EventsSum.sink(typedArgs) ) )}

Copyright © 2014 Criteo

35

Wait, what?class EventsSumJob(args: Args) extends Job(args) with TimeJobArgs {

TypedPipe.from[Events]( Events.source(typedArgs).read, Events.fields ).map(EventsSum.fromEvents) .sumByKey .values .write( TypedSink[EventsSum]( EventsSum.sink(typedArgs) ) )}

Copyright © 2014 Criteo

36

Wait, what?

• the TimeJobArgs trait maps Scalding’s Args object to our own type, available via typedArgs

• fromEvents is just a simple mapping function, Events => EventsSum• sumByKey is a Scalding function that performs an aggregation and

requires a Semigroup and an Ordering

Copyright © 2014 Criteo

37

A Semi-what???

Semigroups are a (mathematical) Set with an associative binary operation

It is also a Monoid without a zero value

Copyright © 2014 Criteo

38

Don’t Freak Out

In Scalding land, a Semigroup[T] is just a thing that describes how two T’s should be added together.

Copyright © 2014 Criteo

39

Semigroup[EventsSum]case class EventsSum( val time: DateTime = new DateTime(0), val events: Long = 0)

object EventsSum extends ScaldingType[…] with TsvSources[…] { … implicit val ordering: Ordering[EventsSum] = Ordering.by(_.time.getMillis)

implicit val semiGroup = new Semigroup[EventsSum] { override def plus( l: EventsSum, r: EventsSum) = { require(l.time == r.time, “l and r times must match!”) l.copy(events = l.events + r.events) } }

}

Copyright © 2014 Criteo

40

And now add that map functioncase class EventsSum( val time: DateTime = new DateTime(0), val events: Long = 0)

object EventsSum extends ScaldingType[…] with TsvSources[…] { … implicit val ordering: Ordering[EventsSum] = Ordering.by(_.time.getMillis)

implicit val semiGroup = new Semigroup[EventsSum] { override def plus( l: EventsSum, r: EventsSum) = { require(l.time == r.time, “l and r times must match!”) l.copy(events = l.events + r.events) } }

def fromEvents(src: Events): EventsSum = EventsSum(src.timestamp, src.events)}

Copyright © 2014 Criteo

41

Progress so far…class EventsSumJob(args: Args) extends Job(args) with TimeJobArgs {

TypedPipe.from[Events]( Events.source(typedArgs).read, Events.fields ).map(EventsSum.fromEvents) <= done! .sumByKey <= done! .values .write( TypedSink[EventsSum]( EventsSum.sink(typedArgs) ) )}

Copyright © 2014 Criteo

42

Let’s parse the Args!trait TypedArgsParser[T] {

def args2TypedArgs(args: Args): T

def args2TypedArgs(args: Array[String]): T = args2TypedArgs(Args(args))

}

case class TimeArgs(root: String, time: DateTime)

trait TimeArgsParser extends TypedArgsParser[TimeArgs] { override def args2TypedArgs(args: Args): TimeArgs = TimeArgs( root = args.required("root"), time = new DateTime(args.required("time"), DateTimeZone.UTC) )}

Copyright © 2014 Criteo

43

Now, expose typedArgs to the Job

trait TypedJobArgs[T] extends Job with TypedArgsParser[T] {

def typedArgs: T = args2TypedArgs(args)

}

trait TimeJobArgs extends TypedJobArgs[TimeArgs] with TimeArgsParser

Copyright © 2014 Criteo

44

Progress so far…class EventsSumJob(args: Args) extends Job(args) with TimeJobArgs { <= done!

TypedPipe.from[Events]( Events.source(typedArgs).read, <= done! Events.fields ).map(EventsSum.fromEvents) <= done! .sumByKey <= done! .values .write( TypedSink[EventsSum]( EventsSum.sink(typedArgs) <= done! ) )}

Copyright © 2014 Criteo

45

Partitions

Partitions are just folders that contain a batch of data to process.

The partition function signature for our two types is the same: TimeKey => Seq[String]

Note also that TimeArgs is equivalent to the TimeKey we defined earlier.

Copyright © 2014 Criteo

46

TimeArgs as Partition Key

object DateUtils { val f = DateTimeFormat.forPattern( “yyyy-MM-dd-HH” ).withZoneUTC()

def dayHour(time: DateTime) = f.print(time)

}

object EventsSum extends ScaldingType[EventsSum, TimeArgs] with TsvSources[EventsSum, TimeArgs] { … def partition(k: TimeArgs) = Seq( s”${k.root}/events_sum/${DateUtils.dayHour(k.time)}” )}

Copyright © 2014 Criteo

47

Looking back at our Job…class EventsSumJob(args: Args) extends Job(args) with TimeJobArgs {

TypedPipe.from[Events]( <= implicit converter Events.source(typedArgs).read, Events.fields ).map(EventsSum.fromEvents) .sumByKey <= semigroup and ordering .values .write( TypedSink[EventsSum]( <= implicit setter EventsSum.sink(typedArgs) ) )}

Copyright © 2014 Criteo

48

Idempotence

In computer science, the term idempotent is used more comprehensively to describe an operation that will produce the same results if executed once or multiple times.ref: https://en.wikipedia.org/wiki/Idempotence#Computer_science_meaning

Copyright © 2014 Criteo

49

Making the Job Idempotent

object EventsSumJob extends IdempotentJob[EventsSum] with IntervalArgsParser {

override def jobClass = classOf[EventsSumJob]

override def partitionsToClean(args: Array[String]) = EventsSum.partitions(TimeArgs(args2TypedArgs(args))

}

Copyright © 2014 Criteo

50

The job with all of that boilerplateclass EventsSumJob(args: Args) extends Job(args) with TimeJobArgs {

implicit val eventsSumTupleSetter: TupleSetter[EventsSum] = new TupleSetter[SomeType] { def apply(arg: SomeType): Tuple = { val t = Tuple.size(arity) t.set(…) … t }

def arity: Int = 2 }

implicit val eventsTupleConverter: TupleConverter[Events] = new TupleConverter[Events] { def apply(te: TupleEntry): Events = new Events(…)

def arity: Int = 3 } val eventsFields = new Fields(“time”, “name”, “events”)

val timeArgs = TimeArgs( root = args.required("root"), time = new DateTime(args.required("time"), DateTimeZone.UTC) )

implicit val eventsSumSemiGroup: SemiGroup[EventsSum] = new Semigroup[EventsSum] { override def plus( l: EventsSum, r: EventsSum) = { require(l.time == r.time, “l and r times must match!”) l.copy(events = l.events + r.events) } }

implicit val eventsSumOrdering: Ordering[EventsSum] = Ordering.by(_.time.getMillis)

val events2EventsSum: Events => EventsSum = { e => EventsSum(…) }

TypedPipe.from[Events]( <= implicit converter Events.source(typedArgs).read, Events.fields ).map(EventsSum.fromEvents) .sumByKey <= semigroup and ordering .values .write( TypedSink[EventsSum]( <= implicit setter EventsSum.sink(typedArgs) ) )}

Copyright © 2014 Criteo

51

Taking it all in

• The type safe API pushes the unsafe parts to the edges (converters and setters)

• ScaldingType[T] formalizes this along with data location, allowing for less boilerplate in your jobs

• TypedArgs permit stable interfaces across many jobs, with copious code sharing

• IdempotentJob[T] makes scheduling and replay of your job safe• In the end your jobs are just logic, with very little boilerplate!

Copyright © 2014 Criteo

52

Execution Optimization

If you’re still with us, here’s a bit more detail on run-time optimizations

Copyright © 2014 Criteo

53

A more complicated Job

TypedPipe.from[Events]( Events.source(typedArgs).read, Events.fields ).map(EventsSum.fromEvents) .groupBy( events => events.type ) .sum(EventsSum.aggregator) dimensions (enrich .values events) .groupBy( e => e.time ) .join(WeatherForecastSource.groupBy( w => w.time )) .values .map(WeatherEvents.fromEventsWithWeather)

Copyright © 2014 Criteo

54

Generated execution

How many M/R jobs will be generated?

Reminder: Scalding code is translated to (at runtime) Cascading pipes. Those pipes are evaluated through an execution plan and produce MapReduce jobs (still at runtime).

Copyright © 2014 Criteo

55

4 jobs?

TypedPipe.from[Events]( Events.source(typedArgs).read, triggers a reduce Events.fields ).map(EventsSum.fromEvents) .groupBy( events => events.type ) .sum(EventsSum.aggregator) .values other source to join with .groupBy( e => e.time ) .join(WeatherForecastSource.groupBy( w => w.time )) .values .map(WeatherEvents.fromEventsWithWeather)

merge of 2 sources

Copyright © 2014 Criteo

56

3 jobs!

TypedPipe.from[Events]( Events.source(typedArgs).read, in the same job as Events.fields scalding reorders ).map(EventsSum.fromEvents) operations .groupBy( events => events.type ) .sum(EventsSum.aggregator) .values .groupBy( e => e.time ) .join(WeatherForecastSource.groupBy( w => w.time )) .values .map(WeatherEvents.fromEventsWithWeather)

Copyright © 2014 Criteo

57

Down to 2 jobs!

TypedPipe.from[Events]( Events.source(typedArgs).read, join done on the Events.fields second job ).map(EventsSum.fromEvents) .join(WeatherForecastSource.groupBy( w => w.time )) .values .map(WeatherEvents.fromEventsWithWeather) first job .sumByLocalKeys(EventsSum.aggregator) .values

sumByLocalKeys: Map-side computation

Copyright © 2014 Criteo

58

Only one job? TypedPipe.from[Events]( Events.source(typedArgs).read, source retrieved Events.fields in memory and joined ).map(EventsSum.fromEvents) in each mapper .hashJoin(WeatherForecastSource.source()) .values .map(WeatherEvents.fromEventsWithWeather) .sumByLocalKeys(EventsSum.aggregator) .values

Generates one job with mappers only: result in HDFS is wrong (until another job aggregates all files).

Due to hashJoin implicitly indicating that the data is fully copied to all mappers: no need for a consolidation phase, therefore no reduce.

Copyright © 2014 Criteo

59

Performance hints

A few number of jobs means less scheduling issues: better latency for end-to-end workflows.

Extreme optimization might lead to data inconsistency.

Copyright © 2014 Criteo

60

Performance hints: mappers

Copyright © 2014 Criteo

61

Performance hints: mappersMappers are not taking the same amount of time: data are not distributed evenly on the mappers.

Reminder : data are stored in blocks in HDFS. Each mapper runs on a block of data. If a file fits in less than a block, a mapper will not take other data (except using a file combiner…) and will finish earlier than the others.

shard(xx)Forces a reduce phase: generates « xx » intermediate files instead of big ones (mappers will handle less files then thanks to the triggering of a new reduce phase). Used to distribute data over mappers in intermediate steps.

Copyright © 2014 Criteo

62

Performance hints: reducersBy default Scalding uses only one reducer. If your data can be partitioned and doesn’t fit in memory (lot of spilling), it’s better to have several reducers.

Warning: the more reducers, the more files, the more mappers for the next step (pay attention to the balance between block size and number of mappers)

Copyright © 2014 Criteo

63

Performance hints: check the counters

JVM use (especially GC counts), spilled records more than 100% of the input size (output data doesn’t fit in memory), distribution of data between mappers, data format (e.g. Parquet)…

Copyright © 2014 Criteo

64

A final word

We’re hiring.

Copyright © 2014 Criteo

65

And we have a nice deck

Copyright © 2014 Criteo

Thank You!Justin Coffey, j.coffey@criteo.com, @jqcoffeySofian Djamaa, s.djamaa@criteo.com, @sdjamaa

http://labs.criteo.com/blog

Also, come see us in Paris on May 26th for a Criteo Sponsored Day with Criteo, Google and Spotify Engineers!

Recommended