Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11925#discussion_r57367862
  
    --- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala
 ---
    @@ -196,4 +194,107 @@ class HDFSMetadataLog[T: ClassTag](sqlContext: 
SQLContext, path: String)
         }
         None
       }
    +
    +  private def createFileManager(): FileManager = {
    +    val hadoopConf = sqlContext.sparkContext.hadoopConfiguration
    +    try {
    +      new FileContextManager(metadataPath, hadoopConf)
    +    } catch {
    +      case e: UnsupportedFileSystemException =>
    +        logWarning("Could not use FileContext API for managing metadata 
log file. The log may be" +
    +          "inconsistent under failures.", e)
    +        new FileSystemManager(metadataPath, hadoopConf)
    +    }
    +  }
    +}
    +
    +object HDFSMetadataLog {
    +
    +  /** A simple trait to abstract out the file management operations needed 
by HDFSMetadataLog */
    +  trait FileManager {
    +    def list(path: Path, filter: PathFilter): Array[FileStatus]
    +    def mkdirs(path: Path): Unit
    +    def exists(path: Path): Boolean
    +    def open(path: Path): FSDataInputStream
    +    def create(path: Path): FSDataOutputStream
    +    def rename(srcPath: Path, destPath: Path): Unit
    +    def deleteOnExit(path: Path): Unit
    +  }
    +
    +  /** Implementation of FileManager using newer FileContext API */
    +  class FileContextManager(path: Path, hadoopConf: Configuration) extends 
FileManager {
    +    private val fc = if (path.toUri.getScheme == null) {
    +      FileContext.getFileContext(hadoopConf)
    +    } else {
    +      FileContext.getFileContext(path.toUri, hadoopConf)
    +    }
    +
    +    override def list(path: Path, filter: PathFilter): Array[FileStatus] = 
{
    +      fc.util.listStatus(path, filter)
    +    }
    +
    +    override def rename(srcPath: Path, destPath: Path): Unit = {
    +      fc.rename(srcPath, destPath)
    +    }
    +
    +    override def mkdirs(path: Path): Unit = {
    +      fc.mkdir(path, FsPermission.getDirDefault, true)
    +    }
    +
    +    override def open(path: Path): FSDataInputStream = {
    +      fc.open(path)
    +    }
    +
    +    override def create(path: Path): FSDataOutputStream = {
    +      fc.create(path, EnumSet.of(CreateFlag.CREATE))
    +    }
    +
    +    override def exists(path: Path): Boolean = {
    +      fc.util().exists(path)
    +    }
    +
    +    override def deleteOnExit(path: Path): Unit = {
    +      fc.deleteOnExit(path)
    +    }
    +  }
    +
    +  /** Implementation of FileManager using older FileSystem API */
    +  class FileSystemManager(path: Path, hadoopConf: Configuration) extends 
FileManager {
    +    private val fs = if (path.toUri.getScheme == null) {
    +      FileSystem.get(hadoopConf)
    +    } else {
    +      FileSystem.get(path.toUri, hadoopConf)
    +    }
    +
    +    override def list(path: Path, filter: PathFilter): Array[FileStatus] = 
{
    +      fs.listStatus(path, filter)
    +    }
    +
    +    override def rename(srcPath: Path, destPath: Path): Unit = {
    +      if (fs.exists(destPath)) {
    +        throw new FileAlreadyExistsException(s"File already exists: 
$destPath")
    +      }
    +      fs.rename(srcPath, destPath)
    --- End diff --
    
    Unlike `fc.rename`, `fs.rename` will return `false` if it fails. We should 
throw an exception for that.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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

Reply via email to