aboutsummaryrefslogtreecommitdiff
path: root/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala
blob: d852255a4fd29b894fa50e19768b75ea686b875e (plain) (blame)
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
/*
 * Licensed to the Apache Software Foundation (ASF) under one or more
 * contributor license agreements.  See the NOTICE file distributed with
 * this work for additional information regarding copyright ownership.
 * The ASF licenses this file to You under the Apache License, Version 2.0
 * (the "License"); you may not use this file except in compliance with
 * the License.  You may obtain a copy of the License at
 *
 *    http://www.apache.org/licenses/LICENSE-2.0
 *
 * Unless required by applicable law or agreed to in writing, software
 * distributed under the License is distributed on an "AS IS" BASIS,
 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 * See the License for the specific language governing permissions and
 * limitations under the License.
 */

package org.apache.spark.input

import java.io.DataOutputStream
import java.io.File
import java.io.FileOutputStream

import scala.collection.immutable.IndexedSeq

import org.apache.hadoop.io.Text
import org.apache.hadoop.io.compress.{CompressionCodecFactory, DefaultCodec, GzipCodec}
import org.scalatest.BeforeAndAfterAll

import org.apache.spark.{Logging, SparkConf, SparkContext, SparkFunSuite}
import org.apache.spark.util.Utils

/**
 * Tests the correctness of
 * [[org.apache.spark.input.WholeTextFileRecordReader WholeTextFileRecordReader]]. A temporary
 * directory is created as fake input. Temporal storage would be deleted in the end.
 */
class WholeTextFileRecordReaderSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
  private var sc: SparkContext = _
  private var factory: CompressionCodecFactory = _

  override def beforeAll() {
    // Hadoop's FileSystem caching does not use the Configuration as part of its cache key, which
    // can cause Filesystem.get(Configuration) to return a cached instance created with a different
    // configuration than the one passed to get() (see HADOOP-8490 for more details). This caused
    // hard-to-reproduce test failures, since any suites that were run after this one would inherit
    // the new value of "fs.local.block.size" (see SPARK-5227 and SPARK-5679). To work around this,
    // we disable FileSystem caching in this suite.
    super.beforeAll()
    val conf = new SparkConf().set("spark.hadoop.fs.file.impl.disable.cache", "true")

    sc = new SparkContext("local", "test", conf)

    // Set the block size of local file system to test whether files are split right or not.
    sc.hadoopConfiguration.setLong("fs.local.block.size", 32)
    sc.hadoopConfiguration.set("io.compression.codecs",
      "org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.DefaultCodec")
    factory = new CompressionCodecFactory(sc.hadoopConfiguration)
  }

  override def afterAll() {
    try {
      sc.stop()
    } finally {
      super.afterAll()
    }
  }

  private def createNativeFile(inputDir: File, fileName: String, contents: Array[Byte],
                               compress: Boolean) = {
    val out = if (compress) {
      val codec = new GzipCodec
      val path = s"${inputDir.toString}/$fileName${codec.getDefaultExtension}"
      codec.createOutputStream(new DataOutputStream(new FileOutputStream(path)))
    } else {
      val path = s"${inputDir.toString}/$fileName"
      new DataOutputStream(new FileOutputStream(path))
    }
    out.write(contents, 0, contents.length)
    out.close()
  }

  /**
   * This code will test the behaviors of WholeTextFileRecordReader based on local disk. There are
   * three aspects to check:
   *   1) Whether all files are read;
   *   2) Whether paths are read correctly;
   *   3) Does the contents be the same.
   */
  test("Correctness of WholeTextFileRecordReader.") {
    val dir = Utils.createTempDir()
    logInfo(s"Local disk address is ${dir.toString}.")

    WholeTextFileRecordReaderSuite.files.foreach { case (filename, contents) =>
      createNativeFile(dir, filename, contents, false)
    }

    val res = sc.wholeTextFiles(dir.toString, 3).collect()

    assert(res.size === WholeTextFileRecordReaderSuite.fileNames.size,
      "Number of files read out does not fit with the actual value.")

    for ((filename, contents) <- res) {
      val shortName = filename.split('/').last
      assert(WholeTextFileRecordReaderSuite.fileNames.contains(shortName),
        s"Missing file name $filename.")
      assert(contents === new Text(WholeTextFileRecordReaderSuite.files(shortName)).toString,
        s"file $filename contents can not match.")
    }

    Utils.deleteRecursively(dir)
  }

  test("Correctness of WholeTextFileRecordReader with GzipCodec.") {
    val dir = Utils.createTempDir()
    logInfo(s"Local disk address is ${dir.toString}.")

    WholeTextFileRecordReaderSuite.files.foreach { case (filename, contents) =>
      createNativeFile(dir, filename, contents, true)
    }

    val res = sc.wholeTextFiles(dir.toString, 3).collect()

    assert(res.size === WholeTextFileRecordReaderSuite.fileNames.size,
      "Number of files read out does not fit with the actual value.")

    for ((filename, contents) <- res) {
      val shortName = filename.split('/').last.split('.')(0)

      assert(WholeTextFileRecordReaderSuite.fileNames.contains(shortName),
        s"Missing file name $filename.")
      assert(contents === new Text(WholeTextFileRecordReaderSuite.files(shortName)).toString,
        s"file $filename contents can not match.")
    }

    Utils.deleteRecursively(dir)
  }
}

/**
 * Files to be tested are defined here.
 */
object WholeTextFileRecordReaderSuite {
  private val testWords: IndexedSeq[Byte] = "Spark is easy to use.\n".map(_.toByte)

  private val fileNames = Array("part-00000", "part-00001", "part-00002")
  private val fileLengths = Array(10, 100, 1000)

  private val files = fileLengths.zip(fileNames).map { case (upperBound, filename) =>
    filename -> Stream.continually(testWords.toList.toStream).flatten.take(upperBound).toArray
  }.toMap
}