You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2019/10/08 08:44:56 UTC

[GitHub] [spark] turboFei commented on a change in pull request #25863: [SPARK-28945][SPARK-29037][CORE][SQL] Fix the issue that spark gives duplicate result and support concurrent file source write operations write to different partitions in the same table.

turboFei commented on a change in pull request #25863: [SPARK-28945][SPARK-29037][CORE][SQL] Fix the issue that spark gives duplicate result and support concurrent file source write operations write to different partitions in the same table.
URL: https://github.com/apache/spark/pull/25863#discussion_r332397399
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala
 ##########
 @@ -266,4 +304,131 @@ case class InsertIntoHadoopFsRelationCommand(
       }
     }.toMap
   }
+
+  /**
+   * Check current committer whether supports several InsertIntoHadoopFsRelation operations write
+   * to different partitions in a same table concurrently. If supports, then detect the conflict
+   * whether there are several operations write to same partition in the same table or write to
+   * a non-partitioned table.
+   */
+  private def detectConflict(
+      commitProtocol: FileCommitProtocol,
+      fs: FileSystem,
+      path: Path,
+      staticPartitionKVs: Seq[(String, String)],
+      appId: String,
+      jobId: String): Unit = {
+    import HadoopMapReduceCommitProtocol._
+
+    val supportConcurrent = commitProtocol.isInstanceOf[HadoopMapReduceCommitProtocol] &&
+      commitProtocol.asInstanceOf[HadoopMapReduceCommitProtocol].supportConcurrent
+    if (supportConcurrent && fs.exists(outputPath) && fs.isDirectory(outputPath)) {
+
+      val insertStagingPath = ".spark-staging-" + staticPartitionKVs.size
+      val checkedPath = new Path(outputPath, Array(insertStagingPath,
+        getEscapedStaticPartitionPath(staticPartitionKVs)).mkString(File.separator))
+      insertStagingDir = new Path(outputPath, insertStagingPath)
+
+      if (fs.exists(checkedPath)) {
+        throwConflictException(fs, insertStagingDir, staticPartitionKVs.size,
+          staticPartitionKVs)
+      }
+      stagingOutputDir = new Path(outputPath, Array(insertStagingPath,
+        getEscapedStaticPartitionPath(staticPartitionKVs), appId, jobId).mkString(File.separator))
+      fs.mkdirs(stagingOutputDir)
+
+      for (i <- 0 to partitionColumns.size) {
+        if (i != staticPartitionKVs.size) {
+          val stagingDir = new Path(path, ".spark-staging-" + i)
+          if (fs.exists(stagingDir)) {
+            val subPath = getEscapedStaticPartitionPath(
+              staticPartitionKVs.slice(0, i))
+            val checkedPath = if (!subPath.isEmpty) {
+              new Path(stagingDir, subPath)
+            } else {
+              stagingDir
+            }
+            if (fs.exists(checkedPath)) {
+              throwConflictException(fs, stagingDir, i, staticPartitionKVs)
+            }
+          }
+        }
+      }
+    }
+  }
+
+  private def throwConflictException(
+      fs: FileSystem,
+      stagingDir: Path,
+      depth: Int,
+      staticPartitionKVs: Seq[(String, String)]): Unit = {
+    val conflictedPaths = ListBuffer[Path]()
+    val currentPath = if (depth == staticPartitionKVs.size || staticPartitionKVs.size == 0) {
+      stagingDir
+    } else {
+      new Path(stagingDir, HadoopMapReduceCommitProtocol.getEscapedStaticPartitionPath(
+        staticPartitionKVs.slice(0, staticPartitionKVs.size - depth)))
+    }
+
+    findConflictedStagingOutputPaths(fs, currentPath, depth, conflictedPaths)
+
+    val pathsInfo = conflictedPaths.toList
+      .map { path =>
+        val absolutePath = path.toUri.getRawPath
+        val relativePath = absolutePath.substring(absolutePath.lastIndexOf(stagingDir.getName))
+        var appId: Option[String] = None
+        var modificationTime: Date = null
+        try {
+          val files = fs.listStatus(path)
+          if (files.size > 0) {
+            appId = Some(files.apply(0).getPath.getName)
+          }
+          modificationTime = new Date(fs.getFileStatus(path).getModificationTime)
+        } catch {
+          case e: Exception => logWarning("Exception occurred", e)
+        }
+        (relativePath, appId.getOrElse("Not Found"), modificationTime)
+      }
+
+    throw new InsertFileSourceConflictException(
+      s"""
+         | Conflict is detected, some other conflicted output path(s) under tablePath:
+         | ($outputPath) existed.
+         | Relative path, appId and last modification time information is shown as below:
+         | ${pathsInfo}.
+         | There may be two possibilities:
+         | 1. Another InsertDataSource operation is executing, you need wait for it to
+         |    complete.
+         | 2. This dir is belong to a killed application and not be cleaned up gracefully.
+         |
+         | Please check the last modification time and use given appId to judge whether
+         | relative application is running now. If not, you should delete responding path
+         | without recursive manually.
+         |""".stripMargin)
+  }
+
+  /**
+   * Find relative staging output paths, which is conflicted with current
+   * InsertIntoHadoopFsRelation operation.
+   */
+  private def findConflictedStagingOutputPaths(
+      fs: FileSystem,
+      path: Path,
+      depth: Int,
+      paths: ListBuffer[Path]): Unit = {
+    try {
+      if (fs.exists(path)) {
+        if (depth == 0) {
+          paths += path
+        } else {
+          for (file <- fs.listStatus(path)) {
 
 Review comment:
   I checked whether this path is existed firstly and caught all exceptions in this code block.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

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