Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/19269#discussion_r144821389
--- Diff:
sql/core/src/test/scala/org/apache/spark/sql/sources/v2/SimpleWritableDataSource.scala
---
@@ -0,0 +1,254 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.sources.v2
+
+import java.io.{BufferedReader, InputStreamReader}
+import java.text.SimpleDateFormat
+import java.util.{Collections, Date, List => JList, Locale, Optional, UUID}
+
+import scala.collection.JavaConverters._
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{FileSystem, Path}
+
+import org.apache.spark.SparkContext
+import org.apache.spark.sql.{Row, SaveMode}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.sources.v2.reader.{DataReader,
DataSourceV2Reader, ReadTask}
+import org.apache.spark.sql.sources.v2.writer._
+import org.apache.spark.sql.types.{DataType, StructType}
+import org.apache.spark.util.SerializableConfiguration
+
+/**
+ * A HDFS based transactional writable data source.
+ * Each task writes data to
`target/_temporary/jobId/$jobId-$partitionId-$attemptNumber`.
+ * Each job moves files from `target/_temporary/jobId/` to `target`.
+ */
+class SimpleWritableDataSource extends DataSourceV2 with ReadSupport with
WriteSupport {
+
+ private val schema = new StructType().add("i", "long").add("j", "long")
+
+ class Reader(path: String, conf: Configuration) extends
DataSourceV2Reader {
+ override def readSchema(): StructType = schema
+
+ override def createReadTasks(): JList[ReadTask[Row]] = {
+ val dataPath = new Path(path)
+ val fs = dataPath.getFileSystem(conf)
+ if (fs.exists(dataPath)) {
+
fs.listStatus(dataPath).filter(!_.getPath.getName.startsWith("_")).map { f =>
+ val serializableConf = new SerializableConfiguration(conf)
+ new SimpleCSVReadTask(f.getPath.toUri.toString,
serializableConf): ReadTask[Row]
+ }.toList.asJava
+ } else {
+ Collections.emptyList()
+ }
+ }
+ }
+
+ class Writer(path: String, conf: Configuration) extends
DataSourceV2Writer {
+ // We can't get the real spark job id here, so we use a timestamp and
random UUID to simulate
+ // a unique job id.
+ private val jobId = new SimpleDateFormat("yyyyMMddHHmmss",
Locale.US).format(new Date()) +
+ "-" + UUID.randomUUID()
+
+ override def createWriterFactory(): DataWriterFactory[Row] = {
+ new SimpleCSVDataWriterFactory(path, jobId, new
SerializableConfiguration(conf))
+ }
+
+ override def commit(messages: Array[WriterCommitMessage]): Unit = {
+ val finalPath = new Path(path)
+ val jobPath = new Path(new Path(finalPath, "_temporary"), jobId)
+ val fs = jobPath.getFileSystem(conf)
+ try {
+ for (file <- fs.listStatus(jobPath).map(_.getPath)) {
+ fs.rename(file, new Path(finalPath, file.getName))
+ }
+ } finally {
+ fs.delete(jobPath, true)
+ }
+ }
+
+ override def abort(messages: Array[WriterCommitMessage]): Unit = {
+ val jobPath = new Path(new Path(path, "_temporary"), jobId)
+ val fs = jobPath.getFileSystem(conf)
+ fs.delete(jobPath, true)
+ }
+ }
+
+ class InternalRowWriter(path: String, conf: Configuration)
+ extends DataSourceV2Writer with SupportsWriteInternalRow {
+
+ private val jobId = new SimpleDateFormat("yyyyMMddHHmmss",
Locale.US).format(new Date())
+
+ override def createInternalRowWriterFactory():
DataWriterFactory[InternalRow] = {
+ new InternalRowCSVDataWriterFactory(path, jobId, new
SerializableConfiguration(conf))
+ }
+
+ override def commit(messages: Array[WriterCommitMessage]): Unit = {
+ val finalPath = new Path(path)
+ val jobPath = new Path(new Path(finalPath, "_temporary"), jobId)
+ val fs = jobPath.getFileSystem(conf)
+ try {
+ for (file <- fs.listStatus(jobPath).map(_.getPath)) {
+ fs.rename(file, new Path(finalPath, file.getName))
+ }
+ } finally {
+ fs.delete(jobPath, true)
+ }
+ }
+
+ override def abort(messages: Array[WriterCommitMessage]): Unit = {
+ val jobPath = new Path(new Path(path, "_temporary"), jobId)
+ val fs = jobPath.getFileSystem(conf)
+ fs.delete(jobPath, true)
+ }
+ }
+
+ override def createReader(options: DataSourceV2Options):
DataSourceV2Reader = {
+ val path = new Path(options.get("path").get())
+ val conf = SparkContext.getActive.get.hadoopConfiguration
+ new Reader(path.toUri.toString, conf)
+ }
+
+ override def createWriter(
+ schema: StructType,
+ mode: SaveMode,
+ options: DataSourceV2Options): Optional[DataSourceV2Writer] = {
+ assert(DataType.equalsStructurally(schema.asNullable,
this.schema.asNullable))
+
assert(!SparkContext.getActive.get.conf.getBoolean("spark.speculation", false))
+
+ val path = new Path(options.get("path").get())
+ val internal = options.get("internal").isPresent
+ val conf = SparkContext.getActive.get.hadoopConfiguration
+ val fs = path.getFileSystem(conf)
+ if (fs.exists(path)) {
+ if (mode == SaveMode.ErrorIfExists) {
+ throw new RuntimeException("data already exists.")
+ }
+
+ if (mode == SaveMode.Ignore) {
+ Optional.empty()
+ } else if (mode == SaveMode.Overwrite) {
+ fs.delete(path, true)
+ Optional.of(createWriter(path, conf, internal))
+ } else {
+ assert(mode == SaveMode.Append)
+ Optional.of(createWriter(path, conf, internal))
+ }
+ } else {
+ Optional.of(createWriter(path, conf, internal))
+ }
+ }
+
+ private def createWriter(
+ path: Path, conf: Configuration, internal: Boolean):
DataSourceV2Writer = {
+ val pathStr = path.toUri.toString
+ if (internal) new InternalRowWriter(pathStr, conf) else new
Writer(pathStr, conf)
+ }
+}
+
+class SimpleCSVReadTask(path: String, conf: SerializableConfiguration)
+ extends ReadTask[Row] with DataReader[Row] {
+
+ @transient private var lines: Iterator[String] = _
+ @transient private var currentLine: String = _
+
+ override def createReader(): DataReader[Row] = {
+ val filePath = new Path(path)
+ val fs = filePath.getFileSystem(conf.value)
+ val in = fs.open(filePath)
+ lines = new BufferedReader(new InputStreamReader(in))
+ .lines().iterator().asScala.toArray.toIterator
+ in.close()
+ this
+ }
+
+ override def next(): Boolean = {
+ if (lines.hasNext) {
+ currentLine = lines.next()
+ true
+ } else {
+ false
+ }
+ }
+
+ override def get(): Row = Row(currentLine.split(",").map(_.trim.toLong):
_*)
+
+ override def close(): Unit = {}
+}
+
+class SimpleCSVDataWriterFactory(path: String, jobId: String, conf:
SerializableConfiguration)
+ extends DataWriterFactory[Row] {
+
+ override def createWriter(stageId: Int, partitionId: Int, attemptNumber:
Int): DataWriter[Row] = {
+ val jobPath = new Path(new Path(path, "_temporary"), jobId)
+ val filePath = new Path(jobPath, s"$jobId-$partitionId-$attemptNumber")
+ val fs = filePath.getFileSystem(conf.value)
+ new SimpleCSVDataWriter(fs, filePath)
+ }
+}
+
+class SimpleCSVDataWriter(fs: FileSystem, file: Path) extends
DataWriter[Row] {
+
+ private val out = fs.create(file)
+
+ override def write(record: Row): Unit = {
+ out.writeBytes(s"${record.getLong(0)},${record.getLong(1)}\n")
+ }
+
+ override def commit(): WriterCommitMessage = {
+ out.close()
+ null
+ }
+
+ override def abort(): Unit = {
+ out.close()
+ fs.delete(file, false)
+ }
+}
+
+class InternalRowCSVDataWriterFactory(path: String, jobId: String, conf:
SerializableConfiguration)
+ extends DataWriterFactory[InternalRow] {
+
+ override def createWriter(
+ stageId: Int, partitionId: Int, attemptNumber: Int):
DataWriter[InternalRow] = {
+ val jobPath = new Path(new Path(path, "_temporary"), jobId)
+ val filePath = new Path(jobPath, s"$jobId-$partitionId-$attemptNumber")
+ val fs = filePath.getFileSystem(conf.value)
+ new InternalRowCSVDataWriter(fs, filePath)
+ }
+}
+
+class InternalRowCSVDataWriter(fs: FileSystem, file: Path) extends
DataWriter[InternalRow] {
+
+ private val out = fs.create(file)
+
+ override def write(record: InternalRow): Unit = {
+ out.writeBytes(s"${record.getLong(0)},${record.getLong(1)}\n")
+ }
+
+ override def commit(): WriterCommitMessage = {
+ out.close()
+ null
+ }
+
+ override def abort(): Unit = {
+ out.close()
--- End diff --
maybe wrap with try/finally, so the delete still gets called
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]