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:
twitter-team
2023-03-31 17:36:31 -05:00
commit ef4c5eb65e
5364 changed files with 460239 additions and 0 deletions

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

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

View File

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

View File

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

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

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

View File

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

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

View File

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

View 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.
}

View File

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

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

View File

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

View File

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

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

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

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

View 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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

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

View File

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

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

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

View File

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

View File

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

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

View 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

View File

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

View File

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

View File

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

View File

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

View File

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

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

View File

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

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

View 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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

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

View File

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

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

View File

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

View File

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

View File

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

View File

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

View File

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