mirror of
https://github.com/twitter/the-algorithm.git
synced 2024-11-16 00:25:11 +01:00
[docx] split commit for file 5200
Signed-off-by: Ari Archer <ari.web.xyz@gmail.com>
This commit is contained in:
parent
2f5f511bb8
commit
bc2f1fc165
Binary file not shown.
@ -1,69 +0,0 @@
|
||||
WITH
|
||||
vars AS (
|
||||
SELECT
|
||||
TIMESTAMP("{START_TIME}") AS start_date,
|
||||
TIMESTAMP("{END_TIME}") AS end_date,
|
||||
),
|
||||
|
||||
-- Get raw user-tweet interaction events from UUA (We will use fav engagements here)
|
||||
raw_engagements AS (
|
||||
SELECT
|
||||
userIdentifier.userId AS userId,
|
||||
eventMetadata.sourceTimestampMs AS tsMillis,
|
||||
CASE
|
||||
WHEN actionType IN ({CONTRIBUTING_ACTION_TYPES_STR}) THEN {CONTRIBUTING_ACTION_TWEET_ID_COLUMN}
|
||||
WHEN actionType IN ({UNDO_ACTION_TYPES_STR}) THEN {UNDO_ACTION_TWEET_ID_COLUMN}
|
||||
END AS tweetId,
|
||||
CASE
|
||||
WHEN actionType IN ({CONTRIBUTING_ACTION_TYPES_STR}) THEN 1
|
||||
WHEN actionType IN ({UNDO_ACTION_TYPES_STR}) THEN -1
|
||||
END AS doOrUndo
|
||||
FROM `twttr-bql-unified-prod.unified_user_actions_engagements.streaming_unified_user_actions_engagements`, vars
|
||||
WHERE (DATE(dateHour) >= DATE(vars.start_date) AND DATE(dateHour) <= DATE(vars.end_date))
|
||||
AND eventMetadata.sourceTimestampMs >= UNIX_MILLIS(vars.start_date)
|
||||
AND eventMetadata.sourceTimestampMs <= UNIX_MILLIS(vars.end_date)
|
||||
AND (actionType IN ({CONTRIBUTING_ACTION_TYPES_STR})
|
||||
OR actionType IN ({UNDO_ACTION_TYPES_STR}))
|
||||
),
|
||||
|
||||
-- Get video tweet ids
|
||||
video_tweet_ids AS (
|
||||
WITH vars AS (
|
||||
SELECT
|
||||
TIMESTAMP("{START_TIME}") AS start_date,
|
||||
TIMESTAMP("{END_TIME}") AS end_date
|
||||
),
|
||||
|
||||
-- Get raw user-tweet interaction events from UUA
|
||||
video_view_engagements AS (
|
||||
SELECT item.tweetInfo.actionTweetId AS tweetId
|
||||
FROM `twttr-bql-unified-prod.unified_user_actions_engagements.streaming_unified_user_actions_engagements`, vars
|
||||
WHERE (DATE(dateHour) >= DATE(vars.start_date) AND DATE(dateHour) <= DATE(vars.end_date))
|
||||
AND eventMetadata.sourceTimestampMs >= UNIX_MILLIS(start_date)
|
||||
AND eventMetadata.sourceTimestampMs <= UNIX_MILLIS(end_date)
|
||||
AND (actionType IN ("ClientTweetVideoPlayback50")
|
||||
OR actionType IN ("ClientTweetVideoPlayback95"))
|
||||
)
|
||||
|
||||
SELECT DISTINCT(tweetId)
|
||||
FROM video_view_engagements
|
||||
),
|
||||
|
||||
-- Join video tweet ids
|
||||
video_tweets_engagements AS (
|
||||
SELECT raw_engagements.*
|
||||
FROM raw_engagements JOIN video_tweet_ids USING(tweetId)
|
||||
),
|
||||
|
||||
-- Group by userId and tweetId
|
||||
user_tweet_engagement_pairs AS (
|
||||
SELECT userId, tweetId, ARRAY_AGG(STRUCT(doOrUndo, tsMillis) ORDER BY tsMillis DESC LIMIT 1) AS details, COUNT(*) AS cnt
|
||||
FROM video_tweets_engagements
|
||||
GROUP BY userId, tweetId
|
||||
)
|
||||
|
||||
-- Remove undo events
|
||||
SELECT userId, tweetId, CAST(dt.tsMillis AS FLOAT64) AS tsMillis
|
||||
FROM user_tweet_engagement_pairs, vars
|
||||
CROSS JOIN UNNEST(details) AS dt
|
||||
WHERE dt.doOrUndo = 1
|
@ -1,110 +0,0 @@
|
||||
scala_library(
|
||||
name = "bq_generation",
|
||||
sources = [
|
||||
"**/*.scala",
|
||||
],
|
||||
platform = "java8",
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"beam-internal/src/main/scala/com/twitter/beam/io/dal",
|
||||
"beam-internal/src/main/scala/com/twitter/scio_internal/job",
|
||||
"beam-internal/src/main/scala/com/twitter/scio_internal/runner/dataflow",
|
||||
"src/scala/com/twitter/simclusters_v2/hdfs_sources:offline_tweet_recommendations_from_interested_in_20M_145K_2020-scala",
|
||||
"src/scala/com/twitter/simclusters_v2/hdfs_sources:offline_tweet_recommendations_from_interested_in_20M_145K_2020_HL_0_EL_15-scala",
|
||||
"src/scala/com/twitter/simclusters_v2/hdfs_sources:offline_tweet_recommendations_from_interested_in_20M_145K_2020_HL_2_EL_15-scala",
|
||||
"src/scala/com/twitter/simclusters_v2/hdfs_sources:offline_tweet_recommendations_from_interested_in_20M_145K_2020_HL_2_EL_50-scala",
|
||||
"src/scala/com/twitter/simclusters_v2/hdfs_sources:offline_tweet_recommendations_from_interested_in_20M_145K_2020_HL_8_EL_50-scala",
|
||||
"src/scala/com/twitter/simclusters_v2/hdfs_sources:offline_tweet_recommendations_from_mts_consumer_embeddings-scala",
|
||||
"src/scala/com/twitter/simclusters_v2/scio/bq_generation/common",
|
||||
"src/scala/com/twitter/simclusters_v2/scio/bq_generation/sql",
|
||||
"src/scala/com/twitter/wtf/beam/bq_embedding_export:bq_embedding_export_lib",
|
||||
"tcdc/bq_blaster/src/main/scala/com/twitter/tcdc/bqblaster/beam",
|
||||
],
|
||||
)
|
||||
|
||||
jvm_binary(
|
||||
name = "iikf-tweets-ann-adhoc-job",
|
||||
main = "com.twitter.simclusters_v2.scio.bq_generation.tweets_ann.IIKF2020TweetsANNBQAdhocJob",
|
||||
platform = "java8",
|
||||
dependencies = [
|
||||
":bq_generation",
|
||||
],
|
||||
)
|
||||
|
||||
jvm_binary(
|
||||
name = "iikf-hl-8-el-50-tweets-ann-adhoc-job",
|
||||
main = "com.twitter.simclusters_v2.scio.bq_generation.tweets_ann.IIKF2020Hl8El50TweetsANNBQAdhocJob",
|
||||
platform = "java8",
|
||||
dependencies = [
|
||||
":bq_generation",
|
||||
],
|
||||
)
|
||||
|
||||
jvm_binary(
|
||||
name = "iikf-tweets-ann-batch-job",
|
||||
main = "com.twitter.simclusters_v2.scio.bq_generation.tweets_ann.IIKF2020TweetsANNBQBatchJob",
|
||||
platform = "java8",
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
":bq_generation",
|
||||
],
|
||||
)
|
||||
|
||||
jvm_binary(
|
||||
name = "iikf-hl-0-el-15-tweets-ann-batch-job",
|
||||
main = "com.twitter.simclusters_v2.scio.bq_generation.tweets_ann.IIKF2020Hl0El15TweetsANNBQBatchJob",
|
||||
platform = "java8",
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
":bq_generation",
|
||||
],
|
||||
)
|
||||
|
||||
jvm_binary(
|
||||
name = "iikf-hl-2-el-15-tweets-ann-batch-job",
|
||||
main = "com.twitter.simclusters_v2.scio.bq_generation.tweets_ann.IIKF2020Hl2El15TweetsANNBQBatchJob",
|
||||
platform = "java8",
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
":bq_generation",
|
||||
],
|
||||
)
|
||||
|
||||
jvm_binary(
|
||||
name = "iikf-hl-2-el-50-tweets-ann-batch-job",
|
||||
main = "com.twitter.simclusters_v2.scio.bq_generation.tweets_ann.IIKF2020Hl2El50TweetsANNBQBatchJob",
|
||||
platform = "java8",
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
":bq_generation",
|
||||
],
|
||||
)
|
||||
|
||||
jvm_binary(
|
||||
name = "iikf-hl-8-el-50-tweets-ann-batch-job",
|
||||
main = "com.twitter.simclusters_v2.scio.bq_generation.tweets_ann.IIKF2020Hl8El50TweetsANNBQBatchJob",
|
||||
platform = "java8",
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
":bq_generation",
|
||||
],
|
||||
)
|
||||
|
||||
jvm_binary(
|
||||
name = "mts-consumer-embeddings-tweets-ann-adhoc-job",
|
||||
main = "com.twitter.simclusters_v2.scio.bq_generation.tweets_ann.MTSConsumerEmbeddingsTweetsANNBQAdhocJob",
|
||||
platform = "java8",
|
||||
dependencies = [
|
||||
":bq_generation",
|
||||
],
|
||||
)
|
||||
|
||||
jvm_binary(
|
||||
name = "mts-consumer-embeddings-tweets-ann-batch-job",
|
||||
main = "com.twitter.simclusters_v2.scio.bq_generation.tweets_ann.MTSConsumerEmbeddingsTweetsANNBQBatchJob",
|
||||
platform = "java8",
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
":bq_generation",
|
||||
],
|
||||
)
|
Binary file not shown.
Binary file not shown.
@ -1,33 +0,0 @@
|
||||
package com.twitter.simclusters_v2.scio.bq_generation.tweets_ann
|
||||
|
||||
object Config {
|
||||
/*
|
||||
* Common root path
|
||||
*/
|
||||
val RootMHPath: String = "manhattan_sequence_files/offline_sann/"
|
||||
val RootThriftPath: String = "processed/offline_sann/"
|
||||
val AdhocRootPath = "adhoc/offline_sann/"
|
||||
|
||||
/*
|
||||
* Variables for MH output path
|
||||
*/
|
||||
val IIKFANNOutputPath: String = "tweets_ann/iikf"
|
||||
val IIKFHL0EL15ANNOutputPath: String = "tweets_ann/iikf_hl_0_el_15"
|
||||
val IIKFHL2EL15ANNOutputPath: String = "tweets_ann/iikf_hl_2_el_15"
|
||||
val IIKFHL2EL50ANNOutputPath: String = "tweets_ann/iikf_hl_2_el_50"
|
||||
val IIKFHL8EL50ANNOutputPath: String = "tweets_ann/iikf_hl_8_el_50"
|
||||
val MTSConsumerEmbeddingsANNOutputPath: String = "tweets_ann/mts_consumer_embeddings"
|
||||
|
||||
/*
|
||||
* Variables for tweet embeddings generation
|
||||
*/
|
||||
val SimClustersTweetEmbeddingsGenerationHalfLife: Int = 28800000 // 8hrs in ms
|
||||
val SimClustersTweetEmbeddingsGenerationEmbeddingLength: Int = 15
|
||||
|
||||
/*
|
||||
* Variables for ANN
|
||||
*/
|
||||
val SimClustersANNTopNClustersPerSourceEmbedding: Int = 20
|
||||
val SimClustersANNTopMTweetsPerCluster: Int = 50
|
||||
val SimClustersANNTopKTweetsPerUserRequest: Int = 200
|
||||
}
|
@ -1,95 +0,0 @@
|
||||
To run iikf-tweets-ann-adhoc-job (adhoc):
|
||||
bin/d6w create \
|
||||
${GCP_PROJECT_NAME}/us-central1/iikf-tweets-ann-adhoc-job \
|
||||
src/scala/com/twitter/simclusters_v2/scio/bq_generation/tweets_ann/iikf-tweets-ann-adhoc-job.d6w \
|
||||
--jar dist/iikf-tweets-ann-adhoc-job.jar \
|
||||
--bind=profile.project=${GCP_PROJECT_NAME} \
|
||||
--bind=profile.user_name=your_ldap \
|
||||
--bind=profile.date="2022-03-28" \
|
||||
--bind=profile.machine="n2-highmem-4" \
|
||||
--bind=profile.job_name="iikf-tweets-ann-adhoc-job" --ignore-existing
|
||||
|
||||
To run iikf-hl-8-el-50-tweets-ann-adhoc-job (adhoc):
|
||||
bin/d6w create \
|
||||
${GCP_PROJECT_NAME}/us-central1/iikf-hl-8-el-50-tweets-ann-adhoc-job \
|
||||
src/scala/com/twitter/simclusters_v2/scio/bq_generation/tweets_ann/iikf-hl-8-el-50-tweets-ann-adhoc-job.d6w \
|
||||
--jar dist/iikf-hl-8-el-50-tweets-ann-adhoc-job.jar \
|
||||
--bind=profile.project=${GCP_PROJECT_NAME} \
|
||||
--bind=profile.user_name=your_ldap \
|
||||
--bind=profile.date="2022-03-28" \
|
||||
--bind=profile.machine="n2-highmem-4" \
|
||||
--bind=profile.job_name="iikf-hl-8-el-50-tweets-ann-adhoc-job" --ignore-existing
|
||||
|
||||
To run mts-consumer-embeddings-tweets-ann-adhoc-job (adhoc)
|
||||
bin/d6w create \
|
||||
${GCP_PROJECT_NAME}/us-central1/mts-consumer-embeddings-tweets-ann-adhoc-job \
|
||||
src/scala/com/twitter/simclusters_v2/scio/bq_generation/tweets_ann/mts-consumer-embeddings-tweets-ann-adhoc-job.d6w \
|
||||
--jar dist/mts-consumer-embeddings-tweets-ann-adhoc-job.jar \
|
||||
--bind=profile.project=${GCP_PROJECT_NAME} \
|
||||
--bind=profile.user_name=your_ldap \
|
||||
--bind=profile.date="2022-03-28" \
|
||||
--bind=profile.machine="n2-highmem-4" \
|
||||
--bind=profile.job_name="mts-consumer-embeddings-tweets-ann-adhoc-job" --ignore-existing
|
||||
|
||||
|
||||
To schedule iikf-tweets-ann-batch-job (batch)
|
||||
bin/d6w schedule \
|
||||
${GCP_PROJECT_NAME}/us-central1/iikf-tweets-ann-batch-job \
|
||||
src/scala/com/twitter/simclusters_v2/scio/bq_generation/tweets_ann/iikf-tweets-ann-batch-job.d6w \
|
||||
--bind=profile.project=${GCP_PROJECT_NAME} \
|
||||
--bind=profile.user_name=cassowary \
|
||||
--bind=profile.date="2022-03-26" \
|
||||
--bind=profile.machine="n2-highmem-4" \
|
||||
--bind=profile.job_name="iikf-tweets-ann-batch-job"
|
||||
|
||||
To schedule iikf-hl-0-el-15-tweets-ann-batch-job (batch)
|
||||
bin/d6w schedule \
|
||||
${GCP_PROJECT_NAME}/us-central1/iikf-hl-0-el-15-tweets-ann-batch-job \
|
||||
src/scala/com/twitter/simclusters_v2/scio/bq_generation/tweets_ann/iikf-hl-0-el-15-tweets-ann-batch-job.d6w \
|
||||
--bind=profile.project=${GCP_PROJECT_NAME} \
|
||||
--bind=profile.user_name=cassowary \
|
||||
--bind=profile.date="2022-03-26" \
|
||||
--bind=profile.machine="n2-highmem-4" \
|
||||
--bind=profile.job_name="iikf-hl-0-el-15-tweets-ann-batch-job"
|
||||
|
||||
To schedule iikf-hl-2-el-15-tweets-ann-batch-job (batch)
|
||||
bin/d6w schedule \
|
||||
${GCP_PROJECT_NAME}/us-central1/iikf-hl-2-el-15-tweets-ann-batch-job \
|
||||
src/scala/com/twitter/simclusters_v2/scio/bq_generation/tweets_ann/iikf-hl-2-el-15-tweets-ann-batch-job.d6w \
|
||||
--bind=profile.project=${GCP_PROJECT_NAME} \
|
||||
--bind=profile.user_name=cassowary \
|
||||
--bind=profile.date="2022-03-26" \
|
||||
--bind=profile.machine="n2-highmem-4" \
|
||||
--bind=profile.job_name="iikf-hl-2-el-15-tweets-ann-batch-job"
|
||||
|
||||
To schedule iikf-hl-2-el-50-tweets-ann-batch-job (batch)
|
||||
bin/d6w schedule \
|
||||
${GCP_PROJECT_NAME}/us-central1/iikf-hl-2-el-50-tweets-ann-batch-job \
|
||||
src/scala/com/twitter/simclusters_v2/scio/bq_generation/tweets_ann/iikf-hl-2-el-50-tweets-ann-batch-job.d6w \
|
||||
--bind=profile.project=${GCP_PROJECT_NAME} \
|
||||
--bind=profile.user_name=cassowary \
|
||||
--bind=profile.date="2022-03-26" \
|
||||
--bind=profile.machine="n2-highmem-4" \
|
||||
--bind=profile.job_name="iikf-hl-2-el-50-tweets-ann-batch-job"
|
||||
|
||||
To schedule iikf-hl-8-el-50-tweets-ann-batch-job (batch)
|
||||
bin/d6w schedule \
|
||||
${GCP_PROJECT_NAME}/us-central1/iikf-hl-8-el-50-tweets-ann-batch-job \
|
||||
src/scala/com/twitter/simclusters_v2/scio/bq_generation/tweets_ann/iikf-hl-8-el-50-tweets-ann-batch-job.d6w \
|
||||
--bind=profile.project=${GCP_PROJECT_NAME} \
|
||||
--bind=profile.user_name=cassowary \
|
||||
--bind=profile.date="2022-03-26" \
|
||||
--bind=profile.machine="n2-highmem-4" \
|
||||
--bind=profile.job_name="iikf-hl-8-el-50-tweets-ann-batch-job"
|
||||
|
||||
To schedule mts-consumer-embeddings-tweets-ann-batch-job(batch)
|
||||
bin/d6w schedule \
|
||||
${GCP_PROJECT_NAME}/us-central1/mts-consumer-embeddings-tweets-ann-batch-job \
|
||||
src/scala/com/twitter/simclusters_v2/scio/bq_generation/tweets_ann/mts-consumer-embeddings-tweets-ann-batch-job.d6w \
|
||||
--bind=profile.project=${GCP_PROJECT_NAME} \
|
||||
--bind=profile.user_name=cassowary \
|
||||
--bind=profile.date="2022-03-26" \
|
||||
--bind=profile.machine="n2-highmem-4" \
|
||||
--bind=profile.job_name="mts-consumer-embeddings-tweets-ann-batch-job"
|
||||
|
||||
|
Binary file not shown.
Binary file not shown.
@ -1,120 +0,0 @@
|
||||
package com.twitter.simclusters_v2.scio.bq_generation
|
||||
package tweets_ann
|
||||
|
||||
import com.spotify.scio.ScioContext
|
||||
import com.spotify.scio.values.SCollection
|
||||
import com.twitter.simclusters_v2.thriftscala.CandidateTweet
|
||||
import com.twitter.wtf.beam.bq_embedding_export.BQQueryUtils
|
||||
import org.apache.avro.generic.GenericData
|
||||
import org.apache.avro.generic.GenericRecord
|
||||
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO
|
||||
import org.apache.beam.sdk.io.gcp.bigquery.SchemaAndRecord
|
||||
import org.apache.beam.sdk.transforms.SerializableFunction
|
||||
import org.joda.time.DateTime
|
||||
import scala.collection.mutable.ListBuffer
|
||||
|
||||
object TweetsANNFromBQ {
|
||||
// Default ANN config variables
|
||||
val topNClustersPerSourceEmbedding = Config.SimClustersANNTopNClustersPerSourceEmbedding
|
||||
val topMTweetsPerCluster = Config.SimClustersANNTopMTweetsPerCluster
|
||||
val topKTweetsPerUserRequest = Config.SimClustersANNTopKTweetsPerUserRequest
|
||||
|
||||
// SQL file paths
|
||||
val tweetsANNSQLPath =
|
||||
s"/com/twitter/simclusters_v2/scio/bq_generation/sql/tweets_ann.sql"
|
||||
val tweetsEmbeddingGenerationSQLPath =
|
||||
s"/com/twitter/simclusters_v2/scio/bq_generation/sql/tweet_embeddings_generation.sql"
|
||||
|
||||
// Function that parses the GenericRecord results we read from BQ
|
||||
val parseUserToTweetRecommendationsFunc =
|
||||
new SerializableFunction[SchemaAndRecord, UserToTweetRecommendations] {
|
||||
override def apply(record: SchemaAndRecord): UserToTweetRecommendations = {
|
||||
val genericRecord: GenericRecord = record.getRecord()
|
||||
UserToTweetRecommendations(
|
||||
userId = genericRecord.get("userId").toString.toLong,
|
||||
tweetCandidates = parseTweetIdColumn(genericRecord, "tweets"),
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
// Parse tweetId candidates column
|
||||
def parseTweetIdColumn(
|
||||
genericRecord: GenericRecord,
|
||||
columnName: String
|
||||
): List[CandidateTweet] = {
|
||||
val tweetIds: GenericData.Array[GenericRecord] =
|
||||
genericRecord.get(columnName).asInstanceOf[GenericData.Array[GenericRecord]]
|
||||
val results: ListBuffer[CandidateTweet] = new ListBuffer[CandidateTweet]()
|
||||
tweetIds.forEach((sc: GenericRecord) => {
|
||||
results += CandidateTweet(
|
||||
tweetId = sc.get("tweetId").toString.toLong,
|
||||
score = Some(sc.get("logCosineSimilarityScore").toString.toDouble)
|
||||
)
|
||||
})
|
||||
results.toList
|
||||
}
|
||||
|
||||
def getTweetEmbeddingsSQL(
|
||||
queryDate: DateTime,
|
||||
consumerEmbeddingsSQL: String,
|
||||
tweetEmbeddingsSQLPath: String,
|
||||
tweetEmbeddingsHalfLife: Int,
|
||||
tweetEmbeddingsLength: Int
|
||||
): String = {
|
||||
// We read one day of fav events to construct our tweet embeddings
|
||||
val templateVariables =
|
||||
Map(
|
||||
"CONSUMER_EMBEDDINGS_SQL" -> consumerEmbeddingsSQL,
|
||||
"QUERY_DATE" -> queryDate.toString(),
|
||||
"START_TIME" -> queryDate.minusDays(1).toString(),
|
||||
"END_TIME" -> queryDate.toString(),
|
||||
"MIN_SCORE_THRESHOLD" -> 0.0.toString,
|
||||
"HALF_LIFE" -> tweetEmbeddingsHalfLife.toString,
|
||||
"TWEET_EMBEDDING_LENGTH" -> tweetEmbeddingsLength.toString,
|
||||
"NO_OLDER_TWEETS_THAN_DATE" -> queryDate.minusDays(1).toString(),
|
||||
)
|
||||
BQQueryUtils.getBQQueryFromSqlFile(tweetEmbeddingsSQLPath, templateVariables)
|
||||
}
|
||||
|
||||
def getTweetRecommendationsBQ(
|
||||
sc: ScioContext,
|
||||
queryTimestamp: DateTime,
|
||||
consumerEmbeddingsSQL: String,
|
||||
tweetEmbeddingsHalfLife: Int,
|
||||
tweetEmbeddingsLength: Int
|
||||
): SCollection[UserToTweetRecommendations] = {
|
||||
// Get the tweet embeddings SQL string based on the provided consumerEmbeddingsSQL
|
||||
val tweetEmbeddingsSQL =
|
||||
getTweetEmbeddingsSQL(
|
||||
queryTimestamp,
|
||||
consumerEmbeddingsSQL,
|
||||
tweetsEmbeddingGenerationSQLPath,
|
||||
tweetEmbeddingsHalfLife,
|
||||
tweetEmbeddingsLength
|
||||
)
|
||||
|
||||
// Define template variables which we would like to be replaced in the corresponding sql file
|
||||
val templateVariables =
|
||||
Map(
|
||||
"CONSUMER_EMBEDDINGS_SQL" -> consumerEmbeddingsSQL,
|
||||
"TWEET_EMBEDDINGS_SQL" -> tweetEmbeddingsSQL,
|
||||
"TOP_N_CLUSTER_PER_SOURCE_EMBEDDING" -> topNClustersPerSourceEmbedding.toString,
|
||||
"TOP_M_TWEETS_PER_CLUSTER" -> topMTweetsPerCluster.toString,
|
||||
"TOP_K_TWEETS_PER_USER_REQUEST" -> topKTweetsPerUserRequest.toString
|
||||
)
|
||||
val query = BQQueryUtils.getBQQueryFromSqlFile(tweetsANNSQLPath, templateVariables)
|
||||
|
||||
// Run SimClusters ANN on BQ and parse the results
|
||||
sc.customInput(
|
||||
s"SimClusters BQ ANN",
|
||||
BigQueryIO
|
||||
.read(parseUserToTweetRecommendationsFunc)
|
||||
.fromQuery(query)
|
||||
.usingStandardSql()
|
||||
)
|
||||
}
|
||||
|
||||
case class UserToTweetRecommendations(
|
||||
userId: Long,
|
||||
tweetCandidates: List[CandidateTweet])
|
||||
}
|
Binary file not shown.
@ -1,297 +0,0 @@
|
||||
package com.twitter.simclusters_v2.scio.bq_generation
|
||||
package tweets_ann
|
||||
|
||||
import com.google.api.services.bigquery.model.TimePartitioning
|
||||
import com.spotify.scio.ScioContext
|
||||
import com.spotify.scio.coders.Coder
|
||||
import com.twitter.beam.io.dal.DAL
|
||||
import com.twitter.beam.io.fs.multiformat.PathLayout
|
||||
import com.twitter.beam.job.DateRangeOptions
|
||||
import com.twitter.conversions.DurationOps.richDurationFromInt
|
||||
import com.twitter.dal.client.dataset.KeyValDALDataset
|
||||
import com.twitter.scalding_internal.multiformat.format.keyval.KeyVal
|
||||
import com.twitter.scio_internal.coders.ThriftStructLazyBinaryScroogeCoder
|
||||
import com.twitter.scio_internal.job.ScioBeamJob
|
||||
import com.twitter.scrooge.ThriftStruct
|
||||
import com.twitter.simclusters_v2.scio.bq_generation.common.BQGenerationUtil.getMTSConsumerEmbeddingsFav90P20MSQL
|
||||
import com.twitter.simclusters_v2.scio.bq_generation.common.BQGenerationUtil.getInterestedIn2020SQL
|
||||
import com.twitter.simclusters_v2.scio.bq_generation.tweets_ann.TweetsANNFromBQ.getTweetRecommendationsBQ
|
||||
import com.twitter.simclusters_v2.hdfs_sources.OfflineTweetRecommendationsFromInterestedIn20M145K2020ScalaDataset
|
||||
import com.twitter.simclusters_v2.hdfs_sources.OfflineTweetRecommendationsFromInterestedIn20M145K2020Hl0El15ScalaDataset
|
||||
import com.twitter.simclusters_v2.hdfs_sources.OfflineTweetRecommendationsFromInterestedIn20M145K2020Hl2El15ScalaDataset
|
||||
import com.twitter.simclusters_v2.hdfs_sources.OfflineTweetRecommendationsFromInterestedIn20M145K2020Hl2El50ScalaDataset
|
||||
import com.twitter.simclusters_v2.hdfs_sources.OfflineTweetRecommendationsFromInterestedIn20M145K2020Hl8El50ScalaDataset
|
||||
import com.twitter.simclusters_v2.hdfs_sources.OfflineTweetRecommendationsFromMtsConsumerEmbeddingsScalaDataset
|
||||
import com.twitter.simclusters_v2.scio.bq_generation.common.BQTableDetails
|
||||
import com.twitter.simclusters_v2.thriftscala.CandidateTweets
|
||||
import com.twitter.simclusters_v2.thriftscala.CandidateTweetsList
|
||||
import com.twitter.tcdc.bqblaster.beam.syntax.BigQueryIOHelpers
|
||||
import com.twitter.tcdc.bqblaster.beam.BQBlasterIO.AvroConverter
|
||||
import com.twitter.tcdc.bqblaster.core.avro.TypedProjection
|
||||
import com.twitter.tcdc.bqblaster.core.transform.RootTransform
|
||||
import java.time.Instant
|
||||
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO
|
||||
import org.joda.time.DateTime
|
||||
|
||||
trait TweetsANNJob extends ScioBeamJob[DateRangeOptions] {
|
||||
// Configs to set for different type of embeddings and jobs
|
||||
val isAdhoc: Boolean
|
||||
val getConsumerEmbeddingsSQLFunc: (DateTime, Int) => String
|
||||
val outputTable: BQTableDetails
|
||||
val keyValDatasetOutputPath: String
|
||||
val tweetRecommentationsSnapshotDataset: KeyValDALDataset[KeyVal[Long, CandidateTweetsList]]
|
||||
val tweetEmbeddingsGenerationHalfLife: Int = Config.SimClustersTweetEmbeddingsGenerationHalfLife
|
||||
val tweetEmbeddingsGenerationEmbeddingLength: Int =
|
||||
Config.SimClustersTweetEmbeddingsGenerationEmbeddingLength
|
||||
|
||||
// Base configs
|
||||
val projectId = "twttr-recos-ml-prod"
|
||||
val environment: DAL.Env = if (isAdhoc) DAL.Environment.Dev else DAL.Environment.Prod
|
||||
|
||||
override implicit def scroogeCoder[T <: ThriftStruct: Manifest]: Coder[T] =
|
||||
ThriftStructLazyBinaryScroogeCoder.scroogeCoder
|
||||
|
||||
override def configurePipeline(sc: ScioContext, opts: DateRangeOptions): Unit = {
|
||||
// The time when the job is scheduled
|
||||
val queryTimestamp = opts.interval.getEnd
|
||||
|
||||
// Read consumer embeddings SQL
|
||||
val consumerEmbeddingsSQL = getConsumerEmbeddingsSQLFunc(queryTimestamp, 14)
|
||||
|
||||
// Generate tweet embeddings and tweet ANN results
|
||||
val tweetRecommendations =
|
||||
getTweetRecommendationsBQ(
|
||||
sc,
|
||||
queryTimestamp,
|
||||
consumerEmbeddingsSQL,
|
||||
tweetEmbeddingsGenerationHalfLife,
|
||||
tweetEmbeddingsGenerationEmbeddingLength
|
||||
)
|
||||
|
||||
// Setup BQ writer
|
||||
val ingestionTime = opts.getDate().value.getEnd.toDate
|
||||
val bqFieldsTransform = RootTransform
|
||||
.Builder()
|
||||
.withPrependedFields("ingestionTime" -> TypedProjection.fromConstant(ingestionTime))
|
||||
val timePartitioning = new TimePartitioning()
|
||||
.setType("HOUR").setField("ingestionTime").setExpirationMs(3.days.inMilliseconds)
|
||||
val bqWriter = BigQueryIO
|
||||
.write[CandidateTweets]
|
||||
.to(outputTable.toString)
|
||||
.withExtendedErrorInfo()
|
||||
.withTimePartitioning(timePartitioning)
|
||||
.withLoadJobProjectId(projectId)
|
||||
.withThriftSupport(bqFieldsTransform.build(), AvroConverter.Legacy)
|
||||
.withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED)
|
||||
.withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_APPEND)
|
||||
|
||||
// Save Tweet ANN results to BQ
|
||||
tweetRecommendations
|
||||
.map { userToTweetRecommendations =>
|
||||
{
|
||||
CandidateTweets(
|
||||
targetUserId = userToTweetRecommendations.userId,
|
||||
recommendedTweets = userToTweetRecommendations.tweetCandidates)
|
||||
}
|
||||
}
|
||||
.saveAsCustomOutput(s"WriteToBQTable - ${outputTable}", bqWriter)
|
||||
|
||||
// Save Tweet ANN results as KeyValSnapshotDataset
|
||||
tweetRecommendations
|
||||
.map { userToTweetRecommendations =>
|
||||
KeyVal(
|
||||
userToTweetRecommendations.userId,
|
||||
CandidateTweetsList(userToTweetRecommendations.tweetCandidates))
|
||||
}.saveAsCustomOutput(
|
||||
name = "WriteTweetRecommendationsToKeyValDataset",
|
||||
DAL.writeVersionedKeyVal(
|
||||
tweetRecommentationsSnapshotDataset,
|
||||
PathLayout.VersionedPath(prefix =
|
||||
((if (!isAdhoc)
|
||||
Config.RootMHPath
|
||||
else
|
||||
Config.AdhocRootPath)
|
||||
+ keyValDatasetOutputPath)),
|
||||
instant = Instant.ofEpochMilli(opts.interval.getEndMillis - 1L),
|
||||
environmentOverride = environment,
|
||||
)
|
||||
)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Scio job for adhoc run for tweet recommendations from IIKF 2020
|
||||
*/
|
||||
object IIKF2020TweetsANNBQAdhocJob extends TweetsANNJob {
|
||||
override val isAdhoc = true
|
||||
override val getConsumerEmbeddingsSQLFunc = getInterestedIn2020SQL
|
||||
override val outputTable = BQTableDetails(
|
||||
"twttr-recos-ml-prod",
|
||||
"multi_type_simclusters",
|
||||
"offline_tweet_recommendations_from_interested_in_20M_145K_2020_adhoc")
|
||||
override val keyValDatasetOutputPath = Config.IIKFANNOutputPath
|
||||
override val tweetRecommentationsSnapshotDataset: KeyValDALDataset[
|
||||
KeyVal[Long, CandidateTweetsList]
|
||||
] =
|
||||
OfflineTweetRecommendationsFromInterestedIn20M145K2020ScalaDataset
|
||||
}
|
||||
|
||||
/**
|
||||
* Scio job for adhoc run for tweet recommendations from IIKF 2020 with
|
||||
* - Half life = 8hrs
|
||||
* - Embedding Length = 50
|
||||
*/
|
||||
object IIKF2020Hl8El50TweetsANNBQAdhocJob extends TweetsANNJob {
|
||||
override val isAdhoc = true
|
||||
override val getConsumerEmbeddingsSQLFunc = getInterestedIn2020SQL
|
||||
override val outputTable = BQTableDetails(
|
||||
"twttr-recos-ml-prod",
|
||||
"multi_type_simclusters",
|
||||
"offline_tweet_recommendations_from_interested_in_20M_145K_2020_HL_8_EL_50_adhoc")
|
||||
override val keyValDatasetOutputPath = Config.IIKFHL8EL50ANNOutputPath
|
||||
override val tweetEmbeddingsGenerationEmbeddingLength: Int = 50
|
||||
override val tweetRecommentationsSnapshotDataset: KeyValDALDataset[
|
||||
KeyVal[Long, CandidateTweetsList]
|
||||
] = {
|
||||
OfflineTweetRecommendationsFromInterestedIn20M145K2020Hl8El50ScalaDataset
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Scio job for adhoc run for tweet recommendations from MTS Consumer Embeddings
|
||||
*/
|
||||
object MTSConsumerEmbeddingsTweetsANNBQAdhocJob extends TweetsANNJob {
|
||||
override val isAdhoc = true
|
||||
override val getConsumerEmbeddingsSQLFunc = getMTSConsumerEmbeddingsFav90P20MSQL
|
||||
override val outputTable = BQTableDetails(
|
||||
"twttr-recos-ml-prod",
|
||||
"multi_type_simclusters",
|
||||
"offline_tweet_recommendations_from_mts_consumer_embeddings_adhoc")
|
||||
override val keyValDatasetOutputPath = Config.MTSConsumerEmbeddingsANNOutputPath
|
||||
override val tweetRecommentationsSnapshotDataset: KeyValDALDataset[
|
||||
KeyVal[Long, CandidateTweetsList]
|
||||
] =
|
||||
OfflineTweetRecommendationsFromMtsConsumerEmbeddingsScalaDataset
|
||||
}
|
||||
|
||||
/**
|
||||
Scio job for batch run for tweet recommendations from IIKF 2020
|
||||
The schedule cmd needs to be run only if there is any change in the config
|
||||
*/
|
||||
object IIKF2020TweetsANNBQBatchJob extends TweetsANNJob {
|
||||
override val isAdhoc = false
|
||||
override val getConsumerEmbeddingsSQLFunc = getInterestedIn2020SQL
|
||||
override val outputTable = BQTableDetails(
|
||||
"twttr-bq-cassowary-prod",
|
||||
"user",
|
||||
"offline_tweet_recommendations_from_interested_in_20M_145K_2020")
|
||||
override val keyValDatasetOutputPath = Config.IIKFANNOutputPath
|
||||
override val tweetRecommentationsSnapshotDataset: KeyValDALDataset[
|
||||
KeyVal[Long, CandidateTweetsList]
|
||||
] =
|
||||
OfflineTweetRecommendationsFromInterestedIn20M145K2020ScalaDataset
|
||||
}
|
||||
|
||||
/**
|
||||
Scio job for batch run for tweet recommendations from IIKF 2020 with parameter setup:
|
||||
- Half Life: None, no decay, direct sum
|
||||
- Embedding Length: 15
|
||||
The schedule cmd needs to be run only if there is any change in the config
|
||||
*/
|
||||
object IIKF2020Hl0El15TweetsANNBQBatchJob extends TweetsANNJob {
|
||||
override val isAdhoc = false
|
||||
override val getConsumerEmbeddingsSQLFunc = getInterestedIn2020SQL
|
||||
override val outputTable = BQTableDetails(
|
||||
"twttr-bq-cassowary-prod",
|
||||
"user",
|
||||
"offline_tweet_recommendations_from_interested_in_20M_145K_2020_HL_0_EL_15")
|
||||
override val keyValDatasetOutputPath = Config.IIKFHL0EL15ANNOutputPath
|
||||
override val tweetEmbeddingsGenerationHalfLife: Int = -1
|
||||
override val tweetRecommentationsSnapshotDataset: KeyValDALDataset[
|
||||
KeyVal[Long, CandidateTweetsList]
|
||||
] =
|
||||
OfflineTweetRecommendationsFromInterestedIn20M145K2020Hl0El15ScalaDataset
|
||||
}
|
||||
|
||||
/**
|
||||
Scio job for batch run for tweet recommendations from IIKF 2020 with parameter setup:
|
||||
- Half Life: 2hrs
|
||||
- Embedding Length: 15
|
||||
The schedule cmd needs to be run only if there is any change in the config
|
||||
*/
|
||||
object IIKF2020Hl2El15TweetsANNBQBatchJob extends TweetsANNJob {
|
||||
override val isAdhoc = false
|
||||
override val getConsumerEmbeddingsSQLFunc = getInterestedIn2020SQL
|
||||
override val outputTable = BQTableDetails(
|
||||
"twttr-bq-cassowary-prod",
|
||||
"user",
|
||||
"offline_tweet_recommendations_from_interested_in_20M_145K_2020_HL_2_EL_15")
|
||||
override val keyValDatasetOutputPath = Config.IIKFHL2EL15ANNOutputPath
|
||||
override val tweetEmbeddingsGenerationHalfLife: Int = 7200000 // 2hrs in ms
|
||||
override val tweetRecommentationsSnapshotDataset: KeyValDALDataset[
|
||||
KeyVal[Long, CandidateTweetsList]
|
||||
] =
|
||||
OfflineTweetRecommendationsFromInterestedIn20M145K2020Hl2El15ScalaDataset
|
||||
}
|
||||
|
||||
/**
|
||||
Scio job for batch run for tweet recommendations from IIKF 2020 with parameter setup:
|
||||
- Half Life: 2hrs
|
||||
- Embedding Length: 50
|
||||
The schedule cmd needs to be run only if there is any change in the config
|
||||
*/
|
||||
object IIKF2020Hl2El50TweetsANNBQBatchJob extends TweetsANNJob {
|
||||
override val isAdhoc = false
|
||||
override val getConsumerEmbeddingsSQLFunc = getInterestedIn2020SQL
|
||||
override val outputTable = BQTableDetails(
|
||||
"twttr-bq-cassowary-prod",
|
||||
"user",
|
||||
"offline_tweet_recommendations_from_interested_in_20M_145K_2020_HL_2_EL_50")
|
||||
override val keyValDatasetOutputPath = Config.IIKFHL2EL50ANNOutputPath
|
||||
override val tweetEmbeddingsGenerationHalfLife: Int = 7200000 // 2hrs in ms
|
||||
override val tweetEmbeddingsGenerationEmbeddingLength: Int = 50
|
||||
override val tweetRecommentationsSnapshotDataset: KeyValDALDataset[
|
||||
KeyVal[Long, CandidateTweetsList]
|
||||
] =
|
||||
OfflineTweetRecommendationsFromInterestedIn20M145K2020Hl2El50ScalaDataset
|
||||
}
|
||||
|
||||
/**
|
||||
Scio job for batch run for tweet recommendations from IIKF 2020 with parameter setup:
|
||||
- Half Life: 8hrs
|
||||
- Embedding Length: 50
|
||||
The schedule cmd needs to be run only if there is any change in the config
|
||||
*/
|
||||
object IIKF2020Hl8El50TweetsANNBQBatchJob extends TweetsANNJob {
|
||||
override val isAdhoc = false
|
||||
override val getConsumerEmbeddingsSQLFunc = getInterestedIn2020SQL
|
||||
override val outputTable = BQTableDetails(
|
||||
"twttr-bq-cassowary-prod",
|
||||
"user",
|
||||
"offline_tweet_recommendations_from_interested_in_20M_145K_2020_HL_8_EL_50")
|
||||
override val keyValDatasetOutputPath = Config.IIKFHL8EL50ANNOutputPath
|
||||
override val tweetEmbeddingsGenerationEmbeddingLength: Int = 50
|
||||
override val tweetRecommentationsSnapshotDataset: KeyValDALDataset[
|
||||
KeyVal[Long, CandidateTweetsList]
|
||||
] =
|
||||
OfflineTweetRecommendationsFromInterestedIn20M145K2020Hl8El50ScalaDataset
|
||||
}
|
||||
|
||||
/**
|
||||
Scio job for batch run for tweet recommendations from MTS Consumer Embeddings
|
||||
The schedule cmd needs to be run only if there is any change in the config
|
||||
*/
|
||||
object MTSConsumerEmbeddingsTweetsANNBQBatchJob extends TweetsANNJob {
|
||||
override val isAdhoc = false
|
||||
override val getConsumerEmbeddingsSQLFunc = getMTSConsumerEmbeddingsFav90P20MSQL
|
||||
override val outputTable = BQTableDetails(
|
||||
"twttr-bq-cassowary-prod",
|
||||
"user",
|
||||
"offline_tweet_recommendations_from_mts_consumer_embeddings")
|
||||
override val keyValDatasetOutputPath = Config.MTSConsumerEmbeddingsANNOutputPath
|
||||
override val tweetRecommentationsSnapshotDataset: KeyValDALDataset[
|
||||
KeyVal[Long, CandidateTweetsList]
|
||||
] =
|
||||
OfflineTweetRecommendationsFromMtsConsumerEmbeddingsScalaDataset
|
||||
}
|
@ -1,39 +0,0 @@
|
||||
class Profile(Struct):
|
||||
project = Required(String)
|
||||
date = Required(String)
|
||||
environment = Default(String, 'dev')
|
||||
job_name = Default(String, 'iikf-hl-0-el-15-tweets-ann-batch-job')
|
||||
machine = Default(String, 'n2-highmem-4')
|
||||
|
||||
job = Job(
|
||||
name='{{profile.job_name}}',
|
||||
project='{{profile.project}}',
|
||||
staging_bucket='{{profile.project}}',
|
||||
service_account='{{profile.user_name}}-shdw@twttr-dp-svc-accounts.iam.gserviceaccount.com',
|
||||
region='us-central1',
|
||||
worker_config=WorkerConfig(
|
||||
num_workers=2,
|
||||
worker_machine_type='{{profile.machine}}',
|
||||
worker_disk_type=WorkerDiskType('HDD'),
|
||||
),
|
||||
extra_args={
|
||||
"date": '{{profile.date}}'
|
||||
},
|
||||
service_identifier='twtr:svc:{{profile.user_name}}:{{profile.job_name}}:{{profile.environment}}:{{profile.cluster}}',
|
||||
deployment_config=BatchDeploymentConfig(
|
||||
role='{{profile.user_name}}',
|
||||
environment='prod',
|
||||
build_target='src/scala/com/twitter/simclusters_v2/scio/bq_generation/tweets_ann:iikf-hl-0-el-15-tweets-ann-batch-job',
|
||||
gcp_deployment_credentials='/var/lib/tss/keys/{{profile.user_name}}/cloud/gcp/dp/shadow.json',
|
||||
statebird_config=StatebirdConfig(
|
||||
batch_width='PT4H',
|
||||
first_time='{{profile.date}}',
|
||||
),
|
||||
workflow_config=WorkflowConfig(
|
||||
play=True,
|
||||
),
|
||||
timeout='PT24H'
|
||||
)
|
||||
)
|
||||
|
||||
jobs=[job]
|
Binary file not shown.
@ -1,39 +0,0 @@
|
||||
class Profile(Struct):
|
||||
project = Required(String)
|
||||
date = Required(String)
|
||||
environment = Default(String, 'dev')
|
||||
job_name = Default(String, 'iikf-hl-2-el-15-tweets-ann-batch-job')
|
||||
machine = Default(String, 'n2-highmem-4')
|
||||
|
||||
job = Job(
|
||||
name='{{profile.job_name}}',
|
||||
project='{{profile.project}}',
|
||||
staging_bucket='{{profile.project}}',
|
||||
service_account='{{profile.user_name}}-shdw@twttr-dp-svc-accounts.iam.gserviceaccount.com',
|
||||
region='us-central1',
|
||||
worker_config=WorkerConfig(
|
||||
num_workers=2,
|
||||
worker_machine_type='{{profile.machine}}',
|
||||
worker_disk_type=WorkerDiskType('HDD'),
|
||||
),
|
||||
extra_args={
|
||||
"date": '{{profile.date}}'
|
||||
},
|
||||
service_identifier='twtr:svc:{{profile.user_name}}:{{profile.job_name}}:{{profile.environment}}:{{profile.cluster}}',
|
||||
deployment_config=BatchDeploymentConfig(
|
||||
role='{{profile.user_name}}',
|
||||
environment='prod',
|
||||
build_target='src/scala/com/twitter/simclusters_v2/scio/bq_generation/tweets_ann:iikf-hl-2-el-15-tweets-ann-batch-job',
|
||||
gcp_deployment_credentials='/var/lib/tss/keys/{{profile.user_name}}/cloud/gcp/dp/shadow.json',
|
||||
statebird_config=StatebirdConfig(
|
||||
batch_width='PT4H',
|
||||
first_time='{{profile.date}}',
|
||||
),
|
||||
workflow_config=WorkflowConfig(
|
||||
play=True,
|
||||
),
|
||||
timeout='PT24H'
|
||||
)
|
||||
)
|
||||
|
||||
jobs=[job]
|
Binary file not shown.
@ -1,39 +0,0 @@
|
||||
class Profile(Struct):
|
||||
project = Required(String)
|
||||
date = Required(String)
|
||||
environment = Default(String, 'dev')
|
||||
job_name = Default(String, 'iikf-hl-2-el-50-tweets-ann-batch-job')
|
||||
machine = Default(String, 'n2-highmem-4')
|
||||
|
||||
job = Job(
|
||||
name='{{profile.job_name}}',
|
||||
project='{{profile.project}}',
|
||||
staging_bucket='{{profile.project}}',
|
||||
service_account='{{profile.user_name}}-shdw@twttr-dp-svc-accounts.iam.gserviceaccount.com',
|
||||
region='us-central1',
|
||||
worker_config=WorkerConfig(
|
||||
num_workers=2,
|
||||
worker_machine_type='{{profile.machine}}',
|
||||
worker_disk_type=WorkerDiskType('HDD'),
|
||||
),
|
||||
extra_args={
|
||||
"date": '{{profile.date}}'
|
||||
},
|
||||
service_identifier='twtr:svc:{{profile.user_name}}:{{profile.job_name}}:{{profile.environment}}:{{profile.cluster}}',
|
||||
deployment_config=BatchDeploymentConfig(
|
||||
role='{{profile.user_name}}',
|
||||
environment='prod',
|
||||
build_target='src/scala/com/twitter/simclusters_v2/scio/bq_generation/tweets_ann:iikf-hl-2-el-50-tweets-ann-batch-job',
|
||||
gcp_deployment_credentials='/var/lib/tss/keys/{{profile.user_name}}/cloud/gcp/dp/shadow.json',
|
||||
statebird_config=StatebirdConfig(
|
||||
batch_width='PT4H',
|
||||
first_time='{{profile.date}}',
|
||||
),
|
||||
workflow_config=WorkflowConfig(
|
||||
play=True,
|
||||
),
|
||||
timeout='PT24H'
|
||||
)
|
||||
)
|
||||
|
||||
jobs=[job]
|
Binary file not shown.
@ -1,39 +0,0 @@
|
||||
class Profile(Struct):
|
||||
project = Required(String)
|
||||
date = Required(String)
|
||||
environment = Default(String, 'dev')
|
||||
job_name = Default(String, 'iikf-hl-8-el-50-tweets-ann-batch-job')
|
||||
machine = Default(String, 'n2-highmem-4')
|
||||
|
||||
job = Job(
|
||||
name='{{profile.job_name}}',
|
||||
project='{{profile.project}}',
|
||||
staging_bucket='{{profile.project}}',
|
||||
service_account='{{profile.user_name}}-shdw@twttr-dp-svc-accounts.iam.gserviceaccount.com',
|
||||
region='us-central1',
|
||||
worker_config=WorkerConfig(
|
||||
num_workers=2,
|
||||
worker_machine_type='{{profile.machine}}',
|
||||
worker_disk_type=WorkerDiskType('HDD'),
|
||||
),
|
||||
extra_args={
|
||||
"date": '{{profile.date}}'
|
||||
},
|
||||
service_identifier='twtr:svc:{{profile.user_name}}:{{profile.job_name}}:{{profile.environment}}:{{profile.cluster}}',
|
||||
deployment_config=BatchDeploymentConfig(
|
||||
role='{{profile.user_name}}',
|
||||
environment='prod',
|
||||
build_target='src/scala/com/twitter/simclusters_v2/scio/bq_generation/tweets_ann:iikf-hl-8-el-50-tweets-ann-batch-job',
|
||||
gcp_deployment_credentials='/var/lib/tss/keys/{{profile.user_name}}/cloud/gcp/dp/shadow.json',
|
||||
statebird_config=StatebirdConfig(
|
||||
batch_width='PT4H',
|
||||
first_time='{{profile.date}}',
|
||||
),
|
||||
workflow_config=WorkflowConfig(
|
||||
play=True,
|
||||
),
|
||||
timeout='PT24H'
|
||||
)
|
||||
)
|
||||
|
||||
jobs=[job]
|
Binary file not shown.
@ -1,39 +0,0 @@
|
||||
class Profile(Struct):
|
||||
project = Required(String)
|
||||
date = Required(String)
|
||||
environment = Default(String, 'dev')
|
||||
job_name = Default(String, 'iikf-hl-8-el-50-tweets-ann-batch-job')
|
||||
machine = Default(String, 'n2-highmem-4')
|
||||
|
||||
job = Job(
|
||||
name='{{profile.job_name}}',
|
||||
project='{{profile.project}}',
|
||||
staging_bucket='{{profile.project}}',
|
||||
service_account='{{profile.user_name}}-shdw@twttr-dp-svc-accounts.iam.gserviceaccount.com',
|
||||
region='us-central1',
|
||||
worker_config=WorkerConfig(
|
||||
num_workers=2,
|
||||
worker_machine_type='{{profile.machine}}',
|
||||
worker_disk_type=WorkerDiskType('HDD'),
|
||||
),
|
||||
extra_args={
|
||||
"date": '{{profile.date}}'
|
||||
},
|
||||
service_identifier='twtr:svc:{{profile.user_name}}:{{profile.job_name}}:{{profile.environment}}:{{profile.cluster}}',
|
||||
deployment_config=BatchDeploymentConfig(
|
||||
role='{{profile.user_name}}',
|
||||
environment='prod',
|
||||
build_target='src/scala/com/twitter/simclusters_v2/scio/bq_generation/tweets_ann:iikf-hl-8-el-50-tweets-ann-batch-job',
|
||||
gcp_deployment_credentials='/var/lib/tss/keys/{{profile.user_name}}/cloud/gcp/dp/shadow.json',
|
||||
statebird_config=StatebirdConfig(
|
||||
batch_width='PT4H',
|
||||
first_time='{{profile.date}}',
|
||||
),
|
||||
workflow_config=WorkflowConfig(
|
||||
play=True,
|
||||
),
|
||||
timeout='PT24H'
|
||||
)
|
||||
)
|
||||
|
||||
jobs=[job]
|
Binary file not shown.
@ -1,34 +0,0 @@
|
||||
class Profile(Struct):
|
||||
project = Required(String)
|
||||
date = Required(String)
|
||||
environment = Default(String, 'dev')
|
||||
job_name = Default(String, 'iikf-tweets-ann-adhoc-job')
|
||||
machine = Default(String, 'n2-highmem-4')
|
||||
|
||||
job = Job(
|
||||
name='{{profile.job_name}}',
|
||||
project='{{profile.project}}',
|
||||
staging_bucket='{{profile.project}}',
|
||||
service_account='{{profile.user_name}}-shdw@twttr-dp-svc-accounts.iam.gserviceaccount.com',
|
||||
region='us-central1',
|
||||
worker_config=WorkerConfig(
|
||||
num_workers=2,
|
||||
worker_machine_type='{{profile.machine}}',
|
||||
worker_disk_type=WorkerDiskType('HDD'),
|
||||
),
|
||||
extra_args={
|
||||
"date": '{{profile.date}}'
|
||||
},
|
||||
service_identifier='twtr:svc:{{profile.user_name}}:{{profile.job_name}}:{{profile.environment}}:{{profile.cluster}}',
|
||||
deployment_config=BatchDeploymentConfig(
|
||||
role='{{profile.user_name}}',
|
||||
build_target='src/scala/com/twitter/simclusters_v2/scio/bq_generation/tweets_ann:iikf-tweets-ann-adhoc-job',
|
||||
gcp_deployment_credentials='/var/lib/tss/keys/{{profile.user_name}}/cloud/gcp/dp/shadow.json',
|
||||
statebird_config=StatebirdConfig(
|
||||
batch_width='PT2H',
|
||||
first_time='{{profile.date}}',
|
||||
),
|
||||
)
|
||||
)
|
||||
|
||||
jobs=[job]
|
Binary file not shown.
@ -1,39 +0,0 @@
|
||||
class Profile(Struct):
|
||||
project = Required(String)
|
||||
date = Required(String)
|
||||
environment = Default(String, 'dev')
|
||||
job_name = Default(String, 'iikf-tweets-ann-batch-job')
|
||||
machine = Default(String, 'n2-highmem-4')
|
||||
|
||||
job = Job(
|
||||
name='{{profile.job_name}}',
|
||||
project='{{profile.project}}',
|
||||
staging_bucket='{{profile.project}}',
|
||||
service_account='{{profile.user_name}}-shdw@twttr-dp-svc-accounts.iam.gserviceaccount.com',
|
||||
region='us-central1',
|
||||
worker_config=WorkerConfig(
|
||||
num_workers=2,
|
||||
worker_machine_type='{{profile.machine}}',
|
||||
worker_disk_type=WorkerDiskType('HDD'),
|
||||
),
|
||||
extra_args={
|
||||
"date": '{{profile.date}}'
|
||||
},
|
||||
service_identifier='twtr:svc:{{profile.user_name}}:{{profile.job_name}}:{{profile.environment}}:{{profile.cluster}}',
|
||||
deployment_config=BatchDeploymentConfig(
|
||||
role='{{profile.user_name}}',
|
||||
environment='prod',
|
||||
build_target='src/scala/com/twitter/simclusters_v2/scio/bq_generation/tweets_ann:iikf-tweets-ann-batch-job',
|
||||
gcp_deployment_credentials='/var/lib/tss/keys/{{profile.user_name}}/cloud/gcp/dp/shadow.json',
|
||||
statebird_config=StatebirdConfig(
|
||||
batch_width='PT4H',
|
||||
first_time='{{profile.date}}',
|
||||
),
|
||||
workflow_config=WorkflowConfig(
|
||||
play=True,
|
||||
),
|
||||
timeout='PT24H'
|
||||
)
|
||||
)
|
||||
|
||||
jobs=[job]
|
Binary file not shown.
@ -1,34 +0,0 @@
|
||||
class Profile(Struct):
|
||||
project = Required(String)
|
||||
date = Required(String)
|
||||
environment = Default(String, 'dev')
|
||||
job_name = Default(String, 'mts-consumer-embeddings-tweets-ann-adhoc-job')
|
||||
machine = Default(String, 'n2-highmem-4')
|
||||
|
||||
job = Job(
|
||||
name='{{profile.job_name}}',
|
||||
project='{{profile.project}}',
|
||||
staging_bucket='{{profile.project}}',
|
||||
service_account='{{profile.user_name}}-shdw@twttr-dp-svc-accounts.iam.gserviceaccount.com',
|
||||
region='us-central1',
|
||||
worker_config=WorkerConfig(
|
||||
num_workers=2,
|
||||
worker_machine_type='{{profile.machine}}',
|
||||
worker_disk_type=WorkerDiskType('HDD'),
|
||||
),
|
||||
extra_args={
|
||||
"date": '{{profile.date}}'
|
||||
},
|
||||
service_identifier='twtr:svc:{{profile.user_name}}:{{profile.job_name}}:{{profile.environment}}:{{profile.cluster}}',
|
||||
deployment_config=BatchDeploymentConfig(
|
||||
role='{{profile.user_name}}',
|
||||
build_target='src/scala/com/twitter/simclusters_v2/scio/bq_generation/tweets_ann:mts-consumer-embeddings-tweets-ann-adhoc-job',
|
||||
gcp_deployment_credentials='/var/lib/tss/keys/{{profile.user_name}}/cloud/gcp/dp/shadow.json',
|
||||
statebird_config=StatebirdConfig(
|
||||
batch_width='PT2H',
|
||||
first_time='{{profile.date}}',
|
||||
),
|
||||
)
|
||||
)
|
||||
|
||||
jobs=[job]
|
Binary file not shown.
@ -1,39 +0,0 @@
|
||||
class Profile(Struct):
|
||||
project = Required(String)
|
||||
date = Required(String)
|
||||
environment = Default(String, 'prod')
|
||||
job_name = Default(String, 'mts-consumer-embeddings-tweets-ann-batch-job')
|
||||
machine = Default(String, 'n2-highmem-4')
|
||||
|
||||
job = Job(
|
||||
name='{{profile.job_name}}',
|
||||
project='{{profile.project}}',
|
||||
staging_bucket='{{profile.project}}',
|
||||
service_account='{{profile.user_name}}-shdw@twttr-dp-svc-accounts.iam.gserviceaccount.com',
|
||||
region='us-central1',
|
||||
worker_config=WorkerConfig(
|
||||
num_workers=2,
|
||||
worker_machine_type='{{profile.machine}}',
|
||||
worker_disk_type=WorkerDiskType('HDD'),
|
||||
),
|
||||
extra_args={
|
||||
"date": '{{profile.date}}'
|
||||
},
|
||||
service_identifier='twtr:svc:{{profile.user_name}}:{{profile.job_name}}:{{profile.environment}}:{{profile.cluster}}',
|
||||
deployment_config=BatchDeploymentConfig(
|
||||
role='{{profile.user_name}}',
|
||||
environment='prod',
|
||||
build_target='src/scala/com/twitter/simclusters_v2/scio/bq_generation/tweets_ann:mts-consumer-embeddings-tweets-ann-batch-job',
|
||||
gcp_deployment_credentials='/var/lib/tss/keys/{{profile.user_name}}/cloud/gcp/dp/shadow.json',
|
||||
statebird_config=StatebirdConfig(
|
||||
batch_width='PT4H',
|
||||
first_time='{{profile.date}}',
|
||||
),
|
||||
workflow_config=WorkflowConfig(
|
||||
play=True,
|
||||
),
|
||||
timeout='PT24H'
|
||||
)
|
||||
)
|
||||
|
||||
jobs=[job]
|
Binary file not shown.
@ -1,21 +0,0 @@
|
||||
scala_library(
|
||||
sources = [
|
||||
"*.scala",
|
||||
],
|
||||
platform = "java8",
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"beam-internal/src/main/scala/com/twitter/beam/io/dal",
|
||||
"beam-internal/src/main/scala/com/twitter/scio_internal/runner/dataflow",
|
||||
"flockdb-tools/datasets/flock:flock-blocks-edges-scala",
|
||||
"flockdb-tools/datasets/flock:flock-follows-edges-scala",
|
||||
"flockdb-tools/datasets/flock:flock-report-as-abuse-edges-scala",
|
||||
"flockdb-tools/datasets/flock:flock-report-as-spam-edges-scala",
|
||||
"iesource/processing/events/src/main/scala/com/twitter/iesource/processing/events/batch:server_engagements-scala",
|
||||
"src/scala/com/twitter/simclusters_v2/scalding",
|
||||
"src/thrift/com/twitter/twadoop/user/gen:gen-scala",
|
||||
"tweetsource/public_tweets/src/main/scala/com/twitter/tweetsource/public_tweets:public_tweets-scala",
|
||||
"usersource/snapshot/src/main/scala/com/twitter/usersource/snapshot/flat:usersource_flat-scala",
|
||||
"usersource/snapshot/src/main/thrift/com/twitter/usersource/snapshot/flat:flat-scala",
|
||||
],
|
||||
)
|
BIN
src/scala/com/twitter/simclusters_v2/scio/common/BUILD.docx
Normal file
BIN
src/scala/com/twitter/simclusters_v2/scio/common/BUILD.docx
Normal file
Binary file not shown.
Binary file not shown.
@ -1,301 +0,0 @@
|
||||
package com.twitter.simclusters_v2.scio.common
|
||||
|
||||
import com.spotify.scio.ScioContext
|
||||
import com.spotify.scio.values.SCollection
|
||||
import com.twitter.beam.io.dal.DAL
|
||||
import com.twitter.common.util.Clock
|
||||
import com.twitter.common_header.thriftscala.CommonHeader
|
||||
import com.twitter.common_header.thriftscala.IdType
|
||||
import com.twitter.common_header.thriftscala.VersionedCommonHeader
|
||||
import com.twitter.frigate.data_pipeline.magicrecs.magicrecs_notifications_lite.thriftscala.MagicRecsNotificationLite
|
||||
import com.twitter.frigate.data_pipeline.scalding.magicrecs.magicrecs_notification_lite.MagicrecsNotificationLite1DayLagScalaDataset
|
||||
import com.twitter.iesource.thriftscala.InteractionEvent
|
||||
import com.twitter.iesource.thriftscala.InteractionTargetType
|
||||
import com.twitter.interests_ds.jobs.interests_service.UserTopicRelationSnapshotScalaDataset
|
||||
import com.twitter.interests.thriftscala.InterestRelationType
|
||||
import com.twitter.interests.thriftscala.UserInterestsRelationSnapshot
|
||||
import com.twitter.penguin.scalding.datasets.PenguinUserLanguagesScalaDataset
|
||||
import com.twitter.search.adaptive.scribing.thriftscala.AdaptiveSearchScribeLog
|
||||
import com.twitter.simclusters_v2.hdfs_sources.UserUserFavGraphScalaDataset
|
||||
import com.twitter.simclusters_v2.scalding.embedding.common.ExternalDataSources.ValidFlockEdgeStateId
|
||||
import com.twitter.simclusters_v2.scalding.embedding.common.ExternalDataSources.getStandardLanguageCode
|
||||
import com.twitter.twadoop.user.gen.thriftscala.CombinedUser
|
||||
import flockdb_tools.datasets.flock.FlockBlocksEdgesScalaDataset
|
||||
import flockdb_tools.datasets.flock.FlockFollowsEdgesScalaDataset
|
||||
import flockdb_tools.datasets.flock.FlockReportAsAbuseEdgesScalaDataset
|
||||
import flockdb_tools.datasets.flock.FlockReportAsSpamEdgesScalaDataset
|
||||
import org.joda.time.Interval
|
||||
import com.twitter.simclusters_v2.thriftscala.EdgeWithDecayedWeights
|
||||
import com.twitter.usersource.snapshot.combined.UsersourceScalaDataset
|
||||
import com.twitter.usersource.snapshot.flat.UsersourceFlatScalaDataset
|
||||
import com.twitter.util.Duration
|
||||
import twadoop_config.configuration.log_categories.group.search.AdaptiveSearchScalaDataset
|
||||
|
||||
object ExternalDataSources {
|
||||
def userSource(
|
||||
noOlderThan: Duration = Duration.fromDays(7)
|
||||
)(
|
||||
implicit sc: ScioContext
|
||||
): SCollection[CombinedUser] = {
|
||||
sc.customInput(
|
||||
"ReadUserSource",
|
||||
DAL
|
||||
.readMostRecentSnapshotNoOlderThan(
|
||||
UsersourceScalaDataset,
|
||||
noOlderThan,
|
||||
Clock.SYSTEM_CLOCK,
|
||||
DAL.Environment.Prod
|
||||
)
|
||||
)
|
||||
}
|
||||
|
||||
def userCountrySource(
|
||||
noOlderThan: Duration = Duration.fromDays(7)
|
||||
)(
|
||||
implicit sc: ScioContext
|
||||
): SCollection[(Long, String)] = {
|
||||
sc.customInput(
|
||||
"ReadUserCountrySource",
|
||||
DAL
|
||||
.readMostRecentSnapshotNoOlderThan(
|
||||
UsersourceFlatScalaDataset,
|
||||
noOlderThan,
|
||||
Clock.SYSTEM_CLOCK,
|
||||
DAL.Environment.Prod,
|
||||
)
|
||||
).flatMap { flatUser =>
|
||||
for {
|
||||
userId <- flatUser.id
|
||||
country <- flatUser.accountCountryCode
|
||||
} yield {
|
||||
(userId, country.toUpperCase)
|
||||
}
|
||||
}.distinct
|
||||
}
|
||||
|
||||
def userUserFavSource(
|
||||
noOlderThan: Duration = Duration.fromDays(14)
|
||||
)(
|
||||
implicit sc: ScioContext
|
||||
): SCollection[EdgeWithDecayedWeights] = {
|
||||
sc.customInput(
|
||||
"ReadUserUserFavSource",
|
||||
DAL
|
||||
.readMostRecentSnapshotNoOlderThan(
|
||||
UserUserFavGraphScalaDataset,
|
||||
noOlderThan,
|
||||
Clock.SYSTEM_CLOCK,
|
||||
DAL.Environment.Prod
|
||||
)
|
||||
)
|
||||
}
|
||||
|
||||
def inferredUserConsumedLanguageSource(
|
||||
noOlderThan: Duration = Duration.fromDays(7)
|
||||
)(
|
||||
implicit sc: ScioContext
|
||||
): SCollection[(Long, Seq[(String, Double)])] = {
|
||||
sc.customInput(
|
||||
"ReadInferredUserConsumedLanguageSource",
|
||||
DAL
|
||||
.readMostRecentSnapshotNoOlderThan(
|
||||
PenguinUserLanguagesScalaDataset,
|
||||
noOlderThan,
|
||||
Clock.SYSTEM_CLOCK,
|
||||
DAL.Environment.Prod
|
||||
)
|
||||
).map { kv =>
|
||||
val consumed = kv.value.consumed
|
||||
.collect {
|
||||
case scoredString if scoredString.weight > 0.001 => //throw away 5% outliers
|
||||
(getStandardLanguageCode(scoredString.item), scoredString.weight)
|
||||
}.collect {
|
||||
case (Some(language), score) => (language, score)
|
||||
}
|
||||
(kv.key, consumed)
|
||||
}
|
||||
}
|
||||
|
||||
def flockBlockSource(
|
||||
noOlderThan: Duration = Duration.fromDays(7)
|
||||
)(
|
||||
implicit sc: ScioContext
|
||||
): SCollection[(Long, Long)] = {
|
||||
sc.customInput(
|
||||
"ReadFlockBlock",
|
||||
DAL.readMostRecentSnapshotNoOlderThan(
|
||||
FlockBlocksEdgesScalaDataset,
|
||||
noOlderThan,
|
||||
Clock.SYSTEM_CLOCK,
|
||||
DAL.Environment.Prod))
|
||||
.collect {
|
||||
case edge if edge.state == ValidFlockEdgeStateId =>
|
||||
(edge.sourceId, edge.destinationId)
|
||||
}
|
||||
}
|
||||
|
||||
def flockFollowSource(
|
||||
noOlderThan: Duration = Duration.fromDays(7)
|
||||
)(
|
||||
implicit sc: ScioContext
|
||||
): SCollection[(Long, Long)] = {
|
||||
sc.customInput(
|
||||
"ReadFlockFollow",
|
||||
DAL
|
||||
.readMostRecentSnapshotNoOlderThan(
|
||||
FlockFollowsEdgesScalaDataset,
|
||||
noOlderThan,
|
||||
Clock.SYSTEM_CLOCK,
|
||||
DAL.Environment.Prod))
|
||||
.collect {
|
||||
case edge if edge.state == ValidFlockEdgeStateId =>
|
||||
(edge.sourceId, edge.destinationId)
|
||||
}
|
||||
}
|
||||
|
||||
def flockReportAsAbuseSource(
|
||||
noOlderThan: Duration = Duration.fromDays(7)
|
||||
)(
|
||||
implicit sc: ScioContext
|
||||
): SCollection[(Long, Long)] = {
|
||||
sc.customInput(
|
||||
"ReadFlockReportAsAbuseJava",
|
||||
DAL
|
||||
.readMostRecentSnapshotNoOlderThan(
|
||||
FlockReportAsAbuseEdgesScalaDataset,
|
||||
noOlderThan,
|
||||
Clock.SYSTEM_CLOCK,
|
||||
DAL.Environment.Prod)
|
||||
)
|
||||
.collect {
|
||||
case edge if edge.state == ValidFlockEdgeStateId =>
|
||||
(edge.sourceId, edge.destinationId)
|
||||
}
|
||||
}
|
||||
|
||||
def flockReportAsSpamSource(
|
||||
noOlderThan: Duration = Duration.fromDays(7)
|
||||
)(
|
||||
implicit sc: ScioContext
|
||||
): SCollection[(Long, Long)] = {
|
||||
sc.customInput(
|
||||
"ReadFlockReportAsSpam",
|
||||
DAL
|
||||
.readMostRecentSnapshotNoOlderThan(
|
||||
FlockReportAsSpamEdgesScalaDataset,
|
||||
noOlderThan,
|
||||
Clock.SYSTEM_CLOCK,
|
||||
DAL.Environment.Prod))
|
||||
.collect {
|
||||
case edge if edge.state == ValidFlockEdgeStateId =>
|
||||
(edge.sourceId, edge.destinationId)
|
||||
}
|
||||
}
|
||||
|
||||
def ieSourceTweetEngagementsSource(
|
||||
interval: Interval
|
||||
)(
|
||||
implicit sc: ScioContext
|
||||
): SCollection[InteractionEvent] = {
|
||||
sc.customInput(
|
||||
"ReadIeSourceTweetEngagementsSource",
|
||||
DAL
|
||||
.read(
|
||||
com.twitter.iesource.processing.events.batch.ServerEngagementsScalaDataset,
|
||||
interval,
|
||||
DAL.Environment.Prod,
|
||||
)
|
||||
).filter { event =>
|
||||
// filter out logged out users because their favorites are less reliable
|
||||
event.engagingUserId > 0L && event.targetType == InteractionTargetType.Tweet
|
||||
}
|
||||
}
|
||||
|
||||
def topicFollowGraphSource(
|
||||
noOlderThan: Duration = Duration.fromDays(7)
|
||||
)(
|
||||
implicit sc: ScioContext
|
||||
): SCollection[(Long, Long)] = {
|
||||
// The implementation here is slightly different than the topicFollowGraphSource function in
|
||||
// src/scala/com/twitter/simclusters_v2/scalding/embedding/common/ExternalDataSources.scala
|
||||
// We don't do an additional hashJoin on uttFollowableEntitiesSource.
|
||||
sc.customInput(
|
||||
"ReadTopicFollowGraphSource",
|
||||
DAL
|
||||
.readMostRecentSnapshotNoOlderThan(
|
||||
UserTopicRelationSnapshotScalaDataset,
|
||||
noOlderThan,
|
||||
Clock.SYSTEM_CLOCK,
|
||||
DAL.Environment.Prod
|
||||
)
|
||||
).collect {
|
||||
case userInterestsRelationSnapshot: UserInterestsRelationSnapshot
|
||||
if userInterestsRelationSnapshot.interestType == "UTT" &&
|
||||
userInterestsRelationSnapshot.relation == InterestRelationType.Followed =>
|
||||
(userInterestsRelationSnapshot.interestId, userInterestsRelationSnapshot.userId)
|
||||
}
|
||||
}
|
||||
|
||||
def magicRecsNotficationOpenOrClickEventsSource(
|
||||
interval: Interval
|
||||
)(
|
||||
implicit sc: ScioContext
|
||||
): SCollection[MagicRecsNotificationLite] = {
|
||||
sc.customInput(
|
||||
"ReadMagicRecsNotficationOpenOrClickEventsSource",
|
||||
DAL
|
||||
.read(MagicrecsNotificationLite1DayLagScalaDataset, interval, DAL.Environment.Prod))
|
||||
.filter { entry =>
|
||||
// keep entries with a valid userId and tweetId, opened or clicked timestamp defined
|
||||
val userIdExists = entry.targetUserId.isDefined
|
||||
val tweetIdExists = entry.tweetId.isDefined
|
||||
val openOrClickExists =
|
||||
entry.openTimestampMs.isDefined || entry.ntabClickTimestampMs.isDefined
|
||||
userIdExists && tweetIdExists && openOrClickExists
|
||||
}
|
||||
}
|
||||
|
||||
def adaptiveSearchScribeLogsSource(
|
||||
interval: Interval
|
||||
)(
|
||||
implicit sc: ScioContext
|
||||
): SCollection[(Long, String)] = {
|
||||
sc.customInput(
|
||||
"ReadAdaptiveSearchScribeLogsSource",
|
||||
DAL
|
||||
.read(AdaptiveSearchScalaDataset, interval, DAL.Environment.Prod))
|
||||
.flatMap({ scribeLog: AdaptiveSearchScribeLog =>
|
||||
for {
|
||||
userId <- userIdFromBlenderAdaptiveScribeLog(scribeLog)
|
||||
// filter out logged out search queries
|
||||
if userId != 0
|
||||
queryString <- scribeLog.requestLog.flatMap(_.request).flatMap(_.rawQuery)
|
||||
} yield {
|
||||
(userId, Set(queryString))
|
||||
}
|
||||
})
|
||||
// if a user searches for the same query multiple times, there could be duplicates.
|
||||
// De-dup them to get the distinct queries searched by a user
|
||||
.sumByKey
|
||||
.flatMap {
|
||||
case (userId, distinctQuerySet) =>
|
||||
distinctQuerySet.map { query =>
|
||||
(userId, query)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private def userIdFromBlenderAdaptiveScribeLog(
|
||||
blenderAdaptiveLog: AdaptiveSearchScribeLog
|
||||
): Option[Long] = {
|
||||
blenderAdaptiveLog.versionedCommonHeader match {
|
||||
case VersionedCommonHeader.CommonHeader(CommonHeader.ServerHeader(serverHeader)) =>
|
||||
serverHeader.requestInfo match {
|
||||
case Some(requestInfo) => requestInfo.ids.get(IdType.UserId).map(_.toLong)
|
||||
case _ => None
|
||||
}
|
||||
case _ => None
|
||||
}
|
||||
}
|
||||
|
||||
}
|
Binary file not shown.
@ -1,39 +0,0 @@
|
||||
package com.twitter.simclusters_v2.scio.multi_type_graph.assemble_multi_type_graph
|
||||
|
||||
/**
|
||||
Build:
|
||||
./bazel bundle src/scala/com/twitter/simclusters_v2/scio/multi_type_graph/assemble_multi_type_graph:assemble-multi-type-graph-scio-adhoc-app
|
||||
|
||||
To kick off an adhoc run:
|
||||
bin/d6w create \
|
||||
${GCP_PROJECT_NAME}/us-central1/assemble-multi-type-graph-scio-adhoc-app \
|
||||
src/scala/com/twitter/simclusters_v2/scio/multi_type_graph/assemble_multi_type_graph/assemble-multi-type-graph-scio-adhoc.d6w \
|
||||
--jar dist/assemble-multi-type-graph-scio-adhoc-app.jar \
|
||||
--bind=profile.project=${GCP_PROJECT_NAME} \
|
||||
--bind=profile.user_name=${USER} \
|
||||
--bind=profile.date="2021-11-04" \
|
||||
--bind=profile.machine="n2-highmem-16"
|
||||
*/
|
||||
|
||||
object AssembleMultiTypeGraphScioAdhocApp extends AssembleMultiTypeGraphScioBaseApp {
|
||||
override val isAdhoc: Boolean = true
|
||||
override val rootMHPath: String = Config.AdhocRootPath
|
||||
override val rootThriftPath: String = Config.AdhocRootPath
|
||||
}
|
||||
|
||||
/**
|
||||
To deploy the job:
|
||||
|
||||
bin/d6w schedule \
|
||||
${GCP_PROJECT_NAME}/us-central1/assemble-multi-type-graph-scio-batch-app \
|
||||
src/scala/com/twitter/simclusters_v2/scio/multi_type_graph/assemble_multi_type_graph/assemble-multi-type-graph-scio-batch.d6w \
|
||||
--bind=profile.project=${GCP_PROJECT_NAME} \
|
||||
--bind=profile.user_name=recos-platform \
|
||||
--bind=profile.date="2021-11-04" \
|
||||
--bind=profile.machine="n2-highmem-16"
|
||||
*/
|
||||
object AssembleMultiTypeGraphScioBatchApp extends AssembleMultiTypeGraphScioBaseApp {
|
||||
override val isAdhoc: Boolean = false
|
||||
override val rootMHPath: String = Config.RootMHPath
|
||||
override val rootThriftPath: String = Config.RootThriftPath
|
||||
}
|
Binary file not shown.
@ -1,574 +0,0 @@
|
||||
package com.twitter.simclusters_v2.scio.multi_type_graph.assemble_multi_type_graph
|
||||
|
||||
import com.spotify.scio.ScioContext
|
||||
import com.spotify.scio.coders.Coder
|
||||
import com.spotify.scio.values.SCollection
|
||||
import com.twitter.beam.io.dal.DAL
|
||||
import com.twitter.beam.io.fs.multiformat.DiskFormat
|
||||
import com.twitter.beam.io.fs.multiformat.PathLayout
|
||||
import com.twitter.beam.job.DateRangeOptions
|
||||
import com.twitter.dal.client.dataset.KeyValDALDataset
|
||||
import com.twitter.dal.client.dataset.SnapshotDALDataset
|
||||
import com.twitter.frigate.data_pipeline.magicrecs.magicrecs_notifications_lite.thriftscala.MagicRecsNotificationLite
|
||||
import com.twitter.iesource.thriftscala.InteractionEvent
|
||||
import com.twitter.iesource.thriftscala.InteractionType
|
||||
import com.twitter.iesource.thriftscala.ReferenceTweet
|
||||
import com.twitter.scalding_internal.multiformat.format.keyval.KeyVal
|
||||
import com.twitter.scio_internal.coders.ThriftStructLazyBinaryScroogeCoder
|
||||
import com.twitter.scio_internal.job.ScioBeamJob
|
||||
import com.twitter.scrooge.ThriftStruct
|
||||
import com.twitter.simclusters_v2.common.Country
|
||||
import com.twitter.simclusters_v2.common.Language
|
||||
import com.twitter.simclusters_v2.common.TopicId
|
||||
import com.twitter.simclusters_v2.common.TweetId
|
||||
import com.twitter.simclusters_v2.common.UserId
|
||||
import com.twitter.simclusters_v2.hdfs_sources.MultiTypeGraphForTopKRightNodesThriftScioScalaDataset
|
||||
import com.twitter.simclusters_v2.hdfs_sources.TopKRightNounsScioScalaDataset
|
||||
import com.twitter.simclusters_v2.hdfs_sources.TruncatedMultiTypeGraphScioScalaDataset
|
||||
import com.twitter.simclusters_v2.scio.common.ExternalDataSources
|
||||
import com.twitter.simclusters_v2.scio.multi_type_graph.assemble_multi_type_graph.Config.GlobalDefaultMinFrequencyOfRightNodeType
|
||||
import com.twitter.simclusters_v2.scio.multi_type_graph.assemble_multi_type_graph.Config.HalfLifeInDaysForFavScore
|
||||
import com.twitter.simclusters_v2.scio.multi_type_graph.assemble_multi_type_graph.Config.NumTopNounsForUnknownRightNodeType
|
||||
import com.twitter.simclusters_v2.scio.multi_type_graph.assemble_multi_type_graph.Config.SampledEmployeeIds
|
||||
import com.twitter.simclusters_v2.scio.multi_type_graph.assemble_multi_type_graph.Config.TopKConfig
|
||||
import com.twitter.simclusters_v2.scio.multi_type_graph.assemble_multi_type_graph.Config.TopKRightNounsForMHDump
|
||||
import com.twitter.simclusters_v2.scio.multi_type_graph.common.MultiTypeGraphUtil
|
||||
import com.twitter.simclusters_v2.thriftscala.EdgeWithDecayedWeights
|
||||
import com.twitter.simclusters_v2.thriftscala.LeftNode
|
||||
import com.twitter.simclusters_v2.thriftscala.MultiTypeGraphEdge
|
||||
import com.twitter.simclusters_v2.thriftscala.Noun
|
||||
import com.twitter.simclusters_v2.thriftscala.NounWithFrequency
|
||||
import com.twitter.simclusters_v2.thriftscala.NounWithFrequencyList
|
||||
import com.twitter.simclusters_v2.thriftscala.RightNode
|
||||
import com.twitter.simclusters_v2.thriftscala.RightNodeType
|
||||
import com.twitter.simclusters_v2.thriftscala.RightNodeTypeStruct
|
||||
import com.twitter.simclusters_v2.thriftscala.RightNodeWithEdgeWeight
|
||||
import com.twitter.simclusters_v2.thriftscala.RightNodeWithEdgeWeightList
|
||||
import com.twitter.twadoop.user.gen.thriftscala.CombinedUser
|
||||
import com.twitter.util.Duration
|
||||
import java.time.Instant
|
||||
import org.joda.time.Interval
|
||||
|
||||
/**
|
||||
* Scio version of
|
||||
* src/scala/com/twitter/simclusters_v2/scalding/multi_type_graph/assemble_multi_type_graph/AssembleMultiTypeGraph.scala
|
||||
*/
|
||||
trait AssembleMultiTypeGraphScioBaseApp extends ScioBeamJob[DateRangeOptions] {
|
||||
// Provides an implicit binary thrift scrooge coder by default.
|
||||
override implicit def scroogeCoder[T <: ThriftStruct: Manifest]: Coder[T] =
|
||||
ThriftStructLazyBinaryScroogeCoder.scroogeCoder
|
||||
|
||||
val isAdhoc: Boolean
|
||||
val rootMHPath: String
|
||||
val rootThriftPath: String
|
||||
|
||||
val truncatedMultiTypeGraphMHOutputDir: String =
|
||||
Config.truncatedMultiTypeGraphMHOutputDir
|
||||
val truncatedMultiTypeGraphThriftOutputDir: String =
|
||||
Config.truncatedMultiTypeGraphThriftOutputDir
|
||||
val topKRightNounsMHOutputDir: String = Config.topKRightNounsMHOutputDir
|
||||
val topKRightNounsOutputDir: String = Config.topKRightNounsOutputDir
|
||||
|
||||
val fullMultiTypeGraphThriftOutputDir: String =
|
||||
Config.fullMultiTypeGraphThriftOutputDir
|
||||
val truncatedMultiTypeGraphKeyValDataset: KeyValDALDataset[
|
||||
KeyVal[LeftNode, RightNodeWithEdgeWeightList]
|
||||
] = TruncatedMultiTypeGraphScioScalaDataset
|
||||
val topKRightNounsKeyValDataset: KeyValDALDataset[
|
||||
KeyVal[RightNodeTypeStruct, NounWithFrequencyList]
|
||||
] = TopKRightNounsScioScalaDataset
|
||||
val topKRightNounsMHKeyValDataset: KeyValDALDataset[
|
||||
KeyVal[RightNodeTypeStruct, NounWithFrequencyList]
|
||||
] = TopKRightNounsMhScioScalaDataset
|
||||
val fullMultiTypeGraphSnapshotDataset: SnapshotDALDataset[MultiTypeGraphEdge] =
|
||||
FullMultiTypeGraphScioScalaDataset
|
||||
val multiTypeGraphTopKForRightNodesSnapshotDataset: SnapshotDALDataset[
|
||||
MultiTypeGraphEdge
|
||||
] =
|
||||
MultiTypeGraphForTopKRightNodesThriftScioScalaDataset
|
||||
|
||||
def getValidUsers(
|
||||
input: SCollection[CombinedUser]
|
||||
): SCollection[UserId] = {
|
||||
input
|
||||
.flatMap { u =>
|
||||
for {
|
||||
user <- u.user
|
||||
if user.id != 0
|
||||
safety <- user.safety
|
||||
if !(safety.suspended || safety.deactivated)
|
||||
} yield {
|
||||
user.id
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
def filterInvalidUsers(
|
||||
flockEdges: SCollection[(UserId, UserId)],
|
||||
validUsers: SCollection[UserId]
|
||||
): SCollection[(UserId, UserId)] = {
|
||||
val validUsersWithValues = validUsers.map(userId => (userId, ()))
|
||||
flockEdges
|
||||
.join(validUsersWithValues)
|
||||
.map {
|
||||
case (srcId, (destId, _)) =>
|
||||
(destId, srcId)
|
||||
}
|
||||
.join(validUsersWithValues)
|
||||
.map {
|
||||
case (destId, (srcId, _)) =>
|
||||
(srcId, destId)
|
||||
}
|
||||
}
|
||||
|
||||
def getFavEdges(
|
||||
input: SCollection[EdgeWithDecayedWeights],
|
||||
halfLifeInDaysForFavScore: Int,
|
||||
): SCollection[(Long, Long, Double)] = {
|
||||
input
|
||||
.flatMap { edge =>
|
||||
if (edge.weights.halfLifeInDaysToDecayedSums.contains(halfLifeInDaysForFavScore)) {
|
||||
Some(
|
||||
(
|
||||
edge.sourceId,
|
||||
edge.destinationId,
|
||||
edge.weights.halfLifeInDaysToDecayedSums(halfLifeInDaysForFavScore)))
|
||||
} else {
|
||||
None
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
def leftRightTuple(
|
||||
leftNodeUserId: UserId,
|
||||
rightNodeType: RightNodeType,
|
||||
rightNoun: Noun,
|
||||
weight: Double = 1.0
|
||||
): (LeftNode, RightNodeWithEdgeWeight) = {
|
||||
(
|
||||
LeftNode.UserId(leftNodeUserId),
|
||||
RightNodeWithEdgeWeight(
|
||||
rightNode = RightNode(rightNodeType = rightNodeType, noun = rightNoun),
|
||||
weight = weight))
|
||||
}
|
||||
|
||||
def getUserFavGraph(
|
||||
userUserFavEdges: SCollection[(UserId, UserId, Double)]
|
||||
): SCollection[(LeftNode, RightNodeWithEdgeWeight)] = {
|
||||
userUserFavEdges.map {
|
||||
case (srcId, destId, edgeWt) =>
|
||||
leftRightTuple(srcId, RightNodeType.FavUser, Noun.UserId(destId), edgeWt)
|
||||
}
|
||||
}
|
||||
|
||||
def getUserFollowGraph(
|
||||
userUserFollowEdges: SCollection[(UserId, UserId)]
|
||||
): SCollection[(LeftNode, RightNodeWithEdgeWeight)] = {
|
||||
userUserFollowEdges.map {
|
||||
case (srcId, destId) =>
|
||||
leftRightTuple(srcId, RightNodeType.FollowUser, Noun.UserId(destId), 1.0)
|
||||
}
|
||||
}
|
||||
|
||||
def getUserBlockGraph(
|
||||
userUserBlockEdges: SCollection[(UserId, UserId)]
|
||||
): SCollection[(LeftNode, RightNodeWithEdgeWeight)] = {
|
||||
userUserBlockEdges.map {
|
||||
case (srcId, destId) =>
|
||||
leftRightTuple(srcId, RightNodeType.BlockUser, Noun.UserId(destId), 1.0)
|
||||
}
|
||||
}
|
||||
|
||||
def getUserAbuseReportGraph(
|
||||
userUserAbuseReportEdges: SCollection[(UserId, UserId)]
|
||||
): SCollection[(LeftNode, RightNodeWithEdgeWeight)] = {
|
||||
userUserAbuseReportEdges.map {
|
||||
case (srcId, destId) =>
|
||||
leftRightTuple(srcId, RightNodeType.AbuseReportUser, Noun.UserId(destId), 1.0)
|
||||
}
|
||||
}
|
||||
|
||||
def getUserSpamReportGraph(
|
||||
userUserSpamReportEdges: SCollection[(UserId, UserId)]
|
||||
): SCollection[(LeftNode, RightNodeWithEdgeWeight)] = {
|
||||
userUserSpamReportEdges.map {
|
||||
case (srcId, destId) =>
|
||||
leftRightTuple(srcId, RightNodeType.SpamReportUser, Noun.UserId(destId), 1.0)
|
||||
}
|
||||
}
|
||||
|
||||
def getUserTopicFollowGraph(
|
||||
topicUserFollowedByEdges: SCollection[(TopicId, UserId)]
|
||||
): SCollection[(LeftNode, RightNodeWithEdgeWeight)] = {
|
||||
topicUserFollowedByEdges.map {
|
||||
case (topicId, userId) =>
|
||||
leftRightTuple(userId, RightNodeType.FollowTopic, Noun.TopicId(topicId), 1.0)
|
||||
}
|
||||
}
|
||||
|
||||
def getUserSignUpCountryGraph(
|
||||
userSignUpCountryEdges: SCollection[(UserId, Country)]
|
||||
): SCollection[(LeftNode, RightNodeWithEdgeWeight)] = {
|
||||
userSignUpCountryEdges.map {
|
||||
case (userId, country) =>
|
||||
leftRightTuple(userId, RightNodeType.SignUpCountry, Noun.Country(country), 1.0)
|
||||
}
|
||||
}
|
||||
|
||||
def getMagicRecsNotifOpenOrClickTweetsGraph(
|
||||
userMRNotifOpenOrClickEvents: SCollection[MagicRecsNotificationLite]
|
||||
): SCollection[(LeftNode, RightNodeWithEdgeWeight)] = {
|
||||
userMRNotifOpenOrClickEvents.flatMap { entry =>
|
||||
for {
|
||||
userId <- entry.targetUserId
|
||||
tweetId <- entry.tweetId
|
||||
} yield {
|
||||
leftRightTuple(userId, RightNodeType.NotifOpenOrClickTweet, Noun.TweetId(tweetId), 1.0)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
def getUserConsumedLanguagesGraph(
|
||||
userConsumedLanguageEdges: SCollection[(UserId, Seq[(Language, Double)])]
|
||||
): SCollection[(LeftNode, RightNodeWithEdgeWeight)] = {
|
||||
userConsumedLanguageEdges.flatMap {
|
||||
case (userId, langWithWeights) =>
|
||||
langWithWeights.map {
|
||||
case (lang, weight) =>
|
||||
leftRightTuple(userId, RightNodeType.ConsumedLanguage, Noun.Language(lang), 1.0)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
def getSearchGraph(
|
||||
userSearchQueryEdges: SCollection[(UserId, String)]
|
||||
): SCollection[(LeftNode, RightNodeWithEdgeWeight)] = {
|
||||
userSearchQueryEdges.map {
|
||||
case (userId, query) =>
|
||||
leftRightTuple(userId, RightNodeType.SearchQuery, Noun.Query(query), 1.0)
|
||||
}
|
||||
}
|
||||
|
||||
def getUserTweetInteractionGraph(
|
||||
tweetInteractionEvents: SCollection[InteractionEvent],
|
||||
): SCollection[(LeftNode, RightNodeWithEdgeWeight)] = {
|
||||
val userTweetInteractionsByType: SCollection[((UserId, TweetId), RightNodeType)] =
|
||||
tweetInteractionEvents
|
||||
.flatMap { event =>
|
||||
val referenceTweet: Option[ReferenceTweet] = event.referenceTweet
|
||||
val targetId: Long = event.targetId
|
||||
val userId: Long = event.engagingUserId
|
||||
|
||||
// To find the id of the tweet that was interacted with
|
||||
// For likes, this is the targetId; for retweet or reply, it is the referenceTweet's id
|
||||
// One thing to note is that for likes, referenceTweet is empty
|
||||
val (tweetIdOpt, rightNodeTypeOpt) = {
|
||||
event.interactionType match {
|
||||
case Some(InteractionType.Favorite) =>
|
||||
// Only allow favorites on original tweets, not retweets, to avoid double-counting
|
||||
// because we have retweet-type tweets in the data source as well
|
||||
(
|
||||
if (referenceTweet.isEmpty) {
|
||||
Some(targetId)
|
||||
} else None,
|
||||
Some(RightNodeType.FavTweet))
|
||||
case Some(InteractionType.Reply) =>
|
||||
(referenceTweet.map(_.tweetId), Some(RightNodeType.ReplyTweet))
|
||||
case Some(InteractionType.Retweet) =>
|
||||
(referenceTweet.map(_.tweetId), Some(RightNodeType.RetweetTweet))
|
||||
case _ => (None, None)
|
||||
}
|
||||
}
|
||||
for {
|
||||
tweetId <- tweetIdOpt
|
||||
rightNodeType <- rightNodeTypeOpt
|
||||
} yield {
|
||||
((userId, tweetId), rightNodeType)
|
||||
}
|
||||
}
|
||||
|
||||
userTweetInteractionsByType
|
||||
.mapValues(Set(_))
|
||||
.sumByKey
|
||||
.flatMap {
|
||||
case ((userId, tweetId), rightNodeTypeSet) =>
|
||||
rightNodeTypeSet.map { rightNodeType =>
|
||||
leftRightTuple(userId, rightNodeType, Noun.TweetId(tweetId), 1.0)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
def getTopKRightNounsWithFrequencies(
|
||||
fullGraph: SCollection[(LeftNode, RightNodeWithEdgeWeight)],
|
||||
topKConfig: Map[RightNodeType, Int],
|
||||
minFrequency: Int,
|
||||
): SCollection[(RightNodeType, Seq[(Noun, Double)])] = {
|
||||
val maxAcrossRightNounType: Int = topKConfig.valuesIterator.max
|
||||
|
||||
fullGraph
|
||||
.map {
|
||||
case (leftNode, rightNodeWithWeight) =>
|
||||
(rightNodeWithWeight.rightNode, 1.0)
|
||||
}
|
||||
.sumByKey
|
||||
.filter(_._2 >= minFrequency)
|
||||
.map {
|
||||
case (rightNode, freq) =>
|
||||
(rightNode.rightNodeType, (rightNode.noun, freq))
|
||||
}
|
||||
.topByKey(maxAcrossRightNounType)(Ordering.by(_._2))
|
||||
.map {
|
||||
case (rightNodeType, nounsListWithFreq) =>
|
||||
val truncatedList = nounsListWithFreq.toSeq
|
||||
.sortBy(-_._2)
|
||||
.take(topKConfig.getOrElse(rightNodeType, NumTopNounsForUnknownRightNodeType))
|
||||
(rightNodeType, truncatedList)
|
||||
}
|
||||
}
|
||||
|
||||
def getTruncatedGraph(
|
||||
fullGraph: SCollection[(LeftNode, RightNodeWithEdgeWeight)],
|
||||
topKWithFrequency: SCollection[(RightNodeType, Seq[(Noun, Double)])]
|
||||
): SCollection[(LeftNode, RightNodeWithEdgeWeight)] = {
|
||||
val topNouns = topKWithFrequency
|
||||
.flatMap {
|
||||
case (rightNodeType, nounsList) =>
|
||||
nounsList
|
||||
.map {
|
||||
case (nounVal, aggregatedFrequency) =>
|
||||
RightNode(rightNodeType, nounVal)
|
||||
}
|
||||
}.map(nouns => (nouns, ()))
|
||||
|
||||
fullGraph
|
||||
.map {
|
||||
case (leftNode, rightNodeWithWeight) =>
|
||||
(rightNodeWithWeight.rightNode, (leftNode, rightNodeWithWeight))
|
||||
}
|
||||
.hashJoin(topNouns)
|
||||
.map {
|
||||
case (rightNode, ((left, rightNodeWithWeight), _)) =>
|
||||
(left, rightNodeWithWeight)
|
||||
}
|
||||
}
|
||||
|
||||
def buildEmployeeGraph(
|
||||
graph: SCollection[(LeftNode, RightNodeWithEdgeWeight)]
|
||||
): SCollection[(LeftNode, RightNodeWithEdgeWeight)] = {
|
||||
val employeeIds = SampledEmployeeIds
|
||||
graph
|
||||
.collect {
|
||||
case (LeftNode.UserId(userId), rightNodeWithWeight) if employeeIds.contains(userId) =>
|
||||
(LeftNode.UserId(userId), rightNodeWithWeight)
|
||||
}
|
||||
}
|
||||
|
||||
override def configurePipeline(sc: ScioContext, opts: DateRangeOptions): Unit = {
|
||||
// Define the implicit ScioContext to read datasets from ExternalDataSources
|
||||
implicit def scioContext: ScioContext = sc
|
||||
|
||||
// DAL.Environment variable for WriteExecs
|
||||
val dalEnv = if (isAdhoc) DAL.Environment.Dev else DAL.Environment.Prod
|
||||
|
||||
// Define date intervals
|
||||
val interval_7days =
|
||||
new Interval(opts.interval.getEnd.minusWeeks(1), opts.interval.getEnd.minusMillis(1))
|
||||
val interval_14days =
|
||||
new Interval(opts.interval.getEnd.minusWeeks(2), opts.interval.getEnd.minusMillis(1))
|
||||
|
||||
/*
|
||||
* Dataset read operations
|
||||
*/
|
||||
// Get list of valid UserIds - to filter out deactivated or suspended user accounts
|
||||
val validUsers = getValidUsers(ExternalDataSources.userSource(Duration.fromDays(7)))
|
||||
|
||||
// ieSource tweet engagements data for tweet favs, replies, retweets - from last 14 days
|
||||
val tweetSource = ExternalDataSources.ieSourceTweetEngagementsSource(interval_14days)
|
||||
|
||||
// Read TFlock datasets
|
||||
val flockFollowSource = ExternalDataSources.flockFollowSource(Duration.fromDays(7))
|
||||
val flockBlockSource = ExternalDataSources.flockBlockSource(Duration.fromDays(7))
|
||||
val flockReportAsAbuseSource =
|
||||
ExternalDataSources.flockReportAsAbuseSource(Duration.fromDays(7))
|
||||
val flockReportAsSpamSource =
|
||||
ExternalDataSources.flockReportAsSpamSource(Duration.fromDays(7))
|
||||
|
||||
// user-user fav edges
|
||||
val userUserFavSource = ExternalDataSources.userUserFavSource(Duration.fromDays(14))
|
||||
val userUserFavEdges = getFavEdges(userUserFavSource, HalfLifeInDaysForFavScore)
|
||||
|
||||
// user-user follow edges
|
||||
val userUserFollowEdges = filterInvalidUsers(flockFollowSource, validUsers)
|
||||
|
||||
// user-user block edges
|
||||
val userUserBlockEdges = filterInvalidUsers(flockBlockSource, validUsers)
|
||||
|
||||
// user-user abuse report edges
|
||||
val userUserAbuseReportEdges = filterInvalidUsers(flockReportAsAbuseSource, validUsers)
|
||||
|
||||
// user-user spam report edges
|
||||
val userUserSpamReportEdges = filterInvalidUsers(flockReportAsSpamSource, validUsers)
|
||||
|
||||
// user-signup country edges
|
||||
val userSignUpCountryEdges = ExternalDataSources
|
||||
.userCountrySource(Duration.fromDays(7))
|
||||
|
||||
// user-consumed language edges
|
||||
val userConsumedLanguageEdges =
|
||||
ExternalDataSources.inferredUserConsumedLanguageSource(Duration.fromDays(7))
|
||||
|
||||
// user-topic follow edges
|
||||
val topicUserFollowedByEdges =
|
||||
ExternalDataSources.topicFollowGraphSource(Duration.fromDays(7))
|
||||
|
||||
// user-MRNotifOpenOrClick events from last 7 days
|
||||
val userMRNotifOpenOrClickEvents =
|
||||
ExternalDataSources.magicRecsNotficationOpenOrClickEventsSource(interval_7days)
|
||||
|
||||
// user-searchQuery strings from last 7 days
|
||||
val userSearchQueryEdges =
|
||||
ExternalDataSources.adaptiveSearchScribeLogsSource(interval_7days)
|
||||
|
||||
/*
|
||||
* Generate the full graph
|
||||
*/
|
||||
val fullGraph =
|
||||
getUserTweetInteractionGraph(tweetSource) ++
|
||||
getUserFavGraph(userUserFavEdges) ++
|
||||
getUserFollowGraph(userUserFollowEdges) ++
|
||||
getUserBlockGraph(userUserBlockEdges) ++
|
||||
getUserAbuseReportGraph(userUserAbuseReportEdges) ++
|
||||
getUserSpamReportGraph(userUserSpamReportEdges) ++
|
||||
getUserSignUpCountryGraph(userSignUpCountryEdges) ++
|
||||
getUserConsumedLanguagesGraph(userConsumedLanguageEdges) ++
|
||||
getUserTopicFollowGraph(topicUserFollowedByEdges) ++
|
||||
getMagicRecsNotifOpenOrClickTweetsGraph(userMRNotifOpenOrClickEvents) ++
|
||||
getSearchGraph(userSearchQueryEdges)
|
||||
|
||||
// Get Top K RightNodes
|
||||
val topKRightNodes: SCollection[(RightNodeType, Seq[(Noun, Double)])] =
|
||||
getTopKRightNounsWithFrequencies(
|
||||
fullGraph,
|
||||
TopKConfig,
|
||||
GlobalDefaultMinFrequencyOfRightNodeType)
|
||||
|
||||
// key transformation - topK nouns, keyed by the RightNodeNounType
|
||||
val topKNounsKeyedByType: SCollection[(RightNodeTypeStruct, NounWithFrequencyList)] =
|
||||
topKRightNodes
|
||||
.map {
|
||||
case (rightNodeType, rightNounsWithScoresList) =>
|
||||
val nounsListWithFrequency: Seq[NounWithFrequency] = rightNounsWithScoresList
|
||||
.map {
|
||||
case (noun, aggregatedFrequency) =>
|
||||
NounWithFrequency(noun, aggregatedFrequency)
|
||||
}
|
||||
(RightNodeTypeStruct(rightNodeType), NounWithFrequencyList(nounsListWithFrequency))
|
||||
}
|
||||
|
||||
// Get Truncated graph based on the top K RightNodes
|
||||
val truncatedGraph: SCollection[(LeftNode, RightNodeWithEdgeWeight)] =
|
||||
getTruncatedGraph(fullGraph, topKRightNodes)
|
||||
|
||||
// key transformations - truncated graph, keyed by LeftNode
|
||||
// Note: By wrapping and unwrapping with the LeftNode.UserId, we don't have to deal
|
||||
// with defining our own customer ordering for LeftNode type
|
||||
val truncatedGraphKeyedBySrc: SCollection[(LeftNode, RightNodeWithEdgeWeightList)] =
|
||||
truncatedGraph
|
||||
.collect {
|
||||
case (LeftNode.UserId(userId), rightNodeWithWeight) =>
|
||||
userId -> List(rightNodeWithWeight)
|
||||
}
|
||||
.sumByKey
|
||||
.map {
|
||||
case (userId, rightNodeWithWeightList) =>
|
||||
(LeftNode.UserId(userId), RightNodeWithEdgeWeightList(rightNodeWithWeightList))
|
||||
}
|
||||
|
||||
// WriteExecs
|
||||
// Write TopK RightNodes to DAL - save all the top K nodes for the clustering step
|
||||
topKNounsKeyedByType
|
||||
.map {
|
||||
case (engagementType, rightList) =>
|
||||
KeyVal(engagementType, rightList)
|
||||
}
|
||||
.saveAsCustomOutput(
|
||||
name = "WriteTopKNouns",
|
||||
DAL.writeVersionedKeyVal(
|
||||
topKRightNounsKeyValDataset,
|
||||
PathLayout.VersionedPath(prefix =
|
||||
rootMHPath + topKRightNounsOutputDir),
|
||||
instant = Instant.ofEpochMilli(opts.interval.getEndMillis - 1L),
|
||||
environmentOverride = dalEnv,
|
||||
)
|
||||
)
|
||||
|
||||
// Write TopK RightNodes to DAL - only take TopKRightNounsForMHDump RightNodes for MH dump
|
||||
topKNounsKeyedByType
|
||||
.map {
|
||||
case (engagementType, rightList) =>
|
||||
val rightListMH =
|
||||
NounWithFrequencyList(rightList.nounWithFrequencyList.take(TopKRightNounsForMHDump))
|
||||
KeyVal(engagementType, rightListMH)
|
||||
}
|
||||
.saveAsCustomOutput(
|
||||
name = "WriteTopKNounsToMHForDebugger",
|
||||
DAL.writeVersionedKeyVal(
|
||||
topKRightNounsMHKeyValDataset,
|
||||
PathLayout.VersionedPath(prefix =
|
||||
rootMHPath + topKRightNounsMHOutputDir),
|
||||
instant = Instant.ofEpochMilli(opts.interval.getEndMillis - 1L),
|
||||
environmentOverride = dalEnv,
|
||||
)
|
||||
)
|
||||
|
||||
// Write truncated graph (MultiTypeGraphTopKForRightNodes) to DAL in KeyVal format
|
||||
truncatedGraphKeyedBySrc
|
||||
.map {
|
||||
case (leftNode, rightNodeWithWeightList) =>
|
||||
KeyVal(leftNode, rightNodeWithWeightList)
|
||||
}.saveAsCustomOutput(
|
||||
name = "WriteTruncatedMultiTypeGraph",
|
||||
DAL.writeVersionedKeyVal(
|
||||
truncatedMultiTypeGraphKeyValDataset,
|
||||
PathLayout.VersionedPath(prefix =
|
||||
rootMHPath + truncatedMultiTypeGraphMHOutputDir),
|
||||
instant = Instant.ofEpochMilli(opts.interval.getEndMillis - 1L),
|
||||
environmentOverride = dalEnv,
|
||||
)
|
||||
)
|
||||
|
||||
// Write truncated graph (MultiTypeGraphTopKForRightNodes) to DAL in thrift format
|
||||
truncatedGraph
|
||||
.map {
|
||||
case (leftNode, rightNodeWithWeight) =>
|
||||
MultiTypeGraphEdge(leftNode, rightNodeWithWeight)
|
||||
}.saveAsCustomOutput(
|
||||
name = "WriteTruncatedMultiTypeGraphThrift",
|
||||
DAL.writeSnapshot(
|
||||
multiTypeGraphTopKForRightNodesSnapshotDataset,
|
||||
PathLayout.FixedPath(rootThriftPath + truncatedMultiTypeGraphThriftOutputDir),
|
||||
Instant.ofEpochMilli(opts.interval.getEndMillis - 1L),
|
||||
DiskFormat.Thrift(),
|
||||
environmentOverride = dalEnv
|
||||
)
|
||||
)
|
||||
|
||||
// Write full graph to DAL
|
||||
fullGraph
|
||||
.map {
|
||||
case (leftNode, rightNodeWithWeight) =>
|
||||
MultiTypeGraphEdge(leftNode, rightNodeWithWeight)
|
||||
}
|
||||
.saveAsCustomOutput(
|
||||
name = "WriteFullMultiTypeGraph",
|
||||
DAL.writeSnapshot(
|
||||
fullMultiTypeGraphSnapshotDataset,
|
||||
PathLayout.FixedPath(rootThriftPath + fullMultiTypeGraphThriftOutputDir),
|
||||
Instant.ofEpochMilli(opts.interval.getEndMillis - 1L),
|
||||
DiskFormat.Thrift(),
|
||||
environmentOverride = dalEnv
|
||||
)
|
||||
)
|
||||
|
||||
}
|
||||
}
|
@ -1,73 +0,0 @@
|
||||
scala_library(
|
||||
name = "assemble-multi-type-graph-scio-lib",
|
||||
sources = [
|
||||
"*.scala",
|
||||
],
|
||||
platform = "java8",
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
":full_multi_type_graph_scio-scala",
|
||||
":top_k_right_nouns_mh_scio-scala",
|
||||
"beam-internal/src/main/scala/com/twitter/beam/io/dal",
|
||||
"beam-internal/src/main/scala/com/twitter/beam/io/manhattan",
|
||||
"beam-internal/src/main/scala/com/twitter/beam/job",
|
||||
"beam-internal/src/main/scala/com/twitter/beam/transform",
|
||||
"beam-internal/src/main/scala/com/twitter/scio_internal/runner/dataflow",
|
||||
"src/scala/com/twitter/simclusters_v2/hdfs_sources",
|
||||
"src/scala/com/twitter/simclusters_v2/scalding/multi_type_graph/assemble_multi_type_graph",
|
||||
"src/scala/com/twitter/simclusters_v2/scio/common",
|
||||
"src/scala/com/twitter/simclusters_v2/scio/multi_type_graph/common",
|
||||
],
|
||||
)
|
||||
|
||||
jvm_binary(
|
||||
name = "assemble-multi-type-graph-scio-adhoc-app",
|
||||
main = "com.twitter.simclusters_v2.scio.multi_type_graph.assemble_multi_type_graph.AssembleMultiTypeGraphScioAdhocApp",
|
||||
platform = "java8",
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
":assemble-multi-type-graph-scio-lib",
|
||||
"beam-internal/src/main/scala/com/twitter/beam/runner/dataflow",
|
||||
],
|
||||
)
|
||||
|
||||
jvm_binary(
|
||||
name = "assemble-multi-type-graph-scio-batch-app",
|
||||
main = "com.twitter.simclusters_v2.scio.multi_type_graph.assemble_multi_type_graph.AssembleMultiTypeGraphScioBatchApp",
|
||||
platform = "java8",
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
":assemble-multi-type-graph-scio-lib",
|
||||
"beam-internal/src/main/scala/com/twitter/beam/runner/dataflow",
|
||||
],
|
||||
)
|
||||
|
||||
create_datasets(
|
||||
base_name = "full_multi_type_graph_scio",
|
||||
java_schema = "com.twitter.simclusters_v2.thriftjava.MultiTypeGraphEdge",
|
||||
platform = "java8",
|
||||
role = "cassowary",
|
||||
scala_schema = "com.twitter.simclusters_v2.thriftscala.MultiTypeGraphEdge",
|
||||
segment_type = "snapshot",
|
||||
tags = ["bazel-compatible"],
|
||||
java_dependencies = [
|
||||
"src/thrift/com/twitter/simclusters_v2:simclusters_v2-thrift-java",
|
||||
],
|
||||
scala_dependencies = [
|
||||
"src/thrift/com/twitter/simclusters_v2:simclusters_v2-thrift-scala",
|
||||
],
|
||||
)
|
||||
|
||||
create_datasets(
|
||||
base_name = "top_k_right_nouns_mh_scio",
|
||||
key_type = "com.twitter.simclusters_v2.thriftscala.RightNodeTypeStruct",
|
||||
platform = "java8",
|
||||
role = "cassowary",
|
||||
scala_schema = "com.twitter.simclusters_v2.hdfs_sources.injections.MultiTypeGraphInjections.topKRightNounListInjection",
|
||||
segment_type = "snapshot",
|
||||
tags = ["bazel-compatible"],
|
||||
val_type = "com.twitter.simclusters_v2.thriftscala.NounWithFrequencyList",
|
||||
scala_dependencies = [
|
||||
"src/scala/com/twitter/simclusters_v2/hdfs_sources/injections",
|
||||
],
|
||||
)
|
Binary file not shown.
Binary file not shown.
@ -1,37 +0,0 @@
|
||||
package com.twitter.simclusters_v2.scio.multi_type_graph.assemble_multi_type_graph
|
||||
|
||||
import com.twitter.simclusters_v2.thriftscala.RightNodeType
|
||||
|
||||
object Config {
|
||||
val RootMHPath: String = "manhattan_sequence_files/multi_type_graph/"
|
||||
val RootThriftPath: String = "processed/multi_type_graph/"
|
||||
val AdhocRootPath = "adhoc/multi_type_graph/"
|
||||
val truncatedMultiTypeGraphMHOutputDir: String = "truncated_graph_mh"
|
||||
val truncatedMultiTypeGraphThriftOutputDir: String = "truncated_graph_thrift"
|
||||
val topKRightNounsMHOutputDir: String = "top_k_right_nouns_mh"
|
||||
val topKRightNounsOutputDir: String = "top_k_right_nouns"
|
||||
val fullMultiTypeGraphThriftOutputDir: String = "full_graph_thrift"
|
||||
val HalfLifeInDaysForFavScore = 100
|
||||
val NumTopNounsForUnknownRightNodeType = 20
|
||||
val GlobalDefaultMinFrequencyOfRightNodeType = 100
|
||||
val TopKRightNounsForMHDump = 1000
|
||||
|
||||
// the topK most frequent nouns for each engagement type
|
||||
val TopKConfig: Map[RightNodeType, Int] = Map(
|
||||
RightNodeType.FollowUser -> 10000000, // 10M, current simclusters_v2 has this value set to 20M, providing this the most weight
|
||||
RightNodeType.FavUser -> 5000000,
|
||||
RightNodeType.BlockUser -> 1000000,
|
||||
RightNodeType.AbuseReportUser -> 1000000,
|
||||
RightNodeType.SpamReportUser -> 1000000,
|
||||
RightNodeType.FollowTopic -> 5000,
|
||||
RightNodeType.SignUpCountry -> 200,
|
||||
RightNodeType.ConsumedLanguage -> 50,
|
||||
RightNodeType.FavTweet -> 500000,
|
||||
RightNodeType.ReplyTweet -> 500000,
|
||||
RightNodeType.RetweetTweet -> 500000,
|
||||
RightNodeType.NotifOpenOrClickTweet -> 500000,
|
||||
RightNodeType.SearchQuery -> 500000
|
||||
)
|
||||
val SampledEmployeeIds: Set[Long] =
|
||||
Set()
|
||||
}
|
Binary file not shown.
@ -1,49 +0,0 @@
|
||||
# Pre-requisites
|
||||
|
||||
## Tutorial
|
||||
Follow the tutorial Batch Job on Dataflow Quickstart on how to run a simple batch job on Dataflow.
|
||||
|
||||
## GCP setup
|
||||
|
||||
Ensure `gcloud` CLI is installed and `application_default_credentials.json` has been generated.
|
||||
|
||||
## Data access
|
||||
|
||||
If you want to run an adhoc job with your ldap, you will need access to multiple LDAP groups to read the datasets.
|
||||
|
||||
# Running the job
|
||||
|
||||
### Running an adhoc job
|
||||
|
||||
```bash
|
||||
export GCP_PROJECT_NAME='twttr-recos-ml-prod'
|
||||
|
||||
./bazel bundle src/scala/com/twitter/simclusters_v2/scio/multi_type_graph/assemble_multi_type_graph:assemble-multi-type-graph-scio-adhoc-app
|
||||
|
||||
bin/d6w create \
|
||||
${GCP_PROJECT_NAME}/us-central1/assemble-multi-type-graph-scio-adhoc-app \
|
||||
src/scala/com/twitter/simclusters_v2/scio/multi_type_graph/assemble_multi_type_graph/assemble-multi-type-graph-scio-adhoc.d6w \
|
||||
--jar dist/assemble-multi-type-graph-scio-adho-app.jar \
|
||||
--bind=profile.project=${GCP_PROJECT_NAME} \
|
||||
--bind=profile.user_name=${USER} \
|
||||
--bind=profile.date="2021-11-04" \
|
||||
--bind=profile.machine="n2-highmem-16"
|
||||
```
|
||||
|
||||
### Scheduling the job on Workflow
|
||||
|
||||
Scheduling a job will require a service account as `recos-platform`.
|
||||
Remember this account will need permissions to read all the required dataset.
|
||||
|
||||
```bash
|
||||
export SERVICE_ACCOUNT='recos-platform'
|
||||
export GCP_PROJECT_NAME='twttr-recos-ml-prod'
|
||||
|
||||
bin/d6w schedule \
|
||||
${GCP_PROJECT_NAME}/us-central1/assemble-multi-type-graph-scio-batch-app \
|
||||
src/scala/com/twitter/simclusters_v2/scio/multi_type_graph/assemble_multi_type_graph/assemble-multi-type-graph-scio-batch.d6w \
|
||||
--bind=profile.project=${GCP_PROJECT_NAME} \
|
||||
--bind=profile.user_name="recos-platform" \
|
||||
--bind=profile.date="2021-11-04" \
|
||||
--bind=profile.machine="n2-highmem-16"
|
||||
```
|
@ -1,36 +0,0 @@
|
||||
# See
|
||||
# Checkout the README to see how to deploy the job
|
||||
|
||||
class Profile(Struct):
|
||||
project = Required(String)
|
||||
date = Required(String)
|
||||
environment = Default(String, 'dev')
|
||||
machine= Default(String, 'n2-highmem-16')
|
||||
|
||||
job = Job(
|
||||
name='assemble-multi-type-graph-scio-adhoc-app',
|
||||
project='{{profile.project}}',
|
||||
staging_bucket='{{profile.project}}',
|
||||
service_account='{{profile.user_name}}-shdw@twttr-dp-svc-accounts.iam.gserviceaccount.com',
|
||||
region='us-central1',
|
||||
worker_config=WorkerConfig(
|
||||
num_workers=2,
|
||||
worker_machine_type='{{profile.machine}}',
|
||||
worker_disk_type=WorkerDiskType('HDD')
|
||||
),
|
||||
extra_args={
|
||||
"environment": '{{profile.environment}}',
|
||||
"date": Quote('{{profile.date}}'),
|
||||
},
|
||||
deployment_config=BatchDeploymentConfig(
|
||||
role='{{profile.user_name}}',
|
||||
build_target='src/scala/com/twitter/simclusters_v2/scio/multi_type_graph/assemble_multi_type_graph:assemble-multi-type-graph-scio-adhoc-app',
|
||||
gcp_deployment_credentials='/var/lib/tss/keys/{{profile.user_name}}/cloud/gcp/dp/shadow.json',
|
||||
statebird_config=StatebirdConfig(
|
||||
batch_width='PT1H',
|
||||
first_time='{{profile.date}}'
|
||||
)
|
||||
)
|
||||
)
|
||||
|
||||
jobs=[job]
|
Binary file not shown.
@ -1,41 +0,0 @@
|
||||
# See
|
||||
# Checkout the README to see how to deploy the job
|
||||
|
||||
class Profile(Struct):
|
||||
project = Required(String)
|
||||
date = Required(String)
|
||||
environment = Default(String, 'prod')
|
||||
machine= Default(String, 'n2-highmem-16')
|
||||
|
||||
job = Job(
|
||||
name='assemble-multi-type-graph-scio-batch-app',
|
||||
project='{{profile.project}}',
|
||||
staging_bucket='{{profile.project}}',
|
||||
service_account='{{profile.user_name}}-shdw@twttr-dp-svc-accounts.iam.gserviceaccount.com',
|
||||
region='us-central1',
|
||||
worker_config=WorkerConfig(
|
||||
num_workers=2,
|
||||
worker_machine_type='{{profile.machine}}',
|
||||
worker_disk_type=WorkerDiskType('HDD')
|
||||
),
|
||||
extra_args={
|
||||
"environment": '{{profile.environment}}',
|
||||
"date": Quote('{{profile.date}}'),
|
||||
},
|
||||
deployment_config=BatchDeploymentConfig(
|
||||
role='{{profile.user_name}}',
|
||||
build_target='src/scala/com/twitter/simclusters_v2/scio/multi_type_graph/assemble_multi_type_graph:assemble-multi-type-graph-scio-batch-app',
|
||||
gcp_deployment_credentials='/var/lib/tss/keys/{{profile.user_name}}/cloud/gcp/dp/shadow.json',
|
||||
environment='prod',
|
||||
statebird_config=StatebirdConfig(
|
||||
batch_width='P1W',
|
||||
first_time='{{profile.date}}'
|
||||
),
|
||||
workflow_config=WorkflowConfig(
|
||||
play=True,
|
||||
),
|
||||
timeout='PT18H'
|
||||
)
|
||||
)
|
||||
|
||||
jobs=[job]
|
Binary file not shown.
@ -1,13 +0,0 @@
|
||||
scala_library(
|
||||
sources = [
|
||||
"*.scala",
|
||||
],
|
||||
platform = "java8",
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"beam-internal/src/main/scala/com/twitter/beam/io/dal",
|
||||
"beam-internal/src/main/scala/com/twitter/scio_internal/runner/dataflow",
|
||||
"src/scala/com/twitter/simclusters_v2/hdfs_sources",
|
||||
"src/scala/com/twitter/simclusters_v2/scalding",
|
||||
],
|
||||
)
|
Binary file not shown.
Binary file not shown.
@ -1,69 +0,0 @@
|
||||
package com.twitter.simclusters_v2.scio
|
||||
package multi_type_graph.common
|
||||
|
||||
import com.spotify.scio.ScioContext
|
||||
import com.spotify.scio.values.SCollection
|
||||
import com.twitter.beam.io.dal.DAL
|
||||
import com.twitter.common.util.Clock
|
||||
import com.twitter.scalding_internal.job.RequiredBinaryComparators.ordSer
|
||||
import com.twitter.scalding_internal.multiformat.format.keyval.KeyVal
|
||||
import com.twitter.simclusters_v2.hdfs_sources.TruncatedMultiTypeGraphScioScalaDataset
|
||||
import com.twitter.simclusters_v2.thriftscala.LeftNode
|
||||
import com.twitter.simclusters_v2.thriftscala.Noun
|
||||
import com.twitter.simclusters_v2.thriftscala.RightNode
|
||||
import com.twitter.simclusters_v2.thriftscala.RightNodeType
|
||||
import com.twitter.util.Duration
|
||||
|
||||
object MultiTypeGraphUtil {
|
||||
val RootMHPath: String = "manhattan_sequence_files/multi_type_graph/"
|
||||
val RootThriftPath: String = "processed/multi_type_graph/"
|
||||
val AdhocRootPath = "adhoc/multi_type_graph/"
|
||||
|
||||
val nounOrdering: Ordering[Noun] = new Ordering[Noun] {
|
||||
// We define an ordering for each noun type as specified in simclusters_v2/multi_type_graph.thrift
|
||||
// Please make sure we don't remove anything here that's still a part of the union Noun thrift and
|
||||
// vice versa, if we add a new noun type to thrift, an ordering for it needs to added here as well.
|
||||
def nounTypeOrder(noun: Noun): Int = noun match {
|
||||
case _: Noun.UserId => 0
|
||||
case _: Noun.Country => 1
|
||||
case _: Noun.Language => 2
|
||||
case _: Noun.Query => 3
|
||||
case _: Noun.TopicId => 4
|
||||
case _: Noun.TweetId => 5
|
||||
}
|
||||
|
||||
override def compare(x: Noun, y: Noun): Int = nounTypeOrder(x) compare nounTypeOrder(y)
|
||||
}
|
||||
|
||||
val rightNodeTypeOrdering: Ordering[RightNodeType] = ordSer[RightNodeType]
|
||||
|
||||
val rightNodeOrdering: Ordering[RightNode] =
|
||||
new Ordering[RightNode] {
|
||||
override def compare(x: RightNode, y: RightNode): Int = {
|
||||
Ordering
|
||||
.Tuple2(rightNodeTypeOrdering, nounOrdering)
|
||||
.compare((x.rightNodeType, x.noun), (y.rightNodeType, y.noun))
|
||||
}
|
||||
}
|
||||
|
||||
def getTruncatedMultiTypeGraph(
|
||||
noOlderThan: Duration = Duration.fromDays(14)
|
||||
)(
|
||||
implicit sc: ScioContext
|
||||
): SCollection[(Long, RightNode, Double)] = {
|
||||
sc.customInput(
|
||||
"ReadTruncatedMultiTypeGraph",
|
||||
DAL
|
||||
.readMostRecentSnapshotNoOlderThan(
|
||||
TruncatedMultiTypeGraphScioScalaDataset,
|
||||
noOlderThan,
|
||||
Clock.SYSTEM_CLOCK,
|
||||
DAL.Environment.Prod
|
||||
)
|
||||
).flatMap {
|
||||
case KeyVal(LeftNode.UserId(userId), rightNodesList) =>
|
||||
rightNodesList.rightNodeWithEdgeWeightList.map(rightNodeWithWeight =>
|
||||
(userId, rightNodeWithWeight.rightNode, rightNodeWithWeight.weight))
|
||||
}
|
||||
}
|
||||
}
|
@ -1,92 +0,0 @@
|
||||
scala_library(
|
||||
name = "multi-type-graph-scio-sims-lib",
|
||||
sources = ["*.scala"],
|
||||
platform = "java8",
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
":right_node_cosine_similarity_scio_adhoc-scala",
|
||||
":right_node_sim_hash_scio_adhoc-scala",
|
||||
"3rdparty/jvm/com/twitter/bijection:scrooge",
|
||||
"beam-internal/src/main/scala/com/twitter/beam/io/dal",
|
||||
"beam-internal/src/main/scala/com/twitter/beam/io/manhattan",
|
||||
"beam-internal/src/main/scala/com/twitter/beam/job",
|
||||
"beam-internal/src/main/scala/com/twitter/beam/transform",
|
||||
"beam-internal/src/main/scala/com/twitter/scio_internal/runner/dataflow",
|
||||
"src/scala/com/twitter/simclusters_v2/hdfs_sources",
|
||||
"src/scala/com/twitter/simclusters_v2/scio/multi_type_graph/common",
|
||||
"src/scala/com/twitter/wtf/dataflow/cosine_similarity/common",
|
||||
],
|
||||
)
|
||||
|
||||
jvm_binary(
|
||||
name = "multi-type-graph-sim-hash-scio-adhoc-app",
|
||||
main = "com.twitter.simclusters_v2.scio.multi_type_graph.multi_type_graph_sims.RightNodeSimHashScioAdhocApp",
|
||||
platform = "java8",
|
||||
dependencies = [
|
||||
":multi-type-graph-scio-sims-lib",
|
||||
"beam-internal/src/main/scala/com/twitter/beam/runner/dataflow",
|
||||
],
|
||||
)
|
||||
|
||||
jvm_binary(
|
||||
name = "multi-type-graph-sim-hash-scio-batch-app",
|
||||
main = "com.twitter.simclusters_v2.scio.multi_type_graph.multi_type_graph_sims.RightNodeSimHashScioBatchApp",
|
||||
platform = "java8",
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
":multi-type-graph-scio-sims-lib",
|
||||
"beam-internal/src/main/scala/com/twitter/beam/runner/dataflow",
|
||||
],
|
||||
)
|
||||
|
||||
jvm_binary(
|
||||
name = "multi-type-graph-cosine-similarity-scio-adhoc-app",
|
||||
main = "com.twitter.simclusters_v2.scio.multi_type_graph.multi_type_graph_sims.RightNodeCosineSimilarityScioAdhocApp",
|
||||
platform = "java8",
|
||||
dependencies = [
|
||||
":multi-type-graph-scio-sims-lib",
|
||||
"beam-internal/src/main/scala/com/twitter/beam/runner/dataflow",
|
||||
],
|
||||
)
|
||||
|
||||
jvm_binary(
|
||||
name = "multi-type-graph-cosine-similarity-scio-batch-app",
|
||||
main = "com.twitter.simclusters_v2.scio.multi_type_graph.multi_type_graph_sims.RightNodeCosineSimilarityScioBatchApp",
|
||||
platform = "java8",
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
":multi-type-graph-scio-sims-lib",
|
||||
"beam-internal/src/main/scala/com/twitter/beam/runner/dataflow",
|
||||
],
|
||||
)
|
||||
|
||||
create_datasets(
|
||||
base_name = "right_node_sim_hash_scio_adhoc",
|
||||
java_schema = "com.twitter.simclusters_v2.thriftjava.RightNodeSimHashSketch",
|
||||
platform = "java8",
|
||||
role = "cassowary",
|
||||
scala_schema = "com.twitter.simclusters_v2.thriftscala.RightNodeSimHashSketch",
|
||||
segment_type = "snapshot",
|
||||
tags = ["bazel-compatible"],
|
||||
java_dependencies = [
|
||||
"src/thrift/com/twitter/simclusters_v2:simclusters_v2-thrift-java",
|
||||
],
|
||||
scala_dependencies = [
|
||||
"src/thrift/com/twitter/simclusters_v2:simclusters_v2-thrift-scala",
|
||||
],
|
||||
)
|
||||
|
||||
create_datasets(
|
||||
base_name = "right_node_cosine_similarity_scio_adhoc",
|
||||
key_type = "com.twitter.simclusters_v2.thriftscala.RightNode",
|
||||
platform = "java8",
|
||||
role = "cassowary",
|
||||
scala_schema = "com.twitter.simclusters_v2.hdfs_sources.injections.MultiTypeGraphInjections.similarRightNodesInjection",
|
||||
segment_type = "snapshot",
|
||||
tags = ["bazel-compatible"],
|
||||
val_type = "com.twitter.simclusters_v2.thriftscala.SimilarRightNodes",
|
||||
scala_dependencies = [
|
||||
"src/scala/com/twitter/scalding_internal/multiformat/format",
|
||||
"src/scala/com/twitter/simclusters_v2/hdfs_sources/injections",
|
||||
],
|
||||
)
|
Binary file not shown.
Binary file not shown.
@ -1,18 +0,0 @@
|
||||
package com.twitter.simclusters_v2.scio
|
||||
package multi_type_graph.multi_type_graph_sims
|
||||
|
||||
object Config {
|
||||
// Config settings for RightNodeSimHashScioBaseApp job
|
||||
// Number of hashes to generate in the sketch
|
||||
val numHashes: Int = 8192 // each is a bit, so this results in 1KB uncompressed sketch/user
|
||||
// Reduce skew by letting each reducers process a limited number of followers/user
|
||||
val maxNumNeighborsPerReducers: Int = 300000
|
||||
val simsHashJobOutputDirectory: String = "right_node/sims/sim_hash"
|
||||
|
||||
// Config settings for RightNodeCosineSimilarityScioBaseApp job
|
||||
val numSims: Int = 500
|
||||
val minCosineSimilarityThreshold: Double = 0.01
|
||||
val maxOutDegree: Int = 10000
|
||||
val cosineSimJobOutputDirectory = "right_node/sims/cosine_similarity"
|
||||
|
||||
}
|
Binary file not shown.
@ -1,55 +0,0 @@
|
||||
package com.twitter.simclusters_v2.scio
|
||||
package multi_type_graph.multi_type_graph_sims
|
||||
|
||||
import com.twitter.dal.client.dataset.KeyValDALDataset
|
||||
import com.twitter.scalding_internal.multiformat.format.keyval.KeyVal
|
||||
import com.twitter.simclusters_v2.hdfs_sources.RightNodeCosineSimilarityScioScalaDataset
|
||||
import com.twitter.simclusters_v2.thriftscala.RightNode
|
||||
import com.twitter.simclusters_v2.thriftscala.SimilarRightNodes
|
||||
import com.twitter.wtf.scalding.jobs.cosine_similarity.common.ApproximateMatrixSelfTransposeMultiplicationJob
|
||||
|
||||
/**
|
||||
Build:
|
||||
./bazel bundle src/scala/com/twitter/simclusters_v2/scio/multi_type_graph/multi_type_graph_sims:multi-type-graph-cosine-similarity-scio-adhoc-app
|
||||
|
||||
To kick off an adhoc run:
|
||||
bin/d6w create \
|
||||
${GCP_PROJECT_NAME}/us-central1/multi-type-graph-cosine-similarity-scio-adhoc-app \
|
||||
src/scala/com/twitter/simclusters_v2/scio/multi_type_graph/multi_type_graph_sims/cosine-similarity-scio-adhoc.d6w \
|
||||
--jar dist/multi-type-graph-cosine-similarity-scio-adhoc-app.jar \
|
||||
--bind=profile.project=${GCP_PROJECT_NAME} \
|
||||
--bind=profile.user_name=${USER} \
|
||||
--bind=profile.date="2022-01-16" \
|
||||
--bind=profile.machine="n2d-highmem-16" --ignore-existing
|
||||
*/
|
||||
|
||||
object RightNodeCosineSimilarityScioAdhocApp extends RightNodeCosineSimilarityScioBaseApp {
|
||||
override val isAdhoc = true
|
||||
override val cosineSimKeyValSnapshotDataset: KeyValDALDataset[
|
||||
KeyVal[RightNode, SimilarRightNodes]
|
||||
] =
|
||||
RightNodeCosineSimilarityScioAdhocScalaDataset
|
||||
override val filterCandidateSimilarityPair: (Double, Double, Double) => Boolean =
|
||||
ApproximateMatrixSelfTransposeMultiplicationJob.filterCandidateSimilarityPair
|
||||
}
|
||||
|
||||
/**
|
||||
To deploy the job:
|
||||
|
||||
bin/d6w schedule \
|
||||
${GCP_PROJECT_NAME}/us-central1/multi-type-graph-cosine-similarity-scio-batch-app \
|
||||
src/scala/com/twitter/simclusters_v2/scio/multi_type_graph/multi_type_graph_sims/cosine-similarity-scio-batch.d6w \
|
||||
--bind=profile.project=${GCP_PROJECT_NAME} \
|
||||
--bind=profile.user_name=recos-platform \
|
||||
--bind=profile.date="2021-12-01" \
|
||||
--bind=profile.machine="n2d-highmem-16"
|
||||
*/
|
||||
object RightNodeCosineSimilarityScioBatchApp extends RightNodeCosineSimilarityScioBaseApp {
|
||||
override val isAdhoc = false
|
||||
override val cosineSimKeyValSnapshotDataset: KeyValDALDataset[
|
||||
KeyVal[RightNode, SimilarRightNodes]
|
||||
] =
|
||||
RightNodeCosineSimilarityScioScalaDataset
|
||||
override val filterCandidateSimilarityPair: (Double, Double, Double) => Boolean =
|
||||
ApproximateMatrixSelfTransposeMultiplicationJob.filterCandidateSimilarityPair
|
||||
}
|
Binary file not shown.
@ -1,96 +0,0 @@
|
||||
package com.twitter.simclusters_v2.scio
|
||||
package multi_type_graph.multi_type_graph_sims
|
||||
|
||||
import com.spotify.scio.ScioContext
|
||||
import com.spotify.scio.coders.Coder
|
||||
import com.spotify.scio.values.SCollection
|
||||
import com.twitter.beam.io.dal.DAL
|
||||
import com.twitter.beam.io.fs.multiformat.PathLayout
|
||||
import com.twitter.beam.job.DateRangeOptions
|
||||
import com.twitter.common.util.Clock
|
||||
import com.twitter.dal.client.dataset.KeyValDALDataset
|
||||
import com.twitter.dal.client.dataset.SnapshotDALDataset
|
||||
import com.twitter.scalding_internal.multiformat.format.keyval.KeyVal
|
||||
import com.twitter.scio_internal.coders.ThriftStructLazyBinaryScroogeCoder
|
||||
import com.twitter.scio_internal.job.ScioBeamJob
|
||||
import com.twitter.scrooge.ThriftStruct
|
||||
import com.twitter.simclusters_v2.hdfs_sources.RightNodeSimHashScioScalaDataset
|
||||
import com.twitter.simclusters_v2.scio.multi_type_graph.common.MultiTypeGraphUtil
|
||||
import com.twitter.simclusters_v2.thriftscala._
|
||||
import com.twitter.util.Duration
|
||||
import com.twitter.wtf.dataflow.cosine_similarity.ApproximateMatrixSelfTransposeMultiplicationJob
|
||||
import java.time.Instant
|
||||
|
||||
trait RightNodeCosineSimilarityScioBaseApp
|
||||
extends ScioBeamJob[DateRangeOptions]
|
||||
with ApproximateMatrixSelfTransposeMultiplicationJob[RightNode] {
|
||||
override implicit def scroogeCoder[T <: ThriftStruct: Manifest]: Coder[T] =
|
||||
ThriftStructLazyBinaryScroogeCoder.scroogeCoder
|
||||
override val ordering: Ordering[RightNode] = MultiTypeGraphUtil.rightNodeOrdering
|
||||
|
||||
val isAdhoc: Boolean
|
||||
val cosineSimKeyValSnapshotDataset: KeyValDALDataset[KeyVal[RightNode, SimilarRightNodes]]
|
||||
val rightNodeSimHashSnapshotDataset: SnapshotDALDataset[RightNodeSimHashSketch] =
|
||||
RightNodeSimHashScioScalaDataset
|
||||
val cosineSimJobOutputDirectory: String = Config.cosineSimJobOutputDirectory
|
||||
|
||||
override def graph(
|
||||
implicit sc: ScioContext,
|
||||
coder: Coder[RightNode]
|
||||
): SCollection[(Long, RightNode, Double)] =
|
||||
MultiTypeGraphUtil.getTruncatedMultiTypeGraph(noOlderThan = Duration.fromDays(14))
|
||||
|
||||
override def simHashSketches(
|
||||
implicit sc: ScioContext,
|
||||
coder: Coder[RightNode]
|
||||
): SCollection[(RightNode, Array[Byte])] = {
|
||||
sc.customInput(
|
||||
"ReadSimHashSketches",
|
||||
DAL
|
||||
.readMostRecentSnapshotNoOlderThan(
|
||||
rightNodeSimHashSnapshotDataset,
|
||||
Duration.fromDays(14),
|
||||
Clock.SYSTEM_CLOCK,
|
||||
DAL.Environment.Prod
|
||||
)
|
||||
).map { sketch =>
|
||||
sketch.rightNode -> sketch.simHashOfEngagers.toArray
|
||||
}
|
||||
}
|
||||
|
||||
override def configurePipeline(
|
||||
sc: ScioContext,
|
||||
opts: DateRangeOptions
|
||||
): Unit = {
|
||||
implicit def scioContext: ScioContext = sc
|
||||
// DAL.Environment variable for WriteExecs
|
||||
val dalEnv = if (isAdhoc) DAL.Environment.Dev else DAL.Environment.Prod
|
||||
|
||||
val topKRightNodes: SCollection[(RightNode, SimilarRightNodes)] = topK
|
||||
.collect {
|
||||
case (rightNode, simRightNodes) =>
|
||||
val sims = simRightNodes.collect {
|
||||
case (simRightNode, score) => SimilarRightNode(simRightNode, score)
|
||||
}
|
||||
(rightNode, SimilarRightNodes(sims))
|
||||
}
|
||||
|
||||
topKRightNodes
|
||||
.map {
|
||||
case (rightNode, sims) => KeyVal(rightNode, sims)
|
||||
}.saveAsCustomOutput(
|
||||
name = "WriteRightNodeCosineSimilarityDataset",
|
||||
DAL.writeVersionedKeyVal(
|
||||
cosineSimKeyValSnapshotDataset,
|
||||
PathLayout.VersionedPath(prefix =
|
||||
((if (!isAdhoc)
|
||||
MultiTypeGraphUtil.RootMHPath
|
||||
else
|
||||
MultiTypeGraphUtil.AdhocRootPath)
|
||||
+ Config.cosineSimJobOutputDirectory)),
|
||||
instant = Instant.ofEpochMilli(opts.interval.getEndMillis - 1L),
|
||||
environmentOverride = dalEnv,
|
||||
)
|
||||
)
|
||||
}
|
||||
}
|
Binary file not shown.
@ -1,43 +0,0 @@
|
||||
package com.twitter.simclusters_v2.scio
|
||||
package multi_type_graph.multi_type_graph_sims
|
||||
|
||||
import com.twitter.dal.client.dataset.SnapshotDALDataset
|
||||
import com.twitter.simclusters_v2.hdfs_sources.RightNodeSimHashScioScalaDataset
|
||||
import com.twitter.simclusters_v2.thriftscala.RightNodeSimHashSketch
|
||||
|
||||
/**
|
||||
Build:
|
||||
./bazel bundle src/scala/com/twitter/simclusters_v2/scio/multi_type_graph/multi_type_graph_sims:multi-type-graph-sim-hash-scio-adhoc-app
|
||||
|
||||
To kick off an adhoc run:
|
||||
bin/d6w create \
|
||||
${GCP_PROJECT_NAME}/us-central1/multi-type-graph-sim-hash-scio-adhoc-app \
|
||||
src/scala/com/twitter/simclusters_v2/scio/multi_type_graph/multi_type_graph_sims/sim-hash-scio-adhoc.d6w \
|
||||
--jar dist/multi-type-graph-sim-hash-scio-adhoc-app.jar \
|
||||
--bind=profile.project=${GCP_PROJECT_NAME} \
|
||||
--bind=profile.user_name=${USER} \
|
||||
--bind=profile.date="2021-12-01" \
|
||||
--bind=profile.machine="n2d-highmem-16" --ignore-existing
|
||||
*/
|
||||
object RightNodeSimHashScioAdhocApp extends RightNodeSimHashScioBaseApp {
|
||||
override val isAdhoc: Boolean = true
|
||||
override val rightNodeSimHashSnapshotDataset: SnapshotDALDataset[RightNodeSimHashSketch] =
|
||||
RightNodeSimHashScioAdhocScalaDataset
|
||||
}
|
||||
|
||||
/**
|
||||
To deploy the job:
|
||||
|
||||
bin/d6w schedule \
|
||||
${GCP_PROJECT_NAME}/us-central1/multi-type-graph-sim-hash-scio-batch-app \
|
||||
src/scala/com/twitter/simclusters_v2/scio/multi_type_graph/multi_type_graph_sims/sim-hash-scio-batch.d6w \
|
||||
--bind=profile.project=${GCP_PROJECT_NAME} \
|
||||
--bind=profile.user_name=recos-platform \
|
||||
--bind=profile.date="2021-12-01" \
|
||||
--bind=profile.machine="n2d-highmem-16"
|
||||
*/
|
||||
object RightNodeSimHashScioBatchApp extends RightNodeSimHashScioBaseApp {
|
||||
override val isAdhoc: Boolean = false
|
||||
override val rightNodeSimHashSnapshotDataset: SnapshotDALDataset[RightNodeSimHashSketch] =
|
||||
RightNodeSimHashScioScalaDataset
|
||||
}
|
Binary file not shown.
@ -1,65 +0,0 @@
|
||||
package com.twitter.simclusters_v2.scio
|
||||
package multi_type_graph.multi_type_graph_sims
|
||||
|
||||
import com.spotify.scio.ScioContext
|
||||
import com.spotify.scio.coders.Coder
|
||||
import com.spotify.scio.values.SCollection
|
||||
import com.twitter.beam.io.dal.DAL
|
||||
import com.twitter.beam.io.fs.multiformat.DiskFormat
|
||||
import com.twitter.beam.io.fs.multiformat.PathLayout
|
||||
import com.twitter.beam.job.DateRangeOptions
|
||||
import com.twitter.dal.client.dataset.SnapshotDALDataset
|
||||
import com.twitter.scio_internal.coders.ThriftStructLazyBinaryScroogeCoder
|
||||
import com.twitter.scio_internal.job.ScioBeamJob
|
||||
import com.twitter.scrooge.ThriftStruct
|
||||
import com.twitter.simclusters_v2.scio.multi_type_graph.common.MultiTypeGraphUtil
|
||||
import com.twitter.simclusters_v2.thriftscala.RightNode
|
||||
import com.twitter.simclusters_v2.thriftscala.RightNodeSimHashSketch
|
||||
import com.twitter.util.Duration
|
||||
import com.twitter.wtf.dataflow.cosine_similarity.SimHashJob
|
||||
import java.time.Instant
|
||||
|
||||
trait RightNodeSimHashScioBaseApp extends ScioBeamJob[DateRangeOptions] with SimHashJob[RightNode] {
|
||||
override implicit def scroogeCoder[T <: ThriftStruct: Manifest]: Coder[T] =
|
||||
ThriftStructLazyBinaryScroogeCoder.scroogeCoder
|
||||
override val ordering: Ordering[RightNode] = MultiTypeGraphUtil.rightNodeOrdering
|
||||
|
||||
val isAdhoc: Boolean
|
||||
val rightNodeSimHashSnapshotDataset: SnapshotDALDataset[RightNodeSimHashSketch]
|
||||
val simsHashJobOutputDirectory: String = Config.simsHashJobOutputDirectory
|
||||
|
||||
override def graph(
|
||||
implicit sc: ScioContext,
|
||||
): SCollection[(Long, RightNode, Double)] =
|
||||
MultiTypeGraphUtil.getTruncatedMultiTypeGraph(noOlderThan = Duration.fromDays(14))
|
||||
|
||||
override def configurePipeline(sc: ScioContext, opts: DateRangeOptions): Unit = {
|
||||
implicit def scioContext: ScioContext = sc
|
||||
|
||||
// DAL.Environment variable for WriteExecs
|
||||
val dalEnv = if (isAdhoc) DAL.Environment.Dev else DAL.Environment.Prod
|
||||
|
||||
val sketches = computeSimHashSketchesForWeightedGraph(graph)
|
||||
.map {
|
||||
case (rightNode, sketch, norm) => RightNodeSimHashSketch(rightNode, sketch, norm)
|
||||
}
|
||||
|
||||
// Write SimHashSketches to DAL
|
||||
sketches
|
||||
.saveAsCustomOutput(
|
||||
name = "WriteSimHashSketches",
|
||||
DAL.writeSnapshot(
|
||||
rightNodeSimHashSnapshotDataset,
|
||||
PathLayout.FixedPath(
|
||||
((if (!isAdhoc)
|
||||
MultiTypeGraphUtil.RootThriftPath
|
||||
else
|
||||
MultiTypeGraphUtil.AdhocRootPath)
|
||||
+ simsHashJobOutputDirectory)),
|
||||
Instant.ofEpochMilli(opts.interval.getEndMillis - 1L),
|
||||
DiskFormat.Thrift(),
|
||||
environmentOverride = dalEnv
|
||||
)
|
||||
)
|
||||
}
|
||||
}
|
@ -1,33 +0,0 @@
|
||||
class Profile(Struct):
|
||||
project = Required(String)
|
||||
date = Required(String)
|
||||
environment = Default(String, 'dev')
|
||||
machine = Default(String, 'n2d-highmem-16')
|
||||
|
||||
job = Job(
|
||||
name='multi-type-graph-cosine-similarity-scio-adhoc-app',
|
||||
project='{{profile.project}}',
|
||||
staging_bucket='{{profile.project}}',
|
||||
service_account='{{profile.user_name}}-shdw@twttr-dp-svc-accounts.iam.gserviceaccount.com',
|
||||
region='us-central1',
|
||||
worker_config=WorkerConfig(
|
||||
num_workers=2,
|
||||
worker_machine_type='{{profile.machine}}',
|
||||
worker_disk_type=WorkerDiskType('HDD'),
|
||||
),
|
||||
extra_args={
|
||||
"environment": '{{profile.environment}}',
|
||||
"date": Quote('{{profile.date}}'),
|
||||
},
|
||||
deployment_config=BatchDeploymentConfig(
|
||||
role='{{profile.user_name}}',
|
||||
build_target='src/scala/com/twitter/simclusters_v2/scio/multi_type_graph/multi_type_graph_sims:multi-type-graph-cosine-similarity-scio-adhoc-app',
|
||||
gcp_deployment_credentials='/var/lib/tss/keys/{{profile.user_name}}/cloud/gcp/dp/shadow.json',
|
||||
statebird_config=StatebirdConfig(
|
||||
batch_width='PT1H',
|
||||
first_time='{{profile.date}}'
|
||||
)
|
||||
)
|
||||
)
|
||||
|
||||
jobs=[job]
|
Binary file not shown.
@ -1,39 +0,0 @@
|
||||
class Profile(Struct):
|
||||
project = Required(String)
|
||||
date = Required(String)
|
||||
environment = Default(String, 'prod')
|
||||
machine = Default(String, 'n2d-highmem-16')
|
||||
|
||||
job = Job(
|
||||
name='multi-type-graph-cosine-similarity-scio-batch-app',
|
||||
project='{{profile.project}}',
|
||||
staging_bucket='{{profile.project}}',
|
||||
service_account='{{profile.user_name}}-shdw@twttr-dp-svc-accounts.iam.gserviceaccount.com',
|
||||
region='us-central1',
|
||||
worker_config=WorkerConfig(
|
||||
num_workers=2,
|
||||
worker_machine_type='{{profile.machine}}',
|
||||
worker_disk_type=WorkerDiskType('HDD'),
|
||||
),
|
||||
extra_args={
|
||||
"environment": '{{profile.environment}}',
|
||||
"date": Quote('{{profile.date}}'),
|
||||
},
|
||||
deployment_config=BatchDeploymentConfig(
|
||||
role='{{profile.user_name}}',
|
||||
build_target='src/scala/com/twitter/simclusters_v2/scio/multi_type_graph/multi_type_graph_sims:multi-type-graph-cosine-similarity-scio-batch-app',
|
||||
gcp_deployment_credentials='/var/lib/tss/keys/{{profile.user_name}}/cloud/gcp/dp/shadow.json',
|
||||
environment='prod',
|
||||
statebird_config=StatebirdConfig(
|
||||
batch_width='P1W',
|
||||
first_time='{{profile.date}}'
|
||||
),
|
||||
workflow_config=WorkflowConfig(
|
||||
play=True,
|
||||
),
|
||||
timeout='PT50H'
|
||||
)
|
||||
)
|
||||
|
||||
jobs=[job]
|
||||
|
Binary file not shown.
@ -1,33 +0,0 @@
|
||||
class Profile(Struct):
|
||||
project = Required(String)
|
||||
date = Required(String)
|
||||
environment = Default(String, 'dev')
|
||||
machine = Default(String, 'n2d-highmem-16')
|
||||
|
||||
job = Job(
|
||||
name='multi-type-graph-sim-hash-scio-adhoc-app',
|
||||
project='{{profile.project}}',
|
||||
staging_bucket='{{profile.project}}',
|
||||
service_account='{{profile.user_name}}-shdw@twttr-dp-svc-accounts.iam.gserviceaccount.com',
|
||||
region='us-central1',
|
||||
worker_config=WorkerConfig(
|
||||
num_workers=2,
|
||||
worker_machine_type='{{profile.machine}}',
|
||||
worker_disk_type=WorkerDiskType('HDD'),
|
||||
),
|
||||
extra_args={
|
||||
"environment": '{{profile.environment}}',
|
||||
"date": Quote('{{profile.date}}'),
|
||||
},
|
||||
deployment_config=BatchDeploymentConfig(
|
||||
role='{{profile.user_name}}',
|
||||
build_target='src/scala/com/twitter/simclusters_v2/scio/multi_type_graph/multi_type_graph_sims:multi-type-graph-sim-hash-scio-adhoc-app',
|
||||
gcp_deployment_credentials='/var/lib/tss/keys/{{profile.user_name}}/cloud/gcp/dp/shadow.json',
|
||||
statebird_config=StatebirdConfig(
|
||||
batch_width='PT1H',
|
||||
first_time='{{profile.date}}'
|
||||
)
|
||||
)
|
||||
)
|
||||
|
||||
jobs=[job]
|
Binary file not shown.
@ -1,38 +0,0 @@
|
||||
class Profile(Struct):
|
||||
project = Required(String)
|
||||
date = Required(String)
|
||||
environment = Default(String, 'prod')
|
||||
machine = Default(String, 'n2d-highmem-16')
|
||||
|
||||
job = Job(
|
||||
name='multi-type-graph-sim-hash-scio-batch-app',
|
||||
project='{{profile.project}}',
|
||||
staging_bucket='{{profile.project}}',
|
||||
service_account='{{profile.user_name}}-shdw@twttr-dp-svc-accounts.iam.gserviceaccount.com',
|
||||
region='us-central1',
|
||||
worker_config=WorkerConfig(
|
||||
num_workers=2,
|
||||
worker_machine_type='{{profile.machine}}',
|
||||
worker_disk_type=WorkerDiskType('HDD'),
|
||||
),
|
||||
extra_args={
|
||||
"environment": '{{profile.environment}}',
|
||||
"date": Quote('{{profile.date}}'),
|
||||
},
|
||||
deployment_config=BatchDeploymentConfig(
|
||||
role='{{profile.user_name}}',
|
||||
build_target='src/scala/com/twitter/simclusters_v2/scio/multi_type_graph/multi_type_graph_sims:multi-type-graph-sim-hash-scio-batch-app',
|
||||
gcp_deployment_credentials='/var/lib/tss/keys/{{profile.user_name}}/cloud/gcp/dp/shadow.json',
|
||||
environment='prod',
|
||||
statebird_config=StatebirdConfig(
|
||||
batch_width='P1W',
|
||||
first_time='{{profile.date}}'
|
||||
),
|
||||
workflow_config=WorkflowConfig(
|
||||
play=True,
|
||||
),
|
||||
timeout='PT20H'
|
||||
)
|
||||
)
|
||||
|
||||
jobs=[job]
|
Binary file not shown.
Binary file not shown.
@ -1,24 +0,0 @@
|
||||
package com.twitter.simclusters_v2.score
|
||||
|
||||
import com.twitter.simclusters_v2.thriftscala.{ScoreId => ThriftScoreId, Score => ThriftScore}
|
||||
import com.twitter.storehaus.ReadableStore
|
||||
|
||||
/**
|
||||
* A wrapper class, used to aggregate the scores calculated by other score stores. It relies on the
|
||||
* results of other ScoreStores registered in the ScoreFacadeStore.
|
||||
*/
|
||||
trait AggregatedScoreStore extends ReadableStore[ThriftScoreId, ThriftScore] {
|
||||
|
||||
// The underlyingScoreStore relies on [[ScoreFacadeStore]] to finish the dependency injection.
|
||||
protected var scoreFacadeStore: ReadableStore[ThriftScoreId, ThriftScore] = ReadableStore.empty
|
||||
|
||||
/**
|
||||
* When registering this store in a ScoreFacadeStore, the facade store calls this function to
|
||||
* provide references to other score stores.
|
||||
*/
|
||||
private[score] def set(facadeStore: ReadableStore[ThriftScoreId, ThriftScore]): Unit = {
|
||||
this.synchronized {
|
||||
scoreFacadeStore = facadeStore
|
||||
}
|
||||
}
|
||||
}
|
@ -1,9 +0,0 @@
|
||||
scala_library(
|
||||
platform = "java8",
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"finagle/finagle-stats",
|
||||
"hermit/hermit-core/src/main/scala/com/twitter/hermit/store/common",
|
||||
"src/scala/com/twitter/simclusters_v2/stores",
|
||||
],
|
||||
)
|
BIN
src/scala/com/twitter/simclusters_v2/score/BUILD.docx
Normal file
BIN
src/scala/com/twitter/simclusters_v2/score/BUILD.docx
Normal file
Binary file not shown.
BIN
src/scala/com/twitter/simclusters_v2/score/Score.docx
Normal file
BIN
src/scala/com/twitter/simclusters_v2/score/Score.docx
Normal file
Binary file not shown.
@ -1,22 +0,0 @@
|
||||
package com.twitter.simclusters_v2.score
|
||||
|
||||
import com.twitter.simclusters_v2.thriftscala.{Score => ThriftScore}
|
||||
|
||||
/**
|
||||
* A uniform value type for all kinds of Calculation Score.
|
||||
**/
|
||||
case class Score(score: Double) {
|
||||
|
||||
implicit lazy val toThrift: ThriftScore = {
|
||||
ThriftScore(score)
|
||||
}
|
||||
}
|
||||
|
||||
object Score {
|
||||
|
||||
/**
|
||||
* Only support Double Type Thrift score
|
||||
*/
|
||||
implicit val fromThriftScore: ThriftScore => Score = { thriftScore => Score(thriftScore.score) }
|
||||
|
||||
}
|
BIN
src/scala/com/twitter/simclusters_v2/score/ScoreFacadeStore.docx
Normal file
BIN
src/scala/com/twitter/simclusters_v2/score/ScoreFacadeStore.docx
Normal file
Binary file not shown.
@ -1,103 +0,0 @@
|
||||
package com.twitter.simclusters_v2.score
|
||||
|
||||
import com.twitter.finagle.stats.BroadcastStatsReceiver
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.hermit.store.common.ObservedReadableStore
|
||||
import com.twitter.simclusters_v2.thriftscala.ScoringAlgorithm
|
||||
import com.twitter.simclusters_v2.thriftscala.{ScoreId => ThriftScoreId}
|
||||
import com.twitter.simclusters_v2.thriftscala.{Score => ThriftScore}
|
||||
import com.twitter.storehaus.ReadableStore
|
||||
import com.twitter.util.Future
|
||||
|
||||
/**
|
||||
* Provide a uniform access layer for all kind of Score.
|
||||
* @param readableStores readable stores indexed by the ScoringAlgorithm they implement
|
||||
*/
|
||||
class ScoreFacadeStore private (
|
||||
stores: Map[ScoringAlgorithm, ReadableStore[ThriftScoreId, ThriftScore]])
|
||||
extends ReadableStore[ThriftScoreId, ThriftScore] {
|
||||
|
||||
override def get(k: ThriftScoreId): Future[Option[ThriftScore]] = {
|
||||
findStore(k).get(k)
|
||||
}
|
||||
|
||||
// Override the multiGet for better batch performance.
|
||||
override def multiGet[K1 <: ThriftScoreId](ks: Set[K1]): Map[K1, Future[Option[ThriftScore]]] = {
|
||||
if (ks.isEmpty) {
|
||||
Map.empty
|
||||
} else {
|
||||
val head = ks.head
|
||||
val notSameType = ks.exists(k => k.algorithm != head.algorithm)
|
||||
if (!notSameType) {
|
||||
findStore(head).multiGet(ks)
|
||||
} else {
|
||||
// Generate a large amount temp objects.
|
||||
// For better performance, avoid querying the multiGet with more than one kind of embedding
|
||||
ks.groupBy(id => id.algorithm).flatMap {
|
||||
case (_, ks) =>
|
||||
findStore(ks.head).multiGet(ks)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// If not store mapping, fast return a IllegalArgumentException.
|
||||
private def findStore(id: ThriftScoreId): ReadableStore[ThriftScoreId, ThriftScore] = {
|
||||
stores.get(id.algorithm) match {
|
||||
case Some(store) => store
|
||||
case None =>
|
||||
throw new IllegalArgumentException(s"The Scoring Algorithm ${id.algorithm} doesn't exist.")
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
object ScoreFacadeStore {
|
||||
/*
|
||||
Build a ScoreFacadeStore which exposes stats for all requests (under "all") and per scoring algorithm:
|
||||
|
||||
score_facade_store/all/<observed readable store metrics for all requests>
|
||||
score_facade_store/<scoring algorithm>/<observed readable store metrics for this algorithm's requests>
|
||||
|
||||
Stores in aggregatedStores may reference stores in readableStores. An instance of ScoreFacadeStore
|
||||
is passed to them after instantiation.
|
||||
*/
|
||||
def buildWithMetrics(
|
||||
readableStores: Map[ScoringAlgorithm, ReadableStore[ThriftScoreId, ThriftScore]],
|
||||
aggregatedStores: Map[ScoringAlgorithm, AggregatedScoreStore],
|
||||
statsReceiver: StatsReceiver
|
||||
) = {
|
||||
val scopedStatsReceiver = statsReceiver.scope("score_facade_store")
|
||||
|
||||
def wrapStore(
|
||||
scoringAlgorithm: ScoringAlgorithm,
|
||||
store: ReadableStore[ThriftScoreId, ThriftScore]
|
||||
): ReadableStore[ThriftScoreId, ThriftScore] = {
|
||||
val sr = BroadcastStatsReceiver(
|
||||
Seq(
|
||||
scopedStatsReceiver.scope("all"),
|
||||
scopedStatsReceiver.scope(scoringAlgorithm.name)
|
||||
))
|
||||
ObservedReadableStore(store)(sr)
|
||||
}
|
||||
|
||||
val stores = (readableStores ++ aggregatedStores).map {
|
||||
case (algo, store) => algo -> wrapStore(algo, store)
|
||||
}
|
||||
val store = new ScoreFacadeStore(stores = stores)
|
||||
|
||||
/*
|
||||
AggregatedScores aggregate scores from multiple non-aggregated stores. They access these via the
|
||||
ScoreFacadeStore itself, and therefore must be passed an instance of it after it has been
|
||||
constructed.
|
||||
*/
|
||||
assert(
|
||||
readableStores.keySet.forall(algorithm => !aggregatedStores.keySet.contains(algorithm)),
|
||||
"Keys for stores are disjoint")
|
||||
|
||||
aggregatedStores.values.foreach(_.set(store))
|
||||
|
||||
store
|
||||
}
|
||||
|
||||
}
|
BIN
src/scala/com/twitter/simclusters_v2/score/ScoreId.docx
Normal file
BIN
src/scala/com/twitter/simclusters_v2/score/ScoreId.docx
Normal file
Binary file not shown.
@ -1,129 +0,0 @@
|
||||
package com.twitter.simclusters_v2.score
|
||||
|
||||
import com.twitter.simclusters_v2.common.SimClustersEmbeddingId._
|
||||
import com.twitter.simclusters_v2.thriftscala.{
|
||||
InternalId,
|
||||
ScoreInternalId,
|
||||
ScoringAlgorithm,
|
||||
SimClustersEmbeddingId,
|
||||
GenericPairScoreId => ThriftGenericPairScoreId,
|
||||
ScoreId => ThriftScoreId,
|
||||
SimClustersEmbeddingPairScoreId => ThriftSimClustersEmbeddingPairScoreId
|
||||
}
|
||||
|
||||
/**
|
||||
* A uniform Identifier type for all kinds of Calculation Score.
|
||||
**/
|
||||
trait ScoreId {
|
||||
|
||||
def algorithm: ScoringAlgorithm
|
||||
|
||||
/**
|
||||
* Convert to a Thrift object. Throw a exception if the operation is not override.
|
||||
*/
|
||||
implicit def toThrift: ThriftScoreId =
|
||||
throw new UnsupportedOperationException(s"ScoreId $this doesn't support Thrift format")
|
||||
}
|
||||
|
||||
object ScoreId {
|
||||
|
||||
implicit val fromThriftScoreId: ThriftScoreId => ScoreId = {
|
||||
case scoreId @ ThriftScoreId(_, ScoreInternalId.GenericPairScoreId(_)) =>
|
||||
PairScoreId.fromThriftScoreId(scoreId)
|
||||
case scoreId @ ThriftScoreId(_, ScoreInternalId.SimClustersEmbeddingPairScoreId(_)) =>
|
||||
SimClustersEmbeddingPairScoreId.fromThriftScoreId(scoreId)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Generic Internal pairwise id. Support all the subtypes in InternalId, which includes TweetId,
|
||||
* UserId, EntityId and more combination ids.
|
||||
**/
|
||||
trait PairScoreId extends ScoreId {
|
||||
|
||||
def id1: InternalId
|
||||
def id2: InternalId
|
||||
|
||||
override implicit lazy val toThrift: ThriftScoreId = {
|
||||
ThriftScoreId(
|
||||
algorithm,
|
||||
ScoreInternalId.GenericPairScoreId(ThriftGenericPairScoreId(id1, id2))
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
object PairScoreId {
|
||||
|
||||
// The default PairScoreId assume id1 <= id2. It used to increase the cache hit rate.
|
||||
def apply(algorithm: ScoringAlgorithm, id1: InternalId, id2: InternalId): PairScoreId = {
|
||||
if (internalIdOrdering.lteq(id1, id2)) {
|
||||
DefaultPairScoreId(algorithm, id1, id2)
|
||||
} else {
|
||||
DefaultPairScoreId(algorithm, id2, id1)
|
||||
}
|
||||
}
|
||||
|
||||
private case class DefaultPairScoreId(
|
||||
algorithm: ScoringAlgorithm,
|
||||
id1: InternalId,
|
||||
id2: InternalId)
|
||||
extends PairScoreId
|
||||
|
||||
implicit val fromThriftScoreId: ThriftScoreId => PairScoreId = {
|
||||
case ThriftScoreId(algorithm, ScoreInternalId.GenericPairScoreId(pairScoreId)) =>
|
||||
DefaultPairScoreId(algorithm, pairScoreId.id1, pairScoreId.id2)
|
||||
case ThriftScoreId(algorithm, ScoreInternalId.SimClustersEmbeddingPairScoreId(pairScoreId)) =>
|
||||
SimClustersEmbeddingPairScoreId(algorithm, pairScoreId.id1, pairScoreId.id2)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* ScoreId for a pair of SimClustersEmbedding.
|
||||
* Used for dot product, cosine similarity and other basic embedding operations.
|
||||
*/
|
||||
trait SimClustersEmbeddingPairScoreId extends PairScoreId {
|
||||
def embeddingId1: SimClustersEmbeddingId
|
||||
|
||||
def embeddingId2: SimClustersEmbeddingId
|
||||
|
||||
override def id1: InternalId = embeddingId1.internalId
|
||||
|
||||
override def id2: InternalId = embeddingId2.internalId
|
||||
|
||||
override implicit lazy val toThrift: ThriftScoreId = {
|
||||
ThriftScoreId(
|
||||
algorithm,
|
||||
ScoreInternalId.SimClustersEmbeddingPairScoreId(
|
||||
ThriftSimClustersEmbeddingPairScoreId(embeddingId1, embeddingId2))
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
object SimClustersEmbeddingPairScoreId {
|
||||
|
||||
// The default PairScoreId assume id1 <= id2. It used to increase the cache hit rate.
|
||||
def apply(
|
||||
algorithm: ScoringAlgorithm,
|
||||
id1: SimClustersEmbeddingId,
|
||||
id2: SimClustersEmbeddingId
|
||||
): SimClustersEmbeddingPairScoreId = {
|
||||
if (simClustersEmbeddingIdOrdering.lteq(id1, id2)) {
|
||||
DefaultSimClustersEmbeddingPairScoreId(algorithm, id1, id2)
|
||||
} else {
|
||||
DefaultSimClustersEmbeddingPairScoreId(algorithm, id2, id1)
|
||||
}
|
||||
}
|
||||
|
||||
private case class DefaultSimClustersEmbeddingPairScoreId(
|
||||
algorithm: ScoringAlgorithm,
|
||||
embeddingId1: SimClustersEmbeddingId,
|
||||
embeddingId2: SimClustersEmbeddingId)
|
||||
extends SimClustersEmbeddingPairScoreId
|
||||
|
||||
implicit val fromThriftScoreId: ThriftScoreId => SimClustersEmbeddingPairScoreId = {
|
||||
case ThriftScoreId(algorithm, ScoreInternalId.SimClustersEmbeddingPairScoreId(pairScoreId)) =>
|
||||
SimClustersEmbeddingPairScoreId(algorithm, pairScoreId.id1, pairScoreId.id2)
|
||||
}
|
||||
}
|
BIN
src/scala/com/twitter/simclusters_v2/score/ScoreStore.docx
Normal file
BIN
src/scala/com/twitter/simclusters_v2/score/ScoreStore.docx
Normal file
Binary file not shown.
@ -1,72 +0,0 @@
|
||||
package com.twitter.simclusters_v2.score
|
||||
|
||||
import com.twitter.simclusters_v2.thriftscala.{Score => ThriftScore, ScoreId => ThriftScoreId}
|
||||
import com.twitter.storehaus.ReadableStore
|
||||
import com.twitter.util.Future
|
||||
|
||||
/**
|
||||
* A Score Store is a readableStore with ScoreId as Key and Score as the Value.
|
||||
* It also needs to include the algorithm type.
|
||||
* A algorithm type should only be used by one Score Store in the application.
|
||||
*/
|
||||
trait ScoreStore[K <: ScoreId] extends ReadableStore[K, Score] {
|
||||
|
||||
def fromThriftScoreId: ThriftScoreId => K
|
||||
|
||||
// Convert to a Thrift version.
|
||||
def toThriftStore: ReadableStore[ThriftScoreId, ThriftScore] = {
|
||||
this
|
||||
.composeKeyMapping[ThriftScoreId](fromThriftScoreId)
|
||||
.mapValues(_.toThrift)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A generic Pairwise Score store.
|
||||
* Requires provide both left and right side feature hydration.
|
||||
*/
|
||||
trait PairScoreStore[K <: PairScoreId, K1, K2, V1, V2] extends ScoreStore[K] {
|
||||
|
||||
def compositeKey1: K => K1
|
||||
def compositeKey2: K => K2
|
||||
|
||||
// Left side feature hydration
|
||||
def underlyingStore1: ReadableStore[K1, V1]
|
||||
|
||||
// Right side feature hydration
|
||||
def underlyingStore2: ReadableStore[K2, V2]
|
||||
|
||||
def score: (V1, V2) => Future[Option[Double]]
|
||||
|
||||
override def get(k: K): Future[Option[Score]] = {
|
||||
for {
|
||||
vs <-
|
||||
Future.join(underlyingStore1.get(compositeKey1(k)), underlyingStore2.get(compositeKey2(k)))
|
||||
v <- vs match {
|
||||
case (Some(v1), Some(v2)) =>
|
||||
score(v1, v2)
|
||||
case _ =>
|
||||
Future.None
|
||||
}
|
||||
} yield {
|
||||
v.map(buildScore)
|
||||
}
|
||||
}
|
||||
|
||||
override def multiGet[KK <: K](ks: Set[KK]): Map[KK, Future[Option[Score]]] = {
|
||||
|
||||
val v1Map = underlyingStore1.multiGet(ks.map { k => compositeKey1(k) })
|
||||
val v2Map = underlyingStore2.multiGet(ks.map { k => compositeKey2(k) })
|
||||
|
||||
ks.map { k =>
|
||||
k -> Future.join(v1Map(compositeKey1(k)), v2Map(compositeKey2(k))).flatMap {
|
||||
case (Some(v1), Some(v2)) =>
|
||||
score(v1, v2).map(_.map(buildScore))
|
||||
case _ =>
|
||||
Future.value(None)
|
||||
}
|
||||
}.toMap
|
||||
}
|
||||
|
||||
private def buildScore(v: Double): Score = Score(v)
|
||||
}
|
Binary file not shown.
@ -1,201 +0,0 @@
|
||||
package com.twitter.simclusters_v2.score
|
||||
|
||||
import com.twitter.simclusters_v2.common.SimClustersEmbedding
|
||||
import com.twitter.simclusters_v2.thriftscala.{SimClustersEmbeddingId, ScoreId => ThriftScoreId}
|
||||
import com.twitter.storehaus.ReadableStore
|
||||
import com.twitter.util.Future
|
||||
|
||||
object SimClustersEmbeddingPairScoreStore {
|
||||
|
||||
/**
|
||||
* Internal Instance of a SimClusters Embedding based Pair Score store.
|
||||
*/
|
||||
private case class SimClustersEmbeddingInternalPairScoreStore(
|
||||
simClustersEmbeddingStore: ReadableStore[SimClustersEmbeddingId, SimClustersEmbedding],
|
||||
score: (SimClustersEmbedding, SimClustersEmbedding) => Future[Option[Double]])
|
||||
extends PairScoreStore[
|
||||
SimClustersEmbeddingPairScoreId,
|
||||
SimClustersEmbeddingId,
|
||||
SimClustersEmbeddingId,
|
||||
SimClustersEmbedding,
|
||||
SimClustersEmbedding
|
||||
] {
|
||||
|
||||
override val compositeKey1: SimClustersEmbeddingPairScoreId => SimClustersEmbeddingId =
|
||||
_.embeddingId1
|
||||
override val compositeKey2: SimClustersEmbeddingPairScoreId => SimClustersEmbeddingId =
|
||||
_.embeddingId2
|
||||
|
||||
override def underlyingStore1: ReadableStore[SimClustersEmbeddingId, SimClustersEmbedding] =
|
||||
simClustersEmbeddingStore
|
||||
|
||||
override def underlyingStore2: ReadableStore[SimClustersEmbeddingId, SimClustersEmbedding] =
|
||||
simClustersEmbeddingStore
|
||||
|
||||
override def fromThriftScoreId: ThriftScoreId => SimClustersEmbeddingPairScoreId =
|
||||
SimClustersEmbeddingPairScoreId.fromThriftScoreId
|
||||
}
|
||||
|
||||
def buildDotProductStore(
|
||||
simClustersEmbeddingStore: ReadableStore[SimClustersEmbeddingId, SimClustersEmbedding]
|
||||
): PairScoreStore[
|
||||
SimClustersEmbeddingPairScoreId,
|
||||
SimClustersEmbeddingId,
|
||||
SimClustersEmbeddingId,
|
||||
SimClustersEmbedding,
|
||||
SimClustersEmbedding
|
||||
] = {
|
||||
|
||||
def dotProduct: (SimClustersEmbedding, SimClustersEmbedding) => Future[Option[Double]] = {
|
||||
case (embedding1, embedding2) =>
|
||||
Future.value(Some(embedding1.dotProduct(embedding2)))
|
||||
}
|
||||
|
||||
SimClustersEmbeddingInternalPairScoreStore(
|
||||
simClustersEmbeddingStore,
|
||||
dotProduct
|
||||
)
|
||||
}
|
||||
|
||||
def buildCosineSimilarityStore(
|
||||
simClustersEmbeddingStore: ReadableStore[SimClustersEmbeddingId, SimClustersEmbedding]
|
||||
): PairScoreStore[
|
||||
SimClustersEmbeddingPairScoreId,
|
||||
SimClustersEmbeddingId,
|
||||
SimClustersEmbeddingId,
|
||||
SimClustersEmbedding,
|
||||
SimClustersEmbedding
|
||||
] = {
|
||||
|
||||
def cosineSimilarity: (SimClustersEmbedding, SimClustersEmbedding) => Future[Option[Double]] = {
|
||||
case (embedding1, embedding2) =>
|
||||
Future.value(Some(embedding1.cosineSimilarity(embedding2)))
|
||||
}
|
||||
|
||||
SimClustersEmbeddingInternalPairScoreStore(
|
||||
simClustersEmbeddingStore,
|
||||
cosineSimilarity
|
||||
)
|
||||
}
|
||||
|
||||
def buildLogCosineSimilarityStore(
|
||||
simClustersEmbeddingStore: ReadableStore[SimClustersEmbeddingId, SimClustersEmbedding]
|
||||
): PairScoreStore[
|
||||
SimClustersEmbeddingPairScoreId,
|
||||
SimClustersEmbeddingId,
|
||||
SimClustersEmbeddingId,
|
||||
SimClustersEmbedding,
|
||||
SimClustersEmbedding
|
||||
] = {
|
||||
|
||||
def logNormCosineSimilarity: (
|
||||
SimClustersEmbedding,
|
||||
SimClustersEmbedding
|
||||
) => Future[Option[Double]] = {
|
||||
case (embedding1, embedding2) =>
|
||||
Future.value(Some(embedding1.logNormCosineSimilarity(embedding2)))
|
||||
}
|
||||
|
||||
SimClustersEmbeddingInternalPairScoreStore(
|
||||
simClustersEmbeddingStore,
|
||||
logNormCosineSimilarity
|
||||
)
|
||||
}
|
||||
|
||||
def buildExpScaledCosineSimilarityStore(
|
||||
simClustersEmbeddingStore: ReadableStore[SimClustersEmbeddingId, SimClustersEmbedding]
|
||||
): PairScoreStore[
|
||||
SimClustersEmbeddingPairScoreId,
|
||||
SimClustersEmbeddingId,
|
||||
SimClustersEmbeddingId,
|
||||
SimClustersEmbedding,
|
||||
SimClustersEmbedding
|
||||
] = {
|
||||
|
||||
def expScaledCosineSimilarity: (
|
||||
SimClustersEmbedding,
|
||||
SimClustersEmbedding
|
||||
) => Future[Option[Double]] = {
|
||||
case (embedding1, embedding2) =>
|
||||
Future.value(Some(embedding1.expScaledCosineSimilarity(embedding2)))
|
||||
}
|
||||
|
||||
SimClustersEmbeddingInternalPairScoreStore(
|
||||
simClustersEmbeddingStore,
|
||||
expScaledCosineSimilarity
|
||||
)
|
||||
}
|
||||
|
||||
def buildJaccardSimilarityStore(
|
||||
simClustersEmbeddingStore: ReadableStore[SimClustersEmbeddingId, SimClustersEmbedding]
|
||||
): PairScoreStore[
|
||||
SimClustersEmbeddingPairScoreId,
|
||||
SimClustersEmbeddingId,
|
||||
SimClustersEmbeddingId,
|
||||
SimClustersEmbedding,
|
||||
SimClustersEmbedding
|
||||
] = {
|
||||
|
||||
def jaccardSimilarity: (
|
||||
SimClustersEmbedding,
|
||||
SimClustersEmbedding
|
||||
) => Future[Option[Double]] = {
|
||||
case (embedding1, embedding2) =>
|
||||
Future.value(Some(embedding1.jaccardSimilarity(embedding2)))
|
||||
}
|
||||
|
||||
SimClustersEmbeddingInternalPairScoreStore(
|
||||
simClustersEmbeddingStore,
|
||||
jaccardSimilarity
|
||||
)
|
||||
}
|
||||
|
||||
def buildEuclideanDistanceStore(
|
||||
simClustersEmbeddingStore: ReadableStore[SimClustersEmbeddingId, SimClustersEmbedding]
|
||||
): PairScoreStore[
|
||||
SimClustersEmbeddingPairScoreId,
|
||||
SimClustersEmbeddingId,
|
||||
SimClustersEmbeddingId,
|
||||
SimClustersEmbedding,
|
||||
SimClustersEmbedding
|
||||
] = {
|
||||
|
||||
def euclideanDistance: (
|
||||
SimClustersEmbedding,
|
||||
SimClustersEmbedding
|
||||
) => Future[Option[Double]] = {
|
||||
case (embedding1, embedding2) =>
|
||||
Future.value(Some(embedding1.euclideanDistance(embedding2)))
|
||||
}
|
||||
|
||||
SimClustersEmbeddingInternalPairScoreStore(
|
||||
simClustersEmbeddingStore,
|
||||
euclideanDistance
|
||||
)
|
||||
}
|
||||
|
||||
def buildManhattanDistanceStore(
|
||||
simClustersEmbeddingStore: ReadableStore[SimClustersEmbeddingId, SimClustersEmbedding]
|
||||
): PairScoreStore[
|
||||
SimClustersEmbeddingPairScoreId,
|
||||
SimClustersEmbeddingId,
|
||||
SimClustersEmbeddingId,
|
||||
SimClustersEmbedding,
|
||||
SimClustersEmbedding
|
||||
] = {
|
||||
|
||||
def manhattanDistance: (
|
||||
SimClustersEmbedding,
|
||||
SimClustersEmbedding
|
||||
) => Future[Option[Double]] = {
|
||||
case (embedding1, embedding2) =>
|
||||
Future.value(Some(embedding1.manhattanDistance(embedding2)))
|
||||
}
|
||||
|
||||
SimClustersEmbeddingInternalPairScoreStore(
|
||||
simClustersEmbeddingStore,
|
||||
manhattanDistance
|
||||
)
|
||||
}
|
||||
|
||||
}
|
Binary file not shown.
@ -1,84 +0,0 @@
|
||||
package com.twitter.simclusters_v2.score
|
||||
|
||||
import com.twitter.simclusters_v2.score.WeightedSumAggregatedScoreStore.WeightedSumAggregatedScoreParameter
|
||||
import com.twitter.simclusters_v2.thriftscala.{
|
||||
EmbeddingType,
|
||||
GenericPairScoreId,
|
||||
ModelVersion,
|
||||
ScoreInternalId,
|
||||
ScoringAlgorithm,
|
||||
SimClustersEmbeddingId,
|
||||
Score => ThriftScore,
|
||||
ScoreId => ThriftScoreId,
|
||||
SimClustersEmbeddingPairScoreId => ThriftSimClustersEmbeddingPairScoreId
|
||||
}
|
||||
import com.twitter.util.Future
|
||||
|
||||
/**
|
||||
* A generic store wrapper to aggregate the scores of N underlying stores in a weighted fashion.
|
||||
*
|
||||
*/
|
||||
case class WeightedSumAggregatedScoreStore(parameters: Seq[WeightedSumAggregatedScoreParameter])
|
||||
extends AggregatedScoreStore {
|
||||
|
||||
override def get(k: ThriftScoreId): Future[Option[ThriftScore]] = {
|
||||
val underlyingScores = parameters.map { parameter =>
|
||||
scoreFacadeStore
|
||||
.get(ThriftScoreId(parameter.scoreAlgorithm, parameter.idTransform(k.internalId)))
|
||||
.map(_.map(s => parameter.scoreTransform(s.score) * parameter.weight))
|
||||
}
|
||||
Future.collect(underlyingScores).map { scores =>
|
||||
if (scores.exists(_.nonEmpty)) {
|
||||
val newScore = scores.foldLeft(0.0) {
|
||||
case (sum, maybeScore) =>
|
||||
sum + maybeScore.getOrElse(0.0)
|
||||
}
|
||||
Some(ThriftScore(score = newScore))
|
||||
} else {
|
||||
// Return None if all of the underlying score is None.
|
||||
None
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object WeightedSumAggregatedScoreStore {
|
||||
|
||||
/**
|
||||
* The parameter of WeightedSumAggregatedScoreStore. Create 0 to N parameters for a WeightedSum
|
||||
* AggregatedScore Store. Please evaluate the performance before productionization any new score.
|
||||
*
|
||||
* @param scoreAlgorithm the underlying score algorithm name
|
||||
* @param weight contribution to weighted sum of this sub-score
|
||||
* @param idTransform transform the source ScoreInternalId to underlying score InternalId.
|
||||
* @param scoreTransform function to apply to sub-score before adding to weighted sum
|
||||
*/
|
||||
case class WeightedSumAggregatedScoreParameter(
|
||||
scoreAlgorithm: ScoringAlgorithm,
|
||||
weight: Double,
|
||||
idTransform: ScoreInternalId => ScoreInternalId,
|
||||
scoreTransform: Double => Double = identityScoreTransform)
|
||||
|
||||
val SameTypeScoreInternalIdTransform: ScoreInternalId => ScoreInternalId = { id => id }
|
||||
val identityScoreTransform: Double => Double = { score => score }
|
||||
|
||||
// Convert Generic Internal Id to a SimClustersEmbeddingId
|
||||
def genericPairScoreIdToSimClustersEmbeddingPairScoreId(
|
||||
embeddingType1: EmbeddingType,
|
||||
embeddingType2: EmbeddingType,
|
||||
modelVersion: ModelVersion
|
||||
): ScoreInternalId => ScoreInternalId = {
|
||||
case id: ScoreInternalId.GenericPairScoreId =>
|
||||
ScoreInternalId.SimClustersEmbeddingPairScoreId(
|
||||
ThriftSimClustersEmbeddingPairScoreId(
|
||||
SimClustersEmbeddingId(embeddingType1, modelVersion, id.genericPairScoreId.id1),
|
||||
SimClustersEmbeddingId(embeddingType2, modelVersion, id.genericPairScoreId.id2)
|
||||
))
|
||||
}
|
||||
|
||||
val simClustersEmbeddingPairScoreIdToGenericPairScoreId: ScoreInternalId => ScoreInternalId = {
|
||||
case ScoreInternalId.SimClustersEmbeddingPairScoreId(simClustersId) =>
|
||||
ScoreInternalId.GenericPairScoreId(
|
||||
GenericPairScoreId(simClustersId.id1.internalId, simClustersId.id2.internalId))
|
||||
}
|
||||
}
|
@ -1,14 +0,0 @@
|
||||
scala_library(
|
||||
platform = "java8",
|
||||
tags = ["bazel-compatible"],
|
||||
dependencies = [
|
||||
"3rdparty/jvm/com/twitter/storehaus:core",
|
||||
"hermit/hermit-core/src/main/scala/com/twitter/hermit/store/common",
|
||||
"src/scala/com/twitter/simclusters_v2/common",
|
||||
"src/scala/com/twitter/storehaus_internal/manhattan",
|
||||
"src/scala/com/twitter/storehaus_internal/util",
|
||||
"src/scala/com/twitter/wtf/scalding/jobs/injection",
|
||||
"src/thrift/com/twitter/recos/entities:entities-thrift-scala",
|
||||
"storage/clients/manhattan/client/src/main/scala",
|
||||
],
|
||||
)
|
BIN
src/scala/com/twitter/simclusters_v2/stores/BUILD.docx
Normal file
BIN
src/scala/com/twitter/simclusters_v2/stores/BUILD.docx
Normal file
Binary file not shown.
Binary file not shown.
@ -1,96 +0,0 @@
|
||||
package com.twitter.simclusters_v2.stores
|
||||
|
||||
import com.twitter.simclusters_v2.common.ClusterId
|
||||
import com.twitter.simclusters_v2.common.SimClustersEmbedding
|
||||
import com.twitter.simclusters_v2.thriftscala.ClusterDetails
|
||||
import com.twitter.simclusters_v2.thriftscala.InternalId
|
||||
import com.twitter.simclusters_v2.thriftscala.ModelVersion
|
||||
import com.twitter.simclusters_v2.thriftscala.SimClustersEmbeddingId
|
||||
import com.twitter.storehaus.ReadableStore
|
||||
import com.twitter.util.Future
|
||||
|
||||
/**
|
||||
* Transfer a Entity SimClustersEmbedding to a language filtered embedding.
|
||||
* The new embedding only contains clusters whose main language is the same as the language field in
|
||||
* the SimClustersEmbeddingId.
|
||||
*
|
||||
* This store is special designed for Topic Tweet and Topic Follow Prompt.
|
||||
* Only support new Ids whose internalId is LocaleEntityId.
|
||||
*/
|
||||
@deprecated
|
||||
case class LanguageFilteredLocaleEntityEmbeddingStore(
|
||||
underlyingStore: ReadableStore[SimClustersEmbeddingId, SimClustersEmbedding],
|
||||
clusterDetailsStore: ReadableStore[(ModelVersion, ClusterId), ClusterDetails],
|
||||
composeKeyMapping: SimClustersEmbeddingId => SimClustersEmbeddingId)
|
||||
extends ReadableStore[SimClustersEmbeddingId, SimClustersEmbedding] {
|
||||
|
||||
import LanguageFilteredLocaleEntityEmbeddingStore._
|
||||
|
||||
override def get(k: SimClustersEmbeddingId): Future[Option[SimClustersEmbedding]] = {
|
||||
for {
|
||||
maybeEmbedding <- underlyingStore.get(composeKeyMapping(k))
|
||||
maybeFilteredEmbedding <- maybeEmbedding match {
|
||||
case Some(embedding) =>
|
||||
embeddingsLanguageFilter(k, embedding).map(Some(_))
|
||||
case None =>
|
||||
Future.None
|
||||
}
|
||||
} yield maybeFilteredEmbedding
|
||||
}
|
||||
|
||||
private def embeddingsLanguageFilter(
|
||||
sourceEmbeddingId: SimClustersEmbeddingId,
|
||||
simClustersEmbedding: SimClustersEmbedding
|
||||
): Future[SimClustersEmbedding] = {
|
||||
val language = getLanguage(sourceEmbeddingId)
|
||||
val modelVersion = sourceEmbeddingId.modelVersion
|
||||
|
||||
val clusterDetailKeys = simClustersEmbedding.sortedClusterIds.map { clusterId =>
|
||||
(modelVersion, clusterId)
|
||||
}.toSet
|
||||
|
||||
Future
|
||||
.collect {
|
||||
clusterDetailsStore.multiGet(clusterDetailKeys)
|
||||
}.map { clusterDetailsMap =>
|
||||
simClustersEmbedding.embedding.filter {
|
||||
case (clusterId, _) =>
|
||||
isDominantLanguage(
|
||||
language,
|
||||
clusterDetailsMap.getOrElse((modelVersion, clusterId), None))
|
||||
}
|
||||
}.map(SimClustersEmbedding(_))
|
||||
}
|
||||
|
||||
private def isDominantLanguage(
|
||||
requestLang: String,
|
||||
clusterDetails: Option[ClusterDetails]
|
||||
): Boolean =
|
||||
clusterDetails match {
|
||||
case Some(details) =>
|
||||
val dominantLanguage =
|
||||
details.languageToFractionDeviceLanguage.map { langMap =>
|
||||
langMap.maxBy {
|
||||
case (_, score) => score
|
||||
}._1
|
||||
}
|
||||
|
||||
dominantLanguage.exists(_.equalsIgnoreCase(requestLang))
|
||||
case _ => true
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
object LanguageFilteredLocaleEntityEmbeddingStore {
|
||||
|
||||
def getLanguage(simClustersEmbeddingId: SimClustersEmbeddingId): String = {
|
||||
simClustersEmbeddingId match {
|
||||
case SimClustersEmbeddingId(_, _, InternalId.LocaleEntityId(localeEntityId)) =>
|
||||
localeEntityId.language
|
||||
case _ =>
|
||||
throw new IllegalArgumentException(
|
||||
s"The Id $simClustersEmbeddingId doesn't contain Locale info")
|
||||
}
|
||||
}
|
||||
|
||||
}
|
Binary file not shown.
@ -1,287 +0,0 @@
|
||||
package com.twitter.simclusters_v2.stores
|
||||
import com.twitter.bijection.Bufferable
|
||||
import com.twitter.bijection.Injection
|
||||
import com.twitter.bijection.scrooge.CompactScalaCodec
|
||||
import com.twitter.simclusters_v2.common.Language
|
||||
import com.twitter.simclusters_v2.thriftscala.ClustersUserIsInterestedIn
|
||||
import com.twitter.simclusters_v2.thriftscala.LeftNode
|
||||
import com.twitter.simclusters_v2.thriftscala.NounWithFrequencyList
|
||||
import com.twitter.simclusters_v2.thriftscala.RightNode
|
||||
import com.twitter.simclusters_v2.thriftscala.RightNodeTypeStruct
|
||||
import com.twitter.simclusters_v2.thriftscala.RightNodeWithEdgeWeightList
|
||||
import com.twitter.simclusters_v2.thriftscala.SimilarRightNodes
|
||||
import com.twitter.simclusters_v2.thriftscala.CandidateTweetsList
|
||||
import com.twitter.storage.client.manhattan.kv.ManhattanKVClientMtlsParams
|
||||
import com.twitter.storehaus.ReadableStore
|
||||
import com.twitter.storehaus_internal.manhattan.Apollo
|
||||
import com.twitter.storehaus_internal.manhattan.ManhattanRO
|
||||
import com.twitter.storehaus_internal.manhattan.ManhattanROConfig
|
||||
import com.twitter.storehaus_internal.util.ApplicationID
|
||||
import com.twitter.storehaus_internal.util.DatasetName
|
||||
import com.twitter.storehaus_internal.util.HDFSPath
|
||||
import com.twitter.scalding_internal.multiformat.format.keyval.KeyValInjection.Long2BigEndian
|
||||
import com.twitter.simclusters_v2.thriftscala.FullClusterId
|
||||
import com.twitter.simclusters_v2.thriftscala.TopKTweetsWithScores
|
||||
|
||||
object MultiTypeGraphStore {
|
||||
|
||||
implicit val leftNodesInject: Injection[LeftNode, Array[Byte]] =
|
||||
CompactScalaCodec(LeftNode)
|
||||
implicit val truncatedMultiTypeGraphInject: Injection[RightNodeWithEdgeWeightList, Array[Byte]] =
|
||||
CompactScalaCodec(RightNodeWithEdgeWeightList)
|
||||
implicit val topKNounsListInject: Injection[NounWithFrequencyList, Array[Byte]] =
|
||||
CompactScalaCodec(NounWithFrequencyList)
|
||||
implicit val rightNodesStructInject: Injection[RightNodeTypeStruct, Array[Byte]] =
|
||||
CompactScalaCodec(RightNodeTypeStruct)
|
||||
implicit val similarRightNodesStructInject: Injection[SimilarRightNodes, Array[Byte]] =
|
||||
CompactScalaCodec(SimilarRightNodes)
|
||||
implicit val rightNodesInject: Injection[RightNode, Array[Byte]] =
|
||||
CompactScalaCodec(RightNode)
|
||||
implicit val tweetCandidatesInject: Injection[CandidateTweetsList, Array[Byte]] =
|
||||
CompactScalaCodec(CandidateTweetsList)
|
||||
implicit val fullClusterIdInject: Injection[FullClusterId, Array[Byte]] =
|
||||
CompactScalaCodec(FullClusterId)
|
||||
implicit val topKTweetsWithScoresInject: Injection[TopKTweetsWithScores, Array[Byte]] =
|
||||
CompactScalaCodec(TopKTweetsWithScores)
|
||||
implicit val clustersUserIsInterestedInInjection: Injection[ClustersUserIsInterestedIn, Array[
|
||||
Byte
|
||||
]] =
|
||||
CompactScalaCodec(ClustersUserIsInterestedIn)
|
||||
|
||||
private val appId = "multi_type_simclusters"
|
||||
|
||||
def getTruncatedMultiTypeGraphRightNodesForUser(
|
||||
mhMtlsParams: ManhattanKVClientMtlsParams
|
||||
): ReadableStore[LeftNode, RightNodeWithEdgeWeightList] = {
|
||||
ManhattanRO.getReadableStoreWithMtls[LeftNode, RightNodeWithEdgeWeightList](
|
||||
ManhattanROConfig(
|
||||
HDFSPath(""),
|
||||
ApplicationID(appId),
|
||||
DatasetName("mts_user_truncated_graph"),
|
||||
Apollo
|
||||
),
|
||||
mhMtlsParams
|
||||
)
|
||||
}
|
||||
|
||||
def getTopKNounsForRightNodeType(
|
||||
mhMtlsParams: ManhattanKVClientMtlsParams
|
||||
): ReadableStore[RightNodeTypeStruct, NounWithFrequencyList] = {
|
||||
ManhattanRO.getReadableStoreWithMtls[RightNodeTypeStruct, NounWithFrequencyList](
|
||||
ManhattanROConfig(
|
||||
HDFSPath(""),
|
||||
ApplicationID(appId),
|
||||
DatasetName("mts_topk_frequent_nouns"),
|
||||
Apollo
|
||||
),
|
||||
mhMtlsParams
|
||||
)
|
||||
}
|
||||
|
||||
def getTopKSimilarRightNodes(
|
||||
mhMtlsParams: ManhattanKVClientMtlsParams
|
||||
): ReadableStore[RightNode, SimilarRightNodes] = {
|
||||
ManhattanRO.getReadableStoreWithMtls[RightNode, SimilarRightNodes](
|
||||
ManhattanROConfig(
|
||||
HDFSPath(""),
|
||||
ApplicationID(appId),
|
||||
DatasetName("mts_topk_similar_right_nodes_scio"),
|
||||
Apollo
|
||||
),
|
||||
mhMtlsParams
|
||||
)
|
||||
}
|
||||
|
||||
def getOfflineTweetMTSCandidateStore(
|
||||
mhMtlsParams: ManhattanKVClientMtlsParams
|
||||
): ReadableStore[Long, CandidateTweetsList] = {
|
||||
ManhattanRO.getReadableStoreWithMtls[Long, CandidateTweetsList](
|
||||
ManhattanROConfig(
|
||||
HDFSPath(""),
|
||||
ApplicationID(appId),
|
||||
DatasetName("offline_tweet_recommendations_from_mts_consumer_embeddings"),
|
||||
Apollo
|
||||
),
|
||||
mhMtlsParams
|
||||
)
|
||||
}
|
||||
|
||||
def getOfflineTweet2020CandidateStore(
|
||||
mhMtlsParams: ManhattanKVClientMtlsParams
|
||||
): ReadableStore[Long, CandidateTweetsList] = {
|
||||
ManhattanRO.getReadableStoreWithMtls[Long, CandidateTweetsList](
|
||||
ManhattanROConfig(
|
||||
HDFSPath(""),
|
||||
ApplicationID(appId),
|
||||
DatasetName("offline_tweet_recommendations_from_interestedin_2020"),
|
||||
Apollo
|
||||
),
|
||||
mhMtlsParams
|
||||
)
|
||||
}
|
||||
|
||||
def getVideoViewBasedClusterTopKTweets(
|
||||
mhMtlsParams: ManhattanKVClientMtlsParams
|
||||
): ReadableStore[FullClusterId, TopKTweetsWithScores] = {
|
||||
ManhattanRO
|
||||
.getReadableStoreWithMtls[FullClusterId, TopKTweetsWithScores](
|
||||
ManhattanROConfig(
|
||||
HDFSPath(""),
|
||||
ApplicationID(appId),
|
||||
DatasetName("video_view_based_cluster_to_tweet_index"),
|
||||
Apollo
|
||||
),
|
||||
mhMtlsParams
|
||||
)
|
||||
}
|
||||
|
||||
def getRetweetBasedClusterTopKTweets(
|
||||
mhMtlsParams: ManhattanKVClientMtlsParams
|
||||
): ReadableStore[FullClusterId, TopKTweetsWithScores] = {
|
||||
ManhattanRO
|
||||
.getReadableStoreWithMtls[FullClusterId, TopKTweetsWithScores](
|
||||
ManhattanROConfig(
|
||||
HDFSPath(""),
|
||||
ApplicationID(appId),
|
||||
DatasetName("retweet_based_simclusters_cluster_to_tweet_index"),
|
||||
Apollo
|
||||
),
|
||||
mhMtlsParams
|
||||
)
|
||||
}
|
||||
|
||||
def getReplyBasedClusterTopKTweets(
|
||||
mhMtlsParams: ManhattanKVClientMtlsParams
|
||||
): ReadableStore[FullClusterId, TopKTweetsWithScores] = {
|
||||
ManhattanRO
|
||||
.getReadableStoreWithMtls[FullClusterId, TopKTweetsWithScores](
|
||||
ManhattanROConfig(
|
||||
HDFSPath(""),
|
||||
ApplicationID(appId),
|
||||
DatasetName("reply_based_simclusters_cluster_to_tweet_index"),
|
||||
Apollo
|
||||
),
|
||||
mhMtlsParams
|
||||
)
|
||||
}
|
||||
|
||||
def getPushOpenBasedClusterTopKTweets(
|
||||
mhMtlsParams: ManhattanKVClientMtlsParams
|
||||
): ReadableStore[FullClusterId, TopKTweetsWithScores] = {
|
||||
ManhattanRO
|
||||
.getReadableStoreWithMtls[FullClusterId, TopKTweetsWithScores](
|
||||
ManhattanROConfig(
|
||||
HDFSPath(""),
|
||||
ApplicationID(appId),
|
||||
DatasetName("push_open_based_simclusters_cluster_to_tweet_index"),
|
||||
Apollo
|
||||
),
|
||||
mhMtlsParams
|
||||
)
|
||||
}
|
||||
|
||||
def getAdsFavBasedClusterTopKTweets(
|
||||
mhMtlsParams: ManhattanKVClientMtlsParams
|
||||
): ReadableStore[FullClusterId, TopKTweetsWithScores] = {
|
||||
ManhattanRO
|
||||
.getReadableStoreWithMtls[FullClusterId, TopKTweetsWithScores](
|
||||
ManhattanROConfig(
|
||||
HDFSPath(""),
|
||||
ApplicationID(appId),
|
||||
DatasetName("ads_fav_based_simclusters_cluster_to_tweet_index"),
|
||||
Apollo
|
||||
),
|
||||
mhMtlsParams
|
||||
)
|
||||
}
|
||||
|
||||
def getAdsFavClickBasedClusterTopKTweets(
|
||||
mhMtlsParams: ManhattanKVClientMtlsParams
|
||||
): ReadableStore[FullClusterId, TopKTweetsWithScores] = {
|
||||
ManhattanRO
|
||||
.getReadableStoreWithMtls[FullClusterId, TopKTweetsWithScores](
|
||||
ManhattanROConfig(
|
||||
HDFSPath(""),
|
||||
ApplicationID(appId),
|
||||
DatasetName("ads_fav_click_based_simclusters_cluster_to_tweet_index"),
|
||||
Apollo
|
||||
),
|
||||
mhMtlsParams
|
||||
)
|
||||
}
|
||||
|
||||
def getFTRPop1000BasedClusterTopKTweets(
|
||||
mhMtlsParams: ManhattanKVClientMtlsParams
|
||||
): ReadableStore[FullClusterId, TopKTweetsWithScores] = {
|
||||
ManhattanRO
|
||||
.getReadableStoreWithMtls[FullClusterId, TopKTweetsWithScores](
|
||||
ManhattanROConfig(
|
||||
HDFSPath(""),
|
||||
ApplicationID(appId),
|
||||
DatasetName("ftr_pop1000_rank_decay_1_1_cluster_to_tweet_index"),
|
||||
Apollo
|
||||
),
|
||||
mhMtlsParams
|
||||
)
|
||||
}
|
||||
|
||||
def getFTRPop10000BasedClusterTopKTweets(
|
||||
mhMtlsParams: ManhattanKVClientMtlsParams
|
||||
): ReadableStore[FullClusterId, TopKTweetsWithScores] = {
|
||||
ManhattanRO
|
||||
.getReadableStoreWithMtls[FullClusterId, TopKTweetsWithScores](
|
||||
ManhattanROConfig(
|
||||
HDFSPath(""),
|
||||
ApplicationID(appId),
|
||||
DatasetName("ftr_pop10000_rank_decay_1_1_cluster_to_tweet_index"),
|
||||
Apollo
|
||||
),
|
||||
mhMtlsParams
|
||||
)
|
||||
}
|
||||
|
||||
def getOONFTRPop1000BasedClusterTopKTweets(
|
||||
mhMtlsParams: ManhattanKVClientMtlsParams
|
||||
): ReadableStore[FullClusterId, TopKTweetsWithScores] = {
|
||||
ManhattanRO
|
||||
.getReadableStoreWithMtls[FullClusterId, TopKTweetsWithScores](
|
||||
ManhattanROConfig(
|
||||
HDFSPath(""),
|
||||
ApplicationID(appId),
|
||||
DatasetName("oon_ftr_pop1000_rnkdecay_cluster_to_tweet_index"),
|
||||
Apollo
|
||||
),
|
||||
mhMtlsParams
|
||||
)
|
||||
}
|
||||
|
||||
def getOfflineLogFavBasedTweetBasedClusterTopKTweets(
|
||||
mhMtlsParams: ManhattanKVClientMtlsParams
|
||||
): ReadableStore[FullClusterId, TopKTweetsWithScores] = {
|
||||
ManhattanRO
|
||||
.getReadableStoreWithMtls[FullClusterId, TopKTweetsWithScores](
|
||||
ManhattanROConfig(
|
||||
HDFSPath(""),
|
||||
ApplicationID(appId),
|
||||
DatasetName("decayed_sum_cluster_to_tweet_index"),
|
||||
Apollo
|
||||
),
|
||||
mhMtlsParams
|
||||
)
|
||||
}
|
||||
|
||||
def getGlobalSimClustersLanguageEmbeddings(
|
||||
mhMtlsParams: ManhattanKVClientMtlsParams
|
||||
): ReadableStore[Language, ClustersUserIsInterestedIn] = {
|
||||
ManhattanRO
|
||||
.getReadableStoreWithMtls[Language, ClustersUserIsInterestedIn](
|
||||
ManhattanROConfig(
|
||||
HDFSPath(""),
|
||||
ApplicationID(appId),
|
||||
DatasetName("global_simclusters_language_embeddings"),
|
||||
Apollo
|
||||
),
|
||||
mhMtlsParams
|
||||
)
|
||||
}
|
||||
}
|
Binary file not shown.
@ -1,120 +0,0 @@
|
||||
package com.twitter.simclusters_v2.stores
|
||||
|
||||
import com.twitter.decider.Decider
|
||||
import com.twitter.finagle.stats.StatsReceiver
|
||||
import com.twitter.hermit.store.common.DeciderableReadableStore
|
||||
import com.twitter.servo.decider.DeciderKeyEnum
|
||||
import com.twitter.simclusters_v2.common.DeciderGateBuilderWithIdHashing
|
||||
import com.twitter.simclusters_v2.common.SimClustersEmbedding
|
||||
import com.twitter.simclusters_v2.thriftscala.EmbeddingType
|
||||
import com.twitter.simclusters_v2.thriftscala.ModelVersion
|
||||
import com.twitter.simclusters_v2.thriftscala.SimClustersEmbeddingId
|
||||
import com.twitter.storehaus.ReadableStore
|
||||
import com.twitter.util.Future
|
||||
|
||||
/**
|
||||
* Facade of all SimClusters Embedding Store.
|
||||
* Provide a uniform access layer for all kind of SimClusters Embedding.
|
||||
*/
|
||||
case class SimClustersEmbeddingStore(
|
||||
stores: Map[
|
||||
(EmbeddingType, ModelVersion),
|
||||
ReadableStore[SimClustersEmbeddingId, SimClustersEmbedding]
|
||||
]) extends ReadableStore[SimClustersEmbeddingId, SimClustersEmbedding] {
|
||||
|
||||
private val lookupStores =
|
||||
stores
|
||||
.groupBy(_._1._1).mapValues(_.map {
|
||||
case ((_, modelVersion), store) =>
|
||||
modelVersion -> store
|
||||
})
|
||||
|
||||
override def get(k: SimClustersEmbeddingId): Future[Option[SimClustersEmbedding]] = {
|
||||
findStore(k) match {
|
||||
case Some(store) => store.get(k)
|
||||
case None => Future.None
|
||||
}
|
||||
}
|
||||
|
||||
// Override the multiGet for better batch performance.
|
||||
override def multiGet[K1 <: SimClustersEmbeddingId](
|
||||
ks: Set[K1]
|
||||
): Map[K1, Future[Option[SimClustersEmbedding]]] = {
|
||||
if (ks.isEmpty) {
|
||||
Map.empty
|
||||
} else {
|
||||
val head = ks.head
|
||||
val notSameType =
|
||||
ks.exists(k => k.embeddingType != head.embeddingType || k.modelVersion != head.modelVersion)
|
||||
if (!notSameType) {
|
||||
findStore(head) match {
|
||||
case Some(store) => store.multiGet(ks)
|
||||
case None => ks.map(_ -> Future.None).toMap
|
||||
}
|
||||
} else {
|
||||
// Generate a large amount temp objects.
|
||||
// For better performance, avoid querying the multiGet with more than one kind of embedding
|
||||
ks.groupBy(id => (id.embeddingType, id.modelVersion)).flatMap {
|
||||
case ((_, _), ks) =>
|
||||
findStore(ks.head) match {
|
||||
case Some(store) => store.multiGet(ks)
|
||||
case None => ks.map(_ -> Future.None).toMap
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private def findStore(
|
||||
id: SimClustersEmbeddingId
|
||||
): Option[ReadableStore[SimClustersEmbeddingId, SimClustersEmbedding]] = {
|
||||
lookupStores.get(id.embeddingType).flatMap(_.get(id.modelVersion))
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
object SimClustersEmbeddingStore {
|
||||
/*
|
||||
Build a SimClustersEmbeddingStore which wraps all stores in DeciderableReadableStore
|
||||
*/
|
||||
def buildWithDecider(
|
||||
underlyingStores: Map[
|
||||
(EmbeddingType, ModelVersion),
|
||||
ReadableStore[SimClustersEmbeddingId, SimClustersEmbedding]
|
||||
],
|
||||
decider: Decider,
|
||||
statsReceiver: StatsReceiver
|
||||
): ReadableStore[SimClustersEmbeddingId, SimClustersEmbedding] = {
|
||||
// To allow for lazy adding of decider config to enable / disable stores, if a value is not found
|
||||
// fall back on returning true (equivalent to availability of 10000)
|
||||
// This overrides default availability of 0 when not decider value is not found
|
||||
val deciderGateBuilder = new DeciderGateBuilderWithIdHashing(decider.orElse(Decider.True))
|
||||
|
||||
val deciderKeyEnum = new DeciderKeyEnum {
|
||||
underlyingStores.keySet.map(key => Value(s"enable_${key._1.name}_${key._2.name}"))
|
||||
}
|
||||
|
||||
def wrapStore(
|
||||
embeddingType: EmbeddingType,
|
||||
modelVersion: ModelVersion,
|
||||
store: ReadableStore[SimClustersEmbeddingId, SimClustersEmbedding]
|
||||
): ReadableStore[SimClustersEmbeddingId, SimClustersEmbedding] = {
|
||||
val gate = deciderGateBuilder.idGateWithHashing[SimClustersEmbeddingId](
|
||||
deciderKeyEnum.withName(s"enable_${embeddingType.name}_${modelVersion.name}"))
|
||||
|
||||
DeciderableReadableStore(
|
||||
underlying = store,
|
||||
gate = gate,
|
||||
statsReceiver = statsReceiver.scope(embeddingType.name, modelVersion.name)
|
||||
)
|
||||
}
|
||||
|
||||
val stores = underlyingStores.map {
|
||||
case ((embeddingType, modelVersion), store) =>
|
||||
(embeddingType, modelVersion) -> wrapStore(embeddingType, modelVersion, store)
|
||||
}
|
||||
|
||||
new SimClustersEmbeddingStore(stores = stores)
|
||||
}
|
||||
|
||||
}
|
Binary file not shown.
@ -1,74 +0,0 @@
|
||||
package com.twitter.simclusters_v2.stores
|
||||
|
||||
import com.twitter.simclusters_v2.common.SimClustersEmbedding
|
||||
import com.twitter.simclusters_v2.common.SimClustersMultiEmbeddingId._
|
||||
import com.twitter.simclusters_v2.thriftscala.{
|
||||
SimClustersMultiEmbedding,
|
||||
SimClustersEmbeddingId,
|
||||
SimClustersMultiEmbeddingId
|
||||
}
|
||||
import com.twitter.storehaus.ReadableStore
|
||||
import com.twitter.util.Future
|
||||
|
||||
/**
|
||||
* The helper methods for SimClusters Multi-Embedding based ReadableStore
|
||||
*/
|
||||
object SimClustersMultiEmbeddingStore {
|
||||
|
||||
/**
|
||||
* Only support the Values based Multi-embedding transformation.
|
||||
*/
|
||||
case class SimClustersMultiEmbeddingWrapperStore(
|
||||
sourceStore: ReadableStore[SimClustersMultiEmbeddingId, SimClustersMultiEmbedding])
|
||||
extends ReadableStore[SimClustersEmbeddingId, SimClustersEmbedding] {
|
||||
|
||||
override def get(k: SimClustersEmbeddingId): Future[Option[SimClustersEmbedding]] = {
|
||||
sourceStore.get(toMultiEmbeddingId(k)).map(_.map(toSimClustersEmbedding(k, _)))
|
||||
}
|
||||
|
||||
// Override the multiGet for better batch performance.
|
||||
override def multiGet[K1 <: SimClustersEmbeddingId](
|
||||
ks: Set[K1]
|
||||
): Map[K1, Future[Option[SimClustersEmbedding]]] = {
|
||||
if (ks.isEmpty) {
|
||||
Map.empty
|
||||
} else {
|
||||
// Aggregate multiple get requests by MultiEmbeddingId
|
||||
val multiEmbeddingIds = ks.map { k =>
|
||||
k -> toMultiEmbeddingId(k)
|
||||
}.toMap
|
||||
|
||||
val multiEmbeddings = sourceStore.multiGet(multiEmbeddingIds.values.toSet)
|
||||
ks.map { k =>
|
||||
k -> multiEmbeddings(multiEmbeddingIds(k)).map(_.map(toSimClustersEmbedding(k, _)))
|
||||
}.toMap
|
||||
}
|
||||
}
|
||||
|
||||
private def toSimClustersEmbedding(
|
||||
id: SimClustersEmbeddingId,
|
||||
multiEmbedding: SimClustersMultiEmbedding
|
||||
): SimClustersEmbedding = {
|
||||
multiEmbedding match {
|
||||
case SimClustersMultiEmbedding.Values(values) =>
|
||||
val subId = toSubId(id)
|
||||
if (subId >= values.embeddings.size) {
|
||||
throw new IllegalArgumentException(
|
||||
s"SimClustersMultiEmbeddingId $id is over the size of ${values.embeddings.size}")
|
||||
} else {
|
||||
values.embeddings(subId).embedding
|
||||
}
|
||||
case _ =>
|
||||
throw new IllegalArgumentException(
|
||||
s"Invalid SimClustersMultiEmbedding $id, $multiEmbedding")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
def toSimClustersEmbeddingStore(
|
||||
sourceStore: ReadableStore[SimClustersMultiEmbeddingId, SimClustersMultiEmbedding]
|
||||
): ReadableStore[SimClustersEmbeddingId, SimClustersEmbedding] = {
|
||||
SimClustersMultiEmbeddingWrapperStore(sourceStore)
|
||||
}
|
||||
|
||||
}
|
Binary file not shown.
@ -1,87 +0,0 @@
|
||||
package com.twitter.simclusters_v2.stores
|
||||
|
||||
import com.twitter.bijection.scrooge.CompactScalaCodec
|
||||
import com.twitter.recos.entities.thriftscala.{SemanticCoreEntityWithLocale, UserScoreList}
|
||||
import com.twitter.storage.client.manhattan.kv.ManhattanKVClientMtlsParams
|
||||
import com.twitter.storehaus.ReadableStore
|
||||
import com.twitter.storehaus_internal.manhattan.{Athena, ManhattanRO, ManhattanROConfig}
|
||||
import com.twitter.storehaus_internal.util.{ApplicationID, DatasetName, HDFSPath}
|
||||
|
||||
object TopicTopProducersStore {
|
||||
val appIdDevel = "recos_platform_dev"
|
||||
val v2DatasetNameDevel = "topic_producers_em"
|
||||
val v3DatasetNameDevel = "topic_producers_agg"
|
||||
val v4DatasetNameDevel = "topic_producers_em_erg"
|
||||
|
||||
val appIdProd = "simclusters_v2"
|
||||
val v1DatasetNameProd = "top_producers_for_topic_from_topic_follow_graph"
|
||||
val v2DatasetNameProd = "top_producers_for_topic_em"
|
||||
|
||||
implicit val keyInj = CompactScalaCodec(SemanticCoreEntityWithLocale)
|
||||
implicit val valInj = CompactScalaCodec(UserScoreList)
|
||||
|
||||
def getTopicTopProducerStoreV1Prod(
|
||||
mhMtlsParams: ManhattanKVClientMtlsParams
|
||||
): ReadableStore[SemanticCoreEntityWithLocale, UserScoreList] =
|
||||
ManhattanRO.getReadableStoreWithMtls[SemanticCoreEntityWithLocale, UserScoreList](
|
||||
ManhattanROConfig(
|
||||
HDFSPath(""),
|
||||
ApplicationID(appIdProd),
|
||||
DatasetName(v1DatasetNameProd),
|
||||
Athena
|
||||
),
|
||||
mhMtlsParams
|
||||
)
|
||||
|
||||
def getTopicTopProducerStoreV2Devel(
|
||||
mhMtlsParams: ManhattanKVClientMtlsParams
|
||||
): ReadableStore[SemanticCoreEntityWithLocale, UserScoreList] =
|
||||
ManhattanRO.getReadableStoreWithMtls[SemanticCoreEntityWithLocale, UserScoreList](
|
||||
ManhattanROConfig(
|
||||
HDFSPath(""),
|
||||
ApplicationID(appIdDevel),
|
||||
DatasetName(v2DatasetNameDevel),
|
||||
Athena
|
||||
),
|
||||
mhMtlsParams
|
||||
)
|
||||
|
||||
def getTopicTopProducerStoreV2Prod(
|
||||
mhMtlsParams: ManhattanKVClientMtlsParams
|
||||
): ReadableStore[SemanticCoreEntityWithLocale, UserScoreList] =
|
||||
ManhattanRO.getReadableStoreWithMtls[SemanticCoreEntityWithLocale, UserScoreList](
|
||||
ManhattanROConfig(
|
||||
HDFSPath(""),
|
||||
ApplicationID(appIdProd),
|
||||
DatasetName(v2DatasetNameProd),
|
||||
Athena
|
||||
),
|
||||
mhMtlsParams
|
||||
)
|
||||
|
||||
def getTopicTopProducerStoreV3Devel(
|
||||
mhMtlsParams: ManhattanKVClientMtlsParams
|
||||
): ReadableStore[SemanticCoreEntityWithLocale, UserScoreList] =
|
||||
ManhattanRO.getReadableStoreWithMtls[SemanticCoreEntityWithLocale, UserScoreList](
|
||||
ManhattanROConfig(
|
||||
HDFSPath(""),
|
||||
ApplicationID(appIdDevel),
|
||||
DatasetName(v3DatasetNameDevel),
|
||||
Athena
|
||||
),
|
||||
mhMtlsParams
|
||||
)
|
||||
|
||||
def getTopicTopProducerStoreV4Devel(
|
||||
mhMtlsParams: ManhattanKVClientMtlsParams
|
||||
): ReadableStore[SemanticCoreEntityWithLocale, UserScoreList] =
|
||||
ManhattanRO.getReadableStoreWithMtls[SemanticCoreEntityWithLocale, UserScoreList](
|
||||
ManhattanROConfig(
|
||||
HDFSPath(""),
|
||||
ApplicationID(appIdDevel),
|
||||
DatasetName(v4DatasetNameDevel),
|
||||
Athena
|
||||
),
|
||||
mhMtlsParams
|
||||
)
|
||||
}
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user