-
Notifications
You must be signed in to change notification settings - Fork 13
/
Copy pathS3ShuffleMapOutputWriter.scala
244 lines (210 loc) · 7.1 KB
/
S3ShuffleMapOutputWriter.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
//
// Copyright 2022- IBM Inc. All rights reserved
// SPDX-License-Identifier: Apache 2.0
//
package org.apache.spark.shuffle
import org.apache.hadoop.fs.FSDataOutputStream
import org.apache.spark.SparkConf
import org.apache.spark.internal.Logging
import org.apache.spark.shuffle.IndexShuffleBlockResolver.NOOP_REDUCE_ID
import org.apache.spark.shuffle.api.metadata.MapOutputCommitMessage
import org.apache.spark.shuffle.api.{ShuffleMapOutputWriter, ShufflePartitionWriter, WritableByteChannelWrapper}
import org.apache.spark.shuffle.helper.{S3ShuffleDispatcher, S3ShuffleHelper}
import org.apache.spark.storage.ShuffleDataBlockId
import java.io.{BufferedOutputStream, IOException, OutputStream}
import java.nio.ByteBuffer
import java.nio.channels.{Channels, WritableByteChannel}
import java.util.Optional
/** Implements the ShuffleMapOutputWriter interface. It stores the shuffle output in one shuffle block.
*
* This file is based on Spark "LocalDiskShuffleMapOutputWriter.java".
*/
class S3ShuffleMapOutputWriter(
conf: SparkConf,
shuffleId: Int,
mapId: Long,
numPartitions: Int
) extends ShuffleMapOutputWriter
with Logging {
val dispatcher = S3ShuffleDispatcher.get
/* Target block for writing */
private val shuffleBlock = ShuffleDataBlockId(shuffleId, mapId, NOOP_REDUCE_ID)
private var stream: FSDataOutputStream = _
private var bufferedStream: OutputStream = _
private var bufferedStreamAsChannel: WritableByteChannel = _
private var reduceIdStreamPosition: Long = 0
def initStream(): Unit = {
if (stream == null) {
stream = dispatcher.createBlock(shuffleBlock)
bufferedStream =
new S3MeasureOutputStream(new BufferedOutputStream(stream, dispatcher.bufferSize), shuffleBlock.name)
}
}
def initChannel(): Unit = {
if (bufferedStreamAsChannel == null) {
initStream()
bufferedStreamAsChannel = Channels.newChannel(bufferedStream)
}
}
private val partitionLengths = Array.fill[Long](numPartitions)(0)
private var totalBytesWritten: Long = 0
private var lastPartitionWriterId: Int = -1
/** @param reducePartitionId
* Monotonically increasing, as per contract in ShuffleMapOutputWriter.
* @return
* An instance of the ShufflePartitionWriter exposing the single output stream.
*/
override def getPartitionWriter(reducePartitionId: Int): ShufflePartitionWriter = {
if (reducePartitionId <= lastPartitionWriterId) {
throw new RuntimeException("Precondition: Expect a monotonically increasing reducePartitionId.")
}
if (reducePartitionId >= numPartitions) {
throw new RuntimeException("Precondition: Invalid partition id.")
}
if (bufferedStream != null) {
bufferedStream.flush()
}
if (stream != null) {
stream.flush()
reduceIdStreamPosition = stream.getPos
}
lastPartitionWriterId = reducePartitionId
new S3ShufflePartitionWriter(reducePartitionId)
}
/** Close all writers and the shuffle block.
*
* @param checksums
* Ignored.
* @return
*/
override def commitAllPartitions(checksums: Array[Long]): MapOutputCommitMessage = {
if (bufferedStream != null) {
bufferedStream.flush()
}
if (stream != null) {
if (stream.getPos != totalBytesWritten) {
throw new RuntimeException(
f"S3ShuffleMapOutputWriter: Unexpected output length ${stream.getPos}, expected: ${totalBytesWritten}."
)
}
}
if (bufferedStreamAsChannel != null) {
bufferedStreamAsChannel.close()
}
if (bufferedStream != null) {
// Closes the underlying stream as well!
bufferedStream.close()
}
// Write index and checksum.
if (partitionLengths.sum > 0 || S3ShuffleDispatcher.get.alwaysCreateIndex) {
S3ShuffleHelper.writePartitionLengths(shuffleId, mapId, partitionLengths)
if (dispatcher.checksumEnabled) {
S3ShuffleHelper.writeChecksum(shuffleId, mapId, checksums)
}
}
MapOutputCommitMessage.of(partitionLengths)
}
override def abort(error: Throwable): Unit = {
cleanUp()
}
private def cleanUp(): Unit = {
if (bufferedStreamAsChannel != null) {
bufferedStreamAsChannel.close()
}
if (bufferedStream != null) {
bufferedStream.close()
}
if (stream != null) {
stream.close()
}
}
private class S3ShufflePartitionWriter(reduceId: Int) extends ShufflePartitionWriter with Logging {
private var partitionStream: S3ShuffleOutputStream = _
private var partitionChannel: S3ShufflePartitionWriterChannel = _
override def openStream(): OutputStream = {
initStream()
if (partitionStream == null) {
partitionStream = new S3ShuffleOutputStream(reduceId)
}
partitionStream
}
override def openChannelWrapper(): Optional[WritableByteChannelWrapper] = {
if (partitionChannel == null) {
initChannel()
partitionChannel = new S3ShufflePartitionWriterChannel(reduceId)
}
Optional.of(partitionChannel)
}
override def getNumBytesWritten: Long = {
if (partitionChannel != null) {
return partitionChannel.numBytesWritten
}
if (partitionStream != null) {
return partitionStream.numBytesWritten
}
// The partition is empty.
0
}
}
private class S3ShuffleOutputStream(reduceId: Int) extends OutputStream {
private var byteCount: Long = 0
private var isClosed = false
def numBytesWritten: Long = byteCount
override def write(b: Int): Unit = {
if (isClosed) {
throw new IOException("S3ShuffleOutputStream is already closed.")
}
bufferedStream.write(b)
byteCount += 1
}
override def write(b: Array[Byte], off: Int, len: Int): Unit = {
if (isClosed) {
throw new IOException("S3ShuffleOutputStream is already closed.")
}
bufferedStream.write(b, off, len)
byteCount += len
}
override def flush(): Unit = {
if (isClosed) {
throw new IOException("S3ShuffleOutputStream is already closed.")
}
bufferedStream.flush()
}
override def close(): Unit = {
partitionLengths(reduceId) = byteCount
totalBytesWritten += byteCount
isClosed = true
}
}
private class S3ShufflePartitionWriterChannel(reduceId: Int) extends WritableByteChannelWrapper {
private val partChannel = new S3PartitionWritableByteChannel(bufferedStreamAsChannel)
override def channel(): WritableByteChannel = {
partChannel
}
def numBytesWritten: Long = {
partChannel.numBytesWritten()
}
override def close(): Unit = {
partitionLengths(reduceId) = numBytesWritten
totalBytesWritten += numBytesWritten
}
}
private class S3PartitionWritableByteChannel(channel: WritableByteChannel) extends WritableByteChannel {
private var count: Long = 0
def numBytesWritten(): Long = {
count
}
override def isOpen(): Boolean = {
channel.isOpen()
}
override def close(): Unit = {}
override def write(x: ByteBuffer): Int = {
var c = 0
while (x.hasRemaining()) {
c += channel.write(x)
}
count += c
c
}
}
}