mirror of
https://github.com/twitter/the-algorithm.git
synced 2025-06-13 12:58:23 +02:00
Twitter Recommendation Algorithm
Please note we have force-pushed a new initial commit in order to remove some publicly-available Twitter user information. Note that this process may be required in the future.
This commit is contained in:
9
src/scala/com/twitter/recos/decider/BUILD
Normal file
9
src/scala/com/twitter/recos/decider/BUILD
Normal file
@ -0,0 +1,9 @@
|
||||
scala_library(
|
||||
sources = ["*.scala"],
|
||||
platform = "java8",
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"decider/src/main/scala",
|
||||
"src/scala/com/twitter/recos/util:recos-util",
|
||||
],
|
||||
)
|
110
src/scala/com/twitter/recos/decider/BaseDecider.scala
Normal file
110
src/scala/com/twitter/recos/decider/BaseDecider.scala
Normal file
@ -0,0 +1,110 @@
|
||||
package com.twitter.recos.decider
|
||||
|
||||
import com.twitter.decider.Decider
|
||||
import com.twitter.decider.DeciderFactory
|
||||
import com.twitter.decider.RandomRecipient
|
||||
import com.twitter.decider.Recipient
|
||||
import com.twitter.decider.SimpleRecipient
|
||||
import com.twitter.recos.util.TeamUsers
|
||||
|
||||
case class GuestRecipient(id: Long) extends Recipient {
|
||||
override def isGuest: Boolean = true
|
||||
}
|
||||
|
||||
sealed trait BaseDecider {
|
||||
def baseConfig: Option[String] = None
|
||||
|
||||
def overlayConfig: Option[String] = None
|
||||
|
||||
lazy val decider: Decider = DeciderFactory(baseConfig, overlayConfig)()
|
||||
|
||||
def isAvailable(feature: String, recipient: Option[Recipient]): Boolean =
|
||||
decider.isAvailable(feature, recipient)
|
||||
|
||||
def isAvailable(feature: String): Boolean = isAvailable(feature, None)
|
||||
|
||||
def isAvailableExceptTeam(feature: String, id: Long, isUser: Boolean = true): Boolean = {
|
||||
if (isUser) TeamUsers.team.contains(id) || isAvailable(feature, Some(SimpleRecipient(id)))
|
||||
else isAvailable(feature, Some(GuestRecipient(id)))
|
||||
}
|
||||
}
|
||||
|
||||
case class RecosDecider(env: String, cluster: String = "atla") extends BaseDecider {
|
||||
override val baseConfig = Some("/com/twitter/recos/config/decider.yml")
|
||||
override val overlayConfig = Some(
|
||||
s"/usr/local/config/overlays/recos/service/prod/$cluster/decider_overlay.yml"
|
||||
)
|
||||
|
||||
def shouldCompute(id: Long, displayLocation: String, isUser: Boolean = true): Boolean = {
|
||||
isAvailableExceptTeam(RecosDecider.recosIncomingTraffic + "_" + displayLocation, id, isUser)
|
||||
}
|
||||
|
||||
def shouldReturn(id: Long, displayLocation: String, isUser: Boolean = true): Boolean = {
|
||||
isAvailableExceptTeam(RecosDecider.recosShouldReturn + "_" + displayLocation, id, isUser)
|
||||
}
|
||||
|
||||
def shouldDarkmode(experiment: String): Boolean = {
|
||||
isAvailable(RecosDecider.recosShouldDark + "_exp_" + experiment, None)
|
||||
}
|
||||
|
||||
def shouldScribe(id: Long, isUser: Boolean = true): Boolean = {
|
||||
if (isUser) (id > 0) && isAvailableExceptTeam(RecosDecider.recosShouldScribe, id, isUser)
|
||||
else false // TODO: define the behavior for guests
|
||||
}
|
||||
|
||||
def shouldWriteMomentCapsuleOpenEdge(): Boolean = {
|
||||
val capsuleOpenDecider = env match {
|
||||
case "prod" => RecosDecider.recosShouldWriteMomentCapsuleOpenEdge
|
||||
case _ => RecosDecider.recosShouldWriteMomentCapsuleOpenEdge + RecosDecider.testSuffix
|
||||
}
|
||||
|
||||
isAvailable(capsuleOpenDecider, Some(RandomRecipient))
|
||||
}
|
||||
}
|
||||
|
||||
object RecosDecider {
|
||||
val testSuffix = "_test"
|
||||
|
||||
val recosIncomingTraffic: String = "recos_incoming_traffic"
|
||||
val recosShouldReturn: String = "recos_should_return"
|
||||
val recosShouldDark: String = "recos_should_dark"
|
||||
val recosRealtimeBlacklist: String = "recos_realtime_blacklist"
|
||||
val recosRealtimeDeveloperlist: String = "recos_realtime_developerlist"
|
||||
val recosShouldScribe: String = "recos_should_scribe"
|
||||
val recosShouldWriteMomentCapsuleOpenEdge: String = "recos_should_write_moment_capsule_open_edge"
|
||||
}
|
||||
|
||||
trait GraphDecider extends BaseDecider {
|
||||
val graphNamePrefix: String
|
||||
|
||||
override val baseConfig = Some("/com/twitter/recos/config/decider.yml")
|
||||
override val overlayConfig = Some(
|
||||
"/usr/local/config/overlays/recos/service/prod/atla/decider_overlay.yml"
|
||||
)
|
||||
}
|
||||
|
||||
case class UserTweetEntityGraphDecider() extends GraphDecider {
|
||||
override val graphNamePrefix: String = "user_tweet_entity_graph"
|
||||
|
||||
def tweetSocialProof: Boolean = {
|
||||
isAvailable("user_tweet_entity_graph_tweet_social_proof")
|
||||
}
|
||||
|
||||
def entitySocialProof: Boolean = {
|
||||
isAvailable("user_tweet_entity_graph_entity_social_proof")
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
case class UserUserGraphDecider() extends GraphDecider {
|
||||
override val graphNamePrefix: String = "user_user_graph"
|
||||
}
|
||||
|
||||
case class UserTweetGraphDecider(env: String, dc: String) extends GraphDecider {
|
||||
override val graphNamePrefix: String = "user-tweet-graph"
|
||||
|
||||
override val baseConfig = Some("/com/twitter/recos/config/user-tweet-graph_decider.yml")
|
||||
override val overlayConfig = Some(
|
||||
s"/usr/local/config/overlays/user-tweet-graph/user-tweet-graph/$env/$dc/decider_overlay.yml"
|
||||
)
|
||||
}
|
@ -0,0 +1,39 @@
|
||||
package com.twitter.recos.decider
|
||||
|
||||
import com.twitter.decider.Decider
|
||||
import com.twitter.decider.RandomRecipient
|
||||
import com.twitter.util.Future
|
||||
import scala.util.control.NoStackTrace
|
||||
|
||||
/*
|
||||
Provides deciders-controlled load shedding for a given endpoint.
|
||||
The format of the decider keys is:
|
||||
|
||||
enable_loadshedding_<graphNamePrefix>_<endpoint name>
|
||||
E.g.:
|
||||
enable_loadshedding_user-tweet-graph_relatedTweets
|
||||
|
||||
Deciders are fractional, so a value of 50.00 will drop 50% of responses. If a decider key is not
|
||||
defined for a particular endpoint, those requests will always be
|
||||
served.
|
||||
|
||||
We should therefore aim to define keys for the endpoints we care most about in decider.yml,
|
||||
so that we can control them during incidents.
|
||||
*/
|
||||
class EndpointLoadShedder(
|
||||
decider: GraphDecider) {
|
||||
import EndpointLoadShedder._
|
||||
|
||||
private val keyPrefix = "enable_loadshedding"
|
||||
|
||||
def apply[T](endpointName: String)(serve: => Future[T]): Future[T] = {
|
||||
val key = s"${keyPrefix}_${decider.graphNamePrefix}_${endpointName}"
|
||||
if (decider.isAvailable(key, recipient = Some(RandomRecipient)))
|
||||
Future.exception(LoadSheddingException)
|
||||
else serve
|
||||
}
|
||||
}
|
||||
|
||||
object EndpointLoadShedder {
|
||||
object LoadSheddingException extends Exception with NoStackTrace
|
||||
}
|
@ -0,0 +1,99 @@
|
||||
package com.twitter.recos.graph_common
|
||||
|
||||
import com.twitter.graphjet.bipartite.api.EdgeTypeMask
|
||||
import com.twitter.recos.recos_common.thriftscala.SocialProofType
|
||||
|
||||
/**
|
||||
* The bit mask is used to encode edge types in the top bits of an integer,
|
||||
* e.g. favorite, retweet, reply and click. Under current segment configuration, each segment
|
||||
* stores up to 128M edges. Assuming that each node on one side is unique, each segment
|
||||
* stores up to 128M unique nodes on one side, which occupies the lower 27 bits of an integer.
|
||||
* This leaves five bits to encode the edge types, which at max can store 32 edge types.
|
||||
* The following implementation utilizes the top four bits and leaves one free bit out.
|
||||
*/
|
||||
class ActionEdgeTypeMask extends EdgeTypeMask {
|
||||
import ActionEdgeTypeMask._
|
||||
|
||||
override def encode(node: Int, edgeType: Byte): Int = {
|
||||
if (edgeType == FAVORITE) {
|
||||
node | EDGEARRAY(FAVORITE)
|
||||
} else if (edgeType == RETWEET) {
|
||||
node | EDGEARRAY(RETWEET)
|
||||
} else if (edgeType == REPLY) {
|
||||
node | EDGEARRAY(REPLY)
|
||||
} else if (edgeType == TWEET) {
|
||||
node | EDGEARRAY(TWEET)
|
||||
} else {
|
||||
// Anything that is not a public engagement (i.e. openlink, share, select, etc.) is a "click"
|
||||
node | EDGEARRAY(CLICK)
|
||||
}
|
||||
}
|
||||
|
||||
override def edgeType(node: Int): Byte = {
|
||||
(node >> 28).toByte
|
||||
}
|
||||
|
||||
override def restore(node: Int): Int = {
|
||||
node & MASK
|
||||
}
|
||||
}
|
||||
|
||||
object ActionEdgeTypeMask {
|
||||
|
||||
/**
|
||||
* Reserve the top four bits of each integer to encode the edge type information.
|
||||
*/
|
||||
val MASK: Int =
|
||||
Integer.parseInt("00001111111111111111111111111111", 2)
|
||||
val CLICK: Byte = 0
|
||||
val FAVORITE: Byte = 1
|
||||
val RETWEET: Byte = 2
|
||||
val REPLY: Byte = 3
|
||||
val TWEET: Byte = 4
|
||||
val SIZE: Byte = 5
|
||||
val UNUSED6: Byte = 6
|
||||
val UNUSED7: Byte = 7
|
||||
val UNUSED8: Byte = 8
|
||||
val UNUSED9: Byte = 9
|
||||
val UNUSED10: Byte = 10
|
||||
val UNUSED11: Byte = 11
|
||||
val UNUSED12: Byte = 12
|
||||
val UNUSED13: Byte = 13
|
||||
val UNUSED14: Byte = 14
|
||||
val UNUSED15: Byte = 15
|
||||
val EDGEARRAY: Array[Int] = Array(
|
||||
0,
|
||||
1 << 28,
|
||||
2 << 28,
|
||||
3 << 28,
|
||||
4 << 28,
|
||||
5 << 28,
|
||||
6 << 28,
|
||||
7 << 28,
|
||||
8 << 28,
|
||||
9 << 28,
|
||||
10 << 28,
|
||||
11 << 28,
|
||||
12 << 28,
|
||||
13 << 28,
|
||||
14 << 28,
|
||||
15 << 28
|
||||
)
|
||||
|
||||
/**
|
||||
* Map valid social proof types specified by clients to an array of bytes. If clients do not
|
||||
* specify any social proof types in thrift, it will return all available social types by
|
||||
* default.
|
||||
*
|
||||
* @param socialProofTypes are the valid socialProofTypes specified by clients
|
||||
* @return an array of bytes representing valid social proof types
|
||||
*/
|
||||
def getUserTweetGraphSocialProofTypes(
|
||||
socialProofTypes: Option[Seq[SocialProofType]]
|
||||
): Array[Byte] = {
|
||||
socialProofTypes
|
||||
.map { _.map { _.getValue }.toArray }
|
||||
.getOrElse((0 until SIZE).toArray)
|
||||
.map { _.toByte }
|
||||
}
|
||||
}
|
12
src/scala/com/twitter/recos/graph_common/BUILD
Normal file
12
src/scala/com/twitter/recos/graph_common/BUILD
Normal file
@ -0,0 +1,12 @@
|
||||
scala_library(
|
||||
sources = ["*.scala"],
|
||||
platform = "java8",
|
||||
strict_deps = False,
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"3rdparty/jvm/com/twitter/graphjet",
|
||||
"finagle/finagle-stats/src/main/scala",
|
||||
"src/scala/com/twitter/recos/util:recos-util",
|
||||
"src/thrift/com/twitter/recos:recos-common-scala",
|
||||
],
|
||||
)
|
@ -0,0 +1,40 @@
|
||||
package com.twitter.recos.graph_common
|
||||
|
||||
import com.twitter.graphjet.algorithms.TweetIDMask
|
||||
import com.twitter.graphjet.bipartite.api.BipartiteGraph
|
||||
import scala.collection.mutable.ListBuffer
|
||||
|
||||
/*
|
||||
* The helper class encodes and decodes tweet ids with tweetypie's card information
|
||||
* when querying recos salsa library. Inside salsa library, all tweet ids are
|
||||
* encoded with card information for the purpose of inline filtering.
|
||||
*/
|
||||
class BipartiteGraphHelper(graph: BipartiteGraph) {
|
||||
private val tweetIDMask = new TweetIDMask
|
||||
|
||||
def getLeftNodeEdges(leftNode: Long): Seq[(Long, Byte)] = {
|
||||
val iterator = graph.getLeftNodeEdges(leftNode)
|
||||
|
||||
val edges: ListBuffer[(Long, Byte)] = ListBuffer()
|
||||
if (iterator != null) {
|
||||
while (iterator.hasNext) {
|
||||
val node = iterator.nextLong()
|
||||
val engagementType = iterator.currentEdgeType()
|
||||
edges += ((tweetIDMask.restore(node), engagementType))
|
||||
}
|
||||
}
|
||||
edges.reverse.distinct // Most recent edges first, no duplications
|
||||
}
|
||||
|
||||
def getRightNodeEdges(rightNode: Long): Seq[Long] = {
|
||||
val iterator = graph.getRightNodeEdges(rightNode)
|
||||
val leftNodes: ListBuffer[Long] = ListBuffer()
|
||||
if (iterator != null) {
|
||||
while (iterator.hasNext) {
|
||||
leftNodes += iterator.nextLong()
|
||||
}
|
||||
}
|
||||
|
||||
leftNodes.reverse.distinct // Most recent edges first, no duplications
|
||||
}
|
||||
}
|
@ -0,0 +1,15 @@
|
||||
package com.twitter.recos.graph_common
|
||||
|
||||
import com.twitter.finagle.stats.Counter
|
||||
import com.twitter.graphjet.stats.{Counter => GraphCounter}
|
||||
|
||||
/**
|
||||
* FinagleCounterWrapper wraps Twitter's Finagle Counter.
|
||||
*
|
||||
* This is because GraphJet is an openly available library which does not
|
||||
* depend on Finagle, but tracks stats using a similar interface.
|
||||
*/
|
||||
class FinagleCounterWrapper(counter: Counter) extends GraphCounter {
|
||||
def incr() = counter.incr()
|
||||
def incr(delta: Int) = counter.incr(delta)
|
||||
}
|
@ -0,0 +1,16 @@
|
||||
package com.twitter.recos.graph_common
|
||||
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.graphjet.stats.{StatsReceiver => GraphStatsReceiver}
|
||||
|
||||
/**
|
||||
* FinagleStatsReceiverWrapper wraps Twitter's Finagle StatsReceiver.
|
||||
*
|
||||
* This is because GraphJet is an openly available library which does not
|
||||
* depend on Finagle, but tracks stats using a similar interface.
|
||||
*/
|
||||
case class FinagleStatsReceiverWrapper(statsReceiver: StatsReceiver) extends GraphStatsReceiver {
|
||||
|
||||
def scope(namespace: String) = new FinagleStatsReceiverWrapper(statsReceiver.scope(namespace))
|
||||
def counter(name: String) = new FinagleCounterWrapper(statsReceiver.counter(name))
|
||||
}
|
@ -0,0 +1,59 @@
|
||||
package com.twitter.recos.graph_common
|
||||
|
||||
import com.twitter.graphjet.bipartite.LeftIndexedPowerLawMultiSegmentBipartiteGraph
|
||||
import com.twitter.graphjet.bipartite.api.EdgeTypeMask
|
||||
import com.twitter.graphjet.stats.StatsReceiver
|
||||
|
||||
/**
|
||||
* The GraphBuilder builds a LeftIndexedPowerLawMultiSegmentBipartiteGraph given a set of
|
||||
* parameters.
|
||||
*/
|
||||
object LeftIndexedPowerLawMultiSegmentBipartiteGraphBuilder {
|
||||
|
||||
/**
|
||||
* This encapsulates all the state needed to initialize the in-memory graph.
|
||||
*
|
||||
* @param maxNumSegments is the maximum number of segments we'll add to the graph.
|
||||
* At that point, the oldest segments will start getting dropped
|
||||
* @param maxNumEdgesPerSegment determines when the implementation decides to fork off a
|
||||
* new segment
|
||||
* @param expectedNumLeftNodes is the expected number of left nodes that would be inserted in
|
||||
* the segment
|
||||
* @param expectedMaxLeftDegree is the maximum degree expected for any left node
|
||||
* @param leftPowerLawExponent is the exponent of the LHS power-law graph. see
|
||||
* [[com.twitter.graphjet.bipartite.edgepool.PowerLawDegreeEdgePool]]
|
||||
* for details
|
||||
* @param expectedNumRightNodes is the expected number of right nodes that would be inserted in
|
||||
* the segment
|
||||
*/
|
||||
case class GraphBuilderConfig(
|
||||
maxNumSegments: Int,
|
||||
maxNumEdgesPerSegment: Int,
|
||||
expectedNumLeftNodes: Int,
|
||||
expectedMaxLeftDegree: Int,
|
||||
leftPowerLawExponent: Double,
|
||||
expectedNumRightNodes: Int,
|
||||
edgeTypeMask: EdgeTypeMask)
|
||||
|
||||
/**
|
||||
* This apply function returns a mutuable bipartiteGraph
|
||||
*
|
||||
* @param graphBuilderConfig is the graph builder config
|
||||
*
|
||||
*/
|
||||
def apply(
|
||||
graphBuilderConfig: GraphBuilderConfig,
|
||||
statsReceiverWrapper: StatsReceiver
|
||||
): LeftIndexedPowerLawMultiSegmentBipartiteGraph = {
|
||||
new LeftIndexedPowerLawMultiSegmentBipartiteGraph(
|
||||
graphBuilderConfig.maxNumSegments,
|
||||
graphBuilderConfig.maxNumEdgesPerSegment,
|
||||
graphBuilderConfig.expectedNumLeftNodes,
|
||||
graphBuilderConfig.expectedMaxLeftDegree,
|
||||
graphBuilderConfig.leftPowerLawExponent,
|
||||
graphBuilderConfig.expectedNumRightNodes,
|
||||
graphBuilderConfig.edgeTypeMask,
|
||||
statsReceiverWrapper
|
||||
)
|
||||
}
|
||||
}
|
@ -0,0 +1,64 @@
|
||||
package com.twitter.recos.graph_common
|
||||
|
||||
import com.twitter.graphjet.stats.StatsReceiver
|
||||
import com.twitter.graphjet.bipartite.MultiSegmentPowerLawBipartiteGraph
|
||||
|
||||
/**
|
||||
* The GraphBuilder builds a MultiSegmentPowerLawBipartiteGraph given a set of parameters.
|
||||
*/
|
||||
object MultiSegmentPowerLawBipartiteGraphBuilder {
|
||||
|
||||
/**
|
||||
* This encapsulates all the state needed to initialize the in-memory graph.
|
||||
*
|
||||
* @param maxNumSegments is the maximum number of segments we'll add to the graph.
|
||||
* At that point, the oldest segments will start getting dropped
|
||||
* @param maxNumEdgesPerSegment determines when the implementation decides to fork off a
|
||||
* new segment
|
||||
* @param expectedNumLeftNodes is the expected number of left nodes that would be inserted in
|
||||
* the segment
|
||||
* @param expectedMaxLeftDegree is the maximum degree expected for any left node
|
||||
* @param leftPowerLawExponent is the exponent of the LHS power-law graph. see
|
||||
* [[com.twitter.graphjet.bipartite.edgepool.PowerLawDegreeEdgePool]]
|
||||
* for details
|
||||
* @param expectedNumRightNodes is the expected number of right nodes that would be inserted in
|
||||
* the segment
|
||||
* @param expectedMaxRightDegree is the maximum degree expected for any right node
|
||||
* @param rightPowerLawExponent is the exponent of the RHS power-law graph. see
|
||||
* [[com.twitter.graphjet.bipartite.edgepool.PowerLawDegreeEdgePool]]
|
||||
* for details
|
||||
*/
|
||||
case class GraphBuilderConfig(
|
||||
maxNumSegments: Int,
|
||||
maxNumEdgesPerSegment: Int,
|
||||
expectedNumLeftNodes: Int,
|
||||
expectedMaxLeftDegree: Int,
|
||||
leftPowerLawExponent: Double,
|
||||
expectedNumRightNodes: Int,
|
||||
expectedMaxRightDegree: Int,
|
||||
rightPowerLawExponent: Double)
|
||||
|
||||
/**
|
||||
* This apply function returns a mutuable bipartiteGraph
|
||||
*
|
||||
* @param graphBuilderConfig is the graph builder config
|
||||
*
|
||||
*/
|
||||
def apply(
|
||||
graphBuilderConfig: GraphBuilderConfig,
|
||||
statsReceiver: StatsReceiver
|
||||
): MultiSegmentPowerLawBipartiteGraph = {
|
||||
new MultiSegmentPowerLawBipartiteGraph(
|
||||
graphBuilderConfig.maxNumSegments,
|
||||
graphBuilderConfig.maxNumEdgesPerSegment,
|
||||
graphBuilderConfig.expectedNumLeftNodes,
|
||||
graphBuilderConfig.expectedMaxLeftDegree,
|
||||
graphBuilderConfig.leftPowerLawExponent,
|
||||
graphBuilderConfig.expectedNumRightNodes,
|
||||
graphBuilderConfig.expectedMaxRightDegree,
|
||||
graphBuilderConfig.rightPowerLawExponent,
|
||||
new ActionEdgeTypeMask(),
|
||||
statsReceiver
|
||||
)
|
||||
}
|
||||
}
|
@ -0,0 +1,59 @@
|
||||
package com.twitter.recos.graph_common
|
||||
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.recos.recos_common.thriftscala.{
|
||||
SocialProofType,
|
||||
GetRecentEdgesRequest,
|
||||
GetRecentEdgesResponse,
|
||||
NodeInfo,
|
||||
RecentEdge
|
||||
}
|
||||
import com.twitter.recos.util.Stats._
|
||||
import com.twitter.servo.request._
|
||||
import com.twitter.util.Future
|
||||
|
||||
/**
|
||||
* Implementation of the Thrift-defined service interface.
|
||||
*/
|
||||
class LeftNodeEdgesHandler(graphHelper: BipartiteGraphHelper, statsReceiver: StatsReceiver)
|
||||
extends RequestHandler[GetRecentEdgesRequest, GetRecentEdgesResponse] {
|
||||
private val stats = statsReceiver.scope(this.getClass.getSimpleName)
|
||||
|
||||
private val CLICK = 0
|
||||
private val FAVORITE = 1
|
||||
private val RETWEET = 2
|
||||
private val REPLY = 3
|
||||
private val TWEET = 4
|
||||
|
||||
override def apply(request: GetRecentEdgesRequest): Future[GetRecentEdgesResponse] = {
|
||||
trackFutureBlockStats(stats) {
|
||||
val recentEdges = graphHelper.getLeftNodeEdges(request.requestId).flatMap {
|
||||
case (node, engagementType) if engagementType == CLICK =>
|
||||
Some(RecentEdge(node, SocialProofType.Click))
|
||||
case (node, engagementType) if engagementType == FAVORITE =>
|
||||
Some(RecentEdge(node, SocialProofType.Favorite))
|
||||
case (node, engagementType) if engagementType == RETWEET =>
|
||||
Some(RecentEdge(node, SocialProofType.Retweet))
|
||||
case (node, engagementType) if engagementType == REPLY =>
|
||||
Some(RecentEdge(node, SocialProofType.Reply))
|
||||
case (node, engagementType) if engagementType == TWEET =>
|
||||
Some(RecentEdge(node, SocialProofType.Tweet))
|
||||
case _ =>
|
||||
None
|
||||
}
|
||||
Future.value(GetRecentEdgesResponse(recentEdges))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class RightNodeInfoHandler(graphHelper: BipartiteGraphHelper, statsReceiver: StatsReceiver)
|
||||
extends RequestHandler[Long, NodeInfo] {
|
||||
private[this] val stats = statsReceiver.scope(this.getClass.getSimpleName)
|
||||
|
||||
override def apply(rightNode: Long): Future[NodeInfo] = {
|
||||
trackFutureBlockStats(stats) {
|
||||
val edges = graphHelper.getRightNodeEdges(rightNode)
|
||||
Future.value(NodeInfo(edges = edges))
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,63 @@
|
||||
package com.twitter.recos.graph_common
|
||||
|
||||
import com.twitter.graphjet.bipartite.api.EdgeTypeMask
|
||||
import com.twitter.graphjet.bipartite.NodeMetadataLeftIndexedPowerLawMultiSegmentBipartiteGraph
|
||||
import com.twitter.graphjet.stats.StatsReceiver
|
||||
|
||||
/**
|
||||
* The GraphBuilder builds a NodeMetadataLeftIndexedPowerLawMultiSegmentBipartiteGraphBuilder given a set of
|
||||
* parameters.
|
||||
*/
|
||||
object NodeMetadataLeftIndexedPowerLawMultiSegmentBipartiteGraphBuilder {
|
||||
|
||||
/**
|
||||
* This encapsulates all the state needed to initialize the in-memory graph.
|
||||
*
|
||||
* @param maxNumSegments is the maximum number of segments we'll add to the graph.
|
||||
* At that point, the oldest segments will start getting dropped
|
||||
* @param maxNumEdgesPerSegment determines when the implementation decides to fork off a
|
||||
* new segment
|
||||
* @param expectedNumLeftNodes is the expected number of left nodes that would be inserted in
|
||||
* the segment
|
||||
* @param expectedMaxLeftDegree is the maximum degree expected for any left node
|
||||
* @param leftPowerLawExponent is the exponent of the LHS power-law graph. see
|
||||
* [[com.twitter.graphjet.bipartite.edgepool.PowerLawDegreeEdgePool]]
|
||||
* for details
|
||||
* @param expectedNumRightNodes is the expected number of right nodes that would be inserted in
|
||||
* the segment
|
||||
* @param numRightNodeMetadataTypes is the max number of node metadata types associated with the
|
||||
* right nodes
|
||||
*/
|
||||
case class GraphBuilderConfig(
|
||||
maxNumSegments: Int,
|
||||
maxNumEdgesPerSegment: Int,
|
||||
expectedNumLeftNodes: Int,
|
||||
expectedMaxLeftDegree: Int,
|
||||
leftPowerLawExponent: Double,
|
||||
expectedNumRightNodes: Int,
|
||||
numRightNodeMetadataTypes: Int,
|
||||
edgeTypeMask: EdgeTypeMask)
|
||||
|
||||
/**
|
||||
* This apply function returns a mutuable bipartiteGraph
|
||||
*
|
||||
* @param graphBuilderConfig is the graph builder config
|
||||
*
|
||||
*/
|
||||
def apply(
|
||||
graphBuilderConfig: GraphBuilderConfig,
|
||||
statsReceiverWrapper: StatsReceiver
|
||||
): NodeMetadataLeftIndexedPowerLawMultiSegmentBipartiteGraph = {
|
||||
new NodeMetadataLeftIndexedPowerLawMultiSegmentBipartiteGraph(
|
||||
graphBuilderConfig.maxNumSegments,
|
||||
graphBuilderConfig.maxNumEdgesPerSegment,
|
||||
graphBuilderConfig.expectedNumLeftNodes,
|
||||
graphBuilderConfig.expectedMaxLeftDegree,
|
||||
graphBuilderConfig.leftPowerLawExponent,
|
||||
graphBuilderConfig.expectedNumRightNodes,
|
||||
graphBuilderConfig.numRightNodeMetadataTypes,
|
||||
graphBuilderConfig.edgeTypeMask,
|
||||
statsReceiverWrapper
|
||||
)
|
||||
}
|
||||
}
|
@ -0,0 +1,63 @@
|
||||
package com.twitter.recos.graph_common
|
||||
|
||||
import com.twitter.graphjet.bipartite.RightNodeMetadataLeftIndexedPowerLawMultiSegmentBipartiteGraph
|
||||
import com.twitter.graphjet.bipartite.api.EdgeTypeMask
|
||||
import com.twitter.graphjet.stats.StatsReceiver
|
||||
|
||||
/**
|
||||
* The GraphBuilder builds a RightNodeMetadataLeftIndexedPowerLawMultiSegmentBipartiteGraphBuilder given a set of
|
||||
* parameters.
|
||||
*/
|
||||
object RightNodeMetadataLeftIndexedPowerLawMultiSegmentBipartiteGraphBuilder {
|
||||
|
||||
/**
|
||||
* This encapsulates all the state needed to initialize the in-memory graph.
|
||||
*
|
||||
* @param maxNumSegments is the maximum number of segments we'll add to the graph.
|
||||
* At that point, the oldest segments will start getting dropped
|
||||
* @param maxNumEdgesPerSegment determines when the implementation decides to fork off a
|
||||
* new segment
|
||||
* @param expectedNumLeftNodes is the expected number of left nodes that would be inserted in
|
||||
* the segment
|
||||
* @param expectedMaxLeftDegree is the maximum degree expected for any left node
|
||||
* @param leftPowerLawExponent is the exponent of the LHS power-law graph. see
|
||||
* [[com.twitter.graphjet.bipartite.edgepool.PowerLawDegreeEdgePool]]
|
||||
* for details
|
||||
* @param expectedNumRightNodes is the expected number of right nodes that would be inserted in
|
||||
* the segment
|
||||
* @param numRightNodeMetadataTypes is the max number of node metadata types associated with the
|
||||
* right nodes
|
||||
*/
|
||||
case class GraphBuilderConfig(
|
||||
maxNumSegments: Int,
|
||||
maxNumEdgesPerSegment: Int,
|
||||
expectedNumLeftNodes: Int,
|
||||
expectedMaxLeftDegree: Int,
|
||||
leftPowerLawExponent: Double,
|
||||
expectedNumRightNodes: Int,
|
||||
numRightNodeMetadataTypes: Int,
|
||||
edgeTypeMask: EdgeTypeMask)
|
||||
|
||||
/**
|
||||
* This apply function returns a mutuable bipartiteGraph
|
||||
*
|
||||
* @param graphBuilderConfig is the graph builder config
|
||||
*
|
||||
*/
|
||||
def apply(
|
||||
graphBuilderConfig: GraphBuilderConfig,
|
||||
statsReceiverWrapper: StatsReceiver
|
||||
): RightNodeMetadataLeftIndexedPowerLawMultiSegmentBipartiteGraph = {
|
||||
new RightNodeMetadataLeftIndexedPowerLawMultiSegmentBipartiteGraph(
|
||||
graphBuilderConfig.maxNumSegments,
|
||||
graphBuilderConfig.maxNumEdgesPerSegment,
|
||||
graphBuilderConfig.expectedNumLeftNodes,
|
||||
graphBuilderConfig.expectedMaxLeftDegree,
|
||||
graphBuilderConfig.leftPowerLawExponent,
|
||||
graphBuilderConfig.expectedNumRightNodes,
|
||||
graphBuilderConfig.numRightNodeMetadataTypes,
|
||||
graphBuilderConfig.edgeTypeMask,
|
||||
statsReceiverWrapper
|
||||
)
|
||||
}
|
||||
}
|
15
src/scala/com/twitter/recos/hose/common/BUILD
Normal file
15
src/scala/com/twitter/recos/hose/common/BUILD
Normal file
@ -0,0 +1,15 @@
|
||||
scala_library(
|
||||
sources = ["*.scala"],
|
||||
strict_deps = False,
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"3rdparty/jvm/com/twitter/graphjet",
|
||||
"3rdparty/jvm/org/apache/kafka:rosette-kafka",
|
||||
"finagle/finagle-stats/src/main/scala",
|
||||
"kafka/finagle-kafka/finatra-kafka/src/main/scala",
|
||||
"kafka/libs/src/main/scala/com/twitter/kafka/client/processor",
|
||||
"servo/repo/src/main/scala",
|
||||
"src/scala/com/twitter/recos/util:recos-util",
|
||||
"src/thrift/com/twitter/recos:recos-internal-scala",
|
||||
],
|
||||
)
|
@ -0,0 +1,48 @@
|
||||
package com.twitter.recos.hose.common
|
||||
|
||||
import com.twitter.finagle.stats.{Stat, StatsReceiver}
|
||||
import com.twitter.logging.Logger
|
||||
import com.twitter.recos.internal.thriftscala.RecosHoseMessage
|
||||
import java.util.concurrent.Semaphore
|
||||
|
||||
/**
|
||||
* This class reads a buffer of edges from the concurrently linked queue
|
||||
* and inserts each edge into the recos graph.
|
||||
* If the queue is empty the thread will sleep for 100ms and attempt to read from the queue again.
|
||||
*/
|
||||
case class BufferedEdgeWriter(
|
||||
queue: java.util.Queue[Array[RecosHoseMessage]],
|
||||
queuelimit: Semaphore,
|
||||
edgeCollector: EdgeCollector,
|
||||
statsReceiver: StatsReceiver,
|
||||
isRunning: () => Boolean)
|
||||
extends Runnable {
|
||||
val logger = Logger()
|
||||
private val queueRemoveCounter = statsReceiver.counter("queueRemove")
|
||||
private val queueSleepCounter = statsReceiver.counter("queueSleep")
|
||||
|
||||
def running: Boolean = {
|
||||
isRunning()
|
||||
}
|
||||
|
||||
override def run(): Unit = {
|
||||
while (running) {
|
||||
val currentBatch = queue.poll
|
||||
if (currentBatch != null) {
|
||||
queueRemoveCounter.incr()
|
||||
queuelimit.release()
|
||||
var i = 0
|
||||
Stat.time(statsReceiver.stat("batchAddEdge")) {
|
||||
while (i < currentBatch.length) {
|
||||
edgeCollector.addEdge(currentBatch(i))
|
||||
i = i + 1
|
||||
}
|
||||
}
|
||||
} else {
|
||||
queueSleepCounter.incr()
|
||||
Thread.sleep(100L)
|
||||
}
|
||||
}
|
||||
logger.info(this.getClass.getSimpleName + " is done")
|
||||
}
|
||||
}
|
42
src/scala/com/twitter/recos/hose/common/EdgeCollector.scala
Normal file
42
src/scala/com/twitter/recos/hose/common/EdgeCollector.scala
Normal file
@ -0,0 +1,42 @@
|
||||
package com.twitter.recos.hose.common
|
||||
|
||||
import com.twitter.finagle.stats.{Stat, StatsReceiver}
|
||||
import com.twitter.recos.internal.thriftscala.RecosHoseMessage
|
||||
import java.util.concurrent.Semaphore
|
||||
|
||||
trait EdgeCollector {
|
||||
def addEdge(message: RecosHoseMessage): Unit
|
||||
}
|
||||
|
||||
/**
|
||||
* The class consumes incoming edges and inserts them into a buffer of a specified bufferSize.
|
||||
* Once the buffer is full of edges, it is written to a concurrently linked queue where the size is bounded by queuelimit.
|
||||
*/
|
||||
case class BufferedEdgeCollector(
|
||||
bufferSize: Int,
|
||||
queue: java.util.Queue[Array[RecosHoseMessage]],
|
||||
queuelimit: Semaphore,
|
||||
statsReceiver: StatsReceiver)
|
||||
extends EdgeCollector {
|
||||
|
||||
private var buffer = new Array[RecosHoseMessage](bufferSize)
|
||||
private var index = 0
|
||||
private val queueAddCounter = statsReceiver.counter("queueAdd")
|
||||
|
||||
override def addEdge(message: RecosHoseMessage): Unit = {
|
||||
buffer(index) = message
|
||||
index = index + 1
|
||||
if (index >= bufferSize) {
|
||||
val oldBuffer = buffer
|
||||
buffer = new Array[RecosHoseMessage](bufferSize)
|
||||
index = 0
|
||||
|
||||
Stat.time(statsReceiver.stat("waitEnqueue")) {
|
||||
queuelimit.acquireUninterruptibly()
|
||||
}
|
||||
|
||||
queue.add(oldBuffer)
|
||||
queueAddCounter.incr()
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,41 @@
|
||||
package com.twitter.recos.hose.common
|
||||
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.recos.internal.thriftscala.RecosHoseMessage
|
||||
import com.twitter.util.Future
|
||||
import org.apache.kafka.clients.consumer.ConsumerRecord
|
||||
|
||||
/**
|
||||
* The class processes RecosHoseMessage and inserts the message as an edge into a recos graph.
|
||||
*/
|
||||
case class RecosEdgeProcessor(
|
||||
edgeCollector: EdgeCollector
|
||||
)(
|
||||
implicit statsReceiver: StatsReceiver) {
|
||||
|
||||
private val scopedStats = statsReceiver.scope("RecosEdgeProcessor")
|
||||
|
||||
private val processEventsCounter = scopedStats.counter("process_events")
|
||||
private val nullPointerEventCounter = scopedStats.counter("null_pointer_num")
|
||||
private val errorCounter = scopedStats.counter("process_errors")
|
||||
|
||||
def process(record: ConsumerRecord[String, RecosHoseMessage]): Future[Unit] = {
|
||||
processEventsCounter.incr()
|
||||
val message = record.value()
|
||||
try {
|
||||
// the message is nullable
|
||||
if (message != null) {
|
||||
edgeCollector.addEdge(message)
|
||||
} else {
|
||||
nullPointerEventCounter.incr()
|
||||
}
|
||||
Future.Unit
|
||||
} catch {
|
||||
case e: Throwable =>
|
||||
errorCounter.incr()
|
||||
e.printStackTrace()
|
||||
Future.Unit
|
||||
}
|
||||
}
|
||||
|
||||
}
|
217
src/scala/com/twitter/recos/hose/common/UnifiedGraphWriter.scala
Normal file
217
src/scala/com/twitter/recos/hose/common/UnifiedGraphWriter.scala
Normal file
@ -0,0 +1,217 @@
|
||||
package com.twitter.recos.hose.common
|
||||
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.finatra.kafka.consumers.FinagleKafkaConsumerBuilder
|
||||
import com.twitter.graphjet.bipartite.LeftIndexedMultiSegmentBipartiteGraph
|
||||
import com.twitter.graphjet.bipartite.segment.LeftIndexedBipartiteGraphSegment
|
||||
import com.twitter.kafka.client.processor.{AtLeastOnceProcessor, ThreadSafeKafkaConsumerClient}
|
||||
import com.twitter.logging.Logger
|
||||
import com.twitter.recos.internal.thriftscala.RecosHoseMessage
|
||||
import java.util.concurrent.atomic.AtomicBoolean
|
||||
import java.util.concurrent.{ConcurrentLinkedQueue, ExecutorService, Executors, Semaphore}
|
||||
|
||||
/**
|
||||
* The class submits a number of graph writer threads, BufferedEdgeWriter,
|
||||
* during service startup. One of them is live writer thread, and the other $(numBootstrapWriters - 1)
|
||||
* are catchup writer threads. All of them consume kafka events from an internal concurrent queue,
|
||||
* which is populated by kafka reader threads. At bootstrap time, the kafka reader threads look
|
||||
* back kafka offset from several hours ago and populate the internal concurrent queue.
|
||||
* Each graph writer thread writes to an individual graph segment separately.
|
||||
* The (numBootstrapWriters - 1) catchup writer threads will stop once all events
|
||||
* between current system time at startup and the time in memcache are processed.
|
||||
* The live writer thread will continue to write all incoming kafka events.
|
||||
* It lives through the entire life cycle of recos graph service.
|
||||
*/
|
||||
trait UnifiedGraphWriter[
|
||||
TSegment <: LeftIndexedBipartiteGraphSegment,
|
||||
TGraph <: LeftIndexedMultiSegmentBipartiteGraph[TSegment]] { writer =>
|
||||
|
||||
import UnifiedGraphWriter._
|
||||
|
||||
def shardId: String
|
||||
def env: String
|
||||
def hosename: String
|
||||
def bufferSize: Int
|
||||
def consumerNum: Int
|
||||
def catchupWriterNum: Int
|
||||
def kafkaConsumerBuilder: FinagleKafkaConsumerBuilder[String, RecosHoseMessage]
|
||||
def clientId: String
|
||||
def statsReceiver: StatsReceiver
|
||||
|
||||
/**
|
||||
* Adds a RecosHoseMessage to the graph. used by live writer to insert edges to the
|
||||
* current segment
|
||||
*/
|
||||
def addEdgeToGraph(graph: TGraph, recosHoseMessage: RecosHoseMessage): Unit
|
||||
|
||||
/**
|
||||
* Adds a RecosHoseMessage to the given segment in the graph. Used by catch up writers to
|
||||
* insert edges to non-current (old) segments
|
||||
*/
|
||||
def addEdgeToSegment(segment: TSegment, recosHoseMessage: RecosHoseMessage): Unit
|
||||
|
||||
private val log = Logger()
|
||||
private val isRunning: AtomicBoolean = new AtomicBoolean(true)
|
||||
private val initialized: AtomicBoolean = new AtomicBoolean(false)
|
||||
private var processors: Seq[AtLeastOnceProcessor[String, RecosHoseMessage]] = Seq.empty
|
||||
private var consumers: Seq[ThreadSafeKafkaConsumerClient[String, RecosHoseMessage]] = Seq.empty
|
||||
private val threadPool: ExecutorService = Executors.newCachedThreadPool()
|
||||
|
||||
def shutdown(): Unit = {
|
||||
processors.foreach { processor =>
|
||||
processor.close()
|
||||
}
|
||||
processors = Seq.empty
|
||||
consumers.foreach { consumer =>
|
||||
consumer.close()
|
||||
}
|
||||
consumers = Seq.empty
|
||||
threadPool.shutdown()
|
||||
isRunning.set(false)
|
||||
}
|
||||
|
||||
def initHose(liveGraph: TGraph): Unit = this.synchronized {
|
||||
if (!initialized.get) {
|
||||
initialized.set(true)
|
||||
|
||||
val queue: java.util.Queue[Array[RecosHoseMessage]] =
|
||||
new ConcurrentLinkedQueue[Array[RecosHoseMessage]]()
|
||||
val queuelimit: Semaphore = new Semaphore(1024)
|
||||
|
||||
initRecosHoseKafka(queue, queuelimit)
|
||||
initGrpahWriters(liveGraph, queue, queuelimit)
|
||||
} else {
|
||||
throw new RuntimeException("attempt to re-init kafka hose")
|
||||
}
|
||||
}
|
||||
|
||||
private def initRecosHoseKafka(
|
||||
queue: java.util.Queue[Array[RecosHoseMessage]],
|
||||
queuelimit: Semaphore,
|
||||
): Unit = {
|
||||
try {
|
||||
consumers = (0 until consumerNum).map { index =>
|
||||
new ThreadSafeKafkaConsumerClient(
|
||||
kafkaConsumerBuilder.clientId(s"clientId-$index").enableAutoCommit(false).config)
|
||||
}
|
||||
processors = consumers.zipWithIndex.map {
|
||||
case (consumer, index) =>
|
||||
val bufferedWriter = BufferedEdgeCollector(bufferSize, queue, queuelimit, statsReceiver)
|
||||
val processor = RecosEdgeProcessor(bufferedWriter)(statsReceiver)
|
||||
|
||||
AtLeastOnceProcessor[String, RecosHoseMessage](
|
||||
s"recos-injector-kafka-$index",
|
||||
hosename,
|
||||
consumer,
|
||||
processor.process,
|
||||
maxPendingRequests = MaxPendingRequests * bufferSize,
|
||||
workerThreads = ProcessorThreads,
|
||||
commitIntervalMs = CommitIntervalMs,
|
||||
statsReceiver = statsReceiver
|
||||
)
|
||||
}
|
||||
|
||||
log.info(s"starting ${processors.size} recosKafka processors")
|
||||
processors.foreach { processor =>
|
||||
processor.start()
|
||||
}
|
||||
} catch {
|
||||
case e: Throwable =>
|
||||
e.printStackTrace()
|
||||
log.error(e, e.toString)
|
||||
processors.foreach { processor =>
|
||||
processor.close()
|
||||
}
|
||||
processors = Seq.empty
|
||||
consumers.foreach { consumer =>
|
||||
consumer.close()
|
||||
}
|
||||
consumers = Seq.empty
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Initialize the graph writers,
|
||||
* by first creating catch up writers to bootstrap the older segments,
|
||||
* and then assigning a live writer to populate the live segment.
|
||||
*/
|
||||
private def initGrpahWriters(
|
||||
liveGraph: TGraph,
|
||||
queue: java.util.Queue[Array[RecosHoseMessage]],
|
||||
queuelimit: Semaphore
|
||||
): Unit = {
|
||||
// define a number of (numBootstrapWriters - 1) catchup writer threads, each of which will write
|
||||
// to a separate graph segment.
|
||||
val catchupWriters = (0 until (catchupWriterNum - 1)).map { index =>
|
||||
val segment = liveGraph.getLiveSegment
|
||||
liveGraph.rollForwardSegment()
|
||||
getCatchupWriter(segment, queue, queuelimit, index)
|
||||
}
|
||||
val threadPool: ExecutorService = Executors.newCachedThreadPool()
|
||||
|
||||
// define one live writer thread
|
||||
val liveWriter = getLiveWriter(liveGraph, queue, queuelimit)
|
||||
log.info("starting live graph writer that runs until service shutdown")
|
||||
threadPool.submit(liveWriter)
|
||||
log.info(
|
||||
"starting catchup graph writer, which will terminate as soon as the catchup segment is full"
|
||||
)
|
||||
catchupWriters.map(threadPool.submit(_))
|
||||
}
|
||||
|
||||
private def getLiveWriter(
|
||||
liveGraph: TGraph,
|
||||
queue: java.util.Queue[Array[RecosHoseMessage]],
|
||||
queuelimit: Semaphore
|
||||
): BufferedEdgeWriter = {
|
||||
val liveEdgeCollector = new EdgeCollector {
|
||||
override def addEdge(message: RecosHoseMessage): Unit = addEdgeToGraph(liveGraph, message)
|
||||
}
|
||||
BufferedEdgeWriter(
|
||||
queue,
|
||||
queuelimit,
|
||||
liveEdgeCollector,
|
||||
statsReceiver.scope("liveWriter"),
|
||||
isRunning.get
|
||||
)
|
||||
}
|
||||
|
||||
private def getCatchupWriter(
|
||||
segment: TSegment,
|
||||
queue: java.util.Queue[Array[RecosHoseMessage]],
|
||||
queuelimit: Semaphore,
|
||||
catchupWriterIndex: Int
|
||||
): BufferedEdgeWriter = {
|
||||
val catchupEdgeCollector = new EdgeCollector {
|
||||
var currentNumEdges = 0
|
||||
|
||||
override def addEdge(message: RecosHoseMessage): Unit = {
|
||||
currentNumEdges += 1
|
||||
addEdgeToSegment(segment, message)
|
||||
}
|
||||
}
|
||||
val maxEdges = segment.getMaxNumEdges
|
||||
|
||||
def runCondition(): Boolean = {
|
||||
isRunning.get && ((maxEdges - catchupEdgeCollector.currentNumEdges) > bufferSize)
|
||||
}
|
||||
|
||||
BufferedEdgeWriter(
|
||||
queue,
|
||||
queuelimit,
|
||||
catchupEdgeCollector,
|
||||
statsReceiver.scope("catcher_" + catchupWriterIndex),
|
||||
runCondition
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
private object UnifiedGraphWriter {
|
||||
|
||||
// The RecosEdgeProcessor is not thread-safe. Only use one thread to process each instance.
|
||||
val ProcessorThreads = 1
|
||||
// Each one cache at most 1000 * bufferSize requests.
|
||||
val MaxPendingRequests = 1000
|
||||
// Short Commit MS to reduce duplicate messages.
|
||||
val CommitIntervalMs: Long = 5000 // 5 seconds, Default Kafka value.
|
||||
}
|
@ -0,0 +1,228 @@
|
||||
package src.scala.com.twitter.recos.hose.common
|
||||
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.finatra.kafka.consumers.FinagleKafkaConsumerBuilder
|
||||
import com.twitter.graphjet.bipartite.LeftIndexedMultiSegmentBipartiteGraph
|
||||
import com.twitter.graphjet.bipartite.segment.LeftIndexedBipartiteGraphSegment
|
||||
import com.twitter.kafka.client.processor.AtLeastOnceProcessor
|
||||
import com.twitter.kafka.client.processor.ThreadSafeKafkaConsumerClient
|
||||
import com.twitter.logging.Logger
|
||||
import com.twitter.recos.hose.common.BufferedEdgeCollector
|
||||
import com.twitter.recos.hose.common.BufferedEdgeWriter
|
||||
import com.twitter.recos.hose.common.EdgeCollector
|
||||
import com.twitter.recos.hose.common.RecosEdgeProcessor
|
||||
import com.twitter.recos.internal.thriftscala.RecosHoseMessage
|
||||
import com.twitter.recos.util.Action
|
||||
import java.util.concurrent.atomic.AtomicBoolean
|
||||
import java.util.concurrent.ConcurrentLinkedQueue
|
||||
import java.util.concurrent.ExecutorService
|
||||
import java.util.concurrent.Executors
|
||||
import java.util.concurrent.Semaphore
|
||||
|
||||
/**
|
||||
* The class is an variation of UnifiedGraphWriter which allow one instance to hold multiple graphs
|
||||
*/
|
||||
trait UnifiedGraphWriterMulti[
|
||||
TSegment <: LeftIndexedBipartiteGraphSegment,
|
||||
TGraph <: LeftIndexedMultiSegmentBipartiteGraph[TSegment]] { writer =>
|
||||
|
||||
import UnifiedGraphWriterMulti._
|
||||
|
||||
def shardId: String
|
||||
def env: String
|
||||
def hosename: String
|
||||
def bufferSize: Int
|
||||
def consumerNum: Int
|
||||
def catchupWriterNum: Int
|
||||
def kafkaConsumerBuilder: FinagleKafkaConsumerBuilder[String, RecosHoseMessage]
|
||||
def clientId: String
|
||||
def statsReceiver: StatsReceiver
|
||||
|
||||
/**
|
||||
* Adds a RecosHoseMessage to the graph. used by live writer to insert edges to the
|
||||
* current segment
|
||||
*/
|
||||
def addEdgeToGraph(
|
||||
graphs: Seq[(TGraph, Set[Action.Value])],
|
||||
recosHoseMessage: RecosHoseMessage
|
||||
): Unit
|
||||
|
||||
/**
|
||||
* Adds a RecosHoseMessage to the given segment in the graph. Used by catch up writers to
|
||||
* insert edges to non-current (old) segments
|
||||
*/
|
||||
def addEdgeToSegment(
|
||||
segment: Seq[(TSegment, Set[Action.Value])],
|
||||
recosHoseMessage: RecosHoseMessage
|
||||
): Unit
|
||||
|
||||
private val log = Logger()
|
||||
private val isRunning: AtomicBoolean = new AtomicBoolean(true)
|
||||
private val initialized: AtomicBoolean = new AtomicBoolean(false)
|
||||
private var processors: Seq[AtLeastOnceProcessor[String, RecosHoseMessage]] = Seq.empty
|
||||
private var consumers: Seq[ThreadSafeKafkaConsumerClient[String, RecosHoseMessage]] = Seq.empty
|
||||
private val threadPool: ExecutorService = Executors.newCachedThreadPool()
|
||||
|
||||
def shutdown(): Unit = {
|
||||
processors.foreach { processor =>
|
||||
processor.close()
|
||||
}
|
||||
processors = Seq.empty
|
||||
consumers.foreach { consumer =>
|
||||
consumer.close()
|
||||
}
|
||||
consumers = Seq.empty
|
||||
threadPool.shutdown()
|
||||
isRunning.set(false)
|
||||
}
|
||||
|
||||
def initHose(liveGraphs: Seq[(TGraph, Set[Action.Value])]): Unit = this.synchronized {
|
||||
if (!initialized.get) {
|
||||
initialized.set(true)
|
||||
|
||||
val queue: java.util.Queue[Array[RecosHoseMessage]] =
|
||||
new ConcurrentLinkedQueue[Array[RecosHoseMessage]]()
|
||||
val queuelimit: Semaphore = new Semaphore(1024)
|
||||
|
||||
initRecosHoseKafka(queue, queuelimit)
|
||||
initGrpahWriters(liveGraphs, queue, queuelimit)
|
||||
} else {
|
||||
throw new RuntimeException("attempt to re-init kafka hose")
|
||||
}
|
||||
}
|
||||
|
||||
private def initRecosHoseKafka(
|
||||
queue: java.util.Queue[Array[RecosHoseMessage]],
|
||||
queuelimit: Semaphore,
|
||||
): Unit = {
|
||||
try {
|
||||
consumers = (0 until consumerNum).map { index =>
|
||||
new ThreadSafeKafkaConsumerClient(
|
||||
kafkaConsumerBuilder.clientId(s"clientId-$index").enableAutoCommit(false).config)
|
||||
}
|
||||
processors = consumers.zipWithIndex.map {
|
||||
case (consumer, index) =>
|
||||
val bufferedWriter = BufferedEdgeCollector(bufferSize, queue, queuelimit, statsReceiver)
|
||||
val processor = RecosEdgeProcessor(bufferedWriter)(statsReceiver)
|
||||
|
||||
AtLeastOnceProcessor[String, RecosHoseMessage](
|
||||
s"recos-injector-kafka-$index",
|
||||
hosename,
|
||||
consumer,
|
||||
processor.process,
|
||||
maxPendingRequests = MaxPendingRequests * bufferSize,
|
||||
workerThreads = ProcessorThreads,
|
||||
commitIntervalMs = CommitIntervalMs,
|
||||
statsReceiver = statsReceiver
|
||||
)
|
||||
}
|
||||
|
||||
log.info(s"starting ${processors.size} recosKafka processors")
|
||||
processors.foreach { processor =>
|
||||
processor.start()
|
||||
}
|
||||
} catch {
|
||||
case e: Throwable =>
|
||||
e.printStackTrace()
|
||||
log.error(e, e.toString)
|
||||
processors.foreach { processor =>
|
||||
processor.close()
|
||||
}
|
||||
processors = Seq.empty
|
||||
consumers.foreach { consumer =>
|
||||
consumer.close()
|
||||
}
|
||||
consumers = Seq.empty
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Initialize the graph writers,
|
||||
* by first creating catch up writers to bootstrap the older segments,
|
||||
* and then assigning a live writer to populate the live segment.
|
||||
*/
|
||||
private def initGrpahWriters(
|
||||
liveGraphs: Seq[(TGraph, Set[Action.Value])],
|
||||
queue: java.util.Queue[Array[RecosHoseMessage]],
|
||||
queuelimit: Semaphore
|
||||
): Unit = {
|
||||
// define a number of (numBootstrapWriters - 1) catchup writer threads, each of which will write
|
||||
// to a separate graph segment.
|
||||
val catchupWriters = (0 until (catchupWriterNum - 1)).map { index =>
|
||||
val segments = liveGraphs.map { case (graph, actions) => (graph.getLiveSegment, actions) }
|
||||
for (liveGraph <- liveGraphs) {
|
||||
liveGraph._1.rollForwardSegment()
|
||||
}
|
||||
getCatchupWriter(segments, queue, queuelimit, index)
|
||||
}
|
||||
val threadPool: ExecutorService = Executors.newCachedThreadPool()
|
||||
|
||||
log.info("starting live graph writer that runs until service shutdown")
|
||||
|
||||
// define one live writer thread
|
||||
val liveWriter = getLiveWriter(liveGraphs, queue, queuelimit)
|
||||
threadPool.submit(liveWriter)
|
||||
|
||||
log.info(
|
||||
"starting catchup graph writer, which will terminate as soon as the catchup segment is full"
|
||||
)
|
||||
catchupWriters.map(threadPool.submit(_))
|
||||
}
|
||||
|
||||
private def getLiveWriter(
|
||||
liveGraphs: Seq[(TGraph, Set[Action.Value])],
|
||||
queue: java.util.Queue[Array[RecosHoseMessage]],
|
||||
queuelimit: Semaphore,
|
||||
): BufferedEdgeWriter = {
|
||||
val liveEdgeCollector = new EdgeCollector {
|
||||
override def addEdge(message: RecosHoseMessage): Unit =
|
||||
addEdgeToGraph(liveGraphs, message)
|
||||
}
|
||||
BufferedEdgeWriter(
|
||||
queue,
|
||||
queuelimit,
|
||||
liveEdgeCollector,
|
||||
statsReceiver.scope("liveWriter"),
|
||||
isRunning.get
|
||||
)
|
||||
}
|
||||
|
||||
private def getCatchupWriter(
|
||||
segments: Seq[(TSegment, Set[Action.Value])],
|
||||
queue: java.util.Queue[Array[RecosHoseMessage]],
|
||||
queuelimit: Semaphore,
|
||||
catchupWriterIndex: Int,
|
||||
): BufferedEdgeWriter = {
|
||||
val catchupEdgeCollector = new EdgeCollector {
|
||||
var currentNumEdges = 0
|
||||
|
||||
override def addEdge(message: RecosHoseMessage): Unit = {
|
||||
currentNumEdges += 1
|
||||
addEdgeToSegment(segments, message)
|
||||
}
|
||||
}
|
||||
val maxEdges = segments.map(_._1.getMaxNumEdges).sum
|
||||
|
||||
def runCondition(): Boolean = {
|
||||
isRunning.get && ((maxEdges - catchupEdgeCollector.currentNumEdges) > bufferSize)
|
||||
}
|
||||
|
||||
BufferedEdgeWriter(
|
||||
queue,
|
||||
queuelimit,
|
||||
catchupEdgeCollector,
|
||||
statsReceiver.scope("catcher_" + catchupWriterIndex),
|
||||
runCondition
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
private object UnifiedGraphWriterMulti {
|
||||
|
||||
// The RecosEdgeProcessor is not thread-safe. Only use one thread to process each instance.
|
||||
val ProcessorThreads = 1
|
||||
// Each one cache at most 1000 * bufferSize requests.
|
||||
val MaxPendingRequests = 1000
|
||||
// Short Commit MS to reduce duplicate messages.
|
||||
val CommitIntervalMs: Long = 5000 // 5 seconds, Default Kafka value.
|
||||
}
|
67
src/scala/com/twitter/recos/user_tweet_entity_graph/BUILD
Normal file
67
src/scala/com/twitter/recos/user_tweet_entity_graph/BUILD
Normal file
@ -0,0 +1,67 @@
|
||||
scala_library(
|
||||
name = "user_tweet_entity_graph",
|
||||
sources = ["*.scala"],
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"3rdparty/jvm/cascading:cascading-local",
|
||||
"3rdparty/jvm/com/backtype:dfs-datastores",
|
||||
"3rdparty/jvm/com/fasterxml/jackson/module:jackson-module-scala",
|
||||
"3rdparty/jvm/com/google/inject:guice",
|
||||
"3rdparty/jvm/com/netflix/curator:curator-framework",
|
||||
"3rdparty/jvm/com/twitter/graphjet",
|
||||
"3rdparty/jvm/io/netty:netty4-tcnative-boringssl-static",
|
||||
"3rdparty/jvm/it/unimi/dsi:fastutil",
|
||||
"3rdparty/jvm/org/apache/hadoop:hadoop-client-default",
|
||||
"3rdparty/jvm/org/apache/kafka:rosette-kafka",
|
||||
"3rdparty/jvm/org/apache/thrift:libthrift",
|
||||
"abdecider/src/main/scala",
|
||||
"decider/src/main/scala",
|
||||
"finagle-internal/mtls/src/main/scala/com/twitter/finagle/mtls/authentication",
|
||||
"finagle-internal/mtls/src/main/scala/com/twitter/finagle/mtls/server",
|
||||
"finagle/finagle-core/src/main",
|
||||
"finagle/finagle-http/src/main/scala",
|
||||
"finagle/finagle-memcached/src/main/scala",
|
||||
"finagle/finagle-stats/src/main/scala",
|
||||
"finagle/finagle-thriftmux/src/main/scala",
|
||||
"frigate/frigate-common/src/main/scala/com/twitter/frigate/common/util",
|
||||
"scrooge/scrooge-core/src/main/scala",
|
||||
"servo/repo/src/main/scala",
|
||||
"servo/request/src/main/scala",
|
||||
"servo/util/src/main/scala",
|
||||
"src/resources/com/twitter/recos:decider",
|
||||
"src/scala/com/twitter/recos/decider",
|
||||
"src/scala/com/twitter/recos/graph_common",
|
||||
"src/scala/com/twitter/recos/hose/common",
|
||||
"src/scala/com/twitter/recos/model:recos-model",
|
||||
"src/scala/com/twitter/recos/serviceapi",
|
||||
"src/scala/com/twitter/recos/util:recos-util",
|
||||
"src/thrift/com/twitter/recos:recos-common-scala",
|
||||
"src/thrift/com/twitter/recos:recos-internal-scala",
|
||||
"src/thrift/com/twitter/recos/user_tweet_entity_graph:user_tweet_entity_graph-scala",
|
||||
"thrift-web-forms/src/main/scala/com/twitter/thriftwebforms",
|
||||
"thrift-web-forms/src/main/scala/com/twitter/thriftwebforms/model",
|
||||
"twitter-server-internal/src/main/scala",
|
||||
"twitter-server/server/src/main/scala",
|
||||
"twitter-server/slf4j-jdk14/src/main/scala/com/twitter/server/logging",
|
||||
"util/util-app/src/main/scala",
|
||||
"util/util-hashing/src/main/scala",
|
||||
"util/util-logging/src/main/scala",
|
||||
"util/util-stats/src/main/scala",
|
||||
],
|
||||
)
|
||||
|
||||
jvm_binary(
|
||||
name = "bin",
|
||||
basename = "user_tweet_entity_graph-server",
|
||||
main = "com.twitter.recos.user_tweet_entity_graph.Main",
|
||||
runtime_platform = "java11",
|
||||
tags = [
|
||||
"bazel-compatible",
|
||||
"known-to-fail-jira:SD-20990",
|
||||
],
|
||||
dependencies = [
|
||||
":user_tweet_entity_graph",
|
||||
"3rdparty/jvm/org/slf4j:slf4j-jdk14",
|
||||
"twitter-server/slf4j-jdk14/src/main/scala",
|
||||
],
|
||||
)
|
@ -0,0 +1,167 @@
|
||||
package com.twitter.recos.user_tweet_entity_graph
|
||||
|
||||
import java.util.Random
|
||||
import com.twitter.concurrent.AsyncQueue
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.graphjet.bipartite.NodeMetadataLeftIndexedPowerLawMultiSegmentBipartiteGraph
|
||||
import com.twitter.graphjet.algorithms.{
|
||||
RecommendationInfo,
|
||||
RecommendationType => JavaRecommendationType
|
||||
}
|
||||
import com.twitter.graphjet.algorithms.socialproof.{
|
||||
NodeMetadataSocialProofGenerator,
|
||||
NodeMetadataSocialProofResult,
|
||||
NodeMetadataSocialProofRequest => SocialProofJavaRequest,
|
||||
SocialProofResponse => SocialProofJavaResponse
|
||||
}
|
||||
import com.twitter.logging.Logger
|
||||
import com.twitter.recos.model.SalsaQueryRunner.SalsaRunnerConfig
|
||||
import com.twitter.recos.user_tweet_entity_graph.thriftscala.{
|
||||
RecommendationType => ThriftRecommendationType,
|
||||
RecommendationSocialProofRequest => SocialProofThriftRequest
|
||||
}
|
||||
import com.twitter.util.{Future, Try}
|
||||
import it.unimi.dsi.fastutil.bytes.{Byte2ObjectArrayMap, Byte2ObjectMap}
|
||||
import it.unimi.dsi.fastutil.ints.{IntOpenHashSet, IntSet}
|
||||
import it.unimi.dsi.fastutil.longs.{Long2DoubleMap, Long2DoubleOpenHashMap}
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
/**
|
||||
* EntitySocialProofRunner creates a queue of reader threads, NodeMetadataProofGenerator,
|
||||
* and each one reads from the graph and computes social proofs.
|
||||
*/
|
||||
class EntitySocialProofRunner(
|
||||
graph: NodeMetadataLeftIndexedPowerLawMultiSegmentBipartiteGraph,
|
||||
salsaRunnerConfig: SalsaRunnerConfig,
|
||||
statsReceiver: StatsReceiver) {
|
||||
private val log: Logger = Logger()
|
||||
private val stats = statsReceiver.scope(this.getClass.getSimpleName)
|
||||
private val socialProofSizeStat = stats.stat("socialProofSize")
|
||||
|
||||
private val socialProofFailureCounter = stats.counter("failure")
|
||||
private val pollCounter = stats.counter("poll")
|
||||
private val pollTimeoutCounter = stats.counter("pollTimeout")
|
||||
private val offerCounter = stats.counter("offer")
|
||||
private val pollLatencyStat = stats.stat("pollLatency")
|
||||
private val socialProofRunnerPool = initSocialProofRunnerPool()
|
||||
|
||||
private def initSocialProofRunnerPool(): AsyncQueue[NodeMetadataSocialProofGenerator] = {
|
||||
val socialProofQueue = new AsyncQueue[NodeMetadataSocialProofGenerator]
|
||||
(0 until salsaRunnerConfig.numSalsaRunners).foreach { _ =>
|
||||
socialProofQueue.offer(new NodeMetadataSocialProofGenerator(graph))
|
||||
}
|
||||
socialProofQueue
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper method to interpret the output of SocialProofJavaResponse
|
||||
*
|
||||
* @param socialProofResponse is the response from running NodeMetadataSocialProof
|
||||
* @return a sequence of SocialProofResult
|
||||
*/
|
||||
private def transformSocialProofResponse(
|
||||
socialProofResponse: Option[SocialProofJavaResponse]
|
||||
): Seq[RecommendationInfo] = {
|
||||
socialProofResponse match {
|
||||
case Some(response) =>
|
||||
val scalaResponse = response.getRankedRecommendations.asScala
|
||||
scalaResponse.foreach { result =>
|
||||
socialProofSizeStat.add(
|
||||
result.asInstanceOf[NodeMetadataSocialProofResult].getSocialProofSize)
|
||||
}
|
||||
scalaResponse.toSeq
|
||||
case _ => Nil
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper method to run social proof computation and convert the results to Option
|
||||
*
|
||||
* @param socialProof is socialProof reader on bipartite graph
|
||||
* @param request is the socialProof request
|
||||
* @return is an option of SocialProofJavaResponse
|
||||
*/
|
||||
private def getSocialProofResponse(
|
||||
socialProof: NodeMetadataSocialProofGenerator,
|
||||
request: SocialProofJavaRequest,
|
||||
random: Random
|
||||
)(
|
||||
implicit statsReceiver: StatsReceiver
|
||||
): Option[SocialProofJavaResponse] = {
|
||||
val attempt = Try(socialProof.computeRecommendations(request, random)).onFailure { e =>
|
||||
socialProofFailureCounter.incr()
|
||||
log.error(e, "SocialProof computation failed")
|
||||
}
|
||||
attempt.toOption
|
||||
}
|
||||
|
||||
/**
|
||||
* Attempt to retrieve a NodeMetadataSocialProof thread from the runner pool
|
||||
* to execute a socialProofRequest
|
||||
*/
|
||||
private def handleSocialProofRequest(socialProofRequest: SocialProofJavaRequest) = {
|
||||
pollCounter.incr()
|
||||
val t0 = System.currentTimeMillis()
|
||||
socialProofRunnerPool.poll().map { entitySocialProof =>
|
||||
val pollTime = System.currentTimeMillis - t0
|
||||
pollLatencyStat.add(pollTime)
|
||||
val socialProofResponse = Try {
|
||||
if (pollTime < salsaRunnerConfig.timeoutSalsaRunner) {
|
||||
val response =
|
||||
getSocialProofResponse(entitySocialProof, socialProofRequest, new Random())(
|
||||
statsReceiver
|
||||
)
|
||||
transformSocialProofResponse(response)
|
||||
} else {
|
||||
// if we did not get a social proof in time, then fail fast here and immediately put it back
|
||||
log.warning("socialProof polling timeout")
|
||||
pollTimeoutCounter.incr()
|
||||
throw new RuntimeException("socialProof poll timeout")
|
||||
Nil
|
||||
}
|
||||
} ensure {
|
||||
socialProofRunnerPool.offer(entitySocialProof)
|
||||
offerCounter.incr()
|
||||
}
|
||||
socialProofResponse.toOption getOrElse Nil
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This apply() supports requests coming from the new social proof endpoint in UTEG that works for
|
||||
* tweet social proof generation, as well as hashtag and url social proof generation.
|
||||
* Currently this endpoint supports url social proof generation for Guide.
|
||||
*/
|
||||
def apply(request: SocialProofThriftRequest): Future[Seq[RecommendationInfo]] = {
|
||||
val nodeMetadataTypeToIdsMap: Byte2ObjectMap[IntSet] = new Byte2ObjectArrayMap[IntSet]()
|
||||
request.recommendationIdsForSocialProof.collect {
|
||||
case (ThriftRecommendationType.Url, urlIds) =>
|
||||
// We must convert the Long url ids into type Int since the underlying library expects Int type metadata ids.
|
||||
val urlIntIds = urlIds.map(_.toInt)
|
||||
nodeMetadataTypeToIdsMap.put(
|
||||
JavaRecommendationType.URL.getValue.toByte,
|
||||
new IntOpenHashSet(urlIntIds.toArray)
|
||||
)
|
||||
case (ThriftRecommendationType.Hashtag, hashtagIds) =>
|
||||
// We must convert the Long hashtag ids into type Int since the underlying library expects Int type metadata ids.
|
||||
val hashtagIntIds = hashtagIds.map(_.toInt)
|
||||
nodeMetadataTypeToIdsMap.put(
|
||||
JavaRecommendationType.HASHTAG.getValue.toByte,
|
||||
new IntOpenHashSet(hashtagIntIds.toArray)
|
||||
)
|
||||
}
|
||||
|
||||
val leftSeedNodes: Long2DoubleMap = new Long2DoubleOpenHashMap(
|
||||
request.seedsWithWeights.keys.toArray,
|
||||
request.seedsWithWeights.values.toArray
|
||||
)
|
||||
|
||||
val socialProofRequest = new SocialProofJavaRequest(
|
||||
nodeMetadataTypeToIdsMap,
|
||||
leftSeedNodes,
|
||||
UserTweetEdgeTypeMask.getUserTweetGraphSocialProofTypes(request.socialProofTypes)
|
||||
)
|
||||
|
||||
handleSocialProofRequest(socialProofRequest)
|
||||
}
|
||||
}
|
@ -0,0 +1,103 @@
|
||||
package com.twitter.recos.user_tweet_entity_graph
|
||||
|
||||
import com.twitter.finagle.tracing.Trace
|
||||
import com.twitter.logging.Logger
|
||||
import com.twitter.recos.user_tweet_entity_graph.thriftscala._
|
||||
import com.twitter.util.Future
|
||||
|
||||
trait LoggingUserTweetEntityGraph extends thriftscala.UserTweetEntityGraph.MethodPerEndpoint {
|
||||
private[this] val accessLog = Logger("access")
|
||||
|
||||
abstract override def recommendTweets(
|
||||
request: RecommendTweetEntityRequest
|
||||
): Future[RecommendTweetEntityResponse] = {
|
||||
val time = System.currentTimeMillis
|
||||
super.recommendTweets(request) onSuccess { resp =>
|
||||
accessLog.info(
|
||||
"%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\tRecommendTweetResponse size: %s\t%s in %d ms"
|
||||
.format(
|
||||
time,
|
||||
Trace.id.toString(),
|
||||
request.requesterId,
|
||||
request.displayLocation,
|
||||
request.recommendationTypes,
|
||||
request.maxResultsByType,
|
||||
request.excludedTweetIds.map(_.take(5)),
|
||||
request.excludedTweetIds.map(_.size),
|
||||
request.seedsWithWeights.take(5),
|
||||
request.seedsWithWeights.size,
|
||||
request.maxTweetAgeInMillis,
|
||||
request.maxUserSocialProofSize,
|
||||
request.maxTweetSocialProofSize,
|
||||
request.minUserSocialProofSizes,
|
||||
request.tweetTypes,
|
||||
request.socialProofTypes,
|
||||
request.socialProofTypeUnions,
|
||||
resp.recommendations.size,
|
||||
resp.recommendations.take(20).toList map {
|
||||
case UserTweetEntityRecommendationUnion.TweetRec(tweetRec) =>
|
||||
(tweetRec.tweetId, tweetRec.socialProofByType.map { case (k, v) => (k, v.size) })
|
||||
case UserTweetEntityRecommendationUnion.HashtagRec(hashtagRec) =>
|
||||
(hashtagRec.id, hashtagRec.socialProofByType.map { case (k, v) => (k, v.size) })
|
||||
case UserTweetEntityRecommendationUnion.UrlRec(urlRec) =>
|
||||
(urlRec.id, urlRec.socialProofByType.map { case (k, v) => (k, v.size) })
|
||||
case _ =>
|
||||
throw new Exception("Unsupported recommendation types")
|
||||
},
|
||||
System.currentTimeMillis - time
|
||||
)
|
||||
)
|
||||
} onFailure { exc =>
|
||||
accessLog.error(
|
||||
"%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s in %d ms".format(
|
||||
time,
|
||||
Trace.id.toString(),
|
||||
request.requesterId,
|
||||
request.displayLocation,
|
||||
request.recommendationTypes,
|
||||
request.maxResultsByType,
|
||||
request.excludedTweetIds.map(_.take(5)),
|
||||
request.excludedTweetIds.map(_.size),
|
||||
request.seedsWithWeights.take(5),
|
||||
request.seedsWithWeights.size,
|
||||
request.maxTweetAgeInMillis,
|
||||
request.maxUserSocialProofSize,
|
||||
request.maxTweetSocialProofSize,
|
||||
request.minUserSocialProofSizes,
|
||||
request.tweetTypes,
|
||||
request.socialProofTypes,
|
||||
request.socialProofTypeUnions,
|
||||
exc,
|
||||
System.currentTimeMillis - time
|
||||
)
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
abstract override def findTweetSocialProofs(
|
||||
request: SocialProofRequest
|
||||
): Future[SocialProofResponse] = {
|
||||
val time = System.currentTimeMillis
|
||||
super.findTweetSocialProofs(request) onSuccess { resp =>
|
||||
accessLog.info(
|
||||
"%s\t%s\t%d\tResponse: %s\tin %d ms".format(
|
||||
Trace.id.toString,
|
||||
request.requesterId,
|
||||
request.seedsWithWeights.size,
|
||||
resp.socialProofResults.toList,
|
||||
System.currentTimeMillis - time
|
||||
)
|
||||
)
|
||||
} onFailure { exc =>
|
||||
accessLog.info(
|
||||
"%s\t%s\t%d\tException: %s\tin %d ms".format(
|
||||
Trace.id.toString,
|
||||
request.requesterId,
|
||||
request.seedsWithWeights.size,
|
||||
exc,
|
||||
System.currentTimeMillis - time
|
||||
)
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
258
src/scala/com/twitter/recos/user_tweet_entity_graph/Main.scala
Normal file
258
src/scala/com/twitter/recos/user_tweet_entity_graph/Main.scala
Normal file
@ -0,0 +1,258 @@
|
||||
package com.twitter.recos.user_tweet_entity_graph
|
||||
|
||||
import com.twitter.abdecider.ABDeciderFactory
|
||||
import com.twitter.abdecider.LoggingABDecider
|
||||
import com.twitter.app.Flag
|
||||
import com.twitter.conversions.DurationOps._
|
||||
import com.twitter.finagle.ThriftMux
|
||||
import com.twitter.finagle.http.HttpMuxer
|
||||
import com.twitter.finagle.mtls.authentication.ServiceIdentifier
|
||||
import com.twitter.finagle.mtls.server.MtlsStackServer._
|
||||
import com.twitter.finagle.mux.transport.OpportunisticTls
|
||||
import com.twitter.finagle.thrift.ClientId
|
||||
import com.twitter.finatra.kafka.consumers.FinagleKafkaConsumerBuilder
|
||||
import com.twitter.finatra.kafka.domain.KafkaGroupId
|
||||
import com.twitter.finatra.kafka.domain.SeekStrategy
|
||||
import com.twitter.finatra.kafka.serde.ScalaSerdes
|
||||
import com.twitter.frigate.common.util.ElfOwlFilter
|
||||
import com.twitter.frigate.common.util.ElfOwlFilter.ByLdapGroup
|
||||
import com.twitter.graphjet.bipartite.NodeMetadataLeftIndexedPowerLawMultiSegmentBipartiteGraph
|
||||
import com.twitter.logging._
|
||||
import com.twitter.recos.decider.UserTweetEntityGraphDecider
|
||||
import com.twitter.recos.graph_common.FinagleStatsReceiverWrapper
|
||||
import com.twitter.recos.graph_common.NodeMetadataLeftIndexedPowerLawMultiSegmentBipartiteGraphBuilder
|
||||
import com.twitter.recos.internal.thriftscala.RecosHoseMessage
|
||||
import com.twitter.recos.model.Constants
|
||||
import com.twitter.recos.user_tweet_entity_graph.RecosConfig._
|
||||
import com.twitter.server.logging.{Logging => JDK14Logging}
|
||||
import com.twitter.server.Deciderable
|
||||
import com.twitter.server.TwitterServer
|
||||
import com.twitter.thriftwebforms.MethodOptions
|
||||
import com.twitter.thriftwebforms.TwitterServerThriftWebForms
|
||||
import com.twitter.util.Await
|
||||
import com.twitter.util.Duration
|
||||
import java.net.InetSocketAddress
|
||||
import java.util.concurrent.TimeUnit
|
||||
import org.apache.kafka.clients.CommonClientConfigs
|
||||
import org.apache.kafka.common.config.SaslConfigs
|
||||
import org.apache.kafka.common.config.SslConfigs
|
||||
import org.apache.kafka.common.security.auth.SecurityProtocol
|
||||
import org.apache.kafka.common.serialization.StringDeserializer
|
||||
|
||||
object Main extends TwitterServer with JDK14Logging with Deciderable {
|
||||
profile =>
|
||||
|
||||
val shardId: Flag[Int] = flag("shardId", 0, "Shard ID")
|
||||
val servicePort: Flag[InetSocketAddress] =
|
||||
flag("service.port", new InetSocketAddress(10143), "Thrift service port")
|
||||
val logDir: Flag[String] = flag("logdir", "recos", "Logging directory")
|
||||
val numShards: Flag[Int] = flag("numShards", 1, "Number of shards for this service")
|
||||
val truststoreLocation: Flag[String] =
|
||||
flag[String]("truststore_location", "", "Truststore file location")
|
||||
val hoseName: Flag[String] =
|
||||
flag("hosename", "recos_injector_user_user", "the kafka stream used for incoming edges")
|
||||
|
||||
val dataCenter: Flag[String] = flag("service.cluster", "atla", "Data Center")
|
||||
val serviceRole: Flag[String] = flag("service.role", "Service Role")
|
||||
val serviceEnv: Flag[String] = flag("service.env", "Service Env")
|
||||
val serviceName: Flag[String] = flag("service.name", "Service Name")
|
||||
|
||||
private val maxNumSegments =
|
||||
flag("maxNumSegments", graphBuilderConfig.maxNumSegments, "the number of segments in the graph")
|
||||
|
||||
private val statsReceiverWrapper = FinagleStatsReceiverWrapper(statsReceiver)
|
||||
|
||||
lazy val clientId = ClientId(s"usertweetentitygraph.${serviceEnv()}")
|
||||
|
||||
private val shutdownTimeout = flag(
|
||||
"service.shutdownTimeout",
|
||||
5.seconds,
|
||||
"Maximum amount of time to wait for pending requests to complete on shutdown"
|
||||
)
|
||||
|
||||
// ********* logging **********
|
||||
|
||||
lazy val loggingLevel: Level = Level.INFO
|
||||
lazy val recosLogPath: String = logDir() + "/recos.log"
|
||||
lazy val graphLogPath: String = logDir() + "/graph.log"
|
||||
lazy val accessLogPath: String = logDir() + "/access.log"
|
||||
|
||||
override def loggerFactories: List[LoggerFactory] =
|
||||
List(
|
||||
LoggerFactory(
|
||||
level = Some(loggingLevel),
|
||||
handlers = QueueingHandler(
|
||||
handler = FileHandler(
|
||||
filename = recosLogPath,
|
||||
level = Some(loggingLevel),
|
||||
rollPolicy = Policy.Hourly,
|
||||
rotateCount = 6,
|
||||
formatter = new Formatter
|
||||
)
|
||||
) :: Nil
|
||||
),
|
||||
LoggerFactory(
|
||||
node = "graph",
|
||||
useParents = false,
|
||||
level = Some(loggingLevel),
|
||||
handlers = QueueingHandler(
|
||||
handler = FileHandler(
|
||||
filename = graphLogPath,
|
||||
level = Some(loggingLevel),
|
||||
rollPolicy = Policy.Hourly,
|
||||
rotateCount = 6,
|
||||
formatter = new Formatter
|
||||
)
|
||||
) :: Nil
|
||||
),
|
||||
LoggerFactory(
|
||||
node = "access",
|
||||
useParents = false,
|
||||
level = Some(loggingLevel),
|
||||
handlers = QueueingHandler(
|
||||
handler = FileHandler(
|
||||
filename = accessLogPath,
|
||||
level = Some(loggingLevel),
|
||||
rollPolicy = Policy.Hourly,
|
||||
rotateCount = 6,
|
||||
formatter = new Formatter
|
||||
)
|
||||
) :: Nil
|
||||
),
|
||||
LoggerFactory(
|
||||
node = "client_event",
|
||||
level = Some(loggingLevel),
|
||||
useParents = false,
|
||||
handlers = QueueingHandler(
|
||||
maxQueueSize = 10000,
|
||||
handler = ScribeHandler(
|
||||
category = "client_event",
|
||||
formatter = BareFormatter
|
||||
)
|
||||
) :: Nil
|
||||
)
|
||||
)
|
||||
// ******** Decider *************
|
||||
|
||||
val graphDecider: UserTweetEntityGraphDecider = UserTweetEntityGraphDecider()
|
||||
|
||||
// ********* ABdecider **********
|
||||
|
||||
val abDeciderYmlPath: String = "/usr/local/config/abdecider/abdecider.yml"
|
||||
|
||||
val scribeLogger: Option[Logger] = Some(Logger.get("client_event"))
|
||||
|
||||
val abDecider: LoggingABDecider =
|
||||
ABDeciderFactory(
|
||||
abDeciderYmlPath = abDeciderYmlPath,
|
||||
scribeLogger = scribeLogger,
|
||||
environment = Some("production")
|
||||
).buildWithLogging()
|
||||
|
||||
// ********* Recos service **********
|
||||
|
||||
private def getKafkaBuilder() = {
|
||||
FinagleKafkaConsumerBuilder[String, RecosHoseMessage]()
|
||||
.dest("/s/kafka/recommendations:kafka-tls")
|
||||
.groupId(KafkaGroupId(f"user_tweet_entity_graph-${shardId()}%06d"))
|
||||
.keyDeserializer(new StringDeserializer)
|
||||
.valueDeserializer(ScalaSerdes.Thrift[RecosHoseMessage].deserializer)
|
||||
.seekStrategy(SeekStrategy.REWIND)
|
||||
.rewindDuration(20.hours)
|
||||
.withConfig(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, SecurityProtocol.SASL_SSL.toString)
|
||||
.withConfig(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, truststoreLocation())
|
||||
.withConfig(SaslConfigs.SASL_MECHANISM, SaslConfigs.GSSAPI_MECHANISM)
|
||||
.withConfig(SaslConfigs.SASL_KERBEROS_SERVICE_NAME, "kafka")
|
||||
.withConfig(SaslConfigs.SASL_KERBEROS_SERVER_NAME, "kafka")
|
||||
}
|
||||
def main(): Unit = {
|
||||
log.info("building graph with maxNumSegments = " + profile.maxNumSegments())
|
||||
val graph = NodeMetadataLeftIndexedPowerLawMultiSegmentBipartiteGraphBuilder(
|
||||
graphBuilderConfig.copy(maxNumSegments = profile.maxNumSegments()),
|
||||
statsReceiverWrapper
|
||||
)
|
||||
|
||||
val kafkaConfigBuilder = getKafkaBuilder()
|
||||
|
||||
val graphWriter =
|
||||
UserTweetEntityGraphWriter(
|
||||
shardId().toString,
|
||||
serviceEnv(),
|
||||
hoseName(),
|
||||
128, // keep the original setting.
|
||||
kafkaConfigBuilder,
|
||||
clientId.name,
|
||||
statsReceiver,
|
||||
)
|
||||
graphWriter.initHose(graph)
|
||||
|
||||
val tweetRecsRunner = new TweetRecommendationsRunner(
|
||||
graph,
|
||||
Constants.salsaRunnerConfig,
|
||||
statsReceiverWrapper
|
||||
)
|
||||
|
||||
val tweetSocialProofRunner = new TweetSocialProofRunner(
|
||||
graph,
|
||||
Constants.salsaRunnerConfig,
|
||||
statsReceiver
|
||||
)
|
||||
|
||||
val entitySocialProofRunner = new EntitySocialProofRunner(
|
||||
graph,
|
||||
Constants.salsaRunnerConfig,
|
||||
statsReceiver
|
||||
)
|
||||
|
||||
val recommendationHandler = new RecommendationHandler(tweetRecsRunner, statsReceiver)
|
||||
|
||||
/*
|
||||
* Old social proof handler retained to support old tweet social proof endpoint.
|
||||
* Future clients should utilize the findRecommendationSocialProofs endpoint which will use
|
||||
* the more broad "SocialProofHandler"
|
||||
*/
|
||||
val tweetSocialProofHandler = new TweetSocialProofHandler(
|
||||
tweetSocialProofRunner,
|
||||
graphDecider,
|
||||
statsReceiver
|
||||
)
|
||||
val socialProofHandler = new SocialProofHandler(
|
||||
tweetSocialProofRunner,
|
||||
entitySocialProofRunner,
|
||||
graphDecider,
|
||||
statsReceiver
|
||||
)
|
||||
val userTweetEntityGraph = new UserTweetEntityGraph(
|
||||
recommendationHandler,
|
||||
tweetSocialProofHandler,
|
||||
socialProofHandler
|
||||
) with LoggingUserTweetEntityGraph
|
||||
|
||||
// For MutualTLS
|
||||
val serviceIdentifier = ServiceIdentifier(
|
||||
role = serviceRole(),
|
||||
service = serviceName(),
|
||||
environment = serviceEnv(),
|
||||
zone = dataCenter()
|
||||
)
|
||||
log.info(s"ServiceIdentifier = ${serviceIdentifier.toString}")
|
||||
|
||||
val thriftServer = ThriftMux.server
|
||||
.withOpportunisticTls(OpportunisticTls.Required)
|
||||
.withMutualTls(serviceIdentifier)
|
||||
.serveIface(servicePort(), userTweetEntityGraph)
|
||||
|
||||
log.info("clientid: " + clientId.toString)
|
||||
log.info("servicePort: " + servicePort().toString)
|
||||
|
||||
log.info("adding shutdown hook")
|
||||
onExit {
|
||||
graphWriter.shutdown()
|
||||
thriftServer.close(shutdownTimeout().fromNow)
|
||||
}
|
||||
log.info("added shutdown hook")
|
||||
|
||||
// Wait on the thriftServer so that shutdownTimeout is respected.
|
||||
Await.result(thriftServer)
|
||||
}
|
||||
}
|
@ -0,0 +1,17 @@
|
||||
# UserTweetEntityGraph (UTEG)
|
||||
|
||||
## What is it
|
||||
User Tweet Entity Graph (UTEG) is a Finalge thrift service built on the GraphJet framework. In maintains a graph of user-tweet relationships and serves user recommendations based on traversals in this graph.
|
||||
|
||||
## How is it used on Twitter
|
||||
UTEG generates the "XXX Liked" out-of-network tweets seen on Twitter's Home Timeline.
|
||||
The core idea behind UTEG is collaborative filtering. UTEG takes a user's weighted follow graph (i.e a list of weighted userIds) as input,
|
||||
performs efficient traversal & aggregation, and returns the top weighted tweets engaged basd on # of users that engaged the tweet, as well as
|
||||
the engaged users' weights.
|
||||
|
||||
UTEG is a stateful service and relies on a Kafka stream to ingest & persist states. It maintains an in-memory user engagements over the past
|
||||
24-48 hours. Older events are dropped and GC'ed.
|
||||
|
||||
For full details on storage & processing, please check out our open-sourced project GraphJet, a general-purpose high performance in-memory storage engine.
|
||||
- https://github.com/twitter/GraphJet
|
||||
- http://www.vldb.org/pvldb/vol9/p1281-sharma.pdf
|
@ -0,0 +1,78 @@
|
||||
package com.twitter.recos.user_tweet_entity_graph
|
||||
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.frigate.common.util.StatsUtil
|
||||
import com.twitter.graphjet.algorithms.RecommendationType
|
||||
import com.twitter.graphjet.algorithms.counting.tweet.TweetMetadataRecommendationInfo
|
||||
import com.twitter.graphjet.algorithms.counting.tweet.TweetRecommendationInfo
|
||||
import com.twitter.recos.user_tweet_entity_graph.thriftscala._
|
||||
import com.twitter.recos.util.Stats
|
||||
import com.twitter.servo.request._
|
||||
import com.twitter.util.Future
|
||||
|
||||
/**
|
||||
* Implementation of the Thrift-defined service interface.
|
||||
*
|
||||
* A wrapper of magicRecsRunner.
|
||||
*/
|
||||
class RecommendationHandler(
|
||||
tweetRecsRunner: TweetRecommendationsRunner,
|
||||
statsReceiver: StatsReceiver)
|
||||
extends RequestHandler[RecommendTweetEntityRequest, RecommendTweetEntityResponse] {
|
||||
private val stats = statsReceiver.scope(this.getClass.getSimpleName)
|
||||
private val socialProofHydrator = new SocialProofHydrator(stats)
|
||||
|
||||
override def apply(request: RecommendTweetEntityRequest): Future[RecommendTweetEntityResponse] = {
|
||||
val scopedStats: StatsReceiver = stats.scope(request.displayLocation.toString)
|
||||
|
||||
StatsUtil.trackBlockStats(scopedStats) {
|
||||
val candidatesFuture = tweetRecsRunner.apply(request)
|
||||
|
||||
candidatesFuture.map { candidates =>
|
||||
if (candidates.isEmpty) scopedStats.counter(Stats.EmptyResult).incr()
|
||||
else scopedStats.counter(Stats.Served).incr(candidates.size)
|
||||
|
||||
RecommendTweetEntityResponse(candidates.flatMap {
|
||||
_ match {
|
||||
case tweetRec: TweetRecommendationInfo =>
|
||||
Some(
|
||||
UserTweetEntityRecommendationUnion.TweetRec(
|
||||
TweetRecommendation(
|
||||
tweetRec.getRecommendation,
|
||||
tweetRec.getWeight,
|
||||
socialProofHydrator.addTweetSocialProofByType(tweetRec),
|
||||
socialProofHydrator.addTweetSocialProofs(tweetRec)
|
||||
)
|
||||
)
|
||||
)
|
||||
case tweetMetadataRec: TweetMetadataRecommendationInfo =>
|
||||
if (tweetMetadataRec.getRecommendationType == RecommendationType.HASHTAG) {
|
||||
Some(
|
||||
UserTweetEntityRecommendationUnion.HashtagRec(
|
||||
HashtagRecommendation(
|
||||
tweetMetadataRec.getRecommendation,
|
||||
tweetMetadataRec.getWeight,
|
||||
socialProofHydrator.addMetadataSocialProofByType(tweetMetadataRec)
|
||||
)
|
||||
)
|
||||
)
|
||||
} else if (tweetMetadataRec.getRecommendationType == RecommendationType.URL) {
|
||||
Some(
|
||||
UserTweetEntityRecommendationUnion.UrlRec(
|
||||
UrlRecommendation(
|
||||
tweetMetadataRec.getRecommendation,
|
||||
tweetMetadataRec.getWeight,
|
||||
socialProofHydrator.addMetadataSocialProofByType(tweetMetadataRec)
|
||||
)
|
||||
)
|
||||
)
|
||||
} else {
|
||||
None: Option[UserTweetEntityRecommendationUnion]
|
||||
}
|
||||
case _ => None
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,44 @@
|
||||
package com.twitter.recos.user_tweet_entity_graph
|
||||
|
||||
import com.twitter.graphjet.algorithms.RecommendationType
|
||||
import com.twitter.recos.model.Constants
|
||||
import com.twitter.recos.graph_common.NodeMetadataLeftIndexedPowerLawMultiSegmentBipartiteGraphBuilder.GraphBuilderConfig
|
||||
|
||||
/**
|
||||
* The class holds all the config parameters for recos graph.
|
||||
*/
|
||||
object RecosConfig {
|
||||
val maxNumSegments: Int = 8 // this value will be overwritten by a parameter from profile config
|
||||
val maxNumEdgesPerSegment: Int = 1 << 27 // 134M edges per segment
|
||||
val expectedNumLeftNodes: Int = 1 << 24 // 16M nodes
|
||||
val expectedMaxLeftDegree: Int = 64
|
||||
val leftPowerLawExponent: Double = 16.0 // steep power law as most nodes will have a small degree
|
||||
val expectedNumRightNodes: Int = 1 << 24 // 16M nodes
|
||||
val numRightNodeMetadataTypes: Int =
|
||||
RecommendationType.METADATASIZE.getValue // two node metadata types: hashtag and url
|
||||
|
||||
val graphBuilderConfig = GraphBuilderConfig(
|
||||
maxNumSegments = maxNumSegments,
|
||||
maxNumEdgesPerSegment = maxNumEdgesPerSegment,
|
||||
expectedNumLeftNodes = expectedNumLeftNodes,
|
||||
expectedMaxLeftDegree = expectedMaxLeftDegree,
|
||||
leftPowerLawExponent = leftPowerLawExponent,
|
||||
expectedNumRightNodes = expectedNumRightNodes,
|
||||
numRightNodeMetadataTypes = numRightNodeMetadataTypes,
|
||||
edgeTypeMask = new UserTweetEdgeTypeMask()
|
||||
)
|
||||
|
||||
val maxUserSocialProofSize: Int = 10
|
||||
val maxTweetSocialProofSize: Int = 10
|
||||
val maxTweetAgeInMillis: Long = 24 * 60 * 60 * 1000
|
||||
val maxEngagementAgeInMillis: Long = Long.MaxValue
|
||||
|
||||
println("RecosConfig - maxNumSegments " + maxNumSegments)
|
||||
println("RecosConfig - maxNumEdgesPerSegment " + maxNumEdgesPerSegment)
|
||||
println("RecosConfig - expectedNumLeftNodes " + expectedNumLeftNodes)
|
||||
println("RecosConfig - expectedMaxLeftDegree " + expectedMaxLeftDegree)
|
||||
println("RecosConfig - leftPowerLawExponent " + leftPowerLawExponent)
|
||||
println("RecosConfig - expectedNumRightNodes " + expectedNumRightNodes)
|
||||
println("RecosConfig - numRightNodeMetadataTypes " + numRightNodeMetadataTypes)
|
||||
println("RecosConfig - salsaRunnerConfig " + Constants.salsaRunnerConfig)
|
||||
}
|
@ -0,0 +1,165 @@
|
||||
package com.twitter.recos.user_tweet_entity_graph
|
||||
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.graphjet.algorithms.{
|
||||
RecommendationInfo,
|
||||
RecommendationType => JavaRecommendationType
|
||||
}
|
||||
import com.twitter.graphjet.algorithms.socialproof.{
|
||||
NodeMetadataSocialProofResult => EntitySocialProofJavaResult,
|
||||
SocialProofResult => SocialProofJavaResult
|
||||
}
|
||||
import com.twitter.recos.decider.UserTweetEntityGraphDecider
|
||||
import com.twitter.recos.util.Stats
|
||||
import com.twitter.recos.util.Stats._
|
||||
import com.twitter.recos.recos_common.thriftscala.{SocialProofType => SocialProofThriftType}
|
||||
import com.twitter.recos.user_tweet_entity_graph.thriftscala.{
|
||||
HashtagRecommendation,
|
||||
TweetRecommendation,
|
||||
UrlRecommendation,
|
||||
UserTweetEntityRecommendationUnion,
|
||||
RecommendationSocialProofRequest => SocialProofThriftRequest,
|
||||
RecommendationSocialProofResponse => SocialProofThriftResponse,
|
||||
RecommendationType => ThriftRecommendationType
|
||||
}
|
||||
import com.twitter.servo.request.RequestHandler
|
||||
import com.twitter.util.{Future, Try}
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
class SocialProofHandler(
|
||||
tweetSocialProofRunner: TweetSocialProofRunner,
|
||||
entitySocialProofRunner: EntitySocialProofRunner,
|
||||
decider: UserTweetEntityGraphDecider,
|
||||
statsReceiver: StatsReceiver)
|
||||
extends RequestHandler[SocialProofThriftRequest, SocialProofThriftResponse] {
|
||||
private val stats = statsReceiver.scope(this.getClass.getSimpleName)
|
||||
|
||||
private def getThriftSocialProof(
|
||||
entitySocialProof: EntitySocialProofJavaResult
|
||||
): Map[SocialProofThriftType, Map[Long, Seq[Long]]] = {
|
||||
val socialProofAttempt = Try(entitySocialProof.getSocialProof)
|
||||
.onFailure { e =>
|
||||
stats.counter(e.getClass.getSimpleName).incr()
|
||||
}
|
||||
|
||||
socialProofAttempt.toOption match {
|
||||
case Some(socialProof) if socialProof.isEmpty =>
|
||||
stats.counter(Stats.EmptyResult).incr()
|
||||
Map.empty[SocialProofThriftType, Map[Long, Seq[Long]]]
|
||||
case Some(socialProof) if !socialProof.isEmpty =>
|
||||
socialProof.asScala.map {
|
||||
case (socialProofType, socialProofUserToTweetsMap) =>
|
||||
val userToTweetsSocialProof = socialProofUserToTweetsMap.asScala.map {
|
||||
case (socialProofUser, connectingTweets) =>
|
||||
(socialProofUser.toLong, connectingTweets.asScala.map(Long2long).toSeq)
|
||||
}.toMap
|
||||
(SocialProofThriftType(socialProofType.toInt), userToTweetsSocialProof)
|
||||
}.toMap
|
||||
case _ =>
|
||||
Map.empty[SocialProofThriftType, Map[Long, Seq[Long]]]
|
||||
}
|
||||
}
|
||||
|
||||
private def getThriftSocialProof(
|
||||
tweetSocialProof: SocialProofJavaResult
|
||||
): Map[SocialProofThriftType, Seq[Long]] = {
|
||||
val socialProofAttempt = Try(tweetSocialProof.getSocialProof)
|
||||
.onFailure { e =>
|
||||
stats.counter(e.getClass.getSimpleName).incr()
|
||||
}
|
||||
|
||||
socialProofAttempt.toOption match {
|
||||
case Some(socialProof) if socialProof.isEmpty =>
|
||||
stats.counter(Stats.EmptyResult).incr()
|
||||
Map.empty[SocialProofThriftType, Seq[Long]]
|
||||
case Some(socialProof) if !socialProof.isEmpty =>
|
||||
socialProof.asScala.map {
|
||||
case (socialProofType, connectingUsers) =>
|
||||
(
|
||||
SocialProofThriftType(socialProofType.toInt),
|
||||
connectingUsers.asScala.map { Long2long }.toSeq)
|
||||
}.toMap
|
||||
case _ =>
|
||||
Map.empty[SocialProofThriftType, Seq[Long]]
|
||||
}
|
||||
}
|
||||
|
||||
private def getEntitySocialProof(
|
||||
request: SocialProofThriftRequest
|
||||
): Future[Seq[UserTweetEntityRecommendationUnion]] = {
|
||||
val socialProofsFuture = entitySocialProofRunner(request)
|
||||
|
||||
socialProofsFuture.map { socialProofs: Seq[RecommendationInfo] =>
|
||||
stats.counter(Stats.Served).incr(socialProofs.size)
|
||||
socialProofs.flatMap { entitySocialProof: RecommendationInfo =>
|
||||
val entitySocialProofJavaResult =
|
||||
entitySocialProof.asInstanceOf[EntitySocialProofJavaResult]
|
||||
if (entitySocialProofJavaResult.getRecommendationType == JavaRecommendationType.URL) {
|
||||
Some(
|
||||
UserTweetEntityRecommendationUnion.UrlRec(
|
||||
UrlRecommendation(
|
||||
entitySocialProofJavaResult.getNodeMetadataId,
|
||||
entitySocialProofJavaResult.getWeight,
|
||||
getThriftSocialProof(entitySocialProofJavaResult)
|
||||
)
|
||||
)
|
||||
)
|
||||
} else if (entitySocialProofJavaResult.getRecommendationType == JavaRecommendationType.HASHTAG) {
|
||||
Some(
|
||||
UserTweetEntityRecommendationUnion.HashtagRec(
|
||||
HashtagRecommendation(
|
||||
entitySocialProofJavaResult.getNodeMetadataId,
|
||||
entitySocialProofJavaResult.getWeight,
|
||||
getThriftSocialProof(entitySocialProofJavaResult)
|
||||
)
|
||||
)
|
||||
)
|
||||
} else {
|
||||
None
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private def getTweetSocialProof(
|
||||
request: SocialProofThriftRequest
|
||||
): Future[Seq[UserTweetEntityRecommendationUnion]] = {
|
||||
val socialProofsFuture = tweetSocialProofRunner(request)
|
||||
|
||||
socialProofsFuture.map { socialProofs: Seq[RecommendationInfo] =>
|
||||
stats.counter(Stats.Served).incr(socialProofs.size)
|
||||
socialProofs.flatMap { tweetSocialProof: RecommendationInfo =>
|
||||
val tweetSocialProofJavaResult = tweetSocialProof.asInstanceOf[SocialProofJavaResult]
|
||||
Some(
|
||||
UserTweetEntityRecommendationUnion.TweetRec(
|
||||
TweetRecommendation(
|
||||
tweetSocialProofJavaResult.getNode,
|
||||
tweetSocialProofJavaResult.getWeight,
|
||||
getThriftSocialProof(tweetSocialProofJavaResult)
|
||||
)
|
||||
)
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
def apply(request: SocialProofThriftRequest): Future[SocialProofThriftResponse] = {
|
||||
trackFutureBlockStats(stats) {
|
||||
val recommendationsWithSocialProofFut = Future
|
||||
.collect {
|
||||
request.recommendationIdsForSocialProof.keys.map {
|
||||
case ThriftRecommendationType.Tweet if decider.tweetSocialProof =>
|
||||
getTweetSocialProof(request)
|
||||
case (ThriftRecommendationType.Url | ThriftRecommendationType.Hashtag)
|
||||
if decider.entitySocialProof =>
|
||||
getEntitySocialProof(request)
|
||||
case _ =>
|
||||
Future.Nil
|
||||
}.toSeq
|
||||
}.map(_.flatten)
|
||||
recommendationsWithSocialProofFut.map { recommendationsWithSocialProof =>
|
||||
SocialProofThriftResponse(recommendationsWithSocialProof)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,111 @@
|
||||
package com.twitter.recos.user_tweet_entity_graph
|
||||
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.graphjet.algorithms.counting.tweet.{
|
||||
TweetMetadataRecommendationInfo,
|
||||
TweetRecommendationInfo
|
||||
}
|
||||
import com.twitter.recos.recos_common.thriftscala.{SocialProof, SocialProofType}
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
class SocialProofHydrator(statsReceiver: StatsReceiver) {
|
||||
private val stats = statsReceiver.scope(this.getClass.getSimpleName)
|
||||
private val socialProofsDup = stats.counter("socialProofsDup")
|
||||
private val socialProofsUni = stats.counter("socialProofsUni")
|
||||
private val socialProofByTypeDup = stats.counter("socialProofByTypeDup")
|
||||
private val socialProofByTypeUni = stats.counter("socialProofByTypeUni")
|
||||
|
||||
// If the social proof type is favorite, there are cases that one user favs, unfavs and then favs the same tweet again.
|
||||
// In this case, UTEG only returns one valid social proof. Note that GraphJet library compares the number of unique users
|
||||
// with the minSocialProofThreshold, so the threshold checking logic is correct.
|
||||
// If the social proof type is reply or quote, there are valid cases that one user replies the same tweet multiple times.
|
||||
// GraphJet does not handle this deduping because this is Twitter specific logic.
|
||||
def getSocialProofs(
|
||||
socialProofType: SocialProofType,
|
||||
users: Seq[Long],
|
||||
metadata: Seq[Long]
|
||||
): Seq[SocialProof] = {
|
||||
if (socialProofType == SocialProofType.Favorite && users.size > 1 && users.size != users.distinct.size) {
|
||||
socialProofsDup.incr()
|
||||
val unique = users
|
||||
.zip(metadata)
|
||||
.foldLeft[Seq[(Long, Long)]](Nil) { (list, next) =>
|
||||
{
|
||||
val test = list find { _._1 == next._1 }
|
||||
if (test.isEmpty) next +: list else list
|
||||
}
|
||||
}
|
||||
.reverse
|
||||
unique.map { case (user, data) => SocialProof(user, Some(data)) }
|
||||
} else {
|
||||
socialProofsUni.incr()
|
||||
users.zip(metadata).map { case (user, data) => SocialProof(user, Some(data)) }
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
// Extract and dedup social proofs from GraphJet. Only Favorite based social proof needs to dedup.
|
||||
// Return the social proofs (userId, metadata) pair in SocialProof thrift objects.
|
||||
def addTweetSocialProofs(
|
||||
tweet: TweetRecommendationInfo
|
||||
): Option[Map[SocialProofType, Seq[SocialProof]]] = {
|
||||
Some(
|
||||
tweet.getSocialProof.asScala.map {
|
||||
case (socialProofType, socialProof) =>
|
||||
val socialProofThriftType = SocialProofType(socialProofType.toByte)
|
||||
(
|
||||
socialProofThriftType,
|
||||
getSocialProofs(
|
||||
socialProofThriftType,
|
||||
socialProof.getConnectingUsers.asScala.map(_.toLong),
|
||||
socialProof.getMetadata.asScala.map(_.toLong)
|
||||
)
|
||||
)
|
||||
}.toMap
|
||||
)
|
||||
}
|
||||
|
||||
def getSocialProofs(users: Seq[Long]): Seq[Long] = {
|
||||
if (users.size > 1) {
|
||||
val distinctUsers = users.distinct
|
||||
if (users.size != distinctUsers.size) {
|
||||
socialProofByTypeDup.incr()
|
||||
} else {
|
||||
socialProofByTypeUni.incr()
|
||||
}
|
||||
distinctUsers
|
||||
} else {
|
||||
socialProofByTypeUni.incr()
|
||||
users
|
||||
}
|
||||
}
|
||||
|
||||
// Extract and dedup social proofs from GraphJet. All social proof types need to dedup.
|
||||
// Return the userId social proofs without metadata.
|
||||
def addTweetSocialProofByType(tweet: TweetRecommendationInfo): Map[SocialProofType, Seq[Long]] = {
|
||||
tweet.getSocialProof.asScala.map {
|
||||
case (socialProofType, socialProof) =>
|
||||
(
|
||||
SocialProofType(socialProofType.toByte),
|
||||
getSocialProofs(socialProof.getConnectingUsers.asScala.map(_.toLong))
|
||||
)
|
||||
}.toMap
|
||||
}
|
||||
|
||||
// The Hashtag and URL Social Proof. Dedup is not necessary.
|
||||
def addMetadataSocialProofByType(
|
||||
tweetMetadataRec: TweetMetadataRecommendationInfo
|
||||
): Map[SocialProofType, Map[Long, Seq[Long]]] = {
|
||||
tweetMetadataRec.getSocialProof.asScala.map {
|
||||
case (socialProofType, socialProof) =>
|
||||
(
|
||||
SocialProofType(socialProofType.toByte),
|
||||
socialProof.asScala.map {
|
||||
case (authorId, tweetIds) =>
|
||||
(authorId.toLong, tweetIds.asScala.map(_.toLong))
|
||||
}.toMap)
|
||||
}.toMap
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,322 @@
|
||||
package com.twitter.recos.user_tweet_entity_graph
|
||||
|
||||
import java.util.Random
|
||||
import com.twitter.concurrent.AsyncQueue
|
||||
import com.twitter.conversions.DurationOps._
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.graphjet.algorithms._
|
||||
import com.twitter.graphjet.algorithms.filters._
|
||||
import com.twitter.graphjet.algorithms.counting.TopSecondDegreeByCountResponse
|
||||
import com.twitter.graphjet.algorithms.counting.tweet.TopSecondDegreeByCountForTweet
|
||||
import com.twitter.graphjet.algorithms.counting.tweet.TopSecondDegreeByCountRequestForTweet
|
||||
import com.twitter.graphjet.bipartite.NodeMetadataLeftIndexedMultiSegmentBipartiteGraph
|
||||
import com.twitter.logging.Logger
|
||||
import com.twitter.recos.graph_common.FinagleStatsReceiverWrapper
|
||||
import com.twitter.recos.model.SalsaQueryRunner.SalsaRunnerConfig
|
||||
import com.twitter.recos.recos_common.thriftscala.SocialProofType
|
||||
import com.twitter.recos.user_tweet_entity_graph.thriftscala.RecommendTweetEntityRequest
|
||||
import com.twitter.recos.user_tweet_entity_graph.thriftscala.TweetEntityDisplayLocation
|
||||
import com.twitter.recos.user_tweet_entity_graph.thriftscala.TweetType
|
||||
import com.twitter.recos.util.Stats.trackBlockStats
|
||||
import com.twitter.util.Future
|
||||
import com.twitter.util.JavaTimer
|
||||
import com.twitter.util.Try
|
||||
import it.unimi.dsi.fastutil.longs.Long2DoubleOpenHashMap
|
||||
import it.unimi.dsi.fastutil.longs.LongOpenHashSet
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
import com.twitter.graphjet.algorithms.RecommendationType
|
||||
import com.twitter.recos.user_tweet_entity_graph.thriftscala.{
|
||||
RecommendationType => ThriftRecommendationType
|
||||
}
|
||||
import scala.collection.Map
|
||||
import scala.collection.Set
|
||||
|
||||
object TweetRecommendationsRunner {
|
||||
private val DefaultTweetTypes: Seq[TweetType] =
|
||||
Seq(TweetType.Regular, TweetType.Summary, TweetType.Photo, TweetType.Player)
|
||||
private val DefaultF1ExactSocialProofSize = 1
|
||||
private val DefaultRareTweetRecencyMillis: Long = 7.days.inMillis
|
||||
|
||||
/**
|
||||
* Map valid social proof types specified by clients to an array of bytes. If clients do not
|
||||
* specify any social proof type unions in thrift, it will return an empty set by default.
|
||||
*/
|
||||
private def getSocialProofTypeUnions(
|
||||
socialProofTypeUnions: Option[Set[Seq[SocialProofType]]]
|
||||
): Set[Array[Byte]] = {
|
||||
socialProofTypeUnions
|
||||
.map {
|
||||
_.map {
|
||||
_.map {
|
||||
_.getValue.toByte
|
||||
}.toArray
|
||||
}
|
||||
}
|
||||
.getOrElse(Set.empty)
|
||||
}
|
||||
|
||||
private def getRecommendationTypes(
|
||||
recommendationTypes: Seq[ThriftRecommendationType]
|
||||
): Set[RecommendationType] = {
|
||||
recommendationTypes.flatMap {
|
||||
_ match {
|
||||
case ThriftRecommendationType.Tweet => Some(RecommendationType.TWEET)
|
||||
case ThriftRecommendationType.Hashtag => Some(RecommendationType.HASHTAG)
|
||||
case ThriftRecommendationType.Url => Some(RecommendationType.URL)
|
||||
case _ =>
|
||||
throw new Exception("Unmatched Recommendation Type in getRecommendationTypes")
|
||||
}
|
||||
}.toSet
|
||||
}
|
||||
|
||||
private def convertThriftEnumsToJavaEnums(
|
||||
maxResults: Option[Map[ThriftRecommendationType, Int]]
|
||||
): Map[RecommendationType, Integer] = {
|
||||
maxResults
|
||||
.map {
|
||||
_.flatMap {
|
||||
_ match {
|
||||
case (ThriftRecommendationType.Tweet, v) => Some((RecommendationType.TWEET, v: Integer))
|
||||
case (ThriftRecommendationType.Hashtag, v) =>
|
||||
Some((RecommendationType.HASHTAG, v: Integer))
|
||||
case (ThriftRecommendationType.Url, v) => Some((RecommendationType.URL, v: Integer))
|
||||
case _ =>
|
||||
throw new Exception("Unmatched Recommendation Type in convertThriftEnumsToJavaEnums")
|
||||
}
|
||||
}
|
||||
}
|
||||
.getOrElse(Map.empty)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* The MagicRecsRunner creates a queue of reader threads, MagicRecs, and each one reads from the
|
||||
* graph and computes recommendations.
|
||||
*/
|
||||
class TweetRecommendationsRunner(
|
||||
bipartiteGraph: NodeMetadataLeftIndexedMultiSegmentBipartiteGraph,
|
||||
salsaRunnerConfig: SalsaRunnerConfig,
|
||||
statsReceiverWrapper: FinagleStatsReceiverWrapper) {
|
||||
|
||||
import TweetRecommendationsRunner._
|
||||
|
||||
private val log: Logger = Logger()
|
||||
|
||||
private val stats = statsReceiverWrapper.statsReceiver.scope(this.getClass.getSimpleName)
|
||||
private val magicRecsFailureCounter = stats.counter("failure")
|
||||
private val pollCounter = stats.counter("poll")
|
||||
private val pollTimeoutCounter = stats.counter("pollTimeout")
|
||||
private val offerCounter = stats.counter("offer")
|
||||
private val pollLatencyStat = stats.stat("pollLatency")
|
||||
|
||||
private val magicRecsQueue = new AsyncQueue[TopSecondDegreeByCountForTweet]
|
||||
(0 until salsaRunnerConfig.numSalsaRunners).foreach { _ =>
|
||||
magicRecsQueue.offer(
|
||||
new TopSecondDegreeByCountForTweet(
|
||||
bipartiteGraph,
|
||||
salsaRunnerConfig.expectedNodesToHitInSalsa,
|
||||
statsReceiverWrapper.scope(this.getClass.getSimpleName)
|
||||
)
|
||||
)
|
||||
}
|
||||
|
||||
private implicit val timer: JavaTimer = new JavaTimer(true)
|
||||
|
||||
private def getBaseFilters(
|
||||
staleTweetDuration: Long,
|
||||
tweetTypes: Seq[TweetType]
|
||||
) = {
|
||||
List(
|
||||
// Keep RecentTweetFilter first since it's the cheapest
|
||||
new RecentTweetFilter(staleTweetDuration, statsReceiverWrapper),
|
||||
new TweetCardFilter(
|
||||
tweetTypes.contains(TweetType.Regular),
|
||||
tweetTypes.contains(TweetType.Summary),
|
||||
tweetTypes.contains(TweetType.Photo),
|
||||
tweetTypes.contains(TweetType.Player),
|
||||
false, // no promoted tweets
|
||||
statsReceiverWrapper
|
||||
),
|
||||
new DirectInteractionsFilter(bipartiteGraph, statsReceiverWrapper),
|
||||
new RequestedSetFilter(statsReceiverWrapper),
|
||||
new SocialProofTypesFilter(statsReceiverWrapper)
|
||||
)
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper method to interpret the output of MagicRecs graph
|
||||
*
|
||||
* @param magicRecsResponse is the response from running MagicRecs
|
||||
* @return a sequence of candidate ids, with score and list of social proofs
|
||||
*/
|
||||
private def transformMagicRecsResponse(
|
||||
magicRecsResponse: Option[TopSecondDegreeByCountResponse]
|
||||
): Seq[RecommendationInfo] = {
|
||||
val responses = magicRecsResponse match {
|
||||
case Some(response) => response.getRankedRecommendations.asScala.toSeq
|
||||
case _ => Nil
|
||||
}
|
||||
responses
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper function to determine different post-process filtering logic in GraphJet,
|
||||
* based on display locations
|
||||
*/
|
||||
private def getFiltersByDisplayLocations(
|
||||
displayLocation: TweetEntityDisplayLocation,
|
||||
whitelistAuthors: LongOpenHashSet,
|
||||
blacklistAuthors: LongOpenHashSet,
|
||||
validSocialProofs: Array[Byte]
|
||||
) = {
|
||||
displayLocation match {
|
||||
case TweetEntityDisplayLocation.MagicRecsF1 =>
|
||||
Seq(
|
||||
new ANDFilters(
|
||||
List[ResultFilter](
|
||||
new TweetAuthorFilter(
|
||||
bipartiteGraph,
|
||||
whitelistAuthors,
|
||||
new LongOpenHashSet(),
|
||||
statsReceiverWrapper),
|
||||
new ExactUserSocialProofSizeFilter(
|
||||
DefaultF1ExactSocialProofSize,
|
||||
validSocialProofs,
|
||||
statsReceiverWrapper
|
||||
)
|
||||
).asJava,
|
||||
statsReceiverWrapper
|
||||
),
|
||||
// Blacklist filter must be applied separately from F1's AND filter chain
|
||||
new TweetAuthorFilter(
|
||||
bipartiteGraph,
|
||||
new LongOpenHashSet(),
|
||||
blacklistAuthors,
|
||||
statsReceiverWrapper)
|
||||
)
|
||||
case TweetEntityDisplayLocation.MagicRecsRareTweet =>
|
||||
Seq(
|
||||
new TweetAuthorFilter(
|
||||
bipartiteGraph,
|
||||
whitelistAuthors,
|
||||
blacklistAuthors,
|
||||
statsReceiverWrapper),
|
||||
new RecentEdgeMetadataFilter(
|
||||
DefaultRareTweetRecencyMillis,
|
||||
UserTweetEdgeTypeMask.Tweet.id.toByte,
|
||||
statsReceiverWrapper
|
||||
)
|
||||
)
|
||||
case _ =>
|
||||
Seq(
|
||||
new TweetAuthorFilter(
|
||||
bipartiteGraph,
|
||||
whitelistAuthors,
|
||||
blacklistAuthors,
|
||||
statsReceiverWrapper))
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper method to run salsa computation and convert the results to Option
|
||||
*
|
||||
* @param magicRecs is magicRecs reader on bipartite graph
|
||||
* @param magicRecsRequest is the magicRecs request
|
||||
* @return is an option of MagicRecsResponse
|
||||
*/
|
||||
private def getMagicRecsResponse(
|
||||
magicRecs: TopSecondDegreeByCountForTweet,
|
||||
magicRecsRequest: TopSecondDegreeByCountRequestForTweet
|
||||
)(
|
||||
implicit statsReceiver: StatsReceiver
|
||||
): Option[TopSecondDegreeByCountResponse] = {
|
||||
trackBlockStats(stats) {
|
||||
val random = new Random()
|
||||
// compute recs -- need to catch and print exceptions here otherwise they are swallowed
|
||||
val magicRecsAttempt =
|
||||
Try(magicRecs.computeRecommendations(magicRecsRequest, random)).onFailure { e =>
|
||||
magicRecsFailureCounter.incr()
|
||||
log.error(e, "MagicRecs computation failed")
|
||||
}
|
||||
magicRecsAttempt.toOption
|
||||
}
|
||||
}
|
||||
|
||||
private def getMagicRecsRequest(
|
||||
request: RecommendTweetEntityRequest
|
||||
): TopSecondDegreeByCountRequestForTweet = {
|
||||
val requesterId = request.requesterId
|
||||
val leftSeedNodes = new Long2DoubleOpenHashMap(
|
||||
request.seedsWithWeights.keys.toArray,
|
||||
request.seedsWithWeights.values.toArray
|
||||
)
|
||||
val tweetsToExcludeArray = new LongOpenHashSet(request.excludedTweetIds.getOrElse(Nil).toArray)
|
||||
val staleTweetDuration = request.maxTweetAgeInMillis.getOrElse(RecosConfig.maxTweetAgeInMillis)
|
||||
val staleEngagementDuration =
|
||||
request.maxEngagementAgeInMillis.getOrElse(RecosConfig.maxEngagementAgeInMillis)
|
||||
val tweetTypes = request.tweetTypes.getOrElse(DefaultTweetTypes)
|
||||
val tweetAuthors = new LongOpenHashSet(request.tweetAuthors.getOrElse(Nil).toArray)
|
||||
val excludedTweetAuthors = new LongOpenHashSet(
|
||||
request.excludedTweetAuthors.getOrElse(Nil).toArray)
|
||||
val validSocialProofs =
|
||||
UserTweetEdgeTypeMask.getUserTweetGraphSocialProofTypes(request.socialProofTypes)
|
||||
|
||||
val resultFilterChain = new ResultFilterChain(
|
||||
(
|
||||
getBaseFilters(staleTweetDuration, tweetTypes) ++
|
||||
getFiltersByDisplayLocations(
|
||||
displayLocation = request.displayLocation,
|
||||
whitelistAuthors = tweetAuthors,
|
||||
blacklistAuthors = excludedTweetAuthors,
|
||||
validSocialProofs = validSocialProofs
|
||||
)
|
||||
).asJava
|
||||
)
|
||||
|
||||
new TopSecondDegreeByCountRequestForTweet(
|
||||
requesterId,
|
||||
leftSeedNodes,
|
||||
tweetsToExcludeArray,
|
||||
getRecommendationTypes(request.recommendationTypes).asJava,
|
||||
convertThriftEnumsToJavaEnums(request.maxResultsByType).asJava,
|
||||
UserTweetEdgeTypeMask.SIZE,
|
||||
request.maxUserSocialProofSize.getOrElse(RecosConfig.maxUserSocialProofSize),
|
||||
request.maxTweetSocialProofSize.getOrElse(RecosConfig.maxTweetSocialProofSize),
|
||||
convertThriftEnumsToJavaEnums(request.minUserSocialProofSizes).asJava,
|
||||
validSocialProofs,
|
||||
staleTweetDuration,
|
||||
staleEngagementDuration,
|
||||
resultFilterChain,
|
||||
getSocialProofTypeUnions(request.socialProofTypeUnions).asJava
|
||||
)
|
||||
}
|
||||
|
||||
def apply(request: RecommendTweetEntityRequest): Future[Seq[RecommendationInfo]] = {
|
||||
pollCounter.incr()
|
||||
val t0 = System.currentTimeMillis
|
||||
magicRecsQueue.poll().map { magicRecs =>
|
||||
val pollTime = System.currentTimeMillis - t0
|
||||
pollLatencyStat.add(pollTime)
|
||||
val magicRecsResponse = Try {
|
||||
if (pollTime < salsaRunnerConfig.timeoutSalsaRunner) {
|
||||
val magicRecsRequest = getMagicRecsRequest(request)
|
||||
transformMagicRecsResponse(
|
||||
getMagicRecsResponse(magicRecs, magicRecsRequest)(statsReceiverWrapper.statsReceiver)
|
||||
)
|
||||
} else {
|
||||
// if we did not get a magicRecs in time, then fail fast here and immediately put it back
|
||||
log.warning("magicRecsQueue polling timeout")
|
||||
pollTimeoutCounter.incr()
|
||||
throw new RuntimeException("magicRecs poll timeout")
|
||||
Nil
|
||||
}
|
||||
} ensure {
|
||||
magicRecsQueue.offer(magicRecs)
|
||||
offerCounter.incr()
|
||||
}
|
||||
magicRecsResponse.toOption getOrElse Nil
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,73 @@
|
||||
package com.twitter.recos.user_tweet_entity_graph
|
||||
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.frigate.common.util.StatsUtil
|
||||
import com.twitter.graphjet.algorithms.RecommendationInfo
|
||||
import com.twitter.graphjet.algorithms.socialproof.{SocialProofResult => SocialProofJavaResult}
|
||||
import com.twitter.recos.decider.UserTweetEntityGraphDecider
|
||||
import com.twitter.recos.util.Stats
|
||||
import com.twitter.recos.util.Stats._
|
||||
import com.twitter.recos.recos_common.thriftscala.{SocialProofType => SocialProofThriftType}
|
||||
import com.twitter.recos.user_tweet_entity_graph.thriftscala.TweetRecommendation
|
||||
import com.twitter.recos.user_tweet_entity_graph.thriftscala.{
|
||||
SocialProofRequest => SocialProofThriftRequest
|
||||
}
|
||||
import com.twitter.recos.user_tweet_entity_graph.thriftscala.{
|
||||
SocialProofResponse => SocialProofThriftResponse
|
||||
}
|
||||
import com.twitter.servo.request.RequestHandler
|
||||
import com.twitter.util.Future
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
class TweetSocialProofHandler(
|
||||
tweetSocialProofRunner: TweetSocialProofRunner,
|
||||
decider: UserTweetEntityGraphDecider,
|
||||
statsReceiver: StatsReceiver)
|
||||
extends RequestHandler[SocialProofThriftRequest, SocialProofThriftResponse] {
|
||||
private val stats = statsReceiver.scope(this.getClass.getSimpleName)
|
||||
|
||||
def getThriftSocialProof(
|
||||
tweetSocialProof: SocialProofJavaResult
|
||||
): Map[SocialProofThriftType, Seq[Long]] = {
|
||||
Option(tweetSocialProof.getSocialProof) match {
|
||||
case Some(socialProof) if socialProof.isEmpty =>
|
||||
stats.counter(Stats.EmptyResult).incr()
|
||||
Map.empty[SocialProofThriftType, Seq[Long]]
|
||||
case Some(socialProof) if !socialProof.isEmpty =>
|
||||
socialProof.asScala.map {
|
||||
case (socialProofType, connectingUsers) =>
|
||||
(
|
||||
SocialProofThriftType(socialProofType.toInt),
|
||||
connectingUsers.asScala.map { Long2long }.toSeq)
|
||||
}.toMap
|
||||
case _ =>
|
||||
throw new Exception("TweetSocialProofHandler gets wrong TweetSocialProof response")
|
||||
}
|
||||
}
|
||||
|
||||
def apply(request: SocialProofThriftRequest): Future[SocialProofThriftResponse] = {
|
||||
StatsUtil.trackBlockStats(stats) {
|
||||
if (decider.tweetSocialProof) {
|
||||
val socialProofsFuture = tweetSocialProofRunner(request)
|
||||
|
||||
socialProofsFuture map { socialProofs: Seq[RecommendationInfo] =>
|
||||
stats.counter(Stats.Served).incr(socialProofs.size)
|
||||
SocialProofThriftResponse(
|
||||
socialProofs.flatMap { tweetSocialProof: RecommendationInfo =>
|
||||
val tweetSocialProofJavaResult = tweetSocialProof.asInstanceOf[SocialProofJavaResult]
|
||||
Some(
|
||||
TweetRecommendation(
|
||||
tweetSocialProofJavaResult.getNode,
|
||||
tweetSocialProofJavaResult.getWeight,
|
||||
getThriftSocialProof(tweetSocialProofJavaResult)
|
||||
)
|
||||
)
|
||||
}
|
||||
)
|
||||
}
|
||||
} else {
|
||||
Future.value(SocialProofThriftResponse())
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,168 @@
|
||||
package com.twitter.recos.user_tweet_entity_graph
|
||||
|
||||
import java.util.Random
|
||||
import com.twitter.concurrent.AsyncQueue
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.graphjet.bipartite.NodeMetadataLeftIndexedMultiSegmentBipartiteGraph
|
||||
import com.twitter.graphjet.algorithms.RecommendationInfo
|
||||
import com.twitter.graphjet.algorithms.socialproof.{
|
||||
SocialProofResult,
|
||||
TweetSocialProofGenerator,
|
||||
SocialProofRequest => SocialProofJavaRequest,
|
||||
SocialProofResponse => SocialProofJavaResponse
|
||||
}
|
||||
import com.twitter.logging.Logger
|
||||
import com.twitter.recos.model.SalsaQueryRunner.SalsaRunnerConfig
|
||||
import com.twitter.recos.user_tweet_entity_graph.thriftscala.{
|
||||
RecommendationType,
|
||||
RecommendationSocialProofRequest => RecommendationSocialProofThriftRequest,
|
||||
SocialProofRequest => SocialProofThriftRequest
|
||||
}
|
||||
import com.twitter.util.{Future, Try}
|
||||
import it.unimi.dsi.fastutil.longs.{Long2DoubleMap, Long2DoubleOpenHashMap, LongArraySet}
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
/**
|
||||
* TweetSocialProofRunner creates a queue of reader threads, TweetSocialProofGenerator, and each one
|
||||
* reads from the graph and computes social proofs.
|
||||
*/
|
||||
class TweetSocialProofRunner(
|
||||
bipartiteGraph: NodeMetadataLeftIndexedMultiSegmentBipartiteGraph,
|
||||
salsaRunnerConfig: SalsaRunnerConfig,
|
||||
statsReceiver: StatsReceiver) {
|
||||
private val log: Logger = Logger()
|
||||
private val stats = statsReceiver.scope(this.getClass.getSimpleName)
|
||||
private val socialProofSizeStat = stats.stat("socialProofSize")
|
||||
|
||||
private val socialProofFailureCounter = stats.counter("failure")
|
||||
private val pollCounter = stats.counter("poll")
|
||||
private val pollTimeoutCounter = stats.counter("pollTimeout")
|
||||
private val offerCounter = stats.counter("offer")
|
||||
private val pollLatencyStat = stats.stat("pollLatency")
|
||||
private val socialProofRunnerPool = initSocialProofRunnerPool()
|
||||
|
||||
private def initSocialProofRunnerPool(): AsyncQueue[TweetSocialProofGenerator] = {
|
||||
val socialProofQueue = new AsyncQueue[TweetSocialProofGenerator]
|
||||
(0 until salsaRunnerConfig.numSalsaRunners).foreach { _ =>
|
||||
socialProofQueue.offer(new TweetSocialProofGenerator(bipartiteGraph))
|
||||
}
|
||||
socialProofQueue
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper method to interpret the output of SocialProofJavaResponse
|
||||
*
|
||||
* @param socialProofResponse is the response from running TweetSocialProof
|
||||
* @return a sequence of SocialProofResult
|
||||
*/
|
||||
private def transformSocialProofResponse(
|
||||
socialProofResponse: Option[SocialProofJavaResponse]
|
||||
): Seq[RecommendationInfo] = {
|
||||
socialProofResponse match {
|
||||
case Some(response) =>
|
||||
val scalaResponse = response.getRankedRecommendations.asScala
|
||||
scalaResponse.foreach { result =>
|
||||
socialProofSizeStat.add(result.asInstanceOf[SocialProofResult].getSocialProofSize)
|
||||
}
|
||||
scalaResponse.toSeq
|
||||
case _ => Nil
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper method to run social proof computation and convert the results to Option
|
||||
*
|
||||
* @param socialProof is socialProof reader on bipartite graph
|
||||
* @param request is the socialProof request
|
||||
* @return is an option of SocialProofJavaResponse
|
||||
*/
|
||||
private def getSocialProofResponse(
|
||||
socialProof: TweetSocialProofGenerator,
|
||||
request: SocialProofJavaRequest,
|
||||
random: Random
|
||||
)(
|
||||
implicit statsReceiver: StatsReceiver
|
||||
): Option[SocialProofJavaResponse] = {
|
||||
val attempt = Try(socialProof.computeRecommendations(request, random)).onFailure { e =>
|
||||
socialProofFailureCounter.incr()
|
||||
log.error(e, "SocialProof computation failed")
|
||||
}
|
||||
attempt.toOption
|
||||
}
|
||||
|
||||
/**
|
||||
* Attempt to retrieve a TweetSocialProof thread from the runner pool
|
||||
* to execute a socialProofRequest
|
||||
*/
|
||||
private def handleSocialProofRequest(socialProofRequest: SocialProofJavaRequest) = {
|
||||
pollCounter.incr()
|
||||
val t0 = System.currentTimeMillis()
|
||||
socialProofRunnerPool.poll().map { tweetSocialProof =>
|
||||
val pollTime = System.currentTimeMillis - t0
|
||||
pollLatencyStat.add(pollTime)
|
||||
val socialProofResponse = Try {
|
||||
if (pollTime < salsaRunnerConfig.timeoutSalsaRunner) {
|
||||
val response = getSocialProofResponse(tweetSocialProof, socialProofRequest, new Random())(
|
||||
statsReceiver
|
||||
)
|
||||
transformSocialProofResponse(response)
|
||||
} else {
|
||||
// if we did not get a social proof in time, then fail fast here and immediately put it back
|
||||
log.warning("socialProof polling timeout")
|
||||
pollTimeoutCounter.incr()
|
||||
throw new RuntimeException("socialProof poll timeout")
|
||||
Nil
|
||||
}
|
||||
} ensure {
|
||||
socialProofRunnerPool.offer(tweetSocialProof)
|
||||
offerCounter.incr()
|
||||
}
|
||||
socialProofResponse.toOption getOrElse Nil
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This apply() supports requests coming from the old tweet social proof endpoint.
|
||||
* Currently this supports clients such as Email Recommendations, MagicRecs, and HomeTimeline.
|
||||
* In order to avoid heavy migration work, we are retaining this endpoint.
|
||||
*/
|
||||
def apply(request: SocialProofThriftRequest): Future[Seq[RecommendationInfo]] = {
|
||||
val tweetSet = new LongArraySet(request.inputTweets.toArray)
|
||||
val leftSeedNodes: Long2DoubleMap = new Long2DoubleOpenHashMap(
|
||||
request.seedsWithWeights.keys.toArray,
|
||||
request.seedsWithWeights.values.toArray
|
||||
)
|
||||
|
||||
val socialProofRequest = new SocialProofJavaRequest(
|
||||
tweetSet,
|
||||
leftSeedNodes,
|
||||
UserTweetEdgeTypeMask.getUserTweetGraphSocialProofTypes(request.socialProofTypes)
|
||||
)
|
||||
|
||||
handleSocialProofRequest(socialProofRequest)
|
||||
}
|
||||
|
||||
/**
|
||||
* This apply() supports requests coming from the new social proof endpoint in UTEG that works for
|
||||
* tweet social proof generation, as well as hashtag and url social proof generation.
|
||||
* Currently this endpoint supports url social proof generation for Guide.
|
||||
*/
|
||||
def apply(request: RecommendationSocialProofThriftRequest): Future[Seq[RecommendationInfo]] = {
|
||||
val tweetIds = request.recommendationIdsForSocialProof.collect {
|
||||
case (RecommendationType.Tweet, ids) => ids
|
||||
}.flatten
|
||||
val tweetSet = new LongArraySet(tweetIds.toArray)
|
||||
val leftSeedNodes: Long2DoubleMap = new Long2DoubleOpenHashMap(
|
||||
request.seedsWithWeights.keys.toArray,
|
||||
request.seedsWithWeights.values.toArray
|
||||
)
|
||||
|
||||
val socialProofRequest = new SocialProofJavaRequest(
|
||||
tweetSet,
|
||||
leftSeedNodes,
|
||||
UserTweetEdgeTypeMask.getUserTweetGraphSocialProofTypes(request.socialProofTypes)
|
||||
)
|
||||
|
||||
handleSocialProofRequest(socialProofRequest)
|
||||
}
|
||||
}
|
@ -0,0 +1,95 @@
|
||||
package com.twitter.recos.user_tweet_entity_graph
|
||||
|
||||
import com.twitter.graphjet.bipartite.api.EdgeTypeMask
|
||||
import com.twitter.recos.recos_common.thriftscala.SocialProofType
|
||||
import com.twitter.recos.util.Action
|
||||
|
||||
/**
|
||||
* The bit mask is used to encode edge types in the top bits of an integer,
|
||||
* e.g. favorite, retweet, reply and click. Under current segment configuration, each segment
|
||||
* stores up to 128M edges. Assuming that each node on one side is unique, each segment
|
||||
* stores up to 128M unique nodes on one side, which occupies the lower 27 bits of an integer.
|
||||
* This leaves five bits to encode the edge types, which at max can store 32 edge types.
|
||||
* The following implementation utilizes the top four bits and leaves one free bit out.
|
||||
*/
|
||||
class UserTweetEdgeTypeMask extends EdgeTypeMask {
|
||||
import UserTweetEdgeTypeMask._
|
||||
|
||||
override def encode(node: Int, edgeType: Byte): Int = {
|
||||
if (edgeType < 0 || edgeType > SIZE || edgeType == Click.id.toByte) {
|
||||
throw new IllegalArgumentException("encode: Illegal edge type argument " + edgeType)
|
||||
} else {
|
||||
node | (edgeType << 28)
|
||||
}
|
||||
}
|
||||
|
||||
override def edgeType(node: Int): Byte = {
|
||||
(node >>> 28).toByte
|
||||
}
|
||||
|
||||
override def restore(node: Int): Int = {
|
||||
node & MASK
|
||||
}
|
||||
}
|
||||
|
||||
object UserTweetEdgeTypeMask extends Enumeration {
|
||||
|
||||
type UserTweetEdgeTypeMask = Value
|
||||
|
||||
/**
|
||||
* Byte values corresponding to the action taken on a tweet, which will be encoded in the
|
||||
* top 4 bits in a tweet Id
|
||||
* NOTE: THERE CAN ONLY BE UP TO 16 TYPES
|
||||
*/
|
||||
val Click: UserTweetEdgeTypeMask = Value(0)
|
||||
val Favorite: UserTweetEdgeTypeMask = Value(1)
|
||||
val Retweet: UserTweetEdgeTypeMask = Value(2)
|
||||
val Reply: UserTweetEdgeTypeMask = Value(3)
|
||||
val Tweet: UserTweetEdgeTypeMask = Value(4)
|
||||
val IsMentioned: UserTweetEdgeTypeMask = Value(5)
|
||||
val IsMediatagged: UserTweetEdgeTypeMask = Value(6)
|
||||
val Quote: UserTweetEdgeTypeMask = Value(7)
|
||||
val Unfavorite: UserTweetEdgeTypeMask = Value(8)
|
||||
|
||||
/**
|
||||
* Reserve the top four bits of each integer to encode the edge type information.
|
||||
*/
|
||||
val MASK: Int = Integer.parseInt("00001111111111111111111111111111", 2)
|
||||
val SIZE: Int = this.values.size
|
||||
|
||||
/**
|
||||
* Map valid social proof types specified by clients to an array of bytes. If clients do not
|
||||
* specify any social proof types in thrift, it will return all available social types by
|
||||
* default.
|
||||
*
|
||||
* @param socialProofTypes are the valid socialProofTypes specified by clients
|
||||
* @return an array of bytes representing valid social proof types
|
||||
*/
|
||||
def getUserTweetGraphSocialProofTypes(
|
||||
socialProofTypes: Option[Seq[SocialProofType]]
|
||||
): Array[Byte] = {
|
||||
socialProofTypes
|
||||
.map { _.map { _.getValue }.toArray }
|
||||
.getOrElse((0 until SIZE).toArray)
|
||||
.map { _.toByte }
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts the action byte in the RecosHoseMessage into GraphJet internal byte mapping
|
||||
*/
|
||||
def actionTypeToEdgeType(actionByte: Byte): Byte = {
|
||||
val edgeType = Action(actionByte) match {
|
||||
case Action.Favorite => Favorite.id
|
||||
case Action.Retweet => Retweet.id
|
||||
case Action.Reply => Reply.id
|
||||
case Action.Tweet => Tweet.id
|
||||
case Action.IsMentioned => IsMentioned.id
|
||||
case Action.IsMediaTagged => IsMediatagged.id
|
||||
case Action.Quote => Quote.id
|
||||
case Action.Unfavorite => Unfavorite.id
|
||||
case _ =>
|
||||
throw new IllegalArgumentException("getEdgeType: Illegal edge type argument " + actionByte)
|
||||
}
|
||||
edgeType.toByte
|
||||
}
|
||||
}
|
@ -0,0 +1,46 @@
|
||||
package com.twitter.recos.user_tweet_entity_graph
|
||||
|
||||
import com.twitter.finagle.thrift.ClientId
|
||||
import com.twitter.finagle.tracing.{Trace, TraceId}
|
||||
import com.twitter.recos.user_tweet_entity_graph.thriftscala._
|
||||
import com.twitter.util.Future
|
||||
|
||||
object UserTweetEntityGraph {
|
||||
def traceId: TraceId = Trace.id
|
||||
def clientId: Option[ClientId] = ClientId.current
|
||||
}
|
||||
|
||||
class UserTweetEntityGraph(
|
||||
recommendationHandler: RecommendationHandler,
|
||||
tweetSocialProofHandler: TweetSocialProofHandler,
|
||||
socialProofHandler: SocialProofHandler)
|
||||
extends thriftscala.UserTweetEntityGraph.MethodPerEndpoint {
|
||||
|
||||
override def recommendTweets(
|
||||
request: RecommendTweetEntityRequest
|
||||
): Future[RecommendTweetEntityResponse] = recommendationHandler(request)
|
||||
|
||||
/**
|
||||
* Given a query user, its seed users, and a set of input tweets, return the social proofs of
|
||||
* input tweets if any.
|
||||
*
|
||||
* Currently this supports clients such as Email Recommendations, MagicRecs, and HomeTimeline.
|
||||
* In order to avoid heavy migration work, we are retaining this endpoint.
|
||||
*/
|
||||
override def findTweetSocialProofs(
|
||||
request: SocialProofRequest
|
||||
): Future[SocialProofResponse] = tweetSocialProofHandler(request)
|
||||
|
||||
/**
|
||||
* Find social proof for the specified RecommendationType given a set of input ids of that type.
|
||||
* Only find social proofs from the specified seed users with the specified social proof types.
|
||||
*
|
||||
* Currently this supports url social proof generation for Guide.
|
||||
*
|
||||
* This endpoint is flexible enough to support social proof generation for all recommendation
|
||||
* types, and should be used for all future clients of this service.
|
||||
*/
|
||||
override def findRecommendationSocialProofs(
|
||||
request: RecommendationSocialProofRequest
|
||||
): Future[RecommendationSocialProofResponse] = socialProofHandler(request)
|
||||
}
|
@ -0,0 +1,105 @@
|
||||
package com.twitter.recos.user_tweet_entity_graph
|
||||
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.finatra.kafka.consumers.FinagleKafkaConsumerBuilder
|
||||
import com.twitter.graphjet.algorithms.{RecommendationType, TweetIDMask}
|
||||
import com.twitter.graphjet.bipartite.NodeMetadataLeftIndexedMultiSegmentBipartiteGraph
|
||||
import com.twitter.graphjet.bipartite.segment.NodeMetadataLeftIndexedBipartiteGraphSegment
|
||||
import com.twitter.recos.hose.common.UnifiedGraphWriter
|
||||
import com.twitter.recos.internal.thriftscala.RecosHoseMessage
|
||||
import com.twitter.recos.serviceapi.Tweetypie._
|
||||
|
||||
/**
|
||||
* The class submits a number of $numBootstrapWriters graph writer threads, BufferedEdgeWriter,
|
||||
* during service startup. One of them is live writer thread, and the other $(numBootstrapWriters - 1)
|
||||
* are catchup writer threads. All of them consume kafka events from an internal concurrent queue,
|
||||
* which is populated by kafka reader threads. At bootstrap time, the kafka reader threads look
|
||||
* back kafka offset from several hours ago and populate the internal concurrent queue.
|
||||
* Each graph writer thread writes to an individual graph segment separately.
|
||||
* The $(numBootstrapWriters - 1) catchup writer threads will stop once all events
|
||||
* between current system time at startup and the time in memcache are processed.
|
||||
* The live writer thread will continue to write all incoming kafka events.
|
||||
* It lives through the entire life cycle of recos graph service.
|
||||
*/
|
||||
case class UserTweetEntityGraphWriter(
|
||||
shardId: String,
|
||||
env: String,
|
||||
hosename: String,
|
||||
bufferSize: Int,
|
||||
kafkaConsumerBuilder: FinagleKafkaConsumerBuilder[String, RecosHoseMessage],
|
||||
clientId: String,
|
||||
statsReceiver: StatsReceiver)
|
||||
extends UnifiedGraphWriter[
|
||||
NodeMetadataLeftIndexedBipartiteGraphSegment,
|
||||
NodeMetadataLeftIndexedMultiSegmentBipartiteGraph
|
||||
] {
|
||||
writer =>
|
||||
// The max throughput for each kafka consumer is around 25MB/s
|
||||
// Use 4 processors for 100MB/s catch-up speed.
|
||||
val consumerNum: Int = 4
|
||||
// Leave 1 Segments to LiveWriter
|
||||
val catchupWriterNum: Int = RecosConfig.maxNumSegments - 1
|
||||
|
||||
private final val EMTPY_LEFT_NODE_METADATA = new Array[Array[Int]](1)
|
||||
|
||||
/**
|
||||
* Adds a RecosHoseMessage to the graph. used by live writer to insert edges to the
|
||||
* current segment
|
||||
*/
|
||||
override def addEdgeToGraph(
|
||||
graph: NodeMetadataLeftIndexedMultiSegmentBipartiteGraph,
|
||||
recosHoseMessage: RecosHoseMessage
|
||||
): Unit = {
|
||||
graph.addEdge(
|
||||
recosHoseMessage.leftId,
|
||||
getMetaEdge(recosHoseMessage.rightId, recosHoseMessage.card),
|
||||
UserTweetEdgeTypeMask.actionTypeToEdgeType(recosHoseMessage.action),
|
||||
recosHoseMessage.edgeMetadata.getOrElse(0L),
|
||||
EMTPY_LEFT_NODE_METADATA,
|
||||
extractEntities(recosHoseMessage)
|
||||
)
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a RecosHoseMessage to the given segment in the graph. Used by catch up writers to
|
||||
* insert edges to non-current (old) segments
|
||||
*/
|
||||
override def addEdgeToSegment(
|
||||
segment: NodeMetadataLeftIndexedBipartiteGraphSegment,
|
||||
recosHoseMessage: RecosHoseMessage
|
||||
): Unit = {
|
||||
segment.addEdge(
|
||||
recosHoseMessage.leftId,
|
||||
getMetaEdge(recosHoseMessage.rightId, recosHoseMessage.card),
|
||||
UserTweetEdgeTypeMask.actionTypeToEdgeType(recosHoseMessage.action),
|
||||
recosHoseMessage.edgeMetadata.getOrElse(0L),
|
||||
EMTPY_LEFT_NODE_METADATA,
|
||||
extractEntities(recosHoseMessage)
|
||||
)
|
||||
}
|
||||
|
||||
private def getMetaEdge(rightId: Long, cardOption: Option[Byte]): Long = {
|
||||
cardOption
|
||||
.map { card =>
|
||||
if (isPhotoCard(card)) TweetIDMask.photo(rightId)
|
||||
else if (isPlayerCard(card)) TweetIDMask.player(rightId)
|
||||
else if (isSummaryCard(card)) TweetIDMask.summary(rightId)
|
||||
else if (isPromotionCard(card)) TweetIDMask.promotion(rightId)
|
||||
else rightId
|
||||
}
|
||||
.getOrElse(rightId)
|
||||
}
|
||||
|
||||
private def extractEntities(message: RecosHoseMessage): Array[Array[Int]] = {
|
||||
val entities: Array[Array[Int]] =
|
||||
new Array[Array[Int]](RecommendationType.METADATASIZE.getValue)
|
||||
message.entities.foreach {
|
||||
_.foreach {
|
||||
case (entityType, ids) =>
|
||||
entities.update(entityType, ids.toArray)
|
||||
}
|
||||
}
|
||||
entities
|
||||
}
|
||||
|
||||
}
|
66
src/scala/com/twitter/recos/user_tweet_graph/BUILD
Normal file
66
src/scala/com/twitter/recos/user_tweet_graph/BUILD
Normal file
@ -0,0 +1,66 @@
|
||||
scala_library(
|
||||
name = "user-tweet-graph",
|
||||
sources = ["*.scala"],
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"3rdparty/jvm/cascading:cascading-local",
|
||||
"3rdparty/jvm/com/backtype:dfs-datastores",
|
||||
"3rdparty/jvm/com/fasterxml/jackson/module:jackson-module-scala",
|
||||
"3rdparty/jvm/com/google/inject:guice",
|
||||
"3rdparty/jvm/com/netflix/curator:curator-framework",
|
||||
"3rdparty/jvm/com/twitter/graphjet",
|
||||
"3rdparty/jvm/io/netty:netty4-tcnative-boringssl-static",
|
||||
"3rdparty/jvm/it/unimi/dsi:fastutil",
|
||||
"3rdparty/jvm/org/apache/hadoop:hadoop-client-default",
|
||||
"3rdparty/jvm/org/apache/kafka:rosette-kafka",
|
||||
"3rdparty/jvm/org/apache/thrift:libthrift",
|
||||
"abdecider/src/main/scala",
|
||||
"decider/src/main/scala",
|
||||
"finagle-internal/mtls/src/main/scala/com/twitter/finagle/mtls/authentication",
|
||||
"finagle-internal/mtls/src/main/scala/com/twitter/finagle/mtls/server",
|
||||
"finagle/finagle-core/src/main",
|
||||
"finagle/finagle-http/src/main/scala",
|
||||
"finagle/finagle-memcached/src/main/scala",
|
||||
"finagle/finagle-stats/src/main/scala",
|
||||
"finagle/finagle-thriftmux/src/main/scala",
|
||||
"frigate/frigate-common/src/main/scala/com/twitter/frigate/common/util",
|
||||
"scrooge/scrooge-core/src/main/scala",
|
||||
"servo/repo/src/main/scala",
|
||||
"servo/request/src/main/scala",
|
||||
"servo/util/src/main/scala",
|
||||
"src/resources/com/twitter/recos:decider",
|
||||
"src/scala/com/twitter/recos/decider",
|
||||
"src/scala/com/twitter/recos/graph_common",
|
||||
"src/scala/com/twitter/recos/hose/common",
|
||||
"src/scala/com/twitter/recos/model:recos-model",
|
||||
"src/scala/com/twitter/recos/serviceapi",
|
||||
"src/scala/com/twitter/recos/user_tweet_graph/relatedTweetHandlers",
|
||||
"src/scala/com/twitter/recos/util:recos-util",
|
||||
"src/scala/com/twitter/simclusters_v2/common",
|
||||
"src/thrift/com/twitter/recos:recos-common-scala",
|
||||
"src/thrift/com/twitter/recos:recos-internal-scala",
|
||||
"src/thrift/com/twitter/recos/user_tweet_graph:user_tweet_graph-scala",
|
||||
"thrift-web-forms/src/main/scala/com/twitter/thriftwebforms",
|
||||
"thrift-web-forms/src/main/scala/com/twitter/thriftwebforms/model",
|
||||
"twitter-server-internal/src/main/scala",
|
||||
"twitter-server/server/src/main/scala",
|
||||
"twitter-server/slf4j-jdk14/src/main/scala/com/twitter/server/logging",
|
||||
"util/util-app/src/main/scala",
|
||||
"util/util-hashing/src/main/scala",
|
||||
"util/util-logging/src/main/scala",
|
||||
"util/util-stats/src/main/scala",
|
||||
],
|
||||
)
|
||||
|
||||
jvm_binary(
|
||||
name = "bin",
|
||||
basename = "user-tweet-graph-server",
|
||||
main = "com.twitter.recos.user_tweet_graph.Main",
|
||||
runtime_platform = "java11",
|
||||
tags = ["known-to-fail-jira:SD-20771"],
|
||||
dependencies = [
|
||||
":user-tweet-graph",
|
||||
"3rdparty/jvm/org/slf4j:slf4j-jdk14",
|
||||
"twitter-server/slf4j-jdk14/src/main/scala",
|
||||
],
|
||||
)
|
291
src/scala/com/twitter/recos/user_tweet_graph/Main.scala
Normal file
291
src/scala/com/twitter/recos/user_tweet_graph/Main.scala
Normal file
@ -0,0 +1,291 @@
|
||||
package com.twitter.recos.user_tweet_graph
|
||||
|
||||
import com.twitter.abdecider.ABDeciderFactory
|
||||
import com.twitter.abdecider.LoggingABDecider
|
||||
import com.twitter.app.Flag
|
||||
import com.twitter.conversions.DurationOps._
|
||||
import com.twitter.finagle.ThriftMux
|
||||
import com.twitter.finagle.http.HttpMuxer
|
||||
import com.twitter.finagle.mtls.authentication.ServiceIdentifier
|
||||
import com.twitter.finagle.mtls.client.MtlsStackClient.MtlsThriftMuxClientSyntax
|
||||
import com.twitter.finagle.mtls.server.MtlsStackServer._
|
||||
import com.twitter.finagle.mux.ClientDiscardedRequestException
|
||||
import com.twitter.finagle.mux.transport.OpportunisticTls
|
||||
import com.twitter.finagle.service.ReqRep
|
||||
import com.twitter.finagle.service.ResponseClass
|
||||
import com.twitter.finagle.thrift.ClientId
|
||||
import com.twitter.finatra.kafka.consumers.FinagleKafkaConsumerBuilder
|
||||
import com.twitter.finatra.kafka.domain.KafkaGroupId
|
||||
import com.twitter.finatra.kafka.domain.SeekStrategy
|
||||
import com.twitter.finatra.kafka.serde.ScalaSerdes
|
||||
import com.twitter.frigate.common.util.ElfOwlFilter
|
||||
import com.twitter.frigate.common.util.ElfOwlFilter.ByLdapGroup
|
||||
import com.twitter.graphjet.bipartite.MultiSegmentPowerLawBipartiteGraph
|
||||
import com.twitter.logging._
|
||||
import com.twitter.recos.decider.EndpointLoadShedder
|
||||
import com.twitter.recos.decider.UserTweetGraphDecider
|
||||
import com.twitter.recos.graph_common.FinagleStatsReceiverWrapper
|
||||
import com.twitter.recos.graph_common.MultiSegmentPowerLawBipartiteGraphBuilder
|
||||
import com.twitter.recos.internal.thriftscala.RecosHoseMessage
|
||||
import com.twitter.recos.user_tweet_graph.RecosConfig._
|
||||
import com.twitter.recos.user_tweet_graph.relatedTweetHandlers.ConsumersBasedRelatedTweetsHandler
|
||||
import com.twitter.recos.user_tweet_graph.relatedTweetHandlers.ProducerBasedRelatedTweetsHandler
|
||||
import com.twitter.recos.user_tweet_graph.relatedTweetHandlers.TweetBasedRelatedTweetsHandler
|
||||
import com.twitter.recos.user_tweet_graph.store.UserRecentFollowersStore
|
||||
import com.twitter.server.Deciderable
|
||||
import com.twitter.server.TwitterServer
|
||||
import com.twitter.server.logging.{Logging => JDK14Logging}
|
||||
import com.twitter.servo.request._
|
||||
import com.twitter.servo.util.ExceptionCounter
|
||||
import com.twitter.simclusters_v2.common.UserId
|
||||
import com.twitter.socialgraph.thriftscala.SocialGraphService
|
||||
import com.twitter.storehaus.ReadableStore
|
||||
import com.twitter.util.Await
|
||||
import com.twitter.util.Duration
|
||||
import com.twitter.util.JavaTimer
|
||||
import com.twitter.util.Throw
|
||||
import com.twitter.util.Timer
|
||||
import java.net.InetSocketAddress
|
||||
import java.util.concurrent.TimeUnit
|
||||
import org.apache.kafka.clients.CommonClientConfigs
|
||||
import org.apache.kafka.common.config.SaslConfigs
|
||||
import org.apache.kafka.common.config.SslConfigs
|
||||
import org.apache.kafka.common.security.auth.SecurityProtocol
|
||||
import org.apache.kafka.common.serialization.StringDeserializer
|
||||
import scala.reflect.ClassTag
|
||||
|
||||
object Main extends TwitterServer with JDK14Logging with Deciderable {
|
||||
profile =>
|
||||
|
||||
val shardId: Flag[Int] = flag("shardId", 0, "Shard ID")
|
||||
val servicePort: Flag[InetSocketAddress] =
|
||||
flag("service.port", new InetSocketAddress(10143), "Thrift service port")
|
||||
val logDir: Flag[String] = flag("logdir", "recos", "Logging directory")
|
||||
val numShards: Flag[Int] = flag("numShards", 1, "Number of shards for this service")
|
||||
val truststoreLocation: Flag[String] =
|
||||
flag[String]("truststore_location", "", "Truststore file location")
|
||||
val hoseName: Flag[String] =
|
||||
flag("hosename", "recos_injector_user_user", "the kafka stream used for incoming edges")
|
||||
|
||||
val dataCenter: Flag[String] = flag("service.cluster", "atla", "Data Center")
|
||||
val serviceRole: Flag[String] = flag("service.role", "Service Role")
|
||||
val serviceEnv: Flag[String] = flag("service.env", "Service Env")
|
||||
val serviceName: Flag[String] = flag("service.name", "Service Name")
|
||||
|
||||
private val maxNumSegments =
|
||||
flag("maxNumSegments", graphBuilderConfig.maxNumSegments, "the number of segments in the graph")
|
||||
|
||||
private val statsReceiverWrapper = FinagleStatsReceiverWrapper(statsReceiver)
|
||||
|
||||
/**
|
||||
* A ClientRequestAuthorizer to be used in a request-authorization RequestFilter.
|
||||
*/
|
||||
lazy val clientAuthorizer: ClientRequestAuthorizer =
|
||||
ClientRequestAuthorizer.observed(
|
||||
ClientRequestAuthorizer.permissive,
|
||||
new ClientRequestObserver(statsReceiver)
|
||||
)
|
||||
|
||||
lazy val clientId = ClientId(s"usertweetgraph.${serviceEnv()}")
|
||||
|
||||
private def makeThriftClient[ThriftServiceType: ClassTag](
|
||||
dest: String,
|
||||
label: String,
|
||||
serviceIdentifier: ServiceIdentifier,
|
||||
requestTimeout: Duration = 100.milliseconds
|
||||
): ThriftServiceType = {
|
||||
ThriftMux.client
|
||||
.withClientId(ClientId("usertweetgraph.prod"))
|
||||
.withOpportunisticTls(OpportunisticTls.Required)
|
||||
.withMutualTls(serviceIdentifier)
|
||||
.withRequestTimeout(requestTimeout)
|
||||
.withStatsReceiver(statsReceiver.scope("clnt"))
|
||||
.withResponseClassifier {
|
||||
case ReqRep(_, Throw(_: ClientDiscardedRequestException)) => ResponseClass.Ignorable
|
||||
}.build[ThriftServiceType](dest, label)
|
||||
}
|
||||
|
||||
private val shutdownTimeout = flag(
|
||||
"service.shutdownTimeout",
|
||||
5.seconds,
|
||||
"Maximum amount of time to wait for pending requests to complete on shutdown"
|
||||
)
|
||||
|
||||
/**
|
||||
* ExceptionCounter for tracking failures from RequestHandler(s).
|
||||
*/
|
||||
lazy val exceptionCounter = new ExceptionCounter(statsReceiver)
|
||||
|
||||
/**
|
||||
* Function for translating exceptions returned by a RequestHandler. Useful
|
||||
* for cases where underlying exception types should be wrapped in those
|
||||
* defined in the project's Thrift IDL.
|
||||
*/
|
||||
lazy val translateExceptions: PartialFunction[Throwable, Throwable] = {
|
||||
case t => t
|
||||
}
|
||||
|
||||
// ********* logging **********
|
||||
|
||||
lazy val loggingLevel: Level = Level.INFO
|
||||
lazy val recosLogPath: String = logDir() + "/recos.log"
|
||||
lazy val graphLogPath: String = logDir() + "/graph.log"
|
||||
lazy val accessLogPath: String = logDir() + "/access.log"
|
||||
|
||||
override def loggerFactories: List[LoggerFactory] =
|
||||
List(
|
||||
LoggerFactory(
|
||||
level = Some(loggingLevel),
|
||||
handlers = QueueingHandler(
|
||||
handler = FileHandler(
|
||||
filename = recosLogPath,
|
||||
level = Some(loggingLevel),
|
||||
rollPolicy = Policy.Hourly,
|
||||
rotateCount = 6,
|
||||
formatter = new Formatter
|
||||
)
|
||||
) :: Nil
|
||||
),
|
||||
LoggerFactory(
|
||||
node = "graph",
|
||||
useParents = false,
|
||||
level = Some(loggingLevel),
|
||||
handlers = QueueingHandler(
|
||||
handler = FileHandler(
|
||||
filename = graphLogPath,
|
||||
level = Some(loggingLevel),
|
||||
rollPolicy = Policy.Hourly,
|
||||
rotateCount = 6,
|
||||
formatter = new Formatter
|
||||
)
|
||||
) :: Nil
|
||||
),
|
||||
LoggerFactory(
|
||||
node = "access",
|
||||
useParents = false,
|
||||
level = Some(loggingLevel),
|
||||
handlers = QueueingHandler(
|
||||
handler = FileHandler(
|
||||
filename = accessLogPath,
|
||||
level = Some(loggingLevel),
|
||||
rollPolicy = Policy.Hourly,
|
||||
rotateCount = 6,
|
||||
formatter = new Formatter
|
||||
)
|
||||
) :: Nil
|
||||
),
|
||||
LoggerFactory(
|
||||
node = "client_event",
|
||||
level = Some(loggingLevel),
|
||||
useParents = false,
|
||||
handlers = QueueingHandler(
|
||||
maxQueueSize = 10000,
|
||||
handler = ScribeHandler(
|
||||
category = "client_event",
|
||||
formatter = BareFormatter
|
||||
)
|
||||
) :: Nil
|
||||
)
|
||||
)
|
||||
// ******** Decider *************
|
||||
|
||||
// ********* ABdecider **********
|
||||
|
||||
val abDeciderYmlPath: String = "/usr/local/config/abdecider/abdecider.yml"
|
||||
|
||||
val scribeLogger: Option[Logger] = Some(Logger.get("client_event"))
|
||||
|
||||
val abDecider: LoggingABDecider =
|
||||
ABDeciderFactory(
|
||||
abDeciderYmlPath = abDeciderYmlPath,
|
||||
scribeLogger = scribeLogger,
|
||||
environment = Some("production")
|
||||
).buildWithLogging()
|
||||
|
||||
// ********* Recos service **********
|
||||
def main(): Unit = {
|
||||
log.info("building graph with maxNumSegments = " + profile.maxNumSegments())
|
||||
|
||||
implicit val timer: Timer = new JavaTimer(true)
|
||||
|
||||
val graph = MultiSegmentPowerLawBipartiteGraphBuilder(
|
||||
graphBuilderConfig.copy(maxNumSegments = profile.maxNumSegments()),
|
||||
statsReceiverWrapper
|
||||
)
|
||||
|
||||
val kafkaConfigBuilder = FinagleKafkaConsumerBuilder[String, RecosHoseMessage]()
|
||||
.dest("/s/kafka/recommendations:kafka-tls")
|
||||
.groupId(KafkaGroupId(f"user_tweet_graph-${shardId()}%06d"))
|
||||
.keyDeserializer(new StringDeserializer)
|
||||
.valueDeserializer(ScalaSerdes.Thrift[RecosHoseMessage].deserializer)
|
||||
.seekStrategy(SeekStrategy.REWIND)
|
||||
.rewindDuration(48.hours)
|
||||
.withConfig(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, SecurityProtocol.SASL_SSL.toString)
|
||||
.withConfig(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, truststoreLocation())
|
||||
.withConfig(SaslConfigs.SASL_MECHANISM, SaslConfigs.GSSAPI_MECHANISM)
|
||||
.withConfig(SaslConfigs.SASL_KERBEROS_SERVICE_NAME, "kafka")
|
||||
.withConfig(SaslConfigs.SASL_KERBEROS_SERVER_NAME, "kafka")
|
||||
|
||||
val graphWriter =
|
||||
UserTweetGraphWriter(
|
||||
shardId().toString,
|
||||
serviceEnv(),
|
||||
hoseName(),
|
||||
128, // keep the original setting.
|
||||
kafkaConfigBuilder,
|
||||
clientId.name,
|
||||
statsReceiver,
|
||||
)
|
||||
graphWriter.initHose(graph)
|
||||
|
||||
// For MutualTLS
|
||||
val serviceIdentifier = ServiceIdentifier(
|
||||
role = serviceRole(),
|
||||
service = serviceName(),
|
||||
environment = serviceEnv(),
|
||||
zone = dataCenter()
|
||||
)
|
||||
log.info(s"ServiceIdentifier = ${serviceIdentifier.toString}")
|
||||
|
||||
val socialGraphClient: SocialGraphService.MethodPerEndpoint =
|
||||
makeThriftClient[SocialGraphService.MethodPerEndpoint](
|
||||
"/s/socialgraph/socialgraph",
|
||||
"socialgraph",
|
||||
serviceIdentifier)
|
||||
val userRecentFollowersStore: ReadableStore[UserRecentFollowersStore.Query, Seq[UserId]] =
|
||||
new UserRecentFollowersStore(socialGraphClient)
|
||||
|
||||
val tweetBasedRelatedTweetsHandler = new TweetBasedRelatedTweetsHandler(graph, statsReceiver)
|
||||
val consumersBasedRelatedTweetsHandler =
|
||||
new ConsumersBasedRelatedTweetsHandler(graph, statsReceiver)
|
||||
val producerBasedRelatedTweetsHandler =
|
||||
new ProducerBasedRelatedTweetsHandler(graph, userRecentFollowersStore, statsReceiver)
|
||||
|
||||
val decider = UserTweetGraphDecider(serviceEnv(), dataCenter())
|
||||
val endpointLoadShedder = new EndpointLoadShedder(decider)
|
||||
val userTweetGraph =
|
||||
new UserTweetGraph(
|
||||
tweetBasedRelatedTweetsHandler,
|
||||
producerBasedRelatedTweetsHandler,
|
||||
consumersBasedRelatedTweetsHandler,
|
||||
endpointLoadShedder)(timer)
|
||||
|
||||
val thriftServer = ThriftMux.server
|
||||
.withOpportunisticTls(OpportunisticTls.Required)
|
||||
.withMutualTls(serviceIdentifier)
|
||||
.serveIface(servicePort(), userTweetGraph)
|
||||
|
||||
log.info("clientid: " + clientId.toString)
|
||||
log.info("servicePort: " + servicePort().toString)
|
||||
|
||||
log.info("adding shutdown hook")
|
||||
onExit {
|
||||
graphWriter.shutdown()
|
||||
thriftServer.close(shutdownTimeout().fromNow)
|
||||
}
|
||||
log.info("added shutdown hook")
|
||||
|
||||
// Wait on the thriftServer so that shutdownTimeout is respected.
|
||||
Await.result(thriftServer)
|
||||
}
|
||||
}
|
17
src/scala/com/twitter/recos/user_tweet_graph/README.md
Normal file
17
src/scala/com/twitter/recos/user_tweet_graph/README.md
Normal file
@ -0,0 +1,17 @@
|
||||
# UserTweetGraph (UTG)
|
||||
|
||||
## What is it
|
||||
User Tweet Graph (UTG) is a Finalge thrift service built on the GraphJet framework. In maintains a graph of user-tweet engagements and serves user recommendations based on traversals of this graph.
|
||||
|
||||
## How is it used on Twitter
|
||||
UTG recommends tweets based on collaborative filtering & random walks. UTG takes a set of seed users or seed tweets as input, and performs
|
||||
1-hop, 2-hop, or even 3+hop traversals on the engagement graph.
|
||||
UTG's user-tweet engagement edges are bi-directional, and this enables it to perform flexible multi-hop traversals. The flipside to this is
|
||||
UTG is more memory demanding compared to other GraphJet services like UTEG, whose engagement edges are single directional.
|
||||
|
||||
UTG is a stateful service and relies on a Kafka stream to ingest & persist states. The Kafka stream is processed and generated by Recos-Injector.
|
||||
It maintains an in-memory user engagements over the past 24-48 hours. Older events are dropped and GC'ed.
|
||||
|
||||
For full details on storage & processing, please check out our open-sourced project GraphJet, a general-purpose high performance in-memory storage engine.
|
||||
- https://github.com/twitter/GraphJet
|
||||
- http://www.vldb.org/pvldb/vol9/p1281-sharma.pdf
|
@ -0,0 +1,98 @@
|
||||
package com.twitter.recos.user_tweet_graph
|
||||
|
||||
import com.twitter.finagle.thrift.ClientId
|
||||
import com.twitter.finagle.tracing.Trace
|
||||
import com.twitter.finagle.tracing.TraceId
|
||||
import com.twitter.recos.decider.EndpointLoadShedder
|
||||
import com.twitter.recos.recos_common.thriftscala._
|
||||
import com.twitter.recos.user_tweet_graph.thriftscala._
|
||||
import com.twitter.util.Duration
|
||||
import com.twitter.util.Future
|
||||
import com.twitter.util.Timer
|
||||
import scala.concurrent.duration.MILLISECONDS
|
||||
import com.twitter.logging.Logger
|
||||
import com.twitter.recos.user_tweet_graph.relatedTweetHandlers.TweetBasedRelatedTweetsHandler
|
||||
import com.twitter.recos.user_tweet_graph.relatedTweetHandlers.ProducerBasedRelatedTweetsHandler
|
||||
import com.twitter.recos.user_tweet_graph.relatedTweetHandlers.ConsumersBasedRelatedTweetsHandler
|
||||
import com.twitter.simclusters_v2.common.TweetId
|
||||
import com.twitter.simclusters_v2.common.UserId
|
||||
|
||||
object UserTweetGraph {
|
||||
def traceId: TraceId = Trace.id
|
||||
def clientId: Option[ClientId] = ClientId.current
|
||||
}
|
||||
|
||||
class UserTweetGraph(
|
||||
tweetBasedRelatedTweetsHandler: TweetBasedRelatedTweetsHandler,
|
||||
producerBasedRelatedTweetsHandler: ProducerBasedRelatedTweetsHandler,
|
||||
consumersBasedRelatedTweetsHandler: ConsumersBasedRelatedTweetsHandler,
|
||||
endpointLoadShedder: EndpointLoadShedder
|
||||
)(
|
||||
implicit timer: Timer)
|
||||
extends thriftscala.UserTweetGraph.MethodPerEndpoint {
|
||||
|
||||
private val defaultTimeout: Duration = Duration(50, MILLISECONDS)
|
||||
private val EmptyResponse = Future.value(RelatedTweetResponse())
|
||||
private val EmptyFeatureResponse = Future.value(UserTweetFeatureResponse())
|
||||
|
||||
private val log = Logger()
|
||||
|
||||
override def recommendTweets(request: RecommendTweetRequest): Future[RecommendTweetResponse] =
|
||||
Future.value(RecommendTweetResponse())
|
||||
|
||||
override def getLeftNodeEdges(request: GetRecentEdgesRequest): Future[GetRecentEdgesResponse] =
|
||||
Future.value(GetRecentEdgesResponse())
|
||||
|
||||
override def getRightNode(tweet: Long): Future[NodeInfo] = Future.value(NodeInfo())
|
||||
|
||||
// deprecated
|
||||
override def relatedTweets(request: RelatedTweetRequest): Future[RelatedTweetResponse] =
|
||||
EmptyResponse
|
||||
|
||||
override def tweetBasedRelatedTweets(
|
||||
request: TweetBasedRelatedTweetRequest
|
||||
): Future[RelatedTweetResponse] =
|
||||
endpointLoadShedder("tweetBasedRelatedTweets") {
|
||||
tweetBasedRelatedTweetsHandler(request).raiseWithin(defaultTimeout)
|
||||
}.rescue {
|
||||
case EndpointLoadShedder.LoadSheddingException =>
|
||||
EmptyResponse
|
||||
case e =>
|
||||
log.info("user-tweet-graph_tweetBasedRelatedTweets" + e)
|
||||
EmptyResponse
|
||||
}
|
||||
|
||||
override def producerBasedRelatedTweets(
|
||||
request: ProducerBasedRelatedTweetRequest
|
||||
): Future[RelatedTweetResponse] =
|
||||
endpointLoadShedder("producerBasedRelatedTweets") {
|
||||
producerBasedRelatedTweetsHandler(request).raiseWithin(defaultTimeout)
|
||||
}.rescue {
|
||||
case EndpointLoadShedder.LoadSheddingException =>
|
||||
EmptyResponse
|
||||
case e =>
|
||||
log.info("user-tweet-graph_producerBasedRelatedTweets" + e)
|
||||
EmptyResponse
|
||||
}
|
||||
|
||||
override def consumersBasedRelatedTweets(
|
||||
request: ConsumersBasedRelatedTweetRequest
|
||||
): Future[RelatedTweetResponse] =
|
||||
endpointLoadShedder("consumersBasedRelatedTweets") {
|
||||
consumersBasedRelatedTweetsHandler(request).raiseWithin(defaultTimeout)
|
||||
}.rescue {
|
||||
case EndpointLoadShedder.LoadSheddingException =>
|
||||
EmptyResponse
|
||||
case e =>
|
||||
log.info("user-tweet-graph_consumersBasedRelatedTweets" + e)
|
||||
EmptyResponse
|
||||
}
|
||||
|
||||
// deprecated
|
||||
override def userTweetFeatures(
|
||||
userId: UserId,
|
||||
tweetId: TweetId
|
||||
): Future[UserTweetFeatureResponse] =
|
||||
EmptyFeatureResponse
|
||||
|
||||
}
|
@ -0,0 +1,39 @@
|
||||
package com.twitter.recos.user_tweet_graph
|
||||
|
||||
import com.twitter.recos.graph_common.MultiSegmentPowerLawBipartiteGraphBuilder.GraphBuilderConfig
|
||||
|
||||
/**
|
||||
* The class holds all the config parameters for recos graph.
|
||||
*/
|
||||
object RecosConfig {
|
||||
val maxNumSegments: Int = 8
|
||||
val maxNumEdgesPerSegment: Int =
|
||||
(1 << 28) // 268M edges per segment, should be able to include 2 days' data
|
||||
val expectedNumLeftNodes: Int =
|
||||
(1 << 26) // should correspond to 67M nodes storage
|
||||
val expectedMaxLeftDegree: Int = 64
|
||||
val leftPowerLawExponent: Double = 16.0 // steep power law as most nodes will have a small degree
|
||||
val expectedNumRightNodes: Int = (1 << 26) // 67M nodes
|
||||
val expectedMaxRightDegree: Int = scala.math.pow(1024, 2).toInt // some nodes will be very popular
|
||||
val rightPowerLawExponent: Double = 4.0 // this will be less steep
|
||||
|
||||
val graphBuilderConfig = GraphBuilderConfig(
|
||||
maxNumSegments = maxNumSegments,
|
||||
maxNumEdgesPerSegment = maxNumEdgesPerSegment,
|
||||
expectedNumLeftNodes = expectedNumLeftNodes,
|
||||
expectedMaxLeftDegree = expectedMaxLeftDegree,
|
||||
leftPowerLawExponent = leftPowerLawExponent,
|
||||
expectedNumRightNodes = expectedNumRightNodes,
|
||||
expectedMaxRightDegree = expectedMaxRightDegree,
|
||||
rightPowerLawExponent = rightPowerLawExponent
|
||||
)
|
||||
|
||||
println("RecosConfig - maxNumSegments " + maxNumSegments)
|
||||
println("RecosConfig - maxNumEdgesPerSegment " + maxNumEdgesPerSegment)
|
||||
println("RecosConfig - expectedNumLeftNodes " + expectedNumLeftNodes)
|
||||
println("RecosConfig - expectedMaxLeftDegree " + expectedMaxLeftDegree)
|
||||
println("RecosConfig - leftPowerLawExponent " + leftPowerLawExponent)
|
||||
println("RecosConfig - expectedNumRightNodes " + expectedNumRightNodes)
|
||||
println("RecosConfig - expectedMaxRightDegree " + expectedMaxRightDegree)
|
||||
println("RecosConfig - rightPowerLawExponent " + rightPowerLawExponent)
|
||||
}
|
@ -0,0 +1,88 @@
|
||||
package com.twitter.recos.user_tweet_graph
|
||||
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.finatra.kafka.consumers.FinagleKafkaConsumerBuilder
|
||||
import com.twitter.graphjet.algorithms.TweetIDMask
|
||||
import com.twitter.recos.util.Action
|
||||
import com.twitter.graphjet.bipartite.MultiSegmentPowerLawBipartiteGraph
|
||||
import com.twitter.graphjet.bipartite.segment.BipartiteGraphSegment
|
||||
import com.twitter.recos.hose.common.UnifiedGraphWriter
|
||||
import com.twitter.recos.internal.thriftscala.RecosHoseMessage
|
||||
import com.twitter.recos.serviceapi.Tweetypie._
|
||||
import com.twitter.recos.user_tweet_graph.util.UserTweetEdgeTypeMask
|
||||
|
||||
/**
|
||||
* The class submits a number of $numBootstrapWriters graph writer threads, BufferedEdgeWriter,
|
||||
* during service startup. One of them is live writer thread, and the other $(numBootstrapWriters - 1)
|
||||
* are catchup writer threads. All of them consume kafka events from an internal concurrent queue,
|
||||
* which is populated by kafka reader threads. At bootstrap time, the kafka reader threads look
|
||||
* back kafka offset from several hours ago and populate the internal concurrent queue.
|
||||
* Each graph writer thread writes to an individual graph segment separately.
|
||||
* The $(numBootstrapWriters - 1) catchup writer threads will stop once all events
|
||||
* between current system time at startup and the time in memcache are processed.
|
||||
* The live writer thread will continue to write all incoming kafka events.
|
||||
* It lives through the entire life cycle of recos graph service.
|
||||
*/
|
||||
case class UserTweetGraphWriter(
|
||||
shardId: String,
|
||||
env: String,
|
||||
hosename: String,
|
||||
bufferSize: Int,
|
||||
kafkaConsumerBuilder: FinagleKafkaConsumerBuilder[String, RecosHoseMessage],
|
||||
clientId: String,
|
||||
statsReceiver: StatsReceiver)
|
||||
extends UnifiedGraphWriter[BipartiteGraphSegment, MultiSegmentPowerLawBipartiteGraph] {
|
||||
writer =>
|
||||
// The max throughput for each kafka consumer is around 25MB/s
|
||||
// Use 4 processors for 100MB/s catch-up speed.
|
||||
val consumerNum: Int = 4
|
||||
// Leave 1 Segments to LiveWriter
|
||||
val catchupWriterNum: Int = RecosConfig.maxNumSegments - 1
|
||||
|
||||
/**
|
||||
* Adds a RecosHoseMessage to the graph. used by live writer to insert edges to the
|
||||
* current segment
|
||||
*/
|
||||
override def addEdgeToGraph(
|
||||
graph: MultiSegmentPowerLawBipartiteGraph,
|
||||
recosHoseMessage: RecosHoseMessage
|
||||
): Unit = {
|
||||
if (Action(recosHoseMessage.action) == Action.Favorite || Action(
|
||||
recosHoseMessage.action) == Action.Retweet)
|
||||
graph.addEdge(
|
||||
recosHoseMessage.leftId,
|
||||
getMetaEdge(recosHoseMessage.rightId, recosHoseMessage.card),
|
||||
UserTweetEdgeTypeMask.actionTypeToEdgeType(recosHoseMessage.action),
|
||||
)
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a RecosHoseMessage to the given segment in the graph. Used by catch up writers to
|
||||
* insert edges to non-current (old) segments
|
||||
*/
|
||||
override def addEdgeToSegment(
|
||||
segment: BipartiteGraphSegment,
|
||||
recosHoseMessage: RecosHoseMessage
|
||||
): Unit = {
|
||||
if (Action(recosHoseMessage.action) == Action.Favorite || Action(
|
||||
recosHoseMessage.action) == Action.Retweet)
|
||||
segment.addEdge(
|
||||
recosHoseMessage.leftId,
|
||||
getMetaEdge(recosHoseMessage.rightId, recosHoseMessage.card),
|
||||
UserTweetEdgeTypeMask.actionTypeToEdgeType(recosHoseMessage.action)
|
||||
)
|
||||
}
|
||||
|
||||
private def getMetaEdge(rightId: Long, cardOption: Option[Byte]): Long = {
|
||||
cardOption
|
||||
.map { card =>
|
||||
if (isPhotoCard(card)) TweetIDMask.photo(rightId)
|
||||
else if (isPlayerCard(card)) TweetIDMask.player(rightId)
|
||||
else if (isSummaryCard(card)) TweetIDMask.summary(rightId)
|
||||
else if (isPromotionCard(card)) TweetIDMask.promotion(rightId)
|
||||
else rightId
|
||||
}
|
||||
.getOrElse(rightId)
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,12 @@
|
||||
scala_library(
|
||||
sources = ["*.scala"],
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"3rdparty/jvm/com/twitter/graphjet",
|
||||
"servo/request/src/main/scala",
|
||||
"src/scala/com/twitter/recos/user_tweet_graph/store",
|
||||
"src/scala/com/twitter/recos/user_tweet_graph/util",
|
||||
"src/scala/com/twitter/recos/util:recos-util",
|
||||
"src/thrift/com/twitter/recos/user_tweet_graph:user_tweet_graph-scala",
|
||||
],
|
||||
)
|
@ -0,0 +1,68 @@
|
||||
package com.twitter.recos.user_tweet_graph.relatedTweetHandlers
|
||||
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.graphjet.bipartite.api.BipartiteGraph
|
||||
import com.twitter.recos.user_tweet_graph.thriftscala._
|
||||
import com.twitter.recos.user_tweet_graph.util.FetchRHSTweetsUtil
|
||||
import com.twitter.recos.user_tweet_graph.util.FilterUtil
|
||||
import com.twitter.recos.user_tweet_graph.util.GetRelatedTweetCandidatesUtil
|
||||
import com.twitter.recos.util.Action
|
||||
import com.twitter.recos.util.Stats._
|
||||
import com.twitter.servo.request._
|
||||
import com.twitter.util.Duration
|
||||
import com.twitter.util.Future
|
||||
import scala.concurrent.duration.HOURS
|
||||
|
||||
/**
|
||||
* Implementation of the Thrift-defined service interface for consumersTweetBasedRelatedTweets.
|
||||
* given a list of consumer userIds, find the tweets they co-engaged with (we're treating input userIds as consumers therefore "consumersTweetBasedRelatedTweets" )
|
||||
* example use case: given a list of user's contacts in their address book, find tweets those contacts engaged with
|
||||
*/
|
||||
class ConsumersBasedRelatedTweetsHandler(
|
||||
bipartiteGraph: BipartiteGraph,
|
||||
statsReceiver: StatsReceiver)
|
||||
extends RequestHandler[ConsumersBasedRelatedTweetRequest, RelatedTweetResponse] {
|
||||
private val stats = statsReceiver.scope(this.getClass.getSimpleName)
|
||||
|
||||
override def apply(request: ConsumersBasedRelatedTweetRequest): Future[RelatedTweetResponse] = {
|
||||
trackFutureBlockStats(stats) {
|
||||
|
||||
val maxResults = request.maxResults.getOrElse(200)
|
||||
val minScore = request.minScore.getOrElse(0.0)
|
||||
val maxTweetAge = request.maxTweetAgeInHours.getOrElse(48)
|
||||
val minResultDegree = request.minResultDegree.getOrElse(50)
|
||||
val minCooccurrence = request.minCooccurrence.getOrElse(3)
|
||||
val excludeTweetIds = request.excludeTweetIds.getOrElse(Seq.empty).toSet
|
||||
|
||||
val consumerSeedSet = request.consumerSeedSet.distinct.filter { userId =>
|
||||
val userDegree = bipartiteGraph.getLeftNodeDegree(userId)
|
||||
// constrain to users that have <100 engagements to avoid spammy behavior
|
||||
userDegree < 100
|
||||
}
|
||||
|
||||
val rhsTweetIds = FetchRHSTweetsUtil.fetchRHSTweets(
|
||||
consumerSeedSet,
|
||||
bipartiteGraph,
|
||||
Set(Action.Favorite, Action.Retweet)
|
||||
)
|
||||
|
||||
val scorePreFactor = 1000.0 / consumerSeedSet.size
|
||||
val relatedTweetCandidates = GetRelatedTweetCandidatesUtil.getRelatedTweetCandidates(
|
||||
rhsTweetIds,
|
||||
minCooccurrence,
|
||||
minResultDegree,
|
||||
scorePreFactor,
|
||||
bipartiteGraph)
|
||||
|
||||
val relatedTweets = relatedTweetCandidates
|
||||
.filter(relatedTweet =>
|
||||
FilterUtil.tweetAgeFilter(
|
||||
relatedTweet.tweetId,
|
||||
Duration(maxTweetAge, HOURS)) && (relatedTweet.score > minScore) && (!excludeTweetIds
|
||||
.contains(relatedTweet.tweetId))).take(maxResults)
|
||||
|
||||
stats.stat("response_size").add(relatedTweets.size)
|
||||
Future.value(RelatedTweetResponse(tweets = relatedTweets))
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,88 @@
|
||||
package com.twitter.recos.user_tweet_graph.relatedTweetHandlers
|
||||
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.graphjet.bipartite.api.BipartiteGraph
|
||||
import com.twitter.recos.user_tweet_graph.thriftscala._
|
||||
import com.twitter.recos.util.Stats._
|
||||
import com.twitter.servo.request._
|
||||
import com.twitter.util.Duration
|
||||
import com.twitter.util.Future
|
||||
import scala.concurrent.duration.HOURS
|
||||
import com.twitter.simclusters_v2.common.UserId
|
||||
import com.twitter.storehaus.ReadableStore
|
||||
import com.twitter.recos.user_tweet_graph.store.UserRecentFollowersStore
|
||||
import com.twitter.recos.user_tweet_graph.util.FetchRHSTweetsUtil
|
||||
import com.twitter.recos.user_tweet_graph.util.FilterUtil
|
||||
import com.twitter.recos.user_tweet_graph.util.GetRelatedTweetCandidatesUtil
|
||||
import com.twitter.recos.util.Action
|
||||
|
||||
/**
|
||||
* Implementation of the Thrift-defined service interface for producerBasedRelatedTweets.
|
||||
*
|
||||
*/
|
||||
class ProducerBasedRelatedTweetsHandler(
|
||||
bipartiteGraph: BipartiteGraph,
|
||||
userRecentFollowersStore: ReadableStore[UserRecentFollowersStore.Query, Seq[UserId]],
|
||||
statsReceiver: StatsReceiver)
|
||||
extends RequestHandler[ProducerBasedRelatedTweetRequest, RelatedTweetResponse] {
|
||||
private val stats = statsReceiver.scope(this.getClass.getSimpleName)
|
||||
|
||||
override def apply(request: ProducerBasedRelatedTweetRequest): Future[RelatedTweetResponse] = {
|
||||
trackFutureBlockStats(stats) {
|
||||
val maxResults = request.maxResults.getOrElse(200)
|
||||
val maxNumFollowers = request.maxNumFollowers.getOrElse(500)
|
||||
val minScore = request.minScore.getOrElse(0.0)
|
||||
val maxTweetAge = request.maxTweetAgeInHours.getOrElse(48)
|
||||
val minResultDegree = request.minResultDegree.getOrElse(50)
|
||||
val minCooccurrence = request.minCooccurrence.getOrElse(4)
|
||||
val excludeTweetIds = request.excludeTweetIds.getOrElse(Seq.empty).toSet
|
||||
|
||||
val followersFut = fetchFollowers(request.producerId, Some(maxNumFollowers))
|
||||
followersFut.map { followers =>
|
||||
val rhsTweetIds = FetchRHSTweetsUtil.fetchRHSTweets(
|
||||
followers,
|
||||
bipartiteGraph,
|
||||
Set(Action.Favorite, Action.Retweet)
|
||||
)
|
||||
|
||||
val scorePreFactor = 1000.0 / followers.size
|
||||
val relatedTweetCandidates = GetRelatedTweetCandidatesUtil.getRelatedTweetCandidates(
|
||||
rhsTweetIds,
|
||||
minCooccurrence,
|
||||
minResultDegree,
|
||||
scorePreFactor,
|
||||
bipartiteGraph)
|
||||
|
||||
val relatedTweets = relatedTweetCandidates
|
||||
.filter { relatedTweet =>
|
||||
FilterUtil.tweetAgeFilter(
|
||||
relatedTweet.tweetId,
|
||||
Duration(maxTweetAge, HOURS)) && (relatedTweet.score > minScore) && (!excludeTweetIds
|
||||
.contains(relatedTweet.tweetId))
|
||||
}.take(maxResults)
|
||||
stats.stat("response_size").add(relatedTweets.size)
|
||||
RelatedTweetResponse(tweets = relatedTweets)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private def fetchFollowers(
|
||||
producerId: Long,
|
||||
maxNumFollower: Option[Int],
|
||||
): Future[Seq[Long]] = {
|
||||
val query =
|
||||
UserRecentFollowersStore.Query(producerId, maxNumFollower, None)
|
||||
|
||||
val followersFut = userRecentFollowersStore.get(query)
|
||||
followersFut.map { followersOpt =>
|
||||
val followers = followersOpt.getOrElse(Seq.empty)
|
||||
val followerIds = followers.distinct.filter { userId =>
|
||||
val userDegree = bipartiteGraph.getLeftNodeDegree(userId)
|
||||
// constrain to more active users that have >1 engagement to optimize latency, and <100 engagements to avoid spammy behavior
|
||||
userDegree > 1 && userDegree < 100
|
||||
}
|
||||
stats.stat("follower_size_after_filter").add(followerIds.size)
|
||||
followerIds
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,93 @@
|
||||
package com.twitter.recos.user_tweet_graph.relatedTweetHandlers
|
||||
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.graphjet.bipartite.api.BipartiteGraph
|
||||
import com.twitter.recos.features.tweet.thriftscala.GraphFeaturesForQuery
|
||||
import com.twitter.recos.user_tweet_graph.thriftscala._
|
||||
import com.twitter.recos.user_tweet_graph.util.FilterUtil
|
||||
import com.twitter.recos.user_tweet_graph.util.FetchRHSTweetsUtil
|
||||
import com.twitter.recos.user_tweet_graph.util.GetAllInternalTweetIdsUtil
|
||||
import com.twitter.recos.user_tweet_graph.util.GetRelatedTweetCandidatesUtil
|
||||
import com.twitter.recos.user_tweet_graph.util.SampleLHSUsersUtil
|
||||
import com.twitter.recos.util.Action
|
||||
import com.twitter.recos.util.Stats._
|
||||
import com.twitter.servo.request._
|
||||
import com.twitter.util.Duration
|
||||
import com.twitter.util.Future
|
||||
import scala.concurrent.duration.HOURS
|
||||
|
||||
/**
|
||||
* Implementation of the Thrift-defined service interface for tweetBasedRelatedTweets.
|
||||
*
|
||||
*/
|
||||
class TweetBasedRelatedTweetsHandler(bipartiteGraph: BipartiteGraph, statsReceiver: StatsReceiver)
|
||||
extends RequestHandler[TweetBasedRelatedTweetRequest, RelatedTweetResponse] {
|
||||
private val stats = statsReceiver.scope(this.getClass.getSimpleName)
|
||||
|
||||
override def apply(request: TweetBasedRelatedTweetRequest): Future[RelatedTweetResponse] = {
|
||||
trackFutureBlockStats(stats) {
|
||||
val internalQueryTweetIds =
|
||||
GetAllInternalTweetIdsUtil.getAllInternalTweetIds(request.tweetId, bipartiteGraph)
|
||||
|
||||
val response = internalQueryTweetIds match {
|
||||
case head +: Nil => getRelatedTweets(request, head)
|
||||
case _ => RelatedTweetResponse()
|
||||
}
|
||||
Future.value(response)
|
||||
}
|
||||
}
|
||||
|
||||
private def getRelatedTweets(
|
||||
request: TweetBasedRelatedTweetRequest,
|
||||
maskedTweetId: Long
|
||||
): RelatedTweetResponse = {
|
||||
|
||||
val maxNumSamplesPerNeighbor = request.maxNumSamplesPerNeighbor.getOrElse(100)
|
||||
val maxResults = request.maxResults.getOrElse(200)
|
||||
val minScore = request.minScore.getOrElse(0.5)
|
||||
val maxTweetAge = request.maxTweetAgeInHours.getOrElse(48)
|
||||
val minResultDegree = request.minResultDegree.getOrElse(50)
|
||||
val minQueryDegree = request.minQueryDegree.getOrElse(10)
|
||||
val minCooccurrence = request.minCooccurrence.getOrElse(3)
|
||||
val excludeTweetIds = request.excludeTweetIds.getOrElse(Seq.empty).toSet
|
||||
|
||||
val queryTweetDegree = bipartiteGraph.getRightNodeDegree(maskedTweetId)
|
||||
stats.stat("queryTweetDegree").add(queryTweetDegree)
|
||||
|
||||
if (queryTweetDegree < minQueryDegree) {
|
||||
stats.counter("queryTweetDegreeLessThanMinQueryDegree").incr()
|
||||
RelatedTweetResponse()
|
||||
} else {
|
||||
|
||||
val sampledLHSuserIds =
|
||||
SampleLHSUsersUtil.sampleLHSUsers(maskedTweetId, maxNumSamplesPerNeighbor, bipartiteGraph)
|
||||
|
||||
val rHStweetIds = FetchRHSTweetsUtil.fetchRHSTweets(
|
||||
sampledLHSuserIds,
|
||||
bipartiteGraph,
|
||||
Set(Action.Favorite, Action.Retweet)
|
||||
)
|
||||
|
||||
val scorePreFactor =
|
||||
queryTweetDegree / math.log(queryTweetDegree) / sampledLHSuserIds.distinct.size
|
||||
val relatedTweetCandidates = GetRelatedTweetCandidatesUtil.getRelatedTweetCandidates(
|
||||
rHStweetIds,
|
||||
minCooccurrence,
|
||||
minResultDegree,
|
||||
scorePreFactor,
|
||||
bipartiteGraph)
|
||||
|
||||
val relatedTweets = relatedTweetCandidates
|
||||
.filter(relatedTweet =>
|
||||
FilterUtil.tweetAgeFilter(
|
||||
relatedTweet.tweetId,
|
||||
Duration(maxTweetAge, HOURS)) && (relatedTweet.score > minScore) && (!excludeTweetIds
|
||||
.contains(relatedTweet.tweetId))).take(maxResults)
|
||||
|
||||
stats.stat("response_size").add(relatedTweets.size)
|
||||
RelatedTweetResponse(
|
||||
tweets = relatedTweets,
|
||||
queryTweetGraphFeatures = Some(GraphFeaturesForQuery(degree = Some(queryTweetDegree))))
|
||||
}
|
||||
}
|
||||
}
|
9
src/scala/com/twitter/recos/user_tweet_graph/store/BUILD
Normal file
9
src/scala/com/twitter/recos/user_tweet_graph/store/BUILD
Normal file
@ -0,0 +1,9 @@
|
||||
scala_library(
|
||||
sources = ["*.scala"],
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"3rdparty/jvm/com/twitter/storehaus:core",
|
||||
"src/scala/com/twitter/simclusters_v2/common",
|
||||
"src/thrift/com/twitter/socialgraph:thrift-scala",
|
||||
],
|
||||
)
|
@ -0,0 +1,50 @@
|
||||
package com.twitter.recos.user_tweet_graph.store
|
||||
|
||||
import com.twitter.simclusters_v2.common.UserId
|
||||
import com.twitter.socialgraph.thriftscala.EdgesRequest
|
||||
import com.twitter.socialgraph.thriftscala.EdgesResult
|
||||
import com.twitter.socialgraph.thriftscala.PageRequest
|
||||
import com.twitter.socialgraph.thriftscala.RelationshipType
|
||||
import com.twitter.socialgraph.thriftscala.SrcRelationship
|
||||
import com.twitter.socialgraph.thriftscala.SocialGraphService
|
||||
import com.twitter.storehaus.ReadableStore
|
||||
import com.twitter.util.Duration
|
||||
import com.twitter.util.Future
|
||||
import com.twitter.util.Time
|
||||
|
||||
class UserRecentFollowersStore(
|
||||
sgsClient: SocialGraphService.MethodPerEndpoint)
|
||||
extends ReadableStore[UserRecentFollowersStore.Query, Seq[UserId]] {
|
||||
|
||||
override def get(key: UserRecentFollowersStore.Query): Future[Option[Seq[UserId]]] = {
|
||||
val edgeRequest = EdgesRequest(
|
||||
relationship = SrcRelationship(key.userId, RelationshipType.FollowedBy),
|
||||
// Could have a better guess at count when k.maxAge != None
|
||||
pageRequest = Some(PageRequest(count = key.maxResults))
|
||||
)
|
||||
|
||||
val lookbackThresholdMillis = key.maxAge
|
||||
.map(maxAge => (Time.now - maxAge).inMilliseconds)
|
||||
.getOrElse(0L)
|
||||
|
||||
sgsClient
|
||||
.edges(Seq(edgeRequest))
|
||||
.map(_.flatMap {
|
||||
case EdgesResult(edges, _, _) =>
|
||||
edges.collect {
|
||||
case e if e.createdAt >= lookbackThresholdMillis =>
|
||||
e.target
|
||||
}
|
||||
})
|
||||
.map(Some(_))
|
||||
}
|
||||
}
|
||||
|
||||
object UserRecentFollowersStore {
|
||||
case class Query(
|
||||
userId: UserId,
|
||||
// maxResults - if Some(count), we return only the `count` most recent follows
|
||||
maxResults: Option[Int] = None,
|
||||
// maxAge - if Some(duration), return only follows since `Time.now - duration`
|
||||
maxAge: Option[Duration] = None)
|
||||
}
|
12
src/scala/com/twitter/recos/user_tweet_graph/util/BUILD
Normal file
12
src/scala/com/twitter/recos/user_tweet_graph/util/BUILD
Normal file
@ -0,0 +1,12 @@
|
||||
scala_library(
|
||||
sources = ["*.scala"],
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"3rdparty/jvm/com/twitter/graphjet",
|
||||
"snowflake:id",
|
||||
"snowflake/src/main/scala/com/twitter/snowflake/id",
|
||||
"src/scala/com/twitter/recos/util:recos-util",
|
||||
"src/scala/com/twitter/simclusters_v2/common",
|
||||
"src/thrift/com/twitter/recos/user_tweet_graph:user_tweet_graph-scala",
|
||||
],
|
||||
)
|
@ -0,0 +1,34 @@
|
||||
package com.twitter.recos.user_tweet_graph.util
|
||||
|
||||
import com.twitter.graphjet.bipartite.MultiSegmentIterator
|
||||
import com.twitter.graphjet.bipartite.api.BipartiteGraph
|
||||
import com.twitter.graphjet.bipartite.segment.BipartiteGraphSegment
|
||||
import scala.collection.mutable.ListBuffer
|
||||
import com.twitter.recos.util.Action
|
||||
|
||||
object FetchRHSTweetsUtil {
|
||||
// get RHS tweets given LHS users
|
||||
def fetchRHSTweets(
|
||||
userIds: Seq[Long],
|
||||
bipartiteGraph: BipartiteGraph,
|
||||
allowedActions: Set[Action.Value]
|
||||
): Seq[Long] = {
|
||||
val allowedActionStrings = allowedActions.map(_.toString)
|
||||
userIds.distinct
|
||||
.flatMap { userId =>
|
||||
val tweetIdsIterator = bipartiteGraph
|
||||
.getLeftNodeEdges(userId).asInstanceOf[MultiSegmentIterator[BipartiteGraphSegment]]
|
||||
|
||||
val tweetIds = new ListBuffer[Long]()
|
||||
if (tweetIdsIterator != null) {
|
||||
while (tweetIdsIterator.hasNext) {
|
||||
val rightNode = tweetIdsIterator.nextLong()
|
||||
val edgeType = tweetIdsIterator.currentEdgeType()
|
||||
if (allowedActionStrings.contains(UserTweetEdgeTypeMask(edgeType).toString))
|
||||
tweetIds += rightNode
|
||||
}
|
||||
}
|
||||
tweetIds.distinct
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,15 @@
|
||||
package com.twitter.recos.user_tweet_graph.util
|
||||
|
||||
import com.twitter.simclusters_v2.common.TweetId
|
||||
import com.twitter.snowflake.id.SnowflakeId
|
||||
import com.twitter.util.Duration
|
||||
import com.twitter.util.Time
|
||||
|
||||
object FilterUtil {
|
||||
def tweetAgeFilter(tweetId: TweetId, maxAge: Duration): Boolean = {
|
||||
SnowflakeId
|
||||
.timeFromIdOpt(tweetId)
|
||||
.map { tweetTime => tweetTime > Time.now - maxAge }.getOrElse(false)
|
||||
// If there's no snowflake timestamp, we have no idea when this tweet happened.
|
||||
}
|
||||
}
|
@ -0,0 +1,33 @@
|
||||
package com.twitter.recos.user_tweet_graph.util
|
||||
|
||||
import com.twitter.graphjet.algorithms.TweetIDMask
|
||||
import com.twitter.graphjet.bipartite.api.BipartiteGraph
|
||||
|
||||
object GetAllInternalTweetIdsUtil {
|
||||
|
||||
def getAllInternalTweetIds(tweetId: Long, bipartiteGraph: BipartiteGraph): Seq[Long] = {
|
||||
val internalTweetIds = getAllMasks(tweetId)
|
||||
sortByDegrees(internalTweetIds, bipartiteGraph)
|
||||
}
|
||||
|
||||
private def getAllMasks(tweetId: Long): Seq[Long] = {
|
||||
Seq(
|
||||
tweetId,
|
||||
TweetIDMask.summary(tweetId),
|
||||
TweetIDMask.photo(tweetId),
|
||||
TweetIDMask.player(tweetId),
|
||||
TweetIDMask.promotion(tweetId)
|
||||
)
|
||||
}
|
||||
|
||||
private def sortByDegrees(
|
||||
encodedTweetIds: Seq[Long],
|
||||
bipartiteGraph: BipartiteGraph
|
||||
): Seq[Long] = {
|
||||
encodedTweetIds
|
||||
.map { encodedTweetId => (encodedTweetId, bipartiteGraph.getRightNodeDegree(encodedTweetId)) }
|
||||
.filter { case (_, degree) => degree > 0 } // keep only tweetds with positive degree
|
||||
.sortBy { case (_, degree) => -degree } // sort by degree in descending order
|
||||
.map { case (encodedTweetId, _) => encodedTweetId }
|
||||
}
|
||||
}
|
@ -0,0 +1,56 @@
|
||||
package com.twitter.recos.user_tweet_graph.util
|
||||
|
||||
import com.twitter.graphjet.bipartite.api.BipartiteGraph
|
||||
import com.twitter.recos.user_tweet_graph.thriftscala._
|
||||
import com.twitter.recos.features.tweet.thriftscala.GraphFeaturesForTweet
|
||||
import com.twitter.graphjet.algorithms.TweetIDMask
|
||||
|
||||
object GetRelatedTweetCandidatesUtil {
|
||||
private val tweetIDMask = new TweetIDMask
|
||||
|
||||
/**
|
||||
* calculate scores for each RHS tweet that we get back
|
||||
* for tweetBasedRelatedTweet, scorePreFactor = queryTweetDegree / log(queryTweetDegree) / LHSuserSize
|
||||
* and the final score will be a log-cosine score
|
||||
* for non-tweetBasedRelatedTweet, We don't have a query tweet, to keep scoring function consistent,
|
||||
* scorePreFactor = 1000.0 / LHSuserSize (queryTweetDegree's average is ~10k, 1000 ~= 10k/log(10k))
|
||||
* Though scorePreFactor is applied for all results within a request, it's still useful to make score comparable across requests,
|
||||
* so we can have a unifed min_score and help with downstream score normalization
|
||||
* **/
|
||||
def getRelatedTweetCandidates(
|
||||
relatedTweetCandidates: Seq[Long],
|
||||
minCooccurrence: Int,
|
||||
minResultDegree: Int,
|
||||
scorePreFactor: Double,
|
||||
bipartiteGraph: BipartiteGraph,
|
||||
): Seq[RelatedTweet] = {
|
||||
relatedTweetCandidates
|
||||
.groupBy(tweetId => tweetId)
|
||||
.filterKeys(tweetId => bipartiteGraph.getRightNodeDegree(tweetId) > minResultDegree)
|
||||
.mapValues(_.size)
|
||||
.filter { case (_, cooccurrence) => cooccurrence >= minCooccurrence }
|
||||
.toSeq
|
||||
.map {
|
||||
case (relatedTweetId, cooccurrence) =>
|
||||
val relatedTweetDegree = bipartiteGraph.getRightNodeDegree(relatedTweetId)
|
||||
val score = scorePreFactor * cooccurrence / math.log(relatedTweetDegree)
|
||||
|
||||
toRelatedTweet(relatedTweetId, score, relatedTweetDegree, cooccurrence)
|
||||
}
|
||||
.sortBy(-_.score)
|
||||
}
|
||||
|
||||
def toRelatedTweet(
|
||||
relatedTweetId: Long,
|
||||
score: Double,
|
||||
relatedTweetDegree: Int,
|
||||
cooccurrence: Int
|
||||
): RelatedTweet = {
|
||||
RelatedTweet(
|
||||
tweetId = tweetIDMask.restore(relatedTweetId),
|
||||
score = score,
|
||||
relatedTweetGraphFeatures = Some(
|
||||
GraphFeaturesForTweet(cooccurrence = Some(cooccurrence), degree = Some(relatedTweetDegree)))
|
||||
)
|
||||
}
|
||||
}
|
@ -0,0 +1,35 @@
|
||||
package com.twitter.recos.user_tweet_graph.util
|
||||
|
||||
import com.twitter.graphjet.bipartite.MultiSegmentIterator
|
||||
import com.twitter.graphjet.bipartite.api.BipartiteGraph
|
||||
import com.twitter.graphjet.bipartite.segment.BipartiteGraphSegment
|
||||
import java.util.Random
|
||||
import scala.collection.mutable.ListBuffer
|
||||
|
||||
object SampleLHSUsersUtil {
|
||||
// sample userId nodes
|
||||
def sampleLHSUsers(
|
||||
maskedTweetId: Long,
|
||||
maxNumSamplesPerNeighbor: Int,
|
||||
bipartiteGraph: BipartiteGraph
|
||||
): Seq[Long] = {
|
||||
val sampledUserIdsIterator = bipartiteGraph
|
||||
.getRandomRightNodeEdges(
|
||||
maskedTweetId,
|
||||
maxNumSamplesPerNeighbor,
|
||||
new Random(System.currentTimeMillis)).asInstanceOf[MultiSegmentIterator[
|
||||
BipartiteGraphSegment
|
||||
]]
|
||||
|
||||
val userIds = new ListBuffer[Long]()
|
||||
if (sampledUserIdsIterator != null) {
|
||||
while (sampledUserIdsIterator.hasNext) {
|
||||
val leftNode = sampledUserIdsIterator.nextLong()
|
||||
// If a user likes too many things, we risk including spammy behavior.
|
||||
if (bipartiteGraph.getLeftNodeDegree(leftNode) < 100)
|
||||
userIds += leftNode
|
||||
}
|
||||
}
|
||||
userIds
|
||||
}
|
||||
}
|
@ -0,0 +1,77 @@
|
||||
package com.twitter.recos.user_tweet_graph.util
|
||||
|
||||
import com.twitter.graphjet.bipartite.api.EdgeTypeMask
|
||||
import com.twitter.recos.util.Action
|
||||
|
||||
/**
|
||||
* The bit mask is used to encode edge types in the top bits of an integer,
|
||||
* e.g. favorite, retweet, reply and click. Under current segment configuration, each segment
|
||||
* stores up to 128M edges. Assuming that each node on one side is unique, each segment
|
||||
* stores up to 128M unique nodes on one side, which occupies the lower 27 bits of an integer.
|
||||
* This leaves five bits to encode the edge types, which at max can store 32 edge types.
|
||||
* The following implementation utilizes the top four bits and leaves one free bit out.
|
||||
*/
|
||||
class UserTweetEdgeTypeMask extends EdgeTypeMask {
|
||||
import UserTweetEdgeTypeMask._
|
||||
|
||||
override def encode(node: Int, edgeType: Byte): Int = {
|
||||
if (edgeType < 0 || edgeType > SIZE || edgeType == Click.id.toByte) {
|
||||
throw new IllegalArgumentException("encode: Illegal edge type argument " + edgeType)
|
||||
} else {
|
||||
node | (edgeType << 28)
|
||||
}
|
||||
}
|
||||
|
||||
override def edgeType(node: Int): Byte = {
|
||||
(node >>> 28).toByte
|
||||
}
|
||||
|
||||
override def restore(node: Int): Int = {
|
||||
node & MASK
|
||||
}
|
||||
}
|
||||
|
||||
object UserTweetEdgeTypeMask extends Enumeration {
|
||||
|
||||
type UserTweetEdgeTypeMask = Value
|
||||
|
||||
/**
|
||||
* Byte values corresponding to the action taken on a tweet, which will be encoded in the
|
||||
* top 4 bits in a tweet Id
|
||||
* NOTE: THERE CAN ONLY BE UP TO 16 TYPES
|
||||
*/
|
||||
val Click: UserTweetEdgeTypeMask = Value(0)
|
||||
val Favorite: UserTweetEdgeTypeMask = Value(1)
|
||||
val Retweet: UserTweetEdgeTypeMask = Value(2)
|
||||
val Reply: UserTweetEdgeTypeMask = Value(3)
|
||||
val Tweet: UserTweetEdgeTypeMask = Value(4)
|
||||
val IsMentioned: UserTweetEdgeTypeMask = Value(5)
|
||||
val IsMediatagged: UserTweetEdgeTypeMask = Value(6)
|
||||
val Quote: UserTweetEdgeTypeMask = Value(7)
|
||||
val Unfavorite: UserTweetEdgeTypeMask = Value(8)
|
||||
|
||||
/**
|
||||
* Reserve the top four bits of each integer to encode the edge type information.
|
||||
*/
|
||||
val MASK: Int = Integer.parseInt("00001111111111111111111111111111", 2)
|
||||
val SIZE: Int = this.values.size
|
||||
|
||||
/**
|
||||
* Converts the action byte in the RecosHoseMessage into GraphJet internal byte mapping
|
||||
*/
|
||||
def actionTypeToEdgeType(actionByte: Byte): Byte = {
|
||||
val edgeType = Action(actionByte) match {
|
||||
case Action.Favorite => Favorite.id
|
||||
case Action.Retweet => Retweet.id
|
||||
case Action.Reply => Reply.id
|
||||
case Action.Tweet => Tweet.id
|
||||
case Action.IsMentioned => IsMentioned.id
|
||||
case Action.IsMediaTagged => IsMediatagged.id
|
||||
case Action.Quote => Quote.id
|
||||
case Action.Unfavorite => Unfavorite.id
|
||||
case _ =>
|
||||
throw new IllegalArgumentException("getEdgeType: Illegal edge type argument " + actionByte)
|
||||
}
|
||||
edgeType.toByte
|
||||
}
|
||||
}
|
45
src/scala/com/twitter/recos/user_user_graph/BUILD
Normal file
45
src/scala/com/twitter/recos/user_user_graph/BUILD
Normal file
@ -0,0 +1,45 @@
|
||||
scala_library(
|
||||
name = "user_user_graph",
|
||||
sources = ["*.scala"],
|
||||
strict_deps = False,
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"3rdparty/jvm/com/google/guava",
|
||||
"3rdparty/jvm/com/twitter/graphjet",
|
||||
"3rdparty/jvm/io/netty:netty4-tcnative-boringssl-static",
|
||||
"3rdparty/jvm/org/apache/kafka:rosette-kafka",
|
||||
"finagle-internal/mtls/src/main/scala/com/twitter/finagle/mtls/authentication",
|
||||
"finagle-internal/mtls/src/main/scala/com/twitter/finagle/mtls/server",
|
||||
"finagle/finagle-core/src/main",
|
||||
"finagle/finagle-http/src/main/scala",
|
||||
"finagle/finagle-memcached/src/main/scala",
|
||||
"finagle/finagle-stats/src/main/scala",
|
||||
"finagle/finagle-thriftmux/src/main/scala",
|
||||
"servo/request/src/main/scala",
|
||||
"servo/util/src/main/scala",
|
||||
"src/resources/com/twitter/recos:decider",
|
||||
"src/scala/com/twitter/recos/decider",
|
||||
"src/scala/com/twitter/recos/graph_common",
|
||||
"src/scala/com/twitter/recos/hose/common",
|
||||
"src/scala/com/twitter/recos/model:recos-model",
|
||||
"src/scala/com/twitter/recos/util:recos-util",
|
||||
"src/thrift/com/twitter/recos/user_user_graph:user_user_graph-scala",
|
||||
"thrift-web-forms/src/main/scala/com/twitter/thriftwebforms",
|
||||
"twitter-server/slf4j-jdk14/src/main/scala/com/twitter/server/logging",
|
||||
"util/util-logging/src/main/scala",
|
||||
"util/util-stats/src/main/scala",
|
||||
],
|
||||
)
|
||||
|
||||
jvm_binary(
|
||||
name = "bin",
|
||||
basename = "user_user_graph-server",
|
||||
main = "com.twitter.recos.user_user_graph.Main",
|
||||
runtime_platform = "java11",
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
":user_user_graph",
|
||||
"3rdparty/jvm/org/slf4j:slf4j-jdk14",
|
||||
"twitter-server/slf4j-jdk14/src/main/scala",
|
||||
],
|
||||
)
|
@ -0,0 +1,13 @@
|
||||
package com.twitter.recos.user_user_graph
|
||||
|
||||
/**
|
||||
* The class holds all the config parameters for kafka queue.
|
||||
*/
|
||||
object KafkaConfig {
|
||||
// The size of the RecosHoseMessage array that is written to the concurrently linked queue
|
||||
// Buffersize of 64 to keep throughput around 64 / (2K edgesPerSec / 150 kafka threads) = 6 seconds, which is lower
|
||||
// than young gen gc cycle, 20 seconds. So that all the incoming messages will be gced in young gen instead of old gen.
|
||||
val bufferSize = 64
|
||||
|
||||
println("KafkaConfig - bufferSize " + bufferSize)
|
||||
}
|
@ -0,0 +1,51 @@
|
||||
package com.twitter.recos.user_user_graph
|
||||
|
||||
import com.twitter.logging.Logger
|
||||
import com.twitter.recos.user_user_graph.thriftscala._
|
||||
import com.twitter.util.Future
|
||||
|
||||
trait LoggingUserUserGraph extends thriftscala.UserUserGraph.MethodPerEndpoint {
|
||||
private[this] val accessLog = Logger("access")
|
||||
|
||||
abstract override def recommendUsers(
|
||||
request: RecommendUserRequest
|
||||
): Future[RecommendUserResponse] = {
|
||||
val time = System.currentTimeMillis
|
||||
super.recommendUsers(request) onSuccess { resp =>
|
||||
val timeTaken = System.currentTimeMillis - time
|
||||
val logText =
|
||||
s"In ${timeTaken}ms, recommendUsers(${requestToString(request)}), response ${responseToString(resp)}"
|
||||
accessLog.info(logText)
|
||||
} onFailure { exc =>
|
||||
val timeTaken = System.currentTimeMillis - time
|
||||
val logText = s"In ${timeTaken}ms, recommendUsers(${requestToString(request)} returned error"
|
||||
accessLog.error(exc, logText)
|
||||
}
|
||||
}
|
||||
|
||||
private def requestToString(request: RecommendUserRequest): String = {
|
||||
Seq(
|
||||
request.requesterId,
|
||||
request.displayLocation,
|
||||
request.seedsWithWeights.size,
|
||||
request.seedsWithWeights.take(5),
|
||||
request.excludedUserIds.map(_.size).getOrElse(0),
|
||||
request.excludedUserIds.map(_.take(5)),
|
||||
request.maxNumResults,
|
||||
request.maxNumSocialProofs,
|
||||
request.minUserPerSocialProof,
|
||||
request.socialProofTypes,
|
||||
request.maxEdgeEngagementAgeInMillis
|
||||
).mkString(",")
|
||||
}
|
||||
|
||||
private def responseToString(response: RecommendUserResponse): String = {
|
||||
response.recommendedUsers.toList.map { recUser =>
|
||||
val socialProof = recUser.socialProofs.map {
|
||||
case (proofType, proofs) =>
|
||||
(proofType, proofs)
|
||||
}
|
||||
(recUser.userId, recUser.score, socialProof)
|
||||
}.toString
|
||||
}
|
||||
}
|
255
src/scala/com/twitter/recos/user_user_graph/Main.scala
Normal file
255
src/scala/com/twitter/recos/user_user_graph/Main.scala
Normal file
@ -0,0 +1,255 @@
|
||||
package com.twitter.recos.user_user_graph
|
||||
|
||||
import com.twitter.abdecider.ABDeciderFactory
|
||||
import com.twitter.abdecider.LoggingABDecider
|
||||
import com.twitter.app.Flag
|
||||
import com.twitter.conversions.DurationOps._
|
||||
import com.twitter.finagle.ThriftMux
|
||||
import com.twitter.finagle.http.HttpMuxer
|
||||
import com.twitter.finagle.mtls.authentication.ServiceIdentifier
|
||||
import com.twitter.finagle.mtls.server.MtlsStackServer._
|
||||
import com.twitter.finagle.mux.transport.OpportunisticTls
|
||||
import com.twitter.finagle.thrift.ClientId
|
||||
import com.twitter.finatra.kafka.consumers.FinagleKafkaConsumerBuilder
|
||||
import com.twitter.finatra.kafka.domain.KafkaGroupId
|
||||
import com.twitter.finatra.kafka.domain.SeekStrategy
|
||||
import com.twitter.finatra.kafka.serde.ScalaSerdes
|
||||
import com.twitter.frigate.common.util.ElfOwlFilter
|
||||
import com.twitter.frigate.common.util.ElfOwlFilter.ByLdapGroup
|
||||
import com.twitter.logging._
|
||||
import com.twitter.recos.decider.UserUserGraphDecider
|
||||
import com.twitter.recos.graph_common.FinagleStatsReceiverWrapper
|
||||
import com.twitter.recos.graph_common.NodeMetadataLeftIndexedPowerLawMultiSegmentBipartiteGraphBuilder
|
||||
import com.twitter.recos.internal.thriftscala.RecosHoseMessage
|
||||
import com.twitter.recos.model.Constants
|
||||
import com.twitter.recos.user_user_graph.KafkaConfig._
|
||||
import com.twitter.recos.user_user_graph.RecosConfig._
|
||||
import com.twitter.server.Deciderable
|
||||
import com.twitter.server.TwitterServer
|
||||
import com.twitter.server.logging.{Logging => JDK14Logging}
|
||||
import com.twitter.servo.request._
|
||||
import com.twitter.servo.util.ExceptionCounter
|
||||
import com.twitter.thriftwebforms._
|
||||
import com.twitter.util.Await
|
||||
import com.twitter.util.Duration
|
||||
import java.net.InetSocketAddress
|
||||
import java.util.concurrent.TimeUnit
|
||||
import org.apache.kafka.clients.CommonClientConfigs
|
||||
import org.apache.kafka.common.config.SaslConfigs
|
||||
import org.apache.kafka.common.config.SslConfigs
|
||||
import org.apache.kafka.common.security.auth.SecurityProtocol
|
||||
import org.apache.kafka.common.serialization.StringDeserializer
|
||||
|
||||
object Main extends TwitterServer with JDK14Logging with Deciderable {
|
||||
profile =>
|
||||
|
||||
val shardId: Flag[Int] = flag("shardId", 0, "Shard ID")
|
||||
val servicePort: Flag[InetSocketAddress] =
|
||||
flag("service.port", new InetSocketAddress(10143), "Thrift service port")
|
||||
val logDir: Flag[String] = flag("logdir", "recos", "Logging directory")
|
||||
val hoseName: Flag[String] =
|
||||
flag("hosename", "recos_injector_user_user", "the kafka stream used for incoming edges")
|
||||
val maxNumSegments: Flag[Int] =
|
||||
flag("maxNumSegments", graphBuilderConfig.maxNumSegments, "the number of segments in the graph")
|
||||
val numShards: Flag[Int] = flag("numShards", 1, "Number of shards for this service")
|
||||
val truststoreLocation: Flag[String] =
|
||||
flag[String]("truststore_location", "", "Truststore file location")
|
||||
|
||||
val dataCenter: Flag[String] = flag("service.cluster", "atla", "Data Center")
|
||||
val serviceRole: Flag[String] = flag("service.role", "Service Role")
|
||||
val serviceEnv: Flag[String] = flag("service.env", "Service Env")
|
||||
val serviceName: Flag[String] = flag("service.name", "Service Name")
|
||||
|
||||
val statsReceiverWrapper: FinagleStatsReceiverWrapper = FinagleStatsReceiverWrapper(
|
||||
statsReceiver
|
||||
)
|
||||
|
||||
/**
|
||||
* A ClientRequestAuthorizer to be used in a request-authorization RequestFilter.
|
||||
*/
|
||||
lazy val clientAuthorizer: ClientRequestAuthorizer =
|
||||
ClientRequestAuthorizer.observed(
|
||||
ClientRequestAuthorizer.permissive,
|
||||
new ClientRequestObserver(statsReceiver)
|
||||
)
|
||||
|
||||
lazy val clientId = ClientId("userusergraph.%s".format(serviceEnv().replace("devel", "dev")))
|
||||
|
||||
val shutdownTimeout: Flag[Duration] = flag(
|
||||
"service.shutdownTimeout",
|
||||
5.seconds,
|
||||
"Maximum amount of time to wait for pending requests to complete on shutdown"
|
||||
)
|
||||
|
||||
/**
|
||||
* ExceptionCounter for tracking failures from RequestHandler(s).
|
||||
*/
|
||||
lazy val exceptionCounter = new ExceptionCounter(statsReceiver)
|
||||
|
||||
/**
|
||||
* Function for translating exceptions returned by a RequestHandler. Useful
|
||||
* for cases where underlying exception types should be wrapped in those
|
||||
* defined in the project's Thrift IDL.
|
||||
*/
|
||||
lazy val translateExceptions: PartialFunction[Throwable, Throwable] = {
|
||||
case t => t
|
||||
}
|
||||
|
||||
// ********* logging **********
|
||||
|
||||
lazy val loggingLevel: Level = Level.INFO
|
||||
lazy val recosLogPath: String = logDir() + "/recos.log"
|
||||
lazy val graphLogPath: String = logDir() + "/graph.log"
|
||||
lazy val accessLogPath: String = logDir() + "/access.log"
|
||||
|
||||
override def loggerFactories: List[LoggerFactory] =
|
||||
List(
|
||||
LoggerFactory(
|
||||
level = Some(loggingLevel),
|
||||
handlers = QueueingHandler(
|
||||
handler = FileHandler(
|
||||
filename = recosLogPath,
|
||||
level = Some(loggingLevel),
|
||||
rollPolicy = Policy.Hourly,
|
||||
rotateCount = 6,
|
||||
formatter = new Formatter
|
||||
)
|
||||
) :: Nil
|
||||
),
|
||||
LoggerFactory(
|
||||
node = "graph",
|
||||
useParents = false,
|
||||
level = Some(loggingLevel),
|
||||
handlers = QueueingHandler(
|
||||
handler = FileHandler(
|
||||
filename = graphLogPath,
|
||||
level = Some(loggingLevel),
|
||||
rollPolicy = Policy.Hourly,
|
||||
rotateCount = 6,
|
||||
formatter = new Formatter
|
||||
)
|
||||
) :: Nil
|
||||
),
|
||||
LoggerFactory(
|
||||
node = "access",
|
||||
useParents = false,
|
||||
level = Some(loggingLevel),
|
||||
handlers = QueueingHandler(
|
||||
handler = FileHandler(
|
||||
filename = accessLogPath,
|
||||
level = Some(loggingLevel),
|
||||
rollPolicy = Policy.Hourly,
|
||||
rotateCount = 6,
|
||||
formatter = new Formatter
|
||||
)
|
||||
) :: Nil
|
||||
),
|
||||
LoggerFactory(
|
||||
node = "client_event",
|
||||
level = Some(loggingLevel),
|
||||
useParents = false,
|
||||
handlers = QueueingHandler(
|
||||
maxQueueSize = 10000,
|
||||
handler = ScribeHandler(
|
||||
category = "client_event",
|
||||
formatter = BareFormatter
|
||||
)
|
||||
) :: Nil
|
||||
)
|
||||
)
|
||||
// ******** Decider *************
|
||||
|
||||
val recosDecider: UserUserGraphDecider = UserUserGraphDecider()
|
||||
|
||||
// ********* ABdecider **********
|
||||
|
||||
val abDeciderYmlPath: String = "/usr/local/config/abdecider/abdecider.yml"
|
||||
|
||||
val scribeLogger: Option[Logger] = Some(Logger.get("client_event"))
|
||||
|
||||
val abDecider: LoggingABDecider =
|
||||
ABDeciderFactory(
|
||||
abDeciderYmlPath = abDeciderYmlPath,
|
||||
scribeLogger = scribeLogger,
|
||||
environment = Some("production")
|
||||
).buildWithLogging()
|
||||
|
||||
val ldapGroups = Seq("eng", "cassowary-group", "timeline-team")
|
||||
|
||||
// ********* Recos service **********
|
||||
|
||||
def main(): Unit = {
|
||||
log.info("building graph with maxNumSegments = " + profile.maxNumSegments())
|
||||
log.info("Reading from: " + hoseName())
|
||||
|
||||
val graph = NodeMetadataLeftIndexedPowerLawMultiSegmentBipartiteGraphBuilder(
|
||||
graphBuilderConfig.copy(maxNumSegments = profile.maxNumSegments()),
|
||||
statsReceiverWrapper
|
||||
)
|
||||
|
||||
val kafkaConfigBuilder = FinagleKafkaConsumerBuilder[String, RecosHoseMessage]()
|
||||
.dest("/s/kafka/recommendations:kafka-tls")
|
||||
.groupId(KafkaGroupId(f"user_user_graph-${shardId()}%06d"))
|
||||
.keyDeserializer(new StringDeserializer)
|
||||
.valueDeserializer(ScalaSerdes.Thrift[RecosHoseMessage].deserializer)
|
||||
.seekStrategy(SeekStrategy.REWIND)
|
||||
.rewindDuration(24.hours)
|
||||
.withConfig(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, SecurityProtocol.SASL_SSL.toString)
|
||||
.withConfig(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, truststoreLocation())
|
||||
.withConfig(SaslConfigs.SASL_MECHANISM, SaslConfigs.GSSAPI_MECHANISM)
|
||||
.withConfig(SaslConfigs.SASL_KERBEROS_SERVICE_NAME, "kafka")
|
||||
.withConfig(SaslConfigs.SASL_KERBEROS_SERVER_NAME, "kafka")
|
||||
|
||||
val graphWriter = UserUserGraphWriter(
|
||||
shardId = shardId().toString,
|
||||
env = serviceEnv(),
|
||||
hosename = hoseName(),
|
||||
bufferSize = bufferSize,
|
||||
kafkaConsumerBuilder = kafkaConfigBuilder,
|
||||
clientId = clientId.name,
|
||||
statsReceiver = statsReceiver
|
||||
)
|
||||
graphWriter.initHose(graph)
|
||||
|
||||
val recommendUsersHandler = RecommendUsersHandlerImpl(
|
||||
graph,
|
||||
Constants.salsaRunnerConfig,
|
||||
recosDecider,
|
||||
statsReceiverWrapper
|
||||
)
|
||||
|
||||
val recos = new UserUserGraph(recommendUsersHandler) with LoggingUserUserGraph
|
||||
|
||||
// For MutualTLS
|
||||
val serviceIdentifier = ServiceIdentifier(
|
||||
role = serviceRole(),
|
||||
service = serviceName(),
|
||||
environment = serviceEnv(),
|
||||
zone = dataCenter()
|
||||
)
|
||||
|
||||
val thriftServer = ThriftMux.server
|
||||
.withOpportunisticTls(OpportunisticTls.Required)
|
||||
.withMutualTls(serviceIdentifier)
|
||||
.serveIface(servicePort(), recos)
|
||||
|
||||
this.addAdminRoute(ElfOwlFilter.getPostbackRoute())
|
||||
|
||||
val elfowlFilter = ElfOwlFilter(
|
||||
ByLdapGroup(ldapGroups),
|
||||
Duration.fromTimeUnit(5, TimeUnit.DAYS)
|
||||
)
|
||||
|
||||
log.info(s"ServiceIdentifier = ${serviceIdentifier.toString}")
|
||||
log.info("clientid: " + clientId.toString)
|
||||
log.info("servicePort: " + servicePort().toString)
|
||||
log.info("adding shutdown hook")
|
||||
onExit {
|
||||
graphWriter.shutdown()
|
||||
thriftServer.close(shutdownTimeout().fromNow)
|
||||
}
|
||||
log.info("added shutdown hook")
|
||||
// Wait on the thriftServer so that shutdownTimeout is respected.
|
||||
Await.result(thriftServer)
|
||||
}
|
||||
}
|
17
src/scala/com/twitter/recos/user_user_graph/README.md
Normal file
17
src/scala/com/twitter/recos/user_user_graph/README.md
Normal file
@ -0,0 +1,17 @@
|
||||
# UserUserGraph (UUG)
|
||||
|
||||
## What is it
|
||||
User User Graph (UUG) is a Finalge thrift service built on the GraphJet framework. In maintains a graph of user-user relationships and serves user recommendations based on traversals of this graph.
|
||||
|
||||
## How is it used on Twitter
|
||||
UUG recommends users to follow based on who your follow graph have recently followed.
|
||||
The core idea behind UUG is collaborative filtering. UUG takes a user's weighted follow graph (i.e a list of weighted userIds) as input,
|
||||
performs efficient traversal & aggregation, and returns the top weighted users basd on # of users that engaged the users, as well as
|
||||
the engaging users' weights.
|
||||
|
||||
UUG is a stateful service and relies on a Kafka stream to ingest & persist states. It maintains an in-memory user engagements over the past
|
||||
week. Older events are dropped and GC'ed.
|
||||
|
||||
For full details on storage & processing, please check out our open-sourced project GraphJet, a general-purpose high performance in-memory storage engine.
|
||||
- https://github.com/twitter/GraphJet
|
||||
- http://www.vldb.org/pvldb/vol9/p1281-sharma.pdf
|
@ -0,0 +1,221 @@
|
||||
package com.twitter.recos.user_user_graph
|
||||
|
||||
import java.util.Random
|
||||
import com.google.common.collect.Lists
|
||||
import com.twitter.concurrent.AsyncQueue
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.graphjet.algorithms.counting.TopSecondDegreeByCountResponse
|
||||
import com.twitter.graphjet.algorithms.counting.user.TopSecondDegreeByCountForUser
|
||||
import com.twitter.graphjet.algorithms.counting.user.TopSecondDegreeByCountRequestForUser
|
||||
import com.twitter.graphjet.algorithms.counting.user.UserRecommendationInfo
|
||||
import com.twitter.graphjet.algorithms.ConnectingUsersWithMetadata
|
||||
import com.twitter.graphjet.algorithms.filters._
|
||||
import com.twitter.graphjet.bipartite.NodeMetadataLeftIndexedPowerLawMultiSegmentBipartiteGraph
|
||||
import com.twitter.logging.Logger
|
||||
import com.twitter.recos.decider.UserUserGraphDecider
|
||||
import com.twitter.recos.graph_common.FinagleStatsReceiverWrapper
|
||||
import com.twitter.recos.model.SalsaQueryRunner.SalsaRunnerConfig
|
||||
import com.twitter.recos.recos_common.thriftscala.UserSocialProofType
|
||||
import com.twitter.recos.user_user_graph.thriftscala._
|
||||
import com.twitter.recos.util.Stats._
|
||||
import com.twitter.servo.request.RequestHandler
|
||||
import com.twitter.util.Future
|
||||
import com.twitter.util.Try
|
||||
import it.unimi.dsi.fastutil.longs.Long2DoubleOpenHashMap
|
||||
import it.unimi.dsi.fastutil.longs.LongOpenHashSet
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
trait RecommendUsersHandler extends RequestHandler[RecommendUserRequest, RecommendUserResponse]
|
||||
|
||||
/**
|
||||
* Computes user recommendations based on a RecommendUserRequest by using
|
||||
* TopSecondDegree algorithm in GraphJet.
|
||||
*/
|
||||
case class RecommendUsersHandlerImpl(
|
||||
bipartiteGraph: NodeMetadataLeftIndexedPowerLawMultiSegmentBipartiteGraph,
|
||||
salsaRunnerConfig: SalsaRunnerConfig,
|
||||
decider: UserUserGraphDecider,
|
||||
statsReceiverWrapper: FinagleStatsReceiverWrapper)
|
||||
extends RecommendUsersHandler {
|
||||
|
||||
private val log: Logger = Logger(this.getClass.getSimpleName)
|
||||
private val stats = statsReceiverWrapper.statsReceiver.scope(this.getClass.getSimpleName)
|
||||
private val failureCounter = stats.counter("failure")
|
||||
private val recsStat = stats.stat("recs_count")
|
||||
private val emptyCounter = stats.counter("empty")
|
||||
private val pollCounter = stats.counter("poll")
|
||||
private val pollTimeoutCounter = stats.counter("pollTimeout")
|
||||
private val offerCounter = stats.counter("offer")
|
||||
private val pollLatencyStat = stats.stat("pollLatency")
|
||||
private val graphJetQueue = new AsyncQueue[TopSecondDegreeByCountForUser]
|
||||
(0 until salsaRunnerConfig.numSalsaRunners).foreach { _ =>
|
||||
graphJetQueue.offer(
|
||||
new TopSecondDegreeByCountForUser(
|
||||
bipartiteGraph,
|
||||
salsaRunnerConfig.expectedNodesToHitInSalsa,
|
||||
statsReceiverWrapper.scope(this.getClass.getSimpleName)
|
||||
)
|
||||
)
|
||||
}
|
||||
|
||||
/**
|
||||
* Given a user_user_graph request, make it conform to GraphJet's request format
|
||||
*/
|
||||
private def convertRequestToJava(
|
||||
request: RecommendUserRequest
|
||||
): TopSecondDegreeByCountRequestForUser = {
|
||||
val queryNode = request.requesterId
|
||||
val leftSeedNodesWithWeight = new Long2DoubleOpenHashMap(
|
||||
request.seedsWithWeights.keys.toArray,
|
||||
request.seedsWithWeights.values.toArray
|
||||
)
|
||||
val toBeFiltered = new LongOpenHashSet(request.excludedUserIds.getOrElse(Nil).toArray)
|
||||
val maxNumResults = request.maxNumResults.getOrElse(DefaultRequestParams.MaxNumResults)
|
||||
val maxNumSocialProofs =
|
||||
request.maxNumSocialProofs.getOrElse(DefaultRequestParams.MaxNumSocialProofs)
|
||||
val minUserPerSocialProof = convertMinUserPerSocialProofToJava(request.minUserPerSocialProof)
|
||||
val socialProofTypes =
|
||||
UserEdgeTypeMask.getUserUserGraphSocialProofTypes(request.socialProofTypes)
|
||||
val maxRightNodeAgeInMillis = DefaultRequestParams.MaxRightNodeAgeThreshold
|
||||
val maxEdgeEngagementAgeInMillis =
|
||||
request.maxEdgeEngagementAgeInMillis.getOrElse(DefaultRequestParams.MaxEdgeAgeThreshold)
|
||||
val resultFilterChain = new ResultFilterChain(
|
||||
Lists.newArrayList(
|
||||
new SocialProofTypesFilter(statsReceiverWrapper),
|
||||
new RequestedSetFilter(statsReceiverWrapper)
|
||||
)
|
||||
)
|
||||
|
||||
new TopSecondDegreeByCountRequestForUser(
|
||||
queryNode,
|
||||
leftSeedNodesWithWeight,
|
||||
toBeFiltered,
|
||||
maxNumResults,
|
||||
maxNumSocialProofs,
|
||||
UserEdgeTypeMask.SIZE.toInt,
|
||||
minUserPerSocialProof,
|
||||
socialProofTypes,
|
||||
maxRightNodeAgeInMillis,
|
||||
maxEdgeEngagementAgeInMillis,
|
||||
resultFilterChain
|
||||
)
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts the thrift scala type to the Java equivalent
|
||||
*/
|
||||
private def convertMinUserPerSocialProofToJava(
|
||||
socialProofInScala: Option[scala.collection.Map[UserSocialProofType, Int]]
|
||||
): java.util.Map[java.lang.Byte, java.lang.Integer] = {
|
||||
socialProofInScala
|
||||
.map {
|
||||
_.map {
|
||||
case (key: UserSocialProofType, value: Int) =>
|
||||
(new java.lang.Byte(key.getValue.toByte), new java.lang.Integer(value))
|
||||
}
|
||||
}
|
||||
.getOrElse(Map.empty[java.lang.Byte, java.lang.Integer])
|
||||
.asJava
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts a byte-array format of social proofs in Java to its Scala equivalent
|
||||
*/
|
||||
private def convertSocialProofsToScala(
|
||||
socialProofs: java.util.Map[java.lang.Byte, ConnectingUsersWithMetadata]
|
||||
): scala.collection.mutable.Map[UserSocialProofType, scala.Seq[Long]] = {
|
||||
socialProofs.asScala.map {
|
||||
case (socialProofByte, socialProof) =>
|
||||
val proofType = UserSocialProofType(socialProofByte.toByte)
|
||||
val ids = socialProof.getConnectingUsers.asScala.map(_.toLong)
|
||||
(proofType, ids)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts Java recommendation results to its Scala equivalent
|
||||
*/
|
||||
private def convertResponseToScala(
|
||||
responseOpt: Option[TopSecondDegreeByCountResponse]
|
||||
): RecommendUserResponse = {
|
||||
responseOpt match {
|
||||
case Some(rawResponse) =>
|
||||
val userSeq = rawResponse.getRankedRecommendations.asScala.toSeq.flatMap {
|
||||
case userRecs: UserRecommendationInfo =>
|
||||
Some(
|
||||
RecommendedUser(
|
||||
userRecs.getRecommendation,
|
||||
userRecs.getWeight,
|
||||
convertSocialProofsToScala(userRecs.getSocialProof)
|
||||
)
|
||||
)
|
||||
case _ =>
|
||||
None
|
||||
}
|
||||
recsStat.add(userSeq.size)
|
||||
if (userSeq.isEmpty) {
|
||||
emptyCounter.incr()
|
||||
}
|
||||
RecommendUserResponse(userSeq)
|
||||
case None =>
|
||||
emptyCounter.incr()
|
||||
RecommendUserResponse(Nil)
|
||||
}
|
||||
}
|
||||
|
||||
private def getGraphJetResponse(
|
||||
graphJet: TopSecondDegreeByCountForUser,
|
||||
request: TopSecondDegreeByCountRequestForUser,
|
||||
random: Random
|
||||
)(
|
||||
implicit statsReceiver: StatsReceiver
|
||||
): Option[TopSecondDegreeByCountResponse] = {
|
||||
trackBlockStats(stats) {
|
||||
// compute recs -- need to catch and print exceptions here otherwise they are swallowed
|
||||
val recAttempt = Try(graphJet.computeRecommendations(request, random)).onFailure { e =>
|
||||
failureCounter.incr()
|
||||
log.error(e, "GraphJet computation failed")
|
||||
}
|
||||
recAttempt.toOption
|
||||
}
|
||||
}
|
||||
|
||||
override def apply(request: RecommendUserRequest): Future[RecommendUserResponse] = {
|
||||
val random = new Random()
|
||||
val graphJetRequest = convertRequestToJava(request)
|
||||
pollCounter.incr()
|
||||
val t0 = System.currentTimeMillis
|
||||
graphJetQueue.poll().map { graphJetRunner =>
|
||||
val pollTime = System.currentTimeMillis - t0
|
||||
pollLatencyStat.add(pollTime)
|
||||
val response = Try {
|
||||
if (pollTime < salsaRunnerConfig.timeoutSalsaRunner) {
|
||||
convertResponseToScala(
|
||||
getGraphJetResponse(
|
||||
graphJetRunner,
|
||||
graphJetRequest,
|
||||
random
|
||||
)(statsReceiverWrapper.statsReceiver)
|
||||
)
|
||||
} else {
|
||||
// if we did not get a runner in time, then fail fast here and immediately put it back
|
||||
log.warning("GraphJet Queue polling timeout")
|
||||
pollTimeoutCounter.incr()
|
||||
throw new RuntimeException("GraphJet poll timeout")
|
||||
RecommendUserResponse(Nil)
|
||||
}
|
||||
} ensure {
|
||||
graphJetQueue.offer(graphJetRunner)
|
||||
offerCounter.incr()
|
||||
}
|
||||
response.toOption.getOrElse(RecommendUserResponse(Nil))
|
||||
}
|
||||
}
|
||||
|
||||
object DefaultRequestParams {
|
||||
val MaxNumResults = 100
|
||||
val MaxNumSocialProofs = 100
|
||||
val MaxRightNodeAgeThreshold: Long = Long.MaxValue
|
||||
val MaxEdgeAgeThreshold: Long = Long.MaxValue
|
||||
}
|
||||
}
|
@ -0,0 +1,37 @@
|
||||
package com.twitter.recos.user_user_graph
|
||||
|
||||
import com.twitter.recos.model.Constants
|
||||
import com.twitter.recos.graph_common.NodeMetadataLeftIndexedPowerLawMultiSegmentBipartiteGraphBuilder.GraphBuilderConfig
|
||||
|
||||
/**
|
||||
* The class holds all the config parameters for recos graph.
|
||||
*/
|
||||
object RecosConfig {
|
||||
val maxNumSegments: Int = 5
|
||||
val maxNumEdgesPerSegment: Int = 1 << 26 // 64M edges per segment
|
||||
val expectedNumLeftNodes: Int = 1 << 24 // should correspond to 16M nodes storage
|
||||
val expectedMaxLeftDegree: Int = 64
|
||||
val leftPowerLawExponent: Double = 16.0 // steep power law as most nodes will have a small degree
|
||||
val expectedNumRightNodes: Int = 1 << 24 // 16M nodes
|
||||
val numRightNodeMetadataTypes = 1 // UUG does not have node metadata
|
||||
|
||||
val graphBuilderConfig = GraphBuilderConfig(
|
||||
maxNumSegments = maxNumSegments,
|
||||
maxNumEdgesPerSegment = maxNumEdgesPerSegment,
|
||||
expectedNumLeftNodes = expectedNumLeftNodes,
|
||||
expectedMaxLeftDegree = expectedMaxLeftDegree,
|
||||
leftPowerLawExponent = leftPowerLawExponent,
|
||||
expectedNumRightNodes = expectedNumRightNodes,
|
||||
numRightNodeMetadataTypes = numRightNodeMetadataTypes,
|
||||
edgeTypeMask = new UserEdgeTypeMask()
|
||||
)
|
||||
|
||||
println("RecosConfig - maxNumSegments " + maxNumSegments)
|
||||
println("RecosConfig - maxNumEdgesPerSegment " + maxNumEdgesPerSegment)
|
||||
println("RecosConfig - expectedNumLeftNodes " + expectedNumLeftNodes)
|
||||
println("RecosConfig - expectedMaxLeftDegree " + expectedMaxLeftDegree)
|
||||
println("RecosConfig - leftPowerLawExponent " + leftPowerLawExponent)
|
||||
println("RecosConfig - expectedNumRightNodes " + expectedNumRightNodes)
|
||||
println("RecosConfig - numRightNodeMetadataTypes " + numRightNodeMetadataTypes)
|
||||
println("RecosConfig - salsaRunnerConfig " + Constants.salsaRunnerConfig)
|
||||
}
|
@ -0,0 +1,91 @@
|
||||
package com.twitter.recos.user_user_graph
|
||||
|
||||
import com.twitter.graphjet.bipartite.api.EdgeTypeMask
|
||||
import com.twitter.recos.recos_common.thriftscala.UserSocialProofType
|
||||
|
||||
/**
|
||||
* The bit mask is used to encode edge types in the top bits of an integer,
|
||||
* e.g. Follow, Mention, and Mediatag. Under current segment configuration, each segment
|
||||
* stores up to 128M edges. Assuming that each node on one side is unique, each segment
|
||||
* stores up to 128M unique nodes on one side, which occupies the lower 27 bits of an integer.
|
||||
* This leaves five bits to encode the edge types, which at max can store 32 edge types.
|
||||
* The following implementation utilizes the top four bits and leaves one free bit out.
|
||||
*/
|
||||
class UserEdgeTypeMask extends EdgeTypeMask {
|
||||
import UserEdgeTypeMask._
|
||||
override def encode(node: Int, edgeType: Byte): Int = {
|
||||
require(
|
||||
edgeType == FOLLOW || edgeType == MENTION || edgeType == MEDIATAG,
|
||||
s"encode: Illegal edge type argument $edgeType")
|
||||
node | EDGEARRAY(edgeType)
|
||||
}
|
||||
|
||||
override def edgeType(node: Int): Byte = {
|
||||
(node >> 28).toByte
|
||||
}
|
||||
|
||||
override def restore(node: Int): Int = {
|
||||
node & MASK
|
||||
}
|
||||
}
|
||||
|
||||
object UserEdgeTypeMask {
|
||||
|
||||
/**
|
||||
* Reserve the top four bits of each integer to encode the edge type information.
|
||||
*/
|
||||
val MASK: Int =
|
||||
Integer.parseInt("00001111111111111111111111111111", 2)
|
||||
val FOLLOW: Byte = 0
|
||||
val MENTION: Byte = 1
|
||||
val MEDIATAG: Byte = 2
|
||||
val SIZE: Byte = 3
|
||||
val UNUSED3: Byte = 3
|
||||
val UNUSED4: Byte = 4
|
||||
val UNUSED5: Byte = 5
|
||||
val UNUSED6: Byte = 6
|
||||
val UNUSED7: Byte = 7
|
||||
val UNUSED8: Byte = 8
|
||||
val UNUSED9: Byte = 9
|
||||
val UNUSED10: Byte = 10
|
||||
val UNUSED11: Byte = 11
|
||||
val UNUSED12: Byte = 12
|
||||
val UNUSED13: Byte = 13
|
||||
val UNUSED14: Byte = 14
|
||||
val UNUSED15: Byte = 15
|
||||
val EDGEARRAY: Array[Int] = Array(
|
||||
0,
|
||||
1 << 28,
|
||||
2 << 28,
|
||||
3 << 28,
|
||||
4 << 28,
|
||||
5 << 28,
|
||||
6 << 28,
|
||||
7 << 28,
|
||||
8 << 28,
|
||||
9 << 28,
|
||||
10 << 28,
|
||||
11 << 28,
|
||||
12 << 28,
|
||||
13 << 28,
|
||||
14 << 28,
|
||||
15 << 28
|
||||
)
|
||||
|
||||
/**
|
||||
* Map valid social proof types specified by clients to an array of bytes. If clients do not
|
||||
* specify any social proof types in thrift, it will return all available social types by
|
||||
* default.
|
||||
*
|
||||
* @param socialProofTypes are the valid socialProofTypes specified by clients
|
||||
* @return an array of bytes representing valid social proof types
|
||||
*/
|
||||
def getUserUserGraphSocialProofTypes(
|
||||
socialProofTypes: Option[Seq[UserSocialProofType]]
|
||||
): Array[Byte] = {
|
||||
socialProofTypes
|
||||
.map { _.map { _.getValue }.toArray }
|
||||
.getOrElse((0 until SIZE).toArray)
|
||||
.map { _.toByte }
|
||||
}
|
||||
}
|
@ -0,0 +1,18 @@
|
||||
package com.twitter.recos.user_user_graph
|
||||
|
||||
import com.twitter.finagle.thrift.ClientId
|
||||
import com.twitter.finagle.tracing.{Trace, TraceId}
|
||||
import com.twitter.recos.user_user_graph.thriftscala._
|
||||
import com.twitter.util.Future
|
||||
|
||||
object UserUserGraph {
|
||||
def traceId: TraceId = Trace.id
|
||||
def clientId: Option[ClientId] = ClientId.current
|
||||
}
|
||||
|
||||
class UserUserGraph(recommendUsersHandler: RecommendUsersHandler)
|
||||
extends thriftscala.UserUserGraph.MethodPerEndpoint {
|
||||
|
||||
override def recommendUsers(request: RecommendUserRequest): Future[RecommendUserResponse] =
|
||||
recommendUsersHandler(request)
|
||||
}
|
@ -0,0 +1,83 @@
|
||||
package com.twitter.recos.user_user_graph
|
||||
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.finatra.kafka.consumers.FinagleKafkaConsumerBuilder
|
||||
import com.twitter.graphjet.bipartite.NodeMetadataLeftIndexedPowerLawMultiSegmentBipartiteGraph
|
||||
import com.twitter.graphjet.bipartite.segment.NodeMetadataLeftIndexedBipartiteGraphSegment
|
||||
import com.twitter.recos.hose.common.UnifiedGraphWriter
|
||||
import com.twitter.recos.internal.thriftscala.RecosHoseMessage
|
||||
import com.twitter.recos.util.Action
|
||||
|
||||
case class UserUserGraphWriter(
|
||||
shardId: String,
|
||||
env: String,
|
||||
hosename: String,
|
||||
bufferSize: Int,
|
||||
kafkaConsumerBuilder: FinagleKafkaConsumerBuilder[String, RecosHoseMessage],
|
||||
clientId: String,
|
||||
statsReceiver: StatsReceiver)
|
||||
extends UnifiedGraphWriter[
|
||||
NodeMetadataLeftIndexedBipartiteGraphSegment,
|
||||
NodeMetadataLeftIndexedPowerLawMultiSegmentBipartiteGraph
|
||||
] {
|
||||
|
||||
// The max throughput for each kafka consumer is around 25MB/s
|
||||
// Use 3 processors for 75MB/s catch-up speed.
|
||||
val consumerNum: Int = 3
|
||||
// Leave 2 Segments for live writer
|
||||
val catchupWriterNum: Int = RecosConfig.maxNumSegments - 2
|
||||
|
||||
import UserUserGraphWriter._
|
||||
|
||||
private def getEdgeType(action: Byte): Byte = {
|
||||
if (action == Action.Follow.id) {
|
||||
UserEdgeTypeMask.FOLLOW
|
||||
} else if (action == Action.Mention.id) {
|
||||
UserEdgeTypeMask.MENTION
|
||||
} else if (action == Action.MediaTag.id) {
|
||||
UserEdgeTypeMask.MEDIATAG
|
||||
} else {
|
||||
throw new IllegalArgumentException("getEdgeType: Illegal edge type argument " + action)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a RecosHoseMessage to the graph. used by live writer to insert edges to the
|
||||
* current segment
|
||||
*/
|
||||
override def addEdgeToGraph(
|
||||
graph: NodeMetadataLeftIndexedPowerLawMultiSegmentBipartiteGraph,
|
||||
recosHoseMessage: RecosHoseMessage
|
||||
): Unit = {
|
||||
graph.addEdge(
|
||||
recosHoseMessage.leftId,
|
||||
recosHoseMessage.rightId,
|
||||
getEdgeType(recosHoseMessage.action),
|
||||
recosHoseMessage.edgeMetadata.getOrElse(0L),
|
||||
EMTPY_NODE_METADATA,
|
||||
EMTPY_NODE_METADATA
|
||||
)
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a RecosHoseMessage to the given segment in the graph. Used by catch up writers to
|
||||
* insert edges to non-current (old) segments
|
||||
*/
|
||||
override def addEdgeToSegment(
|
||||
segment: NodeMetadataLeftIndexedBipartiteGraphSegment,
|
||||
recosHoseMessage: RecosHoseMessage
|
||||
): Unit = {
|
||||
segment.addEdge(
|
||||
recosHoseMessage.leftId,
|
||||
recosHoseMessage.rightId,
|
||||
getEdgeType(recosHoseMessage.action),
|
||||
recosHoseMessage.edgeMetadata.getOrElse(0L),
|
||||
EMTPY_NODE_METADATA,
|
||||
EMTPY_NODE_METADATA
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
private object UserUserGraphWriter {
|
||||
final val EMTPY_NODE_METADATA = new Array[Array[Int]](1)
|
||||
}
|
69
src/scala/com/twitter/recos/user_video_graph/BUILD
Normal file
69
src/scala/com/twitter/recos/user_video_graph/BUILD
Normal file
@ -0,0 +1,69 @@
|
||||
scala_library(
|
||||
name = "user-video-graph",
|
||||
sources = ["*.scala"],
|
||||
tags = [
|
||||
"bazel-compatible",
|
||||
"bazel-only",
|
||||
],
|
||||
dependencies = [
|
||||
"3rdparty/jvm/cascading:cascading-local",
|
||||
"3rdparty/jvm/com/backtype:dfs-datastores",
|
||||
"3rdparty/jvm/com/fasterxml/jackson/module:jackson-module-scala",
|
||||
"3rdparty/jvm/com/google/inject:guice",
|
||||
"3rdparty/jvm/com/netflix/curator:curator-framework",
|
||||
"3rdparty/jvm/com/twitter/graphjet",
|
||||
"3rdparty/jvm/io/netty:netty4-tcnative-boringssl-static",
|
||||
"3rdparty/jvm/it/unimi/dsi:fastutil",
|
||||
"3rdparty/jvm/org/apache/hadoop:hadoop-client-default",
|
||||
"3rdparty/jvm/org/apache/kafka:rosette-kafka",
|
||||
"3rdparty/jvm/org/apache/thrift:libthrift",
|
||||
"abdecider/src/main/scala",
|
||||
"decider/src/main/scala",
|
||||
"finagle-internal/mtls/src/main/scala/com/twitter/finagle/mtls/authentication",
|
||||
"finagle-internal/mtls/src/main/scala/com/twitter/finagle/mtls/server",
|
||||
"finagle/finagle-core/src/main",
|
||||
"finagle/finagle-http/src/main/scala",
|
||||
"finagle/finagle-memcached/src/main/scala",
|
||||
"finagle/finagle-stats/src/main/scala",
|
||||
"finagle/finagle-thriftmux/src/main/scala",
|
||||
"frigate/frigate-common/src/main/scala/com/twitter/frigate/common/util",
|
||||
"scrooge/scrooge-core/src/main/scala",
|
||||
"servo/repo/src/main/scala",
|
||||
"servo/request/src/main/scala",
|
||||
"servo/util/src/main/scala",
|
||||
"src/resources/com/twitter/recos:decider",
|
||||
"src/scala/com/twitter/recos/decider",
|
||||
"src/scala/com/twitter/recos/graph_common",
|
||||
"src/scala/com/twitter/recos/hose/common",
|
||||
"src/scala/com/twitter/recos/model:recos-model",
|
||||
"src/scala/com/twitter/recos/serviceapi",
|
||||
"src/scala/com/twitter/recos/user_video_graph/relatedTweetHandlers",
|
||||
"src/scala/com/twitter/recos/user_video_graph/store",
|
||||
"src/scala/com/twitter/recos/util:recos-util",
|
||||
"src/scala/com/twitter/simclusters_v2/common",
|
||||
"src/thrift/com/twitter/recos:recos-common-scala",
|
||||
"src/thrift/com/twitter/recos:recos-internal-scala",
|
||||
"src/thrift/com/twitter/recos/user_video_graph:user_video_graph-scala",
|
||||
"thrift-web-forms/src/main/scala/com/twitter/thriftwebforms",
|
||||
"thrift-web-forms/src/main/scala/com/twitter/thriftwebforms/model",
|
||||
"twitter-server-internal/src/main/scala",
|
||||
"twitter-server/server/src/main/scala",
|
||||
"twitter-server/slf4j-jdk14/src/main/scala/com/twitter/server/logging",
|
||||
"util/util-app/src/main/scala",
|
||||
"util/util-hashing/src/main/scala",
|
||||
"util/util-stats/src/main/scala",
|
||||
],
|
||||
)
|
||||
|
||||
jvm_binary(
|
||||
name = "bin",
|
||||
basename = "user-video-graph-server",
|
||||
main = "com.twitter.recos.user_video_graph.Main",
|
||||
runtime_platform = "java11",
|
||||
tags = ["known-to-fail-jira:SD-20771"],
|
||||
dependencies = [
|
||||
":user-video-graph",
|
||||
"3rdparty/jvm/org/slf4j:slf4j-jdk14",
|
||||
"twitter-server/slf4j-jdk14/src/main/scala",
|
||||
],
|
||||
)
|
@ -0,0 +1,12 @@
|
||||
package com.twitter.recos.user_video_graph
|
||||
|
||||
import com.twitter.finagle.tracing.Trace
|
||||
import com.twitter.logging.Logger
|
||||
import com.twitter.recos.recos_common.thriftscala._
|
||||
import com.twitter.recos.user_video_graph.thriftscala._
|
||||
import com.twitter.util.Future
|
||||
|
||||
trait LoggingUserVideoGraph extends thriftscala.UserVideoGraph.MethodPerEndpoint {
|
||||
private[this] val accessLog = Logger("access")
|
||||
|
||||
}
|
294
src/scala/com/twitter/recos/user_video_graph/Main.scala
Normal file
294
src/scala/com/twitter/recos/user_video_graph/Main.scala
Normal file
@ -0,0 +1,294 @@
|
||||
package com.twitter.recos.user_video_graph
|
||||
|
||||
import com.twitter.abdecider.ABDeciderFactory
|
||||
import com.twitter.abdecider.LoggingABDecider
|
||||
import com.twitter.app.Flag
|
||||
import com.twitter.conversions.DurationOps._
|
||||
import com.twitter.finagle.ThriftMux
|
||||
import com.twitter.finagle.http.HttpMuxer
|
||||
import com.twitter.finagle.mtls.authentication.ServiceIdentifier
|
||||
import com.twitter.finagle.mtls.client.MtlsStackClient.MtlsThriftMuxClientSyntax
|
||||
import com.twitter.finagle.mtls.server.MtlsStackServer._
|
||||
import com.twitter.finagle.mux.ClientDiscardedRequestException
|
||||
import com.twitter.finagle.mux.transport.OpportunisticTls
|
||||
import com.twitter.finagle.service.ReqRep
|
||||
import com.twitter.finagle.service.ResponseClass
|
||||
import com.twitter.finagle.thrift.ClientId
|
||||
import com.twitter.finatra.kafka.consumers.FinagleKafkaConsumerBuilder
|
||||
import com.twitter.finatra.kafka.domain.KafkaGroupId
|
||||
import com.twitter.finatra.kafka.domain.SeekStrategy
|
||||
import com.twitter.finatra.kafka.serde.ScalaSerdes
|
||||
import com.twitter.frigate.common.util.ElfOwlFilter
|
||||
import com.twitter.frigate.common.util.ElfOwlFilter.ByLdapGroup
|
||||
import com.twitter.graphjet.bipartite.MultiSegmentPowerLawBipartiteGraph
|
||||
import com.twitter.logging._
|
||||
import com.twitter.recos.decider.EndpointLoadShedder
|
||||
import com.twitter.recos.decider.UserTweetGraphDecider
|
||||
import com.twitter.recos.graph_common.FinagleStatsReceiverWrapper
|
||||
import com.twitter.recos.graph_common.MultiSegmentPowerLawBipartiteGraphBuilder
|
||||
import com.twitter.recos.internal.thriftscala.RecosHoseMessage
|
||||
import com.twitter.recos.user_video_graph.RecosConfig._
|
||||
import com.twitter.recos.user_tweet_graph.relatedTweetHandlers.ConsumersBasedRelatedTweetsHandler
|
||||
import com.twitter.recos.user_video_graph.relatedTweetHandlers.TweetBasedRelatedTweetsHandler
|
||||
import com.twitter.recos.user_video_graph.relatedTweetHandlers.ProducerBasedRelatedTweetsHandler
|
||||
import com.twitter.recos.user_video_graph.store.UserRecentFollowersStore
|
||||
import com.twitter.server.Deciderable
|
||||
import com.twitter.server.TwitterServer
|
||||
import com.twitter.server.logging.{Logging => JDK14Logging}
|
||||
import com.twitter.servo.request._
|
||||
import com.twitter.servo.util.ExceptionCounter
|
||||
import com.twitter.simclusters_v2.common.UserId
|
||||
import com.twitter.socialgraph.thriftscala.SocialGraphService
|
||||
import com.twitter.storehaus.ReadableStore
|
||||
import com.twitter.util.Await
|
||||
import com.twitter.util.Duration
|
||||
import com.twitter.util.JavaTimer
|
||||
import com.twitter.util.Throw
|
||||
import com.twitter.util.Timer
|
||||
import java.net.InetSocketAddress
|
||||
import java.util.concurrent.TimeUnit
|
||||
import org.apache.kafka.clients.CommonClientConfigs
|
||||
import org.apache.kafka.common.config.SaslConfigs
|
||||
import org.apache.kafka.common.config.SslConfigs
|
||||
import org.apache.kafka.common.security.auth.SecurityProtocol
|
||||
import org.apache.kafka.common.serialization.StringDeserializer
|
||||
import scala.reflect.ClassTag
|
||||
|
||||
object Main extends TwitterServer with JDK14Logging with Deciderable {
|
||||
profile =>
|
||||
|
||||
val shardId: Flag[Int] = flag("shardId", 0, "Shard ID")
|
||||
val servicePort: Flag[InetSocketAddress] =
|
||||
flag("service.port", new InetSocketAddress(10143), "Thrift service port")
|
||||
val logDir: Flag[String] = flag("logdir", "recos", "Logging directory")
|
||||
val numShards: Flag[Int] = flag("numShards", 1, "Number of shards for this service")
|
||||
val truststoreLocation: Flag[String] =
|
||||
flag[String]("truststore_location", "", "Truststore file location")
|
||||
val hoseName: Flag[String] =
|
||||
flag("hosename", "recos_injector_user_user", "the kafka stream used for incoming edges")
|
||||
|
||||
val dataCenter: Flag[String] = flag("service.cluster", "atla", "Data Center")
|
||||
val serviceRole: Flag[String] = flag("service.role", "Service Role")
|
||||
val serviceEnv: Flag[String] = flag("service.env", "Service Env")
|
||||
val serviceName: Flag[String] = flag("service.name", "Service Name")
|
||||
|
||||
private val maxNumSegments =
|
||||
flag("maxNumSegments", graphBuilderConfig.maxNumSegments, "the number of segments in the graph")
|
||||
|
||||
private val statsReceiverWrapper = FinagleStatsReceiverWrapper(statsReceiver)
|
||||
|
||||
/**
|
||||
* A ClientRequestAuthorizer to be used in a request-authorization RequestFilter.
|
||||
*/
|
||||
lazy val clientAuthorizer: ClientRequestAuthorizer =
|
||||
ClientRequestAuthorizer.observed(
|
||||
ClientRequestAuthorizer.permissive,
|
||||
new ClientRequestObserver(statsReceiver)
|
||||
)
|
||||
|
||||
lazy val clientId = ClientId(s"usertweetgraph.${serviceEnv()}")
|
||||
|
||||
private def makeThriftClient[ThriftServiceType: ClassTag](
|
||||
dest: String,
|
||||
label: String,
|
||||
serviceIdentifier: ServiceIdentifier,
|
||||
requestTimeout: Duration = 100.milliseconds
|
||||
): ThriftServiceType = {
|
||||
ThriftMux.client
|
||||
.withClientId(ClientId("usertweetgraph.prod"))
|
||||
.withOpportunisticTls(OpportunisticTls.Required)
|
||||
.withMutualTls(serviceIdentifier)
|
||||
.withRequestTimeout(requestTimeout)
|
||||
.withStatsReceiver(statsReceiver.scope("clnt"))
|
||||
.withResponseClassifier {
|
||||
case ReqRep(_, Throw(_: ClientDiscardedRequestException)) => ResponseClass.Ignorable
|
||||
}.build[ThriftServiceType](dest, label)
|
||||
}
|
||||
|
||||
private val shutdownTimeout = flag(
|
||||
"service.shutdownTimeout",
|
||||
5.seconds,
|
||||
"Maximum amount of time to wait for pending requests to complete on shutdown"
|
||||
)
|
||||
|
||||
/**
|
||||
* ExceptionCounter for tracking failures from RequestHandler(s).
|
||||
*/
|
||||
lazy val exceptionCounter = new ExceptionCounter(statsReceiver)
|
||||
|
||||
/**
|
||||
* Function for translating exceptions returned by a RequestHandler. Useful
|
||||
* for cases where underlying exception types should be wrapped in those
|
||||
* defined in the project's Thrift IDL.
|
||||
*/
|
||||
lazy val translateExceptions: PartialFunction[Throwable, Throwable] = {
|
||||
case t => t
|
||||
}
|
||||
|
||||
val DefaultLdapAccessGroup: Seq[String] = Seq("eng", "cassowary-group", "timeline-team")
|
||||
|
||||
// ********* logging **********
|
||||
|
||||
lazy val loggingLevel: Level = Level.INFO
|
||||
lazy val recosLogPath: String = logDir() + "/recos.log"
|
||||
lazy val graphLogPath: String = logDir() + "/graph.log"
|
||||
lazy val accessLogPath: String = logDir() + "/access.log"
|
||||
|
||||
override def loggerFactories: List[LoggerFactory] =
|
||||
List(
|
||||
LoggerFactory(
|
||||
level = Some(loggingLevel),
|
||||
handlers = QueueingHandler(
|
||||
handler = FileHandler(
|
||||
filename = recosLogPath,
|
||||
level = Some(loggingLevel),
|
||||
rollPolicy = Policy.Hourly,
|
||||
rotateCount = 6,
|
||||
formatter = new Formatter
|
||||
)
|
||||
) :: Nil
|
||||
),
|
||||
LoggerFactory(
|
||||
node = "graph",
|
||||
useParents = false,
|
||||
level = Some(loggingLevel),
|
||||
handlers = QueueingHandler(
|
||||
handler = FileHandler(
|
||||
filename = graphLogPath,
|
||||
level = Some(loggingLevel),
|
||||
rollPolicy = Policy.Hourly,
|
||||
rotateCount = 6,
|
||||
formatter = new Formatter
|
||||
)
|
||||
) :: Nil
|
||||
),
|
||||
LoggerFactory(
|
||||
node = "access",
|
||||
useParents = false,
|
||||
level = Some(loggingLevel),
|
||||
handlers = QueueingHandler(
|
||||
handler = FileHandler(
|
||||
filename = accessLogPath,
|
||||
level = Some(loggingLevel),
|
||||
rollPolicy = Policy.Hourly,
|
||||
rotateCount = 6,
|
||||
formatter = new Formatter
|
||||
)
|
||||
) :: Nil
|
||||
),
|
||||
LoggerFactory(
|
||||
node = "client_event",
|
||||
level = Some(loggingLevel),
|
||||
useParents = false,
|
||||
handlers = QueueingHandler(
|
||||
maxQueueSize = 10000,
|
||||
handler = ScribeHandler(
|
||||
category = "client_event",
|
||||
formatter = BareFormatter
|
||||
)
|
||||
) :: Nil
|
||||
)
|
||||
)
|
||||
// ******** Decider *************
|
||||
|
||||
// ********* ABdecider **********
|
||||
|
||||
val abDeciderYmlPath: String = "/usr/local/config/abdecider/abdecider.yml"
|
||||
|
||||
val scribeLogger: Option[Logger] = Some(Logger.get("client_event"))
|
||||
|
||||
val abDecider: LoggingABDecider =
|
||||
ABDeciderFactory(
|
||||
abDeciderYmlPath = abDeciderYmlPath,
|
||||
scribeLogger = scribeLogger,
|
||||
environment = Some("production")
|
||||
).buildWithLogging()
|
||||
|
||||
// ********* Recos service **********
|
||||
|
||||
def main(): Unit = {
|
||||
log.info("building graph with maxNumSegments = " + profile.maxNumSegments())
|
||||
|
||||
implicit val timer: Timer = new JavaTimer(true)
|
||||
|
||||
val graph = MultiSegmentPowerLawBipartiteGraphBuilder(
|
||||
graphBuilderConfig.copy(maxNumSegments = profile.maxNumSegments()),
|
||||
statsReceiverWrapper
|
||||
)
|
||||
|
||||
val kafkaConfigBuilder = FinagleKafkaConsumerBuilder[String, RecosHoseMessage]()
|
||||
.dest("/s/kafka/recommendations:kafka-tls")
|
||||
.groupId(KafkaGroupId(f"user_video_graph-${shardId()}%06d"))
|
||||
.keyDeserializer(new StringDeserializer)
|
||||
.valueDeserializer(ScalaSerdes.Thrift[RecosHoseMessage].deserializer)
|
||||
.seekStrategy(SeekStrategy.REWIND)
|
||||
.rewindDuration(48.hours)
|
||||
.withConfig(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, SecurityProtocol.SASL_SSL.toString)
|
||||
.withConfig(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, truststoreLocation())
|
||||
.withConfig(SaslConfigs.SASL_MECHANISM, SaslConfigs.GSSAPI_MECHANISM)
|
||||
.withConfig(SaslConfigs.SASL_KERBEROS_SERVICE_NAME, "kafka")
|
||||
.withConfig(SaslConfigs.SASL_KERBEROS_SERVER_NAME, "kafka")
|
||||
|
||||
val graphWriter =
|
||||
UserVideoGraphWriter(
|
||||
shardId().toString,
|
||||
serviceEnv(),
|
||||
hoseName(),
|
||||
128, // keep the original setting.
|
||||
kafkaConfigBuilder,
|
||||
clientId.name,
|
||||
statsReceiver,
|
||||
)
|
||||
graphWriter.initHose(graph)
|
||||
|
||||
// For MutualTLS
|
||||
val serviceIdentifier = ServiceIdentifier(
|
||||
role = serviceRole(),
|
||||
service = serviceName(),
|
||||
environment = serviceEnv(),
|
||||
zone = dataCenter()
|
||||
)
|
||||
log.info(s"ServiceIdentifier = ${serviceIdentifier.toString}")
|
||||
|
||||
val socialGraphClient: SocialGraphService.MethodPerEndpoint =
|
||||
makeThriftClient[SocialGraphService.MethodPerEndpoint](
|
||||
"/s/socialgraph/socialgraph",
|
||||
"socialgraph",
|
||||
serviceIdentifier)
|
||||
val userRecentFollowersStore: ReadableStore[UserRecentFollowersStore.Query, Seq[UserId]] =
|
||||
new UserRecentFollowersStore(socialGraphClient)
|
||||
|
||||
val tweetBasedRelatedTweetsHandler = new TweetBasedRelatedTweetsHandler(graph, statsReceiver)
|
||||
val consumersBasedRelatedTweetsHandler =
|
||||
new ConsumersBasedRelatedTweetsHandler(graph, statsReceiver)
|
||||
val producerBasedRelatedTweetsHandler =
|
||||
new ProducerBasedRelatedTweetsHandler(graph, userRecentFollowersStore, statsReceiver)
|
||||
|
||||
val decider = UserTweetGraphDecider(serviceEnv(), dataCenter())
|
||||
val endpointLoadShedder = new EndpointLoadShedder(decider)
|
||||
val userVideoGraph =
|
||||
new UserVideoGraph(
|
||||
tweetBasedRelatedTweetsHandler,
|
||||
producerBasedRelatedTweetsHandler,
|
||||
consumersBasedRelatedTweetsHandler,
|
||||
endpointLoadShedder)(timer) with LoggingUserVideoGraph
|
||||
|
||||
val thriftServer = ThriftMux.server
|
||||
.withOpportunisticTls(OpportunisticTls.Required)
|
||||
.withMutualTls(serviceIdentifier)
|
||||
.serveIface(servicePort(), userVideoGraph)
|
||||
|
||||
log.info("clientid: " + clientId.toString)
|
||||
log.info("servicePort: " + servicePort().toString)
|
||||
|
||||
log.info("adding shutdown hook")
|
||||
onExit {
|
||||
graphWriter.shutdown()
|
||||
thriftServer.close(shutdownTimeout().fromNow)
|
||||
}
|
||||
log.info("added shutdown hook")
|
||||
|
||||
// Wait on the thriftServer so that shutdownTimeout is respected.
|
||||
Await.result(thriftServer)
|
||||
}
|
||||
}
|
14
src/scala/com/twitter/recos/user_video_graph/README.md
Normal file
14
src/scala/com/twitter/recos/user_video_graph/README.md
Normal file
@ -0,0 +1,14 @@
|
||||
# UserVideoGraph (UVG)
|
||||
|
||||
## What is it
|
||||
User Video Graph (UVG) is a Finalge thrift service built on the GraphJet framework. In maintains a graph of user-video engagements and serves user recommendations based on traversals in this graph.
|
||||
|
||||
## How is it used on Twitter
|
||||
UVG generates video recommendations from a given seed tweet set. It recommends tweets based on collaborative filtering & random walks.
|
||||
|
||||
UVG is a stateful service and relies on a Kafka stream to ingest & persist states. The Kafka stream is processed and generated by Recos-Injector.
|
||||
It maintains an in-memory user engagements over the past 24-48 hours. Older events are dropped and GC'ed.
|
||||
|
||||
For full details on storage & processing, please check out our open-sourced project GraphJet, a general-purpose high performance in-memory storage engine.
|
||||
- https://github.com/twitter/GraphJet
|
||||
- http://www.vldb.org/pvldb/vol9/p1281-sharma.pdf
|
@ -0,0 +1,62 @@
|
||||
package com.twitter.recos.user_video_graph
|
||||
|
||||
import com.twitter.graphjet.bipartite.api.EdgeTypeMask
|
||||
import com.twitter.recos.util.Action
|
||||
|
||||
/**
|
||||
* The bit mask is used to encode edge types in the top bits of an integer,
|
||||
* e.g. favorite, retweet, reply and click. Under current segment configuration, each segment
|
||||
* stores up to 128M edges. Assuming that each node on one side is unique, each segment
|
||||
* stores up to 128M unique nodes on one side, which occupies the lower 27 bits of an integer.
|
||||
* This leaves five bits to encode the edge types, which at max can store 32 edge types.
|
||||
* The following implementation utilizes the top four bits and leaves one free bit out.
|
||||
*/
|
||||
class UserVideoEdgeTypeMask extends EdgeTypeMask {
|
||||
import UserVideoEdgeTypeMask._
|
||||
|
||||
override def encode(node: Int, edgeType: Byte): Int = {
|
||||
if (edgeType < 0 || edgeType > SIZE) {
|
||||
throw new IllegalArgumentException("encode: Illegal edge type argument " + edgeType)
|
||||
} else {
|
||||
node | (edgeType << 28)
|
||||
}
|
||||
}
|
||||
|
||||
override def edgeType(node: Int): Byte = {
|
||||
(node >>> 28).toByte
|
||||
}
|
||||
|
||||
override def restore(node: Int): Int = {
|
||||
node & MASK
|
||||
}
|
||||
}
|
||||
|
||||
object UserVideoEdgeTypeMask extends Enumeration {
|
||||
|
||||
type UserTweetEdgeTypeMask = Value
|
||||
|
||||
/**
|
||||
* Byte values corresponding to the action taken on a tweet, which will be encoded in the
|
||||
* top 4 bits in a tweet Id
|
||||
* NOTE: THERE CAN ONLY BE UP TO 16 TYPES
|
||||
*/
|
||||
val VideoPlayback50: UserTweetEdgeTypeMask = Value(1)
|
||||
|
||||
/**
|
||||
* Reserve the top four bits of each integer to encode the edge type information.
|
||||
*/
|
||||
val MASK: Int = Integer.parseInt("00001111111111111111111111111111", 2)
|
||||
val SIZE: Int = this.values.size
|
||||
|
||||
/**
|
||||
* Converts the action byte in the RecosHoseMessage into GraphJet internal byte mapping
|
||||
*/
|
||||
def actionTypeToEdgeType(actionByte: Byte): Byte = {
|
||||
val edgeType = Action(actionByte) match {
|
||||
case Action.VideoPlayback50 => VideoPlayback50.id
|
||||
case _ =>
|
||||
throw new IllegalArgumentException("getEdgeType: Illegal edge type argument " + actionByte)
|
||||
}
|
||||
edgeType.toByte
|
||||
}
|
||||
}
|
@ -0,0 +1,73 @@
|
||||
package com.twitter.recos.user_video_graph
|
||||
|
||||
import com.twitter.finagle.thrift.ClientId
|
||||
import com.twitter.finagle.tracing.Trace
|
||||
import com.twitter.finagle.tracing.TraceId
|
||||
import com.twitter.recos.decider.EndpointLoadShedder
|
||||
import com.twitter.recos.user_video_graph.thriftscala._
|
||||
import com.twitter.util.Duration
|
||||
import com.twitter.util.Future
|
||||
import com.twitter.util.Timer
|
||||
import scala.concurrent.duration.MILLISECONDS
|
||||
import com.twitter.logging.Logger
|
||||
import com.twitter.recos.user_tweet_graph.relatedTweetHandlers.ConsumersBasedRelatedTweetsHandler
|
||||
import com.twitter.recos.user_video_graph.relatedTweetHandlers.ProducerBasedRelatedTweetsHandler
|
||||
import com.twitter.recos.user_video_graph.relatedTweetHandlers.TweetBasedRelatedTweetsHandler
|
||||
|
||||
object UserVideoGraph {
|
||||
def traceId: TraceId = Trace.id
|
||||
def clientId: Option[ClientId] = ClientId.current
|
||||
}
|
||||
|
||||
class UserVideoGraph(
|
||||
tweetBasedRelatedTweetsHandler: TweetBasedRelatedTweetsHandler,
|
||||
producerBasedRelatedTweetsHandler: ProducerBasedRelatedTweetsHandler,
|
||||
consumersBasedRelatedTweetsHandler: ConsumersBasedRelatedTweetsHandler,
|
||||
endpointLoadShedder: EndpointLoadShedder
|
||||
)(
|
||||
implicit timer: Timer)
|
||||
extends thriftscala.UserVideoGraph.MethodPerEndpoint {
|
||||
|
||||
private val defaultTimeout: Duration = Duration(50, MILLISECONDS)
|
||||
private val EmptyResponse = Future.value(RelatedTweetResponse())
|
||||
private val log = Logger()
|
||||
|
||||
override def tweetBasedRelatedTweets(
|
||||
request: TweetBasedRelatedTweetRequest
|
||||
): Future[RelatedTweetResponse] =
|
||||
endpointLoadShedder("videoGraphTweetBasedRelatedTweets") {
|
||||
tweetBasedRelatedTweetsHandler(request).raiseWithin(defaultTimeout)
|
||||
}.rescue {
|
||||
case EndpointLoadShedder.LoadSheddingException =>
|
||||
EmptyResponse
|
||||
case e =>
|
||||
log.info("user-video-graph_tweetBasedRelatedTweets" + e)
|
||||
EmptyResponse
|
||||
}
|
||||
|
||||
override def producerBasedRelatedTweets(
|
||||
request: ProducerBasedRelatedTweetRequest
|
||||
): Future[RelatedTweetResponse] =
|
||||
endpointLoadShedder("producerBasedRelatedTweets") {
|
||||
producerBasedRelatedTweetsHandler(request).raiseWithin(defaultTimeout)
|
||||
}.rescue {
|
||||
case EndpointLoadShedder.LoadSheddingException =>
|
||||
EmptyResponse
|
||||
case e =>
|
||||
log.info("user-video-graph_producerBasedRelatedTweets" + e)
|
||||
EmptyResponse
|
||||
}
|
||||
|
||||
override def consumersBasedRelatedTweets(
|
||||
request: ConsumersBasedRelatedTweetRequest
|
||||
): Future[RelatedTweetResponse] =
|
||||
endpointLoadShedder("consumersBasedRelatedTweets") {
|
||||
consumersBasedRelatedTweetsHandler(request).raiseWithin(defaultTimeout)
|
||||
}.rescue {
|
||||
case EndpointLoadShedder.LoadSheddingException =>
|
||||
EmptyResponse
|
||||
case e =>
|
||||
log.info("user-video-graph_consumersBasedRelatedTweets" + e)
|
||||
EmptyResponse
|
||||
}
|
||||
}
|
@ -0,0 +1,39 @@
|
||||
package com.twitter.recos.user_video_graph
|
||||
|
||||
import com.twitter.recos.graph_common.MultiSegmentPowerLawBipartiteGraphBuilder.GraphBuilderConfig
|
||||
|
||||
/**
|
||||
* The class holds all the config parameters for recos graph.
|
||||
*/
|
||||
object RecosConfig {
|
||||
val maxNumSegments: Int = 8
|
||||
val maxNumEdgesPerSegment: Int =
|
||||
(1 << 28) // 268M edges per segment, should be able to include 2 days' data
|
||||
val expectedNumLeftNodes: Int =
|
||||
(1 << 26) // should correspond to 67M nodes storage
|
||||
val expectedMaxLeftDegree: Int = 64
|
||||
val leftPowerLawExponent: Double = 16.0 // steep power law as most nodes will have a small degree
|
||||
val expectedNumRightNodes: Int = (1 << 26) // 67M nodes
|
||||
val expectedMaxRightDegree: Int = scala.math.pow(1024, 2).toInt // some nodes will be very popular
|
||||
val rightPowerLawExponent: Double = 4.0 // this will be less steep
|
||||
|
||||
val graphBuilderConfig = GraphBuilderConfig(
|
||||
maxNumSegments = maxNumSegments,
|
||||
maxNumEdgesPerSegment = maxNumEdgesPerSegment,
|
||||
expectedNumLeftNodes = expectedNumLeftNodes,
|
||||
expectedMaxLeftDegree = expectedMaxLeftDegree,
|
||||
leftPowerLawExponent = leftPowerLawExponent,
|
||||
expectedNumRightNodes = expectedNumRightNodes,
|
||||
expectedMaxRightDegree = expectedMaxRightDegree,
|
||||
rightPowerLawExponent = rightPowerLawExponent
|
||||
)
|
||||
|
||||
println("RecosConfig - maxNumSegments " + maxNumSegments)
|
||||
println("RecosConfig - maxNumEdgesPerSegment " + maxNumEdgesPerSegment)
|
||||
println("RecosConfig - expectedNumLeftNodes " + expectedNumLeftNodes)
|
||||
println("RecosConfig - expectedMaxLeftDegree " + expectedMaxLeftDegree)
|
||||
println("RecosConfig - leftPowerLawExponent " + leftPowerLawExponent)
|
||||
println("RecosConfig - expectedNumRightNodes " + expectedNumRightNodes)
|
||||
println("RecosConfig - expectedMaxRightDegree " + expectedMaxRightDegree)
|
||||
println("RecosConfig - rightPowerLawExponent " + rightPowerLawExponent)
|
||||
}
|
@ -0,0 +1,101 @@
|
||||
package com.twitter.recos.user_video_graph
|
||||
|
||||
import com.twitter.finagle.Service
|
||||
import com.twitter.finagle.http.Request
|
||||
import com.twitter.finagle.http.Response
|
||||
import com.twitter.finagle.http.Status
|
||||
import com.twitter.finagle.http.Version
|
||||
import com.twitter.frigate.common.util.HTMLUtil
|
||||
import com.twitter.graphjet.algorithms.TweetIDMask
|
||||
import com.twitter.graphjet.bipartite.segment.BipartiteGraphSegment
|
||||
import com.twitter.graphjet.bipartite.MultiSegmentIterator
|
||||
import com.twitter.graphjet.bipartite.MultiSegmentPowerLawBipartiteGraph
|
||||
import com.twitter.logging.Logger
|
||||
import com.twitter.util.Future
|
||||
import java.util.Random
|
||||
import scala.collection.mutable.ListBuffer
|
||||
|
||||
class UserTweetGraphEdgeHttpHandler(graph: MultiSegmentPowerLawBipartiteGraph)
|
||||
extends Service[Request, Response] {
|
||||
private val log = Logger("UserTweetGraphEdgeHttpHandler")
|
||||
private val tweetIDMask = new TweetIDMask()
|
||||
|
||||
def getCardInfo(rightNode: Long): String = {
|
||||
val bits: Long = rightNode & TweetIDMask.METAMASK
|
||||
bits match {
|
||||
case TweetIDMask.PHOTO => "Photo"
|
||||
case TweetIDMask.PLAYER => "Video"
|
||||
case TweetIDMask.SUMMARY => "Url"
|
||||
case TweetIDMask.PROMOTION => "Promotion"
|
||||
case _ => "Regular"
|
||||
}
|
||||
}
|
||||
|
||||
private def getUserEdges(userId: Long): ListBuffer[Edge] = {
|
||||
val random = new Random()
|
||||
val iterator =
|
||||
graph
|
||||
.getRandomLeftNodeEdges(userId, 10, random).asInstanceOf[MultiSegmentIterator[
|
||||
BipartiteGraphSegment
|
||||
]]
|
||||
val tweets = new ListBuffer[Edge]()
|
||||
if (iterator != null) {
|
||||
while (iterator.hasNext) {
|
||||
val rightNode = iterator.nextLong()
|
||||
val edgeType = iterator.currentEdgeType()
|
||||
tweets += Edge(
|
||||
tweetIDMask.restore(rightNode),
|
||||
UserVideoEdgeTypeMask(edgeType).toString,
|
||||
getCardInfo(rightNode),
|
||||
)
|
||||
}
|
||||
}
|
||||
tweets
|
||||
}
|
||||
|
||||
def apply(httpRequest: Request): Future[Response] = {
|
||||
log.info("UserTweetGraphEdgeHttpHandler params: " + httpRequest.getParams())
|
||||
val time0 = System.currentTimeMillis
|
||||
|
||||
val tweetId = httpRequest.getLongParam("tweetId")
|
||||
val queryTweetDegree = graph.getRightNodeDegree(tweetId)
|
||||
val tweetEdges = getTweetEdges(tweetId)
|
||||
|
||||
val userId = httpRequest.getLongParam("userId")
|
||||
val queryUserDegree = graph.getLeftNodeDegree(userId)
|
||||
|
||||
val response = Response(Version.Http11, Status.Ok)
|
||||
val userEdges = getUserEdges(userId)
|
||||
val elapsed = System.currentTimeMillis - time0
|
||||
val comment = ("Please specify \"userId\" or \"tweetId\" param." +
|
||||
"\n query tweet degree = " + queryTweetDegree +
|
||||
"\n query user degree = " + queryUserDegree +
|
||||
"\n done in %d ms<br>").format(elapsed)
|
||||
val tweetContent = userEdges.toList
|
||||
.map { edge =>
|
||||
s"<b>TweetId</b>: ${edge.tweetId},\n<b>Action type</b>: ${edge.actionType},\n<b>Card type</b>: ${edge.cardType}"
|
||||
.replaceAll("\n", " ")
|
||||
}.mkString("\n<br>\n")
|
||||
|
||||
response.setContentString(
|
||||
HTMLUtil.html.replace("XXXXX", comment + tweetContent + "\n<hr/>\n" + tweetEdges.toString()))
|
||||
Future.value(response)
|
||||
}
|
||||
|
||||
private def getTweetEdges(tweetId: Long): ListBuffer[Long] = {
|
||||
val random = new Random()
|
||||
val iterator =
|
||||
graph
|
||||
.getRandomRightNodeEdges(tweetId, 500, random).asInstanceOf[MultiSegmentIterator[
|
||||
BipartiteGraphSegment
|
||||
]]
|
||||
val terms = new ListBuffer[Long]()
|
||||
if (iterator != null) {
|
||||
while (iterator.hasNext) { terms += iterator.nextLong() }
|
||||
}
|
||||
terms.distinct
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
case class Edge(tweetId: Long, actionType: String, cardType: String)
|
@ -0,0 +1,82 @@
|
||||
package com.twitter.recos.user_video_graph
|
||||
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.finatra.kafka.consumers.FinagleKafkaConsumerBuilder
|
||||
import com.twitter.graphjet.algorithms.TweetIDMask
|
||||
import com.twitter.graphjet.bipartite.MultiSegmentPowerLawBipartiteGraph
|
||||
import com.twitter.graphjet.bipartite.segment.BipartiteGraphSegment
|
||||
import com.twitter.recos.hose.common.UnifiedGraphWriter
|
||||
import com.twitter.recos.internal.thriftscala.RecosHoseMessage
|
||||
import com.twitter.recos.serviceapi.Tweetypie._
|
||||
|
||||
/**
|
||||
* The class submits a number of $numBootstrapWriters graph writer threads, BufferedEdgeWriter,
|
||||
* during service startup. One of them is live writer thread, and the other $(numBootstrapWriters - 1)
|
||||
* are catchup writer threads. All of them consume kafka events from an internal concurrent queue,
|
||||
* which is populated by kafka reader threads. At bootstrap time, the kafka reader threads look
|
||||
* back kafka offset from several hours ago and populate the internal concurrent queue.
|
||||
* Each graph writer thread writes to an individual graph segment separately.
|
||||
* The $(numBootstrapWriters - 1) catchup writer threads will stop once all events
|
||||
* between current system time at startup and the time in memcache are processed.
|
||||
* The live writer thread will continue to write all incoming kafka events.
|
||||
* It lives through the entire life cycle of recos graph service.
|
||||
*/
|
||||
case class UserVideoGraphWriter(
|
||||
shardId: String,
|
||||
env: String,
|
||||
hosename: String,
|
||||
bufferSize: Int,
|
||||
kafkaConsumerBuilder: FinagleKafkaConsumerBuilder[String, RecosHoseMessage],
|
||||
clientId: String,
|
||||
statsReceiver: StatsReceiver)
|
||||
extends UnifiedGraphWriter[BipartiteGraphSegment, MultiSegmentPowerLawBipartiteGraph] {
|
||||
writer =>
|
||||
// The max throughput for each kafka consumer is around 25MB/s
|
||||
// Use 4 processors for 100MB/s catch-up speed.
|
||||
val consumerNum: Int = 4
|
||||
// Leave 1 Segments to LiveWriter
|
||||
val catchupWriterNum: Int = RecosConfig.maxNumSegments - 1
|
||||
|
||||
/**
|
||||
* Adds a RecosHoseMessage to the graph. used by live writer to insert edges to the
|
||||
* current segment
|
||||
*/
|
||||
override def addEdgeToGraph(
|
||||
graph: MultiSegmentPowerLawBipartiteGraph,
|
||||
recosHoseMessage: RecosHoseMessage
|
||||
): Unit = {
|
||||
graph.addEdge(
|
||||
recosHoseMessage.leftId,
|
||||
getMetaEdge(recosHoseMessage.rightId, recosHoseMessage.card),
|
||||
UserVideoEdgeTypeMask.actionTypeToEdgeType(recosHoseMessage.action),
|
||||
)
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a RecosHoseMessage to the given segment in the graph. Used by catch up writers to
|
||||
* insert edges to non-current (old) segments
|
||||
*/
|
||||
override def addEdgeToSegment(
|
||||
segment: BipartiteGraphSegment,
|
||||
recosHoseMessage: RecosHoseMessage
|
||||
): Unit = {
|
||||
segment.addEdge(
|
||||
recosHoseMessage.leftId,
|
||||
getMetaEdge(recosHoseMessage.rightId, recosHoseMessage.card),
|
||||
UserVideoEdgeTypeMask.actionTypeToEdgeType(recosHoseMessage.action)
|
||||
)
|
||||
}
|
||||
|
||||
private def getMetaEdge(rightId: Long, cardOption: Option[Byte]): Long = {
|
||||
cardOption
|
||||
.map { card =>
|
||||
if (isPhotoCard(card)) TweetIDMask.photo(rightId)
|
||||
else if (isPlayerCard(card)) TweetIDMask.player(rightId)
|
||||
else if (isSummaryCard(card)) TweetIDMask.summary(rightId)
|
||||
else if (isPromotionCard(card)) TweetIDMask.promotion(rightId)
|
||||
else rightId
|
||||
}
|
||||
.getOrElse(rightId)
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,12 @@
|
||||
scala_library(
|
||||
sources = ["*.scala"],
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"3rdparty/jvm/com/twitter/graphjet",
|
||||
"servo/request/src/main/scala",
|
||||
"src/scala/com/twitter/recos/user_video_graph/store",
|
||||
"src/scala/com/twitter/recos/user_video_graph/util",
|
||||
"src/scala/com/twitter/recos/util:recos-util",
|
||||
"src/thrift/com/twitter/recos/user_video_graph:user_video_graph-scala",
|
||||
],
|
||||
)
|
@ -0,0 +1,66 @@
|
||||
package com.twitter.recos.user_tweet_graph.relatedTweetHandlers
|
||||
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.graphjet.bipartite.api.BipartiteGraph
|
||||
import com.twitter.recos.user_video_graph.thriftscala._
|
||||
import com.twitter.recos.user_video_graph.util.FetchRHSTweetsUtil
|
||||
import com.twitter.recos.user_video_graph.util.FilterUtil
|
||||
import com.twitter.recos.user_video_graph.util.GetRelatedTweetCandidatesUtil
|
||||
import com.twitter.recos.util.Stats._
|
||||
import com.twitter.servo.request._
|
||||
import com.twitter.util.Duration
|
||||
import com.twitter.util.Future
|
||||
import scala.concurrent.duration.HOURS
|
||||
|
||||
/**
|
||||
* Implementation of the Thrift-defined service interface for consumersTweetBasedRelatedTweets.
|
||||
* given a list of consumer userIds, find the tweets they co-engaged with (we're treating input userIds as consumers therefore "consumersTweetBasedRelatedTweets" )
|
||||
* example use case: given a list of user's contacts in their address book, find tweets those contacts engaged with
|
||||
*/
|
||||
class ConsumersBasedRelatedTweetsHandler(
|
||||
bipartiteGraph: BipartiteGraph,
|
||||
statsReceiver: StatsReceiver)
|
||||
extends RequestHandler[ConsumersBasedRelatedTweetRequest, RelatedTweetResponse] {
|
||||
private val stats = statsReceiver.scope(this.getClass.getSimpleName)
|
||||
|
||||
override def apply(request: ConsumersBasedRelatedTweetRequest): Future[RelatedTweetResponse] = {
|
||||
trackFutureBlockStats(stats) {
|
||||
|
||||
val maxResults = request.maxResults.getOrElse(200)
|
||||
val minScore = request.minScore.getOrElse(0.0)
|
||||
val maxTweetAge = request.maxTweetAgeInHours.getOrElse(48)
|
||||
val minResultDegree = request.minResultDegree.getOrElse(50)
|
||||
val minCooccurrence = request.minCooccurrence.getOrElse(3)
|
||||
val excludeTweetIds = request.excludeTweetIds.getOrElse(Seq.empty).toSet
|
||||
|
||||
val consumerSeedSet = request.consumerSeedSet.distinct.filter { userId =>
|
||||
val userDegree = bipartiteGraph.getLeftNodeDegree(userId)
|
||||
// constrain to users that have <100 engagements to avoid spammy behavior
|
||||
userDegree < 100
|
||||
}
|
||||
|
||||
val rhsTweetIds = FetchRHSTweetsUtil.fetchRHSTweets(
|
||||
consumerSeedSet,
|
||||
bipartiteGraph
|
||||
)
|
||||
|
||||
val scorePreFactor = 1000.0 / consumerSeedSet.size
|
||||
val relatedTweetCandidates = GetRelatedTweetCandidatesUtil.getRelatedTweetCandidates(
|
||||
rhsTweetIds,
|
||||
minCooccurrence,
|
||||
minResultDegree,
|
||||
scorePreFactor,
|
||||
bipartiteGraph)
|
||||
|
||||
val relatedTweets = relatedTweetCandidates
|
||||
.filter(relatedTweet =>
|
||||
FilterUtil.tweetAgeFilter(
|
||||
relatedTweet.tweetId,
|
||||
Duration(maxTweetAge, HOURS)) && (relatedTweet.score > minScore) && (!excludeTweetIds
|
||||
.contains(relatedTweet.tweetId))).take(maxResults)
|
||||
|
||||
stats.stat("response_size").add(relatedTweets.size)
|
||||
Future.value(RelatedTweetResponse(tweets = relatedTweets))
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,86 @@
|
||||
package com.twitter.recos.user_video_graph.relatedTweetHandlers
|
||||
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.graphjet.bipartite.api.BipartiteGraph
|
||||
import com.twitter.recos.user_video_graph.thriftscala._
|
||||
import com.twitter.recos.util.Stats._
|
||||
import com.twitter.servo.request._
|
||||
import com.twitter.util.Duration
|
||||
import com.twitter.util.Future
|
||||
import scala.concurrent.duration.HOURS
|
||||
import com.twitter.simclusters_v2.common.UserId
|
||||
import com.twitter.storehaus.ReadableStore
|
||||
import com.twitter.recos.user_video_graph.store.UserRecentFollowersStore
|
||||
import com.twitter.recos.user_video_graph.util.FetchRHSTweetsUtil
|
||||
import com.twitter.recos.user_video_graph.util.FilterUtil
|
||||
import com.twitter.recos.user_video_graph.util.GetRelatedTweetCandidatesUtil
|
||||
|
||||
/**
|
||||
* Implementation of the Thrift-defined service interface for producerBasedRelatedTweets.
|
||||
*
|
||||
*/
|
||||
class ProducerBasedRelatedTweetsHandler(
|
||||
bipartiteGraph: BipartiteGraph,
|
||||
userRecentFollowersStore: ReadableStore[UserRecentFollowersStore.Query, Seq[UserId]],
|
||||
statsReceiver: StatsReceiver)
|
||||
extends RequestHandler[ProducerBasedRelatedTweetRequest, RelatedTweetResponse] {
|
||||
private val stats = statsReceiver.scope(this.getClass.getSimpleName)
|
||||
|
||||
override def apply(request: ProducerBasedRelatedTweetRequest): Future[RelatedTweetResponse] = {
|
||||
trackFutureBlockStats(stats) {
|
||||
val maxResults = request.maxResults.getOrElse(200)
|
||||
val maxNumFollowers = request.maxNumFollowers.getOrElse(500)
|
||||
val minScore = request.minScore.getOrElse(0.0)
|
||||
val maxTweetAge = request.maxTweetAgeInHours.getOrElse(48)
|
||||
val minResultDegree = request.minResultDegree.getOrElse(50)
|
||||
val minCooccurrence = request.minCooccurrence.getOrElse(4)
|
||||
val excludeTweetIds = request.excludeTweetIds.getOrElse(Seq.empty).toSet
|
||||
|
||||
val followersFut = fetchFollowers(request.producerId, Some(maxNumFollowers))
|
||||
followersFut.map { followers =>
|
||||
val rhsTweetIds = FetchRHSTweetsUtil.fetchRHSTweets(
|
||||
followers,
|
||||
bipartiteGraph
|
||||
)
|
||||
|
||||
val scorePreFactor = 1000.0 / followers.size
|
||||
val relatedTweetCandidates = GetRelatedTweetCandidatesUtil.getRelatedTweetCandidates(
|
||||
rhsTweetIds,
|
||||
minCooccurrence,
|
||||
minResultDegree,
|
||||
scorePreFactor,
|
||||
bipartiteGraph)
|
||||
|
||||
val relatedTweets = relatedTweetCandidates
|
||||
.filter { relatedTweet =>
|
||||
FilterUtil.tweetAgeFilter(
|
||||
relatedTweet.tweetId,
|
||||
Duration(maxTweetAge, HOURS)) && (relatedTweet.score > minScore) && (!excludeTweetIds
|
||||
.contains(relatedTweet.tweetId))
|
||||
}.take(maxResults)
|
||||
stats.stat("response_size").add(relatedTweets.size)
|
||||
RelatedTweetResponse(tweets = relatedTweets)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private def fetchFollowers(
|
||||
producerId: Long,
|
||||
maxNumFollower: Option[Int],
|
||||
): Future[Seq[Long]] = {
|
||||
val query =
|
||||
UserRecentFollowersStore.Query(producerId, maxNumFollower, None)
|
||||
|
||||
val followersFut = userRecentFollowersStore.get(query)
|
||||
followersFut.map { followersOpt =>
|
||||
val followers = followersOpt.getOrElse(Seq.empty)
|
||||
val followerIds = followers.distinct.filter { userId =>
|
||||
val userDegree = bipartiteGraph.getLeftNodeDegree(userId)
|
||||
// constrain to more active users that have >1 engagement to optimize latency, and <100 engagements to avoid spammy behavior
|
||||
userDegree > 1 && userDegree < 500
|
||||
}
|
||||
stats.stat("follower_size_after_filter").add(followerIds.size)
|
||||
followerIds
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,91 @@
|
||||
package com.twitter.recos.user_video_graph.relatedTweetHandlers
|
||||
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.graphjet.bipartite.api.BipartiteGraph
|
||||
import com.twitter.recos.features.tweet.thriftscala.GraphFeaturesForQuery
|
||||
import com.twitter.recos.user_video_graph.thriftscala._
|
||||
import com.twitter.recos.user_video_graph.util.FilterUtil
|
||||
import com.twitter.recos.user_video_graph.util.FetchRHSTweetsUtil
|
||||
import com.twitter.recos.user_video_graph.util.GetRelatedTweetCandidatesUtil
|
||||
import com.twitter.recos.user_video_graph.util.GetAllInternalTweetIdsUtil
|
||||
import com.twitter.recos.user_video_graph.util.SampleLHSUsersUtil
|
||||
import com.twitter.recos.util.Stats._
|
||||
import com.twitter.servo.request._
|
||||
import com.twitter.util.Duration
|
||||
import com.twitter.util.Future
|
||||
import scala.concurrent.duration.HOURS
|
||||
|
||||
/**
|
||||
* Implementation of the Thrift-defined service interface for tweetBasedRelatedTweets.
|
||||
*
|
||||
*/
|
||||
class TweetBasedRelatedTweetsHandler(bipartiteGraph: BipartiteGraph, statsReceiver: StatsReceiver)
|
||||
extends RequestHandler[TweetBasedRelatedTweetRequest, RelatedTweetResponse] {
|
||||
private val stats = statsReceiver.scope(this.getClass.getSimpleName)
|
||||
|
||||
override def apply(request: TweetBasedRelatedTweetRequest): Future[RelatedTweetResponse] = {
|
||||
trackFutureBlockStats(stats) {
|
||||
val internalQueryTweetIds =
|
||||
GetAllInternalTweetIdsUtil.getAllInternalTweetIds(request.tweetId, bipartiteGraph)
|
||||
|
||||
val response = internalQueryTweetIds match {
|
||||
case head +: Nil => getRelatedTweets(request, head)
|
||||
case _ => RelatedTweetResponse()
|
||||
}
|
||||
Future.value(response)
|
||||
}
|
||||
}
|
||||
|
||||
private def getRelatedTweets(
|
||||
request: TweetBasedRelatedTweetRequest,
|
||||
maskedTweetId: Long
|
||||
): RelatedTweetResponse = {
|
||||
|
||||
val maxNumSamplesPerNeighbor = request.maxNumSamplesPerNeighbor.getOrElse(100)
|
||||
val maxResults = request.maxResults.getOrElse(200)
|
||||
val minScore = request.minScore.getOrElse(0.5)
|
||||
val maxTweetAge = request.maxTweetAgeInHours.getOrElse(48)
|
||||
val minResultDegree = request.minResultDegree.getOrElse(50)
|
||||
val minQueryDegree = request.minQueryDegree.getOrElse(10)
|
||||
val minCooccurrence = request.minCooccurrence.getOrElse(3)
|
||||
val excludeTweetIds = request.excludeTweetIds.getOrElse(Seq.empty).toSet
|
||||
|
||||
val queryTweetDegree = bipartiteGraph.getRightNodeDegree(maskedTweetId)
|
||||
stats.stat("queryTweetDegree").add(queryTweetDegree)
|
||||
|
||||
if (queryTweetDegree < minQueryDegree) {
|
||||
stats.counter("queryTweetDegreeLessThanMinQueryDegree").incr()
|
||||
RelatedTweetResponse()
|
||||
} else {
|
||||
|
||||
val sampledLHSuserIds =
|
||||
SampleLHSUsersUtil.sampleLHSUsers(maskedTweetId, maxNumSamplesPerNeighbor, bipartiteGraph)
|
||||
|
||||
val rHStweetIds = FetchRHSTweetsUtil.fetchRHSTweets(
|
||||
sampledLHSuserIds,
|
||||
bipartiteGraph,
|
||||
)
|
||||
|
||||
val scorePreFactor =
|
||||
queryTweetDegree / math.log(queryTweetDegree) / sampledLHSuserIds.distinct.size
|
||||
val relatedTweetCandidates = GetRelatedTweetCandidatesUtil.getRelatedTweetCandidates(
|
||||
rHStweetIds,
|
||||
minCooccurrence,
|
||||
minResultDegree,
|
||||
scorePreFactor,
|
||||
bipartiteGraph)
|
||||
|
||||
val relatedTweets = relatedTweetCandidates
|
||||
.filter(relatedTweet =>
|
||||
FilterUtil.tweetAgeFilter(
|
||||
relatedTweet.tweetId,
|
||||
Duration(maxTweetAge, HOURS)) && (relatedTweet.score > minScore) && (!excludeTweetIds
|
||||
.contains(relatedTweet.tweetId))).take(maxResults)
|
||||
|
||||
stats.stat("response_size").add(relatedTweets.size)
|
||||
RelatedTweetResponse(
|
||||
tweets = relatedTweets,
|
||||
queryTweetGraphFeatures = Some(GraphFeaturesForQuery(degree = Some(queryTweetDegree))))
|
||||
}
|
||||
}
|
||||
}
|
9
src/scala/com/twitter/recos/user_video_graph/store/BUILD
Normal file
9
src/scala/com/twitter/recos/user_video_graph/store/BUILD
Normal file
@ -0,0 +1,9 @@
|
||||
scala_library(
|
||||
sources = ["*.scala"],
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"3rdparty/jvm/com/twitter/storehaus:core",
|
||||
"src/scala/com/twitter/simclusters_v2/common",
|
||||
"src/thrift/com/twitter/socialgraph:thrift-scala",
|
||||
],
|
||||
)
|
@ -0,0 +1,50 @@
|
||||
package com.twitter.recos.user_video_graph.store
|
||||
|
||||
import com.twitter.simclusters_v2.common.UserId
|
||||
import com.twitter.socialgraph.thriftscala.EdgesRequest
|
||||
import com.twitter.socialgraph.thriftscala.EdgesResult
|
||||
import com.twitter.socialgraph.thriftscala.PageRequest
|
||||
import com.twitter.socialgraph.thriftscala.RelationshipType
|
||||
import com.twitter.socialgraph.thriftscala.SrcRelationship
|
||||
import com.twitter.socialgraph.thriftscala.SocialGraphService
|
||||
import com.twitter.storehaus.ReadableStore
|
||||
import com.twitter.util.Duration
|
||||
import com.twitter.util.Future
|
||||
import com.twitter.util.Time
|
||||
|
||||
class UserRecentFollowersStore(
|
||||
sgsClient: SocialGraphService.MethodPerEndpoint)
|
||||
extends ReadableStore[UserRecentFollowersStore.Query, Seq[UserId]] {
|
||||
|
||||
override def get(key: UserRecentFollowersStore.Query): Future[Option[Seq[UserId]]] = {
|
||||
val edgeRequest = EdgesRequest(
|
||||
relationship = SrcRelationship(key.userId, RelationshipType.FollowedBy),
|
||||
// Could have a better guess at count when k.maxAge != None
|
||||
pageRequest = Some(PageRequest(count = key.maxResults))
|
||||
)
|
||||
|
||||
val lookbackThresholdMillis = key.maxAge
|
||||
.map(maxAge => (Time.now - maxAge).inMilliseconds)
|
||||
.getOrElse(0L)
|
||||
|
||||
sgsClient
|
||||
.edges(Seq(edgeRequest))
|
||||
.map(_.flatMap {
|
||||
case EdgesResult(edges, _, _) =>
|
||||
edges.collect {
|
||||
case e if e.createdAt >= lookbackThresholdMillis =>
|
||||
e.target
|
||||
}
|
||||
})
|
||||
.map(Some(_))
|
||||
}
|
||||
}
|
||||
|
||||
object UserRecentFollowersStore {
|
||||
case class Query(
|
||||
userId: UserId,
|
||||
// maxResults - if Some(count), we return only the `count` most recent follows
|
||||
maxResults: Option[Int] = None,
|
||||
// maxAge - if Some(duration), return only follows since `Time.now - duration`
|
||||
maxAge: Option[Duration] = None)
|
||||
}
|
12
src/scala/com/twitter/recos/user_video_graph/util/BUILD
Normal file
12
src/scala/com/twitter/recos/user_video_graph/util/BUILD
Normal file
@ -0,0 +1,12 @@
|
||||
scala_library(
|
||||
sources = ["*.scala"],
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"3rdparty/jvm/com/twitter/graphjet",
|
||||
"snowflake:id",
|
||||
"snowflake/src/main/scala/com/twitter/snowflake/id",
|
||||
"src/scala/com/twitter/recos/util:recos-util",
|
||||
"src/scala/com/twitter/simclusters_v2/common",
|
||||
"src/thrift/com/twitter/recos/user_video_graph:user_video_graph-scala",
|
||||
],
|
||||
)
|
@ -0,0 +1,29 @@
|
||||
package com.twitter.recos.user_video_graph.util
|
||||
|
||||
import com.twitter.graphjet.bipartite.MultiSegmentIterator
|
||||
import com.twitter.graphjet.bipartite.api.BipartiteGraph
|
||||
import com.twitter.graphjet.bipartite.segment.BipartiteGraphSegment
|
||||
import scala.collection.mutable.ListBuffer
|
||||
|
||||
object FetchRHSTweetsUtil {
|
||||
// get RHS tweets given LHS users
|
||||
def fetchRHSTweets(
|
||||
userIds: Seq[Long],
|
||||
bipartiteGraph: BipartiteGraph
|
||||
): Seq[Long] = {
|
||||
userIds.distinct
|
||||
.flatMap { userId =>
|
||||
val tweetIdsIterator = bipartiteGraph
|
||||
.getLeftNodeEdges(userId).asInstanceOf[MultiSegmentIterator[BipartiteGraphSegment]]
|
||||
|
||||
val tweetIds = new ListBuffer[Long]()
|
||||
if (tweetIdsIterator != null) {
|
||||
while (tweetIdsIterator.hasNext) {
|
||||
val rightNode = tweetIdsIterator.nextLong()
|
||||
tweetIds += rightNode
|
||||
}
|
||||
}
|
||||
tweetIds.distinct
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,15 @@
|
||||
package com.twitter.recos.user_video_graph.util
|
||||
|
||||
import com.twitter.simclusters_v2.common.TweetId
|
||||
import com.twitter.snowflake.id.SnowflakeId
|
||||
import com.twitter.util.Duration
|
||||
import com.twitter.util.Time
|
||||
|
||||
object FilterUtil {
|
||||
def tweetAgeFilter(tweetId: TweetId, maxAge: Duration): Boolean = {
|
||||
SnowflakeId
|
||||
.timeFromIdOpt(tweetId)
|
||||
.map { tweetTime => tweetTime > Time.now - maxAge }.getOrElse(false)
|
||||
// If there's no snowflake timestamp, we have no idea when this tweet happened.
|
||||
}
|
||||
}
|
@ -0,0 +1,33 @@
|
||||
package com.twitter.recos.user_video_graph.util
|
||||
|
||||
import com.twitter.graphjet.algorithms.TweetIDMask
|
||||
import com.twitter.graphjet.bipartite.api.BipartiteGraph
|
||||
|
||||
object GetAllInternalTweetIdsUtil {
|
||||
|
||||
def getAllInternalTweetIds(tweetId: Long, bipartiteGraph: BipartiteGraph): Seq[Long] = {
|
||||
val internalTweetIds = getAllMasks(tweetId)
|
||||
sortByDegrees(internalTweetIds, bipartiteGraph)
|
||||
}
|
||||
|
||||
private def getAllMasks(tweetId: Long): Seq[Long] = {
|
||||
Seq(
|
||||
tweetId,
|
||||
TweetIDMask.summary(tweetId),
|
||||
TweetIDMask.photo(tweetId),
|
||||
TweetIDMask.player(tweetId),
|
||||
TweetIDMask.promotion(tweetId)
|
||||
)
|
||||
}
|
||||
|
||||
private def sortByDegrees(
|
||||
encodedTweetIds: Seq[Long],
|
||||
bipartiteGraph: BipartiteGraph
|
||||
): Seq[Long] = {
|
||||
encodedTweetIds
|
||||
.map { encodedTweetId => (encodedTweetId, bipartiteGraph.getRightNodeDegree(encodedTweetId)) }
|
||||
.filter { case (_, degree) => degree > 0 } // keep only tweetds with positive degree
|
||||
.sortBy { case (_, degree) => -degree } // sort by degree in descending order
|
||||
.map { case (encodedTweetId, _) => encodedTweetId }
|
||||
}
|
||||
}
|
@ -0,0 +1,56 @@
|
||||
package com.twitter.recos.user_video_graph.util
|
||||
|
||||
import com.twitter.graphjet.bipartite.api.BipartiteGraph
|
||||
import com.twitter.recos.user_video_graph.thriftscala._
|
||||
import com.twitter.recos.features.tweet.thriftscala.GraphFeaturesForTweet
|
||||
import com.twitter.graphjet.algorithms.TweetIDMask
|
||||
|
||||
object GetRelatedTweetCandidatesUtil {
|
||||
private val tweetIDMask = new TweetIDMask
|
||||
|
||||
/**
|
||||
* calculate scores for each RHS tweet that we get back
|
||||
* for tweetBasedRelatedTweet, scorePreFactor = queryTweetDegree / log(queryTweetDegree) / LHSuserSize
|
||||
* and the final score will be a log-cosine score
|
||||
* for non-tweetBasedRelatedTweet, We don't have a query tweet, to keep scoring function consistent,
|
||||
* scorePreFactor = 1000.0 / LHSuserSize (queryTweetDegree's average is ~10k, 1000 ~= 10k/log(10k))
|
||||
* Though scorePreFactor is applied for all results within a request, it's still useful to make score comparable across requests,
|
||||
* so we can have a unifed min_score and help with downstream score normalization
|
||||
* **/
|
||||
def getRelatedTweetCandidates(
|
||||
relatedTweetCandidates: Seq[Long],
|
||||
minCooccurrence: Int,
|
||||
minResultDegree: Int,
|
||||
scorePreFactor: Double,
|
||||
bipartiteGraph: BipartiteGraph
|
||||
): Seq[RelatedTweet] = {
|
||||
relatedTweetCandidates
|
||||
.groupBy(tweetId => tweetId)
|
||||
.filterKeys(tweetId => bipartiteGraph.getRightNodeDegree(tweetId) > minResultDegree)
|
||||
.mapValues(_.size)
|
||||
.filter { case (_, cooccurrence) => cooccurrence >= minCooccurrence }
|
||||
.toSeq
|
||||
.map {
|
||||
case (relatedTweetId, cooccurrence) =>
|
||||
val relatedTweetDegree = bipartiteGraph.getRightNodeDegree(relatedTweetId)
|
||||
|
||||
val score = scorePreFactor * cooccurrence / math.log(relatedTweetDegree)
|
||||
toRelatedTweet(relatedTweetId, score, relatedTweetDegree, cooccurrence)
|
||||
}
|
||||
.sortBy(-_.score)
|
||||
}
|
||||
|
||||
def toRelatedTweet(
|
||||
relatedTweetId: Long,
|
||||
score: Double,
|
||||
relatedTweetDegree: Int,
|
||||
cooccurrence: Int
|
||||
): RelatedTweet = {
|
||||
RelatedTweet(
|
||||
tweetId = tweetIDMask.restore(relatedTweetId),
|
||||
score = score,
|
||||
relatedTweetGraphFeatures = Some(
|
||||
GraphFeaturesForTweet(cooccurrence = Some(cooccurrence), degree = Some(relatedTweetDegree)))
|
||||
)
|
||||
}
|
||||
}
|
@ -0,0 +1,35 @@
|
||||
package com.twitter.recos.user_video_graph.util
|
||||
|
||||
import com.twitter.graphjet.bipartite.MultiSegmentIterator
|
||||
import com.twitter.graphjet.bipartite.api.BipartiteGraph
|
||||
import com.twitter.graphjet.bipartite.segment.BipartiteGraphSegment
|
||||
import java.util.Random
|
||||
import scala.collection.mutable.ListBuffer
|
||||
|
||||
object SampleLHSUsersUtil {
|
||||
// sample userId nodes
|
||||
def sampleLHSUsers(
|
||||
maskedTweetId: Long,
|
||||
maxNumSamplesPerNeighbor: Int,
|
||||
bipartiteGraph: BipartiteGraph
|
||||
): Seq[Long] = {
|
||||
val sampledUserIdsIterator = bipartiteGraph
|
||||
.getRandomRightNodeEdges(
|
||||
maskedTweetId,
|
||||
maxNumSamplesPerNeighbor,
|
||||
new Random(System.currentTimeMillis)).asInstanceOf[MultiSegmentIterator[
|
||||
BipartiteGraphSegment
|
||||
]]
|
||||
|
||||
val userIds = new ListBuffer[Long]()
|
||||
if (sampledUserIdsIterator != null) {
|
||||
while (sampledUserIdsIterator.hasNext) {
|
||||
val leftNode = sampledUserIdsIterator.nextLong()
|
||||
// If a user likes too many things, we risk including spammy behavior.
|
||||
if (bipartiteGraph.getLeftNodeDegree(leftNode) < 100)
|
||||
userIds += leftNode
|
||||
}
|
||||
}
|
||||
userIds
|
||||
}
|
||||
}
|
Reference in New Issue
Block a user