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

172

173

174

175

176

177

178

179

180

181

182

183

184

185

186

187

188

189

190

191

192

193

194

195

196

197

198

199

200

201

202

203

204

205

206

207

208

209

210

211

212

213

214

215

216

217

218

219

220

221

222

223

224

225

226

227

228

229

230

231

232

233

234

235

236

237

238

239

240

241

242

243

244

245

246

247

248

249

250

251

252

253

254

255

256

257

258

259

260

261

262

263

264

265

266

267

268

269

270

271

272

273

274

275

276

277

278

279

280

281

282

283

284

285

286

287

288

289

290

291

292

293

294

295

296

297

298

299

300

301

302

303

304

305

306

307

308

309

310

311

312

313

314

# 

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

import shutil 

import tempfile 

import unittest 

 

from pyspark.testing.utils import ReusedPySparkTestCase, SPARK_HOME 

 

 

class InputFormatTests(ReusedPySparkTestCase): 

 

@classmethod 

def setUpClass(cls): 

ReusedPySparkTestCase.setUpClass() 

cls.tempdir = tempfile.NamedTemporaryFile(delete=False) 

os.unlink(cls.tempdir.name) 

cls.sc._jvm.WriteInputFormatTestDataGenerator.generateData(cls.tempdir.name, cls.sc._jsc) 

 

@classmethod 

def tearDownClass(cls): 

ReusedPySparkTestCase.tearDownClass() 

shutil.rmtree(cls.tempdir.name) 

 

def test_oldhadoop(self): 

basepath = self.tempdir.name 

ints = sorted(self.sc.hadoopFile(basepath + "/sftestdata/sfint/", 

"org.apache.hadoop.mapred.SequenceFileInputFormat", 

"org.apache.hadoop.io.IntWritable", 

"org.apache.hadoop.io.Text").collect()) 

ei = [(1, u'aa'), (1, u'aa'), (2, u'aa'), (2, u'bb'), (2, u'bb'), (3, u'cc')] 

self.assertEqual(ints, ei) 

 

hellopath = os.path.join(SPARK_HOME, "python/test_support/hello/hello.txt") 

oldconf = {"mapreduce.input.fileinputformat.inputdir": hellopath} 

hello = self.sc.hadoopRDD("org.apache.hadoop.mapred.TextInputFormat", 

"org.apache.hadoop.io.LongWritable", 

"org.apache.hadoop.io.Text", 

conf=oldconf).collect() 

result = [(0, u'Hello World!')] 

self.assertEqual(hello, result) 

 

def test_newhadoop(self): 

basepath = self.tempdir.name 

ints = sorted(self.sc.newAPIHadoopFile( 

basepath + "/sftestdata/sfint/", 

"org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat", 

"org.apache.hadoop.io.IntWritable", 

"org.apache.hadoop.io.Text").collect()) 

ei = [(1, u'aa'), (1, u'aa'), (2, u'aa'), (2, u'bb'), (2, u'bb'), (3, u'cc')] 

self.assertEqual(ints, ei) 

 

hellopath = os.path.join(SPARK_HOME, "python/test_support/hello/hello.txt") 

newconf = {"mapreduce.input.fileinputformat.inputdir": hellopath} 

hello = self.sc.newAPIHadoopRDD("org.apache.hadoop.mapreduce.lib.input.TextInputFormat", 

"org.apache.hadoop.io.LongWritable", 

"org.apache.hadoop.io.Text", 

conf=newconf).collect() 

result = [(0, u'Hello World!')] 

self.assertEqual(hello, result) 

 

def test_newolderror(self): 

basepath = self.tempdir.name 

self.assertRaises(Exception, lambda: self.sc.hadoopFile( 

basepath + "/sftestdata/sfint/", 

"org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat", 

"org.apache.hadoop.io.IntWritable", 

"org.apache.hadoop.io.Text")) 

 

self.assertRaises(Exception, lambda: self.sc.newAPIHadoopFile( 

basepath + "/sftestdata/sfint/", 

"org.apache.hadoop.mapred.SequenceFileInputFormat", 

"org.apache.hadoop.io.IntWritable", 

"org.apache.hadoop.io.Text")) 

 

def test_bad_inputs(self): 

basepath = self.tempdir.name 

self.assertRaises(Exception, lambda: self.sc.sequenceFile( 

basepath + "/sftestdata/sfint/", 

"org.apache.hadoop.io.NotValidWritable", 

"org.apache.hadoop.io.Text")) 

self.assertRaises(Exception, lambda: self.sc.hadoopFile( 

basepath + "/sftestdata/sfint/", 

"org.apache.hadoop.mapred.NotValidInputFormat", 

"org.apache.hadoop.io.IntWritable", 

"org.apache.hadoop.io.Text")) 

self.assertRaises(Exception, lambda: self.sc.newAPIHadoopFile( 

basepath + "/sftestdata/sfint/", 

"org.apache.hadoop.mapreduce.lib.input.NotValidInputFormat", 

"org.apache.hadoop.io.IntWritable", 

"org.apache.hadoop.io.Text")) 

 

def test_converters(self): 

# use of custom converters 

basepath = self.tempdir.name 

maps = sorted(self.sc.sequenceFile( 

basepath + "/sftestdata/sfmap/", 

"org.apache.hadoop.io.IntWritable", 

"org.apache.hadoop.io.MapWritable", 

keyConverter="org.apache.spark.api.python.TestInputKeyConverter", 

valueConverter="org.apache.spark.api.python.TestInputValueConverter").collect()) 

em = [(u'\x01', []), 

(u'\x01', [3.0]), 

(u'\x02', [1.0]), 

(u'\x02', [1.0]), 

(u'\x03', [2.0])] 

self.assertEqual(maps, em) 

 

def test_binary_files(self): 

path = os.path.join(self.tempdir.name, "binaryfiles") 

os.mkdir(path) 

data = b"short binary data" 

with open(os.path.join(path, "part-0000"), 'wb') as f: 

f.write(data) 

[(p, d)] = self.sc.binaryFiles(path).collect() 

self.assertTrue(p.endswith("part-0000")) 

self.assertEqual(d, data) 

 

def test_binary_records(self): 

path = os.path.join(self.tempdir.name, "binaryrecords") 

os.mkdir(path) 

with open(os.path.join(path, "part-0000"), 'w') as f: 

for i in range(100): 

f.write('%04d' % i) 

result = self.sc.binaryRecords(path, 4).map(int).collect() 

self.assertEqual(list(range(100)), result) 

 

 

class OutputFormatTests(ReusedPySparkTestCase): 

 

def setUp(self): 

self.tempdir = tempfile.NamedTemporaryFile(delete=False) 

os.unlink(self.tempdir.name) 

 

def tearDown(self): 

shutil.rmtree(self.tempdir.name, ignore_errors=True) 

 

def test_oldhadoop(self): 

basepath = self.tempdir.name 

dict_data = [(1, {}), 

(1, {"row1": 1.0}), 

(2, {"row2": 2.0})] 

self.sc.parallelize(dict_data).saveAsHadoopFile( 

basepath + "/oldhadoop/", 

"org.apache.hadoop.mapred.SequenceFileOutputFormat", 

"org.apache.hadoop.io.IntWritable", 

"org.apache.hadoop.io.MapWritable") 

result = self.sc.hadoopFile( 

basepath + "/oldhadoop/", 

"org.apache.hadoop.mapred.SequenceFileInputFormat", 

"org.apache.hadoop.io.IntWritable", 

"org.apache.hadoop.io.MapWritable").collect() 

for v in result: 

self.assertTrue(v, dict_data) 

 

conf = { 

"mapred.output.format.class": "org.apache.hadoop.mapred.SequenceFileOutputFormat", 

"mapreduce.job.output.key.class": "org.apache.hadoop.io.IntWritable", 

"mapreduce.job.output.value.class": "org.apache.hadoop.io.MapWritable", 

"mapreduce.output.fileoutputformat.outputdir": basepath + "/olddataset/" 

} 

self.sc.parallelize(dict_data).saveAsHadoopDataset(conf) 

input_conf = {"mapreduce.input.fileinputformat.inputdir": basepath + "/olddataset/"} 

result = self.sc.hadoopRDD( 

"org.apache.hadoop.mapred.SequenceFileInputFormat", 

"org.apache.hadoop.io.IntWritable", 

"org.apache.hadoop.io.MapWritable", 

conf=input_conf).collect() 

for v in result: 

self.assertTrue(v, dict_data) 

 

def test_newhadoop(self): 

basepath = self.tempdir.name 

data = [(1, ""), 

(1, "a"), 

(2, "bcdf")] 

self.sc.parallelize(data).saveAsNewAPIHadoopFile( 

basepath + "/newhadoop/", 

"org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat", 

"org.apache.hadoop.io.IntWritable", 

"org.apache.hadoop.io.Text") 

result = sorted(self.sc.newAPIHadoopFile( 

basepath + "/newhadoop/", 

"org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat", 

"org.apache.hadoop.io.IntWritable", 

"org.apache.hadoop.io.Text").collect()) 

self.assertEqual(result, data) 

 

conf = { 

"mapreduce.job.outputformat.class": 

"org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat", 

"mapreduce.job.output.key.class": "org.apache.hadoop.io.IntWritable", 

"mapreduce.job.output.value.class": "org.apache.hadoop.io.Text", 

"mapreduce.output.fileoutputformat.outputdir": basepath + "/newdataset/" 

} 

self.sc.parallelize(data).saveAsNewAPIHadoopDataset(conf) 

input_conf = {"mapreduce.input.fileinputformat.inputdir": basepath + "/newdataset/"} 

new_dataset = sorted(self.sc.newAPIHadoopRDD( 

"org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat", 

"org.apache.hadoop.io.IntWritable", 

"org.apache.hadoop.io.Text", 

conf=input_conf).collect()) 

self.assertEqual(new_dataset, data) 

 

def test_newolderror(self): 

basepath = self.tempdir.name 

rdd = self.sc.parallelize(range(1, 4)).map(lambda x: (x, "a" * x)) 

self.assertRaises(Exception, lambda: rdd.saveAsHadoopFile( 

basepath + "/newolderror/saveAsHadoopFile/", 

"org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat")) 

self.assertRaises(Exception, lambda: rdd.saveAsNewAPIHadoopFile( 

basepath + "/newolderror/saveAsNewAPIHadoopFile/", 

"org.apache.hadoop.mapred.SequenceFileOutputFormat")) 

 

def test_bad_inputs(self): 

basepath = self.tempdir.name 

rdd = self.sc.parallelize(range(1, 4)).map(lambda x: (x, "a" * x)) 

self.assertRaises(Exception, lambda: rdd.saveAsHadoopFile( 

basepath + "/badinputs/saveAsHadoopFile/", 

"org.apache.hadoop.mapred.NotValidOutputFormat")) 

self.assertRaises(Exception, lambda: rdd.saveAsNewAPIHadoopFile( 

basepath + "/badinputs/saveAsNewAPIHadoopFile/", 

"org.apache.hadoop.mapreduce.lib.output.NotValidOutputFormat")) 

 

def test_converters(self): 

# use of custom converters 

basepath = self.tempdir.name 

data = [(1, {3.0: u'bb'}), 

(2, {1.0: u'aa'}), 

(3, {2.0: u'dd'})] 

self.sc.parallelize(data).saveAsNewAPIHadoopFile( 

basepath + "/converters/", 

"org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat", 

keyConverter="org.apache.spark.api.python.TestOutputKeyConverter", 

valueConverter="org.apache.spark.api.python.TestOutputValueConverter") 

converted = sorted(self.sc.sequenceFile(basepath + "/converters/").collect()) 

expected = [(u'1', 3.0), 

(u'2', 1.0), 

(u'3', 2.0)] 

self.assertEqual(converted, expected) 

 

def test_reserialization(self): 

basepath = self.tempdir.name 

x = range(1, 5) 

y = range(1001, 1005) 

data = list(zip(x, y)) 

rdd = self.sc.parallelize(x).zip(self.sc.parallelize(y)) 

rdd.saveAsSequenceFile(basepath + "/reserialize/sequence") 

result1 = sorted(self.sc.sequenceFile(basepath + "/reserialize/sequence").collect()) 

self.assertEqual(result1, data) 

 

rdd.saveAsHadoopFile( 

basepath + "/reserialize/hadoop", 

"org.apache.hadoop.mapred.SequenceFileOutputFormat") 

result2 = sorted(self.sc.sequenceFile(basepath + "/reserialize/hadoop").collect()) 

self.assertEqual(result2, data) 

 

rdd.saveAsNewAPIHadoopFile( 

basepath + "/reserialize/newhadoop", 

"org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat") 

result3 = sorted(self.sc.sequenceFile(basepath + "/reserialize/newhadoop").collect()) 

self.assertEqual(result3, data) 

 

conf4 = { 

"mapred.output.format.class": "org.apache.hadoop.mapred.SequenceFileOutputFormat", 

"mapreduce.job.output.key.class": "org.apache.hadoop.io.IntWritable", 

"mapreduce.job.output.value.class": "org.apache.hadoop.io.IntWritable", 

"mapreduce.output.fileoutputformat.outputdir": basepath + "/reserialize/dataset"} 

rdd.saveAsHadoopDataset(conf4) 

result4 = sorted(self.sc.sequenceFile(basepath + "/reserialize/dataset").collect()) 

self.assertEqual(result4, data) 

 

conf5 = {"mapreduce.job.outputformat.class": 

"org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat", 

"mapreduce.job.output.key.class": "org.apache.hadoop.io.IntWritable", 

"mapreduce.job.output.value.class": "org.apache.hadoop.io.IntWritable", 

"mapreduce.output.fileoutputformat.outputdir": basepath + "/reserialize/newdataset" 

} 

rdd.saveAsNewAPIHadoopDataset(conf5) 

result5 = sorted(self.sc.sequenceFile(basepath + "/reserialize/newdataset").collect()) 

self.assertEqual(result5, data) 

 

def test_malformed_RDD(self): 

basepath = self.tempdir.name 

# non-batch-serialized RDD[[(K, V)]] should be rejected 

data = [[(1, "a")], [(2, "aa")], [(3, "aaa")]] 

rdd = self.sc.parallelize(data, len(data)) 

self.assertRaises(Exception, lambda: rdd.saveAsSequenceFile( 

basepath + "/malformed/sequence")) 

 

 

if __name__ == "__main__": 

from pyspark.tests.test_readwrite 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)