@@ -22,7 +22,6 @@ import java.net.URI
2222import java .text .SimpleDateFormat
2323import java .util .{Date , Locale , Random }
2424
25- import org .apache .hadoop .conf .Configuration
2625import org .apache .hadoop .fs .{FileSystem , Path }
2726import org .apache .hadoop .hive .common .FileUtils
2827import org .apache .hadoop .hive .ql .exec .TaskRunner
@@ -86,14 +85,15 @@ case class InsertIntoHiveTable(
8685
8786 val hadoopConf = sessionState.newHadoopConf()
8887 val stagingDir = hadoopConf.get(" hive.exec.stagingdir" , " .hive-staging" )
88+ val scratchDir = hadoopConf.get(" hive.exec.scratchdir" , " /tmp/hive" )
8989
9090 private def executionId : String = {
9191 val rand : Random = new Random
9292 val format = new SimpleDateFormat (" yyyy-MM-dd_HH-mm-ss_SSS" , Locale .US )
9393 " hive_" + format.format(new Date ) + " _" + Math .abs(rand.nextLong)
9494 }
9595
96- private def getStagingDir (inputPath : Path , hadoopConf : Configuration ): Path = {
96+ private def getStagingDir (inputPath : Path ): Path = {
9797 val inputPathUri : URI = inputPath.toUri
9898 val inputPathName : String = inputPathUri.getPath
9999 val fs : FileSystem = inputPath.getFileSystem(hadoopConf)
@@ -121,21 +121,69 @@ case class InsertIntoHiveTable(
121121 return dir
122122 }
123123
124- private def getExternalScratchDir (extURI : URI , hadoopConf : Configuration ): Path = {
125- getStagingDir(new Path (extURI.getScheme, extURI.getAuthority, extURI.getPath), hadoopConf )
124+ private def getExternalScratchDir (extURI : URI ): Path = {
125+ getStagingDir(new Path (extURI.getScheme, extURI.getAuthority, extURI.getPath))
126126 }
127127
128- def getExternalTmpPath (path : Path , hadoopConf : Configuration ): Path = {
128+ def getExternalTmpPath (path : Path ): Path = {
129+ import org .apache .spark .sql .hive .client .hive ._
130+
131+ val hiveVersion = externalCatalog.asInstanceOf [HiveExternalCatalog ].client.version
132+ // Before Hive 1.1, when inserting into a table, Hive will create the staging directory under
133+ // a common scratch directory. After the writing is finished, Hive will simply empty the table
134+ // directory and move the staging directory to it.
135+ // After Hive 1.1, Hive will create the staging directory under the table directory, and when
136+ // moving staging directory to table directory, Hive will still empty the table directory, but
137+ // will exclude the staging directory there.
138+ // We have to follow the Hive behavior here, to avoid troubles. For example, if we create
139+ // staging directory under the table director for Hive prior to 1.1, the staging directory will
140+ // be removed by Hive when Hive is trying to empty the table directory.
141+ if (hiveVersion == v12 || hiveVersion == v13 || hiveVersion == v14 || hiveVersion == v1_0) {
142+ oldVersionExternalTempPath(path)
143+ } else if (hiveVersion == v1_1 || hiveVersion == v1_2) {
144+ newVersionExternalTempPath(path)
145+ } else {
146+ throw new IllegalStateException (" Unsupported hive version: " + hiveVersion.fullVersion)
147+ }
148+ }
149+
150+ // Mostly copied from Context.java#getExternalTmpPath of Hive 0.13
151+ def oldVersionExternalTempPath (path : Path ): Path = {
152+ val extURI : URI = path.toUri
153+ val scratchPath = new Path (scratchDir, executionId)
154+ var dirPath = new Path (
155+ extURI.getScheme,
156+ extURI.getAuthority,
157+ scratchPath.toUri.getPath + " -" + TaskRunner .getTaskRunnerID())
158+
159+ try {
160+ val fs : FileSystem = dirPath.getFileSystem(hadoopConf)
161+ dirPath = new Path (fs.makeQualified(dirPath).toString())
162+
163+ if (! FileUtils .mkdir(fs, dirPath, true , hadoopConf)) {
164+ throw new IllegalStateException (" Cannot create staging directory: " + dirPath.toString)
165+ }
166+ fs.deleteOnExit(dirPath)
167+ } catch {
168+ case e : IOException =>
169+ throw new RuntimeException (" Cannot create staging directory: " + dirPath.toString, e)
170+
171+ }
172+ dirPath
173+ }
174+
175+ // Mostly copied from Context.java#getExternalTmpPath of Hive 1.2
176+ def newVersionExternalTempPath (path : Path ): Path = {
129177 val extURI : URI = path.toUri
130178 if (extURI.getScheme == " viewfs" ) {
131- getExtTmpPathRelTo(path.getParent, hadoopConf )
179+ getExtTmpPathRelTo(path.getParent)
132180 } else {
133- new Path (getExternalScratchDir(extURI, hadoopConf ), " -ext-10000" )
181+ new Path (getExternalScratchDir(extURI), " -ext-10000" )
134182 }
135183 }
136184
137- def getExtTmpPathRelTo (path : Path , hadoopConf : Configuration ): Path = {
138- new Path (getStagingDir(path, hadoopConf ), " -ext-10000" ) // Hive uses 10000
185+ def getExtTmpPathRelTo (path : Path ): Path = {
186+ new Path (getStagingDir(path), " -ext-10000" ) // Hive uses 10000
139187 }
140188
141189 private def saveAsHiveFile (
@@ -172,7 +220,7 @@ case class InsertIntoHiveTable(
172220 // instances within the closure, since Serializer is not serializable while TableDesc is.
173221 val tableDesc = table.tableDesc
174222 val tableLocation = table.hiveQlTable.getDataLocation
175- val tmpLocation = getExternalTmpPath(tableLocation, hadoopConf )
223+ val tmpLocation = getExternalTmpPath(tableLocation)
176224 val fileSinkConf = new FileSinkDesc (tmpLocation.toString, tableDesc, false )
177225 val isCompressed = hadoopConf.get(" hive.exec.compress.output" , " false" ).toBoolean
178226
0 commit comments