Hide keyboard shortcuts

Hot-keys on this page

r m x p   toggle line displays

j k   next/prev highlighted chunk

0   (zero) top of page

1   (one) first highlighted chunk

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

153

154

155

156

157

158

159

160

161

162

163

164

165

166

167

168

169

170

171

# 

# 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. 

# 

 

import shutil 

import tempfile 

 

from pyspark.sql import Row 

from pyspark.sql.types import IntegerType, StructField, StructType, LongType, StringType 

from pyspark.testing.sqlutils import ReusedSQLTestCase 

 

 

class DataSourcesTests(ReusedSQLTestCase): 

 

def test_linesep_text(self): 

df = self.spark.read.text("python/test_support/sql/ages_newlines.csv", lineSep=",") 

expected = [Row(value=u'Joe'), Row(value=u'20'), Row(value=u'"Hi'), 

Row(value=u'\nI am Jeo"\nTom'), Row(value=u'30'), 

Row(value=u'"My name is Tom"\nHyukjin'), Row(value=u'25'), 

Row(value=u'"I am Hyukjin\n\nI love Spark!"\n')] 

self.assertEqual(df.collect(), expected) 

 

tpath = tempfile.mkdtemp() 

shutil.rmtree(tpath) 

try: 

df.write.text(tpath, lineSep="!") 

expected = [Row(value=u'Joe!20!"Hi!'), Row(value=u'I am Jeo"'), 

Row(value=u'Tom!30!"My name is Tom"'), 

Row(value=u'Hyukjin!25!"I am Hyukjin'), 

Row(value=u''), Row(value=u'I love Spark!"'), 

Row(value=u'!')] 

readback = self.spark.read.text(tpath) 

self.assertEqual(readback.collect(), expected) 

finally: 

shutil.rmtree(tpath) 

 

def test_multiline_json(self): 

people1 = self.spark.read.json("python/test_support/sql/people.json") 

people_array = self.spark.read.json("python/test_support/sql/people_array.json", 

multiLine=True) 

self.assertEqual(people1.collect(), people_array.collect()) 

 

def test_encoding_json(self): 

people_array = self.spark.read\ 

.json("python/test_support/sql/people_array_utf16le.json", 

multiLine=True, encoding="UTF-16LE") 

expected = [Row(age=30, name=u'Andy'), Row(age=19, name=u'Justin')] 

self.assertEqual(people_array.collect(), expected) 

 

def test_linesep_json(self): 

df = self.spark.read.json("python/test_support/sql/people.json", lineSep=",") 

expected = [Row(_corrupt_record=None, name=u'Michael'), 

Row(_corrupt_record=u' "age":30}\n{"name":"Justin"', name=None), 

Row(_corrupt_record=u' "age":19}\n', name=None)] 

self.assertEqual(df.collect(), expected) 

 

tpath = tempfile.mkdtemp() 

shutil.rmtree(tpath) 

try: 

df = self.spark.read.json("python/test_support/sql/people.json") 

df.write.json(tpath, lineSep="!!") 

readback = self.spark.read.json(tpath, lineSep="!!") 

self.assertEqual(readback.collect(), df.collect()) 

finally: 

shutil.rmtree(tpath) 

 

def test_multiline_csv(self): 

ages_newlines = self.spark.read.csv( 

"python/test_support/sql/ages_newlines.csv", multiLine=True) 

expected = [Row(_c0=u'Joe', _c1=u'20', _c2=u'Hi,\nI am Jeo'), 

Row(_c0=u'Tom', _c1=u'30', _c2=u'My name is Tom'), 

Row(_c0=u'Hyukjin', _c1=u'25', _c2=u'I am Hyukjin\n\nI love Spark!')] 

self.assertEqual(ages_newlines.collect(), expected) 

 

def test_ignorewhitespace_csv(self): 

tmpPath = tempfile.mkdtemp() 

shutil.rmtree(tmpPath) 

self.spark.createDataFrame([[" a", "b ", " c "]]).write.csv( 

tmpPath, 

ignoreLeadingWhiteSpace=False, 

ignoreTrailingWhiteSpace=False) 

 

expected = [Row(value=u' a,b , c ')] 

readback = self.spark.read.text(tmpPath) 

self.assertEqual(readback.collect(), expected) 

shutil.rmtree(tmpPath) 

 

def test_read_multiple_orc_file(self): 

df = self.spark.read.orc(["python/test_support/sql/orc_partitioned/b=0/c=0", 

"python/test_support/sql/orc_partitioned/b=1/c=1"]) 

self.assertEqual(2, df.count()) 

 

def test_read_text_file_list(self): 

df = self.spark.read.text(['python/test_support/sql/text-test.txt', 

'python/test_support/sql/text-test.txt']) 

count = df.count() 

self.assertEqual(count, 4) 

 

def test_json_sampling_ratio(self): 

rdd = self.spark.sparkContext.range(0, 100, 1, 1) \ 

.map(lambda x: '{"a":0.1}' if x == 1 else '{"a":%s}' % str(x)) 

schema = self.spark.read.option('inferSchema', True) \ 

.option('samplingRatio', 0.5) \ 

.json(rdd).schema 

self.assertEqual(schema, StructType([StructField("a", LongType(), True)])) 

 

def test_csv_sampling_ratio(self): 

rdd = self.spark.sparkContext.range(0, 100, 1, 1) \ 

.map(lambda x: '0.1' if x == 1 else str(x)) 

schema = self.spark.read.option('inferSchema', True)\ 

.csv(rdd, samplingRatio=0.5).schema 

self.assertEqual(schema, StructType([StructField("_c0", IntegerType(), True)])) 

 

def test_checking_csv_header(self): 

path = tempfile.mkdtemp() 

shutil.rmtree(path) 

try: 

self.spark.createDataFrame([[1, 1000], [2000, 2]])\ 

.toDF('f1', 'f2').write.option("header", "true").csv(path) 

schema = StructType([ 

StructField('f2', IntegerType(), nullable=True), 

StructField('f1', IntegerType(), nullable=True)]) 

df = self.spark.read.option('header', 'true').schema(schema)\ 

.csv(path, enforceSchema=False) 

self.assertRaisesRegex( 

Exception, 

"CSV header does not conform to the schema", 

lambda: df.collect()) 

finally: 

shutil.rmtree(path) 

 

def test_ignore_column_of_all_nulls(self): 

path = tempfile.mkdtemp() 

shutil.rmtree(path) 

try: 

df = self.spark.createDataFrame([["""{"a":null, "b":1, "c":3.0}"""], 

["""{"a":null, "b":null, "c":"string"}"""], 

["""{"a":null, "b":null, "c":null}"""]]) 

df.write.text(path) 

schema = StructType([ 

StructField('b', LongType(), nullable=True), 

StructField('c', StringType(), nullable=True)]) 

readback = self.spark.read.json(path, dropFieldIfAllNull=True) 

self.assertEqual(readback.schema, schema) 

finally: 

shutil.rmtree(path) 

 

 

if __name__ == "__main__": 

import unittest 

from pyspark.sql.tests.test_datasources import * # noqa: F401 

 

try: 

import xmlrunner # type: ignore[import] 

testRunner = xmlrunner.XMLTestRunner(output='target/test-reports', verbosity=2) 

except ImportError: 

testRunner = None 

unittest.main(testRunner=testRunner, verbosity=2)