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_r318680962
##########
File path: spark/src/main/scala/org/apache/iceberg/spark/SparkTableUtil.scala
##########
@@ -297,5 +306,110 @@ object SparkTableUtil {
)
}
}
+
+ private def buildManifest(conf: SerializableConfiguration,
+ sparkDataFiles: Seq[SparkDataFile],
+ partitionSpec: PartitionSpec,
+ basePath: String): Iterator[Manifest] = {
+ if (sparkDataFiles.isEmpty) {
+ Seq.empty.iterator
+ }
+
+ val io = new HadoopFileIO(conf.get())
+ val ctx = TaskContext.get()
+ val location = new Path(basePath,
+ s"stage-${ctx.stageId()}-task-${ctx.taskAttemptId()}-manifest")
+ val outputFile =
io.newOutputFile(FileFormat.AVRO.addExtension(location.toString))
+ val writer = ManifestWriter.write(partitionSpec, outputFile)
+ try {
+ sparkDataFiles.foreach { file =>
+ writer.add(file.toDataFile(partitionSpec))
+ }
+ } finally {
+ writer.close()
+ }
+
+ val manifestFile = writer.toManifestFile
+ Seq(Manifest(manifestFile.path, manifestFile.length,
manifestFile.partitionSpecId)).iterator
+ }
+
+ private case class Manifest(location: String, fileLength: Long, specId: Int)
{
+ def toManifestFile: ManifestFile = new ManifestFile {
+ override def path: String = location
+
+ override def length: Long = fileLength
+
+ override def partitionSpecId: Int = specId
+
+ override def snapshotId: java.lang.Long = null
+
+ override def addedFilesCount: Integer = null
+
+ override def existingFilesCount: Integer = null
+
+ override def deletedFilesCount: Integer = null
+
+ override def partitions:
java.util.List[ManifestFile.PartitionFieldSummary] = null
+
+ override def copy: ManifestFile = this
+ }
+ }
+
+ /**
+ * Import a spark table to a iceberg table.
+ *
+ * The import 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 source the database name of the table to be import
+ * @param location the location used to store table metadata
+ * @param numOfManifest the expected number of manifest file to be created
+ * @param stagingDir the staging directory to store temporary manifest file
+ *
+ * @return table the imported table
+ */
+ def importSparkTable(
+ source: TableIdentifier,
+ location: String,
+ numOfManifest: Int,
+ stagingDir: String): Table = {
+ val sparkSession = SparkSession.builder().getOrCreate()
+ import sparkSession.sqlContext.implicits._
+
+ val dbName = source.database.getOrElse("default")
+ val tableName = source.table
+
+ if (!sparkSession.catalog.tableExists(dbName, tableName)) {
+ throw new NoSuchTableException(s"Table $dbName.$tableName does not
exist")
+ }
+
+ val partitionSpec = SparkSchemaUtil.specForTable(sparkSession,
s"$dbName.$tableName")
+ val conf = sparkSession.sparkContext.hadoopConfiguration
+ val serializableConfiguration = new SerializableConfiguration(conf)
+ val tables = new HadoopTables(conf)
+ val schema = SparkSchemaUtil.schemaForTable(sparkSession,
s"$dbName.$tableName")
+ val table = tables.create(schema, partitionSpec, ImmutableMap.of(),
location)
+ val appender = table.newAppend()
+
+ if (partitionSpec == PartitionSpec.unpartitioned) {
+ val catalogTable = Hive.getTable(sparkSession, s"$dbName.$tableName")
+ val files = listPartition(Map.empty[String, String],
catalogTable.location.toString,
+ catalogTable.storage.serde.getOrElse("none"))
+ files.foreach{f =>
appender.appendFile(f.toDataFile(PartitionSpec.unpartitioned))}
+ } else {
+ val partitions = partitionDF(sparkSession, s"$dbName.$tableName")
+ val manifests = partitions.flatMap { row =>
+ listPartition(row.getMap[String, String](0).toMap, row.getString(1),
row.getString(2))
+ }.repartition(numOfManifest).mapPartitions {
Review comment:
I think we need to change a couple of things:
1. Use `spark.sqlContext.conf.numShufflePartitions` instead of passing in a
partition size to repartition. This parallelism setting is already something
users can configure.
2. Add a sort by the file path to cluster data by partition into manifests.
The converted table will have better performance.
I think it should look like this:
```scala
val manifests = partitions.flatMap { row =>
listPartition(row.getMap[String, String](0).toMap, row.getString(1),
row.getString(2))
}.repartition(spark.sqlContext.conf.numShufflePartitions)
.orderBy($"path")
.mapPartitions {
...
}.collect().map(_.toManifestFile)
```
The repartition is still needed to ensure that the partition listing isn't
done twice when calculating the range partitions for the sort.
----------------------------------------------------------------
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:
[email protected]
With regards,
Apache Git Services
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]