Skip to content

Commit 096bbbc

Browse files
committed
Merge pull request #1 from liancheng/refactor-dp
Refactors dynamic partitioning support
2 parents 5033928 + 1093c20 commit 096bbbc

11 files changed

+223
-312
lines changed

sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala renamed to sql/hive/src/main/scala/org/apache/spark/sql/hive/SparkHadoopWriter.scala

Lines changed: 91 additions & 109 deletions
Original file line numberDiff line numberDiff line change
@@ -21,28 +21,32 @@ import java.io.IOException
2121
import java.text.NumberFormat
2222
import java.util.Date
2323

24+
import scala.collection.mutable
25+
2426
import org.apache.hadoop.fs.Path
27+
import org.apache.hadoop.hive.conf.HiveConf.ConfVars
2528
import org.apache.hadoop.hive.ql.exec.{FileSinkOperator, Utilities}
2629
import org.apache.hadoop.hive.ql.io.{HiveFileFormatUtils, HiveOutputFormat}
2730
import org.apache.hadoop.hive.ql.plan.FileSinkDesc
28-
import org.apache.hadoop.mapred._
2931
import org.apache.hadoop.io.Writable
32+
import org.apache.hadoop.mapred._
3033

34+
import org.apache.spark.sql.Row
3135
import org.apache.spark.{Logging, SerializableWritable, SparkHadoopWriter}
3236

3337
/**
3438
* Internal helper class that saves an RDD using a Hive OutputFormat.
3539
* It is based on [[SparkHadoopWriter]].
3640
*/
37-
private[hive] class SparkHiveHadoopWriter(
41+
private[hive] class SparkHiveWriterContainer(
3842
@transient jobConf: JobConf,
3943
fileSinkConf: FileSinkDesc)
4044
extends Logging
4145
with SparkHadoopMapRedUtil
4246
with Serializable {
4347

4448
private val now = new Date()
45-
private val conf = new SerializableWritable(jobConf)
49+
protected val conf = new SerializableWritable(jobConf)
4650

4751
private var jobID = 0
4852
private var splitID = 0
@@ -51,152 +55,75 @@ private[hive] class SparkHiveHadoopWriter(
5155
private var taID: SerializableWritable[TaskAttemptID] = null
5256

5357
@transient private var writer: FileSinkOperator.RecordWriter = null
54-
@transient private var format: HiveOutputFormat[AnyRef, Writable] = null
55-
@transient private var committer: OutputCommitter = null
56-
@transient private var jobContext: JobContext = null
57-
@transient private var taskContext: TaskAttemptContext = null
58+
@transient private lazy val committer = conf.value.getOutputCommitter
59+
@transient private lazy val jobContext = newJobContext(conf.value, jID.value)
60+
@transient private lazy val taskContext = newTaskAttemptContext(conf.value, taID.value)
61+
@transient private lazy val outputFormat =
62+
conf.value.getOutputFormat.asInstanceOf[HiveOutputFormat[AnyRef,Writable]]
5863

59-
def preSetup() {
64+
def driverSideSetup() {
6065
setIDs(0, 0, 0)
6166
setConfParams()
62-
63-
val jCtxt = getJobContext()
64-
getOutputCommitter().setupJob(jCtxt)
67+
committer.setupJob(jobContext)
6568
}
6669

67-
68-
def setup(jobid: Int, splitid: Int, attemptid: Int) {
69-
setIDs(jobid, splitid, attemptid)
70+
def executorSideSetup(jobId: Int, splitId: Int, attemptId: Int) {
71+
setIDs(jobId, splitId, attemptId)
7072
setConfParams()
71-
}
72-
73-
def open() {
74-
val numfmt = NumberFormat.getInstance()
75-
numfmt.setMinimumIntegerDigits(5)
76-
numfmt.setGroupingUsed(false)
77-
78-
val extension = Utilities.getFileExtension(
79-
conf.value,
80-
fileSinkConf.getCompressed,
81-
getOutputFormat())
82-
83-
val outputName = "part-" + numfmt.format(splitID) + extension
84-
val path = FileOutputFormat.getTaskOutputPath(conf.value, outputName)
85-
86-
getOutputCommitter().setupTask(getTaskContext())
87-
writer = HiveFileFormatUtils.getHiveRecordWriter(
88-
conf.value,
89-
fileSinkConf.getTableInfo,
90-
conf.value.getOutputValueClass.asInstanceOf[Class[Writable]],
91-
fileSinkConf,
92-
path,
93-
null)
73+
committer.setupTask(taskContext)
9474
}
9575

9676
/**
97-
* create an HiveRecordWriter. imitate the above function open()
98-
* @param dynamicPartPath the relative path for dynamic partition
99-
*
100-
* since this function is used to create different writer for
101-
* different dynamic partition.So we need a parameter dynamicPartPath
102-
* and use it we can calculate a new path and pass the new path to
103-
* the function HiveFileFormatUtils.getHiveRecordWriter
77+
* Create a `HiveRecordWriter`. A relative dynamic partition path can be used to create a writer
78+
* for writing data to a dynamic partition.
10479
*/
105-
def open(dynamicPartPath: String) {
106-
val numfmt = NumberFormat.getInstance()
107-
numfmt.setMinimumIntegerDigits(5)
108-
numfmt.setGroupingUsed(false)
109-
110-
val extension = Utilities.getFileExtension(
111-
conf.value,
112-
fileSinkConf.getCompressed,
113-
getOutputFormat())
114-
115-
val outputName = "part-" + numfmt.format(splitID) + extension
116-
val outputPath: Path = FileOutputFormat.getOutputPath(conf.value)
117-
if (outputPath == null) {
118-
throw new IOException("Undefined job output-path")
119-
}
120-
val workPath = new Path(outputPath, dynamicPartPath.stripPrefix("/")) // remove "/"
121-
val path = new Path(workPath, outputName)
122-
getOutputCommitter().setupTask(getTaskContext())
80+
def open() {
12381
writer = HiveFileFormatUtils.getHiveRecordWriter(
12482
conf.value,
12583
fileSinkConf.getTableInfo,
12684
conf.value.getOutputValueClass.asInstanceOf[Class[Writable]],
12785
fileSinkConf,
128-
path,
86+
FileOutputFormat.getTaskOutputPath(conf.value, getOutputName),
12987
Reporter.NULL)
13088
}
13189

132-
def write(value: Writable) {
133-
if (writer != null) {
134-
writer.write(value)
135-
} else {
136-
throw new IOException("Writer is null, open() has not been called")
137-
}
90+
protected def getOutputName: String = {
91+
val numberFormat = NumberFormat.getInstance()
92+
numberFormat.setMinimumIntegerDigits(5)
93+
numberFormat.setGroupingUsed(false)
94+
val extension = Utilities.getFileExtension(conf.value, fileSinkConf.getCompressed, outputFormat)
95+
"part-" + numberFormat.format(splitID) + extension
13896
}
13997

98+
def getLocalFileWriter(row: Row): FileSinkOperator.RecordWriter = writer
99+
140100
def close() {
141101
// Seems the boolean value passed into close does not matter.
142102
writer.close(false)
143103
}
144104

145105
def commit() {
146-
val taCtxt = getTaskContext()
147-
val cmtr = getOutputCommitter()
148-
if (cmtr.needsTaskCommit(taCtxt)) {
106+
if (committer.needsTaskCommit(taskContext)) {
149107
try {
150-
cmtr.commitTask(taCtxt)
108+
committer.commitTask(taskContext)
151109
logInfo (taID + ": Committed")
152110
} catch {
153111
case e: IOException =>
154112
logError("Error committing the output of task: " + taID.value, e)
155-
cmtr.abortTask(taCtxt)
113+
committer.abortTask(taskContext)
156114
throw e
157115
}
158116
} else {
159-
logWarning ("No need to commit output of task: " + taID.value)
117+
logInfo("No need to commit output of task: " + taID.value)
160118
}
161119
}
162120

163121
def commitJob() {
164-
// always ? Or if cmtr.needsTaskCommit ?
165-
val cmtr = getOutputCommitter()
166-
cmtr.commitJob(getJobContext())
122+
committer.commitJob(jobContext)
167123
}
168124

169125
// ********* Private Functions *********
170126

171-
private def getOutputFormat(): HiveOutputFormat[AnyRef,Writable] = {
172-
if (format == null) {
173-
format = conf.value.getOutputFormat()
174-
.asInstanceOf[HiveOutputFormat[AnyRef,Writable]]
175-
}
176-
format
177-
}
178-
179-
private def getOutputCommitter(): OutputCommitter = {
180-
if (committer == null) {
181-
committer = conf.value.getOutputCommitter
182-
}
183-
committer
184-
}
185-
186-
private def getJobContext(): JobContext = {
187-
if (jobContext == null) {
188-
jobContext = newJobContext(conf.value, jID.value)
189-
}
190-
jobContext
191-
}
192-
193-
private def getTaskContext(): TaskAttemptContext = {
194-
if (taskContext == null) {
195-
taskContext = newTaskAttemptContext(conf.value, taID.value)
196-
}
197-
taskContext
198-
}
199-
200127
private def setIDs(jobId: Int, splitId: Int, attemptId: Int) {
201128
jobID = jobId
202129
splitID = splitId
@@ -216,7 +143,7 @@ private[hive] class SparkHiveHadoopWriter(
216143
}
217144
}
218145

219-
private[hive] object SparkHiveHadoopWriter {
146+
private[hive] object SparkHiveWriterContainer {
220147
def createPathFromString(path: String, conf: JobConf): Path = {
221148
if (path == null) {
222149
throw new IllegalArgumentException("Output path is null")
@@ -226,6 +153,61 @@ private[hive] object SparkHiveHadoopWriter {
226153
if (outputPath == null || fs == null) {
227154
throw new IllegalArgumentException("Incorrectly formatted output path")
228155
}
229-
outputPath.makeQualified(fs)
156+
outputPath.makeQualified(fs.getUri, fs.getWorkingDirectory)
157+
}
158+
}
159+
160+
private[spark] class SparkHiveDynamicPartitionWriterContainer(
161+
@transient jobConf: JobConf,
162+
fileSinkConf: FileSinkDesc,
163+
dynamicPartColNames: Array[String])
164+
extends SparkHiveWriterContainer(jobConf, fileSinkConf) {
165+
166+
private val defaultPartName = jobConf.get(
167+
ConfVars.DEFAULTPARTITIONNAME.varname, ConfVars.DEFAULTPARTITIONNAME.defaultVal)
168+
169+
@transient private var writers: mutable.HashMap[String, FileSinkOperator.RecordWriter] = _
170+
171+
override def open(): Unit = {
172+
writers = mutable.HashMap.empty[String, FileSinkOperator.RecordWriter]
173+
}
174+
175+
override def close(): Unit = {
176+
writers.values.foreach(_.close(false))
177+
}
178+
179+
override def getLocalFileWriter(row: Row): FileSinkOperator.RecordWriter = {
180+
val dynamicPartPath = dynamicPartColNames
181+
.zip(row.takeRight(dynamicPartColNames.length))
182+
.map { case (col, rawVal) =>
183+
val string = String.valueOf(rawVal)
184+
s"/$col=${if (rawVal == null || string.isEmpty) defaultPartName else string}"
185+
}
186+
.mkString
187+
188+
val path = {
189+
val outputPath = FileOutputFormat.getOutputPath(conf.value)
190+
assert(outputPath != null, "Undefined job output-path")
191+
val workPath = new Path(outputPath, dynamicPartPath.stripPrefix("/"))
192+
new Path(workPath, getOutputName)
193+
}
194+
195+
def newWriter = {
196+
val newFileSinkDesc = new FileSinkDesc(
197+
fileSinkConf.getDirName + dynamicPartPath,
198+
fileSinkConf.getTableInfo,
199+
fileSinkConf.getCompressed)
200+
newFileSinkDesc.setCompressCodec(fileSinkConf.getCompressCodec)
201+
newFileSinkDesc.setCompressType(fileSinkConf.getCompressType)
202+
HiveFileFormatUtils.getHiveRecordWriter(
203+
conf.value,
204+
fileSinkConf.getTableInfo,
205+
conf.value.getOutputValueClass.asInstanceOf[Class[Writable]],
206+
newFileSinkDesc,
207+
path,
208+
Reporter.NULL)
209+
}
210+
211+
writers.getOrElseUpdate(dynamicPartPath, newWriter)
230212
}
231213
}

0 commit comments

Comments
 (0)