Open-sourcing Tweetypie

Tweetypie is the core Tweet service that handles the reading and writing of Tweet data.
This commit is contained in:
twitter-team
2023-05-12 09:11:38 -07:00
parent 90d7ea370e
commit 01dbfee4c0
591 changed files with 68352 additions and 0 deletions

View 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",
],
)

View File

@ -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",
],
)

View File

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

View File

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

View File

@ -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]
}

View File

@ -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
}

View File

@ -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
}

View File

@ -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
}

View File

@ -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]
}

View File

@ -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
}

View File

@ -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
}

View File

@ -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 }
}

View File

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

View File

@ -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
}

View File

@ -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]
}

View File

@ -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]
}

View File

@ -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}")
}
}

View File

@ -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]
}

View File

@ -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]
}

View File

@ -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
}

View File

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

View File

@ -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",
],
)

View File

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

View File

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

View File

@ -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
}
}

View File

@ -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)
}),
)
}

View File

@ -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
}
}
}

View File

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

View File

@ -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
}
}
}

View File

@ -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

View File

@ -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(",")
}

View File

@ -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"))
}

View File

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

View File

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

View File

@ -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")
)
}
}
}

View File

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

View File

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

View File

@ -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
}
}
}

View File

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

View File

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

View File

@ -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
}
}
}

View File

@ -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")
}

View File

@ -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")
}

View File

@ -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
}

View File

@ -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",
],
)

View File

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

View File

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

View File

@ -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

View File

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

View File

@ -0,0 +1,3 @@
package com.twitter.tweetypie.core
case class GeoSearchRequestId(requestID: String)

View File

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

View File

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

View File

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

View File

@ -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
}
}

View File

@ -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)"
}
}

View File

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

View File

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

View File

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

View File

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

View File

@ -0,0 +1,5 @@
package com.twitter.tweetypie
package object core {
type TweetResult = ValueState[TweetData]
}

View File

@ -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",
],
)

View File

@ -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
}: _*
))
}
}

View File

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

View File

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

View File

@ -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",
],
)

View File

@ -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
}
}

View File

@ -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
}
}

View File

@ -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"
}

View File

@ -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"
}
}

View File

@ -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}"))
}
}
}
}
}

View File

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

View File

@ -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"
}

View File

@ -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"
}

View File

@ -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"
}

View File

@ -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,
)
}

View File

@ -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
}
}
}
}

View File

@ -0,0 +1,11 @@
package com.twitter.tweetypie.federated.columns
import com.twitter.strato.config.ContactInfo
object TweetypieContactInfo
extends ContactInfo(
contactEmail = "",
ldapGroup = "",
jiraProject = "",
slackRoomId = ""
)

View File

@ -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"
}

View File

@ -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",
],
)

View File

@ -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,
)
}
}

View File

@ -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",
],
)

View File

@ -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")
)
}
}

View File

@ -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",
],
)

View File

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

View File

@ -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",
],
)

View File

@ -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
}
}

View File

@ -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
}
}
}

View File

@ -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",
],
)

View File

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

View File

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

View File

@ -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
}
}
}

View File

@ -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
}
}
}

View File

@ -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
}
}
}
}

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -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 ()
}
}
}

View File

@ -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