@@ -51,40 +51,66 @@ abstract class StreamFileInputFormat[T]
5151 }
5252
5353 def createRecordReader (split : InputSplit , taContext : TaskAttemptContext ):
54- RecordReader [String ,T ]
54+ RecordReader [String ,T ]
5555
5656}
5757
58+ /**
59+ * A class that allows DataStreams to be serialized and moved around by not creating them
60+ * until they need to be read
61+ * @param split
62+ * @param context
63+ * @param index
64+ */
65+ class PortableDataStream (split : CombineFileSplit , context : TaskAttemptContext , index : Integer )
66+ extends Serializable {
67+ private var path = " "
68+ private var fileIn : FSDataInputStream = null .asInstanceOf [FSDataInputStream ]
69+ private var isOpen = false
70+
71+ def open (): FSDataInputStream = {
72+ val pathp = split.getPath(index)
73+ path = pathp.toString
74+ val fs = pathp.getFileSystem(context.getConfiguration)
75+ fileIn = fs.open(pathp)
76+ isOpen= true
77+ fileIn
78+ }
79+
80+ def close () = {
81+ if (isOpen) {
82+ try {
83+ fileIn.close()
84+ isOpen= false
85+ } catch {
86+ case ioe : java.io.IOException => // do nothing
87+ }
88+ }
89+ }
90+ def getPath (): String = path
91+ }
92+
5893/**
5994 * An abstract class of [[org.apache.hadoop.mapreduce.RecordReader RecordReader ]]
6095 * to reading files out as streams
6196 */
6297abstract class StreamBasedRecordReader [T ](
63- split : CombineFileSplit ,
64- context : TaskAttemptContext ,
65- index : Integer )
98+ split : CombineFileSplit ,
99+ context : TaskAttemptContext ,
100+ index : Integer )
66101 extends RecordReader [String , T ] {
67102
68- private val path = split.getPath(index)
69- private val fs = path.getFileSystem(context.getConfiguration)
103+
70104
71105 // True means the current file has been processed, then skip it.
72106 private var processed = false
73107
74- private val key = path.toString
108+ private var key = " "
75109 private var value : T = null .asInstanceOf [T ]
76- // the file to be read when nextkeyvalue is called
77- private lazy val fileIn : FSDataInputStream = fs.open(path)
110+
78111
79112 override def initialize (split : InputSplit , context : TaskAttemptContext ) = {}
80- override def close () = {
81- // make sure the file is closed
82- try {
83- fileIn.close()
84- } catch {
85- case ioe : java.io.IOException => // do nothing
86- }
87- }
113+ override def close () = {}
88114
89115 override def getProgress = if (processed) 1.0f else 0.0f
90116
@@ -93,10 +119,13 @@ abstract class StreamBasedRecordReader[T](
93119 override def getCurrentValue = value
94120
95121
122+
96123 override def nextKeyValue = {
97124 if (! processed) {
98-
125+ val fileIn = new PortableDataStream (split,context,index)
126+ key = fileIn.getPath
99127 value = parseStream(fileIn)
128+ fileIn.close() // if it has not been open yet, close does nothing
100129 processed = true
101130 true
102131 } else {
@@ -109,29 +138,29 @@ abstract class StreamBasedRecordReader[T](
109138 * @param inStream the stream to be read in
110139 * @return the data formatted as
111140 */
112- def parseStream (inStream : DataInputStream ): T
141+ def parseStream (inStream : PortableDataStream ): T
113142}
114143
115144/**
116145 * Reads the record in directly as a stream for other objects to manipulate and handle
117146 */
118147private [spark] class StreamRecordReader (
119- split : CombineFileSplit ,
120- context : TaskAttemptContext ,
121- index : Integer )
122- extends StreamBasedRecordReader [DataInputStream ](split,context,index) {
148+ split : CombineFileSplit ,
149+ context : TaskAttemptContext ,
150+ index : Integer )
151+ extends StreamBasedRecordReader [PortableDataStream ](split,context,index) {
123152
124- def parseStream (inStream : DataInputStream ): DataInputStream = inStream
153+ def parseStream (inStream : PortableDataStream ): PortableDataStream = inStream
125154}
126155
127156/**
128157 * A class for extracting the information from the file using the
129158 * BinaryRecordReader (as Byte array)
130159 */
131- private [spark] class StreamInputFormat extends StreamFileInputFormat [DataInputStream ] {
160+ private [spark] class StreamInputFormat extends StreamFileInputFormat [PortableDataStream ] {
132161 override def createRecordReader (split : InputSplit , taContext : TaskAttemptContext )=
133162 {
134- new CombineFileRecordReader [String ,DataInputStream ](
163+ new CombineFileRecordReader [String ,PortableDataStream ](
135164 split.asInstanceOf [CombineFileSplit ],taContext,classOf [StreamRecordReader ]
136165 )
137166 }
@@ -143,12 +172,13 @@ private[spark] class StreamInputFormat extends StreamFileInputFormat[DataInputSt
143172 * the file as a byte array
144173 */
145174abstract class BinaryRecordReader [T ](
146- split : CombineFileSplit ,
147- context : TaskAttemptContext ,
148- index : Integer )
175+ split : CombineFileSplit ,
176+ context : TaskAttemptContext ,
177+ index : Integer )
149178 extends StreamBasedRecordReader [T ](split,context,index) {
150179
151- def parseStream (inStream : DataInputStream ): T = {
180+ def parseStream (inpStream : PortableDataStream ): T = {
181+ val inStream = inpStream.open()
152182 val innerBuffer = ByteStreams .toByteArray(inStream)
153183 Closeables .close(inStream, false )
154184 parseByteArray(innerBuffer)
@@ -157,13 +187,14 @@ abstract class BinaryRecordReader[T](
157187}
158188
159189
190+
160191private [spark] class ByteRecordReader (
161- split : CombineFileSplit ,
162- context : TaskAttemptContext ,
163- index : Integer )
192+ split : CombineFileSplit ,
193+ context : TaskAttemptContext ,
194+ index : Integer )
164195 extends BinaryRecordReader [Array [Byte ]](split,context,index) {
165196
166- def parseByteArray (inArray : Array [Byte ]) = inArray
197+ override def parseByteArray (inArray : Array [Byte ]) = inArray
167198}
168199
169200/**
0 commit comments