rdblue commented on a change in pull request #374: Migrate spark table to 
iceberg table
URL: https://github.com/apache/incubator-iceberg/pull/374#discussion_r314399700
 
 

 ##########
 File path: spark/src/main/scala/org/apache/iceberg/spark/SparkTableUtil.scala
 ##########
 @@ -297,5 +301,88 @@ object SparkTableUtil {
       )
     }
   }
+
+  def buildManifest(table: Table,
+                    sparkDataFiles: Seq[SparkDataFile],
+                    partitionSpec: PartitionSpec): ManifestFile = {
+    val outputFile = table.io
+      .newOutputFile(FileFormat.AVRO.addExtension("/tmp/" + 
UUID.randomUUID.toString))
+    val writer = ManifestWriter.write(partitionSpec, outputFile)
+    try {
+      for (file <- sparkDataFiles) {
+        writer.add(file.toDataFile(partitionSpec))
+      }
+    } finally {
+      writer.close()
+    }
+
+    writer.toManifestFile
+  }
+
+  def partitionToMap(partition: String): Map[String, String] = {
+    val map = new mutable.HashMap[String, String]()
+    val list = partition.split("/")
+    list.foreach { str =>
+      val kv = str.split("=")
+      map.put(kv(0), kv(1))
+    }
+
+    map.toMap
+  }
+
+  /**
+   * Migrate a spark table to a iceberg table.
+   *
+   * The migration uses the spark session to get table metadata. It assumes no
+   * operation is going on original table and target table and thus is not
+   * thread-safe.
+   *
+   * @param dbName the database name of the table to be migrated
+   * @param tableName the table to be migrated
+   * @param table the target table to migrate in
+   *
+   * @return table the target table
+   */
+  def migrateSparkTable(dbName: String, tableName: String, table: Table): 
Table = {
+    val sparkSession = SparkSession.builder().getOrCreate()
+
+    if (!sparkSession.catalog.tableExists(dbName, tableName)) {
+      throw new NoSuchTableException(s"Table $dbName.$tableName does not 
exist")
+    }
+
+    val tableMetadata = sparkSession.sessionState.catalog.
+      getTableMetadata(new TableIdentifier(tableName, Some(dbName)))
+
+    val format = tableMetadata.provider.getOrElse("none")
+    if (format != "avro" && format != "parquet" && format != "orc") {
+      throw new UnsupportedOperationException(s"Unsupported format: $format")
+    }
+
+    val location = tableMetadata.location.toString
+    val partitionSpec = SparkSchemaUtil.specForTable(sparkSession, 
s"$dbName.$tableName")
+
+    val fastAppender = table.newFastAppend()
+
+    val partitions = sparkSession.sessionState.catalog.externalCatalog
+      .listPartitionNames(dbName, tableName)
 
 Review comment:
   This should use `partitionDF` and call `listPartition` in parallel for large 
tables. See the example notebook: 
https://github.com/apache/incubator-iceberg/blob/master/examples/Convert%20table%20to%20Iceberg.ipynb

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org

Reply via email to