The Twitter Service Stack - files.meetup.comfiles.meetup.com/2891482/twitterservicestack.pdf · TM...

Post on 04-Oct-2020

0 views 0 download

transcript

TM

The Twitter Service Stack

Marius Eriksen - @mariusBASE 2012/04/25

Monday, April 30, 12

Motivation/background5-minute Scala introComposable futuresServicesFinagleObservabilityConclusion

Agenda

Monday, April 30, 12

serv·ice stack |ˈsərvis stak|

Software infrastructure, for use in services, that is application agnostic.

Comprises a bunch of libraries, loosely coupled.

I’m going to be talking (mostly) about our RPC system.

Monday, April 30, 12

Heavy users of open source projectsMySQL, memcached, redis, ruby(-on-rails), Hadoop, ZooKeeper, Lucene, etc. etc.

Contribute back. Goal: Twitter agnostic infrastructure is open sourced.

github.com/twitterA heterogenous world

Languages, protocols, libraries, runtimes

Twitter infrastructure

Monday, April 30, 12

Monday, April 30, 12

Open source = heterogeneityAntimodular

Thrift, memcache, mysql, zookeeper share no code, or even interfaces.

Difficult to:Reuse code, reason about behavior and performance, address common concerns, observe the system

An RPC system?

Monday, April 30, 12

In fact,[I claim that] The majority of the difficult problems in RPC systems are entirely protocol agnostic.

Monday, April 30, 12

The JVMHotspot, great GC, instrumentation, experience, support for high level languages

Must support any protocolHTTP, Thrift, memcache, redis, MySQL

Must be productiveMust be performantHighly concurrent services

Starting from scratch

Monday, April 30, 12

A (short) detour to Scala

Monday, April 30, 12

First released in 2003 (Odersky at EPFL)Runs primarily on the JVMMixes OO and FPStrong static typingExpressive, conciseInteroperates naturally with JavaSimple in syntax, but with some corners of semantic complexityHas a REPL

Scala elevator pitch

Monday, April 30, 12

: Map[Int, String] [Int, String]

Static typing

val m =Map (1 -> “one”, 2 -> “two”)

Monday, April 30, 12

.. inferred

val m = Map(1 -> “one”, 2 -> “two”)

Monday, April 30, 12

(1, “ok”): (Int, String)

Tuples

Monday, April 30, 12

Function1<Integer, Integer> f = new Function1<Integer, Integer>() { public Integer apply(Integer ) { return x*2 } }

function.java

f

x*2x

Monday, April 30, 12

val = new Function1[Int, Int] { def apply( : Int) = x*2}

function.scala

fx*2x

Monday, April 30, 12

val = { => }

Lightweight syntax!

f x*2x*22x

Monday, April 30, 12

Lightweight syntax!!

val f = _* f 2

Monday, April 30, 12

def schedule(f: () => Unit)

schedule({ () => println("hello, world!")})

(Lightweight syntax)

Monday, April 30, 12

def schedule(f: () => Unit)

schedule { () => println("hello, world!")}

Lightweight syntax

Monday, April 30, 12

def manytimes(n: Int)(f: => Unit) { (0 until n) foreach { _ => f }}

manytimes(100) { println("ok")}

Call by name

Monday, April 30, 12

val f: Int => String = { ...

val g: String => Float = { ...

val h: Int => Float = g compose f // g.compose(f)

A pattern

Monday, April 30, 12

val l = List(1,2,3,4)

val l2 = l map { elem => elem*2 } // l.map({elem => elem*2})

l == List(1,2,3,4)l2 == List(2,4,6,8)

A pattern

Monday, April 30, 12

flatMap[B](f: A⇒List[B]): List[B]

Monday, April 30, 12

List(1,2,3,4) flatMap { elem =>

List(elem, -elem)

} // == List(1,-1,2,-2,3,-3,4,-4)

Monday, April 30, 12

Stream.from(1) flatMap { z =>

Stream.range(1, z) flatMap { x =>

Stream.range(x, z) flatMap { y =>

if (x*x + y*y == z*z) Stream((x, y, z))

else Stream()

}

}

}

flatMap

Monday, April 30, 12

for {

z <- Stream.from(1)

x <- Stream.range(1, z)

y <- Stream.range(x, z)

if x*x + y*y == z*z

} yield (x, y, z)

for-comprehensions

Monday, April 30, 12

Futures

(≠ java.util.concurrent.Future)

Monday, April 30, 12

A deferred value.A long computationAn RPCReading from disk

Computations can fail!Connection failureComputation takes too longArithmetic exception

Futures

Monday, April 30, 12

val f: Future[String]

// wait indefinitelyval result = f.get()

// wait 1 secondval result = f.get(1.second)

Using futures

Monday, April 30, 12

val f: Future[String]

f onSuccess { s => println(“got string “+s)} onFailure { exc => exc.printStackTrace()}

Callbacks

Monday, April 30, 12

ValuesFuture.value[A](a: A): Future[A]Future.exception[A](e: Throwable): Future[A]

A promise is a writable futureval p = new Promise[Int]val f: Future[Int] = p

p.setValue(1)p.setException(new Exception)

Making futures

Monday, April 30, 12

val p = new Promise[Int]p onSuccess { i => println(“value: %d”.format(i))}println(“setting the promise”)p.setValue(1)

Callbacks

Monday, April 30, 12

val p = new Promise[Int]p onSuccess { i => println(“value: %d”.format(i))}println(“setting the promise”)p.setValue(1)

Callbacks

Monday, April 30, 12

val p = new Promise[Int]p onSuccess { i => println(“value: %d”.format(i))}println(“setting the promise”)p.setValue(1)

Callbacks

Monday, April 30, 12

val p = new Promise[Int]p onSuccess { i => println(“value: %d”.format(i))}println(“setting the promise”)p.setValue(1)

Callbacks

Monday, April 30, 12

val p = new Promise[Int]p onSuccess { i => println(“value: %d”.format(i))}println(“setting the promise”)p.setValue(1)

Callbacks

Monday, April 30, 12

val p = new Promise[Int]p onSuccess { i => println(“value: %d”.format(i))}println(“setting the promise”)p.setValue(1)

Callbacks

Monday, April 30, 12

List(1,2,3,4) map { i => i * 2 } // == List(2,4,6,8)

Future.value(1) map { i => i * 2} // == Future.value(2)

Futures are like Lists!

Monday, April 30, 12

flatMap[B](f: A⇒List[B]): List[B]

flatMap?

flatMap[B](f: A⇒ [B]): [B]

Monday, April 30, 12

flatMap[B](f: A⇒Future[B]): Future[B]flatMap[B](f: A⇒ [B]): [B]

Monday, April 30, 12

List(1,2,3,4) flatMap { i => List(i*2) } // == List(2,4,6,8)

Future.value(1) flatMap { i => Future.value(i*2)} // == Future.value(2)

Futures are like Lists!

Monday, April 30, 12

def auth(token: String): Future[Long]def getUser(id: Long) : Future[User]

flatMap

def getUser(token: String): Future[User]

Monday, April 30, 12

flatMap

def getUser(token: String): Future[User] = auth(token) flatMap { id => getUser(id) }

Monday, April 30, 12

Analogous to Scala/Java exceptionsan exception stops a computationcomputation can be recovered by the first handler able to do so

ScopeScala/Java exceptions: stack framesFutures: “transformer” stack

map, flatMap :: handle, rescue

Exceptions

Monday, April 30, 12

val f = lookupUser(123) handle { case UserNotFound => NullUserObject}

Exceptions

Monday, April 30, 12

val f = lookupUser(123) rescue { case ServiceUnavailable => lookupUser(123) // try again}

Exceptions

Monday, April 30, 12

Declarative transformations of dataEvaluation strategy is up to implementationYou don’t prescribe control flow, instead you declare the meaning of a value:auth is the the result of looking up a user, then authenticating the user object, recovering UserLookupError into the NullUserObject

Functional style

Monday, April 30, 12

val ops: = Seq(rpc(), rpc(), rpc())

Future.collect

Seq[Future[Res]]

Monday, April 30, 12

val res: = Future.collect(ops)

Future.collect

Seq[Future[ Res ]]

Monday, April 30, 12

val f1: Future[A]val f2: Future[B]

val f: Future[Unit] = Future.join(f1, f2)

f onSuccess { _ => println(“everything’s done”)}

Future.join

Monday, April 30, 12

val a: Future[A]val b: Future[B]

val c: Future[(A, B)] = a join b

Future#join

Monday, April 30, 12

Works as advertised:

def get(): Future[Item]def continue(item: Item): Boolean

def collect(items: List[Item]): Future[List[Item]] = getItem() flatMap { item => if (continue(item)) collect(item :: items) else Future.value(item :: items) }

val items: Future[List[Item]] = collect(Nil)

Loops Recursion

Monday, April 30, 12

trait User { def follow(other: User) def DM(message: String}}def newUser(): Future[User]

val done: Future[Unit] = for { (a, b) <- newUser() join newUser() _ <- a.follow(b) join b.follow(a) _ <- a.DM(b, “hello, world!”)} yield ()

Putting it all together

Monday, April 30, 12

val done: Future[Unit] = for { (a, b) <- newUser() join newUser() _ <- a.follow(b) join b.follow(a) _ <- a.DM(b, “hello, world!”)} yield ()

Monday, April 30, 12

val f = newUser().join(newUser()).flatMap( new Function1[Tuple2[User, User]], Future[Unit]] { def apply(ab: Tuple2[User, User]): Future[Unit] = { ab._1.follow(ab._2).join(ab._2.follow(ab._1)).flatMap( new Function1[Unit, Unit] { def apply(u: Unit) = { ab._1.DM(ab._2, "hello, world!") } }) } })

Monday, April 30, 12

Services&

Filters

Monday, April 30, 12

It’s a function thatTakes a whileCan fail

What is an RPC?

Monday, April 30, 12

Req => Future[Rep]

Monday, April 30, 12

trait Service[Req, Rep] extends (Req => Future[Rep])

val service: Service[Int, String]val f: Future[String] = service(123)

A service is a function

Monday, April 30, 12

val multiplier = new Service[Int, Int] { def apply(i: Int) = Future.value(i*2) }

multiplier(123) onSuccess { x => println(“multiplied!”, x)}

Monday, April 30, 12

val timeoutFilter: Filter[…]val service: Service[Req, Rep]

val serviceWithTimeout = timeoutFilter andThen service// : Service[Req, Rep]

What’s a filter?

Monday, April 30, 12

val authFilter: Filter[…]val service: Service[AuthedReq, Rep]

val serviceThatAuthenticates = authFilter andThen service// : Service[Req, Rep]

What’s a filter?

Monday, April 30, 12

trait Filter[ReqIn, RepOut, ReqOut, RepIn] (ReqIn, Service[ReqOut, RepIn]) => Future[RepOut])

val authFilter = new Filter[Req, Rep, AuthedReq, Rep] { def apply(req: Req, svc: Service[AuthedReq, Rep]) = if (authenticate(req)) svc(AuthedReq(req)) else Future.exception(new AuthException) }

Filters are service transformers

Monday, April 30, 12

class TimeoutFilter[Req, Rep]( timeout: Duration) extends Filter[Req, Rep, Req, Rep]{ def apply( request: Req, service: Service[Req, Rep] ): Future[Rep] = service(request).within(timeout)}

A timeout filter

Monday, April 30, 12

class RequireAuthentication(passbird: Passbird) extends Filter[HttpReq, HttpRep, AuthHttpReq, HttpRep]{ def apply( req: HttpReq, service: Service[AuthHttpReq, HttpRep] ) = { passbird.auth(req) flatMap { case AuthResult(AuthResultCode.OK, Some(passport), _) => service(AuthHttpReq(req, passport)) case ar: AuthResult => Trace.record("Passbird authentication failed with " + ar) Future.exception( new RequestUnauthenticated(ar.resultCode)) } }}

Authentication

Monday, April 30, 12

class BackupRequestFilter[Req, Rep](after: Duration) extends SimpleFilter[Req, Rep] { def apply(req: Req, service: Service[Req, Rep]) = {   val p = new Promise[Rep]   p.linkTo(service(req) respond(p.updateIfEmpty(_)))   val task = timer.schedule(after.fromNow) {     if (!p.isDefined)       p.linkTo(service(req) respond(p.updateIfEmpty(_)))   }   p ensure {     p.cancel()     task.cancel()   } }}

Backup requests

Monday, April 30, 12

val timeout: Filter[…]val auth: Filter[…]val serve: Service[…]

timeout andThen auth andThen serve

They’re stackable, typesafe

Monday, April 30, 12

Finagleis our RPC system

Monday, April 30, 12

Provides Service instances via clients.Exposes Service instances via servers.Adds behavior, is largely configurable

Retrying, connection pooling, load balancing, rate limiting, monitoring, stats keeping, …

Codecs implement wire protocols.Manages resources for you.

Finagle

Monday, April 30, 12

Most of finagle is protocol agnostic.Codecs for thrift, http, memcache, kestrel, redis, streaming HTTP, generic multiplexer.Supports many RPC styles: request/response, streaming, multiplexing.Writing new codecs is easy.Uses Netty for the event loop.Scala and Java parity.

Finagle

Monday, April 30, 12

ClientBuilder produces a Service instance

ServerBuilderconsumes a Service instance

Builders

Monday, April 30, 12

val client = ClientBuilder() .name(“httploadtest”) .codec(Http) .hosts("host1:80,host2:80,...") .hostConnectionLimit(10) .build()

ClientBuilder

Monday, April 30, 12

val service: Service[HttpReq, HttpResp] val server = ServerBuilder() .name(“httpd”)

.codec(Http) .bindTo(new InetSocketAddress(8080))

.build(service) server.close() // when done

ServerBuilder

Monday, April 30, 12

val client: Service[HttpReq, HttpRep] = ClientBuilder()…build()

ServerBuilder()…build(client)

A proxy

Monday, April 30, 12

service Hello { string hi(1: string a);}

=>

public interface ServiceIface { public Future<String> hi(a: String);}

Thrift codegen

Monday, April 30, 12

≈ Service[Array[Byte], Array[Byte]]

Thrift transport

Monday, April 30, 12

A DISTRIBUTED SEARCH ENGINE IN

200 SLOChttps://github.com/twitter/scala_school/tree/master/searchbird

Monday, April 30, 12

Node

Node Node Node

Monday, April 30, 12

service SearchbirdService { // returned strings are json-encoded // twitter statuses list<string> query(1: list<string> tokens) i64 size() throws(1: SearchbirdException ex)}

searchbird.thrift

Monday, April 30, 12

// Generated codetrait Index { // Query the index for the set of Statuses that // have all of the given tokens. def query(tokens: Seq[String]): Future[Seq[String]]

// Number of Statuses in the index. def size: Future[Long]}

index.scala

Monday, April 30, 12

class InmemoryIndex( tweets: AsyncQueue[Tweet]) extends Index { tweets foreach { tweet => addToIndex(tweet) } .. // a simple posting list}

memoryindex.scala

Monday, April 30, 12

class FanoutIndex(underlying: Seq[Index]) extends Index { def query(tokens: Seq[String]) = Future.collect( underlying map(_.query(tokens)) ) map(_.flatten)

def size = Future.collect( underlying map(_.size) ) map(_.sum)}

fanoutindex.scala

Monday, April 30, 12

val fs = underlying map { index => index.query(tokens)} // : Seq[Future[Seq[String]]]

val coll = Future.collect(fs) // : Future[Seq[Seq[String]]]

coll map { sstr => sstr.flatten } // : Future[Seq[String]]

query, decomposed

Monday, April 30, 12

val mod, count: Int = ..localIndex = { val queue: AsyncQueue[Tweet] = makeReader() val filteredQueue = queue filter { status => (status.id % count) == index }

new InmemoryIndex(filteredQueue)}

server.scala

Monday, April 30, 12

val childAddrs: Seq[SocketAddress] = ..val childIndices: Seq[Index] = childAddrs map { addr => // This creates a transport service val service = ClientBuilder() .hosts(addr) .codec(ThriftClientFramedCodec()) .hostConnectionLimit(1) .build()

new Searchbird.ServiceToClient(service)}

Monday, April 30, 12

val index = new FanoutIndex( localIndex.toSeq ++ childIndices)

val service = new Searchbird.Service(index)

ServerBuilder() .bindTo(new InetSocketAddr(port)) .codec(Thrift) .name("thriftserver") .build(service)

Monday, April 30, 12

A Word on Observability

Monday, April 30, 12

Stats.incr("events", 1)

ostrich

Monday, April 30, 12

Stats.addGauge("npending") { numPending.get() // AtomicInteger}

ostrich

Monday, April 30, 12

ostrich

Stats.addMetric("histo", 100)

Monday, April 30, 12

ostrich% curl localhost:2000/stats.txtcounters: events: 4 ...gauges: npending: 1023metrics: histo: (average=100, count=1, maximum=100, minimum=100, p25=100, p50=100, p75=100, p90=100, p95=100, p99=100, p999=100, p9999=100, sum=100)

Monday, April 30, 12

Monday, April 30, 12

Trace.setId(fromWire)

Trace.recordRpcName(rpcName)

...

Trace.record(“user error “+exc)

Tracing

Monday, April 30, 12

Monday, April 30, 12

% java -Xbootclasspath/a:. \ -agentlib:heapster com.twitter...

% curl 'localhost:9990/pprof/heap?pause=10&sample_period=1024' > /tmp/prof

% pprof /tmp/prof

Heapster

Monday, April 30, 12

Monday, April 30, 12

Monday, April 30, 12

6335MB w=5348 (369MB/s 1213kB/w)50.00% 10 890.00% 18 1595.00% 25 2199.00% 453 38299.90% 813 68799.99% 4966 4192

gcprof

Monday, April 30, 12

In practice

Monday, April 30, 12

The code I showed you is real

Most of our services look much like this

Monday, April 30, 12

Computation is “descriptive” (future composition); finagle controls execution strategy.This has many interesting benefits

Generic tracingGeneric end-to-end cancellationPer request statisticsGeneric request prioritizationCan avoid priority inversion

Abstraction buys you...

Monday, April 30, 12

These have turned out to be extremely productive and versatile primitives.Most of our services are big “future transformers”Greatly enhanced modularity and genericityEasier to reason about semanticsEasier to test: behavior tends to be “pure”

Abstraction buys you...

Monday, April 30, 12

More indirection is more object allocation is more GC pressure.

Most of our “tuning” time is spent dealing with allocation issues.But: we have good tools, and it’s usually obvious.

A few places (Promise, codecs) require special attention to efficiency, but generally the JVM just works for you.

Abstraction costs you...

Monday, April 30, 12

Stack traces don’t make much sense anymoreInstead, future compositions form “dispatch graphs”

The execution model does matterEspecially when mixing with legacy codeThis can lead to a lot of confusion

Harder to reason about performanceTraditional cost models don’t apply

Abstraction costs you...

Monday, April 30, 12

This is not specific to ScalaWe have a lot of Java processes services that use these abstractions effectively

Java

Monday, April 30, 12

Monday, April 30, 12

Monday, April 30, 12

Thanks. Questions?

https://twitter.com/mariusmarius@twitter.com

https://github.com/twitter/utilhttps://github.com/twitter/finaglehttp://twitter.github.com/finagle

Monday, April 30, 12