mirror of
https://github.com/twitter/the-algorithm.git
synced 2025-06-10 14:48:16 -05:00
Open-sourcing Tweetypie
Tweetypie is the core Tweet service that handles the reading and writing of Tweet data.
This commit is contained in:
19
tweetypie/server/src/main/scala/com/twitter/tweetypie/BUILD
Normal file
19
tweetypie/server/src/main/scala/com/twitter/tweetypie/BUILD
Normal file
@ -0,0 +1,19 @@
|
||||
scala_library(
|
||||
sources = ["*.scala"],
|
||||
compiler_option_sets = ["fatal_warnings"],
|
||||
platform = "java8",
|
||||
strict_deps = True,
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"mediaservices/commons/src/main/thrift:thrift-scala",
|
||||
"tweetypie/servo/util",
|
||||
"snowflake:id",
|
||||
"src/thrift/com/twitter/gizmoduck:thrift-scala",
|
||||
"src/thrift/com/twitter/gizmoduck:user-thrift-scala",
|
||||
"tweetypie/common/src/thrift/com/twitter/tweetypie:media-entity-scala",
|
||||
"tweetypie/common/src/thrift/com/twitter/tweetypie:tweet-scala",
|
||||
"tweetypie/server/src/main/thrift:compiled-scala",
|
||||
"tweetypie/common/src/scala/com/twitter/tweetypie/util",
|
||||
"util/util-slf4j-api",
|
||||
],
|
||||
)
|
@ -0,0 +1,48 @@
|
||||
scala_library(
|
||||
sources = ["*.scala"],
|
||||
compiler_option_sets = ["fatal_warnings"],
|
||||
platform = "java8",
|
||||
strict_deps = True,
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"3rdparty/jvm/com/twitter/bijection:thrift",
|
||||
"3rdparty/jvm/org/apache/thrift:libthrift",
|
||||
"configbus/client/src/main/scala/com/twitter/configbus/client",
|
||||
"creatives-container/thrift/src/main/thrift:creatives-container-service-scala",
|
||||
"finagle/finagle-core/src/main",
|
||||
"finagle/finagle-thriftmux/src/main/scala",
|
||||
# "finatra-internal/kafka/src/main/scala/com/twitter/finatra/kafka/producers",
|
||||
"finatra-internal/messaging/kafka/src/main/scala",
|
||||
"finatra-internal/thrift/src/main/thrift:thrift-scala",
|
||||
"flock-client/src/main/scala",
|
||||
"flock-client/src/main/thrift:thrift-scala",
|
||||
# "kafka/finagle-kafka/finatra-kafka/src/main/scala",
|
||||
"limiter/thrift-only/src/main/thrift:thrift-scala",
|
||||
"mediaservices/mediainfo-server/thrift/src/main/thrift:thrift-scala",
|
||||
"tweetypie/servo/util",
|
||||
"src/thrift/com/twitter/dataproducts:service-scala",
|
||||
"src/thrift/com/twitter/escherbird:annotation-service-scala",
|
||||
"src/thrift/com/twitter/escherbird:tweet-annotation-scala",
|
||||
"src/thrift/com/twitter/escherbird/metadata:metadata-service-scala",
|
||||
"src/thrift/com/twitter/expandodo:only-scala",
|
||||
"src/thrift/com/twitter/gizmoduck:thrift-scala",
|
||||
"src/thrift/com/twitter/gizmoduck:user-thrift-scala",
|
||||
"src/thrift/com/twitter/service/scarecrow/gen:scarecrow-scala",
|
||||
"src/thrift/com/twitter/service/scarecrow/gen:tiered-actions-scala",
|
||||
"src/thrift/com/twitter/service/talon/gen:thrift-scala",
|
||||
"src/thrift/com/twitter/servo:servo-exception-scala",
|
||||
"src/thrift/com/twitter/socialgraph:thrift-scala",
|
||||
"src/thrift/com/twitter/timelineservice:thrift-scala",
|
||||
"src/thrift/com/twitter/timelineservice/server/internal:thrift-scala",
|
||||
"tweetypie/common/src/thrift/com/twitter/tweetypie:service-scala",
|
||||
"tweetypie/common/src/thrift/com/twitter/tweetypie:tweet-scala",
|
||||
"stitch/stitch-core",
|
||||
"storage/clients/manhattan/client/src/main/scala",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie/core",
|
||||
"tweetypie/common/src/scala/com/twitter/tweetypie/storage",
|
||||
"tweetypie/common/src/scala/com/twitter/tweetypie/util",
|
||||
"user-image-service/thrift/src/main/thrift:thrift-scala",
|
||||
"util/util-stats/src/main/scala",
|
||||
],
|
||||
)
|
@ -0,0 +1,172 @@
|
||||
package com.twitter.tweetypie
|
||||
package backends
|
||||
|
||||
import com.twitter.finagle.context.Deadline
|
||||
import com.twitter.finagle.service.RetryBudget
|
||||
import com.twitter.finagle.service.RetryPolicy
|
||||
import com.twitter.servo.util.FutureArrow
|
||||
import com.twitter.servo.util.RetryHandler
|
||||
import com.twitter.tweetypie.core.OverCapacity
|
||||
import com.twitter.util.Timer
|
||||
import com.twitter.util.TimeoutException
|
||||
|
||||
object Backend {
|
||||
val log: Logger = Logger(getClass)
|
||||
|
||||
/**
|
||||
* Common stuff that is needed as part of the configuration of all
|
||||
* of the backends.
|
||||
*/
|
||||
case class Context(val timer: Timer, val stats: StatsReceiver)
|
||||
|
||||
/**
|
||||
* All backend operations are encapsulated in the FutureArrow type. The Builder type
|
||||
* represents functions that can decorate the FutureArrow, typically by calling the various
|
||||
* combinator methods on FutureArrow.
|
||||
*/
|
||||
type Builder[A, B] = FutureArrow[A, B] => FutureArrow[A, B]
|
||||
|
||||
/**
|
||||
* A Policy defines some behavior to apply to a FutureArrow that wraps an endpoint.
|
||||
*/
|
||||
trait Policy {
|
||||
|
||||
/**
|
||||
* Using an endpoint name and Context, returns a Builder that does the actual
|
||||
* application of the policy to the FutureArrow.
|
||||
*/
|
||||
def apply[A, B](name: String, ctx: Context): Builder[A, B]
|
||||
|
||||
/**
|
||||
* Sequentially combines policies, first applying this policy and then applying
|
||||
* the next policy. Order matters! For example, to retry on timeouts, the FailureRetryPolicy
|
||||
* needs to be applied after the TimeoutPolicy:
|
||||
*
|
||||
* TimeoutPolicy(100.milliseconds) >>> FailureRetryPolicy(retryPolicy)
|
||||
*/
|
||||
def andThen(next: Policy): Policy = {
|
||||
val first = this
|
||||
new Policy {
|
||||
def apply[A, B](name: String, ctx: Context): Builder[A, B] =
|
||||
first(name, ctx).andThen(next(name, ctx))
|
||||
|
||||
override def toString = s"$first >>> $next"
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* An alias for `andThen`.
|
||||
*/
|
||||
def >>>(next: Policy): Policy = andThen(next)
|
||||
}
|
||||
|
||||
/**
|
||||
* Applies a timeout to the underlying FutureArrow.
|
||||
*/
|
||||
case class TimeoutPolicy(timeout: Duration) extends Policy {
|
||||
def apply[A, B](name: String, ctx: Context): Builder[A, B] = {
|
||||
val stats = ctx.stats.scope(name)
|
||||
val ex = new TimeoutException(name + ": " + timeout)
|
||||
(_: FutureArrow[A, B]).raiseWithin(ctx.timer, timeout, ex)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Attaches a RetryHandler with the given RetryPolicy to retry failures.
|
||||
*/
|
||||
case class FailureRetryPolicy(
|
||||
retryPolicy: RetryPolicy[Try[Nothing]],
|
||||
retryBudget: RetryBudget = RetryBudget())
|
||||
extends Policy {
|
||||
def apply[A, B](name: String, ctx: Context): Builder[A, B] = {
|
||||
val stats = ctx.stats.scope(name)
|
||||
(_: FutureArrow[A, B])
|
||||
.retry(RetryHandler.failuresOnly(retryPolicy, ctx.timer, stats, retryBudget))
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This policy applies standardized endpoint metrics. This should be used with every endpoint.
|
||||
*/
|
||||
case object TrackPolicy extends Policy {
|
||||
def apply[A, B](name: String, ctx: Context): Builder[A, B] = {
|
||||
val stats = ctx.stats.scope(name)
|
||||
(_: FutureArrow[A, B])
|
||||
.onFailure(countOverCapacityExceptions(stats))
|
||||
.trackOutcome(ctx.stats, (_: A) => name)
|
||||
.trackLatency(ctx.stats, (_: A) => name)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* The default "policy" for timeouts, retries, exception counting, latency tracking, etc. to
|
||||
* apply to each backend operation. This returns a Builder type (an endofunction on FutureArrow),
|
||||
* which can be composed with other Builders via simple function composition.
|
||||
*/
|
||||
def defaultPolicy[A, B](
|
||||
name: String,
|
||||
requestTimeout: Duration,
|
||||
retryPolicy: RetryPolicy[Try[B]],
|
||||
ctx: Context,
|
||||
retryBudget: RetryBudget = RetryBudget(),
|
||||
totalTimeout: Duration = Duration.Top,
|
||||
exceptionCategorizer: Throwable => Option[String] = _ => None
|
||||
): Builder[A, B] = {
|
||||
val scopedStats = ctx.stats.scope(name)
|
||||
val requestTimeoutException = new TimeoutException(
|
||||
s"$name: hit request timeout of $requestTimeout"
|
||||
)
|
||||
val totalTimeoutException = new TimeoutException(s"$name: hit total timeout of $totalTimeout")
|
||||
base =>
|
||||
base
|
||||
.raiseWithin(
|
||||
ctx.timer,
|
||||
// We defer to a per-request deadline. When the deadline is missing or wasn't toggled,
|
||||
// 'requestTimeout' is used instead. This mimics the behavior happening within a standard
|
||||
// Finagle client stack and its 'TimeoutFilter'.
|
||||
Deadline.currentToggled.fold(requestTimeout)(_.remaining),
|
||||
requestTimeoutException
|
||||
)
|
||||
.retry(RetryHandler(retryPolicy, ctx.timer, scopedStats, retryBudget))
|
||||
.raiseWithin(ctx.timer, totalTimeout, totalTimeoutException)
|
||||
.onFailure(countOverCapacityExceptions(scopedStats))
|
||||
.trackOutcome(ctx.stats, (_: A) => name, exceptionCategorizer)
|
||||
.trackLatency(ctx.stats, (_: A) => name)
|
||||
}
|
||||
|
||||
/**
|
||||
* An onFailure FutureArrow callback that counts OverCapacity exceptions to a special counter.
|
||||
* These will also be counted as failures and by exception class name, but having a special
|
||||
* counter for this is easier to use in success rate computations where you want to factor out
|
||||
* backpressure responses.
|
||||
*/
|
||||
def countOverCapacityExceptions[A](scopedStats: StatsReceiver): (A, Throwable) => Unit = {
|
||||
val overCapacityCounter = scopedStats.counter("over_capacity")
|
||||
|
||||
{
|
||||
case (_, ex: OverCapacity) => overCapacityCounter.incr()
|
||||
case _ => ()
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Provides a simple mechanism for applying a Policy to an endpoint FutureArrow from
|
||||
* an underlying service interface.
|
||||
*/
|
||||
class PolicyAdvocate[S](backendName: String, ctx: Backend.Context, svc: S) {
|
||||
|
||||
/**
|
||||
* Tacks on the TrackPolicy to the given base policy, and then applies the policy to
|
||||
* a FutureArrow. This is more of a convenience method that every Backend can use to
|
||||
* build the fully configured FutureArrow.
|
||||
*/
|
||||
def apply[A, B](
|
||||
endpointName: String,
|
||||
policy: Policy,
|
||||
endpoint: S => FutureArrow[A, B]
|
||||
): FutureArrow[A, B] = {
|
||||
log.info(s"appling policy to $backendName.$endpointName: $policy")
|
||||
policy.andThen(TrackPolicy)(endpointName, ctx)(endpoint(svc))
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,50 @@
|
||||
package com.twitter.tweetypie.backends
|
||||
|
||||
import com.twitter.configbus.client.ConfigbusClientException
|
||||
import com.twitter.configbus.client.file.PollingConfigSourceBuilder
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.logging.Logger
|
||||
import com.twitter.util.Activity
|
||||
import com.twitter.util.Activity._
|
||||
import com.twitter.conversions.DurationOps._
|
||||
import com.twitter.io.Buf
|
||||
|
||||
trait ConfigBus {
|
||||
def file(path: String): Activity[String]
|
||||
}
|
||||
|
||||
object ConfigBus {
|
||||
private[this] val basePath = "appservices/tweetypie"
|
||||
private[this] val log = Logger(getClass)
|
||||
|
||||
def apply(stats: StatsReceiver, instanceId: Int, instanceCount: Int): ConfigBus = {
|
||||
|
||||
val client = PollingConfigSourceBuilder()
|
||||
.statsReceiver(stats)
|
||||
.pollPeriod(30.seconds)
|
||||
.instanceId(instanceId)
|
||||
.numberOfInstances(instanceCount)
|
||||
.build()
|
||||
|
||||
val validBuffer = stats.counter("valid_buffer")
|
||||
|
||||
def subscribe(path: String) =
|
||||
client.subscribe(s"$basePath/$path").map(_.configs).map {
|
||||
case Buf.Utf8(string) =>
|
||||
validBuffer.incr()
|
||||
string
|
||||
}
|
||||
|
||||
new ConfigBus {
|
||||
def file(path: String): Activity[String] = {
|
||||
val changes = subscribe(path).run.changes.dedupWith {
|
||||
case (Failed(e1: ConfigbusClientException), Failed(e2: ConfigbusClientException)) =>
|
||||
e1.getMessage == e2.getMessage
|
||||
case other =>
|
||||
false
|
||||
}
|
||||
Activity(changes)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,71 @@
|
||||
package com.twitter.tweetypie.backends
|
||||
|
||||
import com.twitter.container.{thriftscala => ccs}
|
||||
import com.twitter.finagle.Backoff
|
||||
import com.twitter.finagle.service.RetryPolicy
|
||||
import com.twitter.finatra.thrift.thriftscala.ServerError
|
||||
import com.twitter.finatra.thrift.thriftscala.ServerErrorCause
|
||||
import com.twitter.servo.util.FutureArrow
|
||||
import com.twitter.tweetypie.Duration
|
||||
import com.twitter.tweetypie.Future
|
||||
import com.twitter.tweetypie.Try
|
||||
import com.twitter.tweetypie.util.RetryPolicyBuilder
|
||||
import com.twitter.tweetypie.{thriftscala => tp}
|
||||
import com.twitter.util.Throw
|
||||
|
||||
object CreativesContainerService {
|
||||
import Backend._
|
||||
|
||||
type MaterializeAsTweet = FutureArrow[ccs.MaterializeAsTweetRequests, Seq[tp.GetTweetResult]]
|
||||
type MaterializeAsTweetFields =
|
||||
FutureArrow[ccs.MaterializeAsTweetFieldsRequests, Seq[tp.GetTweetFieldsResult]]
|
||||
|
||||
def fromClient(
|
||||
client: ccs.CreativesContainerService.MethodPerEndpoint
|
||||
): CreativesContainerService =
|
||||
new CreativesContainerService {
|
||||
val materializeAsTweet: MaterializeAsTweet = FutureArrow(client.materializeAsTweets)
|
||||
val materializeAsTweetFields: MaterializeAsTweetFields = FutureArrow(
|
||||
client.materializeAsTweetFields)
|
||||
|
||||
def ping(): Future[Unit] = client.materializeAsTweets(ccs.MaterializeAsTweetRequests()).unit
|
||||
}
|
||||
|
||||
case class Config(
|
||||
requestTimeout: Duration,
|
||||
timeoutBackoffs: Stream[Duration],
|
||||
serverErrorBackoffs: Stream[Duration]) {
|
||||
def apply(svc: CreativesContainerService, ctx: Backend.Context): CreativesContainerService =
|
||||
new CreativesContainerService {
|
||||
override val materializeAsTweet: MaterializeAsTweet =
|
||||
policy("materializeAsTweets", ctx)(svc.materializeAsTweet)
|
||||
|
||||
override val materializeAsTweetFields: MaterializeAsTweetFields =
|
||||
policy("materializeAsTweetFields", ctx)(svc.materializeAsTweetFields)
|
||||
|
||||
override def ping(): Future[Unit] = svc.ping()
|
||||
}
|
||||
|
||||
private[this] def policy[A, B](name: String, ctx: Context): Builder[A, B] =
|
||||
defaultPolicy(name, requestTimeout, retryPolicy, ctx)
|
||||
|
||||
private[this] def retryPolicy[B]: RetryPolicy[Try[B]] =
|
||||
RetryPolicy.combine[Try[B]](
|
||||
RetryPolicyBuilder.timeouts[B](timeoutBackoffs),
|
||||
RetryPolicy.backoff(Backoff.fromStream(serverErrorBackoffs)) {
|
||||
case Throw(ex: ServerError) if ex.errorCause != ServerErrorCause.NotImplemented => true
|
||||
}
|
||||
)
|
||||
|
||||
implicit val warmup: Warmup[CreativesContainerService] =
|
||||
Warmup[CreativesContainerService]("creativesContainerService")(_.ping())
|
||||
}
|
||||
}
|
||||
|
||||
trait CreativesContainerService {
|
||||
import CreativesContainerService._
|
||||
|
||||
val materializeAsTweet: MaterializeAsTweet
|
||||
val materializeAsTweetFields: MaterializeAsTweetFields
|
||||
def ping(): Future[Unit]
|
||||
}
|
@ -0,0 +1,43 @@
|
||||
package com.twitter.tweetypie
|
||||
package backends
|
||||
|
||||
import com.twitter.escherbird.thriftscala.TweetEntityAnnotation
|
||||
import com.twitter.escherbird.{thriftscala => escherbird}
|
||||
import com.twitter.finagle.service.RetryPolicy
|
||||
import com.twitter.servo.util.FutureArrow
|
||||
import com.twitter.tweetypie.util.RetryPolicyBuilder
|
||||
|
||||
object Escherbird {
|
||||
import Backend._
|
||||
|
||||
type Annotate = FutureArrow[Tweet, Seq[TweetEntityAnnotation]]
|
||||
|
||||
def fromClient(client: escherbird.TweetEntityAnnotationService.MethodPerEndpoint): Escherbird =
|
||||
new Escherbird {
|
||||
val annotate = FutureArrow(client.annotate)
|
||||
}
|
||||
|
||||
case class Config(requestTimeout: Duration, timeoutBackoffs: Stream[Duration]) {
|
||||
|
||||
def apply(svc: Escherbird, ctx: Backend.Context): Escherbird =
|
||||
new Escherbird {
|
||||
val annotate: FutureArrow[Tweet, Seq[TweetEntityAnnotation]] =
|
||||
policy("annotate", requestTimeout, ctx)(svc.annotate)
|
||||
}
|
||||
|
||||
private[this] def policy[A, B](
|
||||
name: String,
|
||||
requestTimeout: Duration,
|
||||
ctx: Context
|
||||
): Builder[A, B] =
|
||||
defaultPolicy(name, requestTimeout, retryPolicy, ctx)
|
||||
|
||||
private[this] def retryPolicy[B]: RetryPolicy[Try[B]] =
|
||||
RetryPolicyBuilder.timeouts[Any](timeoutBackoffs)
|
||||
}
|
||||
}
|
||||
|
||||
trait Escherbird {
|
||||
import Escherbird._
|
||||
val annotate: Annotate
|
||||
}
|
@ -0,0 +1,83 @@
|
||||
package com.twitter.tweetypie
|
||||
package backends
|
||||
|
||||
import com.twitter.expandodo.thriftscala.AttachmentEligibilityRequest
|
||||
import com.twitter.expandodo.thriftscala.AttachmentEligibilityResponses
|
||||
import com.twitter.expandodo.thriftscala.Card2Request
|
||||
import com.twitter.expandodo.thriftscala.Card2RequestOptions
|
||||
import com.twitter.expandodo.thriftscala.Card2Responses
|
||||
import com.twitter.expandodo.thriftscala.CardsResponse
|
||||
import com.twitter.expandodo.thriftscala.GetCardUsersRequests
|
||||
import com.twitter.expandodo.thriftscala.GetCardUsersResponses
|
||||
import com.twitter.expandodo.{thriftscala => expandodo}
|
||||
import com.twitter.finagle.Backoff
|
||||
import com.twitter.finagle.service.RetryPolicy
|
||||
import com.twitter.servo.util.FutureArrow
|
||||
import com.twitter.tweetypie.util.RetryPolicyBuilder
|
||||
|
||||
object Expandodo {
|
||||
import Backend._
|
||||
|
||||
type GetCards = FutureArrow[Set[String], collection.Map[String, expandodo.CardsResponse]]
|
||||
type GetCards2 = FutureArrow[
|
||||
(Seq[expandodo.Card2Request], expandodo.Card2RequestOptions),
|
||||
expandodo.Card2Responses
|
||||
]
|
||||
type GetCardUsers = FutureArrow[expandodo.GetCardUsersRequests, expandodo.GetCardUsersResponses]
|
||||
type CheckAttachmentEligibility =
|
||||
FutureArrow[Seq[
|
||||
expandodo.AttachmentEligibilityRequest
|
||||
], expandodo.AttachmentEligibilityResponses]
|
||||
|
||||
def fromClient(client: expandodo.CardsService.MethodPerEndpoint): Expandodo =
|
||||
new Expandodo {
|
||||
val getCards = FutureArrow(client.getCards _)
|
||||
val getCards2 = FutureArrow((client.getCards2 _).tupled)
|
||||
val getCardUsers = FutureArrow(client.getCardUsers _)
|
||||
val checkAttachmentEligibility = FutureArrow(client.checkAttachmentEligibility _)
|
||||
}
|
||||
|
||||
case class Config(
|
||||
requestTimeout: Duration,
|
||||
timeoutBackoffs: Stream[Duration],
|
||||
serverErrorBackoffs: Stream[Duration]) {
|
||||
def apply(svc: Expandodo, ctx: Backend.Context): Expandodo =
|
||||
new Expandodo {
|
||||
val getCards: FutureArrow[Set[String], collection.Map[String, CardsResponse]] =
|
||||
policy("getCards", ctx)(svc.getCards)
|
||||
val getCards2: FutureArrow[(Seq[Card2Request], Card2RequestOptions), Card2Responses] =
|
||||
policy("getCards2", ctx)(svc.getCards2)
|
||||
val getCardUsers: FutureArrow[GetCardUsersRequests, GetCardUsersResponses] =
|
||||
policy("getCardUsers", ctx)(svc.getCardUsers)
|
||||
val checkAttachmentEligibility: FutureArrow[Seq[
|
||||
AttachmentEligibilityRequest
|
||||
], AttachmentEligibilityResponses] =
|
||||
policy("checkAttachmentEligibility", ctx)(svc.checkAttachmentEligibility)
|
||||
}
|
||||
|
||||
private[this] def policy[A, B](name: String, ctx: Context): Builder[A, B] =
|
||||
defaultPolicy(name, requestTimeout, retryPolicy, ctx)
|
||||
|
||||
private[this] def retryPolicy[B]: RetryPolicy[Try[B]] =
|
||||
RetryPolicy.combine[Try[B]](
|
||||
RetryPolicyBuilder.timeouts[B](timeoutBackoffs),
|
||||
RetryPolicy.backoff(Backoff.fromStream(serverErrorBackoffs)) {
|
||||
case Throw(ex: expandodo.InternalServerError) => true
|
||||
}
|
||||
)
|
||||
}
|
||||
|
||||
implicit val warmup: Warmup[Expandodo] =
|
||||
Warmup[Expandodo]("expandodo")(
|
||||
_.getCards2((Seq.empty, expandodo.Card2RequestOptions("iPhone-13")))
|
||||
)
|
||||
}
|
||||
|
||||
trait Expandodo {
|
||||
import Expandodo._
|
||||
|
||||
val getCards: GetCards
|
||||
val getCards2: GetCards2
|
||||
val getCardUsers: GetCardUsers
|
||||
val checkAttachmentEligibility: CheckAttachmentEligibility
|
||||
}
|
@ -0,0 +1,84 @@
|
||||
package com.twitter.tweetypie
|
||||
package backends
|
||||
|
||||
import com.twitter.servo.util.FutureArrow
|
||||
import com.twitter.stitch.Stitch
|
||||
import com.twitter.storage.client.manhattan.bijections.Bijections._
|
||||
import com.twitter.storage.client.manhattan.kv._
|
||||
import com.twitter.storage.client.manhattan.kv.impl._
|
||||
import com.twitter.util.Time
|
||||
|
||||
/**
|
||||
* Read and write the timestamp of the last delete_location_data request
|
||||
* for a user. This is used as a safeguard to prevent leaking geo data
|
||||
* with tweets that have not yet been scrubbed or were missed during the
|
||||
* geo scrubbing process.
|
||||
*/
|
||||
object GeoScrubEventStore {
|
||||
type GetGeoScrubTimestamp = UserId => Stitch[Option[Time]]
|
||||
type SetGeoScrubTimestamp = FutureArrow[(UserId, Time), Unit]
|
||||
|
||||
private[this] val KeyDesc =
|
||||
KeyDescriptor(
|
||||
Component(LongInjection),
|
||||
Component(LongInjection, StringInjection)
|
||||
).withDataset("geo_scrub")
|
||||
|
||||
private[this] val ValDesc = ValueDescriptor(LongInjection)
|
||||
|
||||
// This modulus determines how user ids get assigned to PKeys, and
|
||||
// thus to shards within the MH cluster. The origin of the specific
|
||||
// value has been lost to time, but it's important that we don't
|
||||
// change it, or else the existing data will be inaccessible.
|
||||
private[this] val PKeyModulus: Long = 25000L
|
||||
|
||||
private[this] def toKey(userId: Long) =
|
||||
KeyDesc
|
||||
.withPkey(userId % PKeyModulus)
|
||||
.withLkey(userId, "_last_scrub")
|
||||
|
||||
def apply(client: ManhattanKVClient, config: Config, ctx: Backend.Context): GeoScrubEventStore = {
|
||||
new GeoScrubEventStore {
|
||||
val getGeoScrubTimestamp: UserId => Stitch[Option[Time]] = {
|
||||
val endpoint = config.read.endpoint(client)
|
||||
|
||||
(userId: UserId) => {
|
||||
endpoint
|
||||
.get(toKey(userId), ValDesc)
|
||||
.map(_.map(value => Time.fromMilliseconds(value.contents)))
|
||||
}
|
||||
}
|
||||
|
||||
val setGeoScrubTimestamp: SetGeoScrubTimestamp = {
|
||||
val endpoint = config.write.endpoint(client)
|
||||
|
||||
FutureArrow {
|
||||
case (userId, timestamp) =>
|
||||
val key = toKey(userId)
|
||||
|
||||
// Use the geo scrub timestamp as the MH entry timestamp. This
|
||||
// ensures that whatever timestamp is highest will win any
|
||||
// update races.
|
||||
val value = ValDesc.withValue(timestamp.inMilliseconds, timestamp)
|
||||
Stitch.run(endpoint.insert(key, value))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
case class EndpointConfig(requestTimeout: Duration, maxRetryCount: Int) {
|
||||
def endpoint(client: ManhattanKVClient): ManhattanKVEndpoint =
|
||||
ManhattanKVEndpointBuilder(client)
|
||||
.defaultMaxTimeout(requestTimeout)
|
||||
.maxRetryCount(maxRetryCount)
|
||||
.build()
|
||||
}
|
||||
|
||||
case class Config(read: EndpointConfig, write: EndpointConfig)
|
||||
}
|
||||
|
||||
trait GeoScrubEventStore {
|
||||
import GeoScrubEventStore._
|
||||
val getGeoScrubTimestamp: GetGeoScrubTimestamp
|
||||
val setGeoScrubTimestamp: SetGeoScrubTimestamp
|
||||
}
|
@ -0,0 +1,93 @@
|
||||
package com.twitter.tweetypie
|
||||
package backends
|
||||
|
||||
import com.twitter.finagle.Backoff
|
||||
import com.twitter.finagle.service.RetryPolicy
|
||||
import com.twitter.gizmoduck.thriftscala.CountsUpdateField
|
||||
import com.twitter.gizmoduck.thriftscala.LookupContext
|
||||
import com.twitter.gizmoduck.thriftscala.ModifiedUser
|
||||
import com.twitter.gizmoduck.thriftscala.UserResult
|
||||
import com.twitter.gizmoduck.{thriftscala => gd}
|
||||
import com.twitter.servo.util.FutureArrow
|
||||
import com.twitter.tweetypie.core.OverCapacity
|
||||
import com.twitter.tweetypie.util.RetryPolicyBuilder
|
||||
|
||||
object Gizmoduck {
|
||||
import Backend._
|
||||
|
||||
type GetById = FutureArrow[(gd.LookupContext, Seq[UserId], Set[UserField]), Seq[gd.UserResult]]
|
||||
type GetByScreenName =
|
||||
FutureArrow[(gd.LookupContext, Seq[String], Set[UserField]), Seq[gd.UserResult]]
|
||||
type IncrCount = FutureArrow[(UserId, gd.CountsUpdateField, Int), Unit]
|
||||
type ModifyAndGet = FutureArrow[(gd.LookupContext, UserId, gd.ModifiedUser), gd.User]
|
||||
|
||||
def fromClient(client: gd.UserService.MethodPerEndpoint): Gizmoduck =
|
||||
new Gizmoduck {
|
||||
val getById = FutureArrow((client.get _).tupled)
|
||||
val getByScreenName = FutureArrow((client.getByScreenName _).tupled)
|
||||
val incrCount = FutureArrow((client.incrCount _).tupled)
|
||||
val modifyAndGet = FutureArrow((client.modifyAndGet _).tupled)
|
||||
def ping(): Future[Unit] = client.get(gd.LookupContext(), Seq.empty, Set.empty).unit
|
||||
}
|
||||
|
||||
case class Config(
|
||||
readTimeout: Duration,
|
||||
writeTimeout: Duration,
|
||||
modifyAndGetTimeout: Duration,
|
||||
modifyAndGetTimeoutBackoffs: Stream[Duration],
|
||||
defaultTimeoutBackoffs: Stream[Duration],
|
||||
gizmoduckExceptionBackoffs: Stream[Duration]) {
|
||||
|
||||
def apply(svc: Gizmoduck, ctx: Backend.Context): Gizmoduck =
|
||||
new Gizmoduck {
|
||||
val getById: FutureArrow[(LookupContext, Seq[UserId], Set[UserField]), Seq[UserResult]] =
|
||||
policy("getById", readTimeout, ctx)(svc.getById)
|
||||
val getByScreenName: FutureArrow[(LookupContext, Seq[String], Set[UserField]), Seq[
|
||||
UserResult
|
||||
]] = policy("getByScreenName", readTimeout, ctx)(svc.getByScreenName)
|
||||
val incrCount: FutureArrow[(UserId, CountsUpdateField, Int), Unit] =
|
||||
policy("incrCount", writeTimeout, ctx)(svc.incrCount)
|
||||
val modifyAndGet: FutureArrow[(LookupContext, UserId, ModifiedUser), User] = policy(
|
||||
"modifyAndGet",
|
||||
modifyAndGetTimeout,
|
||||
ctx,
|
||||
timeoutBackoffs = modifyAndGetTimeoutBackoffs
|
||||
)(svc.modifyAndGet)
|
||||
def ping(): Future[Unit] = svc.ping()
|
||||
}
|
||||
|
||||
private[this] def policy[A, B](
|
||||
name: String,
|
||||
requestTimeout: Duration,
|
||||
ctx: Context,
|
||||
timeoutBackoffs: Stream[Duration] = defaultTimeoutBackoffs
|
||||
): Builder[A, B] =
|
||||
translateExceptions andThen
|
||||
defaultPolicy(name, requestTimeout, retryPolicy(timeoutBackoffs), ctx)
|
||||
|
||||
private[this] def translateExceptions[A, B]: Builder[A, B] =
|
||||
_.translateExceptions {
|
||||
case gd.OverCapacity(msg) => OverCapacity(s"gizmoduck: $msg")
|
||||
}
|
||||
|
||||
private[this] def retryPolicy[B](timeoutBackoffs: Stream[Duration]): RetryPolicy[Try[B]] =
|
||||
RetryPolicy.combine[Try[B]](
|
||||
RetryPolicyBuilder.timeouts[B](timeoutBackoffs),
|
||||
RetryPolicy.backoff(Backoff.fromStream(gizmoduckExceptionBackoffs)) {
|
||||
case Throw(ex: gd.InternalServerError) => true
|
||||
}
|
||||
)
|
||||
}
|
||||
|
||||
implicit val warmup: Warmup[Gizmoduck] =
|
||||
Warmup[Gizmoduck]("gizmoduck")(_.ping())
|
||||
}
|
||||
|
||||
trait Gizmoduck {
|
||||
import Gizmoduck._
|
||||
val getById: GetById
|
||||
val getByScreenName: GetByScreenName
|
||||
val incrCount: IncrCount
|
||||
val modifyAndGet: ModifyAndGet
|
||||
def ping(): Future[Unit]
|
||||
}
|
@ -0,0 +1,42 @@
|
||||
package com.twitter.tweetypie
|
||||
package backends
|
||||
|
||||
import com.twitter.conversions.PercentOps._
|
||||
import com.twitter.conversions.DurationOps._
|
||||
import com.twitter.dataproducts.enrichments.thriftscala._
|
||||
import com.twitter.dataproducts.enrichments.thriftscala.Enricherator
|
||||
import com.twitter.finagle.thriftmux.MethodBuilder
|
||||
import com.twitter.servo.util.FutureArrow
|
||||
|
||||
object GnipEnricherator {
|
||||
|
||||
type HydrateProfileGeo = FutureArrow[ProfileGeoRequest, Seq[ProfileGeoResponse]]
|
||||
|
||||
private def methodPerEndpoint(methodBuilder: MethodBuilder) =
|
||||
Enricherator.MethodPerEndpoint(
|
||||
methodBuilder
|
||||
.servicePerEndpoint[Enricherator.ServicePerEndpoint]
|
||||
.withHydrateProfileGeo(
|
||||
methodBuilder
|
||||
.withTimeoutTotal(300.milliseconds)
|
||||
.withTimeoutPerRequest(100.milliseconds)
|
||||
.idempotent(maxExtraLoad = 1.percent)
|
||||
.servicePerEndpoint[Enricherator.ServicePerEndpoint](methodName = "hydrateProfileGeo")
|
||||
.hydrateProfileGeo
|
||||
)
|
||||
)
|
||||
|
||||
def fromMethod(methodBuilder: MethodBuilder): GnipEnricherator = {
|
||||
val mpe = methodPerEndpoint(methodBuilder)
|
||||
|
||||
new GnipEnricherator {
|
||||
override val hydrateProfileGeo: HydrateProfileGeo =
|
||||
FutureArrow(mpe.hydrateProfileGeo)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
trait GnipEnricherator {
|
||||
import GnipEnricherator._
|
||||
val hydrateProfileGeo: HydrateProfileGeo
|
||||
}
|
@ -0,0 +1,55 @@
|
||||
package com.twitter.tweetypie
|
||||
package backends
|
||||
|
||||
import com.twitter.finagle.service.RetryPolicy
|
||||
import com.twitter.limiter.thriftscala.FeatureRequest
|
||||
import com.twitter.limiter.thriftscala.Usage
|
||||
import com.twitter.limiter.{thriftscala => ls}
|
||||
import com.twitter.servo.util.FutureArrow
|
||||
import com.twitter.tweetypie.util.RetryPolicyBuilder
|
||||
|
||||
object LimiterBackend {
|
||||
import Backend._
|
||||
|
||||
type IncrementFeature = FutureArrow[(ls.FeatureRequest, Int), Unit]
|
||||
type GetFeatureUsage = FutureArrow[ls.FeatureRequest, ls.Usage]
|
||||
|
||||
def fromClient(client: ls.LimitService.MethodPerEndpoint): LimiterBackend =
|
||||
new LimiterBackend {
|
||||
val incrementFeature: IncrementFeature =
|
||||
FutureArrow {
|
||||
case (featureReq, amount) => client.incrementFeature(featureReq, amount).unit
|
||||
}
|
||||
|
||||
val getFeatureUsage: GetFeatureUsage =
|
||||
FutureArrow(featureReq => client.getLimitUsage(None, Some(featureReq)))
|
||||
}
|
||||
|
||||
case class Config(requestTimeout: Duration, timeoutBackoffs: Stream[Duration]) {
|
||||
|
||||
def apply(client: LimiterBackend, ctx: Backend.Context): LimiterBackend =
|
||||
new LimiterBackend {
|
||||
val incrementFeature: FutureArrow[(FeatureRequest, Int), Unit] =
|
||||
policy("incrementFeature", requestTimeout, ctx)(client.incrementFeature)
|
||||
val getFeatureUsage: FutureArrow[FeatureRequest, Usage] =
|
||||
policy("getFeatureUsage", requestTimeout, ctx)(client.getFeatureUsage)
|
||||
}
|
||||
|
||||
private[this] def policy[A, B](
|
||||
name: String,
|
||||
requestTimeout: Duration,
|
||||
ctx: Context
|
||||
): Builder[A, B] =
|
||||
defaultPolicy(name, requestTimeout, retryPolicy, ctx)
|
||||
|
||||
private[this] def retryPolicy[B]: RetryPolicy[Try[B]] =
|
||||
RetryPolicyBuilder.timeouts[Any](timeoutBackoffs)
|
||||
}
|
||||
}
|
||||
|
||||
trait LimiterBackend {
|
||||
import LimiterBackend._
|
||||
|
||||
val incrementFeature: IncrementFeature
|
||||
val getFeatureUsage: GetFeatureUsage
|
||||
}
|
@ -0,0 +1,193 @@
|
||||
package com.twitter.tweetypie
|
||||
package backends
|
||||
|
||||
import com.twitter.finagle.stats.NullStatsReceiver
|
||||
import com.twitter.limiter.thriftscala.FeatureRequest
|
||||
import com.twitter.tweetypie.backends.LimiterBackend.GetFeatureUsage
|
||||
import com.twitter.tweetypie.backends.LimiterBackend.IncrementFeature
|
||||
import com.twitter.tweetypie.backends.LimiterService.Feature
|
||||
|
||||
/**
|
||||
* Why does LimiterService exist?
|
||||
*
|
||||
* The underlying Limiter thrift service doesn't support batching. This trait and implementation
|
||||
* basically exist to allow a batch like interface to the Limiter. This keeps us from having to
|
||||
* spread batching throughout our code base.
|
||||
*
|
||||
* Why is LimiterService in the backends package?
|
||||
*
|
||||
* In some ways it is like a backend if the backend supports batching. There is a modest amount of
|
||||
* business logic LimiterService, but that logic exists here to allow easier consumption throughout
|
||||
* the tweetypie code base. We did look at moving LimiterService to another package, but all likely
|
||||
* candidates (service, serverutil) caused circular dependencies.
|
||||
*
|
||||
* When I need to add functionality, should I add it to LimiterBackend or LimiterService?
|
||||
*
|
||||
* LimiterBackend is used as a simple wrapper around the Limiter thrift client. The LimiterBackend
|
||||
* should be kept as dumb as possible. You will most likely want to add the functionality in
|
||||
* LimiterService.
|
||||
*/
|
||||
object LimiterService {
|
||||
type MinRemaining = (UserId, Option[UserId]) => Future[Int]
|
||||
type HasRemaining = (UserId, Option[UserId]) => Future[Boolean]
|
||||
type Increment = (UserId, Option[UserId], Int) => Future[Unit]
|
||||
type IncrementByOne = (UserId, Option[UserId]) => Future[Unit]
|
||||
|
||||
sealed abstract class Feature(val name: String, val hasPerApp: Boolean = false) {
|
||||
def forUser(userId: UserId): FeatureRequest = FeatureRequest(name, userId = Some(userId))
|
||||
def forApp(appId: AppId): Option[FeatureRequest] =
|
||||
if (hasPerApp) {
|
||||
Some(
|
||||
FeatureRequest(
|
||||
s"${name}_per_app",
|
||||
applicationId = Some(appId),
|
||||
identifier = Some(appId.toString)
|
||||
)
|
||||
)
|
||||
} else {
|
||||
None
|
||||
}
|
||||
}
|
||||
object Feature {
|
||||
case object Updates extends Feature("updates", hasPerApp = true)
|
||||
case object MediaTagCreate extends Feature("media_tag_create")
|
||||
case object TweetCreateFailure extends Feature("tweet_creation_failure")
|
||||
}
|
||||
|
||||
def fromBackend(
|
||||
incrementFeature: IncrementFeature,
|
||||
getFeatureUsage: GetFeatureUsage,
|
||||
getAppId: => Option[
|
||||
AppId
|
||||
], // the call-by-name here to invoke per request to get the current request's app id
|
||||
stats: StatsReceiver = NullStatsReceiver
|
||||
): LimiterService =
|
||||
new LimiterService {
|
||||
def increment(
|
||||
feature: Feature
|
||||
)(
|
||||
userId: UserId,
|
||||
contributorUserId: Option[UserId],
|
||||
amount: Int
|
||||
): Future[Unit] = {
|
||||
Future.when(amount > 0) {
|
||||
def increment(req: FeatureRequest): Future[Unit] = incrementFeature((req, amount))
|
||||
|
||||
val incrementUser: Option[Future[Unit]] =
|
||||
Some(increment(feature.forUser(userId)))
|
||||
|
||||
val incrementContributor: Option[Future[Unit]] =
|
||||
for {
|
||||
id <- contributorUserId
|
||||
if id != userId
|
||||
} yield increment(feature.forUser(id))
|
||||
|
||||
val incrementPerApp: Option[Future[Unit]] =
|
||||
for {
|
||||
appId <- getAppId
|
||||
req <- feature.forApp(appId)
|
||||
} yield increment(req)
|
||||
|
||||
Future.collect(Seq(incrementUser, incrementContributor, incrementPerApp).flatten)
|
||||
}
|
||||
}
|
||||
|
||||
def minRemaining(
|
||||
feature: Feature
|
||||
)(
|
||||
userId: UserId,
|
||||
contributorUserId: Option[UserId]
|
||||
): Future[Int] = {
|
||||
def getRemaining(req: FeatureRequest): Future[Int] = getFeatureUsage(req).map(_.remaining)
|
||||
|
||||
val getUserRemaining: Option[Future[Int]] =
|
||||
Some(getRemaining(feature.forUser(userId)))
|
||||
|
||||
val getContributorRemaining: Option[Future[Int]] =
|
||||
contributorUserId.map(id => getRemaining(feature.forUser(id)))
|
||||
|
||||
val getPerAppRemaining: Option[Future[Int]] =
|
||||
for {
|
||||
appId <- getAppId
|
||||
req <- feature.forApp(appId)
|
||||
} yield getRemaining(req)
|
||||
|
||||
Future
|
||||
.collect(Seq(getUserRemaining, getContributorRemaining, getPerAppRemaining).flatten)
|
||||
.map(_.min)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
trait LimiterService {
|
||||
|
||||
/**
|
||||
* Increment the feature count for both the user and the contributor. If either increment fails,
|
||||
* the resulting future will be the first exception encountered.
|
||||
*
|
||||
* @param feature The feature that is incremented
|
||||
* @param userId The current user tied to the current request
|
||||
* @param contributorUserId The contributor, if one exists, tied to the current request
|
||||
* @param amount The amount that each feature should be incremented.
|
||||
*/
|
||||
def increment(
|
||||
feature: Feature
|
||||
)(
|
||||
userId: UserId,
|
||||
contributorUserId: Option[UserId],
|
||||
amount: Int
|
||||
): Future[Unit]
|
||||
|
||||
/**
|
||||
* Increment the feature count, by one, for both the user and the contributor. If either
|
||||
* increment fails, the resulting future will be the first exception encountered.
|
||||
*
|
||||
* @param feature The feature that is incremented
|
||||
* @param userId The current user tied to the current request
|
||||
* @param contributorUserId The contributor, if one exists, tied to the current request
|
||||
*
|
||||
* @see [[increment]] if you want to increment a feature by a specified amount
|
||||
*/
|
||||
def incrementByOne(
|
||||
feature: Feature
|
||||
)(
|
||||
userId: UserId,
|
||||
contributorUserId: Option[UserId]
|
||||
): Future[Unit] =
|
||||
increment(feature)(userId, contributorUserId, 1)
|
||||
|
||||
/**
|
||||
* The minimum remaining limit between the user and contributor. If an exception occurs, then the
|
||||
* resulting Future will be the first exception encountered.
|
||||
*
|
||||
* @param feature The feature that is queried
|
||||
* @param userId The current user tied to the current request
|
||||
* @param contributorUserId The contributor, if one exists, tied to the current request
|
||||
*
|
||||
* @return a `Future[Int]` with the minimum limit left between the user and contributor
|
||||
*/
|
||||
def minRemaining(feature: Feature)(userId: UserId, contributorUserId: Option[UserId]): Future[Int]
|
||||
|
||||
/**
|
||||
* Can the user and contributor increment the given feature. If the result cannot be determined
|
||||
* because of an exception, then we assume they can increment. This will allow us to continue
|
||||
* servicing requests even if the limiter service isn't responding.
|
||||
*
|
||||
* @param feature The feature that is queried
|
||||
* @param userId The current user tied to the current request
|
||||
* @param contributorUserId The contributor, if one exists, tied to the current request
|
||||
* @return a `Future[Boolean]` with true if both the user and contributor have remaining limit
|
||||
* cap.
|
||||
*
|
||||
* @see [[minRemaining]] if you would like to handle any exceptions that occur on your own
|
||||
*/
|
||||
def hasRemaining(
|
||||
feature: Feature
|
||||
)(
|
||||
userId: UserId,
|
||||
contributorUserId: Option[UserId]
|
||||
): Future[Boolean] =
|
||||
minRemaining(feature)(userId, contributorUserId)
|
||||
.map(_ > 0)
|
||||
.handle { case _ => true }
|
||||
}
|
@ -0,0 +1,46 @@
|
||||
package com.twitter.tweetypie
|
||||
package backends
|
||||
|
||||
import com.twitter.servo.exception.thriftscala
|
||||
import com.twitter.servo.exception.thriftscala.ClientErrorCause
|
||||
import com.twitter.stitch.Stitch
|
||||
import com.twitter.storage.client.manhattan.kv.TimeoutManhattanException
|
||||
import com.twitter.tweetypie.core.OverCapacity
|
||||
import com.twitter.tweetypie.storage.TweetStorageClient.Ping
|
||||
import com.twitter.tweetypie.storage.ClientError
|
||||
import com.twitter.tweetypie.storage.RateLimited
|
||||
import com.twitter.tweetypie.storage.TweetStorageClient
|
||||
import com.twitter.tweetypie.util.StitchUtils
|
||||
import com.twitter.util.TimeoutException
|
||||
|
||||
object Manhattan {
|
||||
def fromClient(underlying: TweetStorageClient): TweetStorageClient =
|
||||
new TweetStorageClient {
|
||||
val addTweet = translateExceptions(underlying.addTweet)
|
||||
val deleteAdditionalFields = translateExceptions(underlying.deleteAdditionalFields)
|
||||
val getDeletedTweets = translateExceptions(underlying.getDeletedTweets)
|
||||
val getTweet = translateExceptions(underlying.getTweet)
|
||||
val getStoredTweet = translateExceptions(underlying.getStoredTweet)
|
||||
val scrub = translateExceptions(underlying.scrub)
|
||||
val softDelete = translateExceptions(underlying.softDelete)
|
||||
val undelete = translateExceptions(underlying.undelete)
|
||||
val updateTweet = translateExceptions(underlying.updateTweet)
|
||||
val hardDeleteTweet = translateExceptions(underlying.hardDeleteTweet)
|
||||
val ping: Ping = underlying.ping
|
||||
val bounceDelete = translateExceptions(underlying.bounceDelete)
|
||||
}
|
||||
|
||||
private[backends] object translateExceptions {
|
||||
private[this] def pf: PartialFunction[Throwable, Throwable] = {
|
||||
case e: RateLimited => OverCapacity(s"storage: ${e.getMessage}")
|
||||
case e: TimeoutManhattanException => new TimeoutException(e.getMessage)
|
||||
case e: ClientError => thriftscala.ClientError(ClientErrorCause.BadRequest, e.message)
|
||||
}
|
||||
|
||||
def apply[A, B](f: A => Stitch[B]): A => Stitch[B] =
|
||||
a => StitchUtils.translateExceptions(f(a), pf)
|
||||
|
||||
def apply[A, B, C](f: (A, B) => Stitch[C]): (A, B) => Stitch[C] =
|
||||
(a, b) => StitchUtils.translateExceptions(f(a, b), pf)
|
||||
}
|
||||
}
|
@ -0,0 +1,43 @@
|
||||
package com.twitter.tweetypie
|
||||
package backends
|
||||
|
||||
import com.twitter.finagle.service.RetryPolicy
|
||||
import com.twitter.mediainfo.server.thriftscala.GetTweetMediaInfoRequest
|
||||
import com.twitter.mediainfo.server.thriftscala.GetTweetMediaInfoResponse
|
||||
import com.twitter.mediainfo.server.{thriftscala => mis}
|
||||
import com.twitter.servo.util.FutureArrow
|
||||
import com.twitter.tweetypie.util.RetryPolicyBuilder
|
||||
|
||||
object MediaInfoService {
|
||||
import Backend._
|
||||
|
||||
type GetTweetMediaInfo = FutureArrow[mis.GetTweetMediaInfoRequest, mis.GetTweetMediaInfoResponse]
|
||||
|
||||
def fromClient(client: mis.MediaInfoService.MethodPerEndpoint): MediaInfoService =
|
||||
new MediaInfoService {
|
||||
val getTweetMediaInfo = FutureArrow(client.getTweetMediaInfo)
|
||||
}
|
||||
|
||||
case class Config(
|
||||
requestTimeout: Duration,
|
||||
totalTimeout: Duration,
|
||||
timeoutBackoffs: Stream[Duration]) {
|
||||
|
||||
def apply(svc: MediaInfoService, ctx: Backend.Context): MediaInfoService =
|
||||
new MediaInfoService {
|
||||
val getTweetMediaInfo: FutureArrow[GetTweetMediaInfoRequest, GetTweetMediaInfoResponse] =
|
||||
policy("getTweetMediaInfo", ctx)(svc.getTweetMediaInfo)
|
||||
}
|
||||
|
||||
private[this] def policy[A, B](name: String, ctx: Context): Builder[A, B] =
|
||||
defaultPolicy(name, requestTimeout, retryPolicy, ctx, totalTimeout = totalTimeout)
|
||||
|
||||
private[this] def retryPolicy[B]: RetryPolicy[Try[B]] =
|
||||
RetryPolicyBuilder.timeouts[Any](timeoutBackoffs)
|
||||
}
|
||||
}
|
||||
|
||||
trait MediaInfoService {
|
||||
import MediaInfoService._
|
||||
val getTweetMediaInfo: GetTweetMediaInfo
|
||||
}
|
@ -0,0 +1,73 @@
|
||||
package com.twitter.tweetypie
|
||||
package backends
|
||||
|
||||
import com.twitter.conversions.DurationOps._
|
||||
import com.twitter.finagle.Backoff
|
||||
import com.twitter.finagle.service.RetryPolicy
|
||||
import com.twitter.service.gen.scarecrow.thriftscala.CheckTweetResponse
|
||||
import com.twitter.service.gen.scarecrow.thriftscala.Retweet
|
||||
import com.twitter.service.gen.scarecrow.thriftscala.TieredAction
|
||||
import com.twitter.service.gen.scarecrow.thriftscala.TweetContext
|
||||
import com.twitter.service.gen.scarecrow.thriftscala.TweetNew
|
||||
import com.twitter.service.gen.scarecrow.{thriftscala => scarecrow}
|
||||
import com.twitter.servo.util.FutureArrow
|
||||
import com.twitter.tweetypie.util.RetryPolicyBuilder
|
||||
|
||||
object Scarecrow {
|
||||
import Backend._
|
||||
|
||||
type CheckTweet2 =
|
||||
FutureArrow[(scarecrow.TweetNew, scarecrow.TweetContext), scarecrow.CheckTweetResponse]
|
||||
type CheckRetweet = FutureArrow[scarecrow.Retweet, scarecrow.TieredAction]
|
||||
|
||||
def fromClient(client: scarecrow.ScarecrowService.MethodPerEndpoint): Scarecrow =
|
||||
new Scarecrow {
|
||||
val checkTweet2 = FutureArrow((client.checkTweet2 _).tupled)
|
||||
val checkRetweet = FutureArrow(client.checkRetweet _)
|
||||
def ping(): Future[Unit] = client.ping()
|
||||
}
|
||||
|
||||
case class Config(
|
||||
readTimeout: Duration,
|
||||
writeTimeout: Duration,
|
||||
timeoutBackoffs: Stream[Duration],
|
||||
scarecrowExceptionBackoffs: Stream[Duration]) {
|
||||
def apply(svc: Scarecrow, ctx: Backend.Context): Scarecrow =
|
||||
new Scarecrow {
|
||||
val checkTweet2: FutureArrow[(TweetNew, TweetContext), CheckTweetResponse] =
|
||||
writePolicy("checkTweet2", ctx)(svc.checkTweet2)
|
||||
val checkRetweet: FutureArrow[Retweet, TieredAction] =
|
||||
writePolicy("checkRetweet", ctx)(svc.checkRetweet)
|
||||
def ping(): Future[Unit] = svc.ping()
|
||||
}
|
||||
|
||||
private[this] def readPolicy[A, B](name: String, ctx: Context): Builder[A, B] =
|
||||
defaultPolicy(name, readTimeout, readRetryPolicy, ctx)
|
||||
|
||||
private[this] def writePolicy[A, B](name: String, ctx: Context): Builder[A, B] =
|
||||
defaultPolicy(name, writeTimeout, nullRetryPolicy, ctx)
|
||||
|
||||
private[this] def readRetryPolicy[B]: RetryPolicy[Try[B]] =
|
||||
RetryPolicy.combine[Try[B]](
|
||||
RetryPolicyBuilder.timeouts[B](timeoutBackoffs),
|
||||
RetryPolicy.backoff(Backoff.fromStream(scarecrowExceptionBackoffs)) {
|
||||
case Throw(ex: scarecrow.InternalServerError) => true
|
||||
}
|
||||
)
|
||||
|
||||
private[this] def nullRetryPolicy[B]: RetryPolicy[Try[B]] =
|
||||
// retry policy that runs once, and will not retry on any exception
|
||||
RetryPolicy.backoff(Backoff.fromStream(Stream(0.milliseconds))) {
|
||||
case Throw(_) => false
|
||||
}
|
||||
}
|
||||
|
||||
implicit val warmup: Warmup[Scarecrow] = Warmup[Scarecrow]("scarecrow")(_.ping())
|
||||
}
|
||||
|
||||
trait Scarecrow {
|
||||
import Scarecrow._
|
||||
val checkTweet2: CheckTweet2
|
||||
val checkRetweet: CheckRetweet
|
||||
def ping(): Future[Unit]
|
||||
}
|
@ -0,0 +1,52 @@
|
||||
package com.twitter.tweetypie
|
||||
package backends
|
||||
|
||||
import com.twitter.finagle.service.RetryPolicy
|
||||
import com.twitter.servo.util.FutureArrow
|
||||
import com.twitter.socialgraph.thriftscala.ExistsRequest
|
||||
import com.twitter.socialgraph.thriftscala.ExistsResult
|
||||
import com.twitter.socialgraph.thriftscala.RequestContext
|
||||
import com.twitter.socialgraph.{thriftscala => sg}
|
||||
import com.twitter.tweetypie.util.RetryPolicyBuilder
|
||||
|
||||
object SocialGraphService {
|
||||
import Backend._
|
||||
|
||||
type Exists =
|
||||
FutureArrow[(Seq[sg.ExistsRequest], Option[sg.RequestContext]), Seq[sg.ExistsResult]]
|
||||
|
||||
def fromClient(client: sg.SocialGraphService.MethodPerEndpoint): SocialGraphService =
|
||||
new SocialGraphService {
|
||||
val exists = FutureArrow((client.exists _).tupled)
|
||||
def ping: Future[Unit] = client.ping().unit
|
||||
}
|
||||
|
||||
case class Config(socialGraphTimeout: Duration, timeoutBackoffs: Stream[Duration]) {
|
||||
|
||||
def apply(svc: SocialGraphService, ctx: Backend.Context): SocialGraphService =
|
||||
new SocialGraphService {
|
||||
val exists: FutureArrow[(Seq[ExistsRequest], Option[RequestContext]), Seq[ExistsResult]] =
|
||||
policy("exists", socialGraphTimeout, ctx)(svc.exists)
|
||||
def ping(): Future[Unit] = svc.ping()
|
||||
}
|
||||
|
||||
private[this] def policy[A, B](
|
||||
name: String,
|
||||
requestTimeout: Duration,
|
||||
ctx: Context
|
||||
): Builder[A, B] =
|
||||
defaultPolicy(name, requestTimeout, retryPolicy, ctx)
|
||||
|
||||
private[this] def retryPolicy[B]: RetryPolicy[Try[B]] =
|
||||
RetryPolicyBuilder.timeouts[Any](timeoutBackoffs)
|
||||
}
|
||||
|
||||
implicit val warmup: Warmup[SocialGraphService] =
|
||||
Warmup[SocialGraphService]("socialgraphservice")(_.ping)
|
||||
}
|
||||
|
||||
trait SocialGraphService {
|
||||
import SocialGraphService._
|
||||
val exists: Exists
|
||||
def ping(): Future[Unit]
|
||||
}
|
@ -0,0 +1,98 @@
|
||||
package com.twitter.tweetypie
|
||||
package backends
|
||||
|
||||
import com.twitter.finagle.Backoff
|
||||
import com.twitter.finagle.service.RetryPolicy
|
||||
import com.twitter.flockdb.client.{thriftscala => flockdb, _}
|
||||
import com.twitter.servo
|
||||
import com.twitter.servo.util.RetryHandler
|
||||
import com.twitter.tweetypie.core.OverCapacity
|
||||
import com.twitter.tweetypie.util.RetryPolicyBuilder
|
||||
import com.twitter.util.Future
|
||||
import com.twitter.util.TimeoutException
|
||||
|
||||
object TFlock {
|
||||
val log = Logger(this.getClass)
|
||||
|
||||
case class Config(
|
||||
requestTimeout: Duration,
|
||||
timeoutBackoffs: Stream[Duration],
|
||||
flockExceptionBackoffs: Stream[Duration],
|
||||
overCapacityBackoffs: Stream[Duration],
|
||||
defaultPageSize: Int = 1000) {
|
||||
def apply(svc: flockdb.FlockDB.MethodPerEndpoint, ctx: Backend.Context): TFlockClient = {
|
||||
val retryHandler =
|
||||
RetryHandler[Any](
|
||||
retryPolicy(timeoutBackoffs, flockExceptionBackoffs, overCapacityBackoffs),
|
||||
ctx.timer,
|
||||
ctx.stats
|
||||
)
|
||||
val rescueHandler = translateExceptions.andThen(Future.exception)
|
||||
val exceptionCounter = new servo.util.ExceptionCounter(ctx.stats, "failures")
|
||||
val timeoutException = new TimeoutException(s"tflock: $requestTimeout")
|
||||
val wrapper =
|
||||
new WrappingFunction {
|
||||
def apply[T](f: => Future[T]): Future[T] =
|
||||
retryHandler {
|
||||
exceptionCounter {
|
||||
f.raiseWithin(ctx.timer, requestTimeout, timeoutException)
|
||||
.onFailure(logFlockExceptions)
|
||||
.rescue(rescueHandler)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
val wrappedClient = new WrappingFlockClient(svc, wrapper, wrapper)
|
||||
val statsClient = new StatsCollectingFlockService(wrappedClient, ctx.stats)
|
||||
new TFlockClient(statsClient, defaultPageSize)
|
||||
}
|
||||
}
|
||||
|
||||
def isOverCapacity(ex: flockdb.FlockException): Boolean =
|
||||
ex.errorCode match {
|
||||
case Some(flockdb.Constants.READ_OVERCAPACITY_ERROR) => true
|
||||
case Some(flockdb.Constants.WRITE_OVERCAPACITY_ERROR) => true
|
||||
case _ => false
|
||||
}
|
||||
|
||||
/**
|
||||
* Builds a RetryPolicy for tflock operations that will retry timeouts with the specified
|
||||
* timeout backoffs, and will retry non-overcapacity FlockExceptions with the
|
||||
* specified flockExceptionBackoffs backoffs, and will retry over-capacity exceptions with
|
||||
* the specified overCapacityBackoffs.
|
||||
*/
|
||||
def retryPolicy(
|
||||
timeoutBackoffs: Stream[Duration],
|
||||
flockExceptionBackoffs: Stream[Duration],
|
||||
overCapacityBackoffs: Stream[Duration]
|
||||
): RetryPolicy[Try[Any]] =
|
||||
RetryPolicy.combine[Try[Any]](
|
||||
RetryPolicyBuilder.timeouts[Any](timeoutBackoffs),
|
||||
RetryPolicy.backoff(Backoff.fromStream(flockExceptionBackoffs)) {
|
||||
case Throw(ex: flockdb.FlockException) if !isOverCapacity(ex) => true
|
||||
case Throw(_: flockdb.FlockQuotaException) => false
|
||||
},
|
||||
RetryPolicy.backoff(Backoff.fromStream(overCapacityBackoffs)) {
|
||||
case Throw(ex: flockdb.FlockException) if isOverCapacity(ex) => true
|
||||
case Throw(_: flockdb.FlockQuotaException) => true
|
||||
case Throw(_: OverCapacity) => true
|
||||
}
|
||||
)
|
||||
|
||||
val logFlockExceptions: Throwable => Unit = {
|
||||
case t: flockdb.FlockException => {
|
||||
log.info("FlockException from TFlock", t)
|
||||
}
|
||||
case _ =>
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts FlockExceptions with overcapacity codes into tweetypie's OverCapacity.
|
||||
*/
|
||||
val translateExceptions: PartialFunction[Throwable, Throwable] = {
|
||||
case t: flockdb.FlockQuotaException =>
|
||||
OverCapacity(s"tflock: throttled ${t.description}")
|
||||
case t: flockdb.FlockException if isOverCapacity(t) =>
|
||||
OverCapacity(s"tflock: ${t.description}")
|
||||
}
|
||||
}
|
@ -0,0 +1,94 @@
|
||||
package com.twitter.tweetypie
|
||||
package backends
|
||||
|
||||
import com.twitter.finagle.Backoff
|
||||
import com.twitter.finagle.service.RetryPolicy
|
||||
import com.twitter.service.talon.thriftscala.ExpandRequest
|
||||
import com.twitter.service.talon.thriftscala.ExpandResponse
|
||||
import com.twitter.service.talon.thriftscala.ResponseCode
|
||||
import com.twitter.service.talon.thriftscala.ShortenRequest
|
||||
import com.twitter.service.talon.thriftscala.ShortenResponse
|
||||
import com.twitter.service.talon.{thriftscala => talon}
|
||||
import com.twitter.servo.util.FutureArrow
|
||||
import com.twitter.tweetypie.core.OverCapacity
|
||||
import com.twitter.tweetypie.util.RetryPolicyBuilder
|
||||
|
||||
object Talon {
|
||||
import Backend._
|
||||
|
||||
type Expand = FutureArrow[talon.ExpandRequest, talon.ExpandResponse]
|
||||
type Shorten = FutureArrow[talon.ShortenRequest, talon.ShortenResponse]
|
||||
|
||||
case object TransientError extends Exception()
|
||||
case object PermanentError extends Exception()
|
||||
|
||||
def fromClient(client: talon.Talon.MethodPerEndpoint): Talon =
|
||||
new Talon {
|
||||
val shorten = FutureArrow(client.shorten _)
|
||||
val expand = FutureArrow(client.expand _)
|
||||
def ping(): Future[Unit] = client.serviceInfo().unit
|
||||
}
|
||||
|
||||
case class Config(
|
||||
shortenTimeout: Duration,
|
||||
expandTimeout: Duration,
|
||||
timeoutBackoffs: Stream[Duration],
|
||||
transientErrorBackoffs: Stream[Duration]) {
|
||||
def apply(svc: Talon, ctx: Backend.Context): Talon =
|
||||
new Talon {
|
||||
val shorten: FutureArrow[ShortenRequest, ShortenResponse] =
|
||||
policy("shorten", shortenTimeout, shortenResponseCode, ctx)(svc.shorten)
|
||||
val expand: FutureArrow[ExpandRequest, ExpandResponse] =
|
||||
policy("expand", expandTimeout, expandResponseCode, ctx)(svc.expand)
|
||||
def ping(): Future[Unit] = svc.ping()
|
||||
}
|
||||
|
||||
private[this] def policy[A, B](
|
||||
name: String,
|
||||
requestTimeout: Duration,
|
||||
getResponseCode: B => talon.ResponseCode,
|
||||
ctx: Context
|
||||
): Builder[A, B] =
|
||||
handleResponseCodes(name, getResponseCode, ctx) andThen
|
||||
defaultPolicy(name, requestTimeout, retryPolicy, ctx)
|
||||
|
||||
private[this] def retryPolicy[B]: RetryPolicy[Try[B]] =
|
||||
RetryPolicy.combine[Try[B]](
|
||||
RetryPolicyBuilder.timeouts[B](timeoutBackoffs),
|
||||
RetryPolicy.backoff(Backoff.fromStream(transientErrorBackoffs)) {
|
||||
case Throw(TransientError) => true
|
||||
}
|
||||
)
|
||||
|
||||
private[this] def handleResponseCodes[A, B](
|
||||
name: String,
|
||||
extract: B => talon.ResponseCode,
|
||||
ctx: Context
|
||||
): Builder[A, B] = {
|
||||
val scopedStats = ctx.stats.scope(name)
|
||||
val responseCodeStats = scopedStats.scope("response_code")
|
||||
_ andThen FutureArrow[B, B] { res =>
|
||||
val responseCode = extract(res)
|
||||
responseCodeStats.counter(responseCode.toString).incr()
|
||||
responseCode match {
|
||||
case talon.ResponseCode.TransientError => Future.exception(TransientError)
|
||||
case talon.ResponseCode.PermanentError => Future.exception(PermanentError)
|
||||
case talon.ResponseCode.ServerOverloaded => Future.exception(OverCapacity("talon"))
|
||||
case _ => Future.value(res)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
def shortenResponseCode(res: talon.ShortenResponse): ResponseCode = res.responseCode
|
||||
def expandResponseCode(res: talon.ExpandResponse): ResponseCode = res.responseCode
|
||||
|
||||
implicit val warmup: Warmup[Talon] = Warmup[Talon]("talon")(_.ping())
|
||||
}
|
||||
|
||||
trait Talon {
|
||||
import Talon._
|
||||
val shorten: Shorten
|
||||
val expand: Expand
|
||||
def ping(): Future[Unit]
|
||||
}
|
@ -0,0 +1,84 @@
|
||||
package com.twitter.tweetypie
|
||||
package backends
|
||||
|
||||
import com.twitter.finagle.Backoff
|
||||
import com.twitter.finagle.service.RetryPolicy
|
||||
import com.twitter.servo.util.FutureArrow
|
||||
import com.twitter.timelineservice.thriftscala.Event
|
||||
import com.twitter.timelineservice.thriftscala.PerspectiveQuery
|
||||
import com.twitter.timelineservice.thriftscala.PerspectiveResult
|
||||
import com.twitter.timelineservice.thriftscala.ProcessEventResult
|
||||
import com.twitter.timelineservice.thriftscala.StatusTimelineResult
|
||||
import com.twitter.timelineservice.thriftscala.TimelineQuery
|
||||
import com.twitter.timelineservice.{thriftscala => tls}
|
||||
import com.twitter.tweetypie.util.RetryPolicyBuilder
|
||||
|
||||
object TimelineService {
|
||||
import Backend._
|
||||
|
||||
type GetStatusTimeline = FutureArrow[Seq[tls.TimelineQuery], Seq[tls.StatusTimelineResult]]
|
||||
type GetPerspectives = FutureArrow[Seq[tls.PerspectiveQuery], Seq[tls.PerspectiveResult]]
|
||||
type ProcessEvent2 = FutureArrow[tls.Event, tls.ProcessEventResult]
|
||||
|
||||
private val warmupQuery =
|
||||
// we need a non-empty query, since tls treats empty queries as an error
|
||||
tls.TimelineQuery(
|
||||
timelineType = tls.TimelineType.User,
|
||||
timelineId = 620530287L, // same user id that timelineservice-api uses for warmup
|
||||
maxCount = 1
|
||||
)
|
||||
|
||||
def fromClient(client: tls.TimelineService.MethodPerEndpoint): TimelineService =
|
||||
new TimelineService {
|
||||
val processEvent2 = FutureArrow(client.processEvent2 _)
|
||||
val getStatusTimeline = FutureArrow(client.getStatusTimeline _)
|
||||
val getPerspectives = FutureArrow(client.getPerspectives _)
|
||||
def ping(): Future[Unit] =
|
||||
client.touchTimeline(Seq(warmupQuery)).handle { case _: tls.InternalServerError => }
|
||||
}
|
||||
|
||||
case class Config(writeRequestPolicy: Policy, readRequestPolicy: Policy) {
|
||||
|
||||
def apply(svc: TimelineService, ctx: Backend.Context): TimelineService = {
|
||||
val build = new PolicyAdvocate("TimelineService", ctx, svc)
|
||||
new TimelineService {
|
||||
val processEvent2: FutureArrow[Event, ProcessEventResult] =
|
||||
build("processEvent2", writeRequestPolicy, _.processEvent2)
|
||||
val getStatusTimeline: FutureArrow[Seq[TimelineQuery], Seq[StatusTimelineResult]] =
|
||||
build("getStatusTimeline", readRequestPolicy, _.getStatusTimeline)
|
||||
val getPerspectives: FutureArrow[Seq[PerspectiveQuery], Seq[PerspectiveResult]] =
|
||||
build("getPerspectives", readRequestPolicy, _.getPerspectives)
|
||||
def ping(): Future[Unit] = svc.ping()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
case class FailureBackoffsPolicy(
|
||||
timeoutBackoffs: Stream[Duration] = Stream.empty,
|
||||
tlsExceptionBackoffs: Stream[Duration] = Stream.empty)
|
||||
extends Policy {
|
||||
def toFailureRetryPolicy: FailureRetryPolicy =
|
||||
FailureRetryPolicy(
|
||||
RetryPolicy.combine(
|
||||
RetryPolicyBuilder.timeouts(timeoutBackoffs),
|
||||
RetryPolicy.backoff(Backoff.fromStream(tlsExceptionBackoffs)) {
|
||||
case Throw(ex: tls.InternalServerError) => true
|
||||
}
|
||||
)
|
||||
)
|
||||
|
||||
def apply[A, B](name: String, ctx: Context): Builder[A, B] =
|
||||
toFailureRetryPolicy(name, ctx)
|
||||
}
|
||||
|
||||
implicit val warmup: Warmup[TimelineService] =
|
||||
Warmup[TimelineService]("timelineservice")(_.ping())
|
||||
}
|
||||
|
||||
trait TimelineService {
|
||||
import TimelineService._
|
||||
val processEvent2: ProcessEvent2
|
||||
val getStatusTimeline: GetStatusTimeline
|
||||
val getPerspectives: GetPerspectives
|
||||
def ping(): Future[Unit]
|
||||
}
|
@ -0,0 +1,71 @@
|
||||
package com.twitter.tweetypie
|
||||
package backends
|
||||
|
||||
import com.twitter.finagle.service.RetryPolicy
|
||||
import com.twitter.servo.util.FutureArrow
|
||||
import com.twitter.tweetypie.util.RetryPolicyBuilder
|
||||
import com.twitter.user_image_service.thriftscala.ProcessTweetMediaRequest
|
||||
import com.twitter.user_image_service.thriftscala.ProcessTweetMediaResponse
|
||||
import com.twitter.user_image_service.thriftscala.UpdateProductMetadataRequest
|
||||
import com.twitter.user_image_service.thriftscala.UpdateTweetMediaRequest
|
||||
import com.twitter.user_image_service.thriftscala.UpdateTweetMediaResponse
|
||||
import com.twitter.user_image_service.{thriftscala => uis}
|
||||
|
||||
object UserImageService {
|
||||
import Backend._
|
||||
|
||||
type ProcessTweetMedia = FutureArrow[uis.ProcessTweetMediaRequest, uis.ProcessTweetMediaResponse]
|
||||
type UpdateProductMetadata = FutureArrow[uis.UpdateProductMetadataRequest, Unit]
|
||||
type UpdateTweetMedia = FutureArrow[uis.UpdateTweetMediaRequest, uis.UpdateTweetMediaResponse]
|
||||
|
||||
def fromClient(client: uis.UserImageService.MethodPerEndpoint): UserImageService =
|
||||
new UserImageService {
|
||||
val processTweetMedia = FutureArrow(client.processTweetMedia)
|
||||
val updateProductMetadata: FutureArrow[UpdateProductMetadataRequest, Unit] = FutureArrow(
|
||||
client.updateProductMetadata).unit
|
||||
val updateTweetMedia = FutureArrow(client.updateTweetMedia)
|
||||
}
|
||||
|
||||
case class Config(
|
||||
processTweetMediaTimeout: Duration,
|
||||
updateTweetMediaTimeout: Duration,
|
||||
timeoutBackoffs: Stream[Duration]) {
|
||||
|
||||
def apply(svc: UserImageService, ctx: Backend.Context): UserImageService =
|
||||
new UserImageService {
|
||||
val processTweetMedia: FutureArrow[ProcessTweetMediaRequest, ProcessTweetMediaResponse] =
|
||||
policy("processTweetMedia", processTweetMediaTimeout, ctx)(svc.processTweetMedia)
|
||||
val updateProductMetadata: FutureArrow[UpdateProductMetadataRequest, Unit] =
|
||||
policy("updateProductMetadata", processTweetMediaTimeout, ctx)(svc.updateProductMetadata)
|
||||
val updateTweetMedia: FutureArrow[UpdateTweetMediaRequest, UpdateTweetMediaResponse] =
|
||||
policy("updateTweetMedia", updateTweetMediaTimeout, ctx)(svc.updateTweetMedia)
|
||||
}
|
||||
|
||||
private[this] def policy[A, B](
|
||||
name: String,
|
||||
requestTimeout: Duration,
|
||||
ctx: Context
|
||||
): Builder[A, B] =
|
||||
defaultPolicy(
|
||||
name = name,
|
||||
requestTimeout = requestTimeout,
|
||||
retryPolicy = retryPolicy,
|
||||
ctx = ctx,
|
||||
exceptionCategorizer = {
|
||||
case _: uis.BadRequest => Some("success")
|
||||
case _ => None
|
||||
}
|
||||
)
|
||||
|
||||
private[this] def retryPolicy[B]: RetryPolicy[Try[B]] =
|
||||
RetryPolicyBuilder.timeouts[Any](timeoutBackoffs)
|
||||
}
|
||||
}
|
||||
|
||||
trait UserImageService {
|
||||
import UserImageService._
|
||||
|
||||
val processTweetMedia: ProcessTweetMedia
|
||||
val updateProductMetadata: UpdateProductMetadata
|
||||
val updateTweetMedia: UpdateTweetMedia
|
||||
}
|
@ -0,0 +1,266 @@
|
||||
package com.twitter.tweetypie
|
||||
package backends
|
||||
|
||||
import com.twitter.concurrent.AsyncSemaphore
|
||||
import com.twitter.util.Timer
|
||||
import com.twitter.util.Promise
|
||||
import scala.util.control.NoStackTrace
|
||||
|
||||
/**
|
||||
* Tools for building warmup actions on backend clients. The basic
|
||||
* idea is to make requests to backends repeatedly until they succeed.
|
||||
*/
|
||||
object Warmup {
|
||||
|
||||
/**
|
||||
* Signals that a warmup action was aborted because warmup is
|
||||
* complete.
|
||||
*/
|
||||
object WarmupComplete extends Exception with NoStackTrace
|
||||
|
||||
/**
|
||||
* Configuration for warmup actions.
|
||||
*
|
||||
* @param maxOutstandingRequests: Limit on total number of outstanding warmup requests.
|
||||
* @param maxWarmupDuration: Total amount of time warmup is allowed to take.
|
||||
* @param requestTimeouts: Time limit for individual warmup actions.
|
||||
* @param reliability: Criteria for how many times each warmup should be run.
|
||||
*/
|
||||
case class Settings(
|
||||
maxOutstandingRequests: Int,
|
||||
maxWarmupDuration: Duration,
|
||||
requestTimeouts: Map[String, Duration],
|
||||
reliability: Reliably) {
|
||||
def toRunner(logger: Logger, timer: Timer): Runner =
|
||||
new WithTimeouts(requestTimeouts, timer)
|
||||
.within(new Logged(logger))
|
||||
.within(new LimitedConcurrency(maxOutstandingRequests))
|
||||
.within(reliability)
|
||||
|
||||
def apply[A: Warmup](value: A, logger: Logger, timer: Timer): Future[Unit] =
|
||||
toRunner(logger, timer)
|
||||
.run(value)
|
||||
.raiseWithin(maxWarmupDuration)(timer)
|
||||
.handle { case _ => }
|
||||
}
|
||||
|
||||
/**
|
||||
* Strategy for running Warmup actions.
|
||||
*/
|
||||
trait Runner { self =>
|
||||
|
||||
/**
|
||||
* Run one single warmup action.
|
||||
*/
|
||||
def runOne(name: String, action: => Future[Unit]): Future[Unit]
|
||||
|
||||
/**
|
||||
* Compose these two Runners by calling this Runner's runOne
|
||||
* inside of other's runOne.
|
||||
*/
|
||||
final def within(other: Runner): Runner =
|
||||
new Runner {
|
||||
override def runOne(name: String, action: => Future[Unit]): Future[Unit] =
|
||||
other.runOne(name, self.runOne(name, action))
|
||||
}
|
||||
|
||||
/**
|
||||
* Execute all of the warmup actions for the given value using
|
||||
* this runner.
|
||||
*/
|
||||
final def run[T](t: T)(implicit w: Warmup[T]): Future[Unit] =
|
||||
Future.join(w.actions.toSeq.map { case (name, f) => runOne(name, f(t).unit) })
|
||||
}
|
||||
|
||||
/**
|
||||
* Set a ceiling on the amount of time each kind of warmup action is
|
||||
* allowed to take.
|
||||
*/
|
||||
class WithTimeouts(timeouts: Map[String, Duration], timer: Timer) extends Runner {
|
||||
override def runOne(name: String, action: => Future[Unit]): Future[Unit] =
|
||||
timeouts.get(name).map(action.raiseWithin(_)(timer)).getOrElse(action)
|
||||
}
|
||||
|
||||
/**
|
||||
* Execute each action until its reliability is estimated to be
|
||||
* above the given threshold. The reliability is initially assumed
|
||||
* to be zero. The reliability is estimated as an exponential moving
|
||||
* average, with the new data point given the appropriate weight so
|
||||
* that a single data point will no longer be able to push the
|
||||
* average below the threshold.
|
||||
*
|
||||
* The warmup action is considered successful if it does not throw
|
||||
* an exception. No timeouts are applied.
|
||||
*
|
||||
* The threshold must be in the interval [0, 1).
|
||||
*
|
||||
* The concurrency level determines how many outstanding requests
|
||||
* to maintain until the threshold is reached. This allows warmup
|
||||
* to happen more rapidly when individual requests have high
|
||||
* latency.
|
||||
*
|
||||
* maxAttempts limits the total number of tries that we are allowed
|
||||
* to try to reach the reliability threshold. This is a safety
|
||||
* measure to prevent overloading whatever subsystem we are
|
||||
* attempting to warm up.
|
||||
*/
|
||||
case class Reliably(reliabilityThreshold: Double, concurrency: Int, maxAttempts: Int)
|
||||
extends Runner {
|
||||
require(reliabilityThreshold < 1)
|
||||
require(reliabilityThreshold >= 0)
|
||||
require(concurrency > 0)
|
||||
require(maxAttempts > 0)
|
||||
|
||||
// Find the weight at which one failure will not push us under the
|
||||
// reliabilityThreshold.
|
||||
val weight: Double = 1 - math.pow(
|
||||
1 - reliabilityThreshold,
|
||||
(1 - reliabilityThreshold) / reliabilityThreshold
|
||||
)
|
||||
|
||||
// Make sure that rounding error did not cause weight to become zero.
|
||||
require(weight > 0)
|
||||
require(weight <= 1)
|
||||
|
||||
// On each iteration, we discount the current reliability by this
|
||||
// factor before adding in the new reliability data point.
|
||||
val decay: Double = 1 - weight
|
||||
|
||||
// Make sure that rounding error did not cause decay to be zero.
|
||||
require(decay < 1)
|
||||
|
||||
override def runOne(name: String, action: => Future[Unit]): Future[Unit] = {
|
||||
def go(attempts: Int, reliability: Double, outstanding: Seq[Future[Unit]]): Future[Unit] =
|
||||
if (reliability >= reliabilityThreshold || (attempts == 0 && outstanding.isEmpty)) {
|
||||
// We hit the threshold or ran out of tries. Don't cancel any
|
||||
// outstanding requests, just wait for them all to complete.
|
||||
Future.join(outstanding.map(_.handle { case _ => }))
|
||||
} else if (attempts > 0 && outstanding.length < concurrency) {
|
||||
// We have not yet hit the reliability threshold, and we
|
||||
// still have available concurrency, so make a new request.
|
||||
go(attempts - 1, reliability, action +: outstanding)
|
||||
} else {
|
||||
val sel = Future.select(outstanding)
|
||||
|
||||
// We need this promise wrapper because if the select is
|
||||
// interrupted, it relays the interrupt to the outstanding
|
||||
// requests but does not itself return with a
|
||||
// failure. Wrapping in a promise lets us differentiate
|
||||
// between an interrupt coming from above and the created
|
||||
// Future failing for another reason.
|
||||
val p = new Promise[(Try[Unit], Seq[Future[Unit]])]
|
||||
p.setInterruptHandler {
|
||||
case e =>
|
||||
// Interrupt the outstanding requests.
|
||||
sel.raise(e)
|
||||
// Halt the computation with a failure.
|
||||
p.updateIfEmpty(Throw(e))
|
||||
}
|
||||
|
||||
// When the select finishes, update the promise with the value.
|
||||
sel.respond(p.updateIfEmpty)
|
||||
p.flatMap {
|
||||
case (tryRes, remaining) =>
|
||||
val delta = if (tryRes.isReturn) weight else 0
|
||||
go(attempts, reliability * decay + delta, remaining)
|
||||
}
|
||||
}
|
||||
|
||||
go(maxAttempts, 0, Seq.empty)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Write a log message recording each invocation of each warmup
|
||||
* action. The log message is comma-separated, with the following
|
||||
* fields:
|
||||
*
|
||||
* name:
|
||||
* The supplied name.
|
||||
*
|
||||
* start time:
|
||||
* The number of milliseconds since the start of the Unix
|
||||
* epoch.
|
||||
*
|
||||
* duration:
|
||||
* How long this warmup action took, in milliseconds.
|
||||
*
|
||||
* result:
|
||||
* "passed" or "failed" depending on whether the Future
|
||||
* returned an exception.
|
||||
*
|
||||
* exception type:
|
||||
* If the result "failed", then this will be the name of
|
||||
* the exception that casued the failure. If it "passed",
|
||||
* it will be the empty string.
|
||||
*
|
||||
* These messages should be sufficient to get a picture of how
|
||||
* warmup proceeded, since they allow the messages to be ordered
|
||||
* and sorted by type. You can use this information to tune the
|
||||
* warmup parameters.
|
||||
*/
|
||||
class Logged(logger: Logger) extends Runner {
|
||||
override def runOne(name: String, action: => Future[Unit]): Future[Unit] = {
|
||||
val start = Time.now
|
||||
val startStr = start.sinceEpoch.inMilliseconds.toString
|
||||
|
||||
action.respond {
|
||||
case Throw(WarmupComplete) =>
|
||||
// Don't log anything for computations that we abandoned
|
||||
// because warmup is complete.
|
||||
|
||||
case r =>
|
||||
val duration = (Time.now - start).inMilliseconds
|
||||
val result = r match {
|
||||
case Throw(e) => "failed," + e.toString.takeWhile(_ != '\n')
|
||||
case _ => "passed,"
|
||||
}
|
||||
logger.info(s"$name,${startStr}ms,${duration}ms,$result")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Ensure that no more than the specified number of invocations of a
|
||||
* warmup action are happening at one time.
|
||||
*/
|
||||
class LimitedConcurrency(limit: Int) extends Runner {
|
||||
private[this] val sem = new AsyncSemaphore(limit)
|
||||
override def runOne(name: String, action: => Future[Unit]): Future[Unit] =
|
||||
sem.acquireAndRun(action)
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new Warmup that performs this single action.
|
||||
*/
|
||||
def apply[A](name: String)(f: A => Future[_]): Warmup[A] = new Warmup(Map(name -> f))
|
||||
|
||||
/**
|
||||
* Create a Warmup that does nothing. This is useful in concert with
|
||||
* warmField.
|
||||
*/
|
||||
def empty[A]: Warmup[A] = new Warmup[A](Map.empty)
|
||||
}
|
||||
|
||||
/**
|
||||
* A set of independent warmup actions. Each action should be the
|
||||
* minimum work that can be done in order to exercise a code
|
||||
* path. Runners can be used to e.g. run the actions repeatedly or
|
||||
* with timeouts.
|
||||
*/
|
||||
class Warmup[A](val actions: Map[String, A => Future[_]]) {
|
||||
def ++(other: Warmup[A]) = new Warmup[A](actions ++ other.actions)
|
||||
|
||||
/**
|
||||
* The names of the individual warmup actions that this warmup is
|
||||
* composed of.
|
||||
*/
|
||||
def names: Set[String] = actions.keySet
|
||||
|
||||
/**
|
||||
* Create a new Warmup that does all of the actions of this warmup
|
||||
* and additionally does warmup on the value specified by `f`.
|
||||
*/
|
||||
def warmField[B](f: A => B)(implicit w: Warmup[B]): Warmup[A] =
|
||||
new Warmup[A](actions ++ (w.actions.mapValues(f.andThen)))
|
||||
}
|
@ -0,0 +1,135 @@
|
||||
scala_library(
|
||||
sources = ["*.scala"],
|
||||
compiler_option_sets = ["fatal_warnings"],
|
||||
strict_deps = True,
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"3rdparty/jvm/com/fasterxml/jackson/dataformat:jackson-dataformat-yaml",
|
||||
"3rdparty/jvm/com/fasterxml/jackson/module:jackson-module-scala",
|
||||
"3rdparty/jvm/io/netty:netty4-tcnative-boringssl-static",
|
||||
"3rdparty/jvm/org/apache/kafka:kafka-clients",
|
||||
"3rdparty/jvm/org/apache/thrift:libthrift",
|
||||
"ads-common/loggingclient/src/main/scala",
|
||||
"core-app-services/failed_task/src/scala/com/twitter/coreservices/failed_task/writer",
|
||||
"creatives-container/thrift/src/main/thrift:creatives-container-service-scala",
|
||||
"decider",
|
||||
"deferredrpc/client",
|
||||
"deferredrpc/client/src/main/thrift:thrift-scala",
|
||||
"eventbus/client",
|
||||
"fanoutservice/thrift/src/main/thrift:thrift-scala",
|
||||
"featureswitches/featureswitches-core:v2",
|
||||
"featureswitches/featureswitches-core/src/main/scala",
|
||||
"finagle-internal/mtls/src/main/scala/com/twitter/finagle/mtls/authentication",
|
||||
"finagle-internal/mtls/src/main/scala/com/twitter/finagle/mtls/authorization",
|
||||
"finagle-internal/mtls/src/main/scala/com/twitter/finagle/mtls/authorization/server",
|
||||
"finagle-internal/mtls/src/main/scala/com/twitter/finagle/mtls/client",
|
||||
"finagle-internal/mtls/src/main/scala/com/twitter/finagle/mtls/server",
|
||||
"finagle-internal/mtls/src/main/scala/com/twitter/finagle/mtls/transport",
|
||||
"finagle/finagle-core/src/main",
|
||||
"finagle/finagle-http/src/main/scala",
|
||||
"finagle/finagle-memcached/src/main/scala",
|
||||
"finagle/finagle-mux/src/main/scala",
|
||||
"finagle/finagle-stats",
|
||||
"finagle/finagle-thrift",
|
||||
"finagle/finagle-thrift/src/main/java",
|
||||
"finagle/finagle-thriftmux",
|
||||
"finatra-internal/kafka/src/main/scala/com/twitter/finatra/kafka/producers",
|
||||
"finatra/inject/inject-slf4j/src/main/scala/com/twitter/inject",
|
||||
"flock-client/src/main/scala",
|
||||
"flock-client/src/main/thrift:thrift-scala",
|
||||
"geoduck/service/src/main/scala/com/twitter/geoduck/service/common/clientmodules",
|
||||
"geoduck/util/src/main/scala/com/twitter/geoduck/util/service",
|
||||
"kafka/finagle-kafka/finatra-kafka/src/main/scala",
|
||||
"limiter-client",
|
||||
"limiter/thrift-only/src/main/thrift:thrift-scala",
|
||||
"mediaservices/mediainfo-server/thrift/src/main/thrift:thrift-scala",
|
||||
"passbird/thrift-only/src/main/thrift:thrift-scala",
|
||||
"quill/capture",
|
||||
"quill/core/src/main/thrift:thrift-scala",
|
||||
"scrooge/scrooge-core",
|
||||
"tweetypie/servo/repo/src/main/scala",
|
||||
"tweetypie/servo/repo/src/main/thrift:thrift-scala",
|
||||
"tweetypie/servo/request/src/main/scala",
|
||||
"tweetypie/servo/util",
|
||||
"snowflake:client",
|
||||
"snowflake/src/main/scala/com/twitter/snowflake/id",
|
||||
"snowflake/src/main/thrift:thrift-scala",
|
||||
"src/scala/com/twitter/ads/internal/pcl/service",
|
||||
"src/scala/com/twitter/search/blender/services/strato",
|
||||
"src/thrift/com/twitter/ads/adserver:adserver_rpc-scala",
|
||||
"src/thrift/com/twitter/context:twitter-context-scala",
|
||||
"src/thrift/com/twitter/escherbird:annotation-service-scala",
|
||||
"src/thrift/com/twitter/escherbird/metadata:metadata-service-scala",
|
||||
"src/thrift/com/twitter/expandodo:cards-scala",
|
||||
"src/thrift/com/twitter/expandodo:only-scala",
|
||||
"src/thrift/com/twitter/geoduck:geoduck-scala",
|
||||
"src/thrift/com/twitter/gizmoduck:thrift-scala",
|
||||
"src/thrift/com/twitter/merlin:thrift-scala",
|
||||
"src/thrift/com/twitter/service/scarecrow/gen:scarecrow-scala",
|
||||
"src/thrift/com/twitter/service/talon/gen:thrift-scala",
|
||||
"src/thrift/com/twitter/socialgraph:thrift-scala",
|
||||
"src/thrift/com/twitter/spam/rtf:tweet-rtf-event-scala",
|
||||
"src/thrift/com/twitter/timelineservice:thrift-scala",
|
||||
"src/thrift/com/twitter/timelineservice/server/internal:thrift-scala",
|
||||
"tweetypie/common/src/thrift/com/twitter/tweetypie:delete_location_data-scala",
|
||||
"tweetypie/common/src/thrift/com/twitter/tweetypie:events-scala",
|
||||
"tweetypie/common/src/thrift/com/twitter/tweetypie:service-scala",
|
||||
"tweetypie/common/src/thrift/com/twitter/tweetypie:stored-tweet-info-scala",
|
||||
"tweetypie/common/src/thrift/com/twitter/tweetypie:tweet-scala",
|
||||
"tweetypie/common/src/thrift/com/twitter/tweetypie:tweet_comparison_service-scala",
|
||||
"stitch/stitch-core",
|
||||
"stitch/stitch-repo/src/main/scala",
|
||||
"stitch/stitch-timelineservice/src/main/scala",
|
||||
"storage/clients/manhattan/client/src/main/scala",
|
||||
"strato/src/main/scala/com/twitter/strato/catalog",
|
||||
"strato/src/main/scala/com/twitter/strato/client",
|
||||
"strato/src/main/scala/com/twitter/strato/fed/server",
|
||||
"strato/src/main/scala/com/twitter/strato/rpc",
|
||||
"strato/src/main/scala/com/twitter/strato/server",
|
||||
"strato/src/main/scala/com/twitter/strato/util",
|
||||
"stringcenter/client/src/main/scala",
|
||||
"tweetypie/server/config",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie/backends",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie/core",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie/federated",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie/federated/warmups",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie/handler",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie/hydrator",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie/media",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie/repository",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie/serverutil",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie/serverutil/logcachewrites",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie/service",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie/store",
|
||||
"tweetypie/server/src/main/thrift:compiled-scala",
|
||||
"tweetypie/common/src/scala/com/twitter/tweetypie/caching",
|
||||
"tweetypie/common/src/scala/com/twitter/tweetypie/client_id",
|
||||
"tweetypie/common/src/scala/com/twitter/tweetypie/context",
|
||||
"tweetypie/common/src/scala/com/twitter/tweetypie/decider",
|
||||
"tweetypie/common/src/scala/com/twitter/tweetypie/jiminy/tweetypie",
|
||||
"tweetypie/common/src/scala/com/twitter/tweetypie/matching",
|
||||
"tweetypie/common/src/scala/com/twitter/tweetypie/media",
|
||||
"tweetypie/common/src/scala/com/twitter/tweetypie/storage",
|
||||
"tweetypie/common/src/scala/com/twitter/tweetypie/tflock",
|
||||
"tweetypie/common/src/scala/com/twitter/tweetypie/thriftscala",
|
||||
"tweetypie/common/src/scala/com/twitter/tweetypie/thriftscala/entities",
|
||||
"tweetypie/common/src/scala/com/twitter/tweetypie/tweettext",
|
||||
"tweetypie/common/src/scala/com/twitter/tweetypie/util",
|
||||
"twitter-config/yaml",
|
||||
"twitter-context",
|
||||
"twitter-server-internal",
|
||||
"twitter-server/server/src/main/scala",
|
||||
"user-image-service/thrift/src/main/thrift:thrift-scala",
|
||||
"util/util-app",
|
||||
"util/util-hashing/src/main/scala",
|
||||
"util/util-slf4j-api/src/main/scala",
|
||||
"util/util-stats/src/main/scala",
|
||||
"visibility/common/src/main/scala/com/twitter/visibility/common",
|
||||
"visibility/common/src/main/scala/com/twitter/visibility/common/tflock",
|
||||
"visibility/lib:tweets",
|
||||
"visibility/lib/src/main/scala/com/twitter/visibility/util",
|
||||
"visibility/writer/src/main/scala/com/twitter/visibility/writer",
|
||||
"visibility/writer/src/main/scala/com/twitter/visibility/writer/interfaces/tweets",
|
||||
],
|
||||
)
|
@ -0,0 +1,796 @@
|
||||
package com.twitter.tweetypie
|
||||
package config
|
||||
|
||||
import com.twitter.ads.internal.pcl.service.CallbackPromotedContentLogger
|
||||
import com.twitter.ads.loggingclient.AdsLoggingClient
|
||||
import com.twitter.adserver.thriftscala.AdCallbackEvent
|
||||
import com.twitter.conversions.DurationOps._
|
||||
import com.twitter.conversions.PercentOps._
|
||||
import com.twitter.container.{thriftscala => ccs}
|
||||
import com.twitter.deferredrpc.client.DeferredThriftService
|
||||
import com.twitter.deferredrpc.thrift.Datacenter
|
||||
import com.twitter.deferredrpc.thrift.DeferredRPC
|
||||
import com.twitter.deferredrpc.thrift.Target
|
||||
import com.twitter.escherbird.thriftscala.TweetEntityAnnotationService$FinagleClient
|
||||
import com.twitter.escherbird.thriftscala.{
|
||||
TweetEntityAnnotationService => TweetEntityAnnotationScroogeIface
|
||||
}
|
||||
import com.twitter.eventbus.client.EventBusPublisher
|
||||
import com.twitter.eventbus.client.EventBusPublisherBuilder
|
||||
import com.twitter.expandodo.thriftscala.CardsService$FinagleClient
|
||||
import com.twitter.expandodo.thriftscala.{CardsService => CardsScroogeIface}
|
||||
import com.twitter.finagle._
|
||||
import com.twitter.finagle.builder.ClientBuilder
|
||||
import com.twitter.finagle.client.Transporter
|
||||
import com.twitter.finagle.factory.TimeoutFactory
|
||||
import com.twitter.finagle.liveness.FailureAccrualFactory
|
||||
import com.twitter.finagle.loadbalancer.Balancers
|
||||
import com.twitter.finagle.mtls.authentication.EmptyServiceIdentifier
|
||||
import com.twitter.finagle.mtls.client.MtlsClientBuilder._
|
||||
import com.twitter.finagle.mtls.client.MtlsStackClient._
|
||||
import com.twitter.finagle.partitioning.param
|
||||
import com.twitter.finagle.service.TimeoutFilter.PropagateDeadlines
|
||||
import com.twitter.finagle.service._
|
||||
import com.twitter.finagle.ssl.OpportunisticTls
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.finagle.thrift.ThriftClientRequest
|
||||
import com.twitter.finagle.thriftmux.MethodBuilder
|
||||
import com.twitter.finagle.tracing.DefaultTracer
|
||||
import com.twitter.flockdb.client.thriftscala.FlockDB
|
||||
import com.twitter.flockdb.client.FlockResponse
|
||||
import com.twitter.flockdb.client.TFlockClient
|
||||
import com.twitter.flockdb.client.UserTimelineGraph
|
||||
import com.twitter.geoduck.backend.hydration.thriftscala.{Hydration => GeoduckHydration}
|
||||
import com.twitter.geoduck.backend.relevance.thriftscala.Relevance
|
||||
import com.twitter.geoduck.backend.relevance.thriftscala.Relevance$FinagleClient
|
||||
import com.twitter.geoduck.backend.relevance.thriftscala.RelevanceContext
|
||||
import com.twitter.geoduck.service.common.clientmodules.GeoduckGeohashLocate
|
||||
import com.twitter.geoduck.thriftscala.ReverseGeocoder
|
||||
import com.twitter.geoduck.util.service.GeoduckLocate
|
||||
import com.twitter.gizmoduck.thriftscala.UserService
|
||||
import com.twitter.hashing.KeyHasher
|
||||
import com.twitter.limiter.client.LimiterClientFactory
|
||||
import com.twitter.mediainfo.server.thriftscala.MediaInfoService$FinagleClient
|
||||
import com.twitter.mediainfo.server.thriftscala.{MediaInfoService => MediaInfoScroogeIface}
|
||||
import com.twitter.merlin.thriftscala.UserRolesService
|
||||
import com.twitter.passbird.thriftscala.PassbirdService
|
||||
import com.twitter.passbird.thriftscala.PassbirdService$FinagleClient
|
||||
import com.twitter.service.gen.scarecrow.thriftscala.ScarecrowService$FinagleClient
|
||||
import com.twitter.service.gen.scarecrow.thriftscala.{ScarecrowService => ScarecrowScroogeIface}
|
||||
import com.twitter.service.talon.thriftscala.Talon$FinagleClient
|
||||
import com.twitter.service.talon.thriftscala.{Talon => TalonScroogeIface}
|
||||
import com.twitter.snowflake.client.SnowflakeClient
|
||||
import com.twitter.snowflake.thriftscala.Snowflake
|
||||
import com.twitter.socialgraph.thriftscala.SocialGraphService$FinagleClient
|
||||
import com.twitter.socialgraph.thriftscala.{SocialGraphService => SocialGraphScroogeIface}
|
||||
import com.twitter.storage.client.manhattan.kv.Experiments
|
||||
import com.twitter.storage.client.manhattan.kv.ManhattanKVClient
|
||||
import com.twitter.storage.client.manhattan.kv.ManhattanKVClientMtlsParams
|
||||
import com.twitter.storage.client.manhattan.kv.NoMtlsParams
|
||||
import com.twitter.strato.client.Strato
|
||||
import com.twitter.strato.client.{Client => StratoClient}
|
||||
import com.twitter.timelineservice.fanout.thriftscala.FanoutService
|
||||
import com.twitter.timelineservice.fanout.thriftscala.FanoutService$FinagleClient
|
||||
import com.twitter.timelineservice.{thriftscala => tls}
|
||||
import com.twitter.tweetypie.backends._
|
||||
import com.twitter.tweetypie.client_id.ClientIdHelper
|
||||
import com.twitter.tweetypie.media.MediaClient
|
||||
import com.twitter.tweetypie.service.ReplicatingTweetService.GatedReplicationClient
|
||||
import com.twitter.tweetypie.storage.ManhattanTweetStorageClient
|
||||
import com.twitter.tweetypie.storage.TweetStorageClient
|
||||
import com.twitter.tweetypie.store._
|
||||
import com.twitter.tweetypie.thriftscala.DeleteLocationData
|
||||
import com.twitter.tweetypie.thriftscala.RetweetArchivalEvent
|
||||
import com.twitter.tweetypie.thriftscala.TweetEvent
|
||||
import com.twitter.tweetypie.thriftscala.TweetServiceInternal$FinagleClient
|
||||
import com.twitter.user_image_service.thriftscala.UserImageService$FinagleClient
|
||||
import com.twitter.user_image_service.thriftscala.{UserImageService => UserImageScroogeIface}
|
||||
import com.twitter.util.Throw
|
||||
import com.twitter.util.Timer
|
||||
import com.twitter.util.{TimeoutException => UtilTimeoutException}
|
||||
import scala.util.Random
|
||||
|
||||
trait BackendClients {
|
||||
|
||||
/** returns all the finagle.Names created while building clients */
|
||||
def referencedNames: Seq[Name]
|
||||
|
||||
val asyncRetryTweetService: ThriftTweetService
|
||||
val asyncTweetDeletionService: ThriftTweetService
|
||||
val asyncTweetService: ThriftTweetService
|
||||
val configBus: ConfigBus
|
||||
val creativesContainerService: CreativesContainerService
|
||||
val darkTrafficClient: Service[Array[Byte], Array[Byte]]
|
||||
val deleteLocationDataPublisher: EventBusPublisher[DeleteLocationData]
|
||||
val escherbird: Escherbird
|
||||
val expandodo: Expandodo
|
||||
val fanoutServiceClient: FanoutService.MethodPerEndpoint
|
||||
val geoHydrationLocate: GeoduckLocate
|
||||
val geoRelevance: Relevance.MethodPerEndpoint
|
||||
val geoScrubEventStore: GeoScrubEventStore
|
||||
val geoduckGeohashLocate: GeoduckGeohashLocate
|
||||
val gizmoduck: Gizmoduck
|
||||
val gnipEnricherator: GnipEnricherator
|
||||
val guano: Guano
|
||||
val limiterService: LimiterService
|
||||
val lowQoSReplicationClients: Seq[GatedReplicationClient]
|
||||
val mediaClient: MediaClient
|
||||
val mediaInfoService: MediaInfoService
|
||||
val memcacheClient: memcached.Client
|
||||
val merlin: UserRolesService.MethodPerEndpoint
|
||||
val passbirdClient: PassbirdService.MethodPerEndpoint
|
||||
val replicationClient: ThriftTweetService
|
||||
val retweetArchivalEventPublisher: EventBusPublisher[RetweetArchivalEvent]
|
||||
val scarecrow: Scarecrow
|
||||
val snowflakeClient: SnowflakeClient.SnowflakeClient
|
||||
val socialGraphService: SocialGraphService
|
||||
val stratoserverClient: StratoClient
|
||||
val talon: Talon
|
||||
val tflockReadClient: TFlockClient
|
||||
val tflockWriteClient: TFlockClient
|
||||
val timelineService: TimelineService
|
||||
val tweetEventsPublisher: EventBusPublisher[TweetEvent]
|
||||
val tweetStorageClient: TweetStorageClient
|
||||
val userImageService: UserImageService
|
||||
val callbackPromotedContentLogger: CallbackPromotedContentLogger
|
||||
}
|
||||
|
||||
/**
|
||||
* default implementation of BackendClients that connects to real, remote
|
||||
* backend services.
|
||||
*/
|
||||
object BackendClients {
|
||||
// for most services, tweetypie typically maintains only a single connection to
|
||||
// each host in the cluster, and that is enough for normal steady-state work.
|
||||
// to prevent ddos'ing backends during unusual traffic influxes, we set the host
|
||||
// connection limit to be 2-3x the steady-state daily peak, giving plenty of head
|
||||
// room but without allowing an excessive number of connections.
|
||||
private val defaultHostConnectionLimit = 3
|
||||
|
||||
// 100ms is greater than most gc pauses; smaller values cause more timeouts
|
||||
private val defaultConnectTimeout = 100.milliseconds
|
||||
// tcpConnect timeout is less than half of defaultConnectTimeout, to allow at least
|
||||
// two tries (except when there is a GC pause)
|
||||
private val defaultTcpConnectTimeout = 20.milliseconds
|
||||
|
||||
private val WriteExceptionsOnly: PartialFunction[Try[Nothing], Boolean] =
|
||||
RetryPolicy.WriteExceptionsOnly
|
||||
|
||||
private val ClosedExceptionsOnly: PartialFunction[Try[Nothing], Boolean] = {
|
||||
case Throw(_: ChannelClosedException) => true
|
||||
}
|
||||
|
||||
private val TimeoutExceptionsOnly: PartialFunction[Try[Nothing], Boolean] = {
|
||||
case Throw(_: TimeoutException) => true
|
||||
case Throw(_: UtilTimeoutException) => true
|
||||
}
|
||||
|
||||
private val NoBackoff = Backoff.const(0.second)
|
||||
|
||||
private def retry(writeExceptions: Int = 100, closedExceptions: Int = 2, timeouts: Int = 0) =
|
||||
RetryPolicy.combine(
|
||||
RetryPolicy.backoff(NoBackoff.take(writeExceptions))(WriteExceptionsOnly),
|
||||
RetryPolicy.backoff(NoBackoff.take(closedExceptions))(ClosedExceptionsOnly),
|
||||
RetryPolicy.backoff(NoBackoff.take(timeouts))(TimeoutExceptionsOnly)
|
||||
)
|
||||
|
||||
implicit val warmup: Warmup[BackendClients] = {
|
||||
// Use a random string so that the keys are likely to hash to
|
||||
// different memcache instances. Request multiple keys at a time so
|
||||
// that we don't consider the backend warm just because we can get a
|
||||
// bunch of successful responses to one cache.
|
||||
val cacheGet = (_: memcached.Client).get(Seq.fill(20)(Random.nextLong.toString))
|
||||
|
||||
Warmup
|
||||
.empty[BackendClients]
|
||||
.warmField(_.expandodo)
|
||||
.warmField(_.gizmoduck)
|
||||
.warmField(_.memcacheClient)(Warmup("memcache")(cacheGet))
|
||||
.warmField(_.talon)
|
||||
.warmField(_.tweetStorageClient)(Warmup("tweetstorage")(_.ping()))
|
||||
.warmField(_.tflockReadClient)(Warmup("tflock")(_.contains(UserTimelineGraph, 0, 0)))
|
||||
.warmField(_.scarecrow)
|
||||
.warmField(_.socialGraphService)
|
||||
.warmField(_.timelineService)
|
||||
.warmField(_.geoRelevance)(Warmup("geo_relevance")(_.placeSearch(RelevanceContext())))
|
||||
}
|
||||
|
||||
def apply(
|
||||
settings: TweetServiceSettings,
|
||||
deciderGates: TweetypieDeciderGates,
|
||||
statsReceiver: StatsReceiver,
|
||||
hostStatsReceiver: StatsReceiver,
|
||||
timer: Timer,
|
||||
clientIdHelper: ClientIdHelper,
|
||||
): BackendClients = {
|
||||
val thriftClientId = settings.thriftClientId
|
||||
val tracer = DefaultTracer
|
||||
|
||||
val env = settings.env.toString
|
||||
val zone = settings.zone
|
||||
val log = Logger(getClass)
|
||||
val backendsScope = statsReceiver.scope("backends")
|
||||
|
||||
/** a Seq builder of finagle.Names loaded via getName */
|
||||
val referencedNamesBuilder = Seq.newBuilder[Name]
|
||||
|
||||
/** the default set of exceptions we believe are safe for Tweetypie to retry */
|
||||
val defaultResponseClassifier: ResponseClassifier =
|
||||
ResponseClassifier.RetryOnChannelClosed.orElse(ResponseClassifier.RetryOnTimeout)
|
||||
|
||||
/**
|
||||
* Resolve a string into a Finagle Name and record it
|
||||
* in referencedNames.
|
||||
*/
|
||||
def eval(address: String): Name = {
|
||||
val name = Resolver.eval(address)
|
||||
referencedNamesBuilder += name
|
||||
name
|
||||
}
|
||||
|
||||
def backendContext(name: String) =
|
||||
Backend.Context(timer, backendsScope.scope(name))
|
||||
|
||||
// by default, retries on most exceptions (see defaultRetryExceptions). if an rpc is not
|
||||
// idempotent, it should use a different retry policy.
|
||||
def clientBuilder(name: String) = {
|
||||
ClientBuilder()
|
||||
.name(name)
|
||||
.reportTo(statsReceiver)
|
||||
.reportHostStats(hostStatsReceiver)
|
||||
.tracer(tracer)
|
||||
.daemon(true)
|
||||
.tcpConnectTimeout(defaultTcpConnectTimeout)
|
||||
.connectTimeout(defaultConnectTimeout)
|
||||
.retryPolicy(retry())
|
||||
}
|
||||
|
||||
def thriftMuxClientBuilder(name: String, address: String, clazz: Class[_]) = {
|
||||
clientBuilder(name)
|
||||
.stack(
|
||||
ThriftMux.client
|
||||
.withClientId(thriftClientId)
|
||||
.withOpportunisticTls(OpportunisticTls.Required)
|
||||
.withServiceClass(clazz))
|
||||
.loadBalancer(balancer())
|
||||
.dest(eval(address))
|
||||
.mutualTls(settings.serviceIdentifier)
|
||||
}
|
||||
|
||||
// Our base ThriftMux.Client
|
||||
// Prefer using thriftMuxMethodBuilder below but
|
||||
// can be used to build custom clients (re: darkTrafficClient)
|
||||
def thriftMuxClient(name: String, propagateDeadlines: Boolean = true): ThriftMux.Client = {
|
||||
ThriftMux.client
|
||||
.withClientId(thriftClientId)
|
||||
.withLabel(name)
|
||||
.withStatsReceiver(statsReceiver)
|
||||
.withTracer(tracer)
|
||||
.withTransport.connectTimeout(defaultTcpConnectTimeout)
|
||||
.withSession.acquisitionTimeout(defaultConnectTimeout)
|
||||
.withMutualTls(settings.serviceIdentifier)
|
||||
.withOpportunisticTls(OpportunisticTls.Required)
|
||||
.configured(PropagateDeadlines(enabled = propagateDeadlines))
|
||||
}
|
||||
|
||||
// If an endpoint is non-idempotent you should add .nonidempotent and
|
||||
// leave off any ResponseClassifiers (it will remove any placed before but not after)
|
||||
// If it is unequivocally idempotent you should add .idempotent and
|
||||
// leave off any ResponseClassifiers (it will retry on all Throws). This will also
|
||||
// enable backup requests
|
||||
def thriftMuxMethodBuilder(
|
||||
name: String,
|
||||
dest: String,
|
||||
): MethodBuilder = {
|
||||
thriftMuxClient(name)
|
||||
.withLoadBalancer(balancer(minAperture = 2))
|
||||
.methodBuilder(dest)
|
||||
.withRetryForClassifier(defaultResponseClassifier)
|
||||
.withTimeoutTotal(2.seconds) // total timeout including 1st attempt and up to 2 retries
|
||||
}
|
||||
|
||||
def balancer(minAperture: Int = 2) = Balancers.aperture(minAperture = minAperture)
|
||||
|
||||
val eventBusPublisherBuilder =
|
||||
EventBusPublisherBuilder()
|
||||
.dest(eval("/s/eventbus/provisioning"))
|
||||
.clientId(settings.thriftClientId)
|
||||
// eventbus stats are further scoped by stream, so put all
|
||||
// publishers under the same stats namespace
|
||||
.statsReceiver(backendsScope.scope("event_bus"))
|
||||
// This makes the underlying kps-client to be resolved over WilyNs vs DNS
|
||||
.serviceIdentifier(settings.serviceIdentifier)
|
||||
|
||||
new BackendClients {
|
||||
def referencedNames: Seq[Name] = referencedNamesBuilder.result()
|
||||
|
||||
val memcacheClient: memcached.Client =
|
||||
Memcached.client
|
||||
.withMutualTls(settings.serviceIdentifier)
|
||||
.connectionsPerEndpoint(2)
|
||||
.configured(param.KeyHasher(KeyHasher.FNV1_32))
|
||||
.configured(Transporter.ConnectTimeout(100.milliseconds))
|
||||
.configured(TimeoutFilter.Param(200.milliseconds))
|
||||
.configured(TimeoutFactory.Param(200.milliseconds))
|
||||
.configured(param.EjectFailedHost(false))
|
||||
.configured(FailureAccrualFactory.Param(numFailures = 20, markDeadFor = 30.second))
|
||||
.configured(
|
||||
PendingRequestFilter.Param(limit = Some(settings.cacheClientPendingRequestLimit))
|
||||
)
|
||||
.filtered(new MemcacheExceptionLoggingFilter)
|
||||
.newRichClient(dest = eval(settings.twemcacheDest), label = "memcache")
|
||||
|
||||
/* clients */
|
||||
val tweetStorageClient: TweetStorageClient =
|
||||
Manhattan.fromClient(
|
||||
new ManhattanTweetStorageClient(
|
||||
settings.tweetStorageConfig,
|
||||
statsReceiver = backendsScope.scope("tweet_storage"),
|
||||
clientIdHelper = clientIdHelper,
|
||||
)
|
||||
)
|
||||
|
||||
val socialGraphService: SocialGraphService = {
|
||||
val finagleClient =
|
||||
new SocialGraphService$FinagleClient(
|
||||
thriftMuxClientBuilder(
|
||||
"socialgraph",
|
||||
"/s/socialgraph/socialgraph",
|
||||
classOf[SocialGraphScroogeIface.MethodPerEndpoint]
|
||||
).loadBalancer(Balancers.aperturePeakEwma(minAperture = 16))
|
||||
.build()
|
||||
)
|
||||
|
||||
settings.socialGraphSeviceConfig(
|
||||
SocialGraphService.fromClient(finagleClient),
|
||||
backendContext("socialgraph")
|
||||
)
|
||||
}
|
||||
|
||||
val tflockClient =
|
||||
new FlockDB.FinagledClient(
|
||||
thriftMuxClientBuilder("tflock", "/s/tflock/tflock", classOf[FlockDB.MethodPerEndpoint])
|
||||
.loadBalancer(balancer(minAperture = 5))
|
||||
.responseClassifier(FlockResponse.classifier)
|
||||
.build(),
|
||||
serviceName = "tflock",
|
||||
stats = statsReceiver
|
||||
)
|
||||
|
||||
val tflockReadClient: TFlockClient =
|
||||
settings.tflockReadConfig(tflockClient, backendContext("tflock"))
|
||||
|
||||
val tflockWriteClient: TFlockClient =
|
||||
settings.tflockWriteConfig(tflockClient, backendContext("tflock"))
|
||||
|
||||
val gizmoduck: Gizmoduck = {
|
||||
val clientBuilder =
|
||||
thriftMuxClientBuilder(
|
||||
"gizmoduck",
|
||||
"/s/gizmoduck/gizmoduck",
|
||||
classOf[UserService.MethodPerEndpoint])
|
||||
.loadBalancer(balancer(minAperture = 63))
|
||||
val mb = MethodBuilder
|
||||
.from(clientBuilder)
|
||||
.idempotent(maxExtraLoad = 1.percent)
|
||||
.servicePerEndpoint[UserService.ServicePerEndpoint]
|
||||
|
||||
val gizmoduckClient = ThriftMux.Client.methodPerEndpoint(mb)
|
||||
settings.gizmoduckConfig(Gizmoduck.fromClient(gizmoduckClient), backendContext("gizmoduck"))
|
||||
}
|
||||
|
||||
val merlin: UserRolesService.MethodPerEndpoint = {
|
||||
val thriftClient = thriftMuxMethodBuilder("merlin", "/s/merlin/merlin")
|
||||
.withTimeoutPerRequest(100.milliseconds)
|
||||
.withTimeoutTotal(400.milliseconds)
|
||||
.idempotent(0.01)
|
||||
.servicePerEndpoint[UserRolesService.ServicePerEndpoint]
|
||||
|
||||
ThriftMux.Client.methodPerEndpoint(thriftClient)
|
||||
}
|
||||
|
||||
val talon: Talon = {
|
||||
val talonClient =
|
||||
new Talon$FinagleClient(
|
||||
thriftMuxClientBuilder(
|
||||
"talon",
|
||||
"/s/talon/backend",
|
||||
classOf[TalonScroogeIface.MethodPerEndpoint])
|
||||
.build()
|
||||
)
|
||||
|
||||
settings.talonConfig(Talon.fromClient(talonClient), backendContext("talon"))
|
||||
}
|
||||
|
||||
val guano = Guano()
|
||||
|
||||
val mediaInfoService: MediaInfoService = {
|
||||
val finagleClient =
|
||||
new MediaInfoService$FinagleClient(
|
||||
thriftMuxClientBuilder(
|
||||
"mediainfo",
|
||||
"/s/photurkey/mediainfo",
|
||||
classOf[MediaInfoScroogeIface.MethodPerEndpoint])
|
||||
.loadBalancer(balancer(minAperture = 75))
|
||||
.build()
|
||||
)
|
||||
|
||||
settings.mediaInfoServiceConfig(
|
||||
MediaInfoService.fromClient(finagleClient),
|
||||
backendContext("mediainfo")
|
||||
)
|
||||
}
|
||||
|
||||
val userImageService: UserImageService = {
|
||||
val finagleClient =
|
||||
new UserImageService$FinagleClient(
|
||||
thriftMuxClientBuilder(
|
||||
"userImage",
|
||||
"/s/user-image-service/uis",
|
||||
classOf[UserImageScroogeIface.MethodPerEndpoint])
|
||||
.build()
|
||||
)
|
||||
|
||||
settings.userImageServiceConfig(
|
||||
UserImageService.fromClient(finagleClient),
|
||||
backendContext("userImage")
|
||||
)
|
||||
}
|
||||
|
||||
val mediaClient: MediaClient =
|
||||
MediaClient.fromBackends(
|
||||
userImageService = userImageService,
|
||||
mediaInfoService = mediaInfoService
|
||||
)
|
||||
|
||||
val timelineService: TimelineService = {
|
||||
val timelineServiceClient =
|
||||
new tls.TimelineService$FinagleClient(
|
||||
thriftMuxClientBuilder(
|
||||
"timelineService",
|
||||
"/s/timelineservice/timelineservice",
|
||||
classOf[tls.TimelineService.MethodPerEndpoint])
|
||||
.loadBalancer(balancer(minAperture = 13))
|
||||
.build()
|
||||
)
|
||||
|
||||
settings.timelineServiceConfig(
|
||||
TimelineService.fromClient(timelineServiceClient),
|
||||
backendContext("timelineService")
|
||||
)
|
||||
}
|
||||
|
||||
val expandodo: Expandodo = {
|
||||
val cardsServiceClient =
|
||||
new CardsService$FinagleClient(
|
||||
thriftMuxClientBuilder(
|
||||
"expandodo",
|
||||
"/s/expandodo/server",
|
||||
classOf[CardsScroogeIface.MethodPerEndpoint])
|
||||
.loadBalancer(balancer(minAperture = 6))
|
||||
.build()
|
||||
)
|
||||
|
||||
settings.expandodoConfig(
|
||||
Expandodo.fromClient(cardsServiceClient),
|
||||
backendContext("expandodo")
|
||||
)
|
||||
}
|
||||
|
||||
val creativesContainerService: CreativesContainerService = {
|
||||
val mb = thriftMuxMethodBuilder(
|
||||
"creativesContainerService",
|
||||
"/s/creatives-container/creatives-container",
|
||||
).withTimeoutTotal(300.milliseconds)
|
||||
.idempotent(maxExtraLoad = 1.percent)
|
||||
.servicePerEndpoint[ccs.CreativesContainerService.ServicePerEndpoint]
|
||||
|
||||
settings.creativesContainerServiceConfig(
|
||||
CreativesContainerService.fromClient(ccs.CreativesContainerService.MethodPerEndpoint(mb)),
|
||||
backendContext("creativesContainerService")
|
||||
)
|
||||
}
|
||||
|
||||
val scarecrow: Scarecrow = {
|
||||
val scarecrowClient = new ScarecrowService$FinagleClient(
|
||||
thriftMuxClientBuilder(
|
||||
"scarecrow",
|
||||
"/s/abuse/scarecrow",
|
||||
classOf[ScarecrowScroogeIface.MethodPerEndpoint])
|
||||
.loadBalancer(balancer(minAperture = 6))
|
||||
.build(),
|
||||
serviceName = "scarecrow",
|
||||
stats = statsReceiver
|
||||
)
|
||||
|
||||
settings.scarecrowConfig(Scarecrow.fromClient(scarecrowClient), backendContext("scarecrow"))
|
||||
}
|
||||
|
||||
val snowflakeClient: Snowflake.MethodPerEndpoint = {
|
||||
eval("/s/snowflake/snowflake") // eagerly resolve the serverset
|
||||
val mb = thriftMuxMethodBuilder(
|
||||
"snowflake",
|
||||
"/s/snowflake/snowflake"
|
||||
).withTimeoutTotal(300.milliseconds)
|
||||
.withTimeoutPerRequest(100.milliseconds)
|
||||
.idempotent(maxExtraLoad = 1.percent)
|
||||
|
||||
SnowflakeClient.snowflakeClient(mb)
|
||||
}
|
||||
|
||||
val deferredRpcClient =
|
||||
new DeferredRPC.FinagledClient(
|
||||
thriftMuxClientBuilder(
|
||||
"deferredrpc",
|
||||
"/s/kafka-shared/krpc-server-main",
|
||||
classOf[DeferredRPC.MethodPerEndpoint])
|
||||
.requestTimeout(200.milliseconds)
|
||||
.retryPolicy(retry(timeouts = 3))
|
||||
.build(),
|
||||
serviceName = "deferredrpc",
|
||||
stats = statsReceiver
|
||||
)
|
||||
|
||||
def deferredTweetypie(target: Target): ThriftTweetService = {
|
||||
// When deferring back to the local datacenter, preserve the finagle
|
||||
// context and dtabs. This will ensure that developer dtabs are honored
|
||||
// and that context is preserved in eventbus. (eventbus enqueues only
|
||||
// happen in async requests within the same datacenter.)
|
||||
//
|
||||
// Effectively, this means we consider deferredrpc requests within the
|
||||
// same datacenter to be part of the same request, but replicated
|
||||
// requests are not.
|
||||
val isLocal: Boolean = target.datacenter == Datacenter.Local
|
||||
|
||||
val deferredThriftService: Service[ThriftClientRequest, Array[Byte]] =
|
||||
new DeferredThriftService(
|
||||
deferredRpcClient,
|
||||
target,
|
||||
serializeFinagleContexts = isLocal,
|
||||
serializeFinagleDtabs = isLocal
|
||||
)
|
||||
|
||||
new TweetServiceInternal$FinagleClient(deferredThriftService)
|
||||
}
|
||||
|
||||
val replicationClient: ThriftTweetService =
|
||||
deferredTweetypie(Target(Datacenter.AllOthers, "tweetypie-replication"))
|
||||
|
||||
// used for read endpoints replication
|
||||
val lowQoSReplicationClients: Seq[GatedReplicationClient] = {
|
||||
val rampUpGate = Gate.linearRampUp(Time.now, settings.forkingRampUp)
|
||||
|
||||
// Gates to avoid sending replicated reads from a cluster to itself
|
||||
val inATLA = if (settings.zone == "atla") Gate.True else Gate.False
|
||||
val inPDXA = if (settings.zone == "pdxa") Gate.True else Gate.False
|
||||
|
||||
Seq(
|
||||
GatedReplicationClient(
|
||||
client = deferredTweetypie(Target(Datacenter.Atla, "tweetypie-lowqos")),
|
||||
gate = rampUpGate & deciderGates.replicateReadsToATLA & !inATLA
|
||||
),
|
||||
GatedReplicationClient(
|
||||
client = deferredTweetypie(Target(Datacenter.Pdxa, "tweetypie-lowqos")),
|
||||
gate = rampUpGate & deciderGates.replicateReadsToPDXA & !inPDXA
|
||||
)
|
||||
)
|
||||
}
|
||||
|
||||
// used for async operations in the write path
|
||||
val asyncTweetService: ThriftTweetService =
|
||||
deferredTweetypie(Target(Datacenter.Local, "tweetypie"))
|
||||
|
||||
// used to trigger asyncEraseUserTweetsRequest
|
||||
val asyncTweetDeletionService: ThriftTweetService =
|
||||
deferredTweetypie(Target(Datacenter.Local, "tweetypie-retweet-deletion"))
|
||||
|
||||
// used for async retries
|
||||
val asyncRetryTweetService: ThriftTweetService =
|
||||
deferredTweetypie(Target(Datacenter.Local, "tweetypie-async-retry"))
|
||||
|
||||
val darkTrafficClient: Service[Array[Byte], Array[Byte]] = {
|
||||
val thriftService =
|
||||
thriftMuxClient(
|
||||
"tweetypie.dark",
|
||||
propagateDeadlines = false
|
||||
).withRequestTimeout(100.milliseconds)
|
||||
.newService("/s/tweetypie/proxy")
|
||||
|
||||
val transformer =
|
||||
new Filter[Array[Byte], Array[Byte], ThriftClientRequest, Array[Byte]] {
|
||||
override def apply(
|
||||
request: Array[Byte],
|
||||
service: Service[ThriftClientRequest, Array[Byte]]
|
||||
): Future[Array[Byte]] =
|
||||
service(new ThriftClientRequest(request, false))
|
||||
}
|
||||
|
||||
transformer andThen thriftService
|
||||
}
|
||||
|
||||
val geoHydrationClient: GeoduckHydration.MethodPerEndpoint = {
|
||||
val mb = thriftMuxMethodBuilder("geoduck_hydration", "/s/geo/hydration")
|
||||
.withTimeoutPerRequest(100.millis)
|
||||
.idempotent(maxExtraLoad = 1.percent)
|
||||
ThriftMux.Client.methodPerEndpoint(
|
||||
mb.servicePerEndpoint[GeoduckHydration.ServicePerEndpoint])
|
||||
}
|
||||
|
||||
val geoHydrationLocate: GeoduckLocate = geoHydrationClient.locate
|
||||
|
||||
val geoReverseGeocoderClient: ReverseGeocoder.MethodPerEndpoint = {
|
||||
val mb = thriftMuxMethodBuilder("geoduck_reversegeocoder", "/s/geo/geoduck_reversegeocoder")
|
||||
.withTimeoutPerRequest(100.millis)
|
||||
.idempotent(maxExtraLoad = 1.percent)
|
||||
ThriftMux.Client.methodPerEndpoint(
|
||||
mb.servicePerEndpoint[ReverseGeocoder.ServicePerEndpoint])
|
||||
}
|
||||
|
||||
val geoduckGeohashLocate: GeoduckGeohashLocate = {
|
||||
new GeoduckGeohashLocate(
|
||||
reverseGeocoderClient = geoReverseGeocoderClient,
|
||||
hydrationClient = geoHydrationClient,
|
||||
classScopedStatsReceiver = statsReceiver.scope("geo_geohash_locate"))
|
||||
}
|
||||
|
||||
val geoRelevance =
|
||||
new Relevance$FinagleClient(
|
||||
thriftMuxClientBuilder(
|
||||
"geoduck_relevance",
|
||||
"/s/geo/relevance",
|
||||
classOf[Relevance.MethodPerEndpoint])
|
||||
.requestTimeout(100.milliseconds)
|
||||
.retryPolicy(retry(timeouts = 1))
|
||||
.build(),
|
||||
stats = statsReceiver
|
||||
)
|
||||
|
||||
val fanoutServiceClient =
|
||||
new FanoutService$FinagleClient(
|
||||
new DeferredThriftService(deferredRpcClient, Target(Datacenter.Local, "fanoutservice")),
|
||||
serviceName = "fanoutservice",
|
||||
stats = statsReceiver
|
||||
)
|
||||
|
||||
val limiterService: LimiterService = {
|
||||
val limiterClient =
|
||||
new LimiterClientFactory(
|
||||
name = "limiter",
|
||||
clientId = thriftClientId,
|
||||
tracer = tracer,
|
||||
statsReceiver = statsReceiver,
|
||||
serviceIdentifier = settings.serviceIdentifier,
|
||||
opportunisticTlsLevel = OpportunisticTls.Required,
|
||||
daemonize = true
|
||||
)(eval("/s/limiter/limiter"))
|
||||
|
||||
val limiterBackend = settings.limiterBackendConfig(
|
||||
LimiterBackend.fromClient(limiterClient),
|
||||
backendContext("limiter")
|
||||
)
|
||||
|
||||
LimiterService.fromBackend(
|
||||
limiterBackend.incrementFeature,
|
||||
limiterBackend.getFeatureUsage,
|
||||
getAppId,
|
||||
backendsScope.scope("limiter")
|
||||
)
|
||||
}
|
||||
|
||||
val passbirdClient =
|
||||
new PassbirdService$FinagleClient(
|
||||
thriftMuxClientBuilder(
|
||||
"passbird",
|
||||
"/s/passbird/passbird",
|
||||
classOf[PassbirdService.MethodPerEndpoint])
|
||||
.requestTimeout(100.milliseconds)
|
||||
.retryPolicy(retry(timeouts = 1))
|
||||
.build(),
|
||||
serviceName = "passbird",
|
||||
stats = statsReceiver
|
||||
)
|
||||
|
||||
val escherbird: Escherbird = {
|
||||
val escherbirdClient =
|
||||
new TweetEntityAnnotationService$FinagleClient(
|
||||
thriftMuxClientBuilder(
|
||||
"escherbird",
|
||||
"/s/escherbird/annotationservice",
|
||||
classOf[TweetEntityAnnotationScroogeIface.MethodPerEndpoint])
|
||||
.build()
|
||||
)
|
||||
settings.escherbirdConfig(
|
||||
Escherbird.fromClient(escherbirdClient),
|
||||
backendContext("escherbird")
|
||||
)
|
||||
}
|
||||
|
||||
val geoScrubEventStore: GeoScrubEventStore = {
|
||||
val mhMtlsParams =
|
||||
if (settings.serviceIdentifier == EmptyServiceIdentifier) NoMtlsParams
|
||||
else
|
||||
ManhattanKVClientMtlsParams(
|
||||
serviceIdentifier = settings.serviceIdentifier,
|
||||
opportunisticTls = OpportunisticTls.Required)
|
||||
|
||||
val mhClient =
|
||||
new ManhattanKVClient(
|
||||
appId = "geoduck_scrub_datastore",
|
||||
dest = "/s/manhattan/omega.native-thrift",
|
||||
mtlsParams = mhMtlsParams,
|
||||
label = "mh_omega",
|
||||
Seq(Experiments.ApertureLoadBalancer)
|
||||
)
|
||||
|
||||
GeoScrubEventStore(
|
||||
mhClient,
|
||||
settings.geoScrubEventStoreConfig,
|
||||
backendContext("geoScrubEventStore")
|
||||
)
|
||||
}
|
||||
|
||||
val tweetEventsPublisher: EventBusPublisher[TweetEvent] =
|
||||
eventBusPublisherBuilder
|
||||
.streamName("tweet_events")
|
||||
.thriftStruct(TweetEvent)
|
||||
.publishTimeout(500.milliseconds)
|
||||
.serializeFinagleDtabs(true)
|
||||
.build()
|
||||
|
||||
val deleteLocationDataPublisher: EventBusPublisher[DeleteLocationData] =
|
||||
eventBusPublisherBuilder
|
||||
.streamName("tweetypie_delete_location_data_prod")
|
||||
.thriftStruct(DeleteLocationData)
|
||||
// deleteLocationData is relatively rare, and publishing to
|
||||
// eventbus is all that the endpoint does. This means that it
|
||||
// is much more likely that we will have to make a connection,
|
||||
// which has much greater latency, and also makes us more
|
||||
// tolerant of slow requests, so we choose a long timeout.
|
||||
.publishTimeout(2.seconds)
|
||||
.build()
|
||||
|
||||
val retweetArchivalEventPublisher: EventBusPublisher[RetweetArchivalEvent] =
|
||||
eventBusPublisherBuilder
|
||||
.streamName("retweet_archival_events")
|
||||
.thriftStruct(RetweetArchivalEvent)
|
||||
.publishTimeout(500.milliseconds)
|
||||
.build()
|
||||
|
||||
val gnipEnricherator: GnipEnricherator = {
|
||||
val gnipEnricherator =
|
||||
thriftMuxMethodBuilder(
|
||||
"enricherator",
|
||||
"/s/datadelivery-enrichments/enricherator"
|
||||
)
|
||||
GnipEnricherator.fromMethod(gnipEnricherator)
|
||||
}
|
||||
|
||||
val stratoserverClient: StratoClient = Strato.client
|
||||
.withMutualTls(
|
||||
serviceIdentifier = settings.serviceIdentifier,
|
||||
opportunisticLevel = OpportunisticTls.Required)
|
||||
.withLabel("stratoserver")
|
||||
.withRequestTimeout(100.milliseconds)
|
||||
.build()
|
||||
|
||||
val configBus: ConfigBus =
|
||||
ConfigBus(backendsScope.scope("config_bus"), settings.instanceId, settings.instanceCount)
|
||||
|
||||
val callbackPromotedContentLogger: CallbackPromotedContentLogger = {
|
||||
val publisher =
|
||||
eventBusPublisherBuilder
|
||||
.streamName(settings.adsLoggingClientTopicName)
|
||||
.thriftStruct(AdCallbackEvent)
|
||||
.publishTimeout(500.milliseconds)
|
||||
.serializeFinagleDtabs(true)
|
||||
.maxQueuedEvents(1000)
|
||||
.kafkaDest("/s/kafka/ads-callback:kafka-tls")
|
||||
.build()
|
||||
|
||||
val stats = backendsScope.scope("promoted_content")
|
||||
val adsLoggingClient = AdsLoggingClient(publisher, stats, "Tweetypie")
|
||||
new CallbackPromotedContentLogger(adsLoggingClient, stats)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,281 @@
|
||||
package com.twitter.tweetypie
|
||||
package config
|
||||
|
||||
import com.twitter.conversions.DurationOps._
|
||||
import com.twitter.finagle.Backoff
|
||||
import com.twitter.finagle.memcached
|
||||
import com.twitter.finagle.stats.Stat
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.servo.cache.{Serializer => CacheSerializer, _}
|
||||
import com.twitter.tweetypie.client_id.ClientIdHelper
|
||||
import com.twitter.tweetypie.core._
|
||||
import com.twitter.tweetypie.handler.CacheBasedTweetCreationLock
|
||||
import com.twitter.tweetypie.repository._
|
||||
import com.twitter.tweetypie.serverutil._
|
||||
import com.twitter.tweetypie.thriftscala._
|
||||
import com.twitter.tweetypie.util._
|
||||
import com.twitter.util.Timer
|
||||
|
||||
/**
|
||||
* Provides configured caches (most backed by memcached) wrapped with appropriate metrics and locks.
|
||||
*
|
||||
* All memcached-backed caches share:
|
||||
* - one Finagle memcached client from backends.memcacheClient
|
||||
* - one in memory caffeine cache
|
||||
* - one Twemcache pool
|
||||
*
|
||||
* Each memcached-backed cache specialization provides its own:
|
||||
* - key prefix or "namespace"
|
||||
* - value serializer/deserializer
|
||||
* - stats scope
|
||||
* - log name
|
||||
*/
|
||||
trait Caches {
|
||||
val memcachedClientWithInProcessCaching: memcached.Client
|
||||
val tweetCache: LockingCache[TweetKey, Cached[CachedTweet]]
|
||||
val tweetResultCache: LockingCache[TweetId, Cached[TweetResult]]
|
||||
val tweetDataCache: LockingCache[TweetId, Cached[TweetData]]
|
||||
val tweetCreateLockerCache: Cache[TweetCreationLock.Key, TweetCreationLock.State]
|
||||
val tweetCountsCache: LockingCache[TweetCountKey, Cached[Count]]
|
||||
val deviceSourceInProcessCache: LockingCache[String, Cached[DeviceSource]]
|
||||
val geoScrubCache: LockingCache[UserId, Cached[Time]]
|
||||
}
|
||||
|
||||
object Caches {
|
||||
object NoCache extends Caches {
|
||||
override val memcachedClientWithInProcessCaching: memcached.Client = new NullMemcacheClient()
|
||||
private val toLockingCache: LockingCacheFactory = NonLockingCacheFactory
|
||||
val tweetCache: LockingCache[TweetKey, Cached[CachedTweet]] =
|
||||
toLockingCache(new NullCache)
|
||||
val tweetResultCache: LockingCache[TweetId, Cached[TweetResult]] =
|
||||
toLockingCache(new NullCache)
|
||||
val tweetDataCache: LockingCache[TweetId, Cached[TweetData]] =
|
||||
toLockingCache(new NullCache)
|
||||
val tweetCreateLockerCache: Cache[TweetCreationLock.Key, TweetCreationLock.State] =
|
||||
new NullCache
|
||||
val tweetCountsCache: LockingCache[TweetCountKey, Cached[Count]] =
|
||||
toLockingCache(new NullCache)
|
||||
val deviceSourceInProcessCache: LockingCache[String, Cached[DeviceSource]] =
|
||||
toLockingCache(new NullCache)
|
||||
val geoScrubCache: LockingCache[UserId, Cached[Time]] =
|
||||
toLockingCache(new NullCache)
|
||||
}
|
||||
|
||||
def apply(
|
||||
settings: TweetServiceSettings,
|
||||
stats: StatsReceiver,
|
||||
timer: Timer,
|
||||
clients: BackendClients,
|
||||
tweetKeyFactory: TweetKeyFactory,
|
||||
deciderGates: TweetypieDeciderGates,
|
||||
clientIdHelper: ClientIdHelper,
|
||||
): Caches = {
|
||||
val cachesStats = stats.scope("caches")
|
||||
val cachesInprocessStats = cachesStats.scope("inprocess")
|
||||
val cachesMemcacheStats = cachesStats.scope("memcache")
|
||||
val cachesMemcacheObserver = new StatsReceiverCacheObserver(cachesStats, 10000, "memcache")
|
||||
val cachesMemcacheTweetStats = cachesMemcacheStats.scope("tweet")
|
||||
val cachesInprocessDeviceSourceStats = cachesInprocessStats.scope("device_source")
|
||||
val cachesMemcacheCountStats = cachesMemcacheStats.scope("count")
|
||||
val cachesMemcacheTweetCreateStats = cachesMemcacheStats.scope("tweet_create")
|
||||
val cachesMemcacheGeoScrubStats = cachesMemcacheStats.scope("geo_scrub")
|
||||
val memcacheClient = clients.memcacheClient
|
||||
|
||||
val caffieneMemcachedClient = settings.inProcessCacheConfigOpt match {
|
||||
case Some(inProcessCacheConfig) =>
|
||||
new CaffeineMemcacheClient(
|
||||
proxyClient = memcacheClient,
|
||||
inProcessCacheConfig.maximumSize,
|
||||
inProcessCacheConfig.ttl,
|
||||
cachesMemcacheStats.scope("caffeine")
|
||||
)
|
||||
case None =>
|
||||
memcacheClient
|
||||
}
|
||||
|
||||
val observedMemcacheWithCaffeineClient =
|
||||
new ObservableMemcache(
|
||||
new FinagleMemcache(
|
||||
caffieneMemcachedClient
|
||||
),
|
||||
cachesMemcacheObserver
|
||||
)
|
||||
|
||||
def observeCache[K, V](
|
||||
cache: Cache[K, V],
|
||||
stats: StatsReceiver,
|
||||
logName: String,
|
||||
windowSize: Int = 10000
|
||||
) =
|
||||
ObservableCache(
|
||||
cache,
|
||||
stats,
|
||||
windowSize,
|
||||
// Need to use an old-school c.t.logging.Logger because that's what servo needs
|
||||
com.twitter.logging.Logger(s"com.twitter.tweetypie.cache.$logName")
|
||||
)
|
||||
|
||||
def mkCache[K, V](
|
||||
ttl: Duration,
|
||||
serializer: CacheSerializer[V],
|
||||
perCacheStats: StatsReceiver,
|
||||
logName: String,
|
||||
windowSize: Int = 10000
|
||||
): Cache[K, V] = {
|
||||
observeCache(
|
||||
new MemcacheCache[K, V](
|
||||
observedMemcacheWithCaffeineClient,
|
||||
ttl,
|
||||
serializer
|
||||
),
|
||||
perCacheStats,
|
||||
logName,
|
||||
windowSize
|
||||
)
|
||||
}
|
||||
|
||||
def toLockingCache[K, V](
|
||||
cache: Cache[K, V],
|
||||
stats: StatsReceiver,
|
||||
backoffs: Stream[Duration] = settings.lockingCacheBackoffs
|
||||
): LockingCache[K, V] =
|
||||
new OptimisticLockingCache(
|
||||
underlyingCache = cache,
|
||||
backoffs = Backoff.fromStream(backoffs),
|
||||
observer = new OptimisticLockingCacheObserver(stats),
|
||||
timer = timer
|
||||
)
|
||||
|
||||
def mkLockingCache[K, V](
|
||||
ttl: Duration,
|
||||
serializer: CacheSerializer[V],
|
||||
stats: StatsReceiver,
|
||||
logName: String,
|
||||
windowSize: Int = 10000,
|
||||
backoffs: Stream[Duration] = settings.lockingCacheBackoffs
|
||||
): LockingCache[K, V] =
|
||||
toLockingCache(
|
||||
mkCache(ttl, serializer, stats, logName, windowSize),
|
||||
stats,
|
||||
backoffs
|
||||
)
|
||||
|
||||
def trackTimeInCache[K, V](
|
||||
cache: Cache[K, Cached[V]],
|
||||
stats: StatsReceiver
|
||||
): Cache[K, Cached[V]] =
|
||||
new CacheWrapper[K, Cached[V]] {
|
||||
val ageStat: Stat = stats.stat("time_in_cache_ms")
|
||||
val underlyingCache: Cache[K, Cached[V]] = cache
|
||||
|
||||
override def get(keys: Seq[K]): Future[KeyValueResult[K, Cached[V]]] =
|
||||
underlyingCache.get(keys).onSuccess(record)
|
||||
|
||||
private def record(res: KeyValueResult[K, Cached[V]]): Unit = {
|
||||
val now = Time.now
|
||||
for (c <- res.found.values) {
|
||||
ageStat.add(c.cachedAt.until(now).inMilliseconds)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
new Caches {
|
||||
override val memcachedClientWithInProcessCaching: memcached.Client = caffieneMemcachedClient
|
||||
|
||||
private val observingTweetCache: Cache[TweetKey, Cached[CachedTweet]] =
|
||||
trackTimeInCache(
|
||||
mkCache(
|
||||
ttl = settings.tweetMemcacheTtl,
|
||||
serializer = Serializer.CachedTweet.CachedCompact,
|
||||
perCacheStats = cachesMemcacheTweetStats,
|
||||
logName = "MemcacheTweetCache"
|
||||
),
|
||||
cachesMemcacheTweetStats
|
||||
)
|
||||
|
||||
// Wrap the tweet cache with a wrapper that will scribe the cache writes
|
||||
// that happen to a fraction of tweets. This was added as part of the
|
||||
// investigation into missing place ids and cache inconsistencies that
|
||||
// were discovered by the additional fields hydrator.
|
||||
private[this] val writeLoggingTweetCache =
|
||||
new ScribeTweetCacheWrites(
|
||||
underlyingCache = observingTweetCache,
|
||||
logYoungTweetCacheWrites = deciderGates.logYoungTweetCacheWrites,
|
||||
logTweetCacheWrites = deciderGates.logTweetCacheWrites
|
||||
)
|
||||
|
||||
val tweetCache: LockingCache[TweetKey, Cached[CachedTweet]] =
|
||||
toLockingCache(
|
||||
cache = writeLoggingTweetCache,
|
||||
stats = cachesMemcacheTweetStats
|
||||
)
|
||||
|
||||
val tweetDataCache: LockingCache[TweetId, Cached[TweetData]] =
|
||||
toLockingCache(
|
||||
cache = TweetDataCache(tweetCache, tweetKeyFactory.fromId),
|
||||
stats = cachesMemcacheTweetStats
|
||||
)
|
||||
|
||||
val tweetResultCache: LockingCache[TweetId, Cached[TweetResult]] =
|
||||
toLockingCache(
|
||||
cache = TweetResultCache(tweetDataCache),
|
||||
stats = cachesMemcacheTweetStats
|
||||
)
|
||||
|
||||
val tweetCountsCache: LockingCache[TweetCountKey, Cached[Count]] =
|
||||
mkLockingCache(
|
||||
ttl = settings.tweetCountsMemcacheTtl,
|
||||
serializer = Serializers.CachedLong.Compact,
|
||||
stats = cachesMemcacheCountStats,
|
||||
logName = "MemcacheTweetCountCache",
|
||||
windowSize = 1000,
|
||||
backoffs = Backoff.linear(0.millis, 2.millis).take(2).toStream
|
||||
)
|
||||
|
||||
val tweetCreateLockerCache: Cache[TweetCreationLock.Key, TweetCreationLock.State] =
|
||||
observeCache(
|
||||
new TtlCacheToCache(
|
||||
underlyingCache = new KeyValueTransformingTtlCache(
|
||||
underlyingCache = observedMemcacheWithCaffeineClient,
|
||||
transformer = TweetCreationLock.State.Serializer,
|
||||
underlyingKey = (_: TweetCreationLock.Key).toString
|
||||
),
|
||||
ttl = CacheBasedTweetCreationLock.ttlChooser(
|
||||
shortTtl = settings.tweetCreateLockingMemcacheTtl,
|
||||
longTtl = settings.tweetCreateLockingMemcacheLongTtl
|
||||
)
|
||||
),
|
||||
stats = cachesMemcacheTweetCreateStats,
|
||||
logName = "MemcacheTweetCreateLockingCache",
|
||||
windowSize = 1000
|
||||
)
|
||||
|
||||
val deviceSourceInProcessCache: LockingCache[String, Cached[DeviceSource]] =
|
||||
toLockingCache(
|
||||
observeCache(
|
||||
new ExpiringLruCache(
|
||||
ttl = settings.deviceSourceInProcessTtl,
|
||||
maximumSize = settings.deviceSourceInProcessCacheMaxSize
|
||||
),
|
||||
stats = cachesInprocessDeviceSourceStats,
|
||||
logName = "InprocessDeviceSourceCache"
|
||||
),
|
||||
stats = cachesInprocessDeviceSourceStats
|
||||
)
|
||||
|
||||
val geoScrubCache: LockingCache[UserId, Cached[Time]] =
|
||||
toLockingCache[UserId, Cached[Time]](
|
||||
new KeyTransformingCache(
|
||||
mkCache[GeoScrubTimestampKey, Cached[Time]](
|
||||
ttl = settings.geoScrubMemcacheTtl,
|
||||
serializer = Serializer.toCached(CacheSerializer.Time),
|
||||
perCacheStats = cachesMemcacheGeoScrubStats,
|
||||
logName = "MemcacheGeoScrubCache"
|
||||
),
|
||||
(userId: UserId) => GeoScrubTimestampKey(userId)
|
||||
),
|
||||
cachesMemcacheGeoScrubStats
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,126 @@
|
||||
package com.twitter.tweetypie.config
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper
|
||||
import com.fasterxml.jackson.dataformat.yaml.YAMLFactory
|
||||
import com.fasterxml.jackson.module.scala.DefaultScalaModule
|
||||
import com.twitter.finagle.mtls.authentication.ServiceIdentifier
|
||||
import com.twitter.util.Try
|
||||
|
||||
case object EmptyConfigException extends Exception
|
||||
|
||||
case class ServiceIdentifierPattern(
|
||||
role: Option[String],
|
||||
service: Option[String],
|
||||
environment: Option[String],
|
||||
) {
|
||||
// Service identifier matches if the fields of service identifier
|
||||
// match all the defined fields of pattern.
|
||||
def matches(id: ServiceIdentifier): Boolean =
|
||||
Seq(
|
||||
role.map(_ == id.role),
|
||||
service.map(_ == id.service),
|
||||
environment.map(_ == id.environment),
|
||||
)
|
||||
.flatten
|
||||
.forall(identity)
|
||||
|
||||
// True if this is the kind of pattern that only specifies environment.
|
||||
// This should be used in rare cases, for example letting all devel clients
|
||||
// use permitted methods - like get_tweet_fields.
|
||||
def onlyEnv: Boolean =
|
||||
role.isEmpty && service.isEmpty && environment.isDefined
|
||||
}
|
||||
|
||||
case class Client(
|
||||
clientId: String,
|
||||
serviceIdentifiers: Seq[ServiceIdentifierPattern],
|
||||
tpsLimit: Option[Int],
|
||||
environments: Seq[String],
|
||||
loadShedEnvs: Seq[String],
|
||||
permittedMethods: Set[String],
|
||||
accessAllMethods: Boolean,
|
||||
bypassVisibilityFiltering: Boolean,
|
||||
enforceRateLimit: Boolean) {
|
||||
|
||||
// Client matches a service identifier if any of its patterns
|
||||
// match.
|
||||
def matches(id: ServiceIdentifier): Boolean =
|
||||
serviceIdentifiers.exists(_.matches(id))
|
||||
}
|
||||
|
||||
object ClientsParser {
|
||||
|
||||
// Case classes for parsing yaml - should match the structure of clients.yml
|
||||
private case class YamlServiceIdentifier(
|
||||
role: Option[String],
|
||||
service: Option[String],
|
||||
environment: Option[String],
|
||||
)
|
||||
private case class YamlClient(
|
||||
client_id: String,
|
||||
service_identifiers: Option[Seq[YamlServiceIdentifier]],
|
||||
service_name: String,
|
||||
tps_quota: String,
|
||||
contact_email: String,
|
||||
environments: Seq[String],
|
||||
load_shed_envs: Option[
|
||||
Seq[String]
|
||||
], // list of environments we can rejects requests from if load shedding
|
||||
comment: Option[String],
|
||||
permitted_methods: Option[Seq[String]],
|
||||
access_all_methods: Boolean,
|
||||
bypass_visibility_filtering: Boolean,
|
||||
bypass_visibility_filtering_reason: Option[String],
|
||||
rate_limit: Boolean) {
|
||||
def toClient: Client = {
|
||||
|
||||
// we provision tps_quota for both DCs during white-listing, to account for full fail-over.
|
||||
val tpsLimit: Option[Int] = Try(tps_quota.replaceAll("[^0-9]", "").toInt * 1000).toOption
|
||||
|
||||
Client(
|
||||
clientId = client_id,
|
||||
serviceIdentifiers = service_identifiers.getOrElse(Nil).flatMap { id =>
|
||||
if (id.role.isDefined || id.service.isDefined || id.environment.isDefined) {
|
||||
Seq(ServiceIdentifierPattern(
|
||||
role = id.role,
|
||||
service = id.service,
|
||||
environment = id.environment,
|
||||
))
|
||||
} else {
|
||||
Seq()
|
||||
}
|
||||
},
|
||||
tpsLimit = tpsLimit,
|
||||
environments = environments,
|
||||
loadShedEnvs = load_shed_envs.getOrElse(Nil),
|
||||
permittedMethods = permitted_methods.getOrElse(Nil).toSet,
|
||||
accessAllMethods = access_all_methods,
|
||||
bypassVisibilityFiltering = bypass_visibility_filtering,
|
||||
enforceRateLimit = rate_limit
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
private val mapper: ObjectMapper = new ObjectMapper(new YAMLFactory())
|
||||
mapper.registerModule(DefaultScalaModule)
|
||||
|
||||
private val yamlClientTypeFactory =
|
||||
mapper
|
||||
.getTypeFactory()
|
||||
.constructCollectionLikeType(
|
||||
classOf[Seq[YamlClient]],
|
||||
classOf[YamlClient]
|
||||
)
|
||||
|
||||
def apply(yamlString: String): Seq[Client] = {
|
||||
val parsed =
|
||||
mapper
|
||||
.readValue[Seq[YamlClient]](yamlString, yamlClientTypeFactory)
|
||||
.map(_.toClient)
|
||||
|
||||
if (parsed.isEmpty)
|
||||
throw EmptyConfigException
|
||||
else
|
||||
parsed
|
||||
}
|
||||
}
|
@ -0,0 +1,100 @@
|
||||
package com.twitter.tweetypie.config
|
||||
|
||||
import com.twitter.finagle.mtls.authentication.ServiceIdentifier
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.tweetypie.Gate
|
||||
import com.twitter.tweetypie.backends.ConfigBus
|
||||
import com.twitter.tweetypie.client_id.ClientIdHelper
|
||||
import com.twitter.util.Activity
|
||||
|
||||
case class DynamicConfig(
|
||||
// A map of fully-qualified client ID (including the environment suffix, e.g. tweetypie.prod) to Client case class
|
||||
clientsByFullyQualifiedId: Option[Map[String, Client]],
|
||||
// Clients by service identifier parts.
|
||||
clientsByRole: Option[Map[String, Seq[Client]]] = None,
|
||||
clientsByService: Option[Map[String, Seq[Client]]] = None,
|
||||
onlyEnvClients: Option[Seq[Client]] = None,
|
||||
// These endpoints do not need permissions to be accessed
|
||||
unprotectedEndpoints: Set[String] = Set("get_tweet_counts", "get_tweet_fields", "get_tweets")) {
|
||||
|
||||
/**
|
||||
* Function that takes a fully qualified client id and says whether it is included in the allowList
|
||||
*/
|
||||
val isAllowListedClient: String => Boolean =
|
||||
clientsByFullyQualifiedId.map(clients => clients.contains _).getOrElse(_ => true)
|
||||
|
||||
def byServiceIdentifier(serviceIdentifier: ServiceIdentifier): Set[Client] =
|
||||
Iterable.concat(
|
||||
get(clientsByRole, serviceIdentifier.role),
|
||||
get(clientsByService, serviceIdentifier.service),
|
||||
onlyEnvClients.getOrElse(Seq()),
|
||||
)
|
||||
.filter(_.matches(serviceIdentifier))
|
||||
.toSet
|
||||
|
||||
private def get(clientsByKey: Option[Map[String, Seq[Client]]], key: String): Seq[Client] =
|
||||
clientsByKey match {
|
||||
case Some(map) => map.getOrElse(key, Seq())
|
||||
case None => Seq()
|
||||
}
|
||||
|
||||
/**
|
||||
* Take a fully qualified client id and says if the client has offered to shed reads if tweetypie
|
||||
* is in an emergency
|
||||
*/
|
||||
val loadShedEligible: Gate[String] = Gate { (clientId: String) =>
|
||||
val env = ClientIdHelper.getClientIdEnv(clientId)
|
||||
clientsByFullyQualifiedId.flatMap(clients => clients.get(clientId)).exists { c =>
|
||||
c.loadShedEnvs.contains(env)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* DynamicConfig uses ConfigBus to update Tweetypie with configuration changes
|
||||
* dynamically. Every time the config changes, the Activity[DynamicConfig] is
|
||||
* updated, and anything relying on that config will be reinitialized.
|
||||
*/
|
||||
object DynamicConfig {
|
||||
def fullyQualifiedClientIds(client: Client): Seq[String] = {
|
||||
val clientId = client.clientId
|
||||
client.environments match {
|
||||
case Nil => Seq(clientId)
|
||||
case envs => envs.map(env => s"$clientId.$env")
|
||||
}
|
||||
}
|
||||
|
||||
// Make a Map of fully qualified client id to Client
|
||||
def byClientId(clients: Seq[Client]): Map[String, Client] =
|
||||
clients.flatMap { client =>
|
||||
fullyQualifiedClientIds(client).map { fullClientId => fullClientId -> client }
|
||||
}.toMap
|
||||
|
||||
def by(get: ServiceIdentifierPattern => Option[String])(clients: Seq[Client]): Map[String, Seq[Client]] =
|
||||
clients.flatMap { c =>
|
||||
c.serviceIdentifiers.collect {
|
||||
case s if get(s).isDefined => (get(s).get, c)
|
||||
}
|
||||
}.groupBy(_._1).mapValues(_.map(_._2))
|
||||
|
||||
private[this] val clientsPath = "config/clients.yml"
|
||||
|
||||
def apply(
|
||||
stats: StatsReceiver,
|
||||
configBus: ConfigBus,
|
||||
settings: TweetServiceSettings
|
||||
): Activity[DynamicConfig] =
|
||||
DynamicConfigLoader(configBus.file)
|
||||
.apply(clientsPath, stats.scope("client_allowlist"), ClientsParser.apply)
|
||||
.map(fromClients)
|
||||
|
||||
def fromClients(clients: Option[Seq[Client]]): DynamicConfig =
|
||||
DynamicConfig(
|
||||
clientsByFullyQualifiedId = clients.map(byClientId),
|
||||
clientsByRole = clients.map(by(_.role)),
|
||||
clientsByService = clients.map(by(_.service)),
|
||||
onlyEnvClients = clients.map(_.filter { client =>
|
||||
client.serviceIdentifiers.exists(_.onlyEnv)
|
||||
}),
|
||||
)
|
||||
}
|
@ -0,0 +1,69 @@
|
||||
package com.twitter.tweetypie.config
|
||||
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.servo.util.ExceptionCounter
|
||||
import com.twitter.tweetypie.serverutil.ActivityUtil
|
||||
import com.twitter.util.{Activity, Return, Try}
|
||||
import com.twitter.util.logging.Logger
|
||||
|
||||
trait DynamicConfigLoader {
|
||||
def apply[T](path: String, stats: StatsReceiver, parse: String => T): Activity[Option[T]]
|
||||
}
|
||||
|
||||
object DynamicConfigLoader {
|
||||
|
||||
def apply(read: String => Activity[String]): DynamicConfigLoader =
|
||||
new DynamicConfigLoader {
|
||||
val logger = Logger(getClass)
|
||||
|
||||
private def snoopState[T](stats: StatsReceiver)(a: Activity[T]): Activity[T] = {
|
||||
val pending = stats.counter("pending")
|
||||
val failure = stats.counter("failure")
|
||||
val success = stats.counter("success")
|
||||
|
||||
a.mapState {
|
||||
case s @ Activity.Ok(_) =>
|
||||
success.incr()
|
||||
s
|
||||
case Activity.Pending =>
|
||||
pending.incr()
|
||||
Activity.Pending
|
||||
case s @ Activity.Failed(_) =>
|
||||
failure.incr()
|
||||
s
|
||||
}
|
||||
}
|
||||
|
||||
def apply[T](path: String, stats: StatsReceiver, parse: String => T): Activity[Option[T]] = {
|
||||
val exceptionCounter = new ExceptionCounter(stats)
|
||||
|
||||
val rawActivity: Activity[T] =
|
||||
snoopState(stats.scope("raw"))(
|
||||
ActivityUtil
|
||||
.strict(read(path))
|
||||
.map(parse)
|
||||
.handle {
|
||||
case e =>
|
||||
exceptionCounter(e)
|
||||
logger.error(s"Invalid config in $path", e)
|
||||
throw e
|
||||
}
|
||||
)
|
||||
|
||||
val stableActivity =
|
||||
snoopState(stats.scope("stabilized"))(rawActivity.stabilize).mapState[Option[T]] {
|
||||
case Activity.Ok(t) => Activity.Ok(Some(t))
|
||||
case _ => Activity.Ok(None)
|
||||
}
|
||||
|
||||
stats.provideGauge("config_state") {
|
||||
Try(stableActivity.sample()) match {
|
||||
case Return(Some(c)) => c.hashCode.abs
|
||||
case _ => 0
|
||||
}
|
||||
}
|
||||
|
||||
stableActivity
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,182 @@
|
||||
package com.twitter.tweetypie
|
||||
package config
|
||||
|
||||
import com.twitter.flockdb.client.StatusGraph
|
||||
import com.twitter.servo.util.FutureArrow
|
||||
import com.twitter.stitch.timelineservice.TimelineService.GetPerspectives
|
||||
import com.twitter.tweetypie.client_id.ClientIdHelper
|
||||
import com.twitter.tweetypie.repository.DeviceSourceRepository.Type
|
||||
import com.twitter.tweetypie.repository._
|
||||
import com.twitter.tweetypie.serverutil._
|
||||
import com.twitter.visibility.common.tflock.UserIsInvitedToConversationRepository
|
||||
|
||||
/**
|
||||
* Tweetypie's read path composes results from many data sources. This
|
||||
* trait is a collection of repositories for external data access.
|
||||
* These repositories should not have (within-Tweetypie) caches,
|
||||
* deciders, etc. applied to them, since that is done when the
|
||||
* repositories are composed together. They should be the minimal
|
||||
* wrapping of the external clients in order to expose an Arrow-based
|
||||
* interface.
|
||||
*/
|
||||
trait ExternalRepositories {
|
||||
def card2Repo: Card2Repository.Type
|
||||
def cardRepo: CardRepository.Type
|
||||
def cardUsersRepo: CardUsersRepository.Type
|
||||
def conversationIdRepo: ConversationIdRepository.Type
|
||||
def containerAsTweetRepo: CreativesContainerMaterializationRepository.GetTweetType
|
||||
def containerAsTweetFieldsRepo: CreativesContainerMaterializationRepository.GetTweetFieldsType
|
||||
def deviceSourceRepo: DeviceSourceRepository.Type
|
||||
def escherbirdAnnotationRepo: EscherbirdAnnotationRepository.Type
|
||||
def stratoSafetyLabelsRepo: StratoSafetyLabelsRepository.Type
|
||||
def stratoCommunityMembershipRepo: StratoCommunityMembershipRepository.Type
|
||||
def stratoCommunityAccessRepo: StratoCommunityAccessRepository.Type
|
||||
def stratoPromotedTweetRepo: StratoPromotedTweetRepository.Type
|
||||
def stratoSuperFollowEligibleRepo: StratoSuperFollowEligibleRepository.Type
|
||||
def stratoSuperFollowRelationsRepo: StratoSuperFollowRelationsRepository.Type
|
||||
def stratoSubscriptionVerificationRepo: StratoSubscriptionVerificationRepository.Type
|
||||
def unmentionedEntitiesRepo: UnmentionedEntitiesRepository.Type
|
||||
def geoScrubTimestampRepo: GeoScrubTimestampRepository.Type
|
||||
def mediaMetadataRepo: MediaMetadataRepository.Type
|
||||
def perspectiveRepo: PerspectiveRepository.Type
|
||||
def placeRepo: PlaceRepository.Type
|
||||
def profileGeoRepo: ProfileGeoRepository.Type
|
||||
def quoterHasAlreadyQuotedRepo: QuoterHasAlreadyQuotedRepository.Type
|
||||
def lastQuoteOfQuoterRepo: LastQuoteOfQuoterRepository.Type
|
||||
def relationshipRepo: RelationshipRepository.Type
|
||||
def retweetSpamCheckRepo: RetweetSpamCheckRepository.Type
|
||||
def tweetCountsRepo: TweetCountsRepository.Type
|
||||
def tweetResultRepo: TweetResultRepository.Type
|
||||
def tweetSpamCheckRepo: TweetSpamCheckRepository.Type
|
||||
def urlRepo: UrlRepository.Type
|
||||
def userIsInvitedToConversationRepo: UserIsInvitedToConversationRepository.Type
|
||||
def userRepo: UserRepository.Type
|
||||
}
|
||||
|
||||
class ExternalServiceRepositories(
|
||||
clients: BackendClients,
|
||||
statsReceiver: StatsReceiver,
|
||||
settings: TweetServiceSettings,
|
||||
clientIdHelper: ClientIdHelper)
|
||||
extends ExternalRepositories {
|
||||
|
||||
lazy val card2Repo: Card2Repository.Type =
|
||||
Card2Repository(clients.expandodo.getCards2, maxRequestSize = 5)
|
||||
|
||||
lazy val cardRepo: CardRepository.Type =
|
||||
CardRepository(clients.expandodo.getCards, maxRequestSize = 5)
|
||||
|
||||
lazy val cardUsersRepo: CardUsersRepository.Type =
|
||||
CardUsersRepository(clients.expandodo.getCardUsers)
|
||||
|
||||
lazy val conversationIdRepo: ConversationIdRepository.Type =
|
||||
ConversationIdRepository(clients.tflockReadClient.multiSelectOne)
|
||||
|
||||
lazy val containerAsTweetRepo: CreativesContainerMaterializationRepository.GetTweetType =
|
||||
CreativesContainerMaterializationRepository(
|
||||
clients.creativesContainerService.materializeAsTweet)
|
||||
|
||||
lazy val containerAsTweetFieldsRepo: CreativesContainerMaterializationRepository.GetTweetFieldsType =
|
||||
CreativesContainerMaterializationRepository.materializeAsTweetFields(
|
||||
clients.creativesContainerService.materializeAsTweetFields)
|
||||
|
||||
lazy val deviceSourceRepo: Type = {
|
||||
DeviceSourceRepository(
|
||||
DeviceSourceParser.parseAppId,
|
||||
FutureArrow(clients.passbirdClient.getClientApplications(_))
|
||||
)
|
||||
}
|
||||
|
||||
lazy val escherbirdAnnotationRepo: EscherbirdAnnotationRepository.Type =
|
||||
EscherbirdAnnotationRepository(clients.escherbird.annotate)
|
||||
|
||||
lazy val quoterHasAlreadyQuotedRepo: QuoterHasAlreadyQuotedRepository.Type =
|
||||
QuoterHasAlreadyQuotedRepository(clients.tflockReadClient)
|
||||
|
||||
lazy val lastQuoteOfQuoterRepo: LastQuoteOfQuoterRepository.Type =
|
||||
LastQuoteOfQuoterRepository(clients.tflockReadClient)
|
||||
|
||||
lazy val stratoSafetyLabelsRepo: StratoSafetyLabelsRepository.Type =
|
||||
StratoSafetyLabelsRepository(clients.stratoserverClient)
|
||||
|
||||
lazy val stratoCommunityMembershipRepo: StratoCommunityMembershipRepository.Type =
|
||||
StratoCommunityMembershipRepository(clients.stratoserverClient)
|
||||
|
||||
lazy val stratoCommunityAccessRepo: StratoCommunityAccessRepository.Type =
|
||||
StratoCommunityAccessRepository(clients.stratoserverClient)
|
||||
|
||||
lazy val stratoSuperFollowEligibleRepo: StratoSuperFollowEligibleRepository.Type =
|
||||
StratoSuperFollowEligibleRepository(clients.stratoserverClient)
|
||||
|
||||
lazy val stratoSuperFollowRelationsRepo: StratoSuperFollowRelationsRepository.Type =
|
||||
StratoSuperFollowRelationsRepository(clients.stratoserverClient)
|
||||
|
||||
lazy val stratoPromotedTweetRepo: StratoPromotedTweetRepository.Type =
|
||||
StratoPromotedTweetRepository(clients.stratoserverClient)
|
||||
|
||||
lazy val stratoSubscriptionVerificationRepo: StratoSubscriptionVerificationRepository.Type =
|
||||
StratoSubscriptionVerificationRepository(clients.stratoserverClient)
|
||||
|
||||
lazy val geoScrubTimestampRepo: GeoScrubTimestampRepository.Type =
|
||||
GeoScrubTimestampRepository(clients.geoScrubEventStore.getGeoScrubTimestamp)
|
||||
|
||||
lazy val mediaMetadataRepo: MediaMetadataRepository.Type =
|
||||
MediaMetadataRepository(clients.mediaClient.getMediaMetadata)
|
||||
|
||||
lazy val perspectiveRepo: GetPerspectives =
|
||||
GetPerspectives(clients.timelineService.getPerspectives)
|
||||
|
||||
lazy val placeRepo: PlaceRepository.Type =
|
||||
GeoduckPlaceRepository(clients.geoHydrationLocate)
|
||||
|
||||
lazy val profileGeoRepo: ProfileGeoRepository.Type =
|
||||
ProfileGeoRepository(clients.gnipEnricherator.hydrateProfileGeo)
|
||||
|
||||
lazy val relationshipRepo: RelationshipRepository.Type =
|
||||
RelationshipRepository(clients.socialGraphService.exists, maxRequestSize = 6)
|
||||
|
||||
lazy val retweetSpamCheckRepo: RetweetSpamCheckRepository.Type =
|
||||
RetweetSpamCheckRepository(clients.scarecrow.checkRetweet)
|
||||
|
||||
lazy val tweetCountsRepo: TweetCountsRepository.Type =
|
||||
TweetCountsRepository(
|
||||
clients.tflockReadClient,
|
||||
maxRequestSize = settings.tweetCountsRepoChunkSize
|
||||
)
|
||||
|
||||
lazy val tweetResultRepo: TweetResultRepository.Type =
|
||||
ManhattanTweetRepository(
|
||||
clients.tweetStorageClient.getTweet,
|
||||
clients.tweetStorageClient.getStoredTweet,
|
||||
settings.shortCircuitLikelyPartialTweetReads,
|
||||
statsReceiver.scope("manhattan_tweet_repo"),
|
||||
clientIdHelper,
|
||||
)
|
||||
|
||||
lazy val tweetSpamCheckRepo: TweetSpamCheckRepository.Type =
|
||||
TweetSpamCheckRepository(clients.scarecrow.checkTweet2)
|
||||
|
||||
lazy val unmentionedEntitiesRepo: UnmentionedEntitiesRepository.Type =
|
||||
UnmentionedEntitiesRepository(clients.stratoserverClient)
|
||||
|
||||
lazy val urlRepo: UrlRepository.Type =
|
||||
UrlRepository(
|
||||
clients.talon.expand,
|
||||
settings.thriftClientId.name,
|
||||
statsReceiver.scope("talon_url_repo"),
|
||||
clientIdHelper,
|
||||
)
|
||||
|
||||
lazy val userRepo: UserRepository.Type =
|
||||
GizmoduckUserRepository(
|
||||
clients.gizmoduck.getById,
|
||||
clients.gizmoduck.getByScreenName,
|
||||
maxRequestSize = 100
|
||||
)
|
||||
|
||||
lazy val userIsInvitedToConversationRepo: UserIsInvitedToConversationRepository.Type =
|
||||
UserIsInvitedToConversationRepository(
|
||||
FutureArrow(clients.tflockReadClient.multiSelectOne(_)),
|
||||
FutureArrow((clients.tflockReadClient.contains(_: StatusGraph, _: Long, _: Long)).tupled))
|
||||
|
||||
}
|
@ -0,0 +1,807 @@
|
||||
package com.twitter.tweetypie
|
||||
package config
|
||||
|
||||
import com.twitter.abdecider.ABDeciderFactory
|
||||
import com.twitter.config.yaml.YamlConfig
|
||||
import com.twitter.decider.Decider
|
||||
import com.twitter.featureswitches.v2.FeatureSwitches
|
||||
import com.twitter.finagle.memcached
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.servo.cache._
|
||||
import com.twitter.servo.cache.{KeyValueResult => _}
|
||||
import com.twitter.servo.repository._
|
||||
import com.twitter.stitch.NotFound
|
||||
import com.twitter.stitch.Stitch
|
||||
import com.twitter.stitch.repo.Repo
|
||||
import com.twitter.stitch.timelineservice.TimelineService
|
||||
import com.twitter.strato.client.{Client => StratoClient}
|
||||
import com.twitter.stringcenter.client.ExternalStringRegistry
|
||||
import com.twitter.stringcenter.client.MultiProjectStringCenter
|
||||
import com.twitter.translation.Languages
|
||||
import com.twitter.translation.YamlConfigLanguages
|
||||
import com.twitter.tweetypie.caching.CacheOperations
|
||||
import com.twitter.tweetypie.caching.Expiry
|
||||
import com.twitter.tweetypie.caching.ServoCachedValueSerializer
|
||||
import com.twitter.tweetypie.caching.StitchCaching
|
||||
import com.twitter.tweetypie.caching.ValueSerializer
|
||||
import com.twitter.tweetypie.client_id.ClientIdHelper
|
||||
import com.twitter.tweetypie.core.FilteredState
|
||||
import com.twitter.tweetypie.core.TweetResult
|
||||
import com.twitter.tweetypie.hydrator.TextRepairer
|
||||
import com.twitter.tweetypie.hydrator.TweetHydration
|
||||
import com.twitter.tweetypie.hydrator.TweetQueryOptionsExpander
|
||||
import com.twitter.tweetypie.repository.TweetRepository
|
||||
import com.twitter.tweetypie.repository.UserRepository
|
||||
import com.twitter.tweetypie.repository._
|
||||
import com.twitter.tweetypie.serverutil.BoringStackTrace
|
||||
import com.twitter.tweetypie.serverutil.ExceptionCounter
|
||||
import com.twitter.tweetypie.thriftscala.DeviceSource
|
||||
import com.twitter.tweetypie.thriftscala.Place
|
||||
import com.twitter.tweetypie.thriftscala.entities.EntityExtractor
|
||||
import com.twitter.tweetypie.util.StitchUtils
|
||||
import com.twitter.util.Duration
|
||||
import com.twitter.util.FuturePool
|
||||
import com.twitter.util.Timer
|
||||
import com.twitter.visibility.VisibilityLibrary
|
||||
import com.twitter.visibility.common.KeywordMatcher
|
||||
import com.twitter.visibility.common.LocalizationSource
|
||||
import com.twitter.visibility.common.TweetMediaMetadataSource
|
||||
import com.twitter.visibility.common.TweetPerspectiveSource
|
||||
import com.twitter.visibility.common.UserRelationshipSource
|
||||
import com.twitter.visibility.common.UserSource
|
||||
import com.twitter.visibility.common.tflock.UserIsInvitedToConversationRepository
|
||||
import com.twitter.visibility.configapi.configs.VisibilityDeciderGates
|
||||
import com.twitter.visibility.generators.CountryNameGenerator
|
||||
import com.twitter.visibility.generators.LocalizedInterstitialGenerator
|
||||
import com.twitter.visibility.generators.TombstoneGenerator
|
||||
import com.twitter.visibility.interfaces.tweets.DeletedTweetVisibilityLibrary
|
||||
import com.twitter.visibility.interfaces.tweets.QuotedTweetVisibilityLibrary
|
||||
import com.twitter.visibility.interfaces.tweets.TweetVisibilityLibrary
|
||||
import com.twitter.visibility.interfaces.tweets.UserUnavailableStateVisibilityLibrary
|
||||
import com.twitter.visibility.util.DeciderUtil
|
||||
import com.twitter.visibility.util.FeatureSwitchUtil
|
||||
import java.util.concurrent.Executors
|
||||
|
||||
/**
|
||||
* LogicalRepositories is a layer above ExternalRepositories. These repos may have additional
|
||||
* logic layered in, such as memcache-caching, hot-key caching, etc. There may
|
||||
* also be multiple logical repositories mapped to an single external repository.
|
||||
*
|
||||
* These repositories are used in tweet hydration and tweet creation.
|
||||
*/
|
||||
trait LogicalRepositories {
|
||||
|
||||
def card2Repo: Card2Repository.Type
|
||||
def cardRepo: CardRepository.Type
|
||||
def cardUsersRepo: CardUsersRepository.Type
|
||||
def conversationIdRepo: ConversationIdRepository.Type
|
||||
def conversationControlRepo: ConversationControlRepository.Type
|
||||
def conversationMutedRepo: ConversationMutedRepository.Type
|
||||
def containerAsGetTweetResultRepo: CreativesContainerMaterializationRepository.GetTweetType
|
||||
def containerAsGetTweetFieldsResultRepo: CreativesContainerMaterializationRepository.GetTweetFieldsType
|
||||
def deviceSourceRepo: DeviceSourceRepository.Type
|
||||
def escherbirdAnnotationRepo: EscherbirdAnnotationRepository.Type
|
||||
def geoScrubTimestampRepo: GeoScrubTimestampRepository.Type
|
||||
def languageRepo: LanguageRepository.Type
|
||||
def mediaMetadataRepo: MediaMetadataRepository.Type
|
||||
def pastedMediaRepo: PastedMediaRepository.Type
|
||||
def perspectiveRepo: PerspectiveRepository.Type
|
||||
def placeRepo: PlaceRepository.Type
|
||||
def profileGeoRepo: ProfileGeoRepository.Type
|
||||
def quoterHasAlreadyQuotedRepo: QuoterHasAlreadyQuotedRepository.Type
|
||||
def lastQuoteOfQuoterRepo: LastQuoteOfQuoterRepository.Type
|
||||
def relationshipRepo: RelationshipRepository.Type
|
||||
def stratoSafetyLabelsRepo: StratoSafetyLabelsRepository.Type
|
||||
def stratoCommunityMembershipRepo: StratoCommunityMembershipRepository.Type
|
||||
def stratoCommunityAccessRepo: StratoCommunityAccessRepository.Type
|
||||
def stratoSuperFollowEligibleRepo: StratoSuperFollowEligibleRepository.Type
|
||||
def stratoSuperFollowRelationsRepo: StratoSuperFollowRelationsRepository.Type
|
||||
def stratoPromotedTweetRepo: StratoPromotedTweetRepository.Type
|
||||
def stratoSubscriptionVerificationRepo: StratoSubscriptionVerificationRepository.Type
|
||||
def takedownRepo: UserTakedownRepository.Type
|
||||
def tweetSpamCheckRepo: TweetSpamCheckRepository.Type
|
||||
def retweetSpamCheckRepo: RetweetSpamCheckRepository.Type
|
||||
def tweetCountsRepo: TweetCountsRepository.Type
|
||||
def tweetVisibilityRepo: TweetVisibilityRepository.Type
|
||||
def quotedTweetVisibilityRepo: QuotedTweetVisibilityRepository.Type
|
||||
def deletedTweetVisibilityRepo: DeletedTweetVisibilityRepository.Type
|
||||
def unmentionedEntitiesRepo: UnmentionedEntitiesRepository.Type
|
||||
def urlRepo: UrlRepository.Type
|
||||
def userRepo: UserRepository.Type
|
||||
def optionalUserRepo: UserRepository.Optional
|
||||
def userIdentityRepo: UserIdentityRepository.Type
|
||||
def userIsInvitedToConversationRepo: UserIsInvitedToConversationRepository.Type
|
||||
def userProtectionRepo: UserProtectionRepository.Type
|
||||
def userViewRepo: UserViewRepository.Type
|
||||
def userVisibilityRepo: UserVisibilityRepository.Type
|
||||
|
||||
def tweetResultRepo: TweetResultRepository.Type
|
||||
def tweetRepo: TweetRepository.Type
|
||||
def optionalTweetRepo: TweetRepository.Optional
|
||||
|
||||
/**
|
||||
* Not actually repositories, but intimately intertwined.
|
||||
*/
|
||||
def tweetHydrators: TweetHydrators
|
||||
}
|
||||
|
||||
object LogicalRepositories {
|
||||
|
||||
/**
|
||||
* Middleware is a function that takes a stitch repo and returns a new stitch repo.
|
||||
*/
|
||||
type Middleware[K, V] = (K => Stitch[V]) => K => Stitch[V]
|
||||
|
||||
// Middleware2 is a function that takes a two-arg stitch repo and returns a new two-arg stitch repo.
|
||||
type Middleware2[K, C, V] = ((K, C) => Stitch[V]) => ((K, C) => Stitch[V])
|
||||
val exceptionLog: Logger = Logger(getClass)
|
||||
|
||||
// Converts a Middleware2 to a Middleware for use with withMiddleware.
|
||||
def tupledMiddleware[K, C, V](middleware2: Middleware2[K, C, V]): Middleware[(K, C), V] =
|
||||
repo => middleware2(Function.untupled(repo)).tupled
|
||||
|
||||
object ObserveStitch {
|
||||
def apply[K, V](
|
||||
repo: K => Stitch[V],
|
||||
repoName: String,
|
||||
stats: StatsReceiver
|
||||
): K => Stitch[V] = {
|
||||
val successCounter = stats.counter("success")
|
||||
val notFoundCounter = stats.counter("not_found")
|
||||
val latencyStat = stats.stat("latency_ms")
|
||||
|
||||
val exceptionCounter =
|
||||
ExceptionCounter(
|
||||
stats,
|
||||
// don't count FilteredState exceptions
|
||||
FilteredState.ignoringCategorizer(ExceptionCounter.defaultCategorizer)
|
||||
)
|
||||
|
||||
(key: K) =>
|
||||
StitchUtils.trackLatency(latencyStat, repo(key)).respond {
|
||||
case Return(_) => successCounter.incr()
|
||||
case Throw(NotFound) => notFoundCounter.incr()
|
||||
case Throw(t) =>
|
||||
val message = s"$repoName: $key"
|
||||
if (BoringStackTrace.isBoring(t)) {
|
||||
exceptionLog.debug(message, t)
|
||||
} else {
|
||||
exceptionLog.warn(message, t)
|
||||
}
|
||||
|
||||
exceptionCounter(t)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Add middleware to configure a repository. The stats receiver is
|
||||
* scoped for the currently-configured repository. The `toRepo` field
|
||||
* is the repository with some set of middleware applied. Each method
|
||||
* adds a new middleware to the current repo, and returns it as a
|
||||
* `RepoConfig`, allowing method chaining.
|
||||
*
|
||||
* Since each method call applies a new middleware, the final middleware is
|
||||
* the outermost middleware, and thus the one that sees the arguments
|
||||
* first.
|
||||
*/
|
||||
class RepoConfig[K, V](
|
||||
val toRepo: K => Stitch[V],
|
||||
stats: StatsReceiver,
|
||||
name: String,
|
||||
memcachedClientWithInProcessCaching: memcached.Client) {
|
||||
def withMiddleware(middleware: Middleware[K, V]): RepoConfig[K, V] =
|
||||
new RepoConfig[K, V](middleware(toRepo), stats, name, memcachedClientWithInProcessCaching)
|
||||
|
||||
/**
|
||||
* Wraps a repo with success/failure/latency stats tracking and logs
|
||||
* exceptions. This will be applied to every repository.
|
||||
*
|
||||
* @param repoName Used when logging exceptions thrown by the underlying repo.
|
||||
*/
|
||||
def observe(repoName: String = s"${name}_repo"): RepoConfig[K, V] = {
|
||||
withMiddleware { repo => ObserveStitch[K, V](repo, repoName, stats) }
|
||||
}
|
||||
|
||||
/**
|
||||
* Use the supplied cache to wrap the repository with a read-through
|
||||
* caching layer.
|
||||
*/
|
||||
def caching(
|
||||
cache: LockingCache[K, Cached[V]],
|
||||
partialHandler: CachedResult.PartialHandler[K, V],
|
||||
maxCacheRequestSize: Int = Int.MaxValue
|
||||
): RepoConfig[K, V] = {
|
||||
val stitchLockingCache = StitchLockingCache(
|
||||
underlying = cache,
|
||||
picker = new PreferNewestCached[V],
|
||||
maxRequestSize = maxCacheRequestSize
|
||||
)
|
||||
|
||||
val handler: CachedResult.Handler[K, V] =
|
||||
CachedResult.Handler(
|
||||
CachedResult.PartialHandler.orElse(
|
||||
partialHandler,
|
||||
CachedResult.failuresAreDoNotCache
|
||||
)
|
||||
)
|
||||
|
||||
withMiddleware { repo =>
|
||||
CacheStitch[K, K, V](
|
||||
repo = repo,
|
||||
cache = stitchLockingCache,
|
||||
identity,
|
||||
handler = handler,
|
||||
cacheable = CacheStitch.cacheFoundAndNotFound
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
def newCaching(
|
||||
keySerializer: K => String,
|
||||
valueSerializer: ValueSerializer[Try[V]]
|
||||
): RepoConfig[K, V] =
|
||||
withMiddleware { repo =>
|
||||
val logger = Logger(s"com.twitter.tweetypie.config.LogicalRepositories.$name")
|
||||
|
||||
val cacheOperations: CacheOperations[K, Try[V]] =
|
||||
new CacheOperations(
|
||||
keySerializer = keySerializer,
|
||||
valueSerializer = valueSerializer,
|
||||
memcachedClient = memcachedClientWithInProcessCaching,
|
||||
statsReceiver = stats.scope("caching"),
|
||||
logger = logger
|
||||
)
|
||||
|
||||
val tryRepo: K => Stitch[Try[V]] = repo.andThen(_.liftToTry)
|
||||
val cachingTryRepo: K => Stitch[Try[V]] = new StitchCaching(cacheOperations, tryRepo)
|
||||
cachingTryRepo.andThen(_.lowerFromTry)
|
||||
}
|
||||
|
||||
def toRepo2[K1, C](implicit tupleToK: ((K1, C)) <:< K): (K1, C) => Stitch[V] =
|
||||
(k1, c) => toRepo(tupleToK((k1, c)))
|
||||
}
|
||||
|
||||
def softTtlPartialHandler[K, V](
|
||||
softTtl: Option[V] => Duration,
|
||||
softTtlPerturbationFactor: Float = 0.05f
|
||||
): CachedResult.PartialHandler[K, V] =
|
||||
CachedResult
|
||||
.softTtlExpiration[K, V](softTtl, CachedResult.randomExpiry(softTtlPerturbationFactor))
|
||||
|
||||
def apply(
|
||||
settings: TweetServiceSettings,
|
||||
stats: StatsReceiver,
|
||||
timer: Timer,
|
||||
deciderGates: TweetypieDeciderGates,
|
||||
external: ExternalRepositories,
|
||||
caches: Caches,
|
||||
stratoClient: StratoClient,
|
||||
hasMedia: Tweet => Boolean,
|
||||
clientIdHelper: ClientIdHelper,
|
||||
featureSwitchesWithoutExperiments: FeatureSwitches,
|
||||
): LogicalRepositories = {
|
||||
val repoStats = stats.scope("repositories")
|
||||
|
||||
def repoConfig[K, V](name: String, repo: K => Stitch[V]): RepoConfig[K, V] =
|
||||
new RepoConfig[K, V](
|
||||
name = name,
|
||||
toRepo = repo,
|
||||
stats = repoStats.scope(name),
|
||||
memcachedClientWithInProcessCaching = caches.memcachedClientWithInProcessCaching)
|
||||
|
||||
def repo2Config[K, C, V](name: String, repo: (K, C) => Stitch[V]): RepoConfig[(K, C), V] =
|
||||
repoConfig[(K, C), V](name, repo.tupled)
|
||||
|
||||
new LogicalRepositories {
|
||||
// the final tweetResultRepo has a circular dependency, where it depends on hydrators
|
||||
// that in turn depend on the tweetResultRepo, so we create a `tweetResultRepo` function
|
||||
// that proxies to `var finalTweetResultRepo`, which gets set at the end of this block.
|
||||
var finalTweetResultRepo: TweetResultRepository.Type = null
|
||||
val tweetResultRepo: TweetResultRepository.Type =
|
||||
(tweetId, opts) => finalTweetResultRepo(tweetId, opts)
|
||||
val tweetRepo: TweetRepository.Type = TweetRepository.fromTweetResult(tweetResultRepo)
|
||||
|
||||
val optionalTweetRepo: TweetRepository.Optional = TweetRepository.optional(tweetRepo)
|
||||
|
||||
val userRepo: UserRepository.Type =
|
||||
repo2Config(repo = external.userRepo, name = "user")
|
||||
.observe()
|
||||
.toRepo2
|
||||
|
||||
val optionalUserRepo: UserRepository.Optional = UserRepository.optional(userRepo)
|
||||
|
||||
private[this] val tweetVisibilityStatsReceiver: StatsReceiver =
|
||||
repoStats.scope("tweet_visibility_library")
|
||||
private[this] val userUnavailableVisibilityStatsReceiver: StatsReceiver =
|
||||
repoStats.scope("user_unavailable_visibility_library")
|
||||
private[this] val quotedTweetVisibilityStatsReceiver: StatsReceiver =
|
||||
repoStats.scope("quoted_tweet_visibility_library")
|
||||
private[this] val deletedTweetVisibilityStatsReceiver: StatsReceiver =
|
||||
repoStats.scope("deleted_tweet_visibility_library")
|
||||
// TweetVisibilityLibrary still uses the old c.t.logging.Logger
|
||||
private[this] val tweetVisibilityLogger =
|
||||
com.twitter.logging.Logger("com.twitter.tweetypie.TweetVisibility")
|
||||
private[this] val visibilityDecider: Decider = DeciderUtil.mkDecider(
|
||||
deciderOverlayPath = settings.vfDeciderOverlayFilename,
|
||||
useLocalDeciderOverrides = true)
|
||||
private[this] val visibilityDeciderGates = VisibilityDeciderGates(visibilityDecider)
|
||||
|
||||
private[this] def visibilityLibrary(statsReceiver: StatsReceiver) = VisibilityLibrary
|
||||
.Builder(
|
||||
log = tweetVisibilityLogger,
|
||||
statsReceiver = statsReceiver,
|
||||
memoizeSafetyLevelParams = visibilityDeciderGates.enableMemoizeSafetyLevelParams
|
||||
)
|
||||
.withDecider(visibilityDecider)
|
||||
.withDefaultABDecider(isLocal = false)
|
||||
.withCaptureDebugStats(Gate.True)
|
||||
.withEnableComposableActions(Gate.True)
|
||||
.withEnableFailClosed(Gate.True)
|
||||
.withEnableShortCircuiting(visibilityDeciderGates.enableShortCircuitingTVL)
|
||||
.withSpecialLogging(visibilityDeciderGates.enableSpecialLogging)
|
||||
.build()
|
||||
|
||||
def countryNameGenerator(statsReceiver: StatsReceiver) = {
|
||||
// TweetVisibilityLibrary, DeletedTweetVisibilityLibrary, and
|
||||
// UserUnavailableVisibilityLibrary do not evaluate any Rules
|
||||
// that require the display of country names in copy
|
||||
CountryNameGenerator.providesWithCustomMap(Map.empty, statsReceiver)
|
||||
}
|
||||
|
||||
def tombstoneGenerator(
|
||||
countryNameGenerator: CountryNameGenerator,
|
||||
statsReceiver: StatsReceiver
|
||||
) =
|
||||
TombstoneGenerator(
|
||||
visibilityLibrary(statsReceiver).visParams,
|
||||
countryNameGenerator,
|
||||
statsReceiver)
|
||||
|
||||
private[this] val userUnavailableVisibilityLibrary =
|
||||
UserUnavailableStateVisibilityLibrary(
|
||||
visibilityLibrary(userUnavailableVisibilityStatsReceiver),
|
||||
visibilityDecider,
|
||||
tombstoneGenerator(
|
||||
countryNameGenerator(userUnavailableVisibilityStatsReceiver),
|
||||
userUnavailableVisibilityStatsReceiver
|
||||
),
|
||||
LocalizedInterstitialGenerator(visibilityDecider, userUnavailableVisibilityStatsReceiver)
|
||||
)
|
||||
|
||||
val userIdentityRepo: UserIdentityRepository.Type =
|
||||
repoConfig(repo = UserIdentityRepository(userRepo), name = "user_identity")
|
||||
.observe()
|
||||
.toRepo
|
||||
|
||||
val userProtectionRepo: UserProtectionRepository.Type =
|
||||
repoConfig(repo = UserProtectionRepository(userRepo), name = "user_protection")
|
||||
.observe()
|
||||
.toRepo
|
||||
|
||||
val userViewRepo: UserViewRepository.Type =
|
||||
repoConfig(repo = UserViewRepository(userRepo), name = "user_view")
|
||||
.observe()
|
||||
.toRepo
|
||||
|
||||
val userVisibilityRepo: UserVisibilityRepository.Type =
|
||||
repoConfig(
|
||||
repo = UserVisibilityRepository(userRepo, userUnavailableVisibilityLibrary),
|
||||
name = "user_visibility"
|
||||
).observe().toRepo
|
||||
|
||||
val urlRepo: UrlRepository.Type =
|
||||
repoConfig(repo = external.urlRepo, name = "url")
|
||||
.observe()
|
||||
.toRepo
|
||||
|
||||
val profileGeoRepo: ProfileGeoRepository.Type =
|
||||
repoConfig(repo = external.profileGeoRepo, name = "profile_geo")
|
||||
.observe()
|
||||
.toRepo
|
||||
|
||||
val quoterHasAlreadyQuotedRepo: QuoterHasAlreadyQuotedRepository.Type =
|
||||
repo2Config(repo = external.quoterHasAlreadyQuotedRepo, name = "quoter_has_already_quoted")
|
||||
.observe()
|
||||
.toRepo2
|
||||
|
||||
val lastQuoteOfQuoterRepo: LastQuoteOfQuoterRepository.Type =
|
||||
repo2Config(repo = external.lastQuoteOfQuoterRepo, name = "last_quote_of_quoter")
|
||||
.observe()
|
||||
.toRepo2
|
||||
|
||||
val mediaMetadataRepo: MediaMetadataRepository.Type =
|
||||
repoConfig(repo = external.mediaMetadataRepo, name = "media_metadata")
|
||||
.observe()
|
||||
.toRepo
|
||||
|
||||
val perspectiveRepo: PerspectiveRepository.Type =
|
||||
repoConfig(repo = external.perspectiveRepo, name = "perspective")
|
||||
.observe()
|
||||
.toRepo
|
||||
|
||||
val conversationMutedRepo: ConversationMutedRepository.Type =
|
||||
TimelineService.GetPerspectives.getConversationMuted(perspectiveRepo)
|
||||
|
||||
// Because observe is applied before caching, only cache misses
|
||||
// (i.e. calls to the underlying repo) are observed.
|
||||
// Note that `newCaching` has stats around cache hit/miss but `caching` does not.
|
||||
val deviceSourceRepo: DeviceSourceRepository.Type =
|
||||
repoConfig(repo = external.deviceSourceRepo, name = "device_source")
|
||||
.observe()
|
||||
.newCaching(
|
||||
keySerializer = appIdStr => DeviceSourceKey(appIdStr).toString,
|
||||
valueSerializer = ServoCachedValueSerializer(
|
||||
codec = DeviceSource,
|
||||
expiry = Expiry.byAge(settings.deviceSourceMemcacheTtl),
|
||||
softTtl = settings.deviceSourceMemcacheSoftTtl
|
||||
)
|
||||
)
|
||||
.caching(
|
||||
cache = caches.deviceSourceInProcessCache,
|
||||
partialHandler = softTtlPartialHandler(_ => settings.deviceSourceInProcessSoftTtl)
|
||||
)
|
||||
.toRepo
|
||||
|
||||
// Because observe is applied before caching, only cache misses
|
||||
// (i.e. calls to the underlying repo) are observed
|
||||
// Note that `newCaching` has stats around cache hit/miss but `caching` does not.
|
||||
val placeRepo: PlaceRepository.Type =
|
||||
repoConfig(repo = external.placeRepo, name = "place")
|
||||
.observe()
|
||||
.newCaching(
|
||||
keySerializer = placeKey => placeKey.toString,
|
||||
valueSerializer = ServoCachedValueSerializer(
|
||||
codec = Place,
|
||||
expiry = Expiry.byAge(settings.placeMemcacheTtl),
|
||||
softTtl = settings.placeMemcacheSoftTtl
|
||||
)
|
||||
)
|
||||
.toRepo
|
||||
|
||||
val cardRepo: CardRepository.Type =
|
||||
repoConfig(repo = external.cardRepo, name = "cards")
|
||||
.observe()
|
||||
.toRepo
|
||||
|
||||
val card2Repo: Card2Repository.Type =
|
||||
repo2Config(repo = external.card2Repo, name = "card2")
|
||||
.observe()
|
||||
.toRepo2
|
||||
|
||||
val cardUsersRepo: CardUsersRepository.Type =
|
||||
repo2Config(repo = external.cardUsersRepo, name = "card_users")
|
||||
.observe()
|
||||
.toRepo2
|
||||
|
||||
val relationshipRepo: RelationshipRepository.Type =
|
||||
repoConfig(repo = external.relationshipRepo, name = "relationship")
|
||||
.observe()
|
||||
.toRepo
|
||||
|
||||
val conversationIdRepo: ConversationIdRepository.Type =
|
||||
repoConfig(repo = external.conversationIdRepo, name = "conversation_id")
|
||||
.observe()
|
||||
.toRepo
|
||||
|
||||
val conversationControlRepo: ConversationControlRepository.Type =
|
||||
repo2Config(
|
||||
repo = ConversationControlRepository(tweetRepo, stats.scope("conversation_control")),
|
||||
name = "conversation_control"
|
||||
).observe().toRepo2
|
||||
|
||||
val containerAsGetTweetResultRepo: CreativesContainerMaterializationRepository.GetTweetType =
|
||||
repo2Config(
|
||||
repo = external.containerAsTweetRepo,
|
||||
name = "container_as_tweet"
|
||||
).observe().toRepo2
|
||||
|
||||
val containerAsGetTweetFieldsResultRepo: CreativesContainerMaterializationRepository.GetTweetFieldsType =
|
||||
repo2Config(
|
||||
repo = external.containerAsTweetFieldsRepo,
|
||||
name = "container_as_tweet_fields"
|
||||
).observe().toRepo2
|
||||
|
||||
val languageRepo: LanguageRepository.Type = {
|
||||
val pool = FuturePool(Executors.newFixedThreadPool(settings.numPenguinThreads))
|
||||
repoConfig(repo = PenguinLanguageRepository(pool), name = "language")
|
||||
.observe()
|
||||
.toRepo
|
||||
}
|
||||
|
||||
// Because observe is applied before caching, only cache misses
|
||||
// (i.e. calls to the underlying repo) are observed
|
||||
// Note that `newCaching` has stats around cache hit/miss but `caching` does not.
|
||||
val tweetCountsRepo: TweetCountsRepository.Type =
|
||||
repoConfig(repo = external.tweetCountsRepo, name = "counts")
|
||||
.observe()
|
||||
.caching(
|
||||
cache = caches.tweetCountsCache,
|
||||
partialHandler = softTtlPartialHandler {
|
||||
case Some(0) => settings.tweetCountsMemcacheZeroSoftTtl
|
||||
case _ => settings.tweetCountsMemcacheNonZeroSoftTtl
|
||||
},
|
||||
maxCacheRequestSize = settings.tweetCountsCacheChunkSize
|
||||
)
|
||||
.toRepo
|
||||
|
||||
val pastedMediaRepo: PastedMediaRepository.Type =
|
||||
repo2Config(repo = PastedMediaRepository(tweetRepo), name = "pasted_media")
|
||||
.observe()
|
||||
.toRepo2
|
||||
|
||||
val escherbirdAnnotationRepo: EscherbirdAnnotationRepository.Type =
|
||||
repoConfig(repo = external.escherbirdAnnotationRepo, name = "escherbird_annotations")
|
||||
.observe()
|
||||
.toRepo
|
||||
|
||||
val stratoSafetyLabelsRepo: StratoSafetyLabelsRepository.Type =
|
||||
repo2Config(repo = external.stratoSafetyLabelsRepo, name = "strato_safety_labels")
|
||||
.observe()
|
||||
.toRepo2
|
||||
|
||||
val stratoCommunityMembershipRepo: StratoCommunityMembershipRepository.Type =
|
||||
repoConfig(
|
||||
repo = external.stratoCommunityMembershipRepo,
|
||||
name = "strato_community_memberships")
|
||||
.observe()
|
||||
.toRepo
|
||||
|
||||
val stratoCommunityAccessRepo: StratoCommunityAccessRepository.Type =
|
||||
repoConfig(repo = external.stratoCommunityAccessRepo, name = "strato_community_access")
|
||||
.observe()
|
||||
.toRepo
|
||||
|
||||
val stratoSuperFollowEligibleRepo: StratoSuperFollowEligibleRepository.Type =
|
||||
repoConfig(
|
||||
repo = external.stratoSuperFollowEligibleRepo,
|
||||
name = "strato_super_follow_eligible")
|
||||
.observe()
|
||||
.toRepo
|
||||
|
||||
val stratoSuperFollowRelationsRepo: StratoSuperFollowRelationsRepository.Type =
|
||||
repo2Config(
|
||||
repo = external.stratoSuperFollowRelationsRepo,
|
||||
name = "strato_super_follow_relations")
|
||||
.observe()
|
||||
.toRepo2
|
||||
|
||||
val stratoPromotedTweetRepo: StratoPromotedTweetRepository.Type =
|
||||
repoConfig(repo = external.stratoPromotedTweetRepo, name = "strato_promoted_tweet")
|
||||
.observe()
|
||||
.toRepo
|
||||
|
||||
val stratoSubscriptionVerificationRepo: StratoSubscriptionVerificationRepository.Type =
|
||||
repo2Config(
|
||||
repo = external.stratoSubscriptionVerificationRepo,
|
||||
name = "strato_subscription_verification")
|
||||
.observe()
|
||||
.toRepo2
|
||||
|
||||
val unmentionedEntitiesRepo: UnmentionedEntitiesRepository.Type =
|
||||
repo2Config(repo = external.unmentionedEntitiesRepo, name = "unmentioned_entities")
|
||||
.observe()
|
||||
.toRepo2
|
||||
|
||||
private[this] val userSource =
|
||||
UserSource.fromRepo(
|
||||
Repo { (k, _) =>
|
||||
val opts = UserQueryOptions(k.fields, UserVisibility.All)
|
||||
userRepo(UserKey(k.id), opts)
|
||||
}
|
||||
)
|
||||
|
||||
private[this] val userRelationshipSource =
|
||||
UserRelationshipSource.fromRepo(
|
||||
Repo[UserRelationshipSource.Key, Unit, Boolean] { (key, _) =>
|
||||
relationshipRepo(
|
||||
RelationshipKey(key.subjectId, key.objectId, key.relationship)
|
||||
)
|
||||
}
|
||||
)
|
||||
|
||||
private[this] val tweetPerspectiveSource =
|
||||
TweetPerspectiveSource.fromGetPerspectives(perspectiveRepo)
|
||||
private[this] val tweetMediaMetadataSource =
|
||||
TweetMediaMetadataSource.fromFunction(mediaMetadataRepo)
|
||||
|
||||
val userIsInvitedToConversationRepo: UserIsInvitedToConversationRepository.Type =
|
||||
repo2Config(
|
||||
repo = external.userIsInvitedToConversationRepo,
|
||||
name = "user_is_invited_to_conversation")
|
||||
.observe()
|
||||
.toRepo2
|
||||
|
||||
private[this] val stringCenterClient: MultiProjectStringCenter = {
|
||||
val stringCenterProjects = settings.flags.stringCenterProjects().toList
|
||||
|
||||
val languages: Languages = new YamlConfigLanguages(
|
||||
new YamlConfig(settings.flags.languagesConfig()))
|
||||
|
||||
val loggingAbDecider = ABDeciderFactory("/usr/local/config/abdecider/abdecider.yml")
|
||||
.withEnvironment("production")
|
||||
.buildWithLogging()
|
||||
|
||||
MultiProjectStringCenter(
|
||||
projects = stringCenterProjects,
|
||||
defaultBundlePath = MultiProjectStringCenter.StandardDefaultBundlePath,
|
||||
refreshingBundlePath = MultiProjectStringCenter.StandardRefreshingBundlePath,
|
||||
refreshingInterval = MultiProjectStringCenter.StandardRefreshingInterval,
|
||||
requireDefaultBundleExists = true,
|
||||
languages = languages,
|
||||
statsReceiver = tweetVisibilityStatsReceiver,
|
||||
loggingABDecider = loggingAbDecider
|
||||
)
|
||||
}
|
||||
private[this] val stringRegistry: ExternalStringRegistry = new ExternalStringRegistry()
|
||||
private[this] val localizationSource: LocalizationSource =
|
||||
LocalizationSource.fromMultiProjectStringCenterClient(stringCenterClient, stringRegistry)
|
||||
|
||||
val tweetVisibilityRepo: TweetVisibilityRepository.Type = {
|
||||
val tweetVisibilityLibrary: TweetVisibilityLibrary.Type =
|
||||
TweetVisibilityLibrary(
|
||||
visibilityLibrary(tweetVisibilityStatsReceiver),
|
||||
userSource = userSource,
|
||||
userRelationshipSource = userRelationshipSource,
|
||||
keywordMatcher = KeywordMatcher.defaultMatcher(stats),
|
||||
stratoClient = stratoClient,
|
||||
localizationSource = localizationSource,
|
||||
decider = visibilityDecider,
|
||||
invitedToConversationRepo = userIsInvitedToConversationRepo,
|
||||
tweetPerspectiveSource = tweetPerspectiveSource,
|
||||
tweetMediaMetadataSource = tweetMediaMetadataSource,
|
||||
tombstoneGenerator = tombstoneGenerator(
|
||||
countryNameGenerator(tweetVisibilityStatsReceiver),
|
||||
tweetVisibilityStatsReceiver
|
||||
),
|
||||
interstitialGenerator =
|
||||
LocalizedInterstitialGenerator(visibilityDecider, tweetVisibilityStatsReceiver),
|
||||
limitedActionsFeatureSwitches =
|
||||
FeatureSwitchUtil.mkLimitedActionsFeatureSwitches(tweetVisibilityStatsReceiver),
|
||||
enableParityTest = deciderGates.tweetVisibilityLibraryEnableParityTest
|
||||
)
|
||||
|
||||
val underlying =
|
||||
TweetVisibilityRepository(
|
||||
tweetVisibilityLibrary,
|
||||
visibilityDeciderGates,
|
||||
tweetVisibilityLogger,
|
||||
repoStats.scope("tweet_visibility_repo")
|
||||
)
|
||||
|
||||
repoConfig(repo = underlying, name = "tweet_visibility")
|
||||
.observe()
|
||||
.toRepo
|
||||
}
|
||||
|
||||
val quotedTweetVisibilityRepo: QuotedTweetVisibilityRepository.Type = {
|
||||
val quotedTweetVisibilityLibrary: QuotedTweetVisibilityLibrary.Type =
|
||||
QuotedTweetVisibilityLibrary(
|
||||
visibilityLibrary(quotedTweetVisibilityStatsReceiver),
|
||||
userSource = userSource,
|
||||
userRelationshipSource = userRelationshipSource,
|
||||
visibilityDecider,
|
||||
userStateVisibilityLibrary = userUnavailableVisibilityLibrary,
|
||||
enableVfFeatureHydration = deciderGates.enableVfFeatureHydrationInQuotedTweetVLShim
|
||||
)
|
||||
|
||||
val underlying =
|
||||
QuotedTweetVisibilityRepository(quotedTweetVisibilityLibrary, visibilityDeciderGates)
|
||||
|
||||
repoConfig(repo = underlying, name = "quoted_tweet_visibility")
|
||||
.observe()
|
||||
.toRepo
|
||||
}
|
||||
|
||||
val deletedTweetVisibilityRepo: DeletedTweetVisibilityRepository.Type = {
|
||||
val deletedTweetVisibilityLibrary: DeletedTweetVisibilityLibrary.Type =
|
||||
DeletedTweetVisibilityLibrary(
|
||||
visibilityLibrary(deletedTweetVisibilityStatsReceiver),
|
||||
visibilityDecider,
|
||||
tombstoneGenerator(
|
||||
countryNameGenerator(deletedTweetVisibilityStatsReceiver),
|
||||
deletedTweetVisibilityStatsReceiver
|
||||
)
|
||||
)
|
||||
|
||||
val underlying = DeletedTweetVisibilityRepository.apply(
|
||||
deletedTweetVisibilityLibrary
|
||||
)
|
||||
|
||||
repoConfig(repo = underlying, name = "deleted_tweet_visibility")
|
||||
.observe()
|
||||
.toRepo
|
||||
}
|
||||
|
||||
val takedownRepo: UserTakedownRepository.Type =
|
||||
repoConfig(repo = UserTakedownRepository(userRepo), name = "takedowns")
|
||||
.observe()
|
||||
.toRepo
|
||||
|
||||
val tweetSpamCheckRepo: TweetSpamCheckRepository.Type =
|
||||
repo2Config(repo = external.tweetSpamCheckRepo, name = "tweet_spam_check")
|
||||
.observe()
|
||||
.toRepo2
|
||||
|
||||
val retweetSpamCheckRepo: RetweetSpamCheckRepository.Type =
|
||||
repoConfig(repo = external.retweetSpamCheckRepo, name = "retweet_spam_check")
|
||||
.observe()
|
||||
.toRepo
|
||||
|
||||
// Because observe is applied before caching, only cache misses
|
||||
// (i.e. calls to the underlying repo) are observed
|
||||
// Note that `newCaching` has stats around cache hit/miss but `caching` does not.
|
||||
val geoScrubTimestampRepo: GeoScrubTimestampRepository.Type =
|
||||
repoConfig(repo = external.geoScrubTimestampRepo, name = "geo_scrub")
|
||||
.observe()
|
||||
.caching(
|
||||
cache = caches.geoScrubCache,
|
||||
partialHandler = (_ => None)
|
||||
)
|
||||
.toRepo
|
||||
|
||||
val tweetHydrators: TweetHydrators =
|
||||
TweetHydrators(
|
||||
stats = stats,
|
||||
deciderGates = deciderGates,
|
||||
repos = this,
|
||||
tweetDataCache = caches.tweetDataCache,
|
||||
hasMedia = hasMedia,
|
||||
featureSwitchesWithoutExperiments = featureSwitchesWithoutExperiments,
|
||||
clientIdHelper = clientIdHelper,
|
||||
)
|
||||
|
||||
val queryOptionsExpander: TweetQueryOptionsExpander.Type =
|
||||
TweetQueryOptionsExpander.threadLocalMemoize(
|
||||
TweetQueryOptionsExpander.expandDependencies
|
||||
)
|
||||
|
||||
// mutations to tweets that we only need to apply when reading from the external
|
||||
// repository, and not when reading from cache
|
||||
val tweetMutation: Mutation[Tweet] =
|
||||
Mutation
|
||||
.all(
|
||||
Seq(
|
||||
EntityExtractor.mutationAll,
|
||||
TextRepairer.BlankLineCollapser,
|
||||
TextRepairer.CoreTextBugPatcher
|
||||
)
|
||||
).onlyIf(_.coreData.isDefined)
|
||||
|
||||
val cachingTweetRepo: TweetResultRepository.Type =
|
||||
repo2Config(repo = external.tweetResultRepo, name = "saved_tweet")
|
||||
.observe()
|
||||
.withMiddleware { repo =>
|
||||
// applies tweetMutation to the results of TweetResultRepository
|
||||
val mutateResult = TweetResult.mutate(tweetMutation)
|
||||
repo.andThen(stitchResult => stitchResult.map(mutateResult))
|
||||
}
|
||||
.withMiddleware(
|
||||
tupledMiddleware(
|
||||
CachingTweetRepository(
|
||||
caches.tweetResultCache,
|
||||
settings.tweetTombstoneTtl,
|
||||
stats.scope("saved_tweet", "cache"),
|
||||
clientIdHelper,
|
||||
deciderGates.logCacheExceptions,
|
||||
)
|
||||
)
|
||||
)
|
||||
.toRepo2
|
||||
|
||||
finalTweetResultRepo = repo2Config(repo = cachingTweetRepo, name = "tweet")
|
||||
.withMiddleware(
|
||||
tupledMiddleware(
|
||||
TweetHydration.hydrateRepo(
|
||||
tweetHydrators.hydrator,
|
||||
tweetHydrators.cacheChangesEffect,
|
||||
queryOptionsExpander
|
||||
)
|
||||
)
|
||||
)
|
||||
.observe()
|
||||
.withMiddleware(tupledMiddleware(TweetResultRepository.shortCircuitInvalidIds))
|
||||
.toRepo2
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,314 @@
|
||||
package com.twitter.tweetypie
|
||||
package config
|
||||
|
||||
import com.twitter.app.Flag
|
||||
import com.twitter.app.Flaggable
|
||||
import com.twitter.app.Flags
|
||||
import com.twitter.finagle.http.HttpMuxer
|
||||
import com.twitter.finagle.mtls.authentication.ServiceIdentifier
|
||||
import com.twitter.finagle.mtls.authorization.server.MtlsServerSessionTrackerFilter
|
||||
import com.twitter.finagle.mtls.server.MtlsStackServer._
|
||||
import com.twitter.finagle.param.Reporter
|
||||
import com.twitter.finagle.ssl.OpportunisticTls
|
||||
import com.twitter.finagle.util.NullReporterFactory
|
||||
import com.twitter.finagle.Thrift
|
||||
import com.twitter.finagle.ThriftMux
|
||||
import com.twitter.flockdb.client.thriftscala.Priority
|
||||
import com.twitter.inject.Injector
|
||||
import com.twitter.inject.annotations.{Flags => InjectFlags}
|
||||
import com.twitter.scrooge.ThriftEnum
|
||||
import com.twitter.scrooge.ThriftEnumObject
|
||||
import com.twitter.server.handler.IndexHandler
|
||||
import com.twitter.strato.catalog.Catalog
|
||||
import com.twitter.strato.fed.StratoFed
|
||||
import com.twitter.strato.fed.server.StratoFedServer
|
||||
import com.twitter.strato.util.Ref
|
||||
import com.twitter.strato.warmup.Warmer
|
||||
import com.twitter.tweetypie.federated.StratoCatalogBuilder
|
||||
import com.twitter.tweetypie.federated.warmups.StratoCatalogWarmups
|
||||
import com.twitter.tweetypie.serverutil.ActivityService
|
||||
import java.net.InetSocketAddress
|
||||
import scala.reflect.ClassTag
|
||||
|
||||
object Env extends Enumeration {
|
||||
val dev: Env.Value = Value
|
||||
val staging: Env.Value = Value
|
||||
val prod: Env.Value = Value
|
||||
}
|
||||
|
||||
class TweetServiceFlags(flag: Flags, injector: => Injector) {
|
||||
implicit object EnvFlaggable extends Flaggable[Env.Value] {
|
||||
def parse(s: String): Env.Value =
|
||||
s match {
|
||||
// Handle Aurora env names that are different from tweetypie's names
|
||||
case "devel" => Env.dev
|
||||
case "test" => Env.staging
|
||||
// Handle Tweetypie env names
|
||||
case other => Env.withName(other)
|
||||
}
|
||||
}
|
||||
|
||||
val zone: Flag[String] =
|
||||
flag("zone", "localhost", "One of: atla, pdxa, localhost, etc.")
|
||||
|
||||
val env: Flag[Env.Value] =
|
||||
flag("env", Env.dev, "One of: testbox, dev, staging, prod")
|
||||
|
||||
val twemcacheDest: Flag[String] =
|
||||
flag(
|
||||
"twemcacheDest",
|
||||
"/s/cache/tweetypie:twemcaches",
|
||||
"The Name for the tweetypie cache cluster."
|
||||
)
|
||||
|
||||
val deciderOverrides: Flag[Map[String, Boolean]] =
|
||||
flag(
|
||||
"deciderOverrides",
|
||||
Map.empty[String, Boolean],
|
||||
"Set deciders to constant values, overriding decider configuration files."
|
||||
)(
|
||||
// Unfortunately, the implicit Flaggable[Boolean] has a default
|
||||
// value and Flaggable.ofMap[K, V] requires that the implicit
|
||||
// Flaggable[V] not have a default. Even less fortunately, it
|
||||
// doesn't say why. We're stuck with this.
|
||||
Flaggable.ofMap(implicitly, Flaggable.mandatory(_.toBoolean))
|
||||
)
|
||||
|
||||
// "/decider.yml" comes from the resources included at
|
||||
// "tweetypie/server/config", so you should not normally need to
|
||||
// override this value. This flag is defined as a step toward making
|
||||
// our command-line usage more similar to the standard
|
||||
// twitter-server-internal flags.
|
||||
def deciderBase(): String =
|
||||
injector.instance[String](InjectFlags.named("decider.base"))
|
||||
|
||||
// Omitting a value for decider overlay flag causes the server to use
|
||||
// only the static decider.
|
||||
def deciderOverlay(): String =
|
||||
injector.instance[String](InjectFlags.named("decider.overlay"))
|
||||
|
||||
// Omitting a value for the VF decider overlay flag causes the server
|
||||
// to use only the static decider.
|
||||
val vfDeciderOverlay: Flag[String] =
|
||||
flag(
|
||||
"vf.decider.overlay",
|
||||
"The location of the overlay decider configuration for Visibility Filtering")
|
||||
|
||||
/**
|
||||
* Warmup Requests happen as part of the initialization process, before any real requests are
|
||||
* processed. This prevents real requests from ever being served from a competely cold state
|
||||
*/
|
||||
val enableWarmupRequests: Flag[Boolean] =
|
||||
flag(
|
||||
"enableWarmupRequests",
|
||||
true,
|
||||
"""| warms up Tweetypie service by generating random requests
|
||||
| to Tweetypie that are processed prior to the actual client requests """.stripMargin
|
||||
)
|
||||
|
||||
val grayListRateLimit: Flag[Double] =
|
||||
flag("graylistRateLimit", 5.0, "rate-limit for non-allowlisted clients")
|
||||
|
||||
val servicePort: Flag[InetSocketAddress] =
|
||||
flag("service.port", "port for tweet-service thrift interface")
|
||||
|
||||
val clientId: Flag[String] =
|
||||
flag("clientId", "tweetypie.staging", "clientId to send in requests")
|
||||
|
||||
val allowlist: Flag[Boolean] =
|
||||
flag("allowlist", true, "enforce client allowlist")
|
||||
|
||||
val clientHostStats: Flag[Boolean] =
|
||||
flag("clientHostStats", false, "enable per client host stats")
|
||||
|
||||
val withCache: Flag[Boolean] =
|
||||
flag("withCache", true, "if set to false, Tweetypie will launch without memcache")
|
||||
|
||||
/**
|
||||
* Make any [[ThriftEnum]] value parseable as a [[Flag]] value. This
|
||||
* will parse case-insensitive values that match the unqualified
|
||||
* names of the values of the enumeration, in the manner of
|
||||
* [[ThriftEnum]]'s `valueOf` method.
|
||||
*
|
||||
* Consider a [[ThriftEnum]] generated from the following Thrift IDL snippet:
|
||||
*
|
||||
* {{{
|
||||
* enum Priority {
|
||||
* Low = 1
|
||||
* Throttled = 2
|
||||
* High = 3
|
||||
* }
|
||||
* }}}
|
||||
*
|
||||
* To enable defining flags that specify one of these enum values:
|
||||
*
|
||||
* {{{
|
||||
* implicit val flaggablePriority: Flaggable[Priority] = flaggableThriftEnum(Priority)
|
||||
* }}}
|
||||
*
|
||||
* In this example, the enumeration value `Priority.Low` can be
|
||||
* represented as the string "Low", "low", or "LOW".
|
||||
*/
|
||||
def flaggableThriftEnum[T <: ThriftEnum: ClassTag](enum: ThriftEnumObject[T]): Flaggable[T] =
|
||||
Flaggable.mandatory[T] { stringValue: String =>
|
||||
enum
|
||||
.valueOf(stringValue)
|
||||
.getOrElse {
|
||||
val validValues = enum.list.map(_.name).mkString(", ")
|
||||
throw new IllegalArgumentException(
|
||||
s"Invalid value ${stringValue}. Valid values include: ${validValues}"
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
implicit val flaggablePriority: Flaggable[Priority] = flaggableThriftEnum(Priority)
|
||||
|
||||
val backgroundIndexingPriority: Flag[Priority] =
|
||||
flag(
|
||||
"backgroundIndexingPriority",
|
||||
Priority.Low,
|
||||
"specifies the queue to use for \"background\" tflock operations, such as removing edges " +
|
||||
"for deleted Tweets. This exists for testing scenarios, when it is useful to see the " +
|
||||
"effects of background indexing operations sooner. In production, this should always be " +
|
||||
"set to \"low\" (the default)."
|
||||
)
|
||||
|
||||
val tflockPageSize: Flag[Int] =
|
||||
flag("tflockPageSize", 1000, "Number of items to return in each page when querying tflock")
|
||||
|
||||
val enableInProcessCache: Flag[Boolean] =
|
||||
flag(
|
||||
"enableInProcessCache",
|
||||
true,
|
||||
"if set to false, Tweetypie will not use the in-process cache"
|
||||
)
|
||||
|
||||
val inProcessCacheSize: Flag[Int] =
|
||||
flag("inProcessCacheSize", 1700, "maximum items in in-process cache")
|
||||
|
||||
val inProcessCacheTtlMs: Flag[Int] =
|
||||
flag("inProcessCacheTtlMs", 10000, "milliseconds that hot keys are stored in memory")
|
||||
|
||||
val memcachePendingRequestLimit: Flag[Int] =
|
||||
flag(
|
||||
"memcachePendingRequestLimit",
|
||||
100,
|
||||
"Number of requests that can be queued on a single memcache connection (4 per cache server)"
|
||||
)
|
||||
|
||||
val instanceId: Flag[Int] =
|
||||
flag(
|
||||
"configbus.instanceId",
|
||||
-1,
|
||||
"InstanceId of the tweetypie service instance for staged configuration distribution"
|
||||
)
|
||||
|
||||
val instanceCount: Flag[Int] =
|
||||
flag(
|
||||
"configbus.instanceCount",
|
||||
-1,
|
||||
"Total number of tweetypie service instances for staged configuration distribution"
|
||||
)
|
||||
|
||||
def serviceIdentifier(): ServiceIdentifier =
|
||||
injector.instance[ServiceIdentifier]
|
||||
|
||||
val enableReplication: Flag[Boolean] =
|
||||
flag(
|
||||
"enableReplication",
|
||||
true,
|
||||
"Enable replication of reads (configurable via tweetypie_replicate_reads decider) and writes (100%) via DRPC"
|
||||
)
|
||||
|
||||
val simulateDeferredrpcCallbacks: Flag[Boolean] =
|
||||
flag(
|
||||
"simulateDeferredrpcCallbacks",
|
||||
false,
|
||||
"""|For async write path, call back into current instance instead of via DRPC.
|
||||
|This is used for test and devel instances so we can ensure the test traffic
|
||||
|is going to the test instance.""".stripMargin
|
||||
)
|
||||
|
||||
val shortCircuitLikelyPartialTweetReadsMs: Flag[Int] =
|
||||
flag(
|
||||
"shortCircuitLikelyPartialTweetReadsMs",
|
||||
1500,
|
||||
"""|Specifies a number of milliseconds before which we will short-circuit likely
|
||||
|partial reads from MH and return a NotFound tweet response state. After
|
||||
|experimenting we went with 1500 ms.""".stripMargin
|
||||
)
|
||||
|
||||
val stringCenterProjects: Flag[Seq[String]] =
|
||||
flag(
|
||||
"stringcenter.projects",
|
||||
Seq.empty[String],
|
||||
"String Center project names, comma separated")(Flaggable.ofSeq(Flaggable.ofString))
|
||||
|
||||
val languagesConfig: Flag[String] =
|
||||
flag("international.languages", "Supported languages config file")
|
||||
}
|
||||
|
||||
class TweetypieMain extends StratoFedServer {
|
||||
override def dest: String = "/s/tweetypie/tweetypie:federated"
|
||||
|
||||
val tweetServiceFlags: TweetServiceFlags = new TweetServiceFlags(flag, injector)
|
||||
|
||||
// display all the registered HttpMuxer handlers
|
||||
HttpMuxer.addHandler("", new IndexHandler)
|
||||
|
||||
private[this] lazy val serverBuilder = {
|
||||
val settings = new TweetServiceSettings(tweetServiceFlags)
|
||||
val serverBuilder = new TweetServerBuilder(settings)
|
||||
|
||||
val mtlsSessionTrackerFilter =
|
||||
new MtlsServerSessionTrackerFilter[Array[Byte], Array[Byte]](statsReceiver)
|
||||
|
||||
val mtlsTrackedService = mtlsSessionTrackerFilter.andThen(ActivityService(serverBuilder.build))
|
||||
|
||||
val thriftMuxServer = ThriftMux.server
|
||||
// by default, finagle logs exceptions to chickadee, which is deprecated and
|
||||
// basically unused. to avoid wasted overhead, we explicitly disable the reporter.
|
||||
.configured(Reporter(NullReporterFactory))
|
||||
.withLabel("tweetypie")
|
||||
.withMutualTls(tweetServiceFlags.serviceIdentifier())
|
||||
.withOpportunisticTls(OpportunisticTls.Required)
|
||||
.configured(Thrift.param.ServiceClass(Some(classOf[ThriftTweetService])))
|
||||
.serve(tweetServiceFlags.servicePort(), mtlsTrackedService)
|
||||
|
||||
closeOnExit(thriftMuxServer)
|
||||
await(thriftMuxServer)
|
||||
|
||||
serverBuilder
|
||||
}
|
||||
|
||||
override def configureRefCatalog(
|
||||
catalog: Ref[Catalog[StratoFed.Column]]
|
||||
): Ref[Catalog[StratoFed.Column]] =
|
||||
catalog
|
||||
.join {
|
||||
Ref(
|
||||
serverBuilder.stratoTweetService.flatMap { tweetService =>
|
||||
StratoCatalogBuilder.catalog(
|
||||
tweetService,
|
||||
serverBuilder.backendClients.stratoserverClient,
|
||||
serverBuilder.backendClients.gizmoduck.getById,
|
||||
serverBuilder.backendClients.callbackPromotedContentLogger,
|
||||
statsReceiver,
|
||||
serverBuilder.deciderGates.enableCommunityTweetCreates,
|
||||
)
|
||||
}
|
||||
)
|
||||
}
|
||||
.map { case (l, r) => l ++ r }
|
||||
|
||||
override def configureWarmer(warmer: Warmer): Unit = {
|
||||
new TweetServiceSettings(tweetServiceFlags).warmupRequestsSettings.foreach { warmupSettings =>
|
||||
warmer.add(
|
||||
"tweetypie strato catalog",
|
||||
() => StratoCatalogWarmups.warmup(warmupSettings, composedOps)
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object Main extends TweetypieMain
|
@ -0,0 +1,62 @@
|
||||
package com.twitter.tweetypie
|
||||
package config
|
||||
|
||||
import com.twitter.io.Buf
|
||||
import com.twitter.finagle.{Service, SimpleFilter}
|
||||
import com.twitter.finagle.memcached.protocol._
|
||||
|
||||
class MemcacheExceptionLoggingFilter extends SimpleFilter[Command, Response] {
|
||||
// Using a custom logger name so that we can target logging rules specifically
|
||||
// for memcache excpetion logging.
|
||||
val logger: Logger = Logger(getClass)
|
||||
|
||||
def apply(command: Command, service: Service[Command, Response]): Future[Response] = {
|
||||
service(command).respond {
|
||||
case Return(Error(e)) =>
|
||||
log(command, e)
|
||||
case Return(ValuesAndErrors(_, errors)) if errors.nonEmpty =>
|
||||
errors.foreach {
|
||||
case (Buf.Utf8(keyStr), e) =>
|
||||
log(command.name, keyStr, e)
|
||||
}
|
||||
case Throw(e) =>
|
||||
log(command, e)
|
||||
|
||||
case _ =>
|
||||
}
|
||||
}
|
||||
|
||||
private def log(command: Command, e: Throwable): Unit = {
|
||||
log(command.name, getKey(command), e)
|
||||
}
|
||||
|
||||
private def log(commandName: String, keyStr: String, e: Throwable): Unit = {
|
||||
logger.debug(
|
||||
s"CACHE_EXCEPTION command: ${commandName} key: ${keyStr} exception: ${e.getClass.getName}",
|
||||
e,
|
||||
)
|
||||
}
|
||||
|
||||
private def getKey(command: Command): String = command match {
|
||||
case Get(keys) => toKeyStr(keys)
|
||||
case Gets(keys) => toKeyStr(keys)
|
||||
|
||||
case Set(Buf.Utf8(key), _, _, _) => key
|
||||
case Add(Buf.Utf8(key), _, _, _) => key
|
||||
case Cas(Buf.Utf8(key), _, _, _, _) => key
|
||||
case Delete(Buf.Utf8(key)) => key
|
||||
case Replace(Buf.Utf8(key), _, _, _) => key
|
||||
case Append(Buf.Utf8(key), _, _, _) => key
|
||||
case Prepend(Buf.Utf8(key), _, _, _) => key
|
||||
|
||||
case Incr(Buf.Utf8(key), _) => key
|
||||
case Decr(Buf.Utf8(key), _) => key
|
||||
case Stats(keys) => toKeyStr(keys)
|
||||
case Quit() => "quit"
|
||||
case Upsert(Buf.Utf8(key), _, _, _, _) => key
|
||||
case Getv(keys) => toKeyStr(keys)
|
||||
}
|
||||
|
||||
private def toKeyStr(keys: Seq[Buf]): String =
|
||||
keys.map { case Buf.Utf8(key) => key }.mkString(",")
|
||||
}
|
@ -0,0 +1,15 @@
|
||||
package com.twitter.tweetypie.config
|
||||
|
||||
import com.twitter.config.yaml.YamlMap
|
||||
import com.twitter.tweetypie.serverutil.PartnerMedia
|
||||
import scala.util.matching.Regex
|
||||
|
||||
/**
|
||||
* Helpers for loading resources bundled with Tweetypie. We load them
|
||||
* through this API in order to be able to unit test the resource
|
||||
* loading code.
|
||||
*/
|
||||
object Resources {
|
||||
def loadPartnerMediaRegexes(): Seq[Regex] =
|
||||
PartnerMedia.load(YamlMap.load("/partner_media.yml"))
|
||||
}
|
@ -0,0 +1,102 @@
|
||||
package com.twitter.tweetypie.config
|
||||
|
||||
import com.twitter.servo.cache.{Cache, Cached, CachedValue, CachedValueStatus}
|
||||
import com.twitter.servo.util.Scribe
|
||||
import com.twitter.tweetypie.TweetId
|
||||
import com.twitter.tweetypie.repository.TweetKey
|
||||
import com.twitter.tweetypie.serverutil.logcachewrites.WriteLoggingCache
|
||||
import com.twitter.snowflake.id.SnowflakeId
|
||||
import com.twitter.tweetypie.thriftscala.{CachedTweet, ComposerSource, TweetCacheWrite}
|
||||
import com.twitter.util.Time
|
||||
|
||||
class ScribeTweetCacheWrites(
|
||||
val underlyingCache: Cache[TweetKey, Cached[CachedTweet]],
|
||||
logYoungTweetCacheWrites: TweetId => Boolean,
|
||||
logTweetCacheWrites: TweetId => Boolean)
|
||||
extends WriteLoggingCache[TweetKey, Cached[CachedTweet]] {
|
||||
private[this] lazy val scribe = Scribe(TweetCacheWrite, "tweetypie_tweet_cache_writes")
|
||||
|
||||
private[this] def mkTweetCacheWrite(
|
||||
id: Long,
|
||||
action: String,
|
||||
cachedValue: CachedValue,
|
||||
cachedTweet: Option[CachedTweet] = None
|
||||
): TweetCacheWrite = {
|
||||
/*
|
||||
* If the Tweet id is a Snowflake id, calculate the offset since Tweet creation.
|
||||
* If it is not a Snowflake id, then the offset should be 0. See [[TweetCacheWrite]]'s Thrift
|
||||
* documentation for more details.
|
||||
*/
|
||||
val timestampOffset =
|
||||
if (SnowflakeId.isSnowflakeId(id)) {
|
||||
SnowflakeId(id).unixTimeMillis.asLong
|
||||
} else {
|
||||
0
|
||||
}
|
||||
|
||||
TweetCacheWrite(
|
||||
tweetId = id,
|
||||
timestamp = Time.now.inMilliseconds - timestampOffset,
|
||||
action = action,
|
||||
cachedValue = cachedValue,
|
||||
cachedTweet = cachedTweet
|
||||
)
|
||||
}
|
||||
|
||||
/**
|
||||
* Scribe a TweetCacheWrite record to tweetypie_tweet_cache_writes. We scribe the
|
||||
* messages instead of writing them to the regular log file because the
|
||||
* primary use of this logging is to get a record over time of the cache
|
||||
* actions that affected a tweet, so we need a durable log that we can
|
||||
* aggregate.
|
||||
*/
|
||||
override def log(action: String, k: TweetKey, v: Option[Cached[CachedTweet]]): Unit =
|
||||
v match {
|
||||
case Some(cachedTweet) => {
|
||||
val cachedValue = CachedValue(
|
||||
status = cachedTweet.status,
|
||||
cachedAtMsec = cachedTweet.cachedAt.inMilliseconds,
|
||||
readThroughAtMsec = cachedTweet.readThroughAt.map(_.inMilliseconds),
|
||||
writtenThroughAtMsec = cachedTweet.writtenThroughAt.map(_.inMilliseconds),
|
||||
doNotCacheUntilMsec = cachedTweet.doNotCacheUntil.map(_.inMilliseconds),
|
||||
)
|
||||
scribe(mkTweetCacheWrite(k.id, action, cachedValue, cachedTweet.value))
|
||||
}
|
||||
// `v` is only None if the action is a "delete" so set CachedValue with a status `Deleted`
|
||||
case None => {
|
||||
val cachedValue =
|
||||
CachedValue(status = CachedValueStatus.Deleted, cachedAtMsec = Time.now.inMilliseconds)
|
||||
scribe(mkTweetCacheWrite(k.id, action, cachedValue))
|
||||
}
|
||||
}
|
||||
|
||||
private[this] val YoungTweetThresholdMs = 3600 * 1000
|
||||
|
||||
private[this] def isYoungTweet(tweetId: TweetId): Boolean =
|
||||
(SnowflakeId.isSnowflakeId(tweetId) &&
|
||||
((Time.now.inMilliseconds - SnowflakeId(tweetId).unixTimeMillis.asLong) <=
|
||||
YoungTweetThresholdMs))
|
||||
|
||||
/**
|
||||
* Select all tweets for which the log_tweet_cache_writes decider returns
|
||||
* true and "young" tweets for which the log_young_tweet_cache_writes decider
|
||||
* returns true.
|
||||
*/
|
||||
override def selectKey(k: TweetKey): Boolean =
|
||||
// When the tweet is young, we log it if it passes either decider. This is
|
||||
// because the deciders will (by design) select a different subset of
|
||||
// tweets. We do this so that we have a full record for all tweets for which
|
||||
// log_tweet_cache_writes is on, but also cast a wider net for tweets that
|
||||
// are more likely to be affected by replication lag, race conditions
|
||||
// between different writes, or other consistency issues
|
||||
logTweetCacheWrites(k.id) || (isYoungTweet(k.id) && logYoungTweetCacheWrites(k.id))
|
||||
|
||||
/**
|
||||
* Log newscamera tweets as well as any tweets for which selectKey returns
|
||||
* true. Note that for newscamera tweets, we will possibly miss "delete"
|
||||
* actions since those do not have access to the value, and so do not call
|
||||
* this method.
|
||||
*/
|
||||
override def select(k: TweetKey, v: Cached[CachedTweet]): Boolean =
|
||||
v.value.exists(_.tweet.composerSource.contains(ComposerSource.Camera)) || selectKey(k)
|
||||
}
|
@ -0,0 +1,300 @@
|
||||
package com.twitter.tweetypie
|
||||
package config
|
||||
|
||||
import com.twitter.featureswitches.v2.FeatureSwitches
|
||||
import com.twitter.stitch.repo.Repo
|
||||
import com.twitter.tweetypie.backends.LimiterService.Feature
|
||||
import com.twitter.tweetypie.handler._
|
||||
import com.twitter.tweetypie.jiminy.tweetypie.NudgeBuilder
|
||||
import com.twitter.tweetypie.repository.RelationshipKey
|
||||
import com.twitter.tweetypie.store.TotalTweetStore
|
||||
import com.twitter.tweetypie.thriftscala._
|
||||
import com.twitter.tweetypie.tweettext.TweetText
|
||||
import com.twitter.visibility.common.TrustedFriendsSource
|
||||
import com.twitter.visibility.common.UserRelationshipSource
|
||||
import com.twitter.visibility.writer.interfaces.tweets.TweetWriteEnforcementLibrary
|
||||
|
||||
trait TweetBuilders {
|
||||
val retweetBuilder: RetweetBuilder.Type
|
||||
val tweetBuilder: TweetBuilder.Type
|
||||
}
|
||||
|
||||
object TweetBuilders {
|
||||
|
||||
def validateCardRefAttachmentByUserAgentGate(
|
||||
android: Gate[Unit],
|
||||
nonAndroid: Gate[Unit]
|
||||
): Gate[Option[String]] =
|
||||
Gate[Option[String]] { (userAgent: Option[String]) =>
|
||||
if (userAgent.exists(_.startsWith("TwitterAndroid"))) {
|
||||
android()
|
||||
} else {
|
||||
nonAndroid()
|
||||
}
|
||||
}
|
||||
|
||||
def apply(
|
||||
settings: TweetServiceSettings,
|
||||
statsReceiver: StatsReceiver,
|
||||
deciderGates: TweetypieDeciderGates,
|
||||
featureSwitchesWithExperiments: FeatureSwitches,
|
||||
clients: BackendClients,
|
||||
caches: Caches,
|
||||
repos: LogicalRepositories,
|
||||
tweetStore: TotalTweetStore,
|
||||
hasMedia: Tweet => Boolean,
|
||||
unretweetEdits: TweetDeletePathHandler.UnretweetEdits,
|
||||
): TweetBuilders = {
|
||||
val urlShortener =
|
||||
UrlShortener.scribeMalware(clients.guano) {
|
||||
UrlShortener.fromTalon(clients.talon.shorten)
|
||||
}
|
||||
|
||||
val urlEntityBuilder = UrlEntityBuilder.fromShortener(urlShortener)
|
||||
|
||||
val geoBuilder =
|
||||
GeoBuilder(
|
||||
repos.placeRepo,
|
||||
ReverseGeocoder.fromGeoduck(clients.geoduckGeohashLocate),
|
||||
statsReceiver.scope("geo_builder")
|
||||
)
|
||||
|
||||
val replyCardUsersFinder: CardUsersFinder.Type = CardUsersFinder(repos.cardUsersRepo)
|
||||
|
||||
val selfThreadBuilder = SelfThreadBuilder(statsReceiver.scope("self_thread_builder"))
|
||||
|
||||
val replyBuilder =
|
||||
ReplyBuilder(
|
||||
repos.userIdentityRepo,
|
||||
repos.optionalTweetRepo,
|
||||
replyCardUsersFinder,
|
||||
selfThreadBuilder,
|
||||
repos.relationshipRepo,
|
||||
repos.unmentionedEntitiesRepo,
|
||||
deciderGates.enableRemoveUnmentionedImplicitMentions,
|
||||
statsReceiver.scope("reply_builder"),
|
||||
TweetText.MaxMentions
|
||||
)
|
||||
|
||||
val mediaBuilder =
|
||||
MediaBuilder(
|
||||
clients.mediaClient.processMedia,
|
||||
CreateMediaTco(urlShortener),
|
||||
statsReceiver.scope("media_builder")
|
||||
)
|
||||
|
||||
val validateAttachments =
|
||||
AttachmentBuilder.validateAttachments(
|
||||
statsReceiver,
|
||||
validateCardRefAttachmentByUserAgentGate(
|
||||
android = deciderGates.validateCardRefAttachmentAndroid,
|
||||
nonAndroid = deciderGates.validateCardRefAttachmentNonAndroid
|
||||
)
|
||||
)
|
||||
|
||||
val attachmentBuilder =
|
||||
AttachmentBuilder(
|
||||
repos.optionalTweetRepo,
|
||||
urlShortener,
|
||||
validateAttachments,
|
||||
statsReceiver.scope("attachment_builder"),
|
||||
deciderGates.denyNonTweetPermalinks
|
||||
)
|
||||
|
||||
val validatePostTweetRequest: FutureEffect[PostTweetRequest] =
|
||||
TweetBuilder.validateAdditionalFields[PostTweetRequest]
|
||||
|
||||
val validateRetweetRequest =
|
||||
TweetBuilder.validateAdditionalFields[RetweetRequest]
|
||||
|
||||
val tweetIdGenerator =
|
||||
() => clients.snowflakeClient.get()
|
||||
|
||||
val retweetSpamChecker =
|
||||
Spam.gated(deciderGates.checkSpamOnRetweet) {
|
||||
Spam.allowOnException(
|
||||
ScarecrowRetweetSpamChecker(
|
||||
statsReceiver.scope("retweet_builder").scope("spam"),
|
||||
repos.retweetSpamCheckRepo
|
||||
)
|
||||
)
|
||||
}
|
||||
|
||||
val tweetSpamChecker =
|
||||
Spam.gated(deciderGates.checkSpamOnTweet) {
|
||||
Spam.allowOnException(
|
||||
ScarecrowTweetSpamChecker.fromSpamCheckRepository(
|
||||
statsReceiver.scope("tweet_builder").scope("spam"),
|
||||
repos.tweetSpamCheckRepo
|
||||
)
|
||||
)
|
||||
}
|
||||
|
||||
val duplicateTweetFinder =
|
||||
DuplicateTweetFinder(
|
||||
settings = settings.duplicateTweetFinderSettings,
|
||||
tweetSource = DuplicateTweetFinder.TweetSource.fromServices(
|
||||
tweetRepo = repos.optionalTweetRepo,
|
||||
getStatusTimeline = clients.timelineService.getStatusTimeline
|
||||
)
|
||||
)
|
||||
|
||||
val validateUpdateRateLimit =
|
||||
RateLimitChecker.validate(
|
||||
clients.limiterService.hasRemaining(Feature.Updates),
|
||||
statsReceiver.scope("rate_limits", Feature.Updates.name),
|
||||
deciderGates.rateLimitByLimiterService
|
||||
)
|
||||
|
||||
val tweetBuilderStats = statsReceiver.scope("tweet_builder")
|
||||
|
||||
val updateUserCounts =
|
||||
TweetBuilder.updateUserCounts(hasMedia)
|
||||
|
||||
val filterInvalidData =
|
||||
TweetBuilder.filterInvalidData(
|
||||
validateTweetMediaTags = TweetBuilder.validateTweetMediaTags(
|
||||
tweetBuilderStats.scope("media_tags_filter"),
|
||||
RateLimitChecker.getMaxMediaTags(
|
||||
clients.limiterService.minRemaining(Feature.MediaTagCreate),
|
||||
TweetBuilder.MaxMediaTagCount
|
||||
),
|
||||
repos.optionalUserRepo
|
||||
),
|
||||
cardReferenceBuilder = TweetBuilder.cardReferenceBuilder(
|
||||
CardReferenceValidationHandler(clients.expandodo.checkAttachmentEligibility),
|
||||
urlShortener
|
||||
)
|
||||
)
|
||||
|
||||
val rateLimitFailures =
|
||||
PostTweet.RateLimitFailures(
|
||||
validateLimit = RateLimitChecker.validate(
|
||||
clients.limiterService.hasRemaining(Feature.TweetCreateFailure),
|
||||
statsReceiver.scope("rate_limits", Feature.TweetCreateFailure.name),
|
||||
deciderGates.rateLimitTweetCreationFailure
|
||||
),
|
||||
clients.limiterService.incrementByOne(Feature.Updates),
|
||||
clients.limiterService.incrementByOne(Feature.TweetCreateFailure)
|
||||
)
|
||||
|
||||
val countFailures =
|
||||
PostTweet.CountFailures[TweetBuilderResult](statsReceiver)
|
||||
|
||||
val tweetBuilderFilter: PostTweet.Filter[TweetBuilderResult] =
|
||||
rateLimitFailures.andThen(countFailures)
|
||||
|
||||
val conversationControlBuilder = ConversationControlBuilder.fromUserIdentityRepo(
|
||||
statsReceiver = statsReceiver.scope("conversation_control_builder"),
|
||||
userIdentityRepo = repos.userIdentityRepo
|
||||
)
|
||||
|
||||
val conversationControlValidator = ConversationControlBuilder.Validate(
|
||||
useFeatureSwitchResults = deciderGates.useConversationControlFeatureSwitchResults,
|
||||
statsReceiver = statsReceiver
|
||||
)
|
||||
|
||||
val communitiesValidator: CommunitiesValidator.Type = CommunitiesValidator()
|
||||
|
||||
val collabControlBuilder: CollabControlBuilder.Type = CollabControlBuilder()
|
||||
|
||||
val userRelationshipSource = UserRelationshipSource.fromRepo(
|
||||
Repo[UserRelationshipSource.Key, Unit, Boolean] { (key, _) =>
|
||||
repos.relationshipRepo(
|
||||
RelationshipKey(key.subjectId, key.objectId, key.relationship)
|
||||
)
|
||||
}
|
||||
)
|
||||
|
||||
val trustedFriendsSource =
|
||||
TrustedFriendsSource.fromStrato(clients.stratoserverClient, statsReceiver)
|
||||
|
||||
val validateTweetWrite = TweetWriteValidator(
|
||||
convoCtlRepo = repos.conversationControlRepo,
|
||||
tweetWriteEnforcementLibrary = TweetWriteEnforcementLibrary(
|
||||
userRelationshipSource,
|
||||
trustedFriendsSource,
|
||||
repos.userIsInvitedToConversationRepo,
|
||||
repos.stratoSuperFollowEligibleRepo,
|
||||
repos.tweetRepo,
|
||||
statsReceiver.scope("tweet_write_enforcement_library")
|
||||
),
|
||||
enableExclusiveTweetControlValidation = deciderGates.enableExclusiveTweetControlValidation,
|
||||
enableTrustedFriendsControlValidation = deciderGates.enableTrustedFriendsControlValidation,
|
||||
enableStaleTweetValidation = deciderGates.enableStaleTweetValidation
|
||||
)
|
||||
|
||||
val nudgeBuilder = NudgeBuilder(
|
||||
clients.stratoserverClient,
|
||||
deciderGates.jiminyDarkRequests,
|
||||
statsReceiver.scope("nudge_builder")
|
||||
)
|
||||
|
||||
val editControlBuilder = EditControlBuilder(
|
||||
tweetRepo = repos.tweetRepo,
|
||||
card2Repo = repos.card2Repo,
|
||||
promotedTweetRepo = repos.stratoPromotedTweetRepo,
|
||||
subscriptionVerificationRepo = repos.stratoSubscriptionVerificationRepo,
|
||||
disablePromotedTweetEdit = deciderGates.disablePromotedTweetEdit,
|
||||
checkTwitterBlueSubscription = deciderGates.checkTwitterBlueSubscriptionForEdit,
|
||||
setEditWindowToSixtyMinutes = deciderGates.setEditTimeWindowToSixtyMinutes,
|
||||
stats = statsReceiver,
|
||||
)
|
||||
|
||||
val validateEdit = EditValidator(repos.optionalTweetRepo)
|
||||
|
||||
// TweetBuilders builds two distinct TweetBuilders (Tweet and Retweet builders).
|
||||
new TweetBuilders {
|
||||
val tweetBuilder: TweetBuilder.Type =
|
||||
tweetBuilderFilter[PostTweetRequest](
|
||||
TweetBuilder(
|
||||
stats = tweetBuilderStats,
|
||||
validateRequest = validatePostTweetRequest,
|
||||
validateEdit = validateEdit,
|
||||
validateUpdateRateLimit = validateUpdateRateLimit,
|
||||
tweetIdGenerator = tweetIdGenerator,
|
||||
userRepo = repos.userRepo,
|
||||
deviceSourceRepo = repos.deviceSourceRepo,
|
||||
communityMembershipRepo = repos.stratoCommunityMembershipRepo,
|
||||
communityAccessRepo = repos.stratoCommunityAccessRepo,
|
||||
urlShortener = urlShortener,
|
||||
urlEntityBuilder = urlEntityBuilder,
|
||||
geoBuilder = geoBuilder,
|
||||
replyBuilder = replyBuilder,
|
||||
mediaBuilder = mediaBuilder,
|
||||
attachmentBuilder = attachmentBuilder,
|
||||
duplicateTweetFinder = duplicateTweetFinder,
|
||||
spamChecker = tweetSpamChecker,
|
||||
filterInvalidData = filterInvalidData,
|
||||
updateUserCounts = updateUserCounts,
|
||||
validateConversationControl = conversationControlValidator,
|
||||
conversationControlBuilder = conversationControlBuilder,
|
||||
validateTweetWrite = validateTweetWrite,
|
||||
nudgeBuilder = nudgeBuilder,
|
||||
communitiesValidator = communitiesValidator,
|
||||
collabControlBuilder = collabControlBuilder,
|
||||
editControlBuilder = editControlBuilder,
|
||||
featureSwitches = featureSwitchesWithExperiments,
|
||||
)
|
||||
)
|
||||
|
||||
val retweetBuilder: RetweetBuilder.Type =
|
||||
tweetBuilderFilter[RetweetRequest](
|
||||
RetweetBuilder(
|
||||
validateRequest = validateRetweetRequest,
|
||||
tweetIdGenerator = tweetIdGenerator,
|
||||
tweetRepo = repos.tweetRepo,
|
||||
userRepo = repos.userRepo,
|
||||
tflock = clients.tflockWriteClient,
|
||||
deviceSourceRepo = repos.deviceSourceRepo,
|
||||
validateUpdateRateLimit = validateUpdateRateLimit,
|
||||
spamChecker = retweetSpamChecker,
|
||||
updateUserCounts = updateUserCounts,
|
||||
superFollowRelationsRepo = repos.stratoSuperFollowRelationsRepo,
|
||||
unretweetEdits = unretweetEdits,
|
||||
setEditWindowToSixtyMinutes = deciderGates.setEditTimeWindowToSixtyMinutes
|
||||
)
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,341 @@
|
||||
package com.twitter.tweetypie
|
||||
package config
|
||||
|
||||
import com.twitter.featureswitches.v2.FeatureSwitches
|
||||
import com.twitter.servo.cache.Cached
|
||||
import com.twitter.servo.cache.LockingCache
|
||||
import com.twitter.servo.util.ExceptionCategorizer
|
||||
import com.twitter.servo.util.ExceptionCounter
|
||||
import com.twitter.servo.util.FutureEffect
|
||||
import com.twitter.servo.util.Scribe
|
||||
import com.twitter.stitch.NotFound
|
||||
import com.twitter.tweetypie.core.FilteredState
|
||||
import com.twitter.tweetypie.core.TweetData
|
||||
import com.twitter.tweetypie.core.ValueState
|
||||
import com.twitter.tweetypie.hydrator._
|
||||
import com.twitter.tweetypie.repository.TweetQuery
|
||||
import com.twitter.tweetypie.serverutil.{ExceptionCounter => TpExceptionCounter}
|
||||
import com.twitter.tweetypie.thriftscala._
|
||||
import com.twitter.tweetypie.client_id.ClientIdHelper
|
||||
|
||||
trait TweetHydrators {
|
||||
|
||||
/**
|
||||
* Hydrator that has all the Tweet hydrators (entire "pipeline") configured
|
||||
* and wired up.
|
||||
* This hydrator is used both on the read and write path and is
|
||||
* customized by different TweetQuery.Options.
|
||||
* Modifications are not automatically written back to cache.
|
||||
* `cacheChanges` must be used for that.
|
||||
*/
|
||||
def hydrator: TweetDataValueHydrator
|
||||
|
||||
/**
|
||||
* The `Effect` to use to write modified tweets back to cache.
|
||||
*/
|
||||
def cacheChangesEffect: Effect[ValueState[TweetData]]
|
||||
}
|
||||
|
||||
object TweetHydrators {
|
||||
|
||||
/**
|
||||
* Creates all the hydrators and calls TweetHydration to wire them up.
|
||||
*/
|
||||
def apply(
|
||||
stats: StatsReceiver,
|
||||
deciderGates: TweetypieDeciderGates,
|
||||
repos: LogicalRepositories,
|
||||
tweetDataCache: LockingCache[TweetId, Cached[TweetData]],
|
||||
hasMedia: Tweet => Boolean,
|
||||
featureSwitchesWithoutExperiments: FeatureSwitches,
|
||||
clientIdHelper: ClientIdHelper
|
||||
): TweetHydrators = {
|
||||
import repos._
|
||||
|
||||
val repairStats = stats.scope("repairs")
|
||||
val hydratorStats = stats.scope("hydrators")
|
||||
|
||||
def scoped[A](stats: StatsReceiver, name: String)(f: StatsReceiver => A): A = {
|
||||
val scopedStats = stats.scope(name)
|
||||
f(scopedStats)
|
||||
}
|
||||
|
||||
val isFailureException: Throwable => Boolean = {
|
||||
case _: FilteredState => false
|
||||
case NotFound => false
|
||||
case _ => true
|
||||
}
|
||||
|
||||
def hydratorExceptionCategorizer(failureScope: String) =
|
||||
ExceptionCategorizer.const("filtered").onlyIf(_.isInstanceOf[FilteredState]) ++
|
||||
ExceptionCategorizer.const("not_found").onlyIf(_ == NotFound) ++
|
||||
TpExceptionCounter.defaultCategorizer(failureScope).onlyIf(isFailureException)
|
||||
|
||||
val hydratorExceptionCounter: (StatsReceiver, String) => ExceptionCounter =
|
||||
(stats, scope) => TpExceptionCounter(stats, hydratorExceptionCategorizer(scope))
|
||||
|
||||
val tweetHydrator =
|
||||
TweetHydration(
|
||||
hydratorStats = hydratorStats,
|
||||
hydrateFeatureSwitchResults =
|
||||
FeatureSwitchResultsHydrator(featureSwitchesWithoutExperiments, clientIdHelper),
|
||||
hydrateMentions = MentionEntitiesHydrator
|
||||
.once(MentionEntityHydrator(userIdentityRepo))
|
||||
.observe(hydratorStats.scope("mentions"), hydratorExceptionCounter),
|
||||
hydrateLanguage = LanguageHydrator(languageRepo)
|
||||
.observe(hydratorStats.scope("language"), hydratorExceptionCounter),
|
||||
hydrateUrls = scoped(hydratorStats, "url") { stats =>
|
||||
UrlEntitiesHydrator
|
||||
.once(UrlEntityHydrator(urlRepo, stats))
|
||||
.observe(stats, hydratorExceptionCounter)
|
||||
},
|
||||
hydrateQuotedTweetRef = QuotedTweetRefHydrator
|
||||
.once(
|
||||
QuotedTweetRefHydrator(tweetRepo)
|
||||
)
|
||||
.observe(hydratorStats.scope("quoted_tweet_ref"), hydratorExceptionCounter),
|
||||
hydrateQuotedTweetRefUrls = QuotedTweetRefUrlsHydrator(userIdentityRepo)
|
||||
.observe(hydratorStats.scope("quoted_tweet_ref_urls"), hydratorExceptionCounter),
|
||||
hydrateMediaCacheable = MediaEntitiesHydrator.Cacheable
|
||||
.once(
|
||||
MediaEntityHydrator.Cacheable(
|
||||
hydrateMediaUrls = MediaUrlFieldsHydrator()
|
||||
.observe(hydratorStats.scope("media_urls"), hydratorExceptionCounter),
|
||||
hydrateMediaIsProtected = MediaIsProtectedHydrator(userProtectionRepo)
|
||||
.observe(hydratorStats.scope("media_is_protected"), hydratorExceptionCounter)
|
||||
)
|
||||
)
|
||||
.observe(hydratorStats.scope("media_cacheable"), hydratorExceptionCounter)
|
||||
.ifEnabled(deciderGates.hydrateMedia),
|
||||
hydrateReplyScreenName = ReplyScreenNameHydrator
|
||||
.once(ReplyScreenNameHydrator(userIdentityRepo))
|
||||
.observe(hydratorStats.scope("in_reply_to_screen_name"), hydratorExceptionCounter),
|
||||
hydrateConvoId = ConversationIdHydrator(conversationIdRepo)
|
||||
.observe(hydratorStats.scope("conversation_id"), hydratorExceptionCounter),
|
||||
hydratePerspective = // Don't cache with the tweet because it depends on the request
|
||||
PerspectiveHydrator(
|
||||
repo = perspectiveRepo,
|
||||
shouldHydrateBookmarksPerspective = deciderGates.hydrateBookmarksPerspective,
|
||||
stats = hydratorStats.scope("perspective_by_safety_label")
|
||||
).observe(hydratorStats.scope("perspective"), hydratorExceptionCounter)
|
||||
.ifEnabled(deciderGates.hydratePerspectives),
|
||||
hydrateEditPerspective = EditPerspectiveHydrator(
|
||||
repo = perspectiveRepo,
|
||||
timelinesGate = deciderGates.hydratePerspectivesEditsForTimelines,
|
||||
tweetDetailsGate = deciderGates.hydratePerspectivesEditsForTweetDetail,
|
||||
otherSafetyLevelsGate = deciderGates.hydratePerspectivesEditsForOtherSafetyLevels,
|
||||
bookmarksGate = deciderGates.hydrateBookmarksPerspective,
|
||||
stats = hydratorStats
|
||||
).observe(hydratorStats.scope("edit_perspective"), hydratorExceptionCounter),
|
||||
hydrateConversationMuted = // Don't cache because it depends on the request. If
|
||||
// possible, this hydrator should be in the same stage as
|
||||
// PerspectiveHydrator, so that the calls can be batched
|
||||
// together.
|
||||
ConversationMutedHydrator(conversationMutedRepo)
|
||||
.observe(hydratorStats.scope("conversation_muted"), hydratorExceptionCounter)
|
||||
.ifEnabled(deciderGates.hydrateConversationMuted),
|
||||
hydrateContributor = ContributorHydrator
|
||||
.once(ContributorHydrator(userIdentityRepo))
|
||||
.observe(hydratorStats.scope("contributors"), hydratorExceptionCounter),
|
||||
hydrateTakedowns = TakedownHydrator(takedownRepo)
|
||||
.observe(hydratorStats.scope("takedowns"), hydratorExceptionCounter),
|
||||
hydrateDirectedAt = scoped(hydratorStats, "directed_at") { stats =>
|
||||
DirectedAtHydrator
|
||||
.once(DirectedAtHydrator(userIdentityRepo, stats))
|
||||
.observe(stats, hydratorExceptionCounter)
|
||||
},
|
||||
hydrateGeoScrub = GeoScrubHydrator(
|
||||
geoScrubTimestampRepo,
|
||||
Scribe("test_tweetypie_read_time_geo_scrubs")
|
||||
.contramap[TweetId](_.toString)
|
||||
).observe(hydratorStats.scope("geo_scrub"), hydratorExceptionCounter),
|
||||
hydrateCacheableRepairs = ValueHydrator
|
||||
.fromMutation[Tweet, TweetQuery.Options](
|
||||
RepairMutation(
|
||||
repairStats.scope("on_read"),
|
||||
"created_at" ->
|
||||
new CreatedAtRepairer(Scribe("test_tweetypie_bad_created_at")),
|
||||
"retweet_media" -> RetweetMediaRepairer,
|
||||
"parent_status_id" -> RetweetParentStatusIdRepairer.tweetMutation,
|
||||
"visible_text_range" -> NegativeVisibleTextRangeRepairer.tweetMutation
|
||||
)
|
||||
)
|
||||
.lensed(TweetData.Lenses.tweet)
|
||||
.onlyIf((td, opts) => opts.cause.reading(td.tweet.id)),
|
||||
hydrateMediaUncacheable = MediaEntityHydrator
|
||||
.Uncacheable(
|
||||
hydrateMediaKey = MediaKeyHydrator()
|
||||
.observe(hydratorStats.scope("media_key"), hydratorExceptionCounter),
|
||||
hydrateMediaInfo = scoped(hydratorStats, "media_info") { stats =>
|
||||
MediaInfoHydrator(mediaMetadataRepo, stats)
|
||||
.observe(stats, hydratorExceptionCounter)
|
||||
}
|
||||
)
|
||||
.observe(hydratorStats.scope("media_uncacheable"), hydratorExceptionCounter)
|
||||
.liftSeq
|
||||
.ifEnabled(deciderGates.hydrateMedia),
|
||||
hydratePostCacheRepairs =
|
||||
// clean-up partially hydrated entities before any of the hydrators that look at
|
||||
// url and media entities run, so that they never see bad entities.
|
||||
ValueHydrator.fromMutation[TweetData, TweetQuery.Options](
|
||||
RepairMutation(
|
||||
repairStats.scope("on_read"),
|
||||
"partial_entity_cleanup" -> PartialEntityCleaner(repairStats),
|
||||
"strip_not_display_coords" -> StripHiddenGeoCoordinates
|
||||
).lensed(TweetData.Lenses.tweet)
|
||||
),
|
||||
hydrateTweetLegacyFormat = scoped(hydratorStats, "tweet_legacy_formatter") { stats =>
|
||||
TweetLegacyFormatter(stats)
|
||||
.observe(stats, hydratorExceptionCounter)
|
||||
.onlyIf((td, opts) => opts.cause.reading(td.tweet.id))
|
||||
},
|
||||
hydrateQuoteTweetVisibility = QuoteTweetVisibilityHydrator(quotedTweetVisibilityRepo)
|
||||
.observe(hydratorStats.scope("quote_tweet_visibility"), hydratorExceptionCounter),
|
||||
hydrateQuotedTweet = QuotedTweetHydrator(tweetResultRepo)
|
||||
.observe(hydratorStats.scope("quoted_tweet"), hydratorExceptionCounter),
|
||||
hydratePastedMedia =
|
||||
// Don't cache with the tweet because we want to automatically drop this media if
|
||||
// the referenced tweet is deleted or becomes non-public.
|
||||
PastedMediaHydrator(pastedMediaRepo)
|
||||
.observe(hydratorStats.scope("pasted_media"))
|
||||
.ifEnabled(deciderGates.hydratePastedMedia),
|
||||
hydrateMediaRefs = MediaRefsHydrator(
|
||||
optionalTweetRepo,
|
||||
deciderGates.mediaRefsHydratorIncludePastedMedia
|
||||
).observe(hydratorStats.scope("media_refs"))
|
||||
.ifEnabled(deciderGates.hydrateMediaRefs),
|
||||
hydrateMediaTags = // depends on AdditionalFieldsHydrator
|
||||
MediaTagsHydrator(userViewRepo)
|
||||
.observe(hydratorStats.scope("media_tags"), hydratorExceptionCounter)
|
||||
.ifEnabled(deciderGates.hydrateMediaTags),
|
||||
hydrateClassicCards = CardHydrator(cardRepo)
|
||||
.observe(hydratorStats.scope("cards"), hydratorExceptionCounter),
|
||||
hydrateCard2 = Card2Hydrator(card2Repo)
|
||||
.observe(hydratorStats.scope("card2")),
|
||||
hydrateContributorVisibility =
|
||||
// Filter out contributors field for all but the user who owns the tweet
|
||||
ContributorVisibilityFilter()
|
||||
.observe(hydratorStats.scope("contributor_visibility"), hydratorExceptionCounter),
|
||||
hydrateHasMedia =
|
||||
// Sets hasMedia. Comes after PastedMediaHydrator in order to include pasted
|
||||
// pics as well as other media & urls.
|
||||
HasMediaHydrator(hasMedia)
|
||||
.observe(hydratorStats.scope("has_media"), hydratorExceptionCounter)
|
||||
.ifEnabled(deciderGates.hydrateHasMedia),
|
||||
hydrateTweetCounts = // Don't cache counts with the tweet because it has its own cache with
|
||||
// a different TTL
|
||||
TweetCountsHydrator(tweetCountsRepo, deciderGates.hydrateBookmarksCount)
|
||||
.observe(hydratorStats.scope("tweet_counts"), hydratorExceptionCounter)
|
||||
.ifEnabled(deciderGates.hydrateCounts),
|
||||
hydratePreviousTweetCounts = // previous counts are not cached
|
||||
scoped(hydratorStats, "previous_counts") { stats =>
|
||||
PreviousTweetCountsHydrator(tweetCountsRepo, deciderGates.hydrateBookmarksCount)
|
||||
.observe(stats, hydratorExceptionCounter)
|
||||
.ifEnabled(deciderGates.hydratePreviousCounts)
|
||||
},
|
||||
hydratePlace =
|
||||
// Don't cache with the tweet because Place has its own tweetypie cache keyspace
|
||||
// with a different TTL, and it's more efficient to store separately.
|
||||
// See com.twitter.tweetypie.repository.PlaceKey
|
||||
PlaceHydrator(placeRepo)
|
||||
.observe(hydratorStats.scope("place"), hydratorExceptionCounter)
|
||||
.ifEnabled(deciderGates.hydratePlaces),
|
||||
hydrateDeviceSource = // Don't cache with the tweet because it has its own cache,
|
||||
// and it's more efficient to cache it separately
|
||||
DeviceSourceHydrator(deviceSourceRepo)
|
||||
.observe(hydratorStats.scope("device_source"), hydratorExceptionCounter)
|
||||
.ifEnabled(deciderGates.hydrateDeviceSources),
|
||||
hydrateProfileGeo =
|
||||
// Don't cache gnip profile geo as read request volume is expected to be low
|
||||
ProfileGeoHydrator(profileGeoRepo)
|
||||
.observe(hydratorStats.scope("profile_geo"), hydratorExceptionCounter)
|
||||
.ifEnabled(deciderGates.hydrateGnipProfileGeoEnrichment),
|
||||
hydrateSourceTweet = scoped(hydratorStats, "source_tweet") { stats =>
|
||||
SourceTweetHydrator(
|
||||
tweetResultRepo,
|
||||
stats,
|
||||
FutureEffect
|
||||
.inParallel(
|
||||
Scribe(DetachedRetweet, "tweetypie_detached_retweets"),
|
||||
Scribe(DetachedRetweet, "test_tweetypie_detached_retweets"),
|
||||
)
|
||||
).observe(stats, hydratorExceptionCounter)
|
||||
},
|
||||
hydrateIM1837State = IM1837FilterHydrator()
|
||||
.observe(hydratorStats.scope("im1837_filter"), hydratorExceptionCounter)
|
||||
.onlyIf { (_, ctx) =>
|
||||
ctx.opts.forExternalConsumption && ctx.opts.cause.reading(ctx.tweetId)
|
||||
},
|
||||
hydrateIM2884State = scoped(hydratorStats, "im2884_filter") { stats =>
|
||||
IM2884FilterHydrator(stats)
|
||||
.observe(stats, hydratorExceptionCounter)
|
||||
.onlyIf { (_, ctx) =>
|
||||
ctx.opts.forExternalConsumption && ctx.opts.cause.reading(ctx.tweetId)
|
||||
}
|
||||
},
|
||||
hydrateIM3433State = scoped(hydratorStats, "im3433_filter") { stats =>
|
||||
IM3433FilterHydrator(stats)
|
||||
.observe(stats, hydratorExceptionCounter)
|
||||
.onlyIf { (_, ctx) =>
|
||||
ctx.opts.forExternalConsumption && ctx.opts.cause.reading(ctx.tweetId)
|
||||
}
|
||||
},
|
||||
hydrateTweetAuthorVisibility = TweetAuthorVisibilityHydrator(userVisibilityRepo)
|
||||
.observe(hydratorStats.scope("tweet_author_visibility"), hydratorExceptionCounter)
|
||||
.onlyIf((_, ctx) => ctx.opts.cause.reading(ctx.tweetId)),
|
||||
hydrateReportedTweetVisibility = ReportedTweetFilter()
|
||||
.observe(hydratorStats.scope("reported_tweet_filter"), hydratorExceptionCounter),
|
||||
scrubSuperfluousUrlEntities = ValueHydrator
|
||||
.fromMutation[Tweet, TweetQuery.Options](SuperfluousUrlEntityScrubber.mutation)
|
||||
.lensed(TweetData.Lenses.tweet),
|
||||
copyFromSourceTweet = CopyFromSourceTweet.hydrator
|
||||
.observe(hydratorStats.scope("copy_from_source_tweet"), hydratorExceptionCounter),
|
||||
hydrateTweetVisibility = scoped(hydratorStats, "tweet_visibility") { stats =>
|
||||
TweetVisibilityHydrator(
|
||||
tweetVisibilityRepo,
|
||||
deciderGates.failClosedInVF,
|
||||
stats
|
||||
).observe(stats, hydratorExceptionCounter)
|
||||
},
|
||||
hydrateEscherbirdAnnotations = EscherbirdAnnotationHydrator(escherbirdAnnotationRepo)
|
||||
.observe(hydratorStats.scope("escherbird_annotations"), hydratorExceptionCounter)
|
||||
.ifEnabled(deciderGates.hydrateEscherbirdAnnotations),
|
||||
hydrateScrubEngagements = ScrubEngagementHydrator()
|
||||
.observe(hydratorStats.scope("scrub_engagements"), hydratorExceptionCounter)
|
||||
.ifEnabled(deciderGates.hydrateScrubEngagements),
|
||||
hydrateConversationControl = scoped(hydratorStats, "tweet_conversation_control") { stats =>
|
||||
ConversationControlHydrator(
|
||||
conversationControlRepo,
|
||||
deciderGates.disableInviteViaMention,
|
||||
stats
|
||||
).observe(stats, hydratorExceptionCounter)
|
||||
},
|
||||
hydrateEditControl = scoped(hydratorStats, "tweet_edit_control") { stats =>
|
||||
EditControlHydrator(
|
||||
tweetRepo,
|
||||
deciderGates.setEditTimeWindowToSixtyMinutes,
|
||||
stats
|
||||
).observe(stats, hydratorExceptionCounter)
|
||||
},
|
||||
hydrateUnmentionData = UnmentionDataHydrator(),
|
||||
hydrateNoteTweetSuffix = NoteTweetSuffixHydrator().observe(stats, hydratorExceptionCounter)
|
||||
)
|
||||
|
||||
new TweetHydrators {
|
||||
val hydrator: TweetDataValueHydrator =
|
||||
tweetHydrator.onlyIf { (tweetData, opts) =>
|
||||
// When the caller requests fetchStoredTweets and Tweets are fetched from Manhattan
|
||||
// irrespective of state, the stored data for some Tweets may be incomplete.
|
||||
// We skip the hydration of those Tweets.
|
||||
!opts.fetchStoredTweets ||
|
||||
tweetData.storedTweetResult.exists(_.canHydrate)
|
||||
}
|
||||
|
||||
val cacheChangesEffect: Effect[ValueState[TweetData]] =
|
||||
TweetHydration.cacheChanges(
|
||||
tweetDataCache,
|
||||
hydratorStats.scope("tweet_caching")
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,300 @@
|
||||
package com.twitter.tweetypie.config
|
||||
|
||||
import com.twitter.decider.Decider
|
||||
import com.twitter.decider.DeciderFactory
|
||||
import com.twitter.decider.LocalOverrides
|
||||
import com.twitter.featureswitches.v2.builder.FeatureSwitchesBuilder
|
||||
import com.twitter.finagle.filter.DarkTrafficFilter
|
||||
import com.twitter.finagle.stats.DefaultStatsReceiver
|
||||
import com.twitter.finagle.stats.NullStatsReceiver
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.finagle.thrift.Protocols
|
||||
import com.twitter.finagle.util.DefaultTimer
|
||||
import com.twitter.finagle.Filter
|
||||
import com.twitter.finagle.Service
|
||||
import com.twitter.finagle.SimpleFilter
|
||||
import com.twitter.quill.capture._
|
||||
import com.twitter.servo.util.MemoizingStatsReceiver
|
||||
import com.twitter.servo.util.WaitForServerSets
|
||||
import com.twitter.tweetypie.ThriftTweetService
|
||||
import com.twitter.tweetypie.client_id.ClientIdHelper
|
||||
import com.twitter.tweetypie.client_id.ConditionalServiceIdentifierStrategy
|
||||
import com.twitter.tweetypie.client_id.PreferForwardedServiceIdentifierForStrato
|
||||
import com.twitter.tweetypie.client_id.UseTransportServiceIdentifier
|
||||
import com.twitter.tweetypie.context.TweetypieContext
|
||||
import com.twitter.tweetypie.matching.Tokenizer
|
||||
import com.twitter.tweetypie.service._
|
||||
import com.twitter.tweetypie.thriftscala.TweetServiceInternal$FinagleService
|
||||
import com.twitter.util._
|
||||
import com.twitter.util.logging.Logger
|
||||
import scala.util.control.NonFatal
|
||||
|
||||
class TweetServerBuilder(settings: TweetServiceSettings) {
|
||||
|
||||
/**
|
||||
* A logger used by some of the built-in initializers.
|
||||
*/
|
||||
val log: Logger = Logger(getClass)
|
||||
|
||||
/**
|
||||
* The top-level stats receiver. Defaults to the default StatsReceiver
|
||||
* embedded in Finagle.
|
||||
*/
|
||||
val statsReceiver: StatsReceiver =
|
||||
new MemoizingStatsReceiver(DefaultStatsReceiver)
|
||||
|
||||
val hostStatsReceiver: StatsReceiver =
|
||||
if (settings.clientHostStats)
|
||||
statsReceiver
|
||||
else
|
||||
NullStatsReceiver
|
||||
|
||||
/**
|
||||
* A timer for scheduling various things.
|
||||
*/
|
||||
val timer: Timer = DefaultTimer
|
||||
|
||||
/**
|
||||
* Creates a decider instance by looking up the decider configuration information
|
||||
* from the settings object.
|
||||
*/
|
||||
val decider: Decider = {
|
||||
val fileBased = DeciderFactory(settings.deciderBaseFilename, settings.deciderOverlayFilename)()
|
||||
|
||||
// Use the tweetypie decider dashboard name for propagating decider overrides.
|
||||
LocalOverrides.decider("tweetypie").orElse(fileBased)
|
||||
}
|
||||
|
||||
val deciderGates: TweetypieDeciderGates = {
|
||||
val deciderGates = TweetypieDeciderGates(decider, settings.deciderOverrides)
|
||||
|
||||
// Write out the configuration overrides to the log so that it's
|
||||
// easy to confirm how this instance has been customized.
|
||||
deciderGates.overrides.foreach {
|
||||
case (overrideName, overrideValue) =>
|
||||
log.info("Decider feature " + overrideName + " overridden to " + overrideValue)
|
||||
if (deciderGates.unusedOverrides.contains(overrideName)) {
|
||||
log.error("Unused decider override flag: " + overrideName)
|
||||
}
|
||||
}
|
||||
|
||||
val scopedReceiver = statsReceiver.scope("decider_values")
|
||||
|
||||
deciderGates.availabilityMap.foreach {
|
||||
case (feature, value) =>
|
||||
scopedReceiver.provideGauge(feature) {
|
||||
// Default value of -1 indicates error state.
|
||||
value.getOrElse(-1).toFloat
|
||||
}
|
||||
}
|
||||
|
||||
deciderGates
|
||||
}
|
||||
|
||||
val featureSwitchesWithExperiments = FeatureSwitchesBuilder
|
||||
.createWithExperiments("/features/tweetypie/main")
|
||||
.build()
|
||||
|
||||
val featureSwitchesWithoutExperiments = FeatureSwitchesBuilder
|
||||
.createWithNoExperiments("/features/tweetypie/main", Some(statsReceiver))
|
||||
.build()
|
||||
|
||||
// ********* initializer **********
|
||||
|
||||
private[this] def warmupTextTokenization(logger: Logger): Unit = {
|
||||
logger.info("Warming up text tokenization")
|
||||
val watch = Stopwatch.start()
|
||||
Tokenizer.warmUp()
|
||||
logger.info(s"Warmed up text tokenization in ${watch()}")
|
||||
}
|
||||
|
||||
private[this] def runWarmup(tweetService: Activity[ThriftTweetService]): Unit = {
|
||||
val tokenizationLogger = Logger("com.twitter.tweetypie.TweetServerBuilder.TokenizationWarmup")
|
||||
warmupTextTokenization(tokenizationLogger)
|
||||
|
||||
val warmupLogger = Logger("com.twitter.tweetypie.TweetServerBuilder.BackendWarmup")
|
||||
// #1 warmup backends
|
||||
Await.ready(settings.backendWarmupSettings(backendClients, warmupLogger, timer))
|
||||
|
||||
// #2 warmup Tweet Service
|
||||
Await.ready {
|
||||
tweetService.values.toFuture.map(_.get).map { service =>
|
||||
settings.warmupRequestsSettings.foreach(new TweetServiceWarmer(_)(service))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private[this] def waitForServerSets(): Unit = {
|
||||
val names = backendClients.referencedNames
|
||||
val startTime = Time.now
|
||||
log.info("will wait for serversets: " + names.mkString("\n", "\t\n", ""))
|
||||
|
||||
try {
|
||||
Await.result(WaitForServerSets.ready(names, settings.waitForServerSetsTimeout, timer))
|
||||
val duration = Time.now.since(startTime)
|
||||
log.info("resolved all serversets in " + duration)
|
||||
} catch {
|
||||
case NonFatal(ex) => log.warn("failed to resolve all serversets", ex)
|
||||
}
|
||||
}
|
||||
|
||||
private[this] def initialize(tweetService: Activity[ThriftTweetService]): Unit = {
|
||||
waitForServerSets()
|
||||
runWarmup(tweetService)
|
||||
|
||||
// try to force a GC before starting to serve requests; this may or may not do anything
|
||||
System.gc()
|
||||
}
|
||||
|
||||
// ********* builders **********
|
||||
|
||||
val clientIdHelper = new ClientIdHelper(
|
||||
new ConditionalServiceIdentifierStrategy(
|
||||
condition = deciderGates.preferForwardedServiceIdentifierForClientId,
|
||||
ifTrue = PreferForwardedServiceIdentifierForStrato,
|
||||
ifFalse = UseTransportServiceIdentifier,
|
||||
),
|
||||
)
|
||||
|
||||
val backendClients: BackendClients =
|
||||
BackendClients(
|
||||
settings = settings,
|
||||
deciderGates = deciderGates,
|
||||
statsReceiver = statsReceiver,
|
||||
hostStatsReceiver = hostStatsReceiver,
|
||||
timer = timer,
|
||||
clientIdHelper = clientIdHelper,
|
||||
)
|
||||
|
||||
val tweetService: Activity[ThriftTweetService] =
|
||||
TweetServiceBuilder(
|
||||
settings = settings,
|
||||
statsReceiver = statsReceiver,
|
||||
timer = timer,
|
||||
deciderGates = deciderGates,
|
||||
featureSwitchesWithExperiments = featureSwitchesWithExperiments,
|
||||
featureSwitchesWithoutExperiments = featureSwitchesWithoutExperiments,
|
||||
backendClients = backendClients,
|
||||
clientIdHelper = clientIdHelper,
|
||||
)
|
||||
|
||||
// Strato columns should use this tweetService
|
||||
def stratoTweetService: Activity[ThriftTweetService] =
|
||||
tweetService.map { service =>
|
||||
// Add quill functionality to the strato tweet service only
|
||||
val quillCapture = QuillCaptureBuilder(settings, deciderGates)
|
||||
new QuillTweetService(quillCapture, service)
|
||||
}
|
||||
|
||||
def build: Activity[Service[Array[Byte], Array[Byte]]] = {
|
||||
|
||||
val quillCapture = QuillCaptureBuilder(settings, deciderGates)
|
||||
|
||||
val darkTrafficFilter: SimpleFilter[Array[Byte], Array[Byte]] =
|
||||
if (!settings.trafficForkingEnabled) {
|
||||
Filter.identity
|
||||
} else {
|
||||
new DarkTrafficFilter(
|
||||
backendClients.darkTrafficClient,
|
||||
_ => deciderGates.forkDarkTraffic(),
|
||||
statsReceiver
|
||||
)
|
||||
}
|
||||
|
||||
val serviceFilter =
|
||||
quillCapture
|
||||
.getServerFilter(ThriftProto.server)
|
||||
.andThen(TweetypieContext.Local.filter[Array[Byte], Array[Byte]])
|
||||
.andThen(darkTrafficFilter)
|
||||
|
||||
initialize(tweetService)
|
||||
|
||||
// tweetService is an Activity[ThriftTweetService], so this callback
|
||||
// is called every time that Activity updates (on ConfigBus changes).
|
||||
tweetService.map { service =>
|
||||
val finagleService =
|
||||
new TweetServiceInternal$FinagleService(
|
||||
service,
|
||||
protocolFactory = Protocols.binaryFactory(),
|
||||
stats = NullStatsReceiver,
|
||||
maxThriftBufferSize = settings.maxThriftBufferSize
|
||||
)
|
||||
|
||||
serviceFilter andThen finagleService
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object QuillCaptureBuilder {
|
||||
val tweetServiceWriteMethods: Set[String] =
|
||||
Set(
|
||||
"async_delete",
|
||||
"async_delete_additional_fields",
|
||||
"async_erase_user_tweets",
|
||||
"async_incr_fav_count",
|
||||
"async_insert",
|
||||
"async_set_additional_fields",
|
||||
"async_set_retweet_visibility",
|
||||
"async_takedown",
|
||||
"async_undelete_tweet",
|
||||
"async_update_possibly_sensitive_tweet",
|
||||
"cascaded_delete_tweet",
|
||||
"delete_additional_fields",
|
||||
"delete_retweets",
|
||||
"delete_tweets",
|
||||
"erase_user_tweets",
|
||||
"flush",
|
||||
"incr_fav_count",
|
||||
"insert",
|
||||
"post_retweet",
|
||||
"post_tweet",
|
||||
"remove",
|
||||
"replicated_delete_additional_fields",
|
||||
"replicated_delete_tweet",
|
||||
"replicated_delete_tweet2",
|
||||
"replicated_incr_fav_count",
|
||||
"replicated_insert_tweet2",
|
||||
"replicated_scrub_geo",
|
||||
"replicated_set_additional_fields",
|
||||
"replicated_set_has_safety_labels",
|
||||
"replicated_set_retweet_visibility",
|
||||
"replicated_takedown",
|
||||
"replicated_undelete_tweet2",
|
||||
"replicated_update_possibly_sensitive_tweet",
|
||||
"scrub_geo",
|
||||
"scrub_geo_update_user_timestamp",
|
||||
"set_additional_fields",
|
||||
"set_has_safety_labels",
|
||||
"set_retweet_visibility",
|
||||
"set_tweet_user_takedown",
|
||||
"takedown",
|
||||
"undelete_tweet"
|
||||
)
|
||||
|
||||
val tweetServiceReadMethods: Set[String] =
|
||||
Set(
|
||||
"get_tweet_counts",
|
||||
"get_tweet_fields",
|
||||
"get_tweets",
|
||||
"replicated_get_tweet_counts",
|
||||
"replicated_get_tweet_fields",
|
||||
"replicated_get_tweets"
|
||||
)
|
||||
|
||||
def apply(settings: TweetServiceSettings, deciderGates: TweetypieDeciderGates): QuillCapture = {
|
||||
val writesStore = SimpleScribeMessageStore("tweetypie_writes")
|
||||
.enabledBy(deciderGates.logWrites)
|
||||
|
||||
val readsStore = SimpleScribeMessageStore("tweetypie_reads")
|
||||
.enabledBy(deciderGates.logReads)
|
||||
|
||||
val messageStore =
|
||||
MessageStore.selected {
|
||||
case msg if tweetServiceWriteMethods.contains(msg.name) => writesStore
|
||||
case msg if tweetServiceReadMethods.contains(msg.name) => readsStore
|
||||
case _ => writesStore
|
||||
}
|
||||
|
||||
new QuillCapture(Store.legacyStore(messageStore), Some(settings.thriftClientId.name))
|
||||
}
|
||||
}
|
@ -0,0 +1,399 @@
|
||||
package com.twitter.tweetypie
|
||||
package config
|
||||
|
||||
import com.twitter.finagle.mtls.authentication.ServiceIdentifier
|
||||
import com.twitter.finagle.mtls.transport.S2STransport
|
||||
import com.twitter.servo.gate.RateLimitingGate
|
||||
import com.twitter.servo.request.ClientRequestAuthorizer.UnauthorizedException
|
||||
import com.twitter.servo.request.{ClientRequestAuthorizer, ClientRequestObserver}
|
||||
import com.twitter.tweetypie.client_id.ClientIdHelper
|
||||
import com.twitter.tweetypie.client_id.PreferForwardedServiceIdentifierForStrato
|
||||
import com.twitter.tweetypie.core.RateLimited
|
||||
import com.twitter.tweetypie.service.MethodAuthorizer
|
||||
import com.twitter.tweetypie.thriftscala._
|
||||
import com.twitter.util.Future
|
||||
|
||||
/**
|
||||
* Compose a ClientRequestAuthorizer for
|
||||
* ClientHandlingTweetService
|
||||
*/
|
||||
object ClientHandlingTweetServiceAuthorizer {
|
||||
private val RateLimitExceeded =
|
||||
RateLimited("Your ClientId has exceeded the rate limit for non-allowListed clients.")
|
||||
|
||||
def apply(
|
||||
settings: TweetServiceSettings,
|
||||
dynamicConfig: DynamicConfig,
|
||||
statsReceiver: StatsReceiver,
|
||||
getServiceIdentifier: () => ServiceIdentifier = S2STransport.peerServiceIdentifier _
|
||||
): ClientRequestAuthorizer = {
|
||||
val authorizer =
|
||||
if (settings.allowlistingRequired) {
|
||||
val limitingGate = RateLimitingGate.uniform(settings.nonAllowListedClientRateLimitPerSec)
|
||||
allowListedOrRateLimitedAuthorizer(dynamicConfig, limitingGate)
|
||||
.andThen(rejectNonAllowListedProdAuthorizer(dynamicConfig))
|
||||
.andThen(permittedMethodsAuthorizer(dynamicConfig))
|
||||
.andThen(allowProductionAuthorizer(settings.allowProductionClients))
|
||||
} else {
|
||||
ClientRequestAuthorizer.withClientId
|
||||
}
|
||||
|
||||
val alternativeClientIdHelper = new ClientIdHelper(PreferForwardedServiceIdentifierForStrato)
|
||||
// pass the authorizer into an observed authorizer for stats tracking.
|
||||
// (observed authorizers can't be composed with andThen)
|
||||
ClientRequestAuthorizer.observed(
|
||||
authorizer,
|
||||
new ClientRequestObserver(statsReceiver) {
|
||||
override def apply(
|
||||
methodName: String,
|
||||
clientIdScopesOpt: Option[Seq[String]]
|
||||
): Future[Unit] = {
|
||||
// Monitor for the migration taking into account forwarded service identifier
|
||||
// as effective client ID for strato.
|
||||
val alternativeClientIdScopes = alternativeClientIdHelper.effectiveClientId.map(Seq(_))
|
||||
if (clientIdScopesOpt != alternativeClientIdScopes) {
|
||||
scopedReceiver.scope(methodName)
|
||||
.scope("before_migration")
|
||||
.scope(clientIdScopesOpt.getOrElse(Seq(ClientIdHelper.UnknownClientId)): _*)
|
||||
.scope("after_migration")
|
||||
.counter(alternativeClientIdScopes.getOrElse(Seq(ClientIdHelper.UnknownClientId)): _*)
|
||||
.incr()
|
||||
} else {
|
||||
scopedReceiver.scope(methodName).counter("migration_indifferent").incr()
|
||||
}
|
||||
super.apply(methodName, clientIdScopesOpt)
|
||||
}
|
||||
|
||||
override def authorized(methodName: String, clientIdStr: String): Unit = {
|
||||
// Monitor for the migration of using service identifier
|
||||
// as identity instead of client ID.
|
||||
val serviceIdentifier = getServiceIdentifier()
|
||||
scopedReceiver.counter(
|
||||
"authorized_request",
|
||||
clientIdStr,
|
||||
serviceIdentifier.role,
|
||||
serviceIdentifier.service,
|
||||
serviceIdentifier.environment
|
||||
).incr()
|
||||
val status = dynamicConfig.byServiceIdentifier(serviceIdentifier).toSeq match {
|
||||
case Seq() => "none"
|
||||
case Seq(client) if client.clientId == clientIdStr => "equal"
|
||||
case Seq(_) => "other"
|
||||
case _ => "ambiguous"
|
||||
}
|
||||
scopedReceiver.counter(
|
||||
"service_id_match_client_id",
|
||||
clientIdStr,
|
||||
serviceIdentifier.role,
|
||||
serviceIdentifier.service,
|
||||
serviceIdentifier.environment,
|
||||
status
|
||||
).incr()
|
||||
}
|
||||
}
|
||||
)
|
||||
}
|
||||
|
||||
/**
|
||||
* @return A ClientRequestAuthorizer that allows unlimited requests for allowlisted client ids and
|
||||
* rate-limited requests for unknown clients.
|
||||
*/
|
||||
def allowListedOrRateLimitedAuthorizer(
|
||||
dynamicConfig: DynamicConfig,
|
||||
nonAllowListedLimiter: Gate[Unit]
|
||||
): ClientRequestAuthorizer =
|
||||
ClientRequestAuthorizer.filtered(
|
||||
{ (_, clientId) =>
|
||||
dynamicConfig.isAllowListedClient(clientId) || nonAllowListedLimiter()
|
||||
},
|
||||
RateLimitExceeded)
|
||||
|
||||
/**
|
||||
* @return A ClientRequestAuthorizer that rejects requests from non-allowListed prod clients.
|
||||
*/
|
||||
def rejectNonAllowListedProdAuthorizer(dynamicConfig: DynamicConfig): ClientRequestAuthorizer = {
|
||||
object UnallowlistedException
|
||||
extends UnauthorizedException(
|
||||
"Traffic is only allowed from allow-listed *.prod clients." +
|
||||
" Please create a ticket to register your clientId to enable production traffic using http://go/tp-new-client."
|
||||
)
|
||||
|
||||
def isProdClient(clientId: String): Boolean =
|
||||
clientId.endsWith(".prod") || clientId.endsWith(".production")
|
||||
|
||||
ClientRequestAuthorizer.filtered(
|
||||
{ (_, clientId) =>
|
||||
!isProdClient(clientId) || dynamicConfig.isAllowListedClient(clientId)
|
||||
},
|
||||
UnallowlistedException)
|
||||
}
|
||||
|
||||
/**
|
||||
* @return A ClientRequestAuthorizer that checks if a given client's
|
||||
* permittedMethods field includes the method they are calling
|
||||
*/
|
||||
def permittedMethodsAuthorizer(dynamicConfig: DynamicConfig): ClientRequestAuthorizer =
|
||||
dynamicConfig.clientsByFullyQualifiedId match {
|
||||
case Some(clientsById) => permittedMethodsAuthorizer(dynamicConfig, clientsById)
|
||||
case None => ClientRequestAuthorizer.permissive
|
||||
}
|
||||
|
||||
private def permittedMethodsAuthorizer(
|
||||
dynamicConfig: DynamicConfig,
|
||||
clientsByFullyQualifiedId: Map[String, Client]
|
||||
): ClientRequestAuthorizer = {
|
||||
ClientRequestAuthorizer.filtered { (methodName, clientId) =>
|
||||
dynamicConfig.unprotectedEndpoints(methodName) ||
|
||||
(clientsByFullyQualifiedId.get(clientId) match {
|
||||
case Some(client) =>
|
||||
client.accessAllMethods ||
|
||||
client.permittedMethods.contains(methodName)
|
||||
case None =>
|
||||
false // If client id is unknown, don't allow access
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @return A ClientRequestAuthorizer that fails the
|
||||
* request if it is coming from a production client
|
||||
* and allowProductionClients is false
|
||||
*/
|
||||
def allowProductionAuthorizer(allowProductionClients: Boolean): ClientRequestAuthorizer =
|
||||
ClientRequestAuthorizer.filtered { (_, clientId) =>
|
||||
allowProductionClients || !(clientId.endsWith(".prod") || clientId.endsWith(".production"))
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Compose a MethodAuthorizer for the `getTweets` endpoint.
|
||||
*/
|
||||
object GetTweetsAuthorizer {
|
||||
import ProtectedTweetsAuthorizer.IncludeProtected
|
||||
|
||||
def apply(
|
||||
config: DynamicConfig,
|
||||
maxRequestSize: Int,
|
||||
instanceCount: Int,
|
||||
enforceRateLimitedClients: Gate[Unit],
|
||||
maxRequestWidthEnabled: Gate[Unit],
|
||||
statsReceiver: StatsReceiver,
|
||||
): MethodAuthorizer[GetTweetsRequest] =
|
||||
MethodAuthorizer.all(
|
||||
Seq(
|
||||
ProtectedTweetsAuthorizer(config.clientsByFullyQualifiedId)
|
||||
.contramap[GetTweetsRequest] { r =>
|
||||
IncludeProtected(r.options.exists(_.bypassVisibilityFiltering))
|
||||
},
|
||||
RequestSizeAuthorizer(maxRequestSize, maxRequestWidthEnabled)
|
||||
.contramap[GetTweetsRequest](_.tweetIds.size),
|
||||
RateLimiterAuthorizer(config, instanceCount, enforceRateLimitedClients, statsReceiver)
|
||||
.contramap[GetTweetsRequest](_.tweetIds.size)
|
||||
)
|
||||
)
|
||||
}
|
||||
|
||||
/**
|
||||
* Compose a MethodAuthorizer for the `getTweetFields` endpoint.
|
||||
*/
|
||||
object GetTweetFieldsAuthorizer {
|
||||
import ProtectedTweetsAuthorizer.IncludeProtected
|
||||
|
||||
def apply(
|
||||
config: DynamicConfig,
|
||||
maxRequestSize: Int,
|
||||
instanceCount: Int,
|
||||
enforceRateLimitedClients: Gate[Unit],
|
||||
maxRequestWidthEnabled: Gate[Unit],
|
||||
statsReceiver: StatsReceiver
|
||||
): MethodAuthorizer[GetTweetFieldsRequest] =
|
||||
MethodAuthorizer.all(
|
||||
Seq(
|
||||
ProtectedTweetsAuthorizer(config.clientsByFullyQualifiedId)
|
||||
.contramap[GetTweetFieldsRequest](r =>
|
||||
IncludeProtected(r.options.visibilityPolicy == TweetVisibilityPolicy.NoFiltering)),
|
||||
RequestSizeAuthorizer(maxRequestSize, maxRequestWidthEnabled)
|
||||
.contramap[GetTweetFieldsRequest](_.tweetIds.size),
|
||||
RateLimiterAuthorizer(config, instanceCount, enforceRateLimitedClients, statsReceiver)
|
||||
.contramap[GetTweetFieldsRequest](_.tweetIds.size)
|
||||
)
|
||||
)
|
||||
}
|
||||
|
||||
object ProtectedTweetsAuthorizer {
|
||||
case class IncludeProtected(include: Boolean) extends AnyVal
|
||||
|
||||
class BypassVisibilityFilteringNotAuthorizedException(message: String)
|
||||
extends UnauthorizedException(message)
|
||||
|
||||
def apply(optClientsById: Option[Map[String, Client]]): MethodAuthorizer[IncludeProtected] = {
|
||||
optClientsById match {
|
||||
case Some(clientsByFullyQualifiedId) =>
|
||||
val clientsWithBypassVisibilityFiltering = clientsByFullyQualifiedId.filter {
|
||||
case (_, client) => client.bypassVisibilityFiltering
|
||||
}
|
||||
apply(clientId => clientsWithBypassVisibilityFiltering.contains(clientId))
|
||||
|
||||
case None =>
|
||||
apply((_: String) => true)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A MethodAuthorizer that fails the request if a client requests to bypass visibility
|
||||
* filtering but doesn't have BypassVisibilityFiltering
|
||||
*/
|
||||
def apply(protectedTweetsAllowlist: String => Boolean): MethodAuthorizer[IncludeProtected] =
|
||||
MethodAuthorizer { (includeProtected, clientId) =>
|
||||
// There is only one unauthorized case, a client requesting
|
||||
// protected tweets when they are not in the allowlist
|
||||
Future.when(includeProtected.include && !protectedTweetsAllowlist(clientId)) {
|
||||
Future.exception(
|
||||
new BypassVisibilityFilteringNotAuthorizedException(
|
||||
s"$clientId is not authorized to bypass visibility filtering"
|
||||
)
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A MethodAuthorizer[Int] that fails large requests.
|
||||
*/
|
||||
object RequestSizeAuthorizer {
|
||||
class ExceededMaxWidthException(message: String) extends UnauthorizedException(message)
|
||||
|
||||
def apply(
|
||||
maxRequestSize: Int,
|
||||
maxWidthLimitEnabled: Gate[Unit] = Gate.False
|
||||
): MethodAuthorizer[Int] =
|
||||
MethodAuthorizer { (requestSize, clientId) =>
|
||||
Future.when(requestSize > maxRequestSize && maxWidthLimitEnabled()) {
|
||||
Future.exception(
|
||||
new ExceededMaxWidthException(
|
||||
s"$requestSize exceeds bulk request size limit. $clientId can request at most $maxRequestSize items per request"
|
||||
)
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object RateLimiterAuthorizer {
|
||||
|
||||
type ClientId = String
|
||||
|
||||
/**
|
||||
* @return client ID to weighted RateLimitingGate map
|
||||
*
|
||||
* We want to rate-limit based on requests per sec for every instance.
|
||||
* When we allowlist new clients to Tweetypie, we assign tweets per sec quota.
|
||||
* That's why, we compute perInstanceQuota [1] and create a weighted rate-limiting gate [2]
|
||||
* which returns true if acquiring requestSize number of permits is successful. [3]
|
||||
*
|
||||
* [1] tps quota during allowlisting is for both DCs and instanceCount is for one DC.
|
||||
* Therefore, we are over-compensating perInstanceQuota for all low-priority clients.
|
||||
* this will act a fudge-factor to account for cluster-wide traffic imbalances.
|
||||
*
|
||||
* val perInstanceQuota : Double = math.max(1.0, math.ceil(tpsLimit.toFloat / instanceCount))
|
||||
*
|
||||
* We have some clients like deferredRPC with 0K tps quota and rate limiter expects > 0 permits.
|
||||
*
|
||||
* [2] if a client has multiple environments - staging, devel, prod. We provision the
|
||||
* same rate-limits for all envs instead of distributing the tps quota across envs.
|
||||
*
|
||||
* Example:
|
||||
*
|
||||
* val c = Client(..., limit = 10k, ...)
|
||||
* Map("foo.prod" -> c, "foo.staging" -> c, "foo.devel" -> c)
|
||||
*
|
||||
* Above client config turns into 3 separate RateLimitingGate.weighted(), each with 10k
|
||||
*
|
||||
* [3] RateLimitingGate will always give permit to the initial request that exceeds
|
||||
* the limit. ex: starting with rate-limit of 1 tps per instance. first request with
|
||||
* 100 batch size is allowed.
|
||||
*
|
||||
* RateLimitFudgeFactor is a multiplier for per-instance quota to account for:
|
||||
*
|
||||
* a) High likelihood of concurrent batches hitting the same tweetypie shard due to
|
||||
* non-uniform load distribution (this can be alleviated by using Deterministic Aperture)
|
||||
* b) Clients with no retry backoffs and custom batching/concurrency.
|
||||
*
|
||||
* We are adding default stitch batch size to per instance quota, to give more headroom for low-tps clients.
|
||||
* https://cgit.twitter.biz/source/tree/stitch/stitch-tweetypie/src/main/scala/com/twitter/stitch/tweetypie/TweetyPie.scala#n47
|
||||
*
|
||||
*/
|
||||
case class RateLimiterConfig(limitingGate: Gate[Int], enforceRateLimit: Boolean)
|
||||
|
||||
def perClientRateLimiters(
|
||||
dynamicConfig: DynamicConfig,
|
||||
instanceCount: Int
|
||||
): Map[ClientId, RateLimiterConfig] = {
|
||||
val RateLimitFudgeFactor: Double = 1.5
|
||||
val DefaultStitchBatchSize: Double = 25.0
|
||||
dynamicConfig.clientsByFullyQualifiedId match {
|
||||
case Some(clients) =>
|
||||
clients.collect {
|
||||
case (clientId, client) if client.tpsLimit.isDefined =>
|
||||
val perInstanceQuota: Double =
|
||||
math.max(
|
||||
1.0,
|
||||
math.ceil(
|
||||
client.tpsLimit.get.toFloat / instanceCount)) * RateLimitFudgeFactor + DefaultStitchBatchSize
|
||||
clientId -> RateLimiterConfig(
|
||||
RateLimitingGate.weighted(perInstanceQuota),
|
||||
client.enforceRateLimit
|
||||
)
|
||||
}
|
||||
case None => Map.empty
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
enforce rate-limiting on get_tweets and get_tweet_fields requests
|
||||
given enable_rate_limited_clients decider is true and rate limiting gate
|
||||
is not giving any more permits.
|
||||
*/
|
||||
def apply(
|
||||
config: DynamicConfig,
|
||||
limiters: Map[ClientId, RateLimiterConfig],
|
||||
instanceCount: Int,
|
||||
enforceRateLimitedClients: Gate[Unit],
|
||||
statsReceiver: StatsReceiver
|
||||
): MethodAuthorizer[Int] = {
|
||||
|
||||
val tpsExceededScope = statsReceiver.scope("tps_exceeded")
|
||||
val tpsRejectedScope = statsReceiver.scope("tps_rejected")
|
||||
val qpsExceededScope = statsReceiver.scope("qps_exceeded")
|
||||
val qpsRejectedScope = statsReceiver.scope("qps_rejected")
|
||||
|
||||
MethodAuthorizer { (requestSize, clientId) =>
|
||||
val positiveRequestSize = math.max(1, requestSize)
|
||||
val shouldRateLimit: Boolean = limiters.get(clientId).exists { config =>
|
||||
val exceededLimit = !config.limitingGate(positiveRequestSize)
|
||||
if (exceededLimit) {
|
||||
qpsExceededScope.counter(clientId).incr()
|
||||
tpsExceededScope.counter(clientId).incr(positiveRequestSize)
|
||||
}
|
||||
exceededLimit && config.enforceRateLimit
|
||||
}
|
||||
|
||||
Future.when(shouldRateLimit && enforceRateLimitedClients()) {
|
||||
qpsRejectedScope.counter(clientId).incr()
|
||||
tpsRejectedScope.counter(clientId).incr(positiveRequestSize)
|
||||
Future.exception(
|
||||
RateLimited(s"Your client ID $clientId has exceeded its reserved tps quota.")
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
def apply(
|
||||
config: DynamicConfig,
|
||||
instanceCount: Int,
|
||||
enforceRateLimitedClients: Gate[Unit],
|
||||
statsReceiver: StatsReceiver
|
||||
): MethodAuthorizer[Int] = {
|
||||
val limiters = perClientRateLimiters(config, instanceCount)
|
||||
apply(config, limiters, instanceCount, enforceRateLimitedClients, statsReceiver)
|
||||
}
|
||||
}
|
@ -0,0 +1,683 @@
|
||||
package com.twitter.tweetypie
|
||||
package config
|
||||
|
||||
import com.twitter.coreservices.failed_task.writer.FailedTaskWriter
|
||||
import com.twitter.featureswitches.v2.FeatureSwitches
|
||||
import com.twitter.flockdb.client._
|
||||
import com.twitter.servo.forked
|
||||
import com.twitter.servo.util.FutureArrow
|
||||
import com.twitter.servo.util.Scribe
|
||||
import com.twitter.stitch.Stitch
|
||||
import com.twitter.tweetypie.client_id.ClientIdHelper
|
||||
import com.twitter.tweetypie.handler._
|
||||
import com.twitter.tweetypie.repository._
|
||||
import com.twitter.tweetypie.service.ReplicatingTweetService
|
||||
import com.twitter.tweetypie.service._
|
||||
import com.twitter.tweetypie.storage.TweetStorageClient
|
||||
import com.twitter.tweetypie.storage.TweetStorageClient.GetTweet
|
||||
import com.twitter.tweetypie.store._
|
||||
import com.twitter.tweetypie.thriftscala._
|
||||
import com.twitter.util.Activity
|
||||
import com.twitter.util.Timer
|
||||
|
||||
/**
|
||||
* Builds a fully configured ThriftTweetService instance.
|
||||
*
|
||||
* The core of the tweet service is a DispatchingTweetService, which is responsible
|
||||
* for dispatching requests to underlying handlers and stores.
|
||||
* The DispatchingTweetService instance is wrapped in:
|
||||
* - ObservedTweetService (adds stats counting)
|
||||
* - ClientHandlingTweetService (authentication, exception handling, etc)
|
||||
* - ReplicatingTweetService (replicates some reads)
|
||||
*
|
||||
* TweetServiceBuilder returns an Activity[ThriftTweetService] which updates
|
||||
* on config changes. See DynamicConfig.scala for more details.
|
||||
*/
|
||||
object TweetServiceBuilder {
|
||||
def apply(
|
||||
settings: TweetServiceSettings,
|
||||
statsReceiver: StatsReceiver,
|
||||
timer: Timer,
|
||||
deciderGates: TweetypieDeciderGates,
|
||||
featureSwitchesWithExperiments: FeatureSwitches,
|
||||
featureSwitchesWithoutExperiments: FeatureSwitches,
|
||||
backendClients: BackendClients,
|
||||
clientIdHelper: ClientIdHelper,
|
||||
): Activity[ThriftTweetService] = {
|
||||
// a forward reference, will be set to the DispatchingTweetService once created
|
||||
val syncTweetService = new MutableTweetServiceProxy(null)
|
||||
|
||||
val tweetServiceScope = statsReceiver.scope("tweet_service")
|
||||
|
||||
val dispatchingTweetService =
|
||||
DispatchingTweetServiceBuilder(
|
||||
settings,
|
||||
statsReceiver,
|
||||
tweetServiceScope,
|
||||
syncTweetService,
|
||||
timer,
|
||||
deciderGates,
|
||||
featureSwitchesWithExperiments,
|
||||
featureSwitchesWithoutExperiments,
|
||||
backendClients,
|
||||
clientIdHelper,
|
||||
)
|
||||
|
||||
val failureLoggingTweetService =
|
||||
// Add the failure writing inside of the authorization filter so
|
||||
// that we don't write out the failures when authorization fails.
|
||||
new FailureLoggingTweetService(
|
||||
failedTaskWriter = FailedTaskWriter("tweetypie_service_failures", identity),
|
||||
underlying = dispatchingTweetService
|
||||
)
|
||||
|
||||
val observedTweetService =
|
||||
new ObservedTweetService(failureLoggingTweetService, tweetServiceScope, clientIdHelper)
|
||||
|
||||
// Every time config is updated, create a new tweet service. Only
|
||||
// ClientHandlingTweetService and ReplicatingTweetService need to
|
||||
// be recreated, as the underlying TweetServices above don't depend
|
||||
// on the config.
|
||||
DynamicConfig(
|
||||
statsReceiver.scope("dynamic_config"),
|
||||
backendClients.configBus,
|
||||
settings
|
||||
).map { dynamicConfig =>
|
||||
val clientHandlingTweetService =
|
||||
new ClientHandlingTweetService(
|
||||
observedTweetService,
|
||||
tweetServiceScope,
|
||||
dynamicConfig.loadShedEligible,
|
||||
deciderGates.shedReadTrafficVoluntarily,
|
||||
ClientHandlingTweetServiceAuthorizer(
|
||||
settings = settings,
|
||||
dynamicConfig = dynamicConfig,
|
||||
statsReceiver = statsReceiver
|
||||
),
|
||||
GetTweetsAuthorizer(
|
||||
config = dynamicConfig,
|
||||
maxRequestSize = settings.maxGetTweetsRequestSize,
|
||||
instanceCount = settings.instanceCount,
|
||||
enforceRateLimitedClients = deciderGates.enforceRateLimitedClients,
|
||||
maxRequestWidthEnabled = deciderGates.maxRequestWidthEnabled,
|
||||
statsReceiver = tweetServiceScope.scope("get_tweets"),
|
||||
),
|
||||
GetTweetFieldsAuthorizer(
|
||||
config = dynamicConfig,
|
||||
maxRequestSize = settings.maxGetTweetsRequestSize,
|
||||
instanceCount = settings.instanceCount,
|
||||
enforceRateLimitedClients = deciderGates.enforceRateLimitedClients,
|
||||
maxRequestWidthEnabled = deciderGates.maxRequestWidthEnabled,
|
||||
statsReceiver = tweetServiceScope.scope("get_tweet_fields"),
|
||||
),
|
||||
RequestSizeAuthorizer(settings.maxRequestSize, deciderGates.maxRequestWidthEnabled),
|
||||
clientIdHelper,
|
||||
)
|
||||
|
||||
syncTweetService.underlying = clientHandlingTweetService
|
||||
|
||||
val replicatingService =
|
||||
if (!settings.enableReplication)
|
||||
clientHandlingTweetService
|
||||
else {
|
||||
new ReplicatingTweetService(
|
||||
underlying = clientHandlingTweetService,
|
||||
replicationTargets = backendClients.lowQoSReplicationClients,
|
||||
executor = new forked.QueueExecutor(
|
||||
100,
|
||||
statsReceiver.scope("replicating_tweet_service")
|
||||
),
|
||||
)
|
||||
}
|
||||
|
||||
replicatingService
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object DispatchingTweetServiceBuilder {
|
||||
val hasMedia: Tweet => Boolean = MediaIndexHelper(Resources.loadPartnerMediaRegexes())
|
||||
|
||||
def apply(
|
||||
settings: TweetServiceSettings,
|
||||
statsReceiver: StatsReceiver,
|
||||
tweetServiceScope: StatsReceiver,
|
||||
syncTweetService: ThriftTweetService,
|
||||
timer: Timer,
|
||||
deciderGates: TweetypieDeciderGates,
|
||||
featureSwitchesWithExperiments: FeatureSwitches,
|
||||
featureSwitchesWithoutExperiments: FeatureSwitches,
|
||||
backendClients: BackendClients,
|
||||
clientIdHelper: ClientIdHelper,
|
||||
): ThriftTweetService = {
|
||||
val (syncInvocationBuilder, asyncInvocationBuilder) = {
|
||||
val b =
|
||||
new ServiceInvocationBuilder(syncTweetService, settings.simulateDeferredrpcCallbacks)
|
||||
(b.withClientId(settings.thriftClientId), b.withClientId(settings.deferredrpcClientId))
|
||||
}
|
||||
|
||||
val tweetKeyFactory = TweetKeyFactory(settings.tweetKeyCacheVersion)
|
||||
|
||||
val caches =
|
||||
if (!settings.withCache)
|
||||
Caches.NoCache
|
||||
else
|
||||
Caches(
|
||||
settings = settings,
|
||||
stats = statsReceiver,
|
||||
timer = timer,
|
||||
clients = backendClients,
|
||||
tweetKeyFactory = tweetKeyFactory,
|
||||
deciderGates = deciderGates,
|
||||
clientIdHelper = clientIdHelper,
|
||||
)
|
||||
|
||||
val logicalRepos =
|
||||
LogicalRepositories(
|
||||
settings = settings,
|
||||
stats = statsReceiver,
|
||||
timer = timer,
|
||||
deciderGates = deciderGates,
|
||||
external = new ExternalServiceRepositories(
|
||||
clients = backendClients,
|
||||
statsReceiver = statsReceiver,
|
||||
settings = settings,
|
||||
clientIdHelper = clientIdHelper,
|
||||
),
|
||||
caches = caches,
|
||||
stratoClient = backendClients.stratoserverClient,
|
||||
hasMedia = hasMedia,
|
||||
clientIdHelper = clientIdHelper,
|
||||
featureSwitchesWithoutExperiments = featureSwitchesWithoutExperiments,
|
||||
)
|
||||
|
||||
val tweetCreationLock =
|
||||
new CacheBasedTweetCreationLock(
|
||||
cache = caches.tweetCreateLockerCache,
|
||||
maxTries = 3,
|
||||
stats = statsReceiver.scope("tweet_save").scope("locker"),
|
||||
logUniquenessId =
|
||||
if (settings.scribeUniquenessIds) CacheBasedTweetCreationLock.ScribeUniquenessId
|
||||
else CacheBasedTweetCreationLock.LogUniquenessId
|
||||
)
|
||||
|
||||
val tweetStores =
|
||||
TweetStores(
|
||||
settings = settings,
|
||||
statsReceiver = statsReceiver,
|
||||
timer = timer,
|
||||
deciderGates = deciderGates,
|
||||
tweetKeyFactory = tweetKeyFactory,
|
||||
clients = backendClients,
|
||||
caches = caches,
|
||||
asyncBuilder = asyncInvocationBuilder,
|
||||
hasMedia = hasMedia,
|
||||
clientIdHelper = clientIdHelper,
|
||||
)
|
||||
|
||||
val tweetDeletePathHandler =
|
||||
new DefaultTweetDeletePathHandler(
|
||||
tweetServiceScope,
|
||||
logicalRepos.tweetResultRepo,
|
||||
logicalRepos.optionalUserRepo,
|
||||
logicalRepos.stratoSafetyLabelsRepo,
|
||||
logicalRepos.lastQuoteOfQuoterRepo,
|
||||
tweetStores,
|
||||
getPerspectives = backendClients.timelineService.getPerspectives,
|
||||
)
|
||||
|
||||
val tweetBuilders =
|
||||
TweetBuilders(
|
||||
settings = settings,
|
||||
statsReceiver = statsReceiver,
|
||||
deciderGates = deciderGates,
|
||||
featureSwitchesWithExperiments = featureSwitchesWithExperiments,
|
||||
clients = backendClients,
|
||||
caches = caches,
|
||||
repos = logicalRepos,
|
||||
tweetStore = tweetStores,
|
||||
hasMedia = hasMedia,
|
||||
unretweetEdits = tweetDeletePathHandler.unretweetEdits,
|
||||
)
|
||||
|
||||
val hydrateTweetForInsert =
|
||||
WritePathHydration.hydrateTweet(
|
||||
logicalRepos.tweetHydrators.hydrator,
|
||||
statsReceiver.scope("insert_tweet")
|
||||
)
|
||||
|
||||
val defaultTweetQueryOptions = TweetQuery.Options(include = GetTweetsHandler.BaseInclude)
|
||||
|
||||
val parentUserIdRepo: ParentUserIdRepository.Type =
|
||||
ParentUserIdRepository(
|
||||
tweetRepo = logicalRepos.tweetRepo
|
||||
)
|
||||
|
||||
val undeleteTweetHandler =
|
||||
UndeleteTweetHandlerBuilder(
|
||||
backendClients.tweetStorageClient,
|
||||
logicalRepos,
|
||||
tweetStores,
|
||||
parentUserIdRepo,
|
||||
statsReceiver
|
||||
)
|
||||
|
||||
val eraseUserTweetsHandler =
|
||||
EraseUserTweetsHandlerBuilder(
|
||||
backendClients,
|
||||
asyncInvocationBuilder,
|
||||
deciderGates,
|
||||
settings,
|
||||
timer,
|
||||
tweetDeletePathHandler,
|
||||
tweetServiceScope
|
||||
)
|
||||
|
||||
val setRetweetVisibilityHandler =
|
||||
SetRetweetVisibilityHandler(
|
||||
tweetGetter =
|
||||
TweetRepository.tweetGetter(logicalRepos.optionalTweetRepo, defaultTweetQueryOptions),
|
||||
tweetStores.setRetweetVisibility
|
||||
)
|
||||
|
||||
val takedownHandler =
|
||||
TakedownHandlerBuilder(
|
||||
logicalRepos = logicalRepos,
|
||||
tweetStores = tweetStores
|
||||
)
|
||||
|
||||
val updatePossiblySensitiveTweetHandler =
|
||||
UpdatePossiblySensitiveTweetHandler(
|
||||
HandlerError.getRequired(
|
||||
TweetRepository.tweetGetter(logicalRepos.optionalTweetRepo, defaultTweetQueryOptions),
|
||||
HandlerError.tweetNotFoundException
|
||||
),
|
||||
HandlerError.getRequired(
|
||||
FutureArrow(
|
||||
UserRepository
|
||||
.userGetter(
|
||||
logicalRepos.optionalUserRepo,
|
||||
UserQueryOptions(Set(UserField.Safety), UserVisibility.All)
|
||||
)
|
||||
.compose(UserKey.byId)
|
||||
),
|
||||
HandlerError.userNotFoundException
|
||||
),
|
||||
tweetStores.updatePossiblySensitiveTweet
|
||||
)
|
||||
|
||||
val userTakedownHandler =
|
||||
UserTakedownHandlerBuilder(
|
||||
logicalRepos = logicalRepos,
|
||||
tweetStores = tweetStores,
|
||||
stats = tweetServiceScope
|
||||
)
|
||||
|
||||
val getDeletedTweetsHandler =
|
||||
GetDeletedTweetsHandler(
|
||||
getDeletedTweets = backendClients.tweetStorageClient.getDeletedTweets,
|
||||
tweetsExist =
|
||||
GetDeletedTweetsHandler.tweetsExist(backendClients.tweetStorageClient.getTweet),
|
||||
stats = tweetServiceScope.scope("get_deleted_tweets_handler")
|
||||
)
|
||||
|
||||
val hydrateQuotedTweet =
|
||||
WritePathHydration.hydrateQuotedTweet(
|
||||
logicalRepos.optionalTweetRepo,
|
||||
logicalRepos.optionalUserRepo,
|
||||
logicalRepos.quoterHasAlreadyQuotedRepo
|
||||
)
|
||||
|
||||
val deleteLocationDataHandler =
|
||||
DeleteLocationDataHandler(
|
||||
backendClients.geoScrubEventStore.getGeoScrubTimestamp,
|
||||
Scribe(DeleteLocationData, "tweetypie_delete_location_data"),
|
||||
backendClients.deleteLocationDataPublisher
|
||||
)
|
||||
|
||||
val getStoredTweetsHandler = GetStoredTweetsHandler(logicalRepos.tweetResultRepo)
|
||||
|
||||
val getStoredTweetsByUserHandler = GetStoredTweetsByUserHandler(
|
||||
getStoredTweetsHandler = getStoredTweetsHandler,
|
||||
getStoredTweet = backendClients.tweetStorageClient.getStoredTweet,
|
||||
selectPage = FutureArrow { select =>
|
||||
backendClients.tflockReadClient
|
||||
.selectPage(select, Some(settings.getStoredTweetsByUserPageSize))
|
||||
},
|
||||
maxPages = settings.getStoredTweetsByUserMaxPages
|
||||
)
|
||||
|
||||
val getTweetsHandler =
|
||||
GetTweetsHandler(
|
||||
logicalRepos.tweetResultRepo,
|
||||
logicalRepos.containerAsGetTweetResultRepo,
|
||||
logicalRepos.deletedTweetVisibilityRepo,
|
||||
statsReceiver.scope("read_path"),
|
||||
deciderGates.shouldMaterializeContainers
|
||||
)
|
||||
|
||||
val getTweetFieldsHandler =
|
||||
GetTweetFieldsHandler(
|
||||
logicalRepos.tweetResultRepo,
|
||||
logicalRepos.deletedTweetVisibilityRepo,
|
||||
logicalRepos.containerAsGetTweetFieldsResultRepo,
|
||||
statsReceiver.scope("read_path"),
|
||||
deciderGates.shouldMaterializeContainers
|
||||
)
|
||||
|
||||
val unretweetHandler =
|
||||
UnretweetHandler(
|
||||
tweetDeletePathHandler.deleteTweets,
|
||||
backendClients.timelineService.getPerspectives,
|
||||
tweetDeletePathHandler.unretweetEdits,
|
||||
logicalRepos.tweetRepo,
|
||||
)
|
||||
|
||||
val hydrateInsertEvent =
|
||||
WritePathHydration.hydrateInsertTweetEvent(
|
||||
hydrateTweet = hydrateTweetForInsert,
|
||||
hydrateQuotedTweet = hydrateQuotedTweet
|
||||
)
|
||||
|
||||
val scrubGeoUpdateUserTimestampBuilder =
|
||||
ScrubGeoEventBuilder.UpdateUserTimestamp(
|
||||
stats = tweetServiceScope.scope("scrub_geo_update_user_timestamp"),
|
||||
userRepo = logicalRepos.optionalUserRepo
|
||||
)
|
||||
|
||||
val scrubGeoScrubTweetsBuilder =
|
||||
ScrubGeoEventBuilder.ScrubTweets(
|
||||
stats = tweetServiceScope.scope("scrub_geo"),
|
||||
userRepo = logicalRepos.optionalUserRepo
|
||||
)
|
||||
|
||||
val handlerFilter =
|
||||
PostTweet
|
||||
.DuplicateHandler(
|
||||
tweetCreationLock = tweetCreationLock,
|
||||
getTweets = getTweetsHandler,
|
||||
stats = statsReceiver.scope("duplicate")
|
||||
)
|
||||
.andThen(PostTweet.RescueTweetCreateFailure)
|
||||
.andThen(PostTweet.LogFailures)
|
||||
|
||||
val postTweetHandler =
|
||||
handlerFilter[PostTweetRequest](
|
||||
PostTweet.Handler(
|
||||
tweetBuilder = tweetBuilders.tweetBuilder,
|
||||
hydrateInsertEvent = hydrateInsertEvent,
|
||||
tweetStore = tweetStores,
|
||||
)
|
||||
)
|
||||
|
||||
val postRetweetHandler =
|
||||
handlerFilter[RetweetRequest](
|
||||
PostTweet.Handler(
|
||||
tweetBuilder = tweetBuilders.retweetBuilder,
|
||||
hydrateInsertEvent = hydrateInsertEvent,
|
||||
tweetStore = tweetStores,
|
||||
)
|
||||
)
|
||||
|
||||
val quotedTweetDeleteBuilder: QuotedTweetDeleteEventBuilder.Type =
|
||||
QuotedTweetDeleteEventBuilder(logicalRepos.optionalTweetRepo)
|
||||
|
||||
val quotedTweetTakedownBuilder: QuotedTweetTakedownEventBuilder.Type =
|
||||
QuotedTweetTakedownEventBuilder(logicalRepos.optionalTweetRepo)
|
||||
|
||||
val setAdditionalFieldsBuilder: SetAdditionalFieldsBuilder.Type =
|
||||
SetAdditionalFieldsBuilder(
|
||||
tweetRepo = logicalRepos.tweetRepo
|
||||
)
|
||||
|
||||
val asyncSetAdditionalFieldsBuilder: AsyncSetAdditionalFieldsBuilder.Type =
|
||||
AsyncSetAdditionalFieldsBuilder(
|
||||
userRepo = logicalRepos.userRepo
|
||||
)
|
||||
|
||||
val deleteAdditionalFieldsBuilder: DeleteAdditionalFieldsBuilder.Type =
|
||||
DeleteAdditionalFieldsBuilder(
|
||||
tweetRepo = logicalRepos.tweetRepo
|
||||
)
|
||||
|
||||
val asyncDeleteAdditionalFieldsBuilder: AsyncDeleteAdditionalFieldsBuilder.Type =
|
||||
AsyncDeleteAdditionalFieldsBuilder(
|
||||
userRepo = logicalRepos.userRepo
|
||||
)
|
||||
|
||||
new DispatchingTweetService(
|
||||
asyncDeleteAdditionalFieldsBuilder = asyncDeleteAdditionalFieldsBuilder,
|
||||
asyncSetAdditionalFieldsBuilder = asyncSetAdditionalFieldsBuilder,
|
||||
deleteAdditionalFieldsBuilder = deleteAdditionalFieldsBuilder,
|
||||
deleteLocationDataHandler = deleteLocationDataHandler,
|
||||
deletePathHandler = tweetDeletePathHandler,
|
||||
eraseUserTweetsHandler = eraseUserTweetsHandler,
|
||||
getDeletedTweetsHandler = getDeletedTweetsHandler,
|
||||
getStoredTweetsHandler = getStoredTweetsHandler,
|
||||
getStoredTweetsByUserHandler = getStoredTweetsByUserHandler,
|
||||
getTweetsHandler = getTweetsHandler,
|
||||
getTweetFieldsHandler = getTweetFieldsHandler,
|
||||
getTweetCountsHandler = GetTweetCountsHandler(logicalRepos.tweetCountsRepo),
|
||||
postTweetHandler = postTweetHandler,
|
||||
postRetweetHandler = postRetweetHandler,
|
||||
quotedTweetDeleteBuilder = quotedTweetDeleteBuilder,
|
||||
quotedTweetTakedownBuilder = quotedTweetTakedownBuilder,
|
||||
scrubGeoUpdateUserTimestampBuilder = scrubGeoUpdateUserTimestampBuilder,
|
||||
scrubGeoScrubTweetsBuilder = scrubGeoScrubTweetsBuilder,
|
||||
setAdditionalFieldsBuilder = setAdditionalFieldsBuilder,
|
||||
setRetweetVisibilityHandler = setRetweetVisibilityHandler,
|
||||
statsReceiver = statsReceiver,
|
||||
takedownHandler = takedownHandler,
|
||||
tweetStore = tweetStores,
|
||||
undeleteTweetHandler = undeleteTweetHandler,
|
||||
unretweetHandler = unretweetHandler,
|
||||
updatePossiblySensitiveTweetHandler = updatePossiblySensitiveTweetHandler,
|
||||
userTakedownHandler = userTakedownHandler,
|
||||
clientIdHelper = clientIdHelper,
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
object TakedownHandlerBuilder {
|
||||
type Type = FutureArrow[TakedownRequest, Unit]
|
||||
|
||||
def apply(logicalRepos: LogicalRepositories, tweetStores: TotalTweetStore) =
|
||||
TakedownHandler(
|
||||
getTweet = HandlerError.getRequired(
|
||||
tweetGetter(logicalRepos),
|
||||
HandlerError.tweetNotFoundException
|
||||
),
|
||||
getUser = HandlerError.getRequired(
|
||||
userGetter(logicalRepos),
|
||||
HandlerError.userNotFoundException
|
||||
),
|
||||
writeTakedown = tweetStores.takedown
|
||||
)
|
||||
|
||||
def tweetGetter(logicalRepos: LogicalRepositories): FutureArrow[TweetId, Option[Tweet]] =
|
||||
FutureArrow(
|
||||
TweetRepository.tweetGetter(
|
||||
logicalRepos.optionalTweetRepo,
|
||||
TweetQuery.Options(
|
||||
include = GetTweetsHandler.BaseInclude.also(
|
||||
tweetFields = Set(
|
||||
Tweet.TweetypieOnlyTakedownCountryCodesField.id,
|
||||
Tweet.TweetypieOnlyTakedownReasonsField.id
|
||||
)
|
||||
)
|
||||
)
|
||||
)
|
||||
)
|
||||
|
||||
def userGetter(logicalRepos: LogicalRepositories): FutureArrow[UserId, Option[User]] =
|
||||
FutureArrow(
|
||||
UserRepository
|
||||
.userGetter(
|
||||
logicalRepos.optionalUserRepo,
|
||||
UserQueryOptions(
|
||||
Set(UserField.Roles, UserField.Safety, UserField.Takedowns),
|
||||
UserVisibility.All
|
||||
)
|
||||
)
|
||||
.compose(UserKey.byId)
|
||||
)
|
||||
}
|
||||
|
||||
object UserTakedownHandlerBuilder {
|
||||
def apply(
|
||||
logicalRepos: LogicalRepositories,
|
||||
tweetStores: TotalTweetStore,
|
||||
stats: StatsReceiver
|
||||
): UserTakedownHandler.Type =
|
||||
UserTakedownHandler(
|
||||
getTweet = TakedownHandlerBuilder.tweetGetter(logicalRepos),
|
||||
tweetTakedown = tweetStores.takedown,
|
||||
)
|
||||
}
|
||||
|
||||
object EraseUserTweetsHandlerBuilder {
|
||||
def apply(
|
||||
backendClients: BackendClients,
|
||||
asyncInvocationBuilder: ServiceInvocationBuilder,
|
||||
deciderGates: TweetypieDeciderGates,
|
||||
settings: TweetServiceSettings,
|
||||
timer: Timer,
|
||||
tweetDeletePathHandler: DefaultTweetDeletePathHandler,
|
||||
tweetServiceScope: StatsReceiver
|
||||
): EraseUserTweetsHandler =
|
||||
EraseUserTweetsHandler(
|
||||
selectPage(backendClients, settings),
|
||||
deleteTweet(tweetDeletePathHandler),
|
||||
eraseUserTweets(backendClients, asyncInvocationBuilder),
|
||||
tweetServiceScope.scope("erase_user_tweets"),
|
||||
sleep(deciderGates, settings, timer)
|
||||
)
|
||||
|
||||
def selectPage(
|
||||
backendClients: BackendClients,
|
||||
settings: TweetServiceSettings
|
||||
): FutureArrow[Select[StatusGraph], PageResult[Long]] =
|
||||
FutureArrow(
|
||||
backendClients.tflockWriteClient.selectPage(_, Some(settings.eraseUserTweetsPageSize))
|
||||
)
|
||||
|
||||
def deleteTweet(
|
||||
tweetDeletePathHandler: DefaultTweetDeletePathHandler
|
||||
): FutureEffect[(TweetId, UserId)] =
|
||||
FutureEffect[(TweetId, UserId)] {
|
||||
case (tweetId, expectedUserId) =>
|
||||
tweetDeletePathHandler
|
||||
.internalDeleteTweets(
|
||||
request = DeleteTweetsRequest(
|
||||
Seq(tweetId),
|
||||
isUserErasure = true,
|
||||
expectedUserId = Some(expectedUserId)
|
||||
),
|
||||
byUserId = None,
|
||||
authenticatedUserId = None,
|
||||
validate = tweetDeletePathHandler.validateTweetsForUserErasureDaemon
|
||||
)
|
||||
.unit
|
||||
}
|
||||
|
||||
def eraseUserTweets(
|
||||
backendClients: BackendClients,
|
||||
asyncInvocationBuilder: ServiceInvocationBuilder
|
||||
): FutureArrow[AsyncEraseUserTweetsRequest, Unit] =
|
||||
asyncInvocationBuilder
|
||||
.asyncVia(backendClients.asyncTweetDeletionService)
|
||||
.method(_.asyncEraseUserTweets)
|
||||
|
||||
def sleep(
|
||||
deciderGates: TweetypieDeciderGates,
|
||||
settings: TweetServiceSettings,
|
||||
timer: Timer
|
||||
): () => Future[Unit] =
|
||||
() =>
|
||||
if (deciderGates.delayEraseUserTweets()) {
|
||||
Future.sleep(settings.eraseUserTweetsDelay)(timer)
|
||||
} else {
|
||||
Future.Unit
|
||||
}
|
||||
}
|
||||
|
||||
object UndeleteTweetHandlerBuilder {
|
||||
def apply(
|
||||
tweetStorage: TweetStorageClient,
|
||||
logicalRepos: LogicalRepositories,
|
||||
tweetStores: TotalTweetStore,
|
||||
parentUserIdRepo: ParentUserIdRepository.Type,
|
||||
statsReceiver: StatsReceiver
|
||||
): UndeleteTweetHandler.Type =
|
||||
UndeleteTweetHandler(
|
||||
undelete = tweetStorage.undelete,
|
||||
tweetExists = tweetExists(tweetStorage),
|
||||
getUser = FutureArrow(
|
||||
UserRepository
|
||||
.userGetter(
|
||||
logicalRepos.optionalUserRepo,
|
||||
UserQueryOptions(
|
||||
// ExtendedProfile is needed to view a user's birthday to
|
||||
// guarantee we are not undeleting tweets from when a user was < 13
|
||||
TweetBuilder.userFields ++ Set(UserField.ExtendedProfile),
|
||||
UserVisibility.All,
|
||||
filteredAsFailure = false
|
||||
)
|
||||
)
|
||||
.compose(UserKey.byId)
|
||||
),
|
||||
getDeletedTweets = tweetStorage.getDeletedTweets,
|
||||
parentUserIdRepo = parentUserIdRepo,
|
||||
save = save(
|
||||
logicalRepos,
|
||||
tweetStores,
|
||||
statsReceiver
|
||||
)
|
||||
)
|
||||
|
||||
private def tweetExists(tweetStorage: TweetStorageClient): FutureArrow[TweetId, Boolean] =
|
||||
FutureArrow { id =>
|
||||
Stitch
|
||||
.run(tweetStorage.getTweet(id))
|
||||
.map {
|
||||
case _: GetTweet.Response.Found => true
|
||||
case _ => false
|
||||
}
|
||||
}
|
||||
|
||||
// 1. hydrates the undeleted tweet
|
||||
// 2. hands a UndeleteTweetEvent to relevant stores.
|
||||
// 3. return the hydrated tweet
|
||||
def save(
|
||||
logicalRepos: LogicalRepositories,
|
||||
tweetStores: TotalTweetStore,
|
||||
statsReceiver: StatsReceiver
|
||||
): FutureArrow[UndeleteTweet.Event, Tweet] = {
|
||||
|
||||
val hydrateTweet =
|
||||
WritePathHydration.hydrateTweet(
|
||||
logicalRepos.tweetHydrators.hydrator,
|
||||
statsReceiver.scope("undelete_tweet")
|
||||
)
|
||||
|
||||
val hydrateQuotedTweet =
|
||||
WritePathHydration.hydrateQuotedTweet(
|
||||
logicalRepos.optionalTweetRepo,
|
||||
logicalRepos.optionalUserRepo,
|
||||
logicalRepos.quoterHasAlreadyQuotedRepo
|
||||
)
|
||||
|
||||
val hydrateUndeleteEvent =
|
||||
WritePathHydration.hydrateUndeleteTweetEvent(
|
||||
hydrateTweet = hydrateTweet,
|
||||
hydrateQuotedTweet = hydrateQuotedTweet
|
||||
)
|
||||
|
||||
FutureArrow[UndeleteTweet.Event, Tweet] { event =>
|
||||
for {
|
||||
hydratedEvent <- hydrateUndeleteEvent(event)
|
||||
_ <- tweetStores.undeleteTweet(hydratedEvent)
|
||||
} yield hydratedEvent.tweet
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,34 @@
|
||||
package com.twitter.tweetypie.config
|
||||
|
||||
import com.twitter.finagle.thrift.ClientId
|
||||
import com.twitter.servo.util.FutureArrow
|
||||
import com.twitter.tweetypie._
|
||||
import com.twitter.tweetypie.service.{ClientIdSettingTweetServiceProxy, TweetServiceProxy}
|
||||
|
||||
/**
|
||||
* This class builds deciderable ThriftTweetService and FutureArrows that respect the
|
||||
* simulateDeferredrpcCallbacks decider. When simulateDeferredrpcCallbacks=true, invocations will
|
||||
* be performed synchronously by the root ThriftTweetService.
|
||||
*/
|
||||
class ServiceInvocationBuilder(
|
||||
val service: ThriftTweetService,
|
||||
simulateDeferredrpcCallbacks: Boolean) {
|
||||
|
||||
def withClientId(clientId: ClientId): ServiceInvocationBuilder =
|
||||
new ServiceInvocationBuilder(
|
||||
new ClientIdSettingTweetServiceProxy(clientId, service),
|
||||
simulateDeferredrpcCallbacks
|
||||
)
|
||||
|
||||
def asyncVia(asyncService: ThriftTweetService): ServiceInvocationBuilder =
|
||||
new ServiceInvocationBuilder(
|
||||
new TweetServiceProxy {
|
||||
override def underlying: ThriftTweetService =
|
||||
if (simulateDeferredrpcCallbacks) service else asyncService
|
||||
},
|
||||
simulateDeferredrpcCallbacks
|
||||
)
|
||||
|
||||
def method[A, B](op: ThriftTweetService => A => Future[B]): FutureArrow[A, B] =
|
||||
FutureArrow(op(service))
|
||||
}
|
@ -0,0 +1,475 @@
|
||||
package com.twitter.tweetypie
|
||||
package config
|
||||
|
||||
import com.twitter.conversions.DurationOps._
|
||||
import com.twitter.finagle.Backoff
|
||||
import com.twitter.finagle.memcached.exp.localMemcachedPort
|
||||
import com.twitter.finagle.mtls.authentication.ServiceIdentifier
|
||||
import com.twitter.finagle.ssl.OpportunisticTls
|
||||
import com.twitter.finagle.thrift.ClientId
|
||||
import com.twitter.flockdb.client.thriftscala.Priority
|
||||
import com.twitter.servo.repository.CachedResult
|
||||
import com.twitter.servo.util.Availability
|
||||
import com.twitter.tweetypie.backends._
|
||||
import com.twitter.tweetypie.caching.SoftTtl
|
||||
import com.twitter.tweetypie.handler.DuplicateTweetFinder
|
||||
import com.twitter.tweetypie.repository.TombstoneTtl
|
||||
import com.twitter.tweetypie.service._
|
||||
import com.twitter.tweetypie.storage.ManhattanTweetStorageClient
|
||||
import com.twitter.util.Duration
|
||||
|
||||
case class InProcessCacheConfig(ttl: Duration, maximumSize: Int)
|
||||
|
||||
class TweetServiceSettings(val flags: TweetServiceFlags) {
|
||||
|
||||
/**
|
||||
* Convert a Boolean to an Option
|
||||
* > optional(true, "my value")
|
||||
* res: Some(my value)
|
||||
*
|
||||
* > optional(false, "my value")
|
||||
* res: None
|
||||
*/
|
||||
def optional[T](b: Boolean, a: => T): Option[T] = if (b) Some(a) else None
|
||||
|
||||
/** atla, localhost, etc. */
|
||||
val zone: String = flags.zone()
|
||||
|
||||
/** dc is less specific than zone, zone=atla, dc=atl */
|
||||
val dc: String = zone.dropRight(1)
|
||||
|
||||
/** one of: prod, staging, dev, testbox */
|
||||
val env: Env.Value = flags.env()
|
||||
|
||||
/** instanceId of this aurora instance */
|
||||
lazy val instanceId: Int = flags.instanceId()
|
||||
|
||||
/** total number of tweetypie aurora instances */
|
||||
val instanceCount: Int = flags.instanceCount()
|
||||
|
||||
/** The Name to resolve to find the memcached cluster */
|
||||
val twemcacheDest: String =
|
||||
// If twemcacheDest is explicitly set, always prefer that to
|
||||
// localMemcachedPort.
|
||||
flags.twemcacheDest.get
|
||||
// Testbox uses this global flag to specify the location of the
|
||||
// local memcached instance.
|
||||
.orElse(localMemcachedPort().map("/$/inet/localhost/" + _))
|
||||
// If no explicit Name is specified, use the default.
|
||||
.getOrElse(flags.twemcacheDest())
|
||||
|
||||
/** Read/write data through Cache */
|
||||
val withCache: Boolean = flags.withCache()
|
||||
|
||||
/**
|
||||
* The TFlock queue to use for background indexing operations. For
|
||||
* production, this should always be the low priority queue, to
|
||||
* allow foreground operations to be processed first.
|
||||
*/
|
||||
val backgroundIndexingPriority: Priority = flags.backgroundIndexingPriority()
|
||||
|
||||
/** Set certain decider gates to this overridden value */
|
||||
val deciderOverrides: Map[String, Boolean] =
|
||||
flags.deciderOverrides()
|
||||
|
||||
/** use per host stats? */
|
||||
val clientHostStats: Boolean =
|
||||
flags.clientHostStats()
|
||||
|
||||
val warmupRequestsSettings: Option[WarmupQueriesSettings] =
|
||||
optional(flags.enableWarmupRequests(), WarmupQueriesSettings())
|
||||
|
||||
/** enables request authorization via a allowlist */
|
||||
val allowlistingRequired: Boolean =
|
||||
flags.allowlist.get.getOrElse(env == Env.prod)
|
||||
|
||||
/** read rate limit for unknown clients (when allowlistingRequired is enabled) */
|
||||
val nonAllowListedClientRateLimitPerSec: Double =
|
||||
flags.grayListRateLimit()
|
||||
|
||||
/** enables requests from production clients */
|
||||
val allowProductionClients: Boolean =
|
||||
env == Env.prod
|
||||
|
||||
/** enables replication via DRPC */
|
||||
val enableReplication: Boolean = flags.enableReplication()
|
||||
|
||||
/** enables forking of some traffic to configured target */
|
||||
val trafficForkingEnabled: Boolean =
|
||||
env == Env.prod
|
||||
|
||||
val scribeUniquenessIds: Boolean =
|
||||
env == Env.prod
|
||||
|
||||
/** ClientId to send to backend services */
|
||||
val thriftClientId: ClientId =
|
||||
flags.clientId.get.map(ClientId(_)).getOrElse {
|
||||
env match {
|
||||
case Env.dev | Env.staging => ClientId("tweetypie.staging")
|
||||
case Env.prod => ClientId("tweetypie.prod")
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Instead of using DRPC for calling into the async code path, call back into the
|
||||
* current instance. Used for development and test to ensure logic in the current
|
||||
* instance is being tested.
|
||||
*/
|
||||
val simulateDeferredrpcCallbacks: Boolean = flags.simulateDeferredrpcCallbacks()
|
||||
|
||||
/**
|
||||
* ClientId to set in 'asynchronous' requests when simulateDeferredrpcCallbacks is
|
||||
* true and Tweetypie ends up just calling itself synchronously.
|
||||
*/
|
||||
val deferredrpcClientId: ClientId = ClientId("deferredrpc.prod")
|
||||
|
||||
/**
|
||||
* ServiceIdentifier used to enable mTLS
|
||||
*/
|
||||
val serviceIdentifier: ServiceIdentifier = flags.serviceIdentifier()
|
||||
|
||||
/**
|
||||
* Decider settings
|
||||
*/
|
||||
val deciderBaseFilename: Option[String] = Option(flags.deciderBase())
|
||||
val deciderOverlayFilename: Option[String] = Option(flags.deciderOverlay())
|
||||
val vfDeciderOverlayFilename: Option[String] = flags.vfDeciderOverlay.get
|
||||
|
||||
/**
|
||||
* Used to determine whether we should fail requests for Tweets that are likely too young
|
||||
* to return a non-partial response. We return NotFound for Tweets that are deemed too young.
|
||||
* Used by [[com.twitter.tweetypie.repository.ManhattanTweetRepository]].
|
||||
*/
|
||||
val shortCircuitLikelyPartialTweetReads: Gate[Duration] = {
|
||||
// interpret the flag as a duration in milliseconds
|
||||
val ageCeiling: Duration = flags.shortCircuitLikelyPartialTweetReadsMs().milliseconds
|
||||
Gate(tweetAge => tweetAge < ageCeiling)
|
||||
}
|
||||
|
||||
// tweet-service internal settings
|
||||
|
||||
val tweetKeyCacheVersion = 1
|
||||
|
||||
/** how often to flush aggregated count updates for tweet counts */
|
||||
val aggregatedTweetCountsFlushInterval: Duration = 5.seconds
|
||||
|
||||
/** maximum number of keys for which aggregated cached count updates may be cached */
|
||||
val maxAggregatedCountsSize = 1000
|
||||
|
||||
/** ramp up period for decidering up forked traffic (if enabled) to the full decidered value */
|
||||
val forkingRampUp: Duration = 3.minutes
|
||||
|
||||
/** how long to wait after startup for serversets to resolve before giving up and moving on */
|
||||
val waitForServerSetsTimeout: Duration = 120.seconds
|
||||
|
||||
/** number of threads to use in thread pool for language identification */
|
||||
val numPenguinThreads = 4
|
||||
|
||||
/** maximum number of tweets that clients can request per getTweets RPC call */
|
||||
val maxGetTweetsRequestSize = 200
|
||||
|
||||
/** maximum batch size for any batched request (getTweets is exempt, it has its own limiting) */
|
||||
val maxRequestSize = 200
|
||||
|
||||
/**
|
||||
* maximum size to allow the thrift response buffer to grow before resetting it. this is set to
|
||||
* approximately the current value of `srv/thrift/response_payload_bytes.p999`, meaning roughly
|
||||
* 1 out of 1000 requests will cause the buffer to be reset.
|
||||
*/
|
||||
val maxThriftBufferSize: Int = 200 * 1024
|
||||
|
||||
// ********* timeouts and backoffs **********
|
||||
|
||||
/** backoffs for OptimisticLockingCache lockAndSet operations */
|
||||
val lockingCacheBackoffs: Stream[Duration] =
|
||||
Backoff.exponentialJittered(10.millisecond, 50.milliseconds).take(3).toStream
|
||||
|
||||
/** retry once on timeout with no backoff */
|
||||
val defaultTimeoutBackoffs: Stream[Duration] = Stream(0.milliseconds).toStream
|
||||
|
||||
/** backoffs when user view is missing */
|
||||
val gizmoduckMissingUserViewBackoffs: Stream[Duration] = Backoff.const(10.millis).take(3).toStream
|
||||
|
||||
/** backoffs for retrying failed async-write actions after first retry failure */
|
||||
val asyncWriteRetryBackoffs: Stream[Duration] =
|
||||
Backoff.exponential(10.milliseconds, 2).take(9).toStream.map(_ min 1.second)
|
||||
|
||||
/** backoffs for retrying failed deferredrpc enqueues */
|
||||
val deferredrpcBackoffs: Stream[Duration] =
|
||||
Backoff.exponential(10.milliseconds, 2).take(3).toStream
|
||||
|
||||
/** backoffs for retrying failed cache updates for replicated events */
|
||||
val replicatedEventCacheBackoffs: Stream[Duration] =
|
||||
Backoff.exponential(100.milliseconds, 2).take(10).toStream
|
||||
|
||||
val escherbirdConfig: Escherbird.Config =
|
||||
Escherbird.Config(
|
||||
requestTimeout = 200.milliseconds,
|
||||
timeoutBackoffs = defaultTimeoutBackoffs
|
||||
)
|
||||
|
||||
val expandodoConfig: Expandodo.Config =
|
||||
Expandodo.Config(
|
||||
requestTimeout = 300.milliseconds,
|
||||
timeoutBackoffs = defaultTimeoutBackoffs,
|
||||
serverErrorBackoffs = Backoff.const(0.millis).take(3).toStream
|
||||
)
|
||||
|
||||
val creativesContainerServiceConfig: CreativesContainerService.Config =
|
||||
CreativesContainerService.Config(
|
||||
requestTimeout = 300.milliseconds,
|
||||
timeoutBackoffs = defaultTimeoutBackoffs,
|
||||
serverErrorBackoffs = Backoff.const(0.millis).take(3).toStream
|
||||
)
|
||||
|
||||
val geoScrubEventStoreConfig: GeoScrubEventStore.Config =
|
||||
GeoScrubEventStore.Config(
|
||||
read = GeoScrubEventStore.EndpointConfig(
|
||||
requestTimeout = 200.milliseconds,
|
||||
maxRetryCount = 1
|
||||
),
|
||||
write = GeoScrubEventStore.EndpointConfig(
|
||||
requestTimeout = 1.second,
|
||||
maxRetryCount = 1
|
||||
)
|
||||
)
|
||||
|
||||
val gizmoduckConfig: Gizmoduck.Config =
|
||||
Gizmoduck.Config(
|
||||
readTimeout = 300.milliseconds,
|
||||
writeTimeout = 300.milliseconds,
|
||||
// We bump the timeout value to 800ms because modifyAndGet is called only in async request path in GeoScrub daemon
|
||||
// and we do not expect sync/realtime apps calling this thrift method
|
||||
modifyAndGetTimeout = 800.milliseconds,
|
||||
modifyAndGetTimeoutBackoffs = Backoff.const(0.millis).take(3).toStream,
|
||||
defaultTimeoutBackoffs = defaultTimeoutBackoffs,
|
||||
gizmoduckExceptionBackoffs = Backoff.const(0.millis).take(3).toStream
|
||||
)
|
||||
|
||||
val limiterBackendConfig: LimiterBackend.Config =
|
||||
LimiterBackend.Config(
|
||||
requestTimeout = 300.milliseconds,
|
||||
timeoutBackoffs = defaultTimeoutBackoffs
|
||||
)
|
||||
|
||||
val mediaInfoServiceConfig: MediaInfoService.Config =
|
||||
MediaInfoService.Config(
|
||||
requestTimeout = 300.milliseconds,
|
||||
totalTimeout = 500.milliseconds,
|
||||
timeoutBackoffs = defaultTimeoutBackoffs
|
||||
)
|
||||
|
||||
val scarecrowConfig: Scarecrow.Config =
|
||||
Scarecrow.Config(
|
||||
readTimeout = 100.milliseconds,
|
||||
writeTimeout = 400.milliseconds,
|
||||
timeoutBackoffs = defaultTimeoutBackoffs,
|
||||
scarecrowExceptionBackoffs = Backoff.const(0.millis).take(3).toStream
|
||||
)
|
||||
|
||||
val socialGraphSeviceConfig: SocialGraphService.Config =
|
||||
SocialGraphService.Config(
|
||||
socialGraphTimeout = 250.milliseconds,
|
||||
timeoutBackoffs = defaultTimeoutBackoffs
|
||||
)
|
||||
|
||||
val talonConfig: Talon.Config =
|
||||
Talon.Config(
|
||||
shortenTimeout = 500.milliseconds,
|
||||
expandTimeout = 150.milliseconds,
|
||||
timeoutBackoffs = defaultTimeoutBackoffs,
|
||||
transientErrorBackoffs = Backoff.const(0.millis).take(3).toStream
|
||||
)
|
||||
|
||||
/**
|
||||
* page size when retrieving tflock pages for tweet deletion and undeletion
|
||||
* tweet erasures have their own page size eraseUserTweetsPageSize
|
||||
*/
|
||||
val tflockPageSize: Int = flags.tflockPageSize()
|
||||
|
||||
val tflockReadConfig: TFlock.Config =
|
||||
TFlock.Config(
|
||||
requestTimeout = 300.milliseconds,
|
||||
timeoutBackoffs = defaultTimeoutBackoffs,
|
||||
flockExceptionBackoffs = Backoff.const(0.millis).take(3).toStream,
|
||||
overCapacityBackoffs = Stream.empty,
|
||||
defaultPageSize = tflockPageSize
|
||||
)
|
||||
|
||||
val tflockWriteConfig: TFlock.Config =
|
||||
TFlock.Config(
|
||||
requestTimeout = 400.milliseconds,
|
||||
timeoutBackoffs = defaultTimeoutBackoffs,
|
||||
flockExceptionBackoffs = Backoff.const(0.millis).take(3).toStream,
|
||||
overCapacityBackoffs = Backoff.exponential(10.millis, 2).take(3).toStream
|
||||
)
|
||||
|
||||
val timelineServiceConfig: TimelineService.Config = {
|
||||
val tlsExceptionBackoffs = Backoff.const(0.millis).take(3).toStream
|
||||
TimelineService.Config(
|
||||
writeRequestPolicy =
|
||||
Backend.TimeoutPolicy(4.seconds) >>>
|
||||
TimelineService.FailureBackoffsPolicy(
|
||||
timeoutBackoffs = defaultTimeoutBackoffs,
|
||||
tlsExceptionBackoffs = tlsExceptionBackoffs
|
||||
),
|
||||
readRequestPolicy =
|
||||
Backend.TimeoutPolicy(400.milliseconds) >>>
|
||||
TimelineService.FailureBackoffsPolicy(
|
||||
timeoutBackoffs = defaultTimeoutBackoffs,
|
||||
tlsExceptionBackoffs = tlsExceptionBackoffs
|
||||
)
|
||||
)
|
||||
}
|
||||
|
||||
val tweetStorageConfig: ManhattanTweetStorageClient.Config = {
|
||||
val remoteZone = zone match {
|
||||
case "atla" => "pdxa"
|
||||
case "pdxa" => "atla"
|
||||
case "atla" | "localhost" => "atla"
|
||||
case _ =>
|
||||
throw new IllegalArgumentException(s"Cannot configure remote DC for unknown zone '$zone'")
|
||||
}
|
||||
ManhattanTweetStorageClient.Config(
|
||||
applicationId = "tbird_mh",
|
||||
localDestination = "/s/manhattan/cylon.native-thrift",
|
||||
localTimeout = 290.milliseconds,
|
||||
remoteDestination = s"/srv#/prod/$remoteZone/manhattan/cylon.native-thrift",
|
||||
remoteTimeout = 1.second,
|
||||
maxRequestsPerBatch = 25,
|
||||
serviceIdentifier = serviceIdentifier,
|
||||
opportunisticTlsLevel = OpportunisticTls.Required
|
||||
)
|
||||
}
|
||||
|
||||
val userImageServiceConfig: UserImageService.Config =
|
||||
UserImageService.Config(
|
||||
processTweetMediaTimeout = 5.seconds,
|
||||
updateTweetMediaTimeout = 2.seconds,
|
||||
timeoutBackoffs = defaultTimeoutBackoffs
|
||||
)
|
||||
|
||||
val adsLoggingClientTopicName = env match {
|
||||
case Env.prod => "ads_client_callback_prod"
|
||||
case Env.dev | Env.staging => "ads_client_callback_staging"
|
||||
}
|
||||
|
||||
/** Delay between successive cascadedDeleteTweet calls when deleting retweets. Applied via decider. */
|
||||
val retweetDeletionDelay: Duration = 20.milliseconds
|
||||
|
||||
/**
|
||||
* Delay to sleep before each tweet deletion of an eraseUserTweets request.
|
||||
* This is a simple rate limiting mechanism. The long term solution is
|
||||
* to move async endpoints like user erasures and retweet deletions out
|
||||
* of the the main tweetypie cluster and into an async cluster with first class
|
||||
* rate limiting support
|
||||
*/
|
||||
val eraseUserTweetsDelay: Duration = 100.milliseconds
|
||||
|
||||
val eraseUserTweetsPageSize = 100
|
||||
|
||||
val getStoredTweetsByUserPageSize = 20
|
||||
val getStoredTweetsByUserMaxPages = 30
|
||||
|
||||
// ********* ttls **********
|
||||
|
||||
// Unfortunately, this tombstone TTL applies equally to the case
|
||||
// where the tweet was deleted and the case that the tweet does not
|
||||
// exist or is unavailable. If we could differentiate between those
|
||||
// cases, we'd cache deleted for a long time and not
|
||||
// found/unavailable for a short time. We chose 100
|
||||
// milliseconds for the minimum TTL because there are known cases in
|
||||
// which a not found result can be erroneously written to cache on
|
||||
// tweet creation. This minimum TTL is a trade-off between a
|
||||
// thundering herd of database requests from clients that just got
|
||||
// the fanned-out tweet and the window for which these inconsistent
|
||||
// results will be available.
|
||||
val tweetTombstoneTtl: CachedResult.CachedNotFound[TweetId] => Duration =
|
||||
TombstoneTtl.linear(min = 100.milliseconds, max = 1.day, from = 5.minutes, to = 5.hours)
|
||||
|
||||
val tweetMemcacheTtl: Duration = 14.days
|
||||
val urlMemcacheTtl: Duration = 1.hour
|
||||
val urlMemcacheSoftTtl: Duration = 1.hour
|
||||
val deviceSourceMemcacheTtl: Duration = 12.hours
|
||||
val deviceSourceMemcacheSoftTtl: SoftTtl.ByAge[Nothing] =
|
||||
SoftTtl.ByAge(softTtl = 1.hour, jitter = 1.minute)
|
||||
val deviceSourceInProcessTtl: Duration = 8.hours
|
||||
val deviceSourceInProcessSoftTtl: Duration = 30.minutes
|
||||
val placeMemcacheTtl: Duration = 1.day
|
||||
val placeMemcacheSoftTtl: SoftTtl.ByAge[Nothing] =
|
||||
SoftTtl.ByAge(softTtl = 3.hours, jitter = 1.minute)
|
||||
val cardMemcacheTtl: Duration = 20.minutes
|
||||
val cardMemcacheSoftTtl: Duration = 30.seconds
|
||||
val tweetCreateLockingMemcacheTtl: Duration = 10.seconds
|
||||
val tweetCreateLockingMemcacheLongTtl: Duration = 12.hours
|
||||
val geoScrubMemcacheTtl: Duration = 30.minutes
|
||||
|
||||
val tweetCountsMemcacheTtl: Duration = 24.hours
|
||||
val tweetCountsMemcacheNonZeroSoftTtl: Duration = 3.hours
|
||||
val tweetCountsMemcacheZeroSoftTtl: Duration = 7.hours
|
||||
|
||||
val cacheClientPendingRequestLimit: Int = flags.memcachePendingRequestLimit()
|
||||
|
||||
val deviceSourceInProcessCacheMaxSize = 10000
|
||||
|
||||
val inProcessCacheConfigOpt: Option[InProcessCacheConfig] =
|
||||
if (flags.enableInProcessCache()) {
|
||||
Some(
|
||||
InProcessCacheConfig(
|
||||
ttl = flags.inProcessCacheTtlMs().milliseconds,
|
||||
maximumSize = flags.inProcessCacheSize()
|
||||
)
|
||||
)
|
||||
} else {
|
||||
None
|
||||
}
|
||||
|
||||
// Begin returning OverCapacity for tweet repo when cache SR falls below 95%,
|
||||
// Scale to rejecting 95% of requests when cache SR <= 80%
|
||||
val tweetCacheAvailabilityFromSuccessRate: Double => Double =
|
||||
Availability.linearlyScaled(0.95, 0.80, 0.05)
|
||||
|
||||
// ******* repository chunking size ********
|
||||
|
||||
val tweetCountsRepoChunkSize = 6
|
||||
// n times `tweetCountsRepoChunkSize`, so chunking at higher level does not
|
||||
// generate small batches at lower level.
|
||||
val tweetCountsCacheChunkSize = 18
|
||||
|
||||
val duplicateTweetFinderSettings: DuplicateTweetFinder.Settings =
|
||||
DuplicateTweetFinder.Settings(numTweetsToCheck = 10, maxDuplicateAge = 12.hours)
|
||||
|
||||
val backendWarmupSettings: Warmup.Settings =
|
||||
Warmup.Settings(
|
||||
// Try for twenty seconds to warm up the backends before giving
|
||||
// up.
|
||||
maxWarmupDuration = 20.seconds,
|
||||
// Only allow up to 50 outstanding warmup requests of any kind
|
||||
// to be outstanding at a time.
|
||||
maxOutstandingRequests = 50,
|
||||
// These timeouts are just over the p999 latency observed in ATLA
|
||||
// for requests to these backends.
|
||||
requestTimeouts = Map(
|
||||
"expandodo" -> 120.milliseconds,
|
||||
"geo_relevance" -> 50.milliseconds,
|
||||
"gizmoduck" -> 200.milliseconds,
|
||||
"memcache" -> 50.milliseconds,
|
||||
"scarecrow" -> 120.milliseconds,
|
||||
"socialgraphservice" -> 180.milliseconds,
|
||||
"talon" -> 70.milliseconds,
|
||||
"tflock" -> 320.milliseconds,
|
||||
"timelineservice" -> 200.milliseconds,
|
||||
"tweetstorage" -> 50.milliseconds
|
||||
),
|
||||
reliability = Warmup.Reliably(
|
||||
// Consider a backend warmed up if 99% of requests are succeeding.
|
||||
reliabilityThreshold = 0.99,
|
||||
// When performing warmup, use a maximum of 10 concurrent
|
||||
// requests to each backend.
|
||||
concurrency = 10,
|
||||
// Do not allow more than this many attempts to perform the
|
||||
// warmup action before giving up.
|
||||
maxAttempts = 1000
|
||||
)
|
||||
)
|
||||
}
|
@ -0,0 +1,577 @@
|
||||
package com.twitter.tweetypie
|
||||
package config
|
||||
|
||||
import com.twitter.servo.util.FutureArrow
|
||||
import com.twitter.servo.util.RetryHandler
|
||||
import com.twitter.servo.util.Scribe
|
||||
import com.twitter.tweetypie.backends.LimiterService.Feature.MediaTagCreate
|
||||
import com.twitter.tweetypie.backends.LimiterService.Feature.Updates
|
||||
import com.twitter.tweetypie.client_id.ClientIdHelper
|
||||
import com.twitter.tweetypie.handler.TweetBuilder
|
||||
import com.twitter.tweetypie.repository.TweetKeyFactory
|
||||
import com.twitter.tweetypie.store._
|
||||
import com.twitter.tweetypie.tflock.TFlockIndexer
|
||||
import com.twitter.tweetypie.thriftscala._
|
||||
import com.twitter.tweetypie.util.RetryPolicyBuilder
|
||||
import com.twitter.util.Timer
|
||||
|
||||
object TweetStores {
|
||||
def apply(
|
||||
settings: TweetServiceSettings,
|
||||
statsReceiver: StatsReceiver,
|
||||
timer: Timer,
|
||||
deciderGates: TweetypieDeciderGates,
|
||||
tweetKeyFactory: TweetKeyFactory,
|
||||
clients: BackendClients,
|
||||
caches: Caches,
|
||||
asyncBuilder: ServiceInvocationBuilder,
|
||||
hasMedia: Tweet => Boolean,
|
||||
clientIdHelper: ClientIdHelper,
|
||||
): TotalTweetStore = {
|
||||
|
||||
val deferredrpcRetryPolicy =
|
||||
// retry all application exceptions for now. however, in the future, deferredrpc
|
||||
// may throw a backpressure exception that should not be retried.
|
||||
RetryPolicyBuilder.anyFailure(settings.deferredrpcBackoffs)
|
||||
|
||||
val asyncWriteRetryPolicy =
|
||||
// currently retries all failures with the same back-off times. might need
|
||||
// to update to handle backpressure exceptions differently.
|
||||
RetryPolicyBuilder.anyFailure(settings.asyncWriteRetryBackoffs)
|
||||
|
||||
val replicatedEventRetryPolicy =
|
||||
RetryPolicyBuilder.anyFailure(settings.replicatedEventCacheBackoffs)
|
||||
|
||||
val logLensStore =
|
||||
LogLensStore(
|
||||
tweetCreationsLogger = Logger("com.twitter.tweetypie.store.TweetCreations"),
|
||||
tweetDeletionsLogger = Logger("com.twitter.tweetypie.store.TweetDeletions"),
|
||||
tweetUndeletionsLogger = Logger("com.twitter.tweetypie.store.TweetUndeletions"),
|
||||
tweetUpdatesLogger = Logger("com.twitter.tweetypie.store.TweetUpdates"),
|
||||
clientIdHelper = clientIdHelper,
|
||||
)
|
||||
|
||||
val tweetStoreStats = statsReceiver.scope("tweet_store")
|
||||
|
||||
val tweetStatsStore = TweetStatsStore(tweetStoreStats.scope("stats"))
|
||||
|
||||
val asyncRetryConfig =
|
||||
new TweetStore.AsyncRetry(
|
||||
asyncWriteRetryPolicy,
|
||||
deferredrpcRetryPolicy,
|
||||
timer,
|
||||
clients.asyncRetryTweetService,
|
||||
Scribe(FailedAsyncWrite, "tweetypie_failed_async_writes")
|
||||
)(_, _)
|
||||
|
||||
val manhattanStore = {
|
||||
val scopedStats = tweetStoreStats.scope("base")
|
||||
ManhattanTweetStore(clients.tweetStorageClient)
|
||||
.tracked(scopedStats)
|
||||
.asyncRetry(asyncRetryConfig(scopedStats, ManhattanTweetStore.Action))
|
||||
}
|
||||
|
||||
val cachingTweetStore = {
|
||||
val cacheStats = tweetStoreStats.scope("caching")
|
||||
CachingTweetStore(
|
||||
tweetKeyFactory = tweetKeyFactory,
|
||||
tweetCache = caches.tweetCache,
|
||||
stats = cacheStats
|
||||
).tracked(cacheStats)
|
||||
.asyncRetry(asyncRetryConfig(cacheStats, CachingTweetStore.Action))
|
||||
.replicatedRetry(RetryHandler.failuresOnly(replicatedEventRetryPolicy, timer, cacheStats))
|
||||
}
|
||||
|
||||
val indexingStore = {
|
||||
val indexingStats = tweetStoreStats.scope("indexing")
|
||||
TweetIndexingStore(
|
||||
new TFlockIndexer(
|
||||
tflock = clients.tflockWriteClient,
|
||||
hasMedia = hasMedia,
|
||||
backgroundIndexingPriority = settings.backgroundIndexingPriority,
|
||||
stats = indexingStats
|
||||
)
|
||||
).tracked(indexingStats)
|
||||
.asyncRetry(asyncRetryConfig(indexingStats, TweetIndexingStore.Action))
|
||||
}
|
||||
|
||||
val timelineUpdatingStore = {
|
||||
val tlsScope = tweetStoreStats.scope("timeline_updating")
|
||||
TlsTimelineUpdatingStore(
|
||||
processEvent2 = clients.timelineService.processEvent2,
|
||||
hasMedia = hasMedia,
|
||||
stats = tlsScope
|
||||
).tracked(tlsScope)
|
||||
.asyncRetry(asyncRetryConfig(tlsScope, TlsTimelineUpdatingStore.Action))
|
||||
}
|
||||
|
||||
val guanoServiceStore = {
|
||||
val guanoStats = tweetStoreStats.scope("guano")
|
||||
GuanoServiceStore(clients.guano, guanoStats)
|
||||
.tracked(guanoStats)
|
||||
.asyncRetry(asyncRetryConfig(guanoStats, GuanoServiceStore.Action))
|
||||
}
|
||||
|
||||
val mediaServiceStore = {
|
||||
val mediaStats = tweetStoreStats.scope("media")
|
||||
MediaServiceStore(clients.mediaClient.deleteMedia, clients.mediaClient.undeleteMedia)
|
||||
.tracked(mediaStats)
|
||||
.asyncRetry(asyncRetryConfig(mediaStats, MediaServiceStore.Action))
|
||||
}
|
||||
|
||||
val userCountsUpdatingStore = {
|
||||
val userCountsStats = tweetStoreStats.scope("user_counts")
|
||||
GizmoduckUserCountsUpdatingStore(clients.gizmoduck.incrCount, hasMedia)
|
||||
.tracked(userCountsStats)
|
||||
.ignoreFailures
|
||||
}
|
||||
|
||||
val tweetCountsUpdatingStore = {
|
||||
val cacheScope = statsReceiver.scope("tweet_counts_cache")
|
||||
val tweetCountsStats = tweetStoreStats.scope("tweet_counts")
|
||||
|
||||
val memcacheCountsStore = {
|
||||
val lockingCacheCountsStore =
|
||||
CachedCountsStore.fromLockingCache(caches.tweetCountsCache)
|
||||
|
||||
new AggregatingCachedCountsStore(
|
||||
lockingCacheCountsStore,
|
||||
timer,
|
||||
settings.aggregatedTweetCountsFlushInterval,
|
||||
settings.maxAggregatedCountsSize,
|
||||
cacheScope
|
||||
)
|
||||
}
|
||||
|
||||
TweetCountsCacheUpdatingStore(memcacheCountsStore)
|
||||
.tracked(tweetCountsStats)
|
||||
.ignoreFailures
|
||||
}
|
||||
|
||||
val replicatingStore = {
|
||||
val replicateStats = tweetStoreStats.scope("replicate_out")
|
||||
ReplicatingTweetStore(
|
||||
clients.replicationClient
|
||||
).tracked(replicateStats)
|
||||
.retry(RetryHandler.failuresOnly(deferredrpcRetryPolicy, timer, replicateStats))
|
||||
.asyncRetry(asyncRetryConfig(replicateStats, ReplicatingTweetStore.Action))
|
||||
.enabledBy(Gate.const(settings.enableReplication))
|
||||
}
|
||||
|
||||
val scribeMediaTagStore =
|
||||
ScribeMediaTagStore()
|
||||
.tracked(tweetStoreStats.scope("scribe_media_tag_store"))
|
||||
|
||||
val limiterStore =
|
||||
LimiterStore(
|
||||
clients.limiterService.incrementByOne(Updates),
|
||||
clients.limiterService.increment(MediaTagCreate)
|
||||
).tracked(tweetStoreStats.scope("limiter_store"))
|
||||
|
||||
val geoSearchRequestIDStore = {
|
||||
val statsScope = tweetStoreStats.scope("geo_search_request_id")
|
||||
GeoSearchRequestIDStore(FutureArrow(clients.geoRelevance.reportConversion _))
|
||||
.tracked(statsScope)
|
||||
.asyncRetry(asyncRetryConfig(statsScope, GeoSearchRequestIDStore.Action))
|
||||
}
|
||||
|
||||
val userGeotagUpdateStore = {
|
||||
val geotagScope = tweetStoreStats.scope("gizmoduck_user_geotag_updating")
|
||||
GizmoduckUserGeotagUpdateStore(
|
||||
clients.gizmoduck.modifyAndGet,
|
||||
geotagScope
|
||||
).tracked(geotagScope)
|
||||
.asyncRetry(asyncRetryConfig(geotagScope, GizmoduckUserGeotagUpdateStore.Action))
|
||||
}
|
||||
|
||||
val fanoutServiceStore = {
|
||||
val fanoutStats = tweetStoreStats.scope("fanout_service_delivery")
|
||||
FanoutServiceStore(clients.fanoutServiceClient, fanoutStats)
|
||||
.tracked(fanoutStats)
|
||||
.asyncRetry(asyncRetryConfig(fanoutStats, FanoutServiceStore.Action))
|
||||
}
|
||||
|
||||
/**
|
||||
* A store that converts Tweetypie TweetEvents to EventBus TweetEvents and sends each event to
|
||||
* the underlying FutureEffect[eventbus.TweetEvent]
|
||||
*/
|
||||
val eventBusEnqueueStore = {
|
||||
val enqueueStats = tweetStoreStats.scope("event_bus_enqueueing")
|
||||
val enqueueEffect = FutureEffect[TweetEvent](clients.tweetEventsPublisher.publish)
|
||||
|
||||
TweetEventBusStore(
|
||||
enqueueEffect
|
||||
).tracked(enqueueStats)
|
||||
.asyncRetry(asyncRetryConfig(enqueueStats, AsyncWriteAction.EventBusEnqueue))
|
||||
}
|
||||
|
||||
val retweetArchivalEnqueueStore = {
|
||||
val enqueueStats = tweetStoreStats.scope("retweet_archival_enqueueing")
|
||||
val enqueueEffect = FutureEffect(clients.retweetArchivalEventPublisher.publish)
|
||||
|
||||
RetweetArchivalEnqueueStore(enqueueEffect)
|
||||
.tracked(enqueueStats)
|
||||
.asyncRetry(asyncRetryConfig(enqueueStats, AsyncWriteAction.RetweetArchivalEnqueue))
|
||||
}
|
||||
|
||||
val asyncEnqueueStore = {
|
||||
val asyncEnqueueStats = tweetStoreStats.scope("async_enqueueing")
|
||||
AsyncEnqueueStore(
|
||||
asyncBuilder.asyncVia(clients.asyncTweetService).service,
|
||||
TweetBuilder.scrubUserInAsyncInserts,
|
||||
TweetBuilder.scrubSourceTweetInAsyncInserts,
|
||||
TweetBuilder.scrubSourceUserInAsyncInserts
|
||||
).tracked(asyncEnqueueStats)
|
||||
.retry(RetryHandler.failuresOnly(deferredrpcRetryPolicy, timer, asyncEnqueueStats))
|
||||
}
|
||||
|
||||
val insertTweetStore =
|
||||
InsertTweet.Store(
|
||||
logLensStore = logLensStore,
|
||||
manhattanStore = manhattanStore,
|
||||
tweetStatsStore = tweetStatsStore,
|
||||
cachingTweetStore = cachingTweetStore,
|
||||
limiterStore = limiterStore,
|
||||
asyncEnqueueStore = asyncEnqueueStore,
|
||||
userCountsUpdatingStore = userCountsUpdatingStore,
|
||||
tweetCountsUpdatingStore = tweetCountsUpdatingStore
|
||||
)
|
||||
|
||||
val asyncInsertStore =
|
||||
AsyncInsertTweet.Store(
|
||||
replicatingStore = replicatingStore,
|
||||
indexingStore = indexingStore,
|
||||
tweetCountsUpdatingStore = tweetCountsUpdatingStore,
|
||||
timelineUpdatingStore = timelineUpdatingStore,
|
||||
eventBusEnqueueStore = eventBusEnqueueStore,
|
||||
fanoutServiceStore = fanoutServiceStore,
|
||||
scribeMediaTagStore = scribeMediaTagStore,
|
||||
userGeotagUpdateStore = userGeotagUpdateStore,
|
||||
geoSearchRequestIDStore = geoSearchRequestIDStore
|
||||
)
|
||||
|
||||
val replicatedInsertTweetStore =
|
||||
ReplicatedInsertTweet.Store(
|
||||
cachingTweetStore = cachingTweetStore,
|
||||
tweetCountsUpdatingStore = tweetCountsUpdatingStore
|
||||
)
|
||||
|
||||
val deleteTweetStore =
|
||||
DeleteTweet.Store(
|
||||
cachingTweetStore = cachingTweetStore,
|
||||
asyncEnqueueStore = asyncEnqueueStore,
|
||||
userCountsUpdatingStore = userCountsUpdatingStore,
|
||||
tweetCountsUpdatingStore = tweetCountsUpdatingStore,
|
||||
logLensStore = logLensStore
|
||||
)
|
||||
|
||||
val asyncDeleteTweetStore =
|
||||
AsyncDeleteTweet.Store(
|
||||
manhattanStore = manhattanStore,
|
||||
cachingTweetStore = cachingTweetStore,
|
||||
replicatingStore = replicatingStore,
|
||||
indexingStore = indexingStore,
|
||||
eventBusEnqueueStore = eventBusEnqueueStore,
|
||||
timelineUpdatingStore = timelineUpdatingStore,
|
||||
tweetCountsUpdatingStore = tweetCountsUpdatingStore,
|
||||
guanoServiceStore = guanoServiceStore,
|
||||
mediaServiceStore = mediaServiceStore
|
||||
)
|
||||
|
||||
val replicatedDeleteTweetStore =
|
||||
ReplicatedDeleteTweet.Store(
|
||||
cachingTweetStore = cachingTweetStore,
|
||||
tweetCountsUpdatingStore = tweetCountsUpdatingStore
|
||||
)
|
||||
|
||||
val incrBookmarkCountStore =
|
||||
IncrBookmarkCount.Store(
|
||||
asyncEnqueueStore = asyncEnqueueStore,
|
||||
replicatingStore = replicatingStore
|
||||
)
|
||||
|
||||
val asyncIncrBookmarkCountStore =
|
||||
AsyncIncrBookmarkCount.Store(
|
||||
tweetCountsUpdatingStore = tweetCountsUpdatingStore
|
||||
)
|
||||
|
||||
val replicatedIncrBookmarkCountStore =
|
||||
ReplicatedIncrBookmarkCount.Store(
|
||||
tweetCountsUpdatingStore = tweetCountsUpdatingStore
|
||||
)
|
||||
|
||||
val incrFavCountStore =
|
||||
IncrFavCount.Store(
|
||||
asyncEnqueueStore = asyncEnqueueStore,
|
||||
replicatingStore = replicatingStore
|
||||
)
|
||||
|
||||
val asyncIncrFavCountStore =
|
||||
AsyncIncrFavCount.Store(
|
||||
tweetCountsUpdatingStore = tweetCountsUpdatingStore
|
||||
)
|
||||
|
||||
val replicatedIncrFavCountStore =
|
||||
ReplicatedIncrFavCount.Store(
|
||||
tweetCountsUpdatingStore = tweetCountsUpdatingStore
|
||||
)
|
||||
|
||||
val scrubGeoStore =
|
||||
ScrubGeo.Store(
|
||||
logLensStore = logLensStore,
|
||||
manhattanStore = manhattanStore,
|
||||
cachingTweetStore = cachingTweetStore,
|
||||
eventBusEnqueueStore = eventBusEnqueueStore,
|
||||
replicatingStore = replicatingStore
|
||||
)
|
||||
|
||||
val replicatedScrubGeoStore =
|
||||
ReplicatedScrubGeo.Store(
|
||||
cachingTweetStore = cachingTweetStore
|
||||
)
|
||||
|
||||
val takedownStore =
|
||||
Takedown.Store(
|
||||
logLensStore = logLensStore,
|
||||
manhattanStore = manhattanStore,
|
||||
cachingTweetStore = cachingTweetStore,
|
||||
asyncEnqueueStore = asyncEnqueueStore
|
||||
)
|
||||
|
||||
val asyncTakedownStore =
|
||||
AsyncTakedown.Store(
|
||||
replicatingStore = replicatingStore,
|
||||
guanoStore = guanoServiceStore,
|
||||
eventBusEnqueueStore = eventBusEnqueueStore
|
||||
)
|
||||
|
||||
val replicatedTakedownStore =
|
||||
ReplicatedTakedown.Store(
|
||||
cachingTweetStore = cachingTweetStore
|
||||
)
|
||||
|
||||
val updatePossiblySensitiveTweetStore =
|
||||
UpdatePossiblySensitiveTweet.Store(
|
||||
manhattanStore = manhattanStore,
|
||||
cachingTweetStore = cachingTweetStore,
|
||||
logLensStore = logLensStore,
|
||||
asyncEnqueueStore = asyncEnqueueStore
|
||||
)
|
||||
|
||||
val asyncUpdatePossiblySensitiveTweetStore =
|
||||
AsyncUpdatePossiblySensitiveTweet.Store(
|
||||
manhattanStore = manhattanStore,
|
||||
cachingTweetStore = cachingTweetStore,
|
||||
replicatingStore = replicatingStore,
|
||||
guanoStore = guanoServiceStore,
|
||||
eventBusStore = eventBusEnqueueStore
|
||||
)
|
||||
|
||||
val replicatedUpdatePossiblySensitiveTweetStore =
|
||||
ReplicatedUpdatePossiblySensitiveTweet.Store(
|
||||
cachingTweetStore = cachingTweetStore
|
||||
)
|
||||
|
||||
val setAdditionalFieldsStore =
|
||||
SetAdditionalFields.Store(
|
||||
manhattanStore = manhattanStore,
|
||||
cachingTweetStore = cachingTweetStore,
|
||||
asyncEnqueueStore = asyncEnqueueStore,
|
||||
logLensStore = logLensStore
|
||||
)
|
||||
|
||||
val asyncSetAdditionalFieldsStore =
|
||||
AsyncSetAdditionalFields.Store(
|
||||
replicatingStore = replicatingStore,
|
||||
eventBusEnqueueStore = eventBusEnqueueStore
|
||||
)
|
||||
|
||||
val replicatedSetAdditionalFieldsStore =
|
||||
ReplicatedSetAdditionalFields.Store(
|
||||
cachingTweetStore = cachingTweetStore
|
||||
)
|
||||
|
||||
val setRetweetVisibilityStore =
|
||||
SetRetweetVisibility.Store(asyncEnqueueStore = asyncEnqueueStore)
|
||||
|
||||
val asyncSetRetweetVisibilityStore =
|
||||
AsyncSetRetweetVisibility.Store(
|
||||
tweetIndexingStore = indexingStore,
|
||||
tweetCountsCacheUpdatingStore = tweetCountsUpdatingStore,
|
||||
replicatingTweetStore = replicatingStore,
|
||||
retweetArchivalEnqueueStore = retweetArchivalEnqueueStore
|
||||
)
|
||||
|
||||
val replicatedSetRetweetVisibilityStore =
|
||||
ReplicatedSetRetweetVisibility.Store(
|
||||
tweetCountsCacheUpdatingStore = tweetCountsUpdatingStore
|
||||
)
|
||||
|
||||
val deleteAdditionalFieldsStore =
|
||||
DeleteAdditionalFields.Store(
|
||||
cachingTweetStore = cachingTweetStore,
|
||||
asyncEnqueueStore = asyncEnqueueStore,
|
||||
logLensStore = logLensStore
|
||||
)
|
||||
|
||||
val asyncDeleteAdditionalFieldsStore =
|
||||
AsyncDeleteAdditionalFields.Store(
|
||||
manhattanStore = manhattanStore,
|
||||
cachingTweetStore = cachingTweetStore,
|
||||
replicatingStore = replicatingStore,
|
||||
eventBusEnqueueStore = eventBusEnqueueStore
|
||||
)
|
||||
|
||||
val replicatedDeleteAdditionalFieldsStore =
|
||||
ReplicatedDeleteAdditionalFields.Store(
|
||||
cachingTweetStore = cachingTweetStore
|
||||
)
|
||||
|
||||
/*
|
||||
* This composed store handles all synchronous side effects of an undelete
|
||||
* but does not execute the undeletion.
|
||||
*
|
||||
* This store is executed after the actual undelete request succeeds.
|
||||
* The undeletion request is initiated by Undelete.apply()
|
||||
*/
|
||||
val undeleteTweetStore =
|
||||
UndeleteTweet.Store(
|
||||
logLensStore = logLensStore,
|
||||
cachingTweetStore = cachingTweetStore,
|
||||
tweetCountsUpdatingStore = tweetCountsUpdatingStore,
|
||||
asyncEnqueueStore = asyncEnqueueStore
|
||||
)
|
||||
|
||||
val asyncUndeleteTweetStore =
|
||||
AsyncUndeleteTweet.Store(
|
||||
cachingTweetStore = cachingTweetStore,
|
||||
eventBusEnqueueStore = eventBusEnqueueStore,
|
||||
indexingStore = indexingStore,
|
||||
replicatingStore = replicatingStore,
|
||||
mediaServiceStore = mediaServiceStore,
|
||||
timelineUpdatingStore = timelineUpdatingStore
|
||||
)
|
||||
|
||||
val replicatedUndeleteTweetStore =
|
||||
ReplicatedUndeleteTweet.Store(
|
||||
cachingTweetStore = cachingTweetStore,
|
||||
tweetCountsUpdatingStore = tweetCountsUpdatingStore
|
||||
)
|
||||
|
||||
val flushStore =
|
||||
Flush.Store(
|
||||
cachingTweetStore = cachingTweetStore,
|
||||
tweetCountsUpdatingStore = tweetCountsUpdatingStore
|
||||
)
|
||||
|
||||
val scrubGeoUpdateUserTimestampStore =
|
||||
ScrubGeoUpdateUserTimestamp.Store(
|
||||
cache = caches.geoScrubCache,
|
||||
setInManhattan = clients.geoScrubEventStore.setGeoScrubTimestamp,
|
||||
geotagUpdateStore = userGeotagUpdateStore,
|
||||
tweetEventBusStore = eventBusEnqueueStore
|
||||
)
|
||||
|
||||
val quotedTweetDeleteStore =
|
||||
QuotedTweetDelete.Store(
|
||||
eventBusEnqueueStore = eventBusEnqueueStore
|
||||
)
|
||||
|
||||
val quotedTweetTakedownStore =
|
||||
QuotedTweetTakedown.Store(
|
||||
eventBusEnqueueStore = eventBusEnqueueStore
|
||||
)
|
||||
|
||||
new TotalTweetStore {
|
||||
val asyncDeleteAdditionalFields: FutureEffect[AsyncDeleteAdditionalFields.Event] =
|
||||
asyncDeleteAdditionalFieldsStore.asyncDeleteAdditionalFields
|
||||
val asyncDeleteTweet: FutureEffect[AsyncDeleteTweet.Event] =
|
||||
asyncDeleteTweetStore.asyncDeleteTweet
|
||||
val asyncIncrBookmarkCount: FutureEffect[AsyncIncrBookmarkCount.Event] =
|
||||
asyncIncrBookmarkCountStore.asyncIncrBookmarkCount
|
||||
val asyncIncrFavCount: FutureEffect[AsyncIncrFavCount.Event] =
|
||||
asyncIncrFavCountStore.asyncIncrFavCount
|
||||
val asyncInsertTweet: FutureEffect[AsyncInsertTweet.Event] = asyncInsertStore.asyncInsertTweet
|
||||
val asyncSetAdditionalFields: FutureEffect[AsyncSetAdditionalFields.Event] =
|
||||
asyncSetAdditionalFieldsStore.asyncSetAdditionalFields
|
||||
val asyncSetRetweetVisibility: FutureEffect[AsyncSetRetweetVisibility.Event] =
|
||||
asyncSetRetweetVisibilityStore.asyncSetRetweetVisibility
|
||||
val asyncTakedown: FutureEffect[AsyncTakedown.Event] = asyncTakedownStore.asyncTakedown
|
||||
val asyncUndeleteTweet: FutureEffect[AsyncUndeleteTweet.Event] =
|
||||
asyncUndeleteTweetStore.asyncUndeleteTweet
|
||||
val asyncUpdatePossiblySensitiveTweet: FutureEffect[AsyncUpdatePossiblySensitiveTweet.Event] =
|
||||
asyncUpdatePossiblySensitiveTweetStore.asyncUpdatePossiblySensitiveTweet
|
||||
val deleteAdditionalFields: FutureEffect[DeleteAdditionalFields.Event] =
|
||||
deleteAdditionalFieldsStore.deleteAdditionalFields
|
||||
val deleteTweet: FutureEffect[DeleteTweet.Event] = deleteTweetStore.deleteTweet
|
||||
val flush: FutureEffect[Flush.Event] = flushStore.flush
|
||||
val incrBookmarkCount: FutureEffect[IncrBookmarkCount.Event] =
|
||||
incrBookmarkCountStore.incrBookmarkCount
|
||||
val incrFavCount: FutureEffect[IncrFavCount.Event] = incrFavCountStore.incrFavCount
|
||||
val insertTweet: FutureEffect[InsertTweet.Event] = insertTweetStore.insertTweet
|
||||
val quotedTweetDelete: FutureEffect[QuotedTweetDelete.Event] =
|
||||
quotedTweetDeleteStore.quotedTweetDelete
|
||||
val quotedTweetTakedown: FutureEffect[QuotedTweetTakedown.Event] =
|
||||
quotedTweetTakedownStore.quotedTweetTakedown
|
||||
val replicatedDeleteAdditionalFields: FutureEffect[ReplicatedDeleteAdditionalFields.Event] =
|
||||
replicatedDeleteAdditionalFieldsStore.replicatedDeleteAdditionalFields
|
||||
val replicatedDeleteTweet: FutureEffect[ReplicatedDeleteTweet.Event] =
|
||||
replicatedDeleteTweetStore.replicatedDeleteTweet
|
||||
val replicatedIncrBookmarkCount: FutureEffect[ReplicatedIncrBookmarkCount.Event] =
|
||||
replicatedIncrBookmarkCountStore.replicatedIncrBookmarkCount
|
||||
val replicatedIncrFavCount: FutureEffect[ReplicatedIncrFavCount.Event] =
|
||||
replicatedIncrFavCountStore.replicatedIncrFavCount
|
||||
val replicatedInsertTweet: FutureEffect[ReplicatedInsertTweet.Event] =
|
||||
replicatedInsertTweetStore.replicatedInsertTweet
|
||||
val replicatedScrubGeo: FutureEffect[ReplicatedScrubGeo.Event] =
|
||||
replicatedScrubGeoStore.replicatedScrubGeo
|
||||
val replicatedSetAdditionalFields: FutureEffect[ReplicatedSetAdditionalFields.Event] =
|
||||
replicatedSetAdditionalFieldsStore.replicatedSetAdditionalFields
|
||||
val replicatedSetRetweetVisibility: FutureEffect[ReplicatedSetRetweetVisibility.Event] =
|
||||
replicatedSetRetweetVisibilityStore.replicatedSetRetweetVisibility
|
||||
val replicatedTakedown: FutureEffect[ReplicatedTakedown.Event] =
|
||||
replicatedTakedownStore.replicatedTakedown
|
||||
val replicatedUndeleteTweet: FutureEffect[ReplicatedUndeleteTweet.Event] =
|
||||
replicatedUndeleteTweetStore.replicatedUndeleteTweet
|
||||
val replicatedUpdatePossiblySensitiveTweet: FutureEffect[
|
||||
ReplicatedUpdatePossiblySensitiveTweet.Event
|
||||
] =
|
||||
replicatedUpdatePossiblySensitiveTweetStore.replicatedUpdatePossiblySensitiveTweet
|
||||
val retryAsyncDeleteAdditionalFields: FutureEffect[
|
||||
TweetStoreRetryEvent[AsyncDeleteAdditionalFields.Event]
|
||||
] =
|
||||
asyncDeleteAdditionalFieldsStore.retryAsyncDeleteAdditionalFields
|
||||
val retryAsyncDeleteTweet: FutureEffect[TweetStoreRetryEvent[AsyncDeleteTweet.Event]] =
|
||||
asyncDeleteTweetStore.retryAsyncDeleteTweet
|
||||
val retryAsyncInsertTweet: FutureEffect[TweetStoreRetryEvent[AsyncInsertTweet.Event]] =
|
||||
asyncInsertStore.retryAsyncInsertTweet
|
||||
val retryAsyncSetAdditionalFields: FutureEffect[
|
||||
TweetStoreRetryEvent[AsyncSetAdditionalFields.Event]
|
||||
] =
|
||||
asyncSetAdditionalFieldsStore.retryAsyncSetAdditionalFields
|
||||
val retryAsyncSetRetweetVisibility: FutureEffect[
|
||||
TweetStoreRetryEvent[AsyncSetRetweetVisibility.Event]
|
||||
] =
|
||||
asyncSetRetweetVisibilityStore.retryAsyncSetRetweetVisibility
|
||||
val retryAsyncTakedown: FutureEffect[TweetStoreRetryEvent[AsyncTakedown.Event]] =
|
||||
asyncTakedownStore.retryAsyncTakedown
|
||||
val retryAsyncUndeleteTweet: FutureEffect[TweetStoreRetryEvent[AsyncUndeleteTweet.Event]] =
|
||||
asyncUndeleteTweetStore.retryAsyncUndeleteTweet
|
||||
val retryAsyncUpdatePossiblySensitiveTweet: FutureEffect[
|
||||
TweetStoreRetryEvent[AsyncUpdatePossiblySensitiveTweet.Event]
|
||||
] =
|
||||
asyncUpdatePossiblySensitiveTweetStore.retryAsyncUpdatePossiblySensitiveTweet
|
||||
val scrubGeo: FutureEffect[ScrubGeo.Event] = scrubGeoStore.scrubGeo
|
||||
val setAdditionalFields: FutureEffect[SetAdditionalFields.Event] =
|
||||
setAdditionalFieldsStore.setAdditionalFields
|
||||
val setRetweetVisibility: FutureEffect[SetRetweetVisibility.Event] =
|
||||
setRetweetVisibilityStore.setRetweetVisibility
|
||||
val takedown: FutureEffect[Takedown.Event] = takedownStore.takedown
|
||||
val undeleteTweet: FutureEffect[UndeleteTweet.Event] = undeleteTweetStore.undeleteTweet
|
||||
val updatePossiblySensitiveTweet: FutureEffect[UpdatePossiblySensitiveTweet.Event] =
|
||||
updatePossiblySensitiveTweetStore.updatePossiblySensitiveTweet
|
||||
val scrubGeoUpdateUserTimestamp: FutureEffect[ScrubGeoUpdateUserTimestamp.Event] =
|
||||
scrubGeoUpdateUserTimestampStore.scrubGeoUpdateUserTimestamp
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,91 @@
|
||||
package com.twitter.tweetypie
|
||||
package config
|
||||
|
||||
import com.twitter.decider.Decider
|
||||
import com.twitter.tweetypie.decider.DeciderGates
|
||||
|
||||
object TweetypieDeciderGates {
|
||||
def apply(
|
||||
_decider: Decider,
|
||||
_overrides: Map[String, Boolean] = Map.empty
|
||||
): TweetypieDeciderGates =
|
||||
new TweetypieDeciderGates {
|
||||
override def decider: Decider = _decider
|
||||
override def overrides: Map[String, Boolean] = _overrides
|
||||
override def prefix: String = "tweetypie"
|
||||
}
|
||||
}
|
||||
|
||||
trait TweetypieDeciderGates extends DeciderGates {
|
||||
val checkSpamOnRetweet: Gate[Unit] = linear("check_spam_on_retweet")
|
||||
val checkSpamOnTweet: Gate[Unit] = linear("check_spam_on_tweet")
|
||||
val delayEraseUserTweets: Gate[Unit] = linear("delay_erase_user_tweets")
|
||||
val denyNonTweetPermalinks: Gate[Unit] = linear("deny_non_tweet_permalinks")
|
||||
val enableCommunityTweetCreates: Gate[Unit] = linear("enable_community_tweet_creates")
|
||||
val useConversationControlFeatureSwitchResults: Gate[Unit] = linear(
|
||||
"conversation_control_use_feature_switch_results")
|
||||
val enableExclusiveTweetControlValidation: Gate[Unit] = linear(
|
||||
"enable_exclusive_tweet_control_validation")
|
||||
val enableTrustedFriendsControlValidation: Gate[Unit] = linear(
|
||||
"enable_trusted_friends_control_validation"
|
||||
)
|
||||
val enableStaleTweetValidation: Gate[Unit] = linear(
|
||||
"enable_stale_tweet_validation"
|
||||
)
|
||||
val enforceRateLimitedClients: Gate[Unit] = linear("enforce_rate_limited_clients")
|
||||
val failClosedInVF: Gate[Unit] = linear("fail_closed_in_vf")
|
||||
val forkDarkTraffic: Gate[Unit] = linear("fork_dark_traffic")
|
||||
val hydrateConversationMuted: Gate[Unit] = linear("hydrate_conversation_muted")
|
||||
val hydrateCounts: Gate[Unit] = linear("hydrate_counts")
|
||||
val hydratePreviousCounts: Gate[Unit] = linear("hydrate_previous_counts")
|
||||
val hydrateDeviceSources: Gate[Unit] = linear("hydrate_device_sources")
|
||||
val hydrateEscherbirdAnnotations: Gate[Unit] = linear("hydrate_escherbird_annotations")
|
||||
val hydrateGnipProfileGeoEnrichment: Gate[Unit] = linear("hydrate_gnip_profile_geo_enrichment")
|
||||
val hydrateHasMedia: Gate[Unit] = linear("hydrate_has_media")
|
||||
val hydrateMedia: Gate[Unit] = linear("hydrate_media")
|
||||
val hydrateMediaRefs: Gate[Unit] = linear("hydrate_media_refs")
|
||||
val hydrateMediaTags: Gate[Unit] = linear("hydrate_media_tags")
|
||||
val hydratePastedMedia: Gate[Unit] = linear("hydrate_pasted_media")
|
||||
val hydratePerspectives: Gate[Unit] = linear("hydrate_perspectives")
|
||||
val hydratePerspectivesEditsForTimelines: Gate[Unit] = linear(
|
||||
"hydrate_perspectives_edits_for_timelines")
|
||||
val hydratePerspectivesEditsForTweetDetail: Gate[Unit] = linear(
|
||||
"hydrate_perspectives_edits_for_tweet_details")
|
||||
val hydratePerspectivesEditsForOtherSafetyLevels: Gate[Unit] =
|
||||
linear("hydrate_perspectives_edits_for_other_levels")
|
||||
val hydratePlaces: Gate[Unit] = linear("hydrate_places")
|
||||
val hydrateScrubEngagements: Gate[Unit] = linear("hydrate_scrub_engagements")
|
||||
val jiminyDarkRequests: Gate[Unit] = linear("jiminy_dark_requests")
|
||||
val logCacheExceptions: Gate[Unit] = linear("log_cache_exceptions")
|
||||
val logReads: Gate[Unit] = linear("log_reads")
|
||||
val logTweetCacheWrites: Gate[TweetId] = byId("log_tweet_cache_writes")
|
||||
val logWrites: Gate[Unit] = linear("log_writes")
|
||||
val logYoungTweetCacheWrites: Gate[TweetId] = byId("log_young_tweet_cache_writes")
|
||||
val maxRequestWidthEnabled: Gate[Unit] = linear("max_request_width_enabled")
|
||||
val mediaRefsHydratorIncludePastedMedia: Gate[Unit] = linear(
|
||||
"media_refs_hydrator_include_pasted_media")
|
||||
val rateLimitByLimiterService: Gate[Unit] = linear("rate_limit_by_limiter_service")
|
||||
val rateLimitTweetCreationFailure: Gate[Unit] = linear("rate_limit_tweet_creation_failure")
|
||||
val replicateReadsToATLA: Gate[Unit] = linear("replicate_reads_to_atla")
|
||||
val replicateReadsToPDXA: Gate[Unit] = linear("replicate_reads_to_pdxa")
|
||||
val disableInviteViaMention: Gate[Unit] = linear("disable_invite_via_mention")
|
||||
val shedReadTrafficVoluntarily: Gate[Unit] = linear("shed_read_traffic_voluntarily")
|
||||
val preferForwardedServiceIdentifierForClientId: Gate[Unit] =
|
||||
linear("prefer_forwarded_service_identifier_for_client_id")
|
||||
val enableRemoveUnmentionedImplicitMentions: Gate[Unit] = linear(
|
||||
"enable_remove_unmentioned_implicit_mentions")
|
||||
val validateCardRefAttachmentAndroid: Gate[Unit] = linear("validate_card_ref_attachment_android")
|
||||
val validateCardRefAttachmentNonAndroid: Gate[Unit] = linear(
|
||||
"validate_card_ref_attachment_non_android")
|
||||
val tweetVisibilityLibraryEnableParityTest: Gate[Unit] = linear(
|
||||
"tweet_visibility_library_enable_parity_test")
|
||||
val enableVfFeatureHydrationInQuotedTweetVLShim: Gate[Unit] = linear(
|
||||
"enable_vf_feature_hydration_in_quoted_tweet_visibility_library_shim")
|
||||
val disablePromotedTweetEdit: Gate[Unit] = linear("disable_promoted_tweet_edit")
|
||||
val shouldMaterializeContainers: Gate[Unit] = linear("should_materialize_containers")
|
||||
val checkTwitterBlueSubscriptionForEdit: Gate[Unit] = linear(
|
||||
"check_twitter_blue_subscription_for_edit")
|
||||
val hydrateBookmarksCount: Gate[Long] = byId("hydrate_bookmarks_count")
|
||||
val hydrateBookmarksPerspective: Gate[Long] = byId("hydrate_bookmarks_perspective")
|
||||
val setEditTimeWindowToSixtyMinutes: Gate[Unit] = linear("set_edit_time_window_to_sixty_minutes")
|
||||
}
|
@ -0,0 +1,223 @@
|
||||
package com.twitter.tweetypie
|
||||
package config
|
||||
|
||||
import com.twitter.servo.util.FutureArrow
|
||||
import com.twitter.stitch.Stitch
|
||||
import com.twitter.tweetypie.core._
|
||||
import com.twitter.tweetypie.handler.TweetBuilder
|
||||
import com.twitter.tweetypie.handler.WritePathQueryOptions
|
||||
import com.twitter.tweetypie.hydrator.EscherbirdAnnotationHydrator
|
||||
import com.twitter.tweetypie.hydrator.LanguageHydrator
|
||||
import com.twitter.tweetypie.hydrator.PlaceHydrator
|
||||
import com.twitter.tweetypie.hydrator.ProfileGeoHydrator
|
||||
import com.twitter.tweetypie.hydrator.TweetDataValueHydrator
|
||||
import com.twitter.tweetypie.repository._
|
||||
import com.twitter.tweetypie.store.InsertTweet
|
||||
import com.twitter.tweetypie.store.UndeleteTweet
|
||||
import com.twitter.tweetypie.thriftscala._
|
||||
import com.twitter.tweetypie.util.EditControlUtil
|
||||
|
||||
object WritePathHydration {
|
||||
type HydrateQuotedTweet =
|
||||
FutureArrow[(User, QuotedTweet, WritePathHydrationOptions), Option[QuoteTweetMetadata]]
|
||||
|
||||
case class QuoteTweetMetadata(
|
||||
quotedTweet: Tweet,
|
||||
quotedUser: User,
|
||||
quoterHasAlreadyQuotedTweet: Boolean)
|
||||
|
||||
private val log = Logger(getClass)
|
||||
|
||||
val UserFieldsForInsert: Set[UserField] =
|
||||
TweetBuilder.userFields
|
||||
|
||||
val AllowedMissingFieldsOnWrite: Set[FieldByPath] =
|
||||
Set(
|
||||
EscherbirdAnnotationHydrator.hydratedField,
|
||||
LanguageHydrator.hydratedField,
|
||||
PlaceHydrator.HydratedField,
|
||||
ProfileGeoHydrator.hydratedField
|
||||
)
|
||||
|
||||
/**
|
||||
* Builds a FutureArrow that performs the necessary hydration in the write-path for a
|
||||
* a InsertTweet.Event. There are two separate hydration steps, pre-cache and post-cache.
|
||||
* The pre-cache hydration step performs the hydration which is safe to cache, while the
|
||||
* post-cache hydration step performs the hydration whose results we don't want to cache
|
||||
* on the tweet.
|
||||
*
|
||||
* TweetInsertEvent contains two tweet fields, `tweet` and `internalTweet`. `tweet` is
|
||||
* the input value used for hydration, and in the updated InsertTweet.Event returned by the
|
||||
* FutureArrow, `tweet` contains the post-cache hydrated tweet while `internalTweet` contains
|
||||
* the pre-cache hydrated tweet.
|
||||
*/
|
||||
def hydrateInsertTweetEvent(
|
||||
hydrateTweet: FutureArrow[(TweetData, TweetQuery.Options), TweetData],
|
||||
hydrateQuotedTweet: HydrateQuotedTweet
|
||||
): FutureArrow[InsertTweet.Event, InsertTweet.Event] =
|
||||
FutureArrow { event =>
|
||||
val cause = TweetQuery.Cause.Insert(event.tweet.id)
|
||||
val hydrationOpts = event.hydrateOptions
|
||||
val isEditControlEdit = event.tweet.editControl.exists(EditControlUtil.isEditControlEdit)
|
||||
val queryOpts: TweetQuery.Options =
|
||||
WritePathQueryOptions.insert(cause, event.user, hydrationOpts, isEditControlEdit)
|
||||
|
||||
val initTweetData =
|
||||
TweetData(
|
||||
tweet = event.tweet,
|
||||
sourceTweetResult = event.sourceTweet.map(TweetResult(_))
|
||||
)
|
||||
|
||||
for {
|
||||
tweetData <- hydrateTweet((initTweetData, queryOpts))
|
||||
hydratedTweet = tweetData.tweet
|
||||
internalTweet =
|
||||
tweetData.cacheableTweetResult
|
||||
.map(_.value.toCachedTweet)
|
||||
.getOrElse(
|
||||
throw new IllegalStateException(s"expected cacheableTweetResult, e=${event}"))
|
||||
|
||||
optQt = getQuotedTweet(hydratedTweet)
|
||||
.orElse(event.sourceTweet.flatMap(getQuotedTweet))
|
||||
|
||||
hydratedQT <- optQt match {
|
||||
case None => Future.value(None)
|
||||
case Some(qt) => hydrateQuotedTweet((event.user, qt, hydrationOpts))
|
||||
}
|
||||
} yield {
|
||||
event.copy(
|
||||
tweet = hydratedTweet,
|
||||
_internalTweet = Some(internalTweet),
|
||||
quotedTweet = hydratedQT.map { case QuoteTweetMetadata(t, _, _) => t },
|
||||
quotedUser = hydratedQT.map { case QuoteTweetMetadata(_, u, _) => u },
|
||||
quoterHasAlreadyQuotedTweet = hydratedQT.exists { case QuoteTweetMetadata(_, _, b) => b }
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Builds a FutureArrow for retrieving a quoted tweet metadata
|
||||
* QuotedTweet struct. If either the quoted tweet or the quoted user
|
||||
* isn't visible to the tweeting user, the FutureArrow will return None.
|
||||
*/
|
||||
def hydrateQuotedTweet(
|
||||
tweetRepo: TweetRepository.Optional,
|
||||
userRepo: UserRepository.Optional,
|
||||
quoterHasAlreadyQuotedRepo: QuoterHasAlreadyQuotedRepository.Type
|
||||
): HydrateQuotedTweet = {
|
||||
FutureArrow {
|
||||
case (tweetingUser, qt, hydrateOptions) =>
|
||||
val tweetQueryOpts = WritePathQueryOptions.quotedTweet(tweetingUser, hydrateOptions)
|
||||
val userQueryOpts =
|
||||
UserQueryOptions(
|
||||
UserFieldsForInsert,
|
||||
UserVisibility.Visible,
|
||||
forUserId = Some(tweetingUser.id)
|
||||
)
|
||||
|
||||
Stitch.run(
|
||||
Stitch
|
||||
.join(
|
||||
tweetRepo(qt.tweetId, tweetQueryOpts),
|
||||
userRepo(UserKey.byId(qt.userId), userQueryOpts),
|
||||
// We're failing open here on tflock exceptions since this should not
|
||||
// affect the ability to quote tweet if tflock goes down. (although if
|
||||
// this call doesn't succeed, quote counts may be inaccurate for a brief
|
||||
// period of time)
|
||||
quoterHasAlreadyQuotedRepo(qt.tweetId, tweetingUser.id).liftToTry
|
||||
)
|
||||
.map {
|
||||
case (Some(tweet), Some(user), isAlreadyQuoted) =>
|
||||
Some(QuoteTweetMetadata(tweet, user, isAlreadyQuoted.getOrElse(false)))
|
||||
case _ => None
|
||||
}
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Builds a FutureArrow that performs any additional hydration on an UndeleteTweet.Event before
|
||||
* being passed to a TweetStore.
|
||||
*/
|
||||
def hydrateUndeleteTweetEvent(
|
||||
hydrateTweet: FutureArrow[(TweetData, TweetQuery.Options), TweetData],
|
||||
hydrateQuotedTweet: HydrateQuotedTweet
|
||||
): FutureArrow[UndeleteTweet.Event, UndeleteTweet.Event] =
|
||||
FutureArrow { event =>
|
||||
val cause = TweetQuery.Cause.Undelete(event.tweet.id)
|
||||
val hydrationOpts = event.hydrateOptions
|
||||
val isEditControlEdit = event.tweet.editControl.exists(EditControlUtil.isEditControlEdit)
|
||||
val queryOpts = WritePathQueryOptions.insert(cause, event.user, hydrationOpts, isEditControlEdit)
|
||||
|
||||
// when undeleting a retweet, don't set sourceTweetResult to enable SourceTweetHydrator to
|
||||
// hydrate it
|
||||
val initTweetData = TweetData(tweet = event.tweet)
|
||||
|
||||
for {
|
||||
tweetData <- hydrateTweet((initTweetData, queryOpts))
|
||||
hydratedTweet = tweetData.tweet
|
||||
internalTweet =
|
||||
tweetData.cacheableTweetResult
|
||||
.map(_.value.toCachedTweet)
|
||||
.getOrElse(
|
||||
throw new IllegalStateException(s"expected cacheableTweetResult, e=${event}"))
|
||||
|
||||
optQt = getQuotedTweet(hydratedTweet)
|
||||
.orElse(tweetData.sourceTweetResult.map(_.value.tweet).flatMap(getQuotedTweet))
|
||||
|
||||
hydratedQt <- optQt match {
|
||||
case None => Future.value(None)
|
||||
case Some(qt) => hydrateQuotedTweet((event.user, qt, hydrationOpts))
|
||||
}
|
||||
} yield {
|
||||
event.copy(
|
||||
tweet = hydratedTweet,
|
||||
_internalTweet = Some(internalTweet),
|
||||
sourceTweet = tweetData.sourceTweetResult.map(_.value.tweet),
|
||||
quotedTweet = hydratedQt.map { case QuoteTweetMetadata(t, _, _) => t },
|
||||
quotedUser = hydratedQt.map { case QuoteTweetMetadata(_, u, _) => u },
|
||||
quoterHasAlreadyQuotedTweet = hydratedQt.exists { case QuoteTweetMetadata(_, _, b) => b }
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts a TweetDataValueHydrator into a FutureArrow that hydrates a tweet for the write-path.
|
||||
*/
|
||||
def hydrateTweet(
|
||||
hydrator: TweetDataValueHydrator,
|
||||
stats: StatsReceiver,
|
||||
allowedMissingFields: Set[FieldByPath] = AllowedMissingFieldsOnWrite
|
||||
): FutureArrow[(TweetData, TweetQuery.Options), TweetData] = {
|
||||
val hydrationStats = stats.scope("hydration")
|
||||
val missingFieldsStats = hydrationStats.scope("missing_fields")
|
||||
|
||||
FutureArrow[(TweetData, TweetQuery.Options), TweetData] {
|
||||
case (td, opts) =>
|
||||
Stitch
|
||||
.run(hydrator(td, opts))
|
||||
.rescue {
|
||||
case ex =>
|
||||
log.warn("Hydration failed with exception", ex)
|
||||
Future.exception(
|
||||
TweetHydrationError("Hydration failed with exception: " + ex, Some(ex))
|
||||
)
|
||||
}
|
||||
.flatMap { r =>
|
||||
// Record missing fields even if the request succeeds)
|
||||
for (missingField <- r.state.failedFields)
|
||||
missingFieldsStats.counter(missingField.fieldIdPath.mkString(".")).incr()
|
||||
|
||||
if ((r.state.failedFields -- allowedMissingFields).nonEmpty) {
|
||||
Future.exception(
|
||||
TweetHydrationError(
|
||||
"Failed to hydrate. Missing Fields: " + r.state.failedFields.mkString(",")
|
||||
)
|
||||
)
|
||||
} else {
|
||||
Future.value(r.value)
|
||||
}
|
||||
}
|
||||
}
|
||||
}.trackOutcome(stats, (_: Any) => "hydration")
|
||||
}
|
@ -0,0 +1,11 @@
|
||||
package com.twitter.tweetypie
|
||||
|
||||
import com.twitter.context.thriftscala.Viewer
|
||||
|
||||
package object config {
|
||||
// Bring Tweetypie permitted TwitterContext into scope
|
||||
private[config] val TwitterContext =
|
||||
com.twitter.context.TwitterContext(com.twitter.tweetypie.TwitterContextPermit)
|
||||
|
||||
def getAppId: Option[AppId] = TwitterContext().getOrElse(Viewer()).clientApplicationId
|
||||
}
|
@ -0,0 +1,19 @@
|
||||
scala_library(
|
||||
sources = ["*.scala"],
|
||||
compiler_option_sets = ["fatal_warnings"],
|
||||
platform = "java8",
|
||||
strict_deps = True,
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"3rdparty/jvm/com/twitter/bijection:core",
|
||||
"featureswitches/featureswitches-core/src/main/scala",
|
||||
"scrooge/scrooge-serializer/src/main/scala",
|
||||
"tweetypie/servo/repo",
|
||||
"tweetypie/servo/util",
|
||||
"tweetypie/common/src/thrift/com/twitter/tweetypie:service-scala",
|
||||
"tweetypie/common/src/thrift/com/twitter/tweetypie:tweet-scala",
|
||||
"stitch/stitch-core",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie",
|
||||
"tweetypie/server/src/main/thrift:compiled-scala",
|
||||
],
|
||||
)
|
@ -0,0 +1,32 @@
|
||||
package com.twitter.tweetypie
|
||||
package core
|
||||
|
||||
import com.twitter.tweetypie.thriftscala.CardReference
|
||||
import java.net.URI
|
||||
|
||||
sealed trait CardUri
|
||||
object Tombstone extends CardUri
|
||||
case class NonTombstone(uri: String) extends CardUri
|
||||
|
||||
object CardReferenceUriExtractor {
|
||||
|
||||
private def parseAsUri(cardRef: CardReference) = Try(new URI(cardRef.cardUri)).toOption
|
||||
private def isTombstone(uri: URI) = uri.getScheme == "tombstone"
|
||||
|
||||
/**
|
||||
* Parses a CardReference to return Option[CardUri] to differentiate among:
|
||||
* - Some(NonTombstone): hydrate card2 with provided uri
|
||||
* - Some(Tombstone): don't hydrate card2
|
||||
* - None: fallback and attempt to use url entities uris
|
||||
*/
|
||||
def unapply(cardRef: CardReference): Option[CardUri] =
|
||||
parseAsUri(cardRef) match {
|
||||
case Some(uri) if !isTombstone(uri) => Some(NonTombstone(uri.toString))
|
||||
case Some(uri) => Some(Tombstone)
|
||||
|
||||
// If a cardReference is set, but does not parse as a URI, it's likely a https? URL with
|
||||
// incorrectly encoded query params. Since these occur frequently in the wild, we'll
|
||||
// attempt a card2 hydration with it
|
||||
case None => Some(NonTombstone(cardRef.cardUri))
|
||||
}
|
||||
}
|
@ -0,0 +1,48 @@
|
||||
package com.twitter.tweetypie.core
|
||||
|
||||
import com.twitter.servo.data.Mutation
|
||||
|
||||
/**
|
||||
* An EditState is a function that changes a value and may generate
|
||||
* some state about what was modified. For instance, it may record
|
||||
* whether an item was changed, or whether there was an error.
|
||||
* EditStates are useful because they are first-class values that can
|
||||
* be composed. In particular, it is useful to concurrently access
|
||||
* external data to build edits and then apply them.
|
||||
*
|
||||
* @tparam A The type of the value that is being edited (for instance,
|
||||
* having fields hydrated with data from another service)
|
||||
*/
|
||||
final case class EditState[A](run: A => ValueState[A]) {
|
||||
|
||||
/**
|
||||
* Composes two EditStates in sequence
|
||||
*/
|
||||
def andThen(other: EditState[A]): EditState[A] =
|
||||
EditState[A] { a0: A =>
|
||||
val ValueState(a1, s1) = run(a0)
|
||||
val ValueState(a2, s2) = other.run(a1)
|
||||
ValueState(a2, s1 ++ s2)
|
||||
}
|
||||
}
|
||||
|
||||
object EditState {
|
||||
|
||||
/**
|
||||
* Creates a "passthrough" EditState:
|
||||
* Leaves A unchanged and produces empty state S
|
||||
*/
|
||||
def unit[A]: EditState[A] =
|
||||
EditState[A](ValueState.unit[A])
|
||||
|
||||
/**
|
||||
* Creates an `EditState[A]` using a `Mutation[A]`.
|
||||
*/
|
||||
def fromMutation[A](mut: Mutation[A]): EditState[A] =
|
||||
EditState[A] { a =>
|
||||
mut(a) match {
|
||||
case None => ValueState.unmodified(a)
|
||||
case Some(a2) => ValueState.modified(a2)
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,14 @@
|
||||
package com.twitter.tweetypie
|
||||
package core
|
||||
|
||||
import scala.util.control.NoStackTrace
|
||||
|
||||
case class InternalServerError(message: String) extends Exception(message) with NoStackTrace
|
||||
|
||||
case class OverCapacity(message: String) extends Exception(message) with NoStackTrace
|
||||
|
||||
case class RateLimited(message: String) extends Exception(message) with NoStackTrace
|
||||
|
||||
case class TweetHydrationError(message: String, cause: Option[Throwable] = None)
|
||||
extends Exception(message, cause.getOrElse(null))
|
||||
with NoStackTrace
|
@ -0,0 +1,96 @@
|
||||
package com.twitter.tweetypie.core
|
||||
|
||||
import com.twitter.servo.util.ExceptionCategorizer
|
||||
import com.twitter.spam.rtf.thriftscala.FilteredReason
|
||||
import scala.util.control.NoStackTrace
|
||||
|
||||
sealed trait FilteredState
|
||||
|
||||
object FilteredState {
|
||||
|
||||
/**
|
||||
* The tweet exists and the filtered state was due to business rules
|
||||
* (e.g. safety label filtering, or protected accounts). Note that
|
||||
* Suppress and Unavailable can both have a FilteredReason.
|
||||
*/
|
||||
sealed trait HasFilteredReason extends FilteredState {
|
||||
def filteredReason: FilteredReason
|
||||
}
|
||||
|
||||
/**
|
||||
* The only FilteredState that is not an exception. It indicates that
|
||||
* the tweet should be returned along with a suppress reason. This is
|
||||
* sometimes known as "soft filtering". Only used by VF.
|
||||
*/
|
||||
case class Suppress(filteredReason: FilteredReason) extends FilteredState with HasFilteredReason
|
||||
|
||||
/**
|
||||
* FilteredStates that cause the tweet to be unavailable are modeled
|
||||
* as an [[Exception]]. (Suppressed filtered states cannot be used as
|
||||
* exceptions because they should not prevent the tweet from being
|
||||
* returned.) This is sometimes known as "hard filtering".
|
||||
*/
|
||||
sealed abstract class Unavailable extends Exception with FilteredState with NoStackTrace
|
||||
|
||||
object Unavailable {
|
||||
// Used for Tweets that should be dropped because of VF rules
|
||||
case class Drop(filteredReason: FilteredReason) extends Unavailable with HasFilteredReason
|
||||
|
||||
// Used for Tweets that should be dropped and replaced with their preview because of VF rules
|
||||
case class Preview(filteredReason: FilteredReason) extends Unavailable with HasFilteredReason
|
||||
|
||||
// Used for Tweets that should be dropped because of Tweetypie business logic
|
||||
case object DropUnspecified extends Unavailable with HasFilteredReason {
|
||||
val filteredReason: FilteredReason = FilteredReason.UnspecifiedReason(true)
|
||||
}
|
||||
|
||||
// Represents a Deleted tweet (NotFound is represented with stitch.NotFound)
|
||||
case object TweetDeleted extends Unavailable
|
||||
|
||||
// Represents a Deleted tweet that violated Twitter Rules (see go/bounced-tweet)
|
||||
case object BounceDeleted extends Unavailable
|
||||
|
||||
// Represents both Deleted and NotFound source tweets
|
||||
case class SourceTweetNotFound(deleted: Boolean) extends Unavailable
|
||||
|
||||
// Used by the [[ReportedTweetFilter]] to signal that a Tweet has a "reported" perspective from TLS
|
||||
case object Reported extends Unavailable with HasFilteredReason {
|
||||
val filteredReason: FilteredReason = FilteredReason.ReportedTweet(true)
|
||||
}
|
||||
|
||||
// The following objects are used by the [[UserRepository]] to signal problems with the Tweet author
|
||||
object Author {
|
||||
case object NotFound extends Unavailable
|
||||
|
||||
case object Deactivated extends Unavailable with HasFilteredReason {
|
||||
val filteredReason: FilteredReason = FilteredReason.AuthorIsDeactivated(true)
|
||||
}
|
||||
|
||||
case object Offboarded extends Unavailable with HasFilteredReason {
|
||||
val filteredReason: FilteredReason = FilteredReason.AuthorAccountIsInactive(true)
|
||||
}
|
||||
|
||||
case object Suspended extends Unavailable with HasFilteredReason {
|
||||
val filteredReason: FilteredReason = FilteredReason.AuthorIsSuspended(true)
|
||||
}
|
||||
|
||||
case object Protected extends Unavailable with HasFilteredReason {
|
||||
val filteredReason: FilteredReason = FilteredReason.AuthorIsProtected(true)
|
||||
}
|
||||
|
||||
case object Unsafe extends Unavailable with HasFilteredReason {
|
||||
val filteredReason: FilteredReason = FilteredReason.AuthorIsUnsafe(true)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new ExceptionCategorizer which returns an empty category for any
|
||||
* Unavailable value, and forwards to `underlying` for anything else.
|
||||
*/
|
||||
def ignoringCategorizer(underlying: ExceptionCategorizer): ExceptionCategorizer =
|
||||
ExceptionCategorizer {
|
||||
case _: Unavailable => Set.empty
|
||||
case t => underlying(t)
|
||||
}
|
||||
}
|
@ -0,0 +1,3 @@
|
||||
package com.twitter.tweetypie.core
|
||||
|
||||
case class GeoSearchRequestId(requestID: String)
|
@ -0,0 +1,122 @@
|
||||
package com.twitter.tweetypie.core
|
||||
|
||||
import com.twitter.tweetypie.thriftscala.FieldByPath
|
||||
import com.twitter.tweetypie.thriftscala.HydrationType
|
||||
|
||||
/**
|
||||
* HydrationState is used to record whether a particular piece of data was modified as a result
|
||||
* of hydration, and/or if there was a failure to hydrate the data.
|
||||
*/
|
||||
sealed trait HydrationState {
|
||||
def isEmpty: Boolean
|
||||
def modified: Boolean
|
||||
def completedHydrations: Set[HydrationType] = Set.empty
|
||||
def failedFields: Set[FieldByPath] = Set.empty
|
||||
def cacheErrorEncountered: Boolean = false
|
||||
def ++(that: HydrationState): HydrationState
|
||||
}
|
||||
|
||||
object HydrationState {
|
||||
|
||||
/**
|
||||
* Base `HydrationState`. It acts as an identity value when combined with any other
|
||||
* `HydrationState`.
|
||||
*/
|
||||
case object Empty extends HydrationState {
|
||||
def isEmpty = true
|
||||
def modified = false
|
||||
def ++(that: HydrationState): HydrationState = that
|
||||
}
|
||||
|
||||
/**
|
||||
* A `HydrationState` with metadata indicating a non-fatal hydration operation.
|
||||
*/
|
||||
case class Success(
|
||||
override val modified: Boolean = false,
|
||||
override val completedHydrations: Set[HydrationType] = Set.empty,
|
||||
override val failedFields: Set[FieldByPath] = Set.empty,
|
||||
override val cacheErrorEncountered: Boolean = false)
|
||||
extends HydrationState {
|
||||
|
||||
def isEmpty: Boolean = !modified && failedFields.isEmpty && !cacheErrorEncountered
|
||||
|
||||
def ++(that: HydrationState): HydrationState =
|
||||
that match {
|
||||
case Empty => this
|
||||
case that: Success =>
|
||||
HydrationState(
|
||||
modified || that.modified,
|
||||
completedHydrations ++ that.completedHydrations,
|
||||
failedFields ++ that.failedFields,
|
||||
cacheErrorEncountered || that.cacheErrorEncountered
|
||||
)
|
||||
}
|
||||
|
||||
/**
|
||||
* An implementation of `copy` that avoids unnecessary allocations, by
|
||||
* using the constant `HydrationState.unmodified` and `HydrationState.modified`
|
||||
* values when possible.
|
||||
*/
|
||||
def copy(
|
||||
modified: Boolean = this.modified,
|
||||
completedHydrations: Set[HydrationType] = this.completedHydrations,
|
||||
failedFields: Set[FieldByPath] = this.failedFields,
|
||||
cacheErrorEncountered: Boolean = this.cacheErrorEncountered
|
||||
): HydrationState =
|
||||
HydrationState(modified, completedHydrations, failedFields, cacheErrorEncountered)
|
||||
}
|
||||
|
||||
val empty: HydrationState = Empty
|
||||
val modified: HydrationState = Success(true)
|
||||
|
||||
def modified(completedHydration: HydrationType): HydrationState =
|
||||
modified(Set(completedHydration))
|
||||
|
||||
def modified(completedHydrations: Set[HydrationType]): HydrationState =
|
||||
Success(modified = true, completedHydrations = completedHydrations)
|
||||
|
||||
def partial(failedField: FieldByPath): HydrationState =
|
||||
partial(Set(failedField))
|
||||
|
||||
def partial(failedFields: Set[FieldByPath]): HydrationState =
|
||||
Success(modified = false, failedFields = failedFields)
|
||||
|
||||
def apply(
|
||||
modified: Boolean,
|
||||
completedHydrations: Set[HydrationType] = Set.empty,
|
||||
failedFields: Set[FieldByPath] = Set.empty,
|
||||
cacheErrorEncountered: Boolean = false
|
||||
): HydrationState =
|
||||
if (completedHydrations.nonEmpty || failedFields.nonEmpty || cacheErrorEncountered) {
|
||||
Success(modified, completedHydrations, failedFields, cacheErrorEncountered)
|
||||
} else if (modified) {
|
||||
HydrationState.modified
|
||||
} else {
|
||||
HydrationState.empty
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new HydrationState with modified set to true if `next` and `prev` are different,
|
||||
* or false if they are the same.
|
||||
*/
|
||||
def delta[A](prev: A, next: A): HydrationState =
|
||||
if (next != prev) modified else empty
|
||||
|
||||
/**
|
||||
* Join a list of HydrationStates into a single HydrationState.
|
||||
*
|
||||
* Note: this could just be a reduce over the HydrationStates but that would allocate
|
||||
* _N_ HydrationStates. This approach also allows for shortcircuiting over the boolean
|
||||
* fields.
|
||||
*/
|
||||
def join(states: HydrationState*): HydrationState = {
|
||||
val statesSet = states.toSet
|
||||
|
||||
HydrationState(
|
||||
modified = states.exists(_.modified),
|
||||
completedHydrations = statesSet.flatMap(_.completedHydrations),
|
||||
failedFields = statesSet.flatMap(_.failedFields),
|
||||
cacheErrorEncountered = states.exists(_.cacheErrorEncountered)
|
||||
)
|
||||
}
|
||||
}
|
@ -0,0 +1,46 @@
|
||||
package com.twitter.tweetypie.core
|
||||
|
||||
import com.twitter.spam.rtf.thriftscala.FilteredReason
|
||||
import com.twitter.util.Return
|
||||
import com.twitter.util.Throw
|
||||
import com.twitter.util.Try
|
||||
|
||||
/**
|
||||
* The data about a quoted tweet that needs to be carried forward to
|
||||
* Tweetypie clients.
|
||||
*/
|
||||
sealed trait QuotedTweetResult {
|
||||
def filteredReason: Option[FilteredReason]
|
||||
def toOption: Option[TweetResult]
|
||||
def map(f: TweetResult => TweetResult): QuotedTweetResult
|
||||
}
|
||||
|
||||
object QuotedTweetResult {
|
||||
case object NotFound extends QuotedTweetResult {
|
||||
def filteredReason: None.type = None
|
||||
def toOption: None.type = None
|
||||
def map(f: TweetResult => TweetResult): NotFound.type = this
|
||||
}
|
||||
case class Filtered(state: FilteredState.Unavailable) extends QuotedTweetResult {
|
||||
def filteredReason: Option[FilteredReason] =
|
||||
state match {
|
||||
case st: FilteredState.HasFilteredReason => Some(st.filteredReason)
|
||||
case _ => None
|
||||
}
|
||||
def toOption: None.type = None
|
||||
def map(f: TweetResult => TweetResult): Filtered = this
|
||||
}
|
||||
case class Found(result: TweetResult) extends QuotedTweetResult {
|
||||
def filteredReason: Option[FilteredReason] = result.value.suppress.map(_.filteredReason)
|
||||
def toOption: Option[TweetResult] = Some(result)
|
||||
def map(f: TweetResult => TweetResult): QuotedTweetResult = Found(f(result))
|
||||
}
|
||||
|
||||
def fromTry(tryResult: Try[TweetResult]): Try[QuotedTweetResult] =
|
||||
tryResult match {
|
||||
case Return(result) => Return(Found(result))
|
||||
case Throw(state: FilteredState.Unavailable) => Return(Filtered(state))
|
||||
case Throw(com.twitter.stitch.NotFound) => Return(NotFound)
|
||||
case Throw(e) => Throw(e)
|
||||
}
|
||||
}
|
@ -0,0 +1,31 @@
|
||||
package com.twitter.tweetypie.core
|
||||
|
||||
import com.twitter.servo.cache
|
||||
import com.twitter.servo.cache.CachedSerializer
|
||||
import com.twitter.tweetypie.thriftscala
|
||||
import com.twitter.tweetypie.thriftscala.CachedTweet
|
||||
import com.twitter.tweetypie.thriftscala.Tweet
|
||||
import org.apache.thrift.protocol.TCompactProtocol
|
||||
|
||||
/**
|
||||
* A container object for serializers.
|
||||
* Creates a serializer for every object type cached by the tweetypie service
|
||||
*/
|
||||
object Serializer {
|
||||
lazy val CompactProtocolFactory: TCompactProtocol.Factory = new TCompactProtocol.Factory
|
||||
|
||||
def toCached[T](underlying: cache.Serializer[T]): cache.CachedSerializer[T] =
|
||||
new cache.CachedSerializer(underlying, CompactProtocolFactory)
|
||||
|
||||
object Tweet {
|
||||
lazy val Compact: cache.ThriftSerializer[thriftscala.Tweet] =
|
||||
new cache.ThriftSerializer(thriftscala.Tweet, CompactProtocolFactory)
|
||||
lazy val CachedCompact: CachedSerializer[Tweet] = toCached(Compact)
|
||||
}
|
||||
|
||||
object CachedTweet {
|
||||
lazy val Compact: cache.ThriftSerializer[thriftscala.CachedTweet] =
|
||||
new cache.ThriftSerializer(thriftscala.CachedTweet, CompactProtocolFactory)
|
||||
lazy val CachedCompact: CachedSerializer[CachedTweet] = toCached(Compact)
|
||||
}
|
||||
}
|
@ -0,0 +1,42 @@
|
||||
package com.twitter.tweetypie.core
|
||||
|
||||
sealed trait StoredTweetResult {
|
||||
def canHydrate: Boolean
|
||||
}
|
||||
|
||||
object StoredTweetResult {
|
||||
sealed trait Error
|
||||
object Error {
|
||||
case object Corrupt extends Error
|
||||
case object ScrubbedFieldsPresent extends Error
|
||||
case object FieldsMissingOrInvalid extends Error
|
||||
case object ShouldBeHardDeleted extends Error
|
||||
}
|
||||
|
||||
case class Present(errors: Seq[Error], canHydrate: Boolean) extends StoredTweetResult
|
||||
|
||||
case class HardDeleted(softDeletedAtMsec: Long, hardDeletedAtMsec: Long)
|
||||
extends StoredTweetResult {
|
||||
override def canHydrate: Boolean = false
|
||||
}
|
||||
|
||||
case class SoftDeleted(softDeletedAtMsec: Long, errors: Seq[Error], canHydrate: Boolean)
|
||||
extends StoredTweetResult
|
||||
|
||||
case class BounceDeleted(deletedAtMsec: Long, errors: Seq[Error], canHydrate: Boolean)
|
||||
extends StoredTweetResult
|
||||
|
||||
case class Undeleted(undeletedAtMsec: Long, errors: Seq[Error], canHydrate: Boolean)
|
||||
extends StoredTweetResult
|
||||
|
||||
case class ForceAdded(addedAtMsec: Long, errors: Seq[Error], canHydrate: Boolean)
|
||||
extends StoredTweetResult
|
||||
|
||||
case class Failed(errors: Seq[Error]) extends StoredTweetResult {
|
||||
override def canHydrate: Boolean = false
|
||||
}
|
||||
|
||||
object NotFound extends StoredTweetResult {
|
||||
override def canHydrate: Boolean = false
|
||||
}
|
||||
}
|
@ -0,0 +1,39 @@
|
||||
package com.twitter.tweetypie.core
|
||||
|
||||
import com.twitter.bouncer.thriftscala.Bounce
|
||||
import com.twitter.tweetypie.TweetId
|
||||
import com.twitter.incentives.jiminy.thriftscala.TweetNudge
|
||||
import com.twitter.tweetypie.thriftscala.PostTweetResult
|
||||
import com.twitter.tweetypie.thriftscala.TweetCreateState
|
||||
|
||||
sealed abstract class TweetCreateFailure extends Exception {
|
||||
def toPostTweetResult: PostTweetResult
|
||||
}
|
||||
|
||||
object TweetCreateFailure {
|
||||
case class Bounced(bounce: Bounce) extends TweetCreateFailure {
|
||||
override def toPostTweetResult: PostTweetResult =
|
||||
PostTweetResult(state = TweetCreateState.Bounce, bounce = Some(bounce))
|
||||
}
|
||||
|
||||
case class AlreadyRetweeted(retweetId: TweetId) extends TweetCreateFailure {
|
||||
override def toPostTweetResult: PostTweetResult =
|
||||
PostTweetResult(state = TweetCreateState.AlreadyRetweeted)
|
||||
}
|
||||
|
||||
case class Nudged(nudge: TweetNudge) extends TweetCreateFailure {
|
||||
override def toPostTweetResult: PostTweetResult =
|
||||
PostTweetResult(state = TweetCreateState.Nudge, nudge = Some(nudge))
|
||||
}
|
||||
|
||||
case class State(state: TweetCreateState, reason: Option[String] = None)
|
||||
extends TweetCreateFailure {
|
||||
require(state != TweetCreateState.Bounce)
|
||||
require(state != TweetCreateState.Ok)
|
||||
require(state != TweetCreateState.Nudge)
|
||||
|
||||
override def toPostTweetResult: PostTweetResult =
|
||||
PostTweetResult(state = state, failureReason = reason)
|
||||
override def toString: String = s"TweetCreateFailure$$State($state, $reason)"
|
||||
}
|
||||
}
|
@ -0,0 +1,86 @@
|
||||
package com.twitter.tweetypie
|
||||
package core
|
||||
|
||||
import com.twitter.featureswitches.v2.FeatureSwitchResults
|
||||
import com.twitter.tweetypie.thriftscala._
|
||||
|
||||
object TweetData {
|
||||
object Lenses {
|
||||
val tweet: Lens[TweetData, Tweet] = Lens[TweetData, Tweet](_.tweet, _.copy(_))
|
||||
|
||||
val suppress: Lens[TweetData, Option[FilteredState.Suppress]] =
|
||||
Lens[TweetData, Option[FilteredState.Suppress]](
|
||||
_.suppress,
|
||||
(td, suppress) => td.copy(suppress = suppress)
|
||||
)
|
||||
|
||||
val sourceTweetResult: Lens[TweetData, Option[TweetResult]] =
|
||||
Lens[TweetData, Option[TweetResult]](
|
||||
_.sourceTweetResult,
|
||||
(td, sourceTweetResult) => td.copy(sourceTweetResult = sourceTweetResult)
|
||||
)
|
||||
|
||||
val quotedTweetResult: Lens[TweetData, Option[QuotedTweetResult]] =
|
||||
Lens[TweetData, Option[QuotedTweetResult]](
|
||||
_.quotedTweetResult,
|
||||
(td, quotedTweetResult) => td.copy(quotedTweetResult = quotedTweetResult)
|
||||
)
|
||||
|
||||
val cacheableTweetResult: Lens[TweetData, Option[TweetResult]] =
|
||||
Lens[TweetData, Option[TweetResult]](
|
||||
_.cacheableTweetResult,
|
||||
(td, cacheableTweetResult) => td.copy(cacheableTweetResult = cacheableTweetResult)
|
||||
)
|
||||
|
||||
val tweetCounts: Lens[TweetData, Option[StatusCounts]] =
|
||||
Lens[TweetData, Option[StatusCounts]](
|
||||
_.tweet.counts,
|
||||
(td, tweetCounts) => td.copy(tweet = td.tweet.copy(counts = tweetCounts))
|
||||
)
|
||||
}
|
||||
|
||||
def fromCachedTweet(cachedTweet: CachedTweet, cachedAt: Time): TweetData =
|
||||
TweetData(
|
||||
tweet = cachedTweet.tweet,
|
||||
completedHydrations = cachedTweet.completedHydrations.toSet,
|
||||
cachedAt = Some(cachedAt),
|
||||
isBounceDeleted = cachedTweet.isBounceDeleted.contains(true)
|
||||
)
|
||||
}
|
||||
|
||||
/**
|
||||
* Encapsulates a tweet and some hydration metadata in the hydration pipeline.
|
||||
*
|
||||
* @param cachedAt if the tweet was read from cache, `cachedAt` contains the time at which
|
||||
* the tweet was written to cache.
|
||||
*/
|
||||
case class TweetData(
|
||||
tweet: Tweet,
|
||||
suppress: Option[FilteredState.Suppress] = None,
|
||||
completedHydrations: Set[HydrationType] = Set.empty,
|
||||
cachedAt: Option[Time] = None,
|
||||
sourceTweetResult: Option[TweetResult] = None,
|
||||
quotedTweetResult: Option[QuotedTweetResult] = None,
|
||||
cacheableTweetResult: Option[TweetResult] = None,
|
||||
storedTweetResult: Option[StoredTweetResult] = None,
|
||||
featureSwitchResults: Option[FeatureSwitchResults] = None,
|
||||
// The isBounceDeleted flag is only used when reading from an underlying
|
||||
// tweet repo and caching records for not-found tweets. It only exists
|
||||
// as a flag on TweetData to marshal bounce-deleted through the layered
|
||||
// transforming caches injected into CachingTweetRepository, ultimately
|
||||
// storing this flag in thrift on CachedTweet.
|
||||
//
|
||||
// During tweet hydration, TweetData.isBounceDeleted is unused and
|
||||
// should always be false.
|
||||
isBounceDeleted: Boolean = false) {
|
||||
|
||||
def addHydrated(fieldIds: Set[HydrationType]): TweetData =
|
||||
copy(completedHydrations = completedHydrations ++ fieldIds)
|
||||
|
||||
def toCachedTweet: CachedTweet =
|
||||
CachedTweet(
|
||||
tweet = tweet,
|
||||
completedHydrations = completedHydrations,
|
||||
isBounceDeleted = if (isBounceDeleted) Some(true) else None
|
||||
)
|
||||
}
|
@ -0,0 +1,39 @@
|
||||
package com.twitter.tweetypie.core
|
||||
|
||||
import com.twitter.servo.data.Lens
|
||||
import com.twitter.tweetypie.Mutation
|
||||
import com.twitter.tweetypie.thriftscala.Tweet
|
||||
|
||||
/**
|
||||
* Helper class for building instances of `TweetResult`, which is a type alias
|
||||
* for `ValueState[TweetData]`.
|
||||
*/
|
||||
object TweetResult {
|
||||
object Lenses {
|
||||
val value: Lens[TweetResult, TweetData] =
|
||||
Lens[TweetResult, TweetData](_.value, (r, value) => r.copy(value = value))
|
||||
val state: Lens[TweetResult, HydrationState] =
|
||||
Lens[TweetResult, HydrationState](_.state, (r, state) => r.copy(state = state))
|
||||
val tweet: Lens[TweetResult, Tweet] = value.andThen(TweetData.Lenses.tweet)
|
||||
}
|
||||
|
||||
def apply(value: TweetData, state: HydrationState = HydrationState.empty): TweetResult =
|
||||
ValueState(value, state)
|
||||
|
||||
def apply(tweet: Tweet): TweetResult =
|
||||
apply(TweetData(tweet = tweet))
|
||||
|
||||
/**
|
||||
* Apply this mutation to the tweet contained in the result, updating the modified flag if the mutation modifies the tweet.
|
||||
*/
|
||||
def mutate(mutation: Mutation[Tweet]): TweetResult => TweetResult =
|
||||
(result: TweetResult) =>
|
||||
mutation(result.value.tweet) match {
|
||||
case None => result
|
||||
case Some(updatedTweet) =>
|
||||
TweetResult(
|
||||
result.value.copy(tweet = updatedTweet),
|
||||
result.state ++ HydrationState.modified
|
||||
)
|
||||
}
|
||||
}
|
@ -0,0 +1,37 @@
|
||||
package com.twitter.tweetypie.core
|
||||
|
||||
import scala.util.control.NoStackTrace
|
||||
|
||||
/**
|
||||
* Parent exception class for failures while talking to upstream services. These will
|
||||
* be counted and then converted to servo.ServerError.DependencyError
|
||||
*/
|
||||
sealed abstract class UpstreamFailure(msg: String) extends Exception(msg) with NoStackTrace
|
||||
|
||||
object UpstreamFailure {
|
||||
case class SnowflakeFailure(t: Throwable) extends UpstreamFailure(t.toString)
|
||||
|
||||
case object UserProfileEmptyException extends UpstreamFailure("User.profile is empty")
|
||||
|
||||
case object UserViewEmptyException extends UpstreamFailure("User.view is empty")
|
||||
|
||||
case object UserSafetyEmptyException extends UpstreamFailure("User.safety is empty")
|
||||
|
||||
case class TweetLookupFailure(t: Throwable) extends UpstreamFailure(t.toString)
|
||||
|
||||
case class UserLookupFailure(t: Throwable) extends UpstreamFailure(t.toString)
|
||||
|
||||
case class DeviceSourceLookupFailure(t: Throwable) extends UpstreamFailure(t.toString)
|
||||
|
||||
case class TFlockLookupFailure(t: Throwable) extends UpstreamFailure(t.toString)
|
||||
|
||||
case class UrlShorteningFailure(t: Throwable) extends UpstreamFailure(t.toString)
|
||||
|
||||
case object MediaShortenUrlMalformedFailure
|
||||
extends UpstreamFailure("Media shortened url is malformed")
|
||||
|
||||
case object MediaExpandedUrlNotValidFailure
|
||||
extends UpstreamFailure("Talon returns badInput on media expanded url")
|
||||
|
||||
case class MediaServiceServerError(t: Throwable) extends UpstreamFailure(t.toString)
|
||||
}
|
@ -0,0 +1,452 @@
|
||||
package com.twitter.tweetypie.core
|
||||
|
||||
import com.twitter.servo.data.Lens
|
||||
import com.twitter.stitch.Stitch
|
||||
import com.twitter.tweetypie.thriftscala.FieldByPath
|
||||
import com.twitter.tweetypie.thriftscala.HydrationType
|
||||
|
||||
/**
|
||||
* Encapsulates a value and associated HydrationState. This class is intended to be used
|
||||
* with `ValueHydrator`, as the result type for hydrators that directly produce updated values,
|
||||
* in contrast with EditHydrator which uses `EditState` as a result type to produce update functions.
|
||||
*
|
||||
* @tparam A The type of the enclosed value, which is the result of hydration.
|
||||
*/
|
||||
final case class ValueState[+A](value: A, state: HydrationState) {
|
||||
|
||||
/**
|
||||
* Applies a function to the enclosed value and produces a new `ValueState` instance.
|
||||
*/
|
||||
def map[B](f: A => B): ValueState[B] =
|
||||
ValueState(f(value), state)
|
||||
|
||||
/**
|
||||
* Produces a new `ValueState` that contains the value generated by `f`, but with state that is
|
||||
* the sum of the state from this `ValueState` and the one produced by `f`.
|
||||
*/
|
||||
def flatMap[B](f: A => ValueState[B]): ValueState[B] = {
|
||||
val ValueState(value2, state2) = f(value)
|
||||
ValueState(value2, state ++ state2)
|
||||
}
|
||||
|
||||
/**
|
||||
* Applies a function to the enclosed state and produces a new `ValueState` instance.
|
||||
*/
|
||||
def mapState[T](f: HydrationState => HydrationState): ValueState[A] =
|
||||
ValueState(value, f(state))
|
||||
|
||||
/**
|
||||
* Converts a `ValueState[A]` to an `EditState[B]`, using a lens. The resulting `EditState`
|
||||
* will overwrite the lensed field with the value from this `ValueState`.
|
||||
*/
|
||||
def edit[B, A2 >: A](lens: Lens[B, A2]): EditState[B] =
|
||||
EditState[B](b => ValueState(lens.set(b, value), state))
|
||||
}
|
||||
|
||||
object ValueState {
|
||||
val UnmodifiedNone: ValueState[None.type] = unmodified(None)
|
||||
val StitchUnmodifiedNone: Stitch[ValueState[None.type]] = Stitch.value(UnmodifiedNone)
|
||||
|
||||
val UnmodifiedUnit: ValueState[Unit] = unmodified(())
|
||||
val StitchUnmodifiedUnit: Stitch[ValueState[Unit]] = Stitch.value(UnmodifiedUnit)
|
||||
|
||||
val UnmodifiedNil: ValueState[Nil.type] = unmodified(Nil)
|
||||
val StitchUnmodifiedNil: Stitch[ValueState[Nil.type]] = Stitch.value(UnmodifiedNil)
|
||||
|
||||
/**
|
||||
* Produces a ValueState instance with the given value and an empty state HydrationState.
|
||||
*/
|
||||
def unit[A](value: A): ValueState[A] =
|
||||
ValueState[A](value, HydrationState.empty)
|
||||
|
||||
def unmodified[A](value: A): ValueState[A] =
|
||||
ValueState(value, HydrationState.empty)
|
||||
|
||||
def modified[A](value: A): ValueState[A] =
|
||||
ValueState(value, HydrationState.modified)
|
||||
|
||||
def modified[A](value: A, hydrationType: HydrationType): ValueState[A] =
|
||||
ValueState(value, HydrationState.modified(hydrationType))
|
||||
|
||||
def success[A](value: A, modified: Boolean): ValueState[A] =
|
||||
ValueState(value, HydrationState(modified))
|
||||
|
||||
def delta[A](prev: A, next: A): ValueState[A] =
|
||||
ValueState(next, HydrationState.delta(prev, next))
|
||||
|
||||
def partial[A](value: A, field: FieldByPath): ValueState[A] =
|
||||
ValueState(value, HydrationState.partial(field))
|
||||
|
||||
def partial[A](value: A, fields: Set[FieldByPath]): ValueState[A] =
|
||||
ValueState(value, HydrationState.partial(fields))
|
||||
|
||||
/**
|
||||
* Converts a `Seq` of `ValueState[A]` to a `ValueState` of `Seq[A]`.
|
||||
*/
|
||||
def sequence[A](seq: Seq[ValueState[A]]): ValueState[Seq[A]] = {
|
||||
ValueState(
|
||||
value = seq.map(_.value),
|
||||
state = HydrationState.join(seq.map(_.state): _*)
|
||||
)
|
||||
}
|
||||
|
||||
def join[A, B](va: ValueState[A], vb: ValueState[B]): ValueState[(A, B)] = {
|
||||
val state =
|
||||
HydrationState.join(
|
||||
va.state,
|
||||
vb.state
|
||||
)
|
||||
|
||||
val value = (
|
||||
va.value,
|
||||
vb.value
|
||||
)
|
||||
|
||||
ValueState(value, state)
|
||||
}
|
||||
|
||||
def join[A, B, C](
|
||||
va: ValueState[A],
|
||||
vb: ValueState[B],
|
||||
vc: ValueState[C]
|
||||
): ValueState[(A, B, C)] = {
|
||||
val state =
|
||||
HydrationState.join(
|
||||
va.state,
|
||||
vb.state,
|
||||
vc.state
|
||||
)
|
||||
|
||||
val value = (
|
||||
va.value,
|
||||
vb.value,
|
||||
vc.value
|
||||
)
|
||||
|
||||
ValueState(value, state)
|
||||
}
|
||||
|
||||
def join[A, B, C, D](
|
||||
va: ValueState[A],
|
||||
vb: ValueState[B],
|
||||
vc: ValueState[C],
|
||||
vd: ValueState[D]
|
||||
): ValueState[(A, B, C, D)] = {
|
||||
val state =
|
||||
HydrationState.join(
|
||||
va.state,
|
||||
vb.state,
|
||||
vc.state,
|
||||
vd.state
|
||||
)
|
||||
|
||||
val value = (
|
||||
va.value,
|
||||
vb.value,
|
||||
vc.value,
|
||||
vd.value
|
||||
)
|
||||
|
||||
ValueState(value, state)
|
||||
}
|
||||
|
||||
def join[A, B, C, D, E](
|
||||
va: ValueState[A],
|
||||
vb: ValueState[B],
|
||||
vc: ValueState[C],
|
||||
vd: ValueState[D],
|
||||
ve: ValueState[E]
|
||||
): ValueState[(A, B, C, D, E)] = {
|
||||
val state =
|
||||
HydrationState.join(
|
||||
va.state,
|
||||
vb.state,
|
||||
vc.state,
|
||||
vd.state,
|
||||
ve.state
|
||||
)
|
||||
|
||||
val value = (
|
||||
va.value,
|
||||
vb.value,
|
||||
vc.value,
|
||||
vd.value,
|
||||
ve.value
|
||||
)
|
||||
|
||||
ValueState(value, state)
|
||||
}
|
||||
|
||||
def join[A, B, C, D, E, F](
|
||||
va: ValueState[A],
|
||||
vb: ValueState[B],
|
||||
vc: ValueState[C],
|
||||
vd: ValueState[D],
|
||||
ve: ValueState[E],
|
||||
vf: ValueState[F]
|
||||
): ValueState[(A, B, C, D, E, F)] = {
|
||||
val state =
|
||||
HydrationState.join(
|
||||
va.state,
|
||||
vb.state,
|
||||
vc.state,
|
||||
vd.state,
|
||||
ve.state,
|
||||
vf.state
|
||||
)
|
||||
|
||||
val value = (
|
||||
va.value,
|
||||
vb.value,
|
||||
vc.value,
|
||||
vd.value,
|
||||
ve.value,
|
||||
vf.value
|
||||
)
|
||||
|
||||
ValueState(value, state)
|
||||
}
|
||||
|
||||
def join[A, B, C, D, E, F, G](
|
||||
va: ValueState[A],
|
||||
vb: ValueState[B],
|
||||
vc: ValueState[C],
|
||||
vd: ValueState[D],
|
||||
ve: ValueState[E],
|
||||
vf: ValueState[F],
|
||||
vg: ValueState[G]
|
||||
): ValueState[(A, B, C, D, E, F, G)] = {
|
||||
val state =
|
||||
HydrationState.join(
|
||||
va.state,
|
||||
vb.state,
|
||||
vc.state,
|
||||
vd.state,
|
||||
ve.state,
|
||||
vf.state,
|
||||
vg.state
|
||||
)
|
||||
|
||||
val value = (
|
||||
va.value,
|
||||
vb.value,
|
||||
vc.value,
|
||||
vd.value,
|
||||
ve.value,
|
||||
vf.value,
|
||||
vg.value
|
||||
)
|
||||
|
||||
ValueState(value, state)
|
||||
}
|
||||
|
||||
def join[A, B, C, D, E, F, G, H](
|
||||
va: ValueState[A],
|
||||
vb: ValueState[B],
|
||||
vc: ValueState[C],
|
||||
vd: ValueState[D],
|
||||
ve: ValueState[E],
|
||||
vf: ValueState[F],
|
||||
vg: ValueState[G],
|
||||
vh: ValueState[H]
|
||||
): ValueState[(A, B, C, D, E, F, G, H)] = {
|
||||
val state =
|
||||
HydrationState.join(
|
||||
va.state,
|
||||
vb.state,
|
||||
vc.state,
|
||||
vd.state,
|
||||
ve.state,
|
||||
vf.state,
|
||||
vg.state,
|
||||
vh.state
|
||||
)
|
||||
|
||||
val value = (
|
||||
va.value,
|
||||
vb.value,
|
||||
vc.value,
|
||||
vd.value,
|
||||
ve.value,
|
||||
vf.value,
|
||||
vg.value,
|
||||
vh.value
|
||||
)
|
||||
|
||||
ValueState(value, state)
|
||||
}
|
||||
|
||||
def join[A, B, C, D, E, F, G, H, I](
|
||||
va: ValueState[A],
|
||||
vb: ValueState[B],
|
||||
vc: ValueState[C],
|
||||
vd: ValueState[D],
|
||||
ve: ValueState[E],
|
||||
vf: ValueState[F],
|
||||
vg: ValueState[G],
|
||||
vh: ValueState[H],
|
||||
vi: ValueState[I]
|
||||
): ValueState[(A, B, C, D, E, F, G, H, I)] = {
|
||||
val state =
|
||||
HydrationState.join(
|
||||
va.state,
|
||||
vb.state,
|
||||
vc.state,
|
||||
vd.state,
|
||||
ve.state,
|
||||
vf.state,
|
||||
vg.state,
|
||||
vh.state,
|
||||
vi.state
|
||||
)
|
||||
|
||||
val value = (
|
||||
va.value,
|
||||
vb.value,
|
||||
vc.value,
|
||||
vd.value,
|
||||
ve.value,
|
||||
vf.value,
|
||||
vg.value,
|
||||
vh.value,
|
||||
vi.value
|
||||
)
|
||||
|
||||
ValueState(value, state)
|
||||
}
|
||||
|
||||
def join[A, B, C, D, E, F, G, H, I, J](
|
||||
va: ValueState[A],
|
||||
vb: ValueState[B],
|
||||
vc: ValueState[C],
|
||||
vd: ValueState[D],
|
||||
ve: ValueState[E],
|
||||
vf: ValueState[F],
|
||||
vg: ValueState[G],
|
||||
vh: ValueState[H],
|
||||
vi: ValueState[I],
|
||||
vj: ValueState[J]
|
||||
): ValueState[(A, B, C, D, E, F, G, H, I, J)] = {
|
||||
val state =
|
||||
HydrationState.join(
|
||||
va.state,
|
||||
vb.state,
|
||||
vc.state,
|
||||
vd.state,
|
||||
ve.state,
|
||||
vf.state,
|
||||
vg.state,
|
||||
vh.state,
|
||||
vi.state,
|
||||
vj.state
|
||||
)
|
||||
|
||||
val value = (
|
||||
va.value,
|
||||
vb.value,
|
||||
vc.value,
|
||||
vd.value,
|
||||
ve.value,
|
||||
vf.value,
|
||||
vg.value,
|
||||
vh.value,
|
||||
vi.value,
|
||||
vj.value
|
||||
)
|
||||
|
||||
ValueState(value, state)
|
||||
}
|
||||
|
||||
def join[A, B, C, D, E, F, G, H, I, J, K](
|
||||
va: ValueState[A],
|
||||
vb: ValueState[B],
|
||||
vc: ValueState[C],
|
||||
vd: ValueState[D],
|
||||
ve: ValueState[E],
|
||||
vf: ValueState[F],
|
||||
vg: ValueState[G],
|
||||
vh: ValueState[H],
|
||||
vi: ValueState[I],
|
||||
vj: ValueState[J],
|
||||
vk: ValueState[K]
|
||||
): ValueState[(A, B, C, D, E, F, G, H, I, J, K)] = {
|
||||
val state =
|
||||
HydrationState.join(
|
||||
va.state,
|
||||
vb.state,
|
||||
vc.state,
|
||||
vd.state,
|
||||
ve.state,
|
||||
vf.state,
|
||||
vg.state,
|
||||
vh.state,
|
||||
vi.state,
|
||||
vj.state,
|
||||
vk.state
|
||||
)
|
||||
|
||||
val value = (
|
||||
va.value,
|
||||
vb.value,
|
||||
vc.value,
|
||||
vd.value,
|
||||
ve.value,
|
||||
vf.value,
|
||||
vg.value,
|
||||
vh.value,
|
||||
vi.value,
|
||||
vj.value,
|
||||
vk.value
|
||||
)
|
||||
|
||||
ValueState(value, state)
|
||||
}
|
||||
|
||||
def join[A, B, C, D, E, F, G, H, I, J, K, L](
|
||||
va: ValueState[A],
|
||||
vb: ValueState[B],
|
||||
vc: ValueState[C],
|
||||
vd: ValueState[D],
|
||||
ve: ValueState[E],
|
||||
vf: ValueState[F],
|
||||
vg: ValueState[G],
|
||||
vh: ValueState[H],
|
||||
vi: ValueState[I],
|
||||
vj: ValueState[J],
|
||||
vk: ValueState[K],
|
||||
vl: ValueState[L]
|
||||
): ValueState[(A, B, C, D, E, F, G, H, I, J, K, L)] = {
|
||||
val state =
|
||||
HydrationState.join(
|
||||
va.state,
|
||||
vb.state,
|
||||
vc.state,
|
||||
vd.state,
|
||||
ve.state,
|
||||
vf.state,
|
||||
vg.state,
|
||||
vh.state,
|
||||
vi.state,
|
||||
vj.state,
|
||||
vk.state,
|
||||
vl.state
|
||||
)
|
||||
|
||||
val value = (
|
||||
va.value,
|
||||
vb.value,
|
||||
vc.value,
|
||||
vd.value,
|
||||
ve.value,
|
||||
vf.value,
|
||||
vg.value,
|
||||
vh.value,
|
||||
vi.value,
|
||||
vj.value,
|
||||
vk.value,
|
||||
vl.value
|
||||
)
|
||||
|
||||
ValueState(value, state)
|
||||
}
|
||||
}
|
@ -0,0 +1,5 @@
|
||||
package com.twitter.tweetypie
|
||||
|
||||
package object core {
|
||||
type TweetResult = ValueState[TweetData]
|
||||
}
|
@ -0,0 +1,35 @@
|
||||
scala_library(
|
||||
sources = ["*.scala"],
|
||||
compiler_option_sets = ["fatal_warnings"],
|
||||
strict_deps = True,
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"finagle-internal/mtls/src/main/scala/com/twitter/finagle/mtls/authentication",
|
||||
"merlin/util/src/main/scala",
|
||||
"tweetypie/servo/util/src/main/scala",
|
||||
"tweetypie/servo/util/src/main/scala:exception",
|
||||
"src/scala/com/twitter/ads/internal/pcl/service",
|
||||
"src/thrift/com/twitter/ads/adserver:adserver_rpc-scala",
|
||||
"src/thrift/com/twitter/gizmoduck:thrift-scala",
|
||||
"src/thrift/com/twitter/merlin:thrift-scala",
|
||||
"tweetypie/common/src/thrift/com/twitter/tweetypie:service-scala",
|
||||
"tweetypie/common/src/thrift/com/twitter/tweetypie:tweet_service_graphql-scala",
|
||||
"stitch/stitch-core/src/main/scala/com/twitter/stitch",
|
||||
"strato/src/main/scala/com/twitter/strato/access/filters",
|
||||
"strato/src/main/scala/com/twitter/strato/catalog",
|
||||
"strato/src/main/scala/com/twitter/strato/client",
|
||||
"strato/src/main/scala/com/twitter/strato/config",
|
||||
"strato/src/main/scala/com/twitter/strato/fed",
|
||||
"strato/src/main/scala/com/twitter/strato/server/context",
|
||||
"strato/src/main/scala/com/twitter/strato/thrift",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie/backends",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie/federated/columns",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie/federated/context",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie/federated/prefetcheddata",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie/federated/promotedcontent",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie/repository",
|
||||
"tweetypie/server/src/main/thrift:compiled-scala",
|
||||
"util/util-slf4j-api/src/main/scala/com/twitter/util/logging",
|
||||
],
|
||||
)
|
@ -0,0 +1,128 @@
|
||||
package com.twitter.tweetypie.federated
|
||||
|
||||
import com.twitter.ads.internal.pcl.service.CallbackPromotedContentLogger
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.scrooge.ThriftStructFieldInfo
|
||||
import com.twitter.servo.util.Gate
|
||||
import com.twitter.strato.catalog.Catalog
|
||||
import com.twitter.strato.client.Client
|
||||
import com.twitter.strato.fed.StratoFed
|
||||
import com.twitter.strato.thrift.ScroogeConv
|
||||
import com.twitter.tweetypie.ThriftTweetService
|
||||
import com.twitter.tweetypie.Tweet
|
||||
import com.twitter.tweetypie.backends.Gizmoduck
|
||||
import com.twitter.tweetypie.federated.columns._
|
||||
import com.twitter.tweetypie.federated.context.GetRequestContext
|
||||
import com.twitter.tweetypie.federated.prefetcheddata.PrefetchedDataRepositoryBuilder
|
||||
import com.twitter.tweetypie.federated.promotedcontent.TweetPromotedContentLogger
|
||||
import com.twitter.tweetypie.repository.UnmentionInfoRepository
|
||||
import com.twitter.tweetypie.repository.VibeRepository
|
||||
import com.twitter.util.Activity
|
||||
import com.twitter.util.logging.Logger
|
||||
|
||||
object StratoCatalogBuilder {
|
||||
|
||||
def catalog(
|
||||
thriftTweetService: ThriftTweetService,
|
||||
stratoClient: Client,
|
||||
getUserResultsById: Gizmoduck.GetById,
|
||||
callbackPromotedContentLogger: CallbackPromotedContentLogger,
|
||||
statsReceiver: StatsReceiver,
|
||||
enableCommunityTweetCreatesDecider: Gate[Unit],
|
||||
): Activity[Catalog[StratoFed.Column]] = {
|
||||
val log = Logger(getClass)
|
||||
|
||||
val getRequestContext = new GetRequestContext()
|
||||
val prefetchedDataRepository =
|
||||
PrefetchedDataRepositoryBuilder(getUserResultsById, statsReceiver)
|
||||
val unmentionInfoRepository = UnmentionInfoRepository(stratoClient)
|
||||
val vibeRepository = VibeRepository(stratoClient)
|
||||
|
||||
val tweetPromotedContentLogger =
|
||||
TweetPromotedContentLogger(callbackPromotedContentLogger)
|
||||
|
||||
// A stitch group builder to be used for Federated Field Column requests. The handler must be the same across
|
||||
// all Federated Field Columns to ensure requests are batched across columns for different fields
|
||||
val federatedFieldGroupBuilder: FederatedFieldGroupBuilder.Type = FederatedFieldGroupBuilder(
|
||||
thriftTweetService.getTweetFields)
|
||||
|
||||
val columns: Seq[StratoFed.Column] = Seq(
|
||||
new UnretweetColumn(
|
||||
thriftTweetService.unretweet,
|
||||
getRequestContext,
|
||||
),
|
||||
new CreateRetweetColumn(
|
||||
thriftTweetService.postRetweet,
|
||||
getRequestContext,
|
||||
prefetchedDataRepository,
|
||||
tweetPromotedContentLogger,
|
||||
statsReceiver
|
||||
),
|
||||
new CreateTweetColumn(
|
||||
thriftTweetService.postTweet,
|
||||
getRequestContext,
|
||||
prefetchedDataRepository,
|
||||
unmentionInfoRepository,
|
||||
vibeRepository,
|
||||
tweetPromotedContentLogger,
|
||||
statsReceiver,
|
||||
enableCommunityTweetCreatesDecider,
|
||||
),
|
||||
new DeleteTweetColumn(
|
||||
thriftTweetService.deleteTweets,
|
||||
getRequestContext,
|
||||
),
|
||||
new GetTweetFieldsColumn(thriftTweetService.getTweetFields, statsReceiver),
|
||||
new GetStoredTweetsColumn(thriftTweetService.getStoredTweets),
|
||||
new GetStoredTweetsByUserColumn(thriftTweetService.getStoredTweetsByUser)
|
||||
)
|
||||
|
||||
// Gather tweet field ids that are eligible to be federated field columns
|
||||
val federatedFieldInfos =
|
||||
Tweet.fieldInfos
|
||||
.filter((info: ThriftStructFieldInfo) =>
|
||||
FederatedFieldColumn.isFederatedField(info.tfield.id))
|
||||
|
||||
// Instantiate the federated field columns
|
||||
val federatedFieldColumns: Seq[FederatedFieldColumn] =
|
||||
federatedFieldInfos.map { fieldInfo: ThriftStructFieldInfo =>
|
||||
val path = FederatedFieldColumn.makeColumnPath(fieldInfo.tfield)
|
||||
val stratoType = ScroogeConv.typeOfFieldInfo(fieldInfo)
|
||||
log.info(f"creating federated column: $path")
|
||||
new FederatedFieldColumn(
|
||||
federatedFieldGroupBuilder,
|
||||
thriftTweetService.setAdditionalFields,
|
||||
stratoType,
|
||||
fieldInfo.tfield,
|
||||
)
|
||||
}
|
||||
|
||||
// Instantiate the federated V1 field columns
|
||||
val federatedV1FieldColumns: Seq[FederatedFieldColumn] =
|
||||
federatedFieldInfos
|
||||
.filter(f => FederatedFieldColumn.isMigrationFederatedField(f.tfield))
|
||||
.map { fieldInfo: ThriftStructFieldInfo =>
|
||||
val v1Path = FederatedFieldColumn.makeV1ColumnPath(fieldInfo.tfield)
|
||||
val stratoType = ScroogeConv.typeOfFieldInfo(fieldInfo)
|
||||
log.info(f"creating V1 federated column: $v1Path")
|
||||
new FederatedFieldColumn(
|
||||
federatedFieldGroupBuilder,
|
||||
thriftTweetService.setAdditionalFields,
|
||||
stratoType,
|
||||
fieldInfo.tfield,
|
||||
Some(v1Path)
|
||||
)
|
||||
}
|
||||
|
||||
// Combine the dynamic and hard coded federated columns
|
||||
val allColumns: Seq[StratoFed.Column] =
|
||||
columns ++ federatedFieldColumns ++ federatedV1FieldColumns
|
||||
|
||||
Activity.value(
|
||||
Catalog(
|
||||
allColumns.map { column =>
|
||||
column.path -> column
|
||||
}: _*
|
||||
))
|
||||
}
|
||||
}
|
@ -0,0 +1,41 @@
|
||||
package com.twitter.tweetypie.federated.columns
|
||||
|
||||
import com.twitter.passbird.bitfield.clientprivileges.thriftscala.{Constants => ClientAppPrivileges}
|
||||
import com.twitter.strato.access.Access.AuthenticatedTwitterUserNotSuspended
|
||||
import com.twitter.strato.access.Access.ClientApplicationPrivilege
|
||||
import com.twitter.strato.access.Access.TwitterUserNotSuspended
|
||||
import com.twitter.strato.access.ClientApplicationPrivilegeVariant
|
||||
import com.twitter.strato.config._
|
||||
|
||||
object AccessPolicy {
|
||||
|
||||
/**
|
||||
* All Tweet Mutation operations require all of:
|
||||
* - Twitter user authentication
|
||||
* - Twitter user is not suspended
|
||||
* - Contributor user, if provided, is not suspended
|
||||
* - "Teams Access": user is acting their own behalf, or is a
|
||||
* contributor using a client with ClientAppPriviledges.CONTRIBUTORS
|
||||
* - Write privileges
|
||||
*/
|
||||
val TweetMutationCommonAccessPolicies: Policy =
|
||||
AllOf(
|
||||
Seq(
|
||||
AllowTwitterUserId,
|
||||
Has(
|
||||
TwitterUserNotSuspended
|
||||
),
|
||||
Has(
|
||||
AuthenticatedTwitterUserNotSuspended
|
||||
),
|
||||
AnyOf(
|
||||
Seq(
|
||||
TwitterUserContributingAsSelf,
|
||||
Has(principal = ClientApplicationPrivilege(ClientApplicationPrivilegeVariant
|
||||
.byId(ClientAppPrivileges.CONTRIBUTORS.toShort).get))
|
||||
)),
|
||||
AllowWritableAccessToken
|
||||
)
|
||||
)
|
||||
|
||||
}
|
@ -0,0 +1,110 @@
|
||||
package com.twitter.tweetypie.federated
|
||||
package columns
|
||||
|
||||
import com.twitter.bouncer.thriftscala.Bounce
|
||||
import com.twitter.finagle.http.Status
|
||||
import com.twitter.finatra.api11
|
||||
import com.twitter.finatra.api11.ApiError
|
||||
import com.twitter.strato.response.Err
|
||||
|
||||
object ApiErrors {
|
||||
// Errs ported from StatusesRetweetController
|
||||
val GenericAccessDeniedErr = toErr(ApiError.GenericAccessDenied)
|
||||
val AlreadyRetweetedErr = toErr(ApiError.AlreadyRetweeted)
|
||||
val DuplicateStatusErr = toErr(ApiError.DuplicateStatusError)
|
||||
val InvalidRetweetForStatusErr = toErr(ApiError.InvalidRetweetForStatus)
|
||||
val StatusNotFoundErr = toErr(ApiError.StatusNotFound)
|
||||
val BlockedUserErr =
|
||||
toErr(ApiError.BlockedUserError, "retweeting this user's tweets at their request")
|
||||
val ClientNotPrivilegedErr = toErr(ApiError.ClientNotPrivileged)
|
||||
val UserDeniedRetweetErr = toErr(ApiError.CurrentUserSuspended)
|
||||
|
||||
// Errs ported from StatusesUpdateController
|
||||
val RateLimitExceededErr = toErr(ApiError.OverStatusUpdateLimit, "User")
|
||||
val TweetUrlSpamErr = toErr(ApiError.TieredActionTweetUrlSpam)
|
||||
val TweetSpammerErr = toErr(ApiError.TieredActionTweetSpammer)
|
||||
val CaptchaChallengeErr = toErr(ApiError.TieredActionChallengeCaptcha)
|
||||
val SafetyRateLimitExceededErr = toErr(ApiError.UserActionRateLimitExceeded, "User")
|
||||
val TweetCannotBeBlankErr = // was MissingRequiredParameterException
|
||||
toErr(ApiError.ForbiddenMissingParameter, "tweet_text or media")
|
||||
val TweetTextTooLongErr = toErr(ApiError.StatusTooLongError)
|
||||
val MalwareTweetErr = toErr(ApiError.StatusMalwareError)
|
||||
val DuplicateTweetErr = toErr(ApiError.DuplicateStatusError)
|
||||
val CurrentUserSuspendedErr = toErr(ApiError.CurrentUserSuspended)
|
||||
val MentionLimitExceededErr = toErr(ApiError.MentionLimitInTweetExceeded)
|
||||
val UrlLimitExceededErr = toErr(ApiError.UrlLimitInTweetExceeded)
|
||||
val HashtagLimitExceededErr = toErr(ApiError.HashtagLimitInTweetExceeded)
|
||||
val CashtagLimitExceededErr = toErr(ApiError.CashtagLimitInTweetExceeded)
|
||||
val HashtagLengthLimitExceededErr = toErr(ApiError.HashtagLengthLimitInTweetExceeded)
|
||||
val TooManyAttachmentTypesErr = toErr(ApiError.AttachmentTypesLimitInTweetExceeded)
|
||||
val InvalidAttachmentUrlErr = toErr(ApiError.InvalidParameter("attachment_url"))
|
||||
val InReplyToTweetNotFoundErr = toErr(ApiError.InReplyToTweetNotFound)
|
||||
val InvalidAdditionalFieldErr = toErr(ApiError.GenericBadRequest)
|
||||
def invalidAdditionalFieldWithReasonErr(failureReason: String) =
|
||||
toErr(ApiError.GenericBadRequest.copy(message = failureReason))
|
||||
val InvalidUrlErr = toErr(ApiError.InvalidUrl)
|
||||
val InvalidCoordinatesErr = toErr(ApiError.InvalidCoordinates)
|
||||
val InvalidGeoSearchRequestIdErr =
|
||||
toErr(ApiError.InvalidParameter("geo_search_request_id"))
|
||||
val ConversationControlNotAuthorizedErr = toErr(ApiError.ConversationControlNotAuthorized)
|
||||
val ConversationControlInvalidErr = toErr(ApiError.ConversationControlInvalid)
|
||||
val ConversationControlReplyRestricted = toErr(ApiError.ConversationControlReplyRestricted)
|
||||
|
||||
// Errors ported from StatusesDestroyController
|
||||
val DeletePermissionErr = toErr(ApiError.StatusActionPermissionError("delete"))
|
||||
|
||||
// See StatusesUpdateController#GenericErrorException
|
||||
val GenericTweetCreateErr = toErr(ApiError.UnknownInterpreterError, "Tweet creation failed")
|
||||
val InvalidBatchModeParameterErr = toErr(ApiError.InvalidParameter("batch_mode"))
|
||||
val CannotConvoControlAndCommunitiesErr =
|
||||
toErr(ApiError.CommunityInvalidParams, "conversation_control")
|
||||
val TooManyCommunitiesErr = toErr(ApiError.CommunityInvalidParams, "communities")
|
||||
val CommunityReplyTweetNotAllowedErr = toErr(ApiError.CommunityReplyTweetNotAllowed)
|
||||
val ConversationControlNotSupportedErr = toErr(ApiError.ConversationControlNotSupported)
|
||||
val CommunityUserNotAuthorizedErr = toErr(ApiError.CommunityUserNotAuthorized)
|
||||
val CommunityNotFoundErr = toErr(ApiError.CommunityNotFound)
|
||||
val CommunityProtectedUserCannotTweetErr = toErr(ApiError.CommunityProtectedUserCannotTweet)
|
||||
|
||||
val SuperFollowCreateNotAuthorizedErr = toErr(ApiError.SuperFollowsCreateNotAuthorized)
|
||||
val SuperFollowInvalidParamsErr = toErr(ApiError.SuperFollowsInvalidParams)
|
||||
val ExclusiveTweetEngagementNotAllowedErr = toErr(ApiError.ExclusiveTweetEngagementNotAllowed)
|
||||
|
||||
val SafetyLevelMissingErr = toErr(ApiError.MissingParameter("safety_level"))
|
||||
|
||||
def accessDeniedByBouncerErr(bounce: Bounce) =
|
||||
toErr(ApiError.AccessDeniedByBouncer, bounce.errorMessage.getOrElse(Seq.empty))
|
||||
|
||||
def tweetEngagementLimitedErr(failureReason: String) =
|
||||
toErr(ApiError.TweetEngagementsLimited(failureReason))
|
||||
|
||||
def invalidMediaErr(failureReason: Option[String]) =
|
||||
toErr(ApiError.invalidMediaId(failureReason))
|
||||
|
||||
val TrustedFriendsInvalidParamsErr = toErr(ApiError.TrustedFriendsInvalidParams)
|
||||
val TrustedFriendsRetweetNotAllowedErr = toErr(ApiError.TrustedFriendsRetweetNotAllowed)
|
||||
val TrustedFriendsEngagementNotAllowedErr = toErr(ApiError.TrustedFriendsEngagementNotAllowed)
|
||||
val TrustedFriendsCreateNotAllowedErr = toErr(ApiError.TrustedFriendsCreateNotAllowed)
|
||||
val TrustedFriendsQuoteTweetNotAllowedErr = toErr(ApiError.TrustedFriendsQuoteTweetNotAllowed)
|
||||
|
||||
val StaleTweetEngagementNotAllowedErr = toErr(ApiError.StaleTweetEngagementNotAllowed)
|
||||
val StaleTweetQuoteTweetNotAllowedErr = toErr(ApiError.StaleTweetQuoteTweetNotAllowed)
|
||||
val StaleTweetRetweetNotAllowedErr = toErr(ApiError.StaleTweetRetweetNotAllowed)
|
||||
|
||||
val CollabTweetInvalidParamsErr = toErr(ApiError.CollabTweetInvalidParams)
|
||||
|
||||
val FieldEditNotAllowedErr = toErr(ApiError.FieldEditNotAllowed)
|
||||
val NotEligibleForEditErr = toErr(ApiError.NotEligibleForEdit)
|
||||
|
||||
def toErr(apiError: api11.ApiError, args: Any*): Err = {
|
||||
val errCode = apiError.status match {
|
||||
case Status.Forbidden => Err.Authorization
|
||||
case Status.Unauthorized => Err.Authentication
|
||||
case Status.NotFound => Err.BadRequest
|
||||
case Status.BadRequest => Err.BadRequest
|
||||
case _ => Err.BadRequest
|
||||
}
|
||||
val errMessage = s"${apiError.message.format(args.mkString(","))} (${apiError.code})"
|
||||
val errContext = Some(Err.Context.Api11Error(apiError.code))
|
||||
Err(errCode, errMessage, errContext)
|
||||
}
|
||||
}
|
@ -0,0 +1,43 @@
|
||||
scala_library(
|
||||
sources = ["*.scala"],
|
||||
compiler_option_sets = ["fatal_warnings"],
|
||||
strict_deps = True,
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"decider",
|
||||
"finagle/finagle-base-http/src/main",
|
||||
"finatra-internal/api11/src/main/scala/com/twitter/finatra/api11:errors",
|
||||
"geo/model/src/main/scala/com/twitter/geo/model",
|
||||
"passbird/bitfields-thrift/src/main/thrift:thrift-scala",
|
||||
"tweetypie/servo/util/src/main/scala:exception",
|
||||
"src/scala/com/twitter/accounts/util:safety-meta",
|
||||
"src/thrift/com/twitter/ads/adserver:ad_engagement_details-scala",
|
||||
"src/thrift/com/twitter/ads/adserver:preroll_metadata-scala",
|
||||
"src/thrift/com/twitter/ads/callback:engagement_request-scala",
|
||||
"src/thrift/com/twitter/bouncer:bounce-action-thrift-scala",
|
||||
"src/thrift/com/twitter/consumer_privacy/mention_controls:thrift-scala",
|
||||
"tweetypie/common/src/thrift/com/twitter/tweetypie:service-scala",
|
||||
"tweetypie/common/src/thrift/com/twitter/tweetypie:stored-tweet-info-scala",
|
||||
"tweetypie/common/src/thrift/com/twitter/tweetypie:tweet-service-federated-scala",
|
||||
"tweetypie/common/src/thrift/com/twitter/tweetypie:tweet_service_graphql-scala",
|
||||
"stitch/stitch-core/src/main/scala/com/twitter/stitch",
|
||||
"strato/config/src/thrift/com/twitter/strato/graphql:api-media-graphql-scala",
|
||||
"strato/config/src/thrift/com/twitter/strato/graphql:graphql-scala",
|
||||
"strato/config/src/thrift/com/twitter/strato/graphql:topics-graphql-scala",
|
||||
"strato/src/main/scala/com/twitter/strato/client",
|
||||
"strato/src/main/scala/com/twitter/strato/fed",
|
||||
"strato/src/main/scala/com/twitter/strato/response",
|
||||
"strato/src/main/scala/com/twitter/strato/thrift",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie/federated/context",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie/federated/prefetcheddata",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie/federated/promotedcontent",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie/repository",
|
||||
"tweetypie/server/src/main/thrift:compiled-scala",
|
||||
"tweetypie/common/src/scala/com/twitter/tweetypie/client_id",
|
||||
"tweetypie/common/src/scala/com/twitter/tweetypie/decider/overrides",
|
||||
"tweetypie/common/src/scala/com/twitter/tweetypie/util",
|
||||
"vibes/src/main/thrift/com/twitter/vibes:vibes-scala",
|
||||
"weaverbird/common/src/main/scala/com/twitter/weaverbird/common",
|
||||
],
|
||||
)
|
@ -0,0 +1,184 @@
|
||||
package com.twitter.tweetypie
|
||||
package federated.columns
|
||||
|
||||
import com.twitter.accounts.util.SafetyMetadataUtils
|
||||
import com.twitter.ads.callback.thriftscala.EngagementRequest
|
||||
import com.twitter.bouncer.thriftscala.{Bounce => BouncerBounce}
|
||||
import com.twitter.stitch.Stitch
|
||||
import com.twitter.strato.catalog.OpMetadata
|
||||
import com.twitter.strato.config.AllOf
|
||||
import com.twitter.strato.config.BouncerAccess
|
||||
import com.twitter.strato.config.ContactInfo
|
||||
import com.twitter.strato.config.Policy
|
||||
import com.twitter.strato.data.Conv
|
||||
import com.twitter.strato.data.Description.PlainText
|
||||
import com.twitter.strato.data.Lifecycle.Production
|
||||
import com.twitter.strato.fed.StratoFed
|
||||
import com.twitter.strato.opcontext.OpContext
|
||||
import com.twitter.strato.response.Err
|
||||
import com.twitter.strato.thrift.ScroogeConv
|
||||
import com.twitter.tweetypie.federated.columns.ApiErrors._
|
||||
import com.twitter.tweetypie.federated.columns.CreateRetweetColumn.toCreateRetweetErr
|
||||
import com.twitter.tweetypie.federated.context.GetRequestContext
|
||||
import com.twitter.tweetypie.federated.prefetcheddata.PrefetchedDataRequest
|
||||
import com.twitter.tweetypie.federated.prefetcheddata.PrefetchedDataResponse
|
||||
import com.twitter.tweetypie.federated.promotedcontent.TweetPromotedContentLogger
|
||||
import com.twitter.tweetypie.federated.promotedcontent.TweetPromotedContentLogger.RetweetEngagement
|
||||
import com.twitter.tweetypie.thriftscala.TweetCreateState._
|
||||
import com.twitter.tweetypie.thriftscala.{graphql => gql}
|
||||
import com.twitter.tweetypie.{thriftscala => thrift}
|
||||
import com.twitter.weaverbird.common.{GetRequestContext => WGetRequestContext}
|
||||
|
||||
class CreateRetweetColumn(
|
||||
retweet: thrift.RetweetRequest => Future[thrift.PostTweetResult],
|
||||
getRequestContext: GetRequestContext,
|
||||
prefetchedDataRepository: PrefetchedDataRequest => Stitch[PrefetchedDataResponse],
|
||||
logTweetPromotedContent: TweetPromotedContentLogger.Type,
|
||||
statsReceiver: StatsReceiver,
|
||||
) extends StratoFed.Column(CreateRetweetColumn.Path)
|
||||
with StratoFed.Execute.StitchWithContext
|
||||
with StratoFed.HandleDarkRequests {
|
||||
|
||||
override val policy: Policy = AllOf(
|
||||
Seq(AccessPolicy.TweetMutationCommonAccessPolicies, BouncerAccess()))
|
||||
|
||||
// The underlying call to thriftTweetService.postRetweet is not idempotent
|
||||
override val isIdempotent: Boolean = false
|
||||
|
||||
override type Arg = gql.CreateRetweetRequest
|
||||
override type Result = gql.CreateRetweetResponseWithSubqueryPrefetchItems
|
||||
|
||||
override val argConv: Conv[Arg] = ScroogeConv.fromStruct
|
||||
override val resultConv: Conv[Result] = ScroogeConv.fromStruct
|
||||
|
||||
override val contactInfo: ContactInfo = TweetypieContactInfo
|
||||
override val metadata: OpMetadata = OpMetadata(
|
||||
Some(Production),
|
||||
Some(PlainText("Creates a retweet by the calling Twitter user of the given source tweet.")))
|
||||
|
||||
private val getWeaverbirdCtx = new WGetRequestContext()
|
||||
|
||||
override def execute(request: Arg, opContext: OpContext): Stitch[Result] = {
|
||||
val ctx = getRequestContext(opContext)
|
||||
|
||||
// First, do any request parameter validation that can result in an error
|
||||
// prior to calling into thriftTweetService.retweet.
|
||||
val safetyLevel = ctx.safetyLevel.getOrElse(throw SafetyLevelMissingErr)
|
||||
|
||||
// Macaw-tweets returns ApiError.ClientNotPrivileged if the caller provides
|
||||
// an impression_id but lacks the PROMOTED_TWEETS_IN_TIMELINE privilege.
|
||||
val trackingId = request.engagementRequest match {
|
||||
case Some(engagementRequest: EngagementRequest) if ctx.hasPrivilegePromotedTweetsInTimeline =>
|
||||
TrackingId.parse(engagementRequest.impressionId, statsReceiver)
|
||||
case Some(e: EngagementRequest) =>
|
||||
throw ClientNotPrivilegedErr
|
||||
case None =>
|
||||
None
|
||||
}
|
||||
|
||||
// DeviceSource is an oauth string computed from the ClientApplicationId.
|
||||
// Macaw-tweets allows non-oauth callers, but GraphQL does not. An undefined
|
||||
// ClientApplicationId is similar to TweetCreateState.DeviceSourceNotFound,
|
||||
// which Macaw-tweets handles via a catch-all that returns
|
||||
// ApiError.GenericAccessDenied
|
||||
val deviceSource = ctx.deviceSource.getOrElse(throw GenericAccessDeniedErr)
|
||||
|
||||
// Macaw-tweets doesn't perform any parameter validation for the components
|
||||
// used as input to makeSafetyMetaData.
|
||||
val safetyMetadata = SafetyMetadataUtils.makeSafetyMetaData(
|
||||
sessionHash = ctx.sessionHash,
|
||||
knownDeviceToken = ctx.knownDeviceToken,
|
||||
contributorId = ctx.contributorId
|
||||
)
|
||||
|
||||
val thriftRetweetRequest = thrift.RetweetRequest(
|
||||
sourceStatusId = request.tweetId,
|
||||
userId = ctx.twitterUserId,
|
||||
contributorUserId = None, // no longer supported, per tweet_service.thrift
|
||||
createdVia = deviceSource,
|
||||
nullcast = request.nullcast,
|
||||
trackingId = trackingId,
|
||||
dark = ctx.isDarkRequest,
|
||||
hydrationOptions = Some(HydrationOptions.writePathHydrationOptions(ctx.cardsPlatformKey)),
|
||||
safetyMetaData = Some(safetyMetadata),
|
||||
)
|
||||
|
||||
val stitchRetweet = Stitch.callFuture(retweet(thriftRetweetRequest))
|
||||
|
||||
request.engagementRequest.foreach { engagement =>
|
||||
logTweetPromotedContent(engagement, RetweetEngagement, ctx.isDarkRequest)
|
||||
}
|
||||
|
||||
stitchRetweet.flatMap { result: thrift.PostTweetResult =>
|
||||
result.state match {
|
||||
case thrift.TweetCreateState.Ok =>
|
||||
val r = PrefetchedDataRequest(
|
||||
tweet = result.tweet.get,
|
||||
sourceTweet = result.sourceTweet,
|
||||
quotedTweet = result.quotedTweet,
|
||||
safetyLevel = safetyLevel,
|
||||
requestContext = getWeaverbirdCtx()
|
||||
)
|
||||
|
||||
prefetchedDataRepository(r)
|
||||
.liftToOption()
|
||||
.map((prefetchedData: Option[PrefetchedDataResponse]) => {
|
||||
gql.CreateRetweetResponseWithSubqueryPrefetchItems(
|
||||
data = Some(gql.CreateRetweetResponse(result.tweet.map(_.id))),
|
||||
subqueryPrefetchItems = prefetchedData.map(_.value)
|
||||
)
|
||||
})
|
||||
case errState =>
|
||||
throw toCreateRetweetErr(errState, result.bounce, result.failureReason)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object CreateRetweetColumn {
|
||||
val Path = "tweetypie/createRetweet.Tweet"
|
||||
|
||||
/**
|
||||
* Ported from:
|
||||
* StatusesRetweetController#retweetStatus rescue block
|
||||
* TweetyPieStatusRepository.toRetweetException
|
||||
*/
|
||||
def toCreateRetweetErr(
|
||||
errState: thrift.TweetCreateState,
|
||||
bounce: Option[BouncerBounce],
|
||||
failureReason: Option[String]
|
||||
): Err = errState match {
|
||||
case CannotRetweetBlockingUser =>
|
||||
BlockedUserErr
|
||||
case AlreadyRetweeted =>
|
||||
AlreadyRetweetedErr
|
||||
case Duplicate =>
|
||||
DuplicateStatusErr
|
||||
case CannotRetweetOwnTweet | CannotRetweetProtectedTweet | CannotRetweetSuspendedUser =>
|
||||
InvalidRetweetForStatusErr
|
||||
case UserNotFound | SourceTweetNotFound | SourceUserNotFound | CannotRetweetDeactivatedUser =>
|
||||
StatusNotFoundErr
|
||||
case UserDeactivated | UserSuspended =>
|
||||
UserDeniedRetweetErr
|
||||
case RateLimitExceeded =>
|
||||
RateLimitExceededErr
|
||||
case UrlSpam =>
|
||||
TweetUrlSpamErr
|
||||
case Spam | UserReadonly =>
|
||||
TweetSpammerErr
|
||||
case SafetyRateLimitExceeded =>
|
||||
SafetyRateLimitExceededErr
|
||||
case Bounce if bounce.isDefined =>
|
||||
accessDeniedByBouncerErr(bounce.get)
|
||||
case DisabledByIpiPolicy =>
|
||||
failureReason
|
||||
.map(tweetEngagementLimitedErr)
|
||||
.getOrElse(GenericAccessDeniedErr)
|
||||
case TrustedFriendsRetweetNotAllowed =>
|
||||
TrustedFriendsRetweetNotAllowedErr
|
||||
case StaleTweetRetweetNotAllowed =>
|
||||
StaleTweetRetweetNotAllowedErr
|
||||
case _ =>
|
||||
GenericAccessDeniedErr
|
||||
}
|
||||
}
|
@ -0,0 +1,546 @@
|
||||
package com.twitter.tweetypie
|
||||
package federated.columns
|
||||
|
||||
import com.twitter.accounts.util.SafetyMetadataUtils
|
||||
import com.twitter.ads.callback.thriftscala.EngagementRequest
|
||||
import com.twitter.bouncer.thriftscala.{Bounce => BouncerBounce}
|
||||
import com.twitter.escherbird.thriftscala.TweetEntityAnnotation
|
||||
import com.twitter.geo.model.LatitudeLongitude
|
||||
import com.twitter.stitch.Stitch
|
||||
import com.twitter.strato.catalog.OpMetadata
|
||||
import com.twitter.strato.config.AllOf
|
||||
import com.twitter.strato.config.BouncerAccess
|
||||
import com.twitter.strato.config.ContactInfo
|
||||
import com.twitter.strato.config.Policy
|
||||
import com.twitter.strato.data.Conv
|
||||
import com.twitter.strato.data.Description.PlainText
|
||||
import com.twitter.strato.data.Lifecycle.Production
|
||||
import com.twitter.strato.fed.StratoFed
|
||||
import com.twitter.strato.opcontext.OpContext
|
||||
import com.twitter.strato.response.Err
|
||||
import com.twitter.strato.thrift.ScroogeConv
|
||||
import com.twitter.tweetypie.decider.overrides.TweetyPieDeciderOverrides
|
||||
import com.twitter.tweetypie.federated.columns.ApiErrors._
|
||||
import com.twitter.tweetypie.federated.columns.CreateTweetColumn.toCreateTweetErr
|
||||
import com.twitter.tweetypie.federated.context.GetRequestContext
|
||||
import com.twitter.tweetypie.federated.prefetcheddata.PrefetchedDataRequest
|
||||
import com.twitter.tweetypie.federated.prefetcheddata.PrefetchedDataResponse
|
||||
import com.twitter.tweetypie.federated.promotedcontent.TweetPromotedContentLogger
|
||||
import com.twitter.tweetypie.federated.promotedcontent.TweetPromotedContentLogger._
|
||||
import com.twitter.tweetypie.repository.UnmentionInfoRepository
|
||||
import com.twitter.tweetypie.repository.VibeRepository
|
||||
import com.twitter.tweetypie.thriftscala.TransientCreateContext
|
||||
import com.twitter.tweetypie.thriftscala.TweetCreateContextKey
|
||||
import com.twitter.tweetypie.thriftscala.TweetCreateState._
|
||||
import com.twitter.tweetypie.thriftscala.{graphql => gql}
|
||||
import com.twitter.tweetypie.util.CommunityAnnotation
|
||||
import com.twitter.tweetypie.util.ConversationControls
|
||||
import com.twitter.tweetypie.util.TransientContextUtil
|
||||
import com.twitter.tweetypie.{thriftscala => thrift}
|
||||
import com.twitter.util.Throwables
|
||||
import com.twitter.weaverbird.common.{GetRequestContext => WGetRequestContext}
|
||||
|
||||
class CreateTweetColumn(
|
||||
postTweet: thrift.PostTweetRequest => Future[thrift.PostTweetResult],
|
||||
getRequestContext: GetRequestContext,
|
||||
prefetchedDataRepository: PrefetchedDataRequest => Stitch[PrefetchedDataResponse],
|
||||
unmentionInfoRepository: UnmentionInfoRepository.Type,
|
||||
vibeRepository: VibeRepository.Type,
|
||||
logTweetPromotedContent: TweetPromotedContentLogger.Type,
|
||||
statsReceiver: StatsReceiver,
|
||||
enableCommunityTweetCreatesDecider: Gate[Unit],
|
||||
) extends StratoFed.Column(CreateTweetColumn.Path)
|
||||
with StratoFed.Execute.StitchWithContext
|
||||
with StratoFed.HandleDarkRequests {
|
||||
|
||||
override val policy: Policy = AllOf(
|
||||
Seq(AccessPolicy.TweetMutationCommonAccessPolicies, BouncerAccess()))
|
||||
|
||||
// The underlying call to thriftTweetService.postRetweet is not idempotent
|
||||
override val isIdempotent: Boolean = false
|
||||
|
||||
override type Arg = gql.CreateTweetRequest
|
||||
override type Result = gql.CreateTweetResponseWithSubqueryPrefetchItems
|
||||
|
||||
override val argConv: Conv[Arg] = ScroogeConv.fromStruct
|
||||
override val resultConv: Conv[Result] = ScroogeConv.fromStruct
|
||||
|
||||
override val contactInfo: ContactInfo = TweetypieContactInfo
|
||||
override val metadata: OpMetadata =
|
||||
OpMetadata(
|
||||
Some(Production),
|
||||
Some(
|
||||
PlainText(
|
||||
"""
|
||||
Creates a tweet using the calling authenticated Twitter user as author.
|
||||
NOTE, not all Tweet space fields are GraphQL queryable in the CreateTweet mutation response.
|
||||
See http://go/missing-create-tweet-fields.
|
||||
"""))
|
||||
)
|
||||
|
||||
private val getWeaverbirdCtx = new WGetRequestContext()
|
||||
|
||||
override def execute(request: Arg, opContext: OpContext): Stitch[Result] = {
|
||||
|
||||
val ctx = getRequestContext(opContext)
|
||||
|
||||
// First, do any request parameter validation that can result in an error
|
||||
// prior to calling into thriftTweetService.postTweet.
|
||||
val safetyLevel = ctx.safetyLevel.getOrElse(throw SafetyLevelMissingErr)
|
||||
|
||||
val trackingId = request.engagementRequest match {
|
||||
case Some(engagementRequest: EngagementRequest) if ctx.hasPrivilegePromotedTweetsInTimeline =>
|
||||
TrackingId.parse(engagementRequest.impressionId, statsReceiver)
|
||||
case Some(e: EngagementRequest) =>
|
||||
throw ClientNotPrivilegedErr
|
||||
case None =>
|
||||
None
|
||||
}
|
||||
|
||||
val deviceSource = ctx.deviceSource.getOrElse(throw GenericAccessDeniedErr)
|
||||
|
||||
if (request.nullcast && !ctx.hasPrivilegeNullcastingAccess) {
|
||||
throw GenericAccessDeniedErr
|
||||
}
|
||||
|
||||
val safetyMetadata = SafetyMetadataUtils.makeSafetyMetaData(
|
||||
sessionHash = ctx.sessionHash,
|
||||
knownDeviceToken = ctx.knownDeviceToken,
|
||||
contributorId = ctx.contributorId
|
||||
)
|
||||
|
||||
val cardReference: Option[thrift.CardReference] =
|
||||
request.cardUri.filter(_.nonEmpty).map(thrift.CardReference(_))
|
||||
|
||||
val escherbirdEntityAnnotations: Option[thrift.EscherbirdEntityAnnotations] =
|
||||
request.semanticAnnotationIds
|
||||
.filter(_.nonEmpty)
|
||||
.map((seq: Seq[gql.TweetAnnotation]) => seq.map(parseTweetEntityAnnotation))
|
||||
.map(thrift.EscherbirdEntityAnnotations(_))
|
||||
|
||||
val mediaEntities = request.media.map(_.mediaEntities)
|
||||
val mediaUploadIds = mediaEntities.map(_.map(_.mediaId)).filter(_.nonEmpty)
|
||||
|
||||
val mediaTags: Option[thrift.TweetMediaTags] = {
|
||||
val mediaTagsAuthorized = !ctx.isContributorRequest
|
||||
|
||||
val tagMap: Map[MediaId, Seq[thrift.MediaTag]] =
|
||||
mediaEntities
|
||||
.getOrElse(Nil)
|
||||
.filter(_ => mediaTagsAuthorized)
|
||||
.filter(_.taggedUsers.nonEmpty)
|
||||
.map(mediaEntity =>
|
||||
mediaEntity.mediaId ->
|
||||
mediaEntity.taggedUsers
|
||||
.map(user_id => thrift.MediaTag(thrift.MediaTagType.User, Some(user_id))))
|
||||
.toMap
|
||||
|
||||
Option(tagMap)
|
||||
.filter(_.nonEmpty)
|
||||
.map(thrift.TweetMediaTags(_))
|
||||
}
|
||||
|
||||
// Can not have both conversation controls and communities defined for a tweet
|
||||
// as they have conflicting permissions on who can reply to the tweet.
|
||||
val communities = parseCommunityIds(escherbirdEntityAnnotations)
|
||||
if (request.conversationControl.isDefined && communities.nonEmpty) {
|
||||
throw CannotConvoControlAndCommunitiesErr
|
||||
}
|
||||
|
||||
// Currently we do not support posting to multiple communities.
|
||||
if (communities.length > 1) {
|
||||
throw TooManyCommunitiesErr
|
||||
}
|
||||
|
||||
// Kill switch for community tweets in case we need to disable them for app security.
|
||||
if (communities.nonEmpty && !enableCommunityTweetCreatesDecider()) {
|
||||
throw CommunityUserNotAuthorizedErr
|
||||
}
|
||||
|
||||
// additionalFields is used to marshal multiple input params and
|
||||
// should only be defined if one or more of those params are defined.
|
||||
val additionalFields: Option[Tweet] =
|
||||
cardReference
|
||||
.orElse(escherbirdEntityAnnotations)
|
||||
.orElse(mediaTags)
|
||||
.map(_ =>
|
||||
thrift.Tweet(
|
||||
0L,
|
||||
cardReference = cardReference,
|
||||
escherbirdEntityAnnotations = escherbirdEntityAnnotations,
|
||||
mediaTags = mediaTags
|
||||
))
|
||||
|
||||
val transientContext: Option[TransientCreateContext] =
|
||||
parseTransientContext(
|
||||
request.batchCompose,
|
||||
request.periscope,
|
||||
ctx.twitterUserId,
|
||||
)
|
||||
|
||||
// PostTweetRequest.additionalContext is marked as deprecated in favor of .transientContext,
|
||||
// but the REST API still supports it and it is still passed along through Tweetypie, and
|
||||
// FanoutService and Notifications still depend on it.
|
||||
val additionalContext: Option[Map[TweetCreateContextKey, String]] =
|
||||
transientContext.map(TransientContextUtil.toAdditionalContext)
|
||||
|
||||
val thriftPostTweetRequest = thrift.PostTweetRequest(
|
||||
userId = ctx.twitterUserId,
|
||||
text = request.tweetText,
|
||||
createdVia = deviceSource,
|
||||
inReplyToTweetId = request.reply.map(_.inReplyToTweetId),
|
||||
geo = request.geo.flatMap(parseTweetCreateGeo),
|
||||
autoPopulateReplyMetadata = request.reply.isDefined,
|
||||
excludeReplyUserIds = request.reply.map(_.excludeReplyUserIds).filter(_.nonEmpty),
|
||||
nullcast = request.nullcast,
|
||||
// Send a dark request to Tweetypie if the dark_request directive is set or
|
||||
// if the Tweet is undo-able.
|
||||
dark = ctx.isDarkRequest || request.undoOptions.exists(_.isUndo),
|
||||
hydrationOptions = Some(HydrationOptions.writePathHydrationOptions(ctx.cardsPlatformKey)),
|
||||
remoteHost = ctx.remoteHost,
|
||||
safetyMetaData = Some(safetyMetadata),
|
||||
attachmentUrl = request.attachmentUrl,
|
||||
mediaUploadIds = mediaUploadIds,
|
||||
mediaMetadata = None,
|
||||
transientContext = transientContext,
|
||||
additionalContext = additionalContext,
|
||||
conversationControl = request.conversationControl.map(parseTweetCreateConversationControl),
|
||||
exclusiveTweetControlOptions = request.exclusiveTweetControlOptions.map { _ =>
|
||||
thrift.ExclusiveTweetControlOptions()
|
||||
},
|
||||
trustedFriendsControlOptions =
|
||||
request.trustedFriendsControlOptions.map(parseTrustedFriendsControlOptions),
|
||||
editOptions = request.editOptions.flatMap(_.previousTweetId.map(thrift.EditOptions(_))),
|
||||
collabControlOptions = request.collabControlOptions.map(parseCollabControlOptions),
|
||||
additionalFields = additionalFields,
|
||||
trackingId = trackingId,
|
||||
noteTweetOptions = request.noteTweetOptions.map(options =>
|
||||
thrift.NoteTweetOptions(
|
||||
options.noteTweetId,
|
||||
options.mentionedScreenNames,
|
||||
options.mentionedUserIds,
|
||||
options.isExpandable))
|
||||
)
|
||||
|
||||
val stitchPostTweet =
|
||||
Stitch.callFuture {
|
||||
TweetyPieDeciderOverrides.ConversationControlUseFeatureSwitchResults.On {
|
||||
postTweet(thriftPostTweetRequest)
|
||||
}
|
||||
}
|
||||
|
||||
for {
|
||||
engagement <- request.engagementRequest
|
||||
if !request.reply.exists(_.inReplyToTweetId == 0) // no op per go/rb/845242
|
||||
engagementType = if (request.reply.isDefined) ReplyEngagement else TweetEngagement
|
||||
} logTweetPromotedContent(engagement, engagementType, ctx.isDarkRequest)
|
||||
|
||||
stitchPostTweet.flatMap { result: thrift.PostTweetResult =>
|
||||
result.state match {
|
||||
|
||||
case thrift.TweetCreateState.Ok =>
|
||||
val unmentionSuccessCounter = statsReceiver.counter("unmention_info_success")
|
||||
val unmentionFailuresCounter = statsReceiver.counter("unmention_info_failures")
|
||||
val unmentionFailuresScope = statsReceiver.scope("unmention_info_failures")
|
||||
|
||||
val unmentionInfoStitch = result.tweet match {
|
||||
case Some(tweet) =>
|
||||
unmentionInfoRepository(tweet)
|
||||
.onFailure { t =>
|
||||
unmentionFailuresCounter.incr()
|
||||
unmentionFailuresScope.counter(Throwables.mkString(t): _*).incr()
|
||||
}
|
||||
.onSuccess { _ =>
|
||||
unmentionSuccessCounter.incr()
|
||||
}
|
||||
.rescue {
|
||||
case _ =>
|
||||
Stitch.None
|
||||
}
|
||||
case _ =>
|
||||
Stitch.None
|
||||
}
|
||||
|
||||
val vibeSuccessCounter = statsReceiver.counter("vibe_success")
|
||||
val vibeFailuresCounter = statsReceiver.counter("vibe_failures")
|
||||
val vibeFailuresScope = statsReceiver.scope("vibe_failures")
|
||||
|
||||
val vibeStitch = result.tweet match {
|
||||
case Some(tweet) =>
|
||||
vibeRepository(tweet)
|
||||
.onSuccess { _ =>
|
||||
vibeSuccessCounter.incr()
|
||||
}
|
||||
.onFailure { t =>
|
||||
vibeFailuresCounter.incr()
|
||||
vibeFailuresScope.counter(Throwables.mkString(t): _*).incr()
|
||||
}
|
||||
.rescue {
|
||||
case _ =>
|
||||
Stitch.None
|
||||
}
|
||||
case _ =>
|
||||
Stitch.None
|
||||
}
|
||||
|
||||
Stitch
|
||||
.join(unmentionInfoStitch, vibeStitch)
|
||||
.liftToOption()
|
||||
.flatMap { prefetchFields =>
|
||||
val r = PrefetchedDataRequest(
|
||||
tweet = result.tweet.get,
|
||||
sourceTweet = result.sourceTweet,
|
||||
quotedTweet = result.quotedTweet,
|
||||
safetyLevel = safetyLevel,
|
||||
unmentionInfo = prefetchFields.flatMap(params => params._1),
|
||||
vibe = prefetchFields.flatMap(params => params._2),
|
||||
requestContext = getWeaverbirdCtx()
|
||||
)
|
||||
|
||||
prefetchedDataRepository(r)
|
||||
.liftToOption()
|
||||
.map((prefetchedData: Option[PrefetchedDataResponse]) => {
|
||||
gql.CreateTweetResponseWithSubqueryPrefetchItems(
|
||||
data = Some(gql.CreateTweetResponse(result.tweet.map(_.id))),
|
||||
subqueryPrefetchItems = prefetchedData.map(_.value)
|
||||
)
|
||||
})
|
||||
}
|
||||
|
||||
case errState =>
|
||||
throw toCreateTweetErr(errState, result.bounce, result.failureReason)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private[this] def parseTweetCreateGeo(gqlGeo: gql.TweetGeo): Option[thrift.TweetCreateGeo] = {
|
||||
val coordinates: Option[thrift.GeoCoordinates] =
|
||||
gqlGeo.coordinates.map { coords =>
|
||||
LatitudeLongitude.of(coords.latitude, coords.longitude) match {
|
||||
case Return(latlon: LatitudeLongitude) =>
|
||||
thrift.GeoCoordinates(
|
||||
latitude = latlon.latitudeDegrees,
|
||||
longitude = latlon.longitudeDegrees,
|
||||
geoPrecision = latlon.precision,
|
||||
display = coords.displayCoordinates
|
||||
)
|
||||
case Throw(_) =>
|
||||
throw InvalidCoordinatesErr
|
||||
}
|
||||
}
|
||||
|
||||
val geoSearchRequestId = gqlGeo.geoSearchRequestId.map { id =>
|
||||
if (id.isEmpty) {
|
||||
throw InvalidGeoSearchRequestIdErr
|
||||
}
|
||||
thrift.TweetGeoSearchRequestID(id)
|
||||
}
|
||||
|
||||
if (coordinates.isEmpty && gqlGeo.placeId.isEmpty) {
|
||||
None
|
||||
} else {
|
||||
Some(
|
||||
thrift.TweetCreateGeo(
|
||||
coordinates = coordinates,
|
||||
placeId = gqlGeo.placeId,
|
||||
geoSearchRequestId = geoSearchRequestId
|
||||
))
|
||||
}
|
||||
}
|
||||
|
||||
private[this] def parseTweetCreateConversationControl(
|
||||
gqlCC: gql.TweetConversationControl
|
||||
): thrift.TweetCreateConversationControl =
|
||||
gqlCC.mode match {
|
||||
case gql.ConversationControlMode.ByInvitation =>
|
||||
ConversationControls.Create.byInvitation()
|
||||
case gql.ConversationControlMode.Community =>
|
||||
ConversationControls.Create.community()
|
||||
case gql.ConversationControlMode.EnumUnknownConversationControlMode(_) =>
|
||||
throw ConversationControlNotSupportedErr
|
||||
}
|
||||
|
||||
private[this] def parseTweetEntityAnnotation(
|
||||
gqlTweetAnnotation: gql.TweetAnnotation
|
||||
): TweetEntityAnnotation =
|
||||
TweetEntityAnnotation(
|
||||
gqlTweetAnnotation.groupId,
|
||||
gqlTweetAnnotation.domainId,
|
||||
gqlTweetAnnotation.entityId
|
||||
)
|
||||
|
||||
private[this] def parseCommunityIds(
|
||||
escherbirdAnnotations: Option[thrift.EscherbirdEntityAnnotations]
|
||||
): Seq[Long] =
|
||||
escherbirdAnnotations
|
||||
.map(_.entityAnnotations).getOrElse(Nil)
|
||||
.flatMap {
|
||||
case CommunityAnnotation(id) => Seq(id)
|
||||
case _ => Nil
|
||||
}
|
||||
|
||||
private[this] def parseBatchMode(
|
||||
gqlBatchComposeMode: gql.BatchComposeMode
|
||||
): thrift.BatchComposeMode = {
|
||||
|
||||
gqlBatchComposeMode match {
|
||||
case gql.BatchComposeMode.BatchFirst =>
|
||||
thrift.BatchComposeMode.BatchFirst
|
||||
case gql.BatchComposeMode.BatchSubsequent =>
|
||||
thrift.BatchComposeMode.BatchSubsequent
|
||||
case gql.BatchComposeMode.EnumUnknownBatchComposeMode(_) =>
|
||||
throw InvalidBatchModeParameterErr
|
||||
}
|
||||
}
|
||||
|
||||
private[this] def parseTransientContext(
|
||||
gqlBatchComposeMode: Option[gql.BatchComposeMode],
|
||||
gqlPeriscope: Option[gql.TweetPeriscopeContext],
|
||||
twitterUserId: UserId,
|
||||
): Option[TransientCreateContext] = {
|
||||
val batchComposeMode = gqlBatchComposeMode.map(parseBatchMode)
|
||||
|
||||
// Per c.t.fanoutservice.model.Tweet#deviceFollowType, isLive=None and Some(false) are
|
||||
// equivalent and the creatorId is discarded in both cases.
|
||||
val periscopeIsLive = gqlPeriscope.map(_.isLive).filter(_ == true)
|
||||
val periscopeCreatorId = if (periscopeIsLive.isDefined) Some(twitterUserId) else None
|
||||
|
||||
if (batchComposeMode.isDefined || periscopeIsLive.isDefined) {
|
||||
Some(
|
||||
thrift.TransientCreateContext(
|
||||
batchCompose = batchComposeMode,
|
||||
periscopeIsLive = periscopeIsLive,
|
||||
periscopeCreatorId = periscopeCreatorId
|
||||
)
|
||||
)
|
||||
} else {
|
||||
None
|
||||
}
|
||||
}
|
||||
|
||||
private[this] def parseTrustedFriendsControlOptions(
|
||||
gqlTrustedFriendsControlOptions: gql.TrustedFriendsControlOptions
|
||||
): thrift.TrustedFriendsControlOptions = {
|
||||
thrift.TrustedFriendsControlOptions(
|
||||
trustedFriendsListId = gqlTrustedFriendsControlOptions.trustedFriendsListId
|
||||
)
|
||||
}
|
||||
|
||||
private[this] def parseCollabControlOptions(
|
||||
gqlCollabControlOptions: gql.CollabControlOptions
|
||||
): thrift.CollabControlOptions = {
|
||||
gqlCollabControlOptions.collabControlType match {
|
||||
case gql.CollabControlType.CollabInvitation =>
|
||||
thrift.CollabControlOptions.CollabInvitation(
|
||||
thrift.CollabInvitationOptions(
|
||||
collaboratorUserIds = gqlCollabControlOptions.collaboratorUserIds
|
||||
)
|
||||
)
|
||||
case gql.CollabControlType.EnumUnknownCollabControlType(_) =>
|
||||
throw CollabTweetInvalidParamsErr
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object CreateTweetColumn {
|
||||
val Path = "tweetypie/createTweet.Tweet"
|
||||
|
||||
def toCreateTweetErr(
|
||||
errState: thrift.TweetCreateState,
|
||||
bounce: Option[BouncerBounce],
|
||||
failureReason: Option[String]
|
||||
): Err = errState match {
|
||||
case TextCannotBeBlank =>
|
||||
TweetCannotBeBlankErr
|
||||
case TextTooLong =>
|
||||
TweetTextTooLongErr
|
||||
case Duplicate =>
|
||||
DuplicateStatusErr
|
||||
case MalwareUrl =>
|
||||
MalwareTweetErr
|
||||
case UserDeactivated | UserSuspended =>
|
||||
// should not occur since this condition is caught by access policy filters
|
||||
CurrentUserSuspendedErr
|
||||
case RateLimitExceeded =>
|
||||
RateLimitExceededErr
|
||||
case UrlSpam =>
|
||||
TweetUrlSpamErr
|
||||
case Spam | UserReadonly =>
|
||||
TweetSpammerErr
|
||||
case SpamCaptcha =>
|
||||
CaptchaChallengeErr
|
||||
case SafetyRateLimitExceeded =>
|
||||
SafetyRateLimitExceededErr
|
||||
case Bounce if bounce.isDefined =>
|
||||
accessDeniedByBouncerErr(bounce.get)
|
||||
case MentionLimitExceeded =>
|
||||
MentionLimitExceededErr
|
||||
case UrlLimitExceeded =>
|
||||
UrlLimitExceededErr
|
||||
case HashtagLimitExceeded =>
|
||||
HashtagLimitExceededErr
|
||||
case CashtagLimitExceeded =>
|
||||
CashtagLimitExceededErr
|
||||
case HashtagLengthLimitExceeded =>
|
||||
HashtagLengthLimitExceededErr
|
||||
case TooManyAttachmentTypes =>
|
||||
TooManyAttachmentTypesErr
|
||||
case InvalidUrl =>
|
||||
InvalidUrlErr
|
||||
case DisabledByIpiPolicy =>
|
||||
failureReason
|
||||
.map(tweetEngagementLimitedErr)
|
||||
.getOrElse(GenericTweetCreateErr)
|
||||
case InvalidAdditionalField =>
|
||||
failureReason
|
||||
.map(invalidAdditionalFieldWithReasonErr)
|
||||
.getOrElse(InvalidAdditionalFieldErr)
|
||||
// InvalidImage has been deprecated by tweetypie. Use InvalidMedia instead.
|
||||
case InvalidMedia | InvalidImage | MediaNotFound =>
|
||||
invalidMediaErr(failureReason)
|
||||
case InReplyToTweetNotFound =>
|
||||
InReplyToTweetNotFoundErr
|
||||
case InvalidAttachmentUrl =>
|
||||
InvalidAttachmentUrlErr
|
||||
case ConversationControlNotAllowed =>
|
||||
ConversationControlNotAuthorizedErr
|
||||
case InvalidConversationControl =>
|
||||
ConversationControlInvalidErr
|
||||
case ReplyTweetNotAllowed =>
|
||||
ConversationControlReplyRestricted
|
||||
case ExclusiveTweetEngagementNotAllowed =>
|
||||
ExclusiveTweetEngagementNotAllowedErr
|
||||
case CommunityReplyTweetNotAllowed =>
|
||||
CommunityReplyTweetNotAllowedErr
|
||||
case CommunityUserNotAuthorized =>
|
||||
CommunityUserNotAuthorizedErr
|
||||
case CommunityNotFound =>
|
||||
CommunityNotFoundErr
|
||||
case SuperFollowsInvalidParams =>
|
||||
SuperFollowInvalidParamsErr
|
||||
case SuperFollowsCreateNotAuthorized =>
|
||||
SuperFollowCreateNotAuthorizedErr
|
||||
case CommunityProtectedUserCannotTweet =>
|
||||
CommunityProtectedUserCannotTweetErr
|
||||
case TrustedFriendsInvalidParams =>
|
||||
TrustedFriendsInvalidParamsErr
|
||||
case TrustedFriendsEngagementNotAllowed =>
|
||||
TrustedFriendsEngagementNotAllowedErr
|
||||
case TrustedFriendsCreateNotAllowed =>
|
||||
TrustedFriendsCreateNotAllowedErr
|
||||
case TrustedFriendsQuoteTweetNotAllowed =>
|
||||
TrustedFriendsQuoteTweetNotAllowedErr
|
||||
case CollabTweetInvalidParams =>
|
||||
CollabTweetInvalidParamsErr
|
||||
case StaleTweetEngagementNotAllowed =>
|
||||
StaleTweetEngagementNotAllowedErr
|
||||
case StaleTweetQuoteTweetNotAllowed =>
|
||||
StaleTweetQuoteTweetNotAllowedErr
|
||||
case FieldEditNotAllowed =>
|
||||
FieldEditNotAllowedErr
|
||||
case NotEligibleForEdit =>
|
||||
NotEligibleForEditErr
|
||||
case _ =>
|
||||
GenericTweetCreateErr
|
||||
}
|
||||
}
|
@ -0,0 +1,81 @@
|
||||
package com.twitter.tweetypie.federated.columns
|
||||
|
||||
import com.twitter.stitch.Stitch
|
||||
import com.twitter.strato.catalog.OpMetadata
|
||||
import com.twitter.strato.config.ContactInfo
|
||||
import com.twitter.strato.config.Policy
|
||||
import com.twitter.strato.data.Conv
|
||||
import com.twitter.strato.data.Description.PlainText
|
||||
import com.twitter.strato.data.Lifecycle.Production
|
||||
import com.twitter.strato.fed.StratoFed
|
||||
import com.twitter.strato.opcontext.OpContext
|
||||
import com.twitter.strato.thrift.ScroogeConv
|
||||
import com.twitter.tweetypie.federated.context.GetRequestContext
|
||||
import com.twitter.tweetypie.federated.prefetcheddata.PrefetchedDataResponse
|
||||
import com.twitter.tweetypie.thriftscala.TweetDeleteState
|
||||
import com.twitter.tweetypie.thriftscala.{graphql => gql}
|
||||
import com.twitter.tweetypie.{thriftscala => thrift}
|
||||
import com.twitter.util.Future
|
||||
|
||||
class DeleteTweetColumn(
|
||||
deleteTweet: thrift.DeleteTweetsRequest => Future[Seq[thrift.DeleteTweetResult]],
|
||||
getRequestContext: GetRequestContext,
|
||||
) extends StratoFed.Column(DeleteTweetColumn.Path)
|
||||
with StratoFed.Execute.StitchWithContext
|
||||
with StratoFed.HandleDarkRequests {
|
||||
|
||||
override val policy: Policy = AccessPolicy.TweetMutationCommonAccessPolicies
|
||||
|
||||
override val isIdempotent: Boolean = true
|
||||
|
||||
override type Arg = gql.DeleteTweetRequest
|
||||
override type Result = gql.DeleteTweetResponseWithSubqueryPrefetchItems
|
||||
|
||||
override val argConv: Conv[Arg] = ScroogeConv.fromStruct
|
||||
override val resultConv: Conv[Result] = ScroogeConv.fromStruct
|
||||
|
||||
override val contactInfo: ContactInfo = TweetypieContactInfo
|
||||
override val metadata: OpMetadata =
|
||||
OpMetadata(Some(Production), Some(PlainText("Deletes a tweet by the calling Twitter user.")))
|
||||
|
||||
override def execute(request: Arg, opContext: OpContext): Stitch[Result] = {
|
||||
val ctx = getRequestContext(opContext)
|
||||
|
||||
val thriftDeleteTweetRequest = thrift.DeleteTweetsRequest(
|
||||
tweetIds = Seq(request.tweetId),
|
||||
// byUserId is picked up by the context in tweetypie.deleteTweet,
|
||||
// but we're passing it in here to be explicit
|
||||
byUserId = Some(ctx.twitterUserId),
|
||||
)
|
||||
|
||||
val stitchDeleteTweet = handleDarkRequest(opContext)(
|
||||
light = {
|
||||
Stitch.callFuture(deleteTweet(thriftDeleteTweetRequest))
|
||||
},
|
||||
// For dark requests, we don't want to send traffic to tweetypie.
|
||||
// Since the response is the same regardless of the request, we take a no-op
|
||||
// action instead.
|
||||
dark = Stitch.value(Seq(thrift.DeleteTweetResult(request.tweetId, TweetDeleteState.Ok)))
|
||||
)
|
||||
|
||||
stitchDeleteTweet.map { result: Seq[thrift.DeleteTweetResult] =>
|
||||
result.headOption match {
|
||||
case Some(thrift.DeleteTweetResult(id, TweetDeleteState.Ok)) =>
|
||||
gql.DeleteTweetResponseWithSubqueryPrefetchItems(
|
||||
data = Some(gql.DeleteTweetResponse(Some(id))),
|
||||
// Prefetch data is always NotFound to prevent subqueries from hydrating via weaverbird
|
||||
// and possibly returning inconsistent results, i.e. a Found tweet.
|
||||
subqueryPrefetchItems = Some(PrefetchedDataResponse.notFound(id).value)
|
||||
)
|
||||
case Some(thrift.DeleteTweetResult(_, TweetDeleteState.PermissionError)) =>
|
||||
throw ApiErrors.DeletePermissionErr
|
||||
case _ =>
|
||||
throw ApiErrors.GenericAccessDeniedErr
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object DeleteTweetColumn {
|
||||
val Path = "tweetypie/deleteTweet.Tweet"
|
||||
}
|
@ -0,0 +1,141 @@
|
||||
package com.twitter.tweetypie
|
||||
package federated.columns
|
||||
|
||||
import com.twitter.io.Buf
|
||||
import com.twitter.scrooge.TFieldBlob
|
||||
import com.twitter.stitch.Stitch
|
||||
import com.twitter.strato.access.Access
|
||||
import com.twitter.strato.catalog.OpMetadata
|
||||
import com.twitter.strato.config.AllowAll
|
||||
import com.twitter.strato.config.ContactInfo
|
||||
import com.twitter.strato.config.Policy
|
||||
import com.twitter.strato.data.Conv
|
||||
import com.twitter.strato.data.Description.PlainText
|
||||
import com.twitter.strato.data.Lifecycle.Production
|
||||
import com.twitter.strato.data.Type
|
||||
import com.twitter.strato.data.Val
|
||||
import com.twitter.strato.fed.StratoFed
|
||||
import com.twitter.strato.opcontext.OpContext
|
||||
import com.twitter.strato.serialization.MVal
|
||||
import com.twitter.strato.serialization.Thrift
|
||||
import com.twitter.strato.util.Strings
|
||||
import com.twitter.tweetypie.thriftscala.GetTweetFieldsResult
|
||||
import com.twitter.tweetypie.thriftscala.SetAdditionalFieldsRequest
|
||||
import com.twitter.tweetypie.thriftscala.Tweet
|
||||
import com.twitter.tweetypie.thriftscala.TweetFieldsResultState.Found
|
||||
import com.twitter.util.Future
|
||||
import org.apache.thrift.protocol.TField
|
||||
|
||||
/**
|
||||
* Federated strato column to return tweet fields
|
||||
* @param federatedFieldsGroup Group to be used for Stitch batching.
|
||||
* This is a function that takes a GroupOptions and returns a FederatedFieldGroup.
|
||||
* Using a function that accepts a GroupOptions allows for Stitch to handle a new group for distinct GroupOptions.
|
||||
* @param setAdditionalFields Handler to set additional fields on tweets.
|
||||
* @param stratoValueType Type to be returned by the strato column.
|
||||
* @param tfield Tweet thrift field to be stored
|
||||
* @param pathName Path to be used in the strato catalog
|
||||
*/
|
||||
class FederatedFieldColumn(
|
||||
federatedFieldsGroup: FederatedFieldGroupBuilder.Type,
|
||||
setAdditionalFields: SetAdditionalFieldsRequest => Future[Unit],
|
||||
stratoValueType: Type,
|
||||
tfield: TField,
|
||||
pathOverride: Option[String] = None)
|
||||
extends StratoFed.Column(pathOverride.getOrElse(FederatedFieldColumn.makeColumnPath(tfield)))
|
||||
with StratoFed.Fetch.StitchWithContext
|
||||
with StratoFed.Put.Stitch {
|
||||
|
||||
type Key = Long
|
||||
type View = Unit
|
||||
type Value = Val.T
|
||||
|
||||
override val keyConv: Conv[Key] = Conv.ofType
|
||||
override val viewConv: Conv[View] = Conv.ofType
|
||||
override val valueConv: Conv[Value] = Conv(stratoValueType, identity, identity)
|
||||
|
||||
override val policy: Policy = AllowAll
|
||||
|
||||
/*
|
||||
* A fetch that proxies GetTweetFieldsColumn.fetch but only requests and
|
||||
* returns one specific field.
|
||||
*/
|
||||
override def fetch(tweetId: Key, view: View, opContext: OpContext): Stitch[Result[Value]] = {
|
||||
|
||||
val twitterUserId: Option[UserId] = Access.getTwitterUserId match {
|
||||
// Access.getTwitterUserId should return a value when request is made on behalf of a user
|
||||
// and will not return a value otherwise
|
||||
case Some(twitterUser) => Some(twitterUser.id)
|
||||
case None => None
|
||||
}
|
||||
|
||||
val stitchGroup = federatedFieldsGroup(GroupOptions(twitterUserId))
|
||||
|
||||
Stitch
|
||||
.call(FederatedFieldReq(tweetId, tfield.id), stitchGroup).map {
|
||||
result: GetTweetFieldsResult =>
|
||||
result.tweetResult match {
|
||||
case Found(f) =>
|
||||
f.tweet.getFieldBlob(tfield.id) match {
|
||||
case Some(v: TFieldBlob) =>
|
||||
found(blobToVal(v))
|
||||
case None => missing
|
||||
}
|
||||
case _ => missing
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/*
|
||||
* A strato put interface for writing a single additional field to a tweet
|
||||
*/
|
||||
override def put(tweetId: Key, value: Val.T): Stitch[Unit] = {
|
||||
val tweet: Tweet = Tweet(id = tweetId).setField(valToBlob(value))
|
||||
val request: SetAdditionalFieldsRequest = SetAdditionalFieldsRequest(tweet)
|
||||
Stitch.callFuture(setAdditionalFields(request))
|
||||
}
|
||||
|
||||
val mval: Thrift.Codec = MVal.codec(stratoValueType).thrift(4)
|
||||
|
||||
def valToBlob(value: Val.T): TFieldBlob =
|
||||
TFieldBlob(tfield, mval.write[Buf](value, Thrift.compactProto))
|
||||
|
||||
def blobToVal(thriftFieldBlob: TFieldBlob): Val.T =
|
||||
mval.read(thriftFieldBlob.content, Thrift.compactProto)
|
||||
|
||||
override val contactInfo: ContactInfo = TweetypieContactInfo
|
||||
override val metadata: OpMetadata = OpMetadata(
|
||||
lifecycle = Some(Production),
|
||||
description = Some(PlainText(s"A federated column for the field tweet.$stratoValueType"))
|
||||
)
|
||||
}
|
||||
|
||||
object FederatedFieldColumn {
|
||||
val idAllowlist: Seq[Short] = Seq(
|
||||
Tweet.CoreDataField.id,
|
||||
Tweet.LanguageField.id,
|
||||
Tweet.ConversationMutedField.id
|
||||
)
|
||||
val ID_START = 157
|
||||
val ID_END = 32000
|
||||
|
||||
private val MigrationFields: Seq[Short] = Seq(157)
|
||||
|
||||
def isFederatedField(id: Short) = id >= ID_START && id < ID_END || idAllowlist.contains(id)
|
||||
|
||||
def isMigrationFederatedField(tField: TField): Boolean = MigrationFields.contains(tField.id)
|
||||
|
||||
/* federated field column strato configs must conform to this
|
||||
* path name scheme for tweetypie to pick them up
|
||||
*/
|
||||
def makeColumnPath(tField: TField) = {
|
||||
val columnName = Strings.toCamelCase(tField.name.stripSuffix("id"))
|
||||
s"tweetypie/fields/${columnName}.Tweet"
|
||||
}
|
||||
|
||||
def makeV1ColumnPath(tField: TField): String = {
|
||||
val columnName = Strings.toCamelCase(tField.name.stripSuffix("id"))
|
||||
s"tweetypie/fields/$columnName-V1.Tweet"
|
||||
}
|
||||
}
|
@ -0,0 +1,88 @@
|
||||
package com.twitter.tweetypie.federated.columns
|
||||
|
||||
import com.twitter.spam.rtf.thriftscala.SafetyLevel
|
||||
import com.twitter.stitch.MapGroup
|
||||
import com.twitter.tweetypie.UserId
|
||||
import com.twitter.tweetypie.federated.columns.FederatedFieldGroupBuilder.allCountFields
|
||||
import com.twitter.tweetypie.federated.columns.FederatedFieldGroupBuilder.countTweetFields
|
||||
import com.twitter.tweetypie.thriftscala.GetTweetFieldsOptions
|
||||
import com.twitter.tweetypie.thriftscala.GetTweetFieldsRequest
|
||||
import com.twitter.tweetypie.thriftscala.GetTweetFieldsResult
|
||||
import com.twitter.tweetypie.thriftscala.StatusCounts
|
||||
import com.twitter.tweetypie.thriftscala.Tweet
|
||||
import com.twitter.tweetypie.thriftscala.TweetInclude
|
||||
import com.twitter.util.Future
|
||||
import com.twitter.util.Throw
|
||||
import com.twitter.util.Try
|
||||
|
||||
case class GroupOptions(twitterUserId: Option[UserId])
|
||||
|
||||
object FederatedFieldGroupBuilder {
|
||||
type Type = GroupOptions => MapGroup[FederatedFieldReq, GetTweetFieldsResult]
|
||||
|
||||
def apply(
|
||||
getTweetFieldsHandler: GetTweetFieldsRequest => Future[Seq[GetTweetFieldsResult]]
|
||||
): Type = {
|
||||
FederatedFieldGroup(getTweetFieldsHandler, _)
|
||||
}
|
||||
|
||||
// The set of non-deprecated count field includes
|
||||
val allCountFields: Set[TweetInclude] = Set(
|
||||
TweetInclude.CountsFieldId(StatusCounts.RetweetCountField.id),
|
||||
TweetInclude.CountsFieldId(StatusCounts.QuoteCountField.id),
|
||||
TweetInclude.CountsFieldId(StatusCounts.FavoriteCountField.id),
|
||||
TweetInclude.CountsFieldId(StatusCounts.ReplyCountField.id),
|
||||
TweetInclude.CountsFieldId(StatusCounts.BookmarkCountField.id),
|
||||
)
|
||||
|
||||
// Tweet field includes which contain counts. These are the only fields where count field includes are relevant.
|
||||
val countTweetFields: Set[TweetInclude] = Set(
|
||||
TweetInclude.TweetFieldId(Tweet.CountsField.id),
|
||||
TweetInclude.TweetFieldId(Tweet.PreviousCountsField.id))
|
||||
}
|
||||
|
||||
case class FederatedFieldGroup(
|
||||
getTweetFieldsHandler: GetTweetFieldsRequest => Future[Seq[GetTweetFieldsResult]],
|
||||
options: GroupOptions)
|
||||
extends MapGroup[FederatedFieldReq, GetTweetFieldsResult] {
|
||||
override protected def run(
|
||||
reqs: Seq[FederatedFieldReq]
|
||||
): Future[FederatedFieldReq => Try[GetTweetFieldsResult]] = {
|
||||
|
||||
// requesting the field ids of the requested additional field ids in this group
|
||||
val fieldIncludes: Set[TweetInclude] = reqs.map { req: FederatedFieldReq =>
|
||||
TweetInclude.TweetFieldId(req.fieldId)
|
||||
}.toSet
|
||||
|
||||
val allIncludes: Set[TweetInclude] = if (fieldIncludes.intersect(countTweetFields).nonEmpty) {
|
||||
// if counts are being requested we include all count fields by default
|
||||
// because there is no way to specify them individually with federated fields,
|
||||
fieldIncludes ++ allCountFields
|
||||
} else {
|
||||
fieldIncludes
|
||||
}
|
||||
|
||||
val gtfOptions = GetTweetFieldsOptions(
|
||||
tweetIncludes = allIncludes,
|
||||
forUserId = options.twitterUserId,
|
||||
// visibility filtering happens at the api layer / tweet top level
|
||||
// and therefore is not required at individual field level
|
||||
safetyLevel = Some(SafetyLevel.FilterNone)
|
||||
)
|
||||
getTweetFieldsHandler(
|
||||
GetTweetFieldsRequest(
|
||||
tweetIds = reqs.map(_.tweetId).distinct,
|
||||
options = gtfOptions
|
||||
)
|
||||
).map {
|
||||
response =>
|
||||
{ req =>
|
||||
response.find(_.tweetId == req.tweetId) match {
|
||||
case Some(result) => Try(result)
|
||||
case None =>
|
||||
Throw(new NoSuchElementException(s"response not found for tweet: ${req.tweetId}"))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,7 @@
|
||||
package com.twitter.tweetypie.federated.columns
|
||||
|
||||
import com.twitter.tweetypie.TweetId
|
||||
|
||||
// Case class to be used for grouping Stitch requests
|
||||
// for Federated Fields
|
||||
case class FederatedFieldReq(tweetId: TweetId, fieldId: Short)
|
@ -0,0 +1,83 @@
|
||||
package com.twitter.tweetypie.federated.columns
|
||||
|
||||
import com.twitter.stitch.Stitch
|
||||
import com.twitter.strato.access.Access.LdapGroup
|
||||
import com.twitter.strato.catalog.Fetch
|
||||
import com.twitter.strato.catalog.OpMetadata
|
||||
import com.twitter.strato.config.AnyOf
|
||||
import com.twitter.strato.config.ContactInfo
|
||||
import com.twitter.strato.config.FromColumns
|
||||
import com.twitter.strato.config.Has
|
||||
import com.twitter.strato.config.Path
|
||||
import com.twitter.strato.config.Policy
|
||||
import com.twitter.strato.data.Conv
|
||||
import com.twitter.strato.data.Description.PlainText
|
||||
import com.twitter.strato.data.Lifecycle.Production
|
||||
import com.twitter.strato.fed.StratoFed
|
||||
import com.twitter.strato.response.Err
|
||||
import com.twitter.strato.thrift.ScroogeConv
|
||||
import com.twitter.tweetypie.UserId
|
||||
import com.twitter.tweetypie.thriftscala.federated.GetStoredTweetsByUserView
|
||||
import com.twitter.tweetypie.thriftscala.federated.GetStoredTweetsByUserResponse
|
||||
import com.twitter.tweetypie.{thriftscala => thrift}
|
||||
import com.twitter.util.Future
|
||||
|
||||
class GetStoredTweetsByUserColumn(
|
||||
handler: thrift.GetStoredTweetsByUserRequest => Future[thrift.GetStoredTweetsByUserResult])
|
||||
extends StratoFed.Column(GetStoredTweetsByUserColumn.Path)
|
||||
with StratoFed.Fetch.Stitch {
|
||||
|
||||
override val contactInfo: ContactInfo = TweetypieContactInfo
|
||||
override val metadata: OpMetadata = OpMetadata(
|
||||
lifecycle = Some(Production),
|
||||
description =
|
||||
Some(PlainText("Fetches hydrated Tweets for a particular User regardless of Tweet state."))
|
||||
)
|
||||
override val policy: Policy = AnyOf(
|
||||
Seq(
|
||||
FromColumns(Set(Path("tweetypie/data-provider/storedTweets.User"))),
|
||||
Has(LdapGroup("tweetypie-team"))
|
||||
))
|
||||
|
||||
override type Key = UserId
|
||||
override type View = GetStoredTweetsByUserView
|
||||
override type Value = GetStoredTweetsByUserResponse
|
||||
|
||||
override val keyConv: Conv[Key] = Conv.ofType
|
||||
override val viewConv: Conv[View] = ScroogeConv.fromStruct[GetStoredTweetsByUserView]
|
||||
override val valueConv: Conv[Value] = ScroogeConv.fromStruct[GetStoredTweetsByUserResponse]
|
||||
|
||||
override def fetch(key: Key, view: View): Stitch[Result[Value]] = {
|
||||
val request = thrift.GetStoredTweetsByUserRequest(
|
||||
userId = key,
|
||||
options = Some(
|
||||
thrift.GetStoredTweetsByUserOptions(
|
||||
bypassVisibilityFiltering = view.bypassVisibilityFiltering,
|
||||
setForUserId = view.setForUserId,
|
||||
startTimeMsec = view.startTimeMsec,
|
||||
endTimeMsec = view.endTimeMsec,
|
||||
cursor = view.cursor,
|
||||
startFromOldest = view.startFromOldest,
|
||||
additionalFieldIds = view.additionalFieldIds
|
||||
))
|
||||
)
|
||||
|
||||
Stitch
|
||||
.callFuture(handler(request))
|
||||
.map { result =>
|
||||
Fetch.Result.found(
|
||||
GetStoredTweetsByUserResponse(
|
||||
storedTweets = result.storedTweets,
|
||||
cursor = result.cursor
|
||||
))
|
||||
}
|
||||
.rescue {
|
||||
case _ => Stitch.exception(Err(Err.Internal))
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
object GetStoredTweetsByUserColumn {
|
||||
val Path = "tweetypie/internal/getStoredTweets.User"
|
||||
}
|
@ -0,0 +1,99 @@
|
||||
package com.twitter.tweetypie.federated.columns
|
||||
|
||||
import com.twitter.stitch.MapGroup
|
||||
import com.twitter.stitch.Stitch
|
||||
import com.twitter.strato.access.Access.LdapGroup
|
||||
import com.twitter.strato.catalog.Fetch
|
||||
import com.twitter.strato.catalog.OpMetadata
|
||||
import com.twitter.strato.config.AnyOf
|
||||
import com.twitter.strato.config.ContactInfo
|
||||
import com.twitter.strato.config.FromColumns
|
||||
import com.twitter.strato.config.Has
|
||||
import com.twitter.strato.config.Path
|
||||
import com.twitter.strato.config.Policy
|
||||
import com.twitter.strato.data.Conv
|
||||
import com.twitter.strato.data.Description.PlainText
|
||||
import com.twitter.strato.data.Lifecycle.Production
|
||||
import com.twitter.strato.fed.StratoFed
|
||||
import com.twitter.strato.response.Err
|
||||
import com.twitter.strato.thrift.ScroogeConv
|
||||
import com.twitter.tweetypie.{thriftscala => thrift}
|
||||
import com.twitter.tweetypie.TweetId
|
||||
import com.twitter.tweetypie.thriftscala.federated.GetStoredTweetsView
|
||||
import com.twitter.tweetypie.thriftscala.federated.GetStoredTweetsResponse
|
||||
import com.twitter.util.Future
|
||||
import com.twitter.util.Return
|
||||
import com.twitter.util.Throw
|
||||
import com.twitter.util.Try
|
||||
|
||||
class GetStoredTweetsColumn(
|
||||
getStoredTweets: thrift.GetStoredTweetsRequest => Future[Seq[thrift.GetStoredTweetsResult]])
|
||||
extends StratoFed.Column(GetStoredTweetsColumn.Path)
|
||||
with StratoFed.Fetch.Stitch {
|
||||
|
||||
override val contactInfo: ContactInfo = TweetypieContactInfo
|
||||
override val metadata: OpMetadata = OpMetadata(
|
||||
lifecycle = Some(Production),
|
||||
description = Some(PlainText("Fetches hydrated Tweets regardless of Tweet state."))
|
||||
)
|
||||
override val policy: Policy = AnyOf(
|
||||
Seq(
|
||||
FromColumns(
|
||||
Set(
|
||||
Path("tweetypie/data-provider/storedTweets.User"),
|
||||
Path("note_tweet/data-provider/noteTweetForZipbird.User"))),
|
||||
Has(LdapGroup("tweetypie-team"))
|
||||
))
|
||||
|
||||
override type Key = TweetId
|
||||
override type View = GetStoredTweetsView
|
||||
override type Value = GetStoredTweetsResponse
|
||||
|
||||
override val keyConv: Conv[Key] = Conv.ofType
|
||||
override val viewConv: Conv[View] = ScroogeConv.fromStruct[GetStoredTweetsView]
|
||||
override val valueConv: Conv[Value] = ScroogeConv.fromStruct[GetStoredTweetsResponse]
|
||||
|
||||
override def fetch(key: Key, view: View): Stitch[Result[Value]] = {
|
||||
Stitch.call(key, Group(view))
|
||||
}
|
||||
|
||||
private case class Group(view: GetStoredTweetsView)
|
||||
extends MapGroup[TweetId, Fetch.Result[GetStoredTweetsResponse]] {
|
||||
override protected def run(
|
||||
keys: Seq[TweetId]
|
||||
): Future[TweetId => Try[Result[GetStoredTweetsResponse]]] = {
|
||||
val options = thrift.GetStoredTweetsOptions(
|
||||
bypassVisibilityFiltering = view.bypassVisibilityFiltering,
|
||||
forUserId = view.forUserId,
|
||||
additionalFieldIds = view.additionalFieldIds
|
||||
)
|
||||
|
||||
getStoredTweets(thrift.GetStoredTweetsRequest(keys, Some(options)))
|
||||
.map(transformAndGroupByTweetId)
|
||||
.handle {
|
||||
case _ =>
|
||||
_ => Throw[Result[GetStoredTweetsResponse]](Err(Err.Internal))
|
||||
}
|
||||
}
|
||||
|
||||
private def transformAndGroupByTweetId(
|
||||
results: Seq[thrift.GetStoredTweetsResult]
|
||||
): Map[TweetId, Try[Fetch.Result[GetStoredTweetsResponse]]] = {
|
||||
results
|
||||
.map(result => GetStoredTweetsResponse(result.storedTweet))
|
||||
.groupBy(_.storedTweet.tweetId)
|
||||
.map {
|
||||
case (tweetId, Seq(result)) => (tweetId, Return(Fetch.Result.found(result)))
|
||||
case (tweetId, multipleResults) =>
|
||||
(
|
||||
tweetId,
|
||||
Throw(Err(Err.BadRequest, s"Got ${multipleResults.size} results for $tweetId")))
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
object GetStoredTweetsColumn {
|
||||
val Path = "tweetypie/internal/getStoredTweets.Tweet"
|
||||
}
|
@ -0,0 +1,172 @@
|
||||
package com.twitter.tweetypie.federated.columns
|
||||
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.stitch.MapGroup
|
||||
import com.twitter.stitch.Stitch
|
||||
import com.twitter.strato.catalog.Fetch
|
||||
import com.twitter.strato.catalog.OpMetadata
|
||||
import com.twitter.strato.config.AllowAll
|
||||
import com.twitter.strato.config.ContactInfo
|
||||
import com.twitter.strato.config.Policy
|
||||
import com.twitter.strato.data.Conv
|
||||
import com.twitter.strato.data.Description.PlainText
|
||||
import com.twitter.strato.data.Lifecycle.Production
|
||||
import com.twitter.strato.fed.StratoFed
|
||||
import com.twitter.strato.opcontext.OpContext
|
||||
import com.twitter.strato.response.Err
|
||||
import com.twitter.strato.thrift.ScroogeConv
|
||||
import com.twitter.tweetypie.TweetId
|
||||
import com.twitter.tweetypie.client_id.PreferForwardedServiceIdentifierForStrato
|
||||
import com.twitter.tweetypie.thriftscala.GetTweetFieldsOptions
|
||||
import com.twitter.tweetypie.thriftscala.GetTweetFieldsRequest
|
||||
import com.twitter.tweetypie.thriftscala.GetTweetFieldsResult
|
||||
import com.twitter.tweetypie.thriftscala.TweetVisibilityPolicy
|
||||
import com.twitter.util.Future
|
||||
import com.twitter.util.Try
|
||||
|
||||
/**
|
||||
* Strato federated column implementing GetTweetFields as a Fetch.
|
||||
*/
|
||||
class GetTweetFieldsColumn(
|
||||
handler: GetTweetFieldsRequest => Future[Seq[GetTweetFieldsResult]],
|
||||
stats: StatsReceiver)
|
||||
extends StratoFed.Column(GetTweetFieldsColumn.Path)
|
||||
with StratoFed.Fetch.StitchWithContext {
|
||||
|
||||
/**
|
||||
* At this point, this fetch op will reject any requests that specify
|
||||
* visibilityPolicy other than USER_VISIBLE, so no access control is needed.
|
||||
*/
|
||||
override val policy: Policy = AllowAll
|
||||
|
||||
override type Key = TweetId
|
||||
override type View = GetTweetFieldsOptions
|
||||
override type Value = GetTweetFieldsResult
|
||||
|
||||
override val keyConv: Conv[Key] = Conv.ofType
|
||||
override val viewConv: Conv[View] = ScroogeConv.fromStruct[GetTweetFieldsOptions]
|
||||
override val valueConv: Conv[Value] = ScroogeConv.fromStruct[GetTweetFieldsResult]
|
||||
|
||||
override val contactInfo: ContactInfo = TweetypieContactInfo
|
||||
override val metadata: OpMetadata = OpMetadata(
|
||||
lifecycle = Some(Production),
|
||||
description =
|
||||
Some(PlainText("Get of tweets that allows fetching only specific subsets of the data.")),
|
||||
)
|
||||
|
||||
val safetyOpContextOnlyCounter = stats.counter("safety_op_context_only")
|
||||
val safetyOpContextOnlyValueScope = stats.scope("safety_op_context_only_value")
|
||||
val safetyOpContextOnlyCallerScope = stats.scope("safety_op_context_only_caller")
|
||||
|
||||
val safetyViewOnlyCounter = stats.counter("safety_view_only")
|
||||
val safetyViewOnlyValueScope = stats.scope("safety_view_only_value")
|
||||
val safetyViewOnlyCallerScope = stats.scope("safety_view_only_caller")
|
||||
|
||||
val safetyLevelInconsistencyCounter = stats.counter("safety_level_inconsistency")
|
||||
val safetyLevelInconsistencyValueScope = stats.scope("safety_level_inconsistency_value")
|
||||
val safetyLevelInconsistencyCallerScope = stats.scope("safety_level_inconsistency_caller")
|
||||
|
||||
override def fetch(key: Key, view: View, ctx: OpContext): Stitch[Result[Value]] = {
|
||||
compareSafetyLevel(view, ctx)
|
||||
checkVisibilityPolicyUserVisible(view).flatMap { _ =>
|
||||
Stitch.call(key, Group(view))
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Only allow [[TweetVisibilityPolicy.UserVisible]] visibilityPolicy.
|
||||
*
|
||||
* This column requires access policy in order to serve requests with visibilityPolicy
|
||||
* other than [[TweetVisibilityPolicy.UserVisible]]. Before we support access control,
|
||||
* reject all requests that are not safe.
|
||||
*/
|
||||
private def checkVisibilityPolicyUserVisible(view: View): Stitch[Unit] =
|
||||
view.visibilityPolicy match {
|
||||
case TweetVisibilityPolicy.UserVisible => Stitch.value(Unit)
|
||||
case otherValue =>
|
||||
Stitch.exception(
|
||||
Err(
|
||||
Err.BadRequest,
|
||||
"GetTweetFields does not support access control on Strato yet. "
|
||||
+ s"Hence visibilityPolicy can only take the default ${TweetVisibilityPolicy.UserVisible} value, "
|
||||
+ s"got: ${otherValue}."
|
||||
))
|
||||
}
|
||||
|
||||
/** Compare the SafetyLevels in the View and OpContext */
|
||||
private def compareSafetyLevel(view: View, ctx: OpContext): Unit =
|
||||
(view.safetyLevel, ctx.safetyLevel) match {
|
||||
case (None, None) =>
|
||||
case (Some(viewSafety), None) => {
|
||||
safetyViewOnlyCounter.incr()
|
||||
safetyViewOnlyValueScope.counter(viewSafety.name).incr()
|
||||
PreferForwardedServiceIdentifierForStrato.serviceIdentifier
|
||||
.foreach(serviceId => safetyViewOnlyCallerScope.counter(serviceId.toString).incr())
|
||||
}
|
||||
case (None, Some(ctxSafety)) => {
|
||||
safetyOpContextOnlyCounter.incr()
|
||||
safetyOpContextOnlyValueScope.counter(ctxSafety.name).incr()
|
||||
PreferForwardedServiceIdentifierForStrato.serviceIdentifier
|
||||
.foreach(serviceId => safetyOpContextOnlyCallerScope.counter(serviceId.toString).incr())
|
||||
}
|
||||
case (Some(viewSafety), Some(ctxSafety)) =>
|
||||
def safeStringEquals(a: String, b: String) =
|
||||
a.toLowerCase().trim().equals(b.toLowerCase().trim())
|
||||
if (!safeStringEquals(viewSafety.name, ctxSafety.name)) {
|
||||
safetyLevelInconsistencyCounter.incr()
|
||||
safetyLevelInconsistencyValueScope.counter(viewSafety.name + '-' + ctxSafety.name).incr()
|
||||
PreferForwardedServiceIdentifierForStrato.serviceIdentifier
|
||||
.foreach(serviceId =>
|
||||
safetyLevelInconsistencyCallerScope.counter(serviceId.toString).incr())
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Means of batching of [[GetTweetFieldsColumn]] calls.
|
||||
*
|
||||
* Only calls issued against the same instance of [[GetTweetFieldsColumn]]
|
||||
* are batched as Stitch clusters group objects based on equality,
|
||||
* and nested case class implicitly captures [[GetTweetFieldsColumn]] reference.
|
||||
*/
|
||||
private case class Group(view: GetTweetFieldsOptions)
|
||||
extends MapGroup[TweetId, Fetch.Result[GetTweetFieldsResult]] {
|
||||
|
||||
/**
|
||||
* Batches given [[TweetId]] lookups in a single [[GetTweetFieldsRequest]]
|
||||
* and returns a result mapped by [[TweetId]].
|
||||
*/
|
||||
override protected def run(
|
||||
keys: Seq[TweetId]
|
||||
): Future[TweetId => Try[Fetch.Result[GetTweetFieldsResult]]] =
|
||||
handler(
|
||||
GetTweetFieldsRequest(
|
||||
// Sorting the keys makes for simpler matchers in the tests
|
||||
// as matching on a Seq needs to be in order.
|
||||
tweetIds = keys.sorted,
|
||||
options = view,
|
||||
)).map(groupByTweetId)
|
||||
|
||||
/**
|
||||
* Groups given [[GetTweetFieldsResult]] objects by [[TweetId]] and returns the mapping.
|
||||
*/
|
||||
private def groupByTweetId(
|
||||
allResults: Seq[GetTweetFieldsResult]
|
||||
): TweetId => Try[Fetch.Result[GetTweetFieldsResult]] = {
|
||||
allResults
|
||||
.groupBy(_.tweetId)
|
||||
.mapValues {
|
||||
case Seq(result) => Try(Fetch.Result.found(result))
|
||||
case manyResults =>
|
||||
Try {
|
||||
throw Err(
|
||||
Err.Dependency,
|
||||
s"Expected one result per tweeet ID, got ${manyResults.length}")
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object GetTweetFieldsColumn {
|
||||
val Path = "tweetypie/getTweetFields.Tweet"
|
||||
}
|
@ -0,0 +1,22 @@
|
||||
package com.twitter.tweetypie.federated.columns
|
||||
|
||||
import com.twitter.tweetypie.{thriftscala => thrift}
|
||||
|
||||
object HydrationOptions {
|
||||
|
||||
def writePathHydrationOptions(
|
||||
cardsPlatformKey: Option[String]
|
||||
) =
|
||||
thrift.WritePathHydrationOptions(
|
||||
// The GraphQL API extracts or "lifts" the ApiTweet.card reference field from the
|
||||
// ApiTweet.card.url returned by Tweetypie. Tweetypie's card hydration business logic
|
||||
// selects the single correct Card URL by first making Expandodo.getCards2 requests for
|
||||
// the Tweet's cardReference, or all of the Tweet's URL entities in cases where Tweet
|
||||
// does not have a stored cardReference, and then selecting the last of the hydrated
|
||||
// cards returned by Expandodo.
|
||||
includeCards = true,
|
||||
cardsPlatformKey = cardsPlatformKey,
|
||||
// The GraphQL API only supports quoted tweet results formatted per go/simplequotedtweet.
|
||||
simpleQuotedTweet = true,
|
||||
)
|
||||
}
|
@ -0,0 +1,29 @@
|
||||
package com.twitter.tweetypie.federated
|
||||
package columns
|
||||
|
||||
import com.twitter.finagle.stats.NullStatsReceiver
|
||||
import com.twitter.tweetypie.StatsReceiver
|
||||
import com.twitter.util.logging.Logger
|
||||
|
||||
object TrackingId {
|
||||
private[this] val log = Logger(getClass)
|
||||
|
||||
def parse(s: String, statsReceiver: StatsReceiver = NullStatsReceiver): Option[Long] = {
|
||||
val trackingStats = statsReceiver.scope("tracking_id_parser")
|
||||
|
||||
val parsedCountCounter = trackingStats.scope("parsed").counter("count")
|
||||
val parseFailedCounter = trackingStats.scope("parse_failed").counter("count")
|
||||
Option(s).map(_.trim).filter(_.nonEmpty).flatMap { idStr =>
|
||||
try {
|
||||
val id = java.lang.Long.parseLong(idStr, 16)
|
||||
parsedCountCounter.incr()
|
||||
Some(id)
|
||||
} catch {
|
||||
case _: NumberFormatException =>
|
||||
parseFailedCounter.incr()
|
||||
log.warn(s"invalid tracking ID: '$s'")
|
||||
None
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,11 @@
|
||||
package com.twitter.tweetypie.federated.columns
|
||||
|
||||
import com.twitter.strato.config.ContactInfo
|
||||
|
||||
object TweetypieContactInfo
|
||||
extends ContactInfo(
|
||||
contactEmail = "",
|
||||
ldapGroup = "",
|
||||
jiraProject = "",
|
||||
slackRoomId = ""
|
||||
)
|
@ -0,0 +1,69 @@
|
||||
package com.twitter.tweetypie
|
||||
package federated.columns
|
||||
|
||||
import com.twitter.stitch.Stitch
|
||||
import com.twitter.strato.catalog.OpMetadata
|
||||
import com.twitter.strato.config.ContactInfo
|
||||
import com.twitter.strato.config.Policy
|
||||
import com.twitter.strato.data.Conv
|
||||
import com.twitter.strato.data.Description.PlainText
|
||||
import com.twitter.strato.data.Lifecycle.Production
|
||||
import com.twitter.strato.fed.StratoFed
|
||||
import com.twitter.strato.opcontext.OpContext
|
||||
import com.twitter.strato.thrift.ScroogeConv
|
||||
import com.twitter.tweetypie.federated.context.GetRequestContext
|
||||
import com.twitter.tweetypie.federated.context.RequestContext
|
||||
import com.twitter.tweetypie.thriftscala.{graphql => gql}
|
||||
import com.twitter.tweetypie.{thriftscala => thrift}
|
||||
|
||||
class UnretweetColumn(
|
||||
unretweet: thrift.UnretweetRequest => Future[thrift.UnretweetResult],
|
||||
getRequestContext: GetRequestContext,
|
||||
) extends StratoFed.Column("tweetypie/unretweet.Tweet")
|
||||
with StratoFed.Execute.StitchWithContext
|
||||
with StratoFed.HandleDarkRequests {
|
||||
|
||||
override val policy: Policy = AccessPolicy.TweetMutationCommonAccessPolicies
|
||||
|
||||
// It's acceptable to retry or reapply an unretweet operation,
|
||||
// as multiple calls result in the same end state.
|
||||
override val isIdempotent: Boolean = true
|
||||
|
||||
override type Arg = gql.UnretweetRequest
|
||||
override type Result = gql.UnretweetResponseWithSubqueryPrefetchItems
|
||||
|
||||
override val argConv: Conv[Arg] = ScroogeConv.fromStruct
|
||||
override val resultConv: Conv[Result] = ScroogeConv.fromStruct
|
||||
|
||||
override val contactInfo: ContactInfo = TweetypieContactInfo
|
||||
override val metadata: OpMetadata =
|
||||
OpMetadata(
|
||||
Some(Production),
|
||||
Some(PlainText("Removes any retweets by the calling user of the given source tweet.")))
|
||||
|
||||
override def execute(gqlRequest: Arg, opContext: OpContext): Stitch[Result] = {
|
||||
val ctx: RequestContext = getRequestContext(opContext)
|
||||
val req = thrift.UnretweetRequest(
|
||||
ctx.twitterUserId,
|
||||
gqlRequest.sourceTweetId,
|
||||
)
|
||||
|
||||
val stitchUnretweet = handleDarkRequest(opContext)(
|
||||
light = Stitch.callFuture(unretweet(req)),
|
||||
// For dark requests, we don't want to send traffic to tweetypie.
|
||||
// Since the response is the same regardless of the request, we take a no-op
|
||||
// action instead.
|
||||
dark = Stitch.value(thrift.UnretweetResult(state = thrift.TweetDeleteState.Ok))
|
||||
)
|
||||
|
||||
stitchUnretweet.map { _ =>
|
||||
gql.UnretweetResponseWithSubqueryPrefetchItems(
|
||||
data = Some(gql.UnretweetResponse(Some(gqlRequest.sourceTweetId)))
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object UnretweetColumn {
|
||||
val Path = "tweetypie/unretweet.Tweet"
|
||||
}
|
@ -0,0 +1,27 @@
|
||||
scala_library(
|
||||
sources = ["*.scala"],
|
||||
compiler_option_sets = ["fatal_warnings"],
|
||||
strict_deps = True,
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"finagle/finagle-core/src/main",
|
||||
"finatra-internal/tfe",
|
||||
"passbird/bitfields-thrift/src/main/thrift:thrift-scala",
|
||||
"src/scala/com/twitter/common/ip_address_utils",
|
||||
"src/thrift/com/twitter/context:feature-context-scala",
|
||||
"src/thrift/com/twitter/context:twitter-context-scala",
|
||||
"src/thrift/com/twitter/ip_address_utils:ip-address-utils-thrift-scala",
|
||||
"src/thrift/com/twitter/spam/rtf:safety-level-scala",
|
||||
"strato/src/main/scala/com/twitter/strato/access",
|
||||
"strato/src/main/scala/com/twitter/strato/config",
|
||||
"strato/src/main/scala/com/twitter/strato/context",
|
||||
"strato/src/main/scala/com/twitter/strato/data",
|
||||
"strato/src/main/scala/com/twitter/strato/opcontext",
|
||||
"strato/src/main/scala/com/twitter/strato/response",
|
||||
"strato/src/main/scala/com/twitter/strato/thrift",
|
||||
"strato/src/main/thrift/com/twitter/strato/context:thrift-scala",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie",
|
||||
"twitter-context/src/main/scala",
|
||||
"weaverbird/common/src/main/scala/com/twitter/weaverbird/common",
|
||||
],
|
||||
)
|
@ -0,0 +1,131 @@
|
||||
package com.twitter.tweetypie
|
||||
package federated.context
|
||||
|
||||
import com.twitter.common.ip_address_utils.ClientIpAddressUtils
|
||||
import com.twitter.context.thriftscala.Viewer
|
||||
import com.twitter.context.TwitterContext
|
||||
import com.twitter.finagle.core.util.InetAddressUtil
|
||||
import com.twitter.passbird.bitfield.clientprivileges.thriftscala.{Constants => ClientAppPrivileges}
|
||||
import com.twitter.finatra.tfe.HttpHeaderNames
|
||||
import com.twitter.spam.rtf.thriftscala.SafetyLevel
|
||||
import com.twitter.strato.access.Access.ClientApplicationPrivilege
|
||||
import com.twitter.strato.access.Access
|
||||
import com.twitter.strato.access.ClientApplicationPrivilegeVariant
|
||||
import com.twitter.strato.context.StratoContext
|
||||
import com.twitter.strato.opcontext.OpContext
|
||||
import com.twitter.strato.response.Err
|
||||
import com.twitter.weaverbird.common.GetPlatformKey
|
||||
|
||||
/**
|
||||
* [[RequestContext]] exists to avoid wiring the federated column
|
||||
* implementations directly to the request data that is derived from the
|
||||
* contextual environment. Columns should not directly reference
|
||||
* TwitterContext, StratoContext, strato.access.Access, HTTP headers, etc.
|
||||
* Each column operation operates on two input parameters: a request (i.e.
|
||||
* a column operation's Arg) and a [[RequestContext]].
|
||||
*/
|
||||
private[federated] case class RequestContext(
|
||||
clientApplicationId: Option[AppId] = None,
|
||||
deviceSource: Option[String] = None,
|
||||
knownDeviceToken: Option[KnownDeviceToken] = None,
|
||||
remoteHost: Option[String] = None,
|
||||
twitterUserId: UserId,
|
||||
contributorId: Option[UserId] = None,
|
||||
isDarkRequest: Boolean = false,
|
||||
hasPrivilegeNullcastingAccess: Boolean = false,
|
||||
hasPrivilegePromotedTweetsInTimeline: Boolean = false,
|
||||
sessionHash: Option[String] = None,
|
||||
cardsPlatformKey: Option[String] = None,
|
||||
safetyLevel: Option[SafetyLevel] = None,
|
||||
) {
|
||||
def isContributorRequest = contributorId.exists(_ != twitterUserId)
|
||||
}
|
||||
|
||||
/**
|
||||
* Provides a single place to derive request data from the contextual
|
||||
* environment. Defined as a sealed class (vs an object) to allow mocking
|
||||
* in unit tests.
|
||||
*/
|
||||
private[federated] sealed class GetRequestContext() {
|
||||
// Bring Tweetypie permitted TwitterContext into scope
|
||||
private[this] val TwitterContext: TwitterContext =
|
||||
com.twitter.context.TwitterContext(com.twitter.tweetypie.TwitterContextPermit)
|
||||
|
||||
/**
|
||||
* When TwitterUserIdNotDefined is thrown, it's likely that the column
|
||||
* access control configuration lacks `AllowTwitterUserId` or other
|
||||
* Policy that ensures the caller is authenticated.
|
||||
*/
|
||||
private[federated] val TwitterUserIdNotDefined =
|
||||
Err(Err.Authentication, "User authentication is required for this operation.")
|
||||
|
||||
private[this] val SessionHashHeaderName = "x-tfe-session-hash"
|
||||
private[this] def hasClientApplicationPrivilege(id: Int): Boolean =
|
||||
Access.getPrincipals.contains(
|
||||
ClientApplicationPrivilege(
|
||||
ClientApplicationPrivilegeVariant
|
||||
.byId(id.toShort).get))
|
||||
|
||||
private[this] def getRequestHeader(headerName: String): Option[String] =
|
||||
StratoContext
|
||||
.current()
|
||||
.propagatedHeaders
|
||||
.flatMap(_.get(headerName))
|
||||
|
||||
def apply(opContext: OpContext): RequestContext = {
|
||||
val twitterUserId = Access.getTwitterUserId match {
|
||||
// Access.getTwitterUserId should return a value as long as the column
|
||||
// policy includes AllowTwitterUserId, which guarantees the presence of
|
||||
// the value.
|
||||
case Some(twitterUser) => twitterUser.id
|
||||
case None => throw TwitterUserIdNotDefined
|
||||
}
|
||||
|
||||
// contributorId should only be defined when the authenticated user differs
|
||||
// from the "Twitter user"
|
||||
val contributorId =
|
||||
Access.getAuthenticatedTwitterUserId.map(_.id).filter(_ != twitterUserId)
|
||||
|
||||
val twitterContext = TwitterContext().getOrElse(Viewer())
|
||||
|
||||
val deviceSource = twitterContext.clientApplicationId.map("oauth:" + _)
|
||||
|
||||
// Ported from StatusesUpdateController#getBirdherdOptions and
|
||||
// BirdherdOption.UserIp(request.clientHost)
|
||||
val remoteHost: Option[String] =
|
||||
getRequestHeader(HttpHeaderNames.X_TWITTER_AUDIT_IP_THRIFT.toLowerCase) // use the new header
|
||||
.flatMap(ClientIpAddressUtils.decodeClientIpAddress(_))
|
||||
.flatMap(ClientIpAddressUtils.getString(_))
|
||||
.orElse(
|
||||
getRequestHeader(
|
||||
HttpHeaderNames.X_TWITTER_AUDIT_IP.toLowerCase
|
||||
) // fallback to old way before migration is completed
|
||||
.map(h => InetAddressUtil.getByName(h.trim).getHostAddress)
|
||||
)
|
||||
|
||||
val isDarkRequest = opContext.darkRequest.isDefined
|
||||
|
||||
val sessionHash = getRequestHeader(SessionHashHeaderName)
|
||||
|
||||
val cardsPlatformKey = twitterContext.clientApplicationId.map(GetPlatformKey(_))
|
||||
|
||||
val safetyLevel = opContext.safetyLevel
|
||||
|
||||
RequestContext(
|
||||
clientApplicationId = twitterContext.clientApplicationId,
|
||||
deviceSource = deviceSource,
|
||||
knownDeviceToken = twitterContext.knownDeviceToken,
|
||||
remoteHost = remoteHost,
|
||||
twitterUserId = twitterUserId,
|
||||
contributorId = contributorId,
|
||||
isDarkRequest = isDarkRequest,
|
||||
hasPrivilegeNullcastingAccess =
|
||||
hasClientApplicationPrivilege(ClientAppPrivileges.NULLCASTING_ACCESS),
|
||||
hasPrivilegePromotedTweetsInTimeline =
|
||||
hasClientApplicationPrivilege(ClientAppPrivileges.PROMOTED_TWEETS_IN_TIMELINE),
|
||||
sessionHash = sessionHash,
|
||||
cardsPlatformKey = cardsPlatformKey,
|
||||
safetyLevel = safetyLevel,
|
||||
)
|
||||
}
|
||||
}
|
@ -0,0 +1,32 @@
|
||||
scala_library(
|
||||
sources = ["*.scala"],
|
||||
compiler_option_sets = ["fatal_warnings"],
|
||||
strict_deps = True,
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"finagle-internal/mtls/src/main/scala/com/twitter/finagle/mtls/authentication",
|
||||
"finagle/finagle-core/src/main",
|
||||
"tweetypie/servo/util/src/main/scala",
|
||||
"src/thrift/com/twitter/consumer_privacy/mention_controls:thrift-scala",
|
||||
"src/thrift/com/twitter/gizmoduck:thrift-scala",
|
||||
"src/thrift/com/twitter/spam/rtf:safety-level-scala",
|
||||
"src/thrift/com/twitter/spam/rtf:safety-result-scala",
|
||||
"tweetypie/common/src/thrift/com/twitter/tweetypie:tweet-scala",
|
||||
"stitch/stitch-compat/src/main/scala/com/twitter/stitch/compat",
|
||||
"stitch/stitch-core/src/main/scala/com/twitter/stitch",
|
||||
"stitch/stitch-gizmoduck",
|
||||
"strato/config/src/thrift/com/twitter/strato/graphql:api-media-graphql-scala",
|
||||
"strato/config/src/thrift/com/twitter/strato/graphql:graphql-scala",
|
||||
"strato/config/src/thrift/com/twitter/strato/graphql:topics-graphql-scala",
|
||||
"strato/src/main/scala/com/twitter/strato/rpc",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie/backends",
|
||||
"vibes/src/main/thrift/com/twitter/vibes:vibes-scala",
|
||||
"weaverbird/common/src/main/scala/com/twitter/weaverbird/common",
|
||||
"weaverbird/common/src/main/scala/com/twitter/weaverbird/converters/common",
|
||||
"weaverbird/common/src/main/scala/com/twitter/weaverbird/converters/tweet",
|
||||
"weaverbird/common/src/main/scala/com/twitter/weaverbird/hydrators",
|
||||
"weaverbird/common/src/main/scala/com/twitter/weaverbird/mappers",
|
||||
"weaverbird/common/src/main/scala/com/twitter/weaverbird/repositories",
|
||||
],
|
||||
)
|
@ -0,0 +1,166 @@
|
||||
package com.twitter.tweetypie
|
||||
package federated
|
||||
package prefetcheddata
|
||||
|
||||
import com.twitter.consumer_privacy.mention_controls.thriftscala.UnmentionInfo
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.gizmoduck.thriftscala.LookupContext
|
||||
import com.twitter.gizmoduck.thriftscala.QueryFields
|
||||
import com.twitter.gizmoduck.thriftscala.UserResult
|
||||
import com.twitter.spam.rtf.thriftscala.SafetyLevel
|
||||
import com.twitter.stitch.compat.LegacySeqGroup
|
||||
import com.twitter.stitch.SeqGroup
|
||||
import com.twitter.stitch.Stitch
|
||||
import com.twitter.strato.graphql.thriftscala.CacheMissStrategy
|
||||
import com.twitter.strato.graphql.thriftscala.PrefetchedData
|
||||
import com.twitter.strato.graphql.thriftscala.TweetResult
|
||||
import com.twitter.tweetypie.backends.Gizmoduck
|
||||
import com.twitter.tweetypie.thriftscala.Tweet
|
||||
import com.twitter.util.Throwables
|
||||
import com.twitter.vibes.thriftscala.VibeV2
|
||||
import com.twitter.weaverbird.common.GetRequestContext
|
||||
import com.twitter.weaverbird.common.PerTOOAppCallerStats
|
||||
import com.twitter.weaverbird.common.RequestContext
|
||||
import com.twitter.weaverbird.converters.tweet.WeaverbirdEntitySetMutations
|
||||
import com.twitter.weaverbird.converters.tweet.WeaverbirdTweetMutations
|
||||
import com.twitter.weaverbird.hydrators._
|
||||
import com.twitter.weaverbird.mappers.ApiTweetPrefetchedMapper
|
||||
import com.twitter.weaverbird.repositories.UserRepository
|
||||
import com.twitter.weaverbird.converters.common.EntityRenderingOptions
|
||||
|
||||
private[federated] final case class PrefetchedDataRequest(
|
||||
tweet: Tweet,
|
||||
sourceTweet: Option[Tweet],
|
||||
quotedTweet: Option[Tweet],
|
||||
unmentionInfo: Option[UnmentionInfo] = None,
|
||||
vibe: Option[VibeV2] = None,
|
||||
safetyLevel: SafetyLevel,
|
||||
requestContext: RequestContext)
|
||||
|
||||
private[federated] final case class PrefetchedDataResponse(value: PrefetchedData)
|
||||
|
||||
private[federated] object PrefetchedDataResponse {
|
||||
// For NotFound, there is no subsequent result or quoted_tweet_results field, so both
|
||||
// settings are false here. These deciders will be removed post migration.
|
||||
private[this] val prefetchedMapper = new ApiTweetPrefetchedMapper(
|
||||
skipTweetResultPrefetchItem = () => false
|
||||
)
|
||||
def notFound(tweetId: Long): PrefetchedDataResponse =
|
||||
PrefetchedDataResponse(
|
||||
value = prefetchedMapper.getPrefetchedData(
|
||||
tweetId = tweetId,
|
||||
apiTweet = None,
|
||||
tweetResult = None
|
||||
)
|
||||
)
|
||||
}
|
||||
|
||||
private[federated] object PrefetchedDataRepository {
|
||||
def apply(
|
||||
thriftTweetToApiTweet: ThriftTweetToApiTweet,
|
||||
prefetchedMapper: ApiTweetPrefetchedMapper,
|
||||
statsReceiver: StatsReceiver,
|
||||
): PrefetchedDataRequest => Stitch[PrefetchedDataResponse] =
|
||||
(request: PrefetchedDataRequest) => {
|
||||
val thriftTweetToApiTweetRequest = ThriftTweetToApiTweetRequest(
|
||||
tweet = request.tweet,
|
||||
sourceTweet = request.sourceTweet,
|
||||
quotedTweet = request.quotedTweet,
|
||||
// For Tweet writes, filteredReason will always be None.
|
||||
filteredReason = None,
|
||||
safetyLevel = request.safetyLevel,
|
||||
requestContext = request.requestContext,
|
||||
entityRenderingOptions = EntityRenderingOptions()
|
||||
)
|
||||
|
||||
val successCounter = statsReceiver.counter("success")
|
||||
val failuresCounter = statsReceiver.counter("failures")
|
||||
val failuresScope = statsReceiver.scope("failures")
|
||||
|
||||
thriftTweetToApiTweet
|
||||
.arrow(thriftTweetToApiTweetRequest)
|
||||
.onSuccess(_ => successCounter.incr())
|
||||
.onFailure { t =>
|
||||
failuresCounter.incr()
|
||||
failuresScope.counter(Throwables.mkString(t): _*).incr()
|
||||
}
|
||||
.map((resp: ThriftTweetToApiTweetResponse) => {
|
||||
val prefetchedData: PrefetchedData = prefetchedMapper.getPrefetchedData(
|
||||
tweetId = request.tweet.id,
|
||||
apiTweet = Some(resp.apiTweet),
|
||||
// since ApiTweet was hydrate, we can fabricate a TweetResult.Tweet
|
||||
tweetResult = Some(TweetResult.Tweet(request.tweet.id)),
|
||||
unmentionInfo = request.unmentionInfo,
|
||||
editControl = request.tweet.editControl,
|
||||
previousCounts = request.tweet.previousCounts,
|
||||
vibe = request.vibe,
|
||||
editPerspective = request.tweet.editPerspective,
|
||||
noteTweet = request.tweet.noteTweet
|
||||
)
|
||||
|
||||
// Notify GraphQL API to not attempt hydration for missing
|
||||
// ApiTweet/TweetResult fields. This is only needed on the
|
||||
// Tweet write path since the newly created Tweet may not
|
||||
// be fully persisted yet in tbird Manhattan.
|
||||
val shortCircuitedPrefetchedData = prefetchedData.copy(
|
||||
onCacheMiss = CacheMissStrategy.ShortCircuitExisting
|
||||
)
|
||||
|
||||
PrefetchedDataResponse(shortCircuitedPrefetchedData)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
private[federated] object PrefetchedDataRepositoryBuilder {
|
||||
def apply(
|
||||
getUserResultsById: Gizmoduck.GetById,
|
||||
statsReceiver: StatsReceiver
|
||||
): PrefetchedDataRequest => Stitch[PrefetchedDataResponse] = {
|
||||
val repoStats = statsReceiver.scope("repositories")
|
||||
|
||||
case class GetUserResultById(
|
||||
queryFields: Set[QueryFields],
|
||||
lookupContext: LookupContext,
|
||||
) extends SeqGroup[UserId, UserResult] {
|
||||
override def run(keys: Seq[UserId]): Future[Seq[Try[UserResult]]] =
|
||||
LegacySeqGroup.liftToSeqTry(getUserResultsById((lookupContext, keys, queryFields)))
|
||||
|
||||
override def maxSize: Int = 100
|
||||
}
|
||||
|
||||
val stitchGetUserResultById: UserRepository.GetUserResultById =
|
||||
(userId: UserId, queryFields: Set[QueryFields], lookupContext: LookupContext) =>
|
||||
Stitch.call(userId, GetUserResultById(queryFields, lookupContext))
|
||||
|
||||
val userRepository = new UserRepository(stitchGetUserResultById, repoStats)
|
||||
|
||||
// Note, this is weaverbird.common.GetRequestContext
|
||||
val getRequestContext = new GetRequestContext()
|
||||
|
||||
// TwiggyUserHydrator is needed to hydrate TwiggyUsers for CWC and misc. logic
|
||||
val twiggyUserHydrator = new TwiggyUserHydrator(userRepository, getRequestContext)
|
||||
|
||||
val weaverbirdMutations = new WeaverbirdTweetMutations(
|
||||
new WeaverbirdEntitySetMutations(
|
||||
new PerTOOAppCallerStats(statsReceiver, getRequestContext)
|
||||
)
|
||||
)
|
||||
|
||||
val prefetchedMapper = new ApiTweetPrefetchedMapper(
|
||||
// do not skip this in mutation path as we depends on it
|
||||
skipTweetResultPrefetchItem = () => false
|
||||
)
|
||||
|
||||
val thriftTweetToApiTweet: ThriftTweetToApiTweet =
|
||||
new FoundThriftTweetToApiTweet(
|
||||
statsReceiver,
|
||||
twiggyUserHydrator,
|
||||
weaverbirdMutations
|
||||
)
|
||||
PrefetchedDataRepository(
|
||||
thriftTweetToApiTweet,
|
||||
prefetchedMapper,
|
||||
repoStats.scope("prefetched_data_repo")
|
||||
)
|
||||
}
|
||||
}
|
@ -0,0 +1,18 @@
|
||||
scala_library(
|
||||
sources = ["*.scala"],
|
||||
compiler_option_sets = ["fatal_warnings"],
|
||||
strict_deps = True,
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"ads-common/loggingclient/src/main/scala",
|
||||
"src/scala/com/twitter/ads/internal/pcl/service",
|
||||
"src/scala/com/twitter/ads/internal/pcl/strato_adaptor",
|
||||
"src/thrift/com/twitter/ads/adserver:ads_shared_types-scala",
|
||||
"src/thrift/com/twitter/ads/callback:engagement_request-scala",
|
||||
"src/thrift/com/twitter/ads/internal/pcl:promoted_content_input-scala",
|
||||
"tweetypie/common/src/thrift/com/twitter/tweetypie:tweet_service_graphql-scala",
|
||||
"strato/src/main/scala/com/twitter/strato/server/context",
|
||||
"twitter-context/src/main/scala",
|
||||
"util/util-stats/src/main/scala/com/twitter/finagle/stats",
|
||||
],
|
||||
)
|
@ -0,0 +1,40 @@
|
||||
package com.twitter.tweetypie
|
||||
package federated
|
||||
package promotedcontent
|
||||
|
||||
import com.twitter.ads.callback.thriftscala.EngagementRequest
|
||||
import com.twitter.ads.internal.pcl.service.CallbackPromotedContentLogger
|
||||
import com.twitter.ads.internal.pcl.strato_adaptor.PromotedContentInputProvider
|
||||
import com.twitter.ads.internal.pcl.thriftscala.PromotedContentInput
|
||||
import com.twitter.adserver.thriftscala.EngagementType
|
||||
import com.twitter.util.Future
|
||||
|
||||
object TweetPromotedContentLogger {
|
||||
sealed abstract class TweetEngagementType(val engagementType: EngagementType)
|
||||
case object TweetEngagement extends TweetEngagementType(EngagementType.Send)
|
||||
case object ReplyEngagement extends TweetEngagementType(EngagementType.Reply)
|
||||
case object RetweetEngagement extends TweetEngagementType(EngagementType.Retweet)
|
||||
|
||||
type Type = (EngagementRequest, TweetEngagementType, Boolean) => Future[Unit]
|
||||
|
||||
private[this] val TwitterContext =
|
||||
com.twitter.context.TwitterContext(com.twitter.tweetypie.TwitterContextPermit)
|
||||
|
||||
def apply(callbackPromotedContentLogger: CallbackPromotedContentLogger): Type =
|
||||
(
|
||||
engagementRequest: EngagementRequest,
|
||||
tweetEngagementType: TweetEngagementType,
|
||||
isDark: Boolean
|
||||
) => {
|
||||
val pci: PromotedContentInput =
|
||||
PromotedContentInputProvider(TwitterContext, engagementRequest)
|
||||
|
||||
// The real logging is fire-and-forget, so we can create the Future and ignore returning it.
|
||||
Future.when(!isDark) {
|
||||
callbackPromotedContentLogger.logNonTrendEngagement(
|
||||
pci,
|
||||
tweetEngagementType.engagementType,
|
||||
pci.impressionId)
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,43 @@
|
||||
scala_library(
|
||||
sources = ["*.scala"],
|
||||
compiler_option_sets = ["fatal_warnings"],
|
||||
strict_deps = True,
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"decider",
|
||||
"finagle/finagle-base-http/src/main",
|
||||
"finatra-internal/api11/src/main/scala/com/twitter/finatra/api11:errors",
|
||||
"geo/model/src/main/scala/com/twitter/geo/model",
|
||||
"passbird/bitfields-thrift/src/main/thrift:thrift-scala",
|
||||
"tweetypie/servo/util/src/main/scala",
|
||||
"tweetypie/servo/util/src/main/scala:exception",
|
||||
"src/scala/com/twitter/accounts/util:safety-meta",
|
||||
"src/thrift/com/twitter/ads/adserver:ad_engagement_details-scala",
|
||||
"src/thrift/com/twitter/ads/adserver:preroll_metadata-scala",
|
||||
"src/thrift/com/twitter/ads/callback:engagement_request-scala",
|
||||
"src/thrift/com/twitter/bouncer:bounce-action-thrift-scala",
|
||||
"src/thrift/com/twitter/context:twitter-context-scala",
|
||||
"tweetypie/common/src/thrift/com/twitter/tweetypie:service-scala",
|
||||
"tweetypie/common/src/thrift/com/twitter/tweetypie:tweet_service_graphql-scala",
|
||||
"stitch/stitch-core/src/main/scala/com/twitter/stitch",
|
||||
"strato/config/src/thrift/com/twitter/strato/graphql:api-media-graphql-scala",
|
||||
"strato/config/src/thrift/com/twitter/strato/graphql:graphql-scala",
|
||||
"strato/config/src/thrift/com/twitter/strato/graphql:topics-graphql-scala",
|
||||
"strato/src/main/scala/com/twitter/strato/client",
|
||||
"strato/src/main/scala/com/twitter/strato/context",
|
||||
"strato/src/main/scala/com/twitter/strato/fed",
|
||||
"strato/src/main/scala/com/twitter/strato/response",
|
||||
"strato/src/main/scala/com/twitter/strato/test/config/bouncer",
|
||||
"strato/src/main/scala/com/twitter/strato/thrift",
|
||||
"strato/src/main/thrift/com/twitter/strato/context:thrift-scala",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie/federated/columns",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie/federated/context",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie/federated/prefetcheddata",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie/federated/promotedcontent",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie/service",
|
||||
"tweetypie/common/src/scala/com/twitter/tweetypie/decider/overrides",
|
||||
"tweetypie/common/src/scala/com/twitter/tweetypie/util",
|
||||
"twitter-context/src/main/scala",
|
||||
],
|
||||
)
|
@ -0,0 +1,140 @@
|
||||
package com.twitter.tweetypie
|
||||
package federated
|
||||
package warmups
|
||||
|
||||
import com.twitter.context.TwitterContext
|
||||
import com.twitter.context.thriftscala.Viewer
|
||||
import com.twitter.spam.rtf.thriftscala.SafetyLevel
|
||||
import com.twitter.stitch.Stitch
|
||||
import com.twitter.strato.access.Access
|
||||
import com.twitter.strato.access.Access.AccessToken
|
||||
import com.twitter.strato.access.Access.AuthenticatedTwitterUserId
|
||||
import com.twitter.strato.access.Access.AuthenticatedTwitterUserNotSuspended
|
||||
import com.twitter.strato.access.Access.TwitterUserId
|
||||
import com.twitter.strato.access.Access.TwitterUserNotSuspended
|
||||
import com.twitter.strato.catalog.Ops
|
||||
import com.twitter.strato.client.StaticClient
|
||||
import com.twitter.strato.context.StratoContext
|
||||
import com.twitter.strato.opcontext.DarkRequest
|
||||
import com.twitter.strato.opcontext.OpContext
|
||||
import com.twitter.strato.test.config.bouncer.TestPrincipals
|
||||
import com.twitter.strato.thrift.ScroogeConvImplicits._
|
||||
import com.twitter.tweetypie.federated.columns.CreateRetweetColumn
|
||||
import com.twitter.tweetypie.federated.columns.CreateTweetColumn
|
||||
import com.twitter.tweetypie.federated.columns.DeleteTweetColumn
|
||||
import com.twitter.tweetypie.federated.columns.UnretweetColumn
|
||||
import com.twitter.tweetypie.service.WarmupQueriesSettings
|
||||
import com.twitter.tweetypie.thriftscala.graphql._
|
||||
import com.twitter.util.logging.Logger
|
||||
import com.twitter.util.Future
|
||||
import com.twitter.util.Stopwatch
|
||||
|
||||
object StratoCatalogWarmups {
|
||||
private[this] val log = Logger(getClass)
|
||||
|
||||
// Performs warmup queries, failing after 30 seconds
|
||||
def warmup(
|
||||
warmupSettings: WarmupQueriesSettings,
|
||||
catalog: PartialFunction[String, Ops]
|
||||
): Future[Unit] = {
|
||||
val elapsed = Stopwatch.start()
|
||||
// note: we need to supply bouncer principals here, because the
|
||||
// columns are gated by a bouncer policy
|
||||
Access
|
||||
.withPrincipals(WarmupPrincipals) {
|
||||
StratoContext.withOpContext(WarmupOpContext) {
|
||||
TwitterContext.let(viewer = WarmupViewer) {
|
||||
warmupSettings.clientId.asCurrent {
|
||||
Stitch.run(executeDarkly(catalog))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
.onSuccess { _ => log.info("warmup completed in %s".format(elapsed())) }
|
||||
.onFailure { t => log.error("could not complete warmup queries before startup.", t) }
|
||||
}
|
||||
|
||||
private val WarmupTwitterUserId = 0L
|
||||
|
||||
private val WarmupPrincipals = Set(
|
||||
TestPrincipals.normalStratoBouncerAccessPrincipal,
|
||||
AuthenticatedTwitterUserId(WarmupTwitterUserId),
|
||||
TwitterUserId(WarmupTwitterUserId),
|
||||
TwitterUserNotSuspended,
|
||||
AuthenticatedTwitterUserNotSuspended,
|
||||
AccessToken(isWritable = true)
|
||||
)
|
||||
|
||||
private[this] val RwebClientId = 0L
|
||||
|
||||
private[this] val WarmupViewer = Viewer(
|
||||
userId = Some(WarmupTwitterUserId),
|
||||
authenticatedUserId = Some(WarmupTwitterUserId),
|
||||
clientApplicationId = Some(RwebClientId),
|
||||
)
|
||||
|
||||
private[this] val WarmupOpContext =
|
||||
OpContext
|
||||
.safetyLevel(SafetyLevel.TweetWritesApi.name)
|
||||
.copy(darkRequest = Some(DarkRequest()))
|
||||
.toThrift()
|
||||
|
||||
private[this] val EllenOscarSelfie = 440322224407314432L
|
||||
|
||||
private[this] val TwitterContext: TwitterContext =
|
||||
com.twitter.context.TwitterContext(com.twitter.tweetypie.TwitterContextPermit)
|
||||
|
||||
private[this] def executeDarkly(catalog: PartialFunction[String, Ops]): Stitch[Unit] = {
|
||||
val stratoClient = new StaticClient(catalog)
|
||||
val tweetCreator =
|
||||
stratoClient.executer[CreateTweetRequest, CreateTweetResponseWithSubqueryPrefetchItems](
|
||||
CreateTweetColumn.Path)
|
||||
|
||||
val tweetDeletor =
|
||||
stratoClient
|
||||
.executer[DeleteTweetRequest, DeleteTweetResponseWithSubqueryPrefetchItems](
|
||||
DeleteTweetColumn.Path)
|
||||
|
||||
val retweetCreator =
|
||||
stratoClient
|
||||
.executer[CreateRetweetRequest, CreateRetweetResponseWithSubqueryPrefetchItems](
|
||||
CreateRetweetColumn.Path)
|
||||
|
||||
val unretweetor =
|
||||
stratoClient
|
||||
.executer[UnretweetRequest, UnretweetResponseWithSubqueryPrefetchItems](
|
||||
UnretweetColumn.Path)
|
||||
|
||||
val stitchCreateTweet =
|
||||
tweetCreator
|
||||
.execute(CreateTweetRequest("getting warmer"))
|
||||
.onSuccess(_ => log.info(s"${CreateTweetColumn.Path} warmup success"))
|
||||
.onFailure(e => log.info(s"${CreateTweetColumn.Path} warmup fail: $e"))
|
||||
|
||||
val stitchDeleteTweet =
|
||||
tweetDeletor
|
||||
.execute(DeleteTweetRequest(-1L))
|
||||
.onSuccess(_ => log.info(s"${DeleteTweetColumn.Path} warmup success"))
|
||||
.onFailure(e => log.info(s"${DeleteTweetColumn.Path} warmup fail: $e"))
|
||||
|
||||
val stitchCreateRetweet =
|
||||
retweetCreator
|
||||
.execute(CreateRetweetRequest(EllenOscarSelfie))
|
||||
.onSuccess(_ => log.info(s"${CreateRetweetColumn.Path} warmup success"))
|
||||
.onFailure(e => log.info(s"${CreateRetweetColumn.Path} warmup fail: $e"))
|
||||
|
||||
val stitchUnretweet =
|
||||
unretweetor
|
||||
.execute(UnretweetRequest(EllenOscarSelfie))
|
||||
.onSuccess(_ => log.info(s"${UnretweetColumn.Path} warmup success"))
|
||||
.onFailure(e => log.info(s"${UnretweetColumn.Path} warmup fail: $e"))
|
||||
|
||||
Stitch
|
||||
.join(
|
||||
stitchCreateTweet,
|
||||
stitchDeleteTweet,
|
||||
stitchCreateRetweet,
|
||||
stitchUnretweet,
|
||||
).unit
|
||||
}
|
||||
}
|
@ -0,0 +1,185 @@
|
||||
package com.twitter.tweetypie
|
||||
package handler
|
||||
|
||||
import com.twitter.snowflake.id.SnowflakeId
|
||||
import com.twitter.tweetutil.DmDeepLink
|
||||
import com.twitter.tweetutil.TweetPermalink
|
||||
import com.twitter.tweetypie.core.CardReferenceUriExtractor
|
||||
import com.twitter.tweetypie.core.NonTombstone
|
||||
import com.twitter.tweetypie.core.TweetCreateFailure
|
||||
import com.twitter.tweetypie.repository.TweetQuery
|
||||
import com.twitter.tweetypie.repository.TweetRepository
|
||||
import com.twitter.tweetypie.thriftscala.CardReference
|
||||
import com.twitter.tweetypie.thriftscala.DeviceSource
|
||||
import com.twitter.tweetypie.thriftscala.QuotedTweet
|
||||
import com.twitter.tweetypie.thriftscala.ShortenedUrl
|
||||
import com.twitter.tweetypie.thriftscala.Tweet
|
||||
import com.twitter.tweetypie.thriftscala.TweetCreateState
|
||||
|
||||
case class AttachmentBuilderRequest(
|
||||
tweetId: TweetId,
|
||||
user: User,
|
||||
mediaUploadIds: Option[Seq[Long]],
|
||||
cardReference: Option[CardReference],
|
||||
attachmentUrl: Option[String],
|
||||
remoteHost: Option[String],
|
||||
darkTraffic: Boolean,
|
||||
deviceSource: DeviceSource) {
|
||||
val ctx: ValidationContext = ValidationContext(
|
||||
user = user,
|
||||
mediaUploadIds = mediaUploadIds,
|
||||
cardReference = cardReference
|
||||
)
|
||||
val passThroughResponse: AttachmentBuilderResult =
|
||||
AttachmentBuilderResult(attachmentUrl = attachmentUrl, validationContext = ctx)
|
||||
}
|
||||
|
||||
case class ValidationContext(
|
||||
user: User,
|
||||
mediaUploadIds: Option[Seq[Long]],
|
||||
cardReference: Option[CardReference])
|
||||
|
||||
case class AttachmentBuilderResult(
|
||||
attachmentUrl: Option[String] = None,
|
||||
quotedTweet: Option[QuotedTweet] = None,
|
||||
extraChars: Int = 0,
|
||||
validationContext: ValidationContext)
|
||||
|
||||
object AttachmentBuilder {
|
||||
|
||||
private[this] val log = Logger(getClass)
|
||||
private[this] val attachmentCountLogger = Logger(
|
||||
"com.twitter.tweetypie.handler.CreateAttachmentCount"
|
||||
)
|
||||
|
||||
type Type = FutureArrow[AttachmentBuilderRequest, AttachmentBuilderResult]
|
||||
type ValidationType = FutureEffect[AttachmentBuilderResult]
|
||||
|
||||
def validateAttachmentUrl(attachmentUrl: Option[String]): Unit.type =
|
||||
attachmentUrl match {
|
||||
case None => Unit
|
||||
case Some(TweetPermalink(_, _)) => Unit
|
||||
case Some(DmDeepLink(_)) => Unit
|
||||
case _ => throw TweetCreateFailure.State(TweetCreateState.InvalidAttachmentUrl)
|
||||
}
|
||||
|
||||
def validateAttachments(
|
||||
stats: StatsReceiver,
|
||||
validateCardRef: Gate[Option[String]]
|
||||
): AttachmentBuilder.ValidationType =
|
||||
FutureEffect { result: AttachmentBuilderResult =>
|
||||
validateAttachmentUrl(result.attachmentUrl)
|
||||
|
||||
val ctx = result.validationContext
|
||||
|
||||
val cardRef = ctx.cardReference.filter {
|
||||
case CardReferenceUriExtractor(NonTombstone(_)) => true
|
||||
case _ => false
|
||||
}
|
||||
|
||||
if (result.quotedTweet.isDefined && cardRef.isEmpty) {
|
||||
Future.Unit
|
||||
} else {
|
||||
val attachmentCount =
|
||||
Seq(
|
||||
ctx.mediaUploadIds,
|
||||
result.attachmentUrl,
|
||||
result.quotedTweet
|
||||
).count(_.nonEmpty)
|
||||
|
||||
val userAgent = TwitterContext().flatMap(_.userAgent)
|
||||
if (attachmentCount + cardRef.count(_ => true) > 1) {
|
||||
attachmentCountLogger.warn(
|
||||
s"Too many attachment types on tweet create from user: ${ctx.user.id}, " +
|
||||
s"agent: '${userAgent}', media: ${ctx.mediaUploadIds}, " +
|
||||
s"attachmentUrl: ${result.attachmentUrl}, cardRef: $cardRef"
|
||||
)
|
||||
stats.counter("too_many_attachment_types_with_cardref").incr()
|
||||
}
|
||||
Future.when(attachmentCount + cardRef.count(_ => validateCardRef(userAgent)) > 1) {
|
||||
Future.exception(TweetCreateFailure.State(TweetCreateState.TooManyAttachmentTypes))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private val queryInclude = TweetQuery.Include(Set(Tweet.CoreDataField.id))
|
||||
|
||||
private val queryOptions = TweetQuery.Options(include = queryInclude)
|
||||
|
||||
def buildUrlShortenerCtx(request: AttachmentBuilderRequest): UrlShortener.Context =
|
||||
UrlShortener.Context(
|
||||
tweetId = request.tweetId,
|
||||
userId = request.user.id,
|
||||
createdAt = SnowflakeId(request.tweetId).time,
|
||||
userProtected = request.user.safety.get.isProtected,
|
||||
clientAppId = request.deviceSource.clientAppId,
|
||||
remoteHost = request.remoteHost,
|
||||
dark = request.darkTraffic
|
||||
)
|
||||
|
||||
def asQuotedTweet(tweet: Tweet, shortenedUrl: ShortenedUrl): QuotedTweet =
|
||||
getShare(tweet) match {
|
||||
case None => QuotedTweet(tweet.id, getUserId(tweet), Some(shortenedUrl))
|
||||
case Some(share) => QuotedTweet(share.sourceStatusId, share.sourceUserId, Some(shortenedUrl))
|
||||
}
|
||||
|
||||
def tweetPermalink(request: AttachmentBuilderRequest): Option[TweetPermalink] =
|
||||
request.attachmentUrl.collectFirst {
|
||||
// prevent tweet-quoting cycles
|
||||
case TweetPermalink(screenName, quotedTweetId) if request.tweetId > quotedTweetId =>
|
||||
TweetPermalink(screenName, quotedTweetId)
|
||||
}
|
||||
|
||||
def apply(
|
||||
tweetRepo: TweetRepository.Optional,
|
||||
urlShortener: UrlShortener.Type,
|
||||
validateAttachments: AttachmentBuilder.ValidationType,
|
||||
stats: StatsReceiver,
|
||||
denyNonTweetPermalinks: Gate[Unit] = Gate.False
|
||||
): Type = {
|
||||
val tweetGetter = TweetRepository.tweetGetter(tweetRepo, queryOptions)
|
||||
val attachmentNotPermalinkCounter = stats.counter("attachment_url_not_tweet_permalink")
|
||||
val quotedTweetFoundCounter = stats.counter("quoted_tweet_found")
|
||||
val quotedTweetNotFoundCounter = stats.counter("quoted_tweet_not_found")
|
||||
|
||||
def buildAttachmentResult(request: AttachmentBuilderRequest) =
|
||||
tweetPermalink(request) match {
|
||||
case Some(qtPermalink) =>
|
||||
tweetGetter(qtPermalink.tweetId).flatMap {
|
||||
case Some(tweet) =>
|
||||
quotedTweetFoundCounter.incr()
|
||||
val ctx = buildUrlShortenerCtx(request)
|
||||
urlShortener((qtPermalink.url, ctx)).map { shortenedUrl =>
|
||||
AttachmentBuilderResult(
|
||||
quotedTweet = Some(asQuotedTweet(tweet, shortenedUrl)),
|
||||
extraChars = shortenedUrl.shortUrl.length + 1,
|
||||
validationContext = request.ctx
|
||||
)
|
||||
}
|
||||
case None =>
|
||||
quotedTweetNotFoundCounter.incr()
|
||||
log.warn(
|
||||
s"unable to extract quote tweet from attachment builder request: $request"
|
||||
)
|
||||
if (denyNonTweetPermalinks()) {
|
||||
throw TweetCreateFailure.State(
|
||||
TweetCreateState.SourceTweetNotFound,
|
||||
Some(s"quoted tweet is not found from given permalink: $qtPermalink")
|
||||
)
|
||||
} else {
|
||||
Future.value(request.passThroughResponse)
|
||||
}
|
||||
}
|
||||
case _ =>
|
||||
attachmentNotPermalinkCounter.incr()
|
||||
Future.value(request.passThroughResponse)
|
||||
}
|
||||
|
||||
FutureArrow { request =>
|
||||
for {
|
||||
result <- buildAttachmentResult(request)
|
||||
() <- validateAttachments(result)
|
||||
} yield result
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,88 @@
|
||||
scala_library(
|
||||
sources = ["*.scala"],
|
||||
compiler_option_sets = ["fatal_warnings"],
|
||||
strict_deps = True,
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"3rdparty/jvm/joda-time",
|
||||
"3rdparty/jvm/org/apache/thrift:libthrift",
|
||||
"3rdparty/jvm/org/geotools:gt-referencing",
|
||||
"3rdparty/jvm/org/locationtech/spatial4j",
|
||||
"compliance/user-consent/src/main/scala/com/twitter/userconsent/compliance/birthdate",
|
||||
"creatives-container/thrift/src/main/thrift:creatives-container-service-scala",
|
||||
"diffshow",
|
||||
"eventbus/client",
|
||||
"featureswitches/featureswitches-core/src/main/scala",
|
||||
"finatra/inject/inject-slf4j/src/main/scala/com/twitter/inject",
|
||||
"flock-client",
|
||||
"flock-client/src/main/thrift:thrift-scala",
|
||||
"geoduck/service/src/main/scala/com/twitter/geoduck/service/common/clientmodules",
|
||||
"geoduck/util/src/main/scala/com/twitter/geoduck/util/primitives",
|
||||
"geoduck/util/src/main/scala/com/twitter/geoduck/util/service",
|
||||
"gizmoduck/common/src/main/scala/com/twitter/gizmoduck/util:scala",
|
||||
"mediaservices/commons/src/main/thrift:thrift-scala",
|
||||
"scrooge-internal/scrooge-schema/src/main/scala/com/twitter/scrooge/schema",
|
||||
"scrooge-internal/scrooge-schema/src/main/scala/com/twitter/scrooge/schema/scrooge/scala",
|
||||
"scrooge-internal/scrooge-schema/src/main/scala/com/twitter/scrooge/schema/tree",
|
||||
"scrooge-internal/src/main/scala/com/twitter/scrooge_internal/linter/known_annotations",
|
||||
"scrooge/scrooge-core",
|
||||
"tweetypie/servo/repo",
|
||||
"tweetypie/servo/util",
|
||||
"snowflake:id",
|
||||
"src/scala/com/twitter/takedown/util",
|
||||
"src/thrift/com/twitter/botmaker:botmaker-scala",
|
||||
"src/thrift/com/twitter/bouncer:bounce-action-thrift-scala",
|
||||
"src/thrift/com/twitter/context:testing-signals-scala",
|
||||
"src/thrift/com/twitter/context:twitter-context-scala",
|
||||
"src/thrift/com/twitter/escherbird:media-annotation-structs-scala",
|
||||
"src/thrift/com/twitter/expandodo:only-scala",
|
||||
"src/thrift/com/twitter/geoduck:geoduck-scala",
|
||||
"src/thrift/com/twitter/gizmoduck:thrift-scala",
|
||||
"src/thrift/com/twitter/gizmoduck:user-thrift-scala",
|
||||
"src/thrift/com/twitter/gizmoduck:user-type-thrift-scala",
|
||||
"src/thrift/com/twitter/relevance/feature_store:feature_store-scala",
|
||||
"src/thrift/com/twitter/service/scarecrow/gen:scarecrow-scala",
|
||||
"src/thrift/com/twitter/service/scarecrow/gen:tiered-actions-scala",
|
||||
"src/thrift/com/twitter/service/talon/gen:thrift-scala",
|
||||
"src/thrift/com/twitter/servo:servo-exception-scala",
|
||||
"src/thrift/com/twitter/spam/features:safety-meta-data-scala",
|
||||
"src/thrift/com/twitter/spam/rtf:safety-level-scala",
|
||||
"src/thrift/com/twitter/spam/rtf:tweet-rtf-event-scala",
|
||||
"src/thrift/com/twitter/timelineservice/server/internal:thrift-scala",
|
||||
"tweetypie/common/src/thrift/com/twitter/tweetypie:audit-scala",
|
||||
"tweetypie/common/src/thrift/com/twitter/tweetypie:delete_location_data-scala",
|
||||
"tweetypie/common/src/thrift/com/twitter/tweetypie:media-entity-scala",
|
||||
"tweetypie/common/src/thrift/com/twitter/tweetypie:service-scala",
|
||||
"tweetypie/common/src/thrift/com/twitter/tweetypie:stored-tweet-info-scala",
|
||||
"tweetypie/common/src/thrift/com/twitter/tweetypie:tweet-scala",
|
||||
"stitch/stitch-core",
|
||||
"tco-util",
|
||||
"tweet-util/src/main/scala",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie/backends",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie/core",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie/media",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie/repository",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie/serverutil",
|
||||
"tweetypie/server/src/main/scala/com/twitter/tweetypie/store",
|
||||
"tweetypie/server/src/main/thrift:compiled-scala",
|
||||
"tweetypie/common/src/scala/com/twitter/tweetypie/additionalfields",
|
||||
"tweetypie/common/src/scala/com/twitter/tweetypie/jiminy/tweetypie",
|
||||
"tweetypie/common/src/scala/com/twitter/tweetypie/media",
|
||||
"tweetypie/common/src/scala/com/twitter/tweetypie/storage",
|
||||
"tweetypie/common/src/scala/com/twitter/tweetypie/thriftscala/entities",
|
||||
"tweetypie/common/src/scala/com/twitter/tweetypie/tweettext",
|
||||
"tweetypie/common/src/scala/com/twitter/tweetypie/util",
|
||||
"twitter-context",
|
||||
"twitter-text/lib/java/src/main/java/com/twitter/twittertext",
|
||||
"util/util-slf4j-api/src/main/scala/com/twitter/util/logging",
|
||||
"util/util-stats",
|
||||
"visibility/common/src/main/scala/com/twitter/visibility/common",
|
||||
"visibility/lib/src/main/scala/com/twitter/visibility/builder",
|
||||
"visibility/lib/src/main/scala/com/twitter/visibility/generators",
|
||||
"visibility/lib/src/main/scala/com/twitter/visibility/models",
|
||||
"visibility/writer/src/main/scala/com/twitter/visibility/writer",
|
||||
"visibility/writer/src/main/scala/com/twitter/visibility/writer/interfaces/tweets",
|
||||
"visibility/writer/src/main/scala/com/twitter/visibility/writer/models",
|
||||
],
|
||||
)
|
@ -0,0 +1,74 @@
|
||||
package com.twitter.tweetypie
|
||||
package handler
|
||||
|
||||
import com.twitter.expandodo.thriftscala.AttachmentEligibilityResponses
|
||||
import com.twitter.expandodo.{thriftscala => expandodo}
|
||||
import com.twitter.tweetypie.backends.Expandodo
|
||||
import com.twitter.twittertext.Extractor
|
||||
import scala.util.control.NoStackTrace
|
||||
import scala.util.control.NonFatal
|
||||
import java.net.URI
|
||||
|
||||
object CardReferenceValidationFailedException extends Exception with NoStackTrace
|
||||
|
||||
object CardReferenceValidationHandler {
|
||||
type Type = FutureArrow[(UserId, CardUri), CardUri]
|
||||
|
||||
def apply(checkEligibility: Expandodo.CheckAttachmentEligibility): Type = {
|
||||
def validateAttachmentForUser(userId: UserId, cardUri: CardUri): Future[CardUri] = {
|
||||
val request = Seq(expandodo.AttachmentEligibilityRequest(cardUri, userId))
|
||||
checkEligibility(request)
|
||||
.flatMap(validatedCardUri)
|
||||
.rescue {
|
||||
case NonFatal(_) => Future.exception(CardReferenceValidationFailedException)
|
||||
}
|
||||
}
|
||||
|
||||
FutureArrow {
|
||||
case (userId, cardUri) =>
|
||||
if (shouldSkipValidation(cardUri)) {
|
||||
Future.value(cardUri)
|
||||
} else {
|
||||
validateAttachmentForUser(userId, cardUri)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private[this] def validatedCardUri(responses: AttachmentEligibilityResponses) = {
|
||||
responses.results.headOption match {
|
||||
case Some(
|
||||
expandodo.AttachmentEligibilityResult
|
||||
.Success(expandodo.ValidCardUri(validatedCardUri))
|
||||
) =>
|
||||
Future.value(validatedCardUri)
|
||||
case _ =>
|
||||
Future.exception(CardReferenceValidationFailedException)
|
||||
}
|
||||
}
|
||||
|
||||
// We're not changing state between calls, so it's safe to share among threads
|
||||
private[this] val extractor = {
|
||||
val extractor = new Extractor
|
||||
extractor.setExtractURLWithoutProtocol(false)
|
||||
extractor
|
||||
}
|
||||
|
||||
// Card References with these URIs don't need validation since cards referenced by URIs in these
|
||||
// schemes are public and hence not subject to restrictions.
|
||||
private[handler] val isWhitelistedSchema = Set("http", "https", "tombstone")
|
||||
|
||||
// NOTE: http://www.ietf.org/rfc/rfc2396.txt
|
||||
private[this] def hasWhitelistedScheme(cardUri: CardUri) =
|
||||
Try(new URI(cardUri)).toOption
|
||||
.map(_.getScheme)
|
||||
.exists(isWhitelistedSchema)
|
||||
|
||||
// Even though URI spec is technically is a superset of http:// and https:// URLs, we have to
|
||||
// resort to using a Regex based parser here as a fallback because many URLs found in the wild
|
||||
// have unescaped components that would fail java.net.URI parsing, yet are still considered acceptable.
|
||||
private[this] def isTwitterUrlEntity(cardUri: CardUri) =
|
||||
extractor.extractURLs(cardUri).size == 1
|
||||
|
||||
private[this] def shouldSkipValidation(cardUri: CardUri) =
|
||||
hasWhitelistedScheme(cardUri) || isTwitterUrlEntity(cardUri)
|
||||
}
|
@ -0,0 +1,52 @@
|
||||
package com.twitter.tweetypie
|
||||
package handler
|
||||
|
||||
import com.twitter.stitch.Stitch
|
||||
import com.twitter.tweetypie.core.CardReferenceUriExtractor
|
||||
import com.twitter.tweetypie.core.NonTombstone
|
||||
import com.twitter.tweetypie.core.Tombstone
|
||||
import com.twitter.tweetypie.repository.CardUsersRepository
|
||||
import com.twitter.tweetypie.repository.CardUsersRepository.Context
|
||||
import com.twitter.tweetypie.thriftscala.CardReference
|
||||
|
||||
/**
|
||||
* Finds a set of UserId that may be mentioned when replying to a tweet that has a card.
|
||||
*
|
||||
* Replies created without 'auto_populate_reply_metadata' include both 'site' and 'author' users to
|
||||
* have a more exhaustive list of mentions to match against. This is needed because iOS and Android
|
||||
* have had different implementations client-side for years.
|
||||
*/
|
||||
object CardUsersFinder {
|
||||
|
||||
case class Request(
|
||||
cardReference: Option[CardReference],
|
||||
urls: Seq[String],
|
||||
perspectiveUserId: UserId) {
|
||||
val uris: Seq[String] = cardReference match {
|
||||
case Some(CardReferenceUriExtractor(cardUri)) =>
|
||||
cardUri match {
|
||||
case NonTombstone(uri) => Seq(uri)
|
||||
case Tombstone => Nil
|
||||
}
|
||||
case _ => urls
|
||||
}
|
||||
|
||||
val context: CardUsersRepository.Context = Context(perspectiveUserId)
|
||||
}
|
||||
|
||||
type Type = Request => Stitch[Set[UserId]]
|
||||
|
||||
/**
|
||||
* From a card-related arguments in [[Request]] select the set of user ids associated with the
|
||||
* card.
|
||||
*
|
||||
* Note that this uses the same "which card do I use?" logic from Card2Hydrator which
|
||||
* prioritizes CardReferenceUri and then falls back to the last resolvable (non-None) url entity.
|
||||
*/
|
||||
def apply(cardUserRepo: CardUsersRepository.Type): Type =
|
||||
request =>
|
||||
Stitch
|
||||
.traverse(request.uris) { uri => cardUserRepo(uri, request.context) }
|
||||
// select the last, non-None Set of users ids
|
||||
.map(r => r.flatten.reverse.headOption.getOrElse(Set.empty))
|
||||
}
|
@ -0,0 +1,109 @@
|
||||
package com.twitter.tweetypie
|
||||
package handler
|
||||
|
||||
import com.twitter.tweetypie.core.TweetCreateFailure
|
||||
import com.twitter.tweetypie.thriftscala.CollabControl
|
||||
import com.twitter.tweetypie.thriftscala.CollabControlOptions
|
||||
import com.twitter.tweetypie.thriftscala.CollabInvitation
|
||||
import com.twitter.tweetypie.thriftscala.CollabInvitationOptions
|
||||
import com.twitter.tweetypie.thriftscala.CollabInvitationStatus
|
||||
import com.twitter.tweetypie.thriftscala.CollabTweet
|
||||
import com.twitter.tweetypie.thriftscala.CollabTweetOptions
|
||||
import com.twitter.tweetypie.thriftscala.Communities
|
||||
import com.twitter.tweetypie.thriftscala.ExclusiveTweetControl
|
||||
import com.twitter.tweetypie.thriftscala.InvitedCollaborator
|
||||
import com.twitter.tweetypie.thriftscala.TrustedFriendsControl
|
||||
import com.twitter.tweetypie.thriftscala.TweetCreateConversationControl
|
||||
import com.twitter.tweetypie.thriftscala.TweetCreateState.CollabTweetInvalidParams
|
||||
import com.twitter.tweetypie.util.CommunityUtil
|
||||
|
||||
object CollabControlBuilder {
|
||||
type Type = Request => Future[Option[CollabControl]]
|
||||
|
||||
case class Request(
|
||||
collabControlOptions: Option[CollabControlOptions],
|
||||
replyResult: Option[ReplyBuilder.Result],
|
||||
communities: Option[Communities],
|
||||
trustedFriendsControl: Option[TrustedFriendsControl],
|
||||
conversationControl: Option[TweetCreateConversationControl],
|
||||
exclusiveTweetControl: Option[ExclusiveTweetControl],
|
||||
userId: UserId)
|
||||
|
||||
def apply(): Type = { request =>
|
||||
val collabControl = convertToCollabControl(request.collabControlOptions, request.userId)
|
||||
|
||||
validateCollabControlParams(
|
||||
collabControl,
|
||||
request.replyResult,
|
||||
request.communities,
|
||||
request.trustedFriendsControl,
|
||||
request.conversationControl,
|
||||
request.exclusiveTweetControl,
|
||||
request.userId
|
||||
) map { _ => collabControl }
|
||||
}
|
||||
|
||||
def convertToCollabControl(
|
||||
collabTweetOptions: Option[CollabControlOptions],
|
||||
authorId: UserId
|
||||
): Option[CollabControl] = {
|
||||
collabTweetOptions flatMap {
|
||||
case CollabControlOptions.CollabInvitation(
|
||||
collabInvitationOptions: CollabInvitationOptions) =>
|
||||
Some(
|
||||
CollabControl.CollabInvitation(
|
||||
CollabInvitation(
|
||||
invitedCollaborators = collabInvitationOptions.collaboratorUserIds.map(userId => {
|
||||
InvitedCollaborator(
|
||||
collaboratorUserId = userId,
|
||||
collabInvitationStatus =
|
||||
if (userId == authorId)
|
||||
CollabInvitationStatus.Accepted
|
||||
else CollabInvitationStatus.Pending
|
||||
)
|
||||
})
|
||||
)
|
||||
)
|
||||
)
|
||||
case CollabControlOptions.CollabTweet(collabTweetOptions: CollabTweetOptions) =>
|
||||
Some(
|
||||
CollabControl.CollabTweet(
|
||||
CollabTweet(
|
||||
collaboratorUserIds = collabTweetOptions.collaboratorUserIds
|
||||
)
|
||||
)
|
||||
)
|
||||
case _ => None
|
||||
}
|
||||
}
|
||||
|
||||
def validateCollabControlParams(
|
||||
collabControl: Option[CollabControl],
|
||||
replyResult: Option[ReplyBuilder.Result],
|
||||
communities: Option[Communities],
|
||||
trustedFriendsControl: Option[TrustedFriendsControl],
|
||||
conversationControl: Option[TweetCreateConversationControl],
|
||||
exclusiveTweetControl: Option[ExclusiveTweetControl],
|
||||
userId: UserId
|
||||
): Future[Unit] = {
|
||||
val isInReplyToTweet = replyResult.exists(_.reply.inReplyToStatusId.isDefined)
|
||||
|
||||
collabControl match {
|
||||
case Some(_: CollabControl)
|
||||
if (isInReplyToTweet ||
|
||||
CommunityUtil.hasCommunity(communities) ||
|
||||
exclusiveTweetControl.isDefined ||
|
||||
trustedFriendsControl.isDefined ||
|
||||
conversationControl.isDefined) =>
|
||||
Future.exception(TweetCreateFailure.State(CollabTweetInvalidParams))
|
||||
case Some(CollabControl.CollabInvitation(collab_invitation))
|
||||
if collab_invitation.invitedCollaborators.head.collaboratorUserId != userId =>
|
||||
Future.exception(TweetCreateFailure.State(CollabTweetInvalidParams))
|
||||
case Some(CollabControl.CollabTweet(collab_tweet))
|
||||
if collab_tweet.collaboratorUserIds.head != userId =>
|
||||
Future.exception(TweetCreateFailure.State(CollabTweetInvalidParams))
|
||||
case _ =>
|
||||
Future.Unit
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,40 @@
|
||||
package com.twitter.tweetypie.handler
|
||||
|
||||
import com.twitter.featureswitches.v2.FeatureSwitchResults
|
||||
import com.twitter.servo.util.Gate
|
||||
import com.twitter.tweetypie.Future
|
||||
import com.twitter.tweetypie.core.TweetCreateFailure
|
||||
import com.twitter.tweetypie.thriftscala.Communities
|
||||
import com.twitter.tweetypie.thriftscala.TweetCreateState.CommunityProtectedUserCannotTweet
|
||||
import com.twitter.tweetypie.util.CommunityUtil
|
||||
|
||||
object CommunitiesValidator {
|
||||
case class Request(
|
||||
matchedResults: Option[FeatureSwitchResults],
|
||||
isProtected: Boolean,
|
||||
community: Option[Communities])
|
||||
|
||||
type Type = Request => Future[Unit]
|
||||
|
||||
val CommunityProtectedCanCreateTweet = "communities_protected_community_tweet_creation_enabled"
|
||||
|
||||
val communityProtectedCanCreateTweetGate: Gate[Request] = Gate { request: Request =>
|
||||
request.matchedResults
|
||||
.flatMap(_.getBoolean(CommunityProtectedCanCreateTweet, shouldLogImpression = true))
|
||||
.contains(false)
|
||||
}
|
||||
|
||||
def apply(): Type =
|
||||
(request: Request) => {
|
||||
// Order is important: the feature-switch gate is checked only when the
|
||||
// request is both protected & community so that the FS experiment measurements
|
||||
// are based only on data from requests that are subject to rejection by this validator.
|
||||
if (request.isProtected &&
|
||||
CommunityUtil.hasCommunity(request.community) &&
|
||||
communityProtectedCanCreateTweetGate(request)) {
|
||||
Future.exception(TweetCreateFailure.State(CommunityProtectedUserCannotTweet))
|
||||
} else {
|
||||
Future.Unit
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,272 @@
|
||||
package com.twitter.tweetypie.handler
|
||||
|
||||
import com.twitter.featureswitches.v2.FeatureSwitchResults
|
||||
import com.twitter.stitch.Stitch
|
||||
import com.twitter.tweetypie.UserId
|
||||
import com.twitter.tweetypie._
|
||||
import com.twitter.tweetypie.core.TweetCreateFailure
|
||||
import com.twitter.tweetypie.repository.UserIdentityRepository
|
||||
import com.twitter.tweetypie.repository.UserKey
|
||||
import com.twitter.tweetypie.thriftscala.ConversationControl
|
||||
import com.twitter.tweetypie.thriftscala.Tweet
|
||||
import com.twitter.tweetypie.thriftscala.TweetCreateConversationControl
|
||||
import com.twitter.tweetypie.thriftscala.TweetCreateState.ConversationControlNotAllowed
|
||||
import com.twitter.tweetypie.thriftscala.TweetCreateState.InvalidConversationControl
|
||||
import com.twitter.tweetypie.util.ConversationControls
|
||||
import com.twitter.util.logging.Logging
|
||||
|
||||
/**
|
||||
* Process request parameters into a ConversationControl value.
|
||||
*/
|
||||
object ConversationControlBuilder extends Logging {
|
||||
type Type = Request => Stitch[Option[ConversationControl]]
|
||||
|
||||
type ScreenName = String
|
||||
|
||||
/**
|
||||
* The fields necessary to create a [[ConversationControl]].
|
||||
*
|
||||
* This is a trait rather than a case class to avoid running the
|
||||
* code to extract the mentions in the cases where handling the
|
||||
* request doesn't need to use them (the common case where
|
||||
* tweetCreateConversationControl is None).
|
||||
*/
|
||||
trait Request {
|
||||
def tweetCreateConversationControl: Option[TweetCreateConversationControl]
|
||||
def tweetAuthorId: UserId
|
||||
def mentionedUserScreenNames: Set[String]
|
||||
|
||||
def noteTweetMentionedUserIds: Option[Set[Long]]
|
||||
}
|
||||
|
||||
object Request {
|
||||
|
||||
/**
|
||||
* Extract the data necessary to create a [[ConversationControl]]
|
||||
* for a new [[Tweet]]. This is intended for use when creating
|
||||
* Tweets. It must be called after the Tweet has had its entities
|
||||
* extracted.
|
||||
*/
|
||||
def fromTweet(
|
||||
tweet: Tweet,
|
||||
tweetCreateConversationControl: Option[TweetCreateConversationControl],
|
||||
noteTweetMentionedUserIdsList: Option[Seq[Long]]
|
||||
): Request = {
|
||||
val cctl = tweetCreateConversationControl
|
||||
new Request {
|
||||
def tweetCreateConversationControl: Option[TweetCreateConversationControl] = cctl
|
||||
def mentionedUserScreenNames: Set[ScreenName] =
|
||||
tweet.mentions
|
||||
// Enforce that the Tweet's mentions have already been
|
||||
// extracted from the text. (Mentions will be None if they
|
||||
// have not yet been extracted.)
|
||||
.getOrElse(
|
||||
throw new RuntimeException(
|
||||
"Mentions must be extracted before applying ConversationControls"))
|
||||
.map(_.screenName)
|
||||
.toSet
|
||||
|
||||
def tweetAuthorId: UserId = tweet.coreData.get.userId
|
||||
def noteTweetMentionedUserIds: Option[Set[Long]] =
|
||||
noteTweetMentionedUserIdsList.map(_.toSet)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a ConversationControlBuilder that looks up user ids for
|
||||
* screen names using the specified UserIdentityRepository.
|
||||
*/
|
||||
def fromUserIdentityRepo(
|
||||
statsReceiver: StatsReceiver,
|
||||
userIdentityRepo: UserIdentityRepository.Type
|
||||
): Request => Stitch[Option[ConversationControl]] =
|
||||
ConversationControlBuilder(
|
||||
getUserId = screenName => userIdentityRepo(UserKey.byScreenName(screenName)).map(_.id),
|
||||
statsReceiver = statsReceiver
|
||||
)
|
||||
|
||||
/**
|
||||
* Extract the inviteViaMention value which does not exist on the TweetCreateConversationControl
|
||||
* itself but does exist on the structures it unions.
|
||||
*/
|
||||
def inviteViaMention(tccc: TweetCreateConversationControl): Boolean =
|
||||
tccc match {
|
||||
case TweetCreateConversationControl.ByInvitation(c) => c.inviteViaMention.contains(true)
|
||||
case TweetCreateConversationControl.Community(c) => c.inviteViaMention.contains(true)
|
||||
case TweetCreateConversationControl.Followers(c) => c.inviteViaMention.contains(true)
|
||||
case _ => false
|
||||
}
|
||||
|
||||
/**
|
||||
* Translates the TweetCreateConversationControl into
|
||||
* ConversationControl using the context from the rest of the tweet
|
||||
* creation. For the most part, this is just a direct translation,
|
||||
* plus filling in the contextual user ids (mentioned users and tweet
|
||||
* author).
|
||||
*/
|
||||
def apply(
|
||||
statsReceiver: StatsReceiver,
|
||||
getUserId: ScreenName => Stitch[UserId]
|
||||
): Request => Stitch[Option[ConversationControl]] = {
|
||||
val userIdLookupsCounter = statsReceiver.counter("user_id_lookups")
|
||||
val conversationControlPresentCounter = statsReceiver.counter("conversation_control_present")
|
||||
val conversationControlInviteViaMentionPresentCounter =
|
||||
statsReceiver.counter("conversation_control_invite_via_mention_present")
|
||||
val failureCounter = statsReceiver.counter("failures")
|
||||
|
||||
// Get the user ids for these screen names. Any users who do not
|
||||
// exist will be silently dropped.
|
||||
def getExistingUserIds(
|
||||
screenNames: Set[ScreenName],
|
||||
mentionedUserIds: Option[Set[Long]]
|
||||
): Stitch[Set[UserId]] = {
|
||||
mentionedUserIds match {
|
||||
case Some(userIds) => Stitch.value(userIds)
|
||||
case _ =>
|
||||
Stitch
|
||||
.traverse(screenNames.toSeq) { screenName =>
|
||||
getUserId(screenName).liftNotFoundToOption
|
||||
.ensure(userIdLookupsCounter.incr())
|
||||
}
|
||||
.map(userIdOptions => userIdOptions.flatten.toSet)
|
||||
}
|
||||
}
|
||||
|
||||
// This is broken out just to make it syntactically nicer to add
|
||||
// the stats handling
|
||||
def process(request: Request): Stitch[Option[ConversationControl]] =
|
||||
request.tweetCreateConversationControl match {
|
||||
case None => Stitch.None
|
||||
case Some(cctl) =>
|
||||
cctl match {
|
||||
case TweetCreateConversationControl.ByInvitation(byInvitationControl) =>
|
||||
for {
|
||||
invitedUserIds <- getExistingUserIds(
|
||||
request.mentionedUserScreenNames,
|
||||
request.noteTweetMentionedUserIds)
|
||||
} yield Some(
|
||||
ConversationControls.byInvitation(
|
||||
invitedUserIds = invitedUserIds.toSeq.filterNot(_ == request.tweetAuthorId),
|
||||
conversationTweetAuthorId = request.tweetAuthorId,
|
||||
byInvitationControl.inviteViaMention
|
||||
)
|
||||
)
|
||||
|
||||
case TweetCreateConversationControl.Community(communityControl) =>
|
||||
for {
|
||||
invitedUserIds <- getExistingUserIds(
|
||||
request.mentionedUserScreenNames,
|
||||
request.noteTweetMentionedUserIds)
|
||||
} yield Some(
|
||||
ConversationControls.community(
|
||||
invitedUserIds = invitedUserIds.toSeq.filterNot(_ == request.tweetAuthorId),
|
||||
conversationTweetAuthorId = request.tweetAuthorId,
|
||||
communityControl.inviteViaMention
|
||||
)
|
||||
)
|
||||
case TweetCreateConversationControl.Followers(followersControl) =>
|
||||
for {
|
||||
invitedUserIds <- getExistingUserIds(
|
||||
request.mentionedUserScreenNames,
|
||||
request.noteTweetMentionedUserIds)
|
||||
} yield Some(
|
||||
ConversationControls.followers(
|
||||
invitedUserIds = invitedUserIds.toSeq.filterNot(_ == request.tweetAuthorId),
|
||||
conversationTweetAuthorId = request.tweetAuthorId,
|
||||
followersControl.inviteViaMention
|
||||
)
|
||||
)
|
||||
// This should only ever happen if a new value is added to the
|
||||
// union and we don't update this code.
|
||||
case TweetCreateConversationControl.UnknownUnionField(fld) =>
|
||||
throw new RuntimeException(s"Unexpected TweetCreateConversationControl: $fld")
|
||||
}
|
||||
}
|
||||
|
||||
(request: Request) => {
|
||||
// Wrap in Stitch to encapsulate any exceptions that happen
|
||||
// before making a Stitch call inside of process.
|
||||
Stitch(process(request)).flatten.respond { response =>
|
||||
// If we count this before doing the work, and the stats are
|
||||
// collected before the RPC completes, then any failures
|
||||
// will get counted in a different minute than the request
|
||||
// that caused it.
|
||||
request.tweetCreateConversationControl.foreach { cc =>
|
||||
conversationControlPresentCounter.incr()
|
||||
if (inviteViaMention(cc)) conversationControlInviteViaMentionPresentCounter.incr()
|
||||
}
|
||||
|
||||
response.onFailure { e =>
|
||||
error(message = "Failed to create conversation control", cause = e)
|
||||
// Don't bother counting individual exceptions, because
|
||||
// the cost of keeping those stats is probably not worth
|
||||
// the convenience of not having to look in the logs.
|
||||
failureCounter.incr()
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Validates if a conversation control request is allowed by feature switches
|
||||
* and is only requested on a root tweet.
|
||||
*/
|
||||
object Validate {
|
||||
case class Request(
|
||||
matchedResults: Option[FeatureSwitchResults],
|
||||
conversationControl: Option[TweetCreateConversationControl],
|
||||
inReplyToTweetId: Option[TweetId])
|
||||
|
||||
type Type = FutureEffect[Request]
|
||||
|
||||
val ExInvalidConversationControl = TweetCreateFailure.State(InvalidConversationControl)
|
||||
val ExConversationControlNotAllowed = TweetCreateFailure.State(ConversationControlNotAllowed)
|
||||
val ConversationControlStatusUpdateEnabledKey = "conversation_control_status_update_enabled"
|
||||
val ConversationControlFollowersEnabledKey = "conversation_control_my_followers_enabled"
|
||||
|
||||
def apply(
|
||||
useFeatureSwitchResults: Gate[Unit],
|
||||
statsReceiver: StatsReceiver
|
||||
): Type = request => {
|
||||
def fsDenied(fsKey: String): Boolean = {
|
||||
val featureEnabledOpt: Option[Boolean] =
|
||||
// Do not log impressions, which would interfere with shared client experiment data.
|
||||
request.matchedResults.flatMap(_.getBoolean(fsKey, shouldLogImpression = false))
|
||||
val fsEnabled = featureEnabledOpt.contains(true)
|
||||
if (!fsEnabled) {
|
||||
statsReceiver.counter(s"check_conversation_control/unauthorized/fs/$fsKey").incr()
|
||||
}
|
||||
!fsEnabled
|
||||
}
|
||||
|
||||
val isCcRequest: Boolean = request.conversationControl.isDefined
|
||||
|
||||
val isCcInvalidParams = isCcRequest && {
|
||||
val isRootTweet = request.inReplyToTweetId.isEmpty
|
||||
if (!isRootTweet) {
|
||||
statsReceiver.counter("check_conversation_control/invalid").incr()
|
||||
}
|
||||
!isRootTweet
|
||||
}
|
||||
|
||||
val isCcDeniedByFs = isCcRequest && {
|
||||
val isFollower = request.conversationControl.exists {
|
||||
case _: TweetCreateConversationControl.Followers => true
|
||||
case _ => false
|
||||
}
|
||||
|
||||
fsDenied(ConversationControlStatusUpdateEnabledKey) ||
|
||||
(isFollower && fsDenied(ConversationControlFollowersEnabledKey))
|
||||
}
|
||||
|
||||
if (isCcDeniedByFs && useFeatureSwitchResults()) {
|
||||
Future.exception(ExConversationControlNotAllowed)
|
||||
} else if (isCcInvalidParams) {
|
||||
Future.exception(ExInvalidConversationControl)
|
||||
} else {
|
||||
Future.Unit
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,66 @@
|
||||
package com.twitter.tweetypie
|
||||
package handler
|
||||
|
||||
import com.twitter.stitch.NotFound
|
||||
import com.twitter.stitch.Stitch
|
||||
import com.twitter.tweetypie.repository.TweetQuery
|
||||
import com.twitter.tweetypie.repository.TweetRepository
|
||||
import com.twitter.tweetypie.repository.UserKey
|
||||
import com.twitter.tweetypie.repository.UserQueryOptions
|
||||
import com.twitter.tweetypie.repository.UserRepository
|
||||
import com.twitter.tweetypie.repository.UserVisibility
|
||||
import com.twitter.tweetypie.store.AsyncDeleteAdditionalFields
|
||||
import com.twitter.tweetypie.store.DeleteAdditionalFields
|
||||
import com.twitter.tweetypie.store.TweetStoreEventOrRetry
|
||||
import com.twitter.tweetypie.thriftscala.AsyncDeleteAdditionalFieldsRequest
|
||||
import com.twitter.tweetypie.thriftscala.DeleteAdditionalFieldsRequest
|
||||
|
||||
object DeleteAdditionalFieldsBuilder {
|
||||
type Type = DeleteAdditionalFieldsRequest => Future[Seq[DeleteAdditionalFields.Event]]
|
||||
|
||||
val tweetQueryOptions = TweetQuery.Options(include = GetTweetsHandler.BaseInclude)
|
||||
|
||||
def apply(tweetRepo: TweetRepository.Type): Type = {
|
||||
def getTweet(tweetId: TweetId) =
|
||||
Stitch.run(
|
||||
tweetRepo(tweetId, tweetQueryOptions)
|
||||
.rescue(HandlerError.translateNotFoundToClientError(tweetId))
|
||||
)
|
||||
|
||||
request => {
|
||||
Future.collect(
|
||||
request.tweetIds.map { tweetId =>
|
||||
getTweet(tweetId).map { tweet =>
|
||||
DeleteAdditionalFields.Event(
|
||||
tweetId = tweetId,
|
||||
fieldIds = request.fieldIds,
|
||||
userId = getUserId(tweet),
|
||||
timestamp = Time.now
|
||||
)
|
||||
}
|
||||
}
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object AsyncDeleteAdditionalFieldsBuilder {
|
||||
type Type = AsyncDeleteAdditionalFieldsRequest => Future[
|
||||
TweetStoreEventOrRetry[AsyncDeleteAdditionalFields.Event]
|
||||
]
|
||||
|
||||
val userQueryOpts: UserQueryOptions = UserQueryOptions(Set(UserField.Safety), UserVisibility.All)
|
||||
|
||||
def apply(userRepo: UserRepository.Type): Type = {
|
||||
def getUser(userId: UserId): Future[User] =
|
||||
Stitch.run(
|
||||
userRepo(UserKey.byId(userId), userQueryOpts)
|
||||
.rescue { case NotFound => Stitch.exception(HandlerError.userNotFound(userId)) }
|
||||
)
|
||||
|
||||
request =>
|
||||
getUser(request.userId).map { user =>
|
||||
AsyncDeleteAdditionalFields.Event.fromAsyncRequest(request, user)
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,62 @@
|
||||
package com.twitter.tweetypie
|
||||
package handler
|
||||
|
||||
import com.twitter.eventbus.client.EventBusPublisher
|
||||
import com.twitter.stitch.Stitch
|
||||
import com.twitter.tweetypie.backends.GeoScrubEventStore.GetGeoScrubTimestamp
|
||||
import com.twitter.tweetypie.thriftscala.DeleteLocationData
|
||||
import com.twitter.tweetypie.thriftscala.DeleteLocationDataRequest
|
||||
|
||||
/**
|
||||
* Initiates the process of removing the geo information from a user's
|
||||
* tweets.
|
||||
*/
|
||||
object DeleteLocationDataHandler {
|
||||
type Type = DeleteLocationDataRequest => Future[Unit]
|
||||
|
||||
def apply(
|
||||
getLastScrubTime: GetGeoScrubTimestamp,
|
||||
scribe: DeleteLocationData => Future[Unit],
|
||||
eventbus: EventBusPublisher[DeleteLocationData]
|
||||
): Type =
|
||||
request => {
|
||||
// Attempt to bound the time range of the tweets that need to be
|
||||
// scrubbed by finding the most recent scrub time on record. This
|
||||
// is an optimization that prevents scrubbing already-scrubbed
|
||||
// tweets, so it is OK if the value that we find is occasionally
|
||||
// stale or if the lookup fails. Primarily, this is intended to
|
||||
// protect against intentional abuse by enqueueing multiple
|
||||
// delete_location_data events that have to traverse a very long
|
||||
// timeline.
|
||||
Stitch
|
||||
.run(getLastScrubTime(request.userId))
|
||||
// If there is no timestamp or the lookup failed, continue with
|
||||
// an unchanged request.
|
||||
.handle { case _ => None }
|
||||
.flatMap { lastScrubTime =>
|
||||
// Due to clock skew, it's possible for the last scrub
|
||||
// timestamp to be larger than the timestamp from the request,
|
||||
// but we ignore that so that we keep a faithful record of
|
||||
// user requests. The execution of such events will end up a
|
||||
// no-op.
|
||||
val event =
|
||||
DeleteLocationData(
|
||||
userId = request.userId,
|
||||
timestampMs = Time.now.inMilliseconds,
|
||||
lastTimestampMs = lastScrubTime.map(_.inMilliseconds)
|
||||
)
|
||||
|
||||
Future.join(
|
||||
Seq(
|
||||
// Scribe the event so that we can reprocess events if
|
||||
// there is a bug or operational issue that causes some
|
||||
// events to be lost.
|
||||
scribe(event),
|
||||
// The actual deletion process is handled by the TweetyPie
|
||||
// geoscrub daemon.
|
||||
eventbus.publish(event)
|
||||
)
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,254 @@
|
||||
package com.twitter.tweetypie
|
||||
package handler
|
||||
|
||||
import com.twitter.stitch.Stitch
|
||||
import com.twitter.timelineservice.{thriftscala => tls}
|
||||
import com.twitter.tweetypie.backends.TimelineService
|
||||
import com.twitter.tweetypie.repository.TweetQuery
|
||||
import com.twitter.tweetypie.repository.TweetRepository
|
||||
import com.twitter.tweetypie.thriftscala.CardReference
|
||||
import com.twitter.tweetypie.thriftscala.ConversationControl
|
||||
import com.twitter.tweetypie.thriftscala.ConversationControlByInvitation
|
||||
import com.twitter.tweetypie.thriftscala.ConversationControlCommunity
|
||||
import com.twitter.tweetypie.thriftscala.ConversationControlFollowers
|
||||
import com.twitter.tweetypie.thriftscala.EditControl
|
||||
import com.twitter.tweetypie.thriftscala.EditOptions
|
||||
import com.twitter.tweetypie.thriftscala.NoteTweetOptions
|
||||
import com.twitter.tweetypie.thriftscala.PostTweetRequest
|
||||
import com.twitter.tweetypie.thriftscala.TweetCreateConversationControl
|
||||
import com.twitter.tweetypie.util.ConversationControls
|
||||
import com.twitter.tweetypie.util.EditControlUtil
|
||||
import com.twitter.util.Time
|
||||
|
||||
/**
|
||||
* Used at tweet creation time to determine whether the tweet creation
|
||||
* request should be considered a duplicate of an existing tweet.
|
||||
*/
|
||||
object DuplicateTweetFinder {
|
||||
|
||||
/**
|
||||
* Return the ids of any tweets that are found to be duplicates of
|
||||
* this request.
|
||||
*/
|
||||
type Type = RequestInfo => Future[Option[TweetId]]
|
||||
|
||||
final case class Settings(
|
||||
// The number of tweets that are loaded from the user's timeline
|
||||
// for the heuristic duplicate check
|
||||
numTweetsToCheck: Int,
|
||||
// The oldest that a tweet can be to still be considered a
|
||||
// duplicate by the heuristic duplicate check
|
||||
maxDuplicateAge: Duration)
|
||||
|
||||
// Takes a ConversationControl from a Tweet and converts to the equivalent
|
||||
// TweetCreateConversationControl. Note: this is a lossy conversion because the
|
||||
// ConversationControl contains additional data from the Tweet.
|
||||
def toTweetCreateConversationControl(
|
||||
conversationControl: ConversationControl
|
||||
): TweetCreateConversationControl =
|
||||
conversationControl match {
|
||||
case ConversationControl.ByInvitation(
|
||||
ConversationControlByInvitation(_, _, inviteViaMention)) =>
|
||||
ConversationControls.Create.byInvitation(inviteViaMention)
|
||||
case ConversationControl.Community(ConversationControlCommunity(_, _, inviteViaMention)) =>
|
||||
ConversationControls.Create.community(inviteViaMention)
|
||||
case ConversationControl.Followers(ConversationControlFollowers(_, _, inviteViaMention)) =>
|
||||
ConversationControls.Create.followers(inviteViaMention)
|
||||
case _ => throw new IllegalArgumentException
|
||||
}
|
||||
|
||||
/**
|
||||
* The parts of the request that we need in order to perform
|
||||
* duplicate detection.
|
||||
*/
|
||||
final case class RequestInfo(
|
||||
userId: UserId,
|
||||
isNarrowcast: Boolean,
|
||||
isNullcast: Boolean,
|
||||
text: String,
|
||||
replyToTweetId: Option[TweetId],
|
||||
mediaUploadIds: Seq[MediaId],
|
||||
cardReference: Option[CardReference],
|
||||
conversationControl: Option[TweetCreateConversationControl],
|
||||
underlyingCreativesContainer: Option[CreativesContainerId],
|
||||
editOptions: Option[EditOptions] = None,
|
||||
noteTweetOptions: Option[NoteTweetOptions] = None) {
|
||||
|
||||
def isDuplicateOf(tweet: Tweet, oldestAcceptableTimestamp: Time): Boolean = {
|
||||
val createdAt = getTimestamp(tweet)
|
||||
val isDuplicateText = text == getText(tweet)
|
||||
val isDuplicateReplyToTweetId = replyToTweetId == getReply(tweet).flatMap(_.inReplyToStatusId)
|
||||
val isDuplicateMedia = getMedia(tweet).map(_.mediaId) == mediaUploadIds
|
||||
val isDuplicateCardReference = getCardReference(tweet) == cardReference
|
||||
val isDuplicateConversationControl =
|
||||
tweet.conversationControl.map(toTweetCreateConversationControl) == conversationControl
|
||||
val isDuplicateConversationContainerId = {
|
||||
tweet.underlyingCreativesContainerId == underlyingCreativesContainer
|
||||
}
|
||||
|
||||
val isDuplicateIfEditRequest = if (editOptions.isDefined) {
|
||||
// We do not count an incoming edit request as creating a duplicate tweet if:
|
||||
// 1) The tweet that is considered a duplicate is a previous version of this tweet OR
|
||||
// 2) The tweet that is considered a duplicate is otherwise stale.
|
||||
val tweetEditChain = tweet.editControl match {
|
||||
case Some(EditControl.Initial(initial)) =>
|
||||
initial.editTweetIds
|
||||
case Some(EditControl.Edit(edit)) =>
|
||||
edit.editControlInitial.map(_.editTweetIds).getOrElse(Nil)
|
||||
case _ => Nil
|
||||
}
|
||||
val tweetIsAPreviousVersion =
|
||||
editOptions.map(_.previousTweetId).exists(tweetEditChain.contains)
|
||||
|
||||
val tweetIsStale = EditControlUtil.isLatestEdit(tweet.editControl, tweet.id) match {
|
||||
case Return(false) => true
|
||||
case _ => false
|
||||
}
|
||||
|
||||
!(tweetIsStale || tweetIsAPreviousVersion)
|
||||
} else {
|
||||
// If not an edit request, this condition is true as duplication checking is not blocked
|
||||
true
|
||||
}
|
||||
|
||||
// Note that this does not prevent you from tweeting the same
|
||||
// image twice with different text, or the same text twice with
|
||||
// different images, because if you upload the same media twice,
|
||||
// we will store two copies of it, each with a different media
|
||||
// URL and thus different t.co URL, and since the text that
|
||||
// we're checking here has that t.co URL added to it already, it
|
||||
// is necessarily different.
|
||||
//
|
||||
// We shouldn't have to check the user id or whether it's a
|
||||
// retweet, because we loaded the tweets from the user's
|
||||
// (non-retweet) timelines, but it doesn't hurt and protects
|
||||
// against possible future changes.
|
||||
(oldestAcceptableTimestamp <= createdAt) &&
|
||||
getShare(tweet).isEmpty &&
|
||||
(getUserId(tweet) == userId) &&
|
||||
isDuplicateText &&
|
||||
isDuplicateReplyToTweetId &&
|
||||
isDuplicateMedia &&
|
||||
isDuplicateCardReference &&
|
||||
isDuplicateConversationControl &&
|
||||
isDuplicateConversationContainerId &&
|
||||
isDuplicateIfEditRequest &&
|
||||
noteTweetOptions.isEmpty // Skip duplicate checks for NoteTweets
|
||||
}
|
||||
}
|
||||
|
||||
object RequestInfo {
|
||||
|
||||
/**
|
||||
* Extract the information relevant to the DuplicateTweetFinder
|
||||
* from the PostTweetRequest.
|
||||
*/
|
||||
def fromPostTweetRequest(req: PostTweetRequest, processedText: String): RequestInfo =
|
||||
RequestInfo(
|
||||
userId = req.userId,
|
||||
isNarrowcast = req.narrowcast.nonEmpty,
|
||||
isNullcast = req.nullcast,
|
||||
text = processedText,
|
||||
replyToTweetId = req.inReplyToTweetId,
|
||||
mediaUploadIds = req.mediaUploadIds.getOrElse[Seq[MediaId]](Seq.empty),
|
||||
cardReference = req.additionalFields.flatMap(_.cardReference),
|
||||
conversationControl = req.conversationControl,
|
||||
underlyingCreativesContainer = req.underlyingCreativesContainerId,
|
||||
editOptions = req.editOptions,
|
||||
noteTweetOptions = req.noteTweetOptions
|
||||
)
|
||||
}
|
||||
|
||||
/**
|
||||
* Encapsulates the external interactions that we need to do for
|
||||
* duplicate checking.
|
||||
*/
|
||||
trait TweetSource {
|
||||
def loadTweets(tweetIds: Seq[TweetId]): Future[Seq[Tweet]]
|
||||
def loadUserTimelineIds(userId: UserId, maxCount: Int): Future[Seq[TweetId]]
|
||||
def loadNarrowcastTimelineIds(userId: UserId, maxCount: Int): Future[Seq[TweetId]]
|
||||
}
|
||||
|
||||
object TweetSource {
|
||||
|
||||
/**
|
||||
* Use the provided services to access tweets.
|
||||
*/
|
||||
def fromServices(
|
||||
tweetRepo: TweetRepository.Optional,
|
||||
getStatusTimeline: TimelineService.GetStatusTimeline
|
||||
): TweetSource =
|
||||
new TweetSource {
|
||||
// only fields needed by RequestInfo.isDuplicateOf()
|
||||
private[this] val tweetQueryOption =
|
||||
TweetQuery.Options(
|
||||
TweetQuery.Include(
|
||||
tweetFields = Set(
|
||||
Tweet.CoreDataField.id,
|
||||
Tweet.MediaField.id,
|
||||
Tweet.ConversationControlField.id,
|
||||
Tweet.EditControlField.id
|
||||
),
|
||||
pastedMedia = true
|
||||
)
|
||||
)
|
||||
|
||||
private[this] def loadTimeline(query: tls.TimelineQuery): Future[Seq[Long]] =
|
||||
getStatusTimeline(Seq(query)).map(_.head.entries.map(_.statusId))
|
||||
|
||||
override def loadUserTimelineIds(userId: UserId, maxCount: Int): Future[Seq[Long]] =
|
||||
loadTimeline(
|
||||
tls.TimelineQuery(
|
||||
timelineType = tls.TimelineType.User,
|
||||
timelineId = userId,
|
||||
maxCount = maxCount.toShort
|
||||
)
|
||||
)
|
||||
|
||||
override def loadNarrowcastTimelineIds(userId: UserId, maxCount: Int): Future[Seq[Long]] =
|
||||
loadTimeline(
|
||||
tls.TimelineQuery(
|
||||
timelineType = tls.TimelineType.Narrowcasted,
|
||||
timelineId = userId,
|
||||
maxCount = maxCount.toShort
|
||||
)
|
||||
)
|
||||
|
||||
override def loadTweets(tweetIds: Seq[TweetId]): Future[Seq[Tweet]] =
|
||||
if (tweetIds.isEmpty) {
|
||||
Future.value(Seq[Tweet]())
|
||||
} else {
|
||||
Stitch
|
||||
.run(
|
||||
Stitch.traverse(tweetIds) { tweetId => tweetRepo(tweetId, tweetQueryOption) }
|
||||
)
|
||||
.map(_.flatten)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
def apply(settings: Settings, tweetSource: TweetSource): Type = { reqInfo =>
|
||||
if (reqInfo.isNullcast) {
|
||||
// iff nullcast, we bypass duplication logic all together
|
||||
Future.None
|
||||
} else {
|
||||
val oldestAcceptableTimestamp = Time.now - settings.maxDuplicateAge
|
||||
val userTweetIdsFut =
|
||||
tweetSource.loadUserTimelineIds(reqInfo.userId, settings.numTweetsToCheck)
|
||||
|
||||
// Check the narrowcast timeline iff this is a narrowcasted tweet
|
||||
val narrowcastTweetIdsFut =
|
||||
if (reqInfo.isNarrowcast) {
|
||||
tweetSource.loadNarrowcastTimelineIds(reqInfo.userId, settings.numTweetsToCheck)
|
||||
} else {
|
||||
Future.value(Seq.empty)
|
||||
}
|
||||
|
||||
for {
|
||||
userTweetIds <- userTweetIdsFut
|
||||
narrowcastTweetIds <- narrowcastTweetIdsFut
|
||||
candidateTweets <- tweetSource.loadTweets(userTweetIds ++ narrowcastTweetIds)
|
||||
} yield candidateTweets.find(reqInfo.isDuplicateOf(_, oldestAcceptableTimestamp)).map(_.id)
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,361 @@
|
||||
package com.twitter.tweetypie
|
||||
package handler
|
||||
|
||||
import com.twitter.expandodo.thriftscala.Card2RequestOptions
|
||||
import com.twitter.featureswitches.v2.FeatureSwitchResults
|
||||
import com.twitter.gizmoduck.util.UserUtil
|
||||
import com.twitter.stitch.Stitch
|
||||
import com.twitter.tweetypie.core.TweetCreateFailure
|
||||
import com.twitter.tweetypie.repository.Card2Repository
|
||||
import com.twitter.tweetypie.repository.StratoPromotedTweetRepository
|
||||
import com.twitter.tweetypie.repository.StratoSubscriptionVerificationRepository
|
||||
import com.twitter.tweetypie.repository.TweetQuery
|
||||
import com.twitter.tweetypie.repository.TweetRepository
|
||||
import com.twitter.tweetypie.repository.UrlCard2Key
|
||||
import com.twitter.tweetypie.thriftscala.EditControl
|
||||
import com.twitter.tweetypie.thriftscala.EditOptions
|
||||
import com.twitter.tweetypie.thriftscala.TweetCreateState
|
||||
import com.twitter.tweetypie.util.EditControlUtil._
|
||||
import com.twitter.tweetypie.thriftscala.CardReference
|
||||
import com.twitter.tweetypie.thriftscala.EditControlInitial
|
||||
import com.twitter.tweetypie.thriftscala.PostTweetRequest
|
||||
import com.twitter.tweetypie.util.CommunityAnnotation
|
||||
import com.twitter.tweetypie.util.EditControlUtil
|
||||
import com.twitter.util.Future
|
||||
|
||||
object EditControlBuilder {
|
||||
type Type = Request => Future[Option[EditControl]]
|
||||
|
||||
val editTweetCountStat = "edit_tweet_count"
|
||||
val editControlQueryOptions = TweetQuery.Options(
|
||||
TweetQuery.Include(Set(Tweet.CoreDataField.id, Tweet.EditControlField.id))
|
||||
)
|
||||
val TweetEditCreationEnabledKey = "tweet_edit_creation_enabled"
|
||||
val TweetEditCreationEnabledForTwitterBlueKey = "tweet_edit_creation_enabled_for_twitter_blue"
|
||||
|
||||
val pollCardNames: Set[String] = Set(
|
||||
"poll2choice_text_only",
|
||||
"poll3choice_text_only",
|
||||
"poll4choice_text_only",
|
||||
"poll2choice_image",
|
||||
"poll3choice_image",
|
||||
"poll4choice_image",
|
||||
"poll2choice_video",
|
||||
"poll3choice_video",
|
||||
"poll4choice_video",
|
||||
)
|
||||
|
||||
/** Used just for checking card name for poll check in case cards platform key not provided. */
|
||||
val defaultCardsPlatformKey = "iPhone-13"
|
||||
|
||||
/**
|
||||
* Do we assume a Tweet has a poll (which makes it not editable) when it has a card
|
||||
* that could be a poll, and it cannot be resolved at create.
|
||||
*/
|
||||
val isPollCardAssumption = true
|
||||
|
||||
val tweetEditSubscriptionResource = "feature/tweet_edit"
|
||||
|
||||
val log: Logger = Logger(getClass)
|
||||
|
||||
case class Request(
|
||||
postTweetRequest: PostTweetRequest,
|
||||
tweet: Tweet,
|
||||
matchedResults: Option[FeatureSwitchResults]) {
|
||||
def editOptions: Option[EditOptions] = postTweetRequest.editOptions
|
||||
|
||||
def authorId: UserId = postTweetRequest.userId
|
||||
|
||||
def createdAt: Time = Time.fromMilliseconds(tweet.coreData.get.createdAtSecs * 1000L)
|
||||
|
||||
def tweetId: TweetId = tweet.id
|
||||
|
||||
def cardReference: Option[CardReference] =
|
||||
postTweetRequest.additionalFields.flatMap(_.cardReference)
|
||||
|
||||
def cardsPlatformKey: Option[String] =
|
||||
postTweetRequest.hydrationOptions.flatMap(_.cardsPlatformKey)
|
||||
}
|
||||
|
||||
def apply(
|
||||
tweetRepo: TweetRepository.Type,
|
||||
card2Repo: Card2Repository.Type,
|
||||
promotedTweetRepo: StratoPromotedTweetRepository.Type,
|
||||
subscriptionVerificationRepo: StratoSubscriptionVerificationRepository.Type,
|
||||
disablePromotedTweetEdit: Gate[Unit],
|
||||
checkTwitterBlueSubscription: Gate[Unit],
|
||||
setEditWindowToSixtyMinutes: Gate[Unit],
|
||||
stats: StatsReceiver
|
||||
): Type = {
|
||||
|
||||
// Nullcast tweets not allowed, except if the tweet has a community annotation
|
||||
def isNullcastedButNotCommunityTweet(request: Request): Boolean = {
|
||||
|
||||
val isNullcasted: Boolean = request.tweet.coreData.get.nullcast
|
||||
|
||||
val communityIds: Option[Seq[CommunityId]] =
|
||||
request.postTweetRequest.additionalFields
|
||||
.flatMap(CommunityAnnotation.additionalFieldsToCommunityIDs)
|
||||
|
||||
isNullcasted && !(communityIds.exists(_.nonEmpty))
|
||||
}
|
||||
|
||||
def isSuperFollow(tweet: Tweet): Boolean = tweet.exclusiveTweetControl.isDefined
|
||||
|
||||
def isCollabTweet(tweet: Tweet): Boolean = tweet.collabControl.isDefined
|
||||
|
||||
def isReplyToTweet(tweet: Tweet): Boolean =
|
||||
getReply(tweet).flatMap(_.inReplyToStatusId).isDefined
|
||||
|
||||
// When card is tombstone, tweet is not considered a poll, and therefore can be edit eligible.
|
||||
val cardReferenceUriIsTombstone = stats.counter("edit_control_builder_card_tombstoned")
|
||||
// We check whether tweets are polls since these are not edit eligible.
|
||||
// If we are not sure due to lookup failure, we take an `isPollCardAssumption`.
|
||||
def isPoll(
|
||||
card2Repo: Card2Repository.Type,
|
||||
cardReference: CardReference,
|
||||
cardsPlatformKey: String,
|
||||
): Stitch[Boolean] = {
|
||||
if (cardReference.cardUri == "tombstone://card") {
|
||||
cardReferenceUriIsTombstone.incr()
|
||||
Stitch.value(false)
|
||||
} else {
|
||||
val key = UrlCard2Key(cardReference.cardUri)
|
||||
// `allowNonTcoUrls = true` This allows us to check if non-tco urls (e.g. apple.com) have a card
|
||||
// at this point in tweet builder urls can be in their original form and not tcoified.
|
||||
val options = Card2RequestOptions(
|
||||
platformKey = cardsPlatformKey,
|
||||
allowNonTcoUrls = true
|
||||
)
|
||||
card2Repo(key, options)
|
||||
.map(card2 => pollCardNames.contains(card2.name))
|
||||
}
|
||||
}
|
||||
|
||||
def isFeatureSwitchEnabled(matchedResults: Option[FeatureSwitchResults], key: String): Boolean =
|
||||
matchedResults.flatMap(_.getBoolean(key, shouldLogImpression = false)).contains(true)
|
||||
|
||||
def wrapInitial(initial: EditControlInitial): Option[EditControl.Initial] =
|
||||
Some(EditControl.Initial(initial = initial))
|
||||
|
||||
// Checks for validity of an edit are implemented as procedures
|
||||
// that throw an error in case a check fails. This composes way better than
|
||||
// returning a Try/Future/Stitch because:
|
||||
// 1. We do not need to decide which of the aforementioned containers to use.
|
||||
// 2. The checks as below compose with callbacks in all the aforementioned containers.
|
||||
|
||||
val editRequestOutsideOfAllowlist = stats.counter("edit_control_builder_rejected", "allowlist")
|
||||
|
||||
// This method uses two feature switches:
|
||||
// - TweetEditCreationEnabledKey authorizes the user to edit tweets directly
|
||||
// - TweetEditCreationEnabledForTwitterBlueKey authorizes the user to edit tweets if they have
|
||||
// a Twitter Blue subscription
|
||||
//
|
||||
// Test users are always authorized to edit tweets.
|
||||
def checkUserEligibility(
|
||||
authorId: UserId,
|
||||
matchedResults: Option[FeatureSwitchResults]
|
||||
): Stitch[Unit] = {
|
||||
val isTestUser = UserUtil.isTestUserId(authorId)
|
||||
val authorizedWithoutTwitterBlue =
|
||||
isFeatureSwitchEnabled(matchedResults, TweetEditCreationEnabledKey)
|
||||
|
||||
if (isTestUser || authorizedWithoutTwitterBlue) {
|
||||
// If the editing user is a test user or is authorized by the non-Twitter Blue feature
|
||||
// switch, allow editing.
|
||||
Stitch.Done
|
||||
} else {
|
||||
// Otherwise, check if they're authorized by the Twitter Blue feature switch and if they're
|
||||
// subscribed to Twitter Blue.
|
||||
val authorizedWithTwitterBlue: Stitch[Boolean] =
|
||||
if (checkTwitterBlueSubscription() &&
|
||||
isFeatureSwitchEnabled(matchedResults, TweetEditCreationEnabledForTwitterBlueKey)) {
|
||||
subscriptionVerificationRepo(authorId, tweetEditSubscriptionResource)
|
||||
} else Stitch.value(false)
|
||||
|
||||
authorizedWithTwitterBlue.flatMap { authorized =>
|
||||
if (!authorized) {
|
||||
log.error(s"User ${authorId} unauthorized to edit")
|
||||
editRequestOutsideOfAllowlist.incr()
|
||||
Stitch.exception(TweetCreateFailure.State(TweetCreateState.EditTweetUserNotAuthorized))
|
||||
} else Stitch.Done
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
val editRequestByNonAuthor = stats.counter("edit_control_builder_rejected", "not_author")
|
||||
def checkAuthor(
|
||||
authorId: UserId,
|
||||
previousTweetAuthorId: UserId
|
||||
): Unit = {
|
||||
if (authorId != previousTweetAuthorId) {
|
||||
editRequestByNonAuthor.incr()
|
||||
throw TweetCreateFailure.State(TweetCreateState.EditTweetUserNotAuthor)
|
||||
}
|
||||
}
|
||||
|
||||
val tweetEditForStaleTweet = stats.counter("edit_control_builder_rejected", "stale")
|
||||
def checkLatestEdit(
|
||||
previousTweetId: TweetId,
|
||||
initial: EditControlInitial,
|
||||
): Unit = {
|
||||
if (previousTweetId != initial.editTweetIds.last) {
|
||||
tweetEditForStaleTweet.incr()
|
||||
throw TweetCreateFailure.State(TweetCreateState.EditTweetNotLatestVersion)
|
||||
}
|
||||
}
|
||||
|
||||
val tweetEditForLimitReached = stats.counter("edit_control_builder_rejected", "edits_limit")
|
||||
def checkEditsRemaining(initial: EditControlInitial): Unit = {
|
||||
initial.editsRemaining match {
|
||||
case Some(number) if number > 0 => // OK
|
||||
case _ =>
|
||||
tweetEditForLimitReached.incr()
|
||||
throw TweetCreateFailure.State(TweetCreateState.EditCountLimitReached)
|
||||
}
|
||||
}
|
||||
|
||||
val editTweetExpired = stats.counter("edit_control_builder_rejected", "expired")
|
||||
val editTweetExpiredNoEditControl =
|
||||
stats.counter("edit_control_builder_rejected", "expired", "no_edit_control")
|
||||
def checkEditTimeWindow(initial: EditControlInitial): Unit = {
|
||||
initial.editableUntilMsecs match {
|
||||
case Some(millis) if Time.now < Time.fromMilliseconds(millis) => // OK
|
||||
case Some(_) =>
|
||||
editTweetExpired.incr()
|
||||
throw TweetCreateFailure.State(TweetCreateState.EditTimeLimitReached)
|
||||
case editable =>
|
||||
editTweetExpired.incr()
|
||||
if (editable.isEmpty) {
|
||||
editTweetExpiredNoEditControl.incr()
|
||||
}
|
||||
throw TweetCreateFailure.State(TweetCreateState.EditTimeLimitReached)
|
||||
}
|
||||
}
|
||||
|
||||
val tweetEditNotEligible = stats.counter("edit_control_builder_rejected", "not_eligible")
|
||||
def checkIsEditEligible(initial: EditControlInitial): Unit = {
|
||||
initial.isEditEligible match {
|
||||
case Some(true) => // OK
|
||||
case _ =>
|
||||
tweetEditNotEligible.incr()
|
||||
throw TweetCreateFailure.State(TweetCreateState.NotEligibleForEdit)
|
||||
}
|
||||
}
|
||||
|
||||
val editControlInitialMissing =
|
||||
stats.counter("edit_control_builder_rejected", "initial_missing")
|
||||
def findEditControlInitial(previousTweet: Tweet): EditControlInitial = {
|
||||
previousTweet.editControl match {
|
||||
case Some(EditControl.Initial(initial)) => initial
|
||||
case Some(EditControl.Edit(edit)) =>
|
||||
edit.editControlInitial.getOrElse {
|
||||
editControlInitialMissing.incr()
|
||||
throw new IllegalStateException(
|
||||
"Encountered edit tweet with missing editControlInitial.")
|
||||
}
|
||||
case _ =>
|
||||
throw TweetCreateFailure.State(TweetCreateState.EditTimeLimitReached)
|
||||
}
|
||||
}
|
||||
|
||||
val editPromotedTweet = stats.counter("tweet_edit_for_promoted_tweet")
|
||||
def checkPromotedTweet(
|
||||
previousTweetId: TweetId,
|
||||
promotedTweetRepo: StratoPromotedTweetRepository.Type,
|
||||
disablePromotedTweetEdit: Gate[Unit]
|
||||
): Stitch[Unit] = {
|
||||
if (disablePromotedTweetEdit()) {
|
||||
promotedTweetRepo(previousTweetId).flatMap {
|
||||
case false =>
|
||||
Stitch.Done
|
||||
case true =>
|
||||
editPromotedTweet.incr()
|
||||
Stitch.exception(TweetCreateFailure.State(TweetCreateState.EditTweetUserNotAuthorized))
|
||||
}
|
||||
} else {
|
||||
Stitch.Done
|
||||
}
|
||||
}
|
||||
|
||||
// Each time edit is made, count how many versions a tweet already has.
|
||||
// Value should be always between 1 and 4.
|
||||
val editTweetCount = 0
|
||||
.to(EditControlUtil.maxTweetEditsAllowed)
|
||||
.map(i => i -> stats.counter("edit_control_builder_edits_count", i.toString))
|
||||
.toMap
|
||||
// Overall counter and failures of card resolution for poll lookups. Needed because polls are not editable.
|
||||
val pollCardResolutionTotal = stats.counter("edit_control_builder_card_resolution", "total")
|
||||
val pollCardResolutionFailure =
|
||||
stats.counter("edit_control_builder_card_resolution", "failures")
|
||||
// Edit of initial tweet requested, and all edit checks successful.
|
||||
val initialEditTweet = stats.counter("edit_control_builder_initial_edit")
|
||||
request =>
|
||||
Stitch.run {
|
||||
request.editOptions match {
|
||||
case None =>
|
||||
val editControl =
|
||||
makeEditControlInitial(
|
||||
tweetId = request.tweetId,
|
||||
createdAt = request.createdAt,
|
||||
setEditWindowToSixtyMinutes = setEditWindowToSixtyMinutes
|
||||
).initial.copy(
|
||||
isEditEligible = Some(
|
||||
!isNullcastedButNotCommunityTweet(request)
|
||||
&& !isSuperFollow(request.tweet)
|
||||
&& !isCollabTweet(request.tweet)
|
||||
&& !isReplyToTweet(request.tweet)
|
||||
),
|
||||
)
|
||||
(editControl.isEditEligible, request.cardReference) match {
|
||||
case (Some(true), Some(reference)) =>
|
||||
pollCardResolutionTotal.incr()
|
||||
isPoll(
|
||||
card2Repo = card2Repo,
|
||||
cardReference = reference,
|
||||
cardsPlatformKey = request.cardsPlatformKey.getOrElse(defaultCardsPlatformKey),
|
||||
).rescue {
|
||||
// Revert to the assumed value if card cannot be resolved.
|
||||
case _ =>
|
||||
pollCardResolutionFailure.incr()
|
||||
Stitch.value(isPollCardAssumption)
|
||||
}
|
||||
.map { tweetIsAPoll =>
|
||||
wrapInitial(editControl.copy(isEditEligible = Some(!tweetIsAPoll)))
|
||||
}
|
||||
case _ => Stitch.value(wrapInitial(editControl))
|
||||
}
|
||||
case Some(editOptions) =>
|
||||
for {
|
||||
(previousTweet, _, _) <- Stitch.join(
|
||||
tweetRepo(editOptions.previousTweetId, editControlQueryOptions),
|
||||
checkPromotedTweet(
|
||||
editOptions.previousTweetId,
|
||||
promotedTweetRepo,
|
||||
disablePromotedTweetEdit),
|
||||
checkUserEligibility(
|
||||
authorId = request.authorId,
|
||||
matchedResults = request.matchedResults)
|
||||
)
|
||||
} yield {
|
||||
val initial = findEditControlInitial(previousTweet)
|
||||
checkAuthor(
|
||||
authorId = request.authorId,
|
||||
previousTweetAuthorId = getUserId(previousTweet))
|
||||
editTweetCount
|
||||
.get(initial.editTweetIds.size)
|
||||
.orElse(editTweetCount.get(EditControlUtil.maxTweetEditsAllowed))
|
||||
.foreach(counter => counter.incr())
|
||||
checkLatestEdit(previousTweet.id, initial)
|
||||
checkEditsRemaining(initial)
|
||||
checkEditTimeWindow(initial)
|
||||
checkIsEditEligible(initial)
|
||||
if (initial.editTweetIds == Seq(previousTweet.id)) {
|
||||
initialEditTweet.incr()
|
||||
}
|
||||
Some(editControlEdit(initialTweetId = initial.editTweetIds.head))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,137 @@
|
||||
package com.twitter.tweetypie
|
||||
package handler
|
||||
|
||||
import com.twitter.scrooge.schema.scrooge.scala.CompiledScroogeDefBuilder
|
||||
import com.twitter.scrooge.schema.scrooge.scala.CompiledScroogeValueExtractor
|
||||
import com.twitter.scrooge.schema.tree.DefinitionTraversal
|
||||
import com.twitter.scrooge.schema.tree.FieldPath
|
||||
import com.twitter.scrooge.schema.{ThriftDefinitions => DEF}
|
||||
import com.twitter.scrooge_internal.linter.known_annotations.AllowedAnnotationKeys.TweetEditAllowed
|
||||
import com.twitter.stitch.Stitch
|
||||
import com.twitter.tweetypie.core.TweetCreateFailure
|
||||
import com.twitter.tweetypie.repository.TweetQuery.Options
|
||||
import com.twitter.tweetypie.repository.TweetQuery
|
||||
import com.twitter.tweetypie.repository.TweetRepository
|
||||
import com.twitter.tweetypie.thriftscala.ConversationControl
|
||||
import com.twitter.tweetypie.thriftscala.TweetCreateState.FieldEditNotAllowed
|
||||
import com.twitter.tweetypie.thriftscala.TweetCreateState.InitialTweetNotFound
|
||||
import com.twitter.tweetypie.thriftscala.EditOptions
|
||||
import com.twitter.tweetypie.thriftscala.Tweet
|
||||
import com.twitter.util.Future
|
||||
import com.twitter.util.logging.Logger
|
||||
|
||||
/**
|
||||
* This class constructs a validator `Tweet => Future[Unit]` which
|
||||
* takes a new edit tweet and performs some validations. Specifically, it
|
||||
*
|
||||
* 1) ensures that no uneditable fields were edited. Uneditable fields are marked
|
||||
* on the tweet.thrift using the thrift annotation "tweetEditAllowed=false".
|
||||
* By default, fields with no annotation are treated as editable.
|
||||
*
|
||||
* 2) ensures that the conversationControl field (which is editable) remains the
|
||||
* same type, e.g. a ConversationControl.ByInvitation doesn't change to a
|
||||
* ConversationControl.Community.
|
||||
*
|
||||
* If either of these validations fail, the validator fails with a `FieldEditNotAllowed`
|
||||
* tweet create state.
|
||||
*/
|
||||
object EditValidator {
|
||||
type Type = (Tweet, Option[EditOptions]) => Future[Unit]
|
||||
|
||||
val log: Logger = Logger(getClass)
|
||||
|
||||
// An object that describes the tweet thrift, used to walk a tweet object looking
|
||||
// for annotated fields.
|
||||
val TweetDef = CompiledScroogeDefBuilder.build[Tweet].asInstanceOf[DEF.StructDef]
|
||||
|
||||
// Collect the `FieldPath` for any nested tweet field with a uneditable field annotation
|
||||
// that is set to false. These are the fields that this validator ensures cannot be edited.
|
||||
val uneditableFieldPaths: Seq[FieldPath] = {
|
||||
DefinitionTraversal().collect(TweetDef) {
|
||||
case (d: DEF.FieldDef, path) if (d.annotations.get(TweetEditAllowed).contains("false")) =>
|
||||
path
|
||||
}
|
||||
}
|
||||
|
||||
// A tweet query options which includes
|
||||
// - any top level tweet field which either is an uneditable field, or contains an uneditable
|
||||
// subfield.
|
||||
// - the conversationControl field
|
||||
// These fields must be present on the initial tweet in order for us to compare them against the
|
||||
// edit tweet.
|
||||
val previousTweetQueryOptions = {
|
||||
// A set of the top level field ids for each (potentially nested) uneditable field.
|
||||
val topLevelUneditableTweetFields = uneditableFieldPaths.map(_.ids.head).toSet
|
||||
Options(
|
||||
TweetQuery.Include(
|
||||
tweetFields = topLevelUneditableTweetFields + Tweet.ConversationControlField.id
|
||||
))
|
||||
}
|
||||
|
||||
def validateUneditableFields(previousTweet: Tweet, editTweet: Tweet): Unit = {
|
||||
// Collect uneditable fields that were edited
|
||||
val invalidEditedFields = uneditableFieldPaths.flatMap { fieldPath =>
|
||||
val previousValue =
|
||||
FieldPath.lensGet(CompiledScroogeValueExtractor, previousTweet, fieldPath)
|
||||
val editValue = FieldPath.lensGet(CompiledScroogeValueExtractor, editTweet, fieldPath)
|
||||
|
||||
if (previousValue != editValue) {
|
||||
Some(fieldPath.toString)
|
||||
} else {
|
||||
None
|
||||
}
|
||||
}
|
||||
|
||||
if (invalidEditedFields.nonEmpty) {
|
||||
// If any inequalities are found, log them and return an exception.
|
||||
val msg = "uneditable fields were edited: " + invalidEditedFields.mkString(",")
|
||||
log.error(msg)
|
||||
throw TweetCreateFailure.State(FieldEditNotAllowed, Some(msg))
|
||||
}
|
||||
}
|
||||
|
||||
def validateConversationControl(
|
||||
previous: Option[ConversationControl],
|
||||
edit: Option[ConversationControl]
|
||||
): Unit = {
|
||||
import ConversationControl.ByInvitation
|
||||
import ConversationControl.Community
|
||||
import ConversationControl.Followers
|
||||
|
||||
(previous, edit) match {
|
||||
case (None, None) => ()
|
||||
case (Some(ByInvitation(_)), Some(ByInvitation(_))) => ()
|
||||
case (Some(Community(_)), Some(Community(_))) => ()
|
||||
case (Some(Followers(_)), Some(Followers(_))) => ()
|
||||
case (_, _) =>
|
||||
val msg = "conversationControl type was edited"
|
||||
log.error(msg)
|
||||
throw TweetCreateFailure.State(FieldEditNotAllowed, Some(msg))
|
||||
}
|
||||
}
|
||||
|
||||
def apply(tweetRepo: TweetRepository.Optional): Type = { (tweet, editOptions) =>
|
||||
Stitch.run(
|
||||
editOptions match {
|
||||
case Some(EditOptions(previousTweetId)) => {
|
||||
// Query for the previous tweet so that we can compare the
|
||||
// fields between the two tweets.
|
||||
tweetRepo(previousTweetId, previousTweetQueryOptions).map {
|
||||
case Some(previousTweet) =>
|
||||
validateUneditableFields(previousTweet, tweet)
|
||||
validateConversationControl(
|
||||
previousTweet.conversationControl,
|
||||
tweet.conversationControl)
|
||||
case _ =>
|
||||
// If the previous tweet is not found we cannot perform validations that
|
||||
// compare tweet fields and we have to fail tweet creation.
|
||||
throw TweetCreateFailure.State(InitialTweetNotFound)
|
||||
}
|
||||
}
|
||||
// This is the case where this isn't an edit tweet (since editOptions = None)
|
||||
// Since this tweet is not an edit there are no fields to validate.
|
||||
case _ => Stitch.Unit
|
||||
}
|
||||
)
|
||||
}
|
||||
}
|
@ -0,0 +1,102 @@
|
||||
package com.twitter.tweetypie
|
||||
package handler
|
||||
|
||||
import com.twitter.finagle.stats.Stat
|
||||
import com.twitter.flockdb.client._
|
||||
import com.twitter.servo.util.FutureArrow
|
||||
import com.twitter.tweetypie.thriftscala._
|
||||
|
||||
trait EraseUserTweetsHandler {
|
||||
|
||||
val eraseUserTweetsRequest: FutureArrow[EraseUserTweetsRequest, Unit]
|
||||
|
||||
val asyncEraseUserTweetsRequest: FutureArrow[AsyncEraseUserTweetsRequest, Unit]
|
||||
}
|
||||
|
||||
/**
|
||||
* This library allows you to erase all of a users's tweets. It's used to clean up
|
||||
* tweets after a user deletes their account.
|
||||
*/
|
||||
object EraseUserTweetsHandler {
|
||||
|
||||
/**
|
||||
* Build a FutureEffect which, when called, deletes one page worth of tweets at the
|
||||
* specified flock cursor. When the page of tweets has been deleted another asyncEraseUserTweets
|
||||
* request is made with the updated cursor location so that the next page of tweets can be processed.
|
||||
*/
|
||||
def apply(
|
||||
selectPage: FutureArrow[Select[StatusGraph], PageResult[Long]],
|
||||
deleteTweet: FutureEffect[(TweetId, UserId)],
|
||||
asyncEraseUserTweets: FutureArrow[AsyncEraseUserTweetsRequest, Unit],
|
||||
stats: StatsReceiver,
|
||||
sleep: () => Future[Unit] = () => Future.Unit
|
||||
): EraseUserTweetsHandler =
|
||||
new EraseUserTweetsHandler {
|
||||
val latencyStat: Stat = stats.stat("latency_ms")
|
||||
val deletedTweetsStat: Stat = stats.stat("tweets_deleted_for_erased_user")
|
||||
|
||||
val selectUserTweets: AsyncEraseUserTweetsRequest => Select[StatusGraph] =
|
||||
(request: AsyncEraseUserTweetsRequest) =>
|
||||
UserTimelineGraph
|
||||
.from(request.userId)
|
||||
.withCursor(Cursor(request.flockCursor))
|
||||
|
||||
// For a provided list of tweetIds, delete each one sequentially, sleeping between each call
|
||||
// This is a rate limiting mechanism to slow down deletions.
|
||||
def deletePage(page: PageResult[Long], expectedUserId: UserId): Future[Unit] =
|
||||
page.entries.foldLeft(Future.Unit) { (previousFuture, nextId) =>
|
||||
for {
|
||||
_ <- previousFuture
|
||||
_ <- sleep()
|
||||
_ <- deleteTweet((nextId, expectedUserId))
|
||||
} yield ()
|
||||
}
|
||||
|
||||
/**
|
||||
* If we aren't on the last page, make another EraseUserTweets request to delete
|
||||
* the next page of tweets
|
||||
*/
|
||||
val nextRequestOrEnd: (AsyncEraseUserTweetsRequest, PageResult[Long]) => Future[Unit] =
|
||||
(request: AsyncEraseUserTweetsRequest, page: PageResult[Long]) =>
|
||||
if (page.nextCursor.isEnd) {
|
||||
latencyStat.add(Time.fromMilliseconds(request.startTimestamp).untilNow.inMillis)
|
||||
deletedTweetsStat.add(request.tweetCount + page.entries.size)
|
||||
Future.Unit
|
||||
} else {
|
||||
asyncEraseUserTweets(
|
||||
request.copy(
|
||||
flockCursor = page.nextCursor.value,
|
||||
tweetCount = request.tweetCount + page.entries.size
|
||||
)
|
||||
)
|
||||
}
|
||||
|
||||
override val eraseUserTweetsRequest: FutureArrow[EraseUserTweetsRequest, Unit] =
|
||||
FutureArrow { request =>
|
||||
asyncEraseUserTweets(
|
||||
AsyncEraseUserTweetsRequest(
|
||||
userId = request.userId,
|
||||
flockCursor = Cursor.start.value,
|
||||
startTimestamp = Time.now.inMillis,
|
||||
tweetCount = 0L
|
||||
)
|
||||
)
|
||||
}
|
||||
|
||||
override val asyncEraseUserTweetsRequest: FutureArrow[AsyncEraseUserTweetsRequest, Unit] =
|
||||
FutureArrow { request =>
|
||||
for {
|
||||
_ <- sleep()
|
||||
|
||||
// get one page of tweets
|
||||
page <- selectPage(selectUserTweets(request))
|
||||
|
||||
// delete tweets
|
||||
_ <- deletePage(page, request.userId)
|
||||
|
||||
// make call to delete the next page of tweets
|
||||
_ <- nextRequestOrEnd(request, page)
|
||||
} yield ()
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,137 @@
|
||||
package com.twitter.tweetypie
|
||||
package handler
|
||||
|
||||
import com.twitter.finagle.stats.Counter
|
||||
import com.twitter.stitch.Stitch
|
||||
import com.twitter.tweetypie.repository.PlaceKey
|
||||
import com.twitter.tweetypie.repository.PlaceRepository
|
||||
import com.twitter.tweetypie.serverutil.ExceptionCounter
|
||||
import com.twitter.tweetypie.thriftscala._
|
||||
|
||||
object GeoStats {
|
||||
val topTenCountryCodes: Set[PlaceLanguage] =
|
||||
Set("US", "JP", "GB", "ID", "BR", "SA", "TR", "MX", "ES", "CA")
|
||||
|
||||
def apply(stats: StatsReceiver): Effect[Option[Place]] = {
|
||||
val totalCount = stats.counter("total")
|
||||
val notFoundCount = stats.counter("not_found")
|
||||
val countryStats: Map[String, Counter] =
|
||||
topTenCountryCodes.map(cc => cc -> stats.scope("with_country_code").counter(cc)).toMap
|
||||
|
||||
val placeTypeStats: Map[PlaceType, Counter] =
|
||||
Map(
|
||||
PlaceType.Admin -> stats.counter("admin"),
|
||||
PlaceType.City -> stats.counter("city"),
|
||||
PlaceType.Country -> stats.counter("country"),
|
||||
PlaceType.Neighborhood -> stats.counter("neighborhood"),
|
||||
PlaceType.Poi -> stats.counter("poi"),
|
||||
PlaceType.Unknown -> stats.counter("unknown")
|
||||
)
|
||||
|
||||
Effect.fromPartial {
|
||||
case Some(place) => {
|
||||
totalCount.incr()
|
||||
placeTypeStats(place.`type`).incr()
|
||||
place.countryCode.foreach(cc => countryStats.get(cc).foreach(_.incr()))
|
||||
}
|
||||
case None => notFoundCount.incr()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object GeoBuilder {
|
||||
case class Request(createGeo: TweetCreateGeo, userGeoEnabled: Boolean, language: String)
|
||||
|
||||
case class Result(geoCoordinates: Option[GeoCoordinates], placeId: Option[PlaceId])
|
||||
|
||||
type Type = FutureArrow[Request, Result]
|
||||
|
||||
def apply(placeRepo: PlaceRepository.Type, rgc: ReverseGeocoder, stats: StatsReceiver): Type = {
|
||||
val exceptionCounters = ExceptionCounter(stats)
|
||||
|
||||
def ignoreFailures[A](future: Future[Option[A]]): Future[Option[A]] =
|
||||
exceptionCounters(future).handle { case _ => None }
|
||||
|
||||
def isValidPlaceId(placeId: String) = PlaceIdRegex.pattern.matcher(placeId).matches
|
||||
|
||||
def isValidLatLon(latitude: Double, longitude: Double): Boolean =
|
||||
latitude >= -90.0 && latitude <= 90.0 &&
|
||||
longitude >= -180.0 && longitude <= 180.0 &&
|
||||
// some clients send (0.0, 0.0) for unknown reasons, but this is highly unlikely to be
|
||||
// valid and should be treated as if no coordinates were sent. if a place Id is provided,
|
||||
// that will still be used.
|
||||
(latitude != 0.0 || longitude != 0.0)
|
||||
|
||||
// Count the number of times we erase geo information based on user preferences.
|
||||
val geoErasedCounter = stats.counter("geo_erased")
|
||||
// Count the number of times we override a user's preferences and add geo anyway.
|
||||
val geoOverriddenCounter = stats.counter("geo_overridden")
|
||||
|
||||
val geoScope = stats.scope("create_geotagged_tweet")
|
||||
|
||||
// Counter for geo tweets with neither lat lon nor place id data
|
||||
val noGeoCounter = geoScope.counter("no_geo_info")
|
||||
val invalidCoordinates = geoScope.counter("invalid_coordinates")
|
||||
val inValidPlaceId = geoScope.counter("invalid_place_id")
|
||||
val latlonStatsEffect = GeoStats(geoScope.scope("from_latlon"))
|
||||
val placeIdStatsEffect = GeoStats(geoScope.scope("from_place_id"))
|
||||
|
||||
def validateCoordinates(coords: GeoCoordinates): Option[GeoCoordinates] =
|
||||
if (isValidLatLon(coords.latitude, coords.longitude)) Some(coords)
|
||||
else {
|
||||
invalidCoordinates.incr()
|
||||
None
|
||||
}
|
||||
|
||||
def validatePlaceId(placeId: String): Option[String] =
|
||||
if (isValidPlaceId(placeId)) Some(placeId)
|
||||
else {
|
||||
inValidPlaceId.incr()
|
||||
None
|
||||
}
|
||||
|
||||
def getPlaceByRGC(coordinates: GeoCoordinates, language: String): Future[Option[Place]] =
|
||||
ignoreFailures(
|
||||
rgc((coordinates, language)).onSuccess(latlonStatsEffect)
|
||||
)
|
||||
|
||||
def getPlaceById(placeId: String, language: String): Future[Option[Place]] =
|
||||
ignoreFailures(
|
||||
Stitch
|
||||
.run(placeRepo(PlaceKey(placeId, language)).liftNotFoundToOption)
|
||||
.onSuccess(placeIdStatsEffect)
|
||||
)
|
||||
|
||||
FutureArrow[Request, Result] { request =>
|
||||
val createGeo = request.createGeo
|
||||
val allowGeo = createGeo.overrideUserGeoSetting || request.userGeoEnabled
|
||||
val overrideGeo = createGeo.overrideUserGeoSetting && !request.userGeoEnabled
|
||||
|
||||
if (createGeo.placeId.isEmpty && createGeo.coordinates.isEmpty) {
|
||||
noGeoCounter.incr()
|
||||
Future.value(Result(None, None))
|
||||
} else if (!allowGeo) {
|
||||
// Record that we had geo information but had to erase it based on user preferences.
|
||||
geoErasedCounter.incr()
|
||||
Future.value(Result(None, None))
|
||||
} else {
|
||||
if (overrideGeo) geoOverriddenCounter.incr()
|
||||
|
||||
// treat invalidate coordinates the same as no-coordinates
|
||||
val validatedCoordinates = createGeo.coordinates.flatMap(validateCoordinates)
|
||||
val validatedPlaceId = createGeo.placeId.flatMap(validatePlaceId)
|
||||
|
||||
for {
|
||||
place <- (createGeo.placeId, validatedPlaceId, validatedCoordinates) match {
|
||||
// if the request contains an invalid place id, we want to return None for the
|
||||
// place instead of reverse-geocoding the coordinates
|
||||
case (Some(_), None, _) => Future.None
|
||||
case (_, Some(placeId), _) => getPlaceById(placeId, request.language)
|
||||
case (_, _, Some(coords)) => getPlaceByRGC(coords, request.language)
|
||||
case _ => Future.None
|
||||
}
|
||||
} yield Result(validatedCoordinates, place.map(_.id))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
Some files were not shown because too many files have changed in this diff Show More
Reference in New Issue
Block a user