Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19269#discussion_r145018796
  
    --- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/sources/v2/SimpleWritableDataSource.scala
 ---
    @@ -0,0 +1,252 @@
    +/*
    + * 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, IOException}
    +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, FSDataInputStream, 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).filterNot { status =>
    +          val name = status.getPath.getName
    +          name.startsWith("_") || name.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.
    +    protected 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)) {
    +          val dest = new Path(finalPath, file.getName)
    +          if(!fs.rename(file, dest)) {
    +            throw new IOException(s"failed to rename($file, $dest)")
    +          }
    +        }
    +      } 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 Writer(path, conf) with SupportsWriteInternalRow {
    +
    +    override def createWriterFactory(): DataWriterFactory[Row] = {
    +      throw new IllegalArgumentException("not expected!")
    +    }
    +
    +    override def createInternalRowWriterFactory(): 
DataWriterFactory[InternalRow] = {
    +      new InternalRowCSVDataWriterFactory(path, jobId, new 
SerializableConfiguration(conf))
    +    }
    +  }
    +
    +  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 (mode == SaveMode.ErrorIfExists) {
    +      if (fs.exists(path)) {
    +        throw new RuntimeException("data already exists.")
    +      }
    +    }
    +    if (mode == SaveMode.Ignore) {
    +      if (fs.exists(path)) {
    +        return Optional.empty()
    +      }
    +    }
    +    if (mode == SaveMode.Overwrite) {
    +      if (fs.exists(path)) {
    +        fs.delete(path, true)
    --- End diff --
    
    cc @steveloughran , overwrite needs this check, because we need to delete 
the root dir here. The writers always create file with unique name, so we can't 
delete old files while overwriting.


---

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

Reply via email to