mirror of
https://github.com/twitter/the-algorithm.git
synced 2024-12-22 09:05:30 +00:00
Open-sourcing Unified User Actions
Unified User Action (UUA) is a centralized, real-time stream of user actions on Twitter, consumed by various product, ML, and marketing teams. UUA makes sure all internal teams consume the uniformed user actions data in an accurate and fast way.
This commit is contained in:
parent
f1b5c32734
commit
617c8c787d
4
unified_user_actions/.gitignore
vendored
Normal file
4
unified_user_actions/.gitignore
vendored
Normal file
|
@ -0,0 +1,4 @@
|
|||
.DS_Store
|
||||
CONFIG.ini
|
||||
PROJECT
|
||||
docs
|
1
unified_user_actions/BUILD.bazel
Normal file
1
unified_user_actions/BUILD.bazel
Normal file
|
@ -0,0 +1 @@
|
|||
# This prevents SQ query from grabbing //:all since it traverses up once to find a BUILD
|
10
unified_user_actions/README.md
Normal file
10
unified_user_actions/README.md
Normal file
|
@ -0,0 +1,10 @@
|
|||
# Unified User Actions (UUA)
|
||||
|
||||
**Unified User Actions** (UUA) is a centralized, real-time stream of user actions on Twitter, consumed by various product, ML, and marketing teams. UUA reads client-side and server-side event streams that contain the user's actions and generates a unified real-time user actions Kafka stream. The Kafka stream is replicated to HDFS, GCP Pubsub, GCP GCS, GCP BigQuery. The user actions include public actions such as favorites, retweets, replies and implicit actions like bookmark, impression, video view.
|
||||
|
||||
## Components
|
||||
|
||||
- adapter: transform the raw inputs to UUA Thrift output
|
||||
- client: Kafka client related utils
|
||||
- kafka: more specific Kafka utils like customized serde
|
||||
- service: deployment, modules and services
|
|
@ -0,0 +1,19 @@
|
|||
package com.twitter.unified_user_actions.adapter
|
||||
|
||||
import com.twitter.finagle.stats.NullStatsReceiver
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
|
||||
trait AbstractAdapter[INPUT, OUTK, OUTV] extends Serializable {
|
||||
|
||||
/**
|
||||
* The basic input -> seq[output] adapter which concrete adapters should extend from
|
||||
* @param input a single INPUT
|
||||
* @return A list of (OUTK, OUTV) tuple. The OUTK is the output key mainly for publishing to Kafka (or Pubsub).
|
||||
* If other processing, e.g. offline batch processing, doesn't require the output key then it can drop it
|
||||
* like source.adaptOneToKeyedMany.map(_._2)
|
||||
*/
|
||||
def adaptOneToKeyedMany(
|
||||
input: INPUT,
|
||||
statsReceiver: StatsReceiver = NullStatsReceiver
|
||||
): Seq[(OUTK, OUTV)]
|
||||
}
|
|
@ -0,0 +1,11 @@
|
|||
scala_library(
|
||||
name = "base",
|
||||
sources = [
|
||||
"AbstractAdapter.scala",
|
||||
],
|
||||
compiler_option_sets = ["fatal_warnings"],
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"util/util-stats/src/main/scala/com/twitter/finagle/stats",
|
||||
],
|
||||
)
|
|
@ -0,0 +1,125 @@
|
|||
package com.twitter.unified_user_actions.adapter.ads_callback_engagements
|
||||
|
||||
import com.twitter.ads.spendserver.thriftscala.SpendServerEvent
|
||||
import com.twitter.unified_user_actions.thriftscala._
|
||||
|
||||
object AdsCallbackEngagement {
|
||||
object PromotedTweetFav extends BaseAdsCallbackEngagement(ActionType.ServerPromotedTweetFav)
|
||||
|
||||
object PromotedTweetUnfav extends BaseAdsCallbackEngagement(ActionType.ServerPromotedTweetUnfav)
|
||||
|
||||
object PromotedTweetReply extends BaseAdsCallbackEngagement(ActionType.ServerPromotedTweetReply)
|
||||
|
||||
object PromotedTweetRetweet
|
||||
extends BaseAdsCallbackEngagement(ActionType.ServerPromotedTweetRetweet)
|
||||
|
||||
object PromotedTweetBlockAuthor
|
||||
extends BaseAdsCallbackEngagement(ActionType.ServerPromotedTweetBlockAuthor)
|
||||
|
||||
object PromotedTweetUnblockAuthor
|
||||
extends BaseAdsCallbackEngagement(ActionType.ServerPromotedTweetUnblockAuthor)
|
||||
|
||||
object PromotedTweetComposeTweet
|
||||
extends BaseAdsCallbackEngagement(ActionType.ServerPromotedTweetComposeTweet)
|
||||
|
||||
object PromotedTweetClick extends BaseAdsCallbackEngagement(ActionType.ServerPromotedTweetClick)
|
||||
|
||||
object PromotedTweetReport extends BaseAdsCallbackEngagement(ActionType.ServerPromotedTweetReport)
|
||||
|
||||
object PromotedProfileFollow
|
||||
extends ProfileAdsCallbackEngagement(ActionType.ServerPromotedProfileFollow)
|
||||
|
||||
object PromotedProfileUnfollow
|
||||
extends ProfileAdsCallbackEngagement(ActionType.ServerPromotedProfileUnfollow)
|
||||
|
||||
object PromotedTweetMuteAuthor
|
||||
extends BaseAdsCallbackEngagement(ActionType.ServerPromotedTweetMuteAuthor)
|
||||
|
||||
object PromotedTweetClickProfile
|
||||
extends BaseAdsCallbackEngagement(ActionType.ServerPromotedTweetClickProfile)
|
||||
|
||||
object PromotedTweetClickHashtag
|
||||
extends BaseAdsCallbackEngagement(ActionType.ServerPromotedTweetClickHashtag)
|
||||
|
||||
object PromotedTweetOpenLink
|
||||
extends BaseAdsCallbackEngagement(ActionType.ServerPromotedTweetOpenLink) {
|
||||
override def getItem(input: SpendServerEvent): Option[Item] = {
|
||||
input.engagementEvent.flatMap { e =>
|
||||
e.impressionData.flatMap { i =>
|
||||
getPromotedTweetInfo(
|
||||
i.promotedTweetId,
|
||||
i.advertiserId,
|
||||
tweetActionInfoOpt = Some(
|
||||
TweetActionInfo.ServerPromotedTweetOpenLink(
|
||||
ServerPromotedTweetOpenLink(url = e.url))))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object PromotedTweetCarouselSwipeNext
|
||||
extends BaseAdsCallbackEngagement(ActionType.ServerPromotedTweetCarouselSwipeNext)
|
||||
|
||||
object PromotedTweetCarouselSwipePrevious
|
||||
extends BaseAdsCallbackEngagement(ActionType.ServerPromotedTweetCarouselSwipePrevious)
|
||||
|
||||
object PromotedTweetLingerImpressionShort
|
||||
extends BaseAdsCallbackEngagement(ActionType.ServerPromotedTweetLingerImpressionShort)
|
||||
|
||||
object PromotedTweetLingerImpressionMedium
|
||||
extends BaseAdsCallbackEngagement(ActionType.ServerPromotedTweetLingerImpressionMedium)
|
||||
|
||||
object PromotedTweetLingerImpressionLong
|
||||
extends BaseAdsCallbackEngagement(ActionType.ServerPromotedTweetLingerImpressionLong)
|
||||
|
||||
object PromotedTweetClickSpotlight
|
||||
extends BaseTrendAdsCallbackEngagement(ActionType.ServerPromotedTweetClickSpotlight)
|
||||
|
||||
object PromotedTweetViewSpotlight
|
||||
extends BaseTrendAdsCallbackEngagement(ActionType.ServerPromotedTweetViewSpotlight)
|
||||
|
||||
object PromotedTrendView
|
||||
extends BaseTrendAdsCallbackEngagement(ActionType.ServerPromotedTrendView)
|
||||
|
||||
object PromotedTrendClick
|
||||
extends BaseTrendAdsCallbackEngagement(ActionType.ServerPromotedTrendClick)
|
||||
|
||||
object PromotedTweetVideoPlayback25
|
||||
extends BaseVideoAdsCallbackEngagement(ActionType.ServerPromotedTweetVideoPlayback25)
|
||||
|
||||
object PromotedTweetVideoPlayback50
|
||||
extends BaseVideoAdsCallbackEngagement(ActionType.ServerPromotedTweetVideoPlayback50)
|
||||
|
||||
object PromotedTweetVideoPlayback75
|
||||
extends BaseVideoAdsCallbackEngagement(ActionType.ServerPromotedTweetVideoPlayback75)
|
||||
|
||||
object PromotedTweetVideoAdPlayback25
|
||||
extends BaseVideoAdsCallbackEngagement(ActionType.ServerPromotedTweetVideoAdPlayback25)
|
||||
|
||||
object PromotedTweetVideoAdPlayback50
|
||||
extends BaseVideoAdsCallbackEngagement(ActionType.ServerPromotedTweetVideoAdPlayback50)
|
||||
|
||||
object PromotedTweetVideoAdPlayback75
|
||||
extends BaseVideoAdsCallbackEngagement(ActionType.ServerPromotedTweetVideoAdPlayback75)
|
||||
|
||||
object TweetVideoAdPlayback25
|
||||
extends BaseVideoAdsCallbackEngagement(ActionType.ServerTweetVideoAdPlayback25)
|
||||
|
||||
object TweetVideoAdPlayback50
|
||||
extends BaseVideoAdsCallbackEngagement(ActionType.ServerTweetVideoAdPlayback50)
|
||||
|
||||
object TweetVideoAdPlayback75
|
||||
extends BaseVideoAdsCallbackEngagement(ActionType.ServerTweetVideoAdPlayback75)
|
||||
|
||||
object PromotedTweetDismissWithoutReason
|
||||
extends BaseAdsCallbackEngagement(ActionType.ServerPromotedTweetDismissWithoutReason)
|
||||
|
||||
object PromotedTweetDismissUninteresting
|
||||
extends BaseAdsCallbackEngagement(ActionType.ServerPromotedTweetDismissUninteresting)
|
||||
|
||||
object PromotedTweetDismissRepetitive
|
||||
extends BaseAdsCallbackEngagement(ActionType.ServerPromotedTweetDismissRepetitive)
|
||||
|
||||
object PromotedTweetDismissSpam
|
||||
extends BaseAdsCallbackEngagement(ActionType.ServerPromotedTweetDismissSpam)
|
||||
}
|
|
@ -0,0 +1,28 @@
|
|||
package com.twitter.unified_user_actions.adapter.ads_callback_engagements
|
||||
|
||||
import com.twitter.finagle.stats.NullStatsReceiver
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.finatra.kafka.serde.UnKeyed
|
||||
import com.twitter.unified_user_actions.adapter.AbstractAdapter
|
||||
import com.twitter.ads.spendserver.thriftscala.SpendServerEvent
|
||||
import com.twitter.unified_user_actions.thriftscala.UnifiedUserAction
|
||||
|
||||
class AdsCallbackEngagementsAdapter
|
||||
extends AbstractAdapter[SpendServerEvent, UnKeyed, UnifiedUserAction] {
|
||||
|
||||
import AdsCallbackEngagementsAdapter._
|
||||
|
||||
override def adaptOneToKeyedMany(
|
||||
input: SpendServerEvent,
|
||||
statsReceiver: StatsReceiver = NullStatsReceiver
|
||||
): Seq[(UnKeyed, UnifiedUserAction)] =
|
||||
adaptEvent(input).map { e => (UnKeyed, e) }
|
||||
}
|
||||
|
||||
object AdsCallbackEngagementsAdapter {
|
||||
def adaptEvent(input: SpendServerEvent): Seq[UnifiedUserAction] = {
|
||||
val baseEngagements: Seq[BaseAdsCallbackEngagement] =
|
||||
EngagementTypeMappings.getEngagementMappings(Option(input).flatMap(_.engagementEvent))
|
||||
baseEngagements.flatMap(_.getUUA(input))
|
||||
}
|
||||
}
|
|
@ -0,0 +1,18 @@
|
|||
scala_library(
|
||||
sources = [
|
||||
"*.scala",
|
||||
],
|
||||
compiler_option_sets = ["fatal_warnings"],
|
||||
tags = [
|
||||
"bazel-compatible",
|
||||
"bazel-only",
|
||||
],
|
||||
dependencies = [
|
||||
"kafka/finagle-kafka/finatra-kafka/src/main/scala",
|
||||
"src/thrift/com/twitter/ads/billing/spendserver:spendserver_thrift-scala",
|
||||
"src/thrift/com/twitter/ads/eventstream:eventstream-scala",
|
||||
"unified_user_actions/adapter/src/main/scala/com/twitter/unified_user_actions/adapter:base",
|
||||
"unified_user_actions/adapter/src/main/scala/com/twitter/unified_user_actions/adapter/common",
|
||||
"unified_user_actions/thrift/src/main/thrift/com/twitter/unified_user_actions:unified_user_actions-scala",
|
||||
],
|
||||
)
|
|
@ -0,0 +1,68 @@
|
|||
package com.twitter.unified_user_actions.adapter.ads_callback_engagements
|
||||
|
||||
import com.twitter.ads.spendserver.thriftscala.SpendServerEvent
|
||||
import com.twitter.unified_user_actions.adapter.common.AdapterUtils
|
||||
import com.twitter.unified_user_actions.thriftscala.ActionType
|
||||
import com.twitter.unified_user_actions.thriftscala.AuthorInfo
|
||||
import com.twitter.unified_user_actions.thriftscala.EventMetadata
|
||||
import com.twitter.unified_user_actions.thriftscala.Item
|
||||
import com.twitter.unified_user_actions.thriftscala.SourceLineage
|
||||
import com.twitter.unified_user_actions.thriftscala.TweetInfo
|
||||
import com.twitter.unified_user_actions.thriftscala.TweetActionInfo
|
||||
import com.twitter.unified_user_actions.thriftscala.UnifiedUserAction
|
||||
import com.twitter.unified_user_actions.thriftscala.UserIdentifier
|
||||
|
||||
abstract class BaseAdsCallbackEngagement(actionType: ActionType) {
|
||||
|
||||
protected def getItem(input: SpendServerEvent): Option[Item] = {
|
||||
input.engagementEvent.flatMap { e =>
|
||||
e.impressionData.flatMap { i =>
|
||||
getPromotedTweetInfo(i.promotedTweetId, i.advertiserId)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
protected def getPromotedTweetInfo(
|
||||
promotedTweetIdOpt: Option[Long],
|
||||
advertiserId: Long,
|
||||
tweetActionInfoOpt: Option[TweetActionInfo] = None
|
||||
): Option[Item] = {
|
||||
promotedTweetIdOpt.map { promotedTweetId =>
|
||||
Item.TweetInfo(
|
||||
TweetInfo(
|
||||
actionTweetId = promotedTweetId,
|
||||
actionTweetAuthorInfo = Some(AuthorInfo(authorId = Some(advertiserId))),
|
||||
tweetActionInfo = tweetActionInfoOpt)
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
def getUUA(input: SpendServerEvent): Option[UnifiedUserAction] = {
|
||||
val userIdentifier: UserIdentifier =
|
||||
UserIdentifier(
|
||||
userId = input.engagementEvent.flatMap(e => e.clientInfo.flatMap(_.userId64)),
|
||||
guestIdMarketing = input.engagementEvent.flatMap(e => e.clientInfo.flatMap(_.guestId)),
|
||||
)
|
||||
|
||||
getItem(input).map { item =>
|
||||
UnifiedUserAction(
|
||||
userIdentifier = userIdentifier,
|
||||
item = item,
|
||||
actionType = actionType,
|
||||
eventMetadata = getEventMetadata(input),
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
protected def getEventMetadata(input: SpendServerEvent): EventMetadata =
|
||||
EventMetadata(
|
||||
sourceTimestampMs = input.engagementEvent
|
||||
.map { e => e.engagementEpochTimeMilliSec }.getOrElse(AdapterUtils.currentTimestampMs),
|
||||
receivedTimestampMs = AdapterUtils.currentTimestampMs,
|
||||
sourceLineage = SourceLineage.ServerAdsCallbackEngagements,
|
||||
language = input.engagementEvent.flatMap { e => e.clientInfo.flatMap(_.languageCode) },
|
||||
countryCode = input.engagementEvent.flatMap { e => e.clientInfo.flatMap(_.countryCode) },
|
||||
clientAppId =
|
||||
input.engagementEvent.flatMap { e => e.clientInfo.flatMap(_.clientId) }.map { _.toLong },
|
||||
)
|
||||
}
|
|
@ -0,0 +1,18 @@
|
|||
package com.twitter.unified_user_actions.adapter.ads_callback_engagements
|
||||
|
||||
import com.twitter.ads.spendserver.thriftscala.SpendServerEvent
|
||||
import com.twitter.unified_user_actions.thriftscala._
|
||||
|
||||
abstract class BaseTrendAdsCallbackEngagement(actionType: ActionType)
|
||||
extends BaseAdsCallbackEngagement(actionType = actionType) {
|
||||
|
||||
override protected def getItem(input: SpendServerEvent): Option[Item] = {
|
||||
input.engagementEvent.flatMap { e =>
|
||||
e.impressionData.flatMap { i =>
|
||||
i.promotedTrendId.map { promotedTrendId =>
|
||||
Item.TrendInfo(TrendInfo(actionTrendId = promotedTrendId))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,54 @@
|
|||
package com.twitter.unified_user_actions.adapter.ads_callback_engagements
|
||||
|
||||
import com.twitter.ads.spendserver.thriftscala.SpendServerEvent
|
||||
import com.twitter.unified_user_actions.thriftscala.ActionType
|
||||
import com.twitter.unified_user_actions.thriftscala.AuthorInfo
|
||||
import com.twitter.unified_user_actions.thriftscala.TweetVideoWatch
|
||||
import com.twitter.unified_user_actions.thriftscala.Item
|
||||
import com.twitter.unified_user_actions.thriftscala.TweetActionInfo
|
||||
import com.twitter.unified_user_actions.thriftscala.TweetInfo
|
||||
|
||||
abstract class BaseVideoAdsCallbackEngagement(actionType: ActionType)
|
||||
extends BaseAdsCallbackEngagement(actionType = actionType) {
|
||||
|
||||
override def getItem(input: SpendServerEvent): Option[Item] = {
|
||||
input.engagementEvent.flatMap { e =>
|
||||
e.impressionData.flatMap { i =>
|
||||
getTweetInfo(i.promotedTweetId, i.organicTweetId, i.advertiserId, input)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private def getTweetInfo(
|
||||
promotedTweetId: Option[Long],
|
||||
organicTweetId: Option[Long],
|
||||
advertiserId: Long,
|
||||
input: SpendServerEvent
|
||||
): Option[Item] = {
|
||||
val actionedTweetIdOpt: Option[Long] =
|
||||
if (promotedTweetId.isEmpty) organicTweetId else promotedTweetId
|
||||
actionedTweetIdOpt.map { actionTweetId =>
|
||||
Item.TweetInfo(
|
||||
TweetInfo(
|
||||
actionTweetId = actionTweetId,
|
||||
actionTweetAuthorInfo = Some(AuthorInfo(authorId = Some(advertiserId))),
|
||||
tweetActionInfo = Some(
|
||||
TweetActionInfo.TweetVideoWatch(
|
||||
TweetVideoWatch(
|
||||
isMonetizable = Some(true),
|
||||
videoOwnerId = input.engagementEvent
|
||||
.flatMap(e => e.cardEngagement).flatMap(_.amplifyDetails).flatMap(_.videoOwnerId),
|
||||
videoUuid = input.engagementEvent
|
||||
.flatMap(_.cardEngagement).flatMap(_.amplifyDetails).flatMap(_.videoUuid),
|
||||
prerollOwnerId = input.engagementEvent
|
||||
.flatMap(e => e.cardEngagement).flatMap(_.amplifyDetails).flatMap(
|
||||
_.prerollOwnerId),
|
||||
prerollUuid = input.engagementEvent
|
||||
.flatMap(_.cardEngagement).flatMap(_.amplifyDetails).flatMap(_.prerollUuid)
|
||||
))
|
||||
)
|
||||
),
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,69 @@
|
|||
package com.twitter.unified_user_actions.adapter.ads_callback_engagements
|
||||
|
||||
import com.twitter.ads.eventstream.thriftscala.EngagementEvent
|
||||
import com.twitter.adserver.thriftscala.EngagementType
|
||||
import com.twitter.unified_user_actions.adapter.ads_callback_engagements.AdsCallbackEngagement._
|
||||
|
||||
object EngagementTypeMappings {
|
||||
|
||||
/**
|
||||
* Ads could be Tweets or non-Tweets. Since UUA explicitly sets the item type, it is
|
||||
* possible that one Ads Callback engagement type maps to multiple UUA action types.
|
||||
*/
|
||||
def getEngagementMappings(
|
||||
engagementEvent: Option[EngagementEvent]
|
||||
): Seq[BaseAdsCallbackEngagement] = {
|
||||
val promotedTweetId: Option[Long] =
|
||||
engagementEvent.flatMap(_.impressionData).flatMap(_.promotedTweetId)
|
||||
engagementEvent
|
||||
.map(event =>
|
||||
event.engagementType match {
|
||||
case EngagementType.Fav => Seq(PromotedTweetFav)
|
||||
case EngagementType.Unfav => Seq(PromotedTweetUnfav)
|
||||
case EngagementType.Reply => Seq(PromotedTweetReply)
|
||||
case EngagementType.Retweet => Seq(PromotedTweetRetweet)
|
||||
case EngagementType.Block => Seq(PromotedTweetBlockAuthor)
|
||||
case EngagementType.Unblock => Seq(PromotedTweetUnblockAuthor)
|
||||
case EngagementType.Send => Seq(PromotedTweetComposeTweet)
|
||||
case EngagementType.Detail => Seq(PromotedTweetClick)
|
||||
case EngagementType.Report => Seq(PromotedTweetReport)
|
||||
case EngagementType.Follow => Seq(PromotedProfileFollow)
|
||||
case EngagementType.Unfollow => Seq(PromotedProfileUnfollow)
|
||||
case EngagementType.Mute => Seq(PromotedTweetMuteAuthor)
|
||||
case EngagementType.ProfilePic => Seq(PromotedTweetClickProfile)
|
||||
case EngagementType.ScreenName => Seq(PromotedTweetClickProfile)
|
||||
case EngagementType.UserName => Seq(PromotedTweetClickProfile)
|
||||
case EngagementType.Hashtag => Seq(PromotedTweetClickHashtag)
|
||||
case EngagementType.Url => Seq(PromotedTweetOpenLink)
|
||||
case EngagementType.CarouselSwipeNext => Seq(PromotedTweetCarouselSwipeNext)
|
||||
case EngagementType.CarouselSwipePrevious => Seq(PromotedTweetCarouselSwipePrevious)
|
||||
case EngagementType.DwellShort => Seq(PromotedTweetLingerImpressionShort)
|
||||
case EngagementType.DwellMedium => Seq(PromotedTweetLingerImpressionMedium)
|
||||
case EngagementType.DwellLong => Seq(PromotedTweetLingerImpressionLong)
|
||||
case EngagementType.SpotlightClick => Seq(PromotedTweetClickSpotlight)
|
||||
case EngagementType.SpotlightView => Seq(PromotedTweetViewSpotlight)
|
||||
case EngagementType.TrendView => Seq(PromotedTrendView)
|
||||
case EngagementType.TrendClick => Seq(PromotedTrendClick)
|
||||
case EngagementType.VideoContentPlayback25 => Seq(PromotedTweetVideoPlayback25)
|
||||
case EngagementType.VideoContentPlayback50 => Seq(PromotedTweetVideoPlayback50)
|
||||
case EngagementType.VideoContentPlayback75 => Seq(PromotedTweetVideoPlayback75)
|
||||
case EngagementType.VideoAdPlayback25 if promotedTweetId.isDefined =>
|
||||
Seq(PromotedTweetVideoAdPlayback25)
|
||||
case EngagementType.VideoAdPlayback25 if promotedTweetId.isEmpty =>
|
||||
Seq(TweetVideoAdPlayback25)
|
||||
case EngagementType.VideoAdPlayback50 if promotedTweetId.isDefined =>
|
||||
Seq(PromotedTweetVideoAdPlayback50)
|
||||
case EngagementType.VideoAdPlayback50 if promotedTweetId.isEmpty =>
|
||||
Seq(TweetVideoAdPlayback50)
|
||||
case EngagementType.VideoAdPlayback75 if promotedTweetId.isDefined =>
|
||||
Seq(PromotedTweetVideoAdPlayback75)
|
||||
case EngagementType.VideoAdPlayback75 if promotedTweetId.isEmpty =>
|
||||
Seq(TweetVideoAdPlayback75)
|
||||
case EngagementType.DismissRepetitive => Seq(PromotedTweetDismissRepetitive)
|
||||
case EngagementType.DismissSpam => Seq(PromotedTweetDismissSpam)
|
||||
case EngagementType.DismissUninteresting => Seq(PromotedTweetDismissUninteresting)
|
||||
case EngagementType.DismissWithoutReason => Seq(PromotedTweetDismissWithoutReason)
|
||||
case _ => Nil
|
||||
}).toSeq.flatten
|
||||
}
|
||||
}
|
|
@ -0,0 +1,26 @@
|
|||
package com.twitter.unified_user_actions.adapter.ads_callback_engagements
|
||||
|
||||
import com.twitter.ads.spendserver.thriftscala.SpendServerEvent
|
||||
import com.twitter.unified_user_actions.thriftscala.ActionType
|
||||
import com.twitter.unified_user_actions.thriftscala.Item
|
||||
import com.twitter.unified_user_actions.thriftscala.ProfileInfo
|
||||
|
||||
abstract class ProfileAdsCallbackEngagement(actionType: ActionType)
|
||||
extends BaseAdsCallbackEngagement(actionType) {
|
||||
|
||||
override protected def getItem(input: SpendServerEvent): Option[Item] = {
|
||||
input.engagementEvent.flatMap { e =>
|
||||
e.impressionData.flatMap { i =>
|
||||
getProfileInfo(i.advertiserId)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
protected def getProfileInfo(advertiserId: Long): Option[Item] = {
|
||||
Some(
|
||||
Item.ProfileInfo(
|
||||
ProfileInfo(
|
||||
actionProfileId = advertiserId
|
||||
)))
|
||||
}
|
||||
}
|
|
@ -0,0 +1,13 @@
|
|||
scala_library(
|
||||
sources = [
|
||||
"*.scala",
|
||||
],
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"client-events/thrift/src/thrift/storage/twitter/behavioral_event:behavioral_event-scala",
|
||||
"kafka/finagle-kafka/finatra-kafka/src/main/scala",
|
||||
"unified_user_actions/adapter/src/main/scala/com/twitter/unified_user_actions/adapter:base",
|
||||
"unified_user_actions/adapter/src/main/scala/com/twitter/unified_user_actions/adapter/common",
|
||||
"unified_user_actions/thrift/src/main/thrift/com/twitter/unified_user_actions:unified_user_actions-scala",
|
||||
],
|
||||
)
|
|
@ -0,0 +1,96 @@
|
|||
package com.twitter.unified_user_actions.adapter.behavioral_client_event
|
||||
|
||||
import com.twitter.client_event_entities.serverside_context_key.latest.thriftscala.FlattenedServersideContextKey
|
||||
import com.twitter.storage.behavioral_event.thriftscala.EventLogContext
|
||||
import com.twitter.storage.behavioral_event.thriftscala.FlattenedEventLog
|
||||
import com.twitter.unified_user_actions.adapter.common.AdapterUtils
|
||||
import com.twitter.unified_user_actions.thriftscala.ActionType
|
||||
import com.twitter.unified_user_actions.thriftscala.BreadcrumbTweet
|
||||
import com.twitter.unified_user_actions.thriftscala.ClientEventNamespace
|
||||
import com.twitter.unified_user_actions.thriftscala.EventMetadata
|
||||
import com.twitter.unified_user_actions.thriftscala.Item
|
||||
import com.twitter.unified_user_actions.thriftscala.ProductSurface
|
||||
import com.twitter.unified_user_actions.thriftscala.ProductSurfaceInfo
|
||||
import com.twitter.unified_user_actions.thriftscala.SourceLineage
|
||||
import com.twitter.unified_user_actions.thriftscala.UnifiedUserAction
|
||||
import com.twitter.unified_user_actions.thriftscala.UserIdentifier
|
||||
|
||||
case class ProductSurfaceRelated(
|
||||
productSurface: Option[ProductSurface],
|
||||
productSurfaceInfo: Option[ProductSurfaceInfo])
|
||||
|
||||
trait BaseBCEAdapter {
|
||||
def toUUA(e: FlattenedEventLog): Seq[UnifiedUserAction]
|
||||
|
||||
protected def getUserIdentifier(c: EventLogContext): UserIdentifier =
|
||||
UserIdentifier(
|
||||
userId = c.userId,
|
||||
guestIdMarketing = c.guestIdMarketing
|
||||
)
|
||||
|
||||
protected def getEventMetadata(e: FlattenedEventLog): EventMetadata =
|
||||
EventMetadata(
|
||||
sourceLineage = SourceLineage.BehavioralClientEvents,
|
||||
sourceTimestampMs =
|
||||
e.context.driftAdjustedEventCreatedAtMs.getOrElse(e.context.eventCreatedAtMs),
|
||||
receivedTimestampMs = AdapterUtils.currentTimestampMs,
|
||||
// Client UI language or from Gizmoduck which is what user set in Twitter App.
|
||||
// Please see more at https://sourcegraph.twitter.biz/git.twitter.biz/source/-/blob/finatra-internal/international/src/main/scala/com/twitter/finatra/international/LanguageIdentifier.scala
|
||||
// The format should be ISO 639-1.
|
||||
language = e.context.languageCode.map(AdapterUtils.normalizeLanguageCode),
|
||||
// Country code could be IP address (geoduck) or User registration country (gizmoduck) and the former takes precedence.
|
||||
// We don’t know exactly which one is applied, unfortunately,
|
||||
// see https://sourcegraph.twitter.biz/git.twitter.biz/source/-/blob/finatra-internal/international/src/main/scala/com/twitter/finatra/international/CountryIdentifier.scala
|
||||
// The format should be ISO_3166-1_alpha-2.
|
||||
countryCode = e.context.countryCode.map(AdapterUtils.normalizeCountryCode),
|
||||
clientAppId = e.context.clientApplicationId,
|
||||
clientVersion = e.context.clientVersion,
|
||||
clientPlatform = e.context.clientPlatform,
|
||||
viewHierarchy = e.v1ViewTypeHierarchy,
|
||||
clientEventNamespace = Some(
|
||||
ClientEventNamespace(
|
||||
page = e.page,
|
||||
section = e.section,
|
||||
element = e.element,
|
||||
action = e.actionName,
|
||||
subsection = e.subsection
|
||||
)),
|
||||
breadcrumbViews = e.v1BreadcrumbViewTypeHierarchy,
|
||||
breadcrumbTweets = e.v1BreadcrumbTweetIds.map { breadcrumbs =>
|
||||
breadcrumbs.map { breadcrumb =>
|
||||
BreadcrumbTweet(
|
||||
tweetId = breadcrumb.serversideContextId.toLong,
|
||||
sourceComponent = breadcrumb.sourceComponent)
|
||||
}
|
||||
}
|
||||
)
|
||||
|
||||
protected def getBreadcrumbTweetIds(
|
||||
breadcrumbTweetIds: Option[Seq[FlattenedServersideContextKey]]
|
||||
): Seq[BreadcrumbTweet] =
|
||||
breadcrumbTweetIds
|
||||
.getOrElse(Nil).map(breadcrumb => {
|
||||
BreadcrumbTweet(
|
||||
tweetId = breadcrumb.serversideContextId.toLong,
|
||||
sourceComponent = breadcrumb.sourceComponent)
|
||||
})
|
||||
|
||||
protected def getBreadcrumbViews(breadcrumbView: Option[Seq[String]]): Seq[String] =
|
||||
breadcrumbView.getOrElse(Nil)
|
||||
|
||||
protected def getUnifiedUserAction(
|
||||
event: FlattenedEventLog,
|
||||
actionType: ActionType,
|
||||
item: Item,
|
||||
productSurface: Option[ProductSurface] = None,
|
||||
productSurfaceInfo: Option[ProductSurfaceInfo] = None
|
||||
): UnifiedUserAction =
|
||||
UnifiedUserAction(
|
||||
userIdentifier = getUserIdentifier(event.context),
|
||||
actionType = actionType,
|
||||
item = item,
|
||||
eventMetadata = getEventMetadata(event),
|
||||
productSurface = productSurface,
|
||||
productSurfaceInfo = productSurfaceInfo
|
||||
)
|
||||
}
|
|
@ -0,0 +1,39 @@
|
|||
package com.twitter.unified_user_actions.adapter.behavioral_client_event
|
||||
|
||||
import com.twitter.finagle.stats.NullStatsReceiver
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.finatra.kafka.serde.UnKeyed
|
||||
import com.twitter.storage.behavioral_event.thriftscala.FlattenedEventLog
|
||||
import com.twitter.unified_user_actions.adapter.AbstractAdapter
|
||||
import com.twitter.unified_user_actions.thriftscala._
|
||||
|
||||
class BehavioralClientEventAdapter
|
||||
extends AbstractAdapter[FlattenedEventLog, UnKeyed, UnifiedUserAction] {
|
||||
|
||||
import BehavioralClientEventAdapter._
|
||||
|
||||
override def adaptOneToKeyedMany(
|
||||
input: FlattenedEventLog,
|
||||
statsReceiver: StatsReceiver = NullStatsReceiver
|
||||
): Seq[(UnKeyed, UnifiedUserAction)] =
|
||||
adaptEvent(input).map { e => (UnKeyed, e) }
|
||||
}
|
||||
|
||||
object BehavioralClientEventAdapter {
|
||||
def adaptEvent(e: FlattenedEventLog): Seq[UnifiedUserAction] =
|
||||
// See go/bcecoverage for event namespaces, usage and coverage details
|
||||
Option(e)
|
||||
.map { e =>
|
||||
(e.page, e.actionName) match {
|
||||
case (Some("tweet_details"), Some("impress")) =>
|
||||
TweetImpressionBCEAdapter.TweetDetails.toUUA(e)
|
||||
case (Some("fullscreen_video"), Some("impress")) =>
|
||||
TweetImpressionBCEAdapter.FullscreenVideo.toUUA(e)
|
||||
case (Some("fullscreen_image"), Some("impress")) =>
|
||||
TweetImpressionBCEAdapter.FullscreenImage.toUUA(e)
|
||||
case (Some("profile"), Some("impress")) =>
|
||||
ProfileImpressionBCEAdapter.Profile.toUUA(e)
|
||||
case _ => Nil
|
||||
}
|
||||
}.getOrElse(Nil)
|
||||
}
|
|
@ -0,0 +1,34 @@
|
|||
package com.twitter.unified_user_actions.adapter.behavioral_client_event
|
||||
|
||||
import com.twitter.client.behavioral_event.action.impress.latest.thriftscala.Impress
|
||||
import com.twitter.client_event_entities.serverside_context_key.latest.thriftscala.FlattenedServersideContextKey
|
||||
import com.twitter.unified_user_actions.thriftscala.Item
|
||||
|
||||
trait ImpressionBCEAdapter extends BaseBCEAdapter {
|
||||
type ImpressedItem <: Item
|
||||
|
||||
def getImpressedItem(
|
||||
context: FlattenedServersideContextKey,
|
||||
impression: Impress
|
||||
): ImpressedItem
|
||||
|
||||
/**
|
||||
* The start time of an impression in milliseconds since epoch. In BCE, the impression
|
||||
* tracking clock will start immediately after the page is visible with no initial delay.
|
||||
*/
|
||||
def getImpressedStartTimestamp(impression: Impress): Long =
|
||||
impression.visibilityPctDwellStartMs
|
||||
|
||||
/**
|
||||
* The end time of an impression in milliseconds since epoch. In BCE, the impression
|
||||
* tracking clock will end before the user exit the page.
|
||||
*/
|
||||
def getImpressedEndTimestamp(impression: Impress): Long =
|
||||
impression.visibilityPctDwellEndMs
|
||||
|
||||
/**
|
||||
* The UI component that hosted the impressed item.
|
||||
*/
|
||||
def getImpressedUISourceComponent(context: FlattenedServersideContextKey): String =
|
||||
context.sourceComponent
|
||||
}
|
|
@ -0,0 +1,52 @@
|
|||
package com.twitter.unified_user_actions.adapter.behavioral_client_event
|
||||
|
||||
import com.twitter.client.behavioral_event.action.impress.latest.thriftscala.Impress
|
||||
import com.twitter.client_event_entities.serverside_context_key.latest.thriftscala.FlattenedServersideContextKey
|
||||
import com.twitter.storage.behavioral_event.thriftscala.FlattenedEventLog
|
||||
import com.twitter.unified_user_actions.thriftscala.ActionType
|
||||
import com.twitter.unified_user_actions.thriftscala.ClientProfileV2Impression
|
||||
import com.twitter.unified_user_actions.thriftscala.Item
|
||||
import com.twitter.unified_user_actions.thriftscala.ProductSurface
|
||||
import com.twitter.unified_user_actions.thriftscala.ProfileActionInfo
|
||||
import com.twitter.unified_user_actions.thriftscala.ProfileInfo
|
||||
import com.twitter.unified_user_actions.thriftscala.UnifiedUserAction
|
||||
|
||||
object ProfileImpressionBCEAdapter {
|
||||
val Profile = new ProfileImpressionBCEAdapter()
|
||||
}
|
||||
|
||||
class ProfileImpressionBCEAdapter extends ImpressionBCEAdapter {
|
||||
override type ImpressedItem = Item.ProfileInfo
|
||||
|
||||
override def toUUA(e: FlattenedEventLog): Seq[UnifiedUserAction] =
|
||||
(e.v2Impress, e.v1UserIds) match {
|
||||
case (Some(v2Impress), Some(v1UserIds)) =>
|
||||
v1UserIds.map { user =>
|
||||
getUnifiedUserAction(
|
||||
event = e,
|
||||
actionType = ActionType.ClientProfileV2Impression,
|
||||
item = getImpressedItem(user, v2Impress),
|
||||
productSurface = Some(ProductSurface.ProfilePage)
|
||||
)
|
||||
}
|
||||
case _ => Nil
|
||||
}
|
||||
|
||||
override def getImpressedItem(
|
||||
context: FlattenedServersideContextKey,
|
||||
impression: Impress
|
||||
): ImpressedItem =
|
||||
Item.ProfileInfo(
|
||||
ProfileInfo(
|
||||
actionProfileId = context.serversideContextId.toLong,
|
||||
profileActionInfo = Some(
|
||||
ProfileActionInfo.ClientProfileV2Impression(
|
||||
ClientProfileV2Impression(
|
||||
impressStartTimestampMs = getImpressedStartTimestamp(impression),
|
||||
impressEndTimestampMs = getImpressedEndTimestamp(impression),
|
||||
sourceComponent = getImpressedUISourceComponent(context)
|
||||
)
|
||||
)
|
||||
)
|
||||
))
|
||||
}
|
|
@ -0,0 +1,84 @@
|
|||
package com.twitter.unified_user_actions.adapter.behavioral_client_event
|
||||
|
||||
import com.twitter.client.behavioral_event.action.impress.latest.thriftscala.Impress
|
||||
import com.twitter.client_event_entities.serverside_context_key.latest.thriftscala.FlattenedServersideContextKey
|
||||
import com.twitter.storage.behavioral_event.thriftscala.FlattenedEventLog
|
||||
import com.twitter.unified_user_actions.thriftscala.ActionType
|
||||
import com.twitter.unified_user_actions.thriftscala.ClientTweetV2Impression
|
||||
import com.twitter.unified_user_actions.thriftscala.Item
|
||||
import com.twitter.unified_user_actions.thriftscala.ProductSurface
|
||||
import com.twitter.unified_user_actions.thriftscala.TweetActionInfo
|
||||
import com.twitter.unified_user_actions.thriftscala.TweetInfo
|
||||
import com.twitter.unified_user_actions.thriftscala.UnifiedUserAction
|
||||
|
||||
object TweetImpressionBCEAdapter {
|
||||
val TweetDetails = new TweetImpressionBCEAdapter(ActionType.ClientTweetV2Impression)
|
||||
val FullscreenVideo = new TweetImpressionBCEAdapter(
|
||||
ActionType.ClientTweetVideoFullscreenV2Impression)
|
||||
val FullscreenImage = new TweetImpressionBCEAdapter(
|
||||
ActionType.ClientTweetImageFullscreenV2Impression)
|
||||
}
|
||||
|
||||
class TweetImpressionBCEAdapter(actionType: ActionType) extends ImpressionBCEAdapter {
|
||||
override type ImpressedItem = Item.TweetInfo
|
||||
|
||||
override def toUUA(e: FlattenedEventLog): Seq[UnifiedUserAction] =
|
||||
(actionType, e.v2Impress, e.v1TweetIds, e.v1BreadcrumbTweetIds) match {
|
||||
case (ActionType.ClientTweetV2Impression, Some(v2Impress), Some(v1TweetIds), _) =>
|
||||
toUUAEvents(e, v2Impress, v1TweetIds)
|
||||
case (
|
||||
ActionType.ClientTweetVideoFullscreenV2Impression,
|
||||
Some(v2Impress),
|
||||
_,
|
||||
Some(v1BreadcrumbTweetIds)) =>
|
||||
toUUAEvents(e, v2Impress, v1BreadcrumbTweetIds)
|
||||
case (
|
||||
ActionType.ClientTweetImageFullscreenV2Impression,
|
||||
Some(v2Impress),
|
||||
_,
|
||||
Some(v1BreadcrumbTweetIds)) =>
|
||||
toUUAEvents(e, v2Impress, v1BreadcrumbTweetIds)
|
||||
case _ => Nil
|
||||
}
|
||||
|
||||
private def toUUAEvents(
|
||||
e: FlattenedEventLog,
|
||||
v2Impress: Impress,
|
||||
v1TweetIds: Seq[FlattenedServersideContextKey]
|
||||
): Seq[UnifiedUserAction] =
|
||||
v1TweetIds.map { tweet =>
|
||||
getUnifiedUserAction(
|
||||
event = e,
|
||||
actionType = actionType,
|
||||
item = getImpressedItem(tweet, v2Impress),
|
||||
productSurface = getProductSurfaceRelated.productSurface,
|
||||
productSurfaceInfo = getProductSurfaceRelated.productSurfaceInfo
|
||||
)
|
||||
}
|
||||
|
||||
override def getImpressedItem(
|
||||
context: FlattenedServersideContextKey,
|
||||
impression: Impress
|
||||
): ImpressedItem =
|
||||
Item.TweetInfo(
|
||||
TweetInfo(
|
||||
actionTweetId = context.serversideContextId.toLong,
|
||||
tweetActionInfo = Some(
|
||||
TweetActionInfo.ClientTweetV2Impression(
|
||||
ClientTweetV2Impression(
|
||||
impressStartTimestampMs = getImpressedStartTimestamp(impression),
|
||||
impressEndTimestampMs = getImpressedEndTimestamp(impression),
|
||||
sourceComponent = getImpressedUISourceComponent(context)
|
||||
)
|
||||
))
|
||||
))
|
||||
|
||||
private def getProductSurfaceRelated: ProductSurfaceRelated =
|
||||
actionType match {
|
||||
case ActionType.ClientTweetV2Impression =>
|
||||
ProductSurfaceRelated(
|
||||
productSurface = Some(ProductSurface.TweetDetailsPage),
|
||||
productSurfaceInfo = None)
|
||||
case _ => ProductSurfaceRelated(productSurface = None, productSurfaceInfo = None)
|
||||
}
|
||||
}
|
|
@ -0,0 +1,16 @@
|
|||
scala_library(
|
||||
sources = [
|
||||
"*.scala",
|
||||
],
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"common-internal/analytics/client-analytics-data-layer/src/main/scala",
|
||||
"kafka/finagle-kafka/finatra-kafka/src/main/scala",
|
||||
"src/scala/com/twitter/loggedout/analytics/common",
|
||||
"src/thrift/com/twitter/clientapp/gen:clientapp-scala",
|
||||
"twadoop_config/configuration/log_categories/group/scribelib:client_event-scala",
|
||||
"unified_user_actions/adapter/src/main/scala/com/twitter/unified_user_actions/adapter:base",
|
||||
"unified_user_actions/adapter/src/main/scala/com/twitter/unified_user_actions/adapter/common",
|
||||
"unified_user_actions/thrift/src/main/thrift/com/twitter/unified_user_actions:unified_user_actions-scala",
|
||||
],
|
||||
)
|
|
@ -0,0 +1,46 @@
|
|||
package com.twitter.unified_user_actions.adapter.client_event
|
||||
|
||||
import com.twitter.clientapp.thriftscala.LogEvent
|
||||
import com.twitter.logbase.thriftscala.LogBase
|
||||
import com.twitter.unified_user_actions.thriftscala.ActionType
|
||||
import com.twitter.unified_user_actions.thriftscala.Item
|
||||
import com.twitter.unified_user_actions.thriftscala.UnifiedUserAction
|
||||
import com.twitter.unified_user_actions.thriftscala._
|
||||
import com.twitter.clientapp.thriftscala.{Item => LogEventItem}
|
||||
|
||||
abstract class BaseCTAClientEvent(actionType: ActionType)
|
||||
extends BaseClientEvent(actionType = actionType) {
|
||||
|
||||
override def toUnifiedUserAction(logEvent: LogEvent): Seq[UnifiedUserAction] = {
|
||||
val logBase: Option[LogBase] = logEvent.logBase
|
||||
val userIdentifier: UserIdentifier = UserIdentifier(
|
||||
userId = logBase.flatMap(_.userId),
|
||||
guestIdMarketing = logBase.flatMap(_.guestIdMarketing))
|
||||
val uuaItem: Item = Item.CtaInfo(CTAInfo())
|
||||
val eventTimestamp = logBase.flatMap(getSourceTimestamp).getOrElse(0L)
|
||||
val ceItem = LogEventItem.unsafeEmpty
|
||||
|
||||
val productSurface: Option[ProductSurface] = ProductSurfaceUtils
|
||||
.getProductSurface(logEvent.eventNamespace)
|
||||
|
||||
val eventMetaData: EventMetadata = ClientEventCommonUtils
|
||||
.getEventMetadata(
|
||||
eventTimestamp = eventTimestamp,
|
||||
logEvent = logEvent,
|
||||
ceItem = ceItem,
|
||||
productSurface = productSurface
|
||||
)
|
||||
|
||||
Seq(
|
||||
UnifiedUserAction(
|
||||
userIdentifier = userIdentifier,
|
||||
item = uuaItem,
|
||||
actionType = actionType,
|
||||
eventMetadata = eventMetaData,
|
||||
productSurface = productSurface,
|
||||
productSurfaceInfo =
|
||||
ProductSurfaceUtils.getProductSurfaceInfo(productSurface, ceItem, logEvent)
|
||||
))
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,26 @@
|
|||
package com.twitter.unified_user_actions.adapter.client_event
|
||||
|
||||
import com.twitter.clientapp.thriftscala.LogEvent
|
||||
import com.twitter.clientapp.thriftscala.{Item => LogEventItem}
|
||||
import com.twitter.clientapp.thriftscala.ItemType
|
||||
import com.twitter.unified_user_actions.thriftscala.ActionType
|
||||
import com.twitter.unified_user_actions.thriftscala.CardInfo
|
||||
import com.twitter.unified_user_actions.thriftscala.Item
|
||||
|
||||
abstract class BaseCardClientEvent(actionType: ActionType)
|
||||
extends BaseClientEvent(actionType = actionType) {
|
||||
|
||||
override def isItemTypeValid(itemTypeOpt: Option[ItemType]): Boolean =
|
||||
ItemTypeFilterPredicates.ignoreItemType(itemTypeOpt)
|
||||
override def getUuaItem(
|
||||
ceItem: LogEventItem,
|
||||
logEvent: LogEvent
|
||||
): Option[Item] = Some(
|
||||
Item.CardInfo(
|
||||
CardInfo(
|
||||
id = ceItem.id,
|
||||
itemType = ceItem.itemType,
|
||||
actionTweetAuthorInfo = ClientEventCommonUtils.getAuthorInfo(ceItem),
|
||||
))
|
||||
)
|
||||
}
|
|
@ -0,0 +1,68 @@
|
|||
package com.twitter.unified_user_actions.adapter.client_event
|
||||
|
||||
import com.twitter.clientapp.thriftscala.ItemType
|
||||
import com.twitter.clientapp.thriftscala.LogEvent
|
||||
import com.twitter.clientapp.thriftscala.{Item => LogEventItem}
|
||||
import com.twitter.logbase.thriftscala.ClientEventReceiver
|
||||
import com.twitter.logbase.thriftscala.LogBase
|
||||
import com.twitter.unified_user_actions.thriftscala._
|
||||
|
||||
abstract class BaseClientEvent(actionType: ActionType) {
|
||||
def toUnifiedUserAction(logEvent: LogEvent): Seq[UnifiedUserAction] = {
|
||||
val logBase: Option[LogBase] = logEvent.logBase
|
||||
|
||||
for {
|
||||
ed <- logEvent.eventDetails.toSeq
|
||||
items <- ed.items.toSeq
|
||||
ceItem <- items
|
||||
eventTimestamp <- logBase.flatMap(getSourceTimestamp)
|
||||
uuaItem <- getUuaItem(ceItem, logEvent)
|
||||
if isItemTypeValid(ceItem.itemType)
|
||||
} yield {
|
||||
val userIdentifier: UserIdentifier = UserIdentifier(
|
||||
userId = logBase.flatMap(_.userId),
|
||||
guestIdMarketing = logBase.flatMap(_.guestIdMarketing))
|
||||
|
||||
val productSurface: Option[ProductSurface] = ProductSurfaceUtils
|
||||
.getProductSurface(logEvent.eventNamespace)
|
||||
|
||||
val eventMetaData: EventMetadata = ClientEventCommonUtils
|
||||
.getEventMetadata(
|
||||
eventTimestamp = eventTimestamp,
|
||||
logEvent = logEvent,
|
||||
ceItem = ceItem,
|
||||
productSurface = productSurface
|
||||
)
|
||||
|
||||
UnifiedUserAction(
|
||||
userIdentifier = userIdentifier,
|
||||
item = uuaItem,
|
||||
actionType = actionType,
|
||||
eventMetadata = eventMetaData,
|
||||
productSurface = productSurface,
|
||||
productSurfaceInfo =
|
||||
ProductSurfaceUtils.getProductSurfaceInfo(productSurface, ceItem, logEvent)
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
def getUuaItem(
|
||||
ceItem: LogEventItem,
|
||||
logEvent: LogEvent
|
||||
): Option[Item] = for (actionTweetId <- ceItem.id)
|
||||
yield Item.TweetInfo(
|
||||
ClientEventCommonUtils
|
||||
.getBasicTweetInfo(actionTweetId, ceItem, logEvent.eventNamespace))
|
||||
|
||||
// default implementation filters items of type tweet
|
||||
// override in the subclass implementation to filter items of other types
|
||||
def isItemTypeValid(itemTypeOpt: Option[ItemType]): Boolean =
|
||||
ItemTypeFilterPredicates.isItemTypeTweet(itemTypeOpt)
|
||||
|
||||
def getSourceTimestamp(logBase: LogBase): Option[Long] =
|
||||
logBase.clientEventReceiver match {
|
||||
case Some(ClientEventReceiver.CesHttp) | Some(ClientEventReceiver.CesThrift) =>
|
||||
logBase.driftAdjustedEventCreatedAtMs
|
||||
case _ => Some(logBase.driftAdjustedEventCreatedAtMs.getOrElse(logBase.timestamp))
|
||||
}
|
||||
}
|
|
@ -0,0 +1,46 @@
|
|||
package com.twitter.unified_user_actions.adapter.client_event
|
||||
|
||||
import com.twitter.clientapp.thriftscala.ItemType
|
||||
import com.twitter.clientapp.thriftscala.LogEvent
|
||||
import com.twitter.clientapp.thriftscala.{Item => LogEventItem}
|
||||
import com.twitter.unified_user_actions.thriftscala._
|
||||
|
||||
abstract class BaseFeedbackSubmitClientEvent(actionType: ActionType)
|
||||
extends BaseClientEvent(actionType = actionType) {
|
||||
|
||||
override def getUuaItem(
|
||||
ceItem: LogEventItem,
|
||||
logEvent: LogEvent
|
||||
): Option[Item] = {
|
||||
logEvent.eventNamespace.flatMap(_.page).flatMap {
|
||||
case "search" =>
|
||||
val searchInfoUtil = new SearchInfoUtils(ceItem)
|
||||
searchInfoUtil.getQueryOptFromItem(logEvent).flatMap { query =>
|
||||
val isRelevant: Boolean = logEvent.eventNamespace
|
||||
.flatMap(_.element)
|
||||
.contains("is_relevant")
|
||||
logEvent.eventNamespace.flatMap(_.component).flatMap {
|
||||
case "relevance_prompt_module" =>
|
||||
for (actionTweetId <- ceItem.id)
|
||||
yield Item.FeedbackPromptInfo(
|
||||
FeedbackPromptInfo(
|
||||
feedbackPromptActionInfo = FeedbackPromptActionInfo.TweetRelevantToSearch(
|
||||
TweetRelevantToSearch(
|
||||
searchQuery = query,
|
||||
tweetId = actionTweetId,
|
||||
isRelevant = Some(isRelevant)))))
|
||||
case "did_you_find_it_module" =>
|
||||
Some(
|
||||
Item.FeedbackPromptInfo(FeedbackPromptInfo(feedbackPromptActionInfo =
|
||||
FeedbackPromptActionInfo.DidYouFindItSearch(
|
||||
DidYouFindItSearch(searchQuery = query, isRelevant = Some(isRelevant))))))
|
||||
}
|
||||
}
|
||||
case _ => None
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
override def isItemTypeValid(itemTypeOpt: Option[ItemType]): Boolean =
|
||||
ItemTypeFilterPredicates.isItemTypeForSearchResultsPageFeedbackSubmit(itemTypeOpt)
|
||||
}
|
|
@ -0,0 +1,48 @@
|
|||
package com.twitter.unified_user_actions.adapter.client_event
|
||||
|
||||
import com.twitter.clientapp.thriftscala.ItemType
|
||||
import com.twitter.clientapp.thriftscala.LogEvent
|
||||
import com.twitter.clientapp.thriftscala.{Item => LogEventItem}
|
||||
import com.twitter.unified_user_actions.thriftscala._
|
||||
|
||||
abstract class BaseNotificationTabClientEvent(actionType: ActionType)
|
||||
extends BaseClientEvent(actionType = actionType) {
|
||||
|
||||
// itemType is `None` for Notification Tab events
|
||||
override def isItemTypeValid(itemTypeOpt: Option[ItemType]): Boolean =
|
||||
ItemTypeFilterPredicates.ignoreItemType(itemTypeOpt)
|
||||
|
||||
override def getUuaItem(
|
||||
ceItem: LogEventItem,
|
||||
logEvent: LogEvent
|
||||
): Option[Item] = for {
|
||||
notificationTabDetails <- ceItem.notificationTabDetails
|
||||
clientEventMetadata <- notificationTabDetails.clientEventMetadata
|
||||
notificationId <- NotificationClientEventUtils.getNotificationIdForNotificationTab(ceItem)
|
||||
} yield {
|
||||
clientEventMetadata.tweetIds match {
|
||||
// if `tweetIds` contain more than one Tweet id, create `MultiTweetNotification`
|
||||
case Some(tweetIds) if tweetIds.size > 1 =>
|
||||
Item.NotificationInfo(
|
||||
NotificationInfo(
|
||||
actionNotificationId = notificationId,
|
||||
content = NotificationContent.MultiTweetNotification(
|
||||
MultiTweetNotification(tweetIds = tweetIds))
|
||||
))
|
||||
// if `tweetIds` contain exactly one Tweet id, create `TweetNotification`
|
||||
case Some(tweetIds) if tweetIds.size == 1 =>
|
||||
Item.NotificationInfo(
|
||||
NotificationInfo(
|
||||
actionNotificationId = notificationId,
|
||||
content =
|
||||
NotificationContent.TweetNotification(TweetNotification(tweetId = tweetIds.head))))
|
||||
// if `tweetIds` are missing, create `UnknownNotification`
|
||||
case _ =>
|
||||
Item.NotificationInfo(
|
||||
NotificationInfo(
|
||||
actionNotificationId = notificationId,
|
||||
content = NotificationContent.UnknownNotification(UnknownNotification())
|
||||
))
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,25 @@
|
|||
package com.twitter.unified_user_actions.adapter.client_event
|
||||
|
||||
import com.twitter.clientapp.thriftscala.ItemType
|
||||
import com.twitter.clientapp.thriftscala.LogEvent
|
||||
import com.twitter.clientapp.thriftscala.{Item => LogEventItem}
|
||||
import com.twitter.unified_user_actions.adapter.client_event.ClientEventCommonUtils.getProfileIdFromUserItem
|
||||
import com.twitter.unified_user_actions.thriftscala.ActionType
|
||||
import com.twitter.unified_user_actions.thriftscala.Item
|
||||
import com.twitter.unified_user_actions.thriftscala.ProfileInfo
|
||||
|
||||
abstract class BaseProfileClientEvent(actionType: ActionType)
|
||||
extends BaseClientEvent(actionType = actionType) {
|
||||
override def isItemTypeValid(itemTypeOpt: Option[ItemType]): Boolean =
|
||||
ItemTypeFilterPredicates.isItemTypeProfile(itemTypeOpt)
|
||||
|
||||
override def getUuaItem(
|
||||
ceItem: LogEventItem,
|
||||
logEvent: LogEvent
|
||||
): Option[Item] =
|
||||
getProfileIdFromUserItem(ceItem).map { id =>
|
||||
Item.ProfileInfo(
|
||||
ProfileInfo(actionProfileId = id)
|
||||
)
|
||||
}
|
||||
}
|
|
@ -0,0 +1,22 @@
|
|||
package com.twitter.unified_user_actions.adapter.client_event
|
||||
|
||||
import com.twitter.clientapp.thriftscala.LogEvent
|
||||
import com.twitter.clientapp.thriftscala.{Item => LogEventItem}
|
||||
import com.twitter.unified_user_actions.thriftscala._
|
||||
|
||||
abstract class BasePushNotificationClientEvent(actionType: ActionType)
|
||||
extends BaseClientEvent(actionType = actionType) {
|
||||
|
||||
override def getUuaItem(
|
||||
ceItem: LogEventItem,
|
||||
logEvent: LogEvent
|
||||
): Option[Item] = for {
|
||||
itemId <- ceItem.id
|
||||
notificationId <- NotificationClientEventUtils.getNotificationIdForPushNotification(logEvent)
|
||||
} yield {
|
||||
Item.NotificationInfo(
|
||||
NotificationInfo(
|
||||
actionNotificationId = notificationId,
|
||||
content = NotificationContent.TweetNotification(TweetNotification(tweetId = itemId))))
|
||||
}
|
||||
}
|
|
@ -0,0 +1,87 @@
|
|||
package com.twitter.unified_user_actions.adapter.client_event
|
||||
|
||||
import com.twitter.clientapp.thriftscala.ItemType
|
||||
import com.twitter.clientapp.thriftscala.LogEvent
|
||||
import com.twitter.clientapp.thriftscala.{Item => LogEventItem}
|
||||
import com.twitter.logbase.thriftscala.LogBase
|
||||
import com.twitter.unified_user_actions.adapter.client_event.ClientEventCommonUtils.getProfileIdFromUserItem
|
||||
import com.twitter.unified_user_actions.thriftscala.ActionType
|
||||
import com.twitter.unified_user_actions.thriftscala.EventMetadata
|
||||
import com.twitter.unified_user_actions.thriftscala.Item
|
||||
import com.twitter.unified_user_actions.thriftscala.ProductSurface
|
||||
import com.twitter.unified_user_actions.thriftscala.TopicQueryResult
|
||||
import com.twitter.unified_user_actions.thriftscala.TypeaheadActionInfo
|
||||
import com.twitter.unified_user_actions.thriftscala.TypeaheadInfo
|
||||
import com.twitter.unified_user_actions.thriftscala.UnifiedUserAction
|
||||
import com.twitter.unified_user_actions.thriftscala.UserIdentifier
|
||||
import com.twitter.unified_user_actions.thriftscala.UserResult
|
||||
|
||||
abstract class BaseSearchTypeaheadEvent(actionType: ActionType)
|
||||
extends BaseClientEvent(actionType = actionType) {
|
||||
|
||||
override def toUnifiedUserAction(logEvent: LogEvent): Seq[UnifiedUserAction] = {
|
||||
val logBase: Option[LogBase] = logEvent.logBase
|
||||
|
||||
for {
|
||||
ed <- logEvent.eventDetails.toSeq
|
||||
targets <- ed.targets.toSeq
|
||||
ceTarget <- targets
|
||||
eventTimestamp <- logBase.flatMap(getSourceTimestamp)
|
||||
uuaItem <- getUuaItem(ceTarget, logEvent)
|
||||
if isItemTypeValid(ceTarget.itemType)
|
||||
} yield {
|
||||
val userIdentifier: UserIdentifier = UserIdentifier(
|
||||
userId = logBase.flatMap(_.userId),
|
||||
guestIdMarketing = logBase.flatMap(_.guestIdMarketing))
|
||||
|
||||
val productSurface: Option[ProductSurface] = ProductSurfaceUtils
|
||||
.getProductSurface(logEvent.eventNamespace)
|
||||
|
||||
val eventMetaData: EventMetadata = ClientEventCommonUtils
|
||||
.getEventMetadata(
|
||||
eventTimestamp = eventTimestamp,
|
||||
logEvent = logEvent,
|
||||
ceItem = ceTarget,
|
||||
productSurface = productSurface
|
||||
)
|
||||
|
||||
UnifiedUserAction(
|
||||
userIdentifier = userIdentifier,
|
||||
item = uuaItem,
|
||||
actionType = actionType,
|
||||
eventMetadata = eventMetaData,
|
||||
productSurface = productSurface,
|
||||
productSurfaceInfo =
|
||||
ProductSurfaceUtils.getProductSurfaceInfo(productSurface, ceTarget, logEvent)
|
||||
)
|
||||
}
|
||||
}
|
||||
override def isItemTypeValid(itemTypeOpt: Option[ItemType]): Boolean =
|
||||
ItemTypeFilterPredicates.isItemTypeTypeaheadResult(itemTypeOpt)
|
||||
|
||||
override def getUuaItem(
|
||||
ceTarget: LogEventItem,
|
||||
logEvent: LogEvent
|
||||
): Option[Item] =
|
||||
logEvent.searchDetails.flatMap(_.query).flatMap { query =>
|
||||
ceTarget.itemType match {
|
||||
case Some(ItemType.User) =>
|
||||
getProfileIdFromUserItem(ceTarget).map { profileId =>
|
||||
Item.TypeaheadInfo(
|
||||
TypeaheadInfo(
|
||||
actionQuery = query,
|
||||
typeaheadActionInfo =
|
||||
TypeaheadActionInfo.UserResult(UserResult(profileId = profileId))))
|
||||
}
|
||||
case Some(ItemType.Search) =>
|
||||
ceTarget.name.map { name =>
|
||||
Item.TypeaheadInfo(
|
||||
TypeaheadInfo(
|
||||
actionQuery = query,
|
||||
typeaheadActionInfo = TypeaheadActionInfo.TopicQueryResult(
|
||||
TopicQueryResult(suggestedTopicQuery = name))))
|
||||
}
|
||||
case _ => None
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,23 @@
|
|||
package com.twitter.unified_user_actions.adapter.client_event
|
||||
|
||||
import com.twitter.clientapp.thriftscala.ItemType
|
||||
import com.twitter.clientapp.thriftscala.LogEvent
|
||||
import com.twitter.clientapp.thriftscala.{Item => LogEventItem}
|
||||
import com.twitter.unified_user_actions.thriftscala.ActionType
|
||||
import com.twitter.unified_user_actions.thriftscala.Item
|
||||
import com.twitter.unified_user_actions.thriftscala.TopicInfo
|
||||
|
||||
abstract class BaseTopicClientEvent(actionType: ActionType)
|
||||
extends BaseClientEvent(actionType = actionType) {
|
||||
override def isItemTypeValid(itemTypeOpt: Option[ItemType]): Boolean =
|
||||
ItemTypeFilterPredicates.isItemTypeTopic(itemTypeOpt)
|
||||
|
||||
override def getUuaItem(
|
||||
ceItem: LogEventItem,
|
||||
logEvent: LogEvent
|
||||
): Option[Item] =
|
||||
for (actionTopicId <- ClientEventCommonUtils.getTopicId(
|
||||
ceItem = ceItem,
|
||||
ceNamespaceOpt = logEvent.eventNamespace))
|
||||
yield Item.TopicInfo(TopicInfo(actionTopicId = actionTopicId))
|
||||
}
|
|
@ -0,0 +1,62 @@
|
|||
package com.twitter.unified_user_actions.adapter.client_event
|
||||
|
||||
import com.twitter.clientapp.thriftscala.LogEvent
|
||||
import com.twitter.clientapp.thriftscala.{Item => LogEventItem}
|
||||
import com.twitter.logbase.thriftscala.LogBase
|
||||
import com.twitter.unified_user_actions.thriftscala.ActionType
|
||||
import com.twitter.unified_user_actions.thriftscala.UnifiedUserAction
|
||||
import com.twitter.unified_user_actions.thriftscala._
|
||||
|
||||
abstract class BaseUASClientEvent(actionType: ActionType)
|
||||
extends BaseClientEvent(actionType = actionType) {
|
||||
|
||||
override def toUnifiedUserAction(logEvent: LogEvent): Seq[UnifiedUserAction] = {
|
||||
val logBase: Option[LogBase] = logEvent.logBase
|
||||
val ceItem = LogEventItem.unsafeEmpty
|
||||
|
||||
val uuaOpt: Option[UnifiedUserAction] = for {
|
||||
eventTimestamp <- logBase.flatMap(getSourceTimestamp)
|
||||
uuaItem <- getUuaItem(ceItem, logEvent)
|
||||
} yield {
|
||||
val userIdentifier: UserIdentifier = UserIdentifier(
|
||||
userId = logBase.flatMap(_.userId),
|
||||
guestIdMarketing = logBase.flatMap(_.guestIdMarketing))
|
||||
|
||||
val productSurface: Option[ProductSurface] = ProductSurfaceUtils
|
||||
.getProductSurface(logEvent.eventNamespace)
|
||||
|
||||
val eventMetaData: EventMetadata = ClientEventCommonUtils
|
||||
.getEventMetadata(
|
||||
eventTimestamp = eventTimestamp,
|
||||
logEvent = logEvent,
|
||||
ceItem = ceItem,
|
||||
productSurface = productSurface
|
||||
)
|
||||
|
||||
UnifiedUserAction(
|
||||
userIdentifier = userIdentifier,
|
||||
item = uuaItem,
|
||||
actionType = actionType,
|
||||
eventMetadata = eventMetaData,
|
||||
productSurface = productSurface,
|
||||
productSurfaceInfo =
|
||||
ProductSurfaceUtils.getProductSurfaceInfo(productSurface, ceItem, logEvent)
|
||||
)
|
||||
}
|
||||
|
||||
uuaOpt match {
|
||||
case Some(uua) => Seq(uua)
|
||||
case _ => Nil
|
||||
}
|
||||
}
|
||||
|
||||
override def getUuaItem(
|
||||
ceItem: LogEventItem,
|
||||
logEvent: LogEvent
|
||||
): Option[Item] = for {
|
||||
performanceDetails <- logEvent.performanceDetails
|
||||
duration <- performanceDetails.durationMs
|
||||
} yield {
|
||||
Item.UasInfo(UASInfo(timeSpentMs = duration))
|
||||
}
|
||||
}
|
|
@ -0,0 +1,34 @@
|
|||
package com.twitter.unified_user_actions.adapter.client_event
|
||||
|
||||
import com.twitter.clientapp.thriftscala.LogEvent
|
||||
import com.twitter.clientapp.thriftscala.{Item => LogEventItem}
|
||||
import com.twitter.unified_user_actions.thriftscala._
|
||||
|
||||
abstract class BaseVideoClientEvent(actionType: ActionType)
|
||||
extends BaseClientEvent(actionType = actionType) {
|
||||
|
||||
override def getUuaItem(
|
||||
ceItem: LogEventItem,
|
||||
logEvent: LogEvent
|
||||
): Option[Item] = for {
|
||||
actionTweetId <- ceItem.id
|
||||
clientMediaEvent <- ceItem.clientMediaEvent
|
||||
sessionState <- clientMediaEvent.sessionState
|
||||
mediaIdentifier <- sessionState.contentVideoIdentifier
|
||||
mediaId <- VideoClientEventUtils.videoIdFromMediaIdentifier(mediaIdentifier)
|
||||
mediaDetails <- ceItem.mediaDetailsV2
|
||||
mediaItems <- mediaDetails.mediaItems
|
||||
videoMetadata <- VideoClientEventUtils.getVideoMetadata(
|
||||
mediaId,
|
||||
mediaItems,
|
||||
ceItem.cardDetails.flatMap(_.amplifyDetails))
|
||||
} yield {
|
||||
Item.TweetInfo(
|
||||
ClientEventCommonUtils
|
||||
.getBasicTweetInfo(
|
||||
actionTweetId = actionTweetId,
|
||||
ceItem = ceItem,
|
||||
ceNamespaceOpt = logEvent.eventNamespace)
|
||||
.copy(tweetActionInfo = Some(videoMetadata)))
|
||||
}
|
||||
}
|
|
@ -0,0 +1,272 @@
|
|||
package com.twitter.unified_user_actions.adapter.client_event
|
||||
|
||||
import com.twitter.finagle.stats.NullStatsReceiver
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.clientapp.thriftscala.EventNamespace
|
||||
import com.twitter.clientapp.thriftscala.LogEvent
|
||||
import com.twitter.finatra.kafka.serde.UnKeyed
|
||||
import com.twitter.unified_user_actions.adapter.AbstractAdapter
|
||||
import com.twitter.unified_user_actions.adapter.client_event.ClientEventImpression._
|
||||
import com.twitter.unified_user_actions.adapter.client_event.ClientEventEngagement._
|
||||
import com.twitter.unified_user_actions.thriftscala.UnifiedUserAction
|
||||
import scala.util.matching.Regex
|
||||
|
||||
class ClientEventAdapter extends AbstractAdapter[LogEvent, UnKeyed, UnifiedUserAction] {
|
||||
import ClientEventAdapter._
|
||||
|
||||
override def adaptOneToKeyedMany(
|
||||
input: LogEvent,
|
||||
statsReceiver: StatsReceiver = NullStatsReceiver
|
||||
): Seq[(UnKeyed, UnifiedUserAction)] =
|
||||
adaptEvent(input).map { e => (UnKeyed, e) }
|
||||
}
|
||||
|
||||
object ClientEventAdapter {
|
||||
// Refer to go/cme-scribing and go/interaction-event-spec for details
|
||||
def isVideoEvent(element: String): Boolean = Seq[String](
|
||||
"gif_player",
|
||||
"periscope_player",
|
||||
"platform_amplify_card",
|
||||
"video_player",
|
||||
"vine_player").contains(element)
|
||||
|
||||
/**
|
||||
* Tweet clicks on the Notification Tab on iOS are a special case because the `element` is different
|
||||
* from Tweet clicks everywhere else on the platform.
|
||||
*
|
||||
* For Notification Tab on iOS, `element` could be one of `user_mentioned_you`,
|
||||
* `user_mentioned_you_in_a_quote_tweet`, `user_replied_to_your_tweet`, or `user_quoted_your_tweet`.
|
||||
*
|
||||
* In other places, `element` = `tweet`.
|
||||
*/
|
||||
def isTweetClickEvent(element: String): Boolean =
|
||||
Seq[String](
|
||||
"tweet",
|
||||
"user_mentioned_you",
|
||||
"user_mentioned_you_in_a_quote_tweet",
|
||||
"user_replied_to_your_tweet",
|
||||
"user_quoted_your_tweet"
|
||||
).contains(element)
|
||||
|
||||
final val validUASIosClientIds = Seq[Long](
|
||||
129032L, // Twitter for iPhone
|
||||
191841L // Twitter for iPad
|
||||
)
|
||||
// Twitter for Android
|
||||
final val validUASAndroidClientIds = Seq[Long](258901L)
|
||||
|
||||
def adaptEvent(inputLogEvent: LogEvent): Seq[UnifiedUserAction] =
|
||||
Option(inputLogEvent).toSeq
|
||||
.filterNot { logEvent: LogEvent =>
|
||||
shouldIgnoreClientEvent(logEvent.eventNamespace)
|
||||
}
|
||||
.flatMap { logEvent: LogEvent =>
|
||||
val actionTypesPerEvent: Seq[BaseClientEvent] = logEvent.eventNamespace.toSeq.flatMap {
|
||||
name =>
|
||||
(name.page, name.section, name.component, name.element, name.action) match {
|
||||
case (_, _, _, _, Some("favorite")) => Seq(TweetFav)
|
||||
case (_, _, _, _, Some("unfavorite")) => Seq(TweetUnfav)
|
||||
case (_, _, Some("stream"), Some("linger"), Some("results")) =>
|
||||
Seq(TweetLingerImpression)
|
||||
case (_, _, Some("stream"), None, Some("results")) =>
|
||||
Seq(TweetRenderImpression)
|
||||
case (_, _, _, _, Some("send_reply")) => Seq(TweetReply)
|
||||
// Different clients may have different actions of the same "send quote"
|
||||
// but it turns out that both send_quote and retweet_with_comment should correspond to
|
||||
// "send quote"
|
||||
case (_, _, _, _, Some("send_quote_tweet")) |
|
||||
(_, _, _, _, Some("retweet_with_comment")) =>
|
||||
Seq(TweetQuote)
|
||||
case (_, _, _, _, Some("retweet")) => Seq(TweetRetweet)
|
||||
case (_, _, _, _, Some("unretweet")) => Seq(TweetUnretweet)
|
||||
case (_, _, _, _, Some("reply")) => Seq(TweetClickReply)
|
||||
case (_, _, _, _, Some("quote")) => Seq(TweetClickQuote)
|
||||
case (_, _, _, Some(element), Some("playback_start")) if isVideoEvent(element) =>
|
||||
Seq(TweetVideoPlaybackStart)
|
||||
case (_, _, _, Some(element), Some("playback_complete")) if isVideoEvent(element) =>
|
||||
Seq(TweetVideoPlaybackComplete)
|
||||
case (_, _, _, Some(element), Some("playback_25")) if isVideoEvent(element) =>
|
||||
Seq(TweetVideoPlayback25)
|
||||
case (_, _, _, Some(element), Some("playback_50")) if isVideoEvent(element) =>
|
||||
Seq(TweetVideoPlayback50)
|
||||
case (_, _, _, Some(element), Some("playback_75")) if isVideoEvent(element) =>
|
||||
Seq(TweetVideoPlayback75)
|
||||
case (_, _, _, Some(element), Some("playback_95")) if isVideoEvent(element) =>
|
||||
Seq(TweetVideoPlayback95)
|
||||
case (_, _, _, Some(element), Some("play_from_tap")) if isVideoEvent(element) =>
|
||||
Seq(TweetVideoPlayFromTap)
|
||||
case (_, _, _, Some(element), Some("video_quality_view")) if isVideoEvent(element) =>
|
||||
Seq(TweetVideoQualityView)
|
||||
case (_, _, _, Some(element), Some("video_view")) if isVideoEvent(element) =>
|
||||
Seq(TweetVideoView)
|
||||
case (_, _, _, Some(element), Some("video_mrc_view")) if isVideoEvent(element) =>
|
||||
Seq(TweetVideoMrcView)
|
||||
case (_, _, _, Some(element), Some("view_threshold")) if isVideoEvent(element) =>
|
||||
Seq(TweetVideoViewThreshold)
|
||||
case (_, _, _, Some(element), Some("cta_url_click")) if isVideoEvent(element) =>
|
||||
Seq(TweetVideoCtaUrlClick)
|
||||
case (_, _, _, Some(element), Some("cta_watch_click")) if isVideoEvent(element) =>
|
||||
Seq(TweetVideoCtaWatchClick)
|
||||
case (_, _, _, Some("platform_photo_card"), Some("click")) => Seq(TweetPhotoExpand)
|
||||
case (_, _, _, Some("platform_card"), Some("click")) => Seq(CardClick)
|
||||
case (_, _, _, _, Some("open_app")) => Seq(CardOpenApp)
|
||||
case (_, _, _, _, Some("install_app")) => Seq(CardAppInstallAttempt)
|
||||
case (_, _, _, Some("platform_card"), Some("vote")) |
|
||||
(_, _, _, Some("platform_forward_card"), Some("vote")) =>
|
||||
Seq(PollCardVote)
|
||||
case (_, _, _, Some("mention"), Some("click")) |
|
||||
(_, _, _, _, Some("mention_click")) =>
|
||||
Seq(TweetClickMentionScreenName)
|
||||
case (_, _, _, Some(element), Some("click")) if isTweetClickEvent(element) =>
|
||||
Seq(TweetClick)
|
||||
case // Follow from the Topic page (or so-called landing page)
|
||||
(_, _, _, Some("topic"), Some("follow")) |
|
||||
// Actually not sure how this is generated ... but saw quite some events in BQ
|
||||
(_, _, _, Some("social_proof"), Some("follow")) |
|
||||
// Click on Tweet's caret menu of "Follow (the topic)", it needs to be:
|
||||
// 1) user follows the Topic already, 2) and clicked on the "Unfollow Topic" first.
|
||||
(_, _, _, Some("feedback_follow_topic"), Some("click")) =>
|
||||
Seq(TopicFollow)
|
||||
case (_, _, _, Some("topic"), Some("unfollow")) |
|
||||
(_, _, _, Some("social_proof"), Some("unfollow")) |
|
||||
(_, _, _, Some("feedback_unfollow_topic"), Some("click")) =>
|
||||
Seq(TopicUnfollow)
|
||||
case (_, _, _, Some("topic"), Some("not_interested")) |
|
||||
(_, _, _, Some("feedback_not_interested_in_topic"), Some("click")) =>
|
||||
Seq(TopicNotInterestedIn)
|
||||
case (_, _, _, Some("topic"), Some("un_not_interested")) |
|
||||
(_, _, _, Some("feedback_not_interested_in_topic"), Some("undo")) =>
|
||||
Seq(TopicUndoNotInterestedIn)
|
||||
case (_, _, _, Some("feedback_givefeedback"), Some("click")) =>
|
||||
Seq(TweetNotHelpful)
|
||||
case (_, _, _, Some("feedback_givefeedback"), Some("undo")) =>
|
||||
Seq(TweetUndoNotHelpful)
|
||||
case (_, _, _, Some("report_tweet"), Some("click")) |
|
||||
(_, _, _, Some("report_tweet"), Some("done")) =>
|
||||
Seq(TweetReport)
|
||||
case (_, _, _, Some("feedback_dontlike"), Some("click")) =>
|
||||
Seq(TweetNotInterestedIn)
|
||||
case (_, _, _, Some("feedback_dontlike"), Some("undo")) =>
|
||||
Seq(TweetUndoNotInterestedIn)
|
||||
case (_, _, _, Some("feedback_notabouttopic"), Some("click")) =>
|
||||
Seq(TweetNotAboutTopic)
|
||||
case (_, _, _, Some("feedback_notabouttopic"), Some("undo")) =>
|
||||
Seq(TweetUndoNotAboutTopic)
|
||||
case (_, _, _, Some("feedback_notrecent"), Some("click")) =>
|
||||
Seq(TweetNotRecent)
|
||||
case (_, _, _, Some("feedback_notrecent"), Some("undo")) =>
|
||||
Seq(TweetUndoNotRecent)
|
||||
case (_, _, _, Some("feedback_seefewer"), Some("click")) =>
|
||||
Seq(TweetSeeFewer)
|
||||
case (_, _, _, Some("feedback_seefewer"), Some("undo")) =>
|
||||
Seq(TweetUndoSeeFewer)
|
||||
// Only when action = "submit" we get all fields in ReportDetails, such as reportType
|
||||
// See https://confluence.twitter.biz/pages/viewpage.action?spaceKey=HEALTH&title=Understanding+ReportDetails
|
||||
case (Some(page), _, _, Some("ticket"), Some("submit"))
|
||||
if page.startsWith("report_") =>
|
||||
Seq(TweetReportServer)
|
||||
case (Some("profile"), _, _, _, Some("block")) =>
|
||||
Seq(ProfileBlock)
|
||||
case (Some("profile"), _, _, _, Some("unblock")) =>
|
||||
Seq(ProfileUnblock)
|
||||
case (Some("profile"), _, _, _, Some("mute_user")) =>
|
||||
Seq(ProfileMute)
|
||||
case (Some("profile"), _, _, _, Some("report")) =>
|
||||
Seq(ProfileReport)
|
||||
case (Some("profile"), _, _, _, Some("show")) =>
|
||||
Seq(ProfileShow)
|
||||
case (_, _, _, Some("follow"), Some("click")) => Seq(TweetFollowAuthor)
|
||||
case (_, _, _, _, Some("follow")) => Seq(TweetFollowAuthor, ProfileFollow)
|
||||
case (_, _, _, Some("unfollow"), Some("click")) => Seq(TweetUnfollowAuthor)
|
||||
case (_, _, _, _, Some("unfollow")) => Seq(TweetUnfollowAuthor)
|
||||
case (_, _, _, Some("block"), Some("click")) => Seq(TweetBlockAuthor)
|
||||
case (_, _, _, Some("unblock"), Some("click")) => Seq(TweetUnblockAuthor)
|
||||
case (_, _, _, Some("mute"), Some("click")) => Seq(TweetMuteAuthor)
|
||||
case (_, _, _, Some(element), Some("click")) if isTweetClickEvent(element) =>
|
||||
Seq(TweetClick)
|
||||
case (_, _, _, _, Some("profile_click")) => Seq(TweetClickProfile, ProfileClick)
|
||||
case (_, _, _, _, Some("share_menu_click")) => Seq(TweetClickShare)
|
||||
case (_, _, _, _, Some("copy_link")) => Seq(TweetShareViaCopyLink)
|
||||
case (_, _, _, _, Some("share_via_dm")) => Seq(TweetClickSendViaDirectMessage)
|
||||
case (_, _, _, _, Some("bookmark")) => Seq(TweetShareViaBookmark, TweetBookmark)
|
||||
case (_, _, _, _, Some("unbookmark")) => Seq(TweetUnbookmark)
|
||||
case (_, _, _, _, Some("hashtag_click")) |
|
||||
// This scribe is triggered on mobile platforms (android/iphone) when user click on hashtag in a tweet.
|
||||
(_, _, _, Some("hashtag"), Some("search")) =>
|
||||
Seq(TweetClickHashtag)
|
||||
case (_, _, _, _, Some("open_link")) => Seq(TweetOpenLink)
|
||||
case (_, _, _, _, Some("take_screenshot")) => Seq(TweetTakeScreenshot)
|
||||
case (_, _, _, Some("feedback_notrelevant"), Some("click")) =>
|
||||
Seq(TweetNotRelevant)
|
||||
case (_, _, _, Some("feedback_notrelevant"), Some("undo")) =>
|
||||
Seq(TweetUndoNotRelevant)
|
||||
case (_, _, _, _, Some("follow_attempt")) => Seq(ProfileFollowAttempt)
|
||||
case (_, _, _, _, Some("favorite_attempt")) => Seq(TweetFavoriteAttempt)
|
||||
case (_, _, _, _, Some("retweet_attempt")) => Seq(TweetRetweetAttempt)
|
||||
case (_, _, _, _, Some("reply_attempt")) => Seq(TweetReplyAttempt)
|
||||
case (_, _, _, _, Some("login")) => Seq(CTALoginClick)
|
||||
case (Some("login"), _, _, _, Some("show")) => Seq(CTALoginStart)
|
||||
case (Some("login"), _, _, _, Some("success")) => Seq(CTALoginSuccess)
|
||||
case (_, _, _, _, Some("signup")) => Seq(CTASignupClick)
|
||||
case (Some("signup"), _, _, _, Some("success")) => Seq(CTASignupSuccess)
|
||||
case // Android app running in the background
|
||||
(Some("notification"), Some("status_bar"), None, _, Some("background_open")) |
|
||||
// Android app running in the foreground
|
||||
(Some("notification"), Some("status_bar"), None, _, Some("open")) |
|
||||
// iOS app running in the background
|
||||
(Some("notification"), Some("notification_center"), None, _, Some("open")) |
|
||||
// iOS app running in the foreground
|
||||
(None, Some("toasts"), Some("social"), Some("favorite"), Some("open")) |
|
||||
// m5
|
||||
(Some("app"), Some("push"), _, _, Some("open")) =>
|
||||
Seq(NotificationOpen)
|
||||
case (Some("ntab"), Some("all"), Some("urt"), _, Some("navigate")) =>
|
||||
Seq(NotificationClick)
|
||||
case (Some("ntab"), Some("all"), Some("urt"), _, Some("see_less_often")) =>
|
||||
Seq(NotificationSeeLessOften)
|
||||
case (Some("notification"), Some("status_bar"), None, _, Some("background_dismiss")) |
|
||||
(Some("notification"), Some("status_bar"), None, _, Some("dismiss")) | (
|
||||
Some("notification"),
|
||||
Some("notification_center"),
|
||||
None,
|
||||
_,
|
||||
Some("dismiss")
|
||||
) =>
|
||||
Seq(NotificationDismiss)
|
||||
case (_, _, _, Some("typeahead"), Some("click")) => Seq(TypeaheadClick)
|
||||
case (Some("search"), _, Some(component), _, Some("click"))
|
||||
if component == "relevance_prompt_module" || component == "did_you_find_it_module" =>
|
||||
Seq(FeedbackPromptSubmit)
|
||||
case (Some("app"), Some("enter_background"), _, _, Some("become_inactive"))
|
||||
if logEvent.logBase
|
||||
.flatMap(_.clientAppId)
|
||||
.exists(validUASIosClientIds.contains(_)) =>
|
||||
Seq(AppExit)
|
||||
case (Some("app"), _, _, _, Some("become_inactive"))
|
||||
if logEvent.logBase
|
||||
.flatMap(_.clientAppId)
|
||||
.exists(validUASAndroidClientIds.contains(_)) =>
|
||||
Seq(AppExit)
|
||||
case (_, _, Some("gallery"), Some("photo"), Some("impression")) =>
|
||||
Seq(TweetGalleryImpression)
|
||||
case (_, _, _, _, _)
|
||||
if TweetDetailsImpression.isTweetDetailsImpression(logEvent.eventNamespace) =>
|
||||
Seq(TweetDetailsImpression)
|
||||
case _ => Nil
|
||||
}
|
||||
}
|
||||
actionTypesPerEvent.map(_.toUnifiedUserAction(logEvent))
|
||||
}.flatten
|
||||
|
||||
def shouldIgnoreClientEvent(eventNamespace: Option[EventNamespace]): Boolean =
|
||||
eventNamespace.exists { name =>
|
||||
(name.page, name.section, name.component, name.element, name.action) match {
|
||||
case (Some("ddg"), _, _, _, Some("experiment")) => true
|
||||
case (Some("qig_ranker"), _, _, _, _) => true
|
||||
case (Some("timelinemixer"), _, _, _, _) => true
|
||||
case (Some("timelineservice"), _, _, _, _) => true
|
||||
case (Some("tweetconvosvc"), _, _, _, _) => true
|
||||
case _ => false
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,169 @@
|
|||
package com.twitter.unified_user_actions.adapter.client_event
|
||||
|
||||
import com.twitter.clientapp.thriftscala.EventNamespace
|
||||
import com.twitter.clientapp.thriftscala.Item
|
||||
import com.twitter.clientapp.thriftscala.ItemType.User
|
||||
import com.twitter.clientapp.thriftscala.LogEvent
|
||||
import com.twitter.clientapp.thriftscala.{Item => LogEventItem}
|
||||
import com.twitter.unified_user_actions.adapter.common.AdapterUtils
|
||||
import com.twitter.unified_user_actions.thriftscala.AuthorInfo
|
||||
import com.twitter.unified_user_actions.thriftscala.ClientEventNamespace
|
||||
import com.twitter.unified_user_actions.thriftscala.EventMetadata
|
||||
import com.twitter.unified_user_actions.thriftscala.ProductSurface
|
||||
import com.twitter.unified_user_actions.thriftscala.SourceLineage
|
||||
import com.twitter.unified_user_actions.thriftscala.TweetAuthorFollowClickSource
|
||||
import com.twitter.unified_user_actions.thriftscala.TweetAuthorUnfollowClickSource
|
||||
import com.twitter.unified_user_actions.thriftscala.TweetInfo
|
||||
|
||||
/**
|
||||
* Comprises helper methods that:
|
||||
* 1. need not be overridden by subclasses of `BaseClientEvent`
|
||||
* 2. need not be invoked by instances of subclasses of `BaseClientEvent`
|
||||
* 3. need to be accessible to subclasses of `BaseClientEvent` and other utils
|
||||
*/
|
||||
object ClientEventCommonUtils {
|
||||
|
||||
def getBasicTweetInfo(
|
||||
actionTweetId: Long,
|
||||
ceItem: LogEventItem,
|
||||
ceNamespaceOpt: Option[EventNamespace]
|
||||
): TweetInfo = TweetInfo(
|
||||
actionTweetId = actionTweetId,
|
||||
actionTweetTopicSocialProofId = getTopicId(ceItem, ceNamespaceOpt),
|
||||
retweetingTweetId = ceItem.tweetDetails.flatMap(_.retweetingTweetId),
|
||||
quotedTweetId = ceItem.tweetDetails.flatMap(_.quotedTweetId),
|
||||
inReplyToTweetId = ceItem.tweetDetails.flatMap(_.inReplyToTweetId),
|
||||
quotingTweetId = ceItem.tweetDetails.flatMap(_.quotingTweetId),
|
||||
// only set AuthorInfo when authorId is present
|
||||
actionTweetAuthorInfo = getAuthorInfo(ceItem),
|
||||
retweetingAuthorId = ceItem.tweetDetails.flatMap(_.retweetAuthorId),
|
||||
quotedAuthorId = ceItem.tweetDetails.flatMap(_.quotedAuthorId),
|
||||
inReplyToAuthorId = ceItem.tweetDetails.flatMap(_.inReplyToAuthorId),
|
||||
tweetPosition = ceItem.position,
|
||||
promotedId = ceItem.promotedId
|
||||
)
|
||||
|
||||
def getTopicId(
|
||||
ceItem: LogEventItem,
|
||||
ceNamespaceOpt: Option[EventNamespace] = None,
|
||||
): Option[Long] =
|
||||
ceNamespaceOpt.flatMap {
|
||||
TopicIdUtils.getTopicId(item = ceItem, _)
|
||||
}
|
||||
|
||||
def getAuthorInfo(
|
||||
ceItem: LogEventItem,
|
||||
): Option[AuthorInfo] =
|
||||
ceItem.tweetDetails.flatMap(_.authorId).map { authorId =>
|
||||
AuthorInfo(
|
||||
authorId = Some(authorId),
|
||||
isFollowedByActingUser = ceItem.isViewerFollowsTweetAuthor,
|
||||
isFollowingActingUser = ceItem.isTweetAuthorFollowsViewer,
|
||||
)
|
||||
}
|
||||
|
||||
def getEventMetadata(
|
||||
eventTimestamp: Long,
|
||||
logEvent: LogEvent,
|
||||
ceItem: LogEventItem,
|
||||
productSurface: Option[ProductSurface] = None
|
||||
): EventMetadata = EventMetadata(
|
||||
sourceTimestampMs = eventTimestamp,
|
||||
receivedTimestampMs = AdapterUtils.currentTimestampMs,
|
||||
sourceLineage = SourceLineage.ClientEvents,
|
||||
// Client UI language or from Gizmoduck which is what user set in Twitter App.
|
||||
// Please see more at https://sourcegraph.twitter.biz/git.twitter.biz/source/-/blob/finatra-internal/international/src/main/scala/com/twitter/finatra/international/LanguageIdentifier.scala
|
||||
// The format should be ISO 639-1.
|
||||
language = logEvent.logBase.flatMap(_.language).map(AdapterUtils.normalizeLanguageCode),
|
||||
// Country code could be IP address (geoduck) or User registration country (gizmoduck) and the former takes precedence.
|
||||
// We don’t know exactly which one is applied, unfortunately,
|
||||
// see https://sourcegraph.twitter.biz/git.twitter.biz/source/-/blob/finatra-internal/international/src/main/scala/com/twitter/finatra/international/CountryIdentifier.scala
|
||||
// The format should be ISO_3166-1_alpha-2.
|
||||
countryCode = logEvent.logBase.flatMap(_.country).map(AdapterUtils.normalizeCountryCode),
|
||||
clientAppId = logEvent.logBase.flatMap(_.clientAppId),
|
||||
clientVersion = logEvent.clientVersion,
|
||||
clientEventNamespace = logEvent.eventNamespace.map(en => toClientEventNamespace(en)),
|
||||
traceId = getTraceId(productSurface, ceItem),
|
||||
requestJoinId = getRequestJoinId(productSurface, ceItem),
|
||||
clientEventTriggeredOn = logEvent.eventDetails.flatMap(_.triggeredOn)
|
||||
)
|
||||
|
||||
def toClientEventNamespace(eventNamespace: EventNamespace): ClientEventNamespace =
|
||||
ClientEventNamespace(
|
||||
page = eventNamespace.page,
|
||||
section = eventNamespace.section,
|
||||
component = eventNamespace.component,
|
||||
element = eventNamespace.element,
|
||||
action = eventNamespace.action
|
||||
)
|
||||
|
||||
/**
|
||||
* Get the profileId from Item.id, which itemType = 'USER'.
|
||||
*
|
||||
* The profileId can be also be found in the event_details.profile_id.
|
||||
* However, the item.id is more reliable than event_details.profile_id,
|
||||
* in particular, 45% of the client events with USER items have
|
||||
* Null for event_details.profile_id while 0.13% item.id is Null.
|
||||
* As such, we only use item.id to populate the profile_id.
|
||||
*/
|
||||
def getProfileIdFromUserItem(item: Item): Option[Long] =
|
||||
if (item.itemType.contains(User))
|
||||
item.id
|
||||
else None
|
||||
|
||||
/**
|
||||
* TraceId is going to be deprecated and replaced by requestJoinId.
|
||||
*
|
||||
* Get the traceId from LogEventItem based on productSurface.
|
||||
*
|
||||
* The traceId is hydrated in controller data from backend. Different product surfaces
|
||||
* populate different controller data. Thus, the product surface is checked first to decide
|
||||
* which controller data should be read to ge the requestJoinId.
|
||||
*/
|
||||
def getTraceId(productSurface: Option[ProductSurface], ceItem: LogEventItem): Option[Long] =
|
||||
productSurface match {
|
||||
case Some(ProductSurface.HomeTimeline) => HomeInfoUtils.getTraceId(ceItem)
|
||||
case Some(ProductSurface.SearchResultsPage) => { new SearchInfoUtils(ceItem) }.getTraceId
|
||||
case _ => None
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the requestJoinId from LogEventItem based on productSurface.
|
||||
*
|
||||
* The requestJoinId is hydrated in controller data from backend. Different product surfaces
|
||||
* populate different controller data. Thus, the product surface is checked first to decide
|
||||
* which controller data should be read to get the requestJoinId.
|
||||
*
|
||||
* Support Home / Home_latest / SearchResults for now, to add other surfaces based on requirement.
|
||||
*/
|
||||
def getRequestJoinId(productSurface: Option[ProductSurface], ceItem: LogEventItem): Option[Long] =
|
||||
productSurface match {
|
||||
case Some(ProductSurface.HomeTimeline) => HomeInfoUtils.getRequestJoinId(ceItem)
|
||||
case Some(ProductSurface.SearchResultsPage) => {
|
||||
new SearchInfoUtils(ceItem)
|
||||
}.getRequestJoinId
|
||||
case _ => None
|
||||
}
|
||||
|
||||
def getTweetAuthorFollowSource(
|
||||
eventNamespace: Option[EventNamespace]
|
||||
): TweetAuthorFollowClickSource = {
|
||||
eventNamespace
|
||||
.map(ns => (ns.element, ns.action)).map {
|
||||
case (Some("follow"), Some("click")) => TweetAuthorFollowClickSource.CaretMenu
|
||||
case (_, Some("follow")) => TweetAuthorFollowClickSource.ProfileImage
|
||||
case _ => TweetAuthorFollowClickSource.Unknown
|
||||
}.getOrElse(TweetAuthorFollowClickSource.Unknown)
|
||||
}
|
||||
|
||||
def getTweetAuthorUnfollowSource(
|
||||
eventNamespace: Option[EventNamespace]
|
||||
): TweetAuthorUnfollowClickSource = {
|
||||
eventNamespace
|
||||
.map(ns => (ns.element, ns.action)).map {
|
||||
case (Some("unfollow"), Some("click")) => TweetAuthorUnfollowClickSource.CaretMenu
|
||||
case (_, Some("unfollow")) => TweetAuthorUnfollowClickSource.ProfileImage
|
||||
case _ => TweetAuthorUnfollowClickSource.Unknown
|
||||
}.getOrElse(TweetAuthorUnfollowClickSource.Unknown)
|
||||
}
|
||||
}
|
|
@ -0,0 +1,687 @@
|
|||
package com.twitter.unified_user_actions.adapter.client_event
|
||||
|
||||
import com.twitter.clientapp.thriftscala.ItemType
|
||||
import com.twitter.clientapp.thriftscala.LogEvent
|
||||
import com.twitter.clientapp.thriftscala.{Item => LogEventItem}
|
||||
import com.twitter.unified_user_actions.thriftscala._
|
||||
|
||||
object ClientEventEngagement {
|
||||
object TweetFav extends BaseClientEvent(ActionType.ClientTweetFav)
|
||||
|
||||
/**
|
||||
* This is fired when a user unlikes a liked(favorited) tweet
|
||||
*/
|
||||
object TweetUnfav extends BaseClientEvent(ActionType.ClientTweetUnfav)
|
||||
|
||||
/**
|
||||
* This is "Send Reply" event to indicate publishing of a reply Tweet as opposed to clicking
|
||||
* on the reply button to initiate a reply Tweet (captured in ClientTweetClickReply).
|
||||
* The difference between this and the ServerTweetReply are:
|
||||
* 1) ServerTweetReply already has the new Tweet Id, 2) A sent reply may be lost during transfer
|
||||
* over the wire and thus may not end up with a follow-up ServerTweetReply.
|
||||
*/
|
||||
object TweetReply extends BaseClientEvent(ActionType.ClientTweetReply)
|
||||
|
||||
/**
|
||||
* This is the "send quote" event to indicate publishing of a quote tweet as opposed to clicking
|
||||
* on the quote button to initiate a quote tweet (captured in ClientTweetClickQuote).
|
||||
* The difference between this and the ServerTweetQuote are:
|
||||
* 1) ServerTweetQuote already has the new Tweet Id, 2) A sent quote may be lost during transfer
|
||||
* over the wire and thus may not ended up with a follow-up ServerTweetQuote.
|
||||
*/
|
||||
object TweetQuote extends BaseClientEvent(ActionType.ClientTweetQuote)
|
||||
|
||||
/**
|
||||
* This is the "retweet" event to indicate publishing of a retweet.
|
||||
*/
|
||||
object TweetRetweet extends BaseClientEvent(ActionType.ClientTweetRetweet)
|
||||
|
||||
/**
|
||||
* "action = reply" indicates that a user expressed the intention to reply to a Tweet by clicking
|
||||
* the reply button. No new tweet is created in this event.
|
||||
*/
|
||||
object TweetClickReply extends BaseClientEvent(ActionType.ClientTweetClickReply)
|
||||
|
||||
/**
|
||||
* Please note that the "action == quote" is NOT the create quote Tweet event like what
|
||||
* we can get from TweetyPie.
|
||||
* It is just click on the "quote tweet" (after clicking on the retweet button there are 2 options,
|
||||
* one is "retweet" and the other is "quote tweet")
|
||||
*
|
||||
* Also checked the CE (BQ Table), the `item.tweet_details.quoting_tweet_id` is always NULL but
|
||||
* `item.tweet_details.retweeting_tweet_id`, `item.tweet_details.in_reply_to_tweet_id`, `item.tweet_details.quoted_tweet_id`
|
||||
* could be NON-NULL and UUA would just include these NON-NULL fields as is. This is also checked in the unit test.
|
||||
*/
|
||||
object TweetClickQuote extends BaseClientEvent(ActionType.ClientTweetClickQuote)
|
||||
|
||||
/**
|
||||
* Refer to go/cme-scribing and go/interaction-event-spec for details.
|
||||
* Fired on the first tick of a track regardless of where in the video it is playing.
|
||||
* For looping playback, this is only fired once and does not reset at loop boundaries.
|
||||
*/
|
||||
object TweetVideoPlaybackStart
|
||||
extends BaseVideoClientEvent(ActionType.ClientTweetVideoPlaybackStart)
|
||||
|
||||
/**
|
||||
* Refer to go/cme-scribing and go/interaction-event-spec for details.
|
||||
* Fired when playback reaches 100% of total track duration.
|
||||
* Not valid for live videos.
|
||||
* For looping playback, this is only fired once and does not reset at loop boundaries.
|
||||
*/
|
||||
object TweetVideoPlaybackComplete
|
||||
extends BaseVideoClientEvent(ActionType.ClientTweetVideoPlaybackComplete)
|
||||
|
||||
/**
|
||||
* Refer to go/cme-scribing and go/interaction-event-spec for details.
|
||||
* This is fired when playback reaches 25% of total track duration. Not valid for live videos.
|
||||
* For looping playback, this is only fired once and does not reset at loop boundaries.
|
||||
*/
|
||||
object TweetVideoPlayback25 extends BaseVideoClientEvent(ActionType.ClientTweetVideoPlayback25)
|
||||
object TweetVideoPlayback50 extends BaseVideoClientEvent(ActionType.ClientTweetVideoPlayback50)
|
||||
object TweetVideoPlayback75 extends BaseVideoClientEvent(ActionType.ClientTweetVideoPlayback75)
|
||||
object TweetVideoPlayback95 extends BaseVideoClientEvent(ActionType.ClientTweetVideoPlayback95)
|
||||
|
||||
/**
|
||||
* Refer to go/cme-scribing and go/interaction-event-spec for details.
|
||||
* This if fired when the video has been played in non-preview
|
||||
* (i.e. not autoplaying in the timeline) mode, and was not started via auto-advance.
|
||||
* For looping playback, this is only fired once and does not reset at loop boundaries.
|
||||
*/
|
||||
object TweetVideoPlayFromTap extends BaseVideoClientEvent(ActionType.ClientTweetVideoPlayFromTap)
|
||||
|
||||
/**
|
||||
* Refer to go/cme-scribing and go/interaction-event-spec for details.
|
||||
* This is fired when 50% of the video has been on-screen and playing for 10 consecutive seconds
|
||||
* or 95% of the video duration, whichever comes first.
|
||||
* For looping playback, this is only fired once and does not reset at loop boundaries.
|
||||
*/
|
||||
object TweetVideoQualityView extends BaseVideoClientEvent(ActionType.ClientTweetVideoQualityView)
|
||||
|
||||
object TweetVideoView extends BaseVideoClientEvent(ActionType.ClientTweetVideoView)
|
||||
object TweetVideoMrcView extends BaseVideoClientEvent(ActionType.ClientTweetVideoMrcView)
|
||||
object TweetVideoViewThreshold
|
||||
extends BaseVideoClientEvent(ActionType.ClientTweetVideoViewThreshold)
|
||||
object TweetVideoCtaUrlClick extends BaseVideoClientEvent(ActionType.ClientTweetVideoCtaUrlClick)
|
||||
object TweetVideoCtaWatchClick
|
||||
extends BaseVideoClientEvent(ActionType.ClientTweetVideoCtaWatchClick)
|
||||
|
||||
/**
|
||||
* This is fired when a user clicks on "Undo retweet" after re-tweeting a tweet
|
||||
*
|
||||
*/
|
||||
object TweetUnretweet extends BaseClientEvent(ActionType.ClientTweetUnretweet)
|
||||
|
||||
/**
|
||||
* This is fired when a user clicks on a photo attached to a tweet and the photo expands to fit
|
||||
* the screen.
|
||||
*/
|
||||
object TweetPhotoExpand extends BaseClientEvent(ActionType.ClientTweetPhotoExpand)
|
||||
|
||||
/**
|
||||
* This is fired when a user clicks on a card, a card could be a photo or video for example
|
||||
*/
|
||||
object CardClick extends BaseCardClientEvent(ActionType.ClientCardClick)
|
||||
object CardOpenApp extends BaseCardClientEvent(ActionType.ClientCardOpenApp)
|
||||
object CardAppInstallAttempt extends BaseCardClientEvent(ActionType.ClientCardAppInstallAttempt)
|
||||
object PollCardVote extends BaseCardClientEvent(ActionType.ClientPollCardVote)
|
||||
|
||||
/**
|
||||
* This is fired when a user clicks on a profile mention inside a tweet.
|
||||
*/
|
||||
object TweetClickMentionScreenName
|
||||
extends BaseClientEvent(ActionType.ClientTweetClickMentionScreenName) {
|
||||
override def getUuaItem(
|
||||
ceItem: LogEventItem,
|
||||
logEvent: LogEvent
|
||||
): Option[Item] =
|
||||
(
|
||||
ceItem.id,
|
||||
logEvent.eventDetails.flatMap(
|
||||
_.targets.flatMap(_.find(_.itemType.contains(ItemType.User))))) match {
|
||||
case (Some(tweetId), Some(target)) =>
|
||||
(target.id, target.name) match {
|
||||
case (Some(profileId), Some(profileHandle)) =>
|
||||
Some(
|
||||
Item.TweetInfo(
|
||||
ClientEventCommonUtils
|
||||
.getBasicTweetInfo(tweetId, ceItem, logEvent.eventNamespace)
|
||||
.copy(tweetActionInfo = Some(
|
||||
TweetActionInfo.ClientTweetClickMentionScreenName(
|
||||
ClientTweetClickMentionScreenName(
|
||||
actionProfileId = profileId,
|
||||
handle = profileHandle
|
||||
))))))
|
||||
case _ => None
|
||||
}
|
||||
case _ => None
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* These are fired when user follows/unfollows a Topic. Please see the comment in the
|
||||
* ClientEventAdapter namespace matching to see the subtle details.
|
||||
*/
|
||||
object TopicFollow extends BaseTopicClientEvent(ActionType.ClientTopicFollow)
|
||||
object TopicUnfollow extends BaseTopicClientEvent(ActionType.ClientTopicUnfollow)
|
||||
|
||||
/**
|
||||
* This is fired when the user clicks the "x" icon next to the topic on their timeline,
|
||||
* and clicks "Not interested in {TOPIC}" in the pop-up prompt
|
||||
* Alternatively, they can also click "See more" button to visit the topic page, and click "Not interested" there.
|
||||
*/
|
||||
object TopicNotInterestedIn extends BaseTopicClientEvent(ActionType.ClientTopicNotInterestedIn)
|
||||
|
||||
/**
|
||||
* This is fired when the user clicks the "Undo" button after clicking "x" or "Not interested" on a Topic
|
||||
* which is captured in ClientTopicNotInterestedIn
|
||||
*/
|
||||
object TopicUndoNotInterestedIn
|
||||
extends BaseTopicClientEvent(ActionType.ClientTopicUndoNotInterestedIn)
|
||||
|
||||
/**
|
||||
* This is fired when a user clicks on "This Tweet's not helpful" flow in the caret menu
|
||||
* of a Tweet result on the Search Results Page
|
||||
*/
|
||||
object TweetNotHelpful extends BaseClientEvent(ActionType.ClientTweetNotHelpful)
|
||||
|
||||
/**
|
||||
* This is fired when a user clicks Undo after clicking on
|
||||
* "This Tweet's not helpful" flow in the caret menu of a Tweet result on the Search Results Page
|
||||
*/
|
||||
object TweetUndoNotHelpful extends BaseClientEvent(ActionType.ClientTweetUndoNotHelpful)
|
||||
|
||||
object TweetReport extends BaseClientEvent(ActionType.ClientTweetReport) {
|
||||
override def getUuaItem(
|
||||
ceItem: LogEventItem,
|
||||
logEvent: LogEvent
|
||||
): Option[Item] = {
|
||||
for {
|
||||
actionTweetId <- ceItem.id
|
||||
} yield {
|
||||
Item.TweetInfo(
|
||||
ClientEventCommonUtils
|
||||
.getBasicTweetInfo(
|
||||
actionTweetId = actionTweetId,
|
||||
ceItem = ceItem,
|
||||
ceNamespaceOpt = logEvent.eventNamespace)
|
||||
.copy(tweetActionInfo = Some(
|
||||
TweetActionInfo.ClientTweetReport(
|
||||
ClientTweetReport(
|
||||
isReportTweetDone =
|
||||
logEvent.eventNamespace.flatMap(_.action).exists(_.contains("done")),
|
||||
reportFlowId = logEvent.reportDetails.flatMap(_.reportFlowId)
|
||||
)
|
||||
))))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Not Interested In (Do Not like) event
|
||||
*/
|
||||
object TweetNotInterestedIn extends BaseClientEvent(ActionType.ClientTweetNotInterestedIn)
|
||||
object TweetUndoNotInterestedIn extends BaseClientEvent(ActionType.ClientTweetUndoNotInterestedIn)
|
||||
|
||||
/**
|
||||
* This is fired when a user FIRST clicks the "Not interested in this Tweet" button in the caret menu of a Tweet
|
||||
* then clicks "This Tweet is not about {TOPIC}" in the subsequent prompt
|
||||
* Note: this button is hidden unless a user clicks "Not interested in this Tweet" first.
|
||||
*/
|
||||
object TweetNotAboutTopic extends BaseClientEvent(ActionType.ClientTweetNotAboutTopic)
|
||||
|
||||
/**
|
||||
* This is fired when a user clicks "Undo" immediately after clicking "This Tweet is not about {TOPIC}",
|
||||
* which is captured in TweetNotAboutTopic
|
||||
*/
|
||||
object TweetUndoNotAboutTopic extends BaseClientEvent(ActionType.ClientTweetUndoNotAboutTopic)
|
||||
|
||||
/**
|
||||
* This is fired when a user FIRST clicks the "Not interested in this Tweet" button in the caret menu of a Tweet
|
||||
* then clicks "This Tweet isn't recent" in the subsequent prompt
|
||||
* Note: this button is hidden unless a user clicks "Not interested in this Tweet" first.
|
||||
*/
|
||||
object TweetNotRecent extends BaseClientEvent(ActionType.ClientTweetNotRecent)
|
||||
|
||||
/**
|
||||
* This is fired when a user clicks "Undo" immediately after clicking "his Tweet isn't recent",
|
||||
* which is captured in TweetNotRecent
|
||||
*/
|
||||
object TweetUndoNotRecent extends BaseClientEvent(ActionType.ClientTweetUndoNotRecent)
|
||||
|
||||
/**
|
||||
* This is fired when a user clicks "Not interested in this Tweet" button in the caret menu of a Tweet
|
||||
* then clicks "Show fewer tweets from" in the subsequent prompt
|
||||
* Note: this button is hidden unless a user clicks "Not interested in this Tweet" first.
|
||||
*/
|
||||
object TweetSeeFewer extends BaseClientEvent(ActionType.ClientTweetSeeFewer)
|
||||
|
||||
/**
|
||||
* This is fired when a user clicks "Undo" immediately after clicking "Show fewer tweets from",
|
||||
* which is captured in TweetSeeFewer
|
||||
*/
|
||||
object TweetUndoSeeFewer extends BaseClientEvent(ActionType.ClientTweetUndoSeeFewer)
|
||||
|
||||
/**
|
||||
* This is fired when a user click "Submit" at the end of a "Report Tweet" flow
|
||||
* ClientTweetReport = 1041 is scribed by HealthClient team, on the client side
|
||||
* This is scribed by spamacaw, on the server side
|
||||
* They can be joined on reportFlowId
|
||||
* See https://confluence.twitter.biz/pages/viewpage.action?spaceKey=HEALTH&title=Understanding+ReportDetails
|
||||
*/
|
||||
object TweetReportServer extends BaseClientEvent(ActionType.ServerTweetReport) {
|
||||
override def getUuaItem(
|
||||
ceItem: LogEventItem,
|
||||
logEvent: LogEvent
|
||||
): Option[Item] =
|
||||
for {
|
||||
actionTweetId <- ceItem.id
|
||||
} yield Item.TweetInfo(
|
||||
ClientEventCommonUtils
|
||||
.getBasicTweetInfo(
|
||||
actionTweetId = actionTweetId,
|
||||
ceItem = ceItem,
|
||||
ceNamespaceOpt = logEvent.eventNamespace)
|
||||
.copy(tweetActionInfo = Some(
|
||||
TweetActionInfo.ServerTweetReport(
|
||||
ServerTweetReport(
|
||||
reportFlowId = logEvent.reportDetails.flatMap(_.reportFlowId),
|
||||
reportType = logEvent.reportDetails.flatMap(_.reportType)
|
||||
)
|
||||
))))
|
||||
}
|
||||
|
||||
/**
|
||||
* This is fired when a user clicks Block in a Profile page
|
||||
* A Profile can also be blocked when a user clicks Block in the menu of a Tweet, which
|
||||
* is captured in ClientTweetBlockAuthor
|
||||
*/
|
||||
object ProfileBlock extends BaseProfileClientEvent(ActionType.ClientProfileBlock)
|
||||
|
||||
/**
|
||||
* This is fired when a user clicks unblock in a pop-up prompt right after blocking a profile
|
||||
* in the profile page or clicks unblock in a drop-down menu in the profile page.
|
||||
*/
|
||||
object ProfileUnblock extends BaseProfileClientEvent(ActionType.ClientProfileUnblock)
|
||||
|
||||
/**
|
||||
* This is fired when a user clicks Mute in a Profile page
|
||||
* A Profile can also be muted when a user clicks Mute in the menu of a Tweet, which
|
||||
* is captured in ClientTweetMuteAuthor
|
||||
*/
|
||||
object ProfileMute extends BaseProfileClientEvent(ActionType.ClientProfileMute)
|
||||
|
||||
/*
|
||||
* This is fired when a user clicks "Report User" action from user profile page
|
||||
* */
|
||||
object ProfileReport extends BaseProfileClientEvent(ActionType.ClientProfileReport)
|
||||
|
||||
// This is fired when a user profile is open in a Profile page
|
||||
object ProfileShow extends BaseProfileClientEvent(ActionType.ClientProfileShow)
|
||||
|
||||
object ProfileClick extends BaseProfileClientEvent(ActionType.ClientProfileClick) {
|
||||
|
||||
/**
|
||||
* ClientTweetClickProfile would emit 2 events, 1 with item type Tweet and 1 with item type User
|
||||
* Both events will go to both actions (the actual classes). For ClientTweetClickProfile,
|
||||
* item type of Tweet will filter out the event with item type User. But for ClientProfileClick,
|
||||
* because we need to include item type of User, then we will also include the event of TweetClickProfile
|
||||
* if we don't do anything here. This override ensures we don't include tweet author clicks events in ProfileClick
|
||||
*/
|
||||
override def getUuaItem(
|
||||
ceItem: LogEventItem,
|
||||
logEvent: LogEvent
|
||||
): Option[Item] =
|
||||
if (logEvent.eventDetails
|
||||
.flatMap(_.items).exists(items => items.exists(_.itemType.contains(ItemType.Tweet)))) {
|
||||
None
|
||||
} else {
|
||||
super.getUuaItem(ceItem, logEvent)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This is fired when a user follows a profile from the
|
||||
* profile page / people module and people tab on the Search Results Page / sidebar on the Home page
|
||||
* A Profile can also be followed when a user clicks follow in the
|
||||
* caret menu of a Tweet / follow button on hovering on profile avatar,
|
||||
* which is captured in ClientTweetFollowAuthor
|
||||
*/
|
||||
object ProfileFollow extends BaseProfileClientEvent(ActionType.ClientProfileFollow) {
|
||||
|
||||
/**
|
||||
* ClientTweetFollowAuthor would emit 2 events, 1 with item type Tweet and 1 with item type User
|
||||
* Both events will go to both actions (the actual classes). For ClientTweetFollowAuthor,
|
||||
* item type of Tweet will filter out the event with item type User. But for ClientProfileFollow,
|
||||
* because we need to include item type of User, then we will also include the event of TweetFollowAuthor
|
||||
* if we don't do anything here. This override ensures we don't include tweet author follow events in ProfileFollow
|
||||
*/
|
||||
override def getUuaItem(
|
||||
ceItem: LogEventItem,
|
||||
logEvent: LogEvent
|
||||
): Option[Item] =
|
||||
if (logEvent.eventDetails
|
||||
.flatMap(_.items).exists(items => items.exists(_.itemType.contains(ItemType.Tweet)))) {
|
||||
None
|
||||
} else {
|
||||
super.getUuaItem(ceItem, logEvent)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This is fired when a user clicks Follow in the caret menu of a Tweet or hovers on the avatar of the tweet author
|
||||
* and clicks on the Follow button. A profile can also be followed by clicking the Follow button on the Profile
|
||||
* page and confirm, which is captured in ClientProfileFollow.
|
||||
* The event emits two items, one of user type and another of tweet type, since the default implementation of
|
||||
* BaseClientEvent only looks for Tweet type, the other item is dropped which is the expected behaviour
|
||||
*/
|
||||
object TweetFollowAuthor extends BaseClientEvent(ActionType.ClientTweetFollowAuthor) {
|
||||
override def getUuaItem(
|
||||
ceItem: LogEventItem,
|
||||
logEvent: LogEvent
|
||||
): Option[Item] = {
|
||||
for {
|
||||
actionTweetId <- ceItem.id
|
||||
} yield {
|
||||
Item.TweetInfo(
|
||||
ClientEventCommonUtils
|
||||
.getBasicTweetInfo(
|
||||
actionTweetId = actionTweetId,
|
||||
ceItem = ceItem,
|
||||
ceNamespaceOpt = logEvent.eventNamespace)
|
||||
.copy(tweetActionInfo = Some(
|
||||
TweetActionInfo.ClientTweetFollowAuthor(
|
||||
ClientTweetFollowAuthor(
|
||||
ClientEventCommonUtils.getTweetAuthorFollowSource(logEvent.eventNamespace))
|
||||
))))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This is fired when a user clicks Unfollow in the caret menu of a Tweet or hovers on the avatar of the tweet author
|
||||
* and clicks on the Unfollow button. A profile can also be unfollowed by clicking the Unfollow button on the Profile
|
||||
* page and confirm, which will be captured in ClientProfileUnfollow.
|
||||
* The event emits two items, one of user type and another of tweet type, since the default implementation of
|
||||
* BaseClientEvent only looks for Tweet type, the other item is dropped which is the expected behaviour
|
||||
*/
|
||||
object TweetUnfollowAuthor extends BaseClientEvent(ActionType.ClientTweetUnfollowAuthor) {
|
||||
override def getUuaItem(
|
||||
ceItem: LogEventItem,
|
||||
logEvent: LogEvent
|
||||
): Option[Item] = {
|
||||
for {
|
||||
actionTweetId <- ceItem.id
|
||||
} yield {
|
||||
Item.TweetInfo(
|
||||
ClientEventCommonUtils
|
||||
.getBasicTweetInfo(
|
||||
actionTweetId = actionTweetId,
|
||||
ceItem = ceItem,
|
||||
ceNamespaceOpt = logEvent.eventNamespace)
|
||||
.copy(tweetActionInfo = Some(
|
||||
TweetActionInfo.ClientTweetUnfollowAuthor(
|
||||
ClientTweetUnfollowAuthor(
|
||||
ClientEventCommonUtils.getTweetAuthorUnfollowSource(logEvent.eventNamespace))
|
||||
))))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This is fired when a user clicks Block in the caret menu of a Tweet to block the profile
|
||||
* that authors this Tweet. A profile can also be blocked in the Profile page, which is captured
|
||||
* in ClientProfileBlock
|
||||
*/
|
||||
object TweetBlockAuthor extends BaseClientEvent(ActionType.ClientTweetBlockAuthor)
|
||||
|
||||
/**
|
||||
* This is fired when a user clicks unblock in a pop-up prompt right after blocking an author
|
||||
* in the drop-down menu of a tweet
|
||||
*/
|
||||
object TweetUnblockAuthor extends BaseClientEvent(ActionType.ClientTweetUnblockAuthor)
|
||||
|
||||
/**
|
||||
* This is fired when a user clicks Mute in the caret menu of a Tweet to mute the profile
|
||||
* that authors this Tweet. A profile can also be muted in the Profile page, which is captured
|
||||
* in ClientProfileMute
|
||||
*/
|
||||
object TweetMuteAuthor extends BaseClientEvent(ActionType.ClientTweetMuteAuthor)
|
||||
|
||||
/**
|
||||
* This is fired when a user clicks on a Tweet to open the Tweet details page. Note that for
|
||||
* Tweets in the Notification Tab product surface, a click can be registered differently
|
||||
* depending on whether the Tweet is a rendered Tweet (a click results in ClientTweetClick)
|
||||
* or a wrapper Notification (a click results in ClientNotificationClick).
|
||||
*/
|
||||
object TweetClick extends BaseClientEvent(ActionType.ClientTweetClick)
|
||||
|
||||
/**
|
||||
* This is fired when a user clicks to view the profile page of another user from a Tweet
|
||||
*/
|
||||
object TweetClickProfile extends BaseClientEvent(ActionType.ClientTweetClickProfile)
|
||||
|
||||
/**
|
||||
* This is fired when a user clicks on the "share" icon on a Tweet to open the share menu.
|
||||
* The user may or may not proceed and finish sharing the Tweet.
|
||||
*/
|
||||
object TweetClickShare extends BaseClientEvent(ActionType.ClientTweetClickShare)
|
||||
|
||||
/**
|
||||
* This is fired when a user clicks "Copy link to Tweet" in a menu appeared after hitting
|
||||
* the "share" icon on a Tweet OR when a user selects share_via -> copy_link after long-click
|
||||
* a link inside a tweet on a mobile device
|
||||
*/
|
||||
object TweetShareViaCopyLink extends BaseClientEvent(ActionType.ClientTweetShareViaCopyLink)
|
||||
|
||||
/**
|
||||
* This is fired when a user clicks "Send via Direct Message" after
|
||||
* clicking on the "share" icon on a Tweet to open the share menu.
|
||||
* The user may or may not proceed and finish Sending the DM.
|
||||
*/
|
||||
object TweetClickSendViaDirectMessage
|
||||
extends BaseClientEvent(ActionType.ClientTweetClickSendViaDirectMessage)
|
||||
|
||||
/**
|
||||
* This is fired when a user clicks "Bookmark" after
|
||||
* clicking on the "share" icon on a Tweet to open the share menu.
|
||||
*/
|
||||
object TweetShareViaBookmark extends BaseClientEvent(ActionType.ClientTweetShareViaBookmark)
|
||||
|
||||
/**
|
||||
* This is fired when a user clicks "Remove Tweet from Bookmarks" after
|
||||
* clicking on the "share" icon on a Tweet to open the share menu.
|
||||
*/
|
||||
object TweetUnbookmark extends BaseClientEvent(ActionType.ClientTweetUnbookmark)
|
||||
|
||||
/**
|
||||
* This event is fired when the user clicks on a hashtag in a Tweet.
|
||||
*/
|
||||
object TweetClickHashtag extends BaseClientEvent(ActionType.ClientTweetClickHashtag) {
|
||||
override def getUuaItem(
|
||||
ceItem: LogEventItem,
|
||||
logEvent: LogEvent
|
||||
): Option[Item] = for {
|
||||
actionTweetId <- ceItem.id
|
||||
} yield Item.TweetInfo(
|
||||
ClientEventCommonUtils
|
||||
.getBasicTweetInfo(
|
||||
actionTweetId = actionTweetId,
|
||||
ceItem = ceItem,
|
||||
ceNamespaceOpt = logEvent.eventNamespace)
|
||||
.copy(tweetActionInfo = logEvent.eventDetails
|
||||
.map(
|
||||
_.targets.flatMap(_.headOption.flatMap(_.name))
|
||||
) // fetch the first item in the details and then the name will have the hashtag value with the '#' sign
|
||||
.map { hashtagOpt =>
|
||||
TweetActionInfo.ClientTweetClickHashtag(
|
||||
ClientTweetClickHashtag(hashtag = hashtagOpt)
|
||||
)
|
||||
}))
|
||||
}
|
||||
|
||||
/**
|
||||
* This is fired when a user clicks "Bookmark" after clicking on the "share" icon on a Tweet to
|
||||
* open the share menu, or when a user clicks on the 'bookmark' icon on a Tweet (bookmark icon
|
||||
* is available to ios only as of March 2023).
|
||||
* TweetBookmark and TweetShareByBookmark log the same events but serve for individual use cases.
|
||||
*/
|
||||
object TweetBookmark extends BaseClientEvent(ActionType.ClientTweetBookmark)
|
||||
|
||||
/**
|
||||
* This is fired when a user clicks on a link in a tweet.
|
||||
* The link could be displayed as a URL or embedded
|
||||
* in a component such as an image or a card in a tweet.
|
||||
*/
|
||||
object TweetOpenLink extends BaseClientEvent(ActionType.ClientTweetOpenLink) {
|
||||
override def getUuaItem(
|
||||
ceItem: LogEventItem,
|
||||
logEvent: LogEvent
|
||||
): Option[Item] =
|
||||
for {
|
||||
actionTweetId <- ceItem.id
|
||||
} yield Item.TweetInfo(
|
||||
ClientEventCommonUtils
|
||||
.getBasicTweetInfo(
|
||||
actionTweetId = actionTweetId,
|
||||
ceItem = ceItem,
|
||||
ceNamespaceOpt = logEvent.eventNamespace)
|
||||
.copy(tweetActionInfo = Some(
|
||||
TweetActionInfo.ClientTweetOpenLink(
|
||||
ClientTweetOpenLink(url = logEvent.eventDetails.flatMap(_.url))
|
||||
))))
|
||||
}
|
||||
|
||||
/**
|
||||
* This is fired when a user takes a screenshot.
|
||||
* This is available for only mobile clients.
|
||||
*/
|
||||
object TweetTakeScreenshot extends BaseClientEvent(ActionType.ClientTweetTakeScreenshot) {
|
||||
override def getUuaItem(
|
||||
ceItem: LogEventItem,
|
||||
logEvent: LogEvent
|
||||
): Option[Item] =
|
||||
for {
|
||||
actionTweetId <- ceItem.id
|
||||
} yield Item.TweetInfo(
|
||||
ClientEventCommonUtils
|
||||
.getBasicTweetInfo(
|
||||
actionTweetId = actionTweetId,
|
||||
ceItem = ceItem,
|
||||
ceNamespaceOpt = logEvent.eventNamespace)
|
||||
.copy(tweetActionInfo = Some(
|
||||
TweetActionInfo.ClientTweetTakeScreenshot(
|
||||
ClientTweetTakeScreenshot(percentVisibleHeight100k = ceItem.percentVisibleHeight100k)
|
||||
))))
|
||||
}
|
||||
|
||||
/**
|
||||
* This is fired when a user clicks the "This Tweet isn't relevant" button in a prompt displayed
|
||||
* after clicking "This Tweet's not helpful" in search result page or "Not Interested in this Tweet"
|
||||
* in the home timeline page.
|
||||
* Note: this button is hidden unless a user clicks "This Tweet isn't relevant" or
|
||||
* "This Tweet's not helpful" first
|
||||
*/
|
||||
object TweetNotRelevant extends BaseClientEvent(ActionType.ClientTweetNotRelevant)
|
||||
|
||||
/**
|
||||
* This is fired when a user clicks "Undo" immediately after clicking "this Tweet isn't relevant",
|
||||
* which is captured in TweetNotRelevant
|
||||
*/
|
||||
object TweetUndoNotRelevant extends BaseClientEvent(ActionType.ClientTweetUndoNotRelevant)
|
||||
|
||||
/**
|
||||
* This is fired when a user is logged out and follows a profile from the
|
||||
* profile page / people module from web.
|
||||
* One can only try to follow from web, iOS and Android do not support logged out browsing
|
||||
*/
|
||||
object ProfileFollowAttempt extends BaseProfileClientEvent(ActionType.ClientProfileFollowAttempt)
|
||||
|
||||
/**
|
||||
* This is fired when a user is logged out and favourite a tweet from web.
|
||||
* One can only try to favourite from web, iOS and Android do not support logged out browsing
|
||||
*/
|
||||
object TweetFavoriteAttempt extends BaseClientEvent(ActionType.ClientTweetFavoriteAttempt)
|
||||
|
||||
/**
|
||||
* This is fired when a user is logged out and Retweet a tweet from web.
|
||||
* One can only try to favourite from web, iOS and Android do not support logged out browsing
|
||||
*/
|
||||
object TweetRetweetAttempt extends BaseClientEvent(ActionType.ClientTweetRetweetAttempt)
|
||||
|
||||
/**
|
||||
* This is fired when a user is logged out and reply on tweet from web.
|
||||
* One can only try to favourite from web, iOS and Android do not support logged out browsing
|
||||
*/
|
||||
object TweetReplyAttempt extends BaseClientEvent(ActionType.ClientTweetReplyAttempt)
|
||||
|
||||
/**
|
||||
* This is fired when a user is logged out and clicks on login button.
|
||||
* Currently seem to be generated only on [m5, LiteNativeWrapper] as of Jan 2023.
|
||||
*/
|
||||
object CTALoginClick extends BaseCTAClientEvent(ActionType.ClientCTALoginClick)
|
||||
|
||||
/**
|
||||
* This is fired when a user is logged out and login window is shown.
|
||||
*/
|
||||
object CTALoginStart extends BaseCTAClientEvent(ActionType.ClientCTALoginStart)
|
||||
|
||||
/**
|
||||
* This is fired when a user is logged out and login is successful.
|
||||
*/
|
||||
object CTALoginSuccess extends BaseCTAClientEvent(ActionType.ClientCTALoginSuccess)
|
||||
|
||||
/**
|
||||
* This is fired when a user is logged out and clicks on signup button.
|
||||
*/
|
||||
object CTASignupClick extends BaseCTAClientEvent(ActionType.ClientCTASignupClick)
|
||||
|
||||
/**
|
||||
* This is fired when a user is logged out and signup is successful.
|
||||
*/
|
||||
object CTASignupSuccess extends BaseCTAClientEvent(ActionType.ClientCTASignupSuccess)
|
||||
|
||||
/**
|
||||
* This is fired when a user opens a Push Notification.
|
||||
* Refer to https://confluence.twitter.biz/pages/viewpage.action?pageId=161811800
|
||||
* for Push Notification scribe details
|
||||
*/
|
||||
object NotificationOpen extends BasePushNotificationClientEvent(ActionType.ClientNotificationOpen)
|
||||
|
||||
/**
|
||||
* This is fired when a user clicks on a notification in the Notification Tab.
|
||||
* Refer to go/ntab-urt-scribe for Notification Tab scribe details.
|
||||
*/
|
||||
object NotificationClick
|
||||
extends BaseNotificationTabClientEvent(ActionType.ClientNotificationClick)
|
||||
|
||||
/**
|
||||
* This is fired when a user taps the "See Less Often" caret menu item of a notification in
|
||||
* the Notification Tab.
|
||||
* Refer to go/ntab-urt-scribe for Notification Tab scribe details.
|
||||
*/
|
||||
object NotificationSeeLessOften
|
||||
extends BaseNotificationTabClientEvent(ActionType.ClientNotificationSeeLessOften)
|
||||
|
||||
/**
|
||||
* This is fired when a user closes or swipes away a Push Notification.
|
||||
* Refer to https://confluence.twitter.biz/pages/viewpage.action?pageId=161811800
|
||||
* for Push Notification scribe details
|
||||
*/
|
||||
object NotificationDismiss
|
||||
extends BasePushNotificationClientEvent(ActionType.ClientNotificationDismiss)
|
||||
|
||||
/**
|
||||
* This is fired when a user clicks on a typeahead suggestion(queries, events, topics, users)
|
||||
* in a drop-down menu of a search box or a tweet compose box.
|
||||
*/
|
||||
object TypeaheadClick extends BaseSearchTypeaheadEvent(ActionType.ClientTypeaheadClick)
|
||||
|
||||
/**
|
||||
* This is a generic event fired when the user submits feedback on a prompt.
|
||||
* Some examples include Did You Find It Prompt and Tweet Relevance on Search Results Page.
|
||||
*/
|
||||
object FeedbackPromptSubmit
|
||||
extends BaseFeedbackSubmitClientEvent(ActionType.ClientFeedbackPromptSubmit)
|
||||
|
||||
object AppExit extends BaseUASClientEvent(ActionType.ClientAppExit)
|
||||
}
|
|
@ -0,0 +1,207 @@
|
|||
package com.twitter.unified_user_actions.adapter.client_event
|
||||
|
||||
import com.twitter.clientapp.thriftscala.EventNamespace
|
||||
import com.twitter.clientapp.thriftscala.LogEvent
|
||||
import com.twitter.clientapp.thriftscala.{Item => LogEventItem}
|
||||
import com.twitter.logbase.thriftscala.LogBase
|
||||
import com.twitter.unified_user_actions.thriftscala._
|
||||
import com.twitter.unified_user_actions.thriftscala.Item.TweetInfo
|
||||
|
||||
object ClientEventImpression {
|
||||
object TweetLingerImpression extends BaseClientEvent(ActionType.ClientTweetLingerImpression) {
|
||||
override def getUuaItem(
|
||||
ceItem: LogEventItem,
|
||||
logEvent: LogEvent
|
||||
): Option[Item] = {
|
||||
for {
|
||||
actionTweetId <- ceItem.id
|
||||
impressionDetails <- ceItem.impressionDetails
|
||||
lingerStartTimestampMs <- impressionDetails.visibilityStart
|
||||
lingerEndTimestampMs <- impressionDetails.visibilityEnd
|
||||
} yield {
|
||||
Item.TweetInfo(
|
||||
ClientEventCommonUtils
|
||||
.getBasicTweetInfo(actionTweetId, ceItem, logEvent.eventNamespace)
|
||||
.copy(tweetActionInfo = Some(
|
||||
TweetActionInfo.ClientTweetLingerImpression(
|
||||
ClientTweetLingerImpression(
|
||||
lingerStartTimestampMs = lingerStartTimestampMs,
|
||||
lingerEndTimestampMs = lingerEndTimestampMs
|
||||
)
|
||||
))))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* To make parity with iesource's definition, render impression for quoted Tweets would emit
|
||||
* 2 events: 1 for the quoting Tweet and 1 for the original Tweet!!!
|
||||
*/
|
||||
object TweetRenderImpression extends BaseClientEvent(ActionType.ClientTweetRenderImpression) {
|
||||
override def toUnifiedUserAction(logEvent: LogEvent): Seq[UnifiedUserAction] = {
|
||||
|
||||
val logBase: Option[LogBase] = logEvent.logBase
|
||||
|
||||
val raw = for {
|
||||
ed <- logEvent.eventDetails.toSeq
|
||||
items <- ed.items.toSeq
|
||||
ceItem <- items
|
||||
eventTimestamp <- logBase.flatMap(getSourceTimestamp)
|
||||
uuaItem <- getUuaItem(ceItem, logEvent)
|
||||
if isItemTypeValid(ceItem.itemType)
|
||||
} yield {
|
||||
val userIdentifier: UserIdentifier = UserIdentifier(
|
||||
userId = logBase.flatMap(_.userId),
|
||||
guestIdMarketing = logBase.flatMap(_.guestIdMarketing))
|
||||
|
||||
val productSurface: Option[ProductSurface] = ProductSurfaceUtils
|
||||
.getProductSurface(logEvent.eventNamespace)
|
||||
|
||||
val eventMetaData: EventMetadata = ClientEventCommonUtils
|
||||
.getEventMetadata(
|
||||
eventTimestamp = eventTimestamp,
|
||||
logEvent = logEvent,
|
||||
ceItem = ceItem,
|
||||
productSurface = productSurface
|
||||
)
|
||||
|
||||
UnifiedUserAction(
|
||||
userIdentifier = userIdentifier,
|
||||
item = uuaItem,
|
||||
actionType = ActionType.ClientTweetRenderImpression,
|
||||
eventMetadata = eventMetaData,
|
||||
productSurface = productSurface,
|
||||
productSurfaceInfo =
|
||||
ProductSurfaceUtils.getProductSurfaceInfo(productSurface, ceItem, logEvent)
|
||||
)
|
||||
}
|
||||
|
||||
raw.flatMap { e =>
|
||||
e.item match {
|
||||
case TweetInfo(t) =>
|
||||
// If it is an impression toward quoted Tweet we emit 2 impressions, 1 for quoting Tweet
|
||||
// and 1 for the original Tweet.
|
||||
if (t.quotedTweetId.isDefined) {
|
||||
val originalItem = t.copy(
|
||||
actionTweetId = t.quotedTweetId.get,
|
||||
actionTweetAuthorInfo = t.quotedAuthorId.map(id => AuthorInfo(authorId = Some(id))),
|
||||
quotingTweetId = Some(t.actionTweetId),
|
||||
quotedTweetId = None,
|
||||
inReplyToTweetId = None,
|
||||
replyingTweetId = None,
|
||||
retweetingTweetId = None,
|
||||
retweetedTweetId = None,
|
||||
quotedAuthorId = None,
|
||||
retweetingAuthorId = None,
|
||||
inReplyToAuthorId = None
|
||||
)
|
||||
val original = e.copy(item = TweetInfo(originalItem))
|
||||
Seq(original, e)
|
||||
} else Seq(e)
|
||||
case _ => Nil
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object TweetGalleryImpression extends BaseClientEvent(ActionType.ClientTweetGalleryImpression)
|
||||
|
||||
object TweetDetailsImpression extends BaseClientEvent(ActionType.ClientTweetDetailsImpression) {
|
||||
|
||||
case class EventNamespaceInternal(
|
||||
client: String,
|
||||
page: String,
|
||||
section: String,
|
||||
component: String,
|
||||
element: String,
|
||||
action: String)
|
||||
|
||||
def isTweetDetailsImpression(eventNamespaceOpt: Option[EventNamespace]): Boolean =
|
||||
eventNamespaceOpt.exists { eventNamespace =>
|
||||
val eventNamespaceInternal = EventNamespaceInternal(
|
||||
client = eventNamespace.client.getOrElse(""),
|
||||
page = eventNamespace.page.getOrElse(""),
|
||||
section = eventNamespace.section.getOrElse(""),
|
||||
component = eventNamespace.component.getOrElse(""),
|
||||
element = eventNamespace.element.getOrElse(""),
|
||||
action = eventNamespace.action.getOrElse(""),
|
||||
)
|
||||
|
||||
isIphoneAppOrMacAppOrIpadAppClientTweetDetailsImpression(
|
||||
eventNamespaceInternal) || isAndroidAppClientTweetDetailsImpression(
|
||||
eventNamespaceInternal) || isWebClientTweetDetailImpression(
|
||||
eventNamespaceInternal) || isTweetDeckAppClientTweetDetailsImpression(
|
||||
eventNamespaceInternal) || isOtherAppClientTweetDetailsImpression(eventNamespaceInternal)
|
||||
}
|
||||
|
||||
private def isWebClientTweetDetailImpression(
|
||||
eventNamespace: EventNamespaceInternal
|
||||
): Boolean = {
|
||||
val eventNameSpaceStr =
|
||||
eventNamespace.client + ":" + eventNamespace.page + ":" + eventNamespace.section + ":" + eventNamespace.component + ":" + eventNamespace.element + ":" + eventNamespace.action
|
||||
eventNameSpaceStr.equalsIgnoreCase("m5:tweet::::show") || eventNameSpaceStr.equalsIgnoreCase(
|
||||
"m5:tweet:landing:::show") || eventNameSpaceStr
|
||||
.equalsIgnoreCase("m2:tweet::::impression") || eventNameSpaceStr.equalsIgnoreCase(
|
||||
"m2:tweet::tweet::impression") || eventNameSpaceStr
|
||||
.equalsIgnoreCase("LiteNativeWrapper:tweet::::show") || eventNameSpaceStr.equalsIgnoreCase(
|
||||
"LiteNativeWrapper:tweet:landing:::show")
|
||||
}
|
||||
|
||||
private def isOtherAppClientTweetDetailsImpression(
|
||||
eventNamespace: EventNamespaceInternal
|
||||
): Boolean = {
|
||||
val excludedClients = Set(
|
||||
"web",
|
||||
"m5",
|
||||
"m2",
|
||||
"LiteNativeWrapper",
|
||||
"iphone",
|
||||
"ipad",
|
||||
"mac",
|
||||
"android",
|
||||
"android_tablet",
|
||||
"deck")
|
||||
(!excludedClients.contains(eventNamespace.client)) && eventNamespace.page
|
||||
.equalsIgnoreCase("tweet") && eventNamespace.section
|
||||
.equalsIgnoreCase("") && eventNamespace.component
|
||||
.equalsIgnoreCase("tweet") && eventNamespace.element
|
||||
.equalsIgnoreCase("") && eventNamespace.action.equalsIgnoreCase("impression")
|
||||
}
|
||||
|
||||
private def isTweetDeckAppClientTweetDetailsImpression(
|
||||
eventNamespace: EventNamespaceInternal
|
||||
): Boolean =
|
||||
eventNamespace.client
|
||||
.equalsIgnoreCase("deck") && eventNamespace.page
|
||||
.equalsIgnoreCase("tweet") && eventNamespace.section
|
||||
.equalsIgnoreCase("") && eventNamespace.component
|
||||
.equalsIgnoreCase("tweet") && eventNamespace.element
|
||||
.equalsIgnoreCase("") && eventNamespace.action.equalsIgnoreCase("impression")
|
||||
|
||||
private def isAndroidAppClientTweetDetailsImpression(
|
||||
eventNamespace: EventNamespaceInternal
|
||||
): Boolean =
|
||||
(eventNamespace.client
|
||||
.equalsIgnoreCase("android") || eventNamespace.client
|
||||
.equalsIgnoreCase("android_tablet")) && eventNamespace.page
|
||||
.equalsIgnoreCase("tweet") && eventNamespace.section.equalsIgnoreCase(
|
||||
"") && (eventNamespace.component
|
||||
.equalsIgnoreCase("tweet") || eventNamespace.component
|
||||
.matches("^suggest.*_tweet.*$") || eventNamespace.component
|
||||
.equalsIgnoreCase("")) && eventNamespace.element
|
||||
.equalsIgnoreCase("") && eventNamespace.action.equalsIgnoreCase("impression")
|
||||
|
||||
private def isIphoneAppOrMacAppOrIpadAppClientTweetDetailsImpression(
|
||||
eventNamespace: EventNamespaceInternal
|
||||
): Boolean =
|
||||
(eventNamespace.client
|
||||
.equalsIgnoreCase("iphone") || eventNamespace.client
|
||||
.equalsIgnoreCase("ipad") || eventNamespace.client
|
||||
.equalsIgnoreCase("mac")) && eventNamespace.page.equalsIgnoreCase(
|
||||
"tweet") && eventNamespace.section
|
||||
.equalsIgnoreCase("") && (eventNamespace.component
|
||||
.equalsIgnoreCase("tweet") || eventNamespace.component
|
||||
.matches("^suggest.*_tweet.*$")) && eventNamespace.element
|
||||
.equalsIgnoreCase("") && eventNamespace.action.equalsIgnoreCase("impression")
|
||||
}
|
||||
}
|
|
@ -0,0 +1,32 @@
|
|||
package com.twitter.unified_user_actions.adapter.client_event
|
||||
|
||||
import com.twitter.clientapp.thriftscala.{Item => LogEventItem}
|
||||
import com.twitter.suggests.controller_data.home_tweets.thriftscala.HomeTweetsControllerData
|
||||
import com.twitter.suggests.controller_data.home_tweets.thriftscala.HomeTweetsControllerDataAliases.V1Alias
|
||||
import com.twitter.suggests.controller_data.thriftscala.ControllerData
|
||||
import com.twitter.suggests.controller_data.v2.thriftscala.{ControllerData => ControllerDataV2}
|
||||
|
||||
object HomeInfoUtils {
|
||||
|
||||
def getHomeTweetControllerDataV1(ceItem: LogEventItem): Option[V1Alias] = {
|
||||
ceItem.suggestionDetails
|
||||
.flatMap(_.decodedControllerData)
|
||||
.flatMap(_ match {
|
||||
case ControllerData.V2(
|
||||
ControllerDataV2.HomeTweets(
|
||||
HomeTweetsControllerData.V1(homeTweetsControllerDataV1)
|
||||
)) =>
|
||||
Some(homeTweetsControllerDataV1)
|
||||
case _ => None
|
||||
})
|
||||
}
|
||||
|
||||
def getTraceId(ceItem: LogEventItem): Option[Long] =
|
||||
getHomeTweetControllerDataV1(ceItem).flatMap(_.traceId)
|
||||
|
||||
def getSuggestType(ceItem: LogEventItem): Option[String] =
|
||||
ceItem.suggestionDetails.flatMap(_.suggestionType)
|
||||
|
||||
def getRequestJoinId(ceItem: LogEventItem): Option[Long] =
|
||||
getHomeTweetControllerDataV1(ceItem).flatMap(_.requestJoinId)
|
||||
}
|
|
@ -0,0 +1,40 @@
|
|||
package com.twitter.unified_user_actions.adapter.client_event
|
||||
|
||||
import com.twitter.clientapp.thriftscala.ItemType
|
||||
|
||||
object ItemTypeFilterPredicates {
|
||||
private val TweetItemTypes = Set[ItemType](ItemType.Tweet, ItemType.QuotedTweet)
|
||||
private val TopicItemTypes = Set[ItemType](ItemType.Tweet, ItemType.QuotedTweet, ItemType.Topic)
|
||||
private val ProfileItemTypes = Set[ItemType](ItemType.User)
|
||||
private val TypeaheadResultItemTypes = Set[ItemType](ItemType.Search, ItemType.User)
|
||||
private val SearchResultsPageFeedbackSubmitItemTypes =
|
||||
Set[ItemType](ItemType.Tweet, ItemType.RelevancePrompt)
|
||||
|
||||
/**
|
||||
* DDG lambda metrics count Tweets based on the `itemType`
|
||||
* Reference code - https://sourcegraph.twitter.biz/git.twitter.biz/source/-/blob/src/scala/com/twitter/experiments/lambda/shared/Timelines.scala?L156
|
||||
* Since enums `PROMOTED_TWEET` and `POPULAR_TWEET` are deprecated in the following thrift
|
||||
* https://sourcegraph.twitter.biz/git.twitter.biz/source/-/blob/src/thrift/com/twitter/clientapp/gen/client_app.thrift?L131
|
||||
* UUA filters two types of Tweets only: `TWEET` and `QUOTED_TWEET`
|
||||
*/
|
||||
def isItemTypeTweet(itemTypeOpt: Option[ItemType]): Boolean =
|
||||
itemTypeOpt.exists(itemType => TweetItemTypes.contains(itemType))
|
||||
|
||||
def isItemTypeTopic(itemTypeOpt: Option[ItemType]): Boolean =
|
||||
itemTypeOpt.exists(itemType => TopicItemTypes.contains(itemType))
|
||||
|
||||
def isItemTypeProfile(itemTypeOpt: Option[ItemType]): Boolean =
|
||||
itemTypeOpt.exists(itemType => ProfileItemTypes.contains(itemType))
|
||||
|
||||
def isItemTypeTypeaheadResult(itemTypeOpt: Option[ItemType]): Boolean =
|
||||
itemTypeOpt.exists(itemType => TypeaheadResultItemTypes.contains(itemType))
|
||||
|
||||
def isItemTypeForSearchResultsPageFeedbackSubmit(itemTypeOpt: Option[ItemType]): Boolean =
|
||||
itemTypeOpt.exists(itemType => SearchResultsPageFeedbackSubmitItemTypes.contains(itemType))
|
||||
|
||||
/**
|
||||
* Always return true. Use this when there is no need to filter based on `item_type` and all
|
||||
* values of `item_type` are acceptable.
|
||||
*/
|
||||
def ignoreItemType(itemTypeOpt: Option[ItemType]): Boolean = true
|
||||
}
|
|
@ -0,0 +1,26 @@
|
|||
package com.twitter.unified_user_actions.adapter.client_event
|
||||
|
||||
import com.twitter.clientapp.thriftscala.LogEvent
|
||||
import com.twitter.clientapp.thriftscala.{Item => LogEventItem}
|
||||
|
||||
object NotificationClientEventUtils {
|
||||
|
||||
// Notification id for notification in the Notification Tab
|
||||
def getNotificationIdForNotificationTab(
|
||||
ceItem: LogEventItem
|
||||
): Option[String] = {
|
||||
for {
|
||||
notificationTabDetails <- ceItem.notificationTabDetails
|
||||
clientEventMetaData <- notificationTabDetails.clientEventMetadata
|
||||
notificationId <- clientEventMetaData.upstreamId
|
||||
} yield {
|
||||
notificationId
|
||||
}
|
||||
}
|
||||
|
||||
// Notification id for Push Notification
|
||||
def getNotificationIdForPushNotification(logEvent: LogEvent): Option[String] = for {
|
||||
pushNotificationDetails <- logEvent.notificationDetails
|
||||
notificationId <- pushNotificationDetails.impressionId
|
||||
} yield notificationId
|
||||
}
|
|
@ -0,0 +1,109 @@
|
|||
package com.twitter.unified_user_actions.adapter.client_event
|
||||
|
||||
import com.twitter.clientapp.thriftscala.EventNamespace
|
||||
import com.twitter.clientapp.thriftscala.LogEvent
|
||||
import com.twitter.clientapp.thriftscala.{Item => LogEventItem}
|
||||
import com.twitter.suggests.controller_data.home_tweets.thriftscala.HomeTweetsControllerDataAliases.V1Alias
|
||||
import com.twitter.unified_user_actions.thriftscala._
|
||||
|
||||
object ProductSurfaceUtils {
|
||||
|
||||
def getProductSurface(eventNamespace: Option[EventNamespace]): Option[ProductSurface] = {
|
||||
(
|
||||
eventNamespace.flatMap(_.page),
|
||||
eventNamespace.flatMap(_.section),
|
||||
eventNamespace.flatMap(_.element)) match {
|
||||
case (Some("home") | Some("home_latest"), _, _) => Some(ProductSurface.HomeTimeline)
|
||||
case (Some("ntab"), _, _) => Some(ProductSurface.NotificationTab)
|
||||
case (Some(page), Some(section), _) if isPushNotification(page, section) =>
|
||||
Some(ProductSurface.PushNotification)
|
||||
case (Some("search"), _, _) => Some(ProductSurface.SearchResultsPage)
|
||||
case (_, _, Some("typeahead")) => Some(ProductSurface.SearchTypeahead)
|
||||
case _ => None
|
||||
}
|
||||
}
|
||||
|
||||
private def isPushNotification(page: String, section: String): Boolean = {
|
||||
Seq[String]("notification", "toasts").contains(page) ||
|
||||
(page == "app" && section == "push")
|
||||
}
|
||||
|
||||
def getProductSurfaceInfo(
|
||||
productSurface: Option[ProductSurface],
|
||||
ceItem: LogEventItem,
|
||||
logEvent: LogEvent
|
||||
): Option[ProductSurfaceInfo] = {
|
||||
productSurface match {
|
||||
case Some(ProductSurface.HomeTimeline) => createHomeTimelineInfo(ceItem)
|
||||
case Some(ProductSurface.NotificationTab) => createNotificationTabInfo(ceItem)
|
||||
case Some(ProductSurface.PushNotification) => createPushNotificationInfo(logEvent)
|
||||
case Some(ProductSurface.SearchResultsPage) => createSearchResultPageInfo(ceItem, logEvent)
|
||||
case Some(ProductSurface.SearchTypeahead) => createSearchTypeaheadInfo(ceItem, logEvent)
|
||||
case _ => None
|
||||
}
|
||||
}
|
||||
|
||||
private def createPushNotificationInfo(logEvent: LogEvent): Option[ProductSurfaceInfo] =
|
||||
NotificationClientEventUtils.getNotificationIdForPushNotification(logEvent) match {
|
||||
case Some(notificationId) =>
|
||||
Some(
|
||||
ProductSurfaceInfo.PushNotificationInfo(
|
||||
PushNotificationInfo(notificationId = notificationId)))
|
||||
case _ => None
|
||||
}
|
||||
|
||||
private def createNotificationTabInfo(ceItem: LogEventItem): Option[ProductSurfaceInfo] =
|
||||
NotificationClientEventUtils.getNotificationIdForNotificationTab(ceItem) match {
|
||||
case Some(notificationId) =>
|
||||
Some(
|
||||
ProductSurfaceInfo.NotificationTabInfo(
|
||||
NotificationTabInfo(notificationId = notificationId)))
|
||||
case _ => None
|
||||
}
|
||||
|
||||
private def createHomeTimelineInfo(ceItem: LogEventItem): Option[ProductSurfaceInfo] = {
|
||||
def suggestType: Option[String] = HomeInfoUtils.getSuggestType(ceItem)
|
||||
def controllerData: Option[V1Alias] = HomeInfoUtils.getHomeTweetControllerDataV1(ceItem)
|
||||
|
||||
if (suggestType.isDefined || controllerData.isDefined) {
|
||||
Some(
|
||||
ProductSurfaceInfo.HomeTimelineInfo(
|
||||
HomeTimelineInfo(
|
||||
suggestionType = suggestType,
|
||||
injectedPosition = controllerData.flatMap(_.injectedPosition)
|
||||
)))
|
||||
} else None
|
||||
}
|
||||
|
||||
private def createSearchResultPageInfo(
|
||||
ceItem: LogEventItem,
|
||||
logEvent: LogEvent
|
||||
): Option[ProductSurfaceInfo] = {
|
||||
val searchInfoUtil = new SearchInfoUtils(ceItem)
|
||||
searchInfoUtil.getQueryOptFromItem(logEvent).map { query =>
|
||||
ProductSurfaceInfo.SearchResultsPageInfo(
|
||||
SearchResultsPageInfo(
|
||||
query = query,
|
||||
querySource = searchInfoUtil.getQuerySourceOptFromControllerDataFromItem,
|
||||
itemPosition = ceItem.position,
|
||||
tweetResultSources = searchInfoUtil.getTweetResultSources,
|
||||
userResultSources = searchInfoUtil.getUserResultSources,
|
||||
queryFilterType = searchInfoUtil.getQueryFilterType(logEvent)
|
||||
))
|
||||
}
|
||||
}
|
||||
|
||||
private def createSearchTypeaheadInfo(
|
||||
ceItem: LogEventItem,
|
||||
logEvent: LogEvent
|
||||
): Option[ProductSurfaceInfo] = {
|
||||
logEvent.searchDetails.flatMap(_.query).map { query =>
|
||||
ProductSurfaceInfo.SearchTypeaheadInfo(
|
||||
SearchTypeaheadInfo(
|
||||
query = query,
|
||||
itemPosition = ceItem.position
|
||||
)
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,129 @@
|
|||
package com.twitter.unified_user_actions.adapter.client_event
|
||||
|
||||
import com.twitter.clientapp.thriftscala.LogEvent
|
||||
import com.twitter.clientapp.thriftscala.{Item => LogEventItem}
|
||||
import com.twitter.search.common.constants.thriftscala.ThriftQuerySource
|
||||
import com.twitter.search.common.constants.thriftscala.TweetResultSource
|
||||
import com.twitter.search.common.constants.thriftscala.UserResultSource
|
||||
import com.twitter.suggests.controller_data.search_response.item_types.thriftscala.ItemTypesControllerData
|
||||
import com.twitter.suggests.controller_data.search_response.item_types.thriftscala.ItemTypesControllerData.TweetTypesControllerData
|
||||
import com.twitter.suggests.controller_data.search_response.item_types.thriftscala.ItemTypesControllerData.UserTypesControllerData
|
||||
import com.twitter.suggests.controller_data.search_response.request.thriftscala.RequestControllerData
|
||||
import com.twitter.suggests.controller_data.search_response.thriftscala.SearchResponseControllerData.V1
|
||||
import com.twitter.suggests.controller_data.search_response.thriftscala.SearchResponseControllerDataAliases.V1Alias
|
||||
import com.twitter.suggests.controller_data.thriftscala.ControllerData.V2
|
||||
import com.twitter.suggests.controller_data.v2.thriftscala.ControllerData.SearchResponse
|
||||
import com.twitter.unified_user_actions.thriftscala.SearchQueryFilterType
|
||||
import com.twitter.unified_user_actions.thriftscala.SearchQueryFilterType._
|
||||
|
||||
class SearchInfoUtils(item: LogEventItem) {
|
||||
private val searchControllerDataOpt: Option[V1Alias] = item.suggestionDetails.flatMap { sd =>
|
||||
sd.decodedControllerData.flatMap { decodedControllerData =>
|
||||
decodedControllerData match {
|
||||
case V2(v2ControllerData) =>
|
||||
v2ControllerData match {
|
||||
case SearchResponse(searchResponseControllerData) =>
|
||||
searchResponseControllerData match {
|
||||
case V1(searchResponseControllerDataV1) =>
|
||||
Some(searchResponseControllerDataV1)
|
||||
case _ => None
|
||||
}
|
||||
case _ =>
|
||||
None
|
||||
}
|
||||
case _ => None
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private val requestControllerDataOptFromItem: Option[RequestControllerData] =
|
||||
searchControllerDataOpt.flatMap { searchControllerData =>
|
||||
searchControllerData.requestControllerData
|
||||
}
|
||||
private val itemTypesControllerDataOptFromItem: Option[ItemTypesControllerData] =
|
||||
searchControllerDataOpt.flatMap { searchControllerData =>
|
||||
searchControllerData.itemTypesControllerData
|
||||
}
|
||||
|
||||
def checkBit(bitmap: Long, idx: Int): Boolean = {
|
||||
(bitmap / Math.pow(2, idx)).toInt % 2 == 1
|
||||
}
|
||||
|
||||
def getQueryOptFromSearchDetails(logEvent: LogEvent): Option[String] = {
|
||||
logEvent.searchDetails.flatMap { sd => sd.query }
|
||||
}
|
||||
|
||||
def getQueryOptFromControllerDataFromItem: Option[String] = {
|
||||
requestControllerDataOptFromItem.flatMap { rd => rd.rawQuery }
|
||||
}
|
||||
|
||||
def getQueryOptFromItem(logEvent: LogEvent): Option[String] = {
|
||||
// First we try to get the query from controller data, and if that's not available, we fall
|
||||
// back to query in search details. If both are None, queryOpt is None.
|
||||
getQueryOptFromControllerDataFromItem.orElse(getQueryOptFromSearchDetails(logEvent))
|
||||
}
|
||||
|
||||
def getTweetTypesOptFromControllerDataFromItem: Option[TweetTypesControllerData] = {
|
||||
itemTypesControllerDataOptFromItem.flatMap { itemTypes =>
|
||||
itemTypes match {
|
||||
case TweetTypesControllerData(tweetTypesControllerData) =>
|
||||
Some(TweetTypesControllerData(tweetTypesControllerData))
|
||||
case _ => None
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
def getUserTypesOptFromControllerDataFromItem: Option[UserTypesControllerData] = {
|
||||
itemTypesControllerDataOptFromItem.flatMap { itemTypes =>
|
||||
itemTypes match {
|
||||
case UserTypesControllerData(userTypesControllerData) =>
|
||||
Some(UserTypesControllerData(userTypesControllerData))
|
||||
case _ => None
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
def getQuerySourceOptFromControllerDataFromItem: Option[ThriftQuerySource] = {
|
||||
requestControllerDataOptFromItem
|
||||
.flatMap { rd => rd.querySource }
|
||||
.flatMap { querySourceVal => ThriftQuerySource.get(querySourceVal) }
|
||||
}
|
||||
|
||||
def getTweetResultSources: Option[Set[TweetResultSource]] = {
|
||||
getTweetTypesOptFromControllerDataFromItem
|
||||
.flatMap { cd => cd.tweetTypesControllerData.tweetTypesBitmap }
|
||||
.map { tweetTypesBitmap =>
|
||||
TweetResultSource.list.filter { t => checkBit(tweetTypesBitmap, t.value) }.toSet
|
||||
}
|
||||
}
|
||||
|
||||
def getUserResultSources: Option[Set[UserResultSource]] = {
|
||||
getUserTypesOptFromControllerDataFromItem
|
||||
.flatMap { cd => cd.userTypesControllerData.userTypesBitmap }
|
||||
.map { userTypesBitmap =>
|
||||
UserResultSource.list.filter { t => checkBit(userTypesBitmap, t.value) }.toSet
|
||||
}
|
||||
}
|
||||
|
||||
def getQueryFilterType(logEvent: LogEvent): Option[SearchQueryFilterType] = {
|
||||
val searchTab = logEvent.eventNamespace.map(_.client).flatMap {
|
||||
case Some("m5") | Some("android") => logEvent.eventNamespace.flatMap(_.element)
|
||||
case _ => logEvent.eventNamespace.flatMap(_.section)
|
||||
}
|
||||
searchTab.flatMap {
|
||||
case "search_filter_top" => Some(Top)
|
||||
case "search_filter_live" => Some(Latest)
|
||||
// android uses search_filter_tweets instead of search_filter_live
|
||||
case "search_filter_tweets" => Some(Latest)
|
||||
case "search_filter_user" => Some(People)
|
||||
case "search_filter_image" => Some(Photos)
|
||||
case "search_filter_video" => Some(Videos)
|
||||
case _ => None
|
||||
}
|
||||
}
|
||||
|
||||
def getRequestJoinId: Option[Long] = requestControllerDataOptFromItem.flatMap(_.requestJoinId)
|
||||
|
||||
def getTraceId: Option[Long] = requestControllerDataOptFromItem.flatMap(_.traceId)
|
||||
|
||||
}
|
|
@ -0,0 +1,157 @@
|
|||
package com.twitter.unified_user_actions.adapter.client_event
|
||||
|
||||
import com.twitter.clientapp.thriftscala.EventNamespace
|
||||
import com.twitter.clientapp.thriftscala.Item
|
||||
import com.twitter.clientapp.thriftscala.ItemType.Topic
|
||||
import com.twitter.guide.scribing.thriftscala.TopicModuleMetadata
|
||||
import com.twitter.guide.scribing.thriftscala.TransparentGuideDetails
|
||||
import com.twitter.suggests.controller_data.home_hitl_topic_annotation_prompt.thriftscala.HomeHitlTopicAnnotationPromptControllerData
|
||||
import com.twitter.suggests.controller_data.home_hitl_topic_annotation_prompt.v1.thriftscala.{
|
||||
HomeHitlTopicAnnotationPromptControllerData => HomeHitlTopicAnnotationPromptControllerDataV1
|
||||
}
|
||||
import com.twitter.suggests.controller_data.home_topic_annotation_prompt.thriftscala.HomeTopicAnnotationPromptControllerData
|
||||
import com.twitter.suggests.controller_data.home_topic_annotation_prompt.v1.thriftscala.{
|
||||
HomeTopicAnnotationPromptControllerData => HomeTopicAnnotationPromptControllerDataV1
|
||||
}
|
||||
import com.twitter.suggests.controller_data.home_topic_follow_prompt.thriftscala.HomeTopicFollowPromptControllerData
|
||||
import com.twitter.suggests.controller_data.home_topic_follow_prompt.v1.thriftscala.{
|
||||
HomeTopicFollowPromptControllerData => HomeTopicFollowPromptControllerDataV1
|
||||
}
|
||||
import com.twitter.suggests.controller_data.home_tweets.thriftscala.HomeTweetsControllerData
|
||||
import com.twitter.suggests.controller_data.home_tweets.v1.thriftscala.{
|
||||
HomeTweetsControllerData => HomeTweetsControllerDataV1
|
||||
}
|
||||
import com.twitter.suggests.controller_data.search_response.item_types.thriftscala.ItemTypesControllerData
|
||||
import com.twitter.suggests.controller_data.search_response.thriftscala.SearchResponseControllerData
|
||||
import com.twitter.suggests.controller_data.search_response.topic_follow_prompt.thriftscala.SearchTopicFollowPromptControllerData
|
||||
import com.twitter.suggests.controller_data.search_response.tweet_types.thriftscala.TweetTypesControllerData
|
||||
import com.twitter.suggests.controller_data.search_response.v1.thriftscala.{
|
||||
SearchResponseControllerData => SearchResponseControllerDataV1
|
||||
}
|
||||
import com.twitter.suggests.controller_data.thriftscala.ControllerData
|
||||
import com.twitter.suggests.controller_data.timelines_topic.thriftscala.TimelinesTopicControllerData
|
||||
import com.twitter.suggests.controller_data.timelines_topic.v1.thriftscala.{
|
||||
TimelinesTopicControllerData => TimelinesTopicControllerDataV1
|
||||
}
|
||||
import com.twitter.suggests.controller_data.v2.thriftscala.{ControllerData => ControllerDataV2}
|
||||
import com.twitter.util.Try
|
||||
|
||||
object TopicIdUtils {
|
||||
val DomainId: Long = 131 // Topical Domain
|
||||
|
||||
def getTopicId(
|
||||
item: Item,
|
||||
namespace: EventNamespace
|
||||
): Option[Long] =
|
||||
getTopicIdFromHomeSearch(item)
|
||||
.orElse(getTopicFromGuide(item))
|
||||
.orElse(getTopicFromOnboarding(item, namespace))
|
||||
.orElse(getTopicIdFromItem(item))
|
||||
|
||||
def getTopicIdFromItem(item: Item): Option[Long] =
|
||||
if (item.itemType.contains(Topic))
|
||||
item.id
|
||||
else None
|
||||
|
||||
def getTopicIdFromHomeSearch(
|
||||
item: Item
|
||||
): Option[Long] = {
|
||||
val decodedControllerData = item.suggestionDetails.flatMap(_.decodedControllerData)
|
||||
decodedControllerData match {
|
||||
case Some(
|
||||
ControllerData.V2(
|
||||
ControllerDataV2.HomeTweets(
|
||||
HomeTweetsControllerData.V1(homeTweets: HomeTweetsControllerDataV1)))
|
||||
) =>
|
||||
homeTweets.topicId
|
||||
case Some(
|
||||
ControllerData.V2(
|
||||
ControllerDataV2.HomeTopicFollowPrompt(
|
||||
HomeTopicFollowPromptControllerData.V1(
|
||||
homeTopicFollowPrompt: HomeTopicFollowPromptControllerDataV1)))
|
||||
) =>
|
||||
homeTopicFollowPrompt.topicId
|
||||
case Some(
|
||||
ControllerData.V2(
|
||||
ControllerDataV2.TimelinesTopic(
|
||||
TimelinesTopicControllerData.V1(
|
||||
timelinesTopic: TimelinesTopicControllerDataV1
|
||||
)))
|
||||
) =>
|
||||
Some(timelinesTopic.topicId)
|
||||
case Some(
|
||||
ControllerData.V2(
|
||||
ControllerDataV2.SearchResponse(
|
||||
SearchResponseControllerData.V1(s: SearchResponseControllerDataV1)))
|
||||
) =>
|
||||
s.itemTypesControllerData match {
|
||||
case Some(
|
||||
ItemTypesControllerData.TopicFollowControllerData(
|
||||
topicFollowControllerData: SearchTopicFollowPromptControllerData)) =>
|
||||
topicFollowControllerData.topicId
|
||||
case Some(
|
||||
ItemTypesControllerData.TweetTypesControllerData(
|
||||
tweetTypesControllerData: TweetTypesControllerData)) =>
|
||||
tweetTypesControllerData.topicId
|
||||
case _ => None
|
||||
}
|
||||
case Some(
|
||||
ControllerData.V2(
|
||||
ControllerDataV2.HomeTopicAnnotationPrompt(
|
||||
HomeTopicAnnotationPromptControllerData.V1(
|
||||
homeTopicAnnotationPrompt: HomeTopicAnnotationPromptControllerDataV1
|
||||
)))
|
||||
) =>
|
||||
Some(homeTopicAnnotationPrompt.topicId)
|
||||
case Some(
|
||||
ControllerData.V2(
|
||||
ControllerDataV2.HomeHitlTopicAnnotationPrompt(
|
||||
HomeHitlTopicAnnotationPromptControllerData.V1(
|
||||
homeHitlTopicAnnotationPrompt: HomeHitlTopicAnnotationPromptControllerDataV1
|
||||
)))
|
||||
) =>
|
||||
Some(homeHitlTopicAnnotationPrompt.topicId)
|
||||
|
||||
case _ => None
|
||||
}
|
||||
}
|
||||
|
||||
def getTopicFromOnboarding(
|
||||
item: Item,
|
||||
namespace: EventNamespace
|
||||
): Option[Long] =
|
||||
if (namespace.page.contains("onboarding") &&
|
||||
(namespace.section.exists(_.contains("topic")) ||
|
||||
namespace.component.exists(_.contains("topic")) ||
|
||||
namespace.element.exists(_.contains("topic")))) {
|
||||
item.description.flatMap { description =>
|
||||
// description: "id=123,main=xyz,row=1"
|
||||
val tokens = description.split(",").headOption.map(_.split("="))
|
||||
tokens match {
|
||||
case Some(Array("id", token, _*)) => Try(token.toLong).toOption
|
||||
case _ => None
|
||||
}
|
||||
}
|
||||
} else None
|
||||
|
||||
def getTopicFromGuide(
|
||||
item: Item
|
||||
): Option[Long] =
|
||||
item.guideItemDetails.flatMap {
|
||||
_.transparentGuideDetails match {
|
||||
case Some(TransparentGuideDetails.TopicMetadata(topicMetadata)) =>
|
||||
topicMetadata match {
|
||||
case TopicModuleMetadata.TttInterest(_) =>
|
||||
None
|
||||
case TopicModuleMetadata.SemanticCoreInterest(semanticCoreInterest) =>
|
||||
if (semanticCoreInterest.domainId == DomainId.toString)
|
||||
Try(semanticCoreInterest.entityId.toLong).toOption
|
||||
else None
|
||||
case TopicModuleMetadata.SimClusterInterest(_) =>
|
||||
None
|
||||
case TopicModuleMetadata.UnknownUnionField(_) => None
|
||||
}
|
||||
case _ => None
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,42 @@
|
|||
package com.twitter.unified_user_actions.adapter.client_event
|
||||
|
||||
import com.twitter.clientapp.thriftscala.AmplifyDetails
|
||||
import com.twitter.clientapp.thriftscala.MediaDetails
|
||||
import com.twitter.unified_user_actions.thriftscala.TweetVideoWatch
|
||||
import com.twitter.unified_user_actions.thriftscala.TweetActionInfo
|
||||
import com.twitter.video.analytics.thriftscala.MediaIdentifier
|
||||
|
||||
object VideoClientEventUtils {
|
||||
|
||||
/**
|
||||
* For Tweets with multiple videos, find the id of the video that generated the client-event
|
||||
*/
|
||||
def videoIdFromMediaIdentifier(mediaIdentifier: MediaIdentifier): Option[String] =
|
||||
mediaIdentifier match {
|
||||
case MediaIdentifier.MediaPlatformIdentifier(mediaPlatformIdentifier) =>
|
||||
mediaPlatformIdentifier.mediaId.map(_.toString)
|
||||
case _ => None
|
||||
}
|
||||
|
||||
/**
|
||||
* Given:
|
||||
* 1. the id of the video (`mediaId`)
|
||||
* 2. details about all the media items in the Tweet (`mediaItems`),
|
||||
* iterate over the `mediaItems` to lookup the metadata about the video with id `mediaId`.
|
||||
*/
|
||||
def getVideoMetadata(
|
||||
mediaId: String,
|
||||
mediaItems: Seq[MediaDetails],
|
||||
amplifyDetails: Option[AmplifyDetails]
|
||||
): Option[TweetActionInfo] = {
|
||||
mediaItems.collectFirst {
|
||||
case media if media.contentId.contains(mediaId) =>
|
||||
TweetActionInfo.TweetVideoWatch(
|
||||
TweetVideoWatch(
|
||||
mediaType = media.mediaType,
|
||||
isMonetizable = media.dynamicAds,
|
||||
videoType = amplifyDetails.flatMap(_.videoType)
|
||||
))
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,15 @@
|
|||
package com.twitter.unified_user_actions.adapter.common
|
||||
|
||||
import com.twitter.snowflake.id.SnowflakeId
|
||||
import com.twitter.util.Time
|
||||
|
||||
object AdapterUtils {
|
||||
def currentTimestampMs: Long = Time.now.inMilliseconds
|
||||
def getTimestampMsFromTweetId(tweetId: Long): Long = SnowflakeId.unixTimeMillisFromId(tweetId)
|
||||
|
||||
// For now just make sure both language code and country code are in upper cases for consistency
|
||||
// For language code, there are mixed lower and upper cases
|
||||
// For country code, there are mixed lower and upper cases
|
||||
def normalizeLanguageCode(inputLanguageCode: String): String = inputLanguageCode.toUpperCase
|
||||
def normalizeCountryCode(inputCountryCode: String): String = inputCountryCode.toUpperCase
|
||||
}
|
|
@ -0,0 +1,10 @@
|
|||
scala_library(
|
||||
sources = [
|
||||
"*.scala",
|
||||
],
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"snowflake/src/main/scala/com/twitter/snowflake/id",
|
||||
"util/util-core:util-core-util",
|
||||
],
|
||||
)
|
|
@ -0,0 +1,14 @@
|
|||
scala_library(
|
||||
sources = [
|
||||
"*.scala",
|
||||
],
|
||||
compiler_option_sets = ["fatal_warnings"],
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"kafka/finagle-kafka/finatra-kafka/src/main/scala",
|
||||
"src/thrift/com/twitter/ibis:logging-scala",
|
||||
"unified_user_actions/adapter/src/main/scala/com/twitter/unified_user_actions/adapter:base",
|
||||
"unified_user_actions/adapter/src/main/scala/com/twitter/unified_user_actions/adapter/common",
|
||||
"unified_user_actions/thrift/src/main/thrift/com/twitter/unified_user_actions:unified_user_actions-scala",
|
||||
],
|
||||
)
|
|
@ -0,0 +1,55 @@
|
|||
package com.twitter.unified_user_actions.adapter.email_notification_event
|
||||
|
||||
import com.twitter.finagle.stats.NullStatsReceiver
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.finatra.kafka.serde.UnKeyed
|
||||
import com.twitter.ibis.thriftscala.NotificationScribe
|
||||
import com.twitter.ibis.thriftscala.NotificationScribeType
|
||||
import com.twitter.unified_user_actions.adapter.AbstractAdapter
|
||||
import com.twitter.unified_user_actions.thriftscala.ActionType
|
||||
import com.twitter.unified_user_actions.thriftscala.EmailNotificationInfo
|
||||
import com.twitter.unified_user_actions.thriftscala.Item
|
||||
import com.twitter.unified_user_actions.thriftscala.ProductSurface
|
||||
import com.twitter.unified_user_actions.thriftscala.ProductSurfaceInfo
|
||||
import com.twitter.unified_user_actions.thriftscala.TweetInfo
|
||||
import com.twitter.unified_user_actions.thriftscala.UnifiedUserAction
|
||||
import com.twitter.unified_user_actions.thriftscala.UserIdentifier
|
||||
|
||||
class EmailNotificationEventAdapter
|
||||
extends AbstractAdapter[NotificationScribe, UnKeyed, UnifiedUserAction] {
|
||||
import EmailNotificationEventAdapter._
|
||||
override def adaptOneToKeyedMany(
|
||||
input: NotificationScribe,
|
||||
statsReceiver: StatsReceiver = NullStatsReceiver
|
||||
): Seq[(UnKeyed, UnifiedUserAction)] =
|
||||
adaptEvent(input).map { e => (UnKeyed, e) }
|
||||
}
|
||||
|
||||
object EmailNotificationEventAdapter {
|
||||
|
||||
def adaptEvent(scribe: NotificationScribe): Seq[UnifiedUserAction] = {
|
||||
Option(scribe).flatMap { e =>
|
||||
e.`type` match {
|
||||
case NotificationScribeType.Click =>
|
||||
val tweetIdOpt = e.logBase.flatMap(EmailNotificationEventUtils.extractTweetId)
|
||||
(tweetIdOpt, e.impressionId) match {
|
||||
case (Some(tweetId), Some(impressionId)) =>
|
||||
Some(
|
||||
UnifiedUserAction(
|
||||
userIdentifier = UserIdentifier(userId = e.userId),
|
||||
item = Item.TweetInfo(TweetInfo(actionTweetId = tweetId)),
|
||||
actionType = ActionType.ClientTweetEmailClick,
|
||||
eventMetadata = EmailNotificationEventUtils.extractEventMetaData(e),
|
||||
productSurface = Some(ProductSurface.EmailNotification),
|
||||
productSurfaceInfo = Some(
|
||||
ProductSurfaceInfo.EmailNotificationInfo(
|
||||
EmailNotificationInfo(notificationId = impressionId)))
|
||||
)
|
||||
)
|
||||
case _ => None
|
||||
}
|
||||
case _ => None
|
||||
}
|
||||
}.toSeq
|
||||
}
|
||||
}
|
|
@ -0,0 +1,39 @@
|
|||
package com.twitter.unified_user_actions.adapter.email_notification_event
|
||||
|
||||
import com.twitter.ibis.thriftscala.NotificationScribe
|
||||
import com.twitter.logbase.thriftscala.LogBase
|
||||
import com.twitter.unified_user_actions.adapter.common.AdapterUtils
|
||||
import com.twitter.unified_user_actions.thriftscala.EventMetadata
|
||||
import com.twitter.unified_user_actions.thriftscala.SourceLineage
|
||||
|
||||
object EmailNotificationEventUtils {
|
||||
|
||||
/*
|
||||
* Extract TweetId from Logbase.page, here is a sample page below
|
||||
* https://twitter.com/i/events/1580827044245544962?cn=ZmxleGlibGVfcmVjcw%3D%3D&refsrc=email
|
||||
* */
|
||||
def extractTweetId(path: String): Option[Long] = {
|
||||
val ptn = raw".*/([0-9]+)\\??.*".r
|
||||
path match {
|
||||
case ptn(tweetId) =>
|
||||
Some(tweetId.toLong)
|
||||
case _ =>
|
||||
None
|
||||
}
|
||||
}
|
||||
|
||||
def extractTweetId(logBase: LogBase): Option[Long] = logBase.page match {
|
||||
case Some(path) => extractTweetId(path)
|
||||
case None => None
|
||||
}
|
||||
|
||||
def extractEventMetaData(scribe: NotificationScribe): EventMetadata =
|
||||
EventMetadata(
|
||||
sourceTimestampMs = scribe.timestamp,
|
||||
receivedTimestampMs = AdapterUtils.currentTimestampMs,
|
||||
sourceLineage = SourceLineage.EmailNotificationEvents,
|
||||
language = scribe.logBase.flatMap(_.language),
|
||||
countryCode = scribe.logBase.flatMap(_.country),
|
||||
clientAppId = scribe.logBase.flatMap(_.clientAppId),
|
||||
)
|
||||
}
|
|
@ -0,0 +1,14 @@
|
|||
scala_library(
|
||||
sources = [
|
||||
"*.scala",
|
||||
],
|
||||
compiler_option_sets = ["fatal_warnings"],
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"fanoutservice/thrift/src/main/thrift:thrift-scala",
|
||||
"kafka/finagle-kafka/finatra-kafka/src/main/scala",
|
||||
"unified_user_actions/adapter/src/main/scala/com/twitter/unified_user_actions/adapter:base",
|
||||
"unified_user_actions/adapter/src/main/scala/com/twitter/unified_user_actions/adapter/common",
|
||||
"unified_user_actions/thrift/src/main/thrift/com/twitter/unified_user_actions:unified_user_actions-scala",
|
||||
],
|
||||
)
|
|
@ -0,0 +1,52 @@
|
|||
package com.twitter.unified_user_actions.adapter.favorite_archival_events
|
||||
|
||||
import com.twitter.finagle.stats.NullStatsReceiver
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.finatra.kafka.serde.UnKeyed
|
||||
import com.twitter.timelineservice.fanout.thriftscala.FavoriteArchivalEvent
|
||||
import com.twitter.unified_user_actions.adapter.AbstractAdapter
|
||||
import com.twitter.unified_user_actions.adapter.common.AdapterUtils
|
||||
import com.twitter.unified_user_actions.thriftscala._
|
||||
|
||||
class FavoriteArchivalEventsAdapter
|
||||
extends AbstractAdapter[FavoriteArchivalEvent, UnKeyed, UnifiedUserAction] {
|
||||
|
||||
import FavoriteArchivalEventsAdapter._
|
||||
override def adaptOneToKeyedMany(
|
||||
input: FavoriteArchivalEvent,
|
||||
statsReceiver: StatsReceiver = NullStatsReceiver
|
||||
): Seq[(UnKeyed, UnifiedUserAction)] =
|
||||
adaptEvent(input).map { e => (UnKeyed, e) }
|
||||
}
|
||||
|
||||
object FavoriteArchivalEventsAdapter {
|
||||
|
||||
def adaptEvent(e: FavoriteArchivalEvent): Seq[UnifiedUserAction] =
|
||||
Option(e).map { e =>
|
||||
UnifiedUserAction(
|
||||
userIdentifier = UserIdentifier(userId = Some(e.favoriterId)),
|
||||
item = getItem(e),
|
||||
actionType =
|
||||
if (e.isArchivingAction.getOrElse(true)) ActionType.ServerTweetArchiveFavorite
|
||||
else ActionType.ServerTweetUnarchiveFavorite,
|
||||
eventMetadata = getEventMetadata(e)
|
||||
)
|
||||
}.toSeq
|
||||
|
||||
def getItem(e: FavoriteArchivalEvent): Item =
|
||||
Item.TweetInfo(
|
||||
TweetInfo(
|
||||
// Please note that here we always use TweetId (not sourceTweetId)!!!
|
||||
actionTweetId = e.tweetId,
|
||||
actionTweetAuthorInfo = Some(AuthorInfo(authorId = e.tweetUserId)),
|
||||
retweetedTweetId = e.sourceTweetId
|
||||
)
|
||||
)
|
||||
|
||||
def getEventMetadata(e: FavoriteArchivalEvent): EventMetadata =
|
||||
EventMetadata(
|
||||
sourceTimestampMs = e.timestampMs,
|
||||
receivedTimestampMs = AdapterUtils.currentTimestampMs,
|
||||
sourceLineage = SourceLineage.ServerFavoriteArchivalEvents,
|
||||
)
|
||||
}
|
|
@ -0,0 +1,14 @@
|
|||
scala_library(
|
||||
sources = [
|
||||
"*.scala",
|
||||
],
|
||||
compiler_option_sets = ["fatal_warnings"],
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"fanoutservice/thrift/src/main/thrift:thrift-scala",
|
||||
"kafka/finagle-kafka/finatra-kafka/src/main/scala",
|
||||
"unified_user_actions/adapter/src/main/scala/com/twitter/unified_user_actions/adapter:base",
|
||||
"unified_user_actions/adapter/src/main/scala/com/twitter/unified_user_actions/adapter/common",
|
||||
"unified_user_actions/thrift/src/main/thrift/com/twitter/unified_user_actions:unified_user_actions-scala",
|
||||
],
|
||||
)
|
|
@ -0,0 +1,51 @@
|
|||
package com.twitter.unified_user_actions.adapter.retweet_archival_events
|
||||
|
||||
import com.twitter.finagle.stats.NullStatsReceiver
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.finatra.kafka.serde.UnKeyed
|
||||
import com.twitter.tweetypie.thriftscala.RetweetArchivalEvent
|
||||
import com.twitter.unified_user_actions.adapter.AbstractAdapter
|
||||
import com.twitter.unified_user_actions.adapter.common.AdapterUtils
|
||||
import com.twitter.unified_user_actions.thriftscala._
|
||||
|
||||
class RetweetArchivalEventsAdapter
|
||||
extends AbstractAdapter[RetweetArchivalEvent, UnKeyed, UnifiedUserAction] {
|
||||
|
||||
import RetweetArchivalEventsAdapter._
|
||||
override def adaptOneToKeyedMany(
|
||||
input: RetweetArchivalEvent,
|
||||
statsReceiver: StatsReceiver = NullStatsReceiver
|
||||
): Seq[(UnKeyed, UnifiedUserAction)] =
|
||||
adaptEvent(input).map { e => (UnKeyed, e) }
|
||||
}
|
||||
|
||||
object RetweetArchivalEventsAdapter {
|
||||
|
||||
def adaptEvent(e: RetweetArchivalEvent): Seq[UnifiedUserAction] =
|
||||
Option(e).map { e =>
|
||||
UnifiedUserAction(
|
||||
userIdentifier = UserIdentifier(userId = Some(e.retweetUserId)),
|
||||
item = getItem(e),
|
||||
actionType =
|
||||
if (e.isArchivingAction.getOrElse(true)) ActionType.ServerTweetArchiveRetweet
|
||||
else ActionType.ServerTweetUnarchiveRetweet,
|
||||
eventMetadata = getEventMetadata(e)
|
||||
)
|
||||
}.toSeq
|
||||
|
||||
def getItem(e: RetweetArchivalEvent): Item =
|
||||
Item.TweetInfo(
|
||||
TweetInfo(
|
||||
actionTweetId = e.srcTweetId,
|
||||
actionTweetAuthorInfo = Some(AuthorInfo(authorId = Some(e.srcTweetUserId))),
|
||||
retweetingTweetId = Some(e.retweetId)
|
||||
)
|
||||
)
|
||||
|
||||
def getEventMetadata(e: RetweetArchivalEvent): EventMetadata =
|
||||
EventMetadata(
|
||||
sourceTimestampMs = e.timestampMs,
|
||||
receivedTimestampMs = AdapterUtils.currentTimestampMs,
|
||||
sourceLineage = SourceLineage.ServerRetweetArchivalEvents,
|
||||
)
|
||||
}
|
|
@ -0,0 +1,14 @@
|
|||
scala_library(
|
||||
sources = [
|
||||
"*.scala",
|
||||
],
|
||||
compiler_option_sets = ["fatal_warnings"],
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"kafka/finagle-kafka/finatra-kafka/src/main/scala",
|
||||
"src/thrift/com/twitter/socialgraph:thrift-scala",
|
||||
"unified_user_actions/adapter/src/main/scala/com/twitter/unified_user_actions/adapter:base",
|
||||
"unified_user_actions/adapter/src/main/scala/com/twitter/unified_user_actions/adapter/common",
|
||||
"unified_user_actions/thrift/src/main/thrift/com/twitter/unified_user_actions:unified_user_actions-scala",
|
||||
],
|
||||
)
|
|
@ -0,0 +1,24 @@
|
|||
package com.twitter.unified_user_actions.adapter.social_graph_event
|
||||
|
||||
import com.twitter.socialgraph.thriftscala.Action
|
||||
import com.twitter.socialgraph.thriftscala.SrcTargetRequest
|
||||
import com.twitter.unified_user_actions.thriftscala.Item
|
||||
import com.twitter.unified_user_actions.thriftscala.ProfileActionInfo
|
||||
import com.twitter.unified_user_actions.thriftscala.ProfileInfo
|
||||
import com.twitter.unified_user_actions.thriftscala.ServerProfileReport
|
||||
|
||||
abstract class BaseReportSocialGraphWriteEvent[T] extends BaseSocialGraphWriteEvent[T] {
|
||||
def socialGraphAction: Action
|
||||
|
||||
override def getSocialGraphItem(socialGraphSrcTargetRequest: SrcTargetRequest): Item = {
|
||||
Item.ProfileInfo(
|
||||
ProfileInfo(
|
||||
actionProfileId = socialGraphSrcTargetRequest.target,
|
||||
profileActionInfo = Some(
|
||||
ProfileActionInfo.ServerProfileReport(
|
||||
ServerProfileReport(reportType = socialGraphAction)
|
||||
))
|
||||
)
|
||||
)
|
||||
}
|
||||
}
|
|
@ -0,0 +1,60 @@
|
|||
package com.twitter.unified_user_actions.adapter.social_graph_event
|
||||
|
||||
import com.twitter.socialgraph.thriftscala.LogEventContext
|
||||
import com.twitter.socialgraph.thriftscala.SrcTargetRequest
|
||||
import com.twitter.socialgraph.thriftscala.WriteEvent
|
||||
import com.twitter.socialgraph.thriftscala.WriteRequestResult
|
||||
import com.twitter.unified_user_actions.adapter.common.AdapterUtils
|
||||
import com.twitter.unified_user_actions.thriftscala.ActionType
|
||||
import com.twitter.unified_user_actions.thriftscala.EventMetadata
|
||||
import com.twitter.unified_user_actions.thriftscala.Item
|
||||
import com.twitter.unified_user_actions.thriftscala.ProfileInfo
|
||||
import com.twitter.unified_user_actions.thriftscala.SourceLineage
|
||||
import com.twitter.unified_user_actions.thriftscala.UnifiedUserAction
|
||||
import com.twitter.unified_user_actions.thriftscala.UserIdentifier
|
||||
|
||||
trait BaseSocialGraphWriteEvent[T] {
|
||||
def uuaActionType: ActionType
|
||||
|
||||
def getSrcTargetRequest(
|
||||
e: WriteEvent
|
||||
): Seq[SrcTargetRequest] = getSubType(e) match {
|
||||
case Some(subType: Seq[T]) =>
|
||||
getWriteRequestResultFromSubType(subType).collect {
|
||||
case r if r.validationError.isEmpty => r.request
|
||||
}
|
||||
case _ => Nil
|
||||
}
|
||||
|
||||
def getSubType(e: WriteEvent): Option[Seq[T]]
|
||||
def getWriteRequestResultFromSubType(subType: Seq[T]): Seq[WriteRequestResult]
|
||||
|
||||
def toUnifiedUserAction(
|
||||
writeEvent: WriteEvent,
|
||||
uuaAction: BaseSocialGraphWriteEvent[_]
|
||||
): Seq[UnifiedUserAction] =
|
||||
uuaAction.getSrcTargetRequest(writeEvent).map { srcTargetRequest =>
|
||||
UnifiedUserAction(
|
||||
userIdentifier = UserIdentifier(userId = writeEvent.context.loggedInUserId),
|
||||
item = getSocialGraphItem(srcTargetRequest),
|
||||
actionType = uuaAction.uuaActionType,
|
||||
eventMetadata = getEventMetadata(writeEvent.context)
|
||||
)
|
||||
}
|
||||
|
||||
def getSocialGraphItem(socialGraphSrcTargetRequest: SrcTargetRequest): Item = {
|
||||
Item.ProfileInfo(
|
||||
ProfileInfo(
|
||||
actionProfileId = socialGraphSrcTargetRequest.target
|
||||
)
|
||||
)
|
||||
}
|
||||
|
||||
def getEventMetadata(context: LogEventContext): EventMetadata = {
|
||||
EventMetadata(
|
||||
sourceTimestampMs = context.timestamp,
|
||||
receivedTimestampMs = AdapterUtils.currentTimestampMs,
|
||||
sourceLineage = SourceLineage.ServerSocialGraphEvents,
|
||||
)
|
||||
}
|
||||
}
|
|
@ -0,0 +1,48 @@
|
|||
package com.twitter.unified_user_actions.adapter.social_graph_event
|
||||
|
||||
import com.twitter.finagle.stats.NullStatsReceiver
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.finatra.kafka.serde.UnKeyed
|
||||
import com.twitter.socialgraph.thriftscala.Action._
|
||||
import com.twitter.socialgraph.thriftscala.WriteEvent
|
||||
import com.twitter.socialgraph.thriftscala.{Action => SocialGraphAction}
|
||||
import com.twitter.unified_user_actions.adapter.AbstractAdapter
|
||||
import com.twitter.unified_user_actions.adapter.social_graph_event.SocialGraphEngagement._
|
||||
import com.twitter.unified_user_actions.thriftscala.UnifiedUserAction
|
||||
|
||||
class SocialGraphAdapter extends AbstractAdapter[WriteEvent, UnKeyed, UnifiedUserAction] {
|
||||
|
||||
import SocialGraphAdapter._
|
||||
|
||||
override def adaptOneToKeyedMany(
|
||||
input: WriteEvent,
|
||||
statsReceiver: StatsReceiver = NullStatsReceiver
|
||||
): Seq[(UnKeyed, UnifiedUserAction)] =
|
||||
adaptEvent(input).map { e => (UnKeyed, e) }
|
||||
}
|
||||
|
||||
object SocialGraphAdapter {
|
||||
|
||||
def adaptEvent(writeEvent: WriteEvent): Seq[UnifiedUserAction] =
|
||||
Option(writeEvent).flatMap { e =>
|
||||
socialGraphWriteEventTypeToUuaEngagementType.get(e.action)
|
||||
} match {
|
||||
case Some(uuaAction) => uuaAction.toUnifiedUserAction(writeEvent, uuaAction)
|
||||
case None => Nil
|
||||
}
|
||||
|
||||
private val socialGraphWriteEventTypeToUuaEngagementType: Map[
|
||||
SocialGraphAction,
|
||||
BaseSocialGraphWriteEvent[_]
|
||||
] =
|
||||
Map[SocialGraphAction, BaseSocialGraphWriteEvent[_]](
|
||||
Follow -> ProfileFollow,
|
||||
Unfollow -> ProfileUnfollow,
|
||||
Block -> ProfileBlock,
|
||||
Unblock -> ProfileUnblock,
|
||||
Mute -> ProfileMute,
|
||||
Unmute -> ProfileUnmute,
|
||||
ReportAsSpam -> ProfileReportAsSpam,
|
||||
ReportAsAbuse -> ProfileReportAsAbuse
|
||||
)
|
||||
}
|
|
@ -0,0 +1,157 @@
|
|||
package com.twitter.unified_user_actions.adapter.social_graph_event
|
||||
|
||||
import com.twitter.socialgraph.thriftscala.Action
|
||||
import com.twitter.socialgraph.thriftscala.BlockGraphEvent
|
||||
import com.twitter.socialgraph.thriftscala.FollowGraphEvent
|
||||
import com.twitter.socialgraph.thriftscala.MuteGraphEvent
|
||||
import com.twitter.socialgraph.thriftscala.ReportAsAbuseGraphEvent
|
||||
import com.twitter.socialgraph.thriftscala.ReportAsSpamGraphEvent
|
||||
import com.twitter.socialgraph.thriftscala.WriteEvent
|
||||
import com.twitter.socialgraph.thriftscala.WriteRequestResult
|
||||
import com.twitter.unified_user_actions.thriftscala.{ActionType => UuaActionType}
|
||||
|
||||
object SocialGraphEngagement {
|
||||
|
||||
/**
|
||||
* This is "Follow" event to indicate user1 follows user2 captured in ServerProfileFollow
|
||||
*/
|
||||
object ProfileFollow extends BaseSocialGraphWriteEvent[FollowGraphEvent] {
|
||||
override def uuaActionType: UuaActionType = UuaActionType.ServerProfileFollow
|
||||
|
||||
override def getSubType(
|
||||
e: WriteEvent
|
||||
): Option[Seq[FollowGraphEvent]] =
|
||||
e.follow
|
||||
|
||||
override def getWriteRequestResultFromSubType(
|
||||
e: Seq[FollowGraphEvent]
|
||||
): Seq[WriteRequestResult] = {
|
||||
// Remove all redundant operations (FollowGraphEvent.redundantOperation == Some(true))
|
||||
e.collect {
|
||||
case fe if !fe.redundantOperation.getOrElse(false) => fe.result
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This is "Unfollow" event to indicate user1 unfollows user2 captured in ServerProfileUnfollow
|
||||
*
|
||||
* Both Unfollow and Follow use the struct FollowGraphEvent, but are treated in its individual case
|
||||
* class.
|
||||
*/
|
||||
object ProfileUnfollow extends BaseSocialGraphWriteEvent[FollowGraphEvent] {
|
||||
override def uuaActionType: UuaActionType = UuaActionType.ServerProfileUnfollow
|
||||
|
||||
override def getSubType(
|
||||
e: WriteEvent
|
||||
): Option[Seq[FollowGraphEvent]] =
|
||||
e.follow
|
||||
|
||||
override def getWriteRequestResultFromSubType(
|
||||
e: Seq[FollowGraphEvent]
|
||||
): Seq[WriteRequestResult] =
|
||||
e.collect {
|
||||
case fe if !fe.redundantOperation.getOrElse(false) => fe.result
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This is "Block" event to indicate user1 blocks user2 captured in ServerProfileBlock
|
||||
*/
|
||||
object ProfileBlock extends BaseSocialGraphWriteEvent[BlockGraphEvent] {
|
||||
override def uuaActionType: UuaActionType = UuaActionType.ServerProfileBlock
|
||||
|
||||
override def getSubType(
|
||||
e: WriteEvent
|
||||
): Option[Seq[BlockGraphEvent]] =
|
||||
e.block
|
||||
|
||||
override def getWriteRequestResultFromSubType(
|
||||
e: Seq[BlockGraphEvent]
|
||||
): Seq[WriteRequestResult] =
|
||||
e.map(_.result)
|
||||
}
|
||||
|
||||
/**
|
||||
* This is "Unblock" event to indicate user1 unblocks user2 captured in ServerProfileUnblock
|
||||
*
|
||||
* Both Unblock and Block use struct BlockGraphEvent, but are treated in its individual case
|
||||
* class.
|
||||
*/
|
||||
object ProfileUnblock extends BaseSocialGraphWriteEvent[BlockGraphEvent] {
|
||||
override def uuaActionType: UuaActionType = UuaActionType.ServerProfileUnblock
|
||||
|
||||
override def getSubType(
|
||||
e: WriteEvent
|
||||
): Option[Seq[BlockGraphEvent]] =
|
||||
e.block
|
||||
|
||||
override def getWriteRequestResultFromSubType(
|
||||
e: Seq[BlockGraphEvent]
|
||||
): Seq[WriteRequestResult] =
|
||||
e.map(_.result)
|
||||
}
|
||||
|
||||
/**
|
||||
* This is "Mute" event to indicate user1 mutes user2 captured in ServerProfileMute
|
||||
*/
|
||||
object ProfileMute extends BaseSocialGraphWriteEvent[MuteGraphEvent] {
|
||||
override def uuaActionType: UuaActionType = UuaActionType.ServerProfileMute
|
||||
|
||||
override def getSubType(
|
||||
e: WriteEvent
|
||||
): Option[Seq[MuteGraphEvent]] =
|
||||
e.mute
|
||||
|
||||
override def getWriteRequestResultFromSubType(e: Seq[MuteGraphEvent]): Seq[WriteRequestResult] =
|
||||
e.map(_.result)
|
||||
}
|
||||
|
||||
/**
|
||||
* This is "Unmute" event to indicate user1 unmutes user2 captured in ServerProfileUnmute
|
||||
*
|
||||
* Both Unmute and Mute use the struct MuteGraphEvent, but are treated in its individual case
|
||||
* class.
|
||||
*/
|
||||
object ProfileUnmute extends BaseSocialGraphWriteEvent[MuteGraphEvent] {
|
||||
override def uuaActionType: UuaActionType = UuaActionType.ServerProfileUnmute
|
||||
|
||||
override def getSubType(
|
||||
e: WriteEvent
|
||||
): Option[Seq[MuteGraphEvent]] =
|
||||
e.mute
|
||||
|
||||
override def getWriteRequestResultFromSubType(e: Seq[MuteGraphEvent]): Seq[WriteRequestResult] =
|
||||
e.map(_.result)
|
||||
}
|
||||
|
||||
object ProfileReportAsSpam extends BaseReportSocialGraphWriteEvent[ReportAsSpamGraphEvent] {
|
||||
override def uuaActionType: UuaActionType = UuaActionType.ServerProfileReport
|
||||
override def socialGraphAction: Action = Action.ReportAsSpam
|
||||
|
||||
override def getSubType(
|
||||
e: WriteEvent
|
||||
): Option[Seq[ReportAsSpamGraphEvent]] =
|
||||
e.reportAsSpam
|
||||
|
||||
override def getWriteRequestResultFromSubType(
|
||||
e: Seq[ReportAsSpamGraphEvent]
|
||||
): Seq[WriteRequestResult] =
|
||||
e.map(_.result)
|
||||
}
|
||||
|
||||
object ProfileReportAsAbuse extends BaseReportSocialGraphWriteEvent[ReportAsAbuseGraphEvent] {
|
||||
override def uuaActionType: UuaActionType = UuaActionType.ServerProfileReport
|
||||
override def socialGraphAction: Action = Action.ReportAsAbuse
|
||||
|
||||
override def getSubType(
|
||||
e: WriteEvent
|
||||
): Option[Seq[ReportAsAbuseGraphEvent]] =
|
||||
e.reportAsAbuse
|
||||
|
||||
override def getWriteRequestResultFromSubType(
|
||||
e: Seq[ReportAsAbuseGraphEvent]
|
||||
): Seq[WriteRequestResult] =
|
||||
e.map(_.result)
|
||||
}
|
||||
}
|
|
@ -0,0 +1,14 @@
|
|||
scala_library(
|
||||
sources = [
|
||||
"*.scala",
|
||||
],
|
||||
compiler_option_sets = ["fatal_warnings"],
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"kafka/finagle-kafka/finatra-kafka/src/main/scala",
|
||||
"src/thrift/com/twitter/timelineservice/server/internal:thrift-scala",
|
||||
"unified_user_actions/adapter/src/main/scala/com/twitter/unified_user_actions/adapter:base",
|
||||
"unified_user_actions/adapter/src/main/scala/com/twitter/unified_user_actions/adapter/common",
|
||||
"unified_user_actions/thrift/src/main/thrift/com/twitter/unified_user_actions:unified_user_actions-scala",
|
||||
],
|
||||
)
|
|
@ -0,0 +1,109 @@
|
|||
package com.twitter.unified_user_actions.adapter.tls_favs_event
|
||||
|
||||
import com.twitter.finagle.stats.NullStatsReceiver
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.finatra.kafka.serde.UnKeyed
|
||||
import com.twitter.timelineservice.thriftscala._
|
||||
import com.twitter.unified_user_actions.adapter.AbstractAdapter
|
||||
import com.twitter.unified_user_actions.adapter.common.AdapterUtils
|
||||
import com.twitter.unified_user_actions.thriftscala._
|
||||
|
||||
class TlsFavsAdapter
|
||||
extends AbstractAdapter[ContextualizedFavoriteEvent, UnKeyed, UnifiedUserAction] {
|
||||
|
||||
import TlsFavsAdapter._
|
||||
|
||||
override def adaptOneToKeyedMany(
|
||||
input: ContextualizedFavoriteEvent,
|
||||
statsReceiver: StatsReceiver = NullStatsReceiver
|
||||
): Seq[(UnKeyed, UnifiedUserAction)] =
|
||||
adaptEvent(input).map { e => (UnKeyed, e) }
|
||||
}
|
||||
|
||||
object TlsFavsAdapter {
|
||||
|
||||
def adaptEvent(e: ContextualizedFavoriteEvent): Seq[UnifiedUserAction] =
|
||||
Option(e).flatMap { e =>
|
||||
e.event match {
|
||||
case FavoriteEventUnion.Favorite(favoriteEvent) =>
|
||||
Some(
|
||||
UnifiedUserAction(
|
||||
userIdentifier = getUserIdentifier(Left(favoriteEvent)),
|
||||
item = getFavItem(favoriteEvent),
|
||||
actionType = ActionType.ServerTweetFav,
|
||||
eventMetadata = getEventMetadata(Left(favoriteEvent), e.context),
|
||||
productSurface = None,
|
||||
productSurfaceInfo = None
|
||||
))
|
||||
|
||||
case FavoriteEventUnion.Unfavorite(unfavoriteEvent) =>
|
||||
Some(
|
||||
UnifiedUserAction(
|
||||
userIdentifier = getUserIdentifier(Right(unfavoriteEvent)),
|
||||
item = getUnfavItem(unfavoriteEvent),
|
||||
actionType = ActionType.ServerTweetUnfav,
|
||||
eventMetadata = getEventMetadata(Right(unfavoriteEvent), e.context),
|
||||
productSurface = None,
|
||||
productSurfaceInfo = None
|
||||
))
|
||||
|
||||
case _ => None
|
||||
}
|
||||
}.toSeq
|
||||
|
||||
def getFavItem(favoriteEvent: FavoriteEvent): Item =
|
||||
Item.TweetInfo(
|
||||
TweetInfo(
|
||||
actionTweetId = favoriteEvent.tweetId,
|
||||
actionTweetAuthorInfo = Some(AuthorInfo(authorId = Some(favoriteEvent.tweetUserId))),
|
||||
retweetingTweetId = favoriteEvent.retweetId
|
||||
)
|
||||
)
|
||||
|
||||
def getUnfavItem(unfavoriteEvent: UnfavoriteEvent): Item =
|
||||
Item.TweetInfo(
|
||||
TweetInfo(
|
||||
actionTweetId = unfavoriteEvent.tweetId,
|
||||
actionTweetAuthorInfo = Some(AuthorInfo(authorId = Some(unfavoriteEvent.tweetUserId))),
|
||||
retweetingTweetId = unfavoriteEvent.retweetId
|
||||
)
|
||||
)
|
||||
|
||||
def getEventMetadata(
|
||||
event: Either[FavoriteEvent, UnfavoriteEvent],
|
||||
context: LogEventContext
|
||||
): EventMetadata = {
|
||||
val sourceTimestampMs = event match {
|
||||
case Left(favoriteEvent) => favoriteEvent.eventTimeMs
|
||||
case Right(unfavoriteEvent) => unfavoriteEvent.eventTimeMs
|
||||
}
|
||||
// Client UI language, see more at http://go/languagepriority. The format should be ISO 639-1.
|
||||
val language = event match {
|
||||
case Left(favoriteEvent) => favoriteEvent.viewerContext.flatMap(_.requestLanguageCode)
|
||||
case Right(unfavoriteEvent) => unfavoriteEvent.viewerContext.flatMap(_.requestLanguageCode)
|
||||
}
|
||||
// From the request (user’s current location),
|
||||
// see https://sourcegraph.twitter.biz/git.twitter.biz/source/-/blob/src/thrift/com/twitter/context/viewer.thrift?L54
|
||||
// The format should be ISO_3166-1_alpha-2.
|
||||
val countryCode = event match {
|
||||
case Left(favoriteEvent) => favoriteEvent.viewerContext.flatMap(_.requestCountryCode)
|
||||
case Right(unfavoriteEvent) => unfavoriteEvent.viewerContext.flatMap(_.requestCountryCode)
|
||||
}
|
||||
EventMetadata(
|
||||
sourceTimestampMs = sourceTimestampMs,
|
||||
receivedTimestampMs = AdapterUtils.currentTimestampMs,
|
||||
sourceLineage = SourceLineage.ServerTlsFavs,
|
||||
language = language.map(AdapterUtils.normalizeLanguageCode),
|
||||
countryCode = countryCode.map(AdapterUtils.normalizeCountryCode),
|
||||
traceId = Some(context.traceId),
|
||||
clientAppId = context.clientApplicationId,
|
||||
)
|
||||
}
|
||||
|
||||
// Get id of the user that took the action
|
||||
def getUserIdentifier(event: Either[FavoriteEvent, UnfavoriteEvent]): UserIdentifier =
|
||||
event match {
|
||||
case Left(favoriteEvent) => UserIdentifier(userId = Some(favoriteEvent.userId))
|
||||
case Right(unfavoriteEvent) => UserIdentifier(userId = Some(unfavoriteEvent.userId))
|
||||
}
|
||||
}
|
|
@ -0,0 +1,16 @@
|
|||
scala_library(
|
||||
sources = [
|
||||
"*.scala",
|
||||
],
|
||||
compiler_option_sets = ["fatal_warnings"],
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"kafka/finagle-kafka/finatra-kafka/src/main/scala",
|
||||
"src/thrift/com/twitter/gizmoduck:user-thrift-scala",
|
||||
"src/thrift/com/twitter/tweetypie:events-scala",
|
||||
"src/thrift/com/twitter/tweetypie:tweet-scala",
|
||||
"unified_user_actions/adapter/src/main/scala/com/twitter/unified_user_actions/adapter:base",
|
||||
"unified_user_actions/adapter/src/main/scala/com/twitter/unified_user_actions/adapter/common",
|
||||
"unified_user_actions/thrift/src/main/thrift/com/twitter/unified_user_actions:unified_user_actions-scala",
|
||||
],
|
||||
)
|
|
@ -0,0 +1,51 @@
|
|||
package com.twitter.unified_user_actions.adapter.tweetypie_event
|
||||
|
||||
import com.twitter.tweetypie.thriftscala.TweetEventFlags
|
||||
import com.twitter.unified_user_actions.thriftscala.ActionType
|
||||
import com.twitter.unified_user_actions.thriftscala.EventMetadata
|
||||
import com.twitter.unified_user_actions.thriftscala.Item
|
||||
import com.twitter.unified_user_actions.thriftscala.UnifiedUserAction
|
||||
import com.twitter.unified_user_actions.thriftscala.UserIdentifier
|
||||
|
||||
/**
|
||||
* Base class for Tweetypie Tweet Event.
|
||||
* Extends this class if you need to implement the parser for a new Tweetypie Tweet Event Type.
|
||||
* @see https://sourcegraph.twitter.biz/git.twitter.biz/source/-/blob/src/thrift/com/twitter/tweetypie/tweet_events.thrift?L225
|
||||
*/
|
||||
trait BaseTweetypieTweetEvent[T] {
|
||||
|
||||
/**
|
||||
* Returns an Optional UnifiedUserAction from the event.
|
||||
*/
|
||||
def getUnifiedUserAction(event: T, flags: TweetEventFlags): Option[UnifiedUserAction]
|
||||
|
||||
/**
|
||||
* Returns UnifiedUserAction.ActionType for each type of event.
|
||||
*/
|
||||
protected def actionType: ActionType
|
||||
|
||||
/**
|
||||
* Output type of the predicate. Could be an input of getItem.
|
||||
*/
|
||||
type ExtractedEvent
|
||||
|
||||
/**
|
||||
* Returns Some(ExtractedEvent) if the event is valid and None otherwise.
|
||||
*/
|
||||
protected def extract(event: T): Option[ExtractedEvent]
|
||||
|
||||
/**
|
||||
* Get the UnifiedUserAction.Item from the event.
|
||||
*/
|
||||
protected def getItem(extractedEvent: ExtractedEvent, event: T): Item
|
||||
|
||||
/**
|
||||
* Get the UnifiedUserAction.UserIdentifier from the event.
|
||||
*/
|
||||
protected def getUserIdentifier(event: T): UserIdentifier
|
||||
|
||||
/**
|
||||
* Get UnifiedUserAction.EventMetadata from the event.
|
||||
*/
|
||||
protected def getEventMetadata(event: T, flags: TweetEventFlags): EventMetadata
|
||||
}
|
|
@ -0,0 +1,200 @@
|
|||
package com.twitter.unified_user_actions.adapter.tweetypie_event
|
||||
|
||||
import com.twitter.tweetypie.thriftscala.QuotedTweet
|
||||
import com.twitter.tweetypie.thriftscala.Share
|
||||
import com.twitter.tweetypie.thriftscala.TweetCreateEvent
|
||||
import com.twitter.tweetypie.thriftscala.TweetEventFlags
|
||||
import com.twitter.unified_user_actions.adapter.common.AdapterUtils
|
||||
import com.twitter.unified_user_actions.thriftscala.ActionType
|
||||
import com.twitter.unified_user_actions.thriftscala.AuthorInfo
|
||||
import com.twitter.unified_user_actions.thriftscala.EventMetadata
|
||||
import com.twitter.unified_user_actions.thriftscala.Item
|
||||
import com.twitter.unified_user_actions.thriftscala.SourceLineage
|
||||
import com.twitter.unified_user_actions.thriftscala.TweetInfo
|
||||
import com.twitter.unified_user_actions.thriftscala.UnifiedUserAction
|
||||
import com.twitter.unified_user_actions.thriftscala.UserIdentifier
|
||||
|
||||
/**
|
||||
* Base class for Tweetypie TweetCreateEvent including Quote, Reply, Retweet, and Create.
|
||||
*/
|
||||
trait BaseTweetypieTweetEventCreate extends BaseTweetypieTweetEvent[TweetCreateEvent] {
|
||||
type ExtractedEvent
|
||||
protected def actionType: ActionType
|
||||
|
||||
/**
|
||||
* This is the country code where actionTweetId is sent from. For the definitions,
|
||||
* check https://sourcegraph.twitter.biz/git.twitter.biz/source/-/blob/src/thrift/com/twitter/tweetypie/tweet.thrift?L1001.
|
||||
*
|
||||
* UUA sets this to be consistent with IESource to meet existing use requirement.
|
||||
*
|
||||
* For ServerTweetReply/Retweet/Quote, the geo-tagging country code is not available in TweetCreatEvent.
|
||||
* Thus, user signup country is picked to meet a customer use case.
|
||||
*
|
||||
* The definition here conflicts with the intention of UUA to log the request country code
|
||||
* rather than the signup / geo-tagging country.
|
||||
*
|
||||
*/
|
||||
protected def getCountryCode(tce: TweetCreateEvent): Option[String] = {
|
||||
tce.tweet.place match {
|
||||
case Some(p) => p.countryCode
|
||||
case _ => tce.user.safety.flatMap(_.signupCountryCode)
|
||||
}
|
||||
}
|
||||
|
||||
protected def getItem(
|
||||
extractedEvent: ExtractedEvent,
|
||||
tweetCreateEvent: TweetCreateEvent
|
||||
): Item
|
||||
protected def extract(tweetCreateEvent: TweetCreateEvent): Option[ExtractedEvent]
|
||||
|
||||
def getUnifiedUserAction(
|
||||
tweetCreateEvent: TweetCreateEvent,
|
||||
tweetEventFlags: TweetEventFlags
|
||||
): Option[UnifiedUserAction] = {
|
||||
extract(tweetCreateEvent).map { extractedEvent =>
|
||||
UnifiedUserAction(
|
||||
userIdentifier = getUserIdentifier(tweetCreateEvent),
|
||||
item = getItem(extractedEvent, tweetCreateEvent),
|
||||
actionType = actionType,
|
||||
eventMetadata = getEventMetadata(tweetCreateEvent, tweetEventFlags),
|
||||
productSurface = None,
|
||||
productSurfaceInfo = None
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
protected def getUserIdentifier(tweetCreateEvent: TweetCreateEvent): UserIdentifier =
|
||||
UserIdentifier(userId = Some(tweetCreateEvent.user.id))
|
||||
|
||||
protected def getEventMetadata(
|
||||
tweetCreateEvent: TweetCreateEvent,
|
||||
flags: TweetEventFlags
|
||||
): EventMetadata =
|
||||
EventMetadata(
|
||||
sourceTimestampMs = flags.timestampMs,
|
||||
receivedTimestampMs = AdapterUtils.currentTimestampMs,
|
||||
sourceLineage = SourceLineage.ServerTweetypieEvents,
|
||||
traceId = None, // Currently traceId is not stored in TweetCreateEvent
|
||||
// UUA sets this to None since there is no request level language info.
|
||||
language = None,
|
||||
countryCode = getCountryCode(tweetCreateEvent),
|
||||
clientAppId = tweetCreateEvent.tweet.deviceSource.flatMap(_.clientAppId),
|
||||
clientVersion = None // Currently clientVersion is not stored in TweetCreateEvent
|
||||
)
|
||||
}
|
||||
|
||||
/**
|
||||
* Get UnifiedUserAction from a tweet Create.
|
||||
* Note the Create is generated when the tweet is not a Quote/Retweet/Reply.
|
||||
*/
|
||||
object TweetypieCreateEvent extends BaseTweetypieTweetEventCreate {
|
||||
type ExtractedEvent = Long
|
||||
override protected val actionType: ActionType = ActionType.ServerTweetCreate
|
||||
override protected def extract(tweetCreateEvent: TweetCreateEvent): Option[Long] =
|
||||
Option(tweetCreateEvent.tweet.id)
|
||||
|
||||
protected def getItem(
|
||||
tweetId: Long,
|
||||
tweetCreateEvent: TweetCreateEvent
|
||||
): Item =
|
||||
Item.TweetInfo(
|
||||
TweetInfo(
|
||||
actionTweetId = tweetId,
|
||||
actionTweetAuthorInfo = Some(AuthorInfo(authorId = Some(tweetCreateEvent.user.id)))
|
||||
))
|
||||
}
|
||||
|
||||
/**
|
||||
* Get UnifiedUserAction from a Reply.
|
||||
* Note the Reply is generated when someone is replying to a tweet.
|
||||
*/
|
||||
object TweetypieReplyEvent extends BaseTweetypieTweetEventCreate {
|
||||
case class PredicateOutput(tweetId: Long, userId: Long)
|
||||
override type ExtractedEvent = PredicateOutput
|
||||
override protected val actionType: ActionType = ActionType.ServerTweetReply
|
||||
override protected def extract(tweetCreateEvent: TweetCreateEvent): Option[PredicateOutput] =
|
||||
tweetCreateEvent.tweet.coreData
|
||||
.flatMap(_.reply).flatMap(r =>
|
||||
r.inReplyToStatusId.map(tweetId => PredicateOutput(tweetId, r.inReplyToUserId)))
|
||||
|
||||
override protected def getItem(
|
||||
repliedTweet: PredicateOutput,
|
||||
tweetCreateEvent: TweetCreateEvent
|
||||
): Item = {
|
||||
Item.TweetInfo(
|
||||
TweetInfo(
|
||||
actionTweetId = repliedTweet.tweetId,
|
||||
actionTweetAuthorInfo = Some(AuthorInfo(authorId = Some(repliedTweet.userId))),
|
||||
replyingTweetId = Some(tweetCreateEvent.tweet.id)
|
||||
)
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get UnifiedUserAction from a Quote.
|
||||
* Note the Quote is generated when someone is quoting (retweeting with comment) a tweet.
|
||||
*/
|
||||
object TweetypieQuoteEvent extends BaseTweetypieTweetEventCreate {
|
||||
override protected val actionType: ActionType = ActionType.ServerTweetQuote
|
||||
type ExtractedEvent = QuotedTweet
|
||||
override protected def extract(tweetCreateEvent: TweetCreateEvent): Option[QuotedTweet] =
|
||||
tweetCreateEvent.tweet.quotedTweet
|
||||
|
||||
override protected def getItem(
|
||||
quotedTweet: QuotedTweet,
|
||||
tweetCreateEvent: TweetCreateEvent
|
||||
): Item =
|
||||
Item.TweetInfo(
|
||||
TweetInfo(
|
||||
actionTweetId = quotedTweet.tweetId,
|
||||
actionTweetAuthorInfo = Some(AuthorInfo(authorId = Some(quotedTweet.userId))),
|
||||
quotingTweetId = Some(tweetCreateEvent.tweet.id)
|
||||
)
|
||||
)
|
||||
}
|
||||
|
||||
/**
|
||||
* Get UnifiedUserAction from a Retweet.
|
||||
* Note the Retweet is generated when someone is retweeting (without comment) a tweet.
|
||||
*/
|
||||
object TweetypieRetweetEvent extends BaseTweetypieTweetEventCreate {
|
||||
override type ExtractedEvent = Share
|
||||
override protected val actionType: ActionType = ActionType.ServerTweetRetweet
|
||||
override protected def extract(tweetCreateEvent: TweetCreateEvent): Option[Share] =
|
||||
tweetCreateEvent.tweet.coreData.flatMap(_.share)
|
||||
|
||||
override protected def getItem(share: Share, tweetCreateEvent: TweetCreateEvent): Item =
|
||||
Item.TweetInfo(
|
||||
TweetInfo(
|
||||
actionTweetId = share.sourceStatusId,
|
||||
actionTweetAuthorInfo = Some(AuthorInfo(authorId = Some(share.sourceUserId))),
|
||||
retweetingTweetId = Some(tweetCreateEvent.tweet.id)
|
||||
)
|
||||
)
|
||||
}
|
||||
|
||||
/**
|
||||
* Get UnifiedUserAction from a TweetEdit.
|
||||
* Note the Edit is generated when someone is editing their quote or default tweet. The edit will
|
||||
* generate a new Tweet.
|
||||
*/
|
||||
object TweetypieEditEvent extends BaseTweetypieTweetEventCreate {
|
||||
override type ExtractedEvent = Long
|
||||
override protected def actionType: ActionType = ActionType.ServerTweetEdit
|
||||
override protected def extract(tweetCreateEvent: TweetCreateEvent): Option[Long] =
|
||||
TweetypieEventUtils.editedTweetIdFromTweet(tweetCreateEvent.tweet)
|
||||
|
||||
override protected def getItem(
|
||||
editedTweetId: Long,
|
||||
tweetCreateEvent: TweetCreateEvent
|
||||
): Item =
|
||||
Item.TweetInfo(
|
||||
TweetInfo(
|
||||
actionTweetId = tweetCreateEvent.tweet.id,
|
||||
actionTweetAuthorInfo = Some(AuthorInfo(authorId = Some(tweetCreateEvent.user.id))),
|
||||
editedTweetId = Some(editedTweetId),
|
||||
quotedTweetId = tweetCreateEvent.tweet.quotedTweet.map(_.tweetId)
|
||||
)
|
||||
)
|
||||
}
|
|
@ -0,0 +1,146 @@
|
|||
package com.twitter.unified_user_actions.adapter.tweetypie_event
|
||||
|
||||
import com.twitter.tweetypie.thriftscala.QuotedTweet
|
||||
import com.twitter.tweetypie.thriftscala.Share
|
||||
import com.twitter.tweetypie.thriftscala.TweetDeleteEvent
|
||||
import com.twitter.tweetypie.thriftscala.TweetEventFlags
|
||||
import com.twitter.unified_user_actions.adapter.common.AdapterUtils
|
||||
import com.twitter.unified_user_actions.thriftscala.ActionType
|
||||
import com.twitter.unified_user_actions.thriftscala.AuthorInfo
|
||||
import com.twitter.unified_user_actions.thriftscala.EventMetadata
|
||||
import com.twitter.unified_user_actions.thriftscala.Item
|
||||
import com.twitter.unified_user_actions.thriftscala.SourceLineage
|
||||
import com.twitter.unified_user_actions.thriftscala.TweetInfo
|
||||
import com.twitter.unified_user_actions.thriftscala.UnifiedUserAction
|
||||
import com.twitter.unified_user_actions.thriftscala.UserIdentifier
|
||||
|
||||
trait BaseTweetypieTweetEventDelete extends BaseTweetypieTweetEvent[TweetDeleteEvent] {
|
||||
type ExtractedEvent
|
||||
protected def actionType: ActionType
|
||||
|
||||
def getUnifiedUserAction(
|
||||
tweetDeleteEvent: TweetDeleteEvent,
|
||||
tweetEventFlags: TweetEventFlags
|
||||
): Option[UnifiedUserAction] =
|
||||
extract(tweetDeleteEvent).map { extractedEvent =>
|
||||
UnifiedUserAction(
|
||||
userIdentifier = getUserIdentifier(tweetDeleteEvent),
|
||||
item = getItem(extractedEvent, tweetDeleteEvent),
|
||||
actionType = actionType,
|
||||
eventMetadata = getEventMetadata(tweetDeleteEvent, tweetEventFlags)
|
||||
)
|
||||
}
|
||||
|
||||
protected def extract(tweetDeleteEvent: TweetDeleteEvent): Option[ExtractedEvent]
|
||||
|
||||
protected def getItem(extractedEvent: ExtractedEvent, tweetDeleteEvent: TweetDeleteEvent): Item
|
||||
|
||||
protected def getUserIdentifier(tweetDeleteEvent: TweetDeleteEvent): UserIdentifier =
|
||||
UserIdentifier(userId = tweetDeleteEvent.user.map(_.id))
|
||||
|
||||
protected def getEventMetadata(
|
||||
tweetDeleteEvent: TweetDeleteEvent,
|
||||
flags: TweetEventFlags
|
||||
): EventMetadata =
|
||||
EventMetadata(
|
||||
sourceTimestampMs = flags.timestampMs,
|
||||
receivedTimestampMs = AdapterUtils.currentTimestampMs,
|
||||
sourceLineage = SourceLineage.ServerTweetypieEvents,
|
||||
traceId = None, // Currently traceId is not stored in TweetDeleteEvent.
|
||||
// UUA sets this to None since there is no request level language info.
|
||||
language = None,
|
||||
// UUA sets this to be consistent with IESource. For the definition,
|
||||
// see https://sourcegraph.twitter.biz/git.twitter.biz/source/-/blob/src/thrift/com/twitter/tweetypie/tweet.thrift?L1001.
|
||||
// The definition here conflicts with the intention of UUA to log the request country code
|
||||
// rather than the signup / geo-tagging country.
|
||||
countryCode = tweetDeleteEvent.tweet.place.flatMap(_.countryCode),
|
||||
/* clientApplicationId is user's app id if the delete is initiated by a user,
|
||||
* or auditor's app id if the delete is initiated by an auditor */
|
||||
clientAppId = tweetDeleteEvent.audit.flatMap(_.clientApplicationId),
|
||||
clientVersion = None // Currently clientVersion is not stored in TweetDeleteEvent.
|
||||
)
|
||||
}
|
||||
|
||||
object TweetypieDeleteEvent extends BaseTweetypieTweetEventDelete {
|
||||
type ExtractedEvent = Long
|
||||
override protected val actionType: ActionType = ActionType.ServerTweetDelete
|
||||
|
||||
override protected def extract(tweetDeleteEvent: TweetDeleteEvent): Option[Long] = Some(
|
||||
tweetDeleteEvent.tweet.id)
|
||||
|
||||
protected def getItem(
|
||||
tweetId: Long,
|
||||
tweetDeleteEvent: TweetDeleteEvent
|
||||
): Item =
|
||||
Item.TweetInfo(
|
||||
TweetInfo(
|
||||
actionTweetId = tweetId,
|
||||
actionTweetAuthorInfo =
|
||||
Some(AuthorInfo(authorId = tweetDeleteEvent.tweet.coreData.map(_.userId)))
|
||||
))
|
||||
}
|
||||
|
||||
object TweetypieUnretweetEvent extends BaseTweetypieTweetEventDelete {
|
||||
override protected val actionType: ActionType = ActionType.ServerTweetUnretweet
|
||||
|
||||
override type ExtractedEvent = Share
|
||||
|
||||
override protected def extract(tweetDeleteEvent: TweetDeleteEvent): Option[Share] =
|
||||
tweetDeleteEvent.tweet.coreData.flatMap(_.share)
|
||||
|
||||
override protected def getItem(share: Share, tweetDeleteEvent: TweetDeleteEvent): Item =
|
||||
Item.TweetInfo(
|
||||
TweetInfo(
|
||||
actionTweetId = share.sourceStatusId,
|
||||
actionTweetAuthorInfo = Some(AuthorInfo(authorId = Some(share.sourceUserId))),
|
||||
retweetingTweetId = Some(tweetDeleteEvent.tweet.id)
|
||||
)
|
||||
)
|
||||
}
|
||||
|
||||
object TweetypieUnreplyEvent extends BaseTweetypieTweetEventDelete {
|
||||
case class PredicateOutput(tweetId: Long, userId: Long)
|
||||
|
||||
override type ExtractedEvent = PredicateOutput
|
||||
|
||||
override protected val actionType: ActionType = ActionType.ServerTweetUnreply
|
||||
|
||||
override protected def extract(tweetDeleteEvent: TweetDeleteEvent): Option[PredicateOutput] =
|
||||
tweetDeleteEvent.tweet.coreData
|
||||
.flatMap(_.reply).flatMap(r =>
|
||||
r.inReplyToStatusId.map(tweetId => PredicateOutput(tweetId, r.inReplyToUserId)))
|
||||
|
||||
override protected def getItem(
|
||||
repliedTweet: PredicateOutput,
|
||||
tweetDeleteEvent: TweetDeleteEvent
|
||||
): Item = {
|
||||
Item.TweetInfo(
|
||||
TweetInfo(
|
||||
actionTweetId = repliedTweet.tweetId,
|
||||
actionTweetAuthorInfo = Some(AuthorInfo(authorId = Some(repliedTweet.userId))),
|
||||
replyingTweetId = Some(tweetDeleteEvent.tweet.id)
|
||||
)
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
object TweetypieUnquoteEvent extends BaseTweetypieTweetEventDelete {
|
||||
override protected val actionType: ActionType = ActionType.ServerTweetUnquote
|
||||
|
||||
type ExtractedEvent = QuotedTweet
|
||||
|
||||
override protected def extract(tweetDeleteEvent: TweetDeleteEvent): Option[QuotedTweet] =
|
||||
tweetDeleteEvent.tweet.quotedTweet
|
||||
|
||||
override protected def getItem(
|
||||
quotedTweet: QuotedTweet,
|
||||
tweetDeleteEvent: TweetDeleteEvent
|
||||
): Item =
|
||||
Item.TweetInfo(
|
||||
TweetInfo(
|
||||
actionTweetId = quotedTweet.tweetId,
|
||||
actionTweetAuthorInfo = Some(AuthorInfo(authorId = Some(quotedTweet.userId))),
|
||||
quotingTweetId = Some(tweetDeleteEvent.tweet.id)
|
||||
)
|
||||
)
|
||||
}
|
|
@ -0,0 +1,78 @@
|
|||
package com.twitter.unified_user_actions.adapter.tweetypie_event
|
||||
|
||||
import com.twitter.finagle.stats.NullStatsReceiver
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.finatra.kafka.serde.UnKeyed
|
||||
import com.twitter.unified_user_actions.adapter.AbstractAdapter
|
||||
import com.twitter.tweetypie.thriftscala.TweetEvent
|
||||
import com.twitter.tweetypie.thriftscala.TweetEventData
|
||||
import com.twitter.tweetypie.thriftscala.TweetCreateEvent
|
||||
import com.twitter.tweetypie.thriftscala.TweetDeleteEvent
|
||||
import com.twitter.tweetypie.thriftscala.TweetEventFlags
|
||||
import com.twitter.unified_user_actions.thriftscala.UnifiedUserAction
|
||||
|
||||
class TweetypieEventAdapter extends AbstractAdapter[TweetEvent, UnKeyed, UnifiedUserAction] {
|
||||
import TweetypieEventAdapter._
|
||||
override def adaptOneToKeyedMany(
|
||||
tweetEvent: TweetEvent,
|
||||
statsReceiver: StatsReceiver = NullStatsReceiver
|
||||
): Seq[(UnKeyed, UnifiedUserAction)] =
|
||||
adaptEvent(tweetEvent).map(e => (UnKeyed, e))
|
||||
}
|
||||
|
||||
object TweetypieEventAdapter {
|
||||
def adaptEvent(tweetEvent: TweetEvent): Seq[UnifiedUserAction] = {
|
||||
Option(tweetEvent).flatMap { e =>
|
||||
e.data match {
|
||||
case TweetEventData.TweetCreateEvent(tweetCreateEvent: TweetCreateEvent) =>
|
||||
getUUAFromTweetCreateEvent(tweetCreateEvent, e.flags)
|
||||
case TweetEventData.TweetDeleteEvent(tweetDeleteEvent: TweetDeleteEvent) =>
|
||||
getUUAFromTweetDeleteEvent(tweetDeleteEvent, e.flags)
|
||||
case _ => None
|
||||
}
|
||||
}.toSeq
|
||||
}
|
||||
|
||||
def getUUAFromTweetCreateEvent(
|
||||
tweetCreateEvent: TweetCreateEvent,
|
||||
tweetEventFlags: TweetEventFlags
|
||||
): Option[UnifiedUserAction] = {
|
||||
val tweetTypeOpt = TweetypieEventUtils.tweetTypeFromTweet(tweetCreateEvent.tweet)
|
||||
|
||||
tweetTypeOpt.flatMap { tweetType =>
|
||||
tweetType match {
|
||||
case TweetTypeReply =>
|
||||
TweetypieReplyEvent.getUnifiedUserAction(tweetCreateEvent, tweetEventFlags)
|
||||
case TweetTypeRetweet =>
|
||||
TweetypieRetweetEvent.getUnifiedUserAction(tweetCreateEvent, tweetEventFlags)
|
||||
case TweetTypeQuote =>
|
||||
TweetypieQuoteEvent.getUnifiedUserAction(tweetCreateEvent, tweetEventFlags)
|
||||
case TweetTypeDefault =>
|
||||
TweetypieCreateEvent.getUnifiedUserAction(tweetCreateEvent, tweetEventFlags)
|
||||
case TweetTypeEdit =>
|
||||
TweetypieEditEvent.getUnifiedUserAction(tweetCreateEvent, tweetEventFlags)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
def getUUAFromTweetDeleteEvent(
|
||||
tweetDeleteEvent: TweetDeleteEvent,
|
||||
tweetEventFlags: TweetEventFlags
|
||||
): Option[UnifiedUserAction] = {
|
||||
val tweetTypeOpt = TweetypieEventUtils.tweetTypeFromTweet(tweetDeleteEvent.tweet)
|
||||
|
||||
tweetTypeOpt.flatMap { tweetType =>
|
||||
tweetType match {
|
||||
case TweetTypeRetweet =>
|
||||
TweetypieUnretweetEvent.getUnifiedUserAction(tweetDeleteEvent, tweetEventFlags)
|
||||
case TweetTypeReply =>
|
||||
TweetypieUnreplyEvent.getUnifiedUserAction(tweetDeleteEvent, tweetEventFlags)
|
||||
case TweetTypeQuote =>
|
||||
TweetypieUnquoteEvent.getUnifiedUserAction(tweetDeleteEvent, tweetEventFlags)
|
||||
case TweetTypeDefault | TweetTypeEdit =>
|
||||
TweetypieDeleteEvent.getUnifiedUserAction(tweetDeleteEvent, tweetEventFlags)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,54 @@
|
|||
package com.twitter.unified_user_actions.adapter.tweetypie_event
|
||||
|
||||
import com.twitter.tweetypie.thriftscala.EditControl
|
||||
import com.twitter.tweetypie.thriftscala.EditControlEdit
|
||||
import com.twitter.tweetypie.thriftscala.Tweet
|
||||
|
||||
sealed trait TweetypieTweetType
|
||||
object TweetTypeDefault extends TweetypieTweetType
|
||||
object TweetTypeReply extends TweetypieTweetType
|
||||
object TweetTypeRetweet extends TweetypieTweetType
|
||||
object TweetTypeQuote extends TweetypieTweetType
|
||||
object TweetTypeEdit extends TweetypieTweetType
|
||||
|
||||
object TweetypieEventUtils {
|
||||
def editedTweetIdFromTweet(tweet: Tweet): Option[Long] = tweet.editControl.flatMap {
|
||||
case EditControl.Edit(EditControlEdit(initialTweetId, _)) => Some(initialTweetId)
|
||||
case _ => None
|
||||
}
|
||||
|
||||
def tweetTypeFromTweet(tweet: Tweet): Option[TweetypieTweetType] = {
|
||||
val data = tweet.coreData
|
||||
val inReplyingToStatusIdOpt = data.flatMap(_.reply).flatMap(_.inReplyToStatusId)
|
||||
val shareOpt = data.flatMap(_.share)
|
||||
val quotedTweetOpt = tweet.quotedTweet
|
||||
val editedTweetIdOpt = editedTweetIdFromTweet(tweet)
|
||||
|
||||
(inReplyingToStatusIdOpt, shareOpt, quotedTweetOpt, editedTweetIdOpt) match {
|
||||
// Reply
|
||||
case (Some(_), None, _, None) =>
|
||||
Some(TweetTypeReply)
|
||||
// For any kind of retweet (be it retweet of quote tweet or retweet of a regular tweet)
|
||||
// we only need to look at the `share` field
|
||||
// https://confluence.twitter.biz/pages/viewpage.action?spaceKey=CSVC&title=TweetyPie+FAQ#TweetypieFAQ-HowdoItellifaTweetisaRetweet
|
||||
case (None, Some(_), _, None) =>
|
||||
Some(TweetTypeRetweet)
|
||||
// quote
|
||||
case (None, None, Some(_), None) =>
|
||||
Some(TweetTypeQuote)
|
||||
// create
|
||||
case (None, None, None, None) =>
|
||||
Some(TweetTypeDefault)
|
||||
// edit
|
||||
case (None, None, _, Some(_)) =>
|
||||
Some(TweetTypeEdit)
|
||||
// reply and retweet shouldn't be present at the same time
|
||||
case (Some(_), Some(_), _, _) =>
|
||||
None
|
||||
// reply and edit / retweet and edit shouldn't be present at the same time
|
||||
case (Some(_), None, _, Some(_)) | (None, Some(_), _, Some(_)) =>
|
||||
None
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,18 @@
|
|||
scala_library(
|
||||
sources = [
|
||||
"*.scala",
|
||||
],
|
||||
compiler_option_sets = ["fatal_warnings"],
|
||||
tags = [
|
||||
"bazel-compatible",
|
||||
"bazel-only",
|
||||
],
|
||||
dependencies = [
|
||||
"kafka/finagle-kafka/finatra-kafka/src/main/scala",
|
||||
"src/thrift/com/twitter/gizmoduck:thrift-scala",
|
||||
"src/thrift/com/twitter/gizmoduck:user-thrift-scala",
|
||||
"unified_user_actions/adapter/src/main/scala/com/twitter/unified_user_actions/adapter:base",
|
||||
"unified_user_actions/adapter/src/main/scala/com/twitter/unified_user_actions/adapter/common",
|
||||
"unified_user_actions/thrift/src/main/thrift/com/twitter/unified_user_actions:unified_user_actions-scala",
|
||||
],
|
||||
)
|
|
@ -0,0 +1,41 @@
|
|||
package com.twitter.unified_user_actions.adapter.user_modification
|
||||
|
||||
import com.twitter.finagle.stats.NullStatsReceiver
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.finatra.kafka.serde.UnKeyed
|
||||
import com.twitter.gizmoduck.thriftscala.UserModification
|
||||
import com.twitter.unified_user_actions.adapter.AbstractAdapter
|
||||
import com.twitter.unified_user_actions.adapter.user_modification_event.UserCreate
|
||||
import com.twitter.unified_user_actions.adapter.user_modification_event.UserUpdate
|
||||
import com.twitter.unified_user_actions.thriftscala.UnifiedUserAction
|
||||
|
||||
class UserModificationAdapter
|
||||
extends AbstractAdapter[UserModification, UnKeyed, UnifiedUserAction] {
|
||||
|
||||
import UserModificationAdapter._
|
||||
|
||||
override def adaptOneToKeyedMany(
|
||||
input: UserModification,
|
||||
statsReceiver: StatsReceiver = NullStatsReceiver
|
||||
): Seq[(UnKeyed, UnifiedUserAction)] =
|
||||
adaptEvent(input).map { e => (UnKeyed, e) }
|
||||
}
|
||||
|
||||
object UserModificationAdapter {
|
||||
|
||||
def adaptEvent(input: UserModification): Seq[UnifiedUserAction] =
|
||||
Option(input).toSeq.flatMap { e =>
|
||||
if (e.create.isDefined) { // User create
|
||||
Some(UserCreate.getUUA(input))
|
||||
} else if (e.update.isDefined) { // User updates
|
||||
Some(UserUpdate.getUUA(input))
|
||||
} else if (e.destroy.isDefined) {
|
||||
None
|
||||
} else if (e.erase.isDefined) {
|
||||
None
|
||||
} else {
|
||||
throw new IllegalArgumentException(
|
||||
"None of the possible events is defined, there must be something with the source")
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,97 @@
|
|||
package com.twitter.unified_user_actions.adapter.user_modification_event
|
||||
|
||||
import com.twitter.gizmoduck.thriftscala.UserModification
|
||||
import com.twitter.unified_user_actions.adapter.common.AdapterUtils
|
||||
import com.twitter.unified_user_actions.thriftscala.ActionType
|
||||
import com.twitter.unified_user_actions.thriftscala.EventMetadata
|
||||
import com.twitter.unified_user_actions.thriftscala.Item
|
||||
import com.twitter.unified_user_actions.thriftscala.ProfileActionInfo
|
||||
import com.twitter.unified_user_actions.thriftscala.ServerUserUpdate
|
||||
import com.twitter.unified_user_actions.thriftscala.ProfileInfo
|
||||
import com.twitter.unified_user_actions.thriftscala.SourceLineage
|
||||
import com.twitter.unified_user_actions.thriftscala.UnifiedUserAction
|
||||
import com.twitter.unified_user_actions.thriftscala.UserIdentifier
|
||||
|
||||
abstract class BaseUserModificationEvent(actionType: ActionType) {
|
||||
|
||||
def getUUA(input: UserModification): UnifiedUserAction = {
|
||||
val userIdentifier: UserIdentifier = UserIdentifier(userId = input.userId)
|
||||
|
||||
UnifiedUserAction(
|
||||
userIdentifier = userIdentifier,
|
||||
item = getItem(input),
|
||||
actionType = actionType,
|
||||
eventMetadata = getEventMetadata(input),
|
||||
)
|
||||
}
|
||||
|
||||
protected def getItem(input: UserModification): Item =
|
||||
Item.ProfileInfo(
|
||||
ProfileInfo(
|
||||
actionProfileId = input.userId
|
||||
.getOrElse(throw new IllegalArgumentException("target user_id is missing"))
|
||||
)
|
||||
)
|
||||
|
||||
protected def getEventMetadata(input: UserModification): EventMetadata =
|
||||
EventMetadata(
|
||||
sourceTimestampMs = input.updatedAtMsec
|
||||
.getOrElse(throw new IllegalArgumentException("timestamp is required")),
|
||||
receivedTimestampMs = AdapterUtils.currentTimestampMs,
|
||||
sourceLineage = SourceLineage.ServerGizmoduckUserModificationEvents,
|
||||
)
|
||||
}
|
||||
|
||||
/**
|
||||
* When there is a new user creation event in Gizmoduck
|
||||
*/
|
||||
object UserCreate extends BaseUserModificationEvent(ActionType.ServerUserCreate) {
|
||||
override protected def getItem(input: UserModification): Item =
|
||||
Item.ProfileInfo(
|
||||
ProfileInfo(
|
||||
actionProfileId = input.create
|
||||
.map { user =>
|
||||
user.id
|
||||
}.getOrElse(throw new IllegalArgumentException("target user_id is missing")),
|
||||
name = input.create.flatMap { user =>
|
||||
user.profile.map(_.name)
|
||||
},
|
||||
handle = input.create.flatMap { user =>
|
||||
user.profile.map(_.screenName)
|
||||
},
|
||||
description = input.create.flatMap { user =>
|
||||
user.profile.map(_.description)
|
||||
}
|
||||
)
|
||||
)
|
||||
|
||||
override protected def getEventMetadata(input: UserModification): EventMetadata =
|
||||
EventMetadata(
|
||||
sourceTimestampMs = input.create
|
||||
.map { user =>
|
||||
user.updatedAtMsec
|
||||
}.getOrElse(throw new IllegalArgumentException("timestamp is required")),
|
||||
receivedTimestampMs = AdapterUtils.currentTimestampMs,
|
||||
sourceLineage = SourceLineage.ServerGizmoduckUserModificationEvents,
|
||||
)
|
||||
}
|
||||
|
||||
object UserUpdate extends BaseUserModificationEvent(ActionType.ServerUserUpdate) {
|
||||
override protected def getItem(input: UserModification): Item =
|
||||
Item.ProfileInfo(
|
||||
ProfileInfo(
|
||||
actionProfileId =
|
||||
input.userId.getOrElse(throw new IllegalArgumentException("userId is required")),
|
||||
profileActionInfo = Some(
|
||||
ProfileActionInfo.ServerUserUpdate(
|
||||
ServerUserUpdate(updates = input.update.getOrElse(Nil), success = input.success)))
|
||||
)
|
||||
)
|
||||
|
||||
override protected def getEventMetadata(input: UserModification): EventMetadata =
|
||||
EventMetadata(
|
||||
sourceTimestampMs = input.updatedAtMsec.getOrElse(AdapterUtils.currentTimestampMs),
|
||||
receivedTimestampMs = AdapterUtils.currentTimestampMs,
|
||||
sourceLineage = SourceLineage.ServerGizmoduckUserModificationEvents,
|
||||
)
|
||||
}
|
|
@ -0,0 +1,14 @@
|
|||
scala_library(
|
||||
sources = [
|
||||
"*.scala",
|
||||
],
|
||||
compiler_option_sets = ["fatal_warnings"],
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"iesource/thrift/src/main/thrift:thrift-scala",
|
||||
"kafka/finagle-kafka/finatra-kafka/src/main/scala",
|
||||
"unified_user_actions/adapter/src/main/scala/com/twitter/unified_user_actions/adapter:base",
|
||||
"unified_user_actions/adapter/src/main/scala/com/twitter/unified_user_actions/adapter/common",
|
||||
"unified_user_actions/thrift/src/main/thrift/com/twitter/unified_user_actions:unified_user_actions-scala",
|
||||
],
|
||||
)
|
|
@ -0,0 +1,11 @@
|
|||
Currently this dir contains multiple adapters.
|
||||
The goal is similar: to generate Rekeyed (key by TweetId) `KeyedUuaTweet` events that can be
|
||||
used for View Counts (aggregation).
|
||||
|
||||
The 2 adapters:
|
||||
1. Reads from UUA-all topic
|
||||
2. Reads from InteractionEvents
|
||||
We have 2 adapters mainly because currently InteractionEvents have 10% more TweetRenderImpressions
|
||||
than what UUA has. Details can be found at https://docs.google.com/document/d/1UcEzAZ7rFrsU_6kl20R3YZ6u_Jt8PH_4-mVHWe216eM/edit#
|
||||
|
||||
It is still unclear which source should be used, but at a time there should be only one service running.
|
|
@ -0,0 +1,33 @@
|
|||
package com.twitter.unified_user_actions.adapter.uua_aggregates
|
||||
|
||||
import com.twitter.finagle.stats.NullStatsReceiver
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.unified_user_actions.adapter.AbstractAdapter
|
||||
import com.twitter.unified_user_actions.thriftscala._
|
||||
|
||||
/**
|
||||
* The main purpose of the rekey adapter and the rekey service is to not break the existing
|
||||
* customers with the existing Unkeyed and also making the value as a super light-weight schema.
|
||||
* After we rekey from Unkeyed to Long (tweetId), downstream KafkaStreams can directly consume
|
||||
* without repartitioning.
|
||||
*/
|
||||
class RekeyUuaAdapter extends AbstractAdapter[UnifiedUserAction, Long, KeyedUuaTweet] {
|
||||
|
||||
import RekeyUuaAdapter._
|
||||
override def adaptOneToKeyedMany(
|
||||
input: UnifiedUserAction,
|
||||
statsReceiver: StatsReceiver = NullStatsReceiver
|
||||
): Seq[(Long, KeyedUuaTweet)] =
|
||||
adaptEvent(input).map { e => (e.tweetId, e) }
|
||||
}
|
||||
|
||||
object RekeyUuaAdapter {
|
||||
def adaptEvent(e: UnifiedUserAction): Seq[KeyedUuaTweet] =
|
||||
Option(e).flatMap { e =>
|
||||
e.actionType match {
|
||||
case ActionType.ClientTweetRenderImpression =>
|
||||
ClientTweetRenderImpressionUua.getRekeyedUUA(e)
|
||||
case _ => None
|
||||
}
|
||||
}.toSeq
|
||||
}
|
|
@ -0,0 +1,86 @@
|
|||
package com.twitter.unified_user_actions.adapter.uua_aggregates
|
||||
|
||||
import com.twitter.finagle.stats.NullStatsReceiver
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.iesource.thriftscala.ClientEventContext
|
||||
import com.twitter.iesource.thriftscala.EngagingContext
|
||||
import com.twitter.unified_user_actions.adapter.AbstractAdapter
|
||||
import com.twitter.iesource.thriftscala.InteractionType
|
||||
import com.twitter.iesource.thriftscala.InteractionEvent
|
||||
import com.twitter.unified_user_actions.adapter.common.AdapterUtils
|
||||
import com.twitter.unified_user_actions.thriftscala.ActionType
|
||||
import com.twitter.unified_user_actions.thriftscala.EventMetadata
|
||||
import com.twitter.unified_user_actions.thriftscala.KeyedUuaTweet
|
||||
import com.twitter.unified_user_actions.thriftscala.SourceLineage
|
||||
import com.twitter.unified_user_actions.thriftscala.UserIdentifier
|
||||
|
||||
/**
|
||||
* This is to read directly from InteractionEvents
|
||||
*/
|
||||
class RekeyUuaFromInteractionEventsAdapter
|
||||
extends AbstractAdapter[InteractionEvent, Long, KeyedUuaTweet] {
|
||||
|
||||
import RekeyUuaFromInteractionEventsAdapter._
|
||||
override def adaptOneToKeyedMany(
|
||||
input: InteractionEvent,
|
||||
statsReceiver: StatsReceiver = NullStatsReceiver
|
||||
): Seq[(Long, KeyedUuaTweet)] =
|
||||
adaptEvent(input, statsReceiver).map { e => (e.tweetId, e) }
|
||||
}
|
||||
|
||||
object RekeyUuaFromInteractionEventsAdapter {
|
||||
|
||||
def adaptEvent(
|
||||
e: InteractionEvent,
|
||||
statsReceiver: StatsReceiver = NullStatsReceiver
|
||||
): Seq[KeyedUuaTweet] =
|
||||
Option(e).flatMap { e =>
|
||||
e.interactionType.flatMap {
|
||||
case InteractionType.TweetRenderImpression if !isDetailImpression(e.engagingContext) =>
|
||||
getRekeyedUUA(
|
||||
input = e,
|
||||
actionType = ActionType.ClientTweetRenderImpression,
|
||||
sourceLineage = SourceLineage.ClientEvents,
|
||||
statsReceiver = statsReceiver)
|
||||
case _ => None
|
||||
}
|
||||
}.toSeq
|
||||
|
||||
def getRekeyedUUA(
|
||||
input: InteractionEvent,
|
||||
actionType: ActionType,
|
||||
sourceLineage: SourceLineage,
|
||||
statsReceiver: StatsReceiver = NullStatsReceiver
|
||||
): Option[KeyedUuaTweet] =
|
||||
input.engagingUserId match {
|
||||
// please see https://docs.google.com/document/d/1-fy2S-8-YMRQgEN0Sco0OLTmeOIUdqgiZ5G1KwTHt2g/edit#
|
||||
// in order to withstand of potential attacks, we filter out the logged-out users.
|
||||
// Checking user id is 0 is the reverse engineering of
|
||||
// https://sourcegraph.twitter.biz/git.twitter.biz/source/-/blob/iesource/thrift/src/main/thrift/com/twitter/iesource/interaction_event.thrift?L220
|
||||
// https://sourcegraph.twitter.biz/git.twitter.biz/source/-/blob/iesource/common/src/main/scala/com/twitter/iesource/common/converters/client/LogEventConverter.scala?L198
|
||||
case 0L =>
|
||||
statsReceiver.counter("loggedOutEvents").incr()
|
||||
None
|
||||
case _ =>
|
||||
Some(
|
||||
KeyedUuaTweet(
|
||||
tweetId = input.targetId,
|
||||
actionType = actionType,
|
||||
userIdentifier = UserIdentifier(userId = Some(input.engagingUserId)),
|
||||
eventMetadata = EventMetadata(
|
||||
sourceTimestampMs = input.triggeredTimestampMillis.getOrElse(input.timestampMillis),
|
||||
receivedTimestampMs = AdapterUtils.currentTimestampMs,
|
||||
sourceLineage = sourceLineage
|
||||
)
|
||||
))
|
||||
}
|
||||
|
||||
def isDetailImpression(engagingContext: EngagingContext): Boolean =
|
||||
engagingContext match {
|
||||
case EngagingContext.ClientEventContext(
|
||||
ClientEventContext(_, _, _, _, _, _, _, Some(isDetailsImpression), _)
|
||||
) if isDetailsImpression =>
|
||||
true
|
||||
case _ => false
|
||||
}
|
||||
}
|
|
@ -0,0 +1,36 @@
|
|||
package com.twitter.unified_user_actions.adapter.uua_aggregates
|
||||
|
||||
import com.twitter.unified_user_actions.adapter.common.AdapterUtils
|
||||
import com.twitter.unified_user_actions.thriftscala.ActionType
|
||||
import com.twitter.unified_user_actions.thriftscala.EventMetadata
|
||||
import com.twitter.unified_user_actions.thriftscala.Item
|
||||
import com.twitter.unified_user_actions.thriftscala.KeyedUuaTweet
|
||||
import com.twitter.unified_user_actions.thriftscala.UnifiedUserAction
|
||||
|
||||
abstract class BaseUuaAction(actionType: ActionType) {
|
||||
def getRekeyedUUA(input: UnifiedUserAction): Option[KeyedUuaTweet] =
|
||||
getTweetIdFromItem(input.item).map { tweetId =>
|
||||
KeyedUuaTweet(
|
||||
tweetId = tweetId,
|
||||
actionType = input.actionType,
|
||||
userIdentifier = input.userIdentifier,
|
||||
eventMetadata = EventMetadata(
|
||||
sourceTimestampMs = input.eventMetadata.sourceTimestampMs,
|
||||
receivedTimestampMs = AdapterUtils.currentTimestampMs,
|
||||
sourceLineage = input.eventMetadata.sourceLineage
|
||||
)
|
||||
)
|
||||
}
|
||||
|
||||
protected def getTweetIdFromItem(item: Item): Option[Long] = {
|
||||
item match {
|
||||
case Item.TweetInfo(tweetInfo) => Some(tweetInfo.actionTweetId)
|
||||
case _ => None
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* When there is a new user creation event in Gizmoduck
|
||||
*/
|
||||
object ClientTweetRenderImpressionUua extends BaseUuaAction(ActionType.ClientTweetRenderImpression)
|
|
@ -0,0 +1,29 @@
|
|||
package com.twitter.unified_user_actions.adapter
|
||||
|
||||
import com.twitter.inject.Test
|
||||
import com.twitter.unified_user_actions.adapter.common.AdapterUtils
|
||||
import com.twitter.util.Time
|
||||
|
||||
class AdapterUtilsSpec extends Test {
|
||||
trait Fixture {
|
||||
|
||||
val frozenTime: Time = Time.fromMilliseconds(1658949273000L)
|
||||
val languageCode = "en"
|
||||
val countryCode = "us"
|
||||
}
|
||||
|
||||
test("tests") {
|
||||
new Fixture {
|
||||
Time.withTimeAt(frozenTime) { _ =>
|
||||
val actual = Time.fromMilliseconds(AdapterUtils.currentTimestampMs)
|
||||
assert(frozenTime === actual)
|
||||
}
|
||||
|
||||
val actionedTweetId = 1554576940756246272L
|
||||
assert(AdapterUtils.getTimestampMsFromTweetId(actionedTweetId) === 1659474999976L)
|
||||
|
||||
assert(languageCode.toUpperCase === AdapterUtils.normalizeLanguageCode(languageCode))
|
||||
assert(countryCode.toUpperCase === AdapterUtils.normalizeCountryCode(countryCode))
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,282 @@
|
|||
package com.twitter.unified_user_actions.adapter
|
||||
|
||||
import com.twitter.ads.spendserver.thriftscala.SpendServerEvent
|
||||
import com.twitter.adserver.thriftscala.EngagementType
|
||||
import com.twitter.clientapp.thriftscala.AmplifyDetails
|
||||
import com.twitter.inject.Test
|
||||
import com.twitter.unified_user_actions.adapter.TestFixtures.AdsCallbackEngagementsFixture
|
||||
import com.twitter.unified_user_actions.adapter.ads_callback_engagements.AdsCallbackEngagementsAdapter
|
||||
import com.twitter.unified_user_actions.thriftscala.ActionType
|
||||
import com.twitter.unified_user_actions.thriftscala.TweetActionInfo
|
||||
import com.twitter.unified_user_actions.thriftscala.UnifiedUserAction
|
||||
import com.twitter.util.Time
|
||||
import org.scalatest.prop.TableDrivenPropertyChecks
|
||||
|
||||
class AdsCallbackEngagementsAdapterSpec extends Test with TableDrivenPropertyChecks {
|
||||
|
||||
test("Test basic conversion for ads callback engagement type fav") {
|
||||
|
||||
new AdsCallbackEngagementsFixture {
|
||||
Time.withTimeAt(frozenTime) { _ =>
|
||||
val events = Table(
|
||||
("inputEvent", "expectedUuaOutput"),
|
||||
( // Test with authorId
|
||||
createSpendServerEvent(EngagementType.Fav),
|
||||
Seq(
|
||||
createExpectedUua(
|
||||
ActionType.ServerPromotedTweetFav,
|
||||
createTweetInfoItem(authorInfo = Some(authorInfo)))))
|
||||
)
|
||||
forEvery(events) { (event: SpendServerEvent, expected: Seq[UnifiedUserAction]) =>
|
||||
val actual = AdsCallbackEngagementsAdapter.adaptEvent(event)
|
||||
assert(expected === actual)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("Test basic conversion for different engagement types") {
|
||||
new AdsCallbackEngagementsFixture {
|
||||
Time.withTimeAt(frozenTime) { _ =>
|
||||
val mappings = Table(
|
||||
("engagementType", "actionType"),
|
||||
(EngagementType.Unfav, ActionType.ServerPromotedTweetUnfav),
|
||||
(EngagementType.Reply, ActionType.ServerPromotedTweetReply),
|
||||
(EngagementType.Retweet, ActionType.ServerPromotedTweetRetweet),
|
||||
(EngagementType.Block, ActionType.ServerPromotedTweetBlockAuthor),
|
||||
(EngagementType.Unblock, ActionType.ServerPromotedTweetUnblockAuthor),
|
||||
(EngagementType.Send, ActionType.ServerPromotedTweetComposeTweet),
|
||||
(EngagementType.Detail, ActionType.ServerPromotedTweetClick),
|
||||
(EngagementType.Report, ActionType.ServerPromotedTweetReport),
|
||||
(EngagementType.Mute, ActionType.ServerPromotedTweetMuteAuthor),
|
||||
(EngagementType.ProfilePic, ActionType.ServerPromotedTweetClickProfile),
|
||||
(EngagementType.ScreenName, ActionType.ServerPromotedTweetClickProfile),
|
||||
(EngagementType.UserName, ActionType.ServerPromotedTweetClickProfile),
|
||||
(EngagementType.Hashtag, ActionType.ServerPromotedTweetClickHashtag),
|
||||
(EngagementType.CarouselSwipeNext, ActionType.ServerPromotedTweetCarouselSwipeNext),
|
||||
(
|
||||
EngagementType.CarouselSwipePrevious,
|
||||
ActionType.ServerPromotedTweetCarouselSwipePrevious),
|
||||
(EngagementType.DwellShort, ActionType.ServerPromotedTweetLingerImpressionShort),
|
||||
(EngagementType.DwellMedium, ActionType.ServerPromotedTweetLingerImpressionMedium),
|
||||
(EngagementType.DwellLong, ActionType.ServerPromotedTweetLingerImpressionLong),
|
||||
(EngagementType.DismissSpam, ActionType.ServerPromotedTweetDismissSpam),
|
||||
(EngagementType.DismissWithoutReason, ActionType.ServerPromotedTweetDismissWithoutReason),
|
||||
(EngagementType.DismissUninteresting, ActionType.ServerPromotedTweetDismissUninteresting),
|
||||
(EngagementType.DismissRepetitive, ActionType.ServerPromotedTweetDismissRepetitive),
|
||||
)
|
||||
|
||||
forEvery(mappings) { (engagementType: EngagementType, actionType: ActionType) =>
|
||||
val event = createSpendServerEvent(engagementType)
|
||||
val actual = AdsCallbackEngagementsAdapter.adaptEvent(event)
|
||||
val expected =
|
||||
Seq(createExpectedUua(actionType, createTweetInfoItem(authorInfo = Some(authorInfo))))
|
||||
assert(expected === actual)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("Test conversion for ads callback engagement type spotlight view and click") {
|
||||
new AdsCallbackEngagementsFixture {
|
||||
Time.withTimeAt(frozenTime) { _ =>
|
||||
val input = Table(
|
||||
("adsEngagement", "uuaAction"),
|
||||
(EngagementType.SpotlightClick, ActionType.ServerPromotedTweetClickSpotlight),
|
||||
(EngagementType.SpotlightView, ActionType.ServerPromotedTweetViewSpotlight),
|
||||
(EngagementType.TrendView, ActionType.ServerPromotedTrendView),
|
||||
(EngagementType.TrendClick, ActionType.ServerPromotedTrendClick),
|
||||
)
|
||||
forEvery(input) { (engagementType: EngagementType, actionType: ActionType) =>
|
||||
val adsEvent = createSpendServerEvent(engagementType)
|
||||
val expected = Seq(createExpectedUua(actionType, trendInfoItem))
|
||||
val actual = AdsCallbackEngagementsAdapter.adaptEvent(adsEvent)
|
||||
assert(expected === actual)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("Test basic conversion for ads callback engagement open link with or without url") {
|
||||
new AdsCallbackEngagementsFixture {
|
||||
Time.withTimeAt(frozenTime) { _ =>
|
||||
val input = Table(
|
||||
("url", "tweetActionInfo"),
|
||||
(Some("go/url"), openLinkWithUrl),
|
||||
(None, openLinkWithoutUrl)
|
||||
)
|
||||
|
||||
forEvery(input) { (url: Option[String], tweetActionInfo: TweetActionInfo) =>
|
||||
val event = createSpendServerEvent(engagementType = EngagementType.Url, url = url)
|
||||
val actual = AdsCallbackEngagementsAdapter.adaptEvent(event)
|
||||
val expected = Seq(createExpectedUua(
|
||||
ActionType.ServerPromotedTweetOpenLink,
|
||||
createTweetInfoItem(authorInfo = Some(authorInfo), actionInfo = Some(tweetActionInfo))))
|
||||
assert(expected === actual)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("Test basic conversion for different engagement types with profile info") {
|
||||
new AdsCallbackEngagementsFixture {
|
||||
Time.withTimeAt(frozenTime) { _ =>
|
||||
val mappings = Table(
|
||||
("engagementType", "actionType"),
|
||||
(EngagementType.Follow, ActionType.ServerPromotedProfileFollow),
|
||||
(EngagementType.Unfollow, ActionType.ServerPromotedProfileUnfollow)
|
||||
)
|
||||
forEvery(mappings) { (engagementType: EngagementType, actionType: ActionType) =>
|
||||
val event = createSpendServerEvent(engagementType)
|
||||
val actual = AdsCallbackEngagementsAdapter.adaptEvent(event)
|
||||
val expected = Seq(createExpectedUuaWithProfileInfo(actionType))
|
||||
assert(expected === actual)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("Test basic conversion for ads callback engagement type video_content_*") {
|
||||
new AdsCallbackEngagementsFixture {
|
||||
Time.withTimeAt(frozenTime) { _ =>
|
||||
val events = Table(
|
||||
("engagementType", "amplifyDetails", "actionType", "tweetActionInfo"),
|
||||
//For video_content_* events on promoted tweets when there is no preroll ad played
|
||||
(
|
||||
EngagementType.VideoContentPlayback25,
|
||||
amplifyDetailsPromotedTweetWithoutAd,
|
||||
ActionType.ServerPromotedTweetVideoPlayback25,
|
||||
tweetActionInfoPromotedTweetWithoutAd),
|
||||
(
|
||||
EngagementType.VideoContentPlayback50,
|
||||
amplifyDetailsPromotedTweetWithoutAd,
|
||||
ActionType.ServerPromotedTweetVideoPlayback50,
|
||||
tweetActionInfoPromotedTweetWithoutAd),
|
||||
(
|
||||
EngagementType.VideoContentPlayback75,
|
||||
amplifyDetailsPromotedTweetWithoutAd,
|
||||
ActionType.ServerPromotedTweetVideoPlayback75,
|
||||
tweetActionInfoPromotedTweetWithoutAd),
|
||||
//For video_content_* events on promoted tweets when there is a preroll ad
|
||||
(
|
||||
EngagementType.VideoContentPlayback25,
|
||||
amplifyDetailsPromotedTweetWithAd,
|
||||
ActionType.ServerPromotedTweetVideoPlayback25,
|
||||
tweetActionInfoPromotedTweetWithAd),
|
||||
(
|
||||
EngagementType.VideoContentPlayback50,
|
||||
amplifyDetailsPromotedTweetWithAd,
|
||||
ActionType.ServerPromotedTweetVideoPlayback50,
|
||||
tweetActionInfoPromotedTweetWithAd),
|
||||
(
|
||||
EngagementType.VideoContentPlayback75,
|
||||
amplifyDetailsPromotedTweetWithAd,
|
||||
ActionType.ServerPromotedTweetVideoPlayback75,
|
||||
tweetActionInfoPromotedTweetWithAd),
|
||||
)
|
||||
forEvery(events) {
|
||||
(
|
||||
engagementType: EngagementType,
|
||||
amplifyDetails: Option[AmplifyDetails],
|
||||
actionType: ActionType,
|
||||
actionInfo: Option[TweetActionInfo]
|
||||
) =>
|
||||
val spendEvent =
|
||||
createVideoSpendServerEvent(engagementType, amplifyDetails, promotedTweetId, None)
|
||||
val expected = Seq(createExpectedVideoUua(actionType, actionInfo, promotedTweetId))
|
||||
|
||||
val actual = AdsCallbackEngagementsAdapter.adaptEvent(spendEvent)
|
||||
assert(expected === actual)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("Test basic conversion for ads callback engagement type video_ad_*") {
|
||||
|
||||
new AdsCallbackEngagementsFixture {
|
||||
Time.withTimeAt(frozenTime) { _ =>
|
||||
val events = Table(
|
||||
(
|
||||
"engagementType",
|
||||
"amplifyDetails",
|
||||
"actionType",
|
||||
"tweetActionInfo",
|
||||
"promotedTweetId",
|
||||
"organicTweetId"),
|
||||
//For video_ad_* events when the preroll ad is on a promoted tweet.
|
||||
(
|
||||
EngagementType.VideoAdPlayback25,
|
||||
amplifyDetailsPrerollAd,
|
||||
ActionType.ServerPromotedTweetVideoAdPlayback25,
|
||||
tweetActionInfoPrerollAd,
|
||||
promotedTweetId,
|
||||
None
|
||||
),
|
||||
(
|
||||
EngagementType.VideoAdPlayback50,
|
||||
amplifyDetailsPrerollAd,
|
||||
ActionType.ServerPromotedTweetVideoAdPlayback50,
|
||||
tweetActionInfoPrerollAd,
|
||||
promotedTweetId,
|
||||
None
|
||||
),
|
||||
(
|
||||
EngagementType.VideoAdPlayback75,
|
||||
amplifyDetailsPrerollAd,
|
||||
ActionType.ServerPromotedTweetVideoAdPlayback75,
|
||||
tweetActionInfoPrerollAd,
|
||||
promotedTweetId,
|
||||
None
|
||||
),
|
||||
// For video_ad_* events when the preroll ad is on an organic tweet.
|
||||
(
|
||||
EngagementType.VideoAdPlayback25,
|
||||
amplifyDetailsPrerollAd,
|
||||
ActionType.ServerTweetVideoAdPlayback25,
|
||||
tweetActionInfoPrerollAd,
|
||||
None,
|
||||
organicTweetId
|
||||
),
|
||||
(
|
||||
EngagementType.VideoAdPlayback50,
|
||||
amplifyDetailsPrerollAd,
|
||||
ActionType.ServerTweetVideoAdPlayback50,
|
||||
tweetActionInfoPrerollAd,
|
||||
None,
|
||||
organicTweetId
|
||||
),
|
||||
(
|
||||
EngagementType.VideoAdPlayback75,
|
||||
amplifyDetailsPrerollAd,
|
||||
ActionType.ServerTweetVideoAdPlayback75,
|
||||
tweetActionInfoPrerollAd,
|
||||
None,
|
||||
organicTweetId
|
||||
),
|
||||
)
|
||||
forEvery(events) {
|
||||
(
|
||||
engagementType: EngagementType,
|
||||
amplifyDetails: Option[AmplifyDetails],
|
||||
actionType: ActionType,
|
||||
actionInfo: Option[TweetActionInfo],
|
||||
promotedTweetId: Option[Long],
|
||||
organicTweetId: Option[Long],
|
||||
) =>
|
||||
val spendEvent =
|
||||
createVideoSpendServerEvent(
|
||||
engagementType,
|
||||
amplifyDetails,
|
||||
promotedTweetId,
|
||||
organicTweetId)
|
||||
val actionTweetId = if (organicTweetId.isDefined) organicTweetId else promotedTweetId
|
||||
val expected = Seq(createExpectedVideoUua(actionType, actionInfo, actionTweetId))
|
||||
|
||||
val actual = AdsCallbackEngagementsAdapter.adaptEvent(spendEvent)
|
||||
assert(expected === actual)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,23 @@
|
|||
junit_tests(
|
||||
sources = ["**/*.scala"],
|
||||
compiler_option_sets = ["fatal_warnings"],
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"3rdparty/jvm/junit",
|
||||
"3rdparty/jvm/org/scalatest",
|
||||
"3rdparty/jvm/org/scalatestplus:junit",
|
||||
"finatra/inject/inject-core/src/test/scala:test-deps",
|
||||
"unified_user_actions/adapter/src/main/scala/com/twitter/unified_user_actions/adapter/ads_callback_engagements",
|
||||
"unified_user_actions/adapter/src/main/scala/com/twitter/unified_user_actions/adapter/client_event",
|
||||
"unified_user_actions/adapter/src/main/scala/com/twitter/unified_user_actions/adapter/common",
|
||||
"unified_user_actions/adapter/src/main/scala/com/twitter/unified_user_actions/adapter/email_notification_event",
|
||||
"unified_user_actions/adapter/src/main/scala/com/twitter/unified_user_actions/adapter/favorite_archival_events",
|
||||
"unified_user_actions/adapter/src/main/scala/com/twitter/unified_user_actions/adapter/retweet_archival_events",
|
||||
"unified_user_actions/adapter/src/main/scala/com/twitter/unified_user_actions/adapter/social_graph_event",
|
||||
"unified_user_actions/adapter/src/main/scala/com/twitter/unified_user_actions/adapter/tls_favs_event",
|
||||
"unified_user_actions/adapter/src/main/scala/com/twitter/unified_user_actions/adapter/tweetypie_event",
|
||||
"unified_user_actions/adapter/src/main/scala/com/twitter/unified_user_actions/adapter/user_modification_event",
|
||||
"unified_user_actions/adapter/src/main/scala/com/twitter/unified_user_actions/adapter/uua_aggregates",
|
||||
"util/util-mock/src/main/scala/com/twitter/util/mock",
|
||||
],
|
||||
)
|
|
@ -0,0 +1,139 @@
|
|||
package com.twitter.unified_user_actions.adapter
|
||||
|
||||
import com.twitter.inject.Test
|
||||
import com.twitter.storage.behavioral_event.thriftscala.FlattenedEventLog
|
||||
import com.twitter.unified_user_actions.adapter.TestFixtures.BCEFixture
|
||||
import com.twitter.unified_user_actions.adapter.behavioral_client_event.BehavioralClientEventAdapter
|
||||
import com.twitter.unified_user_actions.thriftscala._
|
||||
import com.twitter.util.Time
|
||||
import org.scalatest.prop.TableDrivenPropertyChecks
|
||||
|
||||
class BehavioralClientEventAdapterSpec extends Test with TableDrivenPropertyChecks {
|
||||
|
||||
test("basic event conversion should be correct") {
|
||||
new BCEFixture {
|
||||
Time.withTimeAt(frozenTime) { _ =>
|
||||
val tests = Table(
|
||||
("event", "expected", "description"),
|
||||
(
|
||||
makeBCEEvent(),
|
||||
makeUUAImpressEvent(productSurface = Some(ProductSurface.TweetDetailsPage)),
|
||||
"tweet_details conversion"),
|
||||
(makeBCEProfileImpressEvent(), makeUUAProfileImpressEvent(), "profile conversion"),
|
||||
(
|
||||
makeBCEVideoFullscreenImpressEvent(),
|
||||
makeUUAVideoFullscreenImpressEvent(),
|
||||
"fullscreen_video conversion"),
|
||||
(
|
||||
makeBCEImageFullscreenImpressEvent(),
|
||||
makeUUAImageFullscreenImpressEvent(),
|
||||
"fullscreen_image conversion"),
|
||||
)
|
||||
forEvery(tests) { (input: FlattenedEventLog, expected: UnifiedUserAction, desc: String) =>
|
||||
assert(Seq(expected) === BehavioralClientEventAdapter.adaptEvent(input), desc)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test(
|
||||
"tweet_details is NOT missing productSurface[Info] when empty breadcrumb components and breadcrumbs tweets id") {
|
||||
new BCEFixture {
|
||||
Time.withTimeAt(frozenTime) { _ =>
|
||||
val input = makeBCEEvent(v1BreadcrumbViewTypeHierarchy = None, v1BreadcrumbTweetIds = None)
|
||||
val expected =
|
||||
makeUUAImpressEvent(
|
||||
productSurface = Some(ProductSurface.TweetDetailsPage),
|
||||
breadcrumbViews = None,
|
||||
breadcrumbTweets = None)
|
||||
val actual = BehavioralClientEventAdapter.adaptEvent(input)
|
||||
|
||||
assert(Seq(expected) === actual)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("tweet_details is not missing productSurface[Info] when only breadcrumb tweets is empty") {
|
||||
new BCEFixture {
|
||||
Time.withTimeAt(frozenTime) { _ =>
|
||||
val input = makeBCEEvent(v1BreadcrumbTweetIds = None)
|
||||
val expected = makeUUAImpressEvent(
|
||||
productSurface = Some(ProductSurface.TweetDetailsPage),
|
||||
breadcrumbViews = Some(viewBreadcrumbs),
|
||||
breadcrumbTweets = None
|
||||
)
|
||||
val actual = BehavioralClientEventAdapter.adaptEvent(input)
|
||||
|
||||
assert(Seq(expected) === actual)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("unsupported events should be skipped") {
|
||||
new BCEFixture {
|
||||
val unsupportedPage = "unsupported_page"
|
||||
val unsupportedAction = "unsupported_action"
|
||||
val supportedNamespaces = Table(
|
||||
("page", "actions"),
|
||||
("tweet_details", Seq("impress")),
|
||||
("profile", Seq("impress")),
|
||||
)
|
||||
|
||||
forAll(supportedNamespaces) { (page: String, actions: Seq[String]) =>
|
||||
actions.foreach { supportedAction =>
|
||||
assert(
|
||||
BehavioralClientEventAdapter
|
||||
.adaptEvent(
|
||||
makeBCEEvent(
|
||||
currentPage = Some(unsupportedPage),
|
||||
actionName = Some(supportedAction))).isEmpty)
|
||||
|
||||
assert(BehavioralClientEventAdapter
|
||||
.adaptEvent(
|
||||
makeBCEEvent(currentPage = Some(page), actionName = Some(unsupportedAction))).isEmpty)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("event w/ missing info should be skipped") {
|
||||
new BCEFixture {
|
||||
val eventsWithMissingInfo = Table(
|
||||
("event", "description"),
|
||||
(null.asInstanceOf[FlattenedEventLog], "null event"),
|
||||
(makeBCEEvent(v2Impress = None), "impression event missing v2Impress"),
|
||||
(makeBCEEvent(v1TweetIds = None), "tweet event missing v1TweetIds"),
|
||||
(makeBCEProfileImpressEvent(v1UserIds = None), "profile event missing v1UserIds"),
|
||||
(
|
||||
makeBCEVideoFullscreenImpressEvent(v1BreadcrumbTweetIds = None),
|
||||
"fullscreen_video event missing v1BreadcrumbTweetIds"),
|
||||
(
|
||||
makeBCEImageFullscreenImpressEvent(v1BreadcrumbTweetIds = None),
|
||||
"fullscreen_image event missing v1BreadcrumbTweetIds"),
|
||||
)
|
||||
|
||||
forEvery(eventsWithMissingInfo) { (event: FlattenedEventLog, desc: String) =>
|
||||
assert(
|
||||
BehavioralClientEventAdapter
|
||||
.adaptEvent(event).isEmpty,
|
||||
desc)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("use eventCreateAtMs when driftAdjustedTimetampMs is empty") {
|
||||
new BCEFixture {
|
||||
Time.withTimeAt(frozenTime) { _ =>
|
||||
val input = makeBCEEvent(
|
||||
context = makeBCEContext(driftAdjustedEventCreatedAtMs = None)
|
||||
)
|
||||
val expected = makeUUAImpressEvent(
|
||||
createTs = eventCreatedTime,
|
||||
productSurface = Some(ProductSurface.TweetDetailsPage))
|
||||
val actual = BehavioralClientEventAdapter.adaptEvent(input)
|
||||
|
||||
assert(Seq(expected) === actual)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load diff
|
@ -0,0 +1,20 @@
|
|||
package com.twitter.unified_user_actions.adapter
|
||||
|
||||
import com.twitter.inject.Test
|
||||
import com.twitter.unified_user_actions.adapter.TestFixtures.EmailNotificationEventFixture
|
||||
import com.twitter.unified_user_actions.adapter.email_notification_event.EmailNotificationEventAdapter
|
||||
import com.twitter.util.Time
|
||||
|
||||
class EmailNotificationEventAdapterSpec extends Test {
|
||||
|
||||
test("Notifications with click event") {
|
||||
new EmailNotificationEventFixture {
|
||||
Time.withTimeAt(frozenTime) { _ =>
|
||||
val actual = EmailNotificationEventAdapter.adaptEvent(notificationEvent)
|
||||
assert(expectedUua == actual.head)
|
||||
assert(EmailNotificationEventAdapter.adaptEvent(notificationEventWOTweetId).isEmpty)
|
||||
assert(EmailNotificationEventAdapter.adaptEvent(notificationEventWOImpressionId).isEmpty)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,32 @@
|
|||
package com.twitter.unified_user_actions.adapter
|
||||
|
||||
import com.twitter.inject.Test
|
||||
import com.twitter.unified_user_actions.adapter.TestFixtures.EmailNotificationEventFixture
|
||||
import com.twitter.unified_user_actions.adapter.email_notification_event.EmailNotificationEventUtils
|
||||
|
||||
class EmailNotificationEventUtilsSpec extends Test {
|
||||
|
||||
test("Extract TweetId from pageUrl") {
|
||||
new EmailNotificationEventFixture {
|
||||
|
||||
val invalidUrls: Seq[String] =
|
||||
List("", "abc.com/what/not?x=y", "?abc=def", "12345/", "12345/?")
|
||||
val invalidDomain = "https://twitter.app.link/addressbook/"
|
||||
val numericHandle =
|
||||
"https://twitter.com/1234/status/12345?cxt=HBwWgsDTgY3tp&cn=ZmxleGl&refsrc=email)"
|
||||
|
||||
assert(EmailNotificationEventUtils.extractTweetId(pageUrlStatus).contains(tweetIdStatus))
|
||||
assert(EmailNotificationEventUtils.extractTweetId(pageUrlEvent).contains(tweetIdEvent))
|
||||
assert(EmailNotificationEventUtils.extractTweetId(pageUrlNoArgs).contains(tweetIdNoArgs))
|
||||
assert(EmailNotificationEventUtils.extractTweetId(invalidDomain).isEmpty)
|
||||
assert(EmailNotificationEventUtils.extractTweetId(numericHandle).contains(12345L))
|
||||
invalidUrls.foreach(url => assert(EmailNotificationEventUtils.extractTweetId(url).isEmpty))
|
||||
}
|
||||
}
|
||||
|
||||
test("Extract TweetId from LogBase") {
|
||||
new EmailNotificationEventFixture {
|
||||
assert(EmailNotificationEventUtils.extractTweetId(logBase1).contains(tweetIdStatus))
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,132 @@
|
|||
package com.twitter.unified_user_actions.adapter
|
||||
|
||||
import com.twitter.inject.Test
|
||||
import com.twitter.timelineservice.fanout.thriftscala.FavoriteArchivalEvent
|
||||
import com.twitter.unified_user_actions.adapter.favorite_archival_events.FavoriteArchivalEventsAdapter
|
||||
import com.twitter.unified_user_actions.thriftscala._
|
||||
import com.twitter.util.Time
|
||||
import org.scalatest.prop.TableDrivenPropertyChecks
|
||||
|
||||
class FavoriteArchivalEventsAdapterSpec extends Test with TableDrivenPropertyChecks {
|
||||
trait Fixture {
|
||||
|
||||
val frozenTime = Time.fromMilliseconds(1658949273000L)
|
||||
|
||||
val userId = 1L
|
||||
val authorId = 2L
|
||||
val tweetId = 101L
|
||||
val retweetId = 102L
|
||||
|
||||
val favArchivalEventNoRetweet = FavoriteArchivalEvent(
|
||||
favoriterId = userId,
|
||||
tweetId = tweetId,
|
||||
timestampMs = 0L,
|
||||
isArchivingAction = Some(true),
|
||||
tweetUserId = Some(authorId)
|
||||
)
|
||||
val favArchivalEventRetweet = FavoriteArchivalEvent(
|
||||
favoriterId = userId,
|
||||
tweetId = retweetId,
|
||||
timestampMs = 0L,
|
||||
isArchivingAction = Some(true),
|
||||
tweetUserId = Some(authorId),
|
||||
sourceTweetId = Some(tweetId)
|
||||
)
|
||||
val favUnarchivalEventNoRetweet = FavoriteArchivalEvent(
|
||||
favoriterId = userId,
|
||||
tweetId = tweetId,
|
||||
timestampMs = 0L,
|
||||
isArchivingAction = Some(false),
|
||||
tweetUserId = Some(authorId)
|
||||
)
|
||||
val favUnarchivalEventRetweet = FavoriteArchivalEvent(
|
||||
favoriterId = userId,
|
||||
tweetId = retweetId,
|
||||
timestampMs = 0L,
|
||||
isArchivingAction = Some(false),
|
||||
tweetUserId = Some(authorId),
|
||||
sourceTweetId = Some(tweetId)
|
||||
)
|
||||
|
||||
val expectedUua1 = UnifiedUserAction(
|
||||
userIdentifier = UserIdentifier(userId = Some(userId)),
|
||||
item = Item.TweetInfo(
|
||||
TweetInfo(
|
||||
actionTweetId = tweetId,
|
||||
actionTweetAuthorInfo = Some(AuthorInfo(authorId = Some(authorId))),
|
||||
)
|
||||
),
|
||||
actionType = ActionType.ServerTweetArchiveFavorite,
|
||||
eventMetadata = EventMetadata(
|
||||
sourceTimestampMs = 0L,
|
||||
receivedTimestampMs = frozenTime.inMilliseconds,
|
||||
sourceLineage = SourceLineage.ServerFavoriteArchivalEvents,
|
||||
)
|
||||
)
|
||||
val expectedUua2 = UnifiedUserAction(
|
||||
userIdentifier = UserIdentifier(userId = Some(userId)),
|
||||
item = Item.TweetInfo(
|
||||
TweetInfo(
|
||||
actionTweetId = retweetId,
|
||||
actionTweetAuthorInfo = Some(AuthorInfo(authorId = Some(authorId))),
|
||||
retweetedTweetId = Some(tweetId)
|
||||
)
|
||||
),
|
||||
actionType = ActionType.ServerTweetArchiveFavorite,
|
||||
eventMetadata = EventMetadata(
|
||||
sourceTimestampMs = 0L,
|
||||
receivedTimestampMs = frozenTime.inMilliseconds,
|
||||
sourceLineage = SourceLineage.ServerFavoriteArchivalEvents,
|
||||
)
|
||||
)
|
||||
val expectedUua3 = UnifiedUserAction(
|
||||
userIdentifier = UserIdentifier(userId = Some(userId)),
|
||||
item = Item.TweetInfo(
|
||||
TweetInfo(
|
||||
actionTweetId = tweetId,
|
||||
actionTweetAuthorInfo = Some(AuthorInfo(authorId = Some(authorId))),
|
||||
)
|
||||
),
|
||||
actionType = ActionType.ServerTweetUnarchiveFavorite,
|
||||
eventMetadata = EventMetadata(
|
||||
sourceTimestampMs = 0L,
|
||||
receivedTimestampMs = frozenTime.inMilliseconds,
|
||||
sourceLineage = SourceLineage.ServerFavoriteArchivalEvents,
|
||||
)
|
||||
)
|
||||
val expectedUua4 = UnifiedUserAction(
|
||||
userIdentifier = UserIdentifier(userId = Some(userId)),
|
||||
item = Item.TweetInfo(
|
||||
TweetInfo(
|
||||
actionTweetId = retweetId,
|
||||
actionTweetAuthorInfo = Some(AuthorInfo(authorId = Some(authorId))),
|
||||
retweetedTweetId = Some(tweetId)
|
||||
)
|
||||
),
|
||||
actionType = ActionType.ServerTweetUnarchiveFavorite,
|
||||
eventMetadata = EventMetadata(
|
||||
sourceTimestampMs = 0L,
|
||||
receivedTimestampMs = frozenTime.inMilliseconds,
|
||||
sourceLineage = SourceLineage.ServerFavoriteArchivalEvents,
|
||||
)
|
||||
)
|
||||
}
|
||||
|
||||
test("all tests") {
|
||||
new Fixture {
|
||||
Time.withTimeAt(frozenTime) { _ =>
|
||||
val table = Table(
|
||||
("event", "expected"),
|
||||
(favArchivalEventNoRetweet, expectedUua1),
|
||||
(favArchivalEventRetweet, expectedUua2),
|
||||
(favUnarchivalEventNoRetweet, expectedUua3),
|
||||
(favUnarchivalEventRetweet, expectedUua4)
|
||||
)
|
||||
forEvery(table) { (event: FavoriteArchivalEvent, expected: UnifiedUserAction) =>
|
||||
val actual = FavoriteArchivalEventsAdapter.adaptEvent(event)
|
||||
assert(Seq(expected) === actual)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,36 @@
|
|||
package com.twitter.unified_user_actions.adapter
|
||||
|
||||
import com.twitter.inject.Test
|
||||
import com.twitter.unified_user_actions.adapter.TestFixtures.InteractionEventsFixtures
|
||||
import com.twitter.unified_user_actions.adapter.uua_aggregates.RekeyUuaFromInteractionEventsAdapter
|
||||
import com.twitter.util.Time
|
||||
import org.scalatest.prop.TableDrivenPropertyChecks
|
||||
|
||||
class RekeyUuaFromInteractionEventsAdapterSpec extends Test with TableDrivenPropertyChecks {
|
||||
test("ClientTweetRenderImpressions") {
|
||||
new InteractionEventsFixtures {
|
||||
Time.withTimeAt(frozenTime) { _ =>
|
||||
assert(
|
||||
RekeyUuaFromInteractionEventsAdapter.adaptEvent(baseInteractionEvent) === Seq(
|
||||
expectedBaseKeyedUuaTweet))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("Filter out logged out users") {
|
||||
new InteractionEventsFixtures {
|
||||
Time.withTimeAt(frozenTime) { _ =>
|
||||
assert(RekeyUuaFromInteractionEventsAdapter.adaptEvent(loggedOutInteractionEvent) === Nil)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("Filter out detail impressions") {
|
||||
new InteractionEventsFixtures {
|
||||
Time.withTimeAt(frozenTime) { _ =>
|
||||
assert(
|
||||
RekeyUuaFromInteractionEventsAdapter.adaptEvent(detailImpressionInteractionEvent) === Nil)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,86 @@
|
|||
package com.twitter.unified_user_actions.adapter
|
||||
|
||||
import com.twitter.inject.Test
|
||||
import com.twitter.tweetypie.thriftscala.RetweetArchivalEvent
|
||||
import com.twitter.unified_user_actions.adapter.retweet_archival_events.RetweetArchivalEventsAdapter
|
||||
import com.twitter.unified_user_actions.thriftscala._
|
||||
import com.twitter.util.Time
|
||||
import org.scalatest.prop.TableDrivenPropertyChecks
|
||||
|
||||
class RetweetArchivalEventsAdapterSpec extends Test with TableDrivenPropertyChecks {
|
||||
trait Fixture {
|
||||
|
||||
val frozenTime = Time.fromMilliseconds(1658949273000L)
|
||||
|
||||
val authorId = 1L
|
||||
val tweetId = 101L
|
||||
val retweetId = 102L
|
||||
val retweetAuthorId = 2L
|
||||
|
||||
val retweetArchivalEvent = RetweetArchivalEvent(
|
||||
retweetId = retweetId,
|
||||
srcTweetId = tweetId,
|
||||
retweetUserId = retweetAuthorId,
|
||||
srcTweetUserId = authorId,
|
||||
timestampMs = 0L,
|
||||
isArchivingAction = Some(true),
|
||||
)
|
||||
val retweetUnarchivalEvent = RetweetArchivalEvent(
|
||||
retweetId = retweetId,
|
||||
srcTweetId = tweetId,
|
||||
retweetUserId = retweetAuthorId,
|
||||
srcTweetUserId = authorId,
|
||||
timestampMs = 0L,
|
||||
isArchivingAction = Some(false),
|
||||
)
|
||||
|
||||
val expectedUua1 = UnifiedUserAction(
|
||||
userIdentifier = UserIdentifier(userId = Some(retweetAuthorId)),
|
||||
item = Item.TweetInfo(
|
||||
TweetInfo(
|
||||
actionTweetId = tweetId,
|
||||
actionTweetAuthorInfo = Some(AuthorInfo(authorId = Some(authorId))),
|
||||
retweetingTweetId = Some(retweetId)
|
||||
)
|
||||
),
|
||||
actionType = ActionType.ServerTweetArchiveRetweet,
|
||||
eventMetadata = EventMetadata(
|
||||
sourceTimestampMs = 0L,
|
||||
receivedTimestampMs = frozenTime.inMilliseconds,
|
||||
sourceLineage = SourceLineage.ServerRetweetArchivalEvents,
|
||||
)
|
||||
)
|
||||
val expectedUua2 = UnifiedUserAction(
|
||||
userIdentifier = UserIdentifier(userId = Some(retweetAuthorId)),
|
||||
item = Item.TweetInfo(
|
||||
TweetInfo(
|
||||
actionTweetId = tweetId,
|
||||
actionTweetAuthorInfo = Some(AuthorInfo(authorId = Some(authorId))),
|
||||
retweetingTweetId = Some(retweetId)
|
||||
)
|
||||
),
|
||||
actionType = ActionType.ServerTweetUnarchiveRetweet,
|
||||
eventMetadata = EventMetadata(
|
||||
sourceTimestampMs = 0L,
|
||||
receivedTimestampMs = frozenTime.inMilliseconds,
|
||||
sourceLineage = SourceLineage.ServerRetweetArchivalEvents,
|
||||
)
|
||||
)
|
||||
}
|
||||
|
||||
test("all tests") {
|
||||
new Fixture {
|
||||
Time.withTimeAt(frozenTime) { _ =>
|
||||
val table = Table(
|
||||
("event", "expected"),
|
||||
(retweetArchivalEvent, expectedUua1),
|
||||
(retweetUnarchivalEvent, expectedUua2),
|
||||
)
|
||||
forEvery(table) { (event: RetweetArchivalEvent, expected: UnifiedUserAction) =>
|
||||
val actual = RetweetArchivalEventsAdapter.adaptEvent(event)
|
||||
assert(Seq(expected) === actual)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,355 @@
|
|||
package com.twitter.unified_user_actions.adapter
|
||||
|
||||
import com.twitter.clientapp.thriftscala.SuggestionDetails
|
||||
import com.twitter.clientapp.thriftscala._
|
||||
import com.twitter.search.common.constants.thriftscala.ThriftQuerySource
|
||||
import com.twitter.search.common.constants.thriftscala.TweetResultSource
|
||||
import com.twitter.search.common.constants.thriftscala.UserResultSource
|
||||
import com.twitter.suggests.controller_data.search_response.item_types.thriftscala.ItemTypesControllerData
|
||||
import com.twitter.suggests.controller_data.search_response.request.thriftscala.RequestControllerData
|
||||
import com.twitter.suggests.controller_data.search_response.thriftscala.SearchResponseControllerData
|
||||
import com.twitter.suggests.controller_data.search_response.tweet_types.thriftscala.TweetTypesControllerData
|
||||
import com.twitter.suggests.controller_data.search_response.user_types.thriftscala.UserTypesControllerData
|
||||
import com.twitter.suggests.controller_data.search_response.v1.thriftscala.{
|
||||
SearchResponseControllerData => SearchResponseControllerDataV1
|
||||
}
|
||||
import com.twitter.suggests.controller_data.thriftscala.ControllerData
|
||||
import com.twitter.suggests.controller_data.v2.thriftscala.{ControllerData => ControllerDataV2}
|
||||
import com.twitter.util.mock.Mockito
|
||||
import org.junit.runner.RunWith
|
||||
import org.scalatest.funsuite.AnyFunSuite
|
||||
import org.scalatest.matchers.should.Matchers
|
||||
import org.scalatest.prop.TableDrivenPropertyChecks
|
||||
import org.scalatestplus.junit.JUnitRunner
|
||||
import com.twitter.unified_user_actions.adapter.client_event.SearchInfoUtils
|
||||
import com.twitter.unified_user_actions.thriftscala.SearchQueryFilterType
|
||||
import com.twitter.unified_user_actions.thriftscala.SearchQueryFilterType._
|
||||
import org.scalatest.prop.TableFor2
|
||||
|
||||
@RunWith(classOf[JUnitRunner])
|
||||
class SearchInfoUtilsSpec
|
||||
extends AnyFunSuite
|
||||
with Matchers
|
||||
with Mockito
|
||||
with TableDrivenPropertyChecks {
|
||||
|
||||
trait Fixture {
|
||||
def mkControllerData(
|
||||
queryOpt: Option[String],
|
||||
querySourceOpt: Option[Int] = None,
|
||||
traceId: Option[Long] = None,
|
||||
requestJoinId: Option[Long] = None
|
||||
): ControllerData = {
|
||||
ControllerData.V2(
|
||||
ControllerDataV2.SearchResponse(
|
||||
SearchResponseControllerData.V1(
|
||||
SearchResponseControllerDataV1(requestControllerData = Some(
|
||||
RequestControllerData(
|
||||
rawQuery = queryOpt,
|
||||
querySource = querySourceOpt,
|
||||
traceId = traceId,
|
||||
requestJoinId = requestJoinId
|
||||
)))
|
||||
)))
|
||||
}
|
||||
|
||||
def mkTweetTypeControllerData(bitmap: Long, topicId: Option[Long] = None): ControllerData.V2 = {
|
||||
ControllerData.V2(
|
||||
ControllerDataV2.SearchResponse(
|
||||
SearchResponseControllerData.V1(
|
||||
SearchResponseControllerDataV1(itemTypesControllerData = Some(
|
||||
ItemTypesControllerData.TweetTypesControllerData(
|
||||
TweetTypesControllerData(
|
||||
tweetTypesBitmap = Some(bitmap),
|
||||
topicId = topicId
|
||||
))
|
||||
))
|
||||
)))
|
||||
}
|
||||
|
||||
def mkUserTypeControllerData(bitmap: Long): ControllerData.V2 = {
|
||||
ControllerData.V2(
|
||||
ControllerDataV2.SearchResponse(
|
||||
SearchResponseControllerData.V1(
|
||||
SearchResponseControllerDataV1(itemTypesControllerData = Some(
|
||||
ItemTypesControllerData.UserTypesControllerData(UserTypesControllerData(
|
||||
userTypesBitmap = Some(bitmap)
|
||||
))
|
||||
))
|
||||
)))
|
||||
}
|
||||
}
|
||||
|
||||
test("getQueryOptFromControllerDataFromItem should return query if present in controller data") {
|
||||
new Fixture {
|
||||
|
||||
val controllerData: ControllerData = mkControllerData(Some("twitter"))
|
||||
val suggestionDetails: SuggestionDetails =
|
||||
SuggestionDetails(decodedControllerData = Some(controllerData))
|
||||
val item: Item = Item(suggestionDetails = Some(suggestionDetails))
|
||||
val result: Option[String] = new SearchInfoUtils(item).getQueryOptFromControllerDataFromItem
|
||||
result shouldEqual Option("twitter")
|
||||
}
|
||||
}
|
||||
|
||||
test("getRequestJoinId should return requestJoinId if present in controller data") {
|
||||
new Fixture {
|
||||
|
||||
val controllerData: ControllerData = mkControllerData(
|
||||
Some("twitter"),
|
||||
traceId = Some(11L),
|
||||
requestJoinId = Some(12L)
|
||||
)
|
||||
val suggestionDetails: SuggestionDetails =
|
||||
SuggestionDetails(decodedControllerData = Some(controllerData))
|
||||
val item: Item = Item(suggestionDetails = Some(suggestionDetails))
|
||||
val infoUtils = new SearchInfoUtils(item)
|
||||
infoUtils.getTraceId shouldEqual Some(11L)
|
||||
infoUtils.getRequestJoinId shouldEqual Some(12L)
|
||||
}
|
||||
}
|
||||
|
||||
test("getQueryOptFromControllerDataFromItem should return None if no suggestion details") {
|
||||
new Fixture {
|
||||
|
||||
val suggestionDetails: SuggestionDetails = SuggestionDetails()
|
||||
val item: Item = Item(suggestionDetails = Some(suggestionDetails))
|
||||
val result: Option[String] = new SearchInfoUtils(item).getQueryOptFromControllerDataFromItem
|
||||
result shouldEqual None
|
||||
}
|
||||
}
|
||||
|
||||
test("getQueryOptFromSearchDetails should return query if present") {
|
||||
new Fixture {
|
||||
|
||||
val searchDetails: SearchDetails = SearchDetails(query = Some("twitter"))
|
||||
val result: Option[String] = new SearchInfoUtils(Item()).getQueryOptFromSearchDetails(
|
||||
LogEvent(eventName = "", searchDetails = Some(searchDetails))
|
||||
)
|
||||
result shouldEqual Option("twitter")
|
||||
}
|
||||
}
|
||||
|
||||
test("getQueryOptFromSearchDetails should return None if not present") {
|
||||
new Fixture {
|
||||
|
||||
val searchDetails: SearchDetails = SearchDetails()
|
||||
val result: Option[String] = new SearchInfoUtils(Item()).getQueryOptFromSearchDetails(
|
||||
LogEvent(eventName = "", searchDetails = Some(searchDetails))
|
||||
)
|
||||
result shouldEqual None
|
||||
}
|
||||
}
|
||||
|
||||
test("getQuerySourceOptFromControllerDataFromItem should return QuerySource if present") {
|
||||
new Fixture {
|
||||
|
||||
// 1 is Typed Query
|
||||
val controllerData: ControllerData = mkControllerData(Some("twitter"), Some(1))
|
||||
|
||||
val item: Item = Item(
|
||||
suggestionDetails = Some(
|
||||
SuggestionDetails(
|
||||
decodedControllerData = Some(controllerData)
|
||||
))
|
||||
)
|
||||
new SearchInfoUtils(item).getQuerySourceOptFromControllerDataFromItem shouldEqual Some(
|
||||
ThriftQuerySource.TypedQuery)
|
||||
}
|
||||
}
|
||||
|
||||
test("getQuerySourceOptFromControllerDataFromItem should return None if not present") {
|
||||
new Fixture {
|
||||
|
||||
val controllerData: ControllerData = mkControllerData(Some("twitter"), None)
|
||||
|
||||
val item: Item = Item(
|
||||
suggestionDetails = Some(
|
||||
SuggestionDetails(
|
||||
decodedControllerData = Some(controllerData)
|
||||
))
|
||||
)
|
||||
new SearchInfoUtils(item).getQuerySourceOptFromControllerDataFromItem shouldEqual None
|
||||
}
|
||||
}
|
||||
|
||||
test("Decoding Tweet Result Sources bitmap") {
|
||||
new Fixture {
|
||||
|
||||
TweetResultSource.list
|
||||
.foreach { tweetResultSource =>
|
||||
val bitmap = (1 << tweetResultSource.getValue()).toLong
|
||||
val controllerData = mkTweetTypeControllerData(bitmap)
|
||||
|
||||
val item = Item(
|
||||
suggestionDetails = Some(
|
||||
SuggestionDetails(
|
||||
decodedControllerData = Some(controllerData)
|
||||
))
|
||||
)
|
||||
|
||||
val result = new SearchInfoUtils(item).getTweetResultSources
|
||||
result shouldEqual Some(Set(tweetResultSource))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("Decoding multiple Tweet Result Sources") {
|
||||
new Fixture {
|
||||
|
||||
val tweetResultSources: Set[TweetResultSource] =
|
||||
Set(TweetResultSource.QueryInteractionGraph, TweetResultSource.QueryExpansion)
|
||||
val bitmap: Long = tweetResultSources.foldLeft(0L) {
|
||||
case (acc, source) => acc + (1 << source.getValue())
|
||||
}
|
||||
|
||||
val controllerData: ControllerData.V2 = mkTweetTypeControllerData(bitmap)
|
||||
|
||||
val item: Item = Item(
|
||||
suggestionDetails = Some(
|
||||
SuggestionDetails(
|
||||
decodedControllerData = Some(controllerData)
|
||||
))
|
||||
)
|
||||
|
||||
val result: Option[Set[TweetResultSource]] = new SearchInfoUtils(item).getTweetResultSources
|
||||
result shouldEqual Some(tweetResultSources)
|
||||
}
|
||||
}
|
||||
|
||||
test("Decoding User Result Sources bitmap") {
|
||||
new Fixture {
|
||||
|
||||
UserResultSource.list
|
||||
.foreach { userResultSource =>
|
||||
val bitmap = (1 << userResultSource.getValue()).toLong
|
||||
val controllerData = mkUserTypeControllerData(bitmap)
|
||||
|
||||
val item = Item(
|
||||
suggestionDetails = Some(
|
||||
SuggestionDetails(
|
||||
decodedControllerData = Some(controllerData)
|
||||
))
|
||||
)
|
||||
|
||||
val result = new SearchInfoUtils(item).getUserResultSources
|
||||
result shouldEqual Some(Set(userResultSource))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("Decoding multiple User Result Sources") {
|
||||
new Fixture {
|
||||
|
||||
val userResultSources: Set[UserResultSource] =
|
||||
Set(UserResultSource.QueryInteractionGraph, UserResultSource.ExpertSearch)
|
||||
val bitmap: Long = userResultSources.foldLeft(0L) {
|
||||
case (acc, source) => acc + (1 << source.getValue())
|
||||
}
|
||||
|
||||
val controllerData: ControllerData.V2 = mkUserTypeControllerData(bitmap)
|
||||
|
||||
val item: Item = Item(
|
||||
suggestionDetails = Some(
|
||||
SuggestionDetails(
|
||||
decodedControllerData = Some(controllerData)
|
||||
))
|
||||
)
|
||||
|
||||
val result: Option[Set[UserResultSource]] = new SearchInfoUtils(item).getUserResultSources
|
||||
result shouldEqual Some(userResultSources)
|
||||
}
|
||||
}
|
||||
|
||||
test("getQueryFilterTabType should return correct query filter type") {
|
||||
new Fixture {
|
||||
val infoUtils = new SearchInfoUtils(Item())
|
||||
val eventsToBeChecked: TableFor2[Option[EventNamespace], Option[SearchQueryFilterType]] =
|
||||
Table(
|
||||
("eventNamespace", "queryFilterType"),
|
||||
(
|
||||
Some(EventNamespace(client = Some("m5"), element = Some("search_filter_top"))),
|
||||
Some(Top)),
|
||||
(
|
||||
Some(EventNamespace(client = Some("m5"), element = Some("search_filter_live"))),
|
||||
Some(Latest)),
|
||||
(
|
||||
Some(EventNamespace(client = Some("m5"), element = Some("search_filter_user"))),
|
||||
Some(People)),
|
||||
(
|
||||
Some(EventNamespace(client = Some("m5"), element = Some("search_filter_image"))),
|
||||
Some(Photos)),
|
||||
(
|
||||
Some(EventNamespace(client = Some("m5"), element = Some("search_filter_video"))),
|
||||
Some(Videos)),
|
||||
(
|
||||
Some(EventNamespace(client = Some("m5"), section = Some("search_filter_top"))),
|
||||
None
|
||||
), // if client is web, element determines the query filter hence None if element is None
|
||||
(
|
||||
Some(EventNamespace(client = Some("android"), element = Some("search_filter_top"))),
|
||||
Some(Top)),
|
||||
(
|
||||
Some(EventNamespace(client = Some("android"), element = Some("search_filter_tweets"))),
|
||||
Some(Latest)),
|
||||
(
|
||||
Some(EventNamespace(client = Some("android"), element = Some("search_filter_user"))),
|
||||
Some(People)),
|
||||
(
|
||||
Some(EventNamespace(client = Some("android"), element = Some("search_filter_image"))),
|
||||
Some(Photos)),
|
||||
(
|
||||
Some(EventNamespace(client = Some("android"), element = Some("search_filter_video"))),
|
||||
Some(Videos)),
|
||||
(
|
||||
Some(EventNamespace(client = Some("m5"), section = Some("search_filter_top"))),
|
||||
None
|
||||
), // if client is android, element determines the query filter hence None if element is None
|
||||
(
|
||||
Some(EventNamespace(client = Some("iphone"), section = Some("search_filter_top"))),
|
||||
Some(Top)),
|
||||
(
|
||||
Some(EventNamespace(client = Some("iphone"), section = Some("search_filter_live"))),
|
||||
Some(Latest)),
|
||||
(
|
||||
Some(EventNamespace(client = Some("iphone"), section = Some("search_filter_user"))),
|
||||
Some(People)),
|
||||
(
|
||||
Some(EventNamespace(client = Some("iphone"), section = Some("search_filter_image"))),
|
||||
Some(Photos)),
|
||||
(
|
||||
Some(EventNamespace(client = Some("iphone"), section = Some("search_filter_video"))),
|
||||
Some(Videos)),
|
||||
(
|
||||
Some(EventNamespace(client = Some("iphone"), element = Some("search_filter_top"))),
|
||||
None
|
||||
), // if client is iphone, section determines the query filter hence None if section is None
|
||||
(
|
||||
Some(EventNamespace(client = None, section = Some("search_filter_top"))),
|
||||
Some(Top)
|
||||
), // if client is missing, use section by default
|
||||
(
|
||||
Some(EventNamespace(client = None, element = Some("search_filter_top"))),
|
||||
None
|
||||
), // if client is missing, section is used by default hence None since section is missing
|
||||
(
|
||||
Some(EventNamespace(client = Some("iphone"))),
|
||||
None
|
||||
), // if both element and section missing, expect None
|
||||
(None, None), // if namespace is missing from LogEvent, expect None
|
||||
)
|
||||
|
||||
forEvery(eventsToBeChecked) {
|
||||
(
|
||||
eventNamespace: Option[EventNamespace],
|
||||
searchQueryFilterType: Option[SearchQueryFilterType]
|
||||
) =>
|
||||
infoUtils.getQueryFilterType(
|
||||
LogEvent(
|
||||
eventName = "srp_event",
|
||||
eventNamespace = eventNamespace)) shouldEqual searchQueryFilterType
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,359 @@
|
|||
package com.twitter.unified_user_actions.adapter
|
||||
|
||||
import com.twitter.inject.Test
|
||||
import com.twitter.socialgraph.thriftscala.Action
|
||||
import com.twitter.socialgraph.thriftscala.BlockGraphEvent
|
||||
import com.twitter.socialgraph.thriftscala.FollowGraphEvent
|
||||
import com.twitter.socialgraph.thriftscala.FollowRequestGraphEvent
|
||||
import com.twitter.socialgraph.thriftscala.FollowRetweetsGraphEvent
|
||||
import com.twitter.socialgraph.thriftscala.LogEventContext
|
||||
import com.twitter.socialgraph.thriftscala.MuteGraphEvent
|
||||
import com.twitter.socialgraph.thriftscala.ReportAsAbuseGraphEvent
|
||||
import com.twitter.socialgraph.thriftscala.ReportAsSpamGraphEvent
|
||||
import com.twitter.socialgraph.thriftscala.SrcTargetRequest
|
||||
import com.twitter.socialgraph.thriftscala.WriteEvent
|
||||
import com.twitter.socialgraph.thriftscala.WriteRequestResult
|
||||
import com.twitter.unified_user_actions.adapter.social_graph_event.SocialGraphAdapter
|
||||
import com.twitter.unified_user_actions.thriftscala._
|
||||
import com.twitter.util.Time
|
||||
import org.scalatest.prop.TableDrivenPropertyChecks
|
||||
import org.scalatest.prop.TableFor1
|
||||
import org.scalatest.prop.TableFor3
|
||||
|
||||
class SocialGraphAdapterSpec extends Test with TableDrivenPropertyChecks {
|
||||
trait Fixture {
|
||||
|
||||
val frozenTime: Time = Time.fromMilliseconds(1658949273000L)
|
||||
|
||||
val testLogEventContext: LogEventContext = LogEventContext(
|
||||
timestamp = 1001L,
|
||||
hostname = "",
|
||||
transactionId = "",
|
||||
socialGraphClientId = "",
|
||||
loggedInUserId = Some(1111L),
|
||||
)
|
||||
|
||||
val testWriteRequestResult: WriteRequestResult = WriteRequestResult(
|
||||
request = SrcTargetRequest(
|
||||
source = 1111L,
|
||||
target = 2222L
|
||||
)
|
||||
)
|
||||
|
||||
val testWriteRequestResultWithValidationError: WriteRequestResult = WriteRequestResult(
|
||||
request = SrcTargetRequest(
|
||||
source = 1111L,
|
||||
target = 2222L
|
||||
),
|
||||
validationError = Some("action unsuccessful")
|
||||
)
|
||||
|
||||
val baseEvent: WriteEvent = WriteEvent(
|
||||
context = testLogEventContext,
|
||||
action = Action.AcceptFollowRequest
|
||||
)
|
||||
|
||||
val sgFollowEvent: WriteEvent = baseEvent.copy(
|
||||
action = Action.Follow,
|
||||
follow = Some(List(FollowGraphEvent(testWriteRequestResult))))
|
||||
|
||||
val sgUnfollowEvent: WriteEvent = baseEvent.copy(
|
||||
action = Action.Unfollow,
|
||||
follow = Some(List(FollowGraphEvent(testWriteRequestResult))))
|
||||
|
||||
val sgFollowRedundantEvent: WriteEvent = baseEvent.copy(
|
||||
action = Action.Follow,
|
||||
follow = Some(
|
||||
List(
|
||||
FollowGraphEvent(
|
||||
result = testWriteRequestResult,
|
||||
redundantOperation = Some(true)
|
||||
))))
|
||||
|
||||
val sgFollowRedundantIsFalseEvent: WriteEvent = baseEvent.copy(
|
||||
action = Action.Follow,
|
||||
follow = Some(
|
||||
List(
|
||||
FollowGraphEvent(
|
||||
result = testWriteRequestResult,
|
||||
redundantOperation = Some(false)
|
||||
))))
|
||||
|
||||
val sgUnfollowRedundantEvent: WriteEvent = baseEvent.copy(
|
||||
action = Action.Unfollow,
|
||||
follow = Some(
|
||||
List(
|
||||
FollowGraphEvent(
|
||||
result = testWriteRequestResult,
|
||||
redundantOperation = Some(true)
|
||||
))))
|
||||
|
||||
val sgUnfollowRedundantIsFalseEvent: WriteEvent = baseEvent.copy(
|
||||
action = Action.Unfollow,
|
||||
follow = Some(
|
||||
List(
|
||||
FollowGraphEvent(
|
||||
result = testWriteRequestResult,
|
||||
redundantOperation = Some(false)
|
||||
))))
|
||||
|
||||
val sgUnsuccessfulFollowEvent: WriteEvent = baseEvent.copy(
|
||||
action = Action.Follow,
|
||||
follow = Some(List(FollowGraphEvent(testWriteRequestResultWithValidationError))))
|
||||
|
||||
val sgUnsuccessfulUnfollowEvent: WriteEvent = baseEvent.copy(
|
||||
action = Action.Unfollow,
|
||||
follow = Some(List(FollowGraphEvent(testWriteRequestResultWithValidationError))))
|
||||
|
||||
val sgBlockEvent: WriteEvent = baseEvent.copy(
|
||||
action = Action.Block,
|
||||
block = Some(List(BlockGraphEvent(testWriteRequestResult))))
|
||||
|
||||
val sgUnsuccessfulBlockEvent: WriteEvent = baseEvent.copy(
|
||||
action = Action.Block,
|
||||
block = Some(List(BlockGraphEvent(testWriteRequestResultWithValidationError))))
|
||||
|
||||
val sgUnblockEvent: WriteEvent = baseEvent.copy(
|
||||
action = Action.Unblock,
|
||||
block = Some(List(BlockGraphEvent(testWriteRequestResult))))
|
||||
|
||||
val sgUnsuccessfulUnblockEvent: WriteEvent = baseEvent.copy(
|
||||
action = Action.Unblock,
|
||||
block = Some(List(BlockGraphEvent(testWriteRequestResultWithValidationError))))
|
||||
|
||||
val sgMuteEvent: WriteEvent = baseEvent.copy(
|
||||
action = Action.Mute,
|
||||
mute = Some(List(MuteGraphEvent(testWriteRequestResult))))
|
||||
|
||||
val sgUnsuccessfulMuteEvent: WriteEvent = baseEvent.copy(
|
||||
action = Action.Mute,
|
||||
mute = Some(List(MuteGraphEvent(testWriteRequestResultWithValidationError))))
|
||||
|
||||
val sgUnmuteEvent: WriteEvent = baseEvent.copy(
|
||||
action = Action.Unmute,
|
||||
mute = Some(List(MuteGraphEvent(testWriteRequestResult))))
|
||||
|
||||
val sgUnsuccessfulUnmuteEvent: WriteEvent = baseEvent.copy(
|
||||
action = Action.Unmute,
|
||||
mute = Some(List(MuteGraphEvent(testWriteRequestResultWithValidationError))))
|
||||
|
||||
val sgCreateFollowRequestEvent: WriteEvent = baseEvent.copy(
|
||||
action = Action.CreateFollowRequest,
|
||||
followRequest = Some(List(FollowRequestGraphEvent(testWriteRequestResult)))
|
||||
)
|
||||
|
||||
val sgCancelFollowRequestEvent: WriteEvent = baseEvent.copy(
|
||||
action = Action.CancelFollowRequest,
|
||||
followRequest = Some(List(FollowRequestGraphEvent(testWriteRequestResult)))
|
||||
)
|
||||
|
||||
val sgAcceptFollowRequestEvent: WriteEvent = baseEvent.copy(
|
||||
action = Action.AcceptFollowRequest,
|
||||
followRequest = Some(List(FollowRequestGraphEvent(testWriteRequestResult)))
|
||||
)
|
||||
|
||||
val sgAcceptFollowRetweetEvent: WriteEvent = baseEvent.copy(
|
||||
action = Action.FollowRetweets,
|
||||
followRetweets = Some(List(FollowRetweetsGraphEvent(testWriteRequestResult)))
|
||||
)
|
||||
|
||||
val sgAcceptUnfollowRetweetEvent: WriteEvent = baseEvent.copy(
|
||||
action = Action.UnfollowRetweets,
|
||||
followRetweets = Some(List(FollowRetweetsGraphEvent(testWriteRequestResult)))
|
||||
)
|
||||
|
||||
val sgReportAsSpamEvent: WriteEvent = baseEvent.copy(
|
||||
action = Action.ReportAsSpam,
|
||||
reportAsSpam = Some(
|
||||
List(
|
||||
ReportAsSpamGraphEvent(
|
||||
result = testWriteRequestResult
|
||||
))))
|
||||
|
||||
val sgReportAsAbuseEvent: WriteEvent = baseEvent.copy(
|
||||
action = Action.ReportAsAbuse,
|
||||
reportAsAbuse = Some(
|
||||
List(
|
||||
ReportAsAbuseGraphEvent(
|
||||
result = testWriteRequestResult
|
||||
))))
|
||||
|
||||
def getExpectedUUA(
|
||||
userId: Long,
|
||||
actionProfileId: Long,
|
||||
sourceTimestampMs: Long,
|
||||
actionType: ActionType,
|
||||
socialGraphAction: Option[Action] = None
|
||||
): UnifiedUserAction = {
|
||||
val actionItem = socialGraphAction match {
|
||||
case Some(sgAction) =>
|
||||
Item.ProfileInfo(
|
||||
ProfileInfo(
|
||||
actionProfileId = actionProfileId,
|
||||
profileActionInfo = Some(
|
||||
ProfileActionInfo.ServerProfileReport(
|
||||
ServerProfileReport(reportType = sgAction)
|
||||
))
|
||||
)
|
||||
)
|
||||
case _ =>
|
||||
Item.ProfileInfo(
|
||||
ProfileInfo(
|
||||
actionProfileId = actionProfileId
|
||||
)
|
||||
)
|
||||
}
|
||||
|
||||
UnifiedUserAction(
|
||||
userIdentifier = UserIdentifier(userId = Some(userId)),
|
||||
item = actionItem,
|
||||
actionType = actionType,
|
||||
eventMetadata = EventMetadata(
|
||||
sourceTimestampMs = sourceTimestampMs,
|
||||
receivedTimestampMs = frozenTime.inMilliseconds,
|
||||
sourceLineage = SourceLineage.ServerSocialGraphEvents
|
||||
)
|
||||
)
|
||||
}
|
||||
|
||||
val expectedUuaFollow: UnifiedUserAction = getExpectedUUA(
|
||||
userId = 1111L,
|
||||
actionProfileId = 2222L,
|
||||
sourceTimestampMs = 1001L,
|
||||
actionType = ActionType.ServerProfileFollow
|
||||
)
|
||||
|
||||
val expectedUuaUnfollow: UnifiedUserAction = getExpectedUUA(
|
||||
userId = 1111L,
|
||||
actionProfileId = 2222L,
|
||||
sourceTimestampMs = 1001L,
|
||||
actionType = ActionType.ServerProfileUnfollow
|
||||
)
|
||||
|
||||
val expectedUuaMute: UnifiedUserAction = getExpectedUUA(
|
||||
userId = 1111L,
|
||||
actionProfileId = 2222L,
|
||||
sourceTimestampMs = 1001L,
|
||||
actionType = ActionType.ServerProfileMute
|
||||
)
|
||||
|
||||
val expectedUuaUnmute: UnifiedUserAction = getExpectedUUA(
|
||||
userId = 1111L,
|
||||
actionProfileId = 2222L,
|
||||
sourceTimestampMs = 1001L,
|
||||
actionType = ActionType.ServerProfileUnmute
|
||||
)
|
||||
|
||||
val expectedUuaBlock: UnifiedUserAction = getExpectedUUA(
|
||||
userId = 1111L,
|
||||
actionProfileId = 2222L,
|
||||
sourceTimestampMs = 1001L,
|
||||
actionType = ActionType.ServerProfileBlock
|
||||
)
|
||||
|
||||
val expectedUuaUnblock: UnifiedUserAction = getExpectedUUA(
|
||||
userId = 1111L,
|
||||
actionProfileId = 2222L,
|
||||
sourceTimestampMs = 1001L,
|
||||
actionType = ActionType.ServerProfileUnblock
|
||||
)
|
||||
|
||||
val expectedUuaReportAsSpam: UnifiedUserAction = getExpectedUUA(
|
||||
userId = 1111L,
|
||||
actionProfileId = 2222L,
|
||||
sourceTimestampMs = 1001L,
|
||||
actionType = ActionType.ServerProfileReport,
|
||||
socialGraphAction = Some(Action.ReportAsSpam)
|
||||
)
|
||||
|
||||
val expectedUuaReportAsAbuse: UnifiedUserAction = getExpectedUUA(
|
||||
userId = 1111L,
|
||||
actionProfileId = 2222L,
|
||||
sourceTimestampMs = 1001L,
|
||||
actionType = ActionType.ServerProfileReport,
|
||||
socialGraphAction = Some(Action.ReportAsAbuse)
|
||||
)
|
||||
}
|
||||
|
||||
test("SocialGraphAdapter ignore events not in the list") {
|
||||
new Fixture {
|
||||
Time.withTimeAt(frozenTime) { _ =>
|
||||
val ignoredSocialGraphEvents: TableFor1[WriteEvent] = Table(
|
||||
"ignoredSocialGraphEvents",
|
||||
sgAcceptUnfollowRetweetEvent,
|
||||
sgAcceptFollowRequestEvent,
|
||||
sgAcceptFollowRetweetEvent,
|
||||
sgCreateFollowRequestEvent,
|
||||
sgCancelFollowRequestEvent,
|
||||
)
|
||||
forEvery(ignoredSocialGraphEvents) { writeEvent: WriteEvent =>
|
||||
val actual = SocialGraphAdapter.adaptEvent(writeEvent)
|
||||
assert(actual.isEmpty)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("Test SocialGraphAdapter consuming Write events") {
|
||||
new Fixture {
|
||||
Time.withTimeAt(frozenTime) { _ =>
|
||||
val socialProfileActions: TableFor3[String, WriteEvent, UnifiedUserAction] = Table(
|
||||
("actionType", "event", "expectedUnifiedUserAction"),
|
||||
("ProfileFollow", sgFollowEvent, expectedUuaFollow),
|
||||
("ProfileUnfollow", sgUnfollowEvent, expectedUuaUnfollow),
|
||||
("ProfileBlock", sgBlockEvent, expectedUuaBlock),
|
||||
("ProfileUnBlock", sgUnblockEvent, expectedUuaUnblock),
|
||||
("ProfileMute", sgMuteEvent, expectedUuaMute),
|
||||
("ProfileUnmute", sgUnmuteEvent, expectedUuaUnmute),
|
||||
("ProfileReportAsSpam", sgReportAsSpamEvent, expectedUuaReportAsSpam),
|
||||
("ProfileReportAsAbuse", sgReportAsAbuseEvent, expectedUuaReportAsAbuse),
|
||||
)
|
||||
forEvery(socialProfileActions) {
|
||||
(_: String, event: WriteEvent, expected: UnifiedUserAction) =>
|
||||
val actual = SocialGraphAdapter.adaptEvent(event)
|
||||
assert(Seq(expected) === actual)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("SocialGraphAdapter ignore redundant follow/unfollow events") {
|
||||
new Fixture {
|
||||
Time.withTimeAt(frozenTime) { _ =>
|
||||
val socialGraphActions: TableFor3[String, WriteEvent, Seq[UnifiedUserAction]] = Table(
|
||||
("actionType", "ignoredRedundantFollowUnfollowEvents", "expectedUnifiedUserAction"),
|
||||
("ProfileFollow", sgFollowRedundantEvent, Nil),
|
||||
("ProfileFollow", sgFollowRedundantIsFalseEvent, Seq(expectedUuaFollow)),
|
||||
("ProfileUnfollow", sgUnfollowRedundantEvent, Nil),
|
||||
("ProfileUnfollow", sgUnfollowRedundantIsFalseEvent, Seq(expectedUuaUnfollow))
|
||||
)
|
||||
forEvery(socialGraphActions) {
|
||||
(_: String, event: WriteEvent, expected: Seq[UnifiedUserAction]) =>
|
||||
val actual = SocialGraphAdapter.adaptEvent(event)
|
||||
assert(expected === actual)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("SocialGraphAdapter ignore Unsuccessful SocialGraph events") {
|
||||
new Fixture {
|
||||
Time.withTimeAt(frozenTime) { _ =>
|
||||
val unsuccessfulSocialGraphEvents: TableFor1[WriteEvent] = Table(
|
||||
"ignoredSocialGraphEvents",
|
||||
sgUnsuccessfulFollowEvent,
|
||||
sgUnsuccessfulUnfollowEvent,
|
||||
sgUnsuccessfulBlockEvent,
|
||||
sgUnsuccessfulUnblockEvent,
|
||||
sgUnsuccessfulMuteEvent,
|
||||
sgUnsuccessfulUnmuteEvent
|
||||
)
|
||||
|
||||
forEvery(unsuccessfulSocialGraphEvents) { writeEvent: WriteEvent =>
|
||||
val actual = SocialGraphAdapter.adaptEvent(writeEvent)
|
||||
assert(actual.isEmpty)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load diff
|
@ -0,0 +1,205 @@
|
|||
package com.twitter.unified_user_actions.adapter
|
||||
|
||||
import com.twitter.context.thriftscala.Viewer
|
||||
import com.twitter.inject.Test
|
||||
import com.twitter.timelineservice.thriftscala._
|
||||
import com.twitter.unified_user_actions.adapter.tls_favs_event.TlsFavsAdapter
|
||||
import com.twitter.unified_user_actions.thriftscala._
|
||||
import com.twitter.util.Time
|
||||
|
||||
class TlsFavsAdapterSpec extends Test {
|
||||
trait Fixture {
|
||||
|
||||
val frozenTime = Time.fromMilliseconds(1658949273000L)
|
||||
|
||||
val favEventNoRetweet = ContextualizedFavoriteEvent(
|
||||
event = FavoriteEventUnion.Favorite(
|
||||
FavoriteEvent(
|
||||
userId = 91L,
|
||||
tweetId = 1L,
|
||||
tweetUserId = 101L,
|
||||
eventTimeMs = 1001L
|
||||
)
|
||||
),
|
||||
context = LogEventContext(hostname = "", traceId = 31L)
|
||||
)
|
||||
val favEventRetweet = ContextualizedFavoriteEvent(
|
||||
event = FavoriteEventUnion.Favorite(
|
||||
FavoriteEvent(
|
||||
userId = 92L,
|
||||
tweetId = 2L,
|
||||
tweetUserId = 102L,
|
||||
eventTimeMs = 1002L,
|
||||
retweetId = Some(22L)
|
||||
)
|
||||
),
|
||||
context = LogEventContext(hostname = "", traceId = 32L)
|
||||
)
|
||||
val unfavEventNoRetweet = ContextualizedFavoriteEvent(
|
||||
event = FavoriteEventUnion.Unfavorite(
|
||||
UnfavoriteEvent(
|
||||
userId = 93L,
|
||||
tweetId = 3L,
|
||||
tweetUserId = 103L,
|
||||
eventTimeMs = 1003L
|
||||
)
|
||||
),
|
||||
context = LogEventContext(hostname = "", traceId = 33L)
|
||||
)
|
||||
val unfavEventRetweet = ContextualizedFavoriteEvent(
|
||||
event = FavoriteEventUnion.Unfavorite(
|
||||
UnfavoriteEvent(
|
||||
userId = 94L,
|
||||
tweetId = 4L,
|
||||
tweetUserId = 104L,
|
||||
eventTimeMs = 1004L,
|
||||
retweetId = Some(44L)
|
||||
)
|
||||
),
|
||||
context = LogEventContext(hostname = "", traceId = 34L)
|
||||
)
|
||||
val favEventWithLangAndCountry = ContextualizedFavoriteEvent(
|
||||
event = FavoriteEventUnion.Favorite(
|
||||
FavoriteEvent(
|
||||
userId = 91L,
|
||||
tweetId = 1L,
|
||||
tweetUserId = 101L,
|
||||
eventTimeMs = 1001L,
|
||||
viewerContext =
|
||||
Some(Viewer(requestCountryCode = Some("us"), requestLanguageCode = Some("en")))
|
||||
)
|
||||
),
|
||||
context = LogEventContext(hostname = "", traceId = 31L)
|
||||
)
|
||||
|
||||
val expectedUua1 = UnifiedUserAction(
|
||||
userIdentifier = UserIdentifier(userId = Some(91L)),
|
||||
item = Item.TweetInfo(
|
||||
TweetInfo(
|
||||
actionTweetId = 1L,
|
||||
actionTweetAuthorInfo = Some(AuthorInfo(authorId = Some(101L))),
|
||||
)
|
||||
),
|
||||
actionType = ActionType.ServerTweetFav,
|
||||
eventMetadata = EventMetadata(
|
||||
sourceTimestampMs = 1001L,
|
||||
receivedTimestampMs = frozenTime.inMilliseconds,
|
||||
sourceLineage = SourceLineage.ServerTlsFavs,
|
||||
traceId = Some(31L)
|
||||
)
|
||||
)
|
||||
val expectedUua2 = UnifiedUserAction(
|
||||
userIdentifier = UserIdentifier(userId = Some(92L)),
|
||||
item = Item.TweetInfo(
|
||||
TweetInfo(
|
||||
actionTweetId = 2L,
|
||||
actionTweetAuthorInfo = Some(AuthorInfo(authorId = Some(102L))),
|
||||
retweetingTweetId = Some(22L)
|
||||
)
|
||||
),
|
||||
actionType = ActionType.ServerTweetFav,
|
||||
eventMetadata = EventMetadata(
|
||||
sourceTimestampMs = 1002L,
|
||||
receivedTimestampMs = frozenTime.inMilliseconds,
|
||||
sourceLineage = SourceLineage.ServerTlsFavs,
|
||||
traceId = Some(32L)
|
||||
)
|
||||
)
|
||||
val expectedUua3 = UnifiedUserAction(
|
||||
userIdentifier = UserIdentifier(userId = Some(93L)),
|
||||
item = Item.TweetInfo(
|
||||
TweetInfo(
|
||||
actionTweetId = 3L,
|
||||
actionTweetAuthorInfo = Some(AuthorInfo(authorId = Some(103L))),
|
||||
)
|
||||
),
|
||||
actionType = ActionType.ServerTweetUnfav,
|
||||
eventMetadata = EventMetadata(
|
||||
sourceTimestampMs = 1003L,
|
||||
receivedTimestampMs = frozenTime.inMilliseconds,
|
||||
sourceLineage = SourceLineage.ServerTlsFavs,
|
||||
traceId = Some(33L)
|
||||
)
|
||||
)
|
||||
val expectedUua4 = UnifiedUserAction(
|
||||
userIdentifier = UserIdentifier(userId = Some(94L)),
|
||||
item = Item.TweetInfo(
|
||||
TweetInfo(
|
||||
actionTweetId = 4L,
|
||||
actionTweetAuthorInfo = Some(AuthorInfo(authorId = Some(104L))),
|
||||
retweetingTweetId = Some(44L)
|
||||
)
|
||||
),
|
||||
actionType = ActionType.ServerTweetUnfav,
|
||||
eventMetadata = EventMetadata(
|
||||
sourceTimestampMs = 1004L,
|
||||
receivedTimestampMs = frozenTime.inMilliseconds,
|
||||
sourceLineage = SourceLineage.ServerTlsFavs,
|
||||
traceId = Some(34L)
|
||||
)
|
||||
)
|
||||
val expectedUua5 = UnifiedUserAction(
|
||||
userIdentifier = UserIdentifier(userId = Some(91L)),
|
||||
item = Item.TweetInfo(
|
||||
TweetInfo(
|
||||
actionTweetId = 1L,
|
||||
actionTweetAuthorInfo = Some(AuthorInfo(authorId = Some(101L))),
|
||||
)
|
||||
),
|
||||
actionType = ActionType.ServerTweetFav,
|
||||
eventMetadata = EventMetadata(
|
||||
sourceTimestampMs = 1001L,
|
||||
receivedTimestampMs = frozenTime.inMilliseconds,
|
||||
sourceLineage = SourceLineage.ServerTlsFavs,
|
||||
language = Some("EN"),
|
||||
countryCode = Some("US"),
|
||||
traceId = Some(31L)
|
||||
)
|
||||
)
|
||||
}
|
||||
|
||||
test("fav event with no retweet") {
|
||||
new Fixture {
|
||||
Time.withTimeAt(frozenTime) { _ =>
|
||||
val actual = TlsFavsAdapter.adaptEvent(favEventNoRetweet)
|
||||
assert(Seq(expectedUua1) === actual)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("fav event with a retweet") {
|
||||
new Fixture {
|
||||
Time.withTimeAt(frozenTime) { _ =>
|
||||
val actual = TlsFavsAdapter.adaptEvent(favEventRetweet)
|
||||
assert(Seq(expectedUua2) === actual)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("unfav event with no retweet") {
|
||||
new Fixture {
|
||||
Time.withTimeAt(frozenTime) { _ =>
|
||||
val actual = TlsFavsAdapter.adaptEvent(unfavEventNoRetweet)
|
||||
assert(Seq(expectedUua3) === actual)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("unfav event with a retweet") {
|
||||
new Fixture {
|
||||
Time.withTimeAt(frozenTime) { _ =>
|
||||
val actual = TlsFavsAdapter.adaptEvent(unfavEventRetweet)
|
||||
assert(Seq(expectedUua4) === actual)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("fav event with language and country") {
|
||||
new Fixture {
|
||||
Time.withTimeAt(frozenTime) { _ =>
|
||||
val actual = TlsFavsAdapter.adaptEvent(favEventWithLangAndCountry)
|
||||
assert(Seq(expectedUua5) === actual)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,545 @@
|
|||
package com.twitter.unified_user_actions.adapter
|
||||
|
||||
import com.twitter.clientapp.thriftscala._
|
||||
import com.twitter.clientapp.thriftscala.SuggestionDetails
|
||||
import com.twitter.guide.scribing.thriftscala._
|
||||
import com.twitter.guide.scribing.thriftscala.{SemanticCoreInterest => SemanticCoreInterestV1}
|
||||
import com.twitter.guide.scribing.thriftscala.{SimClusterInterest => SimClusterInterestV1}
|
||||
import com.twitter.guide.scribing.thriftscala.TopicModuleMetadata.SemanticCoreInterest
|
||||
import com.twitter.guide.scribing.thriftscala.TopicModuleMetadata.SimClusterInterest
|
||||
import com.twitter.guide.scribing.thriftscala.TransparentGuideDetails.TopicMetadata
|
||||
import com.twitter.logbase.thriftscala.LogBase
|
||||
import com.twitter.scrooge.TFieldBlob
|
||||
import com.twitter.suggests.controller_data.home_hitl_topic_annotation_prompt.thriftscala.HomeHitlTopicAnnotationPromptControllerData
|
||||
import com.twitter.suggests.controller_data.home_hitl_topic_annotation_prompt.v1.thriftscala.{
|
||||
HomeHitlTopicAnnotationPromptControllerData => HomeHitlTopicAnnotationPromptControllerDataV1
|
||||
}
|
||||
import com.twitter.suggests.controller_data.home_topic_annotation_prompt.thriftscala.HomeTopicAnnotationPromptControllerData
|
||||
import com.twitter.suggests.controller_data.home_topic_annotation_prompt.v1.thriftscala.{
|
||||
HomeTopicAnnotationPromptControllerData => HomeTopicAnnotationPromptControllerDataV1
|
||||
}
|
||||
import com.twitter.suggests.controller_data.home_topic_follow_prompt.thriftscala.HomeTopicFollowPromptControllerData
|
||||
import com.twitter.suggests.controller_data.home_topic_follow_prompt.v1.thriftscala.{
|
||||
HomeTopicFollowPromptControllerData => HomeTopicFollowPromptControllerDataV1
|
||||
}
|
||||
import com.twitter.suggests.controller_data.home_tweets.thriftscala.HomeTweetsControllerData
|
||||
import com.twitter.suggests.controller_data.home_tweets.v1.thriftscala.{
|
||||
HomeTweetsControllerData => HomeTweetsControllerDataV1
|
||||
}
|
||||
import com.twitter.suggests.controller_data.search_response.item_types.thriftscala.ItemTypesControllerData
|
||||
import com.twitter.suggests.controller_data.search_response.thriftscala.SearchResponseControllerData
|
||||
import com.twitter.suggests.controller_data.search_response.topic_follow_prompt.thriftscala.SearchTopicFollowPromptControllerData
|
||||
import com.twitter.suggests.controller_data.search_response.tweet_types.thriftscala.TweetTypesControllerData
|
||||
import com.twitter.suggests.controller_data.search_response.v1.thriftscala.{
|
||||
SearchResponseControllerData => SearchResponseControllerDataV1
|
||||
}
|
||||
import com.twitter.suggests.controller_data.thriftscala.ControllerData
|
||||
import com.twitter.suggests.controller_data.timelines_topic.thriftscala.TimelinesTopicControllerData
|
||||
import com.twitter.suggests.controller_data.timelines_topic.v1.thriftscala.{
|
||||
TimelinesTopicControllerData => TimelinesTopicControllerDataV1
|
||||
}
|
||||
import com.twitter.suggests.controller_data.v2.thriftscala.{ControllerData => ControllerDataV2}
|
||||
import org.apache.thrift.protocol.TField
|
||||
import org.junit.runner.RunWith
|
||||
import org.scalatest.funsuite.AnyFunSuite
|
||||
import org.scalatest.matchers.should.Matchers
|
||||
import org.scalatestplus.junit.JUnitRunner
|
||||
import com.twitter.util.mock.Mockito
|
||||
import org.mockito.Mockito.when
|
||||
import org.scalatest.prop.TableDrivenPropertyChecks
|
||||
|
||||
@RunWith(classOf[JUnitRunner])
|
||||
class TopicsIdUtilsSpec
|
||||
extends AnyFunSuite
|
||||
with Matchers
|
||||
with Mockito
|
||||
with TableDrivenPropertyChecks {
|
||||
import com.twitter.unified_user_actions.adapter.client_event.TopicIdUtils._
|
||||
|
||||
trait Fixture {
|
||||
def buildLogBase(userId: Long): LogBase = {
|
||||
val logBase = mock[LogBase]
|
||||
when(logBase.country).thenReturn(Some("US"))
|
||||
when(logBase.userId).thenReturn(Some(userId))
|
||||
when(logBase.timestamp).thenReturn(100L)
|
||||
when(logBase.guestId).thenReturn(Some(1L))
|
||||
when(logBase.userAgent).thenReturn(None)
|
||||
when(logBase.language).thenReturn(Some("en"))
|
||||
logBase
|
||||
}
|
||||
|
||||
def buildItemForTimeline(
|
||||
itemId: Long,
|
||||
itemType: ItemType,
|
||||
topicId: Long,
|
||||
fn: Long => ControllerData.V2
|
||||
): Item = {
|
||||
val item = Item(
|
||||
id = Some(itemId),
|
||||
itemType = Some(itemType),
|
||||
suggestionDetails = Some(SuggestionDetails(decodedControllerData = Some(fn(topicId))))
|
||||
)
|
||||
item
|
||||
}
|
||||
|
||||
def buildClientEventForHomeSearchTimeline(
|
||||
itemId: Long,
|
||||
itemType: ItemType,
|
||||
topicId: Long,
|
||||
fn: Long => ControllerData.V2,
|
||||
userId: Long = 1L,
|
||||
eventNamespaceOpt: Option[EventNamespace] = None,
|
||||
): LogEvent = {
|
||||
val logEvent = mock[LogEvent]
|
||||
when(logEvent.eventNamespace).thenReturn(eventNamespaceOpt)
|
||||
val eventsDetails = mock[EventDetails]
|
||||
when(eventsDetails.items)
|
||||
.thenReturn(Some(Seq(buildItemForTimeline(itemId, itemType, topicId, fn))))
|
||||
val logbase = buildLogBase(userId)
|
||||
when(logEvent.logBase).thenReturn(Some(logbase))
|
||||
when(logEvent.eventDetails).thenReturn(Some(eventsDetails))
|
||||
logEvent
|
||||
}
|
||||
|
||||
def buildClientEventForHomeTweetsTimeline(
|
||||
itemId: Long,
|
||||
itemType: ItemType,
|
||||
topicId: Long,
|
||||
topicIds: Set[Long],
|
||||
fn: (Long, Set[Long]) => ControllerData.V2,
|
||||
userId: Long = 1L,
|
||||
eventNamespaceOpt: Option[EventNamespace] = None,
|
||||
): LogEvent = {
|
||||
val logEvent = mock[LogEvent]
|
||||
when(logEvent.eventNamespace).thenReturn(eventNamespaceOpt)
|
||||
val eventsDetails = mock[EventDetails]
|
||||
when(eventsDetails.items)
|
||||
.thenReturn(Some(Seq(buildItemForHomeTimeline(itemId, itemType, topicId, topicIds, fn))))
|
||||
val logbase = buildLogBase(userId)
|
||||
when(logEvent.logBase).thenReturn(Some(logbase))
|
||||
when(logEvent.eventDetails).thenReturn(Some(eventsDetails))
|
||||
logEvent
|
||||
}
|
||||
|
||||
def buildClientEventForGuide(
|
||||
itemId: Long,
|
||||
itemType: ItemType,
|
||||
topicId: Long,
|
||||
fn: Long => TopicMetadata,
|
||||
userId: Long = 1L,
|
||||
eventNamespaceOpt: Option[EventNamespace] = None,
|
||||
): LogEvent = {
|
||||
val logEvent = mock[LogEvent]
|
||||
when(logEvent.eventNamespace).thenReturn(eventNamespaceOpt)
|
||||
val logbase = buildLogBase(userId)
|
||||
when(logEvent.logBase).thenReturn(Some(logbase))
|
||||
val eventDetails = mock[EventDetails]
|
||||
val item = buildItemForGuide(itemId, itemType, topicId, fn)
|
||||
when(eventDetails.items).thenReturn(Some(Seq(item)))
|
||||
when(logEvent.eventDetails).thenReturn(Some(eventDetails))
|
||||
logEvent
|
||||
}
|
||||
|
||||
def buildClientEventForOnboarding(
|
||||
itemId: Long,
|
||||
topicId: Long,
|
||||
userId: Long = 1L
|
||||
): LogEvent = {
|
||||
val logEvent = mock[LogEvent]
|
||||
val logbase = buildLogBase(userId)
|
||||
when(logEvent.logBase).thenReturn(Some(logbase))
|
||||
when(logEvent.eventNamespace).thenReturn(Some(buildNamespaceForOnboarding))
|
||||
val eventDetails = mock[EventDetails]
|
||||
val item = buildItemForOnboarding(itemId, topicId)
|
||||
when(eventDetails.items)
|
||||
.thenReturn(Some(Seq(item)))
|
||||
when(logEvent.eventDetails).thenReturn(Some(eventDetails))
|
||||
logEvent
|
||||
}
|
||||
|
||||
def buildClientEventForOnboardingBackend(
|
||||
topicId: Long,
|
||||
userId: Long = 1L
|
||||
): LogEvent = {
|
||||
val logEvent = mock[LogEvent]
|
||||
val logbase = buildLogBase(userId)
|
||||
when(logEvent.logBase).thenReturn(Some(logbase))
|
||||
when(logEvent.eventNamespace).thenReturn(Some(buildNamespaceForOnboardingBackend))
|
||||
val eventDetails = buildEventDetailsForOnboardingBackend(topicId)
|
||||
when(logEvent.eventDetails).thenReturn(Some(eventDetails))
|
||||
logEvent
|
||||
}
|
||||
|
||||
def defaultNamespace: EventNamespace = {
|
||||
EventNamespace(Some("iphone"), None, None, None, None, Some("favorite"))
|
||||
}
|
||||
|
||||
def buildNamespaceForOnboardingBackend: EventNamespace = {
|
||||
EventNamespace(
|
||||
Some("iphone"),
|
||||
Some("onboarding_backend"),
|
||||
Some("subtasks"),
|
||||
Some("topics_selector"),
|
||||
Some("removed"),
|
||||
Some("selected"))
|
||||
}
|
||||
|
||||
def buildNamespaceForOnboarding: EventNamespace = {
|
||||
EventNamespace(
|
||||
Some("iphone"),
|
||||
Some("onboarding"),
|
||||
Some("topics_selector"),
|
||||
None,
|
||||
Some("topic"),
|
||||
Some("follow")
|
||||
)
|
||||
}
|
||||
|
||||
def buildItemForHomeTimeline(
|
||||
itemId: Long,
|
||||
itemType: ItemType,
|
||||
topicId: Long,
|
||||
topicIds: Set[Long],
|
||||
fn: (Long, Set[Long]) => ControllerData.V2
|
||||
): Item = {
|
||||
val item = Item(
|
||||
id = Some(itemId),
|
||||
itemType = Some(itemType),
|
||||
suggestionDetails =
|
||||
Some(SuggestionDetails(decodedControllerData = Some(fn(topicId, topicIds))))
|
||||
)
|
||||
item
|
||||
}
|
||||
|
||||
def buildItemForGuide(
|
||||
itemId: Long,
|
||||
itemType: ItemType,
|
||||
topicId: Long,
|
||||
fn: Long => TopicMetadata
|
||||
): Item = {
|
||||
val item = mock[Item]
|
||||
when(item.id).thenReturn(Some(itemId))
|
||||
when(item.itemType).thenReturn(Some(itemType))
|
||||
when(item.suggestionDetails)
|
||||
.thenReturn(Some(SuggestionDetails(suggestionType = Some("ErgTweet"))))
|
||||
val guideItemDetails = mock[GuideItemDetails]
|
||||
when(guideItemDetails.transparentGuideDetails).thenReturn(Some(fn(topicId)))
|
||||
when(item.guideItemDetails).thenReturn(Some(guideItemDetails))
|
||||
item
|
||||
}
|
||||
|
||||
def buildItemForOnboarding(
|
||||
itemId: Long,
|
||||
topicId: Long
|
||||
): Item = {
|
||||
val item = Item(
|
||||
id = Some(itemId),
|
||||
itemType = None,
|
||||
description = Some(s"id=$topicId,row=1")
|
||||
)
|
||||
item
|
||||
}
|
||||
|
||||
def buildEventDetailsForOnboardingBackend(
|
||||
topicId: Long
|
||||
): EventDetails = {
|
||||
val eventDetails = mock[EventDetails]
|
||||
val item = Item(
|
||||
id = Some(topicId)
|
||||
)
|
||||
val itemTmp = buildItemForOnboarding(10, topicId)
|
||||
when(eventDetails.items).thenReturn(Some(Seq(itemTmp)))
|
||||
when(eventDetails.targets).thenReturn(Some(Seq(item)))
|
||||
eventDetails
|
||||
}
|
||||
|
||||
def topicMetadataInGuide(topicId: Long): TopicMetadata =
|
||||
TopicMetadata(
|
||||
SemanticCoreInterest(
|
||||
SemanticCoreInterestV1(domainId = "131", entityId = topicId.toString)
|
||||
)
|
||||
)
|
||||
|
||||
def simClusterMetadataInGuide(simclusterId: Long = 1L): TopicMetadata =
|
||||
TopicMetadata(
|
||||
SimClusterInterest(
|
||||
SimClusterInterestV1(simclusterId.toString)
|
||||
)
|
||||
)
|
||||
|
||||
def timelineTopicControllerData(topicId: Long): ControllerData.V2 =
|
||||
ControllerData.V2(
|
||||
ControllerDataV2.TimelinesTopic(
|
||||
TimelinesTopicControllerData.V1(
|
||||
TimelinesTopicControllerDataV1(
|
||||
topicId = topicId,
|
||||
topicTypesBitmap = 1
|
||||
)
|
||||
)))
|
||||
|
||||
def homeTweetControllerData(topicId: Long): ControllerData.V2 =
|
||||
ControllerData.V2(
|
||||
ControllerDataV2.HomeTweets(
|
||||
HomeTweetsControllerData.V1(
|
||||
HomeTweetsControllerDataV1(
|
||||
topicId = Some(topicId)
|
||||
))))
|
||||
|
||||
def homeTopicFollowPromptControllerData(topicId: Long): ControllerData.V2 =
|
||||
ControllerData.V2(
|
||||
ControllerDataV2.HomeTopicFollowPrompt(HomeTopicFollowPromptControllerData.V1(
|
||||
HomeTopicFollowPromptControllerDataV1(Some(topicId)))))
|
||||
|
||||
def homeTopicAnnotationPromptControllerData(topicId: Long): ControllerData.V2 =
|
||||
ControllerData.V2(
|
||||
ControllerDataV2.HomeTopicAnnotationPrompt(HomeTopicAnnotationPromptControllerData.V1(
|
||||
HomeTopicAnnotationPromptControllerDataV1(tweetId = 1L, topicId = topicId))))
|
||||
|
||||
def homeHitlTopicAnnotationPromptControllerData(topicId: Long): ControllerData.V2 =
|
||||
ControllerData.V2(
|
||||
ControllerDataV2.HomeHitlTopicAnnotationPrompt(
|
||||
HomeHitlTopicAnnotationPromptControllerData.V1(
|
||||
HomeHitlTopicAnnotationPromptControllerDataV1(tweetId = 2L, topicId = topicId))))
|
||||
|
||||
def searchTopicFollowPromptControllerData(topicId: Long): ControllerData.V2 =
|
||||
ControllerData.V2(
|
||||
ControllerDataV2.SearchResponse(
|
||||
SearchResponseControllerData.V1(
|
||||
SearchResponseControllerDataV1(
|
||||
Some(ItemTypesControllerData.TopicFollowControllerData(
|
||||
SearchTopicFollowPromptControllerData(Some(topicId))
|
||||
)),
|
||||
None
|
||||
))))
|
||||
|
||||
def searchTweetTypesControllerData(topicId: Long): ControllerData.V2 =
|
||||
ControllerData.V2(
|
||||
ControllerDataV2.SearchResponse(
|
||||
SearchResponseControllerData.V1(
|
||||
SearchResponseControllerDataV1(
|
||||
Some(ItemTypesControllerData.TweetTypesControllerData(
|
||||
TweetTypesControllerData(None, Some(topicId))
|
||||
)),
|
||||
None
|
||||
)
|
||||
)))
|
||||
|
||||
//used for creating logged out user client events
|
||||
def buildLogBaseWithoutUserId(guestId: Long): LogBase =
|
||||
LogBase(
|
||||
ipAddress = "120.10.10.20",
|
||||
guestId = Some(guestId),
|
||||
userAgent = None,
|
||||
transactionId = "",
|
||||
country = Some("US"),
|
||||
timestamp = 100L,
|
||||
language = Some("en")
|
||||
)
|
||||
}
|
||||
|
||||
test("getTopicId should correctly find topic id from item for home timeline and search") {
|
||||
new Fixture {
|
||||
|
||||
val testData = Table(
|
||||
("ItemType", "topicId", "controllerData"),
|
||||
(ItemType.Tweet, 1L, timelineTopicControllerData(1L)),
|
||||
(ItemType.User, 2L, timelineTopicControllerData(2L)),
|
||||
(ItemType.Topic, 3L, homeTweetControllerData(3L)),
|
||||
(ItemType.Topic, 4L, homeTopicFollowPromptControllerData(4L)),
|
||||
(ItemType.Topic, 5L, searchTopicFollowPromptControllerData(5L)),
|
||||
(ItemType.Topic, 6L, homeHitlTopicAnnotationPromptControllerData(6L))
|
||||
)
|
||||
|
||||
forEvery(testData) {
|
||||
(itemType: ItemType, topicId: Long, controllerDataV2: ControllerData.V2) =>
|
||||
getTopicId(
|
||||
buildItemForTimeline(1, itemType, topicId, _ => controllerDataV2),
|
||||
defaultNamespace) shouldEqual Some(topicId)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("getTopicId should correctly find topic id from item for guide events") {
|
||||
new Fixture {
|
||||
getTopicId(
|
||||
buildItemForGuide(1, ItemType.Tweet, 100, topicMetadataInGuide),
|
||||
defaultNamespace
|
||||
) shouldEqual Some(100)
|
||||
}
|
||||
}
|
||||
|
||||
test("getTopicId should correctly find topic id for onboarding events") {
|
||||
new Fixture {
|
||||
getTopicId(
|
||||
buildItemForOnboarding(1, 100),
|
||||
buildNamespaceForOnboarding
|
||||
) shouldEqual Some(100)
|
||||
}
|
||||
}
|
||||
|
||||
test("should return TopicId From HomeSearch") {
|
||||
val testData = Table(
|
||||
("controllerData", "topicId"),
|
||||
(
|
||||
ControllerData.V2(
|
||||
ControllerDataV2.HomeTweets(
|
||||
HomeTweetsControllerData.V1(HomeTweetsControllerDataV1(topicId = Some(1L))))
|
||||
),
|
||||
Some(1L)),
|
||||
(
|
||||
ControllerData.V2(
|
||||
ControllerDataV2.HomeTopicFollowPrompt(HomeTopicFollowPromptControllerData
|
||||
.V1(HomeTopicFollowPromptControllerDataV1(topicId = Some(2L))))),
|
||||
Some(2L)),
|
||||
(
|
||||
ControllerData.V2(
|
||||
ControllerDataV2.TimelinesTopic(
|
||||
TimelinesTopicControllerData.V1(
|
||||
TimelinesTopicControllerDataV1(topicId = 3L, topicTypesBitmap = 100)
|
||||
))),
|
||||
Some(3L)),
|
||||
(
|
||||
ControllerData.V2(
|
||||
ControllerDataV2.SearchResponse(
|
||||
SearchResponseControllerData.V1(SearchResponseControllerDataV1(itemTypesControllerData =
|
||||
Some(ItemTypesControllerData.TopicFollowControllerData(
|
||||
SearchTopicFollowPromptControllerData(topicId = Some(4L)))))))),
|
||||
Some(4L)),
|
||||
(
|
||||
ControllerData.V2(
|
||||
ControllerDataV2.SearchResponse(
|
||||
SearchResponseControllerData.V1(
|
||||
SearchResponseControllerDataV1(itemTypesControllerData = Some(ItemTypesControllerData
|
||||
.TweetTypesControllerData(TweetTypesControllerData(topicId = Some(5L)))))))),
|
||||
Some(5L)),
|
||||
(
|
||||
ControllerData.V2(
|
||||
ControllerDataV2
|
||||
.SearchResponse(SearchResponseControllerData.V1(SearchResponseControllerDataV1()))),
|
||||
None)
|
||||
)
|
||||
|
||||
forEvery(testData) { (controllerDataV2: ControllerData.V2, topicId: Option[Long]) =>
|
||||
getTopicIdFromHomeSearch(
|
||||
Item(suggestionDetails = Some(
|
||||
SuggestionDetails(decodedControllerData = Some(controllerDataV2))))) shouldEqual topicId
|
||||
}
|
||||
}
|
||||
|
||||
test("test TopicId From Onboarding") {
|
||||
val testData = Table(
|
||||
("Item", "EventNamespace", "topicId"),
|
||||
(
|
||||
Item(description = Some("id=11,key=value")),
|
||||
EventNamespace(
|
||||
page = Some("onboarding"),
|
||||
section = Some("section has topic"),
|
||||
component = Some("component has topic"),
|
||||
element = Some("element has topic")
|
||||
),
|
||||
Some(11L)),
|
||||
(
|
||||
Item(description = Some("id=22,key=value")),
|
||||
EventNamespace(
|
||||
page = Some("onboarding"),
|
||||
section = Some("section has topic")
|
||||
),
|
||||
Some(22L)),
|
||||
(
|
||||
Item(description = Some("id=33,key=value")),
|
||||
EventNamespace(
|
||||
page = Some("onboarding"),
|
||||
component = Some("component has topic")
|
||||
),
|
||||
Some(33L)),
|
||||
(
|
||||
Item(description = Some("id=44,key=value")),
|
||||
EventNamespace(
|
||||
page = Some("onboarding"),
|
||||
element = Some("element has topic")
|
||||
),
|
||||
Some(44L)),
|
||||
(
|
||||
Item(description = Some("id=678,key=value")),
|
||||
EventNamespace(
|
||||
page = Some("onXYZboarding"),
|
||||
section = Some("section has topic"),
|
||||
component = Some("component has topic"),
|
||||
element = Some("element has topic")
|
||||
),
|
||||
None),
|
||||
(
|
||||
Item(description = Some("id=678,key=value")),
|
||||
EventNamespace(
|
||||
page = Some("page has onboarding"),
|
||||
section = Some("section has topPic"),
|
||||
component = Some("component has topPic"),
|
||||
element = Some("element has topPic")
|
||||
),
|
||||
None),
|
||||
(
|
||||
Item(description = Some("key=value,id=678")),
|
||||
EventNamespace(
|
||||
page = Some("page has onboarding"),
|
||||
section = Some("section has topic"),
|
||||
component = Some("component has topic"),
|
||||
element = Some("element has topic")
|
||||
),
|
||||
None)
|
||||
)
|
||||
|
||||
forEvery(testData) { (item: Item, eventNamespace: EventNamespace, topicId: Option[Long]) =>
|
||||
getTopicFromOnboarding(item, eventNamespace) shouldEqual topicId
|
||||
}
|
||||
}
|
||||
|
||||
test("test from Guide") {
|
||||
val testData = Table(
|
||||
("guideItemDetails", "topicId"),
|
||||
(
|
||||
GuideItemDetails(transparentGuideDetails = Some(
|
||||
TransparentGuideDetails.TopicMetadata(
|
||||
TopicModuleMetadata.TttInterest(tttInterest = TttInterest.unsafeEmpty)))),
|
||||
None),
|
||||
(
|
||||
GuideItemDetails(transparentGuideDetails = Some(
|
||||
TransparentGuideDetails.TopicMetadata(
|
||||
TopicModuleMetadata.SimClusterInterest(simClusterInterest =
|
||||
com.twitter.guide.scribing.thriftscala.SimClusterInterest.unsafeEmpty)))),
|
||||
None),
|
||||
(
|
||||
GuideItemDetails(transparentGuideDetails = Some(
|
||||
TransparentGuideDetails.TopicMetadata(TopicModuleMetadata.UnknownUnionField(field =
|
||||
TFieldBlob(new TField(), Array.empty[Byte]))))),
|
||||
None),
|
||||
(
|
||||
GuideItemDetails(transparentGuideDetails = Some(
|
||||
TransparentGuideDetails.TopicMetadata(
|
||||
TopicModuleMetadata.SemanticCoreInterest(
|
||||
com.twitter.guide.scribing.thriftscala.SemanticCoreInterest.unsafeEmpty
|
||||
.copy(domainId = "131", entityId = "1"))))),
|
||||
Some(1L)),
|
||||
)
|
||||
|
||||
forEvery(testData) { (guideItemDetails: GuideItemDetails, topicId: Option[Long]) =>
|
||||
getTopicFromGuide(Item(guideItemDetails = Some(guideItemDetails))) shouldEqual topicId
|
||||
}
|
||||
}
|
||||
|
||||
test("getTopicId should return topicIds") {
|
||||
getTopicId(
|
||||
item = Item(suggestionDetails = Some(
|
||||
SuggestionDetails(decodedControllerData = Some(
|
||||
ControllerData.V2(
|
||||
ControllerDataV2.HomeTweets(
|
||||
HomeTweetsControllerData.V1(HomeTweetsControllerDataV1(topicId = Some(1L))))
|
||||
))))),
|
||||
namespace = EventNamespace(
|
||||
page = Some("onboarding"),
|
||||
section = Some("section has topic"),
|
||||
component = Some("component has topic"),
|
||||
element = Some("element has topic")
|
||||
)
|
||||
) shouldEqual Some(1L)
|
||||
}
|
||||
}
|
|
@ -0,0 +1,852 @@
|
|||
package com.twitter.unified_user_actions.adapter
|
||||
|
||||
import com.twitter.gizmoduck.thriftscala.User
|
||||
import com.twitter.gizmoduck.thriftscala.UserType
|
||||
import com.twitter.inject.Test
|
||||
import com.twitter.snowflake.id.SnowflakeId
|
||||
import com.twitter.tweetypie.thriftscala.AdditionalFieldDeleteEvent
|
||||
import com.twitter.tweetypie.thriftscala.AdditionalFieldUpdateEvent
|
||||
import com.twitter.tweetypie.thriftscala.AuditDeleteTweet
|
||||
import com.twitter.tweetypie.thriftscala.DeviceSource
|
||||
import com.twitter.tweetypie.thriftscala.EditControl
|
||||
import com.twitter.tweetypie.thriftscala.EditControlEdit
|
||||
import com.twitter.tweetypie.thriftscala.Language
|
||||
import com.twitter.tweetypie.thriftscala.Place
|
||||
import com.twitter.tweetypie.thriftscala.PlaceType
|
||||
import com.twitter.tweetypie.thriftscala.QuotedTweet
|
||||
import com.twitter.tweetypie.thriftscala.QuotedTweetDeleteEvent
|
||||
import com.twitter.tweetypie.thriftscala.QuotedTweetTakedownEvent
|
||||
import com.twitter.tweetypie.thriftscala.Reply
|
||||
import com.twitter.tweetypie.thriftscala.Share
|
||||
import com.twitter.tweetypie.thriftscala.Tweet
|
||||
import com.twitter.tweetypie.thriftscala.TweetCoreData
|
||||
import com.twitter.tweetypie.thriftscala.TweetCreateEvent
|
||||
import com.twitter.tweetypie.thriftscala.TweetDeleteEvent
|
||||
import com.twitter.tweetypie.thriftscala.TweetEvent
|
||||
import com.twitter.tweetypie.thriftscala.TweetEventData
|
||||
import com.twitter.tweetypie.thriftscala.TweetEventFlags
|
||||
import com.twitter.tweetypie.thriftscala.TweetPossiblySensitiveUpdateEvent
|
||||
import com.twitter.tweetypie.thriftscala.TweetScrubGeoEvent
|
||||
import com.twitter.tweetypie.thriftscala.TweetTakedownEvent
|
||||
import com.twitter.tweetypie.thriftscala.TweetUndeleteEvent
|
||||
import com.twitter.tweetypie.thriftscala.UserScrubGeoEvent
|
||||
import com.twitter.unified_user_actions.adapter.tweetypie_event.TweetypieEventAdapter
|
||||
import com.twitter.unified_user_actions.thriftscala._
|
||||
import com.twitter.util.Time
|
||||
import org.scalatest.prop.TableDrivenPropertyChecks
|
||||
import org.scalatest.prop.TableFor1
|
||||
import org.scalatest.prop.TableFor2
|
||||
import org.scalatest.prop.TableFor3
|
||||
|
||||
class TweetypieEventAdapterSpec extends Test with TableDrivenPropertyChecks {
|
||||
trait Fixture {
|
||||
val frozenTime: Time = Time.fromMilliseconds(1658949273000L)
|
||||
|
||||
val tweetDeleteEventTime: Time = Time.fromMilliseconds(1658949253000L)
|
||||
|
||||
val tweetId = 1554576940856246272L
|
||||
val timestamp: Long = SnowflakeId.unixTimeMillisFromId(tweetId)
|
||||
val userId = 1L
|
||||
val user: User = User(
|
||||
id = userId,
|
||||
createdAtMsec = 1000L,
|
||||
updatedAtMsec = 1000L,
|
||||
userType = UserType.Normal,
|
||||
)
|
||||
|
||||
val actionedTweetId = 1554576940756246333L
|
||||
val actionedTweetTimestamp: Long = SnowflakeId.unixTimeMillisFromId(actionedTweetId)
|
||||
val actionedTweetAuthorId = 2L
|
||||
|
||||
val actionedByActionedTweetId = 1554566940756246272L
|
||||
val actionedByActionedTweetTimestamp: Long =
|
||||
SnowflakeId.unixTimeMillisFromId(actionedByActionedTweetId)
|
||||
val actionedByActionedTweetAuthorId = 3L
|
||||
|
||||
val tweetEventFlags: TweetEventFlags = TweetEventFlags(timestampMs = timestamp)
|
||||
val language: Option[Language] = Some(Language("EN-US", false))
|
||||
val deviceSource: Option[DeviceSource] = Some(
|
||||
DeviceSource(
|
||||
id = 0,
|
||||
parameter = "",
|
||||
internalName = "",
|
||||
name = "name",
|
||||
url = "url",
|
||||
display = "display",
|
||||
clientAppId = Option(100L)))
|
||||
val place: Option[Place] = Some(
|
||||
Place(
|
||||
id = "id",
|
||||
`type` = PlaceType.City,
|
||||
fullName = "San Francisco",
|
||||
name = "SF",
|
||||
countryCode = Some("US"),
|
||||
))
|
||||
|
||||
// for TweetDeleteEvent
|
||||
val auditDeleteTweet = Some(
|
||||
AuditDeleteTweet(
|
||||
clientApplicationId = Option(200L)
|
||||
))
|
||||
|
||||
val tweetCoreData: TweetCoreData =
|
||||
TweetCoreData(userId, text = "text", createdVia = "created_via", createdAtSecs = timestamp)
|
||||
val baseTweet: Tweet = Tweet(
|
||||
tweetId,
|
||||
coreData = Some(tweetCoreData),
|
||||
language = language,
|
||||
deviceSource = deviceSource,
|
||||
place = place)
|
||||
|
||||
def getCreateTweetCoreData(userId: Long, timestamp: Long): TweetCoreData =
|
||||
tweetCoreData.copy(userId = userId, createdAtSecs = timestamp)
|
||||
def getRetweetTweetCoreData(
|
||||
userId: Long,
|
||||
retweetedTweetId: Long,
|
||||
retweetedAuthorId: Long,
|
||||
parentStatusId: Long,
|
||||
timestamp: Long
|
||||
): TweetCoreData = tweetCoreData.copy(
|
||||
userId = userId,
|
||||
share = Some(
|
||||
Share(
|
||||
sourceStatusId = retweetedTweetId,
|
||||
sourceUserId = retweetedAuthorId,
|
||||
parentStatusId = parentStatusId
|
||||
)),
|
||||
createdAtSecs = timestamp
|
||||
)
|
||||
def getReplyTweetCoreData(
|
||||
userId: Long,
|
||||
repliedTweetId: Long,
|
||||
repliedAuthorId: Long,
|
||||
timestamp: Long
|
||||
): TweetCoreData = tweetCoreData.copy(
|
||||
userId = userId,
|
||||
reply = Some(
|
||||
Reply(
|
||||
inReplyToStatusId = Some(repliedTweetId),
|
||||
inReplyToUserId = repliedAuthorId,
|
||||
)
|
||||
),
|
||||
createdAtSecs = timestamp)
|
||||
def getQuoteTweetCoreData(userId: Long, timestamp: Long): TweetCoreData =
|
||||
tweetCoreData.copy(userId = userId, createdAtSecs = timestamp)
|
||||
|
||||
def getTweet(tweetId: Long, userId: Long, timestamp: Long): Tweet =
|
||||
baseTweet.copy(id = tweetId, coreData = Some(getCreateTweetCoreData(userId, timestamp)))
|
||||
|
||||
def getRetweet(
|
||||
tweetId: Long,
|
||||
userId: Long,
|
||||
timestamp: Long,
|
||||
retweetedTweetId: Long,
|
||||
retweetedUserId: Long,
|
||||
parentStatusId: Option[Long] = None
|
||||
): Tweet =
|
||||
baseTweet.copy(
|
||||
id = tweetId,
|
||||
coreData = Some(
|
||||
getRetweetTweetCoreData(
|
||||
userId,
|
||||
retweetedTweetId,
|
||||
retweetedUserId,
|
||||
parentStatusId.getOrElse(retweetedTweetId),
|
||||
timestamp)))
|
||||
|
||||
def getQuote(
|
||||
tweetId: Long,
|
||||
userId: Long,
|
||||
timestamp: Long,
|
||||
quotedTweetId: Long,
|
||||
quotedUserId: Long
|
||||
): Tweet =
|
||||
baseTweet.copy(
|
||||
id = tweetId,
|
||||
coreData = Some(getQuoteTweetCoreData(userId, timestamp)),
|
||||
quotedTweet = Some(QuotedTweet(quotedTweetId, quotedUserId)))
|
||||
|
||||
def getReply(
|
||||
tweetId: Long,
|
||||
userId: Long,
|
||||
repliedTweetId: Long,
|
||||
repliedAuthorId: Long,
|
||||
timestamp: Long
|
||||
): Tweet =
|
||||
baseTweet.copy(
|
||||
id = tweetId,
|
||||
coreData = Some(getReplyTweetCoreData(userId, repliedTweetId, repliedAuthorId, timestamp)),
|
||||
)
|
||||
|
||||
// ignored tweet events
|
||||
val additionalFieldUpdateEvent: TweetEvent = TweetEvent(
|
||||
TweetEventData.AdditionalFieldUpdateEvent(AdditionalFieldUpdateEvent(baseTweet)),
|
||||
tweetEventFlags)
|
||||
val additionalFieldDeleteEvent: TweetEvent = TweetEvent(
|
||||
TweetEventData.AdditionalFieldDeleteEvent(
|
||||
AdditionalFieldDeleteEvent(Map(tweetId -> Seq.empty))
|
||||
),
|
||||
tweetEventFlags
|
||||
)
|
||||
val tweetUndeleteEvent: TweetEvent = TweetEvent(
|
||||
TweetEventData.TweetUndeleteEvent(TweetUndeleteEvent(baseTweet)),
|
||||
tweetEventFlags
|
||||
)
|
||||
val tweetScrubGeoEvent: TweetEvent = TweetEvent(
|
||||
TweetEventData.TweetScrubGeoEvent(TweetScrubGeoEvent(tweetId, userId)),
|
||||
tweetEventFlags)
|
||||
val tweetTakedownEvent: TweetEvent = TweetEvent(
|
||||
TweetEventData.TweetTakedownEvent(TweetTakedownEvent(tweetId, userId)),
|
||||
tweetEventFlags
|
||||
)
|
||||
val userScrubGeoEvent: TweetEvent = TweetEvent(
|
||||
TweetEventData.UserScrubGeoEvent(UserScrubGeoEvent(userId = userId, maxTweetId = tweetId)),
|
||||
tweetEventFlags
|
||||
)
|
||||
val tweetPossiblySensitiveUpdateEvent: TweetEvent = TweetEvent(
|
||||
TweetEventData.TweetPossiblySensitiveUpdateEvent(
|
||||
TweetPossiblySensitiveUpdateEvent(
|
||||
tweetId = tweetId,
|
||||
userId = userId,
|
||||
nsfwAdmin = false,
|
||||
nsfwUser = false)),
|
||||
tweetEventFlags
|
||||
)
|
||||
val quotedTweetDeleteEvent: TweetEvent = TweetEvent(
|
||||
TweetEventData.QuotedTweetDeleteEvent(
|
||||
QuotedTweetDeleteEvent(
|
||||
quotingTweetId = tweetId,
|
||||
quotingUserId = userId,
|
||||
quotedTweetId = tweetId,
|
||||
quotedUserId = userId)),
|
||||
tweetEventFlags
|
||||
)
|
||||
val quotedTweetTakedownEvent: TweetEvent = TweetEvent(
|
||||
TweetEventData.QuotedTweetTakedownEvent(
|
||||
QuotedTweetTakedownEvent(
|
||||
quotingTweetId = tweetId,
|
||||
quotingUserId = userId,
|
||||
quotedTweetId = tweetId,
|
||||
quotedUserId = userId,
|
||||
takedownCountryCodes = Seq.empty,
|
||||
takedownReasons = Seq.empty
|
||||
)
|
||||
),
|
||||
tweetEventFlags
|
||||
)
|
||||
val replyOnlyTweet =
|
||||
getReply(tweetId, userId, actionedTweetId, actionedTweetAuthorId, timestamp)
|
||||
val replyAndRetweetTweet = replyOnlyTweet.copy(coreData = replyOnlyTweet.coreData.map(
|
||||
_.copy(share = Some(
|
||||
Share(
|
||||
sourceStatusId = actionedTweetId,
|
||||
sourceUserId = actionedTweetAuthorId,
|
||||
parentStatusId = actionedTweetId
|
||||
)))))
|
||||
val replyRetweetPresentEvent: TweetEvent = TweetEvent(
|
||||
TweetEventData.TweetCreateEvent(
|
||||
TweetCreateEvent(
|
||||
tweet = replyAndRetweetTweet,
|
||||
user = user,
|
||||
sourceTweet =
|
||||
Some(getTweet(actionedTweetId, actionedTweetAuthorId, actionedTweetTimestamp))
|
||||
)),
|
||||
tweetEventFlags
|
||||
)
|
||||
|
||||
def getExpectedUUA(
|
||||
userId: Long,
|
||||
actionTweetId: Long,
|
||||
actionTweetAuthorId: Long,
|
||||
sourceTimestampMs: Long,
|
||||
actionType: ActionType,
|
||||
replyingTweetId: Option[Long] = None,
|
||||
quotingTweetId: Option[Long] = None,
|
||||
retweetingTweetId: Option[Long] = None,
|
||||
inReplyToTweetId: Option[Long] = None,
|
||||
quotedTweetId: Option[Long] = None,
|
||||
retweetedTweetId: Option[Long] = None,
|
||||
editedTweetId: Option[Long] = None,
|
||||
appId: Option[Long] = None,
|
||||
): UnifiedUserAction = UnifiedUserAction(
|
||||
userIdentifier = UserIdentifier(userId = Some(userId)),
|
||||
item = Item.TweetInfo(
|
||||
TweetInfo(
|
||||
actionTweetId = actionTweetId,
|
||||
actionTweetAuthorInfo = Some(AuthorInfo(authorId = Some(actionTweetAuthorId))),
|
||||
replyingTweetId = replyingTweetId,
|
||||
quotingTweetId = quotingTweetId,
|
||||
retweetingTweetId = retweetingTweetId,
|
||||
inReplyToTweetId = inReplyToTweetId,
|
||||
quotedTweetId = quotedTweetId,
|
||||
retweetedTweetId = retweetedTweetId,
|
||||
editedTweetId = editedTweetId
|
||||
)
|
||||
),
|
||||
actionType = actionType,
|
||||
eventMetadata = EventMetadata(
|
||||
sourceTimestampMs = sourceTimestampMs,
|
||||
receivedTimestampMs = frozenTime.inMilliseconds,
|
||||
sourceLineage = SourceLineage.ServerTweetypieEvents,
|
||||
language = None,
|
||||
countryCode = Some("US"),
|
||||
clientAppId = appId,
|
||||
)
|
||||
)
|
||||
|
||||
/* Note: This is a deprecated field {ActionTweetType}.
|
||||
* We keep this here to document the behaviors of each unit test.
|
||||
/*
|
||||
* Types of tweets on which actions can take place.
|
||||
* Note that retweets are not included because actions can NOT take place
|
||||
* on retweets. They can only take place on source tweets of retweets,
|
||||
* which are one of the ActionTweetTypes listed below.
|
||||
*/
|
||||
enum ActionTweetType {
|
||||
/* Is a standard (non-retweet, non-reply, non-quote) tweet */
|
||||
Default = 0
|
||||
|
||||
/*
|
||||
* Is a tweet in a reply chain (this includes tweets
|
||||
* without a leading @mention, as long as they are in reply
|
||||
* to some tweet id)
|
||||
*/
|
||||
Reply = 1
|
||||
|
||||
/* Is a retweet with comment */
|
||||
Quote = 2
|
||||
}(persisted='true', hasPersonalData='false')
|
||||
*/
|
||||
|
||||
// tweet create
|
||||
val tweetCreateEvent: TweetEvent = TweetEvent(
|
||||
TweetEventData.TweetCreateEvent(
|
||||
TweetCreateEvent(
|
||||
tweet = getTweet(tweetId, userId, timestamp),
|
||||
user = user,
|
||||
)
|
||||
),
|
||||
tweetEventFlags)
|
||||
val expectedUUACreate = getExpectedUUA(
|
||||
userId = userId,
|
||||
actionTweetId = tweetId,
|
||||
/* @see comment above for ActionTweetType
|
||||
actionTweetType = Some(ActionTweetType.Default),
|
||||
*/
|
||||
actionTweetAuthorId = userId,
|
||||
sourceTimestampMs = timestamp,
|
||||
actionType = ActionType.ServerTweetCreate,
|
||||
appId = deviceSource.flatMap(_.clientAppId)
|
||||
)
|
||||
|
||||
// tweet reply to a default
|
||||
val tweetReplyDefaultEvent: TweetEvent = TweetEvent(
|
||||
TweetEventData.TweetCreateEvent(
|
||||
TweetCreateEvent(
|
||||
tweet = getReply(tweetId, userId, actionedTweetId, actionedTweetAuthorId, timestamp),
|
||||
user = user
|
||||
)
|
||||
),
|
||||
tweetEventFlags
|
||||
)
|
||||
val expectedUUAReplyDefault = getExpectedUUA(
|
||||
userId = userId,
|
||||
actionTweetId = actionedTweetId,
|
||||
/* @see comment above for ActionTweetType
|
||||
actionTweetType = None,
|
||||
*/
|
||||
actionTweetAuthorId = actionedTweetAuthorId,
|
||||
sourceTimestampMs = timestamp,
|
||||
actionType = ActionType.ServerTweetReply,
|
||||
replyingTweetId = Some(tweetId),
|
||||
appId = deviceSource.flatMap(_.clientAppId)
|
||||
)
|
||||
// tweet reply to a reply
|
||||
val tweetReplyToReplyEvent: TweetEvent = TweetEvent(
|
||||
TweetEventData.TweetCreateEvent(
|
||||
TweetCreateEvent(
|
||||
tweet = getReply(tweetId, userId, actionedTweetId, actionedTweetAuthorId, timestamp),
|
||||
user = user
|
||||
)
|
||||
),
|
||||
tweetEventFlags
|
||||
)
|
||||
// tweet reply to a quote
|
||||
val tweetReplyToQuoteEvent: TweetEvent = TweetEvent(
|
||||
TweetEventData.TweetCreateEvent(
|
||||
TweetCreateEvent(
|
||||
tweet = getReply(tweetId, userId, actionedTweetId, actionedTweetAuthorId, timestamp),
|
||||
user = user
|
||||
)
|
||||
),
|
||||
tweetEventFlags
|
||||
)
|
||||
// tweet quote a default
|
||||
val tweetQuoteDefaultEvent: TweetEvent = TweetEvent(
|
||||
TweetEventData.TweetCreateEvent(
|
||||
TweetCreateEvent(
|
||||
tweet = getQuote(tweetId, userId, timestamp, actionedTweetId, actionedTweetAuthorId),
|
||||
user = user,
|
||||
quotedTweet =
|
||||
Some(getTweet(actionedTweetId, actionedTweetAuthorId, actionedTweetTimestamp))
|
||||
)
|
||||
),
|
||||
tweetEventFlags
|
||||
)
|
||||
val expectedUUAQuoteDefault: UnifiedUserAction = getExpectedUUA(
|
||||
userId = userId,
|
||||
actionTweetId = actionedTweetId,
|
||||
/* @see comment above for ActionTweetType
|
||||
actionTweetType = Some(ActionTweetType.Default),
|
||||
*/
|
||||
actionTweetAuthorId = actionedTweetAuthorId,
|
||||
sourceTimestampMs = timestamp,
|
||||
actionType = ActionType.ServerTweetQuote,
|
||||
quotingTweetId = Some(tweetId),
|
||||
appId = deviceSource.flatMap(_.clientAppId)
|
||||
)
|
||||
// tweet quote a reply
|
||||
val tweetQuoteReplyEvent: TweetEvent = TweetEvent(
|
||||
TweetEventData.TweetCreateEvent(
|
||||
TweetCreateEvent(
|
||||
tweet = getQuote(tweetId, userId, timestamp, actionedTweetId, actionedTweetAuthorId),
|
||||
user = user,
|
||||
quotedTweet = Some(
|
||||
getReply(
|
||||
tweetId = actionedTweetId,
|
||||
userId = actionedTweetAuthorId,
|
||||
repliedTweetId = actionedByActionedTweetId,
|
||||
repliedAuthorId = actionedByActionedTweetAuthorId,
|
||||
timestamp = actionedTweetTimestamp
|
||||
))
|
||||
)
|
||||
),
|
||||
tweetEventFlags
|
||||
)
|
||||
val expectedUUAQuoteReply: UnifiedUserAction = getExpectedUUA(
|
||||
userId = userId,
|
||||
actionTweetId = actionedTweetId,
|
||||
/* @see comment above for ActionTweetType
|
||||
actionTweetType = Some(ActionTweetType.Reply),
|
||||
*/
|
||||
actionTweetAuthorId = actionedTweetAuthorId,
|
||||
sourceTimestampMs = timestamp,
|
||||
actionType = ActionType.ServerTweetQuote,
|
||||
quotingTweetId = Some(tweetId),
|
||||
appId = deviceSource.flatMap(_.clientAppId)
|
||||
)
|
||||
// tweet quote a quote
|
||||
val tweetQuoteQuoteEvent: TweetEvent = TweetEvent(
|
||||
TweetEventData.TweetCreateEvent(
|
||||
TweetCreateEvent(
|
||||
tweet = getQuote(tweetId, userId, timestamp, actionedTweetId, actionedTweetAuthorId),
|
||||
user = user,
|
||||
quotedTweet = Some(
|
||||
getQuote(
|
||||
tweetId = actionedTweetId,
|
||||
userId = actionedTweetAuthorId,
|
||||
timestamp = actionedTweetTimestamp,
|
||||
quotedTweetId = actionedByActionedTweetId,
|
||||
quotedUserId = actionedByActionedTweetAuthorId,
|
||||
))
|
||||
)
|
||||
),
|
||||
tweetEventFlags
|
||||
)
|
||||
val expectedUUAQuoteQuote: UnifiedUserAction = getExpectedUUA(
|
||||
userId = userId,
|
||||
actionTweetId = actionedTweetId,
|
||||
/* @see comment above for ActionTweetType
|
||||
actionTweetType = Some(ActionTweetType.Quote),
|
||||
*/
|
||||
actionTweetAuthorId = actionedTweetAuthorId,
|
||||
sourceTimestampMs = timestamp,
|
||||
actionType = ActionType.ServerTweetQuote,
|
||||
quotingTweetId = Some(tweetId),
|
||||
appId = deviceSource.flatMap(_.clientAppId)
|
||||
)
|
||||
// tweet retweet a default
|
||||
val tweetRetweetDefaultEvent: TweetEvent = TweetEvent(
|
||||
TweetEventData.TweetCreateEvent(
|
||||
TweetCreateEvent(
|
||||
tweet = getRetweet(tweetId, userId, timestamp, actionedTweetId, actionedTweetAuthorId),
|
||||
user = user,
|
||||
sourceTweet =
|
||||
Some(getTweet(actionedTweetId, actionedTweetAuthorId, actionedTweetTimestamp))
|
||||
)
|
||||
),
|
||||
tweetEventFlags
|
||||
)
|
||||
val expectedUUARetweetDefault: UnifiedUserAction = getExpectedUUA(
|
||||
userId = userId,
|
||||
actionTweetId = actionedTweetId,
|
||||
/* @see comment above for ActionTweetType
|
||||
actionTweetType = Some(ActionTweetType.Default),
|
||||
*/
|
||||
actionTweetAuthorId = actionedTweetAuthorId,
|
||||
sourceTimestampMs = timestamp,
|
||||
actionType = ActionType.ServerTweetRetweet,
|
||||
retweetingTweetId = Some(tweetId),
|
||||
appId = deviceSource.flatMap(_.clientAppId)
|
||||
)
|
||||
// tweet retweet a reply
|
||||
val tweetRetweetReplyEvent: TweetEvent = TweetEvent(
|
||||
TweetEventData.TweetCreateEvent(
|
||||
TweetCreateEvent(
|
||||
tweet = getRetweet(tweetId, userId, timestamp, actionedTweetId, actionedTweetAuthorId),
|
||||
user = user,
|
||||
sourceTweet = Some(
|
||||
getReply(
|
||||
actionedTweetId,
|
||||
actionedTweetAuthorId,
|
||||
actionedByActionedTweetId,
|
||||
actionedByActionedTweetAuthorId,
|
||||
actionedTweetTimestamp))
|
||||
)
|
||||
),
|
||||
tweetEventFlags
|
||||
)
|
||||
val expectedUUARetweetReply: UnifiedUserAction = getExpectedUUA(
|
||||
userId = userId,
|
||||
actionTweetId = actionedTweetId,
|
||||
/* @see comment above for ActionTweetType
|
||||
actionTweetType = Some(ActionTweetType.Reply),
|
||||
*/
|
||||
actionTweetAuthorId = actionedTweetAuthorId,
|
||||
sourceTimestampMs = timestamp,
|
||||
actionType = ActionType.ServerTweetRetweet,
|
||||
retweetingTweetId = Some(tweetId),
|
||||
appId = deviceSource.flatMap(_.clientAppId)
|
||||
)
|
||||
// tweet retweet a quote
|
||||
val tweetRetweetQuoteEvent: TweetEvent = TweetEvent(
|
||||
TweetEventData.TweetCreateEvent(
|
||||
TweetCreateEvent(
|
||||
tweet = getRetweet(tweetId, userId, timestamp, actionedTweetId, actionedTweetAuthorId),
|
||||
user = user,
|
||||
sourceTweet = Some(
|
||||
getQuote(
|
||||
actionedTweetId,
|
||||
actionedTweetAuthorId,
|
||||
actionedTweetTimestamp,
|
||||
actionedByActionedTweetId,
|
||||
actionedByActionedTweetAuthorId
|
||||
))
|
||||
)
|
||||
),
|
||||
tweetEventFlags
|
||||
)
|
||||
val expectedUUARetweetQuote: UnifiedUserAction = getExpectedUUA(
|
||||
userId = userId,
|
||||
actionTweetId = actionedTweetId,
|
||||
/* @see comment above for ActionTweetType
|
||||
actionTweetType = Some(ActionTweetType.Quote),
|
||||
*/
|
||||
actionTweetAuthorId = actionedTweetAuthorId,
|
||||
sourceTimestampMs = timestamp,
|
||||
actionType = ActionType.ServerTweetRetweet,
|
||||
retweetingTweetId = Some(tweetId),
|
||||
appId = deviceSource.flatMap(_.clientAppId)
|
||||
)
|
||||
// tweet retweet a retweet
|
||||
val tweetRetweetRetweetEvent: TweetEvent = TweetEvent(
|
||||
TweetEventData.TweetCreateEvent(
|
||||
TweetCreateEvent(
|
||||
tweet = getRetweet(
|
||||
tweetId,
|
||||
userId,
|
||||
timestamp,
|
||||
actionedByActionedTweetId,
|
||||
actionedByActionedTweetAuthorId,
|
||||
Some(actionedTweetId)),
|
||||
user = user,
|
||||
sourceTweet = Some(
|
||||
getTweet(
|
||||
actionedByActionedTweetId,
|
||||
actionedByActionedTweetAuthorId,
|
||||
actionedByActionedTweetTimestamp,
|
||||
))
|
||||
)
|
||||
),
|
||||
tweetEventFlags
|
||||
)
|
||||
val expectedUUARetweetRetweet: UnifiedUserAction = getExpectedUUA(
|
||||
userId = userId,
|
||||
actionTweetId = actionedByActionedTweetId,
|
||||
/* @see comment above for ActionTweetType
|
||||
actionTweetType = Some(ActionTweetType.Default),
|
||||
*/
|
||||
actionTweetAuthorId = actionedByActionedTweetAuthorId,
|
||||
sourceTimestampMs = timestamp,
|
||||
actionType = ActionType.ServerTweetRetweet,
|
||||
retweetingTweetId = Some(tweetId),
|
||||
appId = deviceSource.flatMap(_.clientAppId)
|
||||
)
|
||||
// delete a tweet
|
||||
val tweetDeleteEvent: TweetEvent = TweetEvent(
|
||||
TweetEventData.TweetDeleteEvent(
|
||||
TweetDeleteEvent(
|
||||
tweet = getTweet(tweetId, userId, timestamp),
|
||||
user = Some(user),
|
||||
audit = auditDeleteTweet
|
||||
)
|
||||
),
|
||||
tweetEventFlags.copy(timestampMs = tweetDeleteEventTime.inMilliseconds)
|
||||
)
|
||||
val expectedUUADeleteDefault: UnifiedUserAction = getExpectedUUA(
|
||||
userId = user.id,
|
||||
actionTweetId = tweetId,
|
||||
actionTweetAuthorId = userId,
|
||||
sourceTimestampMs = tweetDeleteEventTime.inMilliseconds,
|
||||
actionType = ActionType.ServerTweetDelete,
|
||||
appId = auditDeleteTweet.flatMap(_.clientApplicationId)
|
||||
)
|
||||
// delete a reply - Unreply
|
||||
val tweetUnreplyEvent: TweetEvent = TweetEvent(
|
||||
TweetEventData.TweetDeleteEvent(
|
||||
TweetDeleteEvent(
|
||||
tweet = getReply(tweetId, userId, actionedTweetId, actionedTweetAuthorId, timestamp),
|
||||
user = Some(user),
|
||||
audit = auditDeleteTweet
|
||||
)
|
||||
),
|
||||
tweetEventFlags.copy(timestampMs = tweetDeleteEventTime.inMilliseconds)
|
||||
)
|
||||
val expectedUUAUnreply: UnifiedUserAction = getExpectedUUA(
|
||||
userId = user.id,
|
||||
actionTweetId = actionedTweetId,
|
||||
actionTweetAuthorId = actionedTweetAuthorId,
|
||||
sourceTimestampMs = tweetDeleteEventTime.inMilliseconds,
|
||||
actionType = ActionType.ServerTweetUnreply,
|
||||
replyingTweetId = Some(tweetId),
|
||||
appId = auditDeleteTweet.flatMap(_.clientApplicationId)
|
||||
)
|
||||
// delete a quote - Unquote
|
||||
val tweetUnquoteEvent: TweetEvent = TweetEvent(
|
||||
TweetEventData.TweetDeleteEvent(
|
||||
TweetDeleteEvent(
|
||||
tweet = getQuote(tweetId, userId, timestamp, actionedTweetId, actionedTweetAuthorId),
|
||||
user = Some(user),
|
||||
audit = auditDeleteTweet
|
||||
)
|
||||
),
|
||||
tweetEventFlags.copy(timestampMs = tweetDeleteEventTime.inMilliseconds)
|
||||
)
|
||||
val expectedUUAUnquote: UnifiedUserAction = getExpectedUUA(
|
||||
userId = user.id,
|
||||
actionTweetId = actionedTweetId,
|
||||
actionTweetAuthorId = actionedTweetAuthorId,
|
||||
sourceTimestampMs = tweetDeleteEventTime.inMilliseconds,
|
||||
actionType = ActionType.ServerTweetUnquote,
|
||||
quotingTweetId = Some(tweetId),
|
||||
appId = auditDeleteTweet.flatMap(_.clientApplicationId)
|
||||
)
|
||||
// delete a retweet / unretweet
|
||||
val tweetUnretweetEvent: TweetEvent = TweetEvent(
|
||||
TweetEventData.TweetDeleteEvent(
|
||||
TweetDeleteEvent(
|
||||
tweet = getRetweet(
|
||||
tweetId,
|
||||
userId,
|
||||
timestamp,
|
||||
actionedTweetId,
|
||||
actionedTweetAuthorId,
|
||||
Some(actionedTweetId)),
|
||||
user = Some(user),
|
||||
audit = auditDeleteTweet
|
||||
)
|
||||
),
|
||||
tweetEventFlags.copy(timestampMs = tweetDeleteEventTime.inMilliseconds)
|
||||
)
|
||||
val expectedUUAUnretweet: UnifiedUserAction = getExpectedUUA(
|
||||
userId = user.id,
|
||||
actionTweetId = actionedTweetId,
|
||||
actionTweetAuthorId = actionedTweetAuthorId,
|
||||
sourceTimestampMs = tweetDeleteEventTime.inMilliseconds,
|
||||
actionType = ActionType.ServerTweetUnretweet,
|
||||
retweetingTweetId = Some(tweetId),
|
||||
appId = auditDeleteTweet.flatMap(_.clientApplicationId)
|
||||
)
|
||||
// edit a tweet, the new tweet from edit is a default tweet (not reply/quote/retweet)
|
||||
val regularTweetFromEditEvent: TweetEvent = TweetEvent(
|
||||
TweetEventData.TweetCreateEvent(
|
||||
TweetCreateEvent(
|
||||
tweet = getTweet(
|
||||
tweetId,
|
||||
userId,
|
||||
timestamp
|
||||
).copy(editControl =
|
||||
Some(EditControl.Edit(EditControlEdit(initialTweetId = actionedTweetId)))),
|
||||
user = user,
|
||||
)
|
||||
),
|
||||
tweetEventFlags
|
||||
)
|
||||
val expectedUUARegularTweetFromEdit: UnifiedUserAction = getExpectedUUA(
|
||||
userId = user.id,
|
||||
actionTweetId = tweetId,
|
||||
actionTweetAuthorId = userId,
|
||||
sourceTimestampMs = tweetEventFlags.timestampMs,
|
||||
actionType = ActionType.ServerTweetEdit,
|
||||
editedTweetId = Some(actionedTweetId),
|
||||
appId = deviceSource.flatMap(_.clientAppId)
|
||||
)
|
||||
// edit a tweet, the new tweet from edit is a Quote
|
||||
val quoteFromEditEvent: TweetEvent = TweetEvent(
|
||||
TweetEventData.TweetCreateEvent(
|
||||
TweetCreateEvent(
|
||||
tweet = getQuote(
|
||||
tweetId,
|
||||
userId,
|
||||
timestamp,
|
||||
actionedTweetId,
|
||||
actionedTweetAuthorId
|
||||
).copy(editControl =
|
||||
Some(EditControl.Edit(EditControlEdit(initialTweetId = actionedByActionedTweetId)))),
|
||||
user = user,
|
||||
)
|
||||
),
|
||||
tweetEventFlags
|
||||
)
|
||||
val expectedUUAQuoteFromEdit: UnifiedUserAction = getExpectedUUA(
|
||||
userId = user.id,
|
||||
actionTweetId = tweetId,
|
||||
actionTweetAuthorId = userId,
|
||||
sourceTimestampMs = tweetEventFlags.timestampMs,
|
||||
actionType = ActionType.ServerTweetEdit,
|
||||
editedTweetId = Some(actionedByActionedTweetId),
|
||||
quotedTweetId = Some(actionedTweetId),
|
||||
appId = deviceSource.flatMap(_.clientAppId)
|
||||
)
|
||||
}
|
||||
|
||||
test("ignore non-TweetCreate / non-TweetDelete events") {
|
||||
new Fixture {
|
||||
val ignoredTweetEvents: TableFor1[TweetEvent] = Table(
|
||||
"ignoredTweetEvents",
|
||||
additionalFieldUpdateEvent,
|
||||
additionalFieldDeleteEvent,
|
||||
tweetUndeleteEvent,
|
||||
tweetScrubGeoEvent,
|
||||
tweetTakedownEvent,
|
||||
userScrubGeoEvent,
|
||||
tweetPossiblySensitiveUpdateEvent,
|
||||
quotedTweetDeleteEvent,
|
||||
quotedTweetTakedownEvent
|
||||
)
|
||||
forEvery(ignoredTweetEvents) { tweetEvent: TweetEvent =>
|
||||
val actual = TweetypieEventAdapter.adaptEvent(tweetEvent)
|
||||
assert(actual.isEmpty)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("ignore invalid TweetCreate events") {
|
||||
new Fixture {
|
||||
val ignoredTweetEvents: TableFor2[String, TweetEvent] = Table(
|
||||
("invalidType", "event"),
|
||||
("replyAndRetweetBothPresent", replyRetweetPresentEvent)
|
||||
)
|
||||
forEvery(ignoredTweetEvents) { (_, event) =>
|
||||
val actual = TweetypieEventAdapter.adaptEvent(event)
|
||||
assert(actual.isEmpty)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("TweetypieCreateEvent") {
|
||||
new Fixture {
|
||||
Time.withTimeAt(frozenTime) { _ =>
|
||||
val actual = TweetypieEventAdapter.adaptEvent(tweetCreateEvent)
|
||||
assert(Seq(expectedUUACreate) == actual)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("TweetypieReplyEvent") {
|
||||
new Fixture {
|
||||
Time.withTimeAt(frozenTime) { _ =>
|
||||
val tweetReplies: TableFor3[String, TweetEvent, UnifiedUserAction] = Table(
|
||||
("actionTweetType", "event", "expected"),
|
||||
("Default", tweetReplyDefaultEvent, expectedUUAReplyDefault),
|
||||
("Reply", tweetReplyToReplyEvent, expectedUUAReplyDefault),
|
||||
("Quote", tweetReplyToQuoteEvent, expectedUUAReplyDefault),
|
||||
)
|
||||
forEvery(tweetReplies) { (_: String, event: TweetEvent, expected: UnifiedUserAction) =>
|
||||
val actual = TweetypieEventAdapter.adaptEvent(event)
|
||||
assert(Seq(expected) === actual)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("TweetypieQuoteEvent") {
|
||||
new Fixture {
|
||||
Time.withTimeAt(frozenTime) { _ =>
|
||||
val tweetQuotes: TableFor3[String, TweetEvent, UnifiedUserAction] = Table(
|
||||
("actionTweetType", "event", "expected"),
|
||||
("Default", tweetQuoteDefaultEvent, expectedUUAQuoteDefault),
|
||||
("Reply", tweetQuoteReplyEvent, expectedUUAQuoteReply),
|
||||
("Quote", tweetQuoteQuoteEvent, expectedUUAQuoteQuote),
|
||||
)
|
||||
forEvery(tweetQuotes) { (_: String, event: TweetEvent, expected: UnifiedUserAction) =>
|
||||
val actual = TweetypieEventAdapter.adaptEvent(event)
|
||||
assert(Seq(expected) === actual)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("TweetypieRetweetEvent") {
|
||||
new Fixture {
|
||||
Time.withTimeAt(frozenTime) { _ =>
|
||||
val tweetRetweets: TableFor3[String, TweetEvent, UnifiedUserAction] = Table(
|
||||
("actionTweetType", "event", "expected"),
|
||||
("Default", tweetRetweetDefaultEvent, expectedUUARetweetDefault),
|
||||
("Reply", tweetRetweetReplyEvent, expectedUUARetweetReply),
|
||||
("Quote", tweetRetweetQuoteEvent, expectedUUARetweetQuote),
|
||||
("Retweet", tweetRetweetRetweetEvent, expectedUUARetweetRetweet),
|
||||
)
|
||||
forEvery(tweetRetweets) { (_: String, event: TweetEvent, expected: UnifiedUserAction) =>
|
||||
val actual = TweetypieEventAdapter.adaptEvent(event)
|
||||
assert(Seq(expected) === actual)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("TweetypieDeleteEvent") {
|
||||
new Fixture {
|
||||
Time.withTimeAt(frozenTime) { _ =>
|
||||
val tweetDeletes: TableFor3[String, TweetEvent, UnifiedUserAction] = Table(
|
||||
("actionTweetType", "event", "expected"),
|
||||
("Default", tweetDeleteEvent, expectedUUADeleteDefault),
|
||||
("Reply", tweetUnreplyEvent, expectedUUAUnreply),
|
||||
("Quote", tweetUnquoteEvent, expectedUUAUnquote),
|
||||
("Retweet", tweetUnretweetEvent, expectedUUAUnretweet),
|
||||
)
|
||||
forEvery(tweetDeletes) { (_: String, event: TweetEvent, expected: UnifiedUserAction) =>
|
||||
val actual = TweetypieEventAdapter.adaptEvent(event)
|
||||
assert(Seq(expected) === actual)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("TweetypieEditEvent") {
|
||||
new Fixture {
|
||||
Time.withTimeAt(frozenTime) { _ =>
|
||||
val tweetEdits: TableFor3[String, TweetEvent, UnifiedUserAction] = Table(
|
||||
("actionTweetType", "event", "expected"),
|
||||
("RegularTweetFromEdit", regularTweetFromEditEvent, expectedUUARegularTweetFromEdit),
|
||||
("QuoteFromEdit", quoteFromEditEvent, expectedUUAQuoteFromEdit)
|
||||
)
|
||||
forEvery(tweetEdits) { (_: String, event: TweetEvent, expected: UnifiedUserAction) =>
|
||||
val actual = TweetypieEventAdapter.adaptEvent(event)
|
||||
assert(Seq(expected) === actual)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,25 @@
|
|||
package unified_user_actions.adapter.src.test.scala.com.twitter.unified_user_actions.adapter
|
||||
|
||||
import com.twitter.inject.Test
|
||||
import com.twitter.unified_user_actions.adapter.TestFixtures.UserModificationEventFixture
|
||||
import com.twitter.unified_user_actions.adapter.user_modification.UserModificationAdapter
|
||||
import com.twitter.util.Time
|
||||
import org.scalatest.prop.TableDrivenPropertyChecks
|
||||
|
||||
class UserModificationAdapterSpec extends Test with TableDrivenPropertyChecks {
|
||||
test("User Create") {
|
||||
new UserModificationEventFixture {
|
||||
Time.withTimeAt(frozenTime) { _ =>
|
||||
assert(UserModificationAdapter.adaptEvent(userCreate) === Seq(expectedUuaUserCreate))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("User Update") {
|
||||
new UserModificationEventFixture {
|
||||
Time.withTimeAt(frozenTime) { _ =>
|
||||
assert(UserModificationAdapter.adaptEvent(userUpdate) === Seq(expectedUuaUserUpdate))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,102 @@
|
|||
package com.twitter.unified_user_actions.adapter
|
||||
|
||||
import com.twitter.clientapp.thriftscala.AmplifyDetails
|
||||
import com.twitter.clientapp.thriftscala.MediaDetails
|
||||
import com.twitter.clientapp.thriftscala.MediaType
|
||||
import com.twitter.mediaservices.commons.thriftscala.MediaCategory
|
||||
import com.twitter.unified_user_actions.adapter.client_event.VideoClientEventUtils.getVideoMetadata
|
||||
import com.twitter.unified_user_actions.adapter.client_event.VideoClientEventUtils.videoIdFromMediaIdentifier
|
||||
import com.twitter.unified_user_actions.thriftscala._
|
||||
import com.twitter.util.mock.Mockito
|
||||
import com.twitter.video.analytics.thriftscala._
|
||||
import org.junit.runner.RunWith
|
||||
import org.scalatest.funsuite.AnyFunSuite
|
||||
import org.scalatest.matchers.should.Matchers
|
||||
import org.scalatest.prop.TableDrivenPropertyChecks
|
||||
import org.scalatestplus.junit.JUnitRunner
|
||||
|
||||
@RunWith(classOf[JUnitRunner])
|
||||
class VideoClientEventUtilsSpec
|
||||
extends AnyFunSuite
|
||||
with Matchers
|
||||
with Mockito
|
||||
with TableDrivenPropertyChecks {
|
||||
|
||||
trait Fixture {
|
||||
val mediaDetails = Seq[MediaDetails](
|
||||
MediaDetails(
|
||||
contentId = Some("456"),
|
||||
mediaType = Some(MediaType.ConsumerVideo),
|
||||
dynamicAds = Some(false)),
|
||||
MediaDetails(
|
||||
contentId = Some("123"),
|
||||
mediaType = Some(MediaType.ConsumerVideo),
|
||||
dynamicAds = Some(false)),
|
||||
MediaDetails(
|
||||
contentId = Some("789"),
|
||||
mediaType = Some(MediaType.ConsumerVideo),
|
||||
dynamicAds = Some(false))
|
||||
)
|
||||
|
||||
val videoMetadata: TweetActionInfo = TweetActionInfo.TweetVideoWatch(
|
||||
TweetVideoWatch(mediaType = Some(MediaType.ConsumerVideo), isMonetizable = Some(false)))
|
||||
|
||||
val videoMetadataWithAmplifyDetailsVideoType: TweetActionInfo = TweetActionInfo.TweetVideoWatch(
|
||||
TweetVideoWatch(
|
||||
mediaType = Some(MediaType.ConsumerVideo),
|
||||
isMonetizable = Some(false),
|
||||
videoType = Some("content")))
|
||||
|
||||
val validMediaIdentifier: MediaIdentifier = MediaIdentifier.MediaPlatformIdentifier(
|
||||
MediaPlatformIdentifier(mediaId = 123L, mediaCategory = MediaCategory.TweetVideo))
|
||||
|
||||
val invalidMediaIdentifier: MediaIdentifier = MediaIdentifier.AmplifyCardIdentifier(
|
||||
AmplifyCardIdentifier(vmapUrl = "", contentId = "")
|
||||
)
|
||||
}
|
||||
|
||||
test("findVideoMetadata") {
|
||||
new Fixture {
|
||||
val testData = Table(
|
||||
("testType", "mediaId", "mediaItems", "amplifyDetails", "expectedOutput"),
|
||||
("emptyMediaDetails", "123", Seq[MediaDetails](), None, None),
|
||||
("mediaIdNotFound", "111", mediaDetails, None, None),
|
||||
("mediaIdFound", "123", mediaDetails, None, Some(videoMetadata)),
|
||||
(
|
||||
"mediaIdFound",
|
||||
"123",
|
||||
mediaDetails,
|
||||
Some(AmplifyDetails(videoType = Some("content"))),
|
||||
Some(videoMetadataWithAmplifyDetailsVideoType))
|
||||
)
|
||||
|
||||
forEvery(testData) {
|
||||
(
|
||||
_: String,
|
||||
mediaId: String,
|
||||
mediaItems: Seq[MediaDetails],
|
||||
amplifyDetails: Option[AmplifyDetails],
|
||||
expectedOutput: Option[TweetActionInfo]
|
||||
) =>
|
||||
val actual = getVideoMetadata(mediaId, mediaItems, amplifyDetails)
|
||||
assert(expectedOutput === actual)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("videoIdFromMediaIdentifier") {
|
||||
new Fixture {
|
||||
val testData = Table(
|
||||
("testType", "mediaIdentifier", "expectedOutput"),
|
||||
("validMediaIdentifierType", validMediaIdentifier, Some("123")),
|
||||
("invalidMediaIdentifierType", invalidMediaIdentifier, None)
|
||||
)
|
||||
|
||||
forEvery(testData) {
|
||||
(_: String, mediaIdentifier: MediaIdentifier, expectedOutput: Option[String]) =>
|
||||
val actual = videoIdFromMediaIdentifier(mediaIdentifier)
|
||||
assert(expectedOutput === actual)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,11 @@
|
|||
scala_library(
|
||||
sources = [
|
||||
"*.scala",
|
||||
],
|
||||
compiler_option_sets = ["fatal_warnings"],
|
||||
# Our runtime is using Java 11, but for compatibility with other internal libraries that
|
||||
# are still on Java 8, we'll make our target platform to be Java 8 as well until everyone can
|
||||
# migrate.
|
||||
platform = "java8",
|
||||
tags = ["bazel-compatible"],
|
||||
)
|
|
@ -0,0 +1,24 @@
|
|||
package com.twitter.unified_user_actions.client.config
|
||||
|
||||
sealed trait ClusterConfig {
|
||||
val name: String
|
||||
val environment: EnvironmentConfig
|
||||
}
|
||||
|
||||
object Clusters {
|
||||
/*
|
||||
* Our production cluster for external consumption. Our SLAs are enforced.
|
||||
*/
|
||||
case object ProdCluster extends ClusterConfig {
|
||||
override val name: String = Constants.UuaKafkaProdClusterName
|
||||
override val environment: EnvironmentConfig = Environments.Prod
|
||||
}
|
||||
|
||||
/*
|
||||
* Our staging cluster for external development and pre-releases. No SLAs are enforced.
|
||||
*/
|
||||
case object StagingCluster extends ClusterConfig {
|
||||
override val name: String = Constants.UuaKafkaStagingClusterName
|
||||
override val environment: EnvironmentConfig = Environments.Staging
|
||||
}
|
||||
}
|
|
@ -0,0 +1,10 @@
|
|||
package com.twitter.unified_user_actions.client.config
|
||||
|
||||
object Constants {
|
||||
val UuaKafkaTopicName = "unified_user_actions"
|
||||
val UuaEngagementOnlyKafkaTopicName = "unified_user_actions_engagements"
|
||||
val UuaKafkaProdClusterName = "/s/kafka/bluebird-1"
|
||||
val UuaKafkaStagingClusterName = "/s/kafka/custdevel"
|
||||
val UuaProdEnv = "prod"
|
||||
val UuaStagingEnv = "staging"
|
||||
}
|
|
@ -0,0 +1,15 @@
|
|||
package com.twitter.unified_user_actions.client.config
|
||||
|
||||
sealed trait EnvironmentConfig {
|
||||
val name: String
|
||||
}
|
||||
|
||||
object Environments {
|
||||
case object Prod extends EnvironmentConfig {
|
||||
override val name: String = Constants.UuaProdEnv
|
||||
}
|
||||
|
||||
case object Staging extends EnvironmentConfig {
|
||||
override val name: String = Constants.UuaStagingEnv
|
||||
}
|
||||
}
|
|
@ -0,0 +1,61 @@
|
|||
package com.twitter.unified_user_actions.client.config
|
||||
|
||||
sealed trait ClientConfig {
|
||||
val cluster: ClusterConfig
|
||||
val topic: String
|
||||
val environment: EnvironmentConfig
|
||||
}
|
||||
|
||||
class AbstractClientConfig(isEngagementOnly: Boolean, env: EnvironmentConfig) extends ClientConfig {
|
||||
override val cluster: ClusterConfig = {
|
||||
env match {
|
||||
case Environments.Prod => Clusters.ProdCluster
|
||||
case Environments.Staging => Clusters.StagingCluster
|
||||
case _ => Clusters.ProdCluster
|
||||
}
|
||||
}
|
||||
|
||||
override val topic: String = {
|
||||
if (isEngagementOnly) Constants.UuaEngagementOnlyKafkaTopicName
|
||||
else Constants.UuaKafkaTopicName
|
||||
}
|
||||
|
||||
override val environment: EnvironmentConfig = env
|
||||
}
|
||||
|
||||
object KafkaConfigs {
|
||||
|
||||
/*
|
||||
* Unified User Actions Kafka config with all events (engagements and impressions).
|
||||
* Use this config when you mainly need impression data and data volume is not an issue.
|
||||
*/
|
||||
case object ProdUnifiedUserActions
|
||||
extends AbstractClientConfig(isEngagementOnly = false, env = Environments.Prod)
|
||||
|
||||
/*
|
||||
* Unified User Actions Kafka config with engagements events only.
|
||||
* Use this config when you only need engagement data. The data volume should be a lot smaller
|
||||
* than our main config.
|
||||
*/
|
||||
case object ProdUnifiedUserActionsEngagementOnly
|
||||
extends AbstractClientConfig(isEngagementOnly = true, env = Environments.Prod)
|
||||
|
||||
/*
|
||||
* Staging Environment for integration and testing. This is not a production config.
|
||||
*
|
||||
* Unified User Actions Kafka config with all events (engagements and impressions).
|
||||
* Use this config when you mainly need impression data and data volume is not an issue.
|
||||
*/
|
||||
case object StagingUnifiedUserActions
|
||||
extends AbstractClientConfig(isEngagementOnly = false, env = Environments.Staging)
|
||||
|
||||
/*
|
||||
* Staging Environment for integration and testing. This is not a production config.
|
||||
*
|
||||
* Unified User Actions Kafka config with engagements events only.
|
||||
* Use this config when you only need engagement data. The data volume should be a lot smaller
|
||||
* than our main config.
|
||||
*/
|
||||
case object StagingUnifiedUserActionsEngagementOnly
|
||||
extends AbstractClientConfig(isEngagementOnly = true, env = Environments.Staging)
|
||||
}
|
|
@ -0,0 +1,21 @@
|
|||
scala_library(
|
||||
sources = [
|
||||
"UnifiedUserActionsSourceScrooge.scala",
|
||||
],
|
||||
compiler_option_sets = ["fatal_warnings"],
|
||||
# Our runtime is using Java 11, but for compatibility with other internal libraries that
|
||||
# are still on Java 8, we'll make our target platform to be Java 8 as well until everyone can
|
||||
# migrate.
|
||||
platform = "java8",
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"3rdparty/src/jvm/com/twitter/summingbird:core",
|
||||
"3rdparty/src/jvm/com/twitter/summingbird:storm",
|
||||
"3rdparty/src/jvm/com/twitter/tormenta:core",
|
||||
"src/scala/com/twitter/summingbird_internal/sources/common",
|
||||
"src/scala/com/twitter/tormenta_internal/scheme",
|
||||
"src/scala/com/twitter/tormenta_internal/spout:kafka2",
|
||||
"unified_user_actions/client/src/main/scala/com/twitter/unified_user_actions/client/config",
|
||||
"unified_user_actions/thrift/src/main/thrift/com/twitter/unified_user_actions:unified_user_actions-scala",
|
||||
],
|
||||
)
|
|
@ -0,0 +1,43 @@
|
|||
package com.twitter.unified_user_actions.client.summingbird
|
||||
|
||||
import com.twitter.summingbird.TimeExtractor
|
||||
import com.twitter.summingbird.storm.Storm
|
||||
import com.twitter.summingbird_internal.sources.AppId
|
||||
import com.twitter.summingbird_internal.sources.SourceFactory
|
||||
import com.twitter.tormenta_internal.spout.Kafka2ScroogeSpoutWrapper
|
||||
import com.twitter.unified_user_actions.client.config.ClientConfig
|
||||
import com.twitter.unified_user_actions.thriftscala.UnifiedUserAction
|
||||
import com.twitter.unified_user_actions.client.config.KafkaConfigs
|
||||
|
||||
case class UnifiedUserActionsSourceScrooge(
|
||||
appId: AppId,
|
||||
parallelism: Int,
|
||||
kafkaConfig: ClientConfig = KafkaConfigs.ProdUnifiedUserActions,
|
||||
skipToLatest: Boolean = false,
|
||||
enableTls: Boolean = true)
|
||||
extends SourceFactory[Storm, UnifiedUserAction] {
|
||||
|
||||
override def name: String = "UnifiedUserActionsSource"
|
||||
override def description: String = "Unified User Actions (UUA) events"
|
||||
|
||||
// The event timestamps from summingbird's perspective (client), is our internally
|
||||
// outputted timestamps (producer). This ensures time-continuity between the client and the
|
||||
// producer.
|
||||
val timeExtractor: TimeExtractor[UnifiedUserAction] = TimeExtractor { e =>
|
||||
e.eventMetadata.receivedTimestampMs
|
||||
}
|
||||
|
||||
override def source = {
|
||||
Storm.source(
|
||||
Kafka2ScroogeSpoutWrapper(
|
||||
codec = UnifiedUserAction,
|
||||
cluster = kafkaConfig.cluster.name,
|
||||
topic = kafkaConfig.topic,
|
||||
appId = appId.get,
|
||||
skipToLatest = skipToLatest,
|
||||
enableTls = enableTls
|
||||
),
|
||||
Some(parallelism)
|
||||
)(timeExtractor)
|
||||
}
|
||||
}
|
|
@ -0,0 +1,12 @@
|
|||
junit_tests(
|
||||
sources = ["**/*.scala"],
|
||||
compiler_option_sets = ["fatal_warnings"],
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"3rdparty/jvm/junit",
|
||||
"3rdparty/jvm/org/scalatest",
|
||||
"3rdparty/jvm/org/scalatestplus:junit",
|
||||
"finatra/inject/inject-core/src/test/scala:test-deps",
|
||||
"unified_user_actions/client/src/main/scala/com/twitter/unified_user_actions/client/config",
|
||||
],
|
||||
)
|
|
@ -0,0 +1,38 @@
|
|||
package com.twitter.unified_user_actions.client.config
|
||||
|
||||
import com.twitter.inject.Test
|
||||
|
||||
class KafkaConfigsSpec extends Test {
|
||||
test("configs should be correct") {
|
||||
val states = Seq(
|
||||
(
|
||||
KafkaConfigs.ProdUnifiedUserActions,
|
||||
Constants.UuaProdEnv,
|
||||
Constants.UuaKafkaTopicName,
|
||||
Constants.UuaKafkaProdClusterName),
|
||||
(
|
||||
KafkaConfigs.ProdUnifiedUserActionsEngagementOnly,
|
||||
Constants.UuaProdEnv,
|
||||
Constants.UuaEngagementOnlyKafkaTopicName,
|
||||
Constants.UuaKafkaProdClusterName),
|
||||
(
|
||||
KafkaConfigs.StagingUnifiedUserActions,
|
||||
Constants.UuaStagingEnv,
|
||||
Constants.UuaKafkaTopicName,
|
||||
Constants.UuaKafkaStagingClusterName),
|
||||
(
|
||||
KafkaConfigs.StagingUnifiedUserActionsEngagementOnly,
|
||||
Constants.UuaStagingEnv,
|
||||
Constants.UuaEngagementOnlyKafkaTopicName,
|
||||
Constants.UuaKafkaStagingClusterName)
|
||||
)
|
||||
|
||||
states.foreach {
|
||||
case (actual, expectedEnv, expectedTopic, expectedClusterName) =>
|
||||
assert(expectedEnv == actual.environment.name, s"in $actual")
|
||||
assert(expectedTopic == actual.topic, s"in $actual")
|
||||
assert(expectedClusterName == actual.cluster.name, s"in $actual")
|
||||
case _ =>
|
||||
}
|
||||
}
|
||||
}
|
1
unified_user_actions/enricher/BUILD.bazel
Normal file
1
unified_user_actions/enricher/BUILD.bazel
Normal file
|
@ -0,0 +1 @@
|
|||
# This prevents SQ query from grabbing //:all since it traverses up once to find a BUILD
|
Some files were not shown because too many files have changed in this diff Show more
Loading…
Reference in a new issue