SlideShare une entreprise Scribd logo
1  sur  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
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
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
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")))
Copyright © 2014 Criteo
Our Reality
5
TypedPipe.from(SomeSource(args("some-place"))
.map { TargetType.fromClassWith23Fields }
.groupBy { _.lotsOfDimensions }
.sumByKey
.write(TypedSink[TargetType](args("another-place")))
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}
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
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
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
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
Copyright © 2014 Criteo
Some Context
Before we dig into the nitty-gritty, I think some context is in order.
11
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
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
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
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
File Formats at Criteo
• Our workflow consumes Parquet, Pail/JSON and Text-Delimited data
20
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
Copyright © 2014 Criteo
The Type-Safe API
It compiles, it works! (hardy har har)
22
Copyright © 2014 Criteo
23+ Field Logs
23+ fields and Scala 2.10, means writing your own TupleSetters and Converters
23
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
Copyright © 2014 Criteo
Fine, then
Where’s the beef?
25
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
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
}
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 = ???
}
Copyright © 2014 Criteo
Fun, right?
Remind any one of working with JDBC?
29
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]
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)
}
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] = ???
}
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
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)
)
)
}
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)
)
)
}
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
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
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
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)
}
}
}
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)
}
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)
)
)
}
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)
)
}
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
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!
)
)
}
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
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)}”
)
}
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)
)
)
}
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
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))
}
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)
)
)
}
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
Copyright © 2014 Criteo
Execution Optimization
If you’re still with us, here’s a bit more detail on run-time optimizations
52
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)
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
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
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)
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
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.
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
Copyright © 2014 Criteo
Performance hints: mappers
60
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
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
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
Copyright © 2014 Criteo
A final word
We’re hiring.
64
Copyright © 2014 Criteo
And we have a nice deck
65
Copyright © 2014 Criteo
Thank You!
Justin Coffey, j.coffey@criteo.com, @jqcoffey
Sofian 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!

Contenu connexe

Tendances

Spark ETL Techniques - Creating An Optimal Fantasy Baseball Roster
Spark ETL Techniques - Creating An Optimal Fantasy Baseball RosterSpark ETL Techniques - Creating An Optimal Fantasy Baseball Roster
Spark ETL Techniques - Creating An Optimal Fantasy Baseball RosterDon Drake
 
Introduction to the Hadoop Ecosystem (FrOSCon Edition)
Introduction to the Hadoop Ecosystem (FrOSCon Edition)Introduction to the Hadoop Ecosystem (FrOSCon Edition)
Introduction to the Hadoop Ecosystem (FrOSCon Edition)Uwe Printz
 
A Rusty introduction to Apache Arrow and how it applies to a time series dat...
A Rusty introduction to Apache Arrow and how it applies to a  time series dat...A Rusty introduction to Apache Arrow and how it applies to a  time series dat...
A Rusty introduction to Apache Arrow and how it applies to a time series dat...Andrew Lamb
 
Yahoo - Moving beyond running 100% of Apache Pig jobs on Apache Tez
Yahoo - Moving beyond running 100% of Apache Pig jobs on Apache TezYahoo - Moving beyond running 100% of Apache Pig jobs on Apache Tez
Yahoo - Moving beyond running 100% of Apache Pig jobs on Apache TezDataWorks Summit
 
Easy, Scalable, Fault-tolerant stream processing with Structured Streaming in...
Easy, Scalable, Fault-tolerant stream processing with Structured Streaming in...Easy, Scalable, Fault-tolerant stream processing with Structured Streaming in...
Easy, Scalable, Fault-tolerant stream processing with Structured Streaming in...DataWorks Summit
 
Four Things to Know About Reliable Spark Streaming with Typesafe and Databricks
Four Things to Know About Reliable Spark Streaming with Typesafe and DatabricksFour Things to Know About Reliable Spark Streaming with Typesafe and Databricks
Four Things to Know About Reliable Spark Streaming with Typesafe and DatabricksLegacy Typesafe (now Lightbend)
 
Introduction to the Hadoop EcoSystem
Introduction to the Hadoop EcoSystemIntroduction to the Hadoop EcoSystem
Introduction to the Hadoop EcoSystemShivaji Dutta
 
Engineering fast indexes
Engineering fast indexesEngineering fast indexes
Engineering fast indexesDaniel Lemire
 
Strata NYC 2015 - What's coming for the Spark community
Strata NYC 2015 - What's coming for the Spark communityStrata NYC 2015 - What's coming for the Spark community
Strata NYC 2015 - What's coming for the Spark communityDatabricks
 
Hadoop and Spark for the SAS Developer
Hadoop and Spark for the SAS DeveloperHadoop and Spark for the SAS Developer
Hadoop and Spark for the SAS DeveloperDataWorks Summit
 
Apache® Spark™ 1.6 presented by Databricks co-founder Patrick Wendell
Apache® Spark™ 1.6 presented by Databricks co-founder Patrick WendellApache® Spark™ 1.6 presented by Databricks co-founder Patrick Wendell
Apache® Spark™ 1.6 presented by Databricks co-founder Patrick WendellDatabricks
 
Jump Start into Apache Spark (Seattle Spark Meetup)
Jump Start into Apache Spark (Seattle Spark Meetup)Jump Start into Apache Spark (Seattle Spark Meetup)
Jump Start into Apache Spark (Seattle Spark Meetup)Denny Lee
 
Spark what's new what's coming
Spark what's new what's comingSpark what's new what's coming
Spark what's new what's comingDatabricks
 
Optimizing Delta/Parquet Data Lakes for Apache Spark
Optimizing Delta/Parquet Data Lakes for Apache SparkOptimizing Delta/Parquet Data Lakes for Apache Spark
Optimizing Delta/Parquet Data Lakes for Apache SparkDatabricks
 
Datacubes in Apache Hive at ApacheCon
Datacubes in Apache Hive at ApacheConDatacubes in Apache Hive at ApacheCon
Datacubes in Apache Hive at ApacheConamarsri
 
ETL with SPARK - First Spark London meetup
ETL with SPARK - First Spark London meetupETL with SPARK - First Spark London meetup
ETL with SPARK - First Spark London meetupRafal Kwasny
 

Tendances (20)

Hadoop ecosystem
Hadoop ecosystemHadoop ecosystem
Hadoop ecosystem
 
Polyalgebra
PolyalgebraPolyalgebra
Polyalgebra
 
Spark ETL Techniques - Creating An Optimal Fantasy Baseball Roster
Spark ETL Techniques - Creating An Optimal Fantasy Baseball RosterSpark ETL Techniques - Creating An Optimal Fantasy Baseball Roster
Spark ETL Techniques - Creating An Optimal Fantasy Baseball Roster
 
Introduction to the Hadoop Ecosystem (FrOSCon Edition)
Introduction to the Hadoop Ecosystem (FrOSCon Edition)Introduction to the Hadoop Ecosystem (FrOSCon Edition)
Introduction to the Hadoop Ecosystem (FrOSCon Edition)
 
A Rusty introduction to Apache Arrow and how it applies to a time series dat...
A Rusty introduction to Apache Arrow and how it applies to a  time series dat...A Rusty introduction to Apache Arrow and how it applies to a  time series dat...
A Rusty introduction to Apache Arrow and how it applies to a time series dat...
 
Yahoo - Moving beyond running 100% of Apache Pig jobs on Apache Tez
Yahoo - Moving beyond running 100% of Apache Pig jobs on Apache TezYahoo - Moving beyond running 100% of Apache Pig jobs on Apache Tez
Yahoo - Moving beyond running 100% of Apache Pig jobs on Apache Tez
 
Easy, Scalable, Fault-tolerant stream processing with Structured Streaming in...
Easy, Scalable, Fault-tolerant stream processing with Structured Streaming in...Easy, Scalable, Fault-tolerant stream processing with Structured Streaming in...
Easy, Scalable, Fault-tolerant stream processing with Structured Streaming in...
 
Building data pipelines
Building data pipelinesBuilding data pipelines
Building data pipelines
 
Four Things to Know About Reliable Spark Streaming with Typesafe and Databricks
Four Things to Know About Reliable Spark Streaming with Typesafe and DatabricksFour Things to Know About Reliable Spark Streaming with Typesafe and Databricks
Four Things to Know About Reliable Spark Streaming with Typesafe and Databricks
 
Introduction to the Hadoop EcoSystem
Introduction to the Hadoop EcoSystemIntroduction to the Hadoop EcoSystem
Introduction to the Hadoop EcoSystem
 
Engineering fast indexes
Engineering fast indexesEngineering fast indexes
Engineering fast indexes
 
Strata NYC 2015 - What's coming for the Spark community
Strata NYC 2015 - What's coming for the Spark communityStrata NYC 2015 - What's coming for the Spark community
Strata NYC 2015 - What's coming for the Spark community
 
Hadoop and Spark for the SAS Developer
Hadoop and Spark for the SAS DeveloperHadoop and Spark for the SAS Developer
Hadoop and Spark for the SAS Developer
 
Apache® Spark™ 1.6 presented by Databricks co-founder Patrick Wendell
Apache® Spark™ 1.6 presented by Databricks co-founder Patrick WendellApache® Spark™ 1.6 presented by Databricks co-founder Patrick Wendell
Apache® Spark™ 1.6 presented by Databricks co-founder Patrick Wendell
 
Jump Start into Apache Spark (Seattle Spark Meetup)
Jump Start into Apache Spark (Seattle Spark Meetup)Jump Start into Apache Spark (Seattle Spark Meetup)
Jump Start into Apache Spark (Seattle Spark Meetup)
 
Apache Spark & Hadoop
Apache Spark & HadoopApache Spark & Hadoop
Apache Spark & Hadoop
 
Spark what's new what's coming
Spark what's new what's comingSpark what's new what's coming
Spark what's new what's coming
 
Optimizing Delta/Parquet Data Lakes for Apache Spark
Optimizing Delta/Parquet Data Lakes for Apache SparkOptimizing Delta/Parquet Data Lakes for Apache Spark
Optimizing Delta/Parquet Data Lakes for Apache Spark
 
Datacubes in Apache Hive at ApacheCon
Datacubes in Apache Hive at ApacheConDatacubes in Apache Hive at ApacheCon
Datacubes in Apache Hive at ApacheCon
 
ETL with SPARK - First Spark London meetup
ETL with SPARK - First Spark London meetupETL with SPARK - First Spark London meetup
ETL with SPARK - First Spark London meetup
 

Similaire à Working with the Scalding Type -Safe API

Reactive Stream Processing Using DDS and Rx
Reactive Stream Processing Using DDS and RxReactive Stream Processing Using DDS and Rx
Reactive Stream Processing Using DDS and RxSumant Tambe
 
Kerberizing Spark: Spark Summit East talk by Abel Rincon and Jorge Lopez-Malla
Kerberizing Spark: Spark Summit East talk by Abel Rincon and Jorge Lopez-MallaKerberizing Spark: Spark Summit East talk by Abel Rincon and Jorge Lopez-Malla
Kerberizing Spark: Spark Summit East talk by Abel Rincon and Jorge Lopez-MallaSpark Summit
 
Cloud Orchestration with RightScale Cloud Workflow
Cloud Orchestration with RightScale Cloud WorkflowCloud Orchestration with RightScale Cloud Workflow
Cloud Orchestration with RightScale Cloud WorkflowRightScale
 
Intro to Apache Spark by Marco Vasquez
Intro to Apache Spark by Marco VasquezIntro to Apache Spark by Marco Vasquez
Intro to Apache Spark by Marco VasquezMapR Technologies
 
CDK Meetup: Rule the World through IaC
CDK Meetup: Rule the World through IaCCDK Meetup: Rule the World through IaC
CDK Meetup: Rule the World through IaCsmalltown
 
Codestrong 2012 breakout session hacking titanium
Codestrong 2012 breakout session   hacking titaniumCodestrong 2012 breakout session   hacking titanium
Codestrong 2012 breakout session hacking titaniumAxway Appcelerator
 
Apache Spark Streaming: Architecture and Fault Tolerance
Apache Spark Streaming: Architecture and Fault ToleranceApache Spark Streaming: Architecture and Fault Tolerance
Apache Spark Streaming: Architecture and Fault ToleranceSachin Aggarwal
 
PyTorch 튜토리얼 (Touch to PyTorch)
PyTorch 튜토리얼 (Touch to PyTorch)PyTorch 튜토리얼 (Touch to PyTorch)
PyTorch 튜토리얼 (Touch to PyTorch)Hansol Kang
 
Crossing the Bridge: Connecting Rails and your Front-end Framework
Crossing the Bridge: Connecting Rails and your Front-end FrameworkCrossing the Bridge: Connecting Rails and your Front-end Framework
Crossing the Bridge: Connecting Rails and your Front-end FrameworkDaniel Spector
 
Debasihish da final.ppt
Debasihish da final.pptDebasihish da final.ppt
Debasihish da final.pptKalkey
 
Distributed and Fault Tolerant Realtime Computation with Apache Storm, Apache...
Distributed and Fault Tolerant Realtime Computation with Apache Storm, Apache...Distributed and Fault Tolerant Realtime Computation with Apache Storm, Apache...
Distributed and Fault Tolerant Realtime Computation with Apache Storm, Apache...Folio3 Software
 
Jump Start into Apache® Spark™ and Databricks
Jump Start into Apache® Spark™ and DatabricksJump Start into Apache® Spark™ and Databricks
Jump Start into Apache® Spark™ and DatabricksDatabricks
 
Tajo_Meetup_20141120
Tajo_Meetup_20141120Tajo_Meetup_20141120
Tajo_Meetup_20141120Hyoungjun Kim
 
Gruter_TECHDAY_2014_03_ApacheTajo (in Korean)
Gruter_TECHDAY_2014_03_ApacheTajo (in Korean)Gruter_TECHDAY_2014_03_ApacheTajo (in Korean)
Gruter_TECHDAY_2014_03_ApacheTajo (in Korean)Gruter
 
Terraform: Cloud Configuration Management (WTC/IPC'16)
Terraform: Cloud Configuration Management (WTC/IPC'16)Terraform: Cloud Configuration Management (WTC/IPC'16)
Terraform: Cloud Configuration Management (WTC/IPC'16)Martin Schütte
 
Spring Cloud Data Flow Overview
Spring Cloud Data Flow OverviewSpring Cloud Data Flow Overview
Spring Cloud Data Flow OverviewVMware Tanzu
 
Cascading - A Java Developer’s Companion to the Hadoop World
Cascading - A Java Developer’s Companion to the Hadoop WorldCascading - A Java Developer’s Companion to the Hadoop World
Cascading - A Java Developer’s Companion to the Hadoop WorldCascading
 
Reusable, composable, battle-tested Terraform modules
Reusable, composable, battle-tested Terraform modulesReusable, composable, battle-tested Terraform modules
Reusable, composable, battle-tested Terraform modulesYevgeniy Brikman
 
服务框架: Thrift & PasteScript
服务框架: Thrift & PasteScript服务框架: Thrift & PasteScript
服务框架: Thrift & PasteScriptQiangning Hong
 

Similaire à Working with the Scalding Type -Safe API (20)

Spark etl
Spark etlSpark etl
Spark etl
 
Reactive Stream Processing Using DDS and Rx
Reactive Stream Processing Using DDS and RxReactive Stream Processing Using DDS and Rx
Reactive Stream Processing Using DDS and Rx
 
Kerberizing Spark: Spark Summit East talk by Abel Rincon and Jorge Lopez-Malla
Kerberizing Spark: Spark Summit East talk by Abel Rincon and Jorge Lopez-MallaKerberizing Spark: Spark Summit East talk by Abel Rincon and Jorge Lopez-Malla
Kerberizing Spark: Spark Summit East talk by Abel Rincon and Jorge Lopez-Malla
 
Cloud Orchestration with RightScale Cloud Workflow
Cloud Orchestration with RightScale Cloud WorkflowCloud Orchestration with RightScale Cloud Workflow
Cloud Orchestration with RightScale Cloud Workflow
 
Intro to Apache Spark by Marco Vasquez
Intro to Apache Spark by Marco VasquezIntro to Apache Spark by Marco Vasquez
Intro to Apache Spark by Marco Vasquez
 
CDK Meetup: Rule the World through IaC
CDK Meetup: Rule the World through IaCCDK Meetup: Rule the World through IaC
CDK Meetup: Rule the World through IaC
 
Codestrong 2012 breakout session hacking titanium
Codestrong 2012 breakout session   hacking titaniumCodestrong 2012 breakout session   hacking titanium
Codestrong 2012 breakout session hacking titanium
 
Apache Spark Streaming: Architecture and Fault Tolerance
Apache Spark Streaming: Architecture and Fault ToleranceApache Spark Streaming: Architecture and Fault Tolerance
Apache Spark Streaming: Architecture and Fault Tolerance
 
PyTorch 튜토리얼 (Touch to PyTorch)
PyTorch 튜토리얼 (Touch to PyTorch)PyTorch 튜토리얼 (Touch to PyTorch)
PyTorch 튜토리얼 (Touch to PyTorch)
 
Crossing the Bridge: Connecting Rails and your Front-end Framework
Crossing the Bridge: Connecting Rails and your Front-end FrameworkCrossing the Bridge: Connecting Rails and your Front-end Framework
Crossing the Bridge: Connecting Rails and your Front-end Framework
 
Debasihish da final.ppt
Debasihish da final.pptDebasihish da final.ppt
Debasihish da final.ppt
 
Distributed and Fault Tolerant Realtime Computation with Apache Storm, Apache...
Distributed and Fault Tolerant Realtime Computation with Apache Storm, Apache...Distributed and Fault Tolerant Realtime Computation with Apache Storm, Apache...
Distributed and Fault Tolerant Realtime Computation with Apache Storm, Apache...
 
Jump Start into Apache® Spark™ and Databricks
Jump Start into Apache® Spark™ and DatabricksJump Start into Apache® Spark™ and Databricks
Jump Start into Apache® Spark™ and Databricks
 
Tajo_Meetup_20141120
Tajo_Meetup_20141120Tajo_Meetup_20141120
Tajo_Meetup_20141120
 
Gruter_TECHDAY_2014_03_ApacheTajo (in Korean)
Gruter_TECHDAY_2014_03_ApacheTajo (in Korean)Gruter_TECHDAY_2014_03_ApacheTajo (in Korean)
Gruter_TECHDAY_2014_03_ApacheTajo (in Korean)
 
Terraform: Cloud Configuration Management (WTC/IPC'16)
Terraform: Cloud Configuration Management (WTC/IPC'16)Terraform: Cloud Configuration Management (WTC/IPC'16)
Terraform: Cloud Configuration Management (WTC/IPC'16)
 
Spring Cloud Data Flow Overview
Spring Cloud Data Flow OverviewSpring Cloud Data Flow Overview
Spring Cloud Data Flow Overview
 
Cascading - A Java Developer’s Companion to the Hadoop World
Cascading - A Java Developer’s Companion to the Hadoop WorldCascading - A Java Developer’s Companion to the Hadoop World
Cascading - A Java Developer’s Companion to the Hadoop World
 
Reusable, composable, battle-tested Terraform modules
Reusable, composable, battle-tested Terraform modulesReusable, composable, battle-tested Terraform modules
Reusable, composable, battle-tested Terraform modules
 
服务框架: Thrift & PasteScript
服务框架: Thrift & PasteScript服务框架: Thrift & PasteScript
服务框架: Thrift & PasteScript
 

Plus de DataWorks Summit/Hadoop Summit

Unleashing the Power of Apache Atlas with Apache Ranger
Unleashing the Power of Apache Atlas with Apache RangerUnleashing the Power of Apache Atlas with Apache Ranger
Unleashing the Power of Apache Atlas with Apache RangerDataWorks Summit/Hadoop Summit
 
Enabling Digital Diagnostics with a Data Science Platform
Enabling Digital Diagnostics with a Data Science PlatformEnabling Digital Diagnostics with a Data Science Platform
Enabling Digital Diagnostics with a Data Science PlatformDataWorks Summit/Hadoop Summit
 
Double Your Hadoop Performance with Hortonworks SmartSense
Double Your Hadoop Performance with Hortonworks SmartSenseDouble Your Hadoop Performance with Hortonworks SmartSense
Double Your Hadoop Performance with Hortonworks SmartSenseDataWorks Summit/Hadoop Summit
 
Building a Large-Scale, Adaptive Recommendation Engine with Apache Flink and ...
Building a Large-Scale, Adaptive Recommendation Engine with Apache Flink and ...Building a Large-Scale, Adaptive Recommendation Engine with Apache Flink and ...
Building a Large-Scale, Adaptive Recommendation Engine with Apache Flink and ...DataWorks Summit/Hadoop Summit
 
Real-Time Anomaly Detection using LSTM Auto-Encoders with Deep Learning4J on ...
Real-Time Anomaly Detection using LSTM Auto-Encoders with Deep Learning4J on ...Real-Time Anomaly Detection using LSTM Auto-Encoders with Deep Learning4J on ...
Real-Time Anomaly Detection using LSTM Auto-Encoders with Deep Learning4J on ...DataWorks Summit/Hadoop Summit
 
Mool - Automated Log Analysis using Data Science and ML
Mool - Automated Log Analysis using Data Science and MLMool - Automated Log Analysis using Data Science and ML
Mool - Automated Log Analysis using Data Science and MLDataWorks Summit/Hadoop Summit
 
The Challenge of Driving Business Value from the Analytics of Things (AOT)
The Challenge of Driving Business Value from the Analytics of Things (AOT)The Challenge of Driving Business Value from the Analytics of Things (AOT)
The Challenge of Driving Business Value from the Analytics of Things (AOT)DataWorks Summit/Hadoop Summit
 
From Regulatory Process Verification to Predictive Maintenance and Beyond wit...
From Regulatory Process Verification to Predictive Maintenance and Beyond wit...From Regulatory Process Verification to Predictive Maintenance and Beyond wit...
From Regulatory Process Verification to Predictive Maintenance and Beyond wit...DataWorks Summit/Hadoop Summit
 

Plus de DataWorks Summit/Hadoop Summit (20)

Running Apache Spark & Apache Zeppelin in Production
Running Apache Spark & Apache Zeppelin in ProductionRunning Apache Spark & Apache Zeppelin in Production
Running Apache Spark & Apache Zeppelin in Production
 
State of Security: Apache Spark & Apache Zeppelin
State of Security: Apache Spark & Apache ZeppelinState of Security: Apache Spark & Apache Zeppelin
State of Security: Apache Spark & Apache Zeppelin
 
Unleashing the Power of Apache Atlas with Apache Ranger
Unleashing the Power of Apache Atlas with Apache RangerUnleashing the Power of Apache Atlas with Apache Ranger
Unleashing the Power of Apache Atlas with Apache Ranger
 
Enabling Digital Diagnostics with a Data Science Platform
Enabling Digital Diagnostics with a Data Science PlatformEnabling Digital Diagnostics with a Data Science Platform
Enabling Digital Diagnostics with a Data Science Platform
 
Revolutionize Text Mining with Spark and Zeppelin
Revolutionize Text Mining with Spark and ZeppelinRevolutionize Text Mining with Spark and Zeppelin
Revolutionize Text Mining with Spark and Zeppelin
 
Double Your Hadoop Performance with Hortonworks SmartSense
Double Your Hadoop Performance with Hortonworks SmartSenseDouble Your Hadoop Performance with Hortonworks SmartSense
Double Your Hadoop Performance with Hortonworks SmartSense
 
Hadoop Crash Course
Hadoop Crash CourseHadoop Crash Course
Hadoop Crash Course
 
Data Science Crash Course
Data Science Crash CourseData Science Crash Course
Data Science Crash Course
 
Apache Spark Crash Course
Apache Spark Crash CourseApache Spark Crash Course
Apache Spark Crash Course
 
Dataflow with Apache NiFi
Dataflow with Apache NiFiDataflow with Apache NiFi
Dataflow with Apache NiFi
 
Schema Registry - Set you Data Free
Schema Registry - Set you Data FreeSchema Registry - Set you Data Free
Schema Registry - Set you Data Free
 
Building a Large-Scale, Adaptive Recommendation Engine with Apache Flink and ...
Building a Large-Scale, Adaptive Recommendation Engine with Apache Flink and ...Building a Large-Scale, Adaptive Recommendation Engine with Apache Flink and ...
Building a Large-Scale, Adaptive Recommendation Engine with Apache Flink and ...
 
Real-Time Anomaly Detection using LSTM Auto-Encoders with Deep Learning4J on ...
Real-Time Anomaly Detection using LSTM Auto-Encoders with Deep Learning4J on ...Real-Time Anomaly Detection using LSTM Auto-Encoders with Deep Learning4J on ...
Real-Time Anomaly Detection using LSTM Auto-Encoders with Deep Learning4J on ...
 
Mool - Automated Log Analysis using Data Science and ML
Mool - Automated Log Analysis using Data Science and MLMool - Automated Log Analysis using Data Science and ML
Mool - Automated Log Analysis using Data Science and ML
 
How Hadoop Makes the Natixis Pack More Efficient
How Hadoop Makes the Natixis Pack More Efficient How Hadoop Makes the Natixis Pack More Efficient
How Hadoop Makes the Natixis Pack More Efficient
 
HBase in Practice
HBase in Practice HBase in Practice
HBase in Practice
 
The Challenge of Driving Business Value from the Analytics of Things (AOT)
The Challenge of Driving Business Value from the Analytics of Things (AOT)The Challenge of Driving Business Value from the Analytics of Things (AOT)
The Challenge of Driving Business Value from the Analytics of Things (AOT)
 
Breaking the 1 Million OPS/SEC Barrier in HOPS Hadoop
Breaking the 1 Million OPS/SEC Barrier in HOPS HadoopBreaking the 1 Million OPS/SEC Barrier in HOPS Hadoop
Breaking the 1 Million OPS/SEC Barrier in HOPS Hadoop
 
From Regulatory Process Verification to Predictive Maintenance and Beyond wit...
From Regulatory Process Verification to Predictive Maintenance and Beyond wit...From Regulatory Process Verification to Predictive Maintenance and Beyond wit...
From Regulatory Process Verification to Predictive Maintenance and Beyond wit...
 
Backup and Disaster Recovery in Hadoop
Backup and Disaster Recovery in Hadoop Backup and Disaster Recovery in Hadoop
Backup and Disaster Recovery in Hadoop
 

Dernier

Search Engine Optimization SEO PDF for 2024.pdf
Search Engine Optimization SEO PDF for 2024.pdfSearch Engine Optimization SEO PDF for 2024.pdf
Search Engine Optimization SEO PDF for 2024.pdfRankYa
 
Anypoint Exchange: It’s Not Just a Repo!
Anypoint Exchange: It’s Not Just a Repo!Anypoint Exchange: It’s Not Just a Repo!
Anypoint Exchange: It’s Not Just a Repo!Manik S Magar
 
Are Multi-Cloud and Serverless Good or Bad?
Are Multi-Cloud and Serverless Good or Bad?Are Multi-Cloud and Serverless Good or Bad?
Are Multi-Cloud and Serverless Good or Bad?Mattias Andersson
 
Kotlin Multiplatform & Compose Multiplatform - Starter kit for pragmatics
Kotlin Multiplatform & Compose Multiplatform - Starter kit for pragmaticsKotlin Multiplatform & Compose Multiplatform - Starter kit for pragmatics
Kotlin Multiplatform & Compose Multiplatform - Starter kit for pragmaticscarlostorres15106
 
DevoxxFR 2024 Reproducible Builds with Apache Maven
DevoxxFR 2024 Reproducible Builds with Apache MavenDevoxxFR 2024 Reproducible Builds with Apache Maven
DevoxxFR 2024 Reproducible Builds with Apache MavenHervé Boutemy
 
What's New in Teams Calling, Meetings and Devices March 2024
What's New in Teams Calling, Meetings and Devices March 2024What's New in Teams Calling, Meetings and Devices March 2024
What's New in Teams Calling, Meetings and Devices March 2024Stephanie Beckett
 
Transcript: New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024
Transcript: New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024Transcript: New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024
Transcript: New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024BookNet Canada
 
"ML in Production",Oleksandr Bagan
"ML in Production",Oleksandr Bagan"ML in Production",Oleksandr Bagan
"ML in Production",Oleksandr BaganFwdays
 
Leverage Zilliz Serverless - Up to 50X Saving for Your Vector Storage Cost
Leverage Zilliz Serverless - Up to 50X Saving for Your Vector Storage CostLeverage Zilliz Serverless - Up to 50X Saving for Your Vector Storage Cost
Leverage Zilliz Serverless - Up to 50X Saving for Your Vector Storage CostZilliz
 
Scanning the Internet for External Cloud Exposures via SSL Certs
Scanning the Internet for External Cloud Exposures via SSL CertsScanning the Internet for External Cloud Exposures via SSL Certs
Scanning the Internet for External Cloud Exposures via SSL CertsRizwan Syed
 
SAP Build Work Zone - Overview L2-L3.pptx
SAP Build Work Zone - Overview L2-L3.pptxSAP Build Work Zone - Overview L2-L3.pptx
SAP Build Work Zone - Overview L2-L3.pptxNavinnSomaal
 
"Subclassing and Composition – A Pythonic Tour of Trade-Offs", Hynek Schlawack
"Subclassing and Composition – A Pythonic Tour of Trade-Offs", Hynek Schlawack"Subclassing and Composition – A Pythonic Tour of Trade-Offs", Hynek Schlawack
"Subclassing and Composition – A Pythonic Tour of Trade-Offs", Hynek SchlawackFwdays
 
AI as an Interface for Commercial Buildings
AI as an Interface for Commercial BuildingsAI as an Interface for Commercial Buildings
AI as an Interface for Commercial BuildingsMemoori
 
Dev Dives: Streamline document processing with UiPath Studio Web
Dev Dives: Streamline document processing with UiPath Studio WebDev Dives: Streamline document processing with UiPath Studio Web
Dev Dives: Streamline document processing with UiPath Studio WebUiPathCommunity
 
Gen AI in Business - Global Trends Report 2024.pdf
Gen AI in Business - Global Trends Report 2024.pdfGen AI in Business - Global Trends Report 2024.pdf
Gen AI in Business - Global Trends Report 2024.pdfAddepto
 
Training state-of-the-art general text embedding
Training state-of-the-art general text embeddingTraining state-of-the-art general text embedding
Training state-of-the-art general text embeddingZilliz
 
Unleash Your Potential - Namagunga Girls Coding Club
Unleash Your Potential - Namagunga Girls Coding ClubUnleash Your Potential - Namagunga Girls Coding Club
Unleash Your Potential - Namagunga Girls Coding ClubKalema Edgar
 
New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024
New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024
New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024BookNet Canada
 
Beyond Boundaries: Leveraging No-Code Solutions for Industry Innovation
Beyond Boundaries: Leveraging No-Code Solutions for Industry InnovationBeyond Boundaries: Leveraging No-Code Solutions for Industry Innovation
Beyond Boundaries: Leveraging No-Code Solutions for Industry InnovationSafe Software
 

Dernier (20)

Search Engine Optimization SEO PDF for 2024.pdf
Search Engine Optimization SEO PDF for 2024.pdfSearch Engine Optimization SEO PDF for 2024.pdf
Search Engine Optimization SEO PDF for 2024.pdf
 
Anypoint Exchange: It’s Not Just a Repo!
Anypoint Exchange: It’s Not Just a Repo!Anypoint Exchange: It’s Not Just a Repo!
Anypoint Exchange: It’s Not Just a Repo!
 
Are Multi-Cloud and Serverless Good or Bad?
Are Multi-Cloud and Serverless Good or Bad?Are Multi-Cloud and Serverless Good or Bad?
Are Multi-Cloud and Serverless Good or Bad?
 
E-Vehicle_Hacking_by_Parul Sharma_null_owasp.pptx
E-Vehicle_Hacking_by_Parul Sharma_null_owasp.pptxE-Vehicle_Hacking_by_Parul Sharma_null_owasp.pptx
E-Vehicle_Hacking_by_Parul Sharma_null_owasp.pptx
 
Kotlin Multiplatform & Compose Multiplatform - Starter kit for pragmatics
Kotlin Multiplatform & Compose Multiplatform - Starter kit for pragmaticsKotlin Multiplatform & Compose Multiplatform - Starter kit for pragmatics
Kotlin Multiplatform & Compose Multiplatform - Starter kit for pragmatics
 
DevoxxFR 2024 Reproducible Builds with Apache Maven
DevoxxFR 2024 Reproducible Builds with Apache MavenDevoxxFR 2024 Reproducible Builds with Apache Maven
DevoxxFR 2024 Reproducible Builds with Apache Maven
 
What's New in Teams Calling, Meetings and Devices March 2024
What's New in Teams Calling, Meetings and Devices March 2024What's New in Teams Calling, Meetings and Devices March 2024
What's New in Teams Calling, Meetings and Devices March 2024
 
Transcript: New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024
Transcript: New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024Transcript: New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024
Transcript: New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024
 
"ML in Production",Oleksandr Bagan
"ML in Production",Oleksandr Bagan"ML in Production",Oleksandr Bagan
"ML in Production",Oleksandr Bagan
 
Leverage Zilliz Serverless - Up to 50X Saving for Your Vector Storage Cost
Leverage Zilliz Serverless - Up to 50X Saving for Your Vector Storage CostLeverage Zilliz Serverless - Up to 50X Saving for Your Vector Storage Cost
Leverage Zilliz Serverless - Up to 50X Saving for Your Vector Storage Cost
 
Scanning the Internet for External Cloud Exposures via SSL Certs
Scanning the Internet for External Cloud Exposures via SSL CertsScanning the Internet for External Cloud Exposures via SSL Certs
Scanning the Internet for External Cloud Exposures via SSL Certs
 
SAP Build Work Zone - Overview L2-L3.pptx
SAP Build Work Zone - Overview L2-L3.pptxSAP Build Work Zone - Overview L2-L3.pptx
SAP Build Work Zone - Overview L2-L3.pptx
 
"Subclassing and Composition – A Pythonic Tour of Trade-Offs", Hynek Schlawack
"Subclassing and Composition – A Pythonic Tour of Trade-Offs", Hynek Schlawack"Subclassing and Composition – A Pythonic Tour of Trade-Offs", Hynek Schlawack
"Subclassing and Composition – A Pythonic Tour of Trade-Offs", Hynek Schlawack
 
AI as an Interface for Commercial Buildings
AI as an Interface for Commercial BuildingsAI as an Interface for Commercial Buildings
AI as an Interface for Commercial Buildings
 
Dev Dives: Streamline document processing with UiPath Studio Web
Dev Dives: Streamline document processing with UiPath Studio WebDev Dives: Streamline document processing with UiPath Studio Web
Dev Dives: Streamline document processing with UiPath Studio Web
 
Gen AI in Business - Global Trends Report 2024.pdf
Gen AI in Business - Global Trends Report 2024.pdfGen AI in Business - Global Trends Report 2024.pdf
Gen AI in Business - Global Trends Report 2024.pdf
 
Training state-of-the-art general text embedding
Training state-of-the-art general text embeddingTraining state-of-the-art general text embedding
Training state-of-the-art general text embedding
 
Unleash Your Potential - Namagunga Girls Coding Club
Unleash Your Potential - Namagunga Girls Coding ClubUnleash Your Potential - Namagunga Girls Coding Club
Unleash Your Potential - Namagunga Girls Coding Club
 
New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024
New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024
New from BookNet Canada for 2024: BNC CataList - Tech Forum 2024
 
Beyond Boundaries: Leveraging No-Code Solutions for Industry Innovation
Beyond Boundaries: Leveraging No-Code Solutions for Industry InnovationBeyond Boundaries: Leveraging No-Code Solutions for Industry Innovation
Beyond Boundaries: Leveraging No-Code Solutions for Industry Innovation
 

Working with the Scalding Type -Safe API

  • 1. 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
  • 2. 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
  • 3. 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
  • 4. 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")))
  • 5. Copyright © 2014 Criteo Our Reality 5 TypedPipe.from(SomeSource(args("some-place")) .map { TargetType.fromClassWith23Fields } .groupBy { _.lotsOfDimensions } .sumByKey .write(TypedSink[TargetType](args("another-place")))
  • 6. 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}
  • 7. 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
  • 8. 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
  • 9. 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
  • 10. 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
  • 11. Copyright © 2014 Criteo Some Context Before we dig into the nitty-gritty, I think some context is in order. 11
  • 12. 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
  • 13. 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
  • 14. 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
  • 15. Copyright © 2014 Criteo Langoustine Quick UI Tour
  • 16. Copyright © 2014 Criteo Langoustine Quick UI Tour
  • 17. Copyright © 2014 Criteo Langoustine Quick UI Tour Red is bad.
  • 18. 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 }
  • 19. Copyright © 2014 Criteo Langoustine App object HelloWorldApp extends DefaultLangoustineLauncher { override lazy val run = execute(HelloWorldWorkflow.jobs) }
  • 20. Copyright © 2014 Criteo File Formats at Criteo • Our workflow consumes Parquet, Pail/JSON and Text-Delimited data 20
  • 21. 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
  • 22. Copyright © 2014 Criteo The Type-Safe API It compiles, it works! (hardy har har) 22
  • 23. Copyright © 2014 Criteo 23+ Field Logs 23+ fields and Scala 2.10, means writing your own TupleSetters and Converters 23
  • 24. 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
  • 25. Copyright © 2014 Criteo Fine, then Where’s the beef? 25
  • 26. 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
  • 27. 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 }
  • 28. 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 = ??? }
  • 29. Copyright © 2014 Criteo Fun, right? Remind any one of working with JDBC? 29
  • 30. 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]
  • 31. 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) }
  • 32. 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] = ??? }
  • 33. 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
  • 34. 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) ) ) }
  • 35. 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) ) ) }
  • 36. 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
  • 37. 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
  • 38. 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
  • 39. 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) } } }
  • 40. 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) }
  • 41. 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) ) ) }
  • 42. 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) ) }
  • 43. 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
  • 44. 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! ) ) }
  • 45. 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
  • 46. 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)}” ) }
  • 47. 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) ) ) }
  • 48. 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
  • 49. 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)) }
  • 50. 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) ) ) }
  • 51. 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
  • 52. Copyright © 2014 Criteo Execution Optimization If you’re still with us, here’s a bit more detail on run-time optimizations 52
  • 53. 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)
  • 54. 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
  • 55. 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
  • 56. 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)
  • 57. 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
  • 58. 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.
  • 59. 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
  • 60. Copyright © 2014 Criteo Performance hints: mappers 60
  • 61. 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
  • 62. 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
  • 63. 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
  • 64. Copyright © 2014 Criteo A final word We’re hiring. 64
  • 65. Copyright © 2014 Criteo And we have a nice deck 65
  • 66. Copyright © 2014 Criteo Thank You! Justin Coffey, j.coffey@criteo.com, @jqcoffey Sofian 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!