Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
modify test case
  • Loading branch information
windpiger committed Feb 28, 2017
commit 22bef8be2f242cf54edb06ffad9fd447fb29d1db
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ package org.apache.spark.sql.internal
import java.io.File
import java.net.URI

import org.apache.hadoop.fs.Path
import org.scalatest.BeforeAndAfterEach

import org.apache.spark.SparkFunSuite
Expand Down Expand Up @@ -459,7 +460,9 @@ class CatalogSuite
options = Map("path" -> dir.getAbsolutePath))
val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t"))
assert(table.tableType == CatalogTableType.EXTERNAL)
assert(table.storage.locationUri.get == s"file:${dir.getAbsolutePath}")
val dirPath = new Path(dir.getAbsolutePath)
val fs = dirPath.getFileSystem(spark.sessionState.newHadoopConf())
assert(new Path(table.storage.locationUri.get) == fs.makeQualified(dirPath))

Seq((1)).toDF("i").write.insertInto("t")
assert(dir.exists() && dir.listFiles().nonEmpty)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -85,15 +85,17 @@ class PathOptionSuite extends DataSourceTest with SharedSQLContext {
test("path option also exist for write path") {
withTable("src") {
withTempPath { p =>
val path = new Path(p.getAbsolutePath).toString
val path = new Path(p.getAbsolutePath)
sql(
s"""
|CREATE TABLE src
|USING ${classOf[TestOptionsSource].getCanonicalName}
|OPTIONS (PATH '$path')
|AS SELECT 1
""".stripMargin)
assert(spark.table("src").schema.head.metadata.getString("path") == s"file:$path")
val fs = path.getFileSystem(spark.sessionState.newHadoopConf())
assert(new Path(spark.table("src").schema.head.metadata.getString("path")) ==
fs.makeQualified(path))
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@

package org.apache.spark.sql.hive

import org.apache.hadoop.fs.Path

import org.apache.spark.sql.{QueryTest, Row, SaveMode}
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.catalog.CatalogTableType
Expand Down Expand Up @@ -140,7 +142,9 @@ class DataSourceWithHiveMetastoreCatalogSuite
assert(hiveTable.storage.serde === Some(serde))

assert(hiveTable.tableType === CatalogTableType.EXTERNAL)
assert(hiveTable.storage.locationUri === Some(s"file:${path.toString}"))
val dirPath = new Path(dir.getAbsolutePath)
val fs = dirPath.getFileSystem(spark.sessionState.newHadoopConf())
assert(hiveTable.storage.locationUri === Some(fs.makeQualified(dirPath).toString))

val columns = hiveTable.schema
assert(columns.map(_.name) === Seq("d1", "d2"))
Expand Down