mirror of
https://github.com/twitter/the-algorithm.git
synced 2024-12-18 02:41:37 +03:00
Open-sourcing Tweetypie
Tweetypie is the core Tweet service that handles the reading and writing of Tweet data.
This commit is contained in:
parent
90d7ea370e
commit
01dbfee4c0
@ -24,6 +24,7 @@ Product surfaces at Twitter are built on a shared set of data, models, and softw
|
|||||||
| | [timelines-aggregation-framework](timelines/data_processing/ml_util/aggregation_framework/README.md) | Framework for generating aggregate features in batch or real time. |
|
| | [timelines-aggregation-framework](timelines/data_processing/ml_util/aggregation_framework/README.md) | Framework for generating aggregate features in batch or real time. |
|
||||||
| | [representation-manager](representation-manager/README.md) | Service to retrieve embeddings (i.e. SimClusers and TwHIN). |
|
| | [representation-manager](representation-manager/README.md) | Service to retrieve embeddings (i.e. SimClusers and TwHIN). |
|
||||||
| | [twml](twml/README.md) | Legacy machine learning framework built on TensorFlow v1. |
|
| | [twml](twml/README.md) | Legacy machine learning framework built on TensorFlow v1. |
|
||||||
|
| | [Tweetypie](tweetypie/server/README.md) | Core Tweet service that handles the reading and writing of Tweet data. |
|
||||||
|
|
||||||
The product surface currently included in this repository is the For You Timeline.
|
The product surface currently included in this repository is the For You Timeline.
|
||||||
|
|
||||||
|
@ -0,0 +1,118 @@
|
|||||||
|
package com.twitter.tweetypie.additionalfields
|
||||||
|
|
||||||
|
import com.twitter.tweetypie.thriftscala.Tweet
|
||||||
|
import com.twitter.scrooge.TFieldBlob
|
||||||
|
import com.twitter.scrooge.ThriftStructField
|
||||||
|
|
||||||
|
object AdditionalFields {
|
||||||
|
type FieldId = Short
|
||||||
|
|
||||||
|
/** additional fields really start at 100, be we are ignoring conversation id for now */
|
||||||
|
val StartAdditionalId = 101
|
||||||
|
|
||||||
|
/** all known [[Tweet]] field IDs */
|
||||||
|
val CompiledFieldIds: Seq[FieldId] = Tweet.metaData.fields.map(_.id)
|
||||||
|
|
||||||
|
/** all known [[Tweet]] fields in the "additional-field" range (excludes id) */
|
||||||
|
val CompiledAdditionalFieldMetaDatas: Seq[ThriftStructField[Tweet]] =
|
||||||
|
Tweet.metaData.fields.filter(f => isAdditionalFieldId(f.id))
|
||||||
|
|
||||||
|
val CompiledAdditionalFieldsMap: Map[Short, ThriftStructField[Tweet]] =
|
||||||
|
CompiledAdditionalFieldMetaDatas.map(field => (field.id, field)).toMap
|
||||||
|
|
||||||
|
/** all known [[Tweet]] field IDs in the "additional-field" range */
|
||||||
|
val CompiledAdditionalFieldIds: Seq[FieldId] =
|
||||||
|
CompiledAdditionalFieldsMap.keys.toSeq
|
||||||
|
|
||||||
|
/** all [[Tweet]] field IDs which should be rejected when set as additional
|
||||||
|
* fields on via PostTweetRequest.additionalFields or RetweetRequest.additionalFields */
|
||||||
|
val RejectedFieldIds: Seq[FieldId] = Seq(
|
||||||
|
// Should be provided via PostTweetRequest.conversationControl field. go/convocontrolsbackend
|
||||||
|
Tweet.ConversationControlField.id,
|
||||||
|
// This field should only be set based on whether the client sets the right community
|
||||||
|
// tweet annotation.
|
||||||
|
Tweet.CommunitiesField.id,
|
||||||
|
// This field should not be set by clients and should opt for
|
||||||
|
// [[PostTweetRequest.ExclusiveTweetControlOptions]].
|
||||||
|
// The exclusiveTweetControl field requires the userId to be set
|
||||||
|
// and we shouldn't trust the client to provide the right one.
|
||||||
|
Tweet.ExclusiveTweetControlField.id,
|
||||||
|
// This field should not be set by clients and should opt for
|
||||||
|
// [[PostTweetRequest.TrustedFriendsControlOptions]].
|
||||||
|
// The trustedFriendsControl field requires the trustedFriendsListId to be
|
||||||
|
// set and we shouldn't trust the client to provide the right one.
|
||||||
|
Tweet.TrustedFriendsControlField.id,
|
||||||
|
// This field should not be set by clients and should opt for
|
||||||
|
// [[PostTweetRequest.CollabControlOptions]].
|
||||||
|
// The collabControl field requires a list of Collaborators to be
|
||||||
|
// set and we shouldn't trust the client to provide the right one.
|
||||||
|
Tweet.CollabControlField.id
|
||||||
|
)
|
||||||
|
|
||||||
|
def isAdditionalFieldId(fieldId: FieldId): Boolean =
|
||||||
|
fieldId >= StartAdditionalId
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Provides a list of all additional field IDs on the tweet, which include all
|
||||||
|
* the compiled additional fields and all the provided passthrough fields. This includes
|
||||||
|
* compiled additional fields where the value is None.
|
||||||
|
*/
|
||||||
|
def allAdditionalFieldIds(tweet: Tweet): Seq[FieldId] =
|
||||||
|
CompiledAdditionalFieldIds ++ tweet._passthroughFields.keys
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Provides a list of all field IDs that have a value on the tweet which are not known compiled
|
||||||
|
* additional fields (excludes [[Tweet.id]]).
|
||||||
|
*/
|
||||||
|
def unsettableAdditionalFieldIds(tweet: Tweet): Seq[FieldId] =
|
||||||
|
CompiledFieldIds
|
||||||
|
.filter { id =>
|
||||||
|
!isAdditionalFieldId(id) && id != Tweet.IdField.id && tweet.getFieldBlob(id).isDefined
|
||||||
|
} ++
|
||||||
|
tweet._passthroughFields.keys
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Provides a list of all field IDs that have a value on the tweet which are explicitly disallowed
|
||||||
|
* from being set via PostTweetRequest.additionalFields and RetweetRequest.additionalFields
|
||||||
|
*/
|
||||||
|
def rejectedAdditionalFieldIds(tweet: Tweet): Seq[FieldId] =
|
||||||
|
RejectedFieldIds
|
||||||
|
.filter { id => tweet.getFieldBlob(id).isDefined }
|
||||||
|
|
||||||
|
def unsettableAdditionalFieldIdsErrorMessage(unsettableFieldIds: Seq[FieldId]): String =
|
||||||
|
s"request may not contain fields: [${unsettableFieldIds.sorted.mkString(", ")}]"
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Provides a list of all additional field IDs that have a value on the tweet,
|
||||||
|
* compiled and passthrough (excludes Tweet.id).
|
||||||
|
*/
|
||||||
|
def nonEmptyAdditionalFieldIds(tweet: Tweet): Seq[FieldId] =
|
||||||
|
CompiledAdditionalFieldMetaDatas.collect {
|
||||||
|
case f if f.getValue(tweet) != None => f.id
|
||||||
|
} ++ tweet._passthroughFields.keys
|
||||||
|
|
||||||
|
def additionalFields(tweet: Tweet): Seq[TFieldBlob] =
|
||||||
|
(tweet.getFieldBlobs(CompiledAdditionalFieldIds) ++ tweet._passthroughFields).values.toSeq
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Merge base tweet with additional fields.
|
||||||
|
* Non-additional fields in the additional tweet are ignored.
|
||||||
|
* @param base: a tweet that contains basic fields
|
||||||
|
* @param additional: a tweet object that carries additional fields
|
||||||
|
*/
|
||||||
|
def setAdditionalFields(base: Tweet, additional: Tweet): Tweet =
|
||||||
|
setAdditionalFields(base, additionalFields(additional))
|
||||||
|
|
||||||
|
def setAdditionalFields(base: Tweet, additional: Option[Tweet]): Tweet =
|
||||||
|
additional.map(setAdditionalFields(base, _)).getOrElse(base)
|
||||||
|
|
||||||
|
def setAdditionalFields(base: Tweet, additional: Traversable[TFieldBlob]): Tweet =
|
||||||
|
additional.foldLeft(base) { case (t, f) => t.setField(f) }
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Unsets the specified fields on the given tweet.
|
||||||
|
*/
|
||||||
|
def unsetFields(tweet: Tweet, fieldIds: Iterable[FieldId]): Tweet = {
|
||||||
|
tweet.unsetFields(fieldIds.toSet)
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,15 @@
|
|||||||
|
scala_library(
|
||||||
|
sources = ["*.scala"],
|
||||||
|
compiler_option_sets = ["fatal_warnings"],
|
||||||
|
platform = "java8",
|
||||||
|
strict_deps = True,
|
||||||
|
tags = ["bazel-compatible"],
|
||||||
|
dependencies = [
|
||||||
|
"3rdparty/jvm/org/apache/thrift:libthrift",
|
||||||
|
"mediaservices/commons/src/main/thrift:thrift-scala",
|
||||||
|
"scrooge/scrooge-core",
|
||||||
|
"src/thrift/com/twitter/escherbird:media-annotation-structs-scala",
|
||||||
|
"src/thrift/com/twitter/spam/rtf:safety-label-scala",
|
||||||
|
"tweetypie/common/src/thrift/com/twitter/tweetypie:tweet-scala",
|
||||||
|
],
|
||||||
|
)
|
@ -0,0 +1,15 @@
|
|||||||
|
scala_library(
|
||||||
|
compiler_option_sets = ["fatal_warnings"],
|
||||||
|
strict_deps = True,
|
||||||
|
tags = ["bazel-compatible"],
|
||||||
|
dependencies = [
|
||||||
|
"finagle/finagle-memcached/src/main/scala",
|
||||||
|
"scrooge/scrooge-serializer",
|
||||||
|
"stitch/stitch-core",
|
||||||
|
"util/util-core",
|
||||||
|
"util/util-logging",
|
||||||
|
# CachedValue struct
|
||||||
|
"tweetypie/servo/repo/src/main/thrift:thrift-scala",
|
||||||
|
"util/util-slf4j-api/src/main/scala/com/twitter/util/logging",
|
||||||
|
],
|
||||||
|
)
|
@ -0,0 +1,241 @@
|
|||||||
|
package com.twitter.tweetypie.caching
|
||||||
|
|
||||||
|
import com.twitter.finagle.service.StatsFilter
|
||||||
|
import com.twitter.finagle.stats.StatsReceiver
|
||||||
|
import com.twitter.finagle.stats.ExceptionStatsHandler
|
||||||
|
import com.twitter.finagle.stats.Counter
|
||||||
|
import com.twitter.util.Future
|
||||||
|
import com.twitter.util.logging.Logger
|
||||||
|
import com.twitter.finagle.memcached
|
||||||
|
import scala.util.control.NonFatal
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Wrapper around a memcached client that performs serialization and
|
||||||
|
* deserialization, tracks stats, provides tracing, and provides
|
||||||
|
* per-key fresh/stale/failure/miss results.
|
||||||
|
*
|
||||||
|
* The operations that write values to cache will only write values
|
||||||
|
* that the ValueSerializer says are cacheable. The idea here is that
|
||||||
|
* the deserialize and serialize functions must be coherent, and no
|
||||||
|
* matter how you choose to write these values back to cache, the
|
||||||
|
* serializer will have the appropriate knowledge about whether the
|
||||||
|
* values are cacheable.
|
||||||
|
*
|
||||||
|
* For most cases, you will want to use [[StitchCaching]] rather than
|
||||||
|
* calling this wrapper directly.
|
||||||
|
*
|
||||||
|
* @param keySerializer How to convert a K value to a memcached key.
|
||||||
|
*
|
||||||
|
* @param valueSerializer How to serialize and deserialize V values,
|
||||||
|
* as well as which values are cacheable, and how long to store the
|
||||||
|
* values in cache.
|
||||||
|
*/
|
||||||
|
class CacheOperations[K, V](
|
||||||
|
keySerializer: K => String,
|
||||||
|
valueSerializer: ValueSerializer[V],
|
||||||
|
memcachedClient: memcached.Client,
|
||||||
|
statsReceiver: StatsReceiver,
|
||||||
|
logger: Logger,
|
||||||
|
exceptionStatsHandler: ExceptionStatsHandler = StatsFilter.DefaultExceptions) {
|
||||||
|
// The memcached operations that are performed via this
|
||||||
|
// [[CacheOperations]] instance will be tracked under this stats
|
||||||
|
// receiver.
|
||||||
|
//
|
||||||
|
// We count all memcached failures together under this scope,
|
||||||
|
// because memcached operations should not fail unless there are
|
||||||
|
// communication problems, so differentiating the method that was
|
||||||
|
// being called will not give us any useful information.
|
||||||
|
private[this] val memcachedStats: StatsReceiver = statsReceiver.scope("memcached")
|
||||||
|
|
||||||
|
// Incremented for every attempt to `get` a key from cache.
|
||||||
|
private[this] val memcachedGetCounter: Counter = memcachedStats.counter("get")
|
||||||
|
|
||||||
|
// One of these two counters is incremented for every successful
|
||||||
|
// response returned from a `get` call to memcached.
|
||||||
|
private[this] val memcachedNotFoundCounter: Counter = memcachedStats.counter("not_found")
|
||||||
|
private[this] val memcachedFoundCounter: Counter = memcachedStats.counter("found")
|
||||||
|
|
||||||
|
// Records the state of the cache load after serialization. The
|
||||||
|
// policy may transform a value that was successfully loaded from
|
||||||
|
// cache into any result type, which is why we explicitly track
|
||||||
|
// "found" and "not_found" above. If `stale` + `fresh` is not equal
|
||||||
|
// to `found`, then it means that the policy has translated a found
|
||||||
|
// value into a miss or failure. The policy may do this in order to
|
||||||
|
// cause the caching filter to treat the value that was found in
|
||||||
|
// cache in the way it would have treated a miss or failure from
|
||||||
|
// cache.
|
||||||
|
private[this] val resultStats: StatsReceiver = statsReceiver.scope("result")
|
||||||
|
private[this] val resultFreshCounter: Counter = resultStats.counter("fresh")
|
||||||
|
private[this] val resultStaleCounter: Counter = resultStats.counter("stale")
|
||||||
|
private[this] val resultMissCounter: Counter = resultStats.counter("miss")
|
||||||
|
private[this] val resultFailureCounter: Counter = resultStats.counter("failure")
|
||||||
|
|
||||||
|
// Used for recording exceptions that occurred during
|
||||||
|
// deserialization. This will never be incremented if the
|
||||||
|
// deserializer returns a result, even if the result is a
|
||||||
|
// [[CacheResult.Failure]]. See the comment where this stat is
|
||||||
|
// incremented for more details.
|
||||||
|
private[this] val deserializeFailureStats: StatsReceiver = statsReceiver.scope("deserialize")
|
||||||
|
|
||||||
|
private[this] val notSerializedCounter: Counter = statsReceiver.counter("not_serialized")
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Load a batch of values from cache. Mostly this deals with
|
||||||
|
* converting the [[memcached.GetResult]] to a
|
||||||
|
* [[Seq[CachedResult[V]]]]. The result is in the same order as the
|
||||||
|
* keys, and there will always be an entry for each key. This method
|
||||||
|
* should never return a [[Future.exception]].
|
||||||
|
*/
|
||||||
|
def get(keys: Seq[K]): Future[Seq[CacheResult[V]]] = {
|
||||||
|
memcachedGetCounter.incr(keys.size)
|
||||||
|
val cacheKeys: Seq[String] = keys.map(keySerializer)
|
||||||
|
if (logger.isTraceEnabled) {
|
||||||
|
logger.trace {
|
||||||
|
val lines: Seq[String] = keys.zip(cacheKeys).map { case (k, c) => s"\n $k ($c)" }
|
||||||
|
"Starting load for keys:" + lines.mkString
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
memcachedClient
|
||||||
|
.getResult(cacheKeys)
|
||||||
|
.map { getResult =>
|
||||||
|
memcachedNotFoundCounter.incr(getResult.misses.size)
|
||||||
|
val results: Seq[CacheResult[V]] =
|
||||||
|
cacheKeys.map { cacheKey =>
|
||||||
|
val result: CacheResult[V] =
|
||||||
|
getResult.hits.get(cacheKey) match {
|
||||||
|
case Some(memcachedValue) =>
|
||||||
|
memcachedFoundCounter.incr()
|
||||||
|
try {
|
||||||
|
valueSerializer.deserialize(memcachedValue.value)
|
||||||
|
} catch {
|
||||||
|
case NonFatal(e) =>
|
||||||
|
// If the serializer throws an exception, then
|
||||||
|
// the serialized value was malformed. In that
|
||||||
|
// case, we record the failure so that it can be
|
||||||
|
// detected and fixed, but treat it as a cache
|
||||||
|
// miss. The reason that we treat it as a miss
|
||||||
|
// rather than a failure is that a miss will
|
||||||
|
// cause a write back to cache, and we want to
|
||||||
|
// write a valid result back to cache to replace
|
||||||
|
// the bad entry that we just loaded.
|
||||||
|
//
|
||||||
|
// A serializer is free to return Miss itself to
|
||||||
|
// obtain this behavior if it is expected or
|
||||||
|
// desired, to avoid the logging and stats (and
|
||||||
|
// the minor overhead of catching an exception).
|
||||||
|
//
|
||||||
|
// The exceptions are tracked separately from
|
||||||
|
// other exceptions so that it is easy to see
|
||||||
|
// whether the deserializer itself ever throws an
|
||||||
|
// exception.
|
||||||
|
exceptionStatsHandler.record(deserializeFailureStats, e)
|
||||||
|
logger.warn(s"Failed deserializing value for cache key $cacheKey", e)
|
||||||
|
CacheResult.Miss
|
||||||
|
}
|
||||||
|
|
||||||
|
case None if getResult.misses.contains(cacheKey) =>
|
||||||
|
CacheResult.Miss
|
||||||
|
|
||||||
|
case None =>
|
||||||
|
val exception =
|
||||||
|
getResult.failures.get(cacheKey) match {
|
||||||
|
case None =>
|
||||||
|
// To get here, this was not a hit or a miss,
|
||||||
|
// so we expect the key to be present in
|
||||||
|
// failures. If it is not, then either the
|
||||||
|
// contract of getResult was violated, or this
|
||||||
|
// method is somehow attempting to access a
|
||||||
|
// result for a key that was not
|
||||||
|
// loaded. Either of these indicates a bug, so
|
||||||
|
// we log a high priority log message.
|
||||||
|
logger.error(
|
||||||
|
s"Key $cacheKey not found in hits, misses or failures. " +
|
||||||
|
"This indicates a bug in the memcached library or " +
|
||||||
|
"CacheOperations.load"
|
||||||
|
)
|
||||||
|
// We return this as a failure because that
|
||||||
|
// will cause the repo to be consulted and the
|
||||||
|
// value *not* to be written back to cache,
|
||||||
|
// which is probably the safest thing to do
|
||||||
|
// (if we don't know what's going on, default
|
||||||
|
// to an uncached repo).
|
||||||
|
new IllegalStateException
|
||||||
|
|
||||||
|
case Some(e) =>
|
||||||
|
e
|
||||||
|
}
|
||||||
|
exceptionStatsHandler.record(memcachedStats, exception)
|
||||||
|
CacheResult.Failure(exception)
|
||||||
|
}
|
||||||
|
|
||||||
|
// Count each kind of CacheResult, to make it possible to
|
||||||
|
// see how effective the caching is.
|
||||||
|
result match {
|
||||||
|
case CacheResult.Fresh(_) => resultFreshCounter.incr()
|
||||||
|
case CacheResult.Stale(_) => resultStaleCounter.incr()
|
||||||
|
case CacheResult.Miss => resultMissCounter.incr()
|
||||||
|
case CacheResult.Failure(_) => resultFailureCounter.incr()
|
||||||
|
}
|
||||||
|
|
||||||
|
result
|
||||||
|
}
|
||||||
|
|
||||||
|
if (logger.isTraceEnabled) {
|
||||||
|
logger.trace {
|
||||||
|
val lines: Seq[String] =
|
||||||
|
(keys, cacheKeys, results).zipped.map {
|
||||||
|
case (key, cacheKey, result) => s"\n $key ($cacheKey) -> $result"
|
||||||
|
}
|
||||||
|
|
||||||
|
"Cache results:" + lines.mkString
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
results
|
||||||
|
}
|
||||||
|
.handle {
|
||||||
|
case e =>
|
||||||
|
// If there is a failure from the memcached client, fan it
|
||||||
|
// out to each cache key, so that the caller does not need
|
||||||
|
// to handle failure of the batch differently than failure
|
||||||
|
// of individual keys. This should be rare anyway, since the
|
||||||
|
// memcached client already does this for common Finagle
|
||||||
|
// exceptions
|
||||||
|
resultFailureCounter.incr(keys.size)
|
||||||
|
val theFailure: CacheResult[V] = CacheResult.Failure(e)
|
||||||
|
keys.map { _ =>
|
||||||
|
// Record this as many times as we would if it were in the GetResult
|
||||||
|
exceptionStatsHandler.record(memcachedStats, e)
|
||||||
|
theFailure
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Incremented for every attempt to `set` a key in value.
|
||||||
|
private[this] val memcachedSetCounter: Counter = memcachedStats.counter("set")
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write an entry back to cache, using `set`. If the serializer does
|
||||||
|
* not serialize the value, then this method will immediately return
|
||||||
|
* with success.
|
||||||
|
*/
|
||||||
|
def set(key: K, value: V): Future[Unit] =
|
||||||
|
valueSerializer.serialize(value) match {
|
||||||
|
case Some((expiry, serialized)) =>
|
||||||
|
if (logger.isTraceEnabled) {
|
||||||
|
logger.trace(s"Writing back to cache $key -> $value (expiry = $expiry)")
|
||||||
|
}
|
||||||
|
memcachedSetCounter.incr()
|
||||||
|
memcachedClient
|
||||||
|
.set(key = keySerializer(key), flags = 0, expiry = expiry, value = serialized)
|
||||||
|
.onFailure(exceptionStatsHandler.record(memcachedStats, _))
|
||||||
|
|
||||||
|
case None =>
|
||||||
|
if (logger.isTraceEnabled) {
|
||||||
|
logger.trace(s"Not writing back $key -> $value")
|
||||||
|
}
|
||||||
|
notSerializedCounter.incr()
|
||||||
|
Future.Done
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,45 @@
|
|||||||
|
package com.twitter.tweetypie.caching
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Encodes the possible states of a value loaded from memcached.
|
||||||
|
*
|
||||||
|
* @see [[ValueSerializer]] and [[CacheOperations]]
|
||||||
|
*/
|
||||||
|
sealed trait CacheResult[+V]
|
||||||
|
|
||||||
|
object CacheResult {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Signals that the value could not be successfully loaded from
|
||||||
|
* cache. `Failure` values should not be written back to cache.
|
||||||
|
*
|
||||||
|
* This value may result from an error talking to the memcached
|
||||||
|
* instance or it may be returned from the Serializer when the value
|
||||||
|
* should not be reused, but should also not be overwritten.
|
||||||
|
*/
|
||||||
|
final case class Failure(e: Throwable) extends CacheResult[Nothing]
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Signals that the cache load attempt was successful, but there was
|
||||||
|
* not a usable value.
|
||||||
|
*
|
||||||
|
* When processing a `Miss`, the value should be written back to
|
||||||
|
* cache if it loads successfully.
|
||||||
|
*/
|
||||||
|
case object Miss extends CacheResult[Nothing]
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Signals that the value was found in cache.
|
||||||
|
*
|
||||||
|
* It is not necessary to load the value from the original source.
|
||||||
|
*/
|
||||||
|
case class Fresh[V](value: V) extends CacheResult[V]
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Signals that the value was found in cache.
|
||||||
|
*
|
||||||
|
* This value should be used, but it should be refreshed
|
||||||
|
* out-of-band.
|
||||||
|
*/
|
||||||
|
case class Stale[V](value: V) extends CacheResult[V]
|
||||||
|
}
|
@ -0,0 +1,34 @@
|
|||||||
|
package com.twitter.tweetypie.caching
|
||||||
|
|
||||||
|
import com.twitter.util.Duration
|
||||||
|
import com.twitter.util.Time
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Helpers for creating common expiry functions.
|
||||||
|
*
|
||||||
|
* An expiry function maps from the value to a time in the future when
|
||||||
|
* the value should expire from cache. These are useful in the
|
||||||
|
* implementation of a [[ValueSerializer]].
|
||||||
|
*/
|
||||||
|
object Expiry {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return a time that indicates to memcached to never expire this
|
||||||
|
* value.
|
||||||
|
*
|
||||||
|
* This function takes [[Any]] so that it can be used at any value
|
||||||
|
* type, since it doesn't examine the value at all.
|
||||||
|
*/
|
||||||
|
val Never: Any => Time =
|
||||||
|
_ => Time.Top
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return function that indicates to memcached that the value should
|
||||||
|
* not be used after the `ttl` has elapsed.
|
||||||
|
*
|
||||||
|
* This function takes [[Any]] so that it can be used at any value
|
||||||
|
* type, since it doesn't examine the value at all.
|
||||||
|
*/
|
||||||
|
def byAge(ttl: Duration): Any => Time =
|
||||||
|
_ => Time.now + ttl
|
||||||
|
}
|
@ -0,0 +1,140 @@
|
|||||||
|
package com.twitter.tweetypie.caching
|
||||||
|
|
||||||
|
import com.twitter.io.Buf
|
||||||
|
import com.twitter.scrooge.CompactThriftSerializer
|
||||||
|
import com.twitter.scrooge.ThriftStruct
|
||||||
|
import com.twitter.scrooge.ThriftStructCodec
|
||||||
|
import com.twitter.servo.cache.thriftscala.CachedValue
|
||||||
|
import com.twitter.servo.cache.thriftscala.CachedValueStatus
|
||||||
|
import com.twitter.stitch.NotFound
|
||||||
|
import com.twitter.util.Return
|
||||||
|
import com.twitter.util.Throw
|
||||||
|
import com.twitter.util.Time
|
||||||
|
import com.twitter.util.Try
|
||||||
|
import java.nio.ByteBuffer
|
||||||
|
|
||||||
|
object ServoCachedValueSerializer {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Thrown when the fields of the servo CachedValue struct do not
|
||||||
|
* satisfy the invariants expected by this serialization code.
|
||||||
|
*/
|
||||||
|
case class UnexpectedCachedValueState(cachedValue: CachedValue) extends Exception {
|
||||||
|
def message: String = s"Unexpected state for CachedValue. Value was: $cachedValue"
|
||||||
|
}
|
||||||
|
|
||||||
|
val CachedValueThriftSerializer: CompactThriftSerializer[CachedValue] = CompactThriftSerializer(
|
||||||
|
CachedValue)
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A [[ValueSerializer]] that is compatible with the use of
|
||||||
|
* Servo's [[CachedValue]] struct by tweetypie:
|
||||||
|
*
|
||||||
|
* - The only [[CachedValueStatus]] values that are cacheable are
|
||||||
|
* [[CachedValueStatus.Found]] and [[CachedValueStatus.NotFound]].
|
||||||
|
*
|
||||||
|
* - We only track the `cachedAtMsec` field, because tweetypie's cache
|
||||||
|
* interaction does not use the other fields, and the values that
|
||||||
|
* are cached this way are never updated, so storing readThroughAt
|
||||||
|
* or writtenThroughAt would not add any information.
|
||||||
|
*
|
||||||
|
* - When values are present, they are serialized using
|
||||||
|
* [[org.apache.thrift.protocol.TCompactProtocol]].
|
||||||
|
*
|
||||||
|
* - The CachedValue struct itself is also serialized using TCompactProtocol.
|
||||||
|
*
|
||||||
|
* The serializer operates on [[Try]] values and will cache [[Return]]
|
||||||
|
* and `Throw(NotFound)` values.
|
||||||
|
*/
|
||||||
|
case class ServoCachedValueSerializer[V <: ThriftStruct](
|
||||||
|
codec: ThriftStructCodec[V],
|
||||||
|
expiry: Try[V] => Time,
|
||||||
|
softTtl: SoftTtl[Try[V]])
|
||||||
|
extends ValueSerializer[Try[V]] {
|
||||||
|
import ServoCachedValueSerializer.UnexpectedCachedValueState
|
||||||
|
import ServoCachedValueSerializer.CachedValueThriftSerializer
|
||||||
|
|
||||||
|
private[this] val ValueThriftSerializer = CompactThriftSerializer(codec)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return an expiry based on the value and a
|
||||||
|
* TCompactProtocol-encoded servo CachedValue struct with the
|
||||||
|
* following fields defined:
|
||||||
|
*
|
||||||
|
* - `value`: [[None]]
|
||||||
|
* for {{{Throw(NotFound)}}, {{{Some(encodedStruct)}}} for
|
||||||
|
* [[Return]], where {{{encodedStruct}}} is a
|
||||||
|
* TCompactProtocol-encoding of the value inside of the Return.
|
||||||
|
*
|
||||||
|
* - `status`: [[CachedValueStatus.Found]] if the value is Return,
|
||||||
|
* and [[CachedValueStatus.NotFound]] if it is Throw(NotFound)
|
||||||
|
*
|
||||||
|
* - `cachedAtMsec`: The current time, accoring to [[Time.now]]
|
||||||
|
*
|
||||||
|
* No other fields will be defined.
|
||||||
|
*
|
||||||
|
* @throws IllegalArgumentException if called with a value that
|
||||||
|
* should not be cached.
|
||||||
|
*/
|
||||||
|
override def serialize(value: Try[V]): Option[(Time, Buf)] = {
|
||||||
|
def serializeCachedValue(payload: Option[ByteBuffer]) = {
|
||||||
|
val cachedValue = CachedValue(
|
||||||
|
value = payload,
|
||||||
|
status = if (payload.isDefined) CachedValueStatus.Found else CachedValueStatus.NotFound,
|
||||||
|
cachedAtMsec = Time.now.inMilliseconds)
|
||||||
|
|
||||||
|
val serialized = Buf.ByteArray.Owned(CachedValueThriftSerializer.toBytes(cachedValue))
|
||||||
|
|
||||||
|
(expiry(value), serialized)
|
||||||
|
}
|
||||||
|
|
||||||
|
value match {
|
||||||
|
case Throw(NotFound) =>
|
||||||
|
Some(serializeCachedValue(None))
|
||||||
|
case Return(struct) =>
|
||||||
|
val payload = Some(ByteBuffer.wrap(ValueThriftSerializer.toBytes(struct)))
|
||||||
|
Some(serializeCachedValue(payload))
|
||||||
|
case _ =>
|
||||||
|
None
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Deserializes values serialized by [[serializeValue]]. The
|
||||||
|
* value will be [[CacheResult.Fresh]] or [[CacheResult.Stale]]
|
||||||
|
* depending on the result of {{{softTtl.isFresh}}}.
|
||||||
|
*
|
||||||
|
* @throws UnexpectedCachedValueState if the state of the
|
||||||
|
* [[CachedValue]] could not be produced by [[serialize]]
|
||||||
|
*/
|
||||||
|
override def deserialize(buf: Buf): CacheResult[Try[V]] = {
|
||||||
|
val cachedValue = CachedValueThriftSerializer.fromBytes(Buf.ByteArray.Owned.extract(buf))
|
||||||
|
val hasValue = cachedValue.value.isDefined
|
||||||
|
val isValid =
|
||||||
|
(hasValue && cachedValue.status == CachedValueStatus.Found) ||
|
||||||
|
(!hasValue && cachedValue.status == CachedValueStatus.NotFound)
|
||||||
|
|
||||||
|
if (!isValid) {
|
||||||
|
// Exceptions thrown by deserialization are recorded and treated
|
||||||
|
// as a cache miss by CacheOperations, so throwing this
|
||||||
|
// exception will cause the value in cache to be
|
||||||
|
// overwritten. There will be stats recorded whenever this
|
||||||
|
// happens.
|
||||||
|
throw UnexpectedCachedValueState(cachedValue)
|
||||||
|
}
|
||||||
|
|
||||||
|
val value =
|
||||||
|
cachedValue.value match {
|
||||||
|
case Some(valueBuffer) =>
|
||||||
|
val valueBytes = new Array[Byte](valueBuffer.remaining)
|
||||||
|
valueBuffer.duplicate.get(valueBytes)
|
||||||
|
Return(ValueThriftSerializer.fromBytes(valueBytes))
|
||||||
|
|
||||||
|
case None =>
|
||||||
|
Throw(NotFound)
|
||||||
|
}
|
||||||
|
|
||||||
|
softTtl.toCacheResult(value, Time.fromMilliseconds(cachedValue.cachedAtMsec))
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,120 @@
|
|||||||
|
package com.twitter.tweetypie.caching
|
||||||
|
|
||||||
|
import com.twitter.util.Duration
|
||||||
|
import com.twitter.util.Time
|
||||||
|
import scala.util.Random
|
||||||
|
import com.twitter.logging.Logger
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Used to determine whether values successfully retrieved from cache
|
||||||
|
* are [[CacheResult.Fresh]] or [[CacheResult.Stale]]. This is useful
|
||||||
|
* in the implementation of a [[ValueSerializer]].
|
||||||
|
*/
|
||||||
|
trait SoftTtl[-V] {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Determines whether a cached value was fresh.
|
||||||
|
*
|
||||||
|
* @param cachedAt the time at which the value was cached.
|
||||||
|
*/
|
||||||
|
def isFresh(value: V, cachedAt: Time): Boolean
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Wraps the value in Fresh or Stale depending on the value of `isFresh`.
|
||||||
|
*
|
||||||
|
* (The type variable U exists because it is not allowed to return
|
||||||
|
* values of a contravariant type, so we must define a variable that
|
||||||
|
* is a specific subclass of V. This is worth it because it allows
|
||||||
|
* us to create polymorphic policies without having to specify the
|
||||||
|
* type. Another solution would be to make the type invariant, but
|
||||||
|
* then we would have to specify the type whenever we create an
|
||||||
|
* instance.)
|
||||||
|
*/
|
||||||
|
def toCacheResult[U <: V](value: U, cachedAt: Time): CacheResult[U] =
|
||||||
|
if (isFresh(value, cachedAt)) CacheResult.Fresh(value) else CacheResult.Stale(value)
|
||||||
|
}
|
||||||
|
|
||||||
|
object SoftTtl {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Regardless of the inputs, the value will always be considered
|
||||||
|
* fresh.
|
||||||
|
*/
|
||||||
|
object NeverRefresh extends SoftTtl[Any] {
|
||||||
|
override def isFresh(_unusedValue: Any, _unusedCachedAt: Time): Boolean = true
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Trigger refresh based on the length of time that a value has been
|
||||||
|
* stored in cache, ignoring the value.
|
||||||
|
*
|
||||||
|
* @param softTtl Items that were cached longer ago than this value
|
||||||
|
* will be refreshed when they are accessed.
|
||||||
|
*
|
||||||
|
* @param jitter Add nondeterminism to the soft TTL to prevent a
|
||||||
|
* thundering herd of requests refreshing the value at the same
|
||||||
|
* time. The time at which the value is considered stale will be
|
||||||
|
* uniformly spread out over a range of +/- (jitter/2). It is
|
||||||
|
* valid to set the jitter to zero, which will turn off jittering.
|
||||||
|
*
|
||||||
|
* @param logger If non-null, use this logger rather than one based
|
||||||
|
* on the class name. This logger is only used for trace-level
|
||||||
|
* logging.
|
||||||
|
*/
|
||||||
|
case class ByAge[V](
|
||||||
|
softTtl: Duration,
|
||||||
|
jitter: Duration,
|
||||||
|
specificLogger: Logger = null,
|
||||||
|
rng: Random = Random)
|
||||||
|
extends SoftTtl[Any] {
|
||||||
|
|
||||||
|
private[this] val logger: Logger =
|
||||||
|
if (specificLogger == null) Logger(getClass) else specificLogger
|
||||||
|
|
||||||
|
private[this] val maxJitterMs: Long = jitter.inMilliseconds
|
||||||
|
|
||||||
|
// this requirement is due to using Random.nextInt to choose the
|
||||||
|
// jitter, but it allows jitter of greater than 24 days
|
||||||
|
require(maxJitterMs <= (Int.MaxValue / 2))
|
||||||
|
|
||||||
|
// Negative jitter probably indicates misuse of the API
|
||||||
|
require(maxJitterMs >= 0)
|
||||||
|
|
||||||
|
// we want period +/- jitter, but the random generator
|
||||||
|
// generates non-negative numbers, so we generate [0, 2 *
|
||||||
|
// maxJitter) and subtract maxJitter to obtain [-maxJitter,
|
||||||
|
// maxJitter)
|
||||||
|
private[this] val maxJitterRangeMs: Int = (maxJitterMs * 2).toInt
|
||||||
|
|
||||||
|
// We perform all calculations in milliseconds, so convert the
|
||||||
|
// period to milliseconds out here.
|
||||||
|
private[this] val softTtlMs: Long = softTtl.inMilliseconds
|
||||||
|
|
||||||
|
// If the value is below this age, it will always be fresh,
|
||||||
|
// regardless of jitter.
|
||||||
|
private[this] val alwaysFreshAgeMs: Long = softTtlMs - maxJitterMs
|
||||||
|
|
||||||
|
// If the value is above this age, it will always be stale,
|
||||||
|
// regardless of jitter.
|
||||||
|
private[this] val alwaysStaleAgeMs: Long = softTtlMs + maxJitterMs
|
||||||
|
|
||||||
|
override def isFresh(value: Any, cachedAt: Time): Boolean = {
|
||||||
|
val ageMs: Long = (Time.now - cachedAt).inMilliseconds
|
||||||
|
val fresh =
|
||||||
|
if (ageMs <= alwaysFreshAgeMs) {
|
||||||
|
true
|
||||||
|
} else if (ageMs > alwaysStaleAgeMs) {
|
||||||
|
false
|
||||||
|
} else {
|
||||||
|
val jitterMs: Long = rng.nextInt(maxJitterRangeMs) - maxJitterMs
|
||||||
|
ageMs <= softTtlMs + jitterMs
|
||||||
|
}
|
||||||
|
|
||||||
|
logger.ifTrace(
|
||||||
|
s"Checked soft ttl: fresh = $fresh, " +
|
||||||
|
s"soft_ttl_ms = $softTtlMs, age_ms = $ageMs, value = $value")
|
||||||
|
|
||||||
|
fresh
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,65 @@
|
|||||||
|
package com.twitter.tweetypie.caching
|
||||||
|
|
||||||
|
import scala.collection.mutable
|
||||||
|
import com.twitter.util.Future
|
||||||
|
import com.twitter.stitch.Stitch
|
||||||
|
import com.twitter.stitch.Runner
|
||||||
|
import com.twitter.stitch.FutureRunner
|
||||||
|
import com.twitter.stitch.Group
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Workaround for a infelicity in the implementation of [[Stitch.async]].
|
||||||
|
*
|
||||||
|
* This has the same semantics to [[Stitch.async]], with the exception
|
||||||
|
* that interrupts to the main computation will not interrupt the
|
||||||
|
* async call.
|
||||||
|
*
|
||||||
|
* The problem that this implementation solves is that we do not want
|
||||||
|
* async calls grouped together with synchronous calls. See the
|
||||||
|
* mailing list thread [1] for discussion. This may eventually be
|
||||||
|
* fixed in Stitch.
|
||||||
|
*/
|
||||||
|
private[caching] object StitchAsync {
|
||||||
|
// Contains a deferred Stitch that we want to run asynchronously
|
||||||
|
private[this] class AsyncCall(deferred: => Stitch[_]) {
|
||||||
|
def call(): Stitch[_] = deferred
|
||||||
|
}
|
||||||
|
|
||||||
|
private object AsyncGroup extends Group[AsyncCall, Unit] {
|
||||||
|
override def runner(): Runner[AsyncCall, Unit] =
|
||||||
|
new FutureRunner[AsyncCall, Unit] {
|
||||||
|
// All of the deferred calls of any type. When they are
|
||||||
|
// executed in `run`, the normal Stitch batching and deduping
|
||||||
|
// will occur.
|
||||||
|
private[this] val calls = new mutable.ArrayBuffer[AsyncCall]
|
||||||
|
|
||||||
|
def add(call: AsyncCall): Stitch[Unit] = {
|
||||||
|
// Just remember the deferred call.
|
||||||
|
calls.append(call)
|
||||||
|
|
||||||
|
// Since we don't wait for the completion of the effect,
|
||||||
|
// just return a constant value.
|
||||||
|
Stitch.Unit
|
||||||
|
}
|
||||||
|
|
||||||
|
def run(): Future[_] = {
|
||||||
|
// The future returned from this innter invocation of
|
||||||
|
// Stitch.run is not linked to the returned future, so these
|
||||||
|
// effects are not linked to the outer Run in which this
|
||||||
|
// method was invoked.
|
||||||
|
Stitch.run {
|
||||||
|
Stitch.traverse(calls) { asyncCall: AsyncCall =>
|
||||||
|
asyncCall
|
||||||
|
.call()
|
||||||
|
.liftToTry // So that an exception will not interrupt the other calls
|
||||||
|
}
|
||||||
|
}
|
||||||
|
Future.Unit
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
def apply(call: => Stitch[_]): Stitch[Unit] =
|
||||||
|
// Group together all of the async calls
|
||||||
|
Stitch.call(new AsyncCall(call), AsyncGroup)
|
||||||
|
}
|
@ -0,0 +1,62 @@
|
|||||||
|
package com.twitter.tweetypie.caching
|
||||||
|
|
||||||
|
import com.twitter.stitch.MapGroup
|
||||||
|
import com.twitter.stitch.Group
|
||||||
|
import com.twitter.stitch.Stitch
|
||||||
|
import com.twitter.util.Future
|
||||||
|
import com.twitter.util.Return
|
||||||
|
import com.twitter.util.Try
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Wrapper around [[CacheOperations]] providing a [[Stitch]] API.
|
||||||
|
*/
|
||||||
|
case class StitchCacheOperations[K, V](operations: CacheOperations[K, V]) {
|
||||||
|
import StitchCacheOperations.SetCall
|
||||||
|
|
||||||
|
private[this] val getGroup: Group[K, CacheResult[V]] =
|
||||||
|
MapGroup[K, CacheResult[V]] { keys: Seq[K] =>
|
||||||
|
operations
|
||||||
|
.get(keys)
|
||||||
|
.map(values => keys.zip(values).toMap.mapValues(Return(_)))
|
||||||
|
}
|
||||||
|
|
||||||
|
def get(key: K): Stitch[CacheResult[V]] =
|
||||||
|
Stitch.call(key, getGroup)
|
||||||
|
|
||||||
|
private[this] val setGroup: Group[SetCall[K, V], Unit] =
|
||||||
|
new MapGroup[SetCall[K, V], Unit] {
|
||||||
|
|
||||||
|
override def run(calls: Seq[SetCall[K, V]]): Future[SetCall[K, V] => Try[Unit]] =
|
||||||
|
Future
|
||||||
|
.collectToTry(calls.map(call => operations.set(call.key, call.value)))
|
||||||
|
.map(tries => calls.zip(tries).toMap)
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Performs a [[CacheOperations.set]].
|
||||||
|
*/
|
||||||
|
def set(key: K, value: V): Stitch[Unit] =
|
||||||
|
// This is implemented as a Stitch.call instead of a Stitch.future
|
||||||
|
// in order to handle the case where a batch has a duplicate
|
||||||
|
// key. Each copy of the duplicate key will trigger a write back
|
||||||
|
// to cache, so we dedupe the writes in order to avoid the
|
||||||
|
// extraneous RPC call.
|
||||||
|
Stitch.call(new StitchCacheOperations.SetCall(key, value), setGroup)
|
||||||
|
}
|
||||||
|
|
||||||
|
object StitchCacheOperations {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Used as the "call" for [[SetGroup]]. This is essentially a tuple
|
||||||
|
* where equality is defined only by the key.
|
||||||
|
*/
|
||||||
|
private class SetCall[K, V](val key: K, val value: V) {
|
||||||
|
override def equals(other: Any): Boolean =
|
||||||
|
other match {
|
||||||
|
case setCall: SetCall[_, _] => key == setCall.key
|
||||||
|
case _ => false
|
||||||
|
}
|
||||||
|
|
||||||
|
override def hashCode: Int = key.hashCode
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,36 @@
|
|||||||
|
package com.twitter.tweetypie.caching
|
||||||
|
|
||||||
|
import com.twitter.stitch.Stitch
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Apply caching to a [[Stitch]] function.
|
||||||
|
*
|
||||||
|
* @see CacheResult for more information about the semantics
|
||||||
|
* implemented here.
|
||||||
|
*/
|
||||||
|
class StitchCaching[K, V](operations: CacheOperations[K, V], repo: K => Stitch[V])
|
||||||
|
extends (K => Stitch[V]) {
|
||||||
|
|
||||||
|
private[this] val stitchOps = new StitchCacheOperations(operations)
|
||||||
|
|
||||||
|
override def apply(key: K): Stitch[V] =
|
||||||
|
stitchOps.get(key).flatMap {
|
||||||
|
case CacheResult.Fresh(value) =>
|
||||||
|
Stitch.value(value)
|
||||||
|
|
||||||
|
case CacheResult.Stale(staleValue) =>
|
||||||
|
StitchAsync(repo(key).flatMap(refreshed => stitchOps.set(key, refreshed)))
|
||||||
|
.map(_ => staleValue)
|
||||||
|
|
||||||
|
case CacheResult.Miss =>
|
||||||
|
repo(key)
|
||||||
|
.applyEffect(value => StitchAsync(stitchOps.set(key, value)))
|
||||||
|
|
||||||
|
case CacheResult.Failure(_) =>
|
||||||
|
// In the case of failure, we don't attempt to write back to
|
||||||
|
// cache, because cache failure usually means communication
|
||||||
|
// failure, and sending more requests to the cache that holds
|
||||||
|
// the value for this key could make the situation worse.
|
||||||
|
repo(key)
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,47 @@
|
|||||||
|
package com.twitter.tweetypie.caching
|
||||||
|
|
||||||
|
import com.twitter.io.Buf
|
||||||
|
import com.twitter.util.Time
|
||||||
|
|
||||||
|
/**
|
||||||
|
* How to store values of type V in cache. This includes whether a
|
||||||
|
* given value is cacheable, how to serialize it, when it should
|
||||||
|
* expire from cache, and how to interpret byte patterns from cache.
|
||||||
|
*/
|
||||||
|
trait ValueSerializer[V] {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Prepare the value for storage in cache. When a [[Some]] is
|
||||||
|
* returned, the [[Buf]] should be a valid input to [[deserialize]]
|
||||||
|
* and the [[Time]] will be used as the expiry in the memcached
|
||||||
|
* command. When [[None]] is returned, it indicates that the value
|
||||||
|
* cannot or should not be written back to cache.
|
||||||
|
*
|
||||||
|
* The most common use case for returning None is caching Try
|
||||||
|
* values, where certain exceptional values encode a cacheable state
|
||||||
|
* of a value. In particular, Throw(NotFound) is commonly used to
|
||||||
|
* encode a missing value, and we usually want to cache those
|
||||||
|
* negative lookups, but we don't want to cache e.g. a timeout
|
||||||
|
* exception.
|
||||||
|
*
|
||||||
|
* @return a pair of expiry time for this cache entry and the bytes
|
||||||
|
* to store in cache. If you do not want this value to explicitly
|
||||||
|
* expire, use Time.Top as the expiry.
|
||||||
|
*/
|
||||||
|
def serialize(value: V): Option[(Time, Buf)]
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Deserialize a value found in cache. This function converts the
|
||||||
|
* bytes found in memcache to a [[CacheResult]]. In general, you
|
||||||
|
* probably want to return [[CacheResult.Fresh]] or
|
||||||
|
* [[CacheResult.Stale]], but you are free to return any of the
|
||||||
|
* range of [[CacheResult]]s, depending on the behavior that you
|
||||||
|
* want.
|
||||||
|
*
|
||||||
|
* This is a total function because in the common use case, the
|
||||||
|
* bytes stored in cache will be appropriate for the
|
||||||
|
* serializer. This method is free to throw any exception if the
|
||||||
|
* bytes are not valid.
|
||||||
|
*/
|
||||||
|
def deserialize(serializedValue: Buf): CacheResult[V]
|
||||||
|
}
|
@ -0,0 +1,15 @@
|
|||||||
|
scala_library(
|
||||||
|
sources = ["*.scala"],
|
||||||
|
compiler_option_sets = ["fatal_warnings"],
|
||||||
|
platform = "java8",
|
||||||
|
strict_deps = True,
|
||||||
|
tags = ["bazel-compatible"],
|
||||||
|
dependencies = [
|
||||||
|
"finagle-internal/mtls/src/main/scala/com/twitter/finagle/mtls/authentication",
|
||||||
|
"finagle-internal/mtls/src/main/scala/com/twitter/finagle/mtls/transport",
|
||||||
|
"finagle/finagle-thrift/src/main/scala",
|
||||||
|
"tweetypie/servo/util/src/main/scala:exception",
|
||||||
|
"strato/src/main/scala/com/twitter/strato/access",
|
||||||
|
"strato/src/main/scala/com/twitter/strato/data",
|
||||||
|
],
|
||||||
|
)
|
@ -0,0 +1,185 @@
|
|||||||
|
package com.twitter.tweetypie.client_id
|
||||||
|
|
||||||
|
import com.twitter.finagle.mtls.authentication.EmptyServiceIdentifier
|
||||||
|
import com.twitter.finagle.mtls.authentication.ServiceIdentifier
|
||||||
|
import com.twitter.finagle.mtls.transport.S2STransport
|
||||||
|
import com.twitter.finagle.thrift.ClientId
|
||||||
|
import com.twitter.servo.util.Gate
|
||||||
|
import com.twitter.strato.access.Access
|
||||||
|
import com.twitter.strato.access.Access.ForwardedServiceIdentifier
|
||||||
|
|
||||||
|
object ClientIdHelper {
|
||||||
|
|
||||||
|
val UnknownClientId = "unknown"
|
||||||
|
|
||||||
|
def default: ClientIdHelper = new ClientIdHelper(UseTransportServiceIdentifier)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Trims off the last .element, which is usually .prod or .staging
|
||||||
|
*/
|
||||||
|
def getClientIdRoot(clientId: String): String =
|
||||||
|
clientId.lastIndexOf('.') match {
|
||||||
|
case -1 => clientId
|
||||||
|
case idx => clientId.substring(0, idx)
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the last .element without the '.'
|
||||||
|
*/
|
||||||
|
def getClientIdEnv(clientId: String): String =
|
||||||
|
clientId.lastIndexOf('.') match {
|
||||||
|
case -1 => clientId
|
||||||
|
case idx => clientId.substring(idx + 1)
|
||||||
|
}
|
||||||
|
|
||||||
|
private[client_id] def asClientId(s: ServiceIdentifier): String = s"${s.service}.${s.environment}"
|
||||||
|
}
|
||||||
|
|
||||||
|
class ClientIdHelper(serviceIdentifierStrategy: ServiceIdentifierStrategy) {
|
||||||
|
|
||||||
|
private[client_id] val ProcessPathPrefix = "/p/"
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The effective client id is used for request authorization and metrics
|
||||||
|
* attribution. For calls to Tweetypie's thrift API, the thrift ClientId
|
||||||
|
* is used and is expected in the form of "service-name.env". Federated
|
||||||
|
* Strato clients don't support configured ClientIds and instead provide
|
||||||
|
* a "process path" containing instance-specific information. So for
|
||||||
|
* calls to the federated API, we compute an effective client id from
|
||||||
|
* the ServiceIdentifier, if present, in Strato's Access principles. The
|
||||||
|
* implementation avoids computing this identifier unless necessary,
|
||||||
|
* since this method is invoked on every request.
|
||||||
|
*/
|
||||||
|
def effectiveClientId: Option[String] = {
|
||||||
|
val clientId: Option[String] = ClientId.current.map(_.name)
|
||||||
|
clientId
|
||||||
|
// Exclude process paths because they are instance-specific and aren't
|
||||||
|
// supported by tweetypie for authorization or metrics purposes.
|
||||||
|
.filterNot(_.startsWith(ProcessPathPrefix))
|
||||||
|
// Try computing a value from the ServiceId if the thrift
|
||||||
|
// ClientId is undefined or unsupported.
|
||||||
|
.orElse(serviceIdentifierStrategy.serviceIdentifier.map(ClientIdHelper.asClientId))
|
||||||
|
// Ultimately fall back to the ClientId value, even when given an
|
||||||
|
// unsupported format, so that error text and debug logs include
|
||||||
|
// the value passed by the caller.
|
||||||
|
.orElse(clientId)
|
||||||
|
}
|
||||||
|
|
||||||
|
def effectiveClientIdRoot: Option[String] = effectiveClientId.map(ClientIdHelper.getClientIdRoot)
|
||||||
|
|
||||||
|
def effectiveServiceIdentifier: Option[ServiceIdentifier] =
|
||||||
|
serviceIdentifierStrategy.serviceIdentifier
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Logic how to find a [[ServiceIdentifier]] for the purpose of crafting a client ID. */
|
||||||
|
trait ServiceIdentifierStrategy {
|
||||||
|
def serviceIdentifier: Option[ServiceIdentifier]
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the only element of given [[Set]] or [[None]].
|
||||||
|
*
|
||||||
|
* This utility is used defensively against a set of principals collected
|
||||||
|
* from [[Access.getPrincipals]]. While the contract is that there should be at most one
|
||||||
|
* instance of each principal kind present in that set, in practice that has not been the case
|
||||||
|
* always. The safest strategy to in that case is to abandon a set completely if more than
|
||||||
|
* one principals are competing.
|
||||||
|
*/
|
||||||
|
final protected def onlyElement[T](set: Set[T]): Option[T] =
|
||||||
|
if (set.size <= 1) {
|
||||||
|
set.headOption
|
||||||
|
} else {
|
||||||
|
None
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Picks [[ServiceIdentifier]] from Finagle SSL Transport, if one exists.
|
||||||
|
*
|
||||||
|
* This works for both Thrift API calls as well as StratoFed API calls. Strato's
|
||||||
|
* [[Access#getPrincipals]] collection, which would typically be consulted by StratoFed
|
||||||
|
* column logic, contains the same [[ServiceIdentifier]] derived from the Finagle SSL
|
||||||
|
* transport, so there's no need to have separate strategies for Thrift vs StratoFed
|
||||||
|
* calls.
|
||||||
|
*
|
||||||
|
* This is the default behavior of using [[ServiceIdentifier]] for computing client ID.
|
||||||
|
*/
|
||||||
|
private[client_id] class UseTransportServiceIdentifier(
|
||||||
|
// overridable for testing
|
||||||
|
getPeerServiceIdentifier: => ServiceIdentifier,
|
||||||
|
) extends ServiceIdentifierStrategy {
|
||||||
|
override def serviceIdentifier: Option[ServiceIdentifier] =
|
||||||
|
getPeerServiceIdentifier match {
|
||||||
|
case EmptyServiceIdentifier => None
|
||||||
|
case si => Some(si)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
object UseTransportServiceIdentifier
|
||||||
|
extends UseTransportServiceIdentifier(S2STransport.peerServiceIdentifier)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Picks [[ForwardedServiceIdentifier]] from Strato principals for client ID
|
||||||
|
* if [[ServiceIdentifier]] points at call coming from Strato.
|
||||||
|
* If not present, falls back to [[UseTransportServiceIdentifier]] behavior.
|
||||||
|
*
|
||||||
|
* Tweetypie utilizes the strategy to pick [[ServiceIdentifier]] for the purpose
|
||||||
|
* of generating a client ID when the client ID is absent or unknown.
|
||||||
|
* [[PreferForwardedServiceIdentifierForStrato]] looks for the [[ForwardedServiceIdentifier]]
|
||||||
|
* values set by stratoserver request.
|
||||||
|
* The reason is, stratoserver is effectively a conduit, forwarding the [[ServiceIdentifier]]
|
||||||
|
* of the _actual client_ that is calling stratoserver.
|
||||||
|
* Any direct callers not going through stratoserver will default to [[ServiceIdentfier]].
|
||||||
|
*/
|
||||||
|
private[client_id] class PreferForwardedServiceIdentifierForStrato(
|
||||||
|
// overridable for testing
|
||||||
|
getPeerServiceIdentifier: => ServiceIdentifier,
|
||||||
|
) extends ServiceIdentifierStrategy {
|
||||||
|
val useTransportServiceIdentifier =
|
||||||
|
new UseTransportServiceIdentifier(getPeerServiceIdentifier)
|
||||||
|
|
||||||
|
override def serviceIdentifier: Option[ServiceIdentifier] =
|
||||||
|
useTransportServiceIdentifier.serviceIdentifier match {
|
||||||
|
case Some(serviceIdentifier) if isStrato(serviceIdentifier) =>
|
||||||
|
onlyElement(
|
||||||
|
Access.getPrincipals
|
||||||
|
.collect {
|
||||||
|
case forwarded: ForwardedServiceIdentifier =>
|
||||||
|
forwarded.serviceIdentifier.serviceIdentifier
|
||||||
|
}
|
||||||
|
).orElse(useTransportServiceIdentifier.serviceIdentifier)
|
||||||
|
case other => other
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Strato uses various service names like "stratoserver" and "stratoserver-patient".
|
||||||
|
* They all do start with "stratoserver" though, so at the point of implementing,
|
||||||
|
* the safest bet to recognize strato is to look for this prefix.
|
||||||
|
*
|
||||||
|
* This also works for staged strato instances (which it should), despite allowing
|
||||||
|
* for technically any caller to force this strategy, by creating service certificate
|
||||||
|
* with this service name.
|
||||||
|
*/
|
||||||
|
private def isStrato(serviceIdentifier: ServiceIdentifier): Boolean =
|
||||||
|
serviceIdentifier.service.startsWith("stratoserver")
|
||||||
|
}
|
||||||
|
|
||||||
|
object PreferForwardedServiceIdentifierForStrato
|
||||||
|
extends PreferForwardedServiceIdentifierForStrato(S2STransport.peerServiceIdentifier)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* [[ServiceIdentifierStrategy]] which dispatches between two delegates based on the value
|
||||||
|
* of a unitary decider every time [[serviceIdentifier]] is called.
|
||||||
|
*/
|
||||||
|
class ConditionalServiceIdentifierStrategy(
|
||||||
|
private val condition: Gate[Unit],
|
||||||
|
private val ifTrue: ServiceIdentifierStrategy,
|
||||||
|
private val ifFalse: ServiceIdentifierStrategy)
|
||||||
|
extends ServiceIdentifierStrategy {
|
||||||
|
|
||||||
|
override def serviceIdentifier: Option[ServiceIdentifier] =
|
||||||
|
if (condition()) {
|
||||||
|
ifTrue.serviceIdentifier
|
||||||
|
} else {
|
||||||
|
ifFalse.serviceIdentifier
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,19 @@
|
|||||||
|
scala_library(
|
||||||
|
sources = ["*.scala"],
|
||||||
|
compiler_option_sets = ["fatal_warnings"],
|
||||||
|
platform = "java8",
|
||||||
|
provides = scala_artifact(
|
||||||
|
org = "com.twitter.tweetypie",
|
||||||
|
name = "context",
|
||||||
|
repo = artifactory,
|
||||||
|
),
|
||||||
|
strict_deps = True,
|
||||||
|
tags = ["bazel-compatible"],
|
||||||
|
dependencies = [
|
||||||
|
"finagle/finagle-core/src/main",
|
||||||
|
"graphql/common/src/main/scala/com/twitter/graphql/common/core",
|
||||||
|
"src/thrift/com/twitter/context:twitter-context-scala",
|
||||||
|
"twitter-context/src/main/scala",
|
||||||
|
"util/util-core:scala",
|
||||||
|
],
|
||||||
|
)
|
@ -0,0 +1,135 @@
|
|||||||
|
package com.twitter.tweetypie.context
|
||||||
|
|
||||||
|
import com.twitter.context.TwitterContext
|
||||||
|
import com.twitter.finagle.Filter
|
||||||
|
import com.twitter.finagle.Service
|
||||||
|
import com.twitter.finagle.SimpleFilter
|
||||||
|
import com.twitter.finagle.context.Contexts
|
||||||
|
import com.twitter.io.Buf
|
||||||
|
import com.twitter.io.Buf.ByteArray.Owned
|
||||||
|
import com.twitter.finagle.stats.StatsReceiver
|
||||||
|
import com.twitter.graphql.common.core.GraphQlClientApplication
|
||||||
|
import com.twitter.util.Try
|
||||||
|
import java.nio.charset.StandardCharsets.UTF_8
|
||||||
|
import scala.util.matching.Regex
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Context and filters to help track callers of Tweetypie's endpoints. This context and its
|
||||||
|
* filters were originally added to provide visibility into callers of Tweetypie who are
|
||||||
|
* using the birdherd library to access tweets.
|
||||||
|
*
|
||||||
|
* This context data is intended to be marshalled by callers to Tweetypie, but then the
|
||||||
|
* context data is stripped (moved from broadcast to local). This happens so that the
|
||||||
|
* context data is not forwarded down tweetypie's backend rpc chains, which often result
|
||||||
|
* in transitive calls back into tweetypie. This effectively creates single-hop marshalling.
|
||||||
|
*/
|
||||||
|
object TweetypieContext {
|
||||||
|
// Bring Tweetypie permitted TwitterContext into scope
|
||||||
|
val TwitterContext: TwitterContext =
|
||||||
|
com.twitter.context.TwitterContext(com.twitter.tweetypie.TwitterContextPermit)
|
||||||
|
|
||||||
|
case class Ctx(via: String)
|
||||||
|
val Empty = Ctx("")
|
||||||
|
|
||||||
|
object Broadcast {
|
||||||
|
private[this] object Key extends Contexts.broadcast.Key[Ctx](id = Ctx.getClass.getName) {
|
||||||
|
|
||||||
|
override def marshal(value: Ctx): Buf =
|
||||||
|
Owned(value.via.getBytes(UTF_8))
|
||||||
|
|
||||||
|
override def tryUnmarshal(buf: Buf): Try[Ctx] =
|
||||||
|
Try(Ctx(new String(Owned.extract(buf), UTF_8)))
|
||||||
|
}
|
||||||
|
|
||||||
|
private[TweetypieContext] def current(): Option[Ctx] =
|
||||||
|
Contexts.broadcast.get(Key)
|
||||||
|
|
||||||
|
def currentOrElse(default: Ctx): Ctx =
|
||||||
|
current().getOrElse(default)
|
||||||
|
|
||||||
|
def letClear[T](f: => T): T =
|
||||||
|
Contexts.broadcast.letClear(Key)(f)
|
||||||
|
|
||||||
|
def let[T](ctx: Ctx)(f: => T): T =
|
||||||
|
if (Empty == ctx) {
|
||||||
|
letClear(f)
|
||||||
|
} else {
|
||||||
|
Contexts.broadcast.let(Key, ctx)(f)
|
||||||
|
}
|
||||||
|
|
||||||
|
// ctx has to be by name so we can re-evaluate it for every request (for usage in ServiceTwitter.scala)
|
||||||
|
def filter(ctx: => Ctx): Filter.TypeAgnostic =
|
||||||
|
new Filter.TypeAgnostic {
|
||||||
|
override def toFilter[Req, Rep]: Filter[Req, Rep, Req, Rep] =
|
||||||
|
(request: Req, service: Service[Req, Rep]) => Broadcast.let(ctx)(service(request))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
object Local {
|
||||||
|
private[this] val Key =
|
||||||
|
new Contexts.local.Key[Ctx]
|
||||||
|
|
||||||
|
private[TweetypieContext] def let[T](ctx: Option[Ctx])(f: => T): T =
|
||||||
|
ctx match {
|
||||||
|
case Some(ctx) if ctx != Empty => Contexts.local.let(Key, ctx)(f)
|
||||||
|
case None => Contexts.local.letClear(Key)(f)
|
||||||
|
}
|
||||||
|
|
||||||
|
def current(): Option[Ctx] =
|
||||||
|
Contexts.local.get(Key)
|
||||||
|
|
||||||
|
def filter[Req, Rep]: SimpleFilter[Req, Rep] =
|
||||||
|
(request: Req, service: Service[Req, Rep]) => {
|
||||||
|
val ctx = Broadcast.current()
|
||||||
|
Broadcast.letClear(Local.let(ctx)(service(request)))
|
||||||
|
}
|
||||||
|
|
||||||
|
private[this] def clientAppIdToName(clientAppId: Long) =
|
||||||
|
GraphQlClientApplication.AllById.get(clientAppId).map(_.name).getOrElse("nonTOO")
|
||||||
|
|
||||||
|
private[this] val pathRegexes: Seq[(Regex, String)] = Seq(
|
||||||
|
("timeline_conversation_.*_json".r, "timeline_conversation__slug__json"),
|
||||||
|
("user_timeline_.*_json".r, "user_timeline__user__json"),
|
||||||
|
("[0-9]{2,}".r, "_id_")
|
||||||
|
)
|
||||||
|
|
||||||
|
// `context.via` will either be a string like: "birdherd" or "birdherd:/1.1/statuses/show/123.json,
|
||||||
|
// depending on whether birdherd code was able to determine the path of the request.
|
||||||
|
private[this] def getViaAndPath(via: String): (String, Option[String]) =
|
||||||
|
via.split(":", 2) match {
|
||||||
|
case Array(via, path) =>
|
||||||
|
val sanitizedPath = path
|
||||||
|
.replace('/', '_')
|
||||||
|
.replace('.', '_')
|
||||||
|
|
||||||
|
// Apply each regex in turn
|
||||||
|
val normalizedPath = pathRegexes.foldLeft(sanitizedPath) {
|
||||||
|
case (path, (regex, replacement)) => regex.replaceAllIn(path, replacement)
|
||||||
|
}
|
||||||
|
|
||||||
|
(via, Some(normalizedPath))
|
||||||
|
case Array(via) => (via, None)
|
||||||
|
}
|
||||||
|
|
||||||
|
def trackStats[U](scopes: StatsReceiver*): Unit =
|
||||||
|
for {
|
||||||
|
tweetypieCtx <- TweetypieContext.Local.current()
|
||||||
|
(via, pathOpt) = getViaAndPath(tweetypieCtx.via)
|
||||||
|
twitterCtx <- TwitterContext()
|
||||||
|
clientAppId <- twitterCtx.clientApplicationId
|
||||||
|
} yield {
|
||||||
|
val clientAppName = clientAppIdToName(clientAppId)
|
||||||
|
scopes.foreach { stats =>
|
||||||
|
val ctxStats = stats.scope("context")
|
||||||
|
val viaStats = ctxStats.scope("via", via)
|
||||||
|
viaStats.scope("all").counter("requests").incr()
|
||||||
|
val viaClientStats = viaStats.scope("by_client", clientAppName)
|
||||||
|
viaClientStats.counter("requests").incr()
|
||||||
|
pathOpt.foreach { path =>
|
||||||
|
val viaPathStats = viaStats.scope("by_path", path)
|
||||||
|
viaPathStats.counter("requests").incr()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,15 @@
|
|||||||
|
scala_library(
|
||||||
|
sources = ["DeciderGates.scala"],
|
||||||
|
compiler_option_sets = ["fatal_warnings"],
|
||||||
|
platform = "java8",
|
||||||
|
strict_deps = True,
|
||||||
|
tags = ["bazel-compatible"],
|
||||||
|
dependencies = [
|
||||||
|
"3rdparty/jvm/com/google/guava",
|
||||||
|
"decider",
|
||||||
|
"finagle/finagle-toggle/src/main/scala/com/twitter/finagle/server",
|
||||||
|
"tweetypie/servo/decider",
|
||||||
|
"tweetypie/servo/util/src/main/scala",
|
||||||
|
"util/util-core:scala",
|
||||||
|
],
|
||||||
|
)
|
@ -0,0 +1,60 @@
|
|||||||
|
package com.twitter.tweetypie
|
||||||
|
package decider
|
||||||
|
|
||||||
|
import com.google.common.hash.Hashing
|
||||||
|
import com.twitter.decider.Decider
|
||||||
|
import com.twitter.decider.Feature
|
||||||
|
import com.twitter.servo.gate.DeciderGate
|
||||||
|
import com.twitter.servo.util.Gate
|
||||||
|
import java.nio.charset.StandardCharsets
|
||||||
|
import scala.collection.mutable
|
||||||
|
trait DeciderGates {
|
||||||
|
def overrides: Map[String, Boolean] = Map.empty
|
||||||
|
def decider: Decider
|
||||||
|
def prefix: String
|
||||||
|
|
||||||
|
protected val seenFeatures: mutable.HashSet[String] = new mutable.HashSet[String]
|
||||||
|
|
||||||
|
private def deciderFeature(name: String): Feature = {
|
||||||
|
decider.feature(prefix + "_" + name)
|
||||||
|
}
|
||||||
|
|
||||||
|
def withOverride[T](name: String, mkGate: Feature => Gate[T]): Gate[T] = {
|
||||||
|
seenFeatures += name
|
||||||
|
overrides.get(name).map(Gate.const).getOrElse(mkGate(deciderFeature(name)))
|
||||||
|
}
|
||||||
|
|
||||||
|
protected def linear(name: String): Gate[Unit] = withOverride[Unit](name, DeciderGate.linear)
|
||||||
|
protected def byId(name: String): Gate[Long] = withOverride[Long](name, DeciderGate.byId)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* It returns a Gate[String] that can be used to check availability of the feature.
|
||||||
|
* The string is hashed into a Long and used as an "id" and then used to call servo's
|
||||||
|
* DeciderGate.byId
|
||||||
|
*
|
||||||
|
* @param name decider name
|
||||||
|
* @return Gate[String]
|
||||||
|
*/
|
||||||
|
protected def byStringId(name: String): Gate[String] =
|
||||||
|
byId(name).contramap { s: String =>
|
||||||
|
Hashing.sipHash24().hashString(s, StandardCharsets.UTF_8).asLong()
|
||||||
|
}
|
||||||
|
|
||||||
|
def all: Traversable[String] = seenFeatures
|
||||||
|
|
||||||
|
def unusedOverrides: Set[String] = overrides.keySet.diff(all.toSet)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Generate a map of name -> availability, taking into account overrides.
|
||||||
|
* Overrides are either on or off so map to 10000 or 0, respectively.
|
||||||
|
*/
|
||||||
|
def availabilityMap: Map[String, Option[Int]] =
|
||||||
|
all.map { name =>
|
||||||
|
val availability: Option[Int] = overrides
|
||||||
|
.get(name)
|
||||||
|
.map(on => if (on) 10000 else 0)
|
||||||
|
.orElse(deciderFeature(name).availability)
|
||||||
|
|
||||||
|
name -> availability
|
||||||
|
}.toMap
|
||||||
|
}
|
@ -0,0 +1,10 @@
|
|||||||
|
scala_library(
|
||||||
|
sources = ["*.scala"],
|
||||||
|
compiler_option_sets = ["fatal_warnings"],
|
||||||
|
platform = "java8",
|
||||||
|
strict_deps = True,
|
||||||
|
tags = ["bazel-compatible"],
|
||||||
|
dependencies = [
|
||||||
|
"decider",
|
||||||
|
],
|
||||||
|
)
|
@ -0,0 +1,42 @@
|
|||||||
|
package com.twitter.tweetypie.decider.overrides
|
||||||
|
|
||||||
|
import com.twitter.decider.LocalOverrides
|
||||||
|
|
||||||
|
object TweetyPieDeciderOverrides extends LocalOverrides.Namespace("tweetypie", "tweetypie_") {
|
||||||
|
val CheckSpamOnRetweet: LocalOverrides.Override = feature("check_spam_on_retweet")
|
||||||
|
val CheckSpamOnTweet: LocalOverrides.Override = feature("check_spam_on_tweet")
|
||||||
|
val ConversationControlUseFeatureSwitchResults: LocalOverrides.Override = feature(
|
||||||
|
"conversation_control_use_feature_switch_results")
|
||||||
|
val ConversationControlTweetCreateEnabled: LocalOverrides.Override = feature(
|
||||||
|
"conversation_control_tweet_create_enabled")
|
||||||
|
val EnableExclusiveTweetControlValidation: LocalOverrides.Override = feature(
|
||||||
|
"enable_exclusive_tweet_control_validation")
|
||||||
|
val EnableHotKeyCaches: LocalOverrides.Override = feature("enable_hot_key_caches")
|
||||||
|
val HydrateConversationMuted: LocalOverrides.Override = feature("hydrate_conversation_muted")
|
||||||
|
val HydrateExtensionsOnWrite: LocalOverrides.Override = feature("hydrate_extensions_on_write")
|
||||||
|
val HydrateEscherbirdAnnotations: LocalOverrides.Override = feature(
|
||||||
|
"hydrate_escherbird_annotations")
|
||||||
|
val HydrateGnipProfileGeoEnrichment: LocalOverrides.Override = feature(
|
||||||
|
"hydrate_gnip_profile_geo_enrichment")
|
||||||
|
val HydratePastedPics: LocalOverrides.Override = feature("hydrate_pasted_pics")
|
||||||
|
val HydratePerspectivesEditsForOtherSafetyLevels: LocalOverrides.Override = feature(
|
||||||
|
"hydrate_perspectives_edits_for_other_levels")
|
||||||
|
val HydrateScrubEngagements: LocalOverrides.Override = feature("hydrate_scrub_engagements")
|
||||||
|
val LogRepoExceptions: LocalOverrides.Override = feature("log_repo_exceptions")
|
||||||
|
val MediaRefsHydratorIncludePastedMedia: LocalOverrides.Override = feature(
|
||||||
|
"media_refs_hydrator_include_pasted_media")
|
||||||
|
val ShortCircuitLikelyPartialTweetReads: LocalOverrides.Override = feature(
|
||||||
|
"short_circuit_likely_partial_tweet_reads_ms")
|
||||||
|
val RateLimitByLimiterService: LocalOverrides.Override = feature("rate_limit_by_limiter_service")
|
||||||
|
val RateLimitTweetCreationFailure: LocalOverrides.Override = feature(
|
||||||
|
"rate_limit_tweet_creation_failure")
|
||||||
|
val ReplyTweetConversationControlHydrationEnabled = feature(
|
||||||
|
"reply_tweet_conversation_control_hydration_enabled"
|
||||||
|
)
|
||||||
|
val DisableInviteViaMention = feature(
|
||||||
|
"disable_invite_via_mention"
|
||||||
|
)
|
||||||
|
val EnableRemoveUnmentionedImplicitMentions: LocalOverrides.Override = feature(
|
||||||
|
"enable_remove_unmentioned_implicit_mentions")
|
||||||
|
val useReplicatedDeleteTweet2: LocalOverrides.Override = feature("use_replicated_delete_tweet_2")
|
||||||
|
}
|
@ -0,0 +1,15 @@
|
|||||||
|
scala_library(
|
||||||
|
compiler_option_sets = ["fatal_warnings"],
|
||||||
|
strict_deps = True,
|
||||||
|
tags = ["bazel-compatible"],
|
||||||
|
dependencies = [
|
||||||
|
"finagle/finagle-core/src/main",
|
||||||
|
"incentives/jiminy/src/main/thrift/com/twitter/incentives/jiminy:thrift-scala",
|
||||||
|
"tweetypie/servo/util/src/main/scala",
|
||||||
|
"stitch/stitch-core",
|
||||||
|
"strato/src/main/scala/com/twitter/strato/client",
|
||||||
|
"tweetypie/server/src/main/scala/com/twitter/tweetypie/core",
|
||||||
|
"util/util-core",
|
||||||
|
"util/util-stats",
|
||||||
|
],
|
||||||
|
)
|
@ -0,0 +1,165 @@
|
|||||||
|
package com.twitter.tweetypie.jiminy.tweetypie
|
||||||
|
|
||||||
|
import com.twitter.finagle.stats.CategorizingExceptionStatsHandler
|
||||||
|
import com.twitter.finagle.stats.Stat
|
||||||
|
import com.twitter.finagle.stats.StatsReceiver
|
||||||
|
import com.twitter.incentives.jiminy.thriftscala._
|
||||||
|
import com.twitter.servo.util.FutureArrow
|
||||||
|
import com.twitter.servo.util.Gate
|
||||||
|
import com.twitter.stitch.Stitch
|
||||||
|
import com.twitter.strato.thrift.ScroogeConvImplicits._
|
||||||
|
import com.twitter.strato.client.{Client => StratoClient}
|
||||||
|
import com.twitter.tweetypie.core.TweetCreateFailure
|
||||||
|
import com.twitter.util.Future
|
||||||
|
import com.twitter.util.Return
|
||||||
|
import com.twitter.util.Throw
|
||||||
|
|
||||||
|
case class NudgeBuilderRequest(
|
||||||
|
text: String,
|
||||||
|
inReplyToTweetId: Option[NudgeBuilder.TweetId],
|
||||||
|
conversationId: Option[NudgeBuilder.TweetId],
|
||||||
|
hasQuotedTweet: Boolean,
|
||||||
|
nudgeOptions: Option[CreateTweetNudgeOptions],
|
||||||
|
tweetId: Option[NudgeBuilder.TweetId])
|
||||||
|
|
||||||
|
trait NudgeBuilder extends FutureArrow[NudgeBuilderRequest, Unit] {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Check whether the user should receive a nudge instead of creating
|
||||||
|
* the Tweet. If nudgeOptions is None, then no nudge check will be
|
||||||
|
* performed.
|
||||||
|
*
|
||||||
|
* @return a Future.exception containing a [[TweetCreateFailure]] if the
|
||||||
|
* user should be nudged, or Future.Unit if the user should not be
|
||||||
|
* nudged.
|
||||||
|
*/
|
||||||
|
def apply(
|
||||||
|
request: NudgeBuilderRequest
|
||||||
|
): Future[Unit]
|
||||||
|
}
|
||||||
|
|
||||||
|
object NudgeBuilder {
|
||||||
|
type Type = FutureArrow[NudgeBuilderRequest, Unit]
|
||||||
|
type TweetId = Long
|
||||||
|
|
||||||
|
// darkTrafficCreateNudgeOptions ensure that our dark traffic sends a request that will
|
||||||
|
// accurately test the Jiminy backend. in this case, we specify that we want checks for all
|
||||||
|
// possible nudge types
|
||||||
|
private[this] val darkTrafficCreateNudgeOptions = Some(
|
||||||
|
CreateTweetNudgeOptions(
|
||||||
|
requestedNudgeTypes = Some(
|
||||||
|
Set(
|
||||||
|
TweetNudgeType.PotentiallyToxicTweet,
|
||||||
|
TweetNudgeType.ReviseOrMute,
|
||||||
|
TweetNudgeType.ReviseOrHideThenBlock,
|
||||||
|
TweetNudgeType.ReviseOrBlock
|
||||||
|
)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
|
||||||
|
private[this] def mkJiminyRequest(
|
||||||
|
request: NudgeBuilderRequest,
|
||||||
|
isDarkRequest: Boolean = false
|
||||||
|
): CreateTweetNudgeRequest = {
|
||||||
|
val tweetType =
|
||||||
|
if (request.inReplyToTweetId.nonEmpty) TweetType.Reply
|
||||||
|
else if (request.hasQuotedTweet) TweetType.QuoteTweet
|
||||||
|
else TweetType.OriginalTweet
|
||||||
|
|
||||||
|
CreateTweetNudgeRequest(
|
||||||
|
tweetText = request.text,
|
||||||
|
tweetType = tweetType,
|
||||||
|
inReplyToTweetId = request.inReplyToTweetId,
|
||||||
|
conversationId = request.conversationId,
|
||||||
|
createTweetNudgeOptions =
|
||||||
|
if (isDarkRequest) darkTrafficCreateNudgeOptions else request.nudgeOptions,
|
||||||
|
tweetId = request.tweetId
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* NudgeBuilder implemented by calling the strato column `incentives/createNudge`.
|
||||||
|
*
|
||||||
|
* Stats recorded:
|
||||||
|
* - latency_ms: Latency histogram (also implicitly number of
|
||||||
|
* invocations). This is counted only in the case that a nudge
|
||||||
|
* check was requested (`nudgeOptions` is non-empty)
|
||||||
|
*
|
||||||
|
* - nudge: The nudge check succeeded and a nudge was created.
|
||||||
|
*
|
||||||
|
* - no_nudge: The nudge check succeeded, but no nudge was created.
|
||||||
|
*
|
||||||
|
* - failures: Calling strato to create a nudge failed. Broken out
|
||||||
|
* by exception.
|
||||||
|
*/
|
||||||
|
|
||||||
|
def apply(
|
||||||
|
nudgeArrow: FutureArrow[CreateTweetNudgeRequest, CreateTweetNudgeResponse],
|
||||||
|
enableDarkTraffic: Gate[Unit],
|
||||||
|
stats: StatsReceiver
|
||||||
|
): NudgeBuilder = {
|
||||||
|
new NudgeBuilder {
|
||||||
|
private[this] val nudgeLatencyStat = stats.stat("latency_ms")
|
||||||
|
private[this] val nudgeCounter = stats.counter("nudge")
|
||||||
|
private[this] val noNudgeCounter = stats.counter("no_nudge")
|
||||||
|
private[this] val darkRequestCounter = stats.counter("dark_request")
|
||||||
|
private[this] val nudgeExceptionHandler = new CategorizingExceptionStatsHandler
|
||||||
|
|
||||||
|
override def apply(
|
||||||
|
request: NudgeBuilderRequest
|
||||||
|
): Future[Unit] =
|
||||||
|
request.nudgeOptions match {
|
||||||
|
case None =>
|
||||||
|
if (enableDarkTraffic()) {
|
||||||
|
darkRequestCounter.incr()
|
||||||
|
Stat
|
||||||
|
.timeFuture(nudgeLatencyStat) {
|
||||||
|
nudgeArrow(mkJiminyRequest(request, isDarkRequest = true))
|
||||||
|
}
|
||||||
|
.transform { _ =>
|
||||||
|
// ignore the response since it is a dark request
|
||||||
|
Future.Done
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
Future.Done
|
||||||
|
}
|
||||||
|
|
||||||
|
case Some(_) =>
|
||||||
|
Stat
|
||||||
|
.timeFuture(nudgeLatencyStat) {
|
||||||
|
nudgeArrow(mkJiminyRequest(request))
|
||||||
|
}
|
||||||
|
.transform {
|
||||||
|
case Throw(e) =>
|
||||||
|
nudgeExceptionHandler.record(stats, e)
|
||||||
|
// If we failed to invoke the nudge column, then
|
||||||
|
// just continue on with the Tweet creation.
|
||||||
|
Future.Done
|
||||||
|
|
||||||
|
case Return(CreateTweetNudgeResponse(Some(nudge))) =>
|
||||||
|
nudgeCounter.incr()
|
||||||
|
Future.exception(TweetCreateFailure.Nudged(nudge = nudge))
|
||||||
|
|
||||||
|
case Return(CreateTweetNudgeResponse(None)) =>
|
||||||
|
noNudgeCounter.incr()
|
||||||
|
Future.Done
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
def apply(
|
||||||
|
strato: StratoClient,
|
||||||
|
enableDarkTraffic: Gate[Unit],
|
||||||
|
stats: StatsReceiver
|
||||||
|
): NudgeBuilder = {
|
||||||
|
val executer =
|
||||||
|
strato.executer[CreateTweetNudgeRequest, CreateTweetNudgeResponse](
|
||||||
|
"incentives/createTweetNudge")
|
||||||
|
val nudgeArrow: FutureArrow[CreateTweetNudgeRequest, CreateTweetNudgeResponse] = { req =>
|
||||||
|
Stitch.run(executer.execute(req))
|
||||||
|
}
|
||||||
|
apply(nudgeArrow, enableDarkTraffic, stats)
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,18 @@
|
|||||||
|
scala_library(
|
||||||
|
sources = ["*.scala"],
|
||||||
|
compiler_option_sets = ["fatal_warnings"],
|
||||||
|
platform = "java8",
|
||||||
|
strict_deps = True,
|
||||||
|
tags = ["bazel-compatible"],
|
||||||
|
dependencies = [
|
||||||
|
"src/java/com/twitter/common/text/language:language-identifier",
|
||||||
|
"src/java/com/twitter/common/text/language:locale-util",
|
||||||
|
"src/java/com/twitter/common/text/pipeline",
|
||||||
|
"src/java/com/twitter/common/text/token",
|
||||||
|
"src/java/com/twitter/common_internal/text",
|
||||||
|
"src/java/com/twitter/common_internal/text/version",
|
||||||
|
"tweetypie/src/resources/com/twitter/tweetypie/matching",
|
||||||
|
"util/util-core/src/main/scala/com/twitter/concurrent",
|
||||||
|
"util/util-core/src/main/scala/com/twitter/io",
|
||||||
|
],
|
||||||
|
)
|
@ -0,0 +1,92 @@
|
|||||||
|
package com.twitter.tweetypie.matching
|
||||||
|
|
||||||
|
object TokenSequence {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Is `suffix` a suffix of `s`, starting at `offset` in `s`?
|
||||||
|
*/
|
||||||
|
def hasSuffixAt(s: CharSequence, suffix: CharSequence, offset: Int): Boolean =
|
||||||
|
if (offset == 0 && (s.eq(suffix) || s == suffix)) {
|
||||||
|
true
|
||||||
|
} else if (suffix.length != (s.length - offset)) {
|
||||||
|
false
|
||||||
|
} else {
|
||||||
|
@annotation.tailrec
|
||||||
|
def go(i: Int): Boolean =
|
||||||
|
if (i == suffix.length) true
|
||||||
|
else if (suffix.charAt(i) == s.charAt(offset + i)) go(i + 1)
|
||||||
|
else false
|
||||||
|
|
||||||
|
go(0)
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Do two [[CharSequence]]s contain the same characters?
|
||||||
|
*
|
||||||
|
* [[CharSequence]] equality is not sufficient because
|
||||||
|
* [[CharSequence]]s of different types may not consider other
|
||||||
|
* [[CharSequence]]s containing the same characters equivalent.
|
||||||
|
*/
|
||||||
|
def sameCharacters(s1: CharSequence, s2: CharSequence): Boolean =
|
||||||
|
hasSuffixAt(s1, s2, 0)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This method implements the product definition of a token matching a
|
||||||
|
* keyword. That definition is:
|
||||||
|
*
|
||||||
|
* - The token contains the same characters as the keyword.
|
||||||
|
* - The token contains the same characters as the keyword after
|
||||||
|
* dropping a leading '#' or '@' from the token.
|
||||||
|
*
|
||||||
|
* The intention is that a keyword matches an identical hashtag, but
|
||||||
|
* if the keyword itself is a hashtag, it only matches the hashtag
|
||||||
|
* form.
|
||||||
|
*
|
||||||
|
* The tokenization process should rule out tokens or keywords that
|
||||||
|
* start with multiple '#' characters, even though this implementation
|
||||||
|
* allows for e.g. token "##a" to match "#a".
|
||||||
|
*/
|
||||||
|
def tokenMatches(token: CharSequence, keyword: CharSequence): Boolean =
|
||||||
|
if (sameCharacters(token, keyword)) true
|
||||||
|
else if (token.length == 0) false
|
||||||
|
else {
|
||||||
|
val tokenStart = token.charAt(0)
|
||||||
|
(tokenStart == '#' || tokenStart == '@') && hasSuffixAt(token, keyword, 1)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A sequence of normalized tokens. The sequence depends on the locale
|
||||||
|
* in which the text was parsed and the version of the penguin library
|
||||||
|
* that was used at tokenization time.
|
||||||
|
*/
|
||||||
|
case class TokenSequence private[matching] (toIndexedSeq: IndexedSeq[CharSequence]) {
|
||||||
|
import TokenSequence.tokenMatches
|
||||||
|
|
||||||
|
private def apply(i: Int): CharSequence = toIndexedSeq(i)
|
||||||
|
|
||||||
|
def isEmpty: Boolean = toIndexedSeq.isEmpty
|
||||||
|
def nonEmpty: Boolean = toIndexedSeq.nonEmpty
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Does the supplied sequence of keywords match a consecutive sequence
|
||||||
|
* of tokens within this sequence?
|
||||||
|
*/
|
||||||
|
def containsKeywordSequence(keywords: TokenSequence): Boolean = {
|
||||||
|
val finalIndex = toIndexedSeq.length - keywords.toIndexedSeq.length
|
||||||
|
|
||||||
|
@annotation.tailrec
|
||||||
|
def matchesAt(offset: Int, i: Int): Boolean =
|
||||||
|
if (i >= keywords.toIndexedSeq.length) true
|
||||||
|
else if (tokenMatches(this(i + offset), keywords(i))) matchesAt(offset, i + 1)
|
||||||
|
else false
|
||||||
|
|
||||||
|
@annotation.tailrec
|
||||||
|
def search(offset: Int): Boolean =
|
||||||
|
if (offset > finalIndex) false
|
||||||
|
else if (matchesAt(offset, 0)) true
|
||||||
|
else search(offset + 1)
|
||||||
|
|
||||||
|
search(0)
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,156 @@
|
|||||||
|
package com.twitter.tweetypie.matching
|
||||||
|
|
||||||
|
import com.twitter.common.text.language.LocaleUtil
|
||||||
|
import com.twitter.common_internal.text.pipeline.TwitterTextNormalizer
|
||||||
|
import com.twitter.common_internal.text.pipeline.TwitterTextTokenizer
|
||||||
|
import com.twitter.common_internal.text.version.PenguinVersion
|
||||||
|
import com.twitter.concurrent.Once
|
||||||
|
import com.twitter.io.StreamIO
|
||||||
|
import java.util.Locale
|
||||||
|
import scala.collection.JavaConverters._
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Extract a sequence of normalized tokens from the input text. The
|
||||||
|
* normalization and tokenization are properly configured for keyword
|
||||||
|
* matching between texts.
|
||||||
|
*/
|
||||||
|
trait Tokenizer {
|
||||||
|
def tokenize(input: String): TokenSequence
|
||||||
|
}
|
||||||
|
|
||||||
|
object Tokenizer {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* When a Penguin version is not explicitly specified, use this
|
||||||
|
* version of Penguin to perform normalization and tokenization. If
|
||||||
|
* you cache tokenized text, be sure to store the version as well, to
|
||||||
|
* avoid comparing text that was normalized with different algorithms.
|
||||||
|
*/
|
||||||
|
val DefaultPenguinVersion: PenguinVersion = PenguinVersion.PENGUIN_6
|
||||||
|
|
||||||
|
/**
|
||||||
|
* If you already know the locale of the text that is being tokenized,
|
||||||
|
* use this method to get a tokenizer that is much more efficient than
|
||||||
|
* the Tweet or Query tokenizer, since it does not have to perform
|
||||||
|
* language detection.
|
||||||
|
*/
|
||||||
|
def forLocale(locale: Locale): Tokenizer = get(locale, DefaultPenguinVersion)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Obtain a `Tokenizer` that will tokenize the text for the given
|
||||||
|
* locale and version of the Penguin library.
|
||||||
|
*/
|
||||||
|
def get(locale: Locale, version: PenguinVersion): Tokenizer =
|
||||||
|
TokenizerFactories(version).forLocale(locale)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Encapsulates the configuration and use of [[TwitterTextTokenizer]]
|
||||||
|
* and [[TwitterTextNormalizer]].
|
||||||
|
*/
|
||||||
|
private[this] class TokenizerFactory(version: PenguinVersion) {
|
||||||
|
// The normalizer is thread-safe, so share one instance.
|
||||||
|
private[this] val normalizer =
|
||||||
|
(new TwitterTextNormalizer.Builder(version)).build()
|
||||||
|
|
||||||
|
// The TwitterTextTokenizer is relatively expensive to build,
|
||||||
|
// and is not thread safe, so keep instances of it in a
|
||||||
|
// ThreadLocal.
|
||||||
|
private[this] val local =
|
||||||
|
new ThreadLocal[TwitterTextTokenizer] {
|
||||||
|
override def initialValue: TwitterTextTokenizer =
|
||||||
|
(new TwitterTextTokenizer.Builder(version)).build()
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Obtain a [[Tokenizer]] for this combination of [[PenguinVersion]]
|
||||||
|
* and [[Locale]].
|
||||||
|
*/
|
||||||
|
def forLocale(locale: Locale): Tokenizer =
|
||||||
|
new Tokenizer {
|
||||||
|
override def tokenize(input: String): TokenSequence = {
|
||||||
|
val stream = local.get.getTwitterTokenStreamFor(locale)
|
||||||
|
stream.reset(normalizer.normalize(input, locale))
|
||||||
|
val builder = IndexedSeq.newBuilder[CharSequence]
|
||||||
|
while (stream.incrementToken) builder += stream.term()
|
||||||
|
TokenSequence(builder.result())
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Since there are a small number of Penguin versions, eagerly
|
||||||
|
* initialize a TokenizerFactory for each version, to avoid managing
|
||||||
|
* mutable state.
|
||||||
|
*/
|
||||||
|
private[this] val TokenizerFactories: PenguinVersion => TokenizerFactory =
|
||||||
|
PenguinVersion.values.map(v => v -> new TokenizerFactory(v)).toMap
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The set of locales used in warmup. These locales are mentioned in
|
||||||
|
* the logic of TwitterTextTokenizer and TwitterTextNormalizer.
|
||||||
|
*/
|
||||||
|
private[this] val WarmUpLocales: Seq[Locale] =
|
||||||
|
Seq
|
||||||
|
.concat(
|
||||||
|
Seq(
|
||||||
|
Locale.JAPANESE,
|
||||||
|
Locale.KOREAN,
|
||||||
|
LocaleUtil.UNKNOWN,
|
||||||
|
LocaleUtil.THAI,
|
||||||
|
LocaleUtil.ARABIC,
|
||||||
|
LocaleUtil.SWEDISH
|
||||||
|
),
|
||||||
|
LocaleUtil.CHINESE_JAPANESE_LOCALES.asScala,
|
||||||
|
LocaleUtil.CJK_LOCALES.asScala
|
||||||
|
)
|
||||||
|
.toSet
|
||||||
|
.toArray
|
||||||
|
.toSeq
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Load the default inputs that are used for warming up this library.
|
||||||
|
*/
|
||||||
|
def warmUpCorpus(): Seq[String] = {
|
||||||
|
val stream = getClass.getResourceAsStream("warmup-text.txt")
|
||||||
|
val bytes =
|
||||||
|
try StreamIO.buffer(stream)
|
||||||
|
finally stream.close()
|
||||||
|
bytes.toString("UTF-8").linesIterator.toArray.toSeq
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Exercise the functionality of this library on the specified
|
||||||
|
* strings. In general, prefer [[warmUp]] to this method.
|
||||||
|
*/
|
||||||
|
def warmUpWith(ver: PenguinVersion, texts: Iterable[String]): Unit =
|
||||||
|
texts.foreach { txt =>
|
||||||
|
// Exercise each locale
|
||||||
|
WarmUpLocales.foreach { loc =>
|
||||||
|
Tokenizer.get(loc, ver).tokenize(txt)
|
||||||
|
UserMutes.builder().withPenguinVersion(ver).withLocale(loc).validate(txt)
|
||||||
|
}
|
||||||
|
|
||||||
|
// Exercise language detection
|
||||||
|
TweetTokenizer.get(ver).tokenize(txt)
|
||||||
|
UserMutes.builder().withPenguinVersion(ver).validate(txt)
|
||||||
|
}
|
||||||
|
|
||||||
|
private[this] val warmUpOnce = Once(warmUpWith(DefaultPenguinVersion, warmUpCorpus()))
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The creation of the first TwitterTextTokenizer is relatively
|
||||||
|
* expensive, and tokenizing some texts may cause significant
|
||||||
|
* initialization.
|
||||||
|
*
|
||||||
|
* This method exercises the functionality of this library
|
||||||
|
* with a range of texts in order to perform as much initialization as
|
||||||
|
* possible before the library is used in a latency-sensitive way.
|
||||||
|
*
|
||||||
|
* The warmup routine will only run once. Subsequent invocations of
|
||||||
|
* `warmUp` will no do additional work, and will return once warmup is
|
||||||
|
* complete.
|
||||||
|
*
|
||||||
|
* The warmup will take on the order of seconds.
|
||||||
|
*/
|
||||||
|
def warmUp(): Unit = warmUpOnce()
|
||||||
|
}
|
@ -0,0 +1,45 @@
|
|||||||
|
package com.twitter.tweetypie.matching
|
||||||
|
|
||||||
|
import com.twitter.common.text.pipeline.TwitterLanguageIdentifier
|
||||||
|
import com.twitter.common_internal.text.version.PenguinVersion
|
||||||
|
import java.util.Locale
|
||||||
|
|
||||||
|
object TweetTokenizer extends Tokenizer {
|
||||||
|
type LocalePicking = Option[Locale] => Tokenizer
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get a Tokenizer-producing function that uses the supplied locale
|
||||||
|
* to select an appropriate Tokenizer.
|
||||||
|
*/
|
||||||
|
def localePicking: LocalePicking = {
|
||||||
|
case None => TweetTokenizer
|
||||||
|
case Some(locale) => Tokenizer.forLocale(locale)
|
||||||
|
}
|
||||||
|
|
||||||
|
private[this] val tweetLangIdentifier =
|
||||||
|
(new TwitterLanguageIdentifier.Builder).buildForTweet()
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get a Tokenizer that performs Tweet language detection, and uses
|
||||||
|
* that result to tokenize the text. If you already know the locale of
|
||||||
|
* the tweet text, use `Tokenizer.get`, because it's much
|
||||||
|
* cheaper.
|
||||||
|
*/
|
||||||
|
def get(version: PenguinVersion): Tokenizer =
|
||||||
|
new Tokenizer {
|
||||||
|
override def tokenize(text: String): TokenSequence = {
|
||||||
|
val locale = tweetLangIdentifier.identify(text).getLocale
|
||||||
|
Tokenizer.get(locale, version).tokenize(text)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private[this] val Default = get(Tokenizer.DefaultPenguinVersion)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tokenize the given text using Tweet language detection and
|
||||||
|
* `Tokenizer.DefaultPenguinVersion`. Prefer `Tokenizer.forLocale` if
|
||||||
|
* you already know the language of the text.
|
||||||
|
*/
|
||||||
|
override def tokenize(tweetText: String): TokenSequence =
|
||||||
|
Default.tokenize(tweetText)
|
||||||
|
}
|
@ -0,0 +1,128 @@
|
|||||||
|
package com.twitter.tweetypie.matching
|
||||||
|
|
||||||
|
import com.twitter.common.text.pipeline.TwitterLanguageIdentifier
|
||||||
|
import com.twitter.common_internal.text.version.PenguinVersion
|
||||||
|
import java.util.Locale
|
||||||
|
import scala.collection.JavaConversions.asScalaBuffer
|
||||||
|
|
||||||
|
object UserMutesBuilder {
|
||||||
|
private[matching] val Default =
|
||||||
|
new UserMutesBuilder(Tokenizer.DefaultPenguinVersion, None)
|
||||||
|
|
||||||
|
private val queryLangIdentifier =
|
||||||
|
(new TwitterLanguageIdentifier.Builder).buildForQuery()
|
||||||
|
}
|
||||||
|
|
||||||
|
class UserMutesBuilder private (penguinVersion: PenguinVersion, localeOpt: Option[Locale]) {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Use the specified Penguin version when tokenizing a keyword mute
|
||||||
|
* string. In general, use the default version, unless you need to
|
||||||
|
* specify a particular version for compatibility with another system
|
||||||
|
* that is using that version.
|
||||||
|
*/
|
||||||
|
def withPenguinVersion(ver: PenguinVersion): UserMutesBuilder =
|
||||||
|
if (ver == penguinVersion) this
|
||||||
|
else new UserMutesBuilder(ver, localeOpt)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Use the specified locale when tokenizing a keyword mute string.
|
||||||
|
*/
|
||||||
|
def withLocale(locale: Locale): UserMutesBuilder =
|
||||||
|
if (localeOpt.contains(locale)) this
|
||||||
|
else new UserMutesBuilder(penguinVersion, Some(locale))
|
||||||
|
|
||||||
|
/**
|
||||||
|
* When tokenizing a user mute list, detect the language of the
|
||||||
|
* text. This is significantly more expensive than using a predefined
|
||||||
|
* locale, but is appropriate when the locale is not yet known.
|
||||||
|
*/
|
||||||
|
def detectLocale(): UserMutesBuilder =
|
||||||
|
if (localeOpt.isEmpty) this
|
||||||
|
else new UserMutesBuilder(penguinVersion, localeOpt)
|
||||||
|
|
||||||
|
private[this] lazy val tokenizer =
|
||||||
|
localeOpt match {
|
||||||
|
case None =>
|
||||||
|
// No locale was specified, so use a Tokenizer that performs
|
||||||
|
// language detection before tokenizing.
|
||||||
|
new Tokenizer {
|
||||||
|
override def tokenize(text: String): TokenSequence = {
|
||||||
|
val locale = UserMutesBuilder.queryLangIdentifier.identify(text).getLocale
|
||||||
|
Tokenizer.get(locale, penguinVersion).tokenize(text)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
case Some(locale) =>
|
||||||
|
Tokenizer.get(locale, penguinVersion)
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Given a list of the user's raw keyword mutes, return a preprocessed
|
||||||
|
* set of mutes suitable for matching against tweet text. If the input
|
||||||
|
* contains any phrases that fail validation, then they will be
|
||||||
|
* dropped.
|
||||||
|
*/
|
||||||
|
def build(rawInput: Seq[String]): UserMutes =
|
||||||
|
UserMutes(rawInput.flatMap(validate(_).right.toOption))
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Java-friendly API for processing a user's list of raw keyword mutes
|
||||||
|
* into a preprocessed form suitable for matching against text.
|
||||||
|
*/
|
||||||
|
def fromJavaList(rawInput: java.util.List[String]): UserMutes =
|
||||||
|
build(asScalaBuffer(rawInput).toSeq)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Validate the raw user input muted phrase. Currently, the only
|
||||||
|
* inputs that are not valid for keyword muting are those inputs that
|
||||||
|
* do not contain any keywords, because those inputs would match all
|
||||||
|
* tweets.
|
||||||
|
*/
|
||||||
|
def validate(mutedPhrase: String): Either[UserMutes.ValidationError, TokenSequence] = {
|
||||||
|
val keywords = tokenizer.tokenize(mutedPhrase)
|
||||||
|
if (keywords.isEmpty) UserMutes.EmptyPhraseError else Right(keywords)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
object UserMutes {
|
||||||
|
sealed trait ValidationError
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The phrase's tokenization did not produce any tokens
|
||||||
|
*/
|
||||||
|
case object EmptyPhrase extends ValidationError
|
||||||
|
|
||||||
|
private[matching] val EmptyPhraseError = Left(EmptyPhrase)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get a [[UserMutesBuilder]] that uses the default Penguin version and
|
||||||
|
* performs language identification to choose a locale.
|
||||||
|
*/
|
||||||
|
def builder(): UserMutesBuilder = UserMutesBuilder.Default
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A user's muted keyword list, preprocessed into token sequences.
|
||||||
|
*/
|
||||||
|
case class UserMutes private[matching] (toSeq: Seq[TokenSequence]) {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Do any of the users' muted keyword sequences occur within the
|
||||||
|
* supplied text?
|
||||||
|
*/
|
||||||
|
def matches(text: TokenSequence): Boolean =
|
||||||
|
toSeq.exists(text.containsKeywordSequence)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Find all positions of matching muted keyword from the user's
|
||||||
|
* muted keyword list
|
||||||
|
*/
|
||||||
|
def find(text: TokenSequence): Seq[Int] =
|
||||||
|
toSeq.zipWithIndex.collect {
|
||||||
|
case (token, index) if text.containsKeywordSequence(token) => index
|
||||||
|
}
|
||||||
|
|
||||||
|
def isEmpty: Boolean = toSeq.isEmpty
|
||||||
|
def nonEmpty: Boolean = toSeq.nonEmpty
|
||||||
|
}
|
17
tweetypie/common/src/scala/com/twitter/tweetypie/media/BUILD
Normal file
17
tweetypie/common/src/scala/com/twitter/tweetypie/media/BUILD
Normal file
@ -0,0 +1,17 @@
|
|||||||
|
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",
|
||||||
|
"scrooge/scrooge-core/src/main/scala",
|
||||||
|
"tweetypie/servo/util/src/main/scala",
|
||||||
|
"tweetypie/common/src/thrift/com/twitter/tweetypie:media-entity-scala",
|
||||||
|
"tweetypie/common/src/thrift/com/twitter/tweetypie:tweet-scala",
|
||||||
|
"tco-util",
|
||||||
|
"tweetypie/common/src/scala/com/twitter/tweetypie/util",
|
||||||
|
"util/util-logging/src/main/scala/com/twitter/logging",
|
||||||
|
],
|
||||||
|
)
|
@ -0,0 +1,149 @@
|
|||||||
|
package com.twitter.tweetypie
|
||||||
|
package media
|
||||||
|
|
||||||
|
import com.twitter.mediaservices.commons.thriftscala.MediaCategory
|
||||||
|
import com.twitter.mediaservices.commons.tweetmedia.thriftscala._
|
||||||
|
import com.twitter.tco_util.TcoSlug
|
||||||
|
import com.twitter.tweetypie.thriftscala._
|
||||||
|
import com.twitter.tweetypie.util.TweetLenses
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A smörgåsbord of media-related helper methods.
|
||||||
|
*/
|
||||||
|
object Media {
|
||||||
|
val AnimatedGifContentType = "video/mp4 codecs=avc1.42E0"
|
||||||
|
|
||||||
|
case class MediaTco(expandedUrl: String, url: String, displayUrl: String)
|
||||||
|
|
||||||
|
val ImageContentTypes: Set[MediaContentType] =
|
||||||
|
Set[MediaContentType](
|
||||||
|
MediaContentType.ImageJpeg,
|
||||||
|
MediaContentType.ImagePng,
|
||||||
|
MediaContentType.ImageGif
|
||||||
|
)
|
||||||
|
|
||||||
|
val AnimatedGifContentTypes: Set[MediaContentType] =
|
||||||
|
Set[MediaContentType](
|
||||||
|
MediaContentType.VideoMp4
|
||||||
|
)
|
||||||
|
|
||||||
|
val VideoContentTypes: Set[MediaContentType] =
|
||||||
|
Set[MediaContentType](
|
||||||
|
MediaContentType.VideoGeneric
|
||||||
|
)
|
||||||
|
|
||||||
|
val InUseContentTypes: Set[MediaContentType] =
|
||||||
|
Set[MediaContentType](
|
||||||
|
MediaContentType.ImageGif,
|
||||||
|
MediaContentType.ImageJpeg,
|
||||||
|
MediaContentType.ImagePng,
|
||||||
|
MediaContentType.VideoMp4,
|
||||||
|
MediaContentType.VideoGeneric
|
||||||
|
)
|
||||||
|
|
||||||
|
def isImage(contentType: MediaContentType): Boolean =
|
||||||
|
ImageContentTypes.contains(contentType)
|
||||||
|
|
||||||
|
def contentTypeToString(contentType: MediaContentType): String =
|
||||||
|
contentType match {
|
||||||
|
case MediaContentType.ImageGif => "image/gif"
|
||||||
|
case MediaContentType.ImageJpeg => "image/jpeg"
|
||||||
|
case MediaContentType.ImagePng => "image/png"
|
||||||
|
case MediaContentType.VideoMp4 => "video/mp4"
|
||||||
|
case MediaContentType.VideoGeneric => "video"
|
||||||
|
case _ => throw new IllegalArgumentException(s"UnknownMediaContentType: $contentType")
|
||||||
|
}
|
||||||
|
|
||||||
|
def stringToContentType(str: String): MediaContentType =
|
||||||
|
str match {
|
||||||
|
case "image/gif" => MediaContentType.ImageGif
|
||||||
|
case "image/jpeg" => MediaContentType.ImageJpeg
|
||||||
|
case "image/png" => MediaContentType.ImagePng
|
||||||
|
case "video/mp4" => MediaContentType.VideoMp4
|
||||||
|
case "video" => MediaContentType.VideoGeneric
|
||||||
|
case _ => throw new IllegalArgumentException(s"Unknown Content Type String: $str")
|
||||||
|
}
|
||||||
|
|
||||||
|
def extensionForContentType(cType: MediaContentType): String =
|
||||||
|
cType match {
|
||||||
|
case MediaContentType.ImageJpeg => "jpg"
|
||||||
|
case MediaContentType.ImagePng => "png"
|
||||||
|
case MediaContentType.ImageGif => "gif"
|
||||||
|
case MediaContentType.VideoMp4 => "mp4"
|
||||||
|
case MediaContentType.VideoGeneric => ""
|
||||||
|
case _ => "unknown"
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Extract a URL entity from a media entity.
|
||||||
|
*/
|
||||||
|
def extractUrlEntity(mediaEntity: MediaEntity): UrlEntity =
|
||||||
|
UrlEntity(
|
||||||
|
fromIndex = mediaEntity.fromIndex,
|
||||||
|
toIndex = mediaEntity.toIndex,
|
||||||
|
url = mediaEntity.url,
|
||||||
|
expanded = Some(mediaEntity.expandedUrl),
|
||||||
|
display = Some(mediaEntity.displayUrl)
|
||||||
|
)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Copy the fields from the URL entity into the media entity.
|
||||||
|
*/
|
||||||
|
def copyFromUrlEntity(mediaEntity: MediaEntity, urlEntity: UrlEntity): MediaEntity = {
|
||||||
|
val expandedUrl =
|
||||||
|
urlEntity.expanded.orElse(Option(mediaEntity.expandedUrl)).getOrElse(urlEntity.url)
|
||||||
|
|
||||||
|
val displayUrl =
|
||||||
|
urlEntity.url match {
|
||||||
|
case TcoSlug(slug) => MediaUrl.Display.fromTcoSlug(slug)
|
||||||
|
case _ => urlEntity.expanded.getOrElse(urlEntity.url)
|
||||||
|
}
|
||||||
|
|
||||||
|
mediaEntity.copy(
|
||||||
|
fromIndex = urlEntity.fromIndex,
|
||||||
|
toIndex = urlEntity.toIndex,
|
||||||
|
url = urlEntity.url,
|
||||||
|
expandedUrl = expandedUrl,
|
||||||
|
displayUrl = displayUrl
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
|
def getAspectRatio(size: MediaSize): AspectRatio =
|
||||||
|
getAspectRatio(size.width, size.height)
|
||||||
|
|
||||||
|
def getAspectRatio(width: Int, height: Int): AspectRatio = {
|
||||||
|
if (width == 0 || height == 0) {
|
||||||
|
throw new IllegalArgumentException(s"Dimensions must be non zero: ($width, $height)")
|
||||||
|
}
|
||||||
|
|
||||||
|
def calculateGcd(a: Int, b: Int): Int =
|
||||||
|
if (b == 0) a else calculateGcd(b, a % b)
|
||||||
|
|
||||||
|
val gcd = calculateGcd(math.max(width, height), math.min(width, height))
|
||||||
|
AspectRatio((width / gcd).toShort, (height / gcd).toShort)
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return just the media that belongs to this tweet
|
||||||
|
*/
|
||||||
|
def ownMedia(tweet: Tweet): Seq[MediaEntity] =
|
||||||
|
TweetLenses.media.get(tweet).filter(isOwnMedia(tweet.id, _))
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Does the given media entity, which is was found on the tweet with the specified
|
||||||
|
* tweetId, belong to that tweet?
|
||||||
|
*/
|
||||||
|
def isOwnMedia(tweetId: TweetId, entity: MediaEntity): Boolean =
|
||||||
|
entity.sourceStatusId.forall(_ == tweetId)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Mixed Media is any case where there is more than one media item & any of them is not an image.
|
||||||
|
*/
|
||||||
|
|
||||||
|
def isMixedMedia(mediaEntities: Seq[MediaEntity]): Boolean =
|
||||||
|
mediaEntities.length > 1 && (mediaEntities.flatMap(_.mediaInfo).exists {
|
||||||
|
case _: MediaInfo.ImageInfo => false
|
||||||
|
case _ => true
|
||||||
|
} ||
|
||||||
|
mediaEntities.flatMap(_.mediaKey).map(_.mediaCategory).exists(_ != MediaCategory.TweetImage))
|
||||||
|
}
|
@ -0,0 +1,108 @@
|
|||||||
|
package com.twitter.tweetypie
|
||||||
|
package media
|
||||||
|
|
||||||
|
import com.twitter.logging.Logger
|
||||||
|
import com.twitter.tweetypie.thriftscala.MediaEntity
|
||||||
|
import com.twitter.tweetypie.thriftscala.UrlEntity
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creating and parsing tweet media entity URLs.
|
||||||
|
*
|
||||||
|
* There are four kinds of URL in a media entity:
|
||||||
|
*
|
||||||
|
* - Display URLs: pic.twitter.com aliases for the short URL, for
|
||||||
|
* embedding in the tweet text.
|
||||||
|
*
|
||||||
|
* - Short URLs: regular t.co URLs that expand to the permalink URL.
|
||||||
|
*
|
||||||
|
* - Permalink URLs: link to a page that displays the media after
|
||||||
|
* doing authorization
|
||||||
|
*
|
||||||
|
* - Asset URLs: links to the actual media asset.
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
object MediaUrl {
|
||||||
|
private[this] val log = Logger(getClass)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The URL that should be filled in to the displayUrl field of the
|
||||||
|
* media entity. This URL behaves exactly the same as a t.co link
|
||||||
|
* (only the domain is different.)
|
||||||
|
*/
|
||||||
|
object Display {
|
||||||
|
val Root = "pic.twitter.com/"
|
||||||
|
|
||||||
|
def fromTcoSlug(tcoSlug: String): String = Root + tcoSlug
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The link target for the link in the tweet text (the expanded URL
|
||||||
|
* for the media, copied from the URL entity.) For native photos,
|
||||||
|
* this is the tweet permalink page.
|
||||||
|
*
|
||||||
|
* For users without a screen name ("handleless" or NoScreenName users)
|
||||||
|
* a permalink to /i/status/:tweet_id is used.
|
||||||
|
*/
|
||||||
|
object Permalink {
|
||||||
|
val Root = "https://twitter.com/"
|
||||||
|
val Internal = "i"
|
||||||
|
val PhotoSuffix = "/photo/1"
|
||||||
|
val VideoSuffix = "/video/1"
|
||||||
|
|
||||||
|
def apply(screenName: String, tweetId: TweetId, isVideo: Boolean): String =
|
||||||
|
Root +
|
||||||
|
(if (screenName.isEmpty) Internal else screenName) +
|
||||||
|
"/status/" +
|
||||||
|
tweetId +
|
||||||
|
(if (isVideo) VideoSuffix else PhotoSuffix)
|
||||||
|
|
||||||
|
private[this] val PermalinkRegex =
|
||||||
|
"""https?://twitter.com/(?:#!/)?\w+/status/(\d+)/(?:photo|video)/\d+""".r
|
||||||
|
|
||||||
|
private[this] def getTweetId(permalink: String): Option[TweetId] =
|
||||||
|
permalink match {
|
||||||
|
case PermalinkRegex(tweetIdStr) =>
|
||||||
|
try {
|
||||||
|
Some(tweetIdStr.toLong)
|
||||||
|
} catch {
|
||||||
|
// Digits too big to fit in a Long
|
||||||
|
case _: NumberFormatException => None
|
||||||
|
}
|
||||||
|
case _ => None
|
||||||
|
}
|
||||||
|
|
||||||
|
def getTweetId(urlEntity: UrlEntity): Option[TweetId] =
|
||||||
|
urlEntity.expanded.flatMap(getTweetId)
|
||||||
|
|
||||||
|
def hasTweetId(permalink: String, tweetId: TweetId): Boolean =
|
||||||
|
getTweetId(permalink).contains(tweetId)
|
||||||
|
|
||||||
|
def hasTweetId(mediaEntity: MediaEntity, tweetId: TweetId): Boolean =
|
||||||
|
hasTweetId(mediaEntity.expandedUrl, tweetId)
|
||||||
|
|
||||||
|
def hasTweetId(urlEntity: UrlEntity, tweetId: TweetId): Boolean =
|
||||||
|
getTweetId(urlEntity).contains(tweetId)
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Converts a url that starts with "https://" to one that starts with "http://".
|
||||||
|
*/
|
||||||
|
def httpsToHttp(url: String): String =
|
||||||
|
url.replace("https://", "http://")
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Gets the last path element from an asset url. This exists temporarily to support
|
||||||
|
* the now deprecated mediaPath element in MediaEntity.
|
||||||
|
*/
|
||||||
|
def mediaPathFromUrl(url: String): String =
|
||||||
|
url.lastIndexOf('/') match {
|
||||||
|
case -1 =>
|
||||||
|
log.error("Invalid media path. Could not find last element: " + url)
|
||||||
|
// Better to return a broken preview URL to the client
|
||||||
|
// than to fail the whole request.
|
||||||
|
""
|
||||||
|
|
||||||
|
case idx =>
|
||||||
|
url.substring(idx + 1)
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,7 @@
|
|||||||
|
package com.twitter.tweetypie
|
||||||
|
|
||||||
|
package object media {
|
||||||
|
type TweetId = Long
|
||||||
|
type UserId = Long
|
||||||
|
type MediaId = Long
|
||||||
|
}
|
@ -0,0 +1,80 @@
|
|||||||
|
package com.twitter.tweetypie.storage
|
||||||
|
|
||||||
|
import com.twitter.finagle.stats.StatsReceiver
|
||||||
|
import com.twitter.stitch.Stitch
|
||||||
|
import com.twitter.storage.client.manhattan.kv.ManhattanValue
|
||||||
|
import com.twitter.tweetypie.storage.TweetUtils.collectWithRateLimitCheck
|
||||||
|
import com.twitter.tweetypie.storage_internal.thriftscala.StoredTweet
|
||||||
|
import com.twitter.tweetypie.thriftscala.Tweet
|
||||||
|
import com.twitter.util.Time
|
||||||
|
|
||||||
|
object AddTweetHandler {
|
||||||
|
private[storage] type InternalAddTweet = (
|
||||||
|
Tweet,
|
||||||
|
ManhattanOperations.Insert,
|
||||||
|
Scribe,
|
||||||
|
StatsReceiver,
|
||||||
|
Time
|
||||||
|
) => Stitch[Unit]
|
||||||
|
|
||||||
|
def apply(
|
||||||
|
insert: ManhattanOperations.Insert,
|
||||||
|
scribe: Scribe,
|
||||||
|
stats: StatsReceiver
|
||||||
|
): TweetStorageClient.AddTweet =
|
||||||
|
tweet => doAddTweet(tweet, insert, scribe, stats, Time.now)
|
||||||
|
|
||||||
|
def makeRecords(
|
||||||
|
storedTweet: StoredTweet,
|
||||||
|
timestamp: Time
|
||||||
|
): Seq[TweetManhattanRecord] = {
|
||||||
|
val core = CoreFieldsCodec.fromTweet(storedTweet)
|
||||||
|
val packedCoreFieldsBlob = CoreFieldsCodec.toTFieldBlob(core)
|
||||||
|
val coreRecord =
|
||||||
|
TweetManhattanRecord(
|
||||||
|
TweetKey.coreFieldsKey(storedTweet.id),
|
||||||
|
ManhattanValue(TFieldBlobCodec.toByteBuffer(packedCoreFieldsBlob), Some(timestamp))
|
||||||
|
)
|
||||||
|
|
||||||
|
val otherFieldIds =
|
||||||
|
TweetFields.nonCoreInternalFields ++ TweetFields.getAdditionalFieldIds(storedTweet)
|
||||||
|
|
||||||
|
val otherFields =
|
||||||
|
storedTweet
|
||||||
|
.getFieldBlobs(otherFieldIds)
|
||||||
|
.map {
|
||||||
|
case (fieldId, tFieldBlob) =>
|
||||||
|
TweetManhattanRecord(
|
||||||
|
TweetKey.fieldKey(storedTweet.id, fieldId),
|
||||||
|
ManhattanValue(TFieldBlobCodec.toByteBuffer(tFieldBlob), Some(timestamp))
|
||||||
|
)
|
||||||
|
}
|
||||||
|
.toSeq
|
||||||
|
otherFields :+ coreRecord
|
||||||
|
}
|
||||||
|
|
||||||
|
private[storage] val doAddTweet: InternalAddTweet = (
|
||||||
|
tweet: Tweet,
|
||||||
|
insert: ManhattanOperations.Insert,
|
||||||
|
scribe: Scribe,
|
||||||
|
stats: StatsReceiver,
|
||||||
|
timestamp: Time
|
||||||
|
) => {
|
||||||
|
assert(tweet.coreData.isDefined, s"Tweet ${tweet.id} is missing coreData: $tweet")
|
||||||
|
|
||||||
|
val storedTweet = StorageConversions.toStoredTweet(tweet)
|
||||||
|
val records = makeRecords(storedTweet, timestamp)
|
||||||
|
val inserts = records.map(insert)
|
||||||
|
val insertsWithRateLimitCheck =
|
||||||
|
Stitch.collect(inserts.map(_.liftToTry)).map(collectWithRateLimitCheck).lowerFromTry
|
||||||
|
|
||||||
|
Stats.updatePerFieldQpsCounters(
|
||||||
|
"addTweet",
|
||||||
|
TweetFields.getAdditionalFieldIds(storedTweet),
|
||||||
|
1,
|
||||||
|
stats
|
||||||
|
)
|
||||||
|
|
||||||
|
insertsWithRateLimitCheck.unit.onSuccess { _ => scribe.logAdded(storedTweet) }
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,47 @@
|
|||||||
|
scala_library(
|
||||||
|
sources = ["*.scala"],
|
||||||
|
compiler_option_sets = ["fatal_warnings"],
|
||||||
|
platform = "java8",
|
||||||
|
strict_deps = True,
|
||||||
|
tags = [
|
||||||
|
"bazel-compatible",
|
||||||
|
"bazel-incompatible-scaladoc",
|
||||||
|
],
|
||||||
|
dependencies = [
|
||||||
|
"3rdparty/jvm/com/chuusai:shapeless",
|
||||||
|
"3rdparty/jvm/com/fasterxml/jackson/core:jackson-databind",
|
||||||
|
"3rdparty/jvm/com/fasterxml/jackson/module:jackson-module-scala",
|
||||||
|
"3rdparty/jvm/com/google/guava",
|
||||||
|
"3rdparty/jvm/com/twitter/bijection:core",
|
||||||
|
"3rdparty/jvm/com/twitter/bijection:scrooge",
|
||||||
|
"3rdparty/jvm/com/twitter/bijection:thrift",
|
||||||
|
"3rdparty/jvm/commons-codec",
|
||||||
|
"3rdparty/jvm/org/apache/thrift:libthrift",
|
||||||
|
"diffshow",
|
||||||
|
"finagle-internal/mtls/src/main/scala/com/twitter/finagle/mtls/authorization",
|
||||||
|
"finagle/finagle-core/src/main",
|
||||||
|
"finagle/finagle-stats",
|
||||||
|
"finagle/finagle-thriftmux/src/main/scala",
|
||||||
|
"mediaservices/commons/src/main/thrift:thrift-scala",
|
||||||
|
"scrooge/scrooge-serializer/src/main/scala",
|
||||||
|
"tweetypie/servo/repo/src/main/scala",
|
||||||
|
"tweetypie/servo/util",
|
||||||
|
"snowflake:id",
|
||||||
|
"src/thrift/com/twitter/escherbird:media-annotation-structs-scala",
|
||||||
|
"src/thrift/com/twitter/manhattan:internal-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:tweet-scala",
|
||||||
|
"stitch/stitch-core",
|
||||||
|
"storage/clients/manhattan/client/src/main/scala",
|
||||||
|
"tbird-thrift:scala",
|
||||||
|
"tweetypie/common/src/scala/com/twitter/tweetypie/additionalfields",
|
||||||
|
"tweetypie/common/src/scala/com/twitter/tweetypie/client_id",
|
||||||
|
"tweetypie/common/src/scala/com/twitter/tweetypie/util",
|
||||||
|
"tweetypie/common/src/thrift/com/twitter/tweetypie/storage_internal:storage_internal-scala",
|
||||||
|
"util-internal/scribe",
|
||||||
|
"util/util-core:scala",
|
||||||
|
"util/util-slf4j-api/src/main/scala/com/twitter/util/logging",
|
||||||
|
"util/util-stats/src/main/scala",
|
||||||
|
],
|
||||||
|
)
|
@ -0,0 +1,20 @@
|
|||||||
|
package com.twitter.tweetypie.storage
|
||||||
|
|
||||||
|
import com.twitter.util.Time
|
||||||
|
|
||||||
|
object BounceDeleteHandler {
|
||||||
|
def apply(
|
||||||
|
insert: ManhattanOperations.Insert,
|
||||||
|
scribe: Scribe
|
||||||
|
): TweetStorageClient.BounceDelete =
|
||||||
|
tweetId => {
|
||||||
|
val mhTimestamp = Time.now
|
||||||
|
val bounceDeleteRecord = TweetStateRecord
|
||||||
|
.BounceDeleted(tweetId, mhTimestamp.inMillis)
|
||||||
|
.toTweetMhRecord
|
||||||
|
|
||||||
|
insert(bounceDeleteRecord).onSuccess { _ =>
|
||||||
|
scribe.logRemoved(tweetId, mhTimestamp, isSoftDeleted = true)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,242 @@
|
|||||||
|
package com.twitter.tweetypie.storage
|
||||||
|
|
||||||
|
import com.twitter.bijection.Conversion.asMethod
|
||||||
|
import com.twitter.bijection.Injection
|
||||||
|
import com.twitter.scrooge.TFieldBlob
|
||||||
|
import com.twitter.storage.client.manhattan.kv._
|
||||||
|
import com.twitter.tweetypie.storage.Response.FieldResponse
|
||||||
|
import com.twitter.tweetypie.storage.Response.FieldResponseCode
|
||||||
|
import com.twitter.tweetypie.storage_internal.thriftscala.CoreFields
|
||||||
|
import com.twitter.tweetypie.storage_internal.thriftscala.InternalTweet
|
||||||
|
import com.twitter.tweetypie.storage_internal.thriftscala.StoredTweet
|
||||||
|
import java.io.ByteArrayOutputStream
|
||||||
|
import java.nio.ByteBuffer
|
||||||
|
import org.apache.thrift.protocol.TBinaryProtocol
|
||||||
|
import org.apache.thrift.transport.TIOStreamTransport
|
||||||
|
import org.apache.thrift.transport.TMemoryInputTransport
|
||||||
|
import scala.collection.immutable
|
||||||
|
import scala.util.control.NoStackTrace
|
||||||
|
|
||||||
|
// NOTE: All field ids and Tweet structure in this file correspond to the StoredTweet struct ONLY
|
||||||
|
|
||||||
|
object ByteArrayCodec {
|
||||||
|
def toByteBuffer(byteArray: Array[Byte]): ByteBuffer = byteArray.as[ByteBuffer]
|
||||||
|
def fromByteBuffer(buffer: ByteBuffer): Array[Byte] = buffer.as[Array[Byte]]
|
||||||
|
}
|
||||||
|
|
||||||
|
object StringCodec {
|
||||||
|
private val string2ByteBuffer = Injection.connect[String, Array[Byte], ByteBuffer]
|
||||||
|
def toByteBuffer(strValue: String): ByteBuffer = string2ByteBuffer(strValue)
|
||||||
|
def fromByteBuffer(buffer: ByteBuffer): String = string2ByteBuffer.invert(buffer).get
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Terminology
|
||||||
|
* -----------
|
||||||
|
* Tweet id field : The field number of 'tweetId' in the 'Tweet' thrift structure (i.e "1")
|
||||||
|
*
|
||||||
|
* First AdditionalField id : The ID if the first additional field in 'Tweet' thrift structure. All field Ids less than this are
|
||||||
|
* considered internal and all the ids greater than or equal to this field id are considered 'Additional fields'.
|
||||||
|
* This is set to 100.
|
||||||
|
*
|
||||||
|
* Internal Fields : Fields with ids [1 to firstAdditionalFieldid) (excluding firstAdditionalFieldId)
|
||||||
|
*
|
||||||
|
* Core fields : (Subset of Internal fields)- Fields with ids [1 to 8, 19]. These fields are "packed" together and stored
|
||||||
|
* under a single key. This key is referred to as "CoreFieldsKey" (see @TweetKeyType.CoreFieldsKey).
|
||||||
|
* Note: Actually field 1 is skipped when packing as this field is the tweet id and it need not be
|
||||||
|
* explicitly stored since the pkey already contains the tweet Id)
|
||||||
|
*
|
||||||
|
* Root Core field id : The field id under which the packed core fields are stored in Manhattan. (This is field Id "1")
|
||||||
|
*
|
||||||
|
* Required fields : (Subset of Core fields) - Fields with ids [1 to 5] that MUST be present on every tweet.
|
||||||
|
*
|
||||||
|
* Additional Fields : All fields with field ids >= 'firstAdditionalFieldId'
|
||||||
|
*
|
||||||
|
* Compiled Additional fields : (Subset of Additional Fields) - All fields that the storage library knows about
|
||||||
|
* (i.e present on the latest storage_internal.thrift that is compiled-in).
|
||||||
|
*
|
||||||
|
* Passthrough fields : (Subset of Additional Fields) - The fields on storage_internal.thrift that the storage library is NOT aware of
|
||||||
|
* These field ids are is obtained looking at the "_passThroughFields" member of the scrooge-generated
|
||||||
|
* 'Tweet' object.
|
||||||
|
*
|
||||||
|
* coreFieldsIdInInternalTweet: This is the field id of the core fields (the only field) in the Internal Tweet struct
|
||||||
|
*/
|
||||||
|
object TweetFields {
|
||||||
|
val firstAdditionalFieldId: Short = 100
|
||||||
|
val tweetIdField: Short = 1
|
||||||
|
val geoFieldId: Short = 9
|
||||||
|
|
||||||
|
// The field under which all the core field values are stored (in serialized form).
|
||||||
|
val rootCoreFieldId: Short = 1
|
||||||
|
|
||||||
|
val coreFieldIds: immutable.IndexedSeq[FieldId] = {
|
||||||
|
val quotedTweetFieldId: Short = 19
|
||||||
|
(1 to 8).map(_.toShort) ++ Seq(quotedTweetFieldId)
|
||||||
|
}
|
||||||
|
val requiredFieldIds: immutable.IndexedSeq[FieldId] = (1 to 5).map(_.toShort)
|
||||||
|
|
||||||
|
val coreFieldsIdInInternalTweet: Short = 1
|
||||||
|
|
||||||
|
val compiledAdditionalFieldIds: Seq[FieldId] =
|
||||||
|
StoredTweet.metaData.fields.filter(_.id >= firstAdditionalFieldId).map(_.id)
|
||||||
|
val internalFieldIds: Seq[FieldId] =
|
||||||
|
StoredTweet.metaData.fields.filter(_.id < firstAdditionalFieldId).map(_.id)
|
||||||
|
val nonCoreInternalFields: Seq[FieldId] =
|
||||||
|
(internalFieldIds.toSet -- coreFieldIds.toSet).toSeq
|
||||||
|
def getAdditionalFieldIds(tweet: StoredTweet): Seq[FieldId] =
|
||||||
|
compiledAdditionalFieldIds ++ tweet._passthroughFields.keys.toSeq
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Helper object to convert TFieldBlob to ByteBuffer that gets stored in Manhattan.
|
||||||
|
*
|
||||||
|
* The following is the format in which the TFieldBlob gets stored:
|
||||||
|
* [Version][TField][TFieldBlob]
|
||||||
|
*/
|
||||||
|
object TFieldBlobCodec {
|
||||||
|
val BinaryProtocolFactory: TBinaryProtocol.Factory = new TBinaryProtocol.Factory()
|
||||||
|
val FormatVersion = 1.0
|
||||||
|
|
||||||
|
def toByteBuffer(tFieldBlob: TFieldBlob): ByteBuffer = {
|
||||||
|
val baos = new ByteArrayOutputStream()
|
||||||
|
val prot = BinaryProtocolFactory.getProtocol(new TIOStreamTransport(baos))
|
||||||
|
|
||||||
|
prot.writeDouble(FormatVersion)
|
||||||
|
prot.writeFieldBegin(tFieldBlob.field)
|
||||||
|
prot.writeBinary(ByteArrayCodec.toByteBuffer(tFieldBlob.data))
|
||||||
|
|
||||||
|
ByteArrayCodec.toByteBuffer(baos.toByteArray)
|
||||||
|
}
|
||||||
|
|
||||||
|
def fromByteBuffer(buffer: ByteBuffer): TFieldBlob = {
|
||||||
|
val byteArray = ByteArrayCodec.fromByteBuffer(buffer)
|
||||||
|
val prot = BinaryProtocolFactory.getProtocol(new TMemoryInputTransport(byteArray))
|
||||||
|
|
||||||
|
val version = prot.readDouble()
|
||||||
|
if (version != FormatVersion) {
|
||||||
|
throw new VersionMismatchError(
|
||||||
|
"Version mismatch in decoding ByteBuffer to TFieldBlob. " +
|
||||||
|
"Actual version: " + version + ". Expected version: " + FormatVersion
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
|
val tField = prot.readFieldBegin()
|
||||||
|
val dataBuffer = prot.readBinary()
|
||||||
|
val data = ByteArrayCodec.fromByteBuffer(dataBuffer)
|
||||||
|
|
||||||
|
TFieldBlob(tField, data)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Helper object to help convert 'CoreFields' object to/from TFieldBlob (and also to construct
|
||||||
|
* 'CoreFields' object from a 'StoredTweet' object)
|
||||||
|
*
|
||||||
|
* More details:
|
||||||
|
* - A subset of fields on the 'StoredTweet' thrift structure (2-8,19) are 'packaged' and stored
|
||||||
|
* together as a serialized TFieldBlob object under a single key in Manhattan (see TweetKeyCodec
|
||||||
|
* helper object above for more details).
|
||||||
|
*
|
||||||
|
* - To make the packing/unpacking the fields to/from TFieldBlob object, we created the following
|
||||||
|
* two helper thrift structures 'CoreFields' and 'InternalTweet'
|
||||||
|
*
|
||||||
|
* // The field Ids and types here MUST exactly match field Ids on 'StoredTweet' thrift structure.
|
||||||
|
* struct CoreFields {
|
||||||
|
* 2: optional i64 user_id
|
||||||
|
* ...
|
||||||
|
* 8: optional i64 contributor_id
|
||||||
|
* ...
|
||||||
|
* 19: optional StoredQuotedTweet stored_quoted_tweet
|
||||||
|
*
|
||||||
|
* }
|
||||||
|
*
|
||||||
|
* // The field id of core fields MUST be "1"
|
||||||
|
* struct InternalTweet {
|
||||||
|
* 1: CoreFields coreFields
|
||||||
|
* }
|
||||||
|
*
|
||||||
|
* - Given the above two structures, packing/unpacking fields (2-8,19) on StoredTweet object into a TFieldBlob
|
||||||
|
* becomes very trivial:
|
||||||
|
* For packing:
|
||||||
|
* (i) Copy fields (2-8,19) from StoredTweet object to a new CoreFields object
|
||||||
|
* (ii) Create a new InternalTweet object with the 'CoreFields' object constructed in step (i) above
|
||||||
|
* (iii) Extract field "1" as a TFieldBlob from InternalField (by calling the scrooge generated "getFieldBlob(1)"
|
||||||
|
* function on the InternalField objecton
|
||||||
|
*
|
||||||
|
* For unpacking:
|
||||||
|
* (i) Create an empty 'InternalField' object
|
||||||
|
* (ii) Call scrooge-generated 'setField' by passing the tFieldBlob blob (created by packing steps above)
|
||||||
|
* (iii) Doing step (ii) above will create a hydrated 'CoreField' object that can be accessed by 'coreFields'
|
||||||
|
* member of 'InternalTweet' object.
|
||||||
|
*/
|
||||||
|
object CoreFieldsCodec {
|
||||||
|
val coreFieldIds: Seq[FieldId] = CoreFields.metaData.fields.map(_.id)
|
||||||
|
|
||||||
|
// "Pack" the core fields i.e converts 'CoreFields' object to "packed" tFieldBlob (See description
|
||||||
|
// above for more details)
|
||||||
|
def toTFieldBlob(coreFields: CoreFields): TFieldBlob = {
|
||||||
|
InternalTweet(Some(coreFields)).getFieldBlob(TweetFields.coreFieldsIdInInternalTweet).get
|
||||||
|
}
|
||||||
|
|
||||||
|
// "Unpack" the core fields from a packed TFieldBlob into a CoreFields object (see description above for
|
||||||
|
// more details)
|
||||||
|
def fromTFieldBlob(tFieldBlob: TFieldBlob): CoreFields = {
|
||||||
|
InternalTweet().setField(tFieldBlob).coreFields.get
|
||||||
|
}
|
||||||
|
|
||||||
|
// "Unpack" the core fields from a packed TFieldBlob into a Map of core-fieldId-> TFieldBlob
|
||||||
|
def unpackFields(tFieldBlob: TFieldBlob): Map[Short, TFieldBlob] =
|
||||||
|
fromTFieldBlob(tFieldBlob).getFieldBlobs(coreFieldIds)
|
||||||
|
|
||||||
|
// Create a 'CoreFields' thrift object from 'Tweet' thrift object.
|
||||||
|
def fromTweet(tweet: StoredTweet): CoreFields = {
|
||||||
|
// As mentioned above, the field ids and types on the 'CoreFields' struct exactly match the
|
||||||
|
// corresponding fields on StoredTweet structure. So it is safe to call .getField() on Tweet object and
|
||||||
|
// and pass the returned tFleldBlob a 'setField' on 'CoreFields' object.
|
||||||
|
coreFieldIds.foldLeft(CoreFields()) {
|
||||||
|
case (core, fieldId) =>
|
||||||
|
tweet.getFieldBlob(fieldId) match {
|
||||||
|
case None => core
|
||||||
|
case Some(tFieldBlob) => core.setField(tFieldBlob)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Helper object to convert ManhattanException to FieldResponseCode thrift object
|
||||||
|
*/
|
||||||
|
object FieldResponseCodeCodec {
|
||||||
|
import FieldResponseCodec.ValueNotFoundException
|
||||||
|
|
||||||
|
def fromManhattanException(mhException: ManhattanException): FieldResponseCode = {
|
||||||
|
mhException match {
|
||||||
|
case _: ValueNotFoundException => FieldResponseCode.ValueNotFound
|
||||||
|
case _: InternalErrorManhattanException => FieldResponseCode.Error
|
||||||
|
case _: InvalidRequestManhattanException => FieldResponseCode.InvalidRequest
|
||||||
|
case _: DeniedManhattanException => FieldResponseCode.Error
|
||||||
|
case _: UnsatisfiableManhattanException => FieldResponseCode.Error
|
||||||
|
case _: TimeoutManhattanException => FieldResponseCode.Timeout
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Helper object to construct FieldResponse thrift object from an Exception.
|
||||||
|
* This is typically called to convert 'ManhattanException' object to 'FieldResponse' thrift object
|
||||||
|
*/
|
||||||
|
object FieldResponseCodec {
|
||||||
|
class ValueNotFoundException extends ManhattanException("Value not found!") with NoStackTrace
|
||||||
|
private[storage] val NotFound = new ValueNotFoundException
|
||||||
|
|
||||||
|
def fromThrowable(e: Throwable, additionalMsg: Option[String] = None): FieldResponse = {
|
||||||
|
val (respCode, errMsg) = e match {
|
||||||
|
case mhException: ManhattanException =>
|
||||||
|
(FieldResponseCodeCodec.fromManhattanException(mhException), mhException.getMessage)
|
||||||
|
case _ => (FieldResponseCode.Error, e.getMessage)
|
||||||
|
}
|
||||||
|
|
||||||
|
val respMsg = additionalMsg.map(_ + ". " + errMsg).orElse(Some(errMsg.toString))
|
||||||
|
FieldResponse(respCode, respMsg)
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,67 @@
|
|||||||
|
package com.twitter.tweetypie.storage
|
||||||
|
|
||||||
|
import com.twitter.finagle.stats.StatsReceiver
|
||||||
|
import com.twitter.stitch.Stitch
|
||||||
|
import com.twitter.storage.client.manhattan.kv.DeniedManhattanException
|
||||||
|
import com.twitter.tweetypie.storage.TweetUtils._
|
||||||
|
import com.twitter.util.Throw
|
||||||
|
import com.twitter.util.Time
|
||||||
|
|
||||||
|
object DeleteAdditionalFieldsHandler {
|
||||||
|
def apply(
|
||||||
|
delete: ManhattanOperations.Delete,
|
||||||
|
stats: StatsReceiver
|
||||||
|
): TweetStorageClient.DeleteAdditionalFields =
|
||||||
|
(unfilteredTweetIds: Seq[TweetId], additionalFields: Seq[Field]) => {
|
||||||
|
val tweetIds = unfilteredTweetIds.filter(_ > 0)
|
||||||
|
val additionalFieldIds = additionalFields.map(_.id)
|
||||||
|
require(additionalFields.nonEmpty, "Additional fields to delete cannot be empty")
|
||||||
|
require(
|
||||||
|
additionalFieldIds.min >= TweetFields.firstAdditionalFieldId,
|
||||||
|
s"Additional fields $additionalFields must be in additional field range (>= ${TweetFields.firstAdditionalFieldId})"
|
||||||
|
)
|
||||||
|
|
||||||
|
Stats.addWidthStat("deleteAdditionalFields", "tweetIds", tweetIds.size, stats)
|
||||||
|
Stats.addWidthStat(
|
||||||
|
"deleteAdditionalFields",
|
||||||
|
"additionalFieldIds",
|
||||||
|
additionalFieldIds.size,
|
||||||
|
stats
|
||||||
|
)
|
||||||
|
Stats.updatePerFieldQpsCounters(
|
||||||
|
"deleteAdditionalFields",
|
||||||
|
additionalFieldIds,
|
||||||
|
tweetIds.size,
|
||||||
|
stats
|
||||||
|
)
|
||||||
|
val mhTimestamp = Time.now
|
||||||
|
|
||||||
|
val stitches = tweetIds.map { tweetId =>
|
||||||
|
val (fieldIds, mhKeysToDelete) =
|
||||||
|
additionalFieldIds.map { fieldId =>
|
||||||
|
(fieldId, TweetKey.additionalFieldsKey(tweetId, fieldId))
|
||||||
|
}.unzip
|
||||||
|
|
||||||
|
val deletionStitches = mhKeysToDelete.map { mhKeyToDelete =>
|
||||||
|
delete(mhKeyToDelete, Some(mhTimestamp)).liftToTry
|
||||||
|
}
|
||||||
|
|
||||||
|
Stitch.collect(deletionStitches).map { responsesTries =>
|
||||||
|
val wasRateLimited = responsesTries.exists {
|
||||||
|
case Throw(e: DeniedManhattanException) => true
|
||||||
|
case _ => false
|
||||||
|
}
|
||||||
|
|
||||||
|
val resultsPerTweet = fieldIds.zip(responsesTries).toMap
|
||||||
|
|
||||||
|
if (wasRateLimited) {
|
||||||
|
buildTweetOverCapacityResponse("deleteAdditionalFields", tweetId, resultsPerTweet)
|
||||||
|
} else {
|
||||||
|
buildTweetResponse("deleteAdditionalFields", tweetId, resultsPerTweet)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
Stitch.collect(stitches)
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,41 @@
|
|||||||
|
package com.twitter.tweetypie.storage
|
||||||
|
|
||||||
|
import com.twitter.tweetypie.additionalfields.AdditionalFields
|
||||||
|
import com.twitter.tweetypie.storage_internal.thriftscala.StoredTweet
|
||||||
|
import com.twitter.tweetypie.thriftscala.{Tweet => TpTweet}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A field of the stored version of a tweet to read, update, or delete.
|
||||||
|
*
|
||||||
|
* There is not a one-to-one correspondence between the fields ids of
|
||||||
|
* [[com.twitter.tweetypie.thriftscala.Tweet]] and
|
||||||
|
* [[com.twitter.tweetypie.storage_internal.thriftscala.StoredTweet]]. For example, in StoredTweet,
|
||||||
|
* the nsfwUser property is field 11; in Tweet, it is a property of the coreData struct in field 2.
|
||||||
|
* To circumvent the confusion of using one set of field ids or the other, callers use instances of
|
||||||
|
* [[Field]] to reference the part of the object to modify.
|
||||||
|
*/
|
||||||
|
class Field private[storage] (val id: Short) extends AnyVal {
|
||||||
|
override def toString: String = id.toString
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* NOTE: Make sure `AllUpdatableCompiledFields` is kept up to date when adding any new field
|
||||||
|
*/
|
||||||
|
object Field {
|
||||||
|
import AdditionalFields.isAdditionalFieldId
|
||||||
|
val Geo: Field = new Field(StoredTweet.GeoField.id)
|
||||||
|
val HasTakedown: Field = new Field(StoredTweet.HasTakedownField.id)
|
||||||
|
val NsfwUser: Field = new Field(StoredTweet.NsfwUserField.id)
|
||||||
|
val NsfwAdmin: Field = new Field(StoredTweet.NsfwAdminField.id)
|
||||||
|
val TweetypieOnlyTakedownCountryCodes: Field =
|
||||||
|
new Field(TpTweet.TweetypieOnlyTakedownCountryCodesField.id)
|
||||||
|
val TweetypieOnlyTakedownReasons: Field =
|
||||||
|
new Field(TpTweet.TweetypieOnlyTakedownReasonsField.id)
|
||||||
|
|
||||||
|
val AllUpdatableCompiledFields: Set[Field] = Set(Geo, HasTakedown, NsfwUser, NsfwAdmin)
|
||||||
|
|
||||||
|
def additionalField(id: Short): Field = {
|
||||||
|
require(isAdditionalFieldId(id), "field id must be in the additional field range")
|
||||||
|
new Field(id)
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,150 @@
|
|||||||
|
package com.twitter.tweetypie.storage
|
||||||
|
|
||||||
|
import com.twitter.finagle.stats.StatsReceiver
|
||||||
|
import com.twitter.stitch.Stitch
|
||||||
|
import com.twitter.storage.client.manhattan.kv.DeniedManhattanException
|
||||||
|
import com.twitter.tweetypie.storage.Response.TweetResponseCode
|
||||||
|
import com.twitter.tweetypie.storage.TweetUtils._
|
||||||
|
import com.twitter.tweetypie.storage_internal.thriftscala.StoredTweet
|
||||||
|
import com.twitter.tweetypie.thriftscala.DeletedTweet
|
||||||
|
import scala.util.control.NonFatal
|
||||||
|
|
||||||
|
sealed trait DeleteState
|
||||||
|
object DeleteState {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This tweet is deleted but has not been permanently deleted from Manhattan. Tweets in this state
|
||||||
|
* may be undeleted.
|
||||||
|
*/
|
||||||
|
case object SoftDeleted extends DeleteState
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This tweet is deleted after being bounced for violating the Twitter Rules but has not been
|
||||||
|
* permanently deleted from Manhattan. Tweets in this state may NOT be undeleted.
|
||||||
|
*/
|
||||||
|
case object BounceDeleted extends DeleteState
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This tweet has been permanently deleted from Manhattan.
|
||||||
|
*/
|
||||||
|
case object HardDeleted extends DeleteState
|
||||||
|
|
||||||
|
/**
|
||||||
|
* There is no data in Manhattan to distinguish this tweet id from one that never existed.
|
||||||
|
*/
|
||||||
|
case object NotFound extends DeleteState
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This tweet exists and is not in a deleted state.
|
||||||
|
*/
|
||||||
|
case object NotDeleted extends DeleteState
|
||||||
|
}
|
||||||
|
|
||||||
|
case class DeletedTweetResponse(
|
||||||
|
tweetId: TweetId,
|
||||||
|
overallResponse: TweetResponseCode,
|
||||||
|
deleteState: DeleteState,
|
||||||
|
tweet: Option[DeletedTweet])
|
||||||
|
|
||||||
|
object GetDeletedTweetsHandler {
|
||||||
|
def apply(
|
||||||
|
read: ManhattanOperations.Read,
|
||||||
|
stats: StatsReceiver
|
||||||
|
): TweetStorageClient.GetDeletedTweets =
|
||||||
|
(unfilteredTweetIds: Seq[TweetId]) => {
|
||||||
|
val tweetIds = unfilteredTweetIds.filter(_ > 0)
|
||||||
|
|
||||||
|
Stats.addWidthStat("getDeletedTweets", "tweetIds", tweetIds.size, stats)
|
||||||
|
|
||||||
|
val stitches = tweetIds.map { tweetId =>
|
||||||
|
read(tweetId)
|
||||||
|
.map { mhRecords =>
|
||||||
|
val storedTweet = buildStoredTweet(tweetId, mhRecords)
|
||||||
|
|
||||||
|
TweetStateRecord.mostRecent(mhRecords) match {
|
||||||
|
case Some(m: TweetStateRecord.SoftDeleted) => softDeleted(m, storedTweet)
|
||||||
|
case Some(m: TweetStateRecord.BounceDeleted) => bounceDeleted(m, storedTweet)
|
||||||
|
case Some(m: TweetStateRecord.HardDeleted) => hardDeleted(m, storedTweet)
|
||||||
|
case _ if storedTweet.getFieldBlobs(expectedFields).isEmpty => notFound(tweetId)
|
||||||
|
case _ => notDeleted(tweetId, storedTweet)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
.handle {
|
||||||
|
case _: DeniedManhattanException =>
|
||||||
|
DeletedTweetResponse(
|
||||||
|
tweetId,
|
||||||
|
TweetResponseCode.OverCapacity,
|
||||||
|
DeleteState.NotFound,
|
||||||
|
None
|
||||||
|
)
|
||||||
|
|
||||||
|
case NonFatal(ex) =>
|
||||||
|
TweetUtils.log.warning(
|
||||||
|
ex,
|
||||||
|
s"Unhandled exception in GetDeletedTweetsHandler for tweetId: $tweetId"
|
||||||
|
)
|
||||||
|
DeletedTweetResponse(tweetId, TweetResponseCode.Failure, DeleteState.NotFound, None)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
Stitch.collect(stitches)
|
||||||
|
}
|
||||||
|
|
||||||
|
private def notFound(tweetId: TweetId) =
|
||||||
|
DeletedTweetResponse(
|
||||||
|
tweetId = tweetId,
|
||||||
|
overallResponse = TweetResponseCode.Success,
|
||||||
|
deleteState = DeleteState.NotFound,
|
||||||
|
tweet = None
|
||||||
|
)
|
||||||
|
|
||||||
|
private def softDeleted(record: TweetStateRecord.SoftDeleted, storedTweet: StoredTweet) =
|
||||||
|
DeletedTweetResponse(
|
||||||
|
record.tweetId,
|
||||||
|
TweetResponseCode.Success,
|
||||||
|
DeleteState.SoftDeleted,
|
||||||
|
Some(
|
||||||
|
StorageConversions
|
||||||
|
.toDeletedTweet(storedTweet)
|
||||||
|
.copy(deletedAtMsec = Some(record.createdAt))
|
||||||
|
)
|
||||||
|
)
|
||||||
|
|
||||||
|
private def bounceDeleted(record: TweetStateRecord.BounceDeleted, storedTweet: StoredTweet) =
|
||||||
|
DeletedTweetResponse(
|
||||||
|
record.tweetId,
|
||||||
|
TweetResponseCode.Success,
|
||||||
|
DeleteState.BounceDeleted,
|
||||||
|
Some(
|
||||||
|
StorageConversions
|
||||||
|
.toDeletedTweet(storedTweet)
|
||||||
|
.copy(deletedAtMsec = Some(record.createdAt))
|
||||||
|
)
|
||||||
|
)
|
||||||
|
|
||||||
|
private def hardDeleted(record: TweetStateRecord.HardDeleted, storedTweet: StoredTweet) =
|
||||||
|
DeletedTweetResponse(
|
||||||
|
record.tweetId,
|
||||||
|
TweetResponseCode.Success,
|
||||||
|
DeleteState.HardDeleted,
|
||||||
|
Some(
|
||||||
|
StorageConversions
|
||||||
|
.toDeletedTweet(storedTweet)
|
||||||
|
.copy(
|
||||||
|
hardDeletedAtMsec = Some(record.createdAt),
|
||||||
|
deletedAtMsec = Some(record.deletedAt)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* notDeleted returns a tweet to simplify tweetypie.handler.UndeleteTweetHandler
|
||||||
|
*/
|
||||||
|
private def notDeleted(tweetId: TweetId, storedTweet: StoredTweet) =
|
||||||
|
DeletedTweetResponse(
|
||||||
|
tweetId = tweetId,
|
||||||
|
overallResponse = TweetResponseCode.Success,
|
||||||
|
deleteState = DeleteState.NotDeleted,
|
||||||
|
tweet = Some(StorageConversions.toDeletedTweet(storedTweet))
|
||||||
|
)
|
||||||
|
}
|
@ -0,0 +1,126 @@
|
|||||||
|
package com.twitter.tweetypie.storage
|
||||||
|
|
||||||
|
import com.twitter.conversions.DurationOps._
|
||||||
|
import com.twitter.finagle.stats.StatsReceiver
|
||||||
|
import com.twitter.stitch.Stitch
|
||||||
|
import com.twitter.stitch.StitchSeqGroup
|
||||||
|
import com.twitter.tweetypie.storage.TweetStorageClient.GetStoredTweet
|
||||||
|
import com.twitter.tweetypie.storage.TweetStorageClient.GetStoredTweet.Error
|
||||||
|
import com.twitter.tweetypie.storage.TweetStorageClient.GetStoredTweet.Response._
|
||||||
|
import com.twitter.tweetypie.storage.TweetUtils._
|
||||||
|
import com.twitter.tweetypie.thriftscala.Tweet
|
||||||
|
import com.twitter.util.Time
|
||||||
|
import com.twitter.util.Try
|
||||||
|
import scala.collection.mutable
|
||||||
|
|
||||||
|
object GetStoredTweetHandler {
|
||||||
|
private[this] object DeletedState {
|
||||||
|
def unapply(stateRecord: Option[TweetStateRecord]): Option[TweetStateRecord] =
|
||||||
|
stateRecord match {
|
||||||
|
case state @ (Some(_: TweetStateRecord.SoftDeleted) | Some(
|
||||||
|
_: TweetStateRecord.HardDeleted) | Some(_: TweetStateRecord.BounceDeleted)) =>
|
||||||
|
state
|
||||||
|
case _ => None
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private[this] def deletedAtMs(stateRecord: Option[TweetStateRecord]): Option[Long] =
|
||||||
|
stateRecord match {
|
||||||
|
case Some(d: TweetStateRecord.SoftDeleted) => Some(d.createdAt)
|
||||||
|
case Some(d: TweetStateRecord.BounceDeleted) => Some(d.createdAt)
|
||||||
|
case Some(d: TweetStateRecord.HardDeleted) => Some(d.deletedAt)
|
||||||
|
case _ => None
|
||||||
|
}
|
||||||
|
|
||||||
|
private[this] def tweetResponseFromRecords(
|
||||||
|
tweetId: TweetId,
|
||||||
|
mhRecords: Seq[TweetManhattanRecord],
|
||||||
|
statsReceiver: StatsReceiver,
|
||||||
|
): GetStoredTweet.Response = {
|
||||||
|
val errs =
|
||||||
|
mutable.Buffer[Error]()
|
||||||
|
|
||||||
|
val hasStoredTweetFields: Boolean = mhRecords.exists {
|
||||||
|
case TweetManhattanRecord(TweetKey(_, _: TweetKey.LKey.FieldKey), _) => true
|
||||||
|
case _ => false
|
||||||
|
}
|
||||||
|
|
||||||
|
val storedTweet = if (hasStoredTweetFields) {
|
||||||
|
Try(buildStoredTweet(tweetId, mhRecords, includeScrubbed = true))
|
||||||
|
.onFailure(_ => errs.append(Error.TweetIsCorrupt))
|
||||||
|
.toOption
|
||||||
|
} else {
|
||||||
|
None
|
||||||
|
}
|
||||||
|
|
||||||
|
val scrubbedFields: Set[FieldId] = extractScrubbedFields(mhRecords)
|
||||||
|
val tweet: Option[Tweet] = storedTweet.map(StorageConversions.fromStoredTweetAllowInvalid)
|
||||||
|
val stateRecords: Seq[TweetStateRecord] = TweetStateRecord.fromTweetMhRecords(mhRecords)
|
||||||
|
val tweetState: Option[TweetStateRecord] = TweetStateRecord.mostRecent(mhRecords)
|
||||||
|
|
||||||
|
storedTweet.foreach { storedTweet =>
|
||||||
|
val storedExpectedFields = storedTweet.getFieldBlobs(expectedFields)
|
||||||
|
val missingExpectedFields = expectedFields.filterNot(storedExpectedFields.contains)
|
||||||
|
if (missingExpectedFields.nonEmpty || !isValid(storedTweet)) {
|
||||||
|
errs.append(Error.TweetFieldsMissingOrInvalid)
|
||||||
|
}
|
||||||
|
|
||||||
|
val invalidScrubbedFields = storedTweet.getFieldBlobs(scrubbedFields).keys
|
||||||
|
if (invalidScrubbedFields.nonEmpty) {
|
||||||
|
errs.append(Error.ScrubbedFieldsPresent)
|
||||||
|
}
|
||||||
|
|
||||||
|
if (deletedAtMs(tweetState).exists(_ < Time.now.inMilliseconds - 14.days.inMilliseconds)) {
|
||||||
|
errs.append(Error.TweetShouldBeHardDeleted)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
val err = Option(errs.toList).filter(_.nonEmpty)
|
||||||
|
|
||||||
|
(tweet, tweetState, err) match {
|
||||||
|
case (None, None, None) =>
|
||||||
|
statsReceiver.counter("not_found").incr()
|
||||||
|
NotFound(tweetId)
|
||||||
|
|
||||||
|
case (None, Some(tweetState: TweetStateRecord.HardDeleted), None) =>
|
||||||
|
statsReceiver.counter("hard_deleted").incr()
|
||||||
|
HardDeleted(tweetId, Some(tweetState), stateRecords, scrubbedFields)
|
||||||
|
|
||||||
|
case (None, _, Some(errs)) =>
|
||||||
|
statsReceiver.counter("failed").incr()
|
||||||
|
Failed(tweetId, tweetState, stateRecords, scrubbedFields, errs)
|
||||||
|
|
||||||
|
case (Some(tweet), _, Some(errs)) =>
|
||||||
|
statsReceiver.counter("found_invalid").incr()
|
||||||
|
FoundWithErrors(tweet, tweetState, stateRecords, scrubbedFields, errs)
|
||||||
|
|
||||||
|
case (Some(tweet), DeletedState(state), None) =>
|
||||||
|
statsReceiver.counter("deleted").incr()
|
||||||
|
FoundDeleted(tweet, Some(state), stateRecords, scrubbedFields)
|
||||||
|
|
||||||
|
case (Some(tweet), _, None) =>
|
||||||
|
statsReceiver.counter("found").incr()
|
||||||
|
Found(tweet, tweetState, stateRecords, scrubbedFields)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
def apply(read: ManhattanOperations.Read, statsReceiver: StatsReceiver): GetStoredTweet = {
|
||||||
|
|
||||||
|
object mhGroup extends StitchSeqGroup[TweetId, Seq[TweetManhattanRecord]] {
|
||||||
|
override def run(tweetIds: Seq[TweetId]): Stitch[Seq[Seq[TweetManhattanRecord]]] = {
|
||||||
|
Stats.addWidthStat("getStoredTweet", "tweetIds", tweetIds.size, statsReceiver)
|
||||||
|
Stitch.traverse(tweetIds)(read(_))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
tweetId =>
|
||||||
|
if (tweetId <= 0) {
|
||||||
|
Stitch.NotFound
|
||||||
|
} else {
|
||||||
|
Stitch
|
||||||
|
.call(tweetId, mhGroup)
|
||||||
|
.map(mhRecords =>
|
||||||
|
tweetResponseFromRecords(tweetId, mhRecords, statsReceiver.scope("getStoredTweet")))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,167 @@
|
|||||||
|
package com.twitter.tweetypie.storage
|
||||||
|
|
||||||
|
import com.twitter.conversions.DurationOps._
|
||||||
|
import com.twitter.finagle.stats.Counter
|
||||||
|
import com.twitter.finagle.stats.NullStatsReceiver
|
||||||
|
import com.twitter.finagle.stats.StatsReceiver
|
||||||
|
import com.twitter.logging.Logger
|
||||||
|
import com.twitter.snowflake.id.SnowflakeId
|
||||||
|
import com.twitter.stitch.Stitch
|
||||||
|
import com.twitter.stitch.StitchSeqGroup
|
||||||
|
import com.twitter.storage.client.manhattan.kv.DeniedManhattanException
|
||||||
|
import com.twitter.storage.client.manhattan.kv.ManhattanException
|
||||||
|
import com.twitter.tweetypie.storage.TweetStateRecord.BounceDeleted
|
||||||
|
import com.twitter.tweetypie.storage.TweetStateRecord.HardDeleted
|
||||||
|
import com.twitter.tweetypie.storage.TweetStateRecord.SoftDeleted
|
||||||
|
import com.twitter.tweetypie.storage.TweetStorageClient.GetTweet
|
||||||
|
import com.twitter.tweetypie.storage.TweetUtils._
|
||||||
|
import com.twitter.util.Duration
|
||||||
|
import com.twitter.util.Return
|
||||||
|
import com.twitter.util.Throw
|
||||||
|
import com.twitter.util.Time
|
||||||
|
|
||||||
|
object GetTweetHandler {
|
||||||
|
private[this] val logger = Logger(getClass)
|
||||||
|
|
||||||
|
//////////////////////////////////////////////////
|
||||||
|
// Logging racy reads for later validation.
|
||||||
|
|
||||||
|
val RacyTweetWindow: Duration = 10.seconds
|
||||||
|
|
||||||
|
/**
|
||||||
|
* If this read is soon after the tweet was created, then we would usually
|
||||||
|
* expect it to be served from cache. This early read indicates that this
|
||||||
|
* tweet is prone to consistency issues, so we log what's present in
|
||||||
|
* Manhattan at the time of the read for later analysis.
|
||||||
|
*/
|
||||||
|
private[this] def logRacyRead(tweetId: TweetId, records: Seq[TweetManhattanRecord]): Unit =
|
||||||
|
if (SnowflakeId.isSnowflakeId(tweetId)) {
|
||||||
|
val tweetAge = Time.now.since(SnowflakeId(tweetId).time)
|
||||||
|
if (tweetAge <= RacyTweetWindow) {
|
||||||
|
val sb = new StringBuilder
|
||||||
|
sb.append("racy_tweet_read\t")
|
||||||
|
.append(tweetId)
|
||||||
|
.append('\t')
|
||||||
|
.append(tweetAge.inMilliseconds) // Log the age for analysis purposes
|
||||||
|
records.foreach { rec =>
|
||||||
|
sb.append('\t')
|
||||||
|
.append(rec.lkey)
|
||||||
|
rec.value.timestamp.foreach { ts =>
|
||||||
|
// If there is a timestamp for this key, log it so that we can tell
|
||||||
|
// later on whether a value should have been present. We expect
|
||||||
|
// keys written in a single write to have the same timestamp, and
|
||||||
|
// generally, keys written in separate writes will have different
|
||||||
|
// timestamps. The timestamp value is optional in Manhattan, but
|
||||||
|
// we expect there to always be a value for the timestamp.
|
||||||
|
sb.append(':')
|
||||||
|
.append(ts.inMilliseconds)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
logger.info(sb.toString)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Convert a set of records from Manhattan into a GetTweet.Response.
|
||||||
|
*/
|
||||||
|
def tweetResponseFromRecords(
|
||||||
|
tweetId: TweetId,
|
||||||
|
mhRecords: Seq[TweetManhattanRecord],
|
||||||
|
statsReceiver: StatsReceiver = NullStatsReceiver
|
||||||
|
): GetTweet.Response =
|
||||||
|
if (mhRecords.isEmpty) {
|
||||||
|
GetTweet.Response.NotFound
|
||||||
|
} else {
|
||||||
|
// If no internal fields are present or no required fields present, we consider the tweet
|
||||||
|
// as not returnable (even if some additional fields are present)
|
||||||
|
def tweetFromRecords(tweetId: TweetId, mhRecords: Seq[TweetManhattanRecord]) = {
|
||||||
|
val storedTweet = buildStoredTweet(tweetId, mhRecords)
|
||||||
|
if (storedTweet.getFieldBlobs(expectedFields).nonEmpty) {
|
||||||
|
if (isValid(storedTweet)) {
|
||||||
|
statsReceiver.counter("valid").incr()
|
||||||
|
Some(StorageConversions.fromStoredTweet(storedTweet))
|
||||||
|
} else {
|
||||||
|
log.info(s"Invalid Tweet Id: $tweetId")
|
||||||
|
statsReceiver.counter("invalid").incr()
|
||||||
|
None
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
// The Tweet contained none of the fields defined in `expectedFields`
|
||||||
|
log.info(s"Expected Fields Not Present Tweet Id: $tweetId")
|
||||||
|
statsReceiver.counter("expected_fields_not_present").incr()
|
||||||
|
None
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
val stateRecord = TweetStateRecord.mostRecent(mhRecords)
|
||||||
|
stateRecord match {
|
||||||
|
// some other cases don't require an attempt to construct a Tweet
|
||||||
|
case Some(_: SoftDeleted) | Some(_: HardDeleted) => GetTweet.Response.Deleted
|
||||||
|
|
||||||
|
// all other cases require an attempt to construct a Tweet, which may not be successful
|
||||||
|
case _ =>
|
||||||
|
logRacyRead(tweetId, mhRecords)
|
||||||
|
(stateRecord, tweetFromRecords(tweetId, mhRecords)) match {
|
||||||
|
// BounceDeleted contains the Tweet data so that callers can access data on the the
|
||||||
|
// tweet (e.g. hard delete daemon requires conversationId and userId. There are no
|
||||||
|
// plans for Tweetypie server to make use of the returned tweet at this time.
|
||||||
|
case (Some(_: BounceDeleted), Some(tweet)) => GetTweet.Response.BounceDeleted(tweet)
|
||||||
|
case (Some(_: BounceDeleted), None) => GetTweet.Response.Deleted
|
||||||
|
case (_, Some(tweet)) => GetTweet.Response.Found(tweet)
|
||||||
|
case _ => GetTweet.Response.NotFound
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
def apply(read: ManhattanOperations.Read, statsReceiver: StatsReceiver): GetTweet = {
|
||||||
|
|
||||||
|
object stats {
|
||||||
|
val getTweetScope = statsReceiver.scope("getTweet")
|
||||||
|
val deniedCounter: Counter = getTweetScope.counter("mh_denied")
|
||||||
|
val mhExceptionCounter: Counter = getTweetScope.counter("mh_exception")
|
||||||
|
val nonFatalExceptionCounter: Counter = getTweetScope.counter("non_fatal_exception")
|
||||||
|
val notFoundCounter: Counter = getTweetScope.counter("not_found")
|
||||||
|
}
|
||||||
|
|
||||||
|
object mhGroup extends StitchSeqGroup[TweetId, Seq[TweetManhattanRecord]] {
|
||||||
|
override def run(tweetIds: Seq[TweetId]): Stitch[Seq[Seq[TweetManhattanRecord]]] = {
|
||||||
|
Stats.addWidthStat("getTweet", "tweetIds", tweetIds.size, statsReceiver)
|
||||||
|
Stitch.traverse(tweetIds)(read(_))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
tweetId =>
|
||||||
|
if (tweetId <= 0) {
|
||||||
|
Stitch.NotFound
|
||||||
|
} else {
|
||||||
|
Stitch
|
||||||
|
.call(tweetId, mhGroup)
|
||||||
|
.map(mhRecords => tweetResponseFromRecords(tweetId, mhRecords, stats.getTweetScope))
|
||||||
|
.liftToTry
|
||||||
|
.map {
|
||||||
|
case Throw(mhException: DeniedManhattanException) =>
|
||||||
|
stats.deniedCounter.incr()
|
||||||
|
Throw(RateLimited("", mhException))
|
||||||
|
|
||||||
|
// Encountered some other Manhattan error
|
||||||
|
case t @ Throw(_: ManhattanException) =>
|
||||||
|
stats.mhExceptionCounter.incr()
|
||||||
|
t
|
||||||
|
|
||||||
|
// Something else happened
|
||||||
|
case t @ Throw(ex) =>
|
||||||
|
stats.nonFatalExceptionCounter.incr()
|
||||||
|
TweetUtils.log
|
||||||
|
.warning(ex, s"Unhandled exception in GetTweetHandler for tweetId: $tweetId")
|
||||||
|
t
|
||||||
|
|
||||||
|
case r @ Return(GetTweet.Response.NotFound) =>
|
||||||
|
stats.notFoundCounter.incr()
|
||||||
|
r
|
||||||
|
|
||||||
|
case r @ Return(_) => r
|
||||||
|
}
|
||||||
|
.lowerFromTry
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,153 @@
|
|||||||
|
package com.twitter.tweetypie.storage
|
||||||
|
|
||||||
|
import com.twitter.finagle.stats.StatsReceiver
|
||||||
|
import com.twitter.stitch.Stitch
|
||||||
|
import com.twitter.tweetypie.storage.TweetKey.LKey.ForceAddedStateKey
|
||||||
|
import com.twitter.tweetypie.storage.TweetStorageClient.HardDeleteTweet
|
||||||
|
import com.twitter.tweetypie.storage.TweetStorageClient.HardDeleteTweet.Response._
|
||||||
|
import com.twitter.tweetypie.storage.TweetUtils._
|
||||||
|
import com.twitter.util.Return
|
||||||
|
import com.twitter.util.Throw
|
||||||
|
import com.twitter.util.Time
|
||||||
|
import com.twitter.util.Try
|
||||||
|
|
||||||
|
object HardDeleteTweetHandler {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* When a tweet is removed lkeys with these prefixes will be deleted permanently.
|
||||||
|
*/
|
||||||
|
private[storage] def isKeyToBeDeleted(key: TweetKey): Boolean =
|
||||||
|
key.lKey match {
|
||||||
|
case (TweetKey.LKey.CoreFieldsKey | TweetKey.LKey.InternalFieldsKey(_) |
|
||||||
|
TweetKey.LKey.AdditionalFieldsKey(_) | TweetKey.LKey.SoftDeletionStateKey |
|
||||||
|
TweetKey.LKey.BounceDeletionStateKey | TweetKey.LKey.UnDeletionStateKey |
|
||||||
|
TweetKey.LKey.ForceAddedStateKey) =>
|
||||||
|
true
|
||||||
|
case _ => false
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* When hard deleting, there are two actions, writing the record and
|
||||||
|
* removing the tweet data. If we are performing any action, we will
|
||||||
|
* always try to remove the tweet data. If the tweet does not yet have a
|
||||||
|
* hard deletion record, then we will need to write one. This method
|
||||||
|
* returns the HardDeleted record if it needs to be written, and None
|
||||||
|
* if it has already been written.
|
||||||
|
*
|
||||||
|
* If the tweet is not in a deleted state we signal this with a
|
||||||
|
* Throw(NotDeleted).
|
||||||
|
*/
|
||||||
|
private[storage] def getHardDeleteStateRecord(
|
||||||
|
tweetId: TweetId,
|
||||||
|
records: Seq[TweetManhattanRecord],
|
||||||
|
mhTimestamp: Time,
|
||||||
|
stats: StatsReceiver
|
||||||
|
): Try[Option[TweetStateRecord.HardDeleted]] = {
|
||||||
|
val mostRecent = TweetStateRecord.mostRecent(records)
|
||||||
|
val currentStateStr = mostRecent.map(_.name).getOrElse("no_tweet_state_record")
|
||||||
|
stats.counter(currentStateStr).incr()
|
||||||
|
|
||||||
|
mostRecent match {
|
||||||
|
case Some(
|
||||||
|
record @ (TweetStateRecord.SoftDeleted(_, _) | TweetStateRecord.BounceDeleted(_, _))) =>
|
||||||
|
Return(
|
||||||
|
Some(
|
||||||
|
TweetStateRecord.HardDeleted(
|
||||||
|
tweetId = tweetId,
|
||||||
|
// createdAt is the hard deletion timestamp when dealing with hard deletes in Manhattan
|
||||||
|
createdAt = mhTimestamp.inMillis,
|
||||||
|
// deletedAt is the soft deletion timestamp when dealing with hard deletes in Manhattan
|
||||||
|
deletedAt = record.createdAt
|
||||||
|
)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
|
||||||
|
case Some(_: TweetStateRecord.HardDeleted) =>
|
||||||
|
Return(None)
|
||||||
|
|
||||||
|
case Some(_: TweetStateRecord.ForceAdded) =>
|
||||||
|
Throw(NotDeleted(tweetId, Some(ForceAddedStateKey)))
|
||||||
|
|
||||||
|
case Some(_: TweetStateRecord.Undeleted) =>
|
||||||
|
Throw(NotDeleted(tweetId, Some(TweetKey.LKey.UnDeletionStateKey)))
|
||||||
|
|
||||||
|
case None =>
|
||||||
|
Throw(NotDeleted(tweetId, None))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This handler returns HardDeleteTweet.Response.Deleted if data associated with the tweet is deleted,
|
||||||
|
* either as a result of this request or a previous one.
|
||||||
|
*
|
||||||
|
* The most recently added record determines the tweet's state. This method will only delete data
|
||||||
|
* for tweets in the soft-delete or hard-delete state. (Calling hardDeleteTweet for tweets that have
|
||||||
|
* already been hard-deleted will remove any lkeys that may not have been deleted previously).
|
||||||
|
*/
|
||||||
|
def apply(
|
||||||
|
read: ManhattanOperations.Read,
|
||||||
|
insert: ManhattanOperations.Insert,
|
||||||
|
delete: ManhattanOperations.Delete,
|
||||||
|
scribe: Scribe,
|
||||||
|
stats: StatsReceiver
|
||||||
|
): TweetId => Stitch[HardDeleteTweet.Response] = {
|
||||||
|
val hardDeleteStats = stats.scope("hardDeleteTweet")
|
||||||
|
val hardDeleteTweetCancelled = hardDeleteStats.counter("cancelled")
|
||||||
|
val beforeStateStats = hardDeleteStats.scope("before_state")
|
||||||
|
|
||||||
|
def removeRecords(keys: Seq[TweetKey], mhTimestamp: Time): Stitch[Unit] =
|
||||||
|
Stitch
|
||||||
|
.collect(keys.map(key => delete(key, Some(mhTimestamp)).liftToTry))
|
||||||
|
.map(collectWithRateLimitCheck)
|
||||||
|
.lowerFromTry
|
||||||
|
|
||||||
|
def writeRecord(record: Option[TweetStateRecord.HardDeleted]): Stitch[Unit] =
|
||||||
|
record match {
|
||||||
|
case Some(r) =>
|
||||||
|
insert(r.toTweetMhRecord).onSuccess { _ =>
|
||||||
|
scribe.logRemoved(
|
||||||
|
r.tweetId,
|
||||||
|
Time.fromMilliseconds(r.createdAt),
|
||||||
|
isSoftDeleted = false
|
||||||
|
)
|
||||||
|
}
|
||||||
|
case None => Stitch.Unit
|
||||||
|
}
|
||||||
|
|
||||||
|
tweetId =>
|
||||||
|
read(tweetId)
|
||||||
|
.flatMap { records =>
|
||||||
|
val hardDeletionTimestamp = Time.now
|
||||||
|
|
||||||
|
val keysToBeDeleted: Seq[TweetKey] = records.map(_.key).filter(isKeyToBeDeleted)
|
||||||
|
|
||||||
|
getHardDeleteStateRecord(
|
||||||
|
tweetId,
|
||||||
|
records,
|
||||||
|
hardDeletionTimestamp,
|
||||||
|
beforeStateStats) match {
|
||||||
|
case Return(record) =>
|
||||||
|
Stitch
|
||||||
|
.join(
|
||||||
|
writeRecord(record),
|
||||||
|
removeRecords(keysToBeDeleted, hardDeletionTimestamp)
|
||||||
|
).map(_ =>
|
||||||
|
// If the tweetId is non-snowflake and has previously been hard deleted
|
||||||
|
// there will be no coreData record to fall back on to get the tweet
|
||||||
|
// creation time and createdAtMillis will be None.
|
||||||
|
Deleted(
|
||||||
|
// deletedAtMillis: when the tweet was hard deleted
|
||||||
|
deletedAtMillis = Some(hardDeletionTimestamp.inMillis),
|
||||||
|
// createdAtMillis: when the tweet itself was created
|
||||||
|
// (as opposed to when the deletion record was created)
|
||||||
|
createdAtMillis =
|
||||||
|
TweetUtils.creationTimeFromTweetIdOrMHRecords(tweetId, records)
|
||||||
|
))
|
||||||
|
case Throw(notDeleted: NotDeleted) =>
|
||||||
|
hardDeleteTweetCancelled.incr()
|
||||||
|
Stitch.value(notDeleted)
|
||||||
|
case Throw(e) => Stitch.exception(e) // this should never happen
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,228 @@
|
|||||||
|
package com.twitter.tweetypie.storage
|
||||||
|
|
||||||
|
import com.google.common.base.CaseFormat
|
||||||
|
import com.twitter.conversions.DurationOps._
|
||||||
|
import com.twitter.finagle.mtls.authentication.ServiceIdentifier
|
||||||
|
import com.twitter.scrooge.TFieldBlob
|
||||||
|
import com.twitter.scrooge.ThriftStructFieldInfo
|
||||||
|
import com.twitter.stitch.Stitch
|
||||||
|
import com.twitter.storage.client.manhattan.kv._
|
||||||
|
import com.twitter.tweetypie.additionalfields.AdditionalFields
|
||||||
|
import com.twitter.tweetypie.storage.ManhattanOperations.Read
|
||||||
|
import com.twitter.tweetypie.storage.TweetUtils._
|
||||||
|
import com.twitter.tweetypie.storage_internal.thriftscala.StoredTweet
|
||||||
|
import com.twitter.tweetypie.thriftscala.{Tweet => TweetypieTweet}
|
||||||
|
import com.twitter.util.Duration
|
||||||
|
import com.twitter.util.Future
|
||||||
|
import com.twitter.util.Return
|
||||||
|
import com.twitter.util.Throw
|
||||||
|
import diffshow.Container
|
||||||
|
import diffshow.DiffShow
|
||||||
|
import diffshow.Expr
|
||||||
|
import org.apache.commons.codec.binary.Base64
|
||||||
|
import scala.util.Try
|
||||||
|
import shapeless.Cached
|
||||||
|
import shapeless.Strict
|
||||||
|
|
||||||
|
// This class is used by the Tweetypie Console to inspect tweet field content in Manhattan
|
||||||
|
class InspectFields(svcIdentifier: ServiceIdentifier) {
|
||||||
|
val mhApplicationId = "tbird_mh"
|
||||||
|
val mhDatasetName = "tbird_mh"
|
||||||
|
val mhDestinationName = "/s/manhattan/cylon.native-thrift"
|
||||||
|
val mhTimeout: Duration = 5000.milliseconds
|
||||||
|
|
||||||
|
val localMhEndpoint: ManhattanKVEndpoint =
|
||||||
|
ManhattanKVEndpointBuilder(
|
||||||
|
ManhattanKVClient(
|
||||||
|
mhApplicationId,
|
||||||
|
mhDestinationName,
|
||||||
|
ManhattanKVClientMtlsParams(svcIdentifier)))
|
||||||
|
.defaultGuarantee(Guarantee.SoftDcReadMyWrites)
|
||||||
|
.defaultMaxTimeout(mhTimeout)
|
||||||
|
.build()
|
||||||
|
|
||||||
|
val readOperation: Read = (new ManhattanOperations(mhDatasetName, localMhEndpoint)).read
|
||||||
|
|
||||||
|
def lookup(tweetId: Long): Future[String] = {
|
||||||
|
val result = readOperation(tweetId).liftToTry.map {
|
||||||
|
case Return(mhRecords) =>
|
||||||
|
prettyPrintManhattanRecords(tweetId, TweetKey.padTweetIdStr(tweetId), mhRecords)
|
||||||
|
case Throw(e) => e.toString
|
||||||
|
}
|
||||||
|
|
||||||
|
Stitch.run(result)
|
||||||
|
}
|
||||||
|
|
||||||
|
def storedTweet(tweetId: Long): Future[StoredTweet] = {
|
||||||
|
val result = readOperation(tweetId).liftToTry.map {
|
||||||
|
case Return(mhRecords) =>
|
||||||
|
buildStoredTweet(tweetId, mhRecords)
|
||||||
|
case Throw(e) =>
|
||||||
|
throw e
|
||||||
|
}
|
||||||
|
|
||||||
|
Stitch.run(result)
|
||||||
|
}
|
||||||
|
|
||||||
|
private[this] def prettyPrintManhattanRecords(
|
||||||
|
tweetId: Long,
|
||||||
|
pkey: String,
|
||||||
|
mhRecords: Seq[TweetManhattanRecord]
|
||||||
|
): String = {
|
||||||
|
if (mhRecords.isEmpty) {
|
||||||
|
"Not Found"
|
||||||
|
} else {
|
||||||
|
val formattedRecords = getFormattedManhattanRecords(tweetId, mhRecords)
|
||||||
|
val keyFieldWidth = formattedRecords.map(_.key.length).max + 2
|
||||||
|
val fieldNameFieldWidth = formattedRecords.map(_.fieldName.length).max + 2
|
||||||
|
|
||||||
|
val formatString = s" %-${keyFieldWidth}s %-${fieldNameFieldWidth}s %s"
|
||||||
|
|
||||||
|
val recordsString =
|
||||||
|
formattedRecords
|
||||||
|
.map { record =>
|
||||||
|
val content = record.content.replaceAll("\n", "\n" + formatString.format("", "", ""))
|
||||||
|
formatString.format(record.key, record.fieldName, content)
|
||||||
|
}
|
||||||
|
.mkString("\n")
|
||||||
|
|
||||||
|
"/tbird_mh/" + pkey + "/" + "\n" + recordsString
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private[this] def getFormattedManhattanRecords(
|
||||||
|
tweetId: Long,
|
||||||
|
mhRecords: Seq[TweetManhattanRecord]
|
||||||
|
): Seq[FormattedManhattanRecord] = {
|
||||||
|
val storedTweet = buildStoredTweet(tweetId, mhRecords).copy(updatedAt = None)
|
||||||
|
val tweetypieTweet: Option[TweetypieTweet] =
|
||||||
|
Try(StorageConversions.fromStoredTweet(storedTweet)).toOption
|
||||||
|
|
||||||
|
val blobMap: Map[String, TFieldBlob] = getStoredTweetBlobs(mhRecords).map { blob =>
|
||||||
|
getFieldName(blob.field.id) -> blob
|
||||||
|
}.toMap
|
||||||
|
|
||||||
|
mhRecords
|
||||||
|
.map {
|
||||||
|
case TweetManhattanRecord(fullKey, mhValue) =>
|
||||||
|
FormattedManhattanRecord(
|
||||||
|
key = fullKey.lKey.toString,
|
||||||
|
fieldName = getFieldName(fullKey.lKey),
|
||||||
|
content = prettyPrintManhattanValue(
|
||||||
|
fullKey.lKey,
|
||||||
|
mhValue,
|
||||||
|
storedTweet,
|
||||||
|
tweetypieTweet,
|
||||||
|
tweetId,
|
||||||
|
blobMap
|
||||||
|
)
|
||||||
|
)
|
||||||
|
}
|
||||||
|
.sortBy(_.key.replace("external", "xternal")) // sort by key, with internal first
|
||||||
|
}
|
||||||
|
|
||||||
|
private[this] def getFieldNameFromThrift(
|
||||||
|
fieldId: Short,
|
||||||
|
fieldInfos: List[ThriftStructFieldInfo]
|
||||||
|
): String =
|
||||||
|
fieldInfos
|
||||||
|
.find(info => info.tfield.id == fieldId)
|
||||||
|
.map(_.tfield.name)
|
||||||
|
.getOrElse("<UNKNOWN FIELD>")
|
||||||
|
|
||||||
|
private[this] def isLkeyScrubbedField(lkey: String): Boolean =
|
||||||
|
lkey.split("/")(1) == "scrubbed_fields"
|
||||||
|
|
||||||
|
private[this] def getFieldName(lkey: TweetKey.LKey): String =
|
||||||
|
lkey match {
|
||||||
|
case fieldKey: TweetKey.LKey.FieldKey => getFieldName(fieldKey.fieldId)
|
||||||
|
case _ => ""
|
||||||
|
}
|
||||||
|
|
||||||
|
private[this] def getFieldName(fieldId: Short): String =
|
||||||
|
if (fieldId == 1) {
|
||||||
|
"core_fields"
|
||||||
|
} else if (AdditionalFields.isAdditionalFieldId(fieldId)) {
|
||||||
|
getFieldNameFromThrift(fieldId, TweetypieTweet.fieldInfos)
|
||||||
|
} else {
|
||||||
|
getFieldNameFromThrift(fieldId, StoredTweet.fieldInfos)
|
||||||
|
}
|
||||||
|
|
||||||
|
private[this] def prettyPrintManhattanValue(
|
||||||
|
lkey: TweetKey.LKey,
|
||||||
|
mhValue: TweetManhattanValue,
|
||||||
|
storedTweet: StoredTweet,
|
||||||
|
tweetypieTweet: Option[TweetypieTweet],
|
||||||
|
tweetId: Long,
|
||||||
|
tfieldBlobs: Map[String, TFieldBlob]
|
||||||
|
): String = {
|
||||||
|
val decoded = lkey match {
|
||||||
|
case _: TweetKey.LKey.MetadataKey =>
|
||||||
|
decodeMetadata(mhValue)
|
||||||
|
|
||||||
|
case fieldKey: TweetKey.LKey.FieldKey =>
|
||||||
|
tfieldBlobs
|
||||||
|
.get(getFieldName(fieldKey.fieldId))
|
||||||
|
.map(blob => decodeField(tweetId, blob, storedTweet, tweetypieTweet))
|
||||||
|
|
||||||
|
case _ =>
|
||||||
|
None
|
||||||
|
}
|
||||||
|
|
||||||
|
decoded.getOrElse { // If all else fails, encode the data as a base64 string
|
||||||
|
val contents = mhValue.contents.array
|
||||||
|
if (contents.isEmpty) {
|
||||||
|
"<NO DATA>"
|
||||||
|
} else {
|
||||||
|
Base64.encodeBase64String(contents)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private[this] def decodeMetadata(mhValue: TweetManhattanValue): Option[String] = {
|
||||||
|
val byteArray = ByteArrayCodec.fromByteBuffer(mhValue.contents)
|
||||||
|
Try(Json.decode(byteArray).toString).toOption
|
||||||
|
}
|
||||||
|
|
||||||
|
private[this] def decodeField(
|
||||||
|
tweetId: Long,
|
||||||
|
blob: TFieldBlob,
|
||||||
|
storedTweet: StoredTweet,
|
||||||
|
tweetypieTweet: Option[TweetypieTweet]
|
||||||
|
): String = {
|
||||||
|
val fieldId = blob.field.id
|
||||||
|
|
||||||
|
if (fieldId == 1) {
|
||||||
|
coreFields(storedTweet)
|
||||||
|
} else if (AdditionalFields.isAdditionalFieldId(fieldId)) {
|
||||||
|
decodeTweetWithOneField(TweetypieTweet(tweetId).setField(blob))
|
||||||
|
} else {
|
||||||
|
decodeTweetWithOneField(StoredTweet(tweetId).setField(blob))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Takes a Tweet or StoredTweet with a single field set and returns the value of that field
|
||||||
|
private[this] def decodeTweetWithOneField[T](
|
||||||
|
tweetWithOneField: T
|
||||||
|
)(
|
||||||
|
implicit ev: Cached[Strict[DiffShow[T]]]
|
||||||
|
): String = {
|
||||||
|
val config = diffshow.Config(hideFieldWithEmptyVal = true)
|
||||||
|
val tree: Expr = config.transform(DiffShow.show(tweetWithOneField))
|
||||||
|
|
||||||
|
// matches a Tweet or StoredTweet with two values, the first being the id
|
||||||
|
val value = tree.transform {
|
||||||
|
case Container(_, List(diffshow.Field("id", _), diffshow.Field(_, value))) => value
|
||||||
|
}
|
||||||
|
|
||||||
|
config.exprPrinter.apply(value, width = 80).render
|
||||||
|
}
|
||||||
|
|
||||||
|
private[this] def coreFields(storedTweet: StoredTweet): String =
|
||||||
|
diffshow.show(CoreFieldsCodec.fromTweet(storedTweet), hideFieldWithEmptyVal = true)
|
||||||
|
|
||||||
|
private[this] def toCamelCase(s: String): String =
|
||||||
|
CaseFormat.LOWER_UNDERSCORE.to(CaseFormat.LOWER_CAMEL, s)
|
||||||
|
}
|
||||||
|
|
||||||
|
case class FormattedManhattanRecord(key: String, fieldName: String, content: String)
|
@ -0,0 +1,17 @@
|
|||||||
|
package com.twitter.tweetypie.storage
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.databind.ObjectMapper
|
||||||
|
import com.fasterxml.jackson.module.scala.DefaultScalaModule
|
||||||
|
|
||||||
|
object Json {
|
||||||
|
val TimestampKey = "timestamp"
|
||||||
|
val SoftDeleteTimestampKey = "softdelete_timestamp"
|
||||||
|
|
||||||
|
private val mapper = new ObjectMapper
|
||||||
|
mapper.registerModule(DefaultScalaModule)
|
||||||
|
|
||||||
|
def encode(m: Map[String, Any]): Array[Byte] = mapper.writeValueAsBytes(m)
|
||||||
|
|
||||||
|
def decode(arr: Array[Byte]): Map[String, Any] =
|
||||||
|
mapper.readValue[Map[String, Any]](arr, classOf[Map[String, Any]])
|
||||||
|
}
|
@ -0,0 +1,103 @@
|
|||||||
|
package com.twitter.tweetypie.storage
|
||||||
|
|
||||||
|
import com.twitter.bijection.Injection
|
||||||
|
import com.twitter.io.Buf
|
||||||
|
import com.twitter.stitch.Stitch
|
||||||
|
import com.twitter.storage.client.manhattan.bijections.Bijections.BufInjection
|
||||||
|
import com.twitter.storage.client.manhattan.kv.ManhattanKVEndpoint
|
||||||
|
import com.twitter.storage.client.manhattan.kv.impl.DescriptorP1L1
|
||||||
|
import com.twitter.storage.client.manhattan.kv.impl.Component
|
||||||
|
import com.twitter.storage.client.manhattan.kv.{impl => mh}
|
||||||
|
import com.twitter.storage.client.manhattan.bijections.Bijections.StringInjection
|
||||||
|
import com.twitter.util.Time
|
||||||
|
import java.nio.ByteBuffer
|
||||||
|
import scala.util.control.NonFatal
|
||||||
|
|
||||||
|
case class TweetManhattanRecord(key: TweetKey, value: TweetManhattanValue) {
|
||||||
|
def pkey: TweetId = key.tweetId
|
||||||
|
def lkey: TweetKey.LKey = key.lKey
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Produces a representation that is human-readable, but contains
|
||||||
|
* all of the information from the record. It is not intended for
|
||||||
|
* producing machine-readable values.
|
||||||
|
*
|
||||||
|
* This conversion is relatively expensive, so beware of using it in
|
||||||
|
* hot code paths.
|
||||||
|
*/
|
||||||
|
override def toString: String = {
|
||||||
|
val valueString =
|
||||||
|
try {
|
||||||
|
key.lKey match {
|
||||||
|
case _: TweetKey.LKey.MetadataKey =>
|
||||||
|
StringCodec.fromByteBuffer(value.contents)
|
||||||
|
|
||||||
|
case _: TweetKey.LKey.FieldKey =>
|
||||||
|
val tFieldBlob = TFieldBlobCodec.fromByteBuffer(value.contents)
|
||||||
|
s"TFieldBlob(${tFieldBlob.field}, 0x${Buf.slowHexString(tFieldBlob.content)})"
|
||||||
|
|
||||||
|
case TweetKey.LKey.Unknown(_) =>
|
||||||
|
"0x" + Buf.slowHexString(Buf.ByteBuffer.Shared(value.contents))
|
||||||
|
}
|
||||||
|
} catch {
|
||||||
|
case NonFatal(e) =>
|
||||||
|
val hexValue = Buf.slowHexString(Buf.ByteBuffer.Shared(value.contents))
|
||||||
|
s"0x$hexValue (failed to decode due to $e)"
|
||||||
|
}
|
||||||
|
|
||||||
|
s"$key => ${value.copy(contents = valueString)}"
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
object ManhattanOperations {
|
||||||
|
type Read = TweetId => Stitch[Seq[TweetManhattanRecord]]
|
||||||
|
type Insert = TweetManhattanRecord => Stitch[Unit]
|
||||||
|
type Delete = (TweetKey, Option[Time]) => Stitch[Unit]
|
||||||
|
type DeleteRange = TweetId => Stitch[Unit]
|
||||||
|
|
||||||
|
object PkeyInjection extends Injection[TweetId, String] {
|
||||||
|
override def apply(tweetId: TweetId): String = TweetKey.padTweetIdStr(tweetId)
|
||||||
|
override def invert(str: String): scala.util.Try[TweetId] = scala.util.Try(str.toLong)
|
||||||
|
}
|
||||||
|
|
||||||
|
case class InvalidLkey(lkeyStr: String) extends Exception
|
||||||
|
|
||||||
|
object LkeyInjection extends Injection[TweetKey.LKey, String] {
|
||||||
|
override def apply(lkey: TweetKey.LKey): String = lkey.toString
|
||||||
|
override def invert(str: String): scala.util.Try[TweetKey.LKey] =
|
||||||
|
scala.util.Success(TweetKey.LKey.fromString(str))
|
||||||
|
}
|
||||||
|
|
||||||
|
val KeyDescriptor: DescriptorP1L1.EmptyKey[TweetId, TweetKey.LKey] =
|
||||||
|
mh.KeyDescriptor(
|
||||||
|
Component(PkeyInjection.andThen(StringInjection)),
|
||||||
|
Component(LkeyInjection.andThen(StringInjection))
|
||||||
|
)
|
||||||
|
|
||||||
|
val ValueDescriptor: mh.ValueDescriptor.EmptyValue[ByteBuffer] = mh.ValueDescriptor(BufInjection)
|
||||||
|
}
|
||||||
|
|
||||||
|
class ManhattanOperations(dataset: String, mhEndpoint: ManhattanKVEndpoint) {
|
||||||
|
import ManhattanOperations._
|
||||||
|
|
||||||
|
private[this] def pkey(tweetId: TweetId) = KeyDescriptor.withDataset(dataset).withPkey(tweetId)
|
||||||
|
|
||||||
|
def read: Read = { tweetId =>
|
||||||
|
mhEndpoint.slice(pkey(tweetId).under(), ValueDescriptor).map { mhData =>
|
||||||
|
mhData.map {
|
||||||
|
case (key, value) => TweetManhattanRecord(TweetKey(key.pkey, key.lkey), value)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
def insert: Insert =
|
||||||
|
record => {
|
||||||
|
val mhKey = pkey(record.key.tweetId).withLkey(record.key.lKey)
|
||||||
|
mhEndpoint.insert(mhKey, ValueDescriptor.withValue(record.value))
|
||||||
|
}
|
||||||
|
|
||||||
|
def delete: Delete = (key, time) => mhEndpoint.delete(pkey(key.tweetId).withLkey(key.lKey), time)
|
||||||
|
|
||||||
|
def deleteRange: DeleteRange =
|
||||||
|
tweetId => mhEndpoint.deleteRange(KeyDescriptor.withDataset(dataset).withPkey(tweetId).under())
|
||||||
|
}
|
@ -0,0 +1,451 @@
|
|||||||
|
package com.twitter.tweetypie.storage
|
||||||
|
|
||||||
|
import com.twitter.conversions.DurationOps._
|
||||||
|
import com.twitter.finagle.mtls.authentication.EmptyServiceIdentifier
|
||||||
|
import com.twitter.finagle.mtls.authentication.ServiceIdentifier
|
||||||
|
import com.twitter.finagle.ssl.OpportunisticTls
|
||||||
|
import com.twitter.finagle.stats.NullStatsReceiver
|
||||||
|
import com.twitter.finagle.stats.StatsReceiver
|
||||||
|
import com.twitter.logging.BareFormatter
|
||||||
|
import com.twitter.logging.Level
|
||||||
|
import com.twitter.logging.ScribeHandler
|
||||||
|
import com.twitter.logging._
|
||||||
|
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.ValueDescriptor
|
||||||
|
import com.twitter.tweetypie.client_id.ClientIdHelper
|
||||||
|
import com.twitter.tweetypie.storage.Scribe.ScribeHandlerFactory
|
||||||
|
import com.twitter.tweetypie.storage.TweetStorageClient.BounceDelete
|
||||||
|
import com.twitter.tweetypie.storage.TweetStorageClient.GetTweet
|
||||||
|
import com.twitter.tweetypie.storage.TweetStorageClient.HardDeleteTweet
|
||||||
|
import com.twitter.tweetypie.thriftscala.Tweet
|
||||||
|
import com.twitter.tweetypie.util.StitchUtils
|
||||||
|
import com.twitter.util.Duration
|
||||||
|
import com.twitter.util.Return
|
||||||
|
import com.twitter.util.Throw
|
||||||
|
import scala.util.Random
|
||||||
|
|
||||||
|
object ManhattanTweetStorageClient {
|
||||||
|
object Config {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The Manhattan dataset where tweets are stored is not externally
|
||||||
|
* configurable because writing tweets to a non-production dataset
|
||||||
|
* requires great care. Staging instances using a different dataset will
|
||||||
|
* write tweets to a non-production store, but will publish events, log to
|
||||||
|
* HDFS, and cache data referencing tweets in that store which are not
|
||||||
|
* accessible by the rest of the production cluster.
|
||||||
|
*
|
||||||
|
* In a completely isolated environment it should be safe to write to
|
||||||
|
* other datasets for testing purposes.
|
||||||
|
*/
|
||||||
|
val Dataset = "tbird_mh"
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Once a tweet has been deleted it can only be undeleted within this time
|
||||||
|
* window, after which [[UndeleteHandler]] will return an error on
|
||||||
|
* undelete attempts.
|
||||||
|
*/
|
||||||
|
val UndeleteWindowHours = 240
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Default label used for underlying Manhattan Thrift client metrics
|
||||||
|
*
|
||||||
|
* The finagle client metrics will be exported at clnt/:label.
|
||||||
|
*/
|
||||||
|
val ThriftClientLabel = "mh_cylon"
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the corresponding Wily path for the Cylon cluster in the "other" DC
|
||||||
|
*/
|
||||||
|
def remoteDestination(zone: String): String =
|
||||||
|
s"/srv#/prod/${remoteZone(zone)}/manhattan/cylon.native-thrift"
|
||||||
|
|
||||||
|
private def remoteZone(zone: String) = zone match {
|
||||||
|
case "pdxa" => "atla"
|
||||||
|
case "atla" | "localhost" => "pdxa"
|
||||||
|
case _ =>
|
||||||
|
throw new IllegalArgumentException(s"Cannot configure remote DC for unknown zone '$zone'")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param applicationId Manhattan application id used for quota accounting
|
||||||
|
* @param localDestination Wily path to local Manhattan cluster
|
||||||
|
* @param localTimeout Overall timeout (including retries) for all reads/writes to local cluster
|
||||||
|
* @param remoteDestination Wily path to remote Manhattan cluster, used for undelete and force add
|
||||||
|
* @param remoteTimeout Overall timeout (including retries) for all reads/writes to remote cluster
|
||||||
|
* @param undeleteWindowHours Amount of time during which a deleted tweet can be undeleted
|
||||||
|
* @param thriftClientLabel Label used to scope stats for Manhattan Thrift client
|
||||||
|
* @param maxRequestsPerBatch Configure the Stitch RequestGroup.Generator batch size
|
||||||
|
* @param serviceIdentifier The ServiceIdentifier to use when making connections to a Manhattan cluster
|
||||||
|
* @param opportunisticTlsLevel The level to use for opportunistic TLS for connections to the Manhattan cluster
|
||||||
|
*/
|
||||||
|
case class Config(
|
||||||
|
applicationId: String,
|
||||||
|
localDestination: String,
|
||||||
|
localTimeout: Duration,
|
||||||
|
remoteDestination: String,
|
||||||
|
remoteTimeout: Duration,
|
||||||
|
undeleteWindowHours: Int = Config.UndeleteWindowHours,
|
||||||
|
thriftClientLabel: String = Config.ThriftClientLabel,
|
||||||
|
maxRequestsPerBatch: Int = Int.MaxValue,
|
||||||
|
serviceIdentifier: ServiceIdentifier,
|
||||||
|
opportunisticTlsLevel: OpportunisticTls.Level)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Sanitizes the input for APIs which take in a (Tweet, Seq[Field]) as input.
|
||||||
|
*
|
||||||
|
* NOTE: This function only applies sanity checks which are common to
|
||||||
|
* all APIs which take in a (Tweet, Seq[Field]) as input. API specific
|
||||||
|
* checks are not covered here.
|
||||||
|
*
|
||||||
|
* @param apiStitch the backing API call
|
||||||
|
* @tparam T the output type of the backing API call
|
||||||
|
* @return a stitch function which does some basic input sanity checking
|
||||||
|
*/
|
||||||
|
private[storage] def sanitizeTweetFields[T](
|
||||||
|
apiStitch: (Tweet, Seq[Field]) => Stitch[T]
|
||||||
|
): (Tweet, Seq[Field]) => Stitch[T] =
|
||||||
|
(tweet, fields) => {
|
||||||
|
require(fields.forall(_.id > 0), s"Field ids ${fields} are not positive numbers")
|
||||||
|
apiStitch(tweet, fields)
|
||||||
|
}
|
||||||
|
|
||||||
|
// Returns a handler that asynchronously logs messages to Scribe using the BareFormatter which
|
||||||
|
// logs just the message without any additional metadata
|
||||||
|
def scribeHandler(categoryName: String): HandlerFactory =
|
||||||
|
ScribeHandler(
|
||||||
|
formatter = BareFormatter,
|
||||||
|
maxMessagesPerTransaction = 100,
|
||||||
|
category = categoryName,
|
||||||
|
level = Some(Level.TRACE)
|
||||||
|
)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A Config appropriate for interactive sessions and scripts.
|
||||||
|
*/
|
||||||
|
def develConfig(): Config =
|
||||||
|
Config(
|
||||||
|
applicationId = Option(System.getenv("USER")).getOrElse("<unknown>") + ".devel",
|
||||||
|
localDestination = "/s/manhattan/cylon.native-thrift",
|
||||||
|
localTimeout = 10.seconds,
|
||||||
|
remoteDestination = "/s/manhattan/cylon.native-thrift",
|
||||||
|
remoteTimeout = 10.seconds,
|
||||||
|
undeleteWindowHours = Config.UndeleteWindowHours,
|
||||||
|
thriftClientLabel = Config.ThriftClientLabel,
|
||||||
|
maxRequestsPerBatch = Int.MaxValue,
|
||||||
|
serviceIdentifier = ServiceIdentifier(System.getenv("USER"), "tweetypie", "devel", "local"),
|
||||||
|
opportunisticTlsLevel = OpportunisticTls.Required
|
||||||
|
)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Build a Manhattan tweet storage client for use in interactive
|
||||||
|
* sessions and scripts.
|
||||||
|
*/
|
||||||
|
def devel(): TweetStorageClient =
|
||||||
|
new ManhattanTweetStorageClient(
|
||||||
|
develConfig(),
|
||||||
|
NullStatsReceiver,
|
||||||
|
ClientIdHelper.default,
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
|
class ManhattanTweetStorageClient(
|
||||||
|
config: ManhattanTweetStorageClient.Config,
|
||||||
|
statsReceiver: StatsReceiver,
|
||||||
|
private val clientIdHelper: ClientIdHelper)
|
||||||
|
extends TweetStorageClient {
|
||||||
|
import ManhattanTweetStorageClient._
|
||||||
|
|
||||||
|
lazy val scribeHandlerFactory: ScribeHandlerFactory = scribeHandler _
|
||||||
|
val scribe: Scribe = new Scribe(scribeHandlerFactory, statsReceiver)
|
||||||
|
|
||||||
|
def mkClient(
|
||||||
|
dest: String,
|
||||||
|
label: String
|
||||||
|
): ManhattanKVClient = {
|
||||||
|
val mhMtlsParams =
|
||||||
|
if (config.serviceIdentifier == EmptyServiceIdentifier) NoMtlsParams
|
||||||
|
else
|
||||||
|
ManhattanKVClientMtlsParams(
|
||||||
|
serviceIdentifier = config.serviceIdentifier,
|
||||||
|
opportunisticTls = config.opportunisticTlsLevel
|
||||||
|
)
|
||||||
|
|
||||||
|
new ManhattanKVClient(
|
||||||
|
config.applicationId,
|
||||||
|
dest,
|
||||||
|
mhMtlsParams,
|
||||||
|
label,
|
||||||
|
Seq(Experiments.ApertureLoadBalancer))
|
||||||
|
}
|
||||||
|
|
||||||
|
val localClient: ManhattanKVClient = mkClient(config.localDestination, config.thriftClientLabel)
|
||||||
|
|
||||||
|
val localMhEndpoint: ManhattanKVEndpoint = ManhattanKVEndpointBuilder(localClient)
|
||||||
|
.defaultGuarantee(Guarantee.SoftDcReadMyWrites)
|
||||||
|
.defaultMaxTimeout(config.localTimeout)
|
||||||
|
.maxRequestsPerBatch(config.maxRequestsPerBatch)
|
||||||
|
.build()
|
||||||
|
|
||||||
|
val localManhattanOperations = new ManhattanOperations(Config.Dataset, localMhEndpoint)
|
||||||
|
|
||||||
|
val remoteClient: ManhattanKVClient =
|
||||||
|
mkClient(config.remoteDestination, s"${config.thriftClientLabel}_remote")
|
||||||
|
|
||||||
|
val remoteMhEndpoint: ManhattanKVEndpoint = ManhattanKVEndpointBuilder(remoteClient)
|
||||||
|
.defaultGuarantee(Guarantee.SoftDcReadMyWrites)
|
||||||
|
.defaultMaxTimeout(config.remoteTimeout)
|
||||||
|
.build()
|
||||||
|
|
||||||
|
val remoteManhattanOperations = new ManhattanOperations(Config.Dataset, remoteMhEndpoint)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Note: This translation is only useful for non-batch endpoints. Batch endpoints currently
|
||||||
|
* represent failure without propagating an exception
|
||||||
|
* (e.g. [[com.twitter.tweetypie.storage.Response.TweetResponseCode.Failure]]).
|
||||||
|
*/
|
||||||
|
private[this] def translateExceptions(
|
||||||
|
apiName: String,
|
||||||
|
statsReceiver: StatsReceiver
|
||||||
|
): PartialFunction[Throwable, Throwable] = {
|
||||||
|
case e: IllegalArgumentException => ClientError(e.getMessage, e)
|
||||||
|
case e: DeniedManhattanException => RateLimited(e.getMessage, e)
|
||||||
|
case e: VersionMismatchError =>
|
||||||
|
statsReceiver.scope(apiName).counter("mh_version_mismatches").incr()
|
||||||
|
e
|
||||||
|
case e: InternalError =>
|
||||||
|
TweetUtils.log.error(e, s"Error processing $apiName request: ${e.getMessage}")
|
||||||
|
e
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Count requests per client id producing metrics of the form
|
||||||
|
* .../clients/:root_client_id/requests
|
||||||
|
*/
|
||||||
|
def observeClientId[A, B](
|
||||||
|
apiStitch: A => Stitch[B],
|
||||||
|
statsReceiver: StatsReceiver,
|
||||||
|
clientIdHelper: ClientIdHelper,
|
||||||
|
): A => Stitch[B] = {
|
||||||
|
val clients = statsReceiver.scope("clients")
|
||||||
|
|
||||||
|
val incrementClientRequests = { args: A =>
|
||||||
|
val clientId = clientIdHelper.effectiveClientIdRoot.getOrElse(ClientIdHelper.UnknownClientId)
|
||||||
|
clients.counter(clientId, "requests").incr
|
||||||
|
}
|
||||||
|
|
||||||
|
a => {
|
||||||
|
incrementClientRequests(a)
|
||||||
|
apiStitch(a)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Increment counters based on the overall response status of the returned [[GetTweet.Response]].
|
||||||
|
*/
|
||||||
|
def observeGetTweetResponseCode[A](
|
||||||
|
apiStitch: A => Stitch[GetTweet.Response],
|
||||||
|
statsReceiver: StatsReceiver
|
||||||
|
): A => Stitch[GetTweet.Response] = {
|
||||||
|
val scope = statsReceiver.scope("response_code")
|
||||||
|
|
||||||
|
val success = scope.counter("success")
|
||||||
|
val notFound = scope.counter("not_found")
|
||||||
|
val failure = scope.counter("failure")
|
||||||
|
val overCapacity = scope.counter("over_capacity")
|
||||||
|
val deleted = scope.counter("deleted")
|
||||||
|
val bounceDeleted = scope.counter("bounce_deleted")
|
||||||
|
|
||||||
|
a =>
|
||||||
|
apiStitch(a).respond {
|
||||||
|
case Return(_: GetTweet.Response.Found) => success.incr()
|
||||||
|
case Return(GetTweet.Response.NotFound) => notFound.incr()
|
||||||
|
case Return(_: GetTweet.Response.BounceDeleted) => bounceDeleted.incr()
|
||||||
|
case Return(GetTweet.Response.Deleted) => deleted.incr()
|
||||||
|
case Throw(_: RateLimited) => overCapacity.incr()
|
||||||
|
case Throw(_) => failure.incr()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* We do 3 things here:
|
||||||
|
*
|
||||||
|
* - Bookkeeping for overall requests
|
||||||
|
* - Bookkeeping for per api requests
|
||||||
|
* - Translate exceptions
|
||||||
|
*
|
||||||
|
* @param apiName the API being called
|
||||||
|
* @param apiStitch the implementation of the API
|
||||||
|
* @tparam A template for input type of API
|
||||||
|
* @tparam B template for output type of API
|
||||||
|
* @return Function which executes the given API call
|
||||||
|
*/
|
||||||
|
private[storage] def endpoint[A, B](
|
||||||
|
apiName: String,
|
||||||
|
apiStitch: A => Stitch[B]
|
||||||
|
): A => Stitch[B] = {
|
||||||
|
val translateException = translateExceptions(apiName, statsReceiver)
|
||||||
|
val observe = StitchUtils.observe[B](statsReceiver, apiName)
|
||||||
|
|
||||||
|
a =>
|
||||||
|
StitchUtils.translateExceptions(
|
||||||
|
observe(apiStitch(a)),
|
||||||
|
translateException
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
|
private[storage] def endpoint2[A, B, C](
|
||||||
|
apiName: String,
|
||||||
|
apiStitch: (A, B) => Stitch[C],
|
||||||
|
clientIdHelper: ClientIdHelper,
|
||||||
|
): (A, B) => Stitch[C] =
|
||||||
|
Function.untupled(endpoint(apiName, apiStitch.tupled))
|
||||||
|
|
||||||
|
val getTweet: TweetStorageClient.GetTweet = {
|
||||||
|
val stats = statsReceiver.scope("getTweet")
|
||||||
|
|
||||||
|
observeClientId(
|
||||||
|
observeGetTweetResponseCode(
|
||||||
|
endpoint(
|
||||||
|
"getTweet",
|
||||||
|
GetTweetHandler(
|
||||||
|
read = localManhattanOperations.read,
|
||||||
|
statsReceiver = stats,
|
||||||
|
)
|
||||||
|
),
|
||||||
|
stats,
|
||||||
|
),
|
||||||
|
stats,
|
||||||
|
clientIdHelper,
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
|
val getStoredTweet: TweetStorageClient.GetStoredTweet = {
|
||||||
|
val stats = statsReceiver.scope("getStoredTweet")
|
||||||
|
|
||||||
|
observeClientId(
|
||||||
|
endpoint(
|
||||||
|
"getStoredTweet",
|
||||||
|
GetStoredTweetHandler(
|
||||||
|
read = localManhattanOperations.read,
|
||||||
|
statsReceiver = stats,
|
||||||
|
)
|
||||||
|
),
|
||||||
|
stats,
|
||||||
|
clientIdHelper,
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
|
val addTweet: TweetStorageClient.AddTweet =
|
||||||
|
endpoint(
|
||||||
|
"addTweet",
|
||||||
|
AddTweetHandler(
|
||||||
|
insert = localManhattanOperations.insert,
|
||||||
|
scribe = scribe,
|
||||||
|
stats = statsReceiver
|
||||||
|
)
|
||||||
|
)
|
||||||
|
|
||||||
|
val updateTweet: TweetStorageClient.UpdateTweet =
|
||||||
|
endpoint2(
|
||||||
|
"updateTweet",
|
||||||
|
ManhattanTweetStorageClient.sanitizeTweetFields(
|
||||||
|
UpdateTweetHandler(
|
||||||
|
insert = localManhattanOperations.insert,
|
||||||
|
stats = statsReceiver,
|
||||||
|
)
|
||||||
|
),
|
||||||
|
clientIdHelper,
|
||||||
|
)
|
||||||
|
|
||||||
|
val softDelete: TweetStorageClient.SoftDelete =
|
||||||
|
endpoint(
|
||||||
|
"softDelete",
|
||||||
|
SoftDeleteHandler(
|
||||||
|
insert = localManhattanOperations.insert,
|
||||||
|
scribe = scribe
|
||||||
|
)
|
||||||
|
)
|
||||||
|
|
||||||
|
val bounceDelete: BounceDelete =
|
||||||
|
endpoint(
|
||||||
|
"bounceDelete",
|
||||||
|
BounceDeleteHandler(
|
||||||
|
insert = localManhattanOperations.insert,
|
||||||
|
scribe = scribe
|
||||||
|
)
|
||||||
|
)
|
||||||
|
|
||||||
|
val undelete: TweetStorageClient.Undelete =
|
||||||
|
endpoint(
|
||||||
|
"undelete",
|
||||||
|
UndeleteHandler(
|
||||||
|
read = localManhattanOperations.read,
|
||||||
|
localInsert = localManhattanOperations.insert,
|
||||||
|
remoteInsert = remoteManhattanOperations.insert,
|
||||||
|
delete = localManhattanOperations.delete,
|
||||||
|
undeleteWindowHours = config.undeleteWindowHours,
|
||||||
|
stats = statsReceiver
|
||||||
|
)
|
||||||
|
)
|
||||||
|
|
||||||
|
val getDeletedTweets: TweetStorageClient.GetDeletedTweets =
|
||||||
|
endpoint(
|
||||||
|
"getDeletedTweets",
|
||||||
|
GetDeletedTweetsHandler(
|
||||||
|
read = localManhattanOperations.read,
|
||||||
|
stats = statsReceiver
|
||||||
|
)
|
||||||
|
)
|
||||||
|
|
||||||
|
val deleteAdditionalFields: TweetStorageClient.DeleteAdditionalFields =
|
||||||
|
endpoint2(
|
||||||
|
"deleteAdditionalFields",
|
||||||
|
DeleteAdditionalFieldsHandler(
|
||||||
|
delete = localManhattanOperations.delete,
|
||||||
|
stats = statsReceiver,
|
||||||
|
),
|
||||||
|
clientIdHelper,
|
||||||
|
)
|
||||||
|
|
||||||
|
val scrub: TweetStorageClient.Scrub =
|
||||||
|
endpoint2(
|
||||||
|
"scrub",
|
||||||
|
ScrubHandler(
|
||||||
|
insert = localManhattanOperations.insert,
|
||||||
|
delete = localManhattanOperations.delete,
|
||||||
|
scribe = scribe,
|
||||||
|
stats = statsReceiver,
|
||||||
|
),
|
||||||
|
clientIdHelper,
|
||||||
|
)
|
||||||
|
|
||||||
|
val hardDeleteTweet: HardDeleteTweet =
|
||||||
|
endpoint(
|
||||||
|
"hardDeleteTweet",
|
||||||
|
HardDeleteTweetHandler(
|
||||||
|
read = localManhattanOperations.read,
|
||||||
|
insert = localManhattanOperations.insert,
|
||||||
|
delete = localManhattanOperations.delete,
|
||||||
|
scribe = scribe,
|
||||||
|
stats = statsReceiver
|
||||||
|
)
|
||||||
|
)
|
||||||
|
|
||||||
|
val ping: TweetStorageClient.Ping =
|
||||||
|
() =>
|
||||||
|
Stitch
|
||||||
|
.run(
|
||||||
|
localMhEndpoint
|
||||||
|
.get(
|
||||||
|
ManhattanOperations.KeyDescriptor
|
||||||
|
.withDataset(Config.Dataset)
|
||||||
|
.withPkey(Random.nextLong().abs)
|
||||||
|
.withLkey(TweetKey.LKey.CoreFieldsKey), // could be any lkey
|
||||||
|
ValueDescriptor(BufInjection)
|
||||||
|
).unit
|
||||||
|
)
|
||||||
|
}
|
@ -0,0 +1,30 @@
|
|||||||
|
package com.twitter.tweetypie.storage
|
||||||
|
|
||||||
|
object Response {
|
||||||
|
case class TweetResponse(
|
||||||
|
tweetId: Long,
|
||||||
|
overallResponse: TweetResponseCode,
|
||||||
|
additionalFieldResponses: Option[Map[Short, FieldResponse]] = None)
|
||||||
|
|
||||||
|
sealed trait TweetResponseCode
|
||||||
|
|
||||||
|
object TweetResponseCode {
|
||||||
|
object Success extends TweetResponseCode
|
||||||
|
object Partial extends TweetResponseCode
|
||||||
|
object Failure extends TweetResponseCode
|
||||||
|
object OverCapacity extends TweetResponseCode
|
||||||
|
object Deleted extends TweetResponseCode
|
||||||
|
}
|
||||||
|
|
||||||
|
case class FieldResponse(code: FieldResponseCode, message: Option[String] = None)
|
||||||
|
|
||||||
|
sealed trait FieldResponseCode
|
||||||
|
|
||||||
|
object FieldResponseCode {
|
||||||
|
object Success extends FieldResponseCode
|
||||||
|
object InvalidRequest extends FieldResponseCode
|
||||||
|
object ValueNotFound extends FieldResponseCode
|
||||||
|
object Timeout extends FieldResponseCode
|
||||||
|
object Error extends FieldResponseCode
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,85 @@
|
|||||||
|
package com.twitter.tweetypie.storage
|
||||||
|
|
||||||
|
import com.twitter.servo.util.FutureEffect
|
||||||
|
import com.twitter.finagle.stats.StatsReceiver
|
||||||
|
import com.twitter.logging._
|
||||||
|
import com.twitter.scrooge.BinaryThriftStructSerializer
|
||||||
|
import com.twitter.servo.util.{Scribe => ServoScribe}
|
||||||
|
import com.twitter.tweetypie.storage_internal.thriftscala._
|
||||||
|
import com.twitter.tbird.thriftscala.Added
|
||||||
|
import com.twitter.tbird.thriftscala.Removed
|
||||||
|
import com.twitter.tbird.thriftscala.Scrubbed
|
||||||
|
import com.twitter.util.Time
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Scribe is used to log tweet writes which are used to generate /tables/statuses in HDFS.
|
||||||
|
*
|
||||||
|
* Write Scribe Category Message
|
||||||
|
* ----- --------------- -------
|
||||||
|
* add tbird_add_status [[com.twitter.tbird.thriftscala.Added]]
|
||||||
|
* remove tbird_remove_status [[com.twitter.tbird.thriftscala.Removed]]
|
||||||
|
* scrub tbird_scrub_status [[com.twitter.tbird.thriftscala.Scrubbed]]
|
||||||
|
*
|
||||||
|
* The thrift representation is encoded using binary thrift protocol format, followed by base64
|
||||||
|
* encoding and converted to string using default character set (utf8). The logger uses BareFormatter.
|
||||||
|
*
|
||||||
|
* The thrift ops are scribed only after the write API call has succeeded.
|
||||||
|
*
|
||||||
|
* The class is thread safe except initial configuration and registration routines,
|
||||||
|
* and no exception is expected unless java heap is out of memory.
|
||||||
|
*
|
||||||
|
* If exception does get thrown, add/remove/scrub operations will fail and
|
||||||
|
* client will have to retry
|
||||||
|
*/
|
||||||
|
class Scribe(factory: Scribe.ScribeHandlerFactory, statsReceiver: StatsReceiver) {
|
||||||
|
import Scribe._
|
||||||
|
|
||||||
|
private val AddedSerializer = BinaryThriftStructSerializer(Added)
|
||||||
|
private val RemovedSerializer = BinaryThriftStructSerializer(Removed)
|
||||||
|
private val ScrubbedSerializer = BinaryThriftStructSerializer(Scrubbed)
|
||||||
|
|
||||||
|
private val addCounter = statsReceiver.counter("scribe/add/count")
|
||||||
|
private val removeCounter = statsReceiver.counter("scribe/remove/count")
|
||||||
|
private val scrubCounter = statsReceiver.counter("scribe/scrub/count")
|
||||||
|
|
||||||
|
val addHandler: FutureEffect[String] = ServoScribe(factory(scribeAddedCategory)())
|
||||||
|
val removeHandler: FutureEffect[String] = ServoScribe(factory(scribeRemovedCategory)())
|
||||||
|
val scrubHandler: FutureEffect[String] = ServoScribe(factory(scribeScrubbedCategory)())
|
||||||
|
|
||||||
|
private def addedToString(tweet: StoredTweet): String =
|
||||||
|
AddedSerializer.toString(
|
||||||
|
Added(StatusConversions.toTBirdStatus(tweet), Time.now.inMilliseconds, Some(false))
|
||||||
|
)
|
||||||
|
|
||||||
|
private def removedToString(id: Long, at: Time, isSoftDeleted: Boolean): String =
|
||||||
|
RemovedSerializer.toString(Removed(id, at.inMilliseconds, Some(isSoftDeleted)))
|
||||||
|
|
||||||
|
private def scrubbedToString(id: Long, cols: Seq[Int], at: Time): String =
|
||||||
|
ScrubbedSerializer.toString(Scrubbed(id, cols, at.inMilliseconds))
|
||||||
|
|
||||||
|
def logAdded(tweet: StoredTweet): Unit = {
|
||||||
|
addHandler(addedToString(tweet))
|
||||||
|
addCounter.incr()
|
||||||
|
}
|
||||||
|
|
||||||
|
def logRemoved(id: Long, at: Time, isSoftDeleted: Boolean): Unit = {
|
||||||
|
removeHandler(removedToString(id, at, isSoftDeleted))
|
||||||
|
removeCounter.incr()
|
||||||
|
}
|
||||||
|
|
||||||
|
def logScrubbed(id: Long, cols: Seq[Int], at: Time): Unit = {
|
||||||
|
scrubHandler(scrubbedToString(id, cols, at))
|
||||||
|
scrubCounter.incr()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
object Scribe {
|
||||||
|
type ScribeHandlerFactory = (String) => HandlerFactory
|
||||||
|
|
||||||
|
/** WARNING: These categories are white-listed. If you are changing them, the new categories should be white-listed.
|
||||||
|
* You should followup with CoreWorkflows team (CW) for that.
|
||||||
|
*/
|
||||||
|
private val scribeAddedCategory = "tbird_add_status"
|
||||||
|
private val scribeRemovedCategory = "tbird_remove_status"
|
||||||
|
private val scribeScrubbedCategory = "tbird_scrub_status"
|
||||||
|
}
|
@ -0,0 +1,71 @@
|
|||||||
|
package com.twitter.tweetypie.storage
|
||||||
|
|
||||||
|
import com.twitter.finagle.stats.StatsReceiver
|
||||||
|
import com.twitter.stitch.Stitch
|
||||||
|
import com.twitter.storage.client.manhattan.kv.ManhattanValue
|
||||||
|
import com.twitter.tweetypie.storage.TweetUtils._
|
||||||
|
import com.twitter.util.Time
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Deletes data for the scrubbed field and writes a metadata record.
|
||||||
|
* Provides scrub functionality. Right now, we only allow the scrubbing of the geo field.
|
||||||
|
* It should be simple to add more fields to the allowlist if needed.
|
||||||
|
*/
|
||||||
|
object ScrubHandler {
|
||||||
|
|
||||||
|
val scrubFieldsAllowlist: Set[Field] = Set(Field.Geo)
|
||||||
|
|
||||||
|
def apply(
|
||||||
|
insert: ManhattanOperations.Insert,
|
||||||
|
delete: ManhattanOperations.Delete,
|
||||||
|
scribe: Scribe,
|
||||||
|
stats: StatsReceiver
|
||||||
|
): TweetStorageClient.Scrub =
|
||||||
|
(unfilteredTweetIds: Seq[TweetId], columns: Seq[Field]) => {
|
||||||
|
val tweetIds = unfilteredTweetIds.filter(_ > 0)
|
||||||
|
|
||||||
|
require(columns.nonEmpty, "Must specify fields to scrub")
|
||||||
|
require(
|
||||||
|
columns.toSet.size == columns.size,
|
||||||
|
s"Duplicate fields to scrub specified: $columns"
|
||||||
|
)
|
||||||
|
require(
|
||||||
|
columns.forall(scrubFieldsAllowlist.contains(_)),
|
||||||
|
s"Cannot scrub $columns; scrubbable fields are restricted to $scrubFieldsAllowlist"
|
||||||
|
)
|
||||||
|
|
||||||
|
Stats.addWidthStat("scrub", "ids", tweetIds.size, stats)
|
||||||
|
val mhTimestamp = Time.now
|
||||||
|
|
||||||
|
val stitches = tweetIds.map { tweetId =>
|
||||||
|
val deletionStitches = columns.map { field =>
|
||||||
|
val mhKeyToDelete = TweetKey.fieldKey(tweetId, field.id)
|
||||||
|
delete(mhKeyToDelete, Some(mhTimestamp)).liftToTry
|
||||||
|
}
|
||||||
|
|
||||||
|
val collectedStitch =
|
||||||
|
Stitch.collect(deletionStitches).map(collectWithRateLimitCheck).lowerFromTry
|
||||||
|
|
||||||
|
collectedStitch
|
||||||
|
.flatMap { _ =>
|
||||||
|
val scrubbedStitches = columns.map { column =>
|
||||||
|
val scrubbedKey = TweetKey.scrubbedFieldKey(tweetId, column.id)
|
||||||
|
val record =
|
||||||
|
TweetManhattanRecord(
|
||||||
|
scrubbedKey,
|
||||||
|
ManhattanValue(StringCodec.toByteBuffer(""), Some(mhTimestamp))
|
||||||
|
)
|
||||||
|
|
||||||
|
insert(record).liftToTry
|
||||||
|
}
|
||||||
|
|
||||||
|
Stitch.collect(scrubbedStitches)
|
||||||
|
}
|
||||||
|
.map(collectWithRateLimitCheck)
|
||||||
|
}
|
||||||
|
|
||||||
|
Stitch.collect(stitches).map(collectWithRateLimitCheck).lowerFromTry.onSuccess { _ =>
|
||||||
|
tweetIds.foreach { id => scribe.logScrubbed(id, columns.map(_.id.toInt), mhTimestamp) }
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,20 @@
|
|||||||
|
package com.twitter.tweetypie.storage
|
||||||
|
|
||||||
|
import com.twitter.util.Time
|
||||||
|
|
||||||
|
object SoftDeleteHandler {
|
||||||
|
def apply(
|
||||||
|
insert: ManhattanOperations.Insert,
|
||||||
|
scribe: Scribe
|
||||||
|
): TweetStorageClient.SoftDelete =
|
||||||
|
tweetId => {
|
||||||
|
val mhTimestamp = Time.now
|
||||||
|
val softDeleteRecord = TweetStateRecord
|
||||||
|
.SoftDeleted(tweetId, mhTimestamp.inMillis)
|
||||||
|
.toTweetMhRecord
|
||||||
|
|
||||||
|
insert(softDeleteRecord).onSuccess { _ =>
|
||||||
|
scribe.logRemoved(tweetId, mhTimestamp, isSoftDeleted = true)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,33 @@
|
|||||||
|
package com.twitter.tweetypie.storage
|
||||||
|
|
||||||
|
import com.twitter.finagle.stats.StatsReceiver
|
||||||
|
|
||||||
|
object Stats {
|
||||||
|
// These two methods below (addWidthStat and updatePerFieldQpsCounters) are called per RPC call for most APIs,
|
||||||
|
// so we rely on the stats receiver that is passed in to the library to do memoization.
|
||||||
|
|
||||||
|
private[storage] def addWidthStat(
|
||||||
|
rpcName: String,
|
||||||
|
paramName: String,
|
||||||
|
width: Int,
|
||||||
|
stats: StatsReceiver
|
||||||
|
): Unit =
|
||||||
|
getStat(rpcName, paramName, stats).add(width)
|
||||||
|
|
||||||
|
// Updates the counters for each Additional field. The idea here is to expose the QPS for each
|
||||||
|
// additional field
|
||||||
|
private[storage] def updatePerFieldQpsCounters(
|
||||||
|
rpcName: String,
|
||||||
|
fieldIds: Seq[FieldId],
|
||||||
|
count: Int,
|
||||||
|
stats: StatsReceiver
|
||||||
|
): Unit = {
|
||||||
|
fieldIds.foreach { fieldId => getCounter(rpcName, fieldId, stats).incr(count) }
|
||||||
|
}
|
||||||
|
|
||||||
|
private def getCounter(rpcName: String, fieldId: FieldId, stats: StatsReceiver) =
|
||||||
|
stats.scope(rpcName, "fields", fieldId.toString).counter("count")
|
||||||
|
|
||||||
|
private def getStat(rpcName: String, paramName: String, stats: StatsReceiver) =
|
||||||
|
stats.scope(rpcName, paramName).stat("width")
|
||||||
|
}
|
@ -0,0 +1,129 @@
|
|||||||
|
package com.twitter.tweetypie.storage
|
||||||
|
|
||||||
|
import com.twitter.tweetypie.storage_internal.thriftscala._
|
||||||
|
import com.twitter.tbird.{thriftscala => tbird}
|
||||||
|
|
||||||
|
object StatusConversions {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This is used only in Scribe.scala, when scribing to tbird_add_status
|
||||||
|
* Once we remove that, we can also remove this.
|
||||||
|
*/
|
||||||
|
def toTBirdStatus(tweet: StoredTweet): tbird.Status =
|
||||||
|
tbird.Status(
|
||||||
|
id = tweet.id,
|
||||||
|
userId = tweet.userId.get,
|
||||||
|
text = tweet.text.get,
|
||||||
|
createdVia = tweet.createdVia.get,
|
||||||
|
createdAtSec = tweet.createdAtSec.get,
|
||||||
|
reply = tweet.reply.map(toTBirdReply),
|
||||||
|
share = tweet.share.map(toTBirdShare),
|
||||||
|
contributorId = tweet.contributorId,
|
||||||
|
geo = tweet.geo.map(toTBirdGeo),
|
||||||
|
hasTakedown = tweet.hasTakedown.getOrElse(false),
|
||||||
|
nsfwUser = tweet.nsfwUser.getOrElse(false),
|
||||||
|
nsfwAdmin = tweet.nsfwAdmin.getOrElse(false),
|
||||||
|
media = tweet.media.map(_.map(toTBirdMedia)).getOrElse(Seq()),
|
||||||
|
narrowcast = tweet.narrowcast.map(toTBirdNarrowcast),
|
||||||
|
nullcast = tweet.nullcast.getOrElse(false),
|
||||||
|
trackingId = tweet.trackingId
|
||||||
|
)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This is only used in a test, to verify that the above method `toTBirdStatus`
|
||||||
|
* works, so we can't remove it as long as the above method exists.
|
||||||
|
*/
|
||||||
|
def fromTBirdStatus(status: tbird.Status): StoredTweet = {
|
||||||
|
StoredTweet(
|
||||||
|
id = status.id,
|
||||||
|
userId = Some(status.userId),
|
||||||
|
text = Some(status.text),
|
||||||
|
createdVia = Some(status.createdVia),
|
||||||
|
createdAtSec = Some(status.createdAtSec),
|
||||||
|
reply = status.reply.map(fromTBirdReply),
|
||||||
|
share = status.share.map(fromTBirdShare),
|
||||||
|
contributorId = status.contributorId,
|
||||||
|
geo = status.geo.map(fromTBirdGeo),
|
||||||
|
hasTakedown = Some(status.hasTakedown),
|
||||||
|
nsfwUser = Some(status.nsfwUser),
|
||||||
|
nsfwAdmin = Some(status.nsfwAdmin),
|
||||||
|
media = Some(status.media.map(fromTBirdMedia)),
|
||||||
|
narrowcast = status.narrowcast.map(fromTBirdNarrowcast),
|
||||||
|
nullcast = Some(status.nullcast),
|
||||||
|
trackingId = status.trackingId
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
|
private def fromTBirdReply(reply: tbird.Reply): StoredReply =
|
||||||
|
StoredReply(
|
||||||
|
inReplyToStatusId = reply.inReplyToStatusId,
|
||||||
|
inReplyToUserId = reply.inReplyToUserId
|
||||||
|
)
|
||||||
|
|
||||||
|
private def fromTBirdShare(share: tbird.Share): StoredShare =
|
||||||
|
StoredShare(
|
||||||
|
sourceStatusId = share.sourceStatusId,
|
||||||
|
sourceUserId = share.sourceUserId,
|
||||||
|
parentStatusId = share.parentStatusId
|
||||||
|
)
|
||||||
|
|
||||||
|
private def fromTBirdGeo(geo: tbird.Geo): StoredGeo =
|
||||||
|
StoredGeo(
|
||||||
|
latitude = geo.latitude,
|
||||||
|
longitude = geo.longitude,
|
||||||
|
geoPrecision = geo.geoPrecision,
|
||||||
|
entityId = geo.entityId
|
||||||
|
)
|
||||||
|
|
||||||
|
private def fromTBirdMedia(media: tbird.MediaEntity): StoredMediaEntity =
|
||||||
|
StoredMediaEntity(
|
||||||
|
id = media.id,
|
||||||
|
mediaType = media.mediaType,
|
||||||
|
width = media.width,
|
||||||
|
height = media.height
|
||||||
|
)
|
||||||
|
|
||||||
|
private def fromTBirdNarrowcast(narrowcast: tbird.Narrowcast): StoredNarrowcast =
|
||||||
|
StoredNarrowcast(
|
||||||
|
language = Some(narrowcast.language),
|
||||||
|
location = Some(narrowcast.location),
|
||||||
|
ids = Some(narrowcast.ids)
|
||||||
|
)
|
||||||
|
|
||||||
|
private def toTBirdReply(reply: StoredReply): tbird.Reply =
|
||||||
|
tbird.Reply(
|
||||||
|
inReplyToStatusId = reply.inReplyToStatusId,
|
||||||
|
inReplyToUserId = reply.inReplyToUserId
|
||||||
|
)
|
||||||
|
|
||||||
|
private def toTBirdShare(share: StoredShare): tbird.Share =
|
||||||
|
tbird.Share(
|
||||||
|
sourceStatusId = share.sourceStatusId,
|
||||||
|
sourceUserId = share.sourceUserId,
|
||||||
|
parentStatusId = share.parentStatusId
|
||||||
|
)
|
||||||
|
|
||||||
|
private def toTBirdGeo(geo: StoredGeo): tbird.Geo =
|
||||||
|
tbird.Geo(
|
||||||
|
latitude = geo.latitude,
|
||||||
|
longitude = geo.longitude,
|
||||||
|
geoPrecision = geo.geoPrecision,
|
||||||
|
entityId = geo.entityId,
|
||||||
|
name = geo.name
|
||||||
|
)
|
||||||
|
|
||||||
|
private def toTBirdMedia(media: StoredMediaEntity): tbird.MediaEntity =
|
||||||
|
tbird.MediaEntity(
|
||||||
|
id = media.id,
|
||||||
|
mediaType = media.mediaType,
|
||||||
|
width = media.width,
|
||||||
|
height = media.height
|
||||||
|
)
|
||||||
|
|
||||||
|
private def toTBirdNarrowcast(narrowcast: StoredNarrowcast): tbird.Narrowcast =
|
||||||
|
tbird.Narrowcast(
|
||||||
|
language = narrowcast.language.getOrElse(Nil),
|
||||||
|
location = narrowcast.location.getOrElse(Nil),
|
||||||
|
ids = narrowcast.ids.getOrElse(Nil)
|
||||||
|
)
|
||||||
|
}
|
@ -0,0 +1,346 @@
|
|||||||
|
package com.twitter.tweetypie.storage
|
||||||
|
|
||||||
|
import com.twitter.mediaservices.commons.tweetmedia.thriftscala._
|
||||||
|
import com.twitter.scrooge.TFieldBlob
|
||||||
|
import com.twitter.tweetypie.additionalfields.AdditionalFields
|
||||||
|
import com.twitter.tweetypie.storage_internal.thriftscala._
|
||||||
|
import com.twitter.tweetypie.thriftscala._
|
||||||
|
import com.twitter.tweetypie.util.TweetLenses
|
||||||
|
|
||||||
|
object StorageConversions {
|
||||||
|
private val tbTweetCompiledAdditionalFieldIds =
|
||||||
|
StoredTweet.metaData.fields.map(_.id).filter(AdditionalFields.isAdditionalFieldId)
|
||||||
|
|
||||||
|
def toStoredReply(reply: Reply, conversationId: Option[TweetId]): StoredReply =
|
||||||
|
StoredReply(
|
||||||
|
inReplyToStatusId = reply.inReplyToStatusId.getOrElse(0),
|
||||||
|
inReplyToUserId = reply.inReplyToUserId,
|
||||||
|
conversationId = conversationId
|
||||||
|
)
|
||||||
|
|
||||||
|
def toStoredShare(share: Share): StoredShare =
|
||||||
|
StoredShare(
|
||||||
|
share.sourceStatusId,
|
||||||
|
share.sourceUserId,
|
||||||
|
share.parentStatusId
|
||||||
|
)
|
||||||
|
|
||||||
|
def toStoredQuotedTweet(qt: QuotedTweet, text: String): Option[StoredQuotedTweet] =
|
||||||
|
qt.permalink
|
||||||
|
.filterNot { p =>
|
||||||
|
text.contains(p.shortUrl)
|
||||||
|
} // omit StoredQuotedTweet when url already in text
|
||||||
|
.map { p =>
|
||||||
|
StoredQuotedTweet(
|
||||||
|
qt.tweetId,
|
||||||
|
qt.userId,
|
||||||
|
p.shortUrl
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
|
def toStoredGeo(tweet: Tweet): Option[StoredGeo] =
|
||||||
|
TweetLenses.geoCoordinates.get(tweet) match {
|
||||||
|
case None =>
|
||||||
|
TweetLenses.placeId.get(tweet) match {
|
||||||
|
case None => None
|
||||||
|
case Some(placeId) =>
|
||||||
|
Some(
|
||||||
|
StoredGeo(
|
||||||
|
latitude = 0.0,
|
||||||
|
longitude = 0.0,
|
||||||
|
geoPrecision = 0,
|
||||||
|
entityId = 0,
|
||||||
|
name = Some(placeId)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
}
|
||||||
|
case Some(coords) =>
|
||||||
|
Some(
|
||||||
|
StoredGeo(
|
||||||
|
latitude = coords.latitude,
|
||||||
|
longitude = coords.longitude,
|
||||||
|
geoPrecision = coords.geoPrecision,
|
||||||
|
entityId = if (coords.display) 2 else 0,
|
||||||
|
name = TweetLenses.placeId.get(tweet)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
|
def toStoredMedia(mediaList: Seq[MediaEntity]): Seq[StoredMediaEntity] =
|
||||||
|
mediaList.filter(_.sourceStatusId.isEmpty).flatMap(toStoredMediaEntity)
|
||||||
|
|
||||||
|
def toStoredMediaEntity(media: MediaEntity): Option[StoredMediaEntity] =
|
||||||
|
media.sizes.find(_.sizeType == MediaSizeType.Orig).map { origSize =>
|
||||||
|
StoredMediaEntity(
|
||||||
|
id = media.mediaId,
|
||||||
|
mediaType = origSize.deprecatedContentType.value.toByte,
|
||||||
|
width = origSize.width.toShort,
|
||||||
|
height = origSize.height.toShort
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
|
// The language and ids fields are for compatibility with existing tweets stored in manhattan.
|
||||||
|
def toStoredNarrowcast(narrowcast: Narrowcast): StoredNarrowcast =
|
||||||
|
StoredNarrowcast(
|
||||||
|
language = Some(Seq.empty),
|
||||||
|
location = Some(narrowcast.location),
|
||||||
|
ids = Some(Seq.empty)
|
||||||
|
)
|
||||||
|
|
||||||
|
def toStoredAdditionalFields(from: Seq[TFieldBlob], to: StoredTweet): StoredTweet =
|
||||||
|
from.foldLeft(to) { case (t, f) => t.setField(f) }
|
||||||
|
|
||||||
|
def toStoredAdditionalFields(from: Tweet, to: StoredTweet): StoredTweet =
|
||||||
|
toStoredAdditionalFields(AdditionalFields.additionalFields(from), to)
|
||||||
|
|
||||||
|
def toStoredTweet(tweet: Tweet): StoredTweet = {
|
||||||
|
val storedTweet =
|
||||||
|
StoredTweet(
|
||||||
|
id = tweet.id,
|
||||||
|
userId = Some(TweetLenses.userId(tweet)),
|
||||||
|
text = Some(TweetLenses.text(tweet)),
|
||||||
|
createdVia = Some(TweetLenses.createdVia(tweet)),
|
||||||
|
createdAtSec = Some(TweetLenses.createdAt(tweet)),
|
||||||
|
reply =
|
||||||
|
TweetLenses.reply(tweet).map { r => toStoredReply(r, TweetLenses.conversationId(tweet)) },
|
||||||
|
share = TweetLenses.share(tweet).map(toStoredShare),
|
||||||
|
contributorId = tweet.contributor.map(_.userId),
|
||||||
|
geo = toStoredGeo(tweet),
|
||||||
|
hasTakedown = Some(TweetLenses.hasTakedown(tweet)),
|
||||||
|
nsfwUser = Some(TweetLenses.nsfwUser(tweet)),
|
||||||
|
nsfwAdmin = Some(TweetLenses.nsfwAdmin(tweet)),
|
||||||
|
media = tweet.media.map(toStoredMedia),
|
||||||
|
narrowcast = TweetLenses.narrowcast(tweet).map(toStoredNarrowcast),
|
||||||
|
nullcast = Some(TweetLenses.nullcast(tweet)),
|
||||||
|
trackingId = TweetLenses.trackingId(tweet),
|
||||||
|
quotedTweet = TweetLenses.quotedTweet(tweet).flatMap { qt =>
|
||||||
|
toStoredQuotedTweet(qt, TweetLenses.text(tweet))
|
||||||
|
}
|
||||||
|
)
|
||||||
|
toStoredAdditionalFields(tweet, storedTweet)
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Does not need core data to be set. Constructs on disk tweet by avoiding the TweetLenses object
|
||||||
|
* and only extracting the specified fields.
|
||||||
|
*
|
||||||
|
* NOTE: Assumes that specified fields are set in the tweet.
|
||||||
|
*
|
||||||
|
* @param tpTweet Tweetypie Tweet to be converted
|
||||||
|
* @param fields the fields to be populated in the on disk Tweet
|
||||||
|
*
|
||||||
|
* @return an on disk Tweet which has only the specified fields set
|
||||||
|
*/
|
||||||
|
def toStoredTweetForFields(tpTweet: Tweet, fields: Set[Field]): StoredTweet = {
|
||||||
|
|
||||||
|
// Make sure all the passed in fields are known or additional fields
|
||||||
|
require(
|
||||||
|
(fields -- Field.AllUpdatableCompiledFields)
|
||||||
|
.forall(field => AdditionalFields.isAdditionalFieldId(field.id))
|
||||||
|
)
|
||||||
|
|
||||||
|
val storedTweet =
|
||||||
|
StoredTweet(
|
||||||
|
id = tpTweet.id,
|
||||||
|
geo = if (fields.contains(Field.Geo)) {
|
||||||
|
tpTweet.coreData.get.coordinates match {
|
||||||
|
case None =>
|
||||||
|
tpTweet.coreData.get.placeId match {
|
||||||
|
case None => None
|
||||||
|
case Some(placeId) =>
|
||||||
|
Some(
|
||||||
|
StoredGeo(
|
||||||
|
latitude = 0.0,
|
||||||
|
longitude = 0.0,
|
||||||
|
geoPrecision = 0,
|
||||||
|
entityId = 0,
|
||||||
|
name = Some(placeId)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
}
|
||||||
|
case Some(coords) =>
|
||||||
|
Some(
|
||||||
|
StoredGeo(
|
||||||
|
latitude = coords.latitude,
|
||||||
|
longitude = coords.longitude,
|
||||||
|
geoPrecision = coords.geoPrecision,
|
||||||
|
entityId = if (coords.display) 2 else 0,
|
||||||
|
name = tpTweet.coreData.get.placeId
|
||||||
|
)
|
||||||
|
)
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
None
|
||||||
|
},
|
||||||
|
hasTakedown =
|
||||||
|
if (fields.contains(Field.HasTakedown))
|
||||||
|
Some(tpTweet.coreData.get.hasTakedown)
|
||||||
|
else
|
||||||
|
None,
|
||||||
|
nsfwUser =
|
||||||
|
if (fields.contains(Field.NsfwUser))
|
||||||
|
Some(tpTweet.coreData.get.nsfwUser)
|
||||||
|
else
|
||||||
|
None,
|
||||||
|
nsfwAdmin =
|
||||||
|
if (fields.contains(Field.NsfwAdmin))
|
||||||
|
Some(tpTweet.coreData.get.nsfwAdmin)
|
||||||
|
else
|
||||||
|
None
|
||||||
|
)
|
||||||
|
|
||||||
|
if (fields.map(_.id).exists(AdditionalFields.isAdditionalFieldId))
|
||||||
|
toStoredAdditionalFields(tpTweet, storedTweet)
|
||||||
|
else
|
||||||
|
storedTweet
|
||||||
|
}
|
||||||
|
|
||||||
|
def fromStoredReply(reply: StoredReply): Reply =
|
||||||
|
Reply(
|
||||||
|
Some(reply.inReplyToStatusId).filter(_ > 0),
|
||||||
|
reply.inReplyToUserId
|
||||||
|
)
|
||||||
|
|
||||||
|
def fromStoredShare(share: StoredShare): Share =
|
||||||
|
Share(
|
||||||
|
share.sourceStatusId,
|
||||||
|
share.sourceUserId,
|
||||||
|
share.parentStatusId
|
||||||
|
)
|
||||||
|
|
||||||
|
def fromStoredQuotedTweet(qt: StoredQuotedTweet): QuotedTweet =
|
||||||
|
QuotedTweet(
|
||||||
|
qt.tweetId,
|
||||||
|
qt.userId,
|
||||||
|
Some(
|
||||||
|
ShortenedUrl(
|
||||||
|
shortUrl = qt.shortUrl,
|
||||||
|
longUrl = "", // will be hydrated later via tweetypie's QuotedTweetRefUrlsHydrator
|
||||||
|
displayText = "" //will be hydrated later via tweetypie's QuotedTweetRefUrlsHydrator
|
||||||
|
)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
|
||||||
|
def fromStoredGeo(geo: StoredGeo): GeoCoordinates =
|
||||||
|
GeoCoordinates(
|
||||||
|
latitude = geo.latitude,
|
||||||
|
longitude = geo.longitude,
|
||||||
|
geoPrecision = geo.geoPrecision,
|
||||||
|
display = geo.entityId == 2
|
||||||
|
)
|
||||||
|
|
||||||
|
def fromStoredMediaEntity(media: StoredMediaEntity): MediaEntity =
|
||||||
|
MediaEntity(
|
||||||
|
fromIndex = -1, // will get filled in later
|
||||||
|
toIndex = -1, // will get filled in later
|
||||||
|
url = null, // will get filled in later
|
||||||
|
mediaPath = "", // field is obsolete
|
||||||
|
mediaUrl = null, // will get filled in later
|
||||||
|
mediaUrlHttps = null, // will get filled in later
|
||||||
|
displayUrl = null, // will get filled in later
|
||||||
|
expandedUrl = null, // will get filled in later
|
||||||
|
mediaId = media.id,
|
||||||
|
nsfw = false,
|
||||||
|
sizes = Set(
|
||||||
|
MediaSize(
|
||||||
|
sizeType = MediaSizeType.Orig,
|
||||||
|
resizeMethod = MediaResizeMethod.Fit,
|
||||||
|
deprecatedContentType = MediaContentType(media.mediaType),
|
||||||
|
width = media.width,
|
||||||
|
height = media.height
|
||||||
|
)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
|
||||||
|
def fromStoredNarrowcast(narrowcast: StoredNarrowcast): Narrowcast =
|
||||||
|
Narrowcast(
|
||||||
|
location = narrowcast.location.getOrElse(Seq())
|
||||||
|
)
|
||||||
|
|
||||||
|
def fromStoredTweet(storedTweet: StoredTweet): Tweet = {
|
||||||
|
val coreData =
|
||||||
|
TweetCoreData(
|
||||||
|
userId = storedTweet.userId.get,
|
||||||
|
text = storedTweet.text.get,
|
||||||
|
createdVia = storedTweet.createdVia.get,
|
||||||
|
createdAtSecs = storedTweet.createdAtSec.get,
|
||||||
|
reply = storedTweet.reply.map(fromStoredReply),
|
||||||
|
share = storedTweet.share.map(fromStoredShare),
|
||||||
|
hasTakedown = storedTweet.hasTakedown.getOrElse(false),
|
||||||
|
nsfwUser = storedTweet.nsfwUser.getOrElse(false),
|
||||||
|
nsfwAdmin = storedTweet.nsfwAdmin.getOrElse(false),
|
||||||
|
narrowcast = storedTweet.narrowcast.map(fromStoredNarrowcast),
|
||||||
|
nullcast = storedTweet.nullcast.getOrElse(false),
|
||||||
|
trackingId = storedTweet.trackingId,
|
||||||
|
conversationId = storedTweet.reply.flatMap(_.conversationId),
|
||||||
|
placeId = storedTweet.geo.flatMap(_.name),
|
||||||
|
coordinates = storedTweet.geo.map(fromStoredGeo),
|
||||||
|
hasMedia = if (storedTweet.media.exists(_.nonEmpty)) Some(true) else None
|
||||||
|
)
|
||||||
|
|
||||||
|
// retweets should never have their media, but some tweets incorrectly do.
|
||||||
|
val storedMedia = if (coreData.share.isDefined) Nil else storedTweet.media.toSeq
|
||||||
|
|
||||||
|
val tpTweet =
|
||||||
|
Tweet(
|
||||||
|
id = storedTweet.id,
|
||||||
|
coreData = Some(coreData),
|
||||||
|
contributor = storedTweet.contributorId.map(Contributor(_)),
|
||||||
|
media = Some(storedMedia.flatten.map(fromStoredMediaEntity)),
|
||||||
|
mentions = Some(Seq.empty),
|
||||||
|
urls = Some(Seq.empty),
|
||||||
|
cashtags = Some(Seq.empty),
|
||||||
|
hashtags = Some(Seq.empty),
|
||||||
|
quotedTweet = storedTweet.quotedTweet.map(fromStoredQuotedTweet)
|
||||||
|
)
|
||||||
|
fromStoredAdditionalFields(storedTweet, tpTweet)
|
||||||
|
}
|
||||||
|
|
||||||
|
def fromStoredTweetAllowInvalid(storedTweet: StoredTweet): Tweet = {
|
||||||
|
fromStoredTweet(
|
||||||
|
storedTweet.copy(
|
||||||
|
userId = storedTweet.userId.orElse(Some(-1L)),
|
||||||
|
text = storedTweet.text.orElse(Some("")),
|
||||||
|
createdVia = storedTweet.createdVia.orElse(Some("")),
|
||||||
|
createdAtSec = storedTweet.createdAtSec.orElse(Some(-1L))
|
||||||
|
))
|
||||||
|
}
|
||||||
|
|
||||||
|
def fromStoredAdditionalFields(from: StoredTweet, to: Tweet): Tweet = {
|
||||||
|
val passThroughAdditionalFields =
|
||||||
|
from._passthroughFields.filterKeys(AdditionalFields.isAdditionalFieldId)
|
||||||
|
val allAdditionalFields =
|
||||||
|
from.getFieldBlobs(tbTweetCompiledAdditionalFieldIds) ++ passThroughAdditionalFields
|
||||||
|
allAdditionalFields.values.foldLeft(to) { case (t, f) => t.setField(f) }
|
||||||
|
}
|
||||||
|
|
||||||
|
def toDeletedTweet(storedTweet: StoredTweet): DeletedTweet = {
|
||||||
|
val noteTweetBlob = storedTweet.getFieldBlob(Tweet.NoteTweetField.id)
|
||||||
|
val noteTweetOption = noteTweetBlob.map(blob => NoteTweet.decode(blob.read))
|
||||||
|
DeletedTweet(
|
||||||
|
id = storedTweet.id,
|
||||||
|
userId = storedTweet.userId,
|
||||||
|
text = storedTweet.text,
|
||||||
|
createdAtSecs = storedTweet.createdAtSec,
|
||||||
|
share = storedTweet.share.map(toDeletedShare),
|
||||||
|
media = storedTweet.media.map(_.map(toDeletedMediaEntity)),
|
||||||
|
noteTweetId = noteTweetOption.map(_.id),
|
||||||
|
isExpandable = noteTweetOption.flatMap(_.isExpandable)
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
|
def toDeletedShare(storedShare: StoredShare): DeletedTweetShare =
|
||||||
|
DeletedTweetShare(
|
||||||
|
sourceStatusId = storedShare.sourceStatusId,
|
||||||
|
sourceUserId = storedShare.sourceUserId,
|
||||||
|
parentStatusId = storedShare.parentStatusId
|
||||||
|
)
|
||||||
|
|
||||||
|
def toDeletedMediaEntity(storedMediaEntity: StoredMediaEntity): DeletedTweetMediaEntity =
|
||||||
|
DeletedTweetMediaEntity(
|
||||||
|
id = storedMediaEntity.id,
|
||||||
|
mediaType = storedMediaEntity.mediaType,
|
||||||
|
width = storedMediaEntity.width,
|
||||||
|
height = storedMediaEntity.height
|
||||||
|
)
|
||||||
|
}
|
@ -0,0 +1,92 @@
|
|||||||
|
package com.twitter.tweetypie.storage
|
||||||
|
|
||||||
|
import com.twitter.util.Return
|
||||||
|
import com.twitter.util.Throw
|
||||||
|
import com.twitter.util.Time
|
||||||
|
import com.twitter.util.Try
|
||||||
|
import java.util.Arrays
|
||||||
|
import scala.util.control.NoStackTrace
|
||||||
|
import scala.util.control.NonFatal
|
||||||
|
|
||||||
|
sealed abstract class TimestampType(val keyName: String)
|
||||||
|
object TimestampType {
|
||||||
|
object Default extends TimestampType("timestamp")
|
||||||
|
object SoftDelete extends TimestampType("softdelete_timestamp")
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* TimestampDecoder gets the timestamps associated with state records. The Manhattan timestamp is
|
||||||
|
* used for legacy records (with value "1"), otherwise the timestamp is extracted from the
|
||||||
|
* JSON value.
|
||||||
|
*
|
||||||
|
* See "Metadata" in README.md for further information about state records.
|
||||||
|
*/
|
||||||
|
object TimestampDecoder {
|
||||||
|
case class UnparsableJson(msg: String, t: Throwable) extends Exception(msg, t) with NoStackTrace
|
||||||
|
case class MissingJsonTimestamp(msg: String) extends Exception(msg) with NoStackTrace
|
||||||
|
case class UnexpectedJsonValue(msg: String) extends Exception(msg) with NoStackTrace
|
||||||
|
case class MissingManhattanTimestamp(msg: String) extends Exception(msg) with NoStackTrace
|
||||||
|
|
||||||
|
private[storage] val LegacyValue: Array[Byte] = Array('1')
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The first backfill of tweet data to Manhattan supplied timestamps in milliseconds where
|
||||||
|
* nanoseconds were expected. The result is that some values have an incorrect Manhattan
|
||||||
|
* timestamp. For these bad timestamps, time.inNanoseconds is actually milliseconds.
|
||||||
|
*
|
||||||
|
* For example, the deletion record for tweet 22225781 has Manhattan timestamp 1970-01-01 00:23:24 +0000.
|
||||||
|
* Contrast with the deletion record for tweet 435404491999813632 with Manhattan timestamp 2014-11-09 14:24:04 +0000
|
||||||
|
*
|
||||||
|
* This threshold value comes from the last time in milliseconds that was interpreted
|
||||||
|
* as nanoseconds, e.g. Time.fromNanoseconds(1438387200000L) == 1970-01-01 00:23:58 +0000
|
||||||
|
*/
|
||||||
|
private[storage] val BadTimestampThreshold = Time.at("1970-01-01 00:23:58 +0000")
|
||||||
|
|
||||||
|
def decode(record: TweetManhattanRecord, tsType: TimestampType): Try[Long] =
|
||||||
|
decode(record.value, tsType)
|
||||||
|
|
||||||
|
def decode(mhValue: TweetManhattanValue, tsType: TimestampType): Try[Long] = {
|
||||||
|
val value = ByteArrayCodec.fromByteBuffer(mhValue.contents)
|
||||||
|
if (isLegacyRecord(value)) {
|
||||||
|
nativeManhattanTimestamp(mhValue)
|
||||||
|
} else {
|
||||||
|
jsonTimestamp(value, tsType)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private def isLegacyRecord(value: Array[Byte]) = Arrays.equals(value, LegacyValue)
|
||||||
|
|
||||||
|
private def nativeManhattanTimestamp(mhValue: TweetManhattanValue): Try[Long] =
|
||||||
|
mhValue.timestamp match {
|
||||||
|
case Some(ts) => Return(correctedTimestamp(ts))
|
||||||
|
case None =>
|
||||||
|
Throw(MissingManhattanTimestamp(s"Manhattan timestamp missing in value $mhValue"))
|
||||||
|
}
|
||||||
|
|
||||||
|
private def jsonTimestamp(value: Array[Byte], tsType: TimestampType): Try[Long] =
|
||||||
|
Try { Json.decode(value) }
|
||||||
|
.rescue { case NonFatal(e) => Throw(UnparsableJson(e.getMessage, e)) }
|
||||||
|
.flatMap { m =>
|
||||||
|
m.get(tsType.keyName) match {
|
||||||
|
case Some(v) =>
|
||||||
|
v match {
|
||||||
|
case l: Long => Return(l)
|
||||||
|
case i: Integer => Return(i.toLong)
|
||||||
|
case _ =>
|
||||||
|
Throw(
|
||||||
|
UnexpectedJsonValue(s"Unexpected value for ${tsType.keyName} in record data $m")
|
||||||
|
)
|
||||||
|
}
|
||||||
|
case None =>
|
||||||
|
Throw(MissingJsonTimestamp(s"Missing key ${tsType.keyName} in record data $m"))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
def correctedTime(t: Time): Time =
|
||||||
|
if (t < BadTimestampThreshold) Time.fromMilliseconds(t.inNanoseconds) else t
|
||||||
|
|
||||||
|
def correctedTime(t: Long): Time = correctedTime(Time.fromNanoseconds(t))
|
||||||
|
|
||||||
|
def correctedTimestamp(t: Time): Long =
|
||||||
|
if (t < BadTimestampThreshold) t.inNanoseconds else t.inMilliseconds
|
||||||
|
}
|
@ -0,0 +1,164 @@
|
|||||||
|
package com.twitter.tweetypie.storage
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Responsible for encoding/decoding Tweet records to/from Manhattan keys
|
||||||
|
*
|
||||||
|
* K/V Scheme:
|
||||||
|
* -----------
|
||||||
|
* [TweetId]
|
||||||
|
* /metadata
|
||||||
|
* /delete_state (a.k.a. hard delete)
|
||||||
|
* /soft_delete_state
|
||||||
|
* /bounce_delete_state
|
||||||
|
* /undelete_state
|
||||||
|
* /force_added_state
|
||||||
|
* /scrubbed_fields/
|
||||||
|
* /[ScrubbedFieldId_1]
|
||||||
|
* ..
|
||||||
|
* /[ScrubbedFieldId_M]
|
||||||
|
* /fields
|
||||||
|
* /internal
|
||||||
|
* /1
|
||||||
|
* /9
|
||||||
|
* ..
|
||||||
|
* /99
|
||||||
|
* /external
|
||||||
|
* /100
|
||||||
|
* ..
|
||||||
|
*
|
||||||
|
* IMPORTANT NOTE:
|
||||||
|
* 1) Field Ids 2 to 8 in Tweet thrift struct are considered "core fields" are 'packed' together
|
||||||
|
* into a TFieldBlob and stored under field id 1 (i.e [DatasetName]/[TweetId]/fields/internal/1).
|
||||||
|
* This is why we do not see keys from [DatasetName]/[TweetId]/fields/internal/2 to [DatasetName]/
|
||||||
|
* [TweetId]/fields/internal/8)
|
||||||
|
*
|
||||||
|
* 2) Also, the tweet id (which is the field id 1 in Tweet thrift structure) is not explicitly stored
|
||||||
|
* in Manhattan. There is no need to explicitly store it since it is a part of the Pkey
|
||||||
|
*/
|
||||||
|
case class TweetKey(tweetId: TweetId, lKey: TweetKey.LKey) {
|
||||||
|
override def toString: String =
|
||||||
|
s"/${ManhattanOperations.PkeyInjection(tweetId)}/${ManhattanOperations.LkeyInjection(lKey)}"
|
||||||
|
}
|
||||||
|
|
||||||
|
object TweetKey {
|
||||||
|
// Manhattan uses lexicographical order for keys. To make sure lexicographical order matches the
|
||||||
|
// numerical order, we should pad both tweet id and field ids with leading zeros.
|
||||||
|
// Since tweet id is long and field id is a short, the max width of each can be obtained by doing
|
||||||
|
// Long.MaxValue.toString.length and Short.MaxValue.toString.length respectively
|
||||||
|
private val TweetIdFormatStr = s"%0${Long.MaxValue.toString.length}d"
|
||||||
|
private val FieldIdFormatStr = s"%0${Short.MaxValue.toString.length}d"
|
||||||
|
private[storage] def padTweetIdStr(tweetId: Long): String = TweetIdFormatStr.format(tweetId)
|
||||||
|
private[storage] def padFieldIdStr(fieldId: Short): String = FieldIdFormatStr.format(fieldId)
|
||||||
|
|
||||||
|
def coreFieldsKey(tweetId: TweetId): TweetKey = TweetKey(tweetId, LKey.CoreFieldsKey)
|
||||||
|
def hardDeletionStateKey(tweetId: TweetId): TweetKey =
|
||||||
|
TweetKey(tweetId, LKey.HardDeletionStateKey)
|
||||||
|
def softDeletionStateKey(tweetId: TweetId): TweetKey =
|
||||||
|
TweetKey(tweetId, LKey.SoftDeletionStateKey)
|
||||||
|
def bounceDeletionStateKey(tweetId: TweetId): TweetKey =
|
||||||
|
TweetKey(tweetId, LKey.BounceDeletionStateKey)
|
||||||
|
def unDeletionStateKey(tweetId: TweetId): TweetKey = TweetKey(tweetId, LKey.UnDeletionStateKey)
|
||||||
|
def forceAddedStateKey(tweetId: TweetId): TweetKey = TweetKey(tweetId, LKey.ForceAddedStateKey)
|
||||||
|
def scrubbedGeoFieldKey(tweetId: TweetId): TweetKey = TweetKey(tweetId, LKey.ScrubbedGeoFieldKey)
|
||||||
|
def fieldKey(tweetId: TweetId, fieldId: FieldId): TweetKey =
|
||||||
|
TweetKey(tweetId, LKey.FieldKey(fieldId))
|
||||||
|
def internalFieldsKey(tweetId: TweetId, fieldId: FieldId): TweetKey =
|
||||||
|
TweetKey(tweetId, LKey.InternalFieldsKey(fieldId))
|
||||||
|
def additionalFieldsKey(tweetId: TweetId, fieldId: FieldId): TweetKey =
|
||||||
|
TweetKey(tweetId, LKey.AdditionalFieldsKey(fieldId))
|
||||||
|
def scrubbedFieldKey(tweetId: TweetId, fieldId: FieldId): TweetKey =
|
||||||
|
TweetKey(tweetId, LKey.ScrubbedFieldKey(fieldId))
|
||||||
|
|
||||||
|
// AllFieldsKeyPrefix: fields
|
||||||
|
// CoreFieldsKey: fields/internal/1 (Stores subset of StoredTweet fields which are
|
||||||
|
// "packed" into a single CoreFields record)
|
||||||
|
// HardDeletionStateKey: metadata/delete_state
|
||||||
|
// SoftDeletionStateKey: metadata/soft_delete_state
|
||||||
|
// BounceDeletionStateKey: metadata/bounce_delete_state
|
||||||
|
// UnDeletionStateKey: metadata/undelete_state
|
||||||
|
// ForceAddedStateKey: metadata/force_added_state
|
||||||
|
// FieldKey: fields/<group_name>/<padded_field_id> (where <group_name>
|
||||||
|
// is 'internal' for field ids < 100 and 'external' for all other
|
||||||
|
// fields ids)
|
||||||
|
// InternalFieldsKeyPrefix: fields/internal
|
||||||
|
// PKey: <empty string>
|
||||||
|
// ScrubbedFieldKey: metadata/scrubbed_fields/<padded_field_id>
|
||||||
|
// ScrubbedFieldKeyPrefix: metadata/scrubbed_fields
|
||||||
|
sealed abstract class LKey(override val toString: String)
|
||||||
|
object LKey {
|
||||||
|
private val HardDeletionRecordLiteral = "delete_state"
|
||||||
|
private val SoftDeletionRecordLiteral = "soft_delete_state"
|
||||||
|
private val BounceDeletionRecordLiteral = "bounce_delete_state"
|
||||||
|
private val UnDeletionRecordLiteral = "undelete_state"
|
||||||
|
private val ForceAddRecordLiteral = "force_added_state"
|
||||||
|
private val ScrubbedFieldsGroup = "scrubbed_fields"
|
||||||
|
private val InternalFieldsGroup = "internal"
|
||||||
|
private val ExternalFieldsGroup = "external"
|
||||||
|
private val MetadataCategory = "metadata"
|
||||||
|
private val FieldsCategory = "fields"
|
||||||
|
private val InternalFieldsKeyPrefix = s"$FieldsCategory/$InternalFieldsGroup/"
|
||||||
|
private val ExternalFieldsKeyPrefix = s"$FieldsCategory/$ExternalFieldsGroup/"
|
||||||
|
private val ScrubbedFieldsKeyPrefix = s"$MetadataCategory/$ScrubbedFieldsGroup/"
|
||||||
|
|
||||||
|
sealed abstract class MetadataKey(metadataType: String)
|
||||||
|
extends LKey(s"$MetadataCategory/$metadataType")
|
||||||
|
sealed abstract class StateKey(stateType: String) extends MetadataKey(stateType)
|
||||||
|
case object HardDeletionStateKey extends StateKey(s"$HardDeletionRecordLiteral")
|
||||||
|
case object SoftDeletionStateKey extends StateKey(s"$SoftDeletionRecordLiteral")
|
||||||
|
case object BounceDeletionStateKey extends StateKey(s"$BounceDeletionRecordLiteral")
|
||||||
|
case object UnDeletionStateKey extends StateKey(s"$UnDeletionRecordLiteral")
|
||||||
|
case object ForceAddedStateKey extends StateKey(s"$ForceAddRecordLiteral")
|
||||||
|
|
||||||
|
case class ScrubbedFieldKey(fieldId: FieldId)
|
||||||
|
extends MetadataKey(s"$ScrubbedFieldsGroup/${padFieldIdStr(fieldId)}")
|
||||||
|
val ScrubbedGeoFieldKey: LKey.ScrubbedFieldKey = ScrubbedFieldKey(TweetFields.geoFieldId)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* LKey that has one of many possible fields id. This generalize over
|
||||||
|
* internal and additional fields key.
|
||||||
|
*/
|
||||||
|
sealed abstract class FieldKey(prefix: String) extends LKey(toString) {
|
||||||
|
def fieldId: FieldId
|
||||||
|
override val toString: String = prefix + padFieldIdStr(fieldId)
|
||||||
|
}
|
||||||
|
object FieldKey {
|
||||||
|
def apply(fieldId: FieldId): FieldKey =
|
||||||
|
fieldId match {
|
||||||
|
case id if id < TweetFields.firstAdditionalFieldId => InternalFieldsKey(fieldId)
|
||||||
|
case _ => AdditionalFieldsKey(fieldId)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
case class InternalFieldsKey(fieldId: FieldId) extends FieldKey(InternalFieldsKeyPrefix) {
|
||||||
|
assert(fieldId < TweetFields.firstAdditionalFieldId)
|
||||||
|
}
|
||||||
|
case class AdditionalFieldsKey(fieldId: FieldId) extends FieldKey(ExternalFieldsKeyPrefix) {
|
||||||
|
assert(fieldId >= TweetFields.firstAdditionalFieldId)
|
||||||
|
}
|
||||||
|
val CoreFieldsKey: LKey.InternalFieldsKey = InternalFieldsKey(TweetFields.rootCoreFieldId)
|
||||||
|
|
||||||
|
case class Unknown private (str: String) extends LKey(str)
|
||||||
|
|
||||||
|
def fromString(str: String): LKey = {
|
||||||
|
def extractFieldId(prefix: String): FieldId =
|
||||||
|
str.slice(prefix.length, str.length).toShort
|
||||||
|
|
||||||
|
str match {
|
||||||
|
case CoreFieldsKey.toString => CoreFieldsKey
|
||||||
|
case HardDeletionStateKey.toString => HardDeletionStateKey
|
||||||
|
case SoftDeletionStateKey.toString => SoftDeletionStateKey
|
||||||
|
case BounceDeletionStateKey.toString => BounceDeletionStateKey
|
||||||
|
case UnDeletionStateKey.toString => UnDeletionStateKey
|
||||||
|
case ForceAddedStateKey.toString => ForceAddedStateKey
|
||||||
|
case ScrubbedGeoFieldKey.toString => ScrubbedGeoFieldKey
|
||||||
|
case _ if str.startsWith(InternalFieldsKeyPrefix) =>
|
||||||
|
InternalFieldsKey(extractFieldId(InternalFieldsKeyPrefix))
|
||||||
|
case _ if str.startsWith(ExternalFieldsKeyPrefix) =>
|
||||||
|
AdditionalFieldsKey(extractFieldId(ExternalFieldsKeyPrefix))
|
||||||
|
case _ if str.startsWith(ScrubbedFieldsKeyPrefix) =>
|
||||||
|
ScrubbedFieldKey(extractFieldId(ScrubbedFieldsKeyPrefix))
|
||||||
|
case _ => Unknown(str)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,90 @@
|
|||||||
|
package com.twitter.tweetypie.storage
|
||||||
|
|
||||||
|
import com.twitter.storage.client.manhattan.kv.ManhattanValue
|
||||||
|
import com.twitter.util.Time
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A [[TweetStateRecord]] represents an action taken on a tweet and can be used to determine a tweet's state.
|
||||||
|
*
|
||||||
|
* The state is determined by the record with the most recent timestamp. In the absence of any
|
||||||
|
* record a tweet is considered found, which is to say the tweet has not been through the
|
||||||
|
* deletion process.
|
||||||
|
*
|
||||||
|
* The [[TweetStateRecord]] type is determined by the lkey of a tweet manhattan record:
|
||||||
|
* metadata/delete_state -> HardDeleted
|
||||||
|
* metadata/soft_delete_state -> SoftDeleted
|
||||||
|
* metadata/undelete_state -> Undeleted
|
||||||
|
* metadata/force_added_state -> ForceAdded
|
||||||
|
*
|
||||||
|
* See the README in this directory for more details about the state of a tweet.
|
||||||
|
*/
|
||||||
|
sealed trait TweetStateRecord {
|
||||||
|
def tweetId: TweetId
|
||||||
|
def createdAt: Long
|
||||||
|
def stateKey: TweetKey.LKey.StateKey
|
||||||
|
def values: Map[String, Long] = Map("timestamp" -> createdAt)
|
||||||
|
def name: String
|
||||||
|
|
||||||
|
def toTweetMhRecord: TweetManhattanRecord = {
|
||||||
|
val valByteBuffer = ByteArrayCodec.toByteBuffer(Json.encode(values))
|
||||||
|
val value = ManhattanValue(valByteBuffer, Some(Time.fromMilliseconds(createdAt)))
|
||||||
|
TweetManhattanRecord(TweetKey(tweetId, stateKey), value)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
object TweetStateRecord {
|
||||||
|
|
||||||
|
/** When a soft-deleted or bounce deleted tweet is ultimately hard-deleted by an offline job. */
|
||||||
|
case class HardDeleted(tweetId: TweetId, createdAt: Long, deletedAt: Long)
|
||||||
|
extends TweetStateRecord {
|
||||||
|
// timestamp in the mh backend is the hard deletion timestamp
|
||||||
|
override def values = Map("timestamp" -> createdAt, "softdelete_timestamp" -> deletedAt)
|
||||||
|
def stateKey = TweetKey.LKey.HardDeletionStateKey
|
||||||
|
def name = "hard_deleted"
|
||||||
|
}
|
||||||
|
|
||||||
|
/** When a tweet is deleted by the user. It can still be undeleted while in the soft deleted state. */
|
||||||
|
case class SoftDeleted(tweetId: TweetId, createdAt: Long) extends TweetStateRecord {
|
||||||
|
def stateKey = TweetKey.LKey.SoftDeletionStateKey
|
||||||
|
def name = "soft_deleted"
|
||||||
|
}
|
||||||
|
|
||||||
|
/** When a tweet is deleted by go/bouncer for violating Twitter Rules. It MAY NOT be undeleted. */
|
||||||
|
case class BounceDeleted(tweetId: TweetId, createdAt: Long) extends TweetStateRecord {
|
||||||
|
def stateKey = TweetKey.LKey.BounceDeletionStateKey
|
||||||
|
def name = "bounce_deleted"
|
||||||
|
}
|
||||||
|
|
||||||
|
/** When a tweet is undeleted by an internal system. */
|
||||||
|
case class Undeleted(tweetId: TweetId, createdAt: Long) extends TweetStateRecord {
|
||||||
|
def stateKey = TweetKey.LKey.UnDeletionStateKey
|
||||||
|
def name = "undeleted"
|
||||||
|
}
|
||||||
|
|
||||||
|
/** When a tweet is created using the forceAdd endpoint. */
|
||||||
|
case class ForceAdded(tweetId: TweetId, createdAt: Long) extends TweetStateRecord {
|
||||||
|
def stateKey = TweetKey.LKey.ForceAddedStateKey
|
||||||
|
def name = "force_added"
|
||||||
|
}
|
||||||
|
|
||||||
|
def fromTweetMhRecord(record: TweetManhattanRecord): Option[TweetStateRecord] = {
|
||||||
|
def ts = TimestampDecoder.decode(record, TimestampType.Default).getOrElse(0L)
|
||||||
|
def sdts = TimestampDecoder.decode(record, TimestampType.SoftDelete).getOrElse(0L)
|
||||||
|
def tweetId = record.pkey
|
||||||
|
|
||||||
|
record.lkey match {
|
||||||
|
case TweetKey.LKey.HardDeletionStateKey => Some(HardDeleted(tweetId, ts, sdts))
|
||||||
|
case TweetKey.LKey.SoftDeletionStateKey => Some(SoftDeleted(tweetId, ts))
|
||||||
|
case TweetKey.LKey.BounceDeletionStateKey => Some(BounceDeleted(tweetId, ts))
|
||||||
|
case TweetKey.LKey.UnDeletionStateKey => Some(Undeleted(tweetId, ts))
|
||||||
|
case TweetKey.LKey.ForceAddedStateKey => Some(ForceAdded(tweetId, ts))
|
||||||
|
case _ => None
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
def fromTweetMhRecords(records: Seq[TweetManhattanRecord]): Seq[TweetStateRecord] =
|
||||||
|
records.flatMap(fromTweetMhRecord)
|
||||||
|
|
||||||
|
def mostRecent(records: Seq[TweetManhattanRecord]): Option[TweetStateRecord] =
|
||||||
|
fromTweetMhRecords(records).sortBy(_.createdAt).lastOption
|
||||||
|
}
|
@ -0,0 +1,201 @@
|
|||||||
|
package com.twitter.tweetypie.storage
|
||||||
|
|
||||||
|
import com.twitter.stitch.Stitch
|
||||||
|
import com.twitter.tweetypie.storage.Response.TweetResponse
|
||||||
|
import com.twitter.tweetypie.thriftscala.Tweet
|
||||||
|
import com.twitter.util.Future
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Interface for reading and writing tweet data in Manhattan
|
||||||
|
*/
|
||||||
|
trait TweetStorageClient {
|
||||||
|
import TweetStorageClient._
|
||||||
|
def addTweet: AddTweet
|
||||||
|
def deleteAdditionalFields: DeleteAdditionalFields
|
||||||
|
def getTweet: GetTweet
|
||||||
|
def getStoredTweet: GetStoredTweet
|
||||||
|
def getDeletedTweets: GetDeletedTweets
|
||||||
|
def undelete: Undelete
|
||||||
|
def updateTweet: UpdateTweet
|
||||||
|
def scrub: Scrub
|
||||||
|
def softDelete: SoftDelete
|
||||||
|
def bounceDelete: BounceDelete
|
||||||
|
def hardDeleteTweet: HardDeleteTweet
|
||||||
|
def ping: Ping
|
||||||
|
}
|
||||||
|
|
||||||
|
object TweetStorageClient {
|
||||||
|
type GetTweet = TweetId => Stitch[GetTweet.Response]
|
||||||
|
|
||||||
|
object GetTweet {
|
||||||
|
sealed trait Response
|
||||||
|
object Response {
|
||||||
|
case class Found(tweet: Tweet) extends Response
|
||||||
|
object NotFound extends Response
|
||||||
|
object Deleted extends Response
|
||||||
|
// On BounceDeleted, provide the full Tweet so that implementations
|
||||||
|
// (i.e. ManhattanTweetStorageClient) don't not need to be aware of the specific tweet
|
||||||
|
// fields required by callers for proper processing of bounced deleted tweets.
|
||||||
|
case class BounceDeleted(tweet: Tweet) extends Response
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
type GetStoredTweet = TweetId => Stitch[GetStoredTweet.Response]
|
||||||
|
|
||||||
|
object GetStoredTweet {
|
||||||
|
sealed abstract class Error(val message: String) {
|
||||||
|
override def toString: String = message
|
||||||
|
}
|
||||||
|
object Error {
|
||||||
|
case object TweetIsCorrupt extends Error("stored tweet data is corrupt and cannot be decoded")
|
||||||
|
|
||||||
|
case object ScrubbedFieldsPresent
|
||||||
|
extends Error("stored tweet fields that should be scrubbed are still present")
|
||||||
|
|
||||||
|
case object TweetFieldsMissingOrInvalid
|
||||||
|
extends Error("expected tweet fields are missing or contain invalid values")
|
||||||
|
|
||||||
|
case object TweetShouldBeHardDeleted
|
||||||
|
extends Error("stored tweet that should be hard deleted is still present")
|
||||||
|
}
|
||||||
|
|
||||||
|
sealed trait Response
|
||||||
|
object Response {
|
||||||
|
sealed trait StoredTweetMetadata {
|
||||||
|
def state: Option[TweetStateRecord]
|
||||||
|
def allStates: Seq[TweetStateRecord]
|
||||||
|
def scrubbedFields: Set[FieldId]
|
||||||
|
}
|
||||||
|
|
||||||
|
sealed trait StoredTweetErrors {
|
||||||
|
def errs: Seq[Error]
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tweet data was found, possibly state records and/or scrubbed field records.
|
||||||
|
*/
|
||||||
|
sealed trait FoundAny extends Response with StoredTweetMetadata {
|
||||||
|
def tweet: Tweet
|
||||||
|
}
|
||||||
|
|
||||||
|
object FoundAny {
|
||||||
|
def unapply(
|
||||||
|
response: Response
|
||||||
|
): Option[
|
||||||
|
(Tweet, Option[TweetStateRecord], Seq[TweetStateRecord], Set[FieldId], Seq[Error])
|
||||||
|
] =
|
||||||
|
response match {
|
||||||
|
case f: FoundWithErrors =>
|
||||||
|
Some((f.tweet, f.state, f.allStates, f.scrubbedFields, f.errs))
|
||||||
|
case f: FoundAny => Some((f.tweet, f.state, f.allStates, f.scrubbedFields, Seq.empty))
|
||||||
|
case _ => None
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* No records for this tweet id were found in storage
|
||||||
|
*/
|
||||||
|
case class NotFound(id: TweetId) extends Response
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Data related to the Tweet id was found but could not be loaded successfully. The
|
||||||
|
* errs array contains details of the problems.
|
||||||
|
*/
|
||||||
|
case class Failed(
|
||||||
|
id: TweetId,
|
||||||
|
state: Option[TweetStateRecord],
|
||||||
|
allStates: Seq[TweetStateRecord],
|
||||||
|
scrubbedFields: Set[FieldId],
|
||||||
|
errs: Seq[Error],
|
||||||
|
) extends Response
|
||||||
|
with StoredTweetMetadata
|
||||||
|
with StoredTweetErrors
|
||||||
|
|
||||||
|
/**
|
||||||
|
* No Tweet data was found, and the most recent state record found is HardDeleted
|
||||||
|
*/
|
||||||
|
case class HardDeleted(
|
||||||
|
id: TweetId,
|
||||||
|
state: Option[TweetStateRecord.HardDeleted],
|
||||||
|
allStates: Seq[TweetStateRecord],
|
||||||
|
scrubbedFields: Set[FieldId],
|
||||||
|
) extends Response
|
||||||
|
with StoredTweetMetadata
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tweet data was found, and the most recent state record found, if any, is not
|
||||||
|
* any form of deletion record.
|
||||||
|
*/
|
||||||
|
case class Found(
|
||||||
|
tweet: Tweet,
|
||||||
|
state: Option[TweetStateRecord],
|
||||||
|
allStates: Seq[TweetStateRecord],
|
||||||
|
scrubbedFields: Set[FieldId],
|
||||||
|
) extends FoundAny
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tweet data was found, and the most recent state record found indicates deletion.
|
||||||
|
*/
|
||||||
|
case class FoundDeleted(
|
||||||
|
tweet: Tweet,
|
||||||
|
state: Option[TweetStateRecord],
|
||||||
|
allStates: Seq[TweetStateRecord],
|
||||||
|
scrubbedFields: Set[FieldId],
|
||||||
|
) extends FoundAny
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tweet data was found, however errors were detected in the stored data. Required
|
||||||
|
* fields may be missing from the Tweet struct (e.g. CoreData), stored fields that
|
||||||
|
* should be scrubbed remain present, or Tweets that should be hard-deleted remain
|
||||||
|
* in storage. The errs array contains details of the problems.
|
||||||
|
*/
|
||||||
|
case class FoundWithErrors(
|
||||||
|
tweet: Tweet,
|
||||||
|
state: Option[TweetStateRecord],
|
||||||
|
allStates: Seq[TweetStateRecord],
|
||||||
|
scrubbedFields: Set[FieldId],
|
||||||
|
errs: Seq[Error],
|
||||||
|
) extends FoundAny
|
||||||
|
with StoredTweetErrors
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
type HardDeleteTweet = TweetId => Stitch[HardDeleteTweet.Response]
|
||||||
|
type SoftDelete = TweetId => Stitch[Unit]
|
||||||
|
type BounceDelete = TweetId => Stitch[Unit]
|
||||||
|
|
||||||
|
object HardDeleteTweet {
|
||||||
|
sealed trait Response
|
||||||
|
object Response {
|
||||||
|
case class Deleted(deletedAtMillis: Option[Long], createdAtMillis: Option[Long])
|
||||||
|
extends Response
|
||||||
|
case class NotDeleted(id: TweetId, ineligibleLKey: Option[TweetKey.LKey])
|
||||||
|
extends Throwable
|
||||||
|
with Response
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
type Undelete = TweetId => Stitch[Undelete.Response]
|
||||||
|
object Undelete {
|
||||||
|
case class Response(
|
||||||
|
code: UndeleteResponseCode,
|
||||||
|
tweet: Option[Tweet] = None,
|
||||||
|
createdAtMillis: Option[Long] = None,
|
||||||
|
archivedAtMillis: Option[Long] = None)
|
||||||
|
|
||||||
|
sealed trait UndeleteResponseCode
|
||||||
|
|
||||||
|
object UndeleteResponseCode {
|
||||||
|
object Success extends UndeleteResponseCode
|
||||||
|
object BackupNotFound extends UndeleteResponseCode
|
||||||
|
object NotCreated extends UndeleteResponseCode
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
type AddTweet = Tweet => Stitch[Unit]
|
||||||
|
type UpdateTweet = (Tweet, Seq[Field]) => Stitch[TweetResponse]
|
||||||
|
type GetDeletedTweets = Seq[TweetId] => Stitch[Seq[DeletedTweetResponse]]
|
||||||
|
type DeleteAdditionalFields = (Seq[TweetId], Seq[Field]) => Stitch[Seq[TweetResponse]]
|
||||||
|
type Scrub = (Seq[TweetId], Seq[Field]) => Stitch[Unit]
|
||||||
|
type Ping = () => Future[Unit]
|
||||||
|
}
|
@ -0,0 +1,34 @@
|
|||||||
|
package com.twitter.tweetypie.storage
|
||||||
|
|
||||||
|
import scala.util.control.NoStackTrace
|
||||||
|
|
||||||
|
sealed abstract class TweetStorageException(message: String, cause: Throwable)
|
||||||
|
extends Exception(message, cause)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The request was not properly formed and failed an assertion present in the code. Should not be
|
||||||
|
* retried without modification.
|
||||||
|
*/
|
||||||
|
case class ClientError(message: String, cause: Throwable)
|
||||||
|
extends TweetStorageException(message, cause)
|
||||||
|
with NoStackTrace
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Request was rejected by Manhattan or the in-process rate limiter. Should not be retried.
|
||||||
|
*/
|
||||||
|
case class RateLimited(message: String, cause: Throwable)
|
||||||
|
extends TweetStorageException(message, cause)
|
||||||
|
with NoStackTrace
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Corrupt tweets were requested from Manhattan
|
||||||
|
*/
|
||||||
|
case class VersionMismatchError(message: String, cause: Throwable = null)
|
||||||
|
extends TweetStorageException(message, cause)
|
||||||
|
with NoStackTrace
|
||||||
|
|
||||||
|
/**
|
||||||
|
* All other unhandled exceptions.
|
||||||
|
*/
|
||||||
|
case class InternalError(message: String, cause: Throwable = null)
|
||||||
|
extends TweetStorageException(message, cause)
|
@ -0,0 +1,265 @@
|
|||||||
|
package com.twitter.tweetypie.storage
|
||||||
|
|
||||||
|
import com.twitter.logging.Logger
|
||||||
|
import com.twitter.scrooge.TFieldBlob
|
||||||
|
import com.twitter.snowflake.id.SnowflakeId
|
||||||
|
import com.twitter.storage.client.manhattan.kv.DeniedManhattanException
|
||||||
|
import com.twitter.storage.client.manhattan.kv.ManhattanException
|
||||||
|
import com.twitter.tweetypie.storage.Response._
|
||||||
|
import com.twitter.tweetypie.storage_internal.thriftscala.StoredTweet
|
||||||
|
import com.twitter.util.Return
|
||||||
|
import com.twitter.util.Throw
|
||||||
|
import com.twitter.util.Try
|
||||||
|
|
||||||
|
object TweetUtils {
|
||||||
|
val log: Logger = Logger("com.twitter.tweetypie.storage.TweetStorageLibrary")
|
||||||
|
import FieldResponseCodec.ValueNotFoundException
|
||||||
|
|
||||||
|
/**
|
||||||
|
* It's rare, but we have seen tweets with userId=0, which is likely the result of a
|
||||||
|
* failed/partial delete. Treat these as invalid tweets, which are returned to callers
|
||||||
|
* as not found.
|
||||||
|
*/
|
||||||
|
def isValid(tweet: StoredTweet): Boolean =
|
||||||
|
tweet.userId.exists(_ != 0) && tweet.text.nonEmpty &&
|
||||||
|
tweet.createdVia.nonEmpty && tweet.createdAtSec.nonEmpty
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Helper function to extract Scrubbed field Ids from the result returned by reading entire tweet prefix
|
||||||
|
* function.
|
||||||
|
*
|
||||||
|
* @param records The sequence of MH records for the given tweetId
|
||||||
|
*
|
||||||
|
* @return The set of scrubbed field ids
|
||||||
|
*/
|
||||||
|
private[tweetypie] def extractScrubbedFields(records: Seq[TweetManhattanRecord]): Set[Short] =
|
||||||
|
records
|
||||||
|
.map(r => r.lkey)
|
||||||
|
.collect { case TweetKey.LKey.ScrubbedFieldKey(fieldId) => fieldId }
|
||||||
|
.toSet
|
||||||
|
|
||||||
|
private[tweetypie] val expectedFields =
|
||||||
|
TweetFields.requiredFieldIds.toSet - TweetFields.tweetIdField
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Find the timestamp from a tweetId and a list of MH records. This is used when
|
||||||
|
* you need a timestamp and you aren't sure that tweetId is a snowflake id.
|
||||||
|
*
|
||||||
|
* @param tweetId A tweetId you want the timestamp for.
|
||||||
|
* @param records Tbird_mh records keyed on tweetId, one of which should be the
|
||||||
|
* core fields record.
|
||||||
|
* @return A milliseconds timestamp if one could be found.
|
||||||
|
*/
|
||||||
|
private[tweetypie] def creationTimeFromTweetIdOrMHRecords(
|
||||||
|
tweetId: Long,
|
||||||
|
records: Seq[TweetManhattanRecord]
|
||||||
|
): Option[Long] =
|
||||||
|
SnowflakeId
|
||||||
|
.unixTimeMillisOptFromId(tweetId).orElse({
|
||||||
|
records
|
||||||
|
.find(_.lkey == TweetKey.LKey.CoreFieldsKey)
|
||||||
|
.flatMap { coreFields =>
|
||||||
|
CoreFieldsCodec
|
||||||
|
.fromTFieldBlob(
|
||||||
|
TFieldBlobCodec.fromByteBuffer(coreFields.value.contents)
|
||||||
|
).createdAtSec.map(seconds => seconds * 1000)
|
||||||
|
}
|
||||||
|
})
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Helper function used to parse manhattan results for fields in a tweet (given in the form of
|
||||||
|
* Sequence of (FieldKey, Try[Unit]) pairs) and build a TweetResponse object.
|
||||||
|
*
|
||||||
|
* @param callerName The name of the caller function. Used for error messages
|
||||||
|
* @param tweetId Id of the Tweet for which TweetResponse is being built
|
||||||
|
* @param fieldResults Sequence of (FieldKey, Try[Unit]).
|
||||||
|
*
|
||||||
|
* @return TweetResponse object
|
||||||
|
*/
|
||||||
|
private[tweetypie] def buildTweetResponse(
|
||||||
|
callerName: String,
|
||||||
|
tweetId: Long,
|
||||||
|
fieldResults: Map[FieldId, Try[Unit]]
|
||||||
|
): TweetResponse = {
|
||||||
|
// Count Found/Not Found
|
||||||
|
val successCount =
|
||||||
|
fieldResults.foldLeft(0) {
|
||||||
|
case (count, (_, Return(_))) => count + 1
|
||||||
|
case (count, (_, Throw(_: ValueNotFoundException))) => count + 1
|
||||||
|
case (count, _) => count
|
||||||
|
}
|
||||||
|
|
||||||
|
val fieldResponsesMap = getFieldResponses(callerName, tweetId, fieldResults)
|
||||||
|
|
||||||
|
val overallCode = if (successCount > 0 && successCount == fieldResults.size) {
|
||||||
|
TweetResponseCode.Success
|
||||||
|
} else {
|
||||||
|
|
||||||
|
// If any field was rate limited, then we consider the entire tweet to be rate limited. So first we scan
|
||||||
|
// the field results to check such an occurrence.
|
||||||
|
val wasRateLimited = fieldResults.exists { fieldResult =>
|
||||||
|
fieldResult._2 match {
|
||||||
|
case Throw(e: DeniedManhattanException) => true
|
||||||
|
case _ => false
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Were we rate limited for any of the additional fields?
|
||||||
|
if (wasRateLimited) {
|
||||||
|
TweetResponseCode.OverCapacity
|
||||||
|
} else if (successCount == 0) {
|
||||||
|
// successCount is < fieldResults.size at this point. So if allOrNone is true or
|
||||||
|
// if successCount == 0 (i.e failed on all Fields), the overall code should be 'Failure'
|
||||||
|
TweetResponseCode.Failure
|
||||||
|
} else {
|
||||||
|
// allOrNone == false AND successCount > 0 at this point. Clearly the overallCode should be Partial
|
||||||
|
TweetResponseCode.Partial
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
TweetResponse(tweetId, overallCode, Some(fieldResponsesMap))
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Helper function to convert manhattan results into a Map[FieldId, FieldResponse]
|
||||||
|
*
|
||||||
|
* @param fieldResults Sequence of (TweetKey, TFieldBlob).
|
||||||
|
*/
|
||||||
|
private[tweetypie] def getFieldResponses(
|
||||||
|
callerName: String,
|
||||||
|
tweetId: TweetId,
|
||||||
|
fieldResults: Map[FieldId, Try[_]]
|
||||||
|
): Map[FieldId, FieldResponse] =
|
||||||
|
fieldResults.map {
|
||||||
|
case (fieldId, resp) =>
|
||||||
|
def keyStr = TweetKey.fieldKey(tweetId, fieldId).toString
|
||||||
|
resp match {
|
||||||
|
case Return(_) =>
|
||||||
|
fieldId -> FieldResponse(FieldResponseCode.Success, None)
|
||||||
|
case Throw(mhException: ManhattanException) =>
|
||||||
|
val errMsg = s"Exception in $callerName. Key: $keyStr. Error: $mhException"
|
||||||
|
mhException match {
|
||||||
|
case _: ValueNotFoundException => // ValueNotFound is not an error
|
||||||
|
case _ => log.error(errMsg)
|
||||||
|
}
|
||||||
|
fieldId -> FieldResponseCodec.fromThrowable(mhException, Some(errMsg))
|
||||||
|
case Throw(e) =>
|
||||||
|
val errMsg = s"Exception in $callerName. Key: $keyStr. Error: $e"
|
||||||
|
log.error(errMsg)
|
||||||
|
fieldId -> FieldResponse(FieldResponseCode.Error, Some(errMsg))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Helper function to build a TweetResponse object when being rate limited. Its possible that only some of the fields
|
||||||
|
* got rate limited, so we indicate which fields got processed successfully, and which encountered some sort of error.
|
||||||
|
*
|
||||||
|
* @param tweetId Tweet id
|
||||||
|
* @param callerName name of API calling this function
|
||||||
|
* @param fieldResponses field responses for the case where
|
||||||
|
*
|
||||||
|
* @return The TweetResponse object
|
||||||
|
*/
|
||||||
|
private[tweetypie] def buildTweetOverCapacityResponse(
|
||||||
|
callerName: String,
|
||||||
|
tweetId: Long,
|
||||||
|
fieldResponses: Map[FieldId, Try[Unit]]
|
||||||
|
) = {
|
||||||
|
val fieldResponsesMap = getFieldResponses(callerName, tweetId, fieldResponses)
|
||||||
|
TweetResponse(tweetId, TweetResponseCode.OverCapacity, Some(fieldResponsesMap))
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Build a StoredTweet from a Seq of records. Core fields are handled specially.
|
||||||
|
*/
|
||||||
|
private[tweetypie] def buildStoredTweet(
|
||||||
|
tweetId: TweetId,
|
||||||
|
records: Seq[TweetManhattanRecord],
|
||||||
|
includeScrubbed: Boolean = false,
|
||||||
|
): StoredTweet = {
|
||||||
|
getStoredTweetBlobs(records, includeScrubbed)
|
||||||
|
.flatMap { fieldBlob =>
|
||||||
|
// When fieldId == TweetFields.rootCoreFieldId, we have further work to do since the
|
||||||
|
// 'value' is really serialized/packed version of all core fields. In this case we'll have
|
||||||
|
// to unpack it into many TFieldBlobs.
|
||||||
|
if (fieldBlob.id == TweetFields.rootCoreFieldId) {
|
||||||
|
// We won't throw any error in this function and instead let the caller function handle this
|
||||||
|
// condition (i.e If the caller function does not find any values for the core-fields in
|
||||||
|
// the returned map, it should assume that the tweet is not found)
|
||||||
|
CoreFieldsCodec.unpackFields(fieldBlob).values.toSeq
|
||||||
|
} else {
|
||||||
|
Seq(fieldBlob)
|
||||||
|
}
|
||||||
|
}.foldLeft(StoredTweet(tweetId))(_.setField(_))
|
||||||
|
}
|
||||||
|
|
||||||
|
private[tweetypie] def buildValidStoredTweet(
|
||||||
|
tweetId: TweetId,
|
||||||
|
records: Seq[TweetManhattanRecord]
|
||||||
|
): Option[StoredTweet] = {
|
||||||
|
val storedTweet = buildStoredTweet(tweetId, records)
|
||||||
|
if (storedTweet.getFieldBlobs(expectedFields).nonEmpty && isValid(storedTweet)) {
|
||||||
|
Some(storedTweet)
|
||||||
|
} else {
|
||||||
|
None
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return a TFieldBlob for each StoredTweet field defined in this set of records.
|
||||||
|
* @param includeScrubbed when false, result will not include scrubbed fields even
|
||||||
|
* if the data is present in the set of records.
|
||||||
|
*/
|
||||||
|
private[tweetypie] def getStoredTweetBlobs(
|
||||||
|
records: Seq[TweetManhattanRecord],
|
||||||
|
includeScrubbed: Boolean = false,
|
||||||
|
): Seq[TFieldBlob] = {
|
||||||
|
val scrubbed = extractScrubbedFields(records)
|
||||||
|
|
||||||
|
records
|
||||||
|
.flatMap { r =>
|
||||||
|
// extract LKey.FieldKey records if they are not scrubbed and get their TFieldBlobs
|
||||||
|
r.key match {
|
||||||
|
case fullKey @ TweetKey(_, key: TweetKey.LKey.FieldKey)
|
||||||
|
if includeScrubbed || !scrubbed.contains(key.fieldId) =>
|
||||||
|
try {
|
||||||
|
val fieldBlob = TFieldBlobCodec.fromByteBuffer(r.value.contents)
|
||||||
|
if (fieldBlob.field.id != key.fieldId) {
|
||||||
|
throw new AssertionError(
|
||||||
|
s"Blob stored for $fullKey has unexpected id ${fieldBlob.field.id}"
|
||||||
|
)
|
||||||
|
}
|
||||||
|
Some(fieldBlob)
|
||||||
|
} catch {
|
||||||
|
case e: VersionMismatchError =>
|
||||||
|
log.error(
|
||||||
|
s"Failed to decode bytebuffer for $fullKey: ${e.getMessage}"
|
||||||
|
)
|
||||||
|
throw e
|
||||||
|
}
|
||||||
|
case _ => None
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Its important to bubble up rate limiting exceptions as they would likely be the root cause for other issues
|
||||||
|
* (timeouts etc.), so we scan for this particular exception, and if found, we bubble that up specifically
|
||||||
|
*
|
||||||
|
* @param seqOfTries The sequence of tries which may contain within it a rate limit exception
|
||||||
|
*
|
||||||
|
* @return if a rate limiting exn was detected, this will be a Throw(e: DeniedManhattanException)
|
||||||
|
* otherwise it will be a Return(_) only if all individual tries succeeded
|
||||||
|
*/
|
||||||
|
private[tweetypie] def collectWithRateLimitCheck(seqOfTries: Seq[Try[Unit]]): Try[Unit] = {
|
||||||
|
val rateLimitThrowOpt = seqOfTries.find {
|
||||||
|
case Throw(e: DeniedManhattanException) => true
|
||||||
|
case _ => false
|
||||||
|
}
|
||||||
|
|
||||||
|
rateLimitThrowOpt.getOrElse(
|
||||||
|
Try.collect(seqOfTries).map(_ => ())
|
||||||
|
) // Operation is considered successful only if all the deletions are successful
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,106 @@
|
|||||||
|
package com.twitter.tweetypie.storage
|
||||||
|
|
||||||
|
import com.twitter.finagle.stats.StatsReceiver
|
||||||
|
import com.twitter.stitch.Stitch
|
||||||
|
import com.twitter.tweetypie.storage.TweetStorageClient.Undelete
|
||||||
|
import com.twitter.tweetypie.storage.TweetUtils._
|
||||||
|
import com.twitter.util.Time
|
||||||
|
|
||||||
|
object UndeleteHandler {
|
||||||
|
def apply(
|
||||||
|
read: ManhattanOperations.Read,
|
||||||
|
localInsert: ManhattanOperations.Insert,
|
||||||
|
remoteInsert: ManhattanOperations.Insert,
|
||||||
|
delete: ManhattanOperations.Delete,
|
||||||
|
undeleteWindowHours: Int,
|
||||||
|
stats: StatsReceiver
|
||||||
|
): Undelete = {
|
||||||
|
def withinUndeleteWindow(timestampMs: Long) =
|
||||||
|
(Time.now - Time.fromMilliseconds(timestampMs)).inHours < undeleteWindowHours
|
||||||
|
|
||||||
|
def prepareUndelete(
|
||||||
|
tweetId: TweetId,
|
||||||
|
records: Seq[TweetManhattanRecord]
|
||||||
|
): (Undelete.Response, Option[TweetManhattanRecord]) = {
|
||||||
|
val undeleteRecord =
|
||||||
|
Some(TweetStateRecord.Undeleted(tweetId, Time.now.inMillis).toTweetMhRecord)
|
||||||
|
|
||||||
|
TweetStateRecord.mostRecent(records) match {
|
||||||
|
// check if we need to undo a soft deletion
|
||||||
|
case Some(TweetStateRecord.SoftDeleted(_, createdAt)) =>
|
||||||
|
if (createdAt > 0) {
|
||||||
|
if (withinUndeleteWindow(createdAt)) {
|
||||||
|
(
|
||||||
|
mkSuccessfulUndeleteResponse(tweetId, records, Some(createdAt)),
|
||||||
|
undeleteRecord
|
||||||
|
)
|
||||||
|
} else {
|
||||||
|
(Undelete.Response(Undelete.UndeleteResponseCode.BackupNotFound), None)
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
throw InternalError(s"Timestamp unavailable for $tweetId")
|
||||||
|
}
|
||||||
|
|
||||||
|
// BounceDeleted tweets may not be undeleted. see go/bouncedtweet
|
||||||
|
case Some(_: TweetStateRecord.HardDeleted | _: TweetStateRecord.BounceDeleted) =>
|
||||||
|
(Undelete.Response(Undelete.UndeleteResponseCode.BackupNotFound), None)
|
||||||
|
|
||||||
|
case Some(_: TweetStateRecord.Undeleted) =>
|
||||||
|
// We still want to write the undelete record, because at this point, we only know that the local DC's
|
||||||
|
// winning record is not a soft/hard deletion record, while its possible that the remote DC's winning
|
||||||
|
// record might still be a soft deletion record. Having said that, we don't want to set it to true
|
||||||
|
// if the winning record is forceAdd, as the forceAdd call should have ensured that both DCs had the
|
||||||
|
// forceAdd record.
|
||||||
|
(mkSuccessfulUndeleteResponse(tweetId, records), undeleteRecord)
|
||||||
|
|
||||||
|
case Some(_: TweetStateRecord.ForceAdded) =>
|
||||||
|
(mkSuccessfulUndeleteResponse(tweetId, records), None)
|
||||||
|
|
||||||
|
// lets write the undeletion record just in case there is a softdeletion record in flight
|
||||||
|
case None => (mkSuccessfulUndeleteResponse(tweetId, records), undeleteRecord)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Write the undelete record both locally and remotely to protect
|
||||||
|
// against races with hard delete replication. We only need this
|
||||||
|
// protection for the insertion of the undelete record.
|
||||||
|
def multiInsert(record: TweetManhattanRecord): Stitch[Unit] =
|
||||||
|
Stitch
|
||||||
|
.collect(
|
||||||
|
Seq(
|
||||||
|
localInsert(record).liftToTry,
|
||||||
|
remoteInsert(record).liftToTry
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.map(collectWithRateLimitCheck)
|
||||||
|
.lowerFromTry
|
||||||
|
|
||||||
|
def deleteSoftDeleteRecord(tweetId: TweetId): Stitch[Unit] = {
|
||||||
|
val mhKey = TweetKey.softDeletionStateKey(tweetId)
|
||||||
|
delete(mhKey, None)
|
||||||
|
}
|
||||||
|
|
||||||
|
tweetId =>
|
||||||
|
for {
|
||||||
|
records <- read(tweetId)
|
||||||
|
(response, undeleteRecord) = prepareUndelete(tweetId, records)
|
||||||
|
_ <- Stitch.collect(undeleteRecord.map(multiInsert)).unit
|
||||||
|
_ <- deleteSoftDeleteRecord(tweetId)
|
||||||
|
} yield {
|
||||||
|
response
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private[storage] def mkSuccessfulUndeleteResponse(
|
||||||
|
tweetId: TweetId,
|
||||||
|
records: Seq[TweetManhattanRecord],
|
||||||
|
timestampOpt: Option[Long] = None
|
||||||
|
) =
|
||||||
|
Undelete.Response(
|
||||||
|
Undelete.UndeleteResponseCode.Success,
|
||||||
|
Some(
|
||||||
|
StorageConversions.fromStoredTweet(buildStoredTweet(tweetId, records))
|
||||||
|
),
|
||||||
|
archivedAtMillis = timestampOpt
|
||||||
|
)
|
||||||
|
}
|
@ -0,0 +1,64 @@
|
|||||||
|
package com.twitter.tweetypie.storage
|
||||||
|
|
||||||
|
import com.twitter.finagle.stats.StatsReceiver
|
||||||
|
import com.twitter.stitch.Stitch
|
||||||
|
import com.twitter.storage.client.manhattan.kv.DeniedManhattanException
|
||||||
|
import com.twitter.storage.client.manhattan.kv.ManhattanValue
|
||||||
|
import com.twitter.tweetypie.storage.TweetUtils._
|
||||||
|
import com.twitter.tweetypie.thriftscala.Tweet
|
||||||
|
import com.twitter.util.Throw
|
||||||
|
import com.twitter.util.Time
|
||||||
|
|
||||||
|
object UpdateTweetHandler {
|
||||||
|
def apply(
|
||||||
|
insert: ManhattanOperations.Insert,
|
||||||
|
stats: StatsReceiver
|
||||||
|
): TweetStorageClient.UpdateTweet = { (tpTweet: Tweet, fields: Seq[Field]) =>
|
||||||
|
require(
|
||||||
|
fields.forall(!TweetFields.coreFieldIds.contains(_)),
|
||||||
|
"Core fields cannot be modified by calling updateTweet; use addTweet instead."
|
||||||
|
)
|
||||||
|
require(
|
||||||
|
areAllFieldsDefined(tpTweet, fields),
|
||||||
|
s"Input tweet $tpTweet does not have specified fields $fields set"
|
||||||
|
)
|
||||||
|
|
||||||
|
val now = Time.now
|
||||||
|
val storedTweet = StorageConversions.toStoredTweetForFields(tpTweet, fields.toSet)
|
||||||
|
val tweetId = storedTweet.id
|
||||||
|
Stats.updatePerFieldQpsCounters("updateTweet", fields.map(_.id), 1, stats)
|
||||||
|
|
||||||
|
val (fieldIds, stitchesPerTweet) =
|
||||||
|
fields.map { field =>
|
||||||
|
val fieldId = field.id
|
||||||
|
val tweetKey = TweetKey.fieldKey(tweetId, fieldId)
|
||||||
|
val blob = storedTweet.getFieldBlob(fieldId).get
|
||||||
|
val value = ManhattanValue(TFieldBlobCodec.toByteBuffer(blob), Some(now))
|
||||||
|
val record = TweetManhattanRecord(tweetKey, value)
|
||||||
|
|
||||||
|
(fieldId, insert(record).liftToTry)
|
||||||
|
}.unzip
|
||||||
|
|
||||||
|
Stitch.collect(stitchesPerTweet).map { seqOfTries =>
|
||||||
|
val fieldkeyAndMhResults = fieldIds.zip(seqOfTries).toMap
|
||||||
|
// If even a single field was rate limited, we will send an overall OverCapacity TweetResponse
|
||||||
|
val wasRateLimited = fieldkeyAndMhResults.exists { keyAndResult =>
|
||||||
|
keyAndResult._2 match {
|
||||||
|
case Throw(e: DeniedManhattanException) => true
|
||||||
|
case _ => false
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (wasRateLimited) {
|
||||||
|
buildTweetOverCapacityResponse("updateTweets", tweetId, fieldkeyAndMhResults)
|
||||||
|
} else {
|
||||||
|
buildTweetResponse("updateTweets", tweetId, fieldkeyAndMhResults)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private def areAllFieldsDefined(tpTweet: Tweet, fields: Seq[Field]) = {
|
||||||
|
val storedTweet = StorageConversions.toStoredTweetForFields(tpTweet, fields.toSet)
|
||||||
|
fields.map(_.id).forall(storedTweet.getFieldBlob(_).isDefined)
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,11 @@
|
|||||||
|
package com.twitter.tweetypie
|
||||||
|
|
||||||
|
import com.twitter.storage.client.manhattan.kv.ManhattanValue
|
||||||
|
import java.nio.ByteBuffer
|
||||||
|
|
||||||
|
package object storage {
|
||||||
|
type TweetId = Long
|
||||||
|
type FieldId = Short
|
||||||
|
|
||||||
|
type TweetManhattanValue = ManhattanValue[ByteBuffer]
|
||||||
|
}
|
@ -0,0 +1,20 @@
|
|||||||
|
scala_library(
|
||||||
|
sources = ["*.scala"],
|
||||||
|
compiler_option_sets = ["fatal_warnings"],
|
||||||
|
strict_deps = True,
|
||||||
|
tags = ["bazel-compatible"],
|
||||||
|
dependencies = [
|
||||||
|
"finagle/finagle-core/src/main",
|
||||||
|
"flock-client/src/main/scala",
|
||||||
|
"flock-client/src/main/thrift:thrift-scala",
|
||||||
|
"tweetypie/servo/util/src/main/scala",
|
||||||
|
"snowflake:id",
|
||||||
|
"src/thrift/com/twitter/gizmoduck:thrift-scala",
|
||||||
|
"src/thrift/com/twitter/servo:servo-exception-java",
|
||||||
|
"tweetypie/common/src/thrift/com/twitter/tweetypie:tweet-scala",
|
||||||
|
"tweetypie/server/src/main/scala/com/twitter/tweetypie",
|
||||||
|
"tweetypie/server/src/main/scala/com/twitter/tweetypie/serverutil",
|
||||||
|
"tweetypie/common/src/scala/com/twitter/tweetypie/util",
|
||||||
|
"util/util-core:scala",
|
||||||
|
],
|
||||||
|
)
|
@ -0,0 +1,532 @@
|
|||||||
|
/** Copyright 2010 Twitter, Inc. */
|
||||||
|
package com.twitter.tweetypie
|
||||||
|
package tflock
|
||||||
|
|
||||||
|
import com.twitter.finagle.stats.Counter
|
||||||
|
import com.twitter.flockdb.client._
|
||||||
|
import com.twitter.flockdb.client.thriftscala.Priority
|
||||||
|
import com.twitter.snowflake.id.SnowflakeId
|
||||||
|
import com.twitter.tweetypie.serverutil.StoredCard
|
||||||
|
import com.twitter.tweetypie.thriftscala._
|
||||||
|
import com.twitter.util.Future
|
||||||
|
import scala.collection.mutable.ListBuffer
|
||||||
|
|
||||||
|
object TFlockIndexer {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Printable names for some edge types currently defined in [[com.twitter.flockdb.client]].
|
||||||
|
* Used to defined stats counters for adding edges.
|
||||||
|
*/
|
||||||
|
val graphNames: Map[Int, String] =
|
||||||
|
Map(
|
||||||
|
CardTweetsGraph.id -> "card_tweets",
|
||||||
|
ConversationGraph.id -> "conversation",
|
||||||
|
DirectedAtUserIdGraph.id -> "directed_at_user_id",
|
||||||
|
InvitedUsersGraph.id -> "invited_users",
|
||||||
|
MediaTimelineGraph.id -> "media_timeline",
|
||||||
|
MentionsGraph.id -> "mentions",
|
||||||
|
NarrowcastSentTweetsGraph.id -> "narrowcast_sent_tweets",
|
||||||
|
NullcastedTweetsGraph.id -> "nullcasted_tweets",
|
||||||
|
QuotersGraph.id -> "quoters",
|
||||||
|
QuotesGraph.id -> "quotes",
|
||||||
|
QuoteTweetsIndexGraph.id -> "quote_tweets_index",
|
||||||
|
RepliesToTweetsGraph.id -> "replies_to_tweets",
|
||||||
|
RetweetsByMeGraph.id -> "retweets_by_me",
|
||||||
|
RetweetsGraph.id -> "retweets",
|
||||||
|
RetweetsOfMeGraph.id -> "retweets_of_me",
|
||||||
|
RetweetSourceGraph.id -> "retweet_source",
|
||||||
|
TweetsRetweetedGraph.id -> "tweets_retweeted",
|
||||||
|
UserTimelineGraph.id -> "user_timeline",
|
||||||
|
CreatorSubscriptionTimelineGraph.id -> "creator_subscription_timeline",
|
||||||
|
CreatorSubscriptionMediaTimelineGraph.id -> "creator_subscription_image_timeline",
|
||||||
|
)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* On edge deletion, edges are either archived permanently or retained for 3 months, based on
|
||||||
|
* the retention policy in the above confluence page.
|
||||||
|
*
|
||||||
|
* These two retention policies correspond to the two deletion techniques: archive and remove.
|
||||||
|
* We call removeEdges for edges with a short retention policy and archiveEdges for edges with
|
||||||
|
* a permanent retention policy.
|
||||||
|
*/
|
||||||
|
val graphsWithRemovedEdges: Seq[Int] =
|
||||||
|
Seq(
|
||||||
|
CardTweetsGraph.id,
|
||||||
|
CuratedTimelineGraph.id,
|
||||||
|
CuratedTweetsGraph.id,
|
||||||
|
DirectedAtUserIdGraph.id,
|
||||||
|
MediaTimelineGraph.id,
|
||||||
|
MutedConversationsGraph.id,
|
||||||
|
QuotersGraph.id,
|
||||||
|
QuotesGraph.id,
|
||||||
|
QuoteTweetsIndexGraph.id,
|
||||||
|
ReportedTweetsGraph.id,
|
||||||
|
RetweetsOfMeGraph.id,
|
||||||
|
RetweetSourceGraph.id,
|
||||||
|
SoftLikesGraph.id,
|
||||||
|
TweetsRetweetedGraph.id,
|
||||||
|
CreatorSubscriptionTimelineGraph.id,
|
||||||
|
CreatorSubscriptionMediaTimelineGraph.id,
|
||||||
|
)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* These edges should be left in place when bounced tweets are deleted.
|
||||||
|
* These edges are removed during hard deletion.
|
||||||
|
*
|
||||||
|
* This is done so external teams (timelines) can execute on these edges for
|
||||||
|
* tombstone feature.
|
||||||
|
*/
|
||||||
|
val bounceDeleteGraphIds: Set[Int] =
|
||||||
|
Set(
|
||||||
|
UserTimelineGraph.id,
|
||||||
|
ConversationGraph.id
|
||||||
|
)
|
||||||
|
|
||||||
|
def makeCounters(stats: StatsReceiver, operation: String): Map[Int, Counter] = {
|
||||||
|
TFlockIndexer.graphNames
|
||||||
|
.mapValues(stats.scope(_).counter(operation))
|
||||||
|
.withDefaultValue(stats.scope("unknown").counter(operation))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param backgroundIndexingPriority specifies the queue to use for
|
||||||
|
* background indexing operations. This is useful for making the
|
||||||
|
* effects of background indexing operations (such as deleting edges
|
||||||
|
* for deleted Tweets) available sooner in testing scenarios
|
||||||
|
* (end-to-end tests or development instances). It is set to
|
||||||
|
* Priority.Low in production to reduce the load on high priority
|
||||||
|
* queues that we use for prominently user-visible operations.
|
||||||
|
*/
|
||||||
|
class TFlockIndexer(
|
||||||
|
tflock: TFlockClient,
|
||||||
|
hasMedia: Tweet => Boolean,
|
||||||
|
backgroundIndexingPriority: Priority,
|
||||||
|
stats: StatsReceiver)
|
||||||
|
extends TweetIndexer {
|
||||||
|
private[this] val FutureNil = Future.Nil
|
||||||
|
|
||||||
|
private[this] val archiveCounters = TFlockIndexer.makeCounters(stats, "archive")
|
||||||
|
private[this] val removeCounters = TFlockIndexer.makeCounters(stats, "remove")
|
||||||
|
private[this] val insertCounters = TFlockIndexer.makeCounters(stats, "insert")
|
||||||
|
private[this] val negateCounters = TFlockIndexer.makeCounters(stats, "negate")
|
||||||
|
|
||||||
|
private[this] val foregroundIndexingPriority: Priority = Priority.High
|
||||||
|
|
||||||
|
override def createIndex(tweet: Tweet): Future[Unit] =
|
||||||
|
createEdges(tweet, isUndelete = false)
|
||||||
|
|
||||||
|
override def undeleteIndex(tweet: Tweet): Future[Unit] =
|
||||||
|
createEdges(tweet, isUndelete = true)
|
||||||
|
|
||||||
|
private[this] case class PartitionedEdges(
|
||||||
|
longRetention: Seq[ExecuteEdge[StatusGraph]] = Nil,
|
||||||
|
shortRetention: Seq[ExecuteEdge[StatusGraph]] = Nil,
|
||||||
|
negate: Seq[ExecuteEdge[StatusGraph]] = Nil,
|
||||||
|
ignore: Seq[ExecuteEdge[StatusGraph]] = Nil)
|
||||||
|
|
||||||
|
private[this] def partitionEdgesForDelete(
|
||||||
|
edges: Seq[ExecuteEdge[StatusGraph]],
|
||||||
|
isBounceDelete: Boolean
|
||||||
|
) =
|
||||||
|
edges.foldLeft(PartitionedEdges()) {
|
||||||
|
// Two dependees of UserTimelineGraph edge states to satisfy: timelines & safety tools.
|
||||||
|
// Timelines show bounce-deleted tweets as tombstones; regular deletes are not shown.
|
||||||
|
// - i.e. timelineIds = UserTimelineGraph(Normal || Negative)
|
||||||
|
// Safety tools show deleted tweets to authorized internal review agents
|
||||||
|
// - i.e. deletedIds = UserTimelineGraph(Removed || Negative)
|
||||||
|
case (partitionedEdges, edge) if isBounceDelete && edge.graphId == UserTimelineGraph.id =>
|
||||||
|
partitionedEdges.copy(negate = edge +: partitionedEdges.negate)
|
||||||
|
|
||||||
|
case (partitionedEdges, edge) if isBounceDelete && edge.graphId == ConversationGraph.id =>
|
||||||
|
// Bounce-deleted tweets remain rendered as tombstones in conversations, so do not modify
|
||||||
|
// the ConversationGraph edge state
|
||||||
|
partitionedEdges.copy(ignore = edge +: partitionedEdges.ignore)
|
||||||
|
|
||||||
|
case (partitionedEdges, edge)
|
||||||
|
if TFlockIndexer.graphsWithRemovedEdges.contains(edge.graphId) =>
|
||||||
|
partitionedEdges.copy(shortRetention = edge +: partitionedEdges.shortRetention)
|
||||||
|
|
||||||
|
case (partitionedEdges, edge) =>
|
||||||
|
partitionedEdges.copy(longRetention = edge +: partitionedEdges.longRetention)
|
||||||
|
}
|
||||||
|
|
||||||
|
override def deleteIndex(tweet: Tweet, isBounceDelete: Boolean): Future[Unit] =
|
||||||
|
for {
|
||||||
|
edges <- getEdges(tweet, isCreate = false, isDelete = true, isUndelete = false)
|
||||||
|
partitionedEdges = partitionEdgesForDelete(edges, isBounceDelete)
|
||||||
|
() <-
|
||||||
|
Future
|
||||||
|
.join(
|
||||||
|
tflock
|
||||||
|
.archiveEdges(partitionedEdges.longRetention, backgroundIndexingPriority)
|
||||||
|
.onSuccess(_ =>
|
||||||
|
partitionedEdges.longRetention.foreach(e => archiveCounters(e.graphId).incr())),
|
||||||
|
tflock
|
||||||
|
.removeEdges(partitionedEdges.shortRetention, backgroundIndexingPriority)
|
||||||
|
.onSuccess(_ =>
|
||||||
|
partitionedEdges.shortRetention.foreach(e => removeCounters(e.graphId).incr())),
|
||||||
|
tflock
|
||||||
|
.negateEdges(partitionedEdges.negate, backgroundIndexingPriority)
|
||||||
|
.onSuccess(_ =>
|
||||||
|
partitionedEdges.negate.foreach(e => negateCounters(e.graphId).incr()))
|
||||||
|
)
|
||||||
|
.unit
|
||||||
|
} yield ()
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This operation is called when a user is put into or taken out of
|
||||||
|
* a state in which their retweets should no longer be visible
|
||||||
|
* (e.g. suspended or ROPO).
|
||||||
|
*/
|
||||||
|
override def setRetweetVisibility(retweetId: TweetId, setVisible: Boolean): Future[Unit] = {
|
||||||
|
val retweetEdge = Seq(ExecuteEdge(retweetId, RetweetsGraph, None, Reverse))
|
||||||
|
|
||||||
|
if (setVisible) {
|
||||||
|
tflock
|
||||||
|
.insertEdges(retweetEdge, backgroundIndexingPriority)
|
||||||
|
.onSuccess(_ => insertCounters(RetweetsGraph.id).incr())
|
||||||
|
} else {
|
||||||
|
tflock
|
||||||
|
.archiveEdges(retweetEdge, backgroundIndexingPriority)
|
||||||
|
.onSuccess(_ => archiveCounters(RetweetsGraph.id).incr())
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private[this] def createEdges(tweet: Tweet, isUndelete: Boolean): Future[Unit] =
|
||||||
|
for {
|
||||||
|
edges <- getEdges(tweet = tweet, isCreate = true, isDelete = false, isUndelete = isUndelete)
|
||||||
|
() <- tflock.insertEdges(edges, foregroundIndexingPriority)
|
||||||
|
} yield {
|
||||||
|
// Count all the edges we've successfully added:
|
||||||
|
edges.foreach(e => insertCounters(e.graphId).incr())
|
||||||
|
}
|
||||||
|
|
||||||
|
private[this] def addRTEdges(
|
||||||
|
tweet: Tweet,
|
||||||
|
share: Share,
|
||||||
|
isCreate: Boolean,
|
||||||
|
edges: ListBuffer[ExecuteEdge[StatusGraph]],
|
||||||
|
futureEdges: ListBuffer[Future[Seq[ExecuteEdge[StatusGraph]]]]
|
||||||
|
): Unit = {
|
||||||
|
|
||||||
|
edges += RetweetsOfMeGraph.edge(share.sourceUserId, tweet.id)
|
||||||
|
edges += RetweetsByMeGraph.edge(getUserId(tweet), tweet.id)
|
||||||
|
edges += RetweetsGraph.edge(share.sourceStatusId, tweet.id)
|
||||||
|
|
||||||
|
if (isCreate) {
|
||||||
|
edges += ExecuteEdge(
|
||||||
|
sourceId = getUserId(tweet),
|
||||||
|
graph = RetweetSourceGraph,
|
||||||
|
destinationIds = Some(Seq(share.sourceStatusId)),
|
||||||
|
direction = Forward,
|
||||||
|
position = Some(SnowflakeId(tweet.id).time.inMillis)
|
||||||
|
)
|
||||||
|
edges.append(TweetsRetweetedGraph.edge(share.sourceUserId, share.sourceStatusId))
|
||||||
|
} else {
|
||||||
|
edges += RetweetSourceGraph.edge(getUserId(tweet), share.sourceStatusId)
|
||||||
|
|
||||||
|
// if this is the last retweet we need to remove it from the source user's
|
||||||
|
// tweets retweeted graph
|
||||||
|
futureEdges.append(
|
||||||
|
tflock.count(RetweetsGraph.from(share.sourceStatusId)).flatMap { count =>
|
||||||
|
if (count <= 1) {
|
||||||
|
tflock.selectAll(RetweetsGraph.from(share.sourceStatusId)).map { tweets =>
|
||||||
|
if (tweets.size <= 1)
|
||||||
|
Seq(TweetsRetweetedGraph.edge(share.sourceUserId, share.sourceStatusId))
|
||||||
|
else
|
||||||
|
Nil
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
FutureNil
|
||||||
|
}
|
||||||
|
}
|
||||||
|
)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private[this] def addReplyEdges(
|
||||||
|
tweet: Tweet,
|
||||||
|
edges: ListBuffer[ExecuteEdge[StatusGraph]]
|
||||||
|
): Unit = {
|
||||||
|
getReply(tweet).foreach { reply =>
|
||||||
|
reply.inReplyToStatusId.flatMap { inReplyToStatusId =>
|
||||||
|
edges += RepliesToTweetsGraph.edge(inReplyToStatusId, tweet.id)
|
||||||
|
|
||||||
|
// only index conversationId if this is a reply to another tweet
|
||||||
|
TweetLenses.conversationId.get(tweet).map { conversationId =>
|
||||||
|
edges += ConversationGraph.edge(conversationId, tweet.id)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private[this] def addDirectedAtEdges(
|
||||||
|
tweet: Tweet,
|
||||||
|
edges: ListBuffer[ExecuteEdge[StatusGraph]]
|
||||||
|
): Unit = {
|
||||||
|
TweetLenses.directedAtUser.get(tweet).foreach { directedAtUser =>
|
||||||
|
edges += DirectedAtUserIdGraph.edge(directedAtUser.userId, tweet.id)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private[this] def addMentionEdges(
|
||||||
|
tweet: Tweet,
|
||||||
|
edges: ListBuffer[ExecuteEdge[StatusGraph]]
|
||||||
|
): Unit = {
|
||||||
|
getMentions(tweet)
|
||||||
|
.flatMap(_.userId).foreach { mention =>
|
||||||
|
edges += MentionsGraph.edge(mention, tweet.id)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private[this] def addQTEdges(
|
||||||
|
tweet: Tweet,
|
||||||
|
edges: ListBuffer[ExecuteEdge[StatusGraph]],
|
||||||
|
futureEdges: ListBuffer[Future[Seq[ExecuteEdge[StatusGraph]]]],
|
||||||
|
isCreate: Boolean
|
||||||
|
): Unit = {
|
||||||
|
val userId = getUserId(tweet)
|
||||||
|
|
||||||
|
tweet.quotedTweet.foreach { quotedTweet =>
|
||||||
|
// Regardless of tweet creates/deletes, we add the corresponding edges to the
|
||||||
|
// following two graphs. Note that we're handling the case for
|
||||||
|
// the QuotersGraph slightly differently in the tweet delete case.
|
||||||
|
edges.append(QuotesGraph.edge(quotedTweet.userId, tweet.id))
|
||||||
|
edges.append(QuoteTweetsIndexGraph.edge(quotedTweet.tweetId, tweet.id))
|
||||||
|
if (isCreate) {
|
||||||
|
// As mentioned above, for tweet creates we go ahead and add an edge
|
||||||
|
// to the QuotersGraph without any additional checks.
|
||||||
|
edges.append(QuotersGraph.edge(quotedTweet.tweetId, userId))
|
||||||
|
} else {
|
||||||
|
// For tweet deletes, we only add an edge to be deleted from the
|
||||||
|
// QuotersGraph if the tweeting user isn't quoting the tweet anymore
|
||||||
|
// i.e. if a user has quoted a tweet multiple times, we only delete
|
||||||
|
// an edge from the QuotersGraph if they've deleted all the quotes,
|
||||||
|
// otherwise an edge should exist by definition of what the QuotersGraph
|
||||||
|
// represents.
|
||||||
|
|
||||||
|
// Note: There can be a potential edge case here due to a race condition
|
||||||
|
// in the following scenario.
|
||||||
|
// i) A quotes a tweet T twice resulting in tweets T1 and T2.
|
||||||
|
// ii) There should exist edges in the QuotersGraph from T -> A and T1 <-> T, T2 <-> T in
|
||||||
|
// the QuoteTweetsIndexGraph, but one of the edges haven't been written
|
||||||
|
// to the QuoteTweetsIndex graph in TFlock yet.
|
||||||
|
// iii) In this scenario, we shouldn't really be deleting an edge as we're doing below.
|
||||||
|
// The approach that we're taking below is a "best effort" approach similar to what we
|
||||||
|
// currently do for RTs.
|
||||||
|
|
||||||
|
// Find all the quotes of the quoted tweet from the quoting user
|
||||||
|
val quotesFromQuotingUser = QuoteTweetsIndexGraph
|
||||||
|
.from(quotedTweet.tweetId)
|
||||||
|
.intersect(UserTimelineGraph.from(userId))
|
||||||
|
futureEdges.append(
|
||||||
|
tflock
|
||||||
|
.count(quotesFromQuotingUser).flatMap { count =>
|
||||||
|
// If this is the last quote of the quoted tweet from the quoting user,
|
||||||
|
// we go ahead and delete the edge from the QuotersGraph.
|
||||||
|
if (count <= 1) {
|
||||||
|
tflock.selectAll(quotesFromQuotingUser).map { tweets =>
|
||||||
|
if (tweets.size <= 1) {
|
||||||
|
Seq(QuotersGraph.edge(quotedTweet.tweetId, userId))
|
||||||
|
} else {
|
||||||
|
Nil
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
FutureNil
|
||||||
|
}
|
||||||
|
}
|
||||||
|
)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private[this] def addCardEdges(
|
||||||
|
tweet: Tweet,
|
||||||
|
edges: ListBuffer[ExecuteEdge[StatusGraph]]
|
||||||
|
): Unit = {
|
||||||
|
// Note that we are indexing only the TOO "stored" cards
|
||||||
|
// (cardUri=card://<cardId>). Rest of the cards are ignored here.
|
||||||
|
tweet.cardReference
|
||||||
|
.collect {
|
||||||
|
case StoredCard(id) =>
|
||||||
|
edges.append(CardTweetsGraph.edge(id, tweet.id))
|
||||||
|
}.getOrElse(())
|
||||||
|
}
|
||||||
|
|
||||||
|
// Note: on undelete, this method restores all archived edges, including those that may have
|
||||||
|
// been archived prior to the delete. This is incorrect behavior but in practice rarely
|
||||||
|
// causes problems, as undeletes are so rare.
|
||||||
|
private[this] def addEdgesForDeleteOrUndelete(
|
||||||
|
tweet: Tweet,
|
||||||
|
edges: ListBuffer[ExecuteEdge[StatusGraph]]
|
||||||
|
): Unit = {
|
||||||
|
edges.appendAll(
|
||||||
|
Seq(
|
||||||
|
MentionsGraph.edges(tweet.id, None, Reverse),
|
||||||
|
RepliesToTweetsGraph.edges(tweet.id, None)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
|
||||||
|
// When we delete or undelete a conversation control root Tweet we want to archive or restore
|
||||||
|
// all the edges in InvitedUsersGraph from the Tweet id.
|
||||||
|
if (hasConversationControl(tweet) && isConversationRoot(tweet)) {
|
||||||
|
edges.append(InvitedUsersGraph.edges(tweet.id, None))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private[this] def addSimpleEdges(
|
||||||
|
tweet: Tweet,
|
||||||
|
edges: ListBuffer[ExecuteEdge[StatusGraph]]
|
||||||
|
): Unit = {
|
||||||
|
if (TweetLenses.nullcast.get(tweet)) {
|
||||||
|
edges.append(NullcastedTweetsGraph.edge(getUserId(tweet), tweet.id))
|
||||||
|
} else if (TweetLenses.narrowcast.get(tweet).isDefined) {
|
||||||
|
edges.append(NarrowcastSentTweetsGraph.edge(getUserId(tweet), tweet.id))
|
||||||
|
} else {
|
||||||
|
edges.append(UserTimelineGraph.edge(getUserId(tweet), tweet.id))
|
||||||
|
|
||||||
|
if (hasMedia(tweet))
|
||||||
|
edges.append(MediaTimelineGraph.edge(getUserId(tweet), tweet.id))
|
||||||
|
|
||||||
|
// Index root creator subscription tweets.
|
||||||
|
// Ignore replies because those are not necessarily visible to a user who subscribes to tweet author
|
||||||
|
val isRootTweet: Boolean = tweet.coreData match {
|
||||||
|
case Some(c) => c.reply.isEmpty && c.share.isEmpty
|
||||||
|
case None => true
|
||||||
|
}
|
||||||
|
|
||||||
|
if (tweet.exclusiveTweetControl.isDefined && isRootTweet) {
|
||||||
|
edges.append(CreatorSubscriptionTimelineGraph.edge(getUserId(tweet), tweet.id))
|
||||||
|
|
||||||
|
if (hasMedia(tweet))
|
||||||
|
edges.append(CreatorSubscriptionMediaTimelineGraph.edge(getUserId(tweet), tweet.id))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Issues edges for each mention of user in a conversation-controlled tweet. This way InvitedUsers
|
||||||
|
* graph accumulates complete set of ids for @mention-invited users, by conversation id.
|
||||||
|
*/
|
||||||
|
private def invitedUsersEdgesForCreate(
|
||||||
|
tweet: Tweet,
|
||||||
|
edges: ListBuffer[ExecuteEdge[StatusGraph]]
|
||||||
|
): Unit = {
|
||||||
|
val conversationId: Long = getConversationId(tweet).getOrElse(tweet.id)
|
||||||
|
val mentions: Seq[UserId] = getMentions(tweet).flatMap(_.userId)
|
||||||
|
edges.appendAll(mentions.map(userId => InvitedUsersGraph.edge(conversationId, userId)))
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Issues edges of InviteUsersGraph that ought to be deleted for a conversation controlled reply.
|
||||||
|
* These are mentions of users in the given tweet, only if the user was not mentioned elsewhere
|
||||||
|
* in the conversation. This way for a conversation, InvitedUsersGraph would always hold a set
|
||||||
|
* of all users invited to the conversation, and an edge is removed only after the last mention of
|
||||||
|
* a user is deleted.
|
||||||
|
*/
|
||||||
|
private def invitedUsersEdgesForDelete(
|
||||||
|
tweet: Tweet,
|
||||||
|
futureEdges: ListBuffer[Future[Seq[ExecuteEdge[StatusGraph]]]]
|
||||||
|
): Unit = {
|
||||||
|
getConversationId(tweet).foreach { conversationId: Long =>
|
||||||
|
val mentions: Seq[UserId] = getMentions(tweet).flatMap(_.userId)
|
||||||
|
mentions.foreach { userId =>
|
||||||
|
val tweetIdsWithinConversation = ConversationGraph.from(conversationId)
|
||||||
|
val tweetIdsThatMentionUser = MentionsGraph.from(userId)
|
||||||
|
futureEdges.append(
|
||||||
|
tflock
|
||||||
|
.selectAll(
|
||||||
|
query = tweetIdsThatMentionUser.intersect(tweetIdsWithinConversation),
|
||||||
|
limit = Some(2), // Just need to know if it is >1 or <=1, so 2 are enough.
|
||||||
|
pageSize = None // Provide default, otherwise Mockito complains
|
||||||
|
).map { tweetIds: Seq[Long] =>
|
||||||
|
if (tweetIds.size <= 1) {
|
||||||
|
Seq(InvitedUsersGraph.edge(conversationId, userId))
|
||||||
|
} else {
|
||||||
|
Nil
|
||||||
|
}
|
||||||
|
}
|
||||||
|
)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private def hasInviteViaMention(tweet: Tweet): Boolean = {
|
||||||
|
tweet.conversationControl match {
|
||||||
|
case Some(ConversationControl.ByInvitation(controls)) =>
|
||||||
|
controls.inviteViaMention.getOrElse(false)
|
||||||
|
case Some(ConversationControl.Community(controls)) =>
|
||||||
|
controls.inviteViaMention.getOrElse(false)
|
||||||
|
case Some(ConversationControl.Followers(followers)) =>
|
||||||
|
followers.inviteViaMention.getOrElse(false)
|
||||||
|
case _ =>
|
||||||
|
false
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private def hasConversationControl(tweet: Tweet): Boolean =
|
||||||
|
tweet.conversationControl.isDefined
|
||||||
|
|
||||||
|
// If a Tweet has a ConversationControl, it must have a ConversationId associated with it so we
|
||||||
|
// can compare the ConversationId with the current Tweet ID to determine if it's the root of the
|
||||||
|
// conversation. See ConversationIdHydrator for more details
|
||||||
|
private def isConversationRoot(tweet: Tweet): Boolean =
|
||||||
|
getConversationId(tweet).get == tweet.id
|
||||||
|
|
||||||
|
private def addInvitedUsersEdges(
|
||||||
|
tweet: Tweet,
|
||||||
|
isCreate: Boolean,
|
||||||
|
isUndelete: Boolean,
|
||||||
|
edges: ListBuffer[ExecuteEdge[StatusGraph]],
|
||||||
|
futureEdges: ListBuffer[Future[Seq[ExecuteEdge[StatusGraph]]]]
|
||||||
|
): Unit = {
|
||||||
|
if (hasConversationControl(tweet)) {
|
||||||
|
if (isCreate) {
|
||||||
|
if (isConversationRoot(tweet) && !isUndelete) {
|
||||||
|
// For root Tweets, only add edges for original creates, not for undeletes.
|
||||||
|
// Undeletes are handled by addEdgesForDeleteOrUndelete.
|
||||||
|
invitedUsersEdgesForCreate(tweet, edges)
|
||||||
|
}
|
||||||
|
if (!isConversationRoot(tweet) && hasInviteViaMention(tweet)) {
|
||||||
|
// For replies, only add edges when the conversation control is in inviteViaMention mode.
|
||||||
|
invitedUsersEdgesForCreate(tweet, edges)
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
if (!isConversationRoot(tweet)) {
|
||||||
|
invitedUsersEdgesForDelete(tweet, futureEdges)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private[this] def getEdges(
|
||||||
|
tweet: Tweet,
|
||||||
|
isCreate: Boolean,
|
||||||
|
isDelete: Boolean,
|
||||||
|
isUndelete: Boolean
|
||||||
|
): Future[Seq[ExecuteEdge[StatusGraph]]] = {
|
||||||
|
val edges = ListBuffer[ExecuteEdge[StatusGraph]]()
|
||||||
|
val futureEdges = ListBuffer[Future[Seq[ExecuteEdge[StatusGraph]]]]()
|
||||||
|
|
||||||
|
addSimpleEdges(tweet, edges)
|
||||||
|
getShare(tweet) match {
|
||||||
|
case Some(share) => addRTEdges(tweet, share, isCreate, edges, futureEdges)
|
||||||
|
case _ =>
|
||||||
|
addInvitedUsersEdges(tweet, isCreate, isUndelete, edges, futureEdges)
|
||||||
|
addReplyEdges(tweet, edges)
|
||||||
|
addDirectedAtEdges(tweet, edges)
|
||||||
|
addMentionEdges(tweet, edges)
|
||||||
|
addQTEdges(tweet, edges, futureEdges, isCreate)
|
||||||
|
addCardEdges(tweet, edges)
|
||||||
|
if (isDelete || isUndelete) {
|
||||||
|
addEdgesForDeleteOrUndelete(tweet, edges)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
Future
|
||||||
|
.collect(futureEdges)
|
||||||
|
.map { moreEdges => (edges ++= moreEdges.flatten).toList }
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,30 @@
|
|||||||
|
/** Copyright 2010 Twitter, Inc. */
|
||||||
|
package com.twitter.tweetypie
|
||||||
|
package tflock
|
||||||
|
|
||||||
|
import com.twitter.tweetypie.thriftscala.Tweet
|
||||||
|
import com.twitter.util.Future
|
||||||
|
|
||||||
|
trait TweetIndexer {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Called at tweet-creation time, this method should set up all relevant indices on the tweet.
|
||||||
|
*/
|
||||||
|
def createIndex(tweet: Tweet): Future[Unit] = Future.Unit
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Called at tweet-undelete time (which isn't yet handled), this method should
|
||||||
|
* restore all relevant indices on the tweet.
|
||||||
|
*/
|
||||||
|
def undeleteIndex(tweet: Tweet): Future[Unit] = Future.Unit
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Called at tweet-delete time, this method should archive all relevant indices on the tweet.
|
||||||
|
*/
|
||||||
|
def deleteIndex(tweet: Tweet, isBounceDelete: Boolean): Future[Unit] = Future.Unit
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This method should archive or unarchive the retweet edge in TFlock RetweetsGraph.
|
||||||
|
*/
|
||||||
|
def setRetweetVisibility(retweetId: TweetId, visible: Boolean): Future[Unit] = Future.Unit
|
||||||
|
}
|
@ -0,0 +1,13 @@
|
|||||||
|
scala_library(
|
||||||
|
sources = ["*.scala"],
|
||||||
|
compiler_option_sets = ["fatal_warnings"],
|
||||||
|
platform = "java8",
|
||||||
|
strict_deps = True,
|
||||||
|
tags = ["bazel-compatible"],
|
||||||
|
dependencies = [
|
||||||
|
"finagle/finagle-core/src/main",
|
||||||
|
"scrooge/scrooge-core/src/main/scala",
|
||||||
|
"tweetypie/common/src/thrift/com/twitter/tweetypie:service-scala",
|
||||||
|
"util/util-core:scala",
|
||||||
|
],
|
||||||
|
)
|
@ -0,0 +1,8 @@
|
|||||||
|
package com.twitter.tweetypie.thriftscala
|
||||||
|
|
||||||
|
import com.twitter.finagle.service.FailedService
|
||||||
|
|
||||||
|
class NotImplementedTweetService
|
||||||
|
extends TweetService$FinagleClient(
|
||||||
|
new FailedService(new UnsupportedOperationException("not implemented"))
|
||||||
|
)
|
@ -0,0 +1,79 @@
|
|||||||
|
package com.twitter.tweetypie.thriftscala
|
||||||
|
|
||||||
|
import com.twitter.util.Future
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A trait for TweetService implementations that wrap an underlying
|
||||||
|
* TweetService and need to modify only some of the methods.
|
||||||
|
*/
|
||||||
|
trait TweetServiceProxy extends TweetService.MethodPerEndpoint {
|
||||||
|
protected def underlying: TweetService.MethodPerEndpoint
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Default implementation simply passes through the Future but logic can be added to wrap each
|
||||||
|
* invocation to the underlying TweetService
|
||||||
|
*/
|
||||||
|
protected def wrap[A](f: => Future[A]): Future[A] =
|
||||||
|
f
|
||||||
|
|
||||||
|
override def getTweets(request: GetTweetsRequest): Future[Seq[GetTweetResult]] =
|
||||||
|
wrap(underlying.getTweets(request))
|
||||||
|
|
||||||
|
override def getTweetFields(request: GetTweetFieldsRequest): Future[Seq[GetTweetFieldsResult]] =
|
||||||
|
wrap(underlying.getTweetFields(request))
|
||||||
|
|
||||||
|
override def getTweetCounts(request: GetTweetCountsRequest): Future[Seq[GetTweetCountsResult]] =
|
||||||
|
wrap(underlying.getTweetCounts(request))
|
||||||
|
|
||||||
|
override def setAdditionalFields(request: SetAdditionalFieldsRequest): Future[Unit] =
|
||||||
|
wrap(underlying.setAdditionalFields(request))
|
||||||
|
|
||||||
|
override def deleteAdditionalFields(request: DeleteAdditionalFieldsRequest): Future[Unit] =
|
||||||
|
wrap(underlying.deleteAdditionalFields(request))
|
||||||
|
|
||||||
|
override def postTweet(request: PostTweetRequest): Future[PostTweetResult] =
|
||||||
|
wrap(underlying.postTweet(request))
|
||||||
|
|
||||||
|
override def postRetweet(request: RetweetRequest): Future[PostTweetResult] =
|
||||||
|
wrap(underlying.postRetweet(request))
|
||||||
|
|
||||||
|
override def unretweet(request: UnretweetRequest): Future[UnretweetResult] =
|
||||||
|
wrap(underlying.unretweet(request))
|
||||||
|
|
||||||
|
override def getDeletedTweets(
|
||||||
|
request: GetDeletedTweetsRequest
|
||||||
|
): Future[Seq[GetDeletedTweetResult]] =
|
||||||
|
wrap(underlying.getDeletedTweets(request))
|
||||||
|
|
||||||
|
override def deleteTweets(request: DeleteTweetsRequest): Future[Seq[DeleteTweetResult]] =
|
||||||
|
wrap(underlying.deleteTweets(request))
|
||||||
|
|
||||||
|
override def updatePossiblySensitiveTweet(
|
||||||
|
request: UpdatePossiblySensitiveTweetRequest
|
||||||
|
): Future[Unit] =
|
||||||
|
wrap(underlying.updatePossiblySensitiveTweet(request))
|
||||||
|
|
||||||
|
override def undeleteTweet(request: UndeleteTweetRequest): Future[UndeleteTweetResponse] =
|
||||||
|
wrap(underlying.undeleteTweet(request))
|
||||||
|
|
||||||
|
override def eraseUserTweets(request: EraseUserTweetsRequest): Future[Unit] =
|
||||||
|
wrap(underlying.eraseUserTweets(request))
|
||||||
|
|
||||||
|
override def incrTweetFavCount(request: IncrTweetFavCountRequest): Future[Unit] =
|
||||||
|
wrap(underlying.incrTweetFavCount(request))
|
||||||
|
|
||||||
|
override def deleteLocationData(request: DeleteLocationDataRequest): Future[Unit] =
|
||||||
|
wrap(underlying.deleteLocationData(request))
|
||||||
|
|
||||||
|
override def scrubGeo(request: GeoScrub): Future[Unit] =
|
||||||
|
wrap(underlying.scrubGeo(request))
|
||||||
|
|
||||||
|
override def takedown(request: TakedownRequest): Future[Unit] =
|
||||||
|
wrap(underlying.takedown(request))
|
||||||
|
|
||||||
|
override def flush(request: FlushRequest): Future[Unit] =
|
||||||
|
wrap(underlying.flush(request))
|
||||||
|
|
||||||
|
override def incrTweetBookmarkCount(request: IncrTweetBookmarkCountRequest): Future[Unit] =
|
||||||
|
wrap(underlying.incrTweetBookmarkCount(request))
|
||||||
|
}
|
@ -0,0 +1,15 @@
|
|||||||
|
scala_library(
|
||||||
|
sources = ["*.scala"],
|
||||||
|
compiler_option_sets = ["fatal_warnings"],
|
||||||
|
strict_deps = True,
|
||||||
|
tags = ["bazel-compatible"],
|
||||||
|
dependencies = [
|
||||||
|
"tweetypie/servo/util",
|
||||||
|
"tweetypie/common/src/thrift/com/twitter/tweetypie:media-entity-scala",
|
||||||
|
"tweetypie/common/src/thrift/com/twitter/tweetypie:tweet-scala",
|
||||||
|
"tco-util",
|
||||||
|
"tweetypie/common/src/scala/com/twitter/tweetypie/tweettext",
|
||||||
|
"tweetypie/common/src/scala/com/twitter/tweetypie/util",
|
||||||
|
"twitter-text/lib/java/src/main/java/com/twitter/twittertext",
|
||||||
|
],
|
||||||
|
)
|
@ -0,0 +1,11 @@
|
|||||||
|
package com.twitter.tweetypie.thriftscala.entities
|
||||||
|
|
||||||
|
import com.twitter.tweetypie.thriftscala.CashtagEntity
|
||||||
|
import com.twitter.tweetypie.tweettext.TextEntity
|
||||||
|
|
||||||
|
object CashtagTextEntity extends TextEntity[CashtagEntity] {
|
||||||
|
override def fromIndex(entity: CashtagEntity): Short = entity.fromIndex
|
||||||
|
override def toIndex(entity: CashtagEntity): Short = entity.toIndex
|
||||||
|
override def move(entity: CashtagEntity, fromIndex: Short, toIndex: Short): CashtagEntity =
|
||||||
|
entity.copy(fromIndex = fromIndex, toIndex = toIndex)
|
||||||
|
}
|
@ -0,0 +1,118 @@
|
|||||||
|
package com.twitter.tweetypie.thriftscala.entities
|
||||||
|
|
||||||
|
import com.twitter.servo.data.Mutation
|
||||||
|
import com.twitter.tco_util.TcoUrl
|
||||||
|
import com.twitter.tweetypie.thriftscala._
|
||||||
|
import com.twitter.tweetypie.thriftscala.entities.Implicits._
|
||||||
|
import com.twitter.tweetypie.tweettext.PartialHtmlEncoding
|
||||||
|
import com.twitter.tweetypie.tweettext.TextEntity
|
||||||
|
import com.twitter.tweetypie.tweettext.TextModification
|
||||||
|
import com.twitter.tweetypie.util.TweetLenses
|
||||||
|
import com.twitter.twittertext.Extractor
|
||||||
|
import scala.collection.JavaConverters._
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Contains functions to collect urls, mentions, hashtags, and cashtags from the text of tweets and messages
|
||||||
|
*/
|
||||||
|
object EntityExtractor {
|
||||||
|
// We only use one configuration of com.twitter.twittertext.Extractor, so it's
|
||||||
|
// OK to share one global reference. The only available
|
||||||
|
// configuration option is whether to extract URLs without protocols
|
||||||
|
// (defaults to true)
|
||||||
|
private[this] val extractor = new Extractor
|
||||||
|
|
||||||
|
// The twitter-text library operates on unencoded text, but we store
|
||||||
|
// and process HTML-encoded text. The TextModification returned
|
||||||
|
// from this function contains the decoded text which we will operate on,
|
||||||
|
// but also provides us with the ability to map the indices on
|
||||||
|
// the twitter-text entities back to the entities on the encoded text.
|
||||||
|
private val htmlEncodedTextToEncodeModification: String => TextModification =
|
||||||
|
text =>
|
||||||
|
PartialHtmlEncoding
|
||||||
|
.decodeWithModification(text)
|
||||||
|
.getOrElse(TextModification.identity(text))
|
||||||
|
.inverse
|
||||||
|
|
||||||
|
private[this] val extractAllUrlsFromTextMod: TextModification => Seq[UrlEntity] =
|
||||||
|
extractUrls(false)
|
||||||
|
|
||||||
|
val extractAllUrls: String => Seq[UrlEntity] =
|
||||||
|
htmlEncodedTextToEncodeModification.andThen(extractAllUrlsFromTextMod)
|
||||||
|
|
||||||
|
private[this] val extractTcoUrls: TextModification => Seq[UrlEntity] =
|
||||||
|
extractUrls(true)
|
||||||
|
|
||||||
|
private[this] def extractUrls(tcoOnly: Boolean): TextModification => Seq[UrlEntity] =
|
||||||
|
mkEntityExtractor[UrlEntity](
|
||||||
|
extractor.extractURLsWithIndices(_).asScala.filter { e =>
|
||||||
|
if (tcoOnly) TcoUrl.isTcoUrl(e.getValue) else true
|
||||||
|
},
|
||||||
|
UrlEntity(_, _, _)
|
||||||
|
)
|
||||||
|
|
||||||
|
private[this] val extractMentionsFromTextMod: TextModification => Seq[MentionEntity] =
|
||||||
|
mkEntityExtractor[MentionEntity](
|
||||||
|
extractor.extractMentionedScreennamesWithIndices(_).asScala,
|
||||||
|
MentionEntity(_, _, _)
|
||||||
|
)
|
||||||
|
|
||||||
|
val extractMentions: String => Seq[MentionEntity] =
|
||||||
|
htmlEncodedTextToEncodeModification.andThen(extractMentionsFromTextMod)
|
||||||
|
|
||||||
|
private[this] val extractHashtagsFromTextMod: TextModification => Seq[HashtagEntity] =
|
||||||
|
mkEntityExtractor[HashtagEntity](
|
||||||
|
extractor.extractHashtagsWithIndices(_).asScala,
|
||||||
|
HashtagEntity(_, _, _)
|
||||||
|
)
|
||||||
|
|
||||||
|
val extractHashtags: String => Seq[HashtagEntity] =
|
||||||
|
htmlEncodedTextToEncodeModification.andThen(extractHashtagsFromTextMod)
|
||||||
|
|
||||||
|
private[this] val extractCashtagsFromTextMod: TextModification => Seq[CashtagEntity] =
|
||||||
|
mkEntityExtractor[CashtagEntity](
|
||||||
|
extractor.extractCashtagsWithIndices(_).asScala,
|
||||||
|
CashtagEntity(_, _, _)
|
||||||
|
)
|
||||||
|
|
||||||
|
val extractCashtags: String => Seq[CashtagEntity] =
|
||||||
|
htmlEncodedTextToEncodeModification.andThen(extractCashtagsFromTextMod)
|
||||||
|
|
||||||
|
private[this] def mkEntityExtractor[E: TextEntity](
|
||||||
|
extract: String => Seq[Extractor.Entity],
|
||||||
|
construct: (Short, Short, String) => E
|
||||||
|
): TextModification => Seq[E] =
|
||||||
|
htmlEncodedMod => {
|
||||||
|
val convert: Extractor.Entity => Option[E] =
|
||||||
|
e =>
|
||||||
|
for {
|
||||||
|
start <- asShort(e.getStart.intValue)
|
||||||
|
end <- asShort(e.getEnd.intValue)
|
||||||
|
if e.getValue != null
|
||||||
|
res <- htmlEncodedMod.reindexEntity(construct(start, end, e.getValue))
|
||||||
|
} yield res
|
||||||
|
|
||||||
|
val entities = extract(htmlEncodedMod.original)
|
||||||
|
extractor.modifyIndicesFromUTF16ToUnicode(htmlEncodedMod.original, entities.asJava)
|
||||||
|
entities.map(convert).flatten
|
||||||
|
}
|
||||||
|
|
||||||
|
private[this] def asShort(i: Int): Option[Short] =
|
||||||
|
if (i.isValidShort) Some(i.toShort) else None
|
||||||
|
|
||||||
|
private[this] def mutation(extractUrls: Boolean): Mutation[Tweet] =
|
||||||
|
Mutation { tweet =>
|
||||||
|
val htmlEncodedMod = htmlEncodedTextToEncodeModification(TweetLenses.text.get(tweet))
|
||||||
|
|
||||||
|
Some(
|
||||||
|
tweet.copy(
|
||||||
|
urls = if (extractUrls) Some(extractTcoUrls(htmlEncodedMod)) else tweet.urls,
|
||||||
|
mentions = Some(extractMentionsFromTextMod(htmlEncodedMod)),
|
||||||
|
hashtags = Some(extractHashtagsFromTextMod(htmlEncodedMod)),
|
||||||
|
cashtags = Some(extractCashtagsFromTextMod(htmlEncodedMod))
|
||||||
|
)
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
|
val mutationWithoutUrls: Mutation[Tweet] = mutation(false)
|
||||||
|
val mutationAll: Mutation[Tweet] = mutation(true)
|
||||||
|
}
|
@ -0,0 +1,11 @@
|
|||||||
|
package com.twitter.tweetypie.thriftscala.entities
|
||||||
|
|
||||||
|
import com.twitter.tweetypie.thriftscala.HashtagEntity
|
||||||
|
import com.twitter.tweetypie.tweettext.TextEntity
|
||||||
|
|
||||||
|
object HashtagTextEntity extends TextEntity[HashtagEntity] {
|
||||||
|
override def fromIndex(entity: HashtagEntity): Short = entity.fromIndex
|
||||||
|
override def toIndex(entity: HashtagEntity): Short = entity.toIndex
|
||||||
|
override def move(entity: HashtagEntity, fromIndex: Short, toIndex: Short): HashtagEntity =
|
||||||
|
entity.copy(fromIndex = fromIndex, toIndex = toIndex)
|
||||||
|
}
|
@ -0,0 +1,10 @@
|
|||||||
|
package com.twitter.tweetypie.thriftscala.entities
|
||||||
|
|
||||||
|
object Implicits {
|
||||||
|
implicit val hashtagTextEntity: HashtagTextEntity.type = HashtagTextEntity
|
||||||
|
implicit val cashtagTextEntity: CashtagTextEntity.type = CashtagTextEntity
|
||||||
|
implicit val mentionTextEntity: MentionTextEntity.type = MentionTextEntity
|
||||||
|
implicit val urlTextEntity: UrlTextEntity.type = UrlTextEntity
|
||||||
|
implicit val mediaTextEntity: MediaTextEntity.type = MediaTextEntity
|
||||||
|
implicit val textRangeTextEntity: TextRangeEntityAdapter.type = TextRangeEntityAdapter
|
||||||
|
}
|
@ -0,0 +1,11 @@
|
|||||||
|
package com.twitter.tweetypie.thriftscala.entities
|
||||||
|
|
||||||
|
import com.twitter.tweetypie.thriftscala.MediaEntity
|
||||||
|
import com.twitter.tweetypie.tweettext.TextEntity
|
||||||
|
|
||||||
|
object MediaTextEntity extends TextEntity[MediaEntity] {
|
||||||
|
override def fromIndex(entity: MediaEntity): Short = entity.fromIndex
|
||||||
|
override def toIndex(entity: MediaEntity): Short = entity.toIndex
|
||||||
|
override def move(entity: MediaEntity, fromIndex: Short, toIndex: Short): MediaEntity =
|
||||||
|
entity.copy(fromIndex = fromIndex, toIndex = toIndex)
|
||||||
|
}
|
@ -0,0 +1,11 @@
|
|||||||
|
package com.twitter.tweetypie.thriftscala.entities
|
||||||
|
|
||||||
|
import com.twitter.tweetypie.thriftscala.MentionEntity
|
||||||
|
import com.twitter.tweetypie.tweettext.TextEntity
|
||||||
|
|
||||||
|
object MentionTextEntity extends TextEntity[MentionEntity] {
|
||||||
|
override def fromIndex(entity: MentionEntity): Short = entity.fromIndex
|
||||||
|
override def toIndex(entity: MentionEntity): Short = entity.toIndex
|
||||||
|
override def move(entity: MentionEntity, fromIndex: Short, toIndex: Short): MentionEntity =
|
||||||
|
entity.copy(fromIndex = fromIndex, toIndex = toIndex)
|
||||||
|
}
|
@ -0,0 +1,11 @@
|
|||||||
|
package com.twitter.tweetypie.thriftscala.entities
|
||||||
|
|
||||||
|
import com.twitter.tweetypie.thriftscala.TextRange
|
||||||
|
import com.twitter.tweetypie.tweettext.TextEntity
|
||||||
|
|
||||||
|
object TextRangeEntityAdapter extends TextEntity[TextRange] {
|
||||||
|
override def fromIndex(entity: TextRange): Short = entity.fromIndex.toShort
|
||||||
|
override def toIndex(entity: TextRange): Short = entity.toIndex.toShort
|
||||||
|
override def move(entity: TextRange, fromIndex: Short, toIndex: Short): TextRange =
|
||||||
|
entity.copy(fromIndex = fromIndex, toIndex = toIndex)
|
||||||
|
}
|
@ -0,0 +1,11 @@
|
|||||||
|
package com.twitter.tweetypie.thriftscala.entities
|
||||||
|
|
||||||
|
import com.twitter.tweetypie.thriftscala.UrlEntity
|
||||||
|
import com.twitter.tweetypie.tweettext.TextEntity
|
||||||
|
|
||||||
|
object UrlTextEntity extends TextEntity[UrlEntity] {
|
||||||
|
override def fromIndex(entity: UrlEntity): Short = entity.fromIndex
|
||||||
|
override def toIndex(entity: UrlEntity): Short = entity.toIndex
|
||||||
|
override def move(entity: UrlEntity, fromIndex: Short, toIndex: Short): UrlEntity =
|
||||||
|
entity.copy(fromIndex = fromIndex, toIndex = toIndex)
|
||||||
|
}
|
@ -0,0 +1,16 @@
|
|||||||
|
scala_library(
|
||||||
|
sources = ["*.scala"],
|
||||||
|
compiler_option_sets = ["fatal_warnings"],
|
||||||
|
platform = "java8",
|
||||||
|
provides = scala_artifact(
|
||||||
|
org = "com.twitter",
|
||||||
|
name = "tweetypie-tweettext",
|
||||||
|
repo = artifactory,
|
||||||
|
),
|
||||||
|
strict_deps = True,
|
||||||
|
tags = ["bazel-compatible"],
|
||||||
|
dependencies = [
|
||||||
|
"3rdparty/jvm/com/ibm/icu:icu4j",
|
||||||
|
"twitter-text/lib/java/src/main/java/com/twitter/twittertext",
|
||||||
|
],
|
||||||
|
)
|
@ -0,0 +1,44 @@
|
|||||||
|
package com.twitter.tweetypie.tweettext
|
||||||
|
|
||||||
|
import com.ibm.icu.text.BreakIterator
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Adapt the [[BreakIterator]] interface to a scala [[Iterator]]
|
||||||
|
* over the offsets of user-perceived characters in a String.
|
||||||
|
*/
|
||||||
|
object GraphemeIndexIterator {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Produce an iterator over indices in the string that mark the end
|
||||||
|
* of a user-perceived character (grapheme)
|
||||||
|
*/
|
||||||
|
def ends(s: String): Iterator[Offset.CodeUnit] =
|
||||||
|
// The start of every grapheme but the first is also a grapheme
|
||||||
|
// end. The last grapheme ends at the end of the string.
|
||||||
|
starts(s).drop(1) ++ Iterator(Offset.CodeUnit.length(s))
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Produce an iterator over indices in the string that mark the start
|
||||||
|
* of a user-perceived character (grapheme)
|
||||||
|
*/
|
||||||
|
def starts(s: String): Iterator[Offset.CodeUnit] =
|
||||||
|
new Iterator[Offset.CodeUnit] {
|
||||||
|
private[this] val it = BreakIterator.getCharacterInstance()
|
||||||
|
|
||||||
|
it.setText(s)
|
||||||
|
|
||||||
|
override def hasNext: Boolean = it.current < s.length
|
||||||
|
|
||||||
|
override def next: Offset.CodeUnit = {
|
||||||
|
if (!hasNext) throw new IllegalArgumentException(s"${it.current()}, ${s.length}")
|
||||||
|
|
||||||
|
// No matter what, we will be returning the value of `current`,
|
||||||
|
// which is the index of the start of the next grapheme.
|
||||||
|
val result = it.current()
|
||||||
|
|
||||||
|
it.next()
|
||||||
|
|
||||||
|
Offset.CodeUnit(result)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,85 @@
|
|||||||
|
package com.twitter.tweetypie.tweettext
|
||||||
|
|
||||||
|
/**
|
||||||
|
* An efficient converter of indices between code points and code units.
|
||||||
|
*/
|
||||||
|
class IndexConverter(text: String) {
|
||||||
|
// Keep track of a single corresponding pair of code unit and code point
|
||||||
|
// offsets so that we can re-use counting work if the next requested
|
||||||
|
// entity is near the most recent entity.
|
||||||
|
private var codePointIndex = 0
|
||||||
|
// The code unit index should never split a surrogate pair.
|
||||||
|
private var charIndex = 0
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param offset Index into the string measured in code units.
|
||||||
|
* @return The code point index that corresponds to the specified character index.
|
||||||
|
*/
|
||||||
|
def toCodePoints(offset: Offset.CodeUnit): Offset.CodePoint =
|
||||||
|
Offset.CodePoint(codeUnitsToCodePoints(offset.toInt))
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param charIndex Index into the string measured in code units.
|
||||||
|
* @return The code point index that corresponds to the specified character index.
|
||||||
|
*/
|
||||||
|
def codeUnitsToCodePoints(charIndex: Int): Int = {
|
||||||
|
if (charIndex < this.charIndex) {
|
||||||
|
this.codePointIndex -= text.codePointCount(charIndex, this.charIndex)
|
||||||
|
} else {
|
||||||
|
this.codePointIndex += text.codePointCount(this.charIndex, charIndex)
|
||||||
|
}
|
||||||
|
this.charIndex = charIndex
|
||||||
|
|
||||||
|
// Make sure that charIndex never points to the second code unit of a
|
||||||
|
// surrogate pair.
|
||||||
|
if (charIndex > 0 && Character.isSupplementaryCodePoint(text.codePointAt(charIndex - 1))) {
|
||||||
|
this.charIndex -= 1
|
||||||
|
this.codePointIndex -= 1
|
||||||
|
}
|
||||||
|
|
||||||
|
this.codePointIndex
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param offset Index into the string measured in code points.
|
||||||
|
* @return the corresponding code unit index
|
||||||
|
*/
|
||||||
|
def toCodeUnits(offset: Offset.CodePoint): Offset.CodeUnit = {
|
||||||
|
this.charIndex = text.offsetByCodePoints(charIndex, offset.toInt - this.codePointIndex)
|
||||||
|
this.codePointIndex = offset.toInt
|
||||||
|
Offset.CodeUnit(this.charIndex)
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param codePointIndex Index into the string measured in code points.
|
||||||
|
* @return the corresponding code unit index
|
||||||
|
*/
|
||||||
|
def codePointsToCodeUnits(codePointIndex: Int): Int =
|
||||||
|
toCodeUnits(Offset.CodePoint(codePointIndex)).toInt
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns a substring which begins at the specified code point `from` and extends to the
|
||||||
|
* code point `to`. Since String.substring only works with character, the method first
|
||||||
|
* converts code point offset to code unit offset.
|
||||||
|
*/
|
||||||
|
def substring(from: Offset.CodePoint, to: Offset.CodePoint): String =
|
||||||
|
text.substring(toCodeUnits(from).toInt, toCodeUnits(to).toInt)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns a substring which begins at the specified code point `from` and extends to the
|
||||||
|
* code point `to`. Since String.substring only works with character, the method first
|
||||||
|
* converts code point offset to code unit offset.
|
||||||
|
*/
|
||||||
|
def substringByCodePoints(from: Int, to: Int): String =
|
||||||
|
substring(Offset.CodePoint(from), Offset.CodePoint(to))
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns a substring which begins at the specified code point `from` and extends to the
|
||||||
|
* end of the string. Since String.substring only works with character, the method first
|
||||||
|
* converts code point offset to code unit offset.
|
||||||
|
*/
|
||||||
|
def substringByCodePoints(from: Int): String = {
|
||||||
|
val charFrom = codePointsToCodeUnits(from)
|
||||||
|
text.substring(charFrom)
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,253 @@
|
|||||||
|
package com.twitter.tweetypie.tweettext
|
||||||
|
import scala.collection.immutable
|
||||||
|
|
||||||
|
/**
|
||||||
|
* An Offset is a typed index into a String.
|
||||||
|
*/
|
||||||
|
trait Offset[T] extends Ordering[T] {
|
||||||
|
def toInt(t: T): Int
|
||||||
|
def count(text: String, start: Offset.CodeUnit, end: Offset.CodeUnit): T
|
||||||
|
|
||||||
|
def compare(t1: T, t2: T): Int = toInt(t1).compare(toInt(t2))
|
||||||
|
def length(input: String): T = count(input, Offset.CodeUnit(0), Offset.CodeUnit.length(input))
|
||||||
|
}
|
||||||
|
|
||||||
|
object Offset {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* UTF-16 code unit offsets are the native offsets for Java/Scala
|
||||||
|
* Strings.
|
||||||
|
*/
|
||||||
|
case class CodeUnit(toInt: Int) extends AnyVal with Ordered[CodeUnit] {
|
||||||
|
def compare(other: CodeUnit): Int = toInt.compare(other.toInt)
|
||||||
|
def +(other: CodeUnit) = CodeUnit(toInt + other.toInt)
|
||||||
|
def -(other: CodeUnit) = CodeUnit(toInt - other.toInt)
|
||||||
|
def min(other: CodeUnit): CodeUnit = if (toInt < other.toInt) this else other
|
||||||
|
def max(other: CodeUnit): CodeUnit = if (toInt > other.toInt) this else other
|
||||||
|
def incr: CodeUnit = CodeUnit(toInt + 1)
|
||||||
|
def decr: CodeUnit = CodeUnit(toInt - 1)
|
||||||
|
def until(end: CodeUnit): immutable.IndexedSeq[CodeUnit] =
|
||||||
|
toInt.until(end.toInt).map(CodeUnit(_))
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Converts this `CodeUnit` to the equivalent `CodePoint` within the
|
||||||
|
* given text.
|
||||||
|
*/
|
||||||
|
def toCodePoint(text: String): CodePoint =
|
||||||
|
CodePoint(text.codePointCount(0, toInt))
|
||||||
|
|
||||||
|
def offsetByCodePoints(text: String, codePoints: CodePoint): CodeUnit =
|
||||||
|
CodeUnit(text.offsetByCodePoints(toInt, codePoints.toInt))
|
||||||
|
}
|
||||||
|
|
||||||
|
implicit object CodeUnit extends Offset[CodeUnit] {
|
||||||
|
def toInt(u: CodeUnit): Int = u.toInt
|
||||||
|
override def length(text: String): CodeUnit = CodeUnit(text.length)
|
||||||
|
def count(text: String, start: CodeUnit, end: CodeUnit): CodeUnit = end - start
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Offsets in whole Unicode code points. Any CodePoint is a valid
|
||||||
|
* offset into the String as long as it is >= 0 and less than the
|
||||||
|
* number of code points in the string.
|
||||||
|
*/
|
||||||
|
case class CodePoint(toInt: Int) extends AnyVal with Ordered[CodePoint] {
|
||||||
|
def toShort: Short = toInt.toShort
|
||||||
|
def compare(other: CodePoint): Int = toInt.compare(other.toInt)
|
||||||
|
def +(other: CodePoint) = CodePoint(toInt + other.toInt)
|
||||||
|
def -(other: CodePoint) = CodePoint(toInt - other.toInt)
|
||||||
|
def min(other: CodePoint): CodePoint = if (toInt < other.toInt) this else other
|
||||||
|
def max(other: CodePoint): CodePoint = if (toInt > other.toInt) this else other
|
||||||
|
def until(end: CodePoint): immutable.IndexedSeq[CodePoint] =
|
||||||
|
toInt.until(end.toInt).map(CodePoint(_))
|
||||||
|
|
||||||
|
def toCodeUnit(text: String): CodeUnit =
|
||||||
|
CodeUnit(text.offsetByCodePoints(0, toInt))
|
||||||
|
}
|
||||||
|
|
||||||
|
implicit object CodePoint extends Offset[CodePoint] {
|
||||||
|
def toInt(p: CodePoint): Int = p.toInt
|
||||||
|
|
||||||
|
def count(text: String, start: CodeUnit, end: CodeUnit): CodePoint =
|
||||||
|
CodePoint(text.codePointCount(start.toInt, end.toInt))
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Offsets into the String as if the String were encoded as UTF-8. You
|
||||||
|
* cannot use a [[Utf8]] offset to index a String, because not all
|
||||||
|
* Utf8 indices are valid indices into the String.
|
||||||
|
*/
|
||||||
|
case class Utf8(toInt: Int) extends AnyVal with Ordered[Utf8] {
|
||||||
|
def compare(other: Utf8): Int = toInt.compare(other.toInt)
|
||||||
|
def +(other: Utf8) = Utf8(toInt + other.toInt)
|
||||||
|
def -(other: Utf8) = Utf8(toInt - other.toInt)
|
||||||
|
def min(other: Utf8): Utf8 = if (toInt < other.toInt) this else other
|
||||||
|
def max(other: Utf8): Utf8 = if (toInt > other.toInt) this else other
|
||||||
|
}
|
||||||
|
|
||||||
|
implicit object Utf8 extends Offset[Utf8] {
|
||||||
|
def toInt(u: Utf8): Int = u.toInt
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Count how many bytes this section of text would be when encoded as
|
||||||
|
* UTF-8.
|
||||||
|
*/
|
||||||
|
def count(s: String, start: CodeUnit, end: CodeUnit): Utf8 = {
|
||||||
|
def go(i: CodeUnit, byteLength: Utf8): Utf8 =
|
||||||
|
if (i < end) {
|
||||||
|
val cp = s.codePointAt(i.toInt)
|
||||||
|
go(i + CodeUnit(Character.charCount(cp)), byteLength + forCodePoint(cp))
|
||||||
|
} else {
|
||||||
|
byteLength
|
||||||
|
}
|
||||||
|
|
||||||
|
go(start, Utf8(0))
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Unfortunately, there is no convenient API for finding out how many
|
||||||
|
* bytes a unicode code point would take in UTF-8, so we have to
|
||||||
|
* explicitly calculate it.
|
||||||
|
*
|
||||||
|
* @see http://en.wikipedia.org/wiki/UTF-8#Description
|
||||||
|
*/
|
||||||
|
def forCodePoint(cp: Int): Utf8 =
|
||||||
|
Utf8 {
|
||||||
|
// if the code point is an unpaired surrogate, it will be converted
|
||||||
|
// into a 1 byte replacement character
|
||||||
|
if (Character.getType(cp) == Character.SURROGATE) 1
|
||||||
|
else {
|
||||||
|
cp match {
|
||||||
|
case _ if cp < 0x80 => 1
|
||||||
|
case _ if cp < 0x800 => 2
|
||||||
|
case _ if cp < 0x10000 => 3
|
||||||
|
case _ => 4
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Display units count what we consider a "character" in a
|
||||||
|
* Tweet. [[DisplayUnit]] offsets are only valid for text that is
|
||||||
|
* NFC-normalized (See: http://www.unicode.org/reports/tr15) and
|
||||||
|
* HTML-encoded, though this interface cannot enforce that.
|
||||||
|
*
|
||||||
|
* Currently, a [[DisplayUnit]] is equivalent to a single Unicode code
|
||||||
|
* point combined with treating "<", ">", and "&" each as a
|
||||||
|
* single character (since they are displayed as '<', '>', and '&'
|
||||||
|
* respectively). This implementation is not directly exposed.
|
||||||
|
*
|
||||||
|
* It should be possible to change this definition without breaking
|
||||||
|
* code that uses the [[DisplayUnit]] interface e.g. to count
|
||||||
|
* user-perceived characters (graphemes) rather than code points,
|
||||||
|
* though any change has to be made in concert with changing the
|
||||||
|
* mobile client and Web implementations so that the user experience
|
||||||
|
* of character counting remains consistent.
|
||||||
|
*/
|
||||||
|
case class DisplayUnit(toInt: Int) extends AnyVal with Ordered[DisplayUnit] {
|
||||||
|
def compare(other: DisplayUnit): Int = toInt.compare(other.toInt)
|
||||||
|
def +(other: DisplayUnit) = DisplayUnit(toInt + other.toInt)
|
||||||
|
def -(other: DisplayUnit) = DisplayUnit(toInt - other.toInt)
|
||||||
|
def min(other: DisplayUnit): DisplayUnit = if (toInt < other.toInt) this else other
|
||||||
|
def max(other: DisplayUnit): DisplayUnit = if (toInt > other.toInt) this else other
|
||||||
|
}
|
||||||
|
|
||||||
|
implicit object DisplayUnit extends Offset[DisplayUnit] {
|
||||||
|
def toInt(d: DisplayUnit): Int = d.toInt
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the number of display units in the specified range of the
|
||||||
|
* given text. See [[DisplayUnit]] for a descrption of what we
|
||||||
|
* consider a display unit.
|
||||||
|
*
|
||||||
|
* The input string should already be NFC normalized to get
|
||||||
|
* consistent results. If partially html encoded, it will correctly
|
||||||
|
* count html entities as a single display unit.
|
||||||
|
*
|
||||||
|
* @param text the string containing the characters to count.
|
||||||
|
* @param the index to the first char of the text range
|
||||||
|
* @param the index after the last char of the text range.
|
||||||
|
*/
|
||||||
|
def count(text: String, start: CodeUnit, end: CodeUnit): DisplayUnit = {
|
||||||
|
val stop = end.min(CodeUnit.length(text))
|
||||||
|
|
||||||
|
@annotation.tailrec
|
||||||
|
def go(offset: CodeUnit, total: DisplayUnit): DisplayUnit =
|
||||||
|
if (offset >= stop) total
|
||||||
|
else go(offset + at(text, offset), total + DisplayUnit(1))
|
||||||
|
|
||||||
|
go(start, DisplayUnit(0))
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the length of the display unit at the specified offset in
|
||||||
|
* the (NFC-normalized, HTML-encoded) text.
|
||||||
|
*/
|
||||||
|
def at(text: String, offset: CodeUnit): CodeUnit =
|
||||||
|
CodeUnit {
|
||||||
|
text.codePointAt(offset.toInt) match {
|
||||||
|
case '&' =>
|
||||||
|
if (text.regionMatches(offset.toInt, "&", 0, 5)) 5
|
||||||
|
else if (text.regionMatches(offset.toInt, "<", 0, 4)) 4
|
||||||
|
else if (text.regionMatches(offset.toInt, ">", 0, 4)) 4
|
||||||
|
else 1
|
||||||
|
|
||||||
|
case cp => Character.charCount(cp)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Ranges of offsets, useful for avoiding slicing entities.
|
||||||
|
*/
|
||||||
|
sealed trait Ranges[T] {
|
||||||
|
def contains(t: T): Boolean
|
||||||
|
}
|
||||||
|
|
||||||
|
object Ranges {
|
||||||
|
private[this] case class Impl[T](toSeq: Seq[(T, T)])(implicit off: Offset[T])
|
||||||
|
extends Ranges[T] {
|
||||||
|
def contains(t: T): Boolean = toSeq.exists { case (lo, hi) => off.gt(t, lo) && off.lt(t, hi) }
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Non-inclusive range of offsets (matches values that are strictly
|
||||||
|
* between `hi` and `lo`)
|
||||||
|
*/
|
||||||
|
def between[T](lo: T, hi: T)(implicit off: Offset[T]): Ranges[T] =
|
||||||
|
if (off.toInt(hi) > off.toInt(lo) + 1 && off.toInt(lo) < Int.MaxValue) Impl(Seq((lo, hi)))
|
||||||
|
else Impl(Nil)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The union of all of the specified ranges.
|
||||||
|
*/
|
||||||
|
def all[T](ranges: Seq[Ranges[T]])(implicit off: Offset[T]): Ranges[T] =
|
||||||
|
Impl(
|
||||||
|
// Preprocess the ranges so that each contains check is as cheap
|
||||||
|
// as possible.
|
||||||
|
ranges
|
||||||
|
.flatMap { case r: Impl[T] => r.toSeq }
|
||||||
|
.sortBy(_._1)
|
||||||
|
.foldLeft(Nil: List[(T, T)]) {
|
||||||
|
case ((a, b) :: out, (c, d)) if off.lt(c, b) => (a, d) :: out
|
||||||
|
case (out, r) => r :: out
|
||||||
|
}
|
||||||
|
)
|
||||||
|
|
||||||
|
def Empty[T: Offset]: Ranges[T] = Impl[T](Nil)
|
||||||
|
|
||||||
|
private[this] val HtmlEscapes = """&(?:amp|lt|gt);""".r
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Match [[CodeUnit]]s that would split a HTML entity.
|
||||||
|
*/
|
||||||
|
def htmlEntities(s: String): Ranges[CodeUnit] = {
|
||||||
|
val it = HtmlEscapes.findAllIn(s)
|
||||||
|
all(it.map(_ => between(CodeUnit(it.start), CodeUnit(it.end))).toSeq)
|
||||||
|
}
|
||||||
|
|
||||||
|
def fromCodePointPairs(pairs: Seq[(Int, Int)]): Ranges[CodePoint] =
|
||||||
|
all(pairs.map { case (lo, hi) => between(CodePoint(lo), CodePoint(hi)) })
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,55 @@
|
|||||||
|
package com.twitter.tweetypie.tweettext
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Code used to convert raw user-provided text into an allowable form.
|
||||||
|
*/
|
||||||
|
object PartialHtmlEncoding {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Replaces all `<`, `>`, and '&' chars with "<", ">", and "&", respectively.
|
||||||
|
*
|
||||||
|
* Tweet text is HTML-encoded at tweet creation time, and is stored and processed in encoded form.
|
||||||
|
*/
|
||||||
|
def encode(text: String): String = {
|
||||||
|
val buf = new StringBuilder
|
||||||
|
|
||||||
|
text.foreach {
|
||||||
|
case '<' => buf.append("<")
|
||||||
|
case '>' => buf.append(">")
|
||||||
|
case '&' => buf.append("&")
|
||||||
|
case c => buf.append(c)
|
||||||
|
}
|
||||||
|
|
||||||
|
buf.toString
|
||||||
|
}
|
||||||
|
|
||||||
|
private val AmpLtRegex = "<".r
|
||||||
|
private val AmpGtRegex = ">".r
|
||||||
|
private val AmpAmpRegex = "&".r
|
||||||
|
|
||||||
|
private val partialHtmlDecoder: (String => String) =
|
||||||
|
((s: String) => AmpLtRegex.replaceAllIn(s, "<"))
|
||||||
|
.andThen(s => AmpGtRegex.replaceAllIn(s, ">"))
|
||||||
|
.andThen(s => AmpAmpRegex.replaceAllIn(s, "&"))
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The opposite of encode, it replaces all "<", ">", and "&" with
|
||||||
|
* `<`, `>`, and '&', respectively.
|
||||||
|
*/
|
||||||
|
def decode(text: String): String =
|
||||||
|
decodeWithModification(text) match {
|
||||||
|
case Some(mod) => mod.updated
|
||||||
|
case None => text
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Decodes encoded entities, and returns a `TextModification` if the text was modified.
|
||||||
|
*/
|
||||||
|
def decodeWithModification(text: String): Option[TextModification] =
|
||||||
|
TextModification.replaceAll(
|
||||||
|
text,
|
||||||
|
AmpLtRegex -> "<",
|
||||||
|
AmpGtRegex -> ">",
|
||||||
|
AmpAmpRegex -> "&"
|
||||||
|
)
|
||||||
|
}
|
@ -0,0 +1,251 @@
|
|||||||
|
package com.twitter.tweetypie.tweettext
|
||||||
|
import scala.util.matching.Regex
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Code used to convert raw user-provided text into an allowable form.
|
||||||
|
*/
|
||||||
|
object Preprocessor {
|
||||||
|
import TweetText._
|
||||||
|
import TextModification.replaceAll
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Regex for dos-style line endings.
|
||||||
|
*/
|
||||||
|
val DosLineEndingRegex: Regex = """\r\n""".r
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Converts \r\n to just \n.
|
||||||
|
*/
|
||||||
|
def normalizeNewlines(text: String): String =
|
||||||
|
DosLineEndingRegex.replaceAllIn(text, "\n")
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Characters to strip out of tweet text at write-time.
|
||||||
|
*/
|
||||||
|
val unicodeCharsToStrip: Seq[Char] =
|
||||||
|
Seq(
|
||||||
|
'\uFFFE', '\uFEFF', // BOM
|
||||||
|
'\uFFFF', // Special
|
||||||
|
'\u200E', '\u200F', // ltr, rtl
|
||||||
|
'\u202A', '\u202B', '\u202C', '\u202D', '\u202E', // Directional change
|
||||||
|
'\u0000', '\u0001', '\u0002', '\u0003', '\u0004', '\u0005', '\u0006', '\u0007', '\u0008',
|
||||||
|
'\u0009', '\u000B', '\u000C', '\u000E', '\u000F', '\u0010', '\u0011', '\u0012', '\u0013',
|
||||||
|
'\u0014', '\u0015', '\u0016', '\u0017', '\u0018', '\u0019', '\u001A', '\u001B', '\u001C',
|
||||||
|
'\u001D', '\u001E', '\u001F', '\u007F',
|
||||||
|
'\u2065',
|
||||||
|
)
|
||||||
|
|
||||||
|
val UnicodeCharsToStripRegex: Regex = unicodeCharsToStrip.mkString("[", "", "]").r
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Strips out control characters and other non-textual unicode chars that can break xml and/or
|
||||||
|
* json rendering, or be used for exploits.
|
||||||
|
*/
|
||||||
|
def stripControlCharacters(text: String): String =
|
||||||
|
UnicodeCharsToStripRegex.replaceAllIn(text, "")
|
||||||
|
|
||||||
|
val Tweetypie674UnicodeSequence: String =
|
||||||
|
"\u0633\u0645\u064e\u0640\u064e\u0651\u0648\u064f\u0648\u064f\u062d\u062e " +
|
||||||
|
"\u0337\u0334\u0310\u062e \u0337\u0334\u0310\u062e \u0337\u0334\u0310\u062e " +
|
||||||
|
"\u0627\u0645\u0627\u0631\u062a\u064a\u062e \u0337\u0334\u0310\u062e"
|
||||||
|
|
||||||
|
val Tweetypie674UnicodeRegex: Regex = Tweetypie674UnicodeSequence.r
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Replace each `Tweetypie674UnicodeSequence` of this string to REPLACEMENT
|
||||||
|
* CHARACTER.
|
||||||
|
*
|
||||||
|
* Apple has a bug in its CoreText library. This aims to prevent
|
||||||
|
* ios clients from being crashed when a tweet contains the specific
|
||||||
|
* unicode sequence.
|
||||||
|
*/
|
||||||
|
def avoidCoreTextBug(text: String): String =
|
||||||
|
Tweetypie674UnicodeRegex.replaceAllIn(text, "\ufffd")
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Replace each `Tweetypie674UnicodeSequence` of this string to a REPLACEMENT
|
||||||
|
* CHARACTER, returns a TextModification object that provides information
|
||||||
|
* to also update entity indices.
|
||||||
|
*/
|
||||||
|
def replaceCoreTextBugModification(text: String): Option[TextModification] =
|
||||||
|
replaceAll(text, Tweetypie674UnicodeRegex, "\ufffd")
|
||||||
|
|
||||||
|
private val preprocessor: String => String =
|
||||||
|
((s: String) => nfcNormalize(s))
|
||||||
|
.andThen(stripControlCharacters _)
|
||||||
|
.andThen(trimBlankCharacters _)
|
||||||
|
.andThen(normalizeNewlines _)
|
||||||
|
.andThen(collapseBlankLines _)
|
||||||
|
.andThen(avoidCoreTextBug _)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Performs the text modifications that are necessary in the write-path before extracting URLs.
|
||||||
|
*/
|
||||||
|
def preprocessText(text: String): String =
|
||||||
|
preprocessor(text)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Replaces all `<`, `>`, and '&' chars with "<", ">", and "&", respectively.
|
||||||
|
*
|
||||||
|
* The original purpose of this was presumably to prevent script injections when
|
||||||
|
* displaying tweets without proper escaping. Currently, tweets are encoded before
|
||||||
|
* they are stored in the database.
|
||||||
|
*
|
||||||
|
* Note that the pre-escaping of & < and > also happens in the rich text editor in javascript
|
||||||
|
*/
|
||||||
|
def partialHtmlEncode(text: String): String =
|
||||||
|
PartialHtmlEncoding.encode(text)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The opposite of partialHtmlEncode, it replaces all "<", ">", and "&" with
|
||||||
|
* `<`, `>`, and '&', respectively.
|
||||||
|
*/
|
||||||
|
def partialHtmlDecode(text: String): String =
|
||||||
|
PartialHtmlEncoding.decode(text)
|
||||||
|
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* Detects all forms of whitespace, considering as whitespace the following:
|
||||||
|
* This regex detects characters that always or often are rendered as blank space. We use
|
||||||
|
* this to prevent users from inserting excess blank lines and from tweeting effectively
|
||||||
|
* blank tweets.
|
||||||
|
*
|
||||||
|
* Note that these are not all semantically "whitespace", so this regex should not be used
|
||||||
|
* to process non-blank text, e.g. to separate words.
|
||||||
|
*
|
||||||
|
* Codepoints below and the `\p{Z}` regex character property alias are defined in the Unicode
|
||||||
|
* Character Database (UCD) at https://unicode.org/ucd/ and https://unicode.org/reports/tr44/
|
||||||
|
*
|
||||||
|
* The `\p{Z}` regex character property alias is defined specifically in UCD as:
|
||||||
|
*
|
||||||
|
* Zs | Space_Separator | a space character (of various non-zero widths)
|
||||||
|
* Zl | Line_Separator | U+2028 LINE SEPARATOR only
|
||||||
|
* Zp | Paragraph_Separator | U+2029 PARAGRAPH SEPARATOR only
|
||||||
|
* Z | Separator | Zs | Zl | Zp
|
||||||
|
* ref: https://unicode.org/reports/tr44/#GC_Values_Table
|
||||||
|
*
|
||||||
|
* U+0009 Horizontal Tab (included in \s)
|
||||||
|
* U+000B Vertical Tab (included in \s)
|
||||||
|
* U+000C Form feed (included in \s)
|
||||||
|
* U+000D Carriage return (included in \s)
|
||||||
|
* U+0020 space (included in \s)
|
||||||
|
* U+0085 Next line (included in \u0085)
|
||||||
|
* U+061C arabic letter mark (included in \u061C)
|
||||||
|
* U+00A0 no-break space (included in \p{Z})
|
||||||
|
* U+00AD soft-hyphen marker (included in \u00AD)
|
||||||
|
* U+1680 ogham space mark (included in \p{Z})
|
||||||
|
* U+180E mongolian vowel separator (included in \p{Z} on jdk8 and included in \u180E on jdk11)
|
||||||
|
* U+2000 en quad (included in \p{Z})
|
||||||
|
* U+2001 em quad (included in \p{Z})
|
||||||
|
* U+2002 en space (included in \p{Z})
|
||||||
|
* U+2003 em space (included in \p{Z})
|
||||||
|
* U+2004 three-per-em space (included in \p{Z})
|
||||||
|
* U+2005 four-per-em space (included in \p{Z})
|
||||||
|
* U+2006 six-per-em space (included in \p{Z})
|
||||||
|
* U+2007 figure space (included in \p{Z})
|
||||||
|
* U+2008 punctuation space (included in \p{Z})
|
||||||
|
* U+2009 thin space (included in \p{Z})
|
||||||
|
* U+200A hair space (included in \p{Z})
|
||||||
|
* U+200B zero-width (included in \u200B-\u200D)
|
||||||
|
* U+200C zero-width non-joiner (included in \u200B-\u200D)
|
||||||
|
* U+200D zero-width joiner (included in \u200B-\u200D)
|
||||||
|
* U+2028 line separator (included in \p{Z})
|
||||||
|
* U+2029 paragraph separator (included in \p{Z})
|
||||||
|
* U+202F narrow no-break space (included in \p{Z})
|
||||||
|
* U+205F medium mathematical space (included in \p{Z})
|
||||||
|
* U+2061 function application (included in \u2061-\u2064)
|
||||||
|
* U+2062 invisible times (included in \u2061-\u2064)
|
||||||
|
* U+2063 invisible separator (included in \u2061-\u2064)
|
||||||
|
* U+2064 invisible plus (included in \u2061-\u2064)
|
||||||
|
* U+2066 left-to-right isolate (included in \u2066-\u2069)
|
||||||
|
* U+2067 right-to-left isolate (included in \u2066-\u2069)
|
||||||
|
* U+2068 first strong isolate (included in \u2066-\u2069)
|
||||||
|
* U+2069 pop directional isolate (included in \u2066-\u2069)
|
||||||
|
* U+206A inhibit symmetric swapping (included in \u206A-\u206F)
|
||||||
|
* U+206B activate symmetric swapping (included in \u206A-\u206F)
|
||||||
|
* U+206C inhibit arabic form shaping (included in \u206A-\u206F)
|
||||||
|
* U+206D activate arabic form shaping (included in \u206A-\u206F)
|
||||||
|
* U+206E national digit shapes (included in \u206A-\u206F)
|
||||||
|
* U+206F nominal digit shapes (included in \u206A-\u206F)
|
||||||
|
* U+2800 braille pattern blank (included in \u2800)
|
||||||
|
* U+3164 hongul filler (see UCD Ignorable_Code_Point)
|
||||||
|
* U+FFA0 halfwidth hongul filler (see UCD Ignorable_Code_Point)
|
||||||
|
* U+3000 ideographic space (included in \p{Z})
|
||||||
|
* U+FEFF zero-width no-break space (explicitly included in \uFEFF)
|
||||||
|
*/
|
||||||
|
val BlankTextRegex: Regex =
|
||||||
|
"""[\s\p{Z}\u180E\u0085\u00AD\u061C\u200B-\u200D\u2061-\u2064\u2066-\u2069\u206A-\u206F\u2800\u3164\uFEFF\uFFA0]*""".r
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Some of the above blank characters are valid at the start of a Tweet (and irrelevant at the end)
|
||||||
|
* such as characters that change the direction of text. When trimming from the start
|
||||||
|
* or end of text we use a smaller set of characters
|
||||||
|
*/
|
||||||
|
val BlankWhenLeadingOrTrailingRegex: Regex = """[\s\p{Z}\u180E\u0085\u200B\uFEFF]*""".r
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Matches consecutive blanks, starting at a newline.
|
||||||
|
*/
|
||||||
|
val ConsecutiveBlankLinesRegex: Regex = ("""\n(""" + BlankTextRegex + """\n){2,}""").r
|
||||||
|
|
||||||
|
val LeadingBlankCharactersRegex: Regex = ("^" + BlankWhenLeadingOrTrailingRegex).r
|
||||||
|
val TrailingBlankCharactersRegex: Regex = (BlankWhenLeadingOrTrailingRegex + "$").r
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Is the given text empty or contains nothing but whitespace?
|
||||||
|
*/
|
||||||
|
def isBlank(text: String): Boolean =
|
||||||
|
BlankTextRegex.pattern.matcher(text).matches()
|
||||||
|
|
||||||
|
/**
|
||||||
|
* See http://confluence.local.twitter.com/display/PROD/Displaying+line+breaks+in+Tweets
|
||||||
|
*
|
||||||
|
* Collapses consecutive blanks lines down to a single blank line. We can assume that
|
||||||
|
* all newlines have already been normalized to just \n, so we don't have to worry about
|
||||||
|
* \r\n.
|
||||||
|
*/
|
||||||
|
def collapseBlankLinesModification(text: String): Option[TextModification] =
|
||||||
|
replaceAll(text, ConsecutiveBlankLinesRegex, "\n\n")
|
||||||
|
|
||||||
|
def collapseBlankLines(text: String): String =
|
||||||
|
ConsecutiveBlankLinesRegex.replaceAllIn(text, "\n\n")
|
||||||
|
|
||||||
|
def trimBlankCharacters(text: String): String =
|
||||||
|
TrailingBlankCharactersRegex.replaceFirstIn(
|
||||||
|
LeadingBlankCharactersRegex.replaceFirstIn(text, ""),
|
||||||
|
""
|
||||||
|
)
|
||||||
|
|
||||||
|
/** Characters that are not visible on their own. Some of these are used in combination with
|
||||||
|
* other visible characters, and therefore cannot be always stripped from tweets.
|
||||||
|
*/
|
||||||
|
private[tweettext] val InvisibleCharacters: Seq[Char] =
|
||||||
|
Seq(
|
||||||
|
'\u2060', '\u2061', '\u2062', '\u2063', '\u2064', '\u206A', '\u206B', '\u206C', '\u206D',
|
||||||
|
'\u206D', '\u206E', '\u206F', '\u200C',
|
||||||
|
'\u200D', // non-printing chars with valid use in Arabic
|
||||||
|
'\u2009', '\u200A', '\u200B', // include very skinny spaces too
|
||||||
|
'\ufe00', '\ufe01', '\ufe02', '\ufe03', '\ufe04', '\ufe05', '\ufe06', '\ufe07', '\ufe08',
|
||||||
|
'\ufe09', '\ufe0A', '\ufe0B', '\ufe0C', '\ufe0D', '\ufe0E', '\ufe0F',
|
||||||
|
)
|
||||||
|
|
||||||
|
private[tweetypie] val InvisibleUnicodePattern: Regex =
|
||||||
|
("^[" + InvisibleCharacters.mkString + "]+$").r
|
||||||
|
|
||||||
|
def isInvisibleChar(input: Char): Boolean = {
|
||||||
|
InvisibleCharacters contains input
|
||||||
|
}
|
||||||
|
|
||||||
|
/** If string is only "invisible characters", replace full string with whitespace.
|
||||||
|
* The purpose of this method is to remove invisible characters when ONLY invisible characters
|
||||||
|
* appear between two urls, which can be a security vulnerability due to misleading behavior. These
|
||||||
|
* characters cannot be removed as a rule applied to the tweet, because they are used in
|
||||||
|
* conjuction with other characters.
|
||||||
|
*/
|
||||||
|
def replaceInvisiblesWithWhitespace(text: String): String = {
|
||||||
|
text match {
|
||||||
|
case invisible @ InvisibleUnicodePattern() => " " * TweetText.codePointLength(invisible)
|
||||||
|
case other => other
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,24 @@
|
|||||||
|
package com.twitter.tweetypie.tweettext
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A type class for entities found within a piece of tweet text.
|
||||||
|
*/
|
||||||
|
trait TextEntity[T] {
|
||||||
|
def fromIndex(entity: T): Short
|
||||||
|
def toIndex(entity: T): Short
|
||||||
|
def move(entity: T, fromIndex: Short, toIndex: Short): T
|
||||||
|
}
|
||||||
|
|
||||||
|
object TextEntity {
|
||||||
|
def fromIndex[T: TextEntity](entity: T): Short =
|
||||||
|
implicitly[TextEntity[T]].fromIndex(entity)
|
||||||
|
|
||||||
|
def toIndex[T: TextEntity](entity: T): Short =
|
||||||
|
implicitly[TextEntity[T]].toIndex(entity)
|
||||||
|
|
||||||
|
def move[T: TextEntity](entity: T, fromIndex: Short, toIndex: Short): T =
|
||||||
|
implicitly[TextEntity[T]].move(entity, fromIndex, toIndex)
|
||||||
|
|
||||||
|
def shift[T: TextEntity](entity: T, offset: Short): T =
|
||||||
|
move(entity, (fromIndex(entity) + offset).toShort, (toIndex(entity) + offset).toShort)
|
||||||
|
}
|
@ -0,0 +1,232 @@
|
|||||||
|
package com.twitter.tweetypie.tweettext
|
||||||
|
|
||||||
|
import scala.util.matching.Regex
|
||||||
|
|
||||||
|
object TextModification {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Lift a text into a TextModification where `original` and `updated` text are the same
|
||||||
|
* and `replacements` is empty.
|
||||||
|
*/
|
||||||
|
def identity(text: String): TextModification =
|
||||||
|
TextModification(original = text, updated = text, replacements = Nil)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Replace each substring that matches the regex with the substitution string, returns a
|
||||||
|
* TextModification object that contains the updated text and enough information to also
|
||||||
|
* update entity indices.
|
||||||
|
*
|
||||||
|
* This method should correctly be taking into account surrogate-pairs. The returned
|
||||||
|
* TextModification object has code-point offsets, instead of code-unit offsets.
|
||||||
|
*/
|
||||||
|
def replaceAll(text: String, regex: Regex, substitution: String): Option[TextModification] =
|
||||||
|
replaceAll(text, regex -> substitution)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Replaces substrings that match the given `Regex` with the corresonding substitution
|
||||||
|
* string. Returns a `TextModification` that can be used to reindex entities.
|
||||||
|
*/
|
||||||
|
def replaceAll(
|
||||||
|
text: String,
|
||||||
|
regexAndSubstitutions: (Regex, String)*
|
||||||
|
): Option[TextModification] = {
|
||||||
|
val matches =
|
||||||
|
(for {
|
||||||
|
(r, s) <- regexAndSubstitutions
|
||||||
|
m <- r.findAllIn(text).matchData
|
||||||
|
} yield (m, s)).sortBy { case (m, _) => m.start }
|
||||||
|
|
||||||
|
if (matches.isEmpty) {
|
||||||
|
// no match found, return None to indicate no modifications made
|
||||||
|
None
|
||||||
|
} else {
|
||||||
|
val replacements = List.newBuilder[TextReplacement]
|
||||||
|
val indexConverter = new IndexConverter(text)
|
||||||
|
// contains the retained text, built up as we walk through the regex matches
|
||||||
|
val buf = new StringBuilder(text.length)
|
||||||
|
// the number of code-points copied into buf
|
||||||
|
var codePointsCopied = Offset.CodePoint(0)
|
||||||
|
// always holds the start code-unit offset to copy to buf when we encounter
|
||||||
|
// either a regex match or end-of-string.
|
||||||
|
var anchor = 0
|
||||||
|
|
||||||
|
import indexConverter.toCodePoints
|
||||||
|
|
||||||
|
for ((m, sub) <- matches) {
|
||||||
|
val unchangedText = text.substring(anchor, m.start)
|
||||||
|
val unchangedLen = Offset.CodePoint.length(unchangedText)
|
||||||
|
val subLen = Offset.CodePoint.length(sub)
|
||||||
|
|
||||||
|
// copies the text upto the regex match run, plus the replacement string
|
||||||
|
buf.append(unchangedText).append(sub)
|
||||||
|
codePointsCopied += unchangedLen + subLen
|
||||||
|
|
||||||
|
// the offsets indicate the indices of the matched string in the original
|
||||||
|
// text, and the indices of the replacement string in the updated string
|
||||||
|
replacements +=
|
||||||
|
TextReplacement(
|
||||||
|
originalFrom = toCodePoints(Offset.CodeUnit(m.start)),
|
||||||
|
originalTo = toCodePoints(Offset.CodeUnit(m.end)),
|
||||||
|
updatedFrom = codePointsCopied - subLen,
|
||||||
|
updatedTo = codePointsCopied
|
||||||
|
)
|
||||||
|
|
||||||
|
anchor = m.end
|
||||||
|
}
|
||||||
|
|
||||||
|
buf.append(text.substring(anchor))
|
||||||
|
|
||||||
|
Some(TextModification(text, buf.toString, replacements.result()))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Inserts a string at a specified code point offset.
|
||||||
|
* Returns a `TextModification` that can be used to reindex entities.
|
||||||
|
*/
|
||||||
|
def insertAt(
|
||||||
|
originalText: String,
|
||||||
|
insertAt: Offset.CodePoint,
|
||||||
|
textToInsert: String
|
||||||
|
): TextModification = {
|
||||||
|
val insertAtCodeUnit = insertAt.toCodeUnit(originalText).toInt
|
||||||
|
val (before, after) = originalText.splitAt(insertAtCodeUnit)
|
||||||
|
val updatedText = s"$before$textToInsert$after"
|
||||||
|
val textToInsertLength = TweetText.codePointLength(textToInsert)
|
||||||
|
|
||||||
|
TextModification(
|
||||||
|
original = originalText,
|
||||||
|
updated = updatedText,
|
||||||
|
replacements = List(
|
||||||
|
TextReplacement.fromCodePoints(
|
||||||
|
originalFrom = insertAt.toInt,
|
||||||
|
originalTo = insertAt.toInt,
|
||||||
|
updatedFrom = insertAt.toInt,
|
||||||
|
updatedTo = insertAt.toInt + textToInsertLength
|
||||||
|
))
|
||||||
|
)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Encodes information about insertions/deletions/replacements made to a string, providing
|
||||||
|
* the original string, the updated string, and a list of TextReplacement objects
|
||||||
|
* that encode the indices of the segments that were changed. Using this information,
|
||||||
|
* it is possible to map an offset into the original string to an offset into the updated
|
||||||
|
* string, assuming the text at the offset was not within one of the modified segments.
|
||||||
|
*
|
||||||
|
* All offsets are code-points, not UTF6 code-units.
|
||||||
|
*/
|
||||||
|
case class TextModification(
|
||||||
|
original: String,
|
||||||
|
updated: String,
|
||||||
|
replacements: List[TextReplacement]) {
|
||||||
|
private val originalLen = Offset.CodePoint.length(original)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Using an offset into the original String, computes the equivalent offset into the updated
|
||||||
|
* string. If the offset falls within a segment that was removed/replaced, None is returned.
|
||||||
|
*/
|
||||||
|
def reindex(index: Offset.CodePoint): Option[Offset.CodePoint] =
|
||||||
|
reindex(index, Offset.CodePoint(0), replacements)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Reindexes an entity of type T. Returns the updated entity, or None if either the `fromIndex`
|
||||||
|
* or `toIndex` value is now out of range.
|
||||||
|
*/
|
||||||
|
def reindexEntity[T: TextEntity](e: T): Option[T] =
|
||||||
|
for {
|
||||||
|
from <- reindex(Offset.CodePoint(TextEntity.fromIndex(e)))
|
||||||
|
to <- reindex(Offset.CodePoint(TextEntity.toIndex(e) - 1))
|
||||||
|
} yield TextEntity.move(e, from.toShort, (to.toShort + 1).toShort)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Reindexes a sequence of entities of type T. Some entities could be filtered
|
||||||
|
* out if they span a region of text that has been removed.
|
||||||
|
*/
|
||||||
|
def reindexEntities[T: TextEntity](es: Seq[T]): Seq[T] =
|
||||||
|
for (e <- es; e2 <- reindexEntity(e)) yield e2
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Swaps `original` and `updated` text and inverts all `TextReplacement` instances.
|
||||||
|
*/
|
||||||
|
def inverse: TextModification =
|
||||||
|
TextModification(updated, original, replacements.map(_.inverse))
|
||||||
|
|
||||||
|
// recursively walks through the list of TextReplacement objects computing
|
||||||
|
// offsets to add/substract from 'shift', which accumulates all changes and
|
||||||
|
// then gets added to index at the end.
|
||||||
|
private def reindex(
|
||||||
|
index: Offset.CodePoint,
|
||||||
|
shift: Offset.CodePoint,
|
||||||
|
reps: List[TextReplacement]
|
||||||
|
): Option[Offset.CodePoint] =
|
||||||
|
reps match {
|
||||||
|
case Nil =>
|
||||||
|
if (index.toInt >= 0 && index <= originalLen)
|
||||||
|
Some(index + shift)
|
||||||
|
else
|
||||||
|
None
|
||||||
|
case (r @ TextReplacement(fr, to, _, _)) :: tail =>
|
||||||
|
if (index < fr) Some(index + shift)
|
||||||
|
else if (index < to) None
|
||||||
|
else reindex(index, shift + r.lengthDelta, tail)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
object TextReplacement {
|
||||||
|
def fromCodePoints(
|
||||||
|
originalFrom: Int,
|
||||||
|
originalTo: Int,
|
||||||
|
updatedFrom: Int,
|
||||||
|
updatedTo: Int
|
||||||
|
): TextReplacement =
|
||||||
|
TextReplacement(
|
||||||
|
Offset.CodePoint(originalFrom),
|
||||||
|
Offset.CodePoint(originalTo),
|
||||||
|
Offset.CodePoint(updatedFrom),
|
||||||
|
Offset.CodePoint(updatedTo)
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Encodes the indices of a segment of text in one string that maps to a replacement
|
||||||
|
* segment in an updated version of the text. The replacement segment could be empty
|
||||||
|
* (updatedTo == updatedFrom), indicating the segment was removed.
|
||||||
|
*
|
||||||
|
* All offsets are code-points, not UTF16 code-units.
|
||||||
|
*
|
||||||
|
* `originalFrom` and `updatedFrom` are inclusive.
|
||||||
|
* `originalTo` and `updatedTo` are exclusive.
|
||||||
|
*/
|
||||||
|
case class TextReplacement(
|
||||||
|
originalFrom: Offset.CodePoint,
|
||||||
|
originalTo: Offset.CodePoint,
|
||||||
|
updatedFrom: Offset.CodePoint,
|
||||||
|
updatedTo: Offset.CodePoint) {
|
||||||
|
def originalLength: Offset.CodePoint = originalTo - originalFrom
|
||||||
|
def updatedLength: Offset.CodePoint = updatedTo - updatedFrom
|
||||||
|
def lengthDelta: Offset.CodePoint = updatedLength - originalLength
|
||||||
|
|
||||||
|
def shiftOriginal(offset: Offset.CodePoint): TextReplacement =
|
||||||
|
copy(originalFrom = originalFrom + offset, originalTo = originalTo + offset)
|
||||||
|
|
||||||
|
def shiftUpdated(offset: Offset.CodePoint): TextReplacement =
|
||||||
|
copy(updatedFrom = updatedFrom + offset, updatedTo = updatedTo + offset)
|
||||||
|
|
||||||
|
def shift(offset: Offset.CodePoint): TextReplacement =
|
||||||
|
TextReplacement(
|
||||||
|
originalFrom + offset,
|
||||||
|
originalTo + offset,
|
||||||
|
updatedFrom + offset,
|
||||||
|
updatedTo + offset
|
||||||
|
)
|
||||||
|
|
||||||
|
def inverse: TextReplacement =
|
||||||
|
TextReplacement(
|
||||||
|
originalFrom = updatedFrom,
|
||||||
|
originalTo = updatedTo,
|
||||||
|
updatedFrom = originalFrom,
|
||||||
|
updatedTo = originalTo
|
||||||
|
)
|
||||||
|
}
|
@ -0,0 +1,159 @@
|
|||||||
|
package com.twitter.tweetypie.tweettext
|
||||||
|
|
||||||
|
import com.twitter.tweetypie.tweettext.TweetText._
|
||||||
|
import com.twitter.twittertext.Extractor
|
||||||
|
import java.lang.Character
|
||||||
|
import scala.annotation.tailrec
|
||||||
|
import scala.collection.JavaConverters._
|
||||||
|
|
||||||
|
object Truncator {
|
||||||
|
val Ellipsis = "\u2026"
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Truncate tweet text for a retweet. If the text is longer than
|
||||||
|
* either of the length limits, code points are cut off from the end
|
||||||
|
* of the text and replaced with an ellipsis. We keep as much of the
|
||||||
|
* leading text as possible, subject to these constraints:
|
||||||
|
*
|
||||||
|
* - There are no more than `MaxDisplayLength` characters.
|
||||||
|
*
|
||||||
|
* - When converted to UTF-8, the result does not exceed `MaxByteLength`.
|
||||||
|
*
|
||||||
|
* - We do not break within a single grapheme cluster.
|
||||||
|
*
|
||||||
|
* The input is assumed to be partial HTML-encoded and may or may
|
||||||
|
* not be NFC normalized. The result will be partial HTML-encoded
|
||||||
|
* and will be NFC normalized.
|
||||||
|
*/
|
||||||
|
def truncateForRetweet(input: String): String = truncateWithEllipsis(input, Ellipsis)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Truncate to [[com.twitter.tweetypie.tweettext.TweetText#OrginalMaxDisplayLength]] display
|
||||||
|
* units, using "..." as an ellipsis. The resulting text is guaranteed to pass our tweet length
|
||||||
|
* check, but it is not guaranteed to fit in a SMS message.
|
||||||
|
*/
|
||||||
|
def truncateForSms(input: String): String = truncateWithEllipsis(input, "...")
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Check the length of the given text, and truncate it if it is longer
|
||||||
|
* than the allowed length for a Tweet. The result of this method will
|
||||||
|
* always have:
|
||||||
|
*
|
||||||
|
* - Display length <= OriginalMaxDisplayLength.
|
||||||
|
* - Length when encoded as UTF-8 <= OriginalMaxUtf8Length.
|
||||||
|
*
|
||||||
|
* If the input would violate this, then the text will be
|
||||||
|
* truncated. When the text is truncated, it will be truncated such
|
||||||
|
* that:
|
||||||
|
*
|
||||||
|
* - Grapheme clusters will not be split.
|
||||||
|
* - The last character before the ellipsis will not be a whitespace
|
||||||
|
* character.
|
||||||
|
* - The ellipsis text will be appended to the end.
|
||||||
|
*/
|
||||||
|
private[this] def truncateWithEllipsis(input: String, ellipsis: String): String = {
|
||||||
|
val text = nfcNormalize(input)
|
||||||
|
val truncateAt =
|
||||||
|
truncationPoint(text, OriginalMaxDisplayLength, OriginalMaxUtf8Length, Some(ellipsis))
|
||||||
|
if (truncateAt.codeUnitOffset.toInt == text.length) text
|
||||||
|
else text.take(truncateAt.codeUnitOffset.toInt) + ellipsis
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Indicates a potential TruncationPoint in piece of text.
|
||||||
|
*
|
||||||
|
* @param charOffset the utf-16 character offset of the truncation point
|
||||||
|
* @param codePointOffset the offset in code points
|
||||||
|
*/
|
||||||
|
case class TruncationPoint(codeUnitOffset: Offset.CodeUnit, codePointOffset: Offset.CodePoint)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Computes a TruncationPoint for the given text and length constraints. If `truncated` on
|
||||||
|
* the result is `false`, it means the text will fit within the given constraints without
|
||||||
|
* truncation. Otherwise, the result indicates both the character and code-point offsets
|
||||||
|
* at which to perform the truncation, and the resulting display length and byte length of
|
||||||
|
* the truncated string.
|
||||||
|
*
|
||||||
|
* Text should be NFC normalized first for best results.
|
||||||
|
*
|
||||||
|
* @param withEllipsis if true, then the truncation point will be computed so that there is space
|
||||||
|
* to append an ellipsis and to still remain within the limits. The ellipsis is not counted
|
||||||
|
* in the returned display and byte lengths.
|
||||||
|
*
|
||||||
|
* @param atomicUnits may contain a list of ranges that should be treated as atomic unit and
|
||||||
|
* not split. each tuple is half-open range in code points.
|
||||||
|
*/
|
||||||
|
def truncationPoint(
|
||||||
|
text: String,
|
||||||
|
maxDisplayLength: Int = OriginalMaxDisplayLength,
|
||||||
|
maxByteLength: Int = OriginalMaxUtf8Length,
|
||||||
|
withEllipsis: Option[String] = None,
|
||||||
|
atomicUnits: Offset.Ranges[Offset.CodePoint] = Offset.Ranges.Empty
|
||||||
|
): TruncationPoint = {
|
||||||
|
val breakPoints =
|
||||||
|
GraphemeIndexIterator
|
||||||
|
.ends(text)
|
||||||
|
.filterNot(Offset.Ranges.htmlEntities(text).contains)
|
||||||
|
|
||||||
|
val ellipsisDisplayUnits =
|
||||||
|
withEllipsis.map(Offset.DisplayUnit.length).getOrElse(Offset.DisplayUnit(0))
|
||||||
|
val maxTruncatedDisplayLength = Offset.DisplayUnit(maxDisplayLength) - ellipsisDisplayUnits
|
||||||
|
|
||||||
|
val ellipsisByteLength = withEllipsis.map(Offset.Utf8.length).getOrElse(Offset.Utf8(0))
|
||||||
|
val maxTruncatedByteLength = Offset.Utf8(maxByteLength) - ellipsisByteLength
|
||||||
|
|
||||||
|
var codeUnit = Offset.CodeUnit(0)
|
||||||
|
var codePoint = Offset.CodePoint(0)
|
||||||
|
var displayLength = Offset.DisplayUnit(0)
|
||||||
|
var byteLength = Offset.Utf8(0)
|
||||||
|
var truncateCodeUnit = codeUnit
|
||||||
|
var truncateCodePoint = codePoint
|
||||||
|
|
||||||
|
@tailrec def go(): TruncationPoint =
|
||||||
|
if (displayLength.toInt > maxDisplayLength || byteLength.toInt > maxByteLength) {
|
||||||
|
TruncationPoint(truncateCodeUnit, truncateCodePoint)
|
||||||
|
} else if (codeUnit != truncateCodeUnit &&
|
||||||
|
displayLength <= maxTruncatedDisplayLength &&
|
||||||
|
byteLength <= maxTruncatedByteLength &&
|
||||||
|
(codeUnit.toInt == 0 || !Character.isWhitespace(text.codePointBefore(codeUnit.toInt))) &&
|
||||||
|
!atomicUnits.contains(codePoint)) {
|
||||||
|
// we can advance the truncation point
|
||||||
|
truncateCodeUnit = codeUnit
|
||||||
|
truncateCodePoint = codePoint
|
||||||
|
go()
|
||||||
|
} else if (breakPoints.hasNext) {
|
||||||
|
// there are further truncation points to consider
|
||||||
|
val nextCodeUnit = breakPoints.next
|
||||||
|
codePoint += Offset.CodePoint.count(text, codeUnit, nextCodeUnit)
|
||||||
|
displayLength += Offset.DisplayUnit.count(text, codeUnit, nextCodeUnit)
|
||||||
|
byteLength += Offset.Utf8.count(text, codeUnit, nextCodeUnit)
|
||||||
|
codeUnit = nextCodeUnit
|
||||||
|
go()
|
||||||
|
} else {
|
||||||
|
TruncationPoint(codeUnit, codePoint)
|
||||||
|
}
|
||||||
|
|
||||||
|
go()
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Truncate the given text, avoiding chopping HTML entities and tweet
|
||||||
|
* entities. This should only be used for testing because it performs
|
||||||
|
* entity extraction, and so is very inefficient.
|
||||||
|
*/
|
||||||
|
def truncateForTests(
|
||||||
|
input: String,
|
||||||
|
maxDisplayLength: Int = OriginalMaxDisplayLength,
|
||||||
|
maxByteLength: Int = OriginalMaxUtf8Length
|
||||||
|
): String = {
|
||||||
|
val text = nfcNormalize(input)
|
||||||
|
val extractor = new Extractor
|
||||||
|
val entities = extractor.extractEntitiesWithIndices(text)
|
||||||
|
extractor.modifyIndicesFromUTF16ToUnicode(text, entities)
|
||||||
|
val avoid = Offset.Ranges.fromCodePointPairs(
|
||||||
|
entities.asScala.map(e => (e.getStart().intValue, e.getEnd().intValue))
|
||||||
|
)
|
||||||
|
val truncateAt = truncationPoint(text, maxDisplayLength, maxByteLength, None, avoid)
|
||||||
|
text.take(truncateAt.codeUnitOffset.toInt)
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,62 @@
|
|||||||
|
package com.twitter.tweetypie.tweettext
|
||||||
|
|
||||||
|
import java.text.Normalizer
|
||||||
|
|
||||||
|
object TweetText {
|
||||||
|
|
||||||
|
/** The original maximum tweet length, taking into account normalization */
|
||||||
|
private[tweetypie] val OriginalMaxDisplayLength = 140
|
||||||
|
|
||||||
|
/** Maximum number of visible code points allowed in a tweet when tweet length is counted by code
|
||||||
|
* points, taking into account normalization. See also [[MaxVisibleWeightedEmojiLength]].
|
||||||
|
*/
|
||||||
|
private[tweetypie] val MaxVisibleWeightedLength = 280
|
||||||
|
|
||||||
|
/** Maximum number of visible code points allowed in a tweet when tweet length is counted by
|
||||||
|
* emoji, taking into account normalization. See also [[MaxVisibleWeightedLength]].
|
||||||
|
* 140 is the max number of Emojis, visible, fully-weighted per Twitter's cramming rules
|
||||||
|
* 10 is the max number of Code Points per Emoji
|
||||||
|
*/
|
||||||
|
private[tweetypie] val MaxVisibleWeightedEmojiLength = 140 * 10
|
||||||
|
|
||||||
|
/** Maximum number of bytes when truncating tweet text for a retweet. Originally was the
|
||||||
|
* max UTF-8 length when tweets were at most 140 characters.
|
||||||
|
* See also [[OriginalMaxDisplayLength]].
|
||||||
|
*/
|
||||||
|
private[tweetypie] val OriginalMaxUtf8Length = 600
|
||||||
|
|
||||||
|
/** Maximum number of bytes for tweet text using utf-8 encoding.
|
||||||
|
*/
|
||||||
|
private[tweetypie] val MaxUtf8Length = 5708
|
||||||
|
|
||||||
|
/** Maximum number of mentions allowed in tweet text. This is enforced at tweet creation time */
|
||||||
|
private[tweetypie] val MaxMentions = 50
|
||||||
|
|
||||||
|
/** Maximum number of urls allowed in tweet text. This is enforced at tweet creation time */
|
||||||
|
private[tweetypie] val MaxUrls = 10
|
||||||
|
|
||||||
|
/** Maximum number of hashtags allowed in tweet text. This is enforced at tweet creation time */
|
||||||
|
private[tweetypie] val MaxHashtags = 50
|
||||||
|
|
||||||
|
/** Maximum number of cashtags allowed in tweet text. This is enforced at tweet creation time */
|
||||||
|
private[tweetypie] val MaxCashtags = 50
|
||||||
|
|
||||||
|
/** Maximum length of a hashtag (not including the '#') */
|
||||||
|
private[tweetypie] val MaxHashtagLength = 100
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Normalizes the text according to the unicode NFC spec.
|
||||||
|
*/
|
||||||
|
def nfcNormalize(text: String): String = Normalizer.normalize(text, Normalizer.Form.NFC)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the number of "characters" in this text. See
|
||||||
|
* [[Offset.DisplayUnit]].
|
||||||
|
*/
|
||||||
|
def displayLength(text: String): Int = Offset.DisplayUnit.length(text).toInt
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the number of Unicode code points in this String.
|
||||||
|
*/
|
||||||
|
def codePointLength(text: String): Int = Offset.CodePoint.length(text).toInt
|
||||||
|
}
|
76
tweetypie/common/src/scala/com/twitter/tweetypie/util/BUILD
Normal file
76
tweetypie/common/src/scala/com/twitter/tweetypie/util/BUILD
Normal file
@ -0,0 +1,76 @@
|
|||||||
|
scala_library(
|
||||||
|
sources = ["*.scala"],
|
||||||
|
compiler_option_sets = ["fatal_warnings"],
|
||||||
|
platform = "java8",
|
||||||
|
provides = scala_artifact(
|
||||||
|
org = "com.twitter.tweetypie",
|
||||||
|
name = "util",
|
||||||
|
repo = artifactory,
|
||||||
|
),
|
||||||
|
strict_deps = True,
|
||||||
|
tags = ["bazel-compatible"],
|
||||||
|
dependencies = [
|
||||||
|
"//:scala-reflect",
|
||||||
|
"3rdparty/jvm/commons-codec",
|
||||||
|
"3rdparty/jvm/org/apache/thrift:libthrift",
|
||||||
|
"finagle/finagle-core/src/main",
|
||||||
|
"mediaservices/commons/src/main/thrift:thrift-scala",
|
||||||
|
"scrooge/scrooge-serializer/src/main/scala",
|
||||||
|
"tweetypie/servo/repo",
|
||||||
|
"tweetypie/servo/util",
|
||||||
|
"tweetypie/servo/util/src/main/scala:exception",
|
||||||
|
"src/scala/com/twitter/takedown/util",
|
||||||
|
"src/thrift/com/twitter/dataproducts:enrichments_profilegeo-scala",
|
||||||
|
"src/thrift/com/twitter/escherbird:media-annotation-structs-scala",
|
||||||
|
"src/thrift/com/twitter/expandodo:cards-scala",
|
||||||
|
"src/thrift/com/twitter/gizmoduck:thrift-scala",
|
||||||
|
"src/thrift/com/twitter/servo:servo-exception-scala",
|
||||||
|
"src/thrift/com/twitter/spam/rtf:safety-label-scala",
|
||||||
|
"tweetypie/common/src/thrift/com/twitter/tweetypie:deprecated-scala",
|
||||||
|
"tweetypie/common/src/thrift/com/twitter/tweetypie:service-scala",
|
||||||
|
"tweetypie/common/src/thrift/com/twitter/tweetypie:transient_context-scala",
|
||||||
|
"tweetypie/common/src/thrift/com/twitter/tweetypie:tweet-scala",
|
||||||
|
"stitch/stitch-core",
|
||||||
|
"tweet-util",
|
||||||
|
"util/util-core:scala",
|
||||||
|
],
|
||||||
|
)
|
||||||
|
|
||||||
|
scala_library(
|
||||||
|
name = "EditControlUtil",
|
||||||
|
sources = [
|
||||||
|
"EditControlUtil.scala",
|
||||||
|
"package.scala",
|
||||||
|
],
|
||||||
|
compiler_option_sets = ["fatal_warnings"],
|
||||||
|
platform = "java8",
|
||||||
|
provides = scala_artifact(
|
||||||
|
org = "com.twitter.tweetypie",
|
||||||
|
name = "util-EditControlUtil",
|
||||||
|
repo = artifactory,
|
||||||
|
),
|
||||||
|
strict_deps = True,
|
||||||
|
tags = ["bazel-compatible"],
|
||||||
|
dependencies = [
|
||||||
|
"//:scala-reflect",
|
||||||
|
"3rdparty/jvm/commons-codec",
|
||||||
|
"3rdparty/jvm/org/apache/thrift:libthrift",
|
||||||
|
"finagle/finagle-core/src/main",
|
||||||
|
"mediaservices/commons/src/main/thrift:thrift-scala",
|
||||||
|
"scrooge/scrooge-serializer/src/main/scala",
|
||||||
|
"tweetypie/servo/util/src/main/scala:exception",
|
||||||
|
"src/thrift/com/twitter/dataproducts:enrichments_profilegeo-scala",
|
||||||
|
"src/thrift/com/twitter/escherbird:media-annotation-structs-scala",
|
||||||
|
"src/thrift/com/twitter/expandodo:cards-scala",
|
||||||
|
"src/thrift/com/twitter/gizmoduck:thrift-scala",
|
||||||
|
"src/thrift/com/twitter/servo:servo-exception-scala",
|
||||||
|
"src/thrift/com/twitter/spam/rtf:safety-label-scala",
|
||||||
|
"tweetypie/common/src/thrift/com/twitter/tweetypie:deprecated-scala",
|
||||||
|
"tweetypie/common/src/thrift/com/twitter/tweetypie:service-scala",
|
||||||
|
"tweetypie/common/src/thrift/com/twitter/tweetypie:transient_context-scala",
|
||||||
|
"tweetypie/common/src/thrift/com/twitter/tweetypie:tweet-scala",
|
||||||
|
"stitch/stitch-core",
|
||||||
|
"tweet-util",
|
||||||
|
"util/util-core:scala",
|
||||||
|
],
|
||||||
|
)
|
@ -0,0 +1,29 @@
|
|||||||
|
package com.twitter.tweetypie.util
|
||||||
|
|
||||||
|
import com.twitter.escherbird.thriftscala.TweetEntityAnnotation
|
||||||
|
import com.twitter.tweetypie.thriftscala.EscherbirdEntityAnnotations
|
||||||
|
import com.twitter.tweetypie.thriftscala.Tweet
|
||||||
|
|
||||||
|
object CommunityAnnotation {
|
||||||
|
|
||||||
|
val groupId: Long = 8
|
||||||
|
val domainId: Long = 31
|
||||||
|
|
||||||
|
def apply(communityId: Long): TweetEntityAnnotation =
|
||||||
|
TweetEntityAnnotation(groupId, domainId, entityId = communityId)
|
||||||
|
|
||||||
|
def unapply(annotation: TweetEntityAnnotation): Option[Long] =
|
||||||
|
annotation match {
|
||||||
|
case TweetEntityAnnotation(`groupId`, `domainId`, entityId) => Some(entityId)
|
||||||
|
case _ => None
|
||||||
|
}
|
||||||
|
|
||||||
|
// Returns None instead of Some(Seq()) when there are non-community annotations present
|
||||||
|
def additionalFieldsToCommunityIDs(additionalFields: Tweet): Option[Seq[Long]] = {
|
||||||
|
additionalFields.escherbirdEntityAnnotations
|
||||||
|
.map {
|
||||||
|
case EscherbirdEntityAnnotations(entityAnnotations) =>
|
||||||
|
entityAnnotations.flatMap(CommunityAnnotation.unapply)
|
||||||
|
}.filter(_.nonEmpty)
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,19 @@
|
|||||||
|
package com.twitter.tweetypie.util
|
||||||
|
|
||||||
|
import com.twitter.tweetypie.thriftscala.Communities
|
||||||
|
|
||||||
|
object CommunityUtil {
|
||||||
|
|
||||||
|
def communityIds(maybeCommunities: Option[Communities]): Seq[Long] = {
|
||||||
|
maybeCommunities match {
|
||||||
|
case None =>
|
||||||
|
Nil
|
||||||
|
case Some(Communities(seq)) =>
|
||||||
|
seq
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
def hasCommunity(maybeCommunities: Option[Communities]): Boolean = {
|
||||||
|
maybeCommunities.exists(_.communityIds.nonEmpty)
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,112 @@
|
|||||||
|
package com.twitter.tweetypie
|
||||||
|
package util
|
||||||
|
|
||||||
|
import com.twitter.tweetypie.thriftscala._
|
||||||
|
|
||||||
|
object ConversationControls {
|
||||||
|
object Create {
|
||||||
|
def byInvitation(
|
||||||
|
inviteViaMention: Option[Boolean] = None
|
||||||
|
): TweetCreateConversationControl.ByInvitation = TweetCreateConversationControl.ByInvitation(
|
||||||
|
TweetCreateConversationControlByInvitation(inviteViaMention = inviteViaMention)
|
||||||
|
)
|
||||||
|
|
||||||
|
def community(
|
||||||
|
inviteViaMention: Option[Boolean] = None
|
||||||
|
): TweetCreateConversationControl.Community = TweetCreateConversationControl.Community(
|
||||||
|
TweetCreateConversationControlCommunity(inviteViaMention = inviteViaMention)
|
||||||
|
)
|
||||||
|
|
||||||
|
def followers(
|
||||||
|
inviteViaMention: Option[Boolean] = None
|
||||||
|
): TweetCreateConversationControl.Followers = TweetCreateConversationControl.Followers(
|
||||||
|
TweetCreateConversationControlFollowers(inviteViaMention = inviteViaMention)
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
|
object Scenario {
|
||||||
|
case class CommonScenario(
|
||||||
|
createConversationControl: TweetCreateConversationControl,
|
||||||
|
descriptionSuffix: String,
|
||||||
|
expectedConversationControl: (UserId, Seq[UserId]) => ConversationControl,
|
||||||
|
inviteViaMention: Option[Boolean])
|
||||||
|
|
||||||
|
def mkCommunityScenario(inviteViaMention: Option[Boolean]): CommonScenario =
|
||||||
|
CommonScenario(
|
||||||
|
Create.community(inviteViaMention = inviteViaMention),
|
||||||
|
"community",
|
||||||
|
expectedConversationControl = (authorId, userIds) => {
|
||||||
|
community(userIds, authorId, inviteViaMention)
|
||||||
|
},
|
||||||
|
inviteViaMention
|
||||||
|
)
|
||||||
|
|
||||||
|
def mkByInvitationScenario(inviteViaMention: Option[Boolean]): CommonScenario =
|
||||||
|
CommonScenario(
|
||||||
|
Create.byInvitation(inviteViaMention = inviteViaMention),
|
||||||
|
"invited users",
|
||||||
|
expectedConversationControl = (authorId, userIds) => {
|
||||||
|
byInvitation(userIds, authorId, inviteViaMention)
|
||||||
|
},
|
||||||
|
inviteViaMention
|
||||||
|
)
|
||||||
|
|
||||||
|
def mkFollowersScenario(inviteViaMention: Option[Boolean]): CommonScenario =
|
||||||
|
CommonScenario(
|
||||||
|
Create.followers(inviteViaMention = inviteViaMention),
|
||||||
|
"followers",
|
||||||
|
expectedConversationControl = (authorId, userIds) => {
|
||||||
|
followers(userIds, authorId, inviteViaMention)
|
||||||
|
},
|
||||||
|
inviteViaMention
|
||||||
|
)
|
||||||
|
|
||||||
|
val communityScenario = mkCommunityScenario(None)
|
||||||
|
val communityInviteViaMentionScenario = mkCommunityScenario(Some(true))
|
||||||
|
|
||||||
|
val byInvitationScenario = mkByInvitationScenario(None)
|
||||||
|
val byInvitationInviteViaMentionScenario = mkByInvitationScenario(Some(true))
|
||||||
|
|
||||||
|
val followersScenario = mkFollowersScenario(None)
|
||||||
|
val followersInviteViaMentionScenario = mkFollowersScenario(Some(true))
|
||||||
|
}
|
||||||
|
|
||||||
|
def byInvitation(
|
||||||
|
invitedUserIds: Seq[UserId],
|
||||||
|
conversationTweetAuthorId: UserId,
|
||||||
|
inviteViaMention: Option[Boolean] = None
|
||||||
|
): ConversationControl =
|
||||||
|
ConversationControl.ByInvitation(
|
||||||
|
ConversationControlByInvitation(
|
||||||
|
conversationTweetAuthorId = conversationTweetAuthorId,
|
||||||
|
invitedUserIds = invitedUserIds,
|
||||||
|
inviteViaMention = inviteViaMention
|
||||||
|
)
|
||||||
|
)
|
||||||
|
|
||||||
|
def community(
|
||||||
|
invitedUserIds: Seq[UserId],
|
||||||
|
conversationTweetAuthorId: UserId,
|
||||||
|
inviteViaMention: Option[Boolean] = None
|
||||||
|
): ConversationControl =
|
||||||
|
ConversationControl.Community(
|
||||||
|
ConversationControlCommunity(
|
||||||
|
conversationTweetAuthorId = conversationTweetAuthorId,
|
||||||
|
invitedUserIds = invitedUserIds,
|
||||||
|
inviteViaMention = inviteViaMention
|
||||||
|
)
|
||||||
|
)
|
||||||
|
|
||||||
|
def followers(
|
||||||
|
invitedUserIds: Seq[UserId],
|
||||||
|
conversationTweetAuthorId: UserId,
|
||||||
|
inviteViaMention: Option[Boolean] = None
|
||||||
|
): ConversationControl =
|
||||||
|
ConversationControl.Followers(
|
||||||
|
ConversationControlFollowers(
|
||||||
|
conversationTweetAuthorId = conversationTweetAuthorId,
|
||||||
|
invitedUserIds = invitedUserIds,
|
||||||
|
inviteViaMention = inviteViaMention
|
||||||
|
)
|
||||||
|
)
|
||||||
|
}
|
@ -0,0 +1,174 @@
|
|||||||
|
package com.twitter.tweetypie.util
|
||||||
|
|
||||||
|
import com.twitter.servo.util.Gate
|
||||||
|
import com.twitter.tweetypie.util.TweetEditFailure.TweetEditInvalidEditControlException
|
||||||
|
import com.twitter.tweetypie.util.TweetEditFailure.TweetEditUpdateEditControlException
|
||||||
|
import com.twitter.tweetypie.thriftscala.EditControl
|
||||||
|
import com.twitter.tweetypie.thriftscala.EditControlEdit
|
||||||
|
import com.twitter.tweetypie.thriftscala.EditControlInitial
|
||||||
|
import com.twitter.tweetypie.thriftscala.Tweet
|
||||||
|
import com.twitter.util.Try
|
||||||
|
import com.twitter.util.Return
|
||||||
|
import com.twitter.util.Throw
|
||||||
|
import com.twitter.util.Time
|
||||||
|
import com.twitter.util.Duration
|
||||||
|
|
||||||
|
object EditControlUtil {
|
||||||
|
|
||||||
|
val maxTweetEditsAllowed = 5
|
||||||
|
val oldEditTimeWindow = Duration.fromMinutes(30)
|
||||||
|
val editTimeWindow = Duration.fromMinutes(60)
|
||||||
|
|
||||||
|
def editControlEdit(
|
||||||
|
initialTweetId: TweetId,
|
||||||
|
editControlInitial: Option[EditControlInitial] = None
|
||||||
|
): EditControl.Edit =
|
||||||
|
EditControl.Edit(
|
||||||
|
EditControlEdit(initialTweetId = initialTweetId, editControlInitial = editControlInitial))
|
||||||
|
|
||||||
|
// EditControl for the tweet that is not an edit, that is, any regular tweet we create
|
||||||
|
// that can, potentially, be edited later.
|
||||||
|
def makeEditControlInitial(
|
||||||
|
tweetId: TweetId,
|
||||||
|
createdAt: Time,
|
||||||
|
setEditWindowToSixtyMinutes: Gate[Unit] = Gate(_ => false)
|
||||||
|
): EditControl.Initial = {
|
||||||
|
val editWindow = if (setEditWindowToSixtyMinutes()) editTimeWindow else oldEditTimeWindow
|
||||||
|
val initial = EditControlInitial(
|
||||||
|
editTweetIds = Seq(tweetId),
|
||||||
|
editableUntilMsecs = Some(createdAt.plus(editWindow).inMilliseconds),
|
||||||
|
editsRemaining = Some(maxTweetEditsAllowed),
|
||||||
|
isEditEligible = defaultIsEditEligible,
|
||||||
|
)
|
||||||
|
EditControl.Initial(initial)
|
||||||
|
}
|
||||||
|
|
||||||
|
// Returns if a given latestTweetId is the latest edit in the EditControl
|
||||||
|
def isLatestEdit(
|
||||||
|
tweetEditControl: Option[EditControl],
|
||||||
|
latestTweetId: TweetId
|
||||||
|
): Try[Boolean] = {
|
||||||
|
tweetEditControl match {
|
||||||
|
case Some(EditControl.Initial(initial)) =>
|
||||||
|
isLatestEditFromEditControlInitial(Some(initial), latestTweetId)
|
||||||
|
case Some(EditControl.Edit(edit)) =>
|
||||||
|
isLatestEditFromEditControlInitial(
|
||||||
|
edit.editControlInitial,
|
||||||
|
latestTweetId
|
||||||
|
)
|
||||||
|
case _ => Throw(TweetEditInvalidEditControlException)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Returns if a given latestTweetId is the latest edit in the EditControlInitial
|
||||||
|
private def isLatestEditFromEditControlInitial(
|
||||||
|
initialTweetEditControl: Option[EditControlInitial],
|
||||||
|
latestTweetId: TweetId
|
||||||
|
): Try[Boolean] = {
|
||||||
|
initialTweetEditControl match {
|
||||||
|
case Some(initial) =>
|
||||||
|
Return(latestTweetId == initial.editTweetIds.last)
|
||||||
|
case _ => Throw(TweetEditInvalidEditControlException)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/* Create an updated edit control for an initialTweet given the id of the new edit */
|
||||||
|
def editControlForInitialTweet(
|
||||||
|
initialTweet: Tweet,
|
||||||
|
newEditId: TweetId
|
||||||
|
): Try[EditControl.Initial] = {
|
||||||
|
initialTweet.editControl match {
|
||||||
|
case Some(EditControl.Initial(initial)) =>
|
||||||
|
Return(EditControl.Initial(plusEdit(initial, newEditId)))
|
||||||
|
|
||||||
|
case Some(EditControl.Edit(_)) => Throw(TweetEditUpdateEditControlException)
|
||||||
|
|
||||||
|
case _ =>
|
||||||
|
initialTweet.coreData match {
|
||||||
|
case Some(coreData) =>
|
||||||
|
Return(
|
||||||
|
makeEditControlInitial(
|
||||||
|
tweetId = initialTweet.id,
|
||||||
|
createdAt = Time.fromMilliseconds(coreData.createdAtSecs * 1000),
|
||||||
|
setEditWindowToSixtyMinutes = Gate(_ => true)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
case None => Throw(new Exception("Tweet Missing Required CoreData"))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
def updateEditControl(tweet: Tweet, newEditId: TweetId): Try[Tweet] =
|
||||||
|
editControlForInitialTweet(tweet, newEditId).map { editControl =>
|
||||||
|
tweet.copy(editControl = Some(editControl))
|
||||||
|
}
|
||||||
|
|
||||||
|
def plusEdit(initial: EditControlInitial, newEditId: TweetId): EditControlInitial = {
|
||||||
|
val newEditTweetIds = (initial.editTweetIds :+ newEditId).distinct.sorted
|
||||||
|
val editsCount = newEditTweetIds.size - 1 // as there is the original tweet ID there too.
|
||||||
|
initial.copy(
|
||||||
|
editTweetIds = newEditTweetIds,
|
||||||
|
editsRemaining = Some(maxTweetEditsAllowed - editsCount),
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
|
// The ID of the initial Tweet if this is an edit
|
||||||
|
def getInitialTweetIdIfEdit(tweet: Tweet): Option[TweetId] = tweet.editControl match {
|
||||||
|
case Some(EditControl.Edit(edit)) => Some(edit.initialTweetId)
|
||||||
|
case _ => None
|
||||||
|
}
|
||||||
|
|
||||||
|
// If this is the first tweet in an edit chain, return the same tweet id
|
||||||
|
// otherwise return the result of getInitialTweetId
|
||||||
|
def getInitialTweetId(tweet: Tweet): TweetId =
|
||||||
|
getInitialTweetIdIfEdit(tweet).getOrElse(tweet.id)
|
||||||
|
|
||||||
|
def isInitialTweet(tweet: Tweet): Boolean =
|
||||||
|
getInitialTweetId(tweet) == tweet.id
|
||||||
|
|
||||||
|
// Extracted just so that we can easily track where the values of isEditEligible is coming from.
|
||||||
|
private def defaultIsEditEligible: Option[Boolean] = Some(true)
|
||||||
|
|
||||||
|
// returns true if it's an edit of a Tweet or an initial Tweet that's been edited
|
||||||
|
def isEditTweet(tweet: Tweet): Boolean =
|
||||||
|
tweet.editControl match {
|
||||||
|
case Some(eci: EditControl.Initial) if eci.initial.editTweetIds.size <= 1 => false
|
||||||
|
case Some(_: EditControl.Initial) | Some(_: EditControl.Edit) | Some(
|
||||||
|
EditControl.UnknownUnionField(_)) =>
|
||||||
|
true
|
||||||
|
case None => false
|
||||||
|
}
|
||||||
|
|
||||||
|
// returns true if editControl is from an edit of a Tweet
|
||||||
|
// returns false for any other state, including edit intial.
|
||||||
|
def isEditControlEdit(editControl: EditControl): Boolean = {
|
||||||
|
editControl match {
|
||||||
|
case _: EditControl.Edit | EditControl.UnknownUnionField(_) => true
|
||||||
|
case _ => false
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
def getEditTweetIds(editControl: Option[EditControl]): Try[Seq[TweetId]] = {
|
||||||
|
editControl match {
|
||||||
|
case Some(EditControl.Edit(EditControlEdit(_, Some(eci)))) =>
|
||||||
|
Return(eci.editTweetIds)
|
||||||
|
case Some(EditControl.Initial(initial)) =>
|
||||||
|
Return(initial.editTweetIds)
|
||||||
|
case _ =>
|
||||||
|
Throw(new Exception(s"EditControlInitial not found in $editControl"))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
object TweetEditFailure {
|
||||||
|
abstract class TweetEditException(msg: String) extends Exception(msg)
|
||||||
|
|
||||||
|
case object TweetEditGetInitialEditControlException
|
||||||
|
extends TweetEditException("Initial EditControl not found")
|
||||||
|
|
||||||
|
case object TweetEditInvalidEditControlException
|
||||||
|
extends TweetEditException("Invalid EditControl for initial_tweet")
|
||||||
|
|
||||||
|
case object TweetEditUpdateEditControlException
|
||||||
|
extends TweetEditException("Invalid Edit Control Update")
|
||||||
|
}
|
@ -0,0 +1,45 @@
|
|||||||
|
package com.twitter.tweetypie.util
|
||||||
|
|
||||||
|
import com.twitter.finagle.Backoff
|
||||||
|
import com.twitter.finagle.service.RetryPolicy
|
||||||
|
import com.twitter.finagle.service.RetryPolicy.RetryableWriteException
|
||||||
|
import com.twitter.servo.exception.thriftscala.ServerError
|
||||||
|
import com.twitter.util.Duration
|
||||||
|
import com.twitter.util.Throw
|
||||||
|
import com.twitter.util.TimeoutException
|
||||||
|
import com.twitter.util.Try
|
||||||
|
|
||||||
|
object RetryPolicyBuilder {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Retry on any exception.
|
||||||
|
*/
|
||||||
|
def anyFailure[A](backoffs: Stream[Duration]): RetryPolicy[Try[A]] =
|
||||||
|
RetryPolicy.backoff[Try[A]](Backoff.fromStream(backoffs)) {
|
||||||
|
case Throw(_) => true
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Retry on com.twitter.util.TimeoutException
|
||||||
|
*/
|
||||||
|
def timeouts[A](backoffs: Stream[Duration]): RetryPolicy[Try[A]] =
|
||||||
|
RetryPolicy.backoff[Try[A]](Backoff.fromStream(backoffs)) {
|
||||||
|
case Throw(_: TimeoutException) => true
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Retry on com.twitter.finagle.service.RetryableWriteExceptions
|
||||||
|
*/
|
||||||
|
def writes[A](backoffs: Stream[Duration]): RetryPolicy[Try[A]] =
|
||||||
|
RetryPolicy.backoff[Try[A]](Backoff.fromStream(backoffs)) {
|
||||||
|
case Throw(RetryableWriteException(_)) => true
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Retry on com.twitter.servo.exception.thriftscala.ServerError
|
||||||
|
*/
|
||||||
|
def servoServerError[A](backoffs: Stream[Duration]): RetryPolicy[Try[A]] =
|
||||||
|
RetryPolicy.backoff[Try[A]](Backoff.fromStream(backoffs)) {
|
||||||
|
case Throw(ServerError(_)) => true
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,54 @@
|
|||||||
|
package com.twitter.tweetypie.util
|
||||||
|
|
||||||
|
import com.twitter.finagle.stats.Stat
|
||||||
|
import com.twitter.finagle.stats.StatsReceiver
|
||||||
|
import com.twitter.servo
|
||||||
|
import com.twitter.util.Return
|
||||||
|
import com.twitter.util.Throw
|
||||||
|
import com.twitter.stitch.Stitch
|
||||||
|
|
||||||
|
object StitchUtils {
|
||||||
|
def trackLatency[T](latencyStat: Stat, s: => Stitch[T]): Stitch[T] = {
|
||||||
|
Stitch
|
||||||
|
.time(s)
|
||||||
|
.map {
|
||||||
|
case (res, duration) =>
|
||||||
|
latencyStat.add(duration.inMillis)
|
||||||
|
res
|
||||||
|
}
|
||||||
|
.lowerFromTry
|
||||||
|
}
|
||||||
|
|
||||||
|
def observe[T](statsReceiver: StatsReceiver, apiName: String): Stitch[T] => Stitch[T] = {
|
||||||
|
val stats = statsReceiver.scope(apiName)
|
||||||
|
|
||||||
|
val requests = stats.counter("requests")
|
||||||
|
val success = stats.counter("success")
|
||||||
|
val latencyStat = stats.stat("latency_ms")
|
||||||
|
|
||||||
|
val exceptionCounter =
|
||||||
|
new servo.util.ExceptionCounter(stats, "failures")
|
||||||
|
|
||||||
|
stitch =>
|
||||||
|
trackLatency(latencyStat, stitch)
|
||||||
|
.respond {
|
||||||
|
case Return(_) =>
|
||||||
|
requests.incr()
|
||||||
|
success.incr()
|
||||||
|
|
||||||
|
case Throw(e) =>
|
||||||
|
exceptionCounter(e)
|
||||||
|
requests.incr()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
def translateExceptions[T](
|
||||||
|
stitch: Stitch[T],
|
||||||
|
translateException: PartialFunction[Throwable, Throwable]
|
||||||
|
): Stitch[T] =
|
||||||
|
stitch.rescue {
|
||||||
|
case t if translateException.isDefinedAt(t) =>
|
||||||
|
Stitch.exception(translateException(t))
|
||||||
|
case t => Stitch.exception(t)
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,31 @@
|
|||||||
|
package com.twitter.tweetypie.util
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Escape a String into Java or Scala String literal syntax (adds the
|
||||||
|
* surrounding quotes.)
|
||||||
|
*
|
||||||
|
* This is primarily for printing Strings for debugging or logging.
|
||||||
|
*/
|
||||||
|
object StringLiteral extends (String => String) {
|
||||||
|
private[this] val ControlLimit = ' '
|
||||||
|
private[this] val PrintableLimit = '\u007e'
|
||||||
|
private[this] val Specials =
|
||||||
|
Map('\n' -> 'n', '\r' -> 'r', '\t' -> 't', '"' -> '"', '\'' -> '\'', '\\' -> '\\')
|
||||||
|
|
||||||
|
def apply(str: String): String = {
|
||||||
|
val s = new StringBuilder(str.length)
|
||||||
|
s.append('"')
|
||||||
|
var i = 0
|
||||||
|
while (i < str.length) {
|
||||||
|
val c = str(i)
|
||||||
|
Specials.get(c) match {
|
||||||
|
case None =>
|
||||||
|
if (c >= ControlLimit && c <= PrintableLimit) s.append(c)
|
||||||
|
else s.append("\\u%04x".format(c.toInt))
|
||||||
|
case Some(special) => s.append('\\').append(special)
|
||||||
|
}
|
||||||
|
i += 1
|
||||||
|
}
|
||||||
|
s.append('"').result
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,49 @@
|
|||||||
|
package com.twitter.tweetypie.util
|
||||||
|
|
||||||
|
import com.twitter.takedown.util.TakedownReasons
|
||||||
|
import com.twitter.takedown.util.TakedownReasons.CountryCode
|
||||||
|
import com.twitter.tseng.withholding.thriftscala.TakedownReason
|
||||||
|
import com.twitter.tseng.withholding.thriftscala.UnspecifiedReason
|
||||||
|
import com.twitter.tweetypie.thriftscala.Tweet
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Contains tweetypie-specific utils for working with TakedownReasons.
|
||||||
|
*/
|
||||||
|
object Takedowns {
|
||||||
|
|
||||||
|
type CountryCode = String
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Take a list of [[TakedownReason]] and return values to be saved on the [[Tweet]] in fields
|
||||||
|
* tweetypieOnlyTakedownCountryCode and tweetypieOnlyTakedownReason.
|
||||||
|
*
|
||||||
|
* - tweetypieOnlyTakedownCountryCode contains the country_code of all UnspecifiedReasons
|
||||||
|
* - tweetypieOnlyTakedownReason contains all other reasons
|
||||||
|
*/
|
||||||
|
def partitionReasons(reasons: Seq[TakedownReason]): (Seq[String], Seq[TakedownReason]) = {
|
||||||
|
val (unspecifiedReasons, specifiedReasons) = reasons.partition {
|
||||||
|
case TakedownReason.UnspecifiedReason(UnspecifiedReason(_)) => true
|
||||||
|
case _ => false
|
||||||
|
}
|
||||||
|
val unspecifiedCountryCodes = unspecifiedReasons.collect(TakedownReasons.reasonToCountryCode)
|
||||||
|
(unspecifiedCountryCodes, specifiedReasons)
|
||||||
|
}
|
||||||
|
|
||||||
|
def fromTweet(t: Tweet): Takedowns =
|
||||||
|
Takedowns(
|
||||||
|
Seq
|
||||||
|
.concat(
|
||||||
|
t.tweetypieOnlyTakedownCountryCodes
|
||||||
|
.getOrElse(Nil).map(TakedownReasons.countryCodeToReason),
|
||||||
|
t.tweetypieOnlyTakedownReasons.getOrElse(Nil)
|
||||||
|
).toSet
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This class is used to ensure the caller has access to both the full list of reasons as well
|
||||||
|
* as the backwards-compatible list of country codes.
|
||||||
|
*/
|
||||||
|
case class Takedowns(reasons: Set[TakedownReason]) {
|
||||||
|
def countryCodes: Set[CountryCode] = reasons.collect(TakedownReasons.reasonToCountryCode)
|
||||||
|
}
|
@ -0,0 +1,17 @@
|
|||||||
|
package com.twitter.tweetypie.util
|
||||||
|
|
||||||
|
import com.twitter.tweetypie.thriftscala.TransientCreateContext
|
||||||
|
import com.twitter.tweetypie.thriftscala.TweetCreateContextKey
|
||||||
|
import com.twitter.tweetypie.thriftscala.TweetCreateContextKey.PeriscopeCreatorId
|
||||||
|
import com.twitter.tweetypie.thriftscala.TweetCreateContextKey.PeriscopeIsLive
|
||||||
|
|
||||||
|
object TransientContextUtil {
|
||||||
|
|
||||||
|
def toAdditionalContext(context: TransientCreateContext): Map[TweetCreateContextKey, String] =
|
||||||
|
Seq
|
||||||
|
.concat(
|
||||||
|
context.periscopeIsLive.map(PeriscopeIsLive -> _.toString), // "true" or "false"
|
||||||
|
context.periscopeCreatorId.map(PeriscopeCreatorId -> _.toString) // userId
|
||||||
|
)
|
||||||
|
.toMap
|
||||||
|
}
|
@ -0,0 +1,203 @@
|
|||||||
|
package com.twitter.tweetypie.util
|
||||||
|
|
||||||
|
import com.twitter.conversions.DurationOps._
|
||||||
|
import com.twitter.logging.Logger
|
||||||
|
import com.twitter.mediaservices.commons.mediainformation.thriftscala.UserDefinedProductMetadata
|
||||||
|
import com.twitter.scrooge.BinaryThriftStructSerializer
|
||||||
|
import com.twitter.servo.cache.ScopedCacheKey
|
||||||
|
import com.twitter.servo.util.Transformer
|
||||||
|
import com.twitter.tweetypie.thriftscala.PostTweetRequest
|
||||||
|
import com.twitter.util.Base64Long
|
||||||
|
import com.twitter.util.Time
|
||||||
|
import java.nio.ByteBuffer
|
||||||
|
import java.security.MessageDigest
|
||||||
|
import org.apache.commons.codec.binary.Base64
|
||||||
|
import scala.collection.immutable.SortedMap
|
||||||
|
|
||||||
|
object TweetCreationLock {
|
||||||
|
case class Key private (userId: UserId, typeCode: String, idOrMd5: String)
|
||||||
|
extends ScopedCacheKey("t", "locker", 2, Base64Long.toBase64(userId), typeCode, idOrMd5) {
|
||||||
|
def uniquenessId: Option[String] =
|
||||||
|
if (typeCode == Key.TypeCode.UniquenessId) Some(idOrMd5) else None
|
||||||
|
}
|
||||||
|
|
||||||
|
object Key {
|
||||||
|
private[this] val log = Logger(getClass)
|
||||||
|
|
||||||
|
object TypeCode {
|
||||||
|
val SourceTweetId = "r"
|
||||||
|
val UniquenessId = "u"
|
||||||
|
val PostTweetRequest = "p"
|
||||||
|
}
|
||||||
|
|
||||||
|
private[this] val serializer = BinaryThriftStructSerializer(PostTweetRequest)
|
||||||
|
|
||||||
|
// normalize the representation of no media ids.
|
||||||
|
private[util] def sanitizeMediaUploadIds(mediaUploadIds: Option[Seq[Long]]) =
|
||||||
|
mediaUploadIds.filter(_.nonEmpty)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Request deduplication depends on the hash of a serialized Thrift value.
|
||||||
|
*
|
||||||
|
* In order to guarantee that a Map has a reproducible serialized form,
|
||||||
|
* it's necessary to fix the ordering of its keys.
|
||||||
|
*/
|
||||||
|
private[util] def sanitizeMediaMetadata(
|
||||||
|
mediaMetadata: Option[scala.collection.Map[MediaId, UserDefinedProductMetadata]]
|
||||||
|
): Option[scala.collection.Map[MediaId, UserDefinedProductMetadata]] =
|
||||||
|
mediaMetadata.map(m => SortedMap(m.toSeq: _*))
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Make sure to sanitize request fields with map/set since serialized
|
||||||
|
* bytes ordering is not guaranteed for same thrift values.
|
||||||
|
*/
|
||||||
|
private[util] def sanitizeRequest(request: PostTweetRequest): PostTweetRequest =
|
||||||
|
PostTweetRequest(
|
||||||
|
userId = request.userId,
|
||||||
|
text = request.text,
|
||||||
|
createdVia = "",
|
||||||
|
inReplyToTweetId = request.inReplyToTweetId,
|
||||||
|
geo = request.geo,
|
||||||
|
mediaUploadIds = sanitizeMediaUploadIds(request.mediaUploadIds),
|
||||||
|
narrowcast = request.narrowcast,
|
||||||
|
nullcast = request.nullcast,
|
||||||
|
additionalFields = request.additionalFields,
|
||||||
|
attachmentUrl = request.attachmentUrl,
|
||||||
|
mediaMetadata = sanitizeMediaMetadata(request.mediaMetadata),
|
||||||
|
conversationControl = request.conversationControl,
|
||||||
|
underlyingCreativesContainerId = request.underlyingCreativesContainerId,
|
||||||
|
editOptions = request.editOptions,
|
||||||
|
noteTweetOptions = request.noteTweetOptions
|
||||||
|
)
|
||||||
|
|
||||||
|
def bySourceTweetId(userId: UserId, sourceTweetId: TweetId): Key =
|
||||||
|
Key(userId, TypeCode.SourceTweetId, Base64Long.toBase64(sourceTweetId))
|
||||||
|
|
||||||
|
def byRequest(request: PostTweetRequest): Key =
|
||||||
|
request.uniquenessId match {
|
||||||
|
case Some(uqid) =>
|
||||||
|
byUniquenessId(request.userId, uqid)
|
||||||
|
case None =>
|
||||||
|
val sanitized = sanitizeRequest(request)
|
||||||
|
val sanitizedBytes = serializer.toBytes(sanitized)
|
||||||
|
val digested = MessageDigest.getInstance("SHA-256").digest(sanitizedBytes)
|
||||||
|
val base64Digest = Base64.encodeBase64String(digested)
|
||||||
|
val key = Key(request.userId, TypeCode.PostTweetRequest, base64Digest)
|
||||||
|
log.ifDebug(s"Generated key $key from request:\n${sanitized}")
|
||||||
|
key
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Key for tweets that have a uniqueness id set. There is only one
|
||||||
|
* namespace of uniqueness ids, across all clients. They are
|
||||||
|
* expected to be Snowflake ids, in order to avoid cache
|
||||||
|
* collisions.
|
||||||
|
*/
|
||||||
|
def byUniquenessId(userId: UserId, uniquenessId: Long): Key =
|
||||||
|
Key(userId, TypeCode.UniquenessId, Base64Long.toBase64(uniquenessId))
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The state of tweet creation for a given Key (request).
|
||||||
|
*/
|
||||||
|
sealed trait State
|
||||||
|
|
||||||
|
object State {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* There is no tweet creation currently in progress. (This can
|
||||||
|
* either be represented by no entry in the cache, or this special
|
||||||
|
* marker. This lets us use checkAndSet for deletion to avoid
|
||||||
|
* accidentally overwriting other process' values.)
|
||||||
|
*/
|
||||||
|
case object Unlocked extends State
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Some process is attempting to create the tweet.
|
||||||
|
*/
|
||||||
|
case class InProgress(token: Long, timestamp: Time) extends State
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The tweet has already been successfully created, and has the
|
||||||
|
* specified id.
|
||||||
|
*/
|
||||||
|
case class AlreadyCreated(tweetId: TweetId, timestamp: Time) extends State
|
||||||
|
|
||||||
|
/**
|
||||||
|
* When stored in cache, each state is prefixed by a byte
|
||||||
|
* indicating the type of the entry.
|
||||||
|
*/
|
||||||
|
object TypeCode {
|
||||||
|
val Unlocked: Byte = 0.toByte
|
||||||
|
val InProgress: Byte = 1.toByte // + random long + timestamp
|
||||||
|
val AlreadyCreated: Byte = 2.toByte // + tweet id + timestamp
|
||||||
|
}
|
||||||
|
|
||||||
|
private[this] val BufferSize = 17 // type byte + 64-bit value + 64-bit timestamp
|
||||||
|
|
||||||
|
// Constant buffer to use for storing the serialized form on
|
||||||
|
// Unlocked.
|
||||||
|
private[this] val UnlockedBuf = Array[Byte](TypeCode.Unlocked)
|
||||||
|
|
||||||
|
// Store the serialization function in a ThreadLocal so that we can
|
||||||
|
// reuse the buffer between invocations.
|
||||||
|
private[this] val threadLocalSerialize = new ThreadLocal[State => Array[Byte]] {
|
||||||
|
override def initialValue(): State => Array[Byte] = {
|
||||||
|
// Allocate the thread-local state
|
||||||
|
val ary = new Array[Byte](BufferSize)
|
||||||
|
val buf = ByteBuffer.wrap(ary)
|
||||||
|
|
||||||
|
{
|
||||||
|
case Unlocked => UnlockedBuf
|
||||||
|
case InProgress(token, timestamp) =>
|
||||||
|
buf.clear()
|
||||||
|
buf
|
||||||
|
.put(TypeCode.InProgress)
|
||||||
|
.putLong(token)
|
||||||
|
.putLong(timestamp.sinceEpoch.inNanoseconds)
|
||||||
|
ary
|
||||||
|
case AlreadyCreated(tweetId, timestamp) =>
|
||||||
|
buf.clear()
|
||||||
|
buf
|
||||||
|
.put(TypeCode.AlreadyCreated)
|
||||||
|
.putLong(tweetId)
|
||||||
|
.putLong(timestamp.sinceEpoch.inNanoseconds)
|
||||||
|
ary
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Convert this State to the cache representation.
|
||||||
|
*/
|
||||||
|
private[this] def toBytes(state: State): Array[Byte] =
|
||||||
|
threadLocalSerialize.get()(state)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Convert this byte array into a LockState.
|
||||||
|
*
|
||||||
|
* @throws RuntimeException if the buffer is not of the right size
|
||||||
|
* and format
|
||||||
|
*/
|
||||||
|
private[this] def fromBytes(bytes: Array[Byte]): State = {
|
||||||
|
val buf = ByteBuffer.wrap(bytes)
|
||||||
|
val result = buf.get() match {
|
||||||
|
case TypeCode.Unlocked => Unlocked
|
||||||
|
case TypeCode.InProgress => InProgress(buf.getLong(), buf.getLong().nanoseconds.afterEpoch)
|
||||||
|
case TypeCode.AlreadyCreated =>
|
||||||
|
AlreadyCreated(buf.getLong(), buf.getLong().nanoseconds.afterEpoch)
|
||||||
|
case other => throw new RuntimeException("Invalid type code: " + other)
|
||||||
|
}
|
||||||
|
if (buf.remaining != 0) {
|
||||||
|
throw new RuntimeException("Extra data in buffer: " + bytes)
|
||||||
|
}
|
||||||
|
result
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* How to serialize the State for storage in cache.
|
||||||
|
*/
|
||||||
|
val Serializer: Transformer[State, Array[Byte]] =
|
||||||
|
Transformer[State, Array[Byte]](tTo = toBytes _, tFrom = fromBytes _)
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,506 @@
|
|||||||
|
package com.twitter.tweetypie.util
|
||||||
|
|
||||||
|
import com.twitter.dataproducts.enrichments.thriftscala.ProfileGeoEnrichment
|
||||||
|
import com.twitter.expandodo.thriftscala._
|
||||||
|
import com.twitter.mediaservices.commons.thriftscala.MediaKey
|
||||||
|
import com.twitter.mediaservices.commons.tweetmedia.thriftscala._
|
||||||
|
import com.twitter.servo.data.Lens
|
||||||
|
import com.twitter.spam.rtf.thriftscala.SafetyLabel
|
||||||
|
import com.twitter.tseng.withholding.thriftscala.TakedownReason
|
||||||
|
import com.twitter.tweetypie.thriftscala._
|
||||||
|
import com.twitter.tweetypie.unmentions.thriftscala.UnmentionData
|
||||||
|
|
||||||
|
object TweetLenses {
|
||||||
|
import Lens.checkEq
|
||||||
|
|
||||||
|
def requireSome[A, B](l: Lens[A, Option[B]]): Lens[A, B] =
|
||||||
|
checkEq[A, B](
|
||||||
|
a => l.get(a).get,
|
||||||
|
(a, b) => l.set(a, Some(b))
|
||||||
|
)
|
||||||
|
|
||||||
|
def tweetLens[A](get: Tweet => A, set: (Tweet, A) => Tweet): Lens[Tweet, A] =
|
||||||
|
checkEq[Tweet, A](get, set)
|
||||||
|
|
||||||
|
val id: Lens[Tweet, TweetId] =
|
||||||
|
tweetLens[TweetId](_.id, (t, id) => t.copy(id = id))
|
||||||
|
|
||||||
|
val coreData: Lens[Tweet, Option[TweetCoreData]] =
|
||||||
|
tweetLens[Option[TweetCoreData]](_.coreData, (t, coreData) => t.copy(coreData = coreData))
|
||||||
|
|
||||||
|
val requiredCoreData: Lens[Tweet, TweetCoreData] =
|
||||||
|
requireSome(coreData)
|
||||||
|
|
||||||
|
val optUrls: Lens[Tweet, Option[Seq[UrlEntity]]] =
|
||||||
|
tweetLens[Option[Seq[UrlEntity]]](_.urls, (t, urls) => t.copy(urls = urls))
|
||||||
|
|
||||||
|
val urls: Lens[Tweet, Seq[UrlEntity]] =
|
||||||
|
tweetLens[Seq[UrlEntity]](_.urls.toSeq.flatten, (t, urls) => t.copy(urls = Some(urls)))
|
||||||
|
|
||||||
|
val optMentions: Lens[Tweet, Option[Seq[MentionEntity]]] =
|
||||||
|
tweetLens[Option[Seq[MentionEntity]]](_.mentions, (t, v) => t.copy(mentions = v))
|
||||||
|
|
||||||
|
val mentions: Lens[Tweet, Seq[MentionEntity]] =
|
||||||
|
tweetLens[Seq[MentionEntity]](_.mentions.toSeq.flatten, (t, v) => t.copy(mentions = Some(v)))
|
||||||
|
|
||||||
|
val unmentionData: Lens[Tweet, Option[UnmentionData]] =
|
||||||
|
tweetLens[Option[UnmentionData]](_.unmentionData, (t, v) => t.copy(unmentionData = v))
|
||||||
|
|
||||||
|
val optHashtags: Lens[Tweet, Option[Seq[HashtagEntity]]] =
|
||||||
|
tweetLens[Option[Seq[HashtagEntity]]](_.hashtags, (t, v) => t.copy(hashtags = v))
|
||||||
|
|
||||||
|
val hashtags: Lens[Tweet, Seq[HashtagEntity]] =
|
||||||
|
tweetLens[Seq[HashtagEntity]](_.hashtags.toSeq.flatten, (t, v) => t.copy(hashtags = Some(v)))
|
||||||
|
|
||||||
|
val optCashtags: Lens[Tweet, Option[Seq[CashtagEntity]]] =
|
||||||
|
tweetLens[Option[Seq[CashtagEntity]]](_.cashtags, (t, v) => t.copy(cashtags = v))
|
||||||
|
|
||||||
|
val cashtags: Lens[Tweet, Seq[CashtagEntity]] =
|
||||||
|
tweetLens[Seq[CashtagEntity]](_.cashtags.toSeq.flatten, (t, v) => t.copy(cashtags = Some(v)))
|
||||||
|
|
||||||
|
val optMedia: Lens[Tweet, Option[Seq[MediaEntity]]] =
|
||||||
|
tweetLens[Option[Seq[MediaEntity]]](_.media, (t, v) => t.copy(media = v))
|
||||||
|
|
||||||
|
val media: Lens[Tweet, Seq[MediaEntity]] =
|
||||||
|
tweetLens[Seq[MediaEntity]](_.media.toSeq.flatten, (t, v) => t.copy(media = Some(v)))
|
||||||
|
|
||||||
|
val mediaKeys: Lens[Tweet, Seq[MediaKey]] =
|
||||||
|
tweetLens[Seq[MediaKey]](
|
||||||
|
_.mediaKeys.toSeq.flatten,
|
||||||
|
{
|
||||||
|
case (t, v) => t.copy(mediaKeys = Some(v))
|
||||||
|
})
|
||||||
|
|
||||||
|
val place: Lens[Tweet, Option[Place]] =
|
||||||
|
tweetLens[Option[Place]](
|
||||||
|
_.place,
|
||||||
|
{
|
||||||
|
case (t, v) => t.copy(place = v)
|
||||||
|
})
|
||||||
|
|
||||||
|
val quotedTweet: Lens[Tweet, Option[QuotedTweet]] =
|
||||||
|
tweetLens[Option[QuotedTweet]](
|
||||||
|
_.quotedTweet,
|
||||||
|
{
|
||||||
|
case (t, v) => t.copy(quotedTweet = v)
|
||||||
|
})
|
||||||
|
|
||||||
|
val selfThreadMetadata: Lens[Tweet, Option[SelfThreadMetadata]] =
|
||||||
|
tweetLens[Option[SelfThreadMetadata]](
|
||||||
|
_.selfThreadMetadata,
|
||||||
|
{
|
||||||
|
case (t, v) => t.copy(selfThreadMetadata = v)
|
||||||
|
})
|
||||||
|
|
||||||
|
val composerSource: Lens[Tweet, Option[ComposerSource]] =
|
||||||
|
tweetLens[Option[ComposerSource]](
|
||||||
|
_.composerSource,
|
||||||
|
{
|
||||||
|
case (t, v) => t.copy(composerSource = v)
|
||||||
|
})
|
||||||
|
|
||||||
|
val deviceSource: Lens[Tweet, Option[DeviceSource]] =
|
||||||
|
tweetLens[Option[DeviceSource]](
|
||||||
|
_.deviceSource,
|
||||||
|
{
|
||||||
|
case (t, v) => t.copy(deviceSource = v)
|
||||||
|
})
|
||||||
|
|
||||||
|
val perspective: Lens[Tweet, Option[StatusPerspective]] =
|
||||||
|
tweetLens[Option[StatusPerspective]](
|
||||||
|
_.perspective,
|
||||||
|
{
|
||||||
|
case (t, v) => t.copy(perspective = v)
|
||||||
|
})
|
||||||
|
|
||||||
|
val cards: Lens[Tweet, Option[Seq[Card]]] =
|
||||||
|
tweetLens[Option[Seq[Card]]](
|
||||||
|
_.cards,
|
||||||
|
{
|
||||||
|
case (t, v) => t.copy(cards = v)
|
||||||
|
})
|
||||||
|
|
||||||
|
val card2: Lens[Tweet, Option[Card2]] =
|
||||||
|
tweetLens[Option[Card2]](
|
||||||
|
_.card2,
|
||||||
|
{
|
||||||
|
case (t, v) => t.copy(card2 = v)
|
||||||
|
})
|
||||||
|
|
||||||
|
val cardReference: Lens[Tweet, Option[CardReference]] =
|
||||||
|
tweetLens[Option[CardReference]](
|
||||||
|
_.cardReference,
|
||||||
|
{
|
||||||
|
case (t, v) => t.copy(cardReference = v)
|
||||||
|
})
|
||||||
|
|
||||||
|
val spamLabel: Lens[Tweet, Option[SafetyLabel]] =
|
||||||
|
tweetLens[Option[SafetyLabel]](
|
||||||
|
_.spamLabel,
|
||||||
|
{
|
||||||
|
case (t, v) => t.copy(spamLabel = v)
|
||||||
|
})
|
||||||
|
|
||||||
|
val lowQualityLabel: Lens[Tweet, Option[SafetyLabel]] =
|
||||||
|
tweetLens[Option[SafetyLabel]](
|
||||||
|
_.lowQualityLabel,
|
||||||
|
{
|
||||||
|
case (t, v) => t.copy(lowQualityLabel = v)
|
||||||
|
})
|
||||||
|
|
||||||
|
val nsfwHighPrecisionLabel: Lens[Tweet, Option[SafetyLabel]] =
|
||||||
|
tweetLens[Option[SafetyLabel]](
|
||||||
|
_.nsfwHighPrecisionLabel,
|
||||||
|
{
|
||||||
|
case (t, v) => t.copy(nsfwHighPrecisionLabel = v)
|
||||||
|
})
|
||||||
|
|
||||||
|
val bounceLabel: Lens[Tweet, Option[SafetyLabel]] =
|
||||||
|
tweetLens[Option[SafetyLabel]](
|
||||||
|
_.bounceLabel,
|
||||||
|
{
|
||||||
|
case (t, v) => t.copy(bounceLabel = v)
|
||||||
|
})
|
||||||
|
|
||||||
|
val takedownCountryCodes: Lens[Tweet, Option[Seq[String]]] =
|
||||||
|
tweetLens[Option[Seq[String]]](
|
||||||
|
_.takedownCountryCodes,
|
||||||
|
{
|
||||||
|
case (t, v) => t.copy(takedownCountryCodes = v)
|
||||||
|
})
|
||||||
|
|
||||||
|
val takedownReasons: Lens[Tweet, Option[Seq[TakedownReason]]] =
|
||||||
|
tweetLens[Option[Seq[TakedownReason]]](
|
||||||
|
_.takedownReasons,
|
||||||
|
{
|
||||||
|
case (t, v) => t.copy(takedownReasons = v)
|
||||||
|
})
|
||||||
|
|
||||||
|
val contributor: Lens[Tweet, Option[Contributor]] =
|
||||||
|
tweetLens[Option[Contributor]](
|
||||||
|
_.contributor,
|
||||||
|
{
|
||||||
|
case (t, v) => t.copy(contributor = v)
|
||||||
|
})
|
||||||
|
|
||||||
|
val mediaTags: Lens[Tweet, Option[TweetMediaTags]] =
|
||||||
|
tweetLens[Option[TweetMediaTags]](
|
||||||
|
_.mediaTags,
|
||||||
|
{
|
||||||
|
case (t, v) => t.copy(mediaTags = v)
|
||||||
|
})
|
||||||
|
|
||||||
|
val mediaTagMap: Lens[Tweet, Map[MediaId, Seq[MediaTag]]] =
|
||||||
|
tweetLens[Map[MediaId, Seq[MediaTag]]](
|
||||||
|
_.mediaTags.map { case TweetMediaTags(tagMap) => tagMap.toMap }.getOrElse(Map.empty),
|
||||||
|
(t, v) => {
|
||||||
|
val cleanMap = v.filter { case (_, tags) => tags.nonEmpty }
|
||||||
|
t.copy(mediaTags = if (cleanMap.nonEmpty) Some(TweetMediaTags(cleanMap)) else None)
|
||||||
|
}
|
||||||
|
)
|
||||||
|
|
||||||
|
val escherbirdEntityAnnotations: Lens[Tweet, Option[EscherbirdEntityAnnotations]] =
|
||||||
|
tweetLens[Option[EscherbirdEntityAnnotations]](
|
||||||
|
_.escherbirdEntityAnnotations,
|
||||||
|
{
|
||||||
|
case (t, v) => t.copy(escherbirdEntityAnnotations = v)
|
||||||
|
})
|
||||||
|
|
||||||
|
val communities: Lens[Tweet, Option[Communities]] =
|
||||||
|
tweetLens[Option[Communities]](
|
||||||
|
_.communities,
|
||||||
|
{
|
||||||
|
case (t, v) => t.copy(communities = v)
|
||||||
|
})
|
||||||
|
|
||||||
|
val tweetypieOnlyTakedownCountryCodes: Lens[Tweet, Option[Seq[String]]] =
|
||||||
|
tweetLens[Option[Seq[String]]](
|
||||||
|
_.tweetypieOnlyTakedownCountryCodes,
|
||||||
|
{
|
||||||
|
case (t, v) => t.copy(tweetypieOnlyTakedownCountryCodes = v)
|
||||||
|
})
|
||||||
|
|
||||||
|
val tweetypieOnlyTakedownReasons: Lens[Tweet, Option[Seq[TakedownReason]]] =
|
||||||
|
tweetLens[Option[Seq[TakedownReason]]](
|
||||||
|
_.tweetypieOnlyTakedownReasons,
|
||||||
|
{
|
||||||
|
case (t, v) => t.copy(tweetypieOnlyTakedownReasons = v)
|
||||||
|
})
|
||||||
|
|
||||||
|
val profileGeo: Lens[Tweet, Option[ProfileGeoEnrichment]] =
|
||||||
|
tweetLens[Option[ProfileGeoEnrichment]](
|
||||||
|
_.profileGeoEnrichment,
|
||||||
|
(t, v) => t.copy(profileGeoEnrichment = v)
|
||||||
|
)
|
||||||
|
|
||||||
|
val visibleTextRange: Lens[Tweet, Option[TextRange]] =
|
||||||
|
tweetLens[Option[TextRange]](
|
||||||
|
_.visibleTextRange,
|
||||||
|
{
|
||||||
|
case (t, v) => t.copy(visibleTextRange = v)
|
||||||
|
})
|
||||||
|
|
||||||
|
val selfPermalink: Lens[Tweet, Option[ShortenedUrl]] =
|
||||||
|
tweetLens[Option[ShortenedUrl]](
|
||||||
|
_.selfPermalink,
|
||||||
|
{
|
||||||
|
case (t, v) => t.copy(selfPermalink = v)
|
||||||
|
})
|
||||||
|
|
||||||
|
val extendedTweetMetadata: Lens[Tweet, Option[ExtendedTweetMetadata]] =
|
||||||
|
tweetLens[Option[ExtendedTweetMetadata]](
|
||||||
|
_.extendedTweetMetadata,
|
||||||
|
{
|
||||||
|
case (t, v) => t.copy(extendedTweetMetadata = v)
|
||||||
|
})
|
||||||
|
|
||||||
|
object TweetCoreData {
|
||||||
|
val userId: Lens[TweetCoreData, UserId] = checkEq[TweetCoreData, UserId](
|
||||||
|
_.userId,
|
||||||
|
{ (c, v) =>
|
||||||
|
// Pleases the compiler: https://github.com/scala/bug/issues/9171
|
||||||
|
val userId = v
|
||||||
|
c.copy(userId = userId)
|
||||||
|
})
|
||||||
|
val text: Lens[TweetCoreData, String] = checkEq[TweetCoreData, String](
|
||||||
|
_.text,
|
||||||
|
{ (c, v) =>
|
||||||
|
// Pleases the compiler: https://github.com/scala/bug/issues/9171
|
||||||
|
val text = v
|
||||||
|
c.copy(text = text)
|
||||||
|
})
|
||||||
|
val createdAt: Lens[TweetCoreData, TweetId] =
|
||||||
|
checkEq[TweetCoreData, Long](_.createdAtSecs, (c, v) => c.copy(createdAtSecs = v))
|
||||||
|
val createdVia: Lens[TweetCoreData, String] =
|
||||||
|
checkEq[TweetCoreData, String](
|
||||||
|
_.createdVia,
|
||||||
|
{
|
||||||
|
case (c, v) => c.copy(createdVia = v)
|
||||||
|
})
|
||||||
|
val hasTakedown: Lens[TweetCoreData, Boolean] =
|
||||||
|
checkEq[TweetCoreData, Boolean](
|
||||||
|
_.hasTakedown,
|
||||||
|
{
|
||||||
|
case (c, v) => c.copy(hasTakedown = v)
|
||||||
|
})
|
||||||
|
val nullcast: Lens[TweetCoreData, Boolean] =
|
||||||
|
checkEq[TweetCoreData, Boolean](
|
||||||
|
_.nullcast,
|
||||||
|
{
|
||||||
|
case (c, v) => c.copy(nullcast = v)
|
||||||
|
})
|
||||||
|
val nsfwUser: Lens[TweetCoreData, Boolean] =
|
||||||
|
checkEq[TweetCoreData, Boolean](
|
||||||
|
_.nsfwUser,
|
||||||
|
{
|
||||||
|
case (c, v) => c.copy(nsfwUser = v)
|
||||||
|
})
|
||||||
|
val nsfwAdmin: Lens[TweetCoreData, Boolean] =
|
||||||
|
checkEq[TweetCoreData, Boolean](
|
||||||
|
_.nsfwAdmin,
|
||||||
|
{
|
||||||
|
case (c, v) => c.copy(nsfwAdmin = v)
|
||||||
|
})
|
||||||
|
val reply: Lens[TweetCoreData, Option[Reply]] =
|
||||||
|
checkEq[TweetCoreData, Option[Reply]](
|
||||||
|
_.reply,
|
||||||
|
{
|
||||||
|
case (c, v) => c.copy(reply = v)
|
||||||
|
})
|
||||||
|
val share: Lens[TweetCoreData, Option[Share]] =
|
||||||
|
checkEq[TweetCoreData, Option[Share]](
|
||||||
|
_.share,
|
||||||
|
{
|
||||||
|
case (c, v) => c.copy(share = v)
|
||||||
|
})
|
||||||
|
val narrowcast: Lens[TweetCoreData, Option[Narrowcast]] =
|
||||||
|
checkEq[TweetCoreData, Option[Narrowcast]](
|
||||||
|
_.narrowcast,
|
||||||
|
{
|
||||||
|
case (c, v) => c.copy(narrowcast = v)
|
||||||
|
})
|
||||||
|
val directedAtUser: Lens[TweetCoreData, Option[DirectedAtUser]] =
|
||||||
|
checkEq[TweetCoreData, Option[DirectedAtUser]](
|
||||||
|
_.directedAtUser,
|
||||||
|
{
|
||||||
|
case (c, v) => c.copy(directedAtUser = v)
|
||||||
|
})
|
||||||
|
val conversationId: Lens[TweetCoreData, Option[ConversationId]] =
|
||||||
|
checkEq[TweetCoreData, Option[ConversationId]](
|
||||||
|
_.conversationId,
|
||||||
|
{
|
||||||
|
case (c, v) => c.copy(conversationId = v)
|
||||||
|
})
|
||||||
|
val placeId: Lens[TweetCoreData, Option[String]] =
|
||||||
|
checkEq[TweetCoreData, Option[String]](
|
||||||
|
_.placeId,
|
||||||
|
{
|
||||||
|
case (c, v) => c.copy(placeId = v)
|
||||||
|
})
|
||||||
|
val geoCoordinates: Lens[TweetCoreData, Option[GeoCoordinates]] =
|
||||||
|
checkEq[TweetCoreData, Option[GeoCoordinates]](
|
||||||
|
_.coordinates,
|
||||||
|
(c, v) => c.copy(coordinates = v)
|
||||||
|
)
|
||||||
|
val trackingId: Lens[TweetCoreData, Option[TweetId]] =
|
||||||
|
checkEq[TweetCoreData, Option[Long]](
|
||||||
|
_.trackingId,
|
||||||
|
{
|
||||||
|
case (c, v) => c.copy(trackingId = v)
|
||||||
|
})
|
||||||
|
val hasMedia: Lens[TweetCoreData, Option[Boolean]] =
|
||||||
|
checkEq[TweetCoreData, Option[Boolean]](
|
||||||
|
_.hasMedia,
|
||||||
|
{
|
||||||
|
case (c, v) => c.copy(hasMedia = v)
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|
||||||
|
val counts: Lens[Tweet, Option[StatusCounts]] =
|
||||||
|
tweetLens[Option[StatusCounts]](
|
||||||
|
_.counts,
|
||||||
|
{
|
||||||
|
case (t, v) => t.copy(counts = v)
|
||||||
|
})
|
||||||
|
|
||||||
|
object StatusCounts {
|
||||||
|
val retweetCount: Lens[StatusCounts, Option[TweetId]] =
|
||||||
|
checkEq[StatusCounts, Option[Long]](
|
||||||
|
_.retweetCount,
|
||||||
|
(c, retweetCount) => c.copy(retweetCount = retweetCount)
|
||||||
|
)
|
||||||
|
|
||||||
|
val replyCount: Lens[StatusCounts, Option[TweetId]] =
|
||||||
|
checkEq[StatusCounts, Option[Long]](
|
||||||
|
_.replyCount,
|
||||||
|
(c, replyCount) => c.copy(replyCount = replyCount)
|
||||||
|
)
|
||||||
|
|
||||||
|
val favoriteCount: Lens[StatusCounts, Option[TweetId]] =
|
||||||
|
checkEq[StatusCounts, Option[Long]](
|
||||||
|
_.favoriteCount,
|
||||||
|
{
|
||||||
|
case (c, v) => c.copy(favoriteCount = v)
|
||||||
|
})
|
||||||
|
|
||||||
|
val quoteCount: Lens[StatusCounts, Option[TweetId]] =
|
||||||
|
checkEq[StatusCounts, Option[Long]](
|
||||||
|
_.quoteCount,
|
||||||
|
{
|
||||||
|
case (c, v) => c.copy(quoteCount = v)
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|
||||||
|
val userId: Lens[Tweet, UserId] = requiredCoreData andThen TweetCoreData.userId
|
||||||
|
val text: Lens[Tweet, String] = requiredCoreData andThen TweetCoreData.text
|
||||||
|
val createdVia: Lens[Tweet, String] = requiredCoreData andThen TweetCoreData.createdVia
|
||||||
|
val createdAt: Lens[Tweet, ConversationId] = requiredCoreData andThen TweetCoreData.createdAt
|
||||||
|
val reply: Lens[Tweet, Option[Reply]] = requiredCoreData andThen TweetCoreData.reply
|
||||||
|
val share: Lens[Tweet, Option[Share]] = requiredCoreData andThen TweetCoreData.share
|
||||||
|
val narrowcast: Lens[Tweet, Option[Narrowcast]] =
|
||||||
|
requiredCoreData andThen TweetCoreData.narrowcast
|
||||||
|
val directedAtUser: Lens[Tweet, Option[DirectedAtUser]] =
|
||||||
|
requiredCoreData andThen TweetCoreData.directedAtUser
|
||||||
|
val conversationId: Lens[Tweet, Option[ConversationId]] =
|
||||||
|
requiredCoreData andThen TweetCoreData.conversationId
|
||||||
|
val placeId: Lens[Tweet, Option[String]] = requiredCoreData andThen TweetCoreData.placeId
|
||||||
|
val geoCoordinates: Lens[Tweet, Option[GeoCoordinates]] =
|
||||||
|
requiredCoreData andThen TweetCoreData.geoCoordinates
|
||||||
|
val hasTakedown: Lens[Tweet, Boolean] = requiredCoreData andThen TweetCoreData.hasTakedown
|
||||||
|
val nsfwAdmin: Lens[Tweet, Boolean] = requiredCoreData andThen TweetCoreData.nsfwAdmin
|
||||||
|
val nsfwUser: Lens[Tweet, Boolean] = requiredCoreData andThen TweetCoreData.nsfwUser
|
||||||
|
val nullcast: Lens[Tweet, Boolean] = requiredCoreData andThen TweetCoreData.nullcast
|
||||||
|
val trackingId: Lens[Tweet, Option[ConversationId]] =
|
||||||
|
requiredCoreData andThen TweetCoreData.trackingId
|
||||||
|
val hasMedia: Lens[Tweet, Option[Boolean]] = requiredCoreData andThen TweetCoreData.hasMedia
|
||||||
|
|
||||||
|
object CashtagEntity {
|
||||||
|
val indices: Lens[CashtagEntity, (Short, Short)] =
|
||||||
|
checkEq[CashtagEntity, (Short, Short)](
|
||||||
|
t => (t.fromIndex, t.toIndex),
|
||||||
|
(t, v) => t.copy(fromIndex = v._1, toIndex = v._2)
|
||||||
|
)
|
||||||
|
val text: Lens[CashtagEntity, String] =
|
||||||
|
checkEq[CashtagEntity, String](_.text, (t, text) => t.copy(text = text))
|
||||||
|
}
|
||||||
|
|
||||||
|
object HashtagEntity {
|
||||||
|
val indices: Lens[HashtagEntity, (Short, Short)] =
|
||||||
|
checkEq[HashtagEntity, (Short, Short)](
|
||||||
|
t => (t.fromIndex, t.toIndex),
|
||||||
|
(t, v) => t.copy(fromIndex = v._1, toIndex = v._2)
|
||||||
|
)
|
||||||
|
val text: Lens[HashtagEntity, String] =
|
||||||
|
checkEq[HashtagEntity, String](_.text, (t, text) => t.copy(text = text))
|
||||||
|
}
|
||||||
|
|
||||||
|
object MediaEntity {
|
||||||
|
val indices: Lens[MediaEntity, (Short, Short)] =
|
||||||
|
checkEq[MediaEntity, (Short, Short)](
|
||||||
|
t => (t.fromIndex, t.toIndex),
|
||||||
|
(t, v) => t.copy(fromIndex = v._1, toIndex = v._2)
|
||||||
|
)
|
||||||
|
val mediaSizes: Lens[MediaEntity, collection.Set[MediaSize]] =
|
||||||
|
checkEq[MediaEntity, scala.collection.Set[MediaSize]](
|
||||||
|
_.sizes,
|
||||||
|
(m, sizes) => m.copy(sizes = sizes)
|
||||||
|
)
|
||||||
|
val url: Lens[MediaEntity, String] =
|
||||||
|
checkEq[MediaEntity, String](
|
||||||
|
_.url,
|
||||||
|
{
|
||||||
|
case (t, v) => t.copy(url = v)
|
||||||
|
})
|
||||||
|
val mediaInfo: Lens[MediaEntity, Option[MediaInfo]] =
|
||||||
|
checkEq[MediaEntity, Option[MediaInfo]](
|
||||||
|
_.mediaInfo,
|
||||||
|
{
|
||||||
|
case (t, v) => t.copy(mediaInfo = v)
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|
||||||
|
object MentionEntity {
|
||||||
|
val indices: Lens[MentionEntity, (Short, Short)] =
|
||||||
|
checkEq[MentionEntity, (Short, Short)](
|
||||||
|
t => (t.fromIndex, t.toIndex),
|
||||||
|
(t, v) => t.copy(fromIndex = v._1, toIndex = v._2)
|
||||||
|
)
|
||||||
|
val screenName: Lens[MentionEntity, String] =
|
||||||
|
checkEq[MentionEntity, String](
|
||||||
|
_.screenName,
|
||||||
|
(t, screenName) => t.copy(screenName = screenName)
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
|
object UrlEntity {
|
||||||
|
val indices: Lens[UrlEntity, (Short, Short)] =
|
||||||
|
checkEq[UrlEntity, (Short, Short)](
|
||||||
|
t => (t.fromIndex, t.toIndex),
|
||||||
|
(t, v) => t.copy(fromIndex = v._1, toIndex = v._2)
|
||||||
|
)
|
||||||
|
val url: Lens[UrlEntity, String] =
|
||||||
|
checkEq[UrlEntity, String](_.url, (t, url) => t.copy(url = url))
|
||||||
|
}
|
||||||
|
|
||||||
|
object Contributor {
|
||||||
|
val screenName: Lens[Contributor, Option[String]] =
|
||||||
|
checkEq[Contributor, Option[String]](
|
||||||
|
_.screenName,
|
||||||
|
(c, screenName) => c.copy(screenName = screenName)
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
|
object Reply {
|
||||||
|
val inReplyToScreenName: Lens[Reply, Option[String]] =
|
||||||
|
checkEq[Reply, Option[String]](
|
||||||
|
_.inReplyToScreenName,
|
||||||
|
(c, inReplyToScreenName) => c.copy(inReplyToScreenName = inReplyToScreenName)
|
||||||
|
)
|
||||||
|
|
||||||
|
val inReplyToStatusId: Lens[Reply, Option[TweetId]] =
|
||||||
|
checkEq[Reply, Option[TweetId]](
|
||||||
|
_.inReplyToStatusId,
|
||||||
|
(c, inReplyToStatusId) => c.copy(inReplyToStatusId = inReplyToStatusId)
|
||||||
|
)
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,18 @@
|
|||||||
|
package com.twitter.tweetypie.util
|
||||||
|
|
||||||
|
import com.twitter.tweetutil.TweetPermalink
|
||||||
|
import com.twitter.tweetypie.thriftscala._
|
||||||
|
|
||||||
|
object TweetPermalinkUtil {
|
||||||
|
def lastQuotedTweetPermalink(tweet: Tweet): Option[(UrlEntity, TweetPermalink)] =
|
||||||
|
lastQuotedTweetPermalink(TweetLenses.urls.get(tweet))
|
||||||
|
|
||||||
|
def lastQuotedTweetPermalink(urls: Seq[UrlEntity]): Option[(UrlEntity, TweetPermalink)] =
|
||||||
|
urls.flatMap(matchQuotedTweetPermalink).lastOption
|
||||||
|
|
||||||
|
def matchQuotedTweetPermalink(entity: UrlEntity): Option[(UrlEntity, TweetPermalink)] =
|
||||||
|
for {
|
||||||
|
expanded <- entity.expanded
|
||||||
|
permalink <- TweetPermalink.parse(expanded)
|
||||||
|
} yield (entity, permalink)
|
||||||
|
}
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user