blob: 5ea1f32433699962eb1cc241d67caa89ec2f54cf (
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
|
/*
* 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.sql.sources
import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.execution.datasources.DataSource
class ResolvedDataSourceSuite extends SparkFunSuite {
private def getProvidingClass(name: String): Class[_] =
DataSource(sparkSession = null, className = name).providingClass
test("jdbc") {
assert(
getProvidingClass("jdbc") ===
classOf[org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider])
assert(
getProvidingClass("org.apache.spark.sql.execution.datasources.jdbc") ===
classOf[org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider])
assert(
getProvidingClass("org.apache.spark.sql.jdbc") ===
classOf[org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider])
}
test("json") {
assert(
getProvidingClass("json") ===
classOf[org.apache.spark.sql.execution.datasources.json.JsonFileFormat])
assert(
getProvidingClass("org.apache.spark.sql.execution.datasources.json") ===
classOf[org.apache.spark.sql.execution.datasources.json.JsonFileFormat])
assert(
getProvidingClass("org.apache.spark.sql.json") ===
classOf[org.apache.spark.sql.execution.datasources.json.JsonFileFormat])
}
test("parquet") {
assert(
getProvidingClass("parquet") ===
classOf[org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat])
assert(
getProvidingClass("org.apache.spark.sql.execution.datasources.parquet") ===
classOf[org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat])
assert(
getProvidingClass("org.apache.spark.sql.parquet") ===
classOf[org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat])
}
test("csv") {
assert(
getProvidingClass("csv") ===
classOf[org.apache.spark.sql.execution.datasources.csv.CSVFileFormat])
assert(
getProvidingClass("com.databricks.spark.csv") ===
classOf[org.apache.spark.sql.execution.datasources.csv.CSVFileFormat])
}
test("error message for unknown data sources") {
val error1 = intercept[AnalysisException] {
getProvidingClass("avro")
}
assert(error1.getMessage.contains("spark-packages"))
val error2 = intercept[AnalysisException] {
getProvidingClass("com.databricks.spark.avro")
}
assert(error2.getMessage.contains("spark-packages"))
val error3 = intercept[ClassNotFoundException] {
getProvidingClass("asfdwefasdfasdf")
}
assert(error3.getMessage.contains("spark-packages"))
}
}
|