[SPARK-23514][FOLLOW-UP] Remove more places using sparkContext.hadoopConfiguration directly

## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/20679 I missed a few places in SQL tests.
For hygiene, they should also use the sessionState interface where possible.

## How was this patch tested?

Modified existing tests.

Author: Juliusz Sompolski <julek@databricks.com>

Closes #20718 from juliuszsompolski/SPARK-23514-followup.
This commit is contained in:
Juliusz Sompolski 2018-03-03 09:10:48 +08:00 committed by Wenchen Fan
parent 9e26473c0f
commit dea381dfaa
8 changed files with 9 additions and 9 deletions

View file

@ -124,7 +124,7 @@ class FileBasedDataSourceSuite extends QueryTest with SharedSQLContext with Befo
Seq("1").toDF("a").write.format(format).save(new Path(basePath, "second").toString)
val thirdPath = new Path(basePath, "third")
val fs = thirdPath.getFileSystem(spark.sparkContext.hadoopConfiguration)
val fs = thirdPath.getFileSystem(spark.sessionState.newHadoopConf())
Seq("2").toDF("a").write.format(format).save(thirdPath.toString)
val files = fs.listStatus(thirdPath).filter(_.isFile).map(_.getPath)

View file

@ -1052,7 +1052,7 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils {
val part2 = Map("a" -> "2", "b" -> "6")
val root = new Path(catalog.getTableMetadata(tableIdent).location)
val fs = root.getFileSystem(spark.sparkContext.hadoopConfiguration)
val fs = root.getFileSystem(spark.sessionState.newHadoopConf())
// valid
fs.mkdirs(new Path(new Path(root, "a=1"), "b=5"))
fs.createNewFile(new Path(new Path(root, "a=1/b=5"), "a.csv")) // file

View file

@ -59,7 +59,7 @@ class FileIndexSuite extends SharedSQLContext {
require(!unqualifiedDirPath.toString.contains("file:"))
require(!unqualifiedFilePath.toString.contains("file:"))
val fs = unqualifiedDirPath.getFileSystem(sparkContext.hadoopConfiguration)
val fs = unqualifiedDirPath.getFileSystem(spark.sessionState.newHadoopConf())
val qualifiedFilePath = fs.makeQualified(new Path(file.getCanonicalPath))
require(qualifiedFilePath.toString.startsWith("file:"))

View file

@ -101,7 +101,7 @@ class ParquetCommitterSuite extends SparkFunSuite with SQLTestUtils
if (check) {
result = Some(MarkingFileOutput.checkMarker(
destPath,
spark.sparkContext.hadoopConfiguration))
spark.sessionState.newHadoopConf()))
}
}
}

View file

@ -29,7 +29,7 @@ class ParquetFileFormatSuite extends QueryTest with ParquetTest with SharedSQLCo
test("read parquet footers in parallel") {
def testReadFooters(ignoreCorruptFiles: Boolean): Unit = {
withTempDir { dir =>
val fs = FileSystem.get(sparkContext.hadoopConfiguration)
val fs = FileSystem.get(spark.sessionState.newHadoopConf())
val basePath = dir.getCanonicalPath
val path1 = new Path(basePath, "first")
@ -44,7 +44,7 @@ class ParquetFileFormatSuite extends QueryTest with ParquetTest with SharedSQLCo
Seq(fs.listStatus(path1), fs.listStatus(path2), fs.listStatus(path3)).flatten
val footers = ParquetFileFormat.readParquetFootersInParallel(
sparkContext.hadoopConfiguration, fileStatuses, ignoreCorruptFiles)
spark.sessionState.newHadoopConf(), fileStatuses, ignoreCorruptFiles)
assert(footers.size == 2)
}

View file

@ -163,7 +163,7 @@ class ParquetInteroperabilitySuite extends ParquetCompatibilityTest with SharedS
// Just to be defensive in case anything ever changes in parquet, this test checks
// the assumption on column stats, and also the end-to-end behavior.
val hadoopConf = sparkContext.hadoopConfiguration
val hadoopConf = spark.sessionState.newHadoopConf()
val fs = FileSystem.get(hadoopConf)
val parts = fs.listStatus(new Path(tableDir.getAbsolutePath), new PathFilter {
override def accept(path: Path): Boolean = !path.getName.startsWith("_")

View file

@ -819,7 +819,7 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext
val path = dir.getCanonicalPath
spark.range(3).write.parquet(path)
val fs = FileSystem.get(sparkContext.hadoopConfiguration)
val fs = FileSystem.get(spark.sessionState.newHadoopConf())
val files = fs.listFiles(new Path(path), true)
while (files.hasNext) {

View file

@ -353,7 +353,7 @@ class FileStreamSinkSuite extends StreamTest {
}
test("FileStreamSink.ancestorIsMetadataDirectory()") {
val hadoopConf = spark.sparkContext.hadoopConfiguration
val hadoopConf = spark.sessionState.newHadoopConf()
def assertAncestorIsMetadataDirectory(path: String): Unit =
assert(FileStreamSink.ancestorIsMetadataDirectory(new Path(path), hadoopConf))
def assertAncestorIsNotMetadataDirectory(path: String): Unit =