├── 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 | --------------------------------------------------------------------------------