File tree Expand file tree Collapse file tree 1 file changed +9
-3
lines changed
core/src/main/scala/org/apache/spark/input Expand file tree Collapse file tree 1 file changed +9
-3
lines changed Original file line number Diff line number Diff line change @@ -61,16 +61,22 @@ abstract class StreamFileInputFormat[T]
6161 */
6262class PortableDataStream (split : CombineFileSplit , context : TaskAttemptContext , index : Integer )
6363 extends Serializable {
64- private var path = " "
64+
6565 private var fileIn : FSDataInputStream = null .asInstanceOf [FSDataInputStream ]
6666 private var isOpen = false
67+ /**
68+ * Calculate the path name independently of opening the file
69+ */
70+ private lazy val path = {
71+ val pathp = split.getPath(index)
72+ path = pathp.toString
73+ }
6774
6875 /**
6976 * create a new DataInputStream from the split and context
7077 */
7178 def open (): FSDataInputStream = {
7279 val pathp = split.getPath(index)
73- path = pathp.toString
7480 val fs = pathp.getFileSystem(context.getConfiguration)
7581 fileIn = fs.open(pathp)
7682 isOpen= true
@@ -126,9 +132,9 @@ abstract class StreamBasedRecordReader[T](
126132 override def nextKeyValue = {
127133 if (! processed) {
128134 val fileIn = new PortableDataStream (split,context,index)
129- key = fileIn.getPath
130135 value = parseStream(fileIn)
131136 fileIn.close() // if it has not been open yet, close does nothing
137+ key = fileIn.getPath
132138 processed = true
133139 true
134140 } else {
You can’t perform that action at this time.
0 commit comments