├── src
├── main
│ ├── scala
│ │ └── com
│ │ │ └── netflix
│ │ │ └── migration
│ │ │ ├── utils
│ │ │ ├── revert_jobs.txt
│ │ │ ├── jobs.txt
│ │ │ ├── StringUtils.scala
│ │ │ ├── BulkRevertFinalizedJobs.scala
│ │ │ ├── IngestJobs.scala
│ │ │ ├── MigrationRunnerUtils.scala
│ │ │ ├── MigrationConsts.scala
│ │ │ ├── EmailSender.scala
│ │ │ └── Utils.scala
│ │ │ ├── modes
│ │ │ ├── CommandMode.scala
│ │ │ ├── Shadower.scala
│ │ │ ├── Preprocessor.scala
│ │ │ ├── Migrator.scala
│ │ │ ├── Reverter.scala
│ │ │ └── Communicator.scala
│ │ │ ├── data
│ │ │ ├── JobState.scala
│ │ │ ├── MigrationCommandMode.scala
│ │ │ ├── MigrationConfig.scala
│ │ │ ├── Job.scala
│ │ │ └── EmailStrings.scala
│ │ │ ├── providers
│ │ │ ├── NdcServiceProvider.scala
│ │ │ ├── SparkSessionProvider.scala
│ │ │ ├── MetacatServiceProvider.scala
│ │ │ ├── MigrationConfigProvider.scala
│ │ │ └── IcebergTableServiceProvider.scala
│ │ │ ├── services
│ │ │ ├── IcebergTableService.scala
│ │ │ ├── StorageService.scala
│ │ │ ├── NdcService.scala
│ │ │ ├── TrinoIcebergTableServiceImpl.scala
│ │ │ ├── NdcServiceImpl.scala
│ │ │ ├── MetacatService.scala
│ │ │ ├── StorageServiceImpl.scala
│ │ │ └── MetacatServiceImpl.scala
│ │ │ └── MigrationRunner.scala
│ └── main.iml
└── test
│ ├── test.iml
│ └── scala
│ └── com
│ └── netflix
│ └── migration
│ ├── utils
│ ├── StorageUtils.scala
│ ├── TestUtils.scala
│ ├── insertFakeRows.sh
│ └── SparkTablesGenerator.scala
│ └── MigrationIntegrationTest.scala
├── README.md
└── LICENSE
/src/main/scala/com/netflix/migration/utils/revert_jobs.txt:
--------------------------------------------------------------------------------
1 | prod/example_db/example_table2
2 |
--------------------------------------------------------------------------------
/src/main/scala/com/netflix/migration/utils/jobs.txt:
--------------------------------------------------------------------------------
1 | prod/example_db/example_table1
2 | prod/example_db/example_table2
3 | prod/example_db/example_table3
--------------------------------------------------------------------------------
/src/main/scala/com/netflix/migration/modes/CommandMode.scala:
--------------------------------------------------------------------------------
1 | package com.netflix.migration.modes
2 |
3 | /**
4 | * Methods that each commandMode implements
5 | */
6 | trait CommandMode {
7 |
8 | /**
9 | * Run the migration tool in the implemented command mode.
10 | */
11 | def run(): Unit
12 | }
13 |
--------------------------------------------------------------------------------
/src/main/scala/com/netflix/migration/data/JobState.scala:
--------------------------------------------------------------------------------
1 | package com.netflix.migration.data
2 |
3 | object JobState extends Enumeration {
4 | type JobState = Value
5 |
6 | val Ready, WritesBlocked, IcebergReady, IcebergPrimary, WritesUnblocked, HiveDropped,
7 | HivePrimary, SyncHive, IcebergDropped, Undefined = Value
8 | }
9 |
--------------------------------------------------------------------------------
/src/main/main.iml:
--------------------------------------------------------------------------------
1 |
2 |
3 |
4 |
5 |
6 |
7 |
8 |
9 |
10 |
11 |
--------------------------------------------------------------------------------
/src/test/test.iml:
--------------------------------------------------------------------------------
1 |
2 |
3 |
4 |
5 |
6 |
7 |
8 |
9 |
10 |
11 |
--------------------------------------------------------------------------------
/src/main/scala/com/netflix/migration/providers/NdcServiceProvider.scala:
--------------------------------------------------------------------------------
1 | package com.netflix.migration.providers
2 |
3 | import com.netflix.migration.services.NdcService
4 |
5 | object NdcServiceProvider {
6 | private var service: NdcService = _
7 |
8 | /**
9 | * Initialize the ndc service.
10 | *
11 | * @param ndcService the ndc service instance.
12 | */
13 | def init(ndcService: NdcService): Unit = service = ndcService
14 |
15 | def getNdcService: NdcService = service
16 | }
17 |
--------------------------------------------------------------------------------
/src/main/scala/com/netflix/migration/providers/SparkSessionProvider.scala:
--------------------------------------------------------------------------------
1 | package com.netflix.migration.providers
2 |
3 | import org.apache.spark.sql.SparkSession
4 |
5 | object SparkSessionProvider {
6 | private var spark: SparkSession = _
7 |
8 | /**
9 | * Initialize the SparkSession.
10 | *
11 | * @param sparkSession
12 | * the SparkSession to be used for the migration
13 | */
14 | def init(sparkSession: SparkSession): Unit = spark = sparkSession
15 |
16 | def getSparkSession: SparkSession = spark
17 | }
18 |
--------------------------------------------------------------------------------
/src/main/scala/com/netflix/migration/providers/MetacatServiceProvider.scala:
--------------------------------------------------------------------------------
1 | package com.netflix.migration.providers
2 |
3 | import com.netflix.migration.services.MetacatService
4 |
5 | object MetacatServiceProvider {
6 | private var service: MetacatService = _
7 |
8 | /**
9 | * Initialize the metacat service.
10 | *
11 | * @param metacatService the metacat service instance.
12 | */
13 | def init(metacatService: MetacatService): Unit = service = metacatService
14 |
15 | def getMetacatService: MetacatService = service
16 | }
17 |
--------------------------------------------------------------------------------
/src/main/scala/com/netflix/migration/providers/MigrationConfigProvider.scala:
--------------------------------------------------------------------------------
1 | package com.netflix.migration.providers
2 |
3 | import com.netflix.migration.data.MigrationConfig
4 |
5 | object MigrationConfigProvider {
6 | private var conf: MigrationConfig = _
7 |
8 | /**
9 | * Initialize the migration config.
10 | *
11 | * @param migrationConfig
12 | * the config to be used for the migration
13 | */
14 | def init(migrationConfig: MigrationConfig): Unit = conf = migrationConfig
15 |
16 | def getMigrationConfig: MigrationConfig = conf
17 | }
18 |
--------------------------------------------------------------------------------
/src/main/scala/com/netflix/migration/providers/IcebergTableServiceProvider.scala:
--------------------------------------------------------------------------------
1 | package com.netflix.migration.providers
2 |
3 | import com.netflix.migration.services.{IcebergTableService}
4 |
5 | object IcebergTableServiceProvider {
6 | private var service: IcebergTableService = _
7 |
8 | /**
9 | * Initialize the iceberg service.
10 | *
11 | * @param icebergTableService the service instance.
12 | */
13 | def init(icebergTableService: IcebergTableService): Unit = service = icebergTableService
14 |
15 | def getIcebergTableService: IcebergTableService = service
16 | }
17 |
--------------------------------------------------------------------------------
/src/main/scala/com/netflix/migration/services/IcebergTableService.scala:
--------------------------------------------------------------------------------
1 | package com.netflix.migration.services
2 |
3 |
4 | /**
5 | * An interface for interacting with iceberg table metadata.
6 | */
7 | trait IcebergTableService {
8 |
9 | /**
10 | * Set the necessary grants for this table.
11 | *
12 | * @param catalogName The catalog name.
13 | * @param dbName The db name.
14 | * @param tableName The table name.
15 | * @param grantor name of entity with existing grant permissions on the table.
16 | */
17 | def setGrants(catalogName: String, dbName: String, tableName: String, grantor: String)
18 | }
19 |
--------------------------------------------------------------------------------
/src/test/scala/com/netflix/migration/utils/StorageUtils.scala:
--------------------------------------------------------------------------------
1 | package com.netflix.migration.utils
2 |
3 | import com.netflix.migration.data.{Job, MigrationConfig}
4 | import com.netflix.migration.providers.MigrationConfigProvider
5 | import com.netflix.migration.services.StorageServiceImpl
6 |
7 | object StorageUtils {
8 |
9 | val conf: MigrationConfig = MigrationConfigProvider.getMigrationConfig
10 | val storageService: StorageServiceImpl = StorageServiceImpl(conf)
11 |
12 | def getAllJobs(): List[Job] = {
13 | val jobsQuery =
14 | s"""
15 | |SELECT *
16 | |FROM ${conf.queueName}
17 | |""".stripMargin
18 | storageService.getJobs(jobsQuery)
19 | }
20 | }
21 |
--------------------------------------------------------------------------------
/src/test/scala/com/netflix/migration/utils/TestUtils.scala:
--------------------------------------------------------------------------------
1 | package com.netflix.migration.utils
2 |
3 | import com.netflix.migration.providers.SparkSessionProvider
4 | import org.apache.spark.sql.SparkSession
5 |
6 | object TestUtils {
7 |
8 | val spark: SparkSession = SparkSessionProvider.getSparkSession
9 |
10 | def getHiveTableRowCount(table: String): Long = {
11 | spark.sql(s"SELECT COUNT(1) FROM $table").collect()(0).getLong(0)
12 | }
13 |
14 | def getIcebergTableRowCount(table: String): Long = {
15 | spark
16 | .sql(s"""
17 | |SELECT SUM(record_count)
18 | |FROM ${table}.partitions
19 | |""".stripMargin)
20 | .collect()(0)
21 | .getLong(0)
22 | }
23 | }
24 |
--------------------------------------------------------------------------------
/src/main/scala/com/netflix/migration/data/MigrationCommandMode.scala:
--------------------------------------------------------------------------------
1 | package com.netflix.migration.data
2 |
3 | /**
4 | * Specifies the command modes for the Migration Runner.
5 | */
6 | object MigrationCommandMode extends Enumeration {
7 | type MigrationCommandMode = Value
8 | val COMMUNICATOR, PREPROCESSOR, MIGRATOR, REVERTER, SHADOWER = Value
9 |
10 | /**
11 | * Parse the mode string and return the corresponding value
12 | *
13 | * @param mode
14 | * the mode string to be parsed
15 | * @return
16 | * the corresponding value for the mode string
17 | */
18 | def parse(mode: String): Value = {
19 | values
20 | .find(_.toString.equalsIgnoreCase(mode))
21 | .getOrElse(throw new NoSuchElementException(s"No value found for '$mode'"))
22 | }
23 | }
24 |
--------------------------------------------------------------------------------
/src/main/scala/com/netflix/migration/utils/StringUtils.scala:
--------------------------------------------------------------------------------
1 | package com.netflix.migration.utils
2 |
3 | object StringUtils {
4 |
5 | /**
6 | * Parses String Array with format a=b and returns a Map of key,value pairs
7 | *
8 | * @param args
9 | * String Array with format a=b
10 | * @return
11 | */
12 | def parseArgs(args: Array[String]): Map[String, String] = {
13 | var result: Map[String, String] = Map()
14 | if (args == null) return result
15 |
16 | for (arg <- args) {
17 | val foo = arg.split("=", 2)
18 | val value = if (foo.length == 1) "" else foo(1)
19 | result += (foo(0) -> value)
20 | }
21 | result
22 | }
23 |
24 | def getTableQualifiedName(
25 | catalogName: String,
26 | dbName: String,
27 | tableName: String,
28 | delim: String = "/"): String = {
29 | require(
30 | catalogName != null && dbName != null && tableName != null,
31 | "Qualified name arguments cannot be null")
32 | s"$catalogName$delim$dbName$delim$tableName"
33 | }
34 | }
35 |
--------------------------------------------------------------------------------
/src/main/scala/com/netflix/migration/utils/BulkRevertFinalizedJobs.scala:
--------------------------------------------------------------------------------
1 | package com.netflix.migration.utils
2 |
3 | import com.netflix.migration.MigrationRunner
4 | import com.netflix.migration.modes.Reverter
5 |
6 | import java.io.File
7 | import scala.io.Source
8 |
9 | object BulkRevertJobs extends App {
10 |
11 | // Set the path to the file containing the list of table names
12 | val tableFile = new File(
13 | "./revert_jobs.txt").getAbsoluteFile
14 |
15 | // Set conf to prod env and get migration conf
16 | val migrationConf = MigrationRunner.createMigrationConf(Map("dbEnv" -> "prod"))
17 |
18 | // Read the table file and generate a (catalogName, dbName, tableName) triplet for each table
19 | val tablesSet = Source
20 | .fromFile(tableFile)
21 | .getLines()
22 | .map { table =>
23 | val Array(catalogName, dbName, tableName) = table.split("/")
24 | (catalogName, dbName, tableName)
25 | }
26 | .toSet
27 |
28 | // Instantiate Reverter object to bulk revert tables
29 | val reverter = Reverter()
30 | reverter.bulkRevertFinalizedTables(tablesSet)
31 | }
32 |
--------------------------------------------------------------------------------
/src/main/scala/com/netflix/migration/services/StorageService.scala:
--------------------------------------------------------------------------------
1 | package com.netflix.migration.services
2 |
3 | import com.netflix.migration.data.Job
4 |
5 | /**
6 | * Methods interacting with the Jobs database
7 | */
8 | trait StorageService {
9 |
10 | /**
11 | * Get the job for the given table
12 | *
13 | * @param catalogName
14 | * Name of the catalog
15 | * @param dbName
16 | * Name of the database
17 | * @param tableName
18 | * Name of the table
19 | * @return
20 | * Job corresponding to table
21 | */
22 | def getJob(catalogName: String, dbName: String, tableName: String): Job
23 |
24 | /**
25 | * Get the list of jobs for the given query
26 | *
27 | * @param jobsQuery
28 | * Jobs query
29 | * @return
30 | * list of jobs
31 | */
32 | def getJobs(jobsQuery: String): List[Job]
33 |
34 | /**
35 | * Update the given job's column value.
36 | *
37 | * @param job
38 | * The job.
39 | */
40 | def updateJob(job: Job): Unit
41 |
42 | /**
43 | * Insert new job into the jobs queue
44 | *
45 | * @param job
46 | * The job.
47 | */
48 | def enqueueJob(job: Job): Unit
49 |
50 | /**
51 | * Remove job from the jobs queue
52 | *
53 | * @param job
54 | * The job.
55 | */
56 | def removeJob(job: Job): Unit
57 | }
58 |
--------------------------------------------------------------------------------
/src/test/scala/com/netflix/migration/utils/insertFakeRows.sh:
--------------------------------------------------------------------------------
1 | #!/bin/bash
2 |
3 | # Set LC_CTYPE to C.UTF-8
4 | export LC_CTYPE=C.UTF-8
5 |
6 | # Check if argument is provided
7 | if [ $# -eq 0 ]
8 | then
9 | # Prompt user for number of rows to insert
10 | echo -n "Enter number of rows to insert: "
11 | read num_rows
12 | else
13 | # Use argument as number of rows to insert
14 | num_rows=$1
15 | fi
16 |
17 | # Generate insert statement with fake data
18 | echo "INSERT INTO prodhive.sandbox.job_sla_edges VALUES"
19 | for (( i=1; i<=$num_rows; i++ ))
20 | do
21 | id="$i"
22 | rand=$(openssl rand -hex 18 | base64)
23 | label=${rand:0:6}
24 | from_id=${rand:6:6}
25 | to_id=${rand:12:6}
26 | from_name=${rand:18:6}
27 | to_name=${rand:24:6}
28 | properties="map('property1', 'value1', 'property2', 'value2')"
29 | dateint=$(( (RANDOM % 100) + 1616329200 ))
30 | cluster_name="cluster_name$i"
31 |
32 | if [ $i -eq $num_rows ]
33 | then
34 | # Last row, so end with semicolon
35 | echo "('$id', '$label', '$from_id', '$to_id', '$from_name', '$to_name', $properties, $dateint, '$cluster_name');"
36 | else
37 | # Not the last row, so end with comma
38 | echo "('$id', '$label', '$from_id', '$to_id', '$from_name', '$to_name', $properties, $dateint, '$cluster_name'),"
39 | fi
40 | done
41 |
42 |
--------------------------------------------------------------------------------
/src/main/scala/com/netflix/migration/services/NdcService.scala:
--------------------------------------------------------------------------------
1 | package com.netflix.migration.services
2 |
3 | import com.netflix.migration.utils.MigrationConsts.DataCategory.DataCategory
4 | import com.netflix.ndc.common.dto.{MetadataDto, MetadataResponseDto}
5 |
6 | /**
7 | * An interface for interacting with NDC.
8 | */
9 | trait NdcService {
10 |
11 | /**
12 | * Get table metadata
13 | *
14 | * @param catalogName
15 | * @param dbName
16 | * @param tableName
17 | */
18 | def getMetadata(catalogName: String, dbName: String, tableName: String): MetadataResponseDto
19 |
20 | /**
21 | * Sets table metadata
22 | *
23 | * @param metadataDtos
24 | */
25 | def setMetadata(metadataDtos: List[MetadataDto]): Unit
26 |
27 | /**
28 | * Get table owners
29 | *
30 | * @param catalogName
31 | * @param dbName
32 | * @param tableName
33 | */
34 | def getTableOwners(catalogName: String, dbName: String, tableName: String): List[String]
35 |
36 | /**
37 | * Create MetadataDto object from MetadataResponseDto
38 | *
39 | * @param metadataResponseDto
40 | */
41 | def createMetadataDto(metadataResponseDto: MetadataResponseDto): MetadataDto
42 |
43 | /**
44 | * Get table data category
45 | *
46 | * @param catalogName
47 | * @param dbName
48 | * @param tableName
49 | */
50 | def getDataCategory(catalogName: String, dbName: String, tableName: String): DataCategory
51 |
52 | }
53 |
--------------------------------------------------------------------------------
/src/main/scala/com/netflix/migration/data/MigrationConfig.scala:
--------------------------------------------------------------------------------
1 | package com.netflix.migration.data
2 |
3 | import com.netflix.migration.data.MigrationCommandMode.MigrationCommandMode
4 | import com.netflix.migration.utils.MigrationConsts
5 |
6 | import java.time.{Instant, LocalDate}
7 |
8 | /**
9 | * Migration tool configuration for a given run.
10 | *
11 | * @param commandMode
12 | * The command mode to run in: MIGRATOR, PREPROCESSOR, REVERTER etc.
13 | * @param runLocally
14 | * Whether the tool should be run locally in Spark.
15 | * @param dryRun
16 | * Whether this is a dry run.
17 | * @param processDate
18 | * The processing date.
19 | * @param processStartTime
20 | * The processing start time.
21 | * @param distributedMode
22 | * Whether the jobs should be processed on executors in a distributed manner.
23 | * @param runId
24 | * The genie job id, or UUID if run locally.
25 | * @param dbEnv
26 | * The database environment, prod or test.
27 | */
28 | case class MigrationConfig(
29 | commandMode: MigrationCommandMode,
30 | runLocally: Boolean,
31 | dryRun: Boolean,
32 | processDate: LocalDate,
33 | processStartTime: Instant,
34 | distributedMode: Boolean,
35 | runId: String,
36 | dbEnv: String,
37 | jobsProcessBatchSize: Int,
38 | queueName: String,
39 | archiveHive: Boolean,
40 | setIcebergAcls: Boolean,
41 | dbName: String,
42 | batchId: Option[String],
43 | batchTableName: String,
44 | migratorId: Int,
45 | numMigrators: Int,
46 | preprocessorId: Int,
47 | numPreprocessors: Int) {
48 | require(
49 | dbEnv.equals(MigrationConsts.TEST_ENV) || dbEnv.equals(MigrationConsts.PROD_ENV),
50 | "Database environment should be one of 'prod' or 'test'")
51 | }
52 |
--------------------------------------------------------------------------------
/src/main/scala/com/netflix/migration/utils/IngestJobs.scala:
--------------------------------------------------------------------------------
1 | package com.netflix.migration.utils
2 |
3 | import com.netflix.migration.MigrationRunner
4 | import com.netflix.migration.services.StorageServiceImpl
5 |
6 | import java.io.File
7 | import scala.io.Source
8 |
9 | object IngestJobs extends App {
10 |
11 | // Set the path to the file containing the list of table names
12 | val tableFile = new File(
13 | "./jobs.txt").getAbsoluteFile
14 |
15 | // Set conf to prod env and get migration conf
16 | val migrationConf = MigrationRunner.createMigrationConf(Map("dbEnv" -> "prod"))
17 |
18 | // Connect to the MySQL database and retrieve the existing table names from the jobs table
19 | val connection = StorageServiceImpl.makeDbConnection(migrationConf)
20 | val statement = connection.createStatement()
21 | val resultSet = statement.executeQuery("SELECT catalog_name, db_name, tbl_name FROM jobs")
22 | val existingTableNames = collection.mutable.Set[(String, String, String)]()
23 | while (resultSet.next()) {
24 | existingTableNames.add(
25 | (
26 | resultSet.getString("catalog_name"),
27 | resultSet.getString("db_name"),
28 | resultSet.getString("tbl_name")))
29 | }
30 | resultSet.close()
31 | statement.close()
32 |
33 | // Read the table file and generate a VALUES clause for each table that is not already in the jobs table
34 | val valuesSet = Source
35 | .fromFile(tableFile)
36 | .getLines()
37 | .map { table =>
38 | val Array(catalogName, dbName, tableName) = table.split("/")
39 | (catalogName, dbName, tableName)
40 | }
41 | .filterNot(existingTableNames.contains)
42 | .toSet
43 |
44 | val values = valuesSet
45 | .map { case (catalogName, dbName, tableName) =>
46 | s"('$catalogName', '$dbName', '$tableName')"
47 | }
48 | .mkString(",")
49 |
50 | // Insert the new table names into the jobs table
51 | if (values.nonEmpty) {
52 | val insertStatement = s"INSERT INTO jobs (catalog_name, db_name, tbl_name) VALUES $values;"
53 | val statement = connection.createStatement()
54 | statement.executeUpdate(insertStatement)
55 | statement.close()
56 | }
57 |
58 | // Close the database connection
59 | connection.close()
60 | }
61 |
--------------------------------------------------------------------------------
/src/main/scala/com/netflix/migration/data/Job.scala:
--------------------------------------------------------------------------------
1 | package com.netflix.migration.data
2 |
3 | /**
4 | * Represents a job entity for migrating a table.
5 | *
6 | * @param id
7 | * @param catalogName
8 | * @param dbName
9 | * @param tblName
10 | * @param stgFormat
11 | * @param tblOwners
12 | * @param downstreamUsers
13 | * @param toBeProcessed
14 | * @param inProcess
15 | * @param state
16 | * @param desiredState
17 | * @param commLevel1Date
18 | * @param commLevel2Date
19 | * @param commLevel3Date
20 | * @param shadowWatermark
21 | * @param migrationPaused
22 | * @param shadowStatus
23 | * @param createdAt
24 | * @param lastUpdatedTime
25 | */
26 | class Job(
27 | var id: Integer,
28 | var catalogName: String,
29 | var dbName: String,
30 | var tblName: String,
31 | var stgFormat: String,
32 | var dataCategory: String,
33 | var tblOwners: Set[String],
34 | var downstreamUsers: Set[String],
35 | var toBeProcessed: Int,
36 | var inProcess: Int,
37 | var state: JobState.Value,
38 | var desiredState: JobState.Value,
39 | var initialGapInDays: Int,
40 | var probationGapInDays: Int,
41 | var commLevel1Date: Long,
42 | var commLevel2Date: Long,
43 | var commLevel3Date: Long,
44 | var shadowWatermark: Long,
45 | var migrationPaused: Int,
46 | var pauseReason: String,
47 | var runId: String,
48 | var shadowStatus: String,
49 | var createdAt: Long,
50 | var lastUpdatedTime: Long) {
51 | override def toString(): String = {
52 | s"Job[id=$id, " +
53 | s"catalog=$catalogName, " +
54 | s"db=$dbName, " +
55 | s"tbl=$tblName, " +
56 | s"format=$stgFormat, " +
57 | s"data_category=$dataCategory, " +
58 | s"owners=$tblOwners, " +
59 | s"to_be_processed=$toBeProcessed, " +
60 | s"in-process=$inProcess, " +
61 | s"state=$state, " +
62 | s"desired=$desiredState, " +
63 | s"initial_gap_in_days=$initialGapInDays, " +
64 | s"probation_gap_in_days=$probationGapInDays, " +
65 | s"comm_level1=$commLevel1Date, " +
66 | s"comm_level2=$commLevel2Date, " +
67 | s"comm_level3=$commLevel3Date, " +
68 | s"shadow_watermark=$shadowWatermark, " +
69 | s"migration_paused=$migrationPaused, " +
70 | s"pause_reason=$pauseReason, " +
71 | s"runId=$runId, " +
72 | s"shadow_status=$shadowStatus]"
73 | }
74 | }
75 |
--------------------------------------------------------------------------------
/src/main/scala/com/netflix/migration/utils/MigrationRunnerUtils.scala:
--------------------------------------------------------------------------------
1 | package com.netflix.migration.utils
2 |
3 | import com.netflix.metacat.common.dto.TableDto
4 | import com.netflix.metacat.shaded.com.fasterxml.jackson.databind.ObjectMapper
5 | import com.netflix.metacat.shaded.com.fasterxml.jackson.databind.node.ObjectNode
6 | import com.netflix.migration.data.{Job, MigrationConfig}
7 | import com.typesafe.scalalogging.StrictLogging
8 | import org.apache.spark.sql.SparkSession
9 |
10 | import scala.util.{Failure, Success, Try}
11 |
12 | /**
13 | * Generic Migration Runner utility methods.
14 | */
15 | object MigrationRunnerUtils extends StrictLogging {
16 |
17 | @annotation.tailrec
18 | def retry[T](n: Int)(fn: => T): T = {
19 | Try {
20 | fn
21 | } match {
22 | case Success(x) => x
23 | case _ if n > 1 =>
24 | logger.error(
25 | s"Failed to execute ${getFnName(fn)} on attempt: $n. Retrying after 5 secs..")
26 | Thread.sleep(5000)
27 | retry(n - 1)(fn)
28 | case Failure(e) => throw e
29 | }
30 | }
31 |
32 | def getFnName[T](fn: => T): String = {
33 | fn.getClass.getName.split("\\$").last
34 | }
35 |
36 | def applyBatchIdFilterOnJobs(
37 | spark: SparkSession,
38 | conf: MigrationConfig,
39 | jobs: List[Job]): List[Job] = {
40 | logger.info(
41 | s"Applying batchId: ${conf.batchId} filter on ${jobs.size} using batchId queue: ${conf.batchTableName}")
42 | // Get the list of tables matching the batch Id
43 | val tablesToProcess: Set[String] = spark
44 | .sql(s"""
45 | |select distinct table_name from ${conf.batchTableName} t
46 | |where t.batchId = '${conf.batchId.get.toLowerCase}'
47 | |""".stripMargin)
48 | .toDF()
49 | .collect()
50 | .map(_(0).asInstanceOf[String].toLowerCase)
51 | .toSet
52 | jobs.filter(j =>
53 | tablesToProcess.contains(
54 | StringUtils.getTableQualifiedName(j.catalogName, j.dbName, j.tblName).toLowerCase))
55 | }
56 |
57 | def getMetacatFieldValue(dto: TableDto, fieldName: String): Option[String] = {
58 | Option(dto.getDefinitionMetadata)
59 | .flatMap(o => Option(o.get(fieldName)))
60 | .map(_.asText)
61 | }
62 |
63 | def setMetacatFieldValue(dto: TableDto, fieldName: String, fieldValue: String) = {
64 | var definitionMetadataNode: ObjectNode = null
65 | if (dto.getDefinitionMetadata == null) {
66 | val objectMapper = new ObjectMapper()
67 | definitionMetadataNode = objectMapper.createObjectNode()
68 | dto.setDefinitionMetadata(definitionMetadataNode)
69 | } else {
70 | definitionMetadataNode = dto.getDefinitionMetadata
71 | }
72 | definitionMetadataNode.put(fieldName, fieldValue)
73 | }
74 | }
75 |
--------------------------------------------------------------------------------
/src/main/scala/com/netflix/migration/modes/Shadower.scala:
--------------------------------------------------------------------------------
1 | package com.netflix.migration.modes
2 |
3 | import com.netflix.bdp.IncrementalRefreshShadowTool
4 | import com.netflix.migration.data.{Job, MigrationConfig}
5 | import com.netflix.migration.providers.{MigrationConfigProvider, SparkSessionProvider}
6 | import com.netflix.migration.services.{StorageService, StorageServiceImpl}
7 | import com.netflix.migration.utils.MigrationConsts.HIVE_SUFFIX
8 | import com.netflix.migration.utils.Utils
9 | import com.typesafe.scalalogging.StrictLogging
10 | import org.apache.spark.sql.SparkSession
11 |
12 | case class Shadower() extends CommandMode with StrictLogging {
13 |
14 | val conf: MigrationConfig = MigrationConfigProvider.getMigrationConfig
15 | val spark: SparkSession = SparkSessionProvider.getSparkSession
16 | // val sparkSessionPool: Array[SparkSession] = SparkSessionPoolProvider.getSparkSessionPool
17 | @transient private[modes] lazy val storageService: StorageService = StorageServiceImpl(conf)
18 |
19 | def getRelevantJobs(): List[Job] = {
20 | val jobsQuery =
21 | s"""
22 | |SELECT *
23 | |FROM ${conf.queueName}
24 | |WHERE state = 'WritesUnblocked'
25 | |AND desired_state = 'WritesUnblocked'
26 | |AND probation_gap_days > 0
27 | |AND shadow_status is NULL
28 | |AND migration_paused = false
29 | |""".stripMargin
30 | storageService.getJobs(jobsQuery)
31 | }
32 |
33 | def run(): Unit = {
34 | val jobs = getRelevantJobs()
35 | logger.info(s"Shadower with runid=${conf.runId} processing jobs=$jobs")
36 | for (job <- jobs) {
37 | if (!Utils.hiveIcebergTablesInSync(job)) {
38 | shadowJob(job)
39 | }
40 | }
41 | }
42 |
43 | /**
44 | * Shadow the job entity.
45 | *
46 | * @param job
47 | * the job entity to be shadowed
48 | */
49 | def shadowJob(job: Job): Unit = {
50 | val tableName = Utils.getFullyQualifiedTableName(job)
51 | val destinationTableName = s"$tableName$HIVE_SUFFIX"
52 | job.shadowStatus = "Shadowing"
53 | logger.info(s"Running shadower for $tableName to destination $destinationTableName")
54 | storageService.updateJob(job)
55 | copyDataSrcToDst(tableName, destinationTableName)
56 | job.shadowStatus = null
57 | storageService.updateJob(job)
58 | }
59 |
60 | /**
61 | * Copy data from source table to destination table.
62 | *
63 | * @param srcTable
64 | * The name of the source table.
65 | * @param dstTable
66 | * The name of the destination table.
67 | */
68 | def copyDataSrcToDst(srcTable: String, dstTable: String): Unit = {
69 | val shadowTool =
70 | new IncrementalRefreshShadowTool(srcTable, dstTable, spark, 50, 3, 50)
71 | shadowTool.mirrorUpdates()
72 | }
73 | }
74 |
--------------------------------------------------------------------------------
/src/main/scala/com/netflix/migration/services/TrinoIcebergTableServiceImpl.scala:
--------------------------------------------------------------------------------
1 | package com.netflix.migration.services
2 |
3 | import com.netflix.migration.data.MigrationConfig
4 | import com.netflix.migration.services.TrinoIcebergTableServiceImpl.getTrinoConnection
5 | import com.netflix.migration.utils.{MigrationConsts, StringUtils}
6 | import com.typesafe.scalalogging.StrictLogging
7 |
8 | /**
9 | * Manage Iceberg table metadata using Trino.
10 | */
11 | case class TrinoIcebergTableServiceImpl(conf: MigrationConfig)
12 | extends IcebergTableService
13 | with StrictLogging {
14 |
15 | /**
16 | * Set the necessary grants for this table.
17 | *
18 | * @param catalogName
19 | * The catalog name.
20 | * @param dbName
21 | * The db name.
22 | * @param tableName
23 | * The table name.
24 | * @param grantor
25 | * name of entity with existing grant permissions on the table.
26 | */
27 | override def setGrants(
28 | grantor: String,
29 | catalogName: String,
30 | dbName: String,
31 | tableName: String): Unit = {
32 | // For now, grant all permissions to this GROUP.
33 | val granteeNames = List("common-access@bdp")
34 | val tableQName = StringUtils.getTableQualifiedName(catalogName, dbName, tableName)
35 | var connection: Connection = null
36 | try {
37 | connection = getTrinoConnection(grantor)
38 | // For now, grant all permissions (including ADMIN).
39 | for (granteeName <- granteeNames) {
40 | logger.info(
41 | s"Attempting to grant ADMIN perms to $granteeName using grantor: $grantor on table: $tableQName")
42 | val grantSqlQuery =
43 | "GRANT ALL PRIVILEGES ON " + catalogName + "." + dbName + ".\"" + tableName + "\"" + "TO ROLE \"" + granteeName + "\" WITH GRANT OPTION"
44 | val stmt = connection.createStatement
45 | stmt.executeUpdate(grantSqlQuery)
46 | logger.info(
47 | s"Successfully granted ADMIN perms to $granteeName using grantor: $grantor on table: $tableName")
48 | }
49 | } catch {
50 | case e: Exception =>
51 | logger.error(s"Encountered exception setting table: $tableName grants. Exception: $e")
52 | throw e
53 | } finally {
54 | if (connection != null) {
55 | connection.close()
56 | }
57 | }
58 | }
59 | }
60 |
61 | object TrinoIcebergTableServiceImpl {
62 |
63 | def getTrinoConnection(user: String): Connection = {
64 | val trinoJdbcUrl = MigrationConsts.TRINO_JDBC_URL
65 | val properties = getConnProperties(user)
66 | DriverManager.registerDriver(new TrinoDriver())
67 | DriverManager.getConnection(trinoJdbcUrl, properties)
68 | }
69 |
70 | def getConnProperties(user: String): Properties = {
71 | val properties = new Properties()
72 | properties.setProperty("user", user)
73 | properties.setProperty("SSL", "true")
74 | properties.setProperty("source", "hive_iceberg_migration")
75 | properties
76 | }
77 | }
78 |
--------------------------------------------------------------------------------
/src/main/scala/com/netflix/migration/utils/MigrationConsts.scala:
--------------------------------------------------------------------------------
1 | package com.netflix.migration.utils
2 |
3 | /**
4 | * Migration tool constants.
5 | */
6 | object MigrationConsts {
7 |
8 | /**
9 | * URL for the test jobs db.
10 | */
11 | final val JOBS_DB_TEST_URL: String = "your-test-rds-db-hostname.region.rds.amazonaws.com"
12 | final val JOBS_DB_PROD_URL: String =
13 | "your-prod-rds-db-hostname.region.rds.amazonaws.com"
14 | final val JOBS_DB_NAME: String = "migrate_01"
15 | final val HIVE_SUFFIX = "_hive"
16 | final val ICEBERG_SUFFIX = "_iceberg"
17 | final val REVERT_SUFFIX = "_revert"
18 | final val ARCHIVED_SUFFIX = "_archived"
19 | final val millisecondsPerDay = 86400000L
20 | final val ICEBERG_MIGRATION_DO_NOT_MODIFY_TAG: String = "iceberg_migration_do_not_modify"
21 | final val MIGRATED_FROM_HIVE_TAG: String = "h2i_migrated_from_hive"
22 | final val DO_NOT_RENAME_TAG: String = "do_not_rename"
23 | final val DO_NOT_DROP_TAG: String = "do_not_drop"
24 | final val HIVE_ARCHIVED_TAG: String = "h2i_archive"
25 | final val HIVE_PARQUET: String = "parquet"
26 | final val HIVE_CSV_TEXT: String = "csv/text"
27 | final val PROD_ENV = "prod"
28 | final val TEST_ENV = "test"
29 | final val TEST_CATALOG = "testhive"
30 | final val TEST_DATABASE = "transport_temp"
31 | final val TEST_TABLE_PREFIX = "table_"
32 | final val DEFAULT_TABLE_OWNER = "akayyoor"
33 | final val TEST_THREAD_POOL_SIZE = 10
34 | final val NUM_RETRIES = 3
35 | final val HIVE_TIMESTAMP_COLUMN_NAME = "timestamp"
36 | final val METACAT_HOST_URL_PROPERTY = "http://metacat-hosturl"
37 | final val METACAT_USERNAME_PROPERTY = "hiveToIcebergMigrator"
38 | final val METACAT_CLIENT_APP_NAME_PROPERTY = METACAT_USERNAME_PROPERTY
39 | final val METACAT_DATA_MIGRATED_PROPERTY = "migrated_data_location"
40 | final val NDC_HOST_URL_PROPERTY = "https://data-catalog-host-url"
41 | final val NDC_USERNAME_PROPERTY = METACAT_USERNAME_PROPERTY
42 | final val NDC_CLIENT_APP_NAME_PROPERTY = METACAT_CLIENT_APP_NAME_PROPERTY
43 | final val MIGRATOR_ID = 1
44 | final val NUM_MIGRATORS = 1
45 | final val PREPROCESSOR_ID = 1
46 | final val NUM_PREPROCESSORS = 1
47 | final val RECIPIENT_BATCH_SIZE = 50
48 | final val MYSQL_USER = 'user'
49 | final val MYSQL_PASS = 'pass'
50 | final val TRINO_JDBC_URL = "jdbc:trino://trino.master.net:7004/iceberg/default"
51 | final val AWS_SES_SOURCE_ARN = "arn:aws:ses:region:etcetc"
52 | final val COMMUNICATION_EMAIL_ADDRESS = "h2i-migration-noreply@some-domain.com"
53 |
54 | final val jobsProcessBatchSize = 10
55 | final val queueName = "jobs"
56 | final val batchTableName = "prod.bdp.h2i_batches"
57 |
58 | object TableFormat extends Enumeration {
59 | type TableFormat = Value
60 | val ICEBERG, HIVE_PARQUET = Value
61 | }
62 |
63 | object PauseReason extends Enumeration {
64 | type PauseReason = Value
65 | val IS_CSV_TEXT, IS_ALREADY_ICEBERG, IS_WAP_TABLE, IS_PSYCHO_TABLE, TABLE_NOT_FOUND,
66 | MISSING_OWNER, HAS_TIMESTAMP_COLUMN, EMPTY_TABLE, REVERTED =
67 | Value
68 | }
69 |
70 | object DataCategory extends Enumeration {
71 | type DataCategory = Value
72 | val PI, CORE_PI, NO_PI, UNKNOWN = Value
73 | }
74 | }
75 |
--------------------------------------------------------------------------------
/src/main/scala/com/netflix/migration/utils/EmailSender.scala:
--------------------------------------------------------------------------------
1 | package com.netflix.migration.utils
2 |
3 | import com.amazonaws.regions.Regions
4 | import com.amazonaws.services.simpleemail.{AmazonSimpleEmailService, AmazonSimpleEmailServiceClientBuilder}
5 | import com.amazonaws.services.simpleemail.model._
6 | import com.netflix.migration.utils.Utils.migrationConf
7 |
8 | import scala.collection.JavaConverters.setAsJavaSetConverter
9 |
10 | object EmailSender {
11 |
12 | // val log = Logger.apply(this.getClass)
13 | val region = "us-west-2"
14 |
15 | val client: AmazonSimpleEmailService = AmazonSimpleEmailServiceClientBuilder
16 | .standard()
17 | .withRegion(Regions.fromName(region))
18 | .build()
19 |
20 | sealed trait EmailBodyType
21 | case object Text extends EmailBodyType
22 | case object Html extends EmailBodyType
23 |
24 | /**
25 | * Send an email to the specified recipients
26 | *
27 | * @param to
28 | * set of recipients' email addresses
29 | * @param subject
30 | * subject of the email
31 | * @param body
32 | * body of the email
33 | * @param bodyType
34 | * type of the email body
35 | */
36 | def send(
37 | to: Set[String],
38 | subject: String,
39 | body: String = "",
40 | bodyType: EmailBodyType = Html): Unit = {
41 | // log.info(s"Sending an email to $to with subject $subject")
42 | var request = new SendEmailRequest()
43 | .withDestination(new Destination()
44 | .withToAddresses(to.asJava))
45 | .withMessage(
46 | new Message()
47 | .withBody(getBody(bodyType, body))
48 | .withSubject(new Content()
49 | .withCharset("UTF-8")
50 | .withData(s"do-not-reply: $subject")))
51 | .withSourceArn(MigrationConsts.AWS_SES_SOURCE_ARN)
52 | request = request.withSource(MigrationConsts.COMMUNICATION_EMAIL_ADDRESS)
53 | client.sendEmail(request)
54 | }
55 |
56 | /**
57 | * Convert plain text email body to HTML email body
58 | *
59 | * @param emailText
60 | * plain text email body that includes newlines and list items
61 | * @return
62 | * HTML formatted email body with proper line breaks and list items
63 | */
64 | def textToHtmlEmailBody(emailText: String): String = {
65 | val emailHtml =
66 | emailText
67 | .replaceAll("\\*", "
")
68 | .replaceAll("[.] \n", "")
69 | .replaceAll("\n\n", "
")
70 | .replaceAll("\n", "
")
71 |
72 | s"""
73 |
74 |
75 |
76 |
77 | $emailHtml
78 |
79 |
80 | """
81 | }
82 |
83 | /**
84 | * Get the body of the email based on the body type
85 | *
86 | * @param bodyType
87 | * type of the email body
88 | * @param body
89 | * the text or html content of the email body
90 | * @return
91 | * the email body in the specified format
92 | */
93 | def getBody(bodyType: EmailBodyType, body: String): Body = {
94 | bodyType match {
95 | case Text => new Body().withText(new Content().withCharset("UTF-8").withData(body))
96 | case Html =>
97 | new Body().withHtml(
98 | new Content().withCharset("UTF-8").withData(textToHtmlEmailBody(body)))
99 | case _ => throw new UnsupportedOperationException
100 | }
101 | }
102 |
103 | def main(args: Array[String]): Unit = {
104 | EmailSender.send(
105 | Set("akayyoor@netflix.com"),
106 | "Test Email",
107 | "Test email body",
108 | EmailSender.Html)
109 | }
110 | }
111 |
--------------------------------------------------------------------------------
/src/main/scala/com/netflix/migration/data/EmailStrings.scala:
--------------------------------------------------------------------------------
1 | package com.netflix.migration.data
2 |
3 | import com.netflix.migration.providers.MigrationConfigProvider
4 | import com.netflix.migration.utils.Utils.getDayNDaysFromNow
5 |
6 | case class EmailStrings(initialGapInDays: Int, probationGapInDays: Int) {
7 |
8 | val migrationConf: MigrationConfig = MigrationConfigProvider.getMigrationConfig
9 |
10 | val level1Str1: String =
11 | if (initialGapInDays > 1) s"${initialGapInDays} Days"
12 | else if (initialGapInDays == 1) s"${initialGapInDays} Day"
13 | else "Shortly"
14 | val level1Str2: String =
15 | if (initialGapInDays > 0) s"in ${level1Str1.toLowerCase}"
16 | else s"${level1Str1.toLowerCase}"
17 | val level1Str3: String =
18 | if (initialGapInDays > 0) s"within the next ${level1Str1.toLowerCase}" else ""
19 | val migrationDate = getDayNDaysFromNow(initialGapInDays)
20 | val lowerLevel1Str1 = level1Str1.toLowerCase
21 | val level1DownstreamUserMsg: (String, String) = (
22 | s"[Hive To Iceberg Migration]: Table %s Scheduled for Migration to Iceberg on $migrationDate",
23 | "Dear User, \n\nWe are writing to inform you that we will be migrating the Hive table\n '%s' \n[https://somelink/tables/%s] " +
24 | s"to the Iceberg table \nformat on $migrationDate, which is $lowerLevel1Str1 from now. Our records " +
25 | s"indicate \nthat you have used this table in the past, so we wanted to provide ample notice." +
26 | s"\n\nUpon completion of the migration, the table owner/s (%s) will have the \n" +
27 | s"option to restrict access. By default, the table will remain accessible to everyone " +
28 | s"\n('Common Access')." +
29 | s"\n\nWe do not anticipate any significant differences or disruptions during the migration " +
30 | s"\nprocess. For more information about the migration, please visit our migration portal at " +
31 | s"\n[http://go/h2i-docs]. \n\nIf you have any concerns, questions, or need to pause the migration " +
32 | s"\nbefore $migrationDate, please reach out to #bigdatasupport Slack channel, and \nwe will be glad to assist you." +
33 | s"\n\nThanks,\nData Platform")
34 |
35 | val level1TableOwnerMsg: (String, String) = (
36 | s"[Hive To Iceberg Migration]: Tables Owned by You are Scheduled for Migration to Iceberg on $migrationDate",
37 | "This email serves as a notification to the owner of following Hive tables %s that they will " +
38 | s"be migrated as part of the Hive to Iceberg migration process on $migrationDate, \nwhich is " +
39 | s"$lowerLevel1Str1 from now.\n\n Note that the migrated tables will be granted '" +
40 | s"""Some docs' """ +
41 | s"by default. However, as \nthe table owner, you will have the ability to restrict ACLs on the " +
42 | s"migrated table.\n\n Note that we are currently NOT migrating tables if they use following " +
43 | s"features: \n* Tables read or written using Metaflow / Fast Data Access. \n* Tables written to " +
44 | s"using Kragle. \n* Tables using Psycho pattern. \n* Tables using parallel writer WAP pattern. \nIf " +
45 | s"your table uses any of the above listed features or if there are any other anticipated issues with " +
46 | s"this \nmigration then please contact the #bigdatasupport Slack channel before $migrationDate to " +
47 | s"request a \npause and deferral of the migration.")
48 |
49 | val level2Str1: String =
50 | if (probationGapInDays > 0) s"for ${probationGapInDays} Days"
51 | else ""
52 | val level2Str2: String =
53 | if (probationGapInDays > 0)
54 | s"within the next ${probationGapInDays} Days"
55 | else ""
56 | val level2: (String, String) = (
57 | s"[Hive To Iceberg Migration]: Table %s on Probation Period $level2Str1",
58 | "This email is to inform you that the Hive table %s has been migrated to " +
59 | "the Iceberg format \nas part of the Hive to Iceberg migration process. \n\n If users encounter any " +
60 | s"issues with the newly-created Iceberg table $level2Str2, please refer to " +
61 | "https://go/h2i-docs \nfor instructions on how to revert back to the Hive table and pause the migration or " +
62 | "reach out to #bigdatasupport Slack channel.")
63 |
64 | val level3: (String, String) = (
65 | "[Hive To Iceberg Migration]: Migration of Table %s from Hive to Iceberg Table Format Complete",
66 | "This email serves as a notification to the owner or downstream user of " +
67 | "Hive table %s \nthat it has been successfully migrated to the Iceberg format as part of the " +
68 | "Hive to Iceberg migration process. \n\nNo further action is required from the table users at this point.")
69 | }
70 |
--------------------------------------------------------------------------------
/src/main/scala/com/netflix/migration/modes/Preprocessor.scala:
--------------------------------------------------------------------------------
1 | package com.netflix.migration.modes
2 |
3 | import com.netflix.migration.data.{Job, JobState, MigrationConfig}
4 | import com.netflix.migration.providers.{MetacatServiceProvider, MigrationConfigProvider, SparkSessionProvider}
5 | import com.netflix.migration.services.{MetacatService, StorageService, StorageServiceImpl}
6 | import com.netflix.migration.utils.MigrationConsts.{HIVE_CSV_TEXT, PauseReason}
7 | import com.netflix.migration.utils.Utils
8 | import com.netflix.migration.utils.Utils.{consistentHash, getDataCategory, getDownstreamUsers, getTableOwners, getTableStorageFormat, isHiveTable, isHiveTableEmpty, tableHasTimestampColumn}
9 | import com.typesafe.scalalogging.StrictLogging
10 | import org.apache.spark.sql.SparkSession
11 |
12 | import scala.util.Random.shuffle
13 |
14 | case class Preprocessor() extends CommandMode with StrictLogging {
15 |
16 | val conf: MigrationConfig = MigrationConfigProvider.getMigrationConfig
17 | val spark: SparkSession = SparkSessionProvider.getSparkSession
18 | @transient private[modes] lazy val storageService: StorageService = StorageServiceImpl(conf)
19 | val metacatService: MetacatService = MetacatServiceProvider.getMetacatService
20 |
21 | def getRelevantJobs(): List[Job] = {
22 | val jobsQuery =
23 | s"""
24 | |SELECT *
25 | |FROM ${conf.queueName}
26 | |WHERE ((to_be_processed = false
27 | |AND in_process = false
28 | |AND state = desired_state
29 | |AND desired_state = 'Undefined')
30 | |OR (in_process = true
31 | |AND state = desired_state
32 | |AND desired_state = 'WritesUnblocked'
33 | |AND comm_level2_date < NOW() - INTERVAL probation_gap_days DAY)
34 | |OR (in_process = true
35 | |AND state = desired_state
36 | |AND desired_state = 'HiveDropped'))
37 | |AND migration_paused = false
38 | |LIMIT ${conf.jobsProcessBatchSize}
39 | |""".stripMargin
40 | storageService.getJobs(jobsQuery)
41 | }
42 |
43 | def run(): Unit = {
44 | val jobs = getRelevantJobs()
45 | logger.info(s"Preprocessor with runid=${conf.runId} processing jobs=$jobs")
46 | for (job <- jobs) {
47 | if (consistentHash(job.id, conf.numPreprocessors) == conf.preprocessorId) {
48 | processJob(job)
49 | }
50 | }
51 | }
52 |
53 | /**
54 | * Process the job entity and update its status.
55 | *
56 | * @param job
57 | * the job entity to be processed
58 | */
59 | def processJob(job: Job): Unit = {
60 | val fullTableName = Utils.getFullyQualifiedTableName(job)
61 | println(s"Preprocessing job: $fullTableName")
62 | (job.toBeProcessed, job.inProcess, job.state, job.desiredState) match {
63 | case (0, 0, JobState.Undefined, JobState.Undefined) =>
64 | val tableDtoOpt = metacatService.getTable(job.catalogName, job.dbName, job.tblName)
65 | if (tableDtoOpt.isEmpty) {
66 | job.migrationPaused = 1
67 | job.pauseReason = PauseReason.TABLE_NOT_FOUND.toString
68 | storageService.updateJob(job)
69 | logger.info(
70 | s"Migration paused for table $fullTableName. Pause reason: Table not found.")
71 | return
72 | }
73 |
74 | job.stgFormat = getTableStorageFormat(job)
75 | job.dataCategory = getDataCategory(job).toString
76 | if (job.tblOwners.isEmpty) {
77 | job.tblOwners = getTableOwners(job)
78 | }
79 | if (job.downstreamUsers.isEmpty) {
80 | job.downstreamUsers = getDownstreamUsers(job)
81 | }
82 | storageService.updateJob(job)
83 |
84 | if (isHiveTable(job.catalogName, job.dbName, job.tblName) || isHiveTableEmpty(job)) {
85 | job.state = JobState.Ready
86 | job.desiredState = JobState.WritesBlocked
87 | job.toBeProcessed = 1
88 | storageService.updateJob(job)
89 | } else {
90 | job.migrationPaused = 1
91 | job.pauseReason = PauseReason.IS_ALREADY_ICEBERG.toString
92 | storageService.updateJob(job)
93 | logger.info(
94 | s"The table: $fullTableName is already " +
95 | s"in Iceberg format. Skipping and pausing migration. Job: $job")
96 | }
97 |
98 | case (_, 1, JobState.WritesUnblocked, JobState.WritesUnblocked) =>
99 | job.desiredState = JobState.HiveDropped
100 | storageService.updateJob(job)
101 |
102 | case (_, 1, JobState.HiveDropped, JobState.HiveDropped) =>
103 | job.inProcess = 0
104 | storageService.updateJob(job)
105 |
106 | case _ => // do nothing
107 | }
108 | }
109 | }
110 |
--------------------------------------------------------------------------------
/src/test/scala/com/netflix/migration/utils/SparkTablesGenerator.scala:
--------------------------------------------------------------------------------
1 | package com.netflix.migration.utils
2 |
3 | import com.netflix.migration.data.{Job, MigrationConfig}
4 | import com.netflix.migration.providers.{MetacatServiceProvider, MigrationConfigProvider, SparkSessionProvider}
5 | import com.netflix.migration.services.{MetacatService, StorageService}
6 | import com.netflix.migration.utils.MigrationConsts.TableFormat.TableFormat
7 | import com.netflix.migration.utils.MigrationConsts.{DEFAULT_TABLE_OWNER, DO_NOT_DROP_TAG, DO_NOT_RENAME_TAG, HIVE_SUFFIX, ICEBERG_SUFFIX, TEST_CATALOG, TEST_DATABASE, TEST_TABLE_PREFIX, TEST_THREAD_POOL_SIZE}
8 | import com.netflix.migration.utils.Utils.getMockJobObject
9 | import org.apache.spark.sql
10 | import org.apache.spark.sql.{Encoders, SparkSession}
11 |
12 | import java.util
13 | import java.util.concurrent.{Callable, Executors}
14 | import scala.util.Random
15 |
16 | class SparkTablesGenerator(
17 | numTables: Int,
18 | numRowsPerTable: Int,
19 | format: TableFormat,
20 | storageService: StorageService) {
21 |
22 | val spark: SparkSession = SparkSessionProvider.getSparkSession
23 | val metacatService: MetacatService = MetacatServiceProvider.getMetacatService
24 | val migrationConf: MigrationConfig = MigrationConfigProvider.getMigrationConfig
25 |
26 | def createTables(): Unit = {
27 | dropTables()
28 | val executor = Executors.newFixedThreadPool(TEST_THREAD_POOL_SIZE)
29 | val tasks = new util.ArrayList[Callable[Void]]()
30 | (1 to numTables).foreach { i =>
31 | val table = s"$TEST_CATALOG.$TEST_DATABASE.${TEST_TABLE_PREFIX}$i"
32 | tasks.add(new Callable[Void] {
33 | override def call(): Void = {
34 | createTable(table, format)
35 | writeNumRowsToTable(table)
36 | null
37 | }
38 | })
39 | }
40 | executor.invokeAll(tasks)
41 | executor.shutdown()
42 | }
43 |
44 | def createTable(table: String, format: TableFormat): Unit = {
45 | val table_format = format.toString.toLowerCase
46 | val tableParts = table.split("[.]")
47 | val (catalog_name, db_name, table_name) = (tableParts(0), tableParts(1), tableParts(2))
48 | spark.sql(
49 | s"CREATE TABLE IF NOT EXISTS $table " +
50 | s"(id INT, name STRING, date INT, hour INT) " +
51 | s"USING $table_format PARTITIONED BY (date, hour)")
52 | Utils.metacatService.setOwner(catalog_name, db_name, table_name, DEFAULT_TABLE_OWNER)
53 | println(s"Created test table: $table")
54 | metacatService.setTableTags(
55 | catalog_name,
56 | db_name,
57 | table_name,
58 | Set(DO_NOT_RENAME_TAG, DO_NOT_DROP_TAG))
59 | val job = getMockJobObject(catalog_name, db_name, table_name)
60 | storageService.enqueueJob(job)
61 | }
62 |
63 | def writeNumRowsToTable(table: String): Unit = {
64 | val encoder = Encoders.product[(Int, String, Int, Int)]
65 | val random = new Random()
66 | val data = spark
67 | .range(numRowsPerTable)
68 | .map { i =>
69 | (i.toInt, s"name_$i", 20230101, random.nextInt(10000000))
70 | }(encoder)
71 | .toDF("id", "name", "date", "hour")
72 | .repartition(spark.conf.get("spark.sql.shuffle.partitions").toInt * 2)
73 | writeToTable(table, data)
74 | }
75 |
76 | def writeToTable(table: String, data: sql.DataFrame): Unit = {
77 | data.write.insertInto(s"$table")
78 | }
79 |
80 | def dropTables(): Unit = {
81 | (1 to numTables).foreach { i =>
82 | dropTable(getMockJobObject(TEST_CATALOG, TEST_DATABASE, s"$TEST_TABLE_PREFIX$i"))
83 | }
84 | }
85 |
86 | def dropTable(job: Job): Unit = {
87 | try {
88 | if (metacatService.tableExists(job.catalogName, job.dbName, job.tblName)) {
89 | metacatService.unBlockTableWrites(job.catalogName, job.dbName, job.tblName)
90 | metacatService.forceDropTable(job.catalogName, job.dbName, job.tblName)
91 | }
92 | if (metacatService.tableExists(job.catalogName, job.dbName, job.tblName + HIVE_SUFFIX)) {
93 | metacatService.unBlockTableWrites(job.catalogName, job.dbName, job.tblName + HIVE_SUFFIX)
94 | metacatService.forceDropTable(job.catalogName, job.dbName, job.tblName + HIVE_SUFFIX)
95 | }
96 | if (metacatService.tableExists(job.catalogName, job.dbName, job.tblName + ICEBERG_SUFFIX)) {
97 | metacatService.unBlockTableWrites(
98 | job.catalogName,
99 | job.dbName,
100 | job.tblName + ICEBERG_SUFFIX)
101 | metacatService.forceDropTable(job.catalogName, job.dbName, job.tblName + ICEBERG_SUFFIX)
102 | }
103 | } catch {
104 | case e: Exception => // do nothing
105 | } finally {
106 | storageService.removeJob(job)
107 | }
108 | }
109 | }
110 |
--------------------------------------------------------------------------------
/src/main/scala/com/netflix/migration/modes/Migrator.scala:
--------------------------------------------------------------------------------
1 | package com.netflix.migration.modes
2 |
3 | import com.netflix.migration.data.{Job, MigrationConfig}
4 | import com.netflix.migration.providers.{MetacatServiceProvider, MigrationConfigProvider, SparkSessionProvider}
5 | import com.netflix.migration.services.{MetacatService, StorageService, StorageServiceImpl}
6 | import com.netflix.migration.utils.MigrationConsts.{HIVE_CSV_TEXT, HIVE_PARQUET, HIVE_SUFFIX, PauseReason}
7 | import com.netflix.migration.utils.MigrationRunnerUtils.getFnName
8 | import com.netflix.migration.utils.{MigrationRunnerUtils, Utils}
9 | import com.netflix.migration.utils.Utils.{consistentHash, isHiveTable, isHiveTableEmpty}
10 | import com.typesafe.scalalogging.StrictLogging
11 | import org.apache.spark.sql.SparkSession
12 |
13 | case class Migrator() extends CommandMode with StrictLogging {
14 |
15 | val conf: MigrationConfig = MigrationConfigProvider.getMigrationConfig
16 | val spark: SparkSession = SparkSessionProvider.getSparkSession
17 | val metacatService: MetacatService = MetacatServiceProvider.getMetacatService
18 | @transient private[modes] lazy val storageService: StorageService = StorageServiceImpl(conf)
19 |
20 | def getRelevantJobs(): List[Job] = {
21 | val jobsQuery =
22 | s"""
23 | |SELECT *
24 | |FROM ${conf.queueName}
25 | |WHERE ((to_be_processed = true
26 | |AND comm_level1_date IS NOT NULL
27 | |AND (comm_level1_date < NOW() - INTERVAL initial_gap_days DAY)
28 | |AND state = 'Ready' AND desired_state = 'WritesBlocked')
29 | |OR (in_process = true
30 | |AND state != desired_state))
31 | |AND migration_paused = false
32 | |""".stripMargin
33 | storageService.getJobs(jobsQuery)
34 | }
35 |
36 | def run(): Unit = {
37 | var jobs = getRelevantJobs()
38 | if (jobs.isEmpty) {
39 | logger.warn(s"Migrator with runid=${conf.runId} did not find any jobs to process")
40 | return
41 | }
42 | if (!conf.batchId.isEmpty) {
43 | val numJobsBeforeFilter = jobs.size
44 | jobs = MigrationRunnerUtils.applyBatchIdFilterOnJobs(spark, conf, jobs)
45 | logger.info(
46 | s"Found ${jobs.size} matching batchId filter: ${conf.batchId.get} out of $numJobsBeforeFilter")
47 | }
48 |
49 | logger.info(
50 | s"Migrator with runid=${conf.runId} found ${jobs.size} jobs to process. Jobs list: $jobs")
51 | for (job <- jobs) {
52 | if (consistentHash(job.id, conf.numMigrators) == conf.migratorId) {
53 | migrateJob(job)
54 | }
55 | }
56 | }
57 |
58 | /**
59 | * Migrate the job entity.
60 | *
61 | * @param job
62 | * the job entity to be migrated
63 | */
64 | def migrateJob(job: Job): Unit = {
65 | val fullTableName = Utils.getFullyQualifiedTableName(job)
66 | val tableDtoOpt = metacatService.getTable(job.catalogName, job.dbName, job.tblName)
67 | if (tableDtoOpt.isEmpty && !metacatService
68 | .tableExists(job.catalogName, job.dbName, job.tblName + HIVE_SUFFIX)) {
69 | job.migrationPaused = 1
70 | job.pauseReason = PauseReason.TABLE_NOT_FOUND.toString
71 | storageService.updateJob(job)
72 | logger.warn(s"Migration paused for table $fullTableName. Pause reason: Table not found.")
73 | return
74 | }
75 |
76 | if (job.toBeProcessed == 1) {
77 | if (isHiveTable(job.catalogName, job.dbName, job.tblName)) {
78 | if (job.stgFormat == HIVE_PARQUET || isHiveTableEmpty(job)) {
79 | job.inProcess = 1
80 | job.toBeProcessed = 0
81 | storageService.updateJob(job)
82 | logger.info(s"Started Migrator for table: $fullTableName. Job: $job")
83 | } else {
84 | job.migrationPaused = 1
85 | job.pauseReason = PauseReason.IS_CSV_TEXT.toString
86 | storageService.updateJob(job)
87 | logger.info(
88 | s"Migration paused for table $fullTableName. Table is in $HIVE_CSV_TEXT format.")
89 | return
90 | }
91 | } else {
92 | job.migrationPaused = 1
93 | job.pauseReason = PauseReason.IS_ALREADY_ICEBERG.toString
94 | storageService.updateJob(job)
95 | logger.info(
96 | s"The table: $fullTableName is already in Iceberg format. " +
97 | s"Skipping and pausing migration. Job: $job")
98 | return
99 | }
100 | }
101 | var fun = Utils.getJobNextAction(job)
102 | while (!fun.isEmpty) {
103 | val f = fun.get
104 | f(job)
105 | fun = Utils.getJobNextAction(job)
106 | storageService.updateJob(job)
107 | logger.info(s"Migrator transitioning table: $fullTableName from " +
108 | s"state: ${job.state} to state: ${job.desiredState} using function: ${getFnName(f)}. Job: $job")
109 | }
110 | }
111 | }
112 |
--------------------------------------------------------------------------------
/src/main/scala/com/netflix/migration/modes/Reverter.scala:
--------------------------------------------------------------------------------
1 | package com.netflix.migration.modes
2 |
3 | import com.netflix.migration.MigrationRunner
4 | import com.netflix.migration.data.{Job, JobState, MigrationConfig}
5 | import com.netflix.migration.providers.{MetacatServiceProvider, MigrationConfigProvider}
6 | import com.netflix.migration.services.{MetacatService, StorageService, StorageServiceImpl}
7 | import com.netflix.migration.utils.MigrationConsts.PauseReason
8 | import com.netflix.migration.utils.Utils
9 | import com.netflix.migration.utils.Utils.{isHiveTable, revertFinalizedTable, revertHelper}
10 | import com.typesafe.scalalogging.StrictLogging
11 |
12 | case class Reverter() extends CommandMode with StrictLogging {
13 |
14 | val conf: MigrationConfig = MigrationConfigProvider.getMigrationConfig
15 | val metacatService: MetacatService = MetacatServiceProvider.getMetacatService
16 | @transient private[modes] lazy val storageService: StorageService = StorageServiceImpl(conf)
17 |
18 | def run(): Unit = {
19 | val tablesToBeReverted = MigrationRunner.tablesToBeReverted.split(",")
20 | for (table <- tablesToBeReverted) {
21 | val tableToBeReverted = table.trim.toLowerCase
22 | if (tableToBeReverted == null) {
23 | throw new IllegalArgumentException("Table to be reverted is null")
24 | }
25 | val tableParts = tableToBeReverted.split("[.]")
26 | if (tableParts.length != 3) {
27 | throw new IllegalArgumentException(
28 | "Table name is not fully qualified. " +
29 | "Table name format must be ..")
30 | }
31 | val jobsQuery =
32 | s"""
33 | |SELECT *
34 | |FROM ${conf.queueName}
35 | |WHERE catalog_name = '${tableParts(0)}'
36 | |AND db_name = '${tableParts(1)}'
37 | |AND tbl_name = '${tableParts(2)}'
38 | |""".stripMargin
39 | val jobs = storageService.getJobs(jobsQuery)
40 | logger.info(s"Reverter with runid=${conf.runId} processing jobs=$jobs")
41 | if (jobs.size > 1) {
42 | throw new IllegalArgumentException(
43 | "There is more than single table to be reverted. " +
44 | "Please specify single table to be reverted")
45 | }
46 | val job = jobs.head
47 | if (!isHiveTable(job.catalogName, job.dbName, job.tblName) && metacatService.tableExists(
48 | job.catalogName,
49 | job.dbName,
50 | job.tblName)) {
51 | revertHelper(job)
52 | revertJob(job)
53 | }
54 | }
55 | }
56 |
57 | /**
58 | * Revert the state of a job entity.
59 | *
60 | * @param job
61 | * the job entity to be reverted
62 | */
63 | def revertJob(job: Job): Unit = {
64 | val fullTableName = Utils.getFullyQualifiedTableName(job)
65 | job.migrationPaused = 1
66 | job.pauseReason = PauseReason.REVERTED.toString
67 | job.state = JobState.WritesUnblocked
68 | job.desiredState = JobState.WritesBlocked
69 | storageService.updateJob(job)
70 | var fun = Utils.getJobNextAction(job)
71 | logger.info(s"Revert started for table: $fullTableName at state: ${job.state}. Job: $job")
72 | while (!fun.isEmpty) {
73 | val f = fun.get
74 | f(job)
75 | fun = Utils.getJobNextAction(job)
76 | storageService.updateJob(job)
77 | logger.info(
78 | s"Reverter transitioning table: $fullTableName from " +
79 | s"state: ${job.state} to state: ${job.desiredState}. Job: $job")
80 | }
81 | }
82 |
83 | /**
84 | * Bulk revert set of migration tables to Hive table format.
85 | *
86 | * @param tables
87 | * The set of tables to be reverted
88 | */
89 | def bulkRevertTables(tables: Set[(String, String, String)]): Unit = {
90 | for (table <- tables) {
91 | val (catalogName, dbName, tableName) = table
92 | val job = storageService.getJob(catalogName, dbName, tableName)
93 | revertHelper(job)
94 | revertJob(job)
95 | }
96 | }
97 |
98 | /**
99 | * Bulk revert set of migration finalized tables to Hive table format.
100 | *
101 | * @param tables
102 | * The set of tables to be reverted
103 | */
104 | def bulkRevertFinalizedTables(tables: Set[(String, String, String)]): Unit = {
105 | for (table <- tables) {
106 | if (revertFinalizedTable(table)) {
107 | val (catalogName, dbName, tableName) = table
108 | val job = storageService.getJob(catalogName, dbName, tableName)
109 | job.toBeProcessed = 1
110 | job.inProcess = 0
111 | job.state = JobState.Ready
112 | job.desiredState = JobState.WritesBlocked
113 | job.migrationPaused = 1
114 | job.pauseReason = PauseReason.REVERTED.toString
115 | storageService.updateJob(job)
116 | }
117 | }
118 | }
119 | }
120 |
--------------------------------------------------------------------------------
/src/main/scala/com/netflix/migration/MigrationRunner.scala:
--------------------------------------------------------------------------------
1 | package com.netflix.migration
2 |
3 | import com.netflix.migration.data.MigrationCommandMode._
4 | import com.netflix.migration.data.{MigrationCommandMode, MigrationConfig}
5 | import com.netflix.migration.modes._
6 | import com.netflix.migration.providers.{IcebergTableServiceProvider, MetacatServiceProvider, MigrationConfigProvider, NdcServiceProvider, SparkSessionProvider}
7 | import com.netflix.migration.services.{MetacatService, MetacatServiceImpl, NdcService, NdcServiceImpl, TrinoIcebergTableServiceImpl}
8 | import com.netflix.migration.utils.{MigrationConsts, StringUtils, Utils}
9 | import com.typesafe.scalalogging.StrictLogging
10 | import org.apache.spark.sql.SparkSession
11 |
12 | import java.time.{Instant, LocalDate}
13 |
14 | case class MigrationRunner() extends StrictLogging {
15 |
16 | val conf: MigrationConfig = MigrationConfigProvider.getMigrationConfig
17 | val spark: SparkSession = SparkSessionProvider.getSparkSession
18 |
19 | def process(): Unit = {
20 | logger.info(
21 | s"Started ${conf.commandMode} with date=${conf.processDate} and runid=${conf.runId}")
22 | try {
23 | conf.commandMode match {
24 | case COMMUNICATOR => Communicator().run()
25 | case PREPROCESSOR => Preprocessor().run()
26 | case MIGRATOR => Migrator().run()
27 | case REVERTER => Reverter().run()
28 | case SHADOWER => Shadower().run()
29 | }
30 | } finally {
31 | logger.info(
32 | s"Finished ${conf.commandMode} with date=${conf.processDate} and runid=${conf.runId}")
33 | }
34 | }
35 | }
36 |
37 | object MigrationRunner extends StrictLogging {
38 | var tablesToBeReverted: String = _
39 | def main(args: Array[String]): Unit = {
40 | logger.info("Migration app. Instantiating Spark Session")
41 | val argsMap = StringUtils.parseArgs(args)
42 | logger.info("Args received: " + argsMap)
43 | createMigrationConf(argsMap)
44 | val spark: SparkSession = SparkSessionProvider.getSparkSession
45 | try {
46 | if (Utils.migrationConf.dryRun) {
47 | // TODO: Extend the dryRun mode (Each command mode should have a dryRun implementation)
48 | logger.info("Spark version running: " + spark.version)
49 | } else {
50 | tablesToBeReverted = argsMap.getOrElse("tablesToBeReverted", null)
51 | val runner = MigrationRunner()
52 | runner.process()
53 | }
54 | } finally {
55 | logger.info("Stopping spark session..")
56 | spark.stop()
57 | }
58 | }
59 |
60 | /**
61 | * Create a MigrationConfig object from command line arguments
62 | * @param argsMap
63 | * A map of command line arguments, where the key is the argument name and the value is the
64 | * argument value
65 | * @return
66 | * A MigrationConfig object
67 | */
68 | def createMigrationConf(argsMap: Map[String, String]): MigrationConfig = {
69 | val migrationConf = MigrationConfig(
70 | commandMode = MigrationCommandMode.parse(argsMap.getOrElse("mode", "MIGRATOR")),
71 | runLocally = argsMap.getOrElse("local", "true").toUpperCase.equals("TRUE"),
72 | dryRun = argsMap
73 | .getOrElse("dryrun", "false")
74 | .toUpperCase
75 | .equals("TRUE"), // default to dryRun mode for now
76 | processDate = LocalDate.now(),
77 | processStartTime = Instant.now(),
78 | runId = sys.env.getOrElse("GENIE_JOB_ID", java.util.UUID.randomUUID.toString),
79 | distributedMode = argsMap.getOrElse("distributed", "false").toUpperCase.equals("TRUE"),
80 | dbEnv = argsMap
81 | .getOrElse("dbEnv", MigrationConsts.TEST_ENV)
82 | .toLowerCase, // Use test database by default
83 | jobsProcessBatchSize = argsMap
84 | .getOrElse("jobsProcessBatchSize", MigrationConsts.jobsProcessBatchSize.toString)
85 | .toInt,
86 | queueName = argsMap.getOrElse("queueName", MigrationConsts.queueName),
87 | archiveHive = argsMap.getOrElse("archiveHive", "true").toUpperCase.equals("TRUE"),
88 | setIcebergAcls = argsMap.getOrElse("setIcebergAcls", "true").toUpperCase.equals("TRUE"),
89 | dbName = argsMap.getOrElse("dbName", MigrationConsts.JOBS_DB_NAME),
90 | batchId = argsMap.get("batchid"),
91 | batchTableName = argsMap.getOrElse("batchTableName", MigrationConsts.batchTableName),
92 | migratorId = argsMap.getOrElse("migratorId", MigrationConsts.MIGRATOR_ID.toString).toInt,
93 | numMigrators =
94 | argsMap.getOrElse("numMigrators", MigrationConsts.NUM_MIGRATORS.toString).toInt,
95 | preprocessorId =
96 | argsMap.getOrElse("preprocessorId", MigrationConsts.PREPROCESSOR_ID.toString).toInt,
97 | numPreprocessors =
98 | argsMap.getOrElse("numPreprocessors", MigrationConsts.NUM_PREPROCESSORS.toString).toInt)
99 |
100 | MigrationConfigProvider.init(migrationConf)
101 | val metacatService: MetacatService = MetacatServiceImpl(migrationConf)
102 | MetacatServiceProvider.init(metacatService)
103 | val ndcService: NdcService = NdcServiceImpl(migrationConf)
104 | NdcServiceProvider.init(ndcService)
105 | IcebergTableServiceProvider.init(TrinoIcebergTableServiceImpl(migrationConf))
106 | if (SparkSessionProvider.getSparkSession == null) {
107 | lazy val spark: SparkSession = Utils.getSparkSession()
108 | SparkSessionProvider.init(spark)
109 | }
110 | migrationConf
111 | }
112 | }
113 |
--------------------------------------------------------------------------------
/src/main/scala/com/netflix/migration/services/NdcServiceImpl.scala:
--------------------------------------------------------------------------------
1 | package com.netflix.migration.services
2 |
3 | import com.netflix.ndc.client.Client
4 | import com.netflix.migration.data.MigrationConfig
5 | import com.netflix.migration.utils.MigrationRunnerUtils.retry
6 | import com.netflix.migration.utils.StringUtils.getTableQualifiedName
7 | import com.netflix.migration.utils.MigrationConsts
8 | import com.netflix.migration.utils.MigrationConsts.DataCategory.{CORE_PI, DataCategory, NO_PI, PI, UNKNOWN}
9 | import com.netflix.ndc.common.QualifiedName
10 | import com.netflix.ndc.common.dto.MetadataDto.MetadataDtoBuilder
11 | import com.netflix.ndc.common.dto.{MetadataDto, MetadataResponseDto}
12 | import com.typesafe.scalalogging.StrictLogging
13 |
14 | import scala.collection.JavaConverters._
15 |
16 | case class NdcServiceImpl(conf: MigrationConfig) extends NdcService with StrictLogging {
17 | @transient private[services] lazy val ndcClient = Client
18 | .builder()
19 | .withHost(MigrationConsts.NDC_HOST_URL_PROPERTY)
20 | .withUserName(MigrationConsts.NDC_USERNAME_PROPERTY)
21 | .withClientAppName(MigrationConsts.NDC_CLIENT_APP_NAME_PROPERTY)
22 | .build()
23 |
24 | /**
25 | * Get table metadata
26 | *
27 | * @param catalogName
28 | * @param dbName
29 | * @param tableName
30 | */
31 | def getMetadata(catalogName: String, dbName: String, tableName: String): MetadataResponseDto = {
32 | val qualifiedName = QualifiedName.fromString(
33 | String.format("ndc://hive/%s/%s/%s", catalogName, dbName, tableName))
34 | val metadataDtos = ndcClient.getMetadataV0Api.getMetadata(qualifiedName, true)
35 | if (!metadataDtos.isEmpty) {
36 | logger.info(s"Successfully getting NDC metadata for table ${qualifiedName.getFullName}")
37 | metadataDtos.get(0)
38 | } else null
39 | }
40 |
41 | /**
42 | * Sets table metadata
43 | *
44 | * @param metadataDtos
45 | */
46 | def setMetadata(metadataDtos: List[MetadataDto]): Unit = {
47 | retry(MigrationConsts.NUM_RETRIES)(
48 | ndcClient.getBulkMetadataV0Api.setMetadata(metadataDtos.asJava))
49 | logger.info(
50 | s"Successfully setting NDC metadata for table ${metadataDtos.head.getName.getFullName}")
51 | }
52 |
53 | /**
54 | * Get table owners
55 | *
56 | * @param catalogName
57 | * @param dbName
58 | * @param tableName
59 | */
60 | def getTableOwners(catalogName: String, dbName: String, tableName: String): List[String] = {
61 | val tableQualifiedName = getTableQualifiedName(catalogName, dbName, tableName)
62 | val metadataResponseDto = getMetadata(catalogName, dbName, tableName)
63 | if (metadataResponseDto != null) {
64 | metadataResponseDto.getVerifiedTechnicalContacts.asScala.toList
65 | } else {
66 | logger.warn(s"Failed to get table owners information for table $tableQualifiedName")
67 | List.empty[String]
68 | }
69 | }
70 |
71 | /**
72 | * Get table data category
73 | *
74 | * @param catalogName
75 | * @param dbName
76 | * @param tableName
77 | */
78 | def getDataCategory(catalogName: String, dbName: String, tableName: String): DataCategory = {
79 | val tableQualifiedName = getTableQualifiedName(catalogName, dbName, tableName)
80 | Option(getMetadata(catalogName, dbName, tableName)) match {
81 | case Some(metadataResponseDto) =>
82 | Option(metadataResponseDto.getDlmDataCategoryTags) match {
83 | case Some(dlmDataCategoryTags) =>
84 | dlmDataCategoryTags.asScala.get("core_pi") match {
85 | case Some(_) => CORE_PI
86 | case None =>
87 | dlmDataCategoryTags.asScala.get("pi") match {
88 | case Some("yes") => PI
89 | case Some("no") => NO_PI
90 | case _ => UNKNOWN
91 | }
92 | }
93 | case None => UNKNOWN
94 | }
95 | case None =>
96 | logger.warn(s"Failed to get data category information for table $tableQualifiedName")
97 | UNKNOWN
98 | }
99 | }
100 |
101 | /**
102 | * Create MetadataDto object from MetadataResponseDto
103 | *
104 | * @param metadataResponseDto
105 | */
106 | def createMetadataDto(metadataResponseDto: MetadataResponseDto): MetadataDto = {
107 | MetadataDto
108 | .builder()
109 | .asInstanceOf[MetadataDtoBuilder[MetadataDto, _]]
110 | .name(metadataResponseDto.getName)
111 | .asInstanceOf[MetadataDtoBuilder[MetadataDto, _]]
112 | .id(metadataResponseDto.getId)
113 | .asInstanceOf[MetadataDtoBuilder[MetadataDto, _]]
114 | .audit(metadataResponseDto.getAudit)
115 | .asInstanceOf[MetadataDtoBuilder[MetadataDto, _]]
116 | .dataSize(metadataResponseDto.getDataSize)
117 | .asInstanceOf[MetadataDtoBuilder[MetadataDto, _]]
118 | .disavowedTechnicalContactPandoraIDs(
119 | metadataResponseDto.getDisavowedTechnicalContactPandoraIDs)
120 | .asInstanceOf[MetadataDtoBuilder[MetadataDto, _]]
121 | .dlmDataCategory(metadataResponseDto.getDlmDataCategory)
122 | .asInstanceOf[MetadataDtoBuilder[MetadataDto, _]]
123 | .dlmDataCategoryTags(metadataResponseDto.getDlmDataCategoryTags)
124 | .asInstanceOf[MetadataDtoBuilder[MetadataDto, _]]
125 | .dlmMetadata(metadataResponseDto.getDlmMetadata)
126 | .asInstanceOf[MetadataDtoBuilder[MetadataDto, _]]
127 | .dlmOwners(metadataResponseDto.getDlmOwners)
128 | .asInstanceOf[MetadataDtoBuilder[MetadataDto, _]]
129 | .inferredTechnicalContactPandoraIDs(
130 | metadataResponseDto.getInferredTechnicalContactPandoraIDs)
131 | .asInstanceOf[MetadataDtoBuilder[MetadataDto, _]]
132 | .isTemporary(metadataResponseDto.getIsTemporary)
133 | .asInstanceOf[MetadataDtoBuilder[MetadataDto, _]]
134 | .labels(metadataResponseDto.getLabels)
135 | .asInstanceOf[MetadataDtoBuilder[MetadataDto, _]]
136 | .lifetime(metadataResponseDto.getLifetime)
137 | .asInstanceOf[MetadataDtoBuilder[MetadataDto, _]]
138 | .location(metadataResponseDto.getLocation)
139 | .asInstanceOf[MetadataDtoBuilder[MetadataDto, _]]
140 | .metadata(metadataResponseDto.getMetadata)
141 | .asInstanceOf[MetadataDtoBuilder[MetadataDto, _]]
142 | .sourceLink(metadataResponseDto.getSourceLink)
143 | .asInstanceOf[MetadataDtoBuilder[MetadataDto, _]]
144 | .syncDate(metadataResponseDto.getSyncDate)
145 | .asInstanceOf[MetadataDtoBuilder[MetadataDto, _]]
146 | .verifiedTechnicalContactPandoraIDs(
147 | metadataResponseDto.getVerifiedTechnicalContactPandoraIDs)
148 | .asInstanceOf[MetadataDtoBuilder[MetadataDto, _]]
149 | .build
150 | }
151 | }
152 |
--------------------------------------------------------------------------------
/src/main/scala/com/netflix/migration/services/MetacatService.scala:
--------------------------------------------------------------------------------
1 | package com.netflix.migration.services
2 |
3 | import com.netflix.metacat.common.dto.TableDto
4 |
5 | import scala.collection.mutable
6 |
7 | /**
8 | * An interface for interacting with Metacat.
9 | */
10 | trait MetacatService {
11 |
12 | /**
13 | * Renames the given table to the new table name.
14 | *
15 | * @param catalogName
16 | * The catalog name.
17 | * @param dbName
18 | * The database name.
19 | * @param tableName
20 | * The table name.
21 | * @param newTableName
22 | * The new table name.
23 | */
24 | def renameTable(
25 | catalogName: String,
26 | dbName: String,
27 | tableName: String,
28 | newTableName: String): Unit
29 |
30 | /**
31 | * Gets tags on the table
32 | *
33 | * @param catalogName
34 | * @param dbName
35 | * @param tableName
36 | */
37 | def getTableTags(catalogName: String, dbName: String, tableName: String): mutable.Set[String]
38 |
39 | /**
40 | * Sets tags on the table
41 | *
42 | * @param catalogName
43 | * @param dbName
44 | * @param tableName
45 | * @param tag
46 | */
47 | def setTableTag(catalogName: String, dbName: String, tableName: String, tag: String): Unit
48 |
49 | /**
50 | * Append the given tags to the set of existing tags on the table.
51 | *
52 | * @param catalogName The catalog name.
53 | * @param dbName The db name.
54 | * @param tableName The table name.
55 | * @param tag The set of tags.
56 | */
57 | def appendTableTags(catalogName: String, dbName: String, tableName: String, tags: Set[String]): Unit
58 |
59 | /**
60 | * Sets and replaces the tags on the table
61 | *
62 | * @param catalogName
63 | * @param dbName
64 | * @param tableName
65 | * @param tag
66 | */
67 | def setTableTags(catalogName: String, dbName: String, tableName: String, tag: Set[String]): Unit
68 |
69 | /**
70 | * Sets and replaces the tags on the table if they existed before
71 | *
72 | * @param catalogName
73 | * @param dbName
74 | * @param tableName
75 | * @param tag
76 | */
77 | def setTableTagsIfExists(
78 | catalogName: String,
79 | dbName: String,
80 | tableName: String,
81 | preExistingTags: mutable.Set[String],
82 | tag: Set[String]): Unit
83 |
84 | /**
85 | * Remove tags from the table
86 | *
87 | * @param catalogName
88 | * @param dbName
89 | * @param tableName
90 | * @param tag
91 | */
92 | def removeTableTag(catalogName: String, dbName: String, tableName: String, tag: String): Unit
93 |
94 | /**
95 | * Remove tags from the table
96 | *
97 | * @param catalogName
98 | * @param dbName
99 | * @param tableName
100 | * @param tag
101 | */
102 | def removeTableTags(
103 | catalogName: String,
104 | dbName: String,
105 | tableName: String,
106 | tag: Set[String]): Unit
107 |
108 | /**
109 | * Remove tags from the table if they existed before
110 | *
111 | * @param catalogName
112 | * @param dbName
113 | * @param tableName
114 | * @param preExistingTags
115 | * @param tag
116 | */
117 | def removeTableTagsIfExists(
118 | catalogName: String,
119 | dbName: String,
120 | tableName: String,
121 | preExistingTags: mutable.Set[String],
122 | tag: Set[String]): Unit
123 |
124 | /**
125 | * Sets a tag on the table that blocks table updates (renames/deletes/updates).
126 | *
127 | * @param catalogName
128 | * @param dbName
129 | * @param tableName
130 | */
131 | def blockTableWrites(catalogName: String, dbName: String, tableName: String): Unit
132 |
133 | /**
134 | * Removes a tag on the table that unblocks table updates (renames/deletes/updates).
135 | *
136 | * @param catalogName
137 | * @param dbName
138 | * @param tableName
139 | */
140 | def unBlockTableWrites(catalogName: String, dbName: String, tableName: String): Unit
141 |
142 | /**
143 | * Checks if the given table exists.
144 | *
145 | * @param catalogName
146 | * The catalog name.
147 | * @param dbName
148 | * The database name.
149 | * @param tableName
150 | * The table name.
151 | * @return
152 | */
153 | def tableExists(catalogName: String, dbName: String, tableName: String): Boolean
154 |
155 | /**
156 | * Deletes the given table.
157 | *
158 | * @param catalogName
159 | * @param dbName
160 | * @param tableName
161 | */
162 | def deleteTable(catalogName: String, dbName: String, tableName: String): Unit
163 |
164 | /**
165 | * Force drop given table.
166 | *
167 | * @param catalogName
168 | * @param dbName
169 | * @param tableName
170 | */
171 | def forceDropTable(catalogName: String, dbName: String, tblName: String): Unit
172 |
173 | /**
174 | * Update the table with the given Dto.
175 | *
176 | * @param catalogName
177 | * @param databaseName
178 | * @param tableName
179 | * @return
180 | */
181 | def updateTable(
182 | catalogName: String,
183 | databaseName: String,
184 | tableName: String,
185 | tableDto: TableDto): Unit
186 |
187 | /**
188 | * Get the table from metacat.
189 | *
190 | * @param catalogName
191 | * @param databaseName
192 | * @param tableName
193 | * @param includeInfo
194 | * @param includeDefinitionMetadata
195 | * @param includeDataMetadata
196 | * @return
197 | */
198 | def getTable(
199 | catalogName: String,
200 | databaseName: String,
201 | tableName: String,
202 | includeInfo: Boolean = true,
203 | includeDefinitionMetadata: Boolean = true,
204 | includeDataMetadata: Boolean = true): Option[TableDto]
205 |
206 | /**
207 | * Get table storage format.
208 | *
209 | * @param catalogName
210 | * @param databaseName
211 | * @param tableName
212 | * @return
213 | */
214 | def getTableStorageFormat(catalogName: String, databaseName: String, tableName: String): String
215 |
216 | /**
217 | * Get table partition count.
218 | *
219 | * @param catalogName
220 | * @param databaseName
221 | * @param tableName
222 | * @return
223 | */
224 | def getPartitionCount(catalogName: String, databaseName: String, tableName: String): Int
225 |
226 | /**
227 | * Gets table data size
228 | *
229 | * @param catalogName
230 | * @param dbName
231 | * @param tableName
232 | */
233 | def getTableDataSize(catalogName: String, dbName: String, tableName: String): Long
234 |
235 | /**
236 | * Set auth to strict secure
237 | *
238 | * @param catalogName
239 | * @param dbName
240 | * @param tableName
241 | */
242 | def setStrictSecure(catalogName: String, dbName: String, tableName: String): Unit
243 |
244 | /**
245 | * Set table owner
246 | *
247 | * @param catalogName
248 | * @param dbName
249 | * @param tableName
250 | */
251 | def setOwner(catalogName: String, dbName: String, tableName: String, owner: String): Unit
252 | }
253 |
--------------------------------------------------------------------------------
/src/main/scala/com/netflix/migration/modes/Communicator.scala:
--------------------------------------------------------------------------------
1 | package com.netflix.migration.modes
2 |
3 | import com.netflix.migration.data.{EmailStrings, Job, JobState, MigrationConfig}
4 | import com.netflix.migration.providers.{MetacatServiceProvider, MigrationConfigProvider, SparkSessionProvider}
5 | import com.netflix.migration.services.{MetacatService, StorageService, StorageServiceImpl}
6 | import com.netflix.migration.utils.{EmailSender, Utils}
7 | import com.netflix.migration.utils.MigrationConsts.{HIVE_CSV_TEXT, HIVE_PARQUET, PauseReason, RECIPIENT_BATCH_SIZE, millisecondsPerDay}
8 | import com.netflix.migration.utils.Utils.{createBooleanMap, isHiveTable, isHiveTableEmpty, jobsGroupByOwners}
9 | import com.typesafe.scalalogging.StrictLogging
10 | import org.apache.spark.sql.SparkSession
11 |
12 | import java.time.Instant
13 | import scala.util.control.Breaks.break
14 |
15 | case class Communicator() extends CommandMode with StrictLogging {
16 | val conf: MigrationConfig = MigrationConfigProvider.getMigrationConfig
17 | val metacatService: MetacatService = MetacatServiceProvider.getMetacatService
18 |
19 | val spark: SparkSession = SparkSessionProvider.getSparkSession
20 | @transient private[modes] lazy val storageService: StorageService = StorageServiceImpl(conf)
21 |
22 | def getRelevantJobs(): List[Job] = {
23 | val jobsQuery =
24 | s"""
25 | |SELECT *
26 | |FROM ${conf.queueName}
27 | |WHERE ((to_be_processed = 1
28 | |AND comm_level1_date IS NULL)
29 | |OR (in_process = 1
30 | |AND comm_level1_date IS NOT NULL
31 | |AND state = 'WritesUnblocked'
32 | |AND desired_state = 'WritesUnblocked'
33 | |AND comm_level2_date IS NULL)
34 | |OR (in_process = 0
35 | |AND comm_level2_date IS NOT NULL
36 | |AND state = 'HiveDropped'
37 | |AND desired_state = 'HiveDropped'
38 | |AND comm_level3_date IS NULL))
39 | |AND migration_paused = false
40 | |""".stripMargin
41 | storageService.getJobs(jobsQuery)
42 | }
43 |
44 | def run(): Unit = {
45 | val jobs = getRelevantJobs()
46 | logger.info(s"Communicator with runid=${conf.runId} processing jobs=$jobs")
47 | communicate(jobs)
48 | }
49 |
50 | /**
51 | * Send email communication to the job's downstream users and owners.
52 | *
53 | * @param jobs
54 | * the list of job entities for which the communication needs to be sent
55 | */
56 | private def communicate(jobs: List[Job]): Unit = {
57 |
58 | val ownerToTablesMap = jobsGroupByOwners(jobs)
59 | val ownerToCommunicatedMap = createBooleanMap(ownerToTablesMap)
60 |
61 | for (job <- jobs) {
62 | val table = s"${job.catalogName}.${job.dbName}.${job.tblName}"
63 | val currentTimeMillis = Instant.now().toEpochMilli
64 | val initialDaysInMillis: Long = job.initialGapInDays * millisecondsPerDay
65 | val probationDaysInMillis: Long = job.probationGapInDays * millisecondsPerDay
66 | val tblOwners = job.tblOwners.filter(email => email.contains("@netflix.com"))
67 | var downstreamUsers = job.downstreamUsers.diff(job.tblOwners)
68 | downstreamUsers = downstreamUsers - "bdw_data_detection@netflix.com"
69 | downstreamUsers = downstreamUsers.filter(email => email.contains("@netflix.com"))
70 |
71 | def sendEmail(to: Set[String], subject: String, body: String): Unit = {
72 | EmailSender.send(
73 | to,
74 | subject.format(table),
75 | body.format(table, table, tblOwners.mkString(", ")),
76 | EmailSender.Text)
77 | }
78 |
79 | def sendEmailToOwners(subject: String, body: String): Unit = {
80 | for (owner <- tblOwners) {
81 | if (!ownerToCommunicatedMap(owner) && owner.contains("@netflix.com")) {
82 | val tables = ownerToTablesMap(owner)
83 | EmailSender.send(
84 | Set(owner),
85 | subject,
86 | body.format("\n" + tables.map(table => s"* $table. ").mkString("\n") + "\n"),
87 | EmailSender.Html)
88 | ownerToCommunicatedMap.put(owner, true)
89 | }
90 | }
91 | }
92 |
93 | (job.toBeProcessed, job.inProcess, job.state, job.desiredState) match {
94 | case (1, _, _, _) if job.commLevel1Date < 0 =>
95 | val fullTableName = Utils.getFullyQualifiedTableName(job)
96 | if (metacatService.tableExists(job.catalogName, job.dbName, job.tblName)) {
97 | if (isHiveTable(job.catalogName, job.dbName, job.tblName)) {
98 | if (job.stgFormat == HIVE_PARQUET || isHiveTableEmpty(job)) {
99 | if (tblOwners.nonEmpty) {
100 | if (job.initialGapInDays > 0) {
101 | if (downstreamUsers.nonEmpty) {
102 | val (subjectForDownstreamUser, bodyForDownstreamUser) =
103 | EmailStrings(
104 | job.initialGapInDays,
105 | job.probationGapInDays).level1DownstreamUserMsg
106 | val downstreamUsersSeq = downstreamUsers.grouped(RECIPIENT_BATCH_SIZE)
107 | downstreamUsersSeq.foreach { downstreamUsersBatch =>
108 | sendEmail(downstreamUsersBatch, subjectForDownstreamUser, bodyForDownstreamUser)
109 | }
110 | }
111 |
112 | val (subjectForOwner, bodyForOwner) =
113 | EmailStrings(
114 | job.initialGapInDays,
115 | job.probationGapInDays).level1TableOwnerMsg
116 | sendEmailToOwners(subjectForOwner, bodyForOwner)
117 | }
118 | }
119 | job.commLevel1Date = currentTimeMillis
120 | storageService.updateJob(job)
121 | } else {
122 | job.migrationPaused = 1
123 | job.pauseReason = PauseReason.IS_CSV_TEXT.toString
124 | storageService.updateJob(job)
125 | logger.info(
126 | s"Migration paused for table $fullTableName. Pause reason: Table is in $HIVE_CSV_TEXT format.")
127 | }
128 | } else {
129 | job.migrationPaused = 1
130 | job.pauseReason = PauseReason.IS_ALREADY_ICEBERG.toString
131 | storageService.updateJob(job)
132 | logger.info(
133 | s"The table: $fullTableName is already " +
134 | s"in Iceberg format. Skipping and pausing migration. Job: $job")
135 | }
136 | } else {
137 | job.migrationPaused = 1
138 | job.pauseReason = PauseReason.TABLE_NOT_FOUND.toString
139 | storageService.updateJob(job)
140 | logger.info(
141 | s"Migration paused for table $fullTableName. Pause reason: Table not found.")
142 | }
143 |
144 | case (_, 1, JobState.WritesUnblocked, JobState.WritesUnblocked)
145 | if job.commLevel2Date < 0 && (currentTimeMillis - job.commLevel1Date) >= initialDaysInMillis =>
146 | if (job.probationGapInDays > 0) {
147 | val (subject, body) =
148 | EmailStrings(job.initialGapInDays, job.probationGapInDays).level2
149 | val to = downstreamUsers.union(tblOwners)
150 | val toUsers = to.grouped(RECIPIENT_BATCH_SIZE)
151 | toUsers.foreach { toUsersBatch =>
152 | sendEmail(toUsersBatch, subject, body)
153 | }
154 | }
155 | job.commLevel2Date = currentTimeMillis
156 | storageService.updateJob(job)
157 |
158 | case (_, 0, JobState.HiveDropped, JobState.HiveDropped)
159 | if job.commLevel3Date < 0 && (currentTimeMillis - job.commLevel2Date) >= probationDaysInMillis =>
160 | if (job.initialGapInDays > 0 || job.probationGapInDays > 0) {
161 | val (subject, body) =
162 | EmailStrings(job.initialGapInDays, job.probationGapInDays).level3
163 | val to = downstreamUsers.union(tblOwners)
164 | val toUsers = to.grouped(RECIPIENT_BATCH_SIZE)
165 | toUsers.foreach { toUsersBatch =>
166 | sendEmail(toUsersBatch, subject, body)
167 | }
168 | }
169 | job.commLevel3Date = currentTimeMillis
170 | storageService.updateJob(job)
171 |
172 | case _ => // do nothing
173 | }
174 | }
175 | }
176 | }
177 |
--------------------------------------------------------------------------------
/README.md:
--------------------------------------------------------------------------------
1 | # Hive to Iceberg Migration Tooling
2 |
3 | Note: This repository is intended to be a reference for broader community and will not be actively maintained by Netflix.
4 |
5 | ## Purpose
6 |
7 | The key purpose of this tool is to (bulk) migrate tables in Hive table format into an Iceberg table format.
8 |
9 | ## Use
10 |
11 | This tool uses few Netflix's internal services such as Lineage Logging, Metacat, Netflix Data Catalog, but we believe these can be replaced by OSS alternatives or similar systems that your organization could be using.
12 |
13 | Here we present few pointers to use this tool. The Hive tables that need to be migrated can be ingested into migration queue by listing the tables in ```src/main/scala/com/netflix/migration/utils/jobs.txt``` and by running ```src/main/scala/com/netflix/migration/utils/IngestJobs.scala```.
14 |
15 | Once jobs are ingested, using daily (or any other schedule granularity) scheduled workflows, invoke different components (PREPROCESSOR, COMMUNICATOR, MIGRATOR, SHADOWER, REVERTER). Each workflow could correspond to a different component.
16 |
17 | For an instance, below workflow creates an scheduled instance for PREPROCESSOR:
18 | ```
19 | Trigger:
20 | cron: 0 0 * * * # Means: run everyday at midnight
21 | tz: US/Pacific # Means: specified cron is in Pacific Time.
22 | Variables:
23 | migration_jar: ../build/libs/hive2iceberg-migration-all.jar
24 | migration_main: com.netflix.migration.MigrationRunner
25 | migration_spark_version: 3.5
26 | Workflow:
27 | id: hive2iceberg_migration_preprocessor
28 | name: hive2iceberg_migration_preprocessor
29 | jobs:
30 | - job:
31 | id: hive_to_iceberg_migration_job
32 | spark:
33 | app_args:
34 | - mode=PREPROCESSOR
35 | - jobsProcessBatchSize=5000
36 | - dbEnv="prod"
37 | - local=false
38 | - dryrun=false
39 | class: ${migration_main}
40 | script: ${migration_jar}
41 | version: ${migration_spark_version}
42 | type: Spark
43 | ```
44 |
45 | ## Design
46 |
47 | ### Components and Responsibilities:
48 |
49 | To automate the migration from Hive to Iceberg, we can use several key components. The "Metadata Table" (MT) acts as both a job queue and metadata store for tables to be migrated.
50 | For the newly inserted jobs in the MT, the “Preprocessor” (PR) populates the information such as table owners, downstream users, current state, and desired state etc and makes it available for other components to process the jobs. The "Communicator" (CR) scans the MT for jobs and retrieves the list of table owners and downstream users from the MT for these jobs. Based on the stage of migration recorded in the MT, the CR sends the appropriate level of communication (level 1, level 2, or level 3) and stores the communication level and timestamp in the MT. The "Migrator" (MR) looks at the MT for jobs to be processed and advances to the next state of migration based on the communication level. The MR also stores information about the current state of migration in the MT. These components can be scheduled as workflow jobs that run on a regular basis. The "Shadower" (SR) selects tables in the probation period and performs shadowing from the new Iceberg table to the original Hive table with the _hive suffix. The "Reverter" (RT) can be used to revert the Iceberg table to the original Hive table and pause the migration during the probation period.
51 |
52 | ### Metadata Table / Job Queue
53 |
54 | Migration tooling uses `jobs` table as the queue. The table is created using the following SQL command:
55 |
56 | ```sql
57 | CREATE TABLE `jobs` (
58 | `task_id` int NOT NULL AUTO_INCREMENT, -- Unique identifier for each task
59 | `catalog_name` varchar(30) NOT NULL, -- Name of the catalog
60 | `db_name` varchar(255) NOT NULL, -- Name of the database
61 | `tbl_name` varchar(255) NOT NULL, -- Name of the table
62 | `stg_format` varchar(30) DEFAULT NULL, -- Storage format of the table
63 | `data_category` varchar(255) NOT NULL DEFAULT 'UNKNOWN', -- Category of the data
64 | `tbl_owners` json DEFAULT NULL, -- JSON array of table owners
65 | `downstream_users` json DEFAULT NULL, -- JSON array of downstream users
66 | `to_be_processed` tinyint(1) NOT NULL DEFAULT '0', -- Flag indicating if the job is ready to be processed
67 | `in_process` tinyint(1) NOT NULL DEFAULT '0', -- Flag indicating if the job is currently being processed
68 | `state` varchar(30) NOT NULL DEFAULT 'Undefined', -- Current state of the job
69 | `desired_state` varchar(30) NOT NULL DEFAULT 'Undefined', -- Desired state of the job
70 | `initial_gap_days` int NOT NULL DEFAULT '14', -- Initial gap days before processing the job
71 | `probation_gap_days` int NOT NULL DEFAULT '0', -- Probation gap days before processing the job
72 | `comm_level1_date` timestamp NULL DEFAULT NULL, -- Timestamp of level 1 communication
73 | `comm_level2_date` timestamp NULL DEFAULT NULL, -- Timestamp of level 2 communication
74 | `comm_level3_date` timestamp NULL DEFAULT NULL, -- Timestamp of level 3 communication
75 | `shadow_watermark` mediumtext, -- Watermark for shadowing process
76 | `migration_paused` tinyint(1) NOT NULL DEFAULT '0', -- Flag indicating if the migration is paused
77 | `pause_reason` varchar(512) NOT NULL DEFAULT 'None', -- Reason for pausing the migration
78 | `shadow_status` varchar(30) DEFAULT NULL, -- Status of the shadowing process
79 | `created_at` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP, -- Timestamp when the job was created
80 | `last_updated_time` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP, -- Timestamp when the job was last updated
81 | PRIMARY KEY (`task_id`), -- Primary key of the table
82 | UNIQUE KEY `uniq_name` (`catalog_name`,`db_name`,`tbl_name`) -- Unique key constraint on catalog_name, db_name, and tbl_name
83 | ) ENGINE=InnoDB AUTO_INCREMENT=452391 DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_0900_ai_ci; -- Table engine and character set details
84 | ```
85 |
86 | ### Preprocessor
87 |
88 | Preprocessor is a process that selects a job with to_be_processed set to 0 and in_process set to 0, extracts the table owner and downstream users, and assigns them to the job. Also, sets state to Ready and desired_state to WritesBlocked, and sets the to_be_processed flag to 1.
89 |
90 | ### Migrator
91 |
92 | If it is determined that two weeks have passed since level 1 communication was sent for a specific job/table, Migrator will mark the in_process flag as 1 and reset the to_be_processed flag to 0. The migration of that table then begins until the desired_state and state are both set to WritesUnblocked, which starts the probation period marked by Communicator sending level 2 communication. After two weeks have passed since level 2 communication was sent, Migrator completes the migration by dropping the original Hive table with the _hive suffix. At this point, both the state and desired_state become equal to the HiveDropped state and reset the in_process flag to 0.
93 |
94 | ### Communicator
95 |
96 | If the to_be_processed flag is set to 1, the Communicator workflow will send Level 1 communication to notify users that the table will be migrated in 1-2 weeks. This sets the date for comm_level1_date. After 1-2 weeks, the migration tool will select the table for migration and set the in_process flag to 1. The migration of the table will then begin. Once the table is migrated, if the desired_state and state are both set to WritesUnblocked, the Communicator will send Level 2 communication to notify users that the probation period has begun. This sets the date for comm_level2_date. After two weeks, the migration tool will delete the original Hive table. If the state and desired_state are both set to HiveDropped at this point, the Communicator will send Level 3 communication to notify users that the migration is complete and set the comm_level3_date.
97 |
98 | ### Shadower
99 |
100 | If the Hive table watermark does not match the current snapshot_id of the Iceberg table, and if the desired_state and state are both set to WritesUnblocked and shadow_status is set to NULL, then the Shadower will set shadow_status to NOT NULL (some meaningful value) and invoke the Shadow Tool (ST) to incrementally copy the data from the new Iceberg table to the Hive table with the _hive suffix. Once the current ST incremental copy is successful, the most recent snapshot_id that the Hive and Iceberg tables are in sync with is set as the watermark in the Hive TBLPROPERTIES, and the ST sets shadow_status to NULL in a single transaction.
101 |
102 | ### Reverter
103 |
104 | Reverter is a feature that allows users or migration administrators to revert the primary table to Hive if they encounter issues with the newly created Iceberg table during the probation period. Reverter will not allow user requests if the table migration is in-process (where in_process is set to 1) and it is not in the probation period. When Reverter is activated during the probation period, writes to the Iceberg table are first blocked. Then, the shadow tool is used to update the Hive table (with the _hive suffix) with the newly written data in the Iceberg table. The Iceberg table name is then appended with the _iceberg suffix, while the _hive suffix is removed from the Hive table, making it the primary table. Finally, writes to the primary table are unblocked, while the Iceberg table with the _iceberg suffix is deleted and the migration_paused field is set to true.
105 |
--------------------------------------------------------------------------------
/LICENSE:
--------------------------------------------------------------------------------
1 |
2 | Apache License
3 | Version 2.0, January 2004
4 | http://www.apache.org/licenses/
5 |
6 | TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
7 |
8 | 1. Definitions.
9 |
10 | "License" shall mean the terms and conditions for use, reproduction,
11 | and distribution as defined by Sections 1 through 9 of this document.
12 |
13 | "Licensor" shall mean the copyright owner or entity authorized by
14 | the copyright owner that is granting the License.
15 |
16 | "Legal Entity" shall mean the union of the acting entity and all
17 | other entities that control, are controlled by, or are under common
18 | control with that entity. For the purposes of this definition,
19 | "control" means (i) the power, direct or indirect, to cause the
20 | direction or management of such entity, whether by contract or
21 | otherwise, or (ii) ownership of fifty percent (50%) or more of the
22 | outstanding shares, or (iii) beneficial ownership of such entity.
23 |
24 | "You" (or "Your") shall mean an individual or Legal Entity
25 | exercising permissions granted by this License.
26 |
27 | "Source" form shall mean the preferred form for making modifications,
28 | including but not limited to software source code, documentation
29 | source, and configuration files.
30 |
31 | "Object" form shall mean any form resulting from mechanical
32 | transformation or translation of a Source form, including but
33 | not limited to compiled object code, generated documentation,
34 | and conversions to other media types.
35 |
36 | "Work" shall mean the work of authorship, whether in Source or
37 | Object form, made available under the License, as indicated by a
38 | copyright notice that is included in or attached to the work
39 | (an example is provided in the Appendix below).
40 |
41 | "Derivative Works" shall mean any work, whether in Source or Object
42 | form, that is based on (or derived from) the Work and for which the
43 | editorial revisions, annotations, elaborations, or other modifications
44 | represent, as a whole, an original work of authorship. For the purposes
45 | of this License, Derivative Works shall not include works that remain
46 | separable from, or merely link (or bind by name) to the interfaces of,
47 | the Work and Derivative Works thereof.
48 |
49 | "Contribution" shall mean any work of authorship, including
50 | the original version of the Work and any modifications or additions
51 | to that Work or Derivative Works thereof, that is intentionally
52 | submitted to Licensor for inclusion in the Work by the copyright owner
53 | or by an individual or Legal Entity authorized to submit on behalf of
54 | the copyright owner. For the purposes of this definition, "submitted"
55 | means any form of electronic, verbal, or written communication sent
56 | to the Licensor or its representatives, including but not limited to
57 | communication on electronic mailing lists, source code control systems,
58 | and issue tracking systems that are managed by, or on behalf of, the
59 | Licensor for the purpose of discussing and improving the Work, but
60 | excluding communication that is conspicuously marked or otherwise
61 | designated in writing by the copyright owner as "Not a Contribution."
62 |
63 | "Contributor" shall mean Licensor and any individual or Legal Entity
64 | on behalf of whom a Contribution has been received by Licensor and
65 | subsequently incorporated within the Work.
66 |
67 | 2. Grant of Copyright License. Subject to the terms and conditions of
68 | this License, each Contributor hereby grants to You a perpetual,
69 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable
70 | copyright license to reproduce, prepare Derivative Works of,
71 | publicly display, publicly perform, sublicense, and distribute the
72 | Work and such Derivative Works in Source or Object form.
73 |
74 | 3. Grant of Patent License. Subject to the terms and conditions of
75 | this License, each Contributor hereby grants to You a perpetual,
76 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable
77 | (except as stated in this section) patent license to make, have made,
78 | use, offer to sell, sell, import, and otherwise transfer the Work,
79 | where such license applies only to those patent claims licensable
80 | by such Contributor that are necessarily infringed by their
81 | Contribution(s) alone or by combination of their Contribution(s)
82 | with the Work to which such Contribution(s) was submitted. If You
83 | institute patent litigation against any entity (including a
84 | cross-claim or counterclaim in a lawsuit) alleging that the Work
85 | or a Contribution incorporated within the Work constitutes direct
86 | or contributory patent infringement, then any patent licenses
87 | granted to You under this License for that Work shall terminate
88 | as of the date such litigation is filed.
89 |
90 | 4. Redistribution. You may reproduce and distribute copies of the
91 | Work or Derivative Works thereof in any medium, with or without
92 | modifications, and in Source or Object form, provided that You
93 | meet the following conditions:
94 |
95 | (a) You must give any other recipients of the Work or
96 | Derivative Works a copy of this License; and
97 |
98 | (b) You must cause any modified files to carry prominent notices
99 | stating that You changed the files; and
100 |
101 | (c) You must retain, in the Source form of any Derivative Works
102 | that You distribute, all copyright, patent, trademark, and
103 | attribution notices from the Source form of the Work,
104 | excluding those notices that do not pertain to any part of
105 | the Derivative Works; and
106 |
107 | (d) If the Work includes a "NOTICE" text file as part of its
108 | distribution, then any Derivative Works that You distribute must
109 | include a readable copy of the attribution notices contained
110 | within such NOTICE file, excluding those notices that do not
111 | pertain to any part of the Derivative Works, in at least one
112 | of the following places: within a NOTICE text file distributed
113 | as part of the Derivative Works; within the Source form or
114 | documentation, if provided along with the Derivative Works; or,
115 | within a display generated by the Derivative Works, if and
116 | wherever such third-party notices normally appear. The contents
117 | of the NOTICE file are for informational purposes only and
118 | do not modify the License. You may add Your own attribution
119 | notices within Derivative Works that You distribute, alongside
120 | or as an addendum to the NOTICE text from the Work, provided
121 | that such additional attribution notices cannot be construed
122 | as modifying the License.
123 |
124 | You may add Your own copyright statement to Your modifications and
125 | may provide additional or different license terms and conditions
126 | for use, reproduction, or distribution of Your modifications, or
127 | for any such Derivative Works as a whole, provided Your use,
128 | reproduction, and distribution of the Work otherwise complies with
129 | the conditions stated in this License.
130 |
131 | 5. Submission of Contributions. Unless You explicitly state otherwise,
132 | any Contribution intentionally submitted for inclusion in the Work
133 | by You to the Licensor shall be under the terms and conditions of
134 | this License, without any additional terms or conditions.
135 | Notwithstanding the above, nothing herein shall supersede or modify
136 | the terms of any separate license agreement you may have executed
137 | with Licensor regarding such Contributions.
138 |
139 | 6. Trademarks. This License does not grant permission to use the trade
140 | names, trademarks, service marks, or product names of the Licensor,
141 | except as required for reasonable and customary use in describing the
142 | origin of the Work and reproducing the content of the NOTICE file.
143 |
144 | 7. Disclaimer of Warranty. Unless required by applicable law or
145 | agreed to in writing, Licensor provides the Work (and each
146 | Contributor provides its Contributions) on an "AS IS" BASIS,
147 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
148 | implied, including, without limitation, any warranties or conditions
149 | of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
150 | PARTICULAR PURPOSE. You are solely responsible for determining the
151 | appropriateness of using or redistributing the Work and assume any
152 | risks associated with Your exercise of permissions under this License.
153 |
154 | 8. Limitation of Liability. In no event and under no legal theory,
155 | whether in tort (including negligence), contract, or otherwise,
156 | unless required by applicable law (such as deliberate and grossly
157 | negligent acts) or agreed to in writing, shall any Contributor be
158 | liable to You for damages, including any direct, indirect, special,
159 | incidental, or consequential damages of any character arising as a
160 | result of this License or out of the use or inability to use the
161 | Work (including but not limited to damages for loss of goodwill,
162 | work stoppage, computer failure or malfunction, or any and all
163 | other commercial damages or losses), even if such Contributor
164 | has been advised of the possibility of such damages.
165 |
166 | 9. Accepting Warranty or Additional Liability. While redistributing
167 | the Work or Derivative Works thereof, You may choose to offer,
168 | and charge a fee for, acceptance of support, warranty, indemnity,
169 | or other liability obligations and/or rights consistent with this
170 | License. However, in accepting such obligations, You may act only
171 | on Your own behalf and on Your sole responsibility, not on behalf
172 | of any other Contributor, and only if You agree to indemnify,
173 | defend, and hold each Contributor harmless for any liability
174 | incurred by, or claims asserted against, such Contributor by reason
175 | of your accepting any such warranty or additional liability.
176 |
177 | END OF TERMS AND CONDITIONS
178 |
179 | APPENDIX: How to apply the Apache License to your work.
180 |
181 | To apply the Apache License to your work, attach the following
182 | boilerplate notice, with the fields enclosed by brackets "[]"
183 | replaced with your own identifying information. (Don't include
184 | the brackets!) The text should be enclosed in the appropriate
185 | comment syntax for the file format. We also recommend that a
186 | file or class name and description of purpose be included on the
187 | same "printed page" as the copyright notice for easier
188 | identification within third-party archives.
189 |
190 | Copyright 2023 Netflix, Inc.
191 |
192 | Licensed under the Apache License, Version 2.0 (the "License");
193 | you may not use this file except in compliance with the License.
194 | You may obtain a copy of the License at
195 |
196 | http://www.apache.org/licenses/LICENSE-2.0
197 |
198 | Unless required by applicable law or agreed to in writing, software
199 | distributed under the License is distributed on an "AS IS" BASIS,
200 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
201 | See the License for the specific language governing permissions and
202 | limitations under the License.
203 |
--------------------------------------------------------------------------------
/src/main/scala/com/netflix/migration/services/StorageServiceImpl.scala:
--------------------------------------------------------------------------------
1 | package com.netflix.migration.services
2 |
3 | import com.netflix.migration.data.{Job, JobState, MigrationConfig}
4 | import com.netflix.migration.providers.MigrationConfigProvider
5 | import com.netflix.migration.services.StorageServiceImpl.{makeDbConnection, rsToJobs}
6 | import com.netflix.migration.utils.MigrationConsts
7 | import com.typesafe.scalalogging.StrictLogging
8 | import play.api.libs.json.Json
9 |
10 | import java.sql.{Connection, DriverManager, JDBCType, ResultSet}
11 | import java.time.Instant
12 |
13 | case class StorageServiceImpl(conf: MigrationConfig) extends StorageService with StrictLogging {
14 |
15 | /**
16 | * Get the job for the given table
17 | *
18 | * @param catalogName
19 | * Name of the catalog
20 | * @param dbName
21 | * Name of the database
22 | * @param tableName
23 | * Name of the table
24 | * @return
25 | * Job corresponding to table
26 | */
27 | override def getJob(catalogName: String, dbName: String, tableName: String): Job = {
28 | val conf: MigrationConfig = MigrationConfigProvider.getMigrationConfig
29 | val jobsQuery =
30 | s"""
31 | |SELECT *
32 | |FROM ${conf.queueName}
33 | |WHERE catalog_name = '$catalogName'
34 | |AND db_name = '$dbName'
35 | |AND tbl_name = '$tableName'
36 | |""".stripMargin
37 | getJobs(jobsQuery).head
38 | }
39 |
40 | /**
41 | * Get the list of jobs for the given query
42 | *
43 | * @param jobsQuery
44 | * Jobs query
45 | * @return
46 | * list of jobs
47 | */
48 | override def getJobs(jobsQuery: String): List[Job] = {
49 | var connection: Connection = null
50 | try {
51 | connection = makeDbConnection(conf)
52 | val statement = connection.createStatement
53 | val rs = statement.executeQuery(jobsQuery)
54 | rsToJobs(rs)
55 | } catch {
56 | case e: Exception =>
57 | logger.error(
58 | s"Failed to get jobs using query: $jobsQuery. Exception: ${e.printStackTrace()}")
59 | throw e
60 | } finally {
61 | connection.close()
62 | }
63 | }
64 |
65 | /**
66 | * Insert new job into the jobs queue
67 | *
68 | * @param job
69 | * The job.
70 | */
71 | def enqueueJob(job: Job): Unit = {
72 | var connection: Connection = null
73 | try {
74 | connection = makeDbConnection(conf)
75 | val insertQuery =
76 | s"""
77 | |INSERT INTO ${conf.queueName} (catalog_name, db_name, tbl_name)
78 | |VALUES (?, ?, ?)
79 | |""".stripMargin
80 | val statement = connection.prepareStatement(insertQuery)
81 |
82 | val catalogNameStr =
83 | Option(job.catalogName).getOrElse(throw new RuntimeException("catalog is empty"))
84 | statement.setObject(1, catalogNameStr, JDBCType.VARCHAR)
85 |
86 | val databaseNameStr = if (Option(job.dbName).isEmpty) "" else job.dbName
87 | statement.setObject(2, databaseNameStr, JDBCType.VARCHAR)
88 |
89 | val tableNameStr = if (Option(job.tblName).isEmpty) "" else job.tblName
90 | statement.setObject(3, tableNameStr, JDBCType.VARCHAR)
91 |
92 | statement.executeUpdate
93 | } catch {
94 | case e: Exception =>
95 | logger.error(
96 | s"Encountered exception enqueuing Job: $job. Exception: ${e.printStackTrace()}")
97 | throw e
98 | } finally {
99 | connection.close()
100 | }
101 | }
102 |
103 | /**
104 | * Remove job from the jobs queue
105 | *
106 | * @param job
107 | * The job.
108 | */
109 | def removeJob(job: Job): Unit = {
110 | var connection: Connection = null
111 | try {
112 | connection = makeDbConnection(conf)
113 | val deleteQuery =
114 | s"""
115 | |DELETE FROM ${conf.queueName}
116 | |WHERE catalog_name = ?
117 | |AND db_name = ?
118 | |AND tbl_name = ?
119 | |""".stripMargin
120 | val statement = connection.prepareStatement(deleteQuery)
121 |
122 | val catalogNameStr =
123 | Option(job.catalogName).getOrElse(throw new RuntimeException("catalog is empty"))
124 | statement.setObject(1, catalogNameStr, JDBCType.VARCHAR)
125 |
126 | val databaseNameStr = if (Option(job.dbName).isEmpty) "" else job.dbName
127 | statement.setObject(2, databaseNameStr, JDBCType.VARCHAR)
128 |
129 | val tableNameStr = if (Option(job.tblName).isEmpty) "" else job.tblName
130 | statement.setObject(3, tableNameStr, JDBCType.VARCHAR)
131 |
132 | statement.executeUpdate
133 | } catch {
134 | case e: Exception =>
135 | logger.error(
136 | s"Encountered exception removing Job: $job. Exception: ${e.printStackTrace()}")
137 | throw e
138 | } finally {
139 | connection.close()
140 | }
141 | }
142 |
143 | /**
144 | * Update the given job's column value.
145 | *
146 | * @param job
147 | * The job.
148 | */
149 | override def updateJob(job: Job): Unit = {
150 | var connection: Connection = null
151 | try {
152 | connection = makeDbConnection(conf)
153 | val updateQuery =
154 | s"""
155 | |UPDATE ${conf.queueName}
156 | |SET stg_format = ?, data_category = ?, tbl_owners = ?, downstream_users = ?,
157 | |to_be_processed = ?, in_process = ?, state = ?, desired_state = ?,
158 | |initial_gap_days = ?, probation_gap_days = ?, comm_level1_date = ?,
159 | |comm_level2_date = ?, comm_level3_date = ?, shadow_watermark = ?,
160 | |migration_paused = ?, pause_reason = ?, run_id = ?, shadow_status = ?
161 | |WHERE task_id = ?
162 | |AND catalog_name = ?
163 | |AND db_name = ?
164 | |AND tbl_name = ?
165 | |""".stripMargin
166 | val statement = connection.prepareStatement(updateQuery)
167 |
168 | val stgFormatStr = if (Option(job.stgFormat).isEmpty) null else job.stgFormat
169 | statement.setObject(1, stgFormatStr, JDBCType.VARCHAR)
170 |
171 | val dataCategoryStr = if (Option(job.dataCategory).isEmpty) null else job.dataCategory
172 | statement.setObject(2, dataCategoryStr, JDBCType.VARCHAR)
173 |
174 | val tblOwnersJson = if (job.tblOwners.isEmpty) null else Json.toJson(job.tblOwners).toString
175 | statement.setObject(3, tblOwnersJson, JDBCType.VARCHAR)
176 |
177 | val tblDownstreamUsersJson =
178 | if (job.downstreamUsers.isEmpty) null else Json.toJson(job.downstreamUsers).toString
179 | statement.setObject(4, tblDownstreamUsersJson, JDBCType.VARCHAR)
180 |
181 | statement.setObject(5, job.toBeProcessed, JDBCType.BOOLEAN)
182 | statement.setObject(6, job.inProcess, JDBCType.BOOLEAN)
183 |
184 | val jobState =
185 | if (Option(job.state.toString).isEmpty || job.state == JobState.Undefined)
186 | JobState.Undefined
187 | else job.state.toString
188 | statement.setObject(7, jobState.toString, JDBCType.VARCHAR)
189 |
190 | val jobDesiredState =
191 | if (Option(job.desiredState.toString).isEmpty || job.desiredState == JobState.Undefined)
192 | JobState.Undefined
193 | else job.desiredState
194 | statement.setObject(8, jobDesiredState.toString, JDBCType.VARCHAR)
195 |
196 | statement.setObject(9, job.initialGapInDays, JDBCType.INTEGER)
197 | statement.setObject(10, job.probationGapInDays, JDBCType.INTEGER)
198 |
199 | val jobCommLevel1Date =
200 | if (job.commLevel1Date != -1)
201 | java.sql.Timestamp.from(Instant.ofEpochMilli(job.commLevel1Date))
202 | else null
203 | statement.setTimestamp(11, jobCommLevel1Date)
204 |
205 | val jobCommLevel2Date =
206 | if (job.commLevel2Date != -1)
207 | java.sql.Timestamp.from(Instant.ofEpochMilli(job.commLevel2Date))
208 | else null
209 | statement.setTimestamp(12, jobCommLevel2Date)
210 |
211 | val jobCommLevel3Date =
212 | if (job.commLevel3Date != -1)
213 | java.sql.Timestamp.from(Instant.ofEpochMilli(job.commLevel3Date))
214 | else null
215 | statement.setTimestamp(13, jobCommLevel3Date)
216 |
217 | statement.setObject(14, job.shadowWatermark, JDBCType.BIGINT)
218 | statement.setObject(15, job.migrationPaused, JDBCType.BOOLEAN)
219 | statement.setObject(16, job.pauseReason, JDBCType.VARCHAR)
220 | statement.setObject(17, job.runId, JDBCType.VARCHAR)
221 |
222 | val jobShadowStatusStr = Option(job.shadowStatus).orNull
223 | statement.setObject(18, jobShadowStatusStr, JDBCType.VARCHAR)
224 |
225 | statement.setObject(19, job.id, JDBCType.INTEGER)
226 |
227 | val catalogNameStr = if (Option(job.catalogName).isEmpty) "" else job.catalogName
228 | statement.setObject(20, catalogNameStr, JDBCType.VARCHAR)
229 |
230 | val databaseNameStr = if (Option(job.dbName).isEmpty) "" else job.dbName
231 | statement.setObject(21, databaseNameStr, JDBCType.VARCHAR)
232 |
233 | val tableNameStr = if (Option(job.tblName).isEmpty) "" else job.tblName
234 | statement.setObject(22, tableNameStr, JDBCType.VARCHAR)
235 |
236 | statement.executeUpdate
237 | } catch {
238 | case e: Exception =>
239 | logger.error(
240 | s"Encountered exception updating Job: $job. Exception: ${e.printStackTrace()}")
241 | throw e
242 | } finally {
243 | connection.close()
244 | }
245 | }
246 | }
247 |
248 | object StorageServiceImpl {
249 |
250 | def makeDbConnection(conf: MigrationConfig): Connection = {
251 | val dbServer =
252 | if (conf.dbEnv.equals(MigrationConsts.PROD_ENV))
253 | MigrationConsts.JOBS_DB_PROD_URL
254 | else MigrationConsts.JOBS_DB_TEST_URL
255 | val url =
256 | s"jdbc:mysql://$dbServer/${conf.dbName}?useUnicode=true&connectTimeout=30000&characterEncoding=latin1&autoReconnect=true&sessionVariables=@@innodb_lock_wait_timeout=300&enabledTLSProtocols=TLSv1.2"
257 | val driver = "com.mysql.cj.jdbc.Driver"
258 | val username = MigrationConsts.MYSQL_USER
259 | val password = MigrationConsts.MYSQL_PASS
260 | Class.forName(driver)
261 | DriverManager.getConnection(url, username, password)
262 | }
263 |
264 | def rsToJobs(rs: ResultSet): List[Job] = {
265 | var res: List[Job] = List()
266 | while (rs.next) {
267 | val id = Option(rs.getInt("task_id")).getOrElse(-1)
268 | val catalog = Option(rs.getString("catalog_name")).getOrElse("")
269 | val db = Option(rs.getString("db_name")).getOrElse("")
270 | val tbl = Option(rs.getString("tbl_name")).getOrElse("")
271 | val storageFormat = Option(rs.getString("stg_format")).getOrElse("")
272 | val dataCategory = Option(rs.getString("data_category")).getOrElse("")
273 | val tableOwners =
274 | Option(rs.getString("tbl_owners")).map(Json.parse(_).as[Set[String]]).getOrElse(Set())
275 | val downstreamUsers = Option(rs.getString("downstream_users"))
276 | .map(Json.parse(_).as[Set[String]])
277 | .getOrElse(Set())
278 | val toBeProcessed = Option(rs.getInt("to_be_processed")).getOrElse(0)
279 | val inProcess = Option(rs.getInt("in_process")).getOrElse(0)
280 | val state =
281 | Option(rs.getString("state")).map(JobState.withName).getOrElse(JobState.Undefined)
282 | val desired =
283 | Option(rs.getString("desired_state")).map(JobState.withName).getOrElse(JobState.Undefined)
284 | val initialGapInDays =
285 | Option(rs.getInt("initial_gap_days")).getOrElse(0)
286 | val probationGapInDays =
287 | Option(rs.getInt("probation_gap_days")).getOrElse(0)
288 | val commLevel1Date =
289 | Option(rs.getTimestamp("comm_level1_date")).fold(-1L)(_.toInstant.toEpochMilli)
290 | val commLevel2Date =
291 | Option(rs.getTimestamp("comm_level2_date")).fold(-1L)(_.toInstant.toEpochMilli)
292 | val commLevel3Date =
293 | Option(rs.getTimestamp("comm_level3_date")).fold(-1L)(_.toInstant.toEpochMilli)
294 | val shadowWatermark = Option(rs.getLong("shadow_watermark")).getOrElse(-1L)
295 | val migrationPaused = Option(rs.getInt("migration_paused")).getOrElse(0)
296 | val pauseReason = Option(rs.getString("pause_reason")).getOrElse("None")
297 | val runId = Option(rs.getString("run_id")).orNull
298 | val shadowStatus = Option(rs.getString("shadow_status")).orNull
299 | val createdAt = Option(rs.getTimestamp("created_at")).fold(-1L)(_.toInstant.toEpochMilli)
300 | val lastUpdatedTime =
301 | Option(rs.getTimestamp("last_updated_time")).fold(-1L)(_.toInstant.toEpochMilli)
302 | res = new Job(
303 | id,
304 | catalog,
305 | db,
306 | tbl,
307 | storageFormat,
308 | dataCategory,
309 | tableOwners,
310 | downstreamUsers,
311 | toBeProcessed,
312 | inProcess,
313 | state,
314 | desired,
315 | initialGapInDays,
316 | probationGapInDays,
317 | commLevel1Date,
318 | commLevel2Date,
319 | commLevel3Date,
320 | shadowWatermark,
321 | migrationPaused,
322 | pauseReason,
323 | runId,
324 | shadowStatus,
325 | createdAt,
326 | lastUpdatedTime) :: res
327 | }
328 | res
329 | }
330 | }
331 |
--------------------------------------------------------------------------------
/src/main/scala/com/netflix/migration/services/MetacatServiceImpl.scala:
--------------------------------------------------------------------------------
1 | package com.netflix.migration.services
2 |
3 | import com.netflix.metacat.client.Client
4 | import com.netflix.metacat.common.dto.TableDto
5 | import com.netflix.metacat.shaded.com.fasterxml.jackson.databind.node.ObjectNode
6 | import com.netflix.migration.data.MigrationConfig
7 | import com.netflix.migration.utils.MigrationConsts.{DEFAULT_TABLE_OWNER, DO_NOT_DROP_TAG, DO_NOT_RENAME_TAG, HIVE_CSV_TEXT, HIVE_PARQUET, ICEBERG_MIGRATION_DO_NOT_MODIFY_TAG}
8 | import com.netflix.migration.utils.MigrationRunnerUtils.retry
9 | import com.netflix.migration.utils.Utils.metacatService
10 | import com.netflix.migration.utils.{MigrationConsts, StringUtils}
11 | import com.typesafe.scalalogging.StrictLogging
12 |
13 | import scala.collection.JavaConverters._
14 | import scala.collection.mutable
15 |
16 | case class MetacatServiceImpl(conf: MigrationConfig) extends MetacatService with StrictLogging {
17 | @transient private[services] lazy val metacatClient = Client
18 | .builder()
19 | .withHost(MigrationConsts.METACAT_HOST_URL_PROPERTY)
20 | .withDataTypeContext("hive")
21 | .withUserName(MigrationConsts.METACAT_USERNAME_PROPERTY)
22 | .withClientAppName(MigrationConsts.METACAT_CLIENT_APP_NAME_PROPERTY)
23 | .build()
24 |
25 | override def renameTable(
26 | catalogName: String,
27 | dbName: String,
28 | tableName: String,
29 | newTableName: String): Unit = {
30 | val currentTableQName = StringUtils.getTableQualifiedName(catalogName, dbName, tableName)
31 | val newTableQName = StringUtils.getTableQualifiedName(catalogName, dbName, newTableName)
32 |
33 | try {
34 | val preExistingTags = getTableTags(catalogName, dbName, tableName)
35 | removeTableTagsIfExists(
36 | catalogName,
37 | dbName,
38 | tableName,
39 | preExistingTags,
40 | Set(MigrationConsts.DO_NOT_RENAME_TAG, MigrationConsts.DO_NOT_DROP_TAG))
41 |
42 | metacatClient.getApi.renameTable(catalogName, dbName, tableName, newTableName)
43 |
44 | setTableTagsIfExists(
45 | catalogName,
46 | dbName,
47 | newTableName,
48 | preExistingTags,
49 | Set(MigrationConsts.DO_NOT_RENAME_TAG, MigrationConsts.DO_NOT_DROP_TAG))
50 |
51 | logger.info(s"Successfully renamed table $currentTableQName to $newTableQName")
52 | } catch {
53 | case e: Exception =>
54 | println(s"Failed to rename table $currentTableQName to $newTableQName. Exception: $e")
55 | throw e
56 | }
57 | }
58 |
59 | /**
60 | * Gets table dto object
61 | *
62 | * @param catalogName
63 | * @param dbName
64 | * @param tableName
65 | */
66 | def getTableDto(catalogName: String, dbName: String, tableName: String): TableDto = {
67 | val fullTableName = s"$catalogName/$dbName/$tableName"
68 | val optionalTable = getTable(catalogName, dbName, tableName, includeDataMetadata = false)
69 | if (optionalTable.isEmpty) {
70 | println(s"Table: $fullTableName not found in metacat")
71 | throw new RuntimeException(s"Table $fullTableName not found in metacat!")
72 | }
73 | optionalTable.get
74 | }
75 |
76 | /**
77 | * Gets tags on the table
78 | *
79 | * @param catalogName
80 | * @param dbName
81 | * @param tableName
82 | */
83 | def getTableTags(
84 | catalogName: String,
85 | dbName: String,
86 | tableName: String): mutable.Set[String] = {
87 | val tableDto = getTableDto(catalogName, dbName, tableName)
88 |
89 | val tagsSet: mutable.HashSet[String] = mutable.HashSet()
90 | if (tableDto.getDefinitionMetadata != null) {
91 | val tagsNode = tableDto.getDefinitionMetadata.path("tags")
92 | if (tagsNode != null &&
93 | tagsNode.isArray &&
94 | tagsNode.size() > 0) {
95 | // Get the existing set of tags and append to them since
96 | // the tags API is a replace all tags API.
97 | for (tag <- tagsNode.asScala) {
98 | tagsSet.add(tag.asText().trim)
99 | }
100 | }
101 | } else {
102 | return null
103 | }
104 | tagsSet
105 | }
106 |
107 | /**
108 | * Sets tags on the table
109 | *
110 | * @param catalogName
111 | * @param dbName
112 | * @param tableName
113 | * @param tag
114 | */
115 | override def setTableTag(
116 | catalogName: String,
117 | dbName: String,
118 | tableName: String,
119 | tag: String): Unit = {
120 | val tableQName = StringUtils.getTableQualifiedName(catalogName, dbName, tableName)
121 | if (!tableExists(catalogName, dbName, tableName)) {
122 | logger.error(s"Table: $tableQName does not exist. Skipping tag update.")
123 | return
124 | }
125 | var tagsSet: mutable.Set[String] = getTableTags(catalogName, dbName, tableName)
126 | if (tagsSet != null) {
127 | if (!tagsSet.contains(tag)) {
128 | tagsSet += tag
129 | }
130 | setTableTags(catalogName, dbName, tableName, tagsSet.toSet)
131 | logger.info(s"Successfully setting tag $tag from table $tableQName")
132 | }
133 | }
134 |
135 | /**
136 | * Append the given tags to the set of existing tags on the table.
137 | *
138 | * @param catalogName
139 | * The catalog name.
140 | * @param dbName
141 | * The db name.
142 | * @param tableName
143 | * The table name.
144 | * @param tag
145 | * The set of tags.
146 | */
147 | def appendTableTags(
148 | catalogName: String,
149 | dbName: String,
150 | tableName: String,
151 | tags: Set[String]): Unit = {
152 | if (!tableExists(catalogName, dbName, tableName)) {
153 | logger.error(s"Table: $tableName does not exist. Skipping tag append operation.")
154 | return
155 | }
156 | if (tags.isEmpty) {
157 | logger.error(s"Tags set to append is empty. Skipping tag append operation.")
158 | return
159 | }
160 |
161 | // Get the existing tags and append, deduplication will be handled by ++= operator
162 | val existingTags = getTableTags(catalogName, dbName, tableName)
163 | existingTags ++= tags
164 | setTableTags(catalogName, dbName, tableName, existingTags.toSet)
165 | }
166 |
167 | /**
168 | * @param catalogName
169 | * @param dbName
170 | * @param tableName
171 | * @param tag
172 | */
173 | def setTableTags(
174 | catalogName: String,
175 | dbName: String,
176 | tableName: String,
177 | tagsSet: Set[String]): Unit = {
178 | if (!tableExists(catalogName, dbName, tableName)) {
179 | logger.error(s"Table: $tableName does not exist. Skipping tag update.")
180 | return
181 | }
182 | retry(MigrationConsts.NUM_RETRIES)(
183 | metacatClient.getTagApi.setTableTags(catalogName, dbName, tableName, tagsSet.asJava))
184 | }
185 |
186 | /**
187 | * Sets and replaces the tags on the table if they existed before
188 | *
189 | * @param catalogName
190 | * @param dbName
191 | * @param tableName
192 | * @param tagsSet
193 | */
194 | def setTableTagsIfExists(
195 | catalogName: String,
196 | dbName: String,
197 | tableName: String,
198 | preExistingTags: mutable.Set[String],
199 | tags: Set[String]): Unit = {
200 | for (tag <- tags) {
201 | if (preExistingTags != null && preExistingTags.contains(tag)) {
202 | retry(MigrationConsts.NUM_RETRIES)(setTableTag(catalogName, dbName, tableName, tag))
203 | }
204 | }
205 | }
206 |
207 | /**
208 | * Remove tags from the table
209 | *
210 | * @param catalogName
211 | * @param dbName
212 | * @param tableName
213 | * @param tag
214 | */
215 | def removeTableTag(
216 | catalogName: String,
217 | dbName: String,
218 | tableName: String,
219 | tag: String): Unit = {
220 | val tableQName = StringUtils.getTableQualifiedName(catalogName, dbName, tableName)
221 | metacatClient.getTagApi.removeTableTags(
222 | catalogName,
223 | dbName,
224 | tableName,
225 | false,
226 | Set(tag).asJava)
227 | logger.info(s"Successfully removed tag $tag from table $tableQName")
228 | }
229 |
230 | /**
231 | * Remove tags from the table
232 | *
233 | * @param catalogName
234 | * @param dbName
235 | * @param tableName
236 | * @param tagsSet
237 | */
238 | def removeTableTags(
239 | catalogName: String,
240 | dbName: String,
241 | tableName: String,
242 | tags: Set[String]): Unit = {
243 | val preExistingTags = getTableTags(catalogName, dbName, tableName)
244 | removeTableTagsIfExists(catalogName, dbName, tableName, preExistingTags, tags)
245 | }
246 |
247 | /**
248 | * Remove tags from the table if they existed before
249 | *
250 | * @param catalogName
251 | * @param dbName
252 | * @param tableName
253 | * @param preExistingTags
254 | * @param tagsSet
255 | */
256 | def removeTableTagsIfExists(
257 | catalogName: String,
258 | dbName: String,
259 | tableName: String,
260 | preExistingTags: mutable.Set[String],
261 | tags: Set[String]): Unit = {
262 | for (tag <- tags) {
263 | if (preExistingTags != null && preExistingTags.contains(tag)) {
264 | removeTableTag(catalogName, dbName, tableName, tag)
265 | }
266 | }
267 | }
268 |
269 | /**
270 | * Sets a tag on the table that blocks table updates (renames/deletes/updates).
271 | *
272 | * @param catalogName
273 | * @param dbName
274 | * @param tableName
275 | */
276 | override def blockTableWrites(catalogName: String, dbName: String, tableName: String): Unit = {
277 | setTableTag(
278 | catalogName,
279 | dbName,
280 | tableName,
281 | MigrationConsts.ICEBERG_MIGRATION_DO_NOT_MODIFY_TAG)
282 | }
283 |
284 | /**
285 | * Removes a tag on the table that unblocks table updates (renames/deletes/updates).
286 | *
287 | * @param catalogName
288 | * @param dbName
289 | * @param tableName
290 | */
291 | override def unBlockTableWrites(
292 | catalogName: String,
293 | dbName: String,
294 | tableName: String): Unit = {
295 | val fullTableName = s"$catalogName/$dbName/$tableName"
296 | if (tableExists(catalogName, dbName, tableName)) {
297 | logger.info(s"Unblocking writes for table: $fullTableName")
298 | metacatClient.getTagApi.removeTableTags(
299 | catalogName,
300 | dbName,
301 | tableName,
302 | false,
303 | Set(MigrationConsts.ICEBERG_MIGRATION_DO_NOT_MODIFY_TAG).asJava)
304 | logger.info(s"Successfully unblocked writes to table: $fullTableName")
305 | }
306 | }
307 |
308 | /**
309 | * Get the table from metacat.
310 | *
311 | * @param catalogName
312 | * @param databaseName
313 | * @param tableName
314 | * @param includeInfo
315 | * @param includeDefinitionMetadata
316 | * @param includeDataMetadata
317 | * @return
318 | */
319 | override def getTable(
320 | catalogName: String,
321 | databaseName: String,
322 | tableName: String,
323 | includeInfo: Boolean,
324 | includeDefinitionMetadata: Boolean,
325 | includeDataMetadata: Boolean): Option[TableDto] = {
326 | val fullTableName = s"$catalogName/$databaseName/$tableName"
327 |
328 | try {
329 | val result = metacatClient.getApi
330 | .getTable(
331 | catalogName,
332 | databaseName,
333 | tableName,
334 | includeInfo,
335 | includeDefinitionMetadata,
336 | includeDataMetadata)
337 | if (result == null) {
338 | println(s"Table $fullTableName not found in metacat!")
339 | None
340 | } else {
341 | Some(result)
342 | }
343 | } catch {
344 | case e: Exception =>
345 | println(s"getTable($fullTableName) from metacat failed. Exception: $e")
346 | None
347 | }
348 | }
349 |
350 | /**
351 | * Checks if the given table exists.
352 | *
353 | * @param catalogName
354 | * The catalog name.
355 | * @param dbName
356 | * The database name.
357 | * @param tableName
358 | * The table name.
359 | * @return
360 | */
361 | override def tableExists(catalogName: String, dbName: String, tableName: String): Boolean = {
362 | val tableQName = StringUtils.getTableQualifiedName(catalogName, dbName, tableName)
363 | try {
364 | logger.info(
365 | s"Checking if table ${StringUtils.getTableQualifiedName(catalogName, dbName, tableName)} exists")
366 | metacatClient.getApi.doesTableExist(catalogName, dbName, tableName)
367 | } catch {
368 | case e: Exception =>
369 | logger.error(s"Failed to check if table: $tableQName exists in metacat. Exception: $e")
370 | throw e
371 | }
372 | }
373 |
374 | /**
375 | * Deletes the given table.
376 | *
377 | * @param catalogName
378 | * @param dbName
379 | * @param tableName
380 | */
381 | override def deleteTable(catalogName: String, dbName: String, tableName: String): Unit = {
382 | val tableQName = StringUtils.getTableQualifiedName(catalogName, dbName, tableName)
383 | try {
384 | metacatService.removeTableTags(
385 | catalogName,
386 | dbName,
387 | tableName,
388 | Set(DO_NOT_RENAME_TAG, DO_NOT_DROP_TAG, ICEBERG_MIGRATION_DO_NOT_MODIFY_TAG))
389 | metacatClient.getApi.deleteTable(catalogName, dbName, tableName)
390 | logger.info(s"Successfully deleted table: $tableQName from metacat.")
391 | } catch {
392 | case e: Exception =>
393 | logger.error(s"Failed to delete table: $tableQName from metacat. Exception: $e")
394 | throw e
395 | }
396 | }
397 |
398 | /**
399 | * Force drop given table.
400 | *
401 | * @param catalogName
402 | * @param dbName
403 | * @param tableName
404 | */
405 | def forceDropTable(catalogName: String, dbName: String, tblName: String): Unit = {
406 | try {
407 | metacatService.removeTableTags(
408 | catalogName,
409 | dbName,
410 | tblName,
411 | Set(DO_NOT_RENAME_TAG, DO_NOT_DROP_TAG, ICEBERG_MIGRATION_DO_NOT_MODIFY_TAG))
412 | metacatService.deleteTable(catalogName, dbName, tblName)
413 | } catch {
414 | case e: Throwable =>
415 | println(s"Ignoring exception during deleteTable for $tblName")
416 | }
417 | }
418 |
419 | /**
420 | * Update the table with the given Dto.
421 | *
422 | * @param catalogName
423 | * @param databaseName
424 | * @param tableName
425 | * @return
426 | */
427 | override def updateTable(
428 | catalogName: String,
429 | databaseName: String,
430 | tableName: String,
431 | tableDto: TableDto): Unit = {
432 | val tableQName = StringUtils.getTableQualifiedName(catalogName, databaseName, tableName)
433 | try {
434 | metacatClient.getApi.updateTable(catalogName, databaseName, tableName, tableDto)
435 | logger.info(s"Successfully updated table: $tableQName in metacat.")
436 | } catch {
437 | case e: Exception =>
438 | logger.error(s"Failed to update table: $tableQName in metacat. Exception: $e")
439 | throw e
440 | }
441 | }
442 |
443 | /**
444 | * Get table storage format.
445 | *
446 | * @param catalogName
447 | * @param databaseName
448 | * @param tableName
449 | * @return
450 | */
451 | override def getTableStorageFormat(
452 | catalogName: String,
453 | databaseName: String,
454 | tableName: String): String = {
455 | val tableDto = getTableDto(catalogName, databaseName, tableName)
456 | if (tableDto.getSerde.getOutputFormat.toLowerCase.contains("parquet")) {
457 | HIVE_PARQUET
458 | } else {
459 | HIVE_CSV_TEXT
460 | }
461 | }
462 |
463 | /**
464 | * Get table partition count.
465 | *
466 | * @param catalogName
467 | * @param databaseName
468 | * @param tableName
469 | * @return
470 | */
471 | def getPartitionCount(catalogName: String, databaseName: String, tableName: String): Int = {
472 | metacatClient.getPartitionApi.getPartitionCount(catalogName, databaseName, tableName)
473 | }
474 |
475 | /**
476 | * Gets table data size
477 | *
478 | * @param catalogName
479 | * @param dbName
480 | * @param tableName
481 | */
482 | def getTableDataSize(catalogName: String, dbName: String, tableName: String): Long = {
483 | val tableDto = getTableDto(catalogName, dbName, tableName)
484 |
485 | Option(tableDto.getDefinitionMetadata)
486 | .flatMap { metadata =>
487 | Option(metadata.get("data_size")).map(_.asLong)
488 | }
489 | .getOrElse(-1L)
490 | }
491 |
492 | /**
493 | * Set auth to strict secure
494 | *
495 | * @param catalogName
496 | * @param dbName
497 | * @param tableName
498 | */
499 | def setStrictSecure(catalogName: String, dbName: String, tableName: String): Unit = {
500 | val tableDto = getTableDto(catalogName, dbName, tableName)
501 | tableDto.getDefinitionMetadata.put("secure", true)
502 | metacatClient.getApi.updateTable(catalogName, dbName, tableName, tableDto)
503 | }
504 |
505 | /**
506 | * Set table owner
507 | *
508 | * @param catalogName
509 | * @param dbName
510 | * @param tableName
511 | */
512 | def setOwner(catalogName: String, dbName: String, tableName: String, owner: String): Unit = {
513 | val tableDto = getTableDto(catalogName, dbName, tableName)
514 | val ownerNode = tableDto.getDefinitionMetadata.get("owner").asInstanceOf[ObjectNode]
515 | ownerNode.put("userId", DEFAULT_TABLE_OWNER)
516 | metacatClient.getApi.updateTable(catalogName, dbName, tableName, tableDto)
517 | }
518 | }
519 |
--------------------------------------------------------------------------------
/src/test/scala/com/netflix/migration/MigrationIntegrationTest.scala:
--------------------------------------------------------------------------------
1 | package com.netflix.migration
2 |
3 | import com.netflix.migration.MigrationRunner.createMigrationConf
4 | import com.netflix.migration.data.{Job, JobState}
5 | import com.netflix.migration.modes.{Communicator, Migrator, Preprocessor, Reverter, Shadower}
6 | import com.netflix.migration.providers.{MigrationConfigProvider, SparkSessionProvider}
7 | import com.netflix.migration.services.{StorageService, StorageServiceImpl}
8 | import com.netflix.migration.utils.MigrationConsts.{ARCHIVED_SUFFIX, HIVE_CSV_TEXT, HIVE_SUFFIX, PauseReason, TEST_CATALOG, TEST_DATABASE, TableFormat}
9 | import com.netflix.migration.utils.SparkTablesGenerator
10 | import com.netflix.migration.utils.StorageUtils.getAllJobs
11 | import com.netflix.migration.utils.TestUtils.{getHiveTableRowCount, getIcebergTableRowCount}
12 | import com.netflix.migration.utils.Utils.{getFullyQualifiedTableName, isHiveTable, metacatService}
13 | import org.apache.spark.sql.SparkSession
14 | import org.scalatest.{BeforeAndAfterAll, FunSuite}
15 |
16 | import java.nio.file.Files
17 | import java.time.{Instant, LocalDateTime, ZoneId}
18 | import scala.collection.mutable
19 |
20 | class MigrationIntegrationTest extends FunSuite with BeforeAndAfterAll {
21 |
22 | private var spark: SparkSession = _
23 | private final val numTestTables = 2
24 | private final var numRowsPerTable = 100
25 | private final val shadowIters = 1
26 | private final val shadowWriteNumRows = 100
27 | var storageService: StorageService = _
28 |
29 | override def beforeAll(): Unit = {
30 | val map = Map(
31 | "local" -> "true",
32 | "dbEnv" -> "test",
33 | "queueName" -> "jobs_int_test",
34 | "setIcebergAcls" -> "false",
35 | "archiveHive" -> "false")
36 | createMigrationConf(map)
37 | val conf = MigrationConfigProvider.getMigrationConfig
38 | storageService = StorageServiceImpl(conf)
39 | super.beforeAll()
40 | }
41 |
42 | override def afterAll(): Unit = {
43 | if (spark != null) {
44 | spark.stop()
45 | }
46 | super.afterAll()
47 | }
48 |
49 | def mock_preprocessor(invocation: Int): Unit = {
50 | val jobs = Preprocessor().getRelevantJobs()
51 | for (job <- jobs) {
52 | if (invocation == 1) {
53 | job.tblOwners = Set("akayyoor@netflix.com")
54 | job.toBeProcessed = 1
55 | job.state = JobState.Ready
56 | job.desiredState = JobState.WritesBlocked
57 | job.stgFormat = "parquet"
58 | storageService.updateJob(job)
59 | } else {
60 | job.inProcess = 1
61 | job.state = JobState.WritesUnblocked
62 | job.state = JobState.HiveDropped
63 | storageService.updateJob(job)
64 | }
65 | }
66 | }
67 |
68 | def getTimestampDaysAgo(numDays: Int): Long = {
69 | val zoneId = ZoneId.of("America/Los_Angeles")
70 | val dateTime = LocalDateTime.now().minusDays(numDays)
71 | val zonedDateTime = dateTime.atZone(zoneId)
72 | zonedDateTime.toInstant.toEpochMilli
73 | }
74 |
75 | test("H2I Migration Integration Test") {
76 | var tables = new SparkTablesGenerator(
77 | numTestTables,
78 | numRowsPerTable,
79 | TableFormat.HIVE_PARQUET,
80 | storageService)
81 | tables.createTables()
82 |
83 | var conf_map = Map(
84 | "mode" -> "PREPROCESSOR",
85 | "jobsProcessBatchSize" -> "10",
86 | "dbEnv" -> "test",
87 | "local" -> "false",
88 | "dryrun" -> "false",
89 | "distributed" -> "false",
90 | "queueName" -> "jobs_int_test",
91 | "setIcebergAcls" -> "false",
92 | "archiveHive" -> "false")
93 | createMigrationConf(conf_map)
94 | Console.println("RUNNING PREPROCESSOR-1")
95 | mock_preprocessor(1)
96 |
97 | var jobs = getAllJobs()
98 | assert(jobs.nonEmpty)
99 | for (job <- jobs) {
100 | assert(job.stgFormat == "parquet")
101 | assert(job.toBeProcessed == 1)
102 | assert(job.inProcess == 0)
103 | assert(job.state == JobState.Ready)
104 | assert(job.desiredState == JobState.WritesBlocked)
105 | assert(job.commLevel1Date == -1L)
106 | assert(job.commLevel2Date == -1L)
107 | assert(job.commLevel3Date == -1L)
108 | assert(job.migrationPaused == 0)
109 | assert(job.shadowStatus == null)
110 | }
111 |
112 | conf_map += ("mode" -> "COMMUNICATOR")
113 | createMigrationConf(conf_map)
114 | Console.println("RUNNING COMMUNICATOR-1")
115 | MigrationRunner().process()
116 |
117 | jobs = getAllJobs()
118 | assert(jobs.nonEmpty)
119 | for (job <- jobs) {
120 | assert(job.stgFormat == "parquet")
121 | assert(job.toBeProcessed == 1)
122 | assert(job.inProcess == 0)
123 | assert(job.state == JobState.Ready)
124 | assert(job.desiredState == JobState.WritesBlocked)
125 | assert(job.commLevel1Date > -1L)
126 | assert(job.commLevel2Date == -1L)
127 | assert(job.commLevel3Date == -1L)
128 | assert(job.migrationPaused == 0)
129 | assert(job.shadowStatus == null)
130 | }
131 |
132 | jobs = getAllJobs()
133 | assert(jobs.nonEmpty)
134 | for (job <- jobs) {
135 | job.initialGapInDays = 2
136 | storageService.updateJob(job)
137 | }
138 |
139 | conf_map += ("mode" -> "MIGRATOR")
140 | createMigrationConf(conf_map)
141 | Console.println("RUNNING MIGRATOR-1 -- Not yet time to Migrate test tables")
142 | jobs = Migrator().getRelevantJobs()
143 | assert(jobs.isEmpty)
144 |
145 | jobs = getAllJobs()
146 | assert(jobs.nonEmpty)
147 | for (job <- jobs) {
148 | job.initialGapInDays = 2
149 | job.commLevel1Date = getTimestampDaysAgo(job.initialGapInDays + 1)
150 | storageService.updateJob(job)
151 | }
152 |
153 | conf_map += ("mode" -> "MIGRATOR")
154 | createMigrationConf(conf_map)
155 | Console.println("RUNNING MIGRATOR-1 -- Time to Migrate test tables")
156 | MigrationRunner().process()
157 |
158 | jobs = getAllJobs()
159 | assert(jobs.nonEmpty)
160 | for (job <- jobs) {
161 | assert(job.stgFormat == "parquet")
162 | assert(job.toBeProcessed == 0)
163 | assert(job.inProcess == 1)
164 | assert(job.state == JobState.WritesUnblocked)
165 | assert(job.desiredState == JobState.WritesUnblocked)
166 | assert(job.commLevel1Date > -1L)
167 | assert(job.commLevel2Date == -1L)
168 | assert(job.commLevel3Date == -1L)
169 | assert(job.migrationPaused == 0)
170 | assert(job.shadowStatus == null)
171 | }
172 |
173 | val shadower = new Shadower
174 | numRowsPerTable = shadowWriteNumRows
175 | tables = new SparkTablesGenerator(
176 | numTestTables,
177 | numRowsPerTable,
178 | TableFormat.HIVE_PARQUET,
179 | storageService)
180 | Console.println(
181 | s"TESTING SHADOWER functionality -- $shadowIters shadow iterations on $jobs tables")
182 | jobs = getAllJobs()
183 | assert(jobs.nonEmpty)
184 | for (i <- 1 to shadowIters) {
185 | for (job <- jobs) {
186 | val icebergTable = getFullyQualifiedTableName(job)
187 | tables.writeNumRowsToTable(getFullyQualifiedTableName(job))
188 | Console.println(s"Writing $numRowsPerTable rows to Iceberg table $icebergTable")
189 | Console.println(
190 | s"Syncing Hive table ${icebergTable + HIVE_SUFFIX} from Iceberg table $icebergTable")
191 | shadower.shadowJob(job)
192 | val icebergTableName = getFullyQualifiedTableName(job)
193 | val icebergTableRowCnt = getIcebergTableRowCount(icebergTableName)
194 | val hiveTableName = icebergTableName + HIVE_SUFFIX
195 | val hiveTableRowCnt = getHiveTableRowCount(hiveTableName)
196 | assert(hiveTableRowCnt == icebergTableRowCnt)
197 | }
198 | }
199 |
200 | conf_map += ("mode" -> "COMMUNICATOR")
201 | createMigrationConf(conf_map)
202 | Console.println("RUNNING COMMUNICATOR-2")
203 | MigrationRunner().process()
204 |
205 | jobs = getAllJobs()
206 | assert(jobs.nonEmpty)
207 | for (job <- jobs) {
208 | assert(job.stgFormat == "parquet")
209 | assert(job.toBeProcessed == 0)
210 | assert(job.inProcess == 1)
211 | assert(job.state == JobState.WritesUnblocked)
212 | assert(job.desiredState == JobState.WritesUnblocked)
213 | assert(job.commLevel1Date > -1L)
214 | assert(job.commLevel2Date > -1L)
215 | assert(job.commLevel3Date == -1L)
216 | assert(job.migrationPaused == 0)
217 | assert(job.shadowStatus == null)
218 | }
219 |
220 | jobs = getAllJobs()
221 | assert(jobs.nonEmpty)
222 | for (job <- jobs) {
223 | job.probationGapInDays = 5
224 | storageService.updateJob(job)
225 | }
226 |
227 | conf_map += ("mode" -> "PREPROCESSOR")
228 | createMigrationConf(conf_map)
229 | Console.println("RUNNING PREPROCESSOR-2 -- Not yet time to Preprocess")
230 | jobs = Preprocessor().getRelevantJobs()
231 | assert(jobs.isEmpty)
232 |
233 | jobs = getAllJobs()
234 | assert(jobs.nonEmpty)
235 | for (job <- jobs) {
236 | job.probationGapInDays = 3
237 | job.commLevel2Date = getTimestampDaysAgo(job.probationGapInDays + 1)
238 | storageService.updateJob(job)
239 | }
240 |
241 | conf_map += ("mode" -> "PREPROCESSOR")
242 | createMigrationConf(conf_map)
243 | Console.println("RUNNING PREPROCESSOR-2 -- Time to Preprocess")
244 | MigrationRunner().process()
245 |
246 | jobs = getAllJobs()
247 | assert(jobs.nonEmpty)
248 | for (job <- jobs) {
249 | assert(job.stgFormat == "parquet")
250 | assert(job.toBeProcessed == 0)
251 | assert(job.inProcess == 1)
252 | assert(job.state == JobState.WritesUnblocked)
253 | assert(job.desiredState == JobState.HiveDropped)
254 | assert(job.commLevel1Date > -1L)
255 | assert(job.commLevel2Date > -1L)
256 | assert(job.commLevel3Date == -1L)
257 | assert(job.migrationPaused == 0)
258 | assert(job.shadowStatus == null)
259 | }
260 |
261 | conf_map += ("mode" -> "MIGRATOR")
262 | createMigrationConf(conf_map)
263 | Console.println("RUNNING MIGRATOR-2")
264 | val startTime = System.currentTimeMillis().toDouble
265 | MigrationRunner().process()
266 | val endTime = System.currentTimeMillis().toDouble
267 | val elapsedTime = (endTime - startTime) / 1000.0 / 60.0
268 | println(
269 | s"Average Migrator Elapsed time: ${elapsedTime / numTestTables} " +
270 | s"minutes per table with $numRowsPerTable rows")
271 |
272 | jobs = getAllJobs()
273 | assert(jobs.nonEmpty)
274 | for (job <- jobs) {
275 | assert(job.stgFormat == "parquet")
276 | assert(job.toBeProcessed == 0)
277 | assert(job.inProcess == 1)
278 | assert(job.state == JobState.HiveDropped)
279 | assert(job.desiredState == JobState.HiveDropped)
280 | assert(job.commLevel1Date > -1L)
281 | assert(job.commLevel2Date > -1L)
282 | assert(job.commLevel3Date == -1L)
283 | assert(job.migrationPaused == 0)
284 | assert(job.shadowStatus == null)
285 | }
286 |
287 | conf_map += ("mode" -> "PREPROCESSOR")
288 | createMigrationConf(conf_map)
289 | Console.println("RUNNING PREPROCESSOR-3")
290 | MigrationRunner().process()
291 |
292 | jobs = getAllJobs()
293 | assert(jobs.nonEmpty)
294 | for (job <- jobs) {
295 | assert(job.stgFormat == "parquet")
296 | assert(job.toBeProcessed == 0)
297 | assert(job.inProcess == 0)
298 | assert(job.state == JobState.HiveDropped)
299 | assert(job.desiredState == JobState.HiveDropped)
300 | assert(job.commLevel1Date > -1L)
301 | assert(job.commLevel2Date > -1L)
302 | assert(job.commLevel3Date == -1L)
303 | assert(job.migrationPaused == 0)
304 | assert(job.shadowStatus == null)
305 | }
306 |
307 | conf_map += ("mode" -> "COMMUNICATOR")
308 | createMigrationConf(conf_map)
309 | Console.println("RUNNING COMMUNICATOR-3")
310 | MigrationRunner().process()
311 |
312 | jobs = getAllJobs()
313 | assert(jobs.nonEmpty)
314 | val tablesSet = mutable.Set[(String, String, String)]()
315 | for (job <- jobs) {
316 | assert(job.stgFormat == "parquet")
317 | assert(job.toBeProcessed == 0)
318 | assert(job.inProcess == 0)
319 | assert(job.state == JobState.HiveDropped)
320 | assert(job.desiredState == JobState.HiveDropped)
321 | assert(job.commLevel1Date > -1L)
322 | assert(job.commLevel2Date > -1L)
323 | assert(job.commLevel3Date > -1L)
324 | assert(job.migrationPaused == 0)
325 | assert(job.shadowStatus == null)
326 | tablesSet.add((job.catalogName, job.dbName, job.tblName))
327 | }
328 |
329 | Console.println("RUNNING BULK-REVERTER")
330 | val reverter = Reverter()
331 | reverter.bulkRevertFinalizedTables(tablesSet.toSet)
332 |
333 | for (table <- tablesSet) {
334 | assert(isHiveTable(table._1, table._2, table._3))
335 | }
336 |
337 | jobs = getAllJobs()
338 | assert(jobs.nonEmpty)
339 | for (job <- jobs) {
340 | assert(job.stgFormat == "parquet")
341 | assert(job.toBeProcessed == 1)
342 | assert(job.inProcess == 0)
343 | assert(job.state == JobState.Ready)
344 | assert(job.desiredState == JobState.WritesBlocked)
345 | assert(job.commLevel1Date > -1L)
346 | assert(job.commLevel2Date > -1L)
347 | assert(job.commLevel3Date > -1L)
348 | assert(job.migrationPaused == 1)
349 | assert(job.shadowStatus == null)
350 | metacatService.forceDropTable(
351 | job.catalogName,
352 | job.dbName,
353 | job.tblName + HIVE_SUFFIX + ARCHIVED_SUFFIX)
354 | }
355 | }
356 |
357 | ignore("H2I Migration Integration Test -- skip Iceberg tables in PREPROCESSOR") {
358 | val tables = new SparkTablesGenerator(numTestTables, 10, TableFormat.ICEBERG, storageService)
359 | tables.createTables()
360 |
361 | var conf_map = Map(
362 | "mode" -> "PREPROCESSOR",
363 | "jobsProcessBatchSize" -> "10",
364 | "initialGapInWeeks" -> "0",
365 | "probationGapInWeeks" -> "0",
366 | "dbEnv" -> "test",
367 | "local" -> "false",
368 | "dryrun" -> "false",
369 | "distributed" -> "false",
370 | "queueName" -> "jobs_int_test",
371 | "archiveHive" -> "false")
372 | createMigrationConf(conf_map)
373 | Console.println("RUNNING PREPROCESSOR-1")
374 | MigrationRunner().process()
375 |
376 | var jobs = getAllJobs()
377 | for (job <- jobs) {
378 | assert(job.stgFormat.isEmpty)
379 | assert(job.toBeProcessed == 0)
380 | assert(job.inProcess == 0)
381 | assert(job.state == JobState.Undefined)
382 | assert(job.desiredState == JobState.Undefined)
383 | assert(job.commLevel1Date == -1L)
384 | assert(job.commLevel2Date == -1L)
385 | assert(job.commLevel3Date == -1L)
386 | assert(job.migrationPaused == 1)
387 | assert(job.shadowStatus == null)
388 | }
389 |
390 | conf_map += ("mode" -> "COMMUNICATOR")
391 | createMigrationConf(conf_map)
392 | Console.println("RUNNING COMMUNICATOR-1")
393 | jobs = Communicator().getRelevantJobs()
394 | assert(jobs.isEmpty)
395 | MigrationRunner().process()
396 |
397 | conf_map += ("mode" -> "MIGRATOR")
398 | createMigrationConf(conf_map)
399 | Console.println("RUNNING MIGRATOR-1")
400 | jobs = Migrator().getRelevantJobs()
401 | assert(jobs.isEmpty)
402 | }
403 |
404 | ignore("H2I Migration Integration Test -- skip Iceberg tables in COMMUNICATOR") {
405 | val rowsPerTable = 10
406 | var tables = new SparkTablesGenerator(
407 | numTestTables,
408 | rowsPerTable,
409 | TableFormat.HIVE_PARQUET,
410 | storageService)
411 | tables.createTables()
412 |
413 | var conf_map = Map(
414 | "mode" -> "PREPROCESSOR",
415 | "jobsProcessBatchSize" -> "10",
416 | "initialGapInWeeks" -> "0",
417 | "probationGapInWeeks" -> "0",
418 | "dbEnv" -> "test",
419 | "local" -> "false",
420 | "dryrun" -> "false",
421 | "distributed" -> "false",
422 | "queueName" -> "jobs_int_test",
423 | "archiveHive" -> "false")
424 | createMigrationConf(conf_map)
425 | Console.println("RUNNING PREPROCESSOR-1")
426 | mock_preprocessor(1)
427 |
428 | var jobs = getAllJobs()
429 | for (job <- jobs) {
430 | assert(job.stgFormat == "parquet")
431 | assert(job.toBeProcessed == 1)
432 | assert(job.inProcess == 0)
433 | assert(job.state == JobState.Ready)
434 | assert(job.desiredState == JobState.WritesBlocked)
435 | assert(job.commLevel1Date == -1L)
436 | assert(job.commLevel2Date == -1L)
437 | assert(job.commLevel3Date == -1L)
438 | assert(job.migrationPaused == 0)
439 | assert(job.shadowStatus == null)
440 | }
441 |
442 | tables =
443 | new SparkTablesGenerator(numTestTables, rowsPerTable, TableFormat.ICEBERG, storageService)
444 | tables.createTables()
445 | mock_preprocessor(1)
446 |
447 | conf_map += ("mode" -> "COMMUNICATOR")
448 | createMigrationConf(conf_map)
449 | Console.println("RUNNING COMMUNICATOR-1")
450 | MigrationRunner().process()
451 |
452 | jobs = getAllJobs()
453 | for (job <- jobs) {
454 | assert(job.stgFormat == "parquet")
455 | assert(job.toBeProcessed == 1)
456 | assert(job.inProcess == 0)
457 | assert(job.state == JobState.Ready)
458 | assert(job.desiredState == JobState.WritesBlocked)
459 | assert(job.commLevel1Date == -1L)
460 | assert(job.commLevel2Date == -1L)
461 | assert(job.commLevel3Date == -1L)
462 | assert(job.migrationPaused == 1)
463 | assert(job.shadowStatus == null)
464 | }
465 |
466 | conf_map += ("mode" -> "MIGRATOR")
467 | createMigrationConf(conf_map)
468 | Console.println("RUNNING MIGRATOR-1")
469 | jobs = Migrator().getRelevantJobs()
470 | assert(jobs.isEmpty)
471 | }
472 |
473 | ignore("H2I Migration Integration Test -- skip Iceberg tables in MIGRATOR") {
474 | val rowsPerTable = 10
475 | var tables = new SparkTablesGenerator(
476 | numTestTables,
477 | rowsPerTable,
478 | TableFormat.HIVE_PARQUET,
479 | storageService)
480 | tables.createTables()
481 |
482 | var conf_map = Map(
483 | "mode" -> "PREPROCESSOR",
484 | "jobsProcessBatchSize" -> "10",
485 | "initialGapInWeeks" -> "0",
486 | "probationGapInWeeks" -> "0",
487 | "dbEnv" -> "test",
488 | "local" -> "false",
489 | "dryrun" -> "false",
490 | "distributed" -> "false",
491 | "queueName" -> "jobs_int_test",
492 | "archiveHive" -> "false")
493 | createMigrationConf(conf_map)
494 | Console.println("RUNNING PREPROCESSOR-1")
495 | mock_preprocessor(1)
496 |
497 | var jobs = getAllJobs()
498 | for (job <- jobs) {
499 | assert(job.stgFormat == "parquet")
500 | assert(job.toBeProcessed == 1)
501 | assert(job.inProcess == 0)
502 | assert(job.state == JobState.Ready)
503 | assert(job.desiredState == JobState.WritesBlocked)
504 | assert(job.commLevel1Date == -1L)
505 | assert(job.commLevel2Date == -1L)
506 | assert(job.commLevel3Date == -1L)
507 | assert(job.migrationPaused == 0)
508 | assert(job.shadowStatus == null)
509 | }
510 |
511 | conf_map += ("mode" -> "COMMUNICATOR")
512 | createMigrationConf(conf_map)
513 | Console.println("RUNNING COMMUNICATOR-1")
514 | MigrationRunner().process()
515 |
516 | jobs = getAllJobs()
517 | for (job <- jobs) {
518 | assert(job.stgFormat == "parquet")
519 | assert(job.toBeProcessed == 1)
520 | assert(job.inProcess == 0)
521 | assert(job.state == JobState.Ready)
522 | assert(job.desiredState == JobState.WritesBlocked)
523 | assert(job.commLevel1Date > -1L)
524 | assert(job.commLevel2Date == -1L)
525 | assert(job.commLevel3Date == -1L)
526 | assert(job.migrationPaused == 0)
527 | assert(job.shadowStatus == null)
528 | }
529 |
530 | tables =
531 | new SparkTablesGenerator(numTestTables, rowsPerTable, TableFormat.ICEBERG, storageService)
532 | tables.createTables()
533 | jobs = getAllJobs()
534 | for (job <- jobs) {
535 | job.stgFormat = "parquet"
536 | job.toBeProcessed = 1
537 | job.state = JobState.Ready
538 | job.desiredState = JobState.WritesBlocked
539 | job.commLevel1Date = Instant.now().toEpochMilli
540 | storageService.updateJob(job)
541 | }
542 |
543 | conf_map += ("mode" -> "MIGRATOR")
544 | createMigrationConf(conf_map)
545 | Console.println("RUNNING MIGRATOR-1")
546 | MigrationRunner().process()
547 |
548 | jobs = getAllJobs()
549 | for (job <- jobs) {
550 | assert(job.stgFormat == "parquet")
551 | assert(job.toBeProcessed == 1)
552 | assert(job.inProcess == 0)
553 | assert(job.state == JobState.Ready)
554 | assert(job.desiredState == JobState.WritesBlocked)
555 | assert(job.commLevel1Date > -1L)
556 | assert(job.commLevel2Date == -1L)
557 | assert(job.commLevel3Date == -1L)
558 | assert(job.migrationPaused == 1)
559 | assert(job.shadowStatus == null)
560 | }
561 | }
562 |
563 | ignore("H2I Migration Integration Test -- skip csv/text Hive tables") {
564 | val rowsPerTable = 10
565 | val tables = new SparkTablesGenerator(
566 | numTestTables,
567 | rowsPerTable,
568 | TableFormat.HIVE_PARQUET,
569 | storageService)
570 | tables.createTables()
571 |
572 | var conf_map = Map(
573 | "mode" -> "PREPROCESSOR",
574 | "jobsProcessBatchSize" -> "10",
575 | "initialGapInWeeks" -> "0",
576 | "probationGapInWeeks" -> "0",
577 | "dbEnv" -> "test",
578 | "local" -> "false",
579 | "dryrun" -> "false",
580 | "distributed" -> "false",
581 | "queueName" -> "jobs_int_test",
582 | "archiveHive" -> "false")
583 | createMigrationConf(conf_map)
584 | Console.println("RUNNING PREPROCESSOR-1")
585 | mock_preprocessor(1)
586 |
587 | var jobs = getAllJobs()
588 | for (job <- jobs) {
589 | job.stgFormat = "csv/text"
590 | storageService.updateJob(job)
591 | }
592 |
593 | conf_map += ("mode" -> "COMMUNICATOR")
594 | createMigrationConf(conf_map)
595 | Console.println("RUNNING COMMUNICATOR-1")
596 | MigrationRunner().process()
597 |
598 | jobs = getAllJobs()
599 | for (job <- jobs) {
600 | assert(job.stgFormat == HIVE_CSV_TEXT)
601 | assert(job.toBeProcessed == 1)
602 | assert(job.inProcess == 0)
603 | assert(job.state == JobState.Ready)
604 | assert(job.desiredState == JobState.WritesBlocked)
605 | assert(job.commLevel1Date == -1L)
606 | assert(job.commLevel2Date == -1L)
607 | assert(job.commLevel3Date == -1L)
608 | assert(job.migrationPaused == 1)
609 | assert(job.pauseReason == PauseReason.IS_CSV_TEXT.toString)
610 | assert(job.shadowStatus == null)
611 | }
612 |
613 | conf_map += ("mode" -> "MIGRATOR")
614 | createMigrationConf(conf_map)
615 | Console.println("RUNNING MIGRATOR-1")
616 | jobs = Migrator().getRelevantJobs()
617 | assert(jobs.isEmpty)
618 |
619 | conf_map += ("mode" -> "SHADOWER")
620 | createMigrationConf(conf_map)
621 | Console.println("RUNNING SHADOWER-1")
622 | jobs = Shadower().getRelevantJobs()
623 | assert(jobs.isEmpty)
624 | }
625 |
626 | ignore("H2I Migration Integration Test -- Pause Migration on TABLE_NOT_FOUND by PREPROCESSOR") {
627 | val tables =
628 | new SparkTablesGenerator(numTestTables, 0, TableFormat.HIVE_PARQUET, storageService)
629 | tables.dropTables()
630 |
631 | val job = new Job(
632 | null,
633 | TEST_CATALOG,
634 | TEST_DATABASE,
635 | "this_table_does_not_exist",
636 | "",
637 | "",
638 | null,
639 | null,
640 | 0,
641 | 0,
642 | JobState.Undefined,
643 | JobState.Undefined,
644 | 0,
645 | 0,
646 | 0,
647 | 0,
648 | 0,
649 | 0,
650 | 0,
651 | "",
652 | null,
653 | null,
654 | 0,
655 | 0)
656 | storageService.enqueueJob(job)
657 |
658 | val conf_map = Map(
659 | "mode" -> "PREPROCESSOR",
660 | "jobsProcessBatchSize" -> "10",
661 | "initialGapInWeeks" -> "0",
662 | "probationGapInWeeks" -> "0",
663 | "dbEnv" -> "test",
664 | "local" -> "false",
665 | "dryrun" -> "false",
666 | "distributed" -> "false",
667 | "queueName" -> "jobs_int_test",
668 | "archiveHive" -> "false")
669 | createMigrationConf(conf_map)
670 | Console.println("RUNNING PREPROCESSOR-1")
671 | MigrationRunner().process()
672 |
673 | val jobs = getAllJobs()
674 | for (job <- jobs) {
675 | assert(job.stgFormat == "")
676 | assert(job.toBeProcessed == 0)
677 | assert(job.inProcess == 0)
678 | assert(job.state == JobState.Undefined)
679 | assert(job.desiredState == JobState.Undefined)
680 | assert(job.commLevel1Date == -1L)
681 | assert(job.commLevel2Date == -1L)
682 | assert(job.commLevel3Date == -1L)
683 | assert(job.migrationPaused == 1)
684 | assert(job.pauseReason == PauseReason.TABLE_NOT_FOUND.toString)
685 | assert(job.shadowStatus == null)
686 | }
687 | }
688 | }
689 |
--------------------------------------------------------------------------------
/src/main/scala/com/netflix/migration/utils/Utils.scala:
--------------------------------------------------------------------------------
1 | package com.netflix.migration.utils
2 |
3 | import com.netflix.metacat.common.dto.TableDto
4 | import com.netflix.migration.data.{Job, JobState, MigrationCommandMode, MigrationConfig}
5 | import com.netflix.migration.modes.Shadower
6 | import com.netflix.migration.providers.{IcebergTableServiceProvider, MetacatServiceProvider, MigrationConfigProvider, NdcServiceProvider, SparkSessionProvider}
7 | import com.netflix.migration.services.{IcebergTableService, MetacatService, NdcService, StorageService, StorageServiceImpl}
8 | import com.netflix.migration.utils.MigrationConsts.DataCategory.DataCategory
9 | import com.netflix.migration.utils.MigrationConsts.{ARCHIVED_SUFFIX, DO_NOT_DROP_TAG, DO_NOT_RENAME_TAG, HIVE_ARCHIVED_TAG, HIVE_SUFFIX, HIVE_TIMESTAMP_COLUMN_NAME, ICEBERG_SUFFIX, PROD_ENV, PauseReason, REVERT_SUFFIX}
10 | import com.netflix.migration.utils.MigrationRunnerUtils.retry
11 | import com.typesafe.scalalogging.StrictLogging
12 | import org.apache.spark.SparkConf
13 | import org.apache.spark.sql.SparkSession
14 |
15 | import java.nio.file.Files
16 | import java.security.MessageDigest
17 | import java.time.LocalDate
18 | import java.time.format.DateTimeFormatter
19 | import scala.collection.JavaConverters.{asScalaIteratorConverter, iterableAsScalaIterableConverter}
20 | import scala.collection.mutable
21 |
22 | object Utils extends StrictLogging {
23 |
24 | val migrationConf: MigrationConfig = MigrationConfigProvider.getMigrationConfig
25 | var spark: SparkSession = SparkSessionProvider.getSparkSession
26 | var metacatService: MetacatService = MetacatServiceProvider.getMetacatService
27 | var ndcService: NdcService = NdcServiceProvider.getNdcService
28 | var icebergTableService: IcebergTableService =
29 | IcebergTableServiceProvider.getIcebergTableService
30 |
31 | type transition = Job => Unit
32 |
33 | private val stageChangeActionMap: Map[(JobState.JobState, JobState.JobState), transition] =
34 | Map(
35 | (JobState.Ready, JobState.WritesBlocked) -> blockWrites,
36 | (JobState.WritesBlocked, JobState.IcebergReady) -> migrateTableIceberg,
37 | (JobState.IcebergReady, JobState.IcebergPrimary) -> makeIcebergPrimary,
38 | (JobState.IcebergPrimary, JobState.WritesUnblocked) -> unblockWrites,
39 | (JobState.WritesUnblocked, JobState.WritesBlocked) -> blockWrites,
40 | (JobState.WritesBlocked, JobState.SyncHive) -> syncHiveFromIceberg,
41 | (JobState.SyncHive, JobState.HivePrimary) -> makeHivePrimary,
42 | (JobState.HivePrimary, JobState.WritesUnblocked) -> unblockWrites,
43 | (JobState.WritesUnblocked, JobState.IcebergDropped) -> dropIceberg,
44 | (JobState.WritesUnblocked, JobState.HiveDropped) -> dropHive)
45 |
46 | /**
47 | * @param job
48 | * @return
49 | */
50 | def getJobNextAction(job: Job): Option[transition] = {
51 | stageChangeActionMap.get((job.state, job.desiredState))
52 | }
53 |
54 | /**
55 | * Update the state of a job entity.
56 | *
57 | * @param job
58 | * the job entity to be updated
59 | * @param cur
60 | * the current state of the job
61 | * @param desired
62 | * the desired state of the job
63 | */
64 | private def updateJobState(
65 | job: Job,
66 | cur: JobState.JobState,
67 | desired: JobState.JobState): Unit = {
68 | job.desiredState = desired
69 | job.state = cur
70 | }
71 |
72 | /**
73 | * Block writes for the specified job
74 | *
75 | * @param job
76 | * the job for which the writes needs to be blocked
77 | */
78 | private def blockWrites(job: Job): Unit = {
79 | val fullTableName = s"${job.catalogName}/${job.dbName}/${job.tblName}"
80 | println(s"Blocking writes for table: $fullTableName")
81 | metacatService.blockTableWrites(job.catalogName, job.dbName, job.tblName)
82 | println(s"Successfully blocked writes for table: $fullTableName")
83 |
84 | if (migrationConf.commandMode == MigrationCommandMode.REVERTER) {
85 | updateJobState(job, JobState.WritesBlocked, JobState.SyncHive)
86 | } else {
87 | updateJobState(job, JobState.WritesBlocked, JobState.IcebergReady)
88 | }
89 | }
90 |
91 | /**
92 | * Unblock writes for the specified job
93 | *
94 | * @param job
95 | * the job for which the writes needs to be unblocked
96 | */
97 | private def unblockWrites(job: Job): Unit = {
98 | val tblName = job.tblName
99 | val hiveTblName = job.tblName + HIVE_SUFFIX
100 | metacatService.unBlockTableWrites(job.catalogName, job.dbName, tblName)
101 | if (migrationConf.dbEnv == PROD_ENV) {
102 | metacatService.setStrictSecure(job.catalogName, job.dbName, tblName)
103 | }
104 | metacatService.unBlockTableWrites(job.catalogName, job.dbName, hiveTblName)
105 | metacatService.setTableTag(job.catalogName, job.dbName, hiveTblName, HIVE_ARCHIVED_TAG)
106 |
107 | if (migrationConf.commandMode == MigrationCommandMode.REVERTER) {
108 | updateJobState(job, JobState.WritesUnblocked, JobState.IcebergDropped)
109 | } else {
110 | // for now transition to hive dropped is manual for safety, can be automated down the line
111 | updateJobState(job, JobState.WritesUnblocked, JobState.WritesUnblocked)
112 | }
113 | }
114 |
115 | /**
116 | * Returns the qualified table name for a job based on the Spark version
117 | *
118 | * @param spark
119 | * SparkSession
120 | * @param job
121 | * Job
122 | * @return
123 | * String
124 | */
125 | private def jobQualTblNameForSpark(spark: SparkSession, job: Job): String = {
126 | val base = s"${job.dbName}.${job.tblName}"
127 | if (spark.version >= "2.4") return s"${job.catalogName}." + base
128 | if (job.catalogName == "prodhive") base else null
129 | }
130 |
131 | def getSparkSession(): SparkSession = {
132 | if (spark == null) {
133 | spark = buildSparkSession()
134 | }
135 | spark
136 | }
137 |
138 | /**
139 | * Build a SparkSession with the appropriate configurations
140 | *
141 | * @return
142 | * The SparkSession object
143 | */
144 | private def buildSparkSession(): SparkSession = {
145 | val sparkConf = initializeSparkConf()
146 | if (migrationConf.runLocally) {
147 | SparkSession
148 | .builder()
149 | .master("local[*]")
150 | .config(initializeLocalSparkConfLocal(sparkConf))
151 | .enableHiveSupport()
152 | .getOrCreate()
153 | } else {
154 | SparkSession
155 | .builder()
156 | .config(sparkConf)
157 | .enableHiveSupport()
158 | .getOrCreate()
159 | }
160 | }
161 |
162 | /**
163 | * Initialize the SparkConf with settings specific to the Spark Iceberg Migration job
164 | *
165 | * @return
166 | * The SparkConf object
167 | */
168 | private def initializeSparkConf(): SparkConf = {
169 | new SparkConf(true)
170 | .setAppName("Spark Iceberg Migration")
171 | .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
172 | .set("spark.sql.shuffle.partitions", "1500")
173 | .set("spark.speculation", "false")
174 | .set("spark.task.maxFailures", "30") // default=4
175 | .set("spark.network.timeout", "900")
176 | .set("spark.shuffle.io.maxRetries", "30") // default=3
177 | .set("spark.reducer.maxReqsInFlight", "10")
178 | .set("spark.rpc.message.maxSize", "2047")
179 | .set("spark.executor.memoryOverhead", "3g")
180 | .set("spark.driver.memory", "15g")
181 | .set("spark.executor.memory", "15g")
182 | .set("spark.executor.cores", "8")
183 | .set("spark.task.cpus", "2")
184 | .set(
185 | "spark.sql.extensions",
186 | "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions")
187 | }
188 |
189 | /**
190 | * Initialize the SparkConf with settings specific to a local environment
191 | *
192 | * @param sparkConf
193 | * SparkConf object to be modified
194 | * @return
195 | * The modified SparkConf object
196 | */
197 | def initializeLocalSparkConfLocal(sparkConf: SparkConf): SparkConf = {
198 | val tmp_dir = Files.createTempDirectory("MigrationTooling-").toFile.getAbsolutePath
199 | sparkConf
200 | .set(
201 | "spark.hadoop.bdp.s3.credentials-provider",
202 | "com.netflix.hadoop.aws.ConfigurableCredentialsProvider")
203 | .set("spark.hadoop.bdp.s3.staging-directory", s"$tmp_dir/s3")
204 | .set("spark.hadoop.bdp.s3.use-instance-credentials", "false")
205 | .set("spark.hadoop.fs.defaultFS", s"file:$tmp_dir")
206 | .set("spark.hadoop.hive-metastore.host", "http://hive-metastore-host-url")
207 | }
208 |
209 | /**
210 | * Migrate the table associated with the given job to Iceberg
211 | *
212 | * @param job
213 | * Job object containing information about the table and its location
214 | */
215 | private def migrateTableIceberg(job: Job): Unit = {
216 | if (!isHiveTable(job.catalogName, job.dbName, job.tblName)) {
217 | metacatService.unBlockTableWrites(job.catalogName, job.dbName, job.tblName)
218 | job.migrationPaused = 1
219 | job.pauseReason = PauseReason.IS_ALREADY_ICEBERG.toString
220 | job.state = JobState.Ready
221 | job.desiredState = JobState.WritesBlocked
222 | @transient lazy val storageService: StorageService = StorageServiceImpl(migrationConf)
223 | storageService.updateJob(job)
224 | return
225 | }
226 |
227 | val jobQualTblName = jobQualTblNameForSpark(spark, job)
228 | val icebergJobQualTblName = jobQualTblName + ICEBERG_SUFFIX
229 | if (jobQualTblName == null) {
230 | logger.warn(s"can only migrate prodhive for this spark version. Table: $jobQualTblName")
231 | updateJobState(job, JobState.IcebergReady, JobState.IcebergReady)
232 | return
233 | }
234 |
235 | val icebergTableName = job.tblName + ICEBERG_SUFFIX
236 | val icebergQName =
237 | StringUtils.getTableQualifiedName(job.catalogName, job.dbName, icebergTableName)
238 | // If the iceberg table already exists, drop it.
239 | if (metacatService.tableExists(job.catalogName, job.dbName, icebergTableName)) {
240 | logger.warn(s"Iceberg table: $icebergQName already exists. Attempting to drop it..")
241 | metacatService.deleteTable(job.catalogName, job.dbName, icebergTableName)
242 | }
243 |
244 | var hasTimestampColumn = false
245 | val tableDtoOpt = metacatService.getTable(job.catalogName, job.dbName, job.tblName)
246 | tableDtoOpt match {
247 | case Some(tableDto) if Utils.tableHasTimestampColumn(tableDto) =>
248 | hasTimestampColumn = true
249 | case _ => // Do nothing
250 | }
251 |
252 | try {
253 | if ((job.stgFormat.equals("parquet") && !job.dbName.startsWith("ae_") && !hasTimestampColumn) ||
254 | isHiveTableEmpty(job)) {
255 | spark.sql(s"SNAPSHOT TABLE $jobQualTblName AS $icebergJobQualTblName USING iceberg")
256 | } else {
257 | throw new org.apache.spark.SparkException(
258 | "Cannot convert table with non-Parquet partitions")
259 | }
260 | } catch {
261 | case e: org.apache.spark.SparkException =>
262 | job.stgFormat = MigrationConsts.HIVE_CSV_TEXT
263 | if (e.getMessage.contains("Cannot convert table with non-Parquet partitions") ||
264 | e.getMessage.contains("is not a Parquet file")) {
265 | spark.sql(s"CREATE TABLE $icebergJobQualTblName LIKE $jobQualTblName USING iceberg")
266 | spark.sql(s"ALTER TABLE $icebergJobQualTblName " +
267 | s"SET TBLPROPERTIES ('spark.behavior.compatibility'='true','migrated-from-hive'='true')")
268 | val shadower = new Shadower
269 | shadower.copyDataSrcToDst(jobQualTblName, icebergJobQualTblName)
270 | } else {
271 | throw e // if the exception message is not the one we expect, rethrow the exception
272 | }
273 | }
274 | updateJobState(job, JobState.IcebergReady, JobState.IcebergPrimary)
275 | }
276 |
277 | /**
278 | * Set the watermark snapshot ID for the given job's table in Hive
279 | *
280 | * @param job
281 | * Job object containing information about the table and its location
282 | */
283 | private def setWatermarkSnapshotId(job: Job): Unit = {
284 | val tblName = getFullyQualifiedTableName(job)
285 | val hiveFullTableName = tblName + HIVE_SUFFIX
286 | val hiveTableName = job.tblName + HIVE_SUFFIX
287 | val icebergTableName = tblName
288 | try {
289 | val currentSnapshotId = getCurrentSnapshotId(icebergTableName)
290 | // Since renameTable carries over tags, hiveName table will also have writes blocked
291 | metacatService.unBlockTableWrites(job.catalogName, job.dbName, hiveTableName)
292 | spark.sql(
293 | s"ALTER TABLE $hiveFullTableName SET TBLPROPERTIES('watermark_snapshot_id' = $currentSnapshotId)")
294 | metacatService.blockTableWrites(job.catalogName, job.dbName, hiveTableName)
295 | } catch {
296 | case _ =>
297 | logger.warn(
298 | "Iceberg table is unaccessible, possible migrated from legacy Hive table with STRICT permission issue")
299 | }
300 | }
301 |
302 | private def syncHiveFromIceberg(job: Job): Unit = {
303 | val shadower = new Shadower
304 | shadower.shadowJob(job)
305 | updateJobState(job, JobState.SyncHive, JobState.HivePrimary)
306 | }
307 |
308 | /**
309 | * Make Hive primary for the given job
310 | *
311 | * @param job
312 | * Job object containing information about the table and its location
313 | */
314 | private def makeHivePrimary(job: Job): Unit = {
315 | val tblName = job.tblName
316 | val hiveName = tblName + HIVE_SUFFIX
317 | val icebergName = tblName + ICEBERG_SUFFIX
318 | metacatService.renameTable(job.catalogName, job.dbName, tblName, icebergName)
319 | metacatService.renameTable(job.catalogName, job.dbName, hiveName, tblName)
320 | updateJobState(job, JobState.HivePrimary, JobState.WritesUnblocked)
321 | }
322 |
323 | /**
324 | * Make Iceberg primary for the given job
325 | *
326 | * @param job
327 | * Job object containing information about the table and its location
328 | */
329 | private def makeIcebergPrimary(job: Job): Unit = {
330 | val tblName = job.tblName
331 | val hiveName = tblName + HIVE_SUFFIX
332 | val icebergName = tblName + ICEBERG_SUFFIX
333 | val hiveQName = StringUtils.getTableQualifiedName(job.catalogName, job.dbName, hiveName)
334 | val icebergQName = StringUtils.getTableQualifiedName(job.catalogName, job.dbName, icebergName)
335 |
336 | // Ensure hive table doesn't already exist (this could be a retry)
337 | if (!metacatService.tableExists(job.catalogName, job.dbName, hiveName)) {
338 | metacatService.renameTable(job.catalogName, job.dbName, tblName, hiveName)
339 | } else {
340 | logger.warn(s"Hive Table $hiveQName already exists. Skipping rename.")
341 | }
342 |
343 | // On a retry, the iceberg table won't exist. So make sure it exists the first time.
344 | if (metacatService.tableExists(job.catalogName, job.dbName, icebergName)) {
345 | metacatService.renameTable(job.catalogName, job.dbName, icebergName, tblName)
346 | } else {
347 | logger.warn(s"Iceberg Table $icebergQName does not exist. Skipping rename.")
348 | }
349 |
350 | setIcebergTableACLs(job)
351 | copyHiveDefinitionMetadata(job.catalogName, job.dbName, hiveName, tblName)
352 | copyHiveClassificationMetadata(job.catalogName, job.dbName, hiveName, tblName)
353 | setWatermarkSnapshotId(job)
354 | updateJobState(job, JobState.IcebergPrimary, JobState.WritesUnblocked)
355 | }
356 |
357 | private def copyHiveClassificationMetadata(
358 | catalogName: String,
359 | dbName: String,
360 | hiveTableName: String,
361 | icebergTableName: String): Unit = {
362 | val srcMetadataResponseDto = ndcService.getMetadata(catalogName, dbName, hiveTableName)
363 | if (srcMetadataResponseDto != null) {
364 | val dstMetadataResponseDto = ndcService.getMetadata(catalogName, dbName, icebergTableName)
365 | val metadataDto = ndcService.createMetadataDto(srcMetadataResponseDto)
366 | if (dstMetadataResponseDto != null) {
367 | metadataDto.setId(dstMetadataResponseDto.getId)
368 | metadataDto.setName(dstMetadataResponseDto.getName)
369 | metadataDto.setLocation(dstMetadataResponseDto.getLocation)
370 | metadataDto.setSourceLink(dstMetadataResponseDto.getSourceLink)
371 | }
372 | ndcService.setMetadata(List(metadataDto))
373 | }
374 | }
375 |
376 | private def copyHiveDefinitionMetadata(
377 | catalogName: String,
378 | dbName: String,
379 | hiveTableName: String,
380 | icebergTableName: String): Unit = {
381 | val hiveTableDtoOpt = metacatService.getTable(catalogName, dbName, hiveTableName)
382 | if (hiveTableDtoOpt.isEmpty) {
383 | logger.warn(s"Hive table: $hiveTableName doesn't exist. Skipping copy definition metadata.")
384 | return
385 | }
386 | val hiveTableDto = hiveTableDtoOpt.get
387 | val hiveDefinitionMetadata = hiveTableDto.getDefinitionMetadata
388 | if (hiveDefinitionMetadata == null) {
389 | logger.warn(s"Hive table: $hiveTableName definition metadata is empty.")
390 | }
391 |
392 | val icebergTableDtoOpt = metacatService.getTable(catalogName, dbName, icebergTableName)
393 | if (icebergTableDtoOpt.isEmpty) {
394 | logger.warn(
395 | s"Iceberg table: $icebergTableName doesn't exist. Skipping copy definition metadata.")
396 | return
397 | }
398 | val icebergTableDto = icebergTableDtoOpt.get
399 | if (getTagsFromTableMetadata(icebergTableDto)
400 | .getOrElse(Set())
401 | .contains(MigrationConsts.ICEBERG_MIGRATION_DO_NOT_MODIFY_TAG)) {
402 | logger.warn(s"Iceberg table: $icebergTableName already contains definitionMetadata " +
403 | s"and tag ${MigrationConsts.ICEBERG_MIGRATION_DO_NOT_MODIFY_TAG}. Skipping copy definition metadata.")
404 | return
405 | }
406 |
407 | try {
408 | icebergTableDto.setDefinitionMetadata(hiveDefinitionMetadata)
409 | setDataMigratedLocation(hiveTableDto, icebergTableDto)
410 | metacatService.updateTable(catalogName, dbName, icebergTableName, icebergTableDto)
411 | var tagsSet: Set[String] = getTagsFromTableMetadata(hiveTableDto).getOrElse(Set())
412 | tagsSet = tagsSet ++ Set(MigrationConsts.MIGRATED_FROM_HIVE_TAG)
413 | metacatService.setTableTags(catalogName, dbName, icebergTableName, tagsSet)
414 | logger.info(
415 | s"Successfully copied definition from hive table: $hiveTableName to $icebergTableName")
416 | } catch {
417 | case e: Exception =>
418 | logger.error(
419 | s"Failed to update iceberg table: $icebergTableName exists in metacat. Exception: $e")
420 | throw e
421 | }
422 | }
423 |
424 | def setDataMigratedLocation(hiveTableDto: TableDto, icebergTableDto: TableDto): Unit = {
425 | if (MigrationRunnerUtils
426 | .getMetacatFieldValue(icebergTableDto, MigrationConsts.METACAT_DATA_MIGRATED_PROPERTY)
427 | .isDefined) {
428 | logger.warn(s"Migrated data location already set for ${icebergTableDto.getName}")
429 | return
430 | }
431 |
432 | // Get the data location from the hive table
433 | val hiveTableLocation = hiveTableDto.getDataUri
434 | if (hiveTableLocation == null) {
435 | throw new RuntimeException(
436 | s"Could not retrieve data location for metacat table: ${hiveTableDto.getName}")
437 | }
438 |
439 | MigrationRunnerUtils.setMetacatFieldValue(
440 | icebergTableDto,
441 | MigrationConsts.METACAT_DATA_MIGRATED_PROPERTY,
442 | hiveTableLocation)
443 | }
444 |
445 | /**
446 | * Grant the appropriate secure Iceberg permissions to the original hive table owner.
447 | *
448 | * @param job
449 | * The migration job.
450 | */
451 | private def setIcebergTableACLs(job: Job): Unit = {
452 | if (!migrationConf.setIcebergAcls) {
453 | logger.warn(
454 | s"setIcebergAcls conf set to false. Skipping granting ACLs for table: ${job.tblName}")
455 | return
456 | }
457 | // Grant the permissions using the creds of the owner of this genie job
458 | // Since they'll have ownership of the iceberg table too.
459 | val jobUserName = spark.sparkContext.sparkUser
460 | try {
461 | retry(2)(icebergTableService.setGrants(jobUserName, job.catalogName, job.dbName, job.tblName))
462 | } catch {
463 | case e =>
464 | if (e.getMessage.contains("Access Denied")) {
465 | logger.warn(
466 | "Iceberg table is unaccessible, possible migrated from legacy Hive table with STRICT permission issue")
467 | }
468 | }
469 | }
470 |
471 | /**
472 | * Drops the Iceberg table associated with the given job
473 | *
474 | * @param job
475 | * Job object containing information about the table and its location
476 | */
477 | private def dropIceberg(job: Job): Unit = {
478 | val tblName = job.tblName + ICEBERG_SUFFIX
479 | val tblQName = StringUtils.getTableQualifiedName(job.catalogName, job.dbName, tblName)
480 | val revertTblName = tblName + REVERT_SUFFIX
481 | val revertQTblName = StringUtils.getTableQualifiedName(job.catalogName, job.dbName, tblName)
482 | if (!metacatService.tableExists(job.catalogName, job.dbName, revertTblName)) {
483 | metacatService.unBlockTableWrites(job.catalogName, job.dbName, tblName)
484 | metacatService.renameTable(job.catalogName, job.dbName, tblName, revertTblName)
485 | } else {
486 | logger.warn(s"Table $revertQTblName already exists. Skipping rename of $tblQName")
487 | }
488 | updateJobState(job, JobState.IcebergDropped, JobState.IcebergDropped)
489 | }
490 |
491 | /**
492 | * Drops the Hive table associated with the given job
493 | *
494 | * @param job
495 | * Job object containing information about the table and its location
496 | */
497 | private def dropHive(job: Job): Unit = {
498 | val tblQualifiedName = jobQualTblNameForSpark(spark, job) + HIVE_SUFFIX
499 | val tblName = job.tblName + HIVE_SUFFIX
500 | // Archive Hive table in all cases for correctness reasons
501 | // Keep this change till appropriate changes are made to Janitor.
502 | if (migrationConf.archiveHive || !migrationConf.archiveHive) {
503 | val archivedTblName = tblName + ARCHIVED_SUFFIX
504 | val archivedQualifiedTblName = tblQualifiedName + ARCHIVED_SUFFIX
505 | if (!metacatService.tableExists(job.catalogName, job.dbName, archivedTblName)) {
506 | metacatService.unBlockTableWrites(job.catalogName, job.dbName, tblName)
507 | metacatService.renameTable(job.catalogName, job.dbName, tblName, archivedTblName)
508 | metacatService.appendTableTags(
509 | job.catalogName,
510 | job.dbName,
511 | archivedTblName,
512 | Set(HIVE_ARCHIVED_TAG, DO_NOT_RENAME_TAG, DO_NOT_DROP_TAG))
513 | } else {
514 | logger.warn(
515 | s"Table $archivedQualifiedTblName already exists. Skipping rename of $tblQualifiedName")
516 | }
517 | } else {
518 | spark.sql(s"DROP TABLE IF EXISTS $tblName")
519 | logger.info(s"Successfully dropped table: $tblName")
520 | }
521 | updateJobState(job, JobState.HiveDropped, JobState.HiveDropped)
522 | }
523 |
524 | /**
525 | * Get the table owners for a given job
526 | *
527 | * @param job
528 | * Job object containing information about the table and its location
529 | * @return
530 | * Set of table owners as a string
531 | */
532 | def getTableOwners(job: Job): Set[String] = {
533 | var tableOwners: Set[String] = Set()
534 | tableOwners = ndcService.getTableOwners(job.catalogName, job.dbName, job.tblName).toSet
535 | if (tableOwners == null || tableOwners.isEmpty) {
536 | val ndc_query =
537 | s"""
538 | |SELECT verifiedTechnicalContacts
539 | |FROM bdp.ndc_metadata
540 | |WHERE nameDetails['catalogName'] = '${job.catalogName}'
541 | |AND nameDetails['database'] = '${job.dbName}'
542 | |AND nameDetails['table'] = '${job.tblName}'
543 | |AND dateint = DATE_FORMAT(DATE_SUB(CURRENT_DATE(), 1), 'YYYYMMdd')
544 | |""".stripMargin
545 | var result = spark.sql(ndc_query)
546 | val firstResult = result.head(1)
547 | if (!firstResult.isEmpty) {
548 | tableOwners ++= firstResult.head.getAs[Seq[String]]("verifiedTechnicalContacts").toSet
549 | } else {
550 | val metacat_query =
551 | s"""
552 | |SELECT CONCAT(nf_json_extract_scalar(other_properties['current'], '$$.definitionMetadata.owner.userId'), '@netflix.com')
553 | |AS table_owner
554 | |FROM bdp_metacat_table_events
555 | |WHERE other_properties['name'] = '${job.catalogName}/${job.dbName}/${job.tblName}'
556 | |AND other_properties['name'] IS NOT NULL
557 | |AND other_properties['name'] != ''
558 | |AND other_properties['current'] IS NOT NULL
559 | |AND other_properties['current'] != ''
560 | |LIMIT 1
561 | |""".stripMargin
562 | result = spark.sql(metacat_query)
563 | val firstResult = result.head(1)
564 | if (!firstResult.isEmpty) {
565 | tableOwners += firstResult.head.getAs[String]("table_owner")
566 | }
567 | }
568 | }
569 | tableOwners
570 | }
571 |
572 | /**
573 | * Get the downstream users for a given job
574 | *
575 | * @param job
576 | * Job object containing information about the table and its location
577 | * @return
578 | * Set of downstream users as a string
579 | */
580 | def getDownstreamUsers(job: Job): Set[String] = {
581 | val query =
582 | s"""
583 | |SELECT COLLECT_SET(DISTINCT(user_id)) AS downstream_user
584 | |FROM lineage.lineage_daily_agg
585 | |WHERE source_name = '${job.catalogName}/${job.dbName}/${job.tblName}'
586 | |AND operation = 'read'
587 | |""".stripMargin
588 | val result = spark.sql(query)
589 | var downstreamUsers: Set[String] = Set()
590 | val firstResult = result.head(1)
591 | if (!firstResult.isEmpty) {
592 | downstreamUsers ++= firstResult.head.getAs[Seq[String]]("downstream_user").toSet
593 | }
594 | var returnDownstreamUsers: Set[String] = Set()
595 | for (user <- downstreamUsers) {
596 | if (user.contains("@netflix.com")) {
597 | returnDownstreamUsers += user
598 | } else {
599 | returnDownstreamUsers += s"$user@netflix.com"
600 | }
601 | }
602 | returnDownstreamUsers
603 | }
604 |
605 | /**
606 | * Get the storage format of a given table of the job
607 | *
608 | * @param job
609 | * Job object containing information about the table and its location
610 | * @return
611 | * Storage format of the table as a string (e.g. "parquet" or "csv/text")
612 | */
613 | def getTableStorageFormat(job: Job): String = {
614 | metacatService.getTableStorageFormat(job.catalogName, job.dbName, job.tblName)
615 | }
616 |
617 | /**
618 | * Get table data category
619 | *
620 | * @param job
621 | */
622 | def getDataCategory(job: Job): DataCategory = {
623 | ndcService.getDataCategory(job.catalogName, job.dbName, job.tblName)
624 | }
625 |
626 | /**
627 | * Get the fully qualified table name of a given job
628 | *
629 | * @param job
630 | * Job object containing information about the table and its location
631 | * @return
632 | * Fully qualified table name as a string (e.g. "catalogName.dbName.tblName")
633 | */
634 | def getFullyQualifiedTableName(job: Job): String = {
635 | s"${job.catalogName}.${job.dbName}.${job.tblName}"
636 | }
637 |
638 | /**
639 | * Get the snapshot id of a current snapshot of the given iceberg table
640 | *
641 | * @param icebergTableName
642 | * Name of the iceberg table as a string
643 | * @return
644 | * Snapshot id of the current snapshot
645 | */
646 | private def getCurrentSnapshotId(icebergTableName: String): Any = {
647 | spark.sql(s"REFRESH TABLE $icebergTableName")
648 | spark
649 | .sql(
650 | s"SELECT snapshot_id FROM $icebergTableName.snapshots ORDER BY committed_at DESC LIMIT 1")
651 | .first()
652 | .get(0)
653 | }
654 |
655 | /**
656 | * Check if the given table is an Hive table
657 | *
658 | * @param catalogName
659 | * Name of the catalog
660 | * @param dbName
661 | * Name of the database
662 | * @param tableName
663 | * Table name to check
664 | * @return
665 | * Boolean indicating if the table is an Hive table
666 | */
667 | def isHiveTable(catalogName: String, dbName: String, tableName: String): Boolean = {
668 | if (!metacatService.tableExists(catalogName, dbName, tableName)) {
669 | return false
670 | }
671 | val tableDto = metacatService.getTable(catalogName, dbName, tableName).get
672 | Option(tableDto.getMetadata.get("table_type")) match {
673 | case Some(tableType) => !tableType.toLowerCase.contains("iceberg")
674 | case None => Option(tableDto.getSerde.getUri).exists(_.contains("hive"))
675 | }
676 | }
677 |
678 | /**
679 | * Get the snapshot id of the watermark snapshot of the given hive table
680 | *
681 | * @param hiveTableName
682 | * Name of the hive table as a string
683 | * @return
684 | * Snapshot id of the watermark snapshot or -1 if it does not exist
685 | */
686 | private def getWatermarkSnapshotId(hiveTableName: String): Any = {
687 | spark.sql(s"REFRESH TABLE $hiveTableName")
688 | spark
689 | .sql(s"SHOW TBLPROPERTIES $hiveTableName('watermark_snapshot_id')")
690 | .first()
691 | .get(1) match {
692 | case s: String if s.startsWith("Table") => -1
693 | case v => v
694 | }
695 | }
696 |
697 | /**
698 | * Check if the hive and iceberg tables of a given job are in sync
699 | *
700 | * @param job
701 | * Job object containing information about the table and its location
702 | * @return
703 | * Boolean indicating if the tables are in sync
704 | */
705 | def hiveIcebergTablesInSync(job: Job): Boolean = {
706 | val tableName = getFullyQualifiedTableName(job)
707 | val hiveTableName = s"$tableName$HIVE_SUFFIX"
708 | val icebergTableName = tableName
709 | val currentSnapshotId = getCurrentSnapshotId(icebergTableName)
710 | val watermarkSnapshotId = getWatermarkSnapshotId(hiveTableName)
711 | watermarkSnapshotId.==(currentSnapshotId)
712 | }
713 |
714 | private def getTagsFromTableMetadata(tableDto: TableDto): Option[Set[String]] = {
715 | if (tableDto.getDefinitionMetadata != null) {
716 | val tagsNode = tableDto.getDefinitionMetadata.path("tags")
717 | if (tagsNode != null &&
718 | tagsNode.isArray &&
719 | tagsNode.size() > 0) {
720 | val tagsSet: mutable.HashSet[String] = mutable.HashSet()
721 | // Get the existing set of tags and append to them since
722 | // the tags API is a replace all tags API.
723 | for (tag <- tagsNode.asScala) {
724 | tagsSet.add(tag.asText().trim)
725 | }
726 | return Some(tagsSet.toSet)
727 | }
728 | }
729 | None
730 | }
731 |
732 | def tableHasTimestampColumn(tableDto: TableDto): Boolean = {
733 | val metacatTableSchema = tableDto.getFields
734 | if (metacatTableSchema != null &&
735 | metacatTableSchema.size() > 0) {
736 | return metacatTableSchema
737 | .iterator()
738 | .asScala
739 | .toStream
740 | .count(f => {
741 | !f.getName.equals(HIVE_TIMESTAMP_COLUMN_NAME) &&
742 | f.getSource_type != null &&
743 | f.getSource_type.toLowerCase.contains(HIVE_TIMESTAMP_COLUMN_NAME) &&
744 | !f.getSource_type.toLowerCase.contains(HIVE_TIMESTAMP_COLUMN_NAME + ":")
745 | }) > 0
746 | }
747 | false
748 | }
749 |
750 | /**
751 | * Get mock Job object
752 | *
753 | * @param catalog_name
754 | * catalog name of the table
755 | * @param db_name
756 | * database name of the table
757 | * @param table_name
758 | * name of the table
759 | * @return
760 | * Boolean indicating if the tables are in sync
761 | */
762 | def getMockJobObject(catalog_name: String, db_name: String, table_name: String): Job = {
763 | new Job(
764 | null,
765 | catalog_name,
766 | db_name,
767 | table_name,
768 | "",
769 | dataCategory = "",
770 | null,
771 | null,
772 | 0,
773 | 0,
774 | JobState.Undefined,
775 | JobState.Undefined,
776 | 0,
777 | 0,
778 | 0,
779 | 0,
780 | 0,
781 | 0,
782 | 0,
783 | "",
784 | null,
785 | null,
786 | 0,
787 | 0)
788 | }
789 |
790 | /**
791 | * Groups the tables in the given list of jobs by their owners.
792 | *
793 | * @param jobs
794 | * a list of Job objects to be grouped
795 | * @return
796 | * a map of owner names to sets of table names owned by each owner
797 | */
798 | def jobsGroupByOwners(jobs: List[Job]): Map[String, Set[String]] = {
799 | jobs
800 | .flatMap(job => job.tblOwners.map(owner => (owner, getFullyQualifiedTableName(job))))
801 | .groupBy(_._1)
802 | .mapValues(_.map(_._2).toSet)
803 | }
804 |
805 | /**
806 | * Creates a new mutable.HashMap with owner names from ownerToTablesMap as keys and Boolean as
807 | * values. The initial value for all the keys will be false.
808 | *
809 | * @param ownerToTablesMap
810 | * the map with owner names as keys and corresponding table names as values
811 | * @return
812 | * a mutable.HashMap with owner names as keys and false as initial values
813 | */
814 | def createBooleanMap(
815 | ownerToTablesMap: Map[String, Set[String]]): mutable.HashMap[String, Boolean] = {
816 | val booleanMap = mutable.HashMap.empty[String, Boolean]
817 | for (owner <- ownerToTablesMap.keys) {
818 | booleanMap(owner) = false
819 | }
820 | booleanMap
821 | }
822 |
823 | /**
824 | * Check if Hive table is empty or not
825 | *
826 | * @param job
827 | * Job object containing information about the table
828 | */
829 | def isHiveTableEmpty(job: Job): Boolean = {
830 | def fetchDataSize: Long =
831 | metacatService.getTableDataSize(job.catalogName, job.dbName, job.tblName)
832 |
833 | def isEmptyWithSpark: Boolean = {
834 | val result =
835 | spark.sql(s"SELECT 1 FROM ${job.catalogName}.${job.dbName}.${job.tblName} LIMIT 1")
836 | result.isEmpty || result.head.getAs[Int](0) == 0
837 | }
838 |
839 | if (isHiveTable(job.catalogName, job.dbName, job.tblName)) {
840 | fetchDataSize match {
841 | case 0L => true
842 | case -1L => isEmptyWithSpark
843 | case _ => false
844 | }
845 | } else {
846 | false
847 | }
848 | }
849 |
850 | /**
851 | * Return future day, date N days from now
852 | *
853 | * @param N
854 | * Number of days from now
855 | */
856 | def getDayNDaysFromNow(N: Int): String = {
857 | val today = LocalDate.now()
858 | val futureDate = today.plusDays(N)
859 | val formatter = DateTimeFormatter.ofPattern("EEEE, MMMM d, yyyy")
860 | futureDate.format(formatter)
861 | }
862 |
863 | def consistentHash(value: Int, numBuckets: Int): Int = {
864 | val md = MessageDigest.getInstance("MD5")
865 | val hashBytes = md.digest(value.toString.getBytes("UTF-8"))
866 | val hashString = hashBytes.map("%02x".format(_)).mkString
867 | val bucket = BigInt(hashString, 16) % numBuckets
868 | bucket.toInt + 1
869 | }
870 |
871 | /**
872 | * Revert a migration finalized table back to Hive table format using shadow tool.
873 | *
874 | * @param table
875 | * The table to be reverted
876 | * @return
877 | * Return revert status: true or false
878 | */
879 | def revertHelper(job: Job): Unit = {
880 | val (catalogName, dbName, tableName) = (job.catalogName, job.dbName, job.tblName)
881 | var archivedHiveTableName = tableName + HIVE_SUFFIX + ARCHIVED_SUFFIX
882 | if (metacatService.tableExists(catalogName, dbName, archivedHiveTableName)) {
883 | metacatService.removeTableTags(
884 | catalogName,
885 | dbName,
886 | archivedHiveTableName,
887 | Set(DO_NOT_DROP_TAG, DO_NOT_RENAME_TAG))
888 | metacatService.renameTable(catalogName, dbName, archivedHiveTableName, tableName + HIVE_SUFFIX)
889 | }
890 | }
891 |
892 | /**
893 | * Revert a migration finalized table back to Hive table format.
894 | *
895 | * @param table
896 | * The table to be reverted
897 | * @return
898 | * Return revert status: true or false
899 | */
900 | def revertFinalizedTable(table: (String, String, String)): Boolean = {
901 | val (catalogName, dbName, tableName) = table
902 | val archivedHiveTableName = tableName + HIVE_SUFFIX + ARCHIVED_SUFFIX
903 | val archivedIcebergTableName = tableName + ICEBERG_SUFFIX + ARCHIVED_SUFFIX
904 | logger.info(
905 | s"Starting revert from $catalogName.$dbName.$archivedHiveTableName to " +
906 | s"$catalogName.$dbName.$tableName")
907 | val isIceberg = !isHiveTable(catalogName, dbName, tableName)
908 | if (isIceberg && metacatService.tableExists(catalogName, dbName, archivedHiveTableName)) {
909 | val preExistingTags = metacatService.getTableTags(catalogName, dbName, tableName)
910 | metacatService.removeTableTags(
911 | catalogName,
912 | dbName,
913 | archivedHiveTableName,
914 | Set(DO_NOT_DROP_TAG, DO_NOT_RENAME_TAG))
915 | if (metacatService.tableExists(catalogName, dbName, archivedIcebergTableName)) {
916 | metacatService.forceDropTable(catalogName, dbName, archivedIcebergTableName)
917 | }
918 | metacatService.renameTable(catalogName, dbName, tableName, archivedIcebergTableName)
919 | metacatService.renameTable(catalogName, dbName, archivedHiveTableName, tableName)
920 | metacatService.appendTableTags(catalogName, dbName, tableName, preExistingTags.toSet)
921 | logger.info(s"Successfully reverted table $catalogName.$dbName.$tableName")
922 | true
923 | } else {
924 | logger.warn(s"Revert failed for table $catalogName.$dbName.$tableName. ")
925 | if (!isIceberg) {
926 | logger.warn("Table is already in Hive format.")
927 | } else {
928 | logger.warn(
929 | "Table migration has not yet finalized." +
930 | " Wait for table migration to be finalized and then retry.")
931 | }
932 | false
933 | }
934 | }
935 | }
936 |
--------------------------------------------------------------------------------