working with the scalding type-safe api

66
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

Upload: criteolabs

Post on 09-Feb-2017

124 views

Category:

Engineering


3 download

TRANSCRIPT

Page 1: Working with the Scalding Type-Safe API

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

Page 2: Working with the Scalding Type-Safe API

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

Page 3: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

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

3

Page 4: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

The Promise

4

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

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

.groupBy { word => word }

.size

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

Page 5: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

Our Reality

5

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

.map { TargetType.fromClassWith23Fields }

.groupBy { _.lotsOfDimensions }

.sumByKey

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

Page 6: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

Our Reality, continued

6

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

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

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

Page 7: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

3 hours later, you ask yourself…

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

7

Page 8: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

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!

8

Page 9: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

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.

9

Page 10: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

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 ;)

10

Page 11: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

Some Context

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

11

Page 12: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

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)

12

Page 13: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

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

13

Page 14: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

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/)

14

Page 15: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

Langoustine Quick UI Tour

Page 16: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

Langoustine Quick UI Tour

Page 17: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

Langoustine Quick UI Tour

Red is bad.

Page 18: Working with the Scalding Type-Safe API

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

}

Page 19: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

Langoustine App

object HelloWorldApp extends DefaultLangoustineLauncher {

override lazy val run = execute(HelloWorldWorkflow.jobs)

}

Page 20: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

File Formats at Criteo

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

20

Page 21: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

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

21

Page 22: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

The Type-Safe API

It compiles, it works! (hardy har har)

22

Page 23: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

23+ Field Logs

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

23

Page 24: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

JobTest and Arity

JobTest with non-Products means Arity exceptions.

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

24

Page 25: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

Fine, then

Where’s the beef?

25

Page 26: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

For a given type, we need

• A TupleSetter and TupleConverter

• A Source and a Sink

• A partition finder/data locality service

26

Page 27: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

ScaldingType[T, K]

27

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

}

Page 28: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

TupleConverter and TupleSetter

28

new 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 = ???

}

Page 29: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

Fun, right?

Remind any one of working with JDBC?

29

Page 30: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

Macros to the rescue!

30

import com.criteo.scalding.utils._

val setter = SchemaUtils.scaldingTupleSetterFor[SomeType]

val converter = SchemaUtils.scaldingTupleConverterFor[SomeType]

val fields = SchemaUtils.scaldingFieldsFor[SomeType]

Page 31: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

And what about Sources?

31

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)

}

Page 32: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

Ty(p)ing it all together

32

class 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] = ???

}

Page 33: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

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)

33

Page 34: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

EventsSumJob

34

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)

)

)

}

Page 35: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

Wait, what?

35

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)

)

)

}

Page 36: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

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

36

Page 37: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

A Semi-what???

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

It is also a Monoid without a zero value

37

Page 38: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

Don’t Freak Out

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

38

Page 39: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

Semigroup[EventsSum]

39

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)

}

}

}

Page 40: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

And now add that map function

40

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)

}

}

def fromEvents(src: Events): EventsSum =

EventsSum(src.timestamp, src.events)

}

Page 41: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

Progress so far…

41

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)

)

)

}

Page 42: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

Let’s parse the Args!

42

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)

)

}

Page 43: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

Now, expose typedArgs to the Job

43

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

def typedArgs: T = args2TypedArgs(args)

}

trait TimeJobArgs extends TypedJobArgs[TimeArgs]

with TimeArgsParser

Page 44: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

Progress so far…

44

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!

)

)

}

Page 45: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

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.

45

Page 46: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

TimeArgs as Partition Key

46

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)}”

)

}

Page 47: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

Looking back at our Job…

47

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)

)

)

}

Page 48: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

Idempotence

48

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

Page 49: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

Making the Job Idempotent

49

object EventsSumJob extends IdempotentJob[EventsSum]

with IntervalArgsParser {

override def jobClass = classOf[EventsSumJob]

override def partitionsToClean(args: Array[String]) =

EventsSum.partitions(TimeArgs(args2TypedArgs(args))

}

Page 50: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

The job with all of that boilerplate

50

class 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)

)

)

}

Page 51: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

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!

51

Page 52: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

Execution Optimization

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

52

Page 53: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

A more complicated Job

53

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)

Page 54: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

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).

54

Page 55: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

4 jobs?

55

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

Page 56: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

3 jobs!

56

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)

Page 57: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

Down to 2 jobs!

57

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

Page 58: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

Only one job?

58

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.

Page 59: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

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.

59

Page 60: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

Performance hints: mappers

60

Page 61: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

Performance hints: mappers

Mappers 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.

61

Page 62: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

Performance hints: reducers

By 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)

62

Page 63: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

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)…

63

Page 64: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

A final word

We’re hiring.

64

Page 65: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

And we have a nice deck

65

Page 66: Working with the Scalding Type-Safe API

Copyright © 2014 Criteo

Thank You!

Justin Coffey, [email protected], @jqcoffey

Sofian Djamaa, [email protected], @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!