forked from woltapp/spark-osm-datasource
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Migrate to Spark 3 DataSource v2 interfaces
Fixes woltapp#3
- Loading branch information
Showing
6 changed files
with
99 additions
and
70 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
11 changes: 7 additions & 4 deletions
11
src/main/scala/com/wolt/osm/spark/OsmSource/OsmPartition.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -1,9 +1,12 @@ | ||
package com.wolt.osm.spark.OsmSource | ||
|
||
import org.apache.spark.sql.catalyst.InternalRow | ||
import org.apache.spark.sql.sources.v2.reader.{InputPartition, InputPartitionReader} | ||
import org.apache.spark.sql.connector.read.InputPartition | ||
import org.apache.spark.sql.types.StructType | ||
|
||
class OsmPartition(input: String, hadoop: SerializableHadoopConfigration, schema: StructType, threads: Int, partitionsNo: Int, partition: Int, useLocal: Boolean) extends InputPartition[InternalRow] { | ||
override def createPartitionReader(): InputPartitionReader[InternalRow] = new OsmPartitionReader(input, hadoop, schema, threads, partitionsNo, partition, useLocal) | ||
class OsmPartition(val input: String, val hadoop: SerializableHadoopConfigration, val schema: StructType, val threads: Int, val partitionsNo: Int, val partition: Int, val useLocal: Boolean) extends InputPartition | ||
|
||
object OsmPartition { | ||
def unapply(inputPartition: OsmPartition): Option[(String, SerializableHadoopConfigration, StructType, Int, Int, Int, Boolean)] = { | ||
Some((inputPartition.input, inputPartition.hadoop, inputPartition.schema, inputPartition.threads, inputPartition.partitionsNo, inputPartition.partition, inputPartition.useLocal)) | ||
} | ||
} |
15 changes: 7 additions & 8 deletions
15
src/main/scala/com/wolt/osm/spark/OsmSource/OsmPartitionReader.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,50 @@ | ||
package com.wolt.osm.spark.OsmSource | ||
|
||
import org.apache.hadoop.fs.Path | ||
import org.apache.spark.SparkFiles | ||
import org.apache.spark.sql.SparkSession | ||
|
||
import org.apache.spark.sql.connector.read.{Batch, InputPartition, PartitionReaderFactory, Scan} | ||
|
||
import java.io.File | ||
import org.apache.spark.sql.types.StructType | ||
import org.apache.spark.sql.util.CaseInsensitiveStringMap | ||
|
||
import scala.util.Try | ||
|
||
class OsmScan(val options: CaseInsensitiveStringMap) extends Scan with Batch { | ||
private val path = options.get("path") | ||
private val useLocal = Option(options.get("useLocalFile")).getOrElse("").equalsIgnoreCase("true") | ||
|
||
private val spark = SparkSession.active | ||
private val hadoop = spark.sessionState.newHadoopConf() | ||
private val hadoopConfigration = new SerializableHadoopConfigration(hadoop) | ||
|
||
if (useLocal) { | ||
if (!new File(SparkFiles.get(path)).canRead) { | ||
throw new RuntimeException(s"Input unavailable: $path") | ||
} | ||
} else { | ||
val source = new Path(path) | ||
val fs = source.getFileSystem(hadoop) | ||
if (!fs.exists(source)) { | ||
throw new RuntimeException(s"Input unavailable: $path") | ||
} | ||
} | ||
|
||
private val partitions = Option(options.get("partitions")).getOrElse("1") | ||
private val threads = Option(options.get("threads")).getOrElse("1") | ||
|
||
override def readSchema(): StructType = OsmSource.schema | ||
|
||
override def planInputPartitions(): Array[InputPartition] = { | ||
val partitionsNo = Try(partitions.toInt).getOrElse(1) | ||
val threadsNo = Try(threads.toInt).getOrElse(1) | ||
val shiftedPartitions = partitionsNo - 1 | ||
(0 to shiftedPartitions).map(p => new OsmPartition(path, hadoopConfigration, readSchema(), threadsNo, partitionsNo, p, useLocal)).toArray | ||
} | ||
|
||
override def toBatch: Batch = this | ||
|
||
override def createReaderFactory(): PartitionReaderFactory = new OsmPartitionReaderFactory() | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
26 changes: 0 additions & 26 deletions
26
src/main/scala/com/wolt/osm/spark/OsmSource/OsmSourceReader.scala
This file was deleted.
Oops, something went wrong.