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

315

316

317

318

319

320

321

322

323

324

325

326

327

328

329

330

331

332

333

334

335

336

337

338

339

340

341

342

343

344

345

346

347

348

349

350

351

352

353

354

355

356

357

358

359

360

361

362

363

364

365

366

367

368

369

370

371

372

373

374

375

376

377

378

379

380

381

382

383

384

385

386

387

388

389

390

391

392

393

394

395

396

397

398

399

400

401

402

403

404

405

406

407

408

409

410

411

412

413

414

415

416

417

418

419

420

421

422

423

424

425

426

427

428

429

430

431

432

433

434

435

436

437

438

439

440

441

442

443

444

445

446

447

448

449

450

451

452

453

454

455

456

457

458

459

460

461

462

463

464

465

466

467

468

469

470

471

472

473

474

475

476

477

478

479

480

481

482

483

484

485

486

487

488

489

490

491

492

493

494

495

496

497

498

499

500

501

502

503

504

505

506

507

508

509

510

511

512

513

514

515

516

517

518

519

520

521

522

523

524

525

526

527

528

529

530

531

532

533

534

535

536

537

538

539

540

541

542

543

544

545

546

547

548

549

550

551

552

553

554

555

556

557

558

559

560

561

562

563

564

565

566

567

568

569

570

571

572

573

574

575

576

577

578

579

580

581

582

583

584

585

586

587

588

589

590

591

592

593

594

595

596

597

598

599

600

601

602

603

604

605

606

607

608

609

610

611

612

613

614

615

616

617

618

619

620

621

622

623

624

625

626

627

628

629

630

631

632

633

634

635

636

637

638

639

640

641

642

643

644

645

646

647

648

649

650

651

652

653

654

655

656

657

658

659

660

661

662

663

664

665

666

667

668

669

670

671

672

673

674

675

676

677

678

679

680

681

682

683

684

685

686

687

688

689

690

691

692

693

694

695

696

697

698

699

700

701

702

703

704

705

706

707

708

709

710

711

712

713

714

715

716

717

718

719

720

721

722

723

724

725

726

727

728

729

730

731

732

733

734

735

736

737

738

739

740

741

742

743

744

745

746

747

748

749

750

751

752

753

754

755

756

757

758

759

760

761

762

763

764

765

766

767

768

769

770

771

772

773

774

775

776

777

778

779

780

781

782

783

784

785

786

787

788

789

790

791

792

793

794

795

796

797

798

799

800

801

802

803

804

805

806

807

808

809

810

811

812

813

814

815

816

817

818

819

820

821

822

823

824

825

826

827

828

829

830

831

832

833

834

835

836

837

838

839

840

841

842

843

844

845

846

847

848

849

850

851

852

853

854

# 

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

import warnings 

from functools import reduce 

from threading import RLock 

 

from pyspark import since 

from pyspark.rdd import RDD 

from pyspark.sql.conf import RuntimeConfig 

from pyspark.sql.dataframe import DataFrame 

from pyspark.sql.pandas.conversion import SparkConversionMixin 

from pyspark.sql.readwriter import DataFrameReader 

from pyspark.sql.streaming import DataStreamReader 

from pyspark.sql.types import DataType, StructType, \ 

_make_type_verifier, _infer_schema, _has_nulltype, _merge_type, _create_converter, \ 

_parse_datatype_string 

from pyspark.sql.utils import install_exception_handler 

 

__all__ = ["SparkSession"] 

 

 

def _monkey_patch_RDD(sparkSession): 

def toDF(self, schema=None, sampleRatio=None): 

""" 

Converts current :class:`RDD` into a :class:`DataFrame` 

 

This is a shorthand for ``spark.createDataFrame(rdd, schema, sampleRatio)`` 

 

Parameters 

---------- 

schema : :class:`pyspark.sql.types.DataType`, str or list, optional 

a :class:`pyspark.sql.types.DataType` or a datatype string or a list of 

column names, default is None. The data type string format equals to 

:class:`pyspark.sql.types.DataType.simpleString`, except that top level struct type can 

omit the ``struct<>`` and atomic types use ``typeName()`` as their format, e.g. use 

``byte`` instead of ``tinyint`` for :class:`pyspark.sql.types.ByteType`. 

We can also use ``int`` as a short name for :class:`pyspark.sql.types.IntegerType`. 

sampleRatio : float, optional 

the sample ratio of rows used for inferring 

 

Returns 

------- 

:class:`DataFrame` 

 

Examples 

-------- 

>>> rdd.toDF().collect() 

[Row(name='Alice', age=1)] 

""" 

return sparkSession.createDataFrame(self, schema, sampleRatio) 

 

RDD.toDF = toDF 

 

 

class SparkSession(SparkConversionMixin): 

"""The entry point to programming Spark with the Dataset and DataFrame API. 

 

A SparkSession can be used create :class:`DataFrame`, register :class:`DataFrame` as 

tables, execute SQL over tables, cache tables, and read parquet files. 

To create a :class:`SparkSession`, use the following builder pattern: 

 

.. autoattribute:: builder 

:annotation: 

 

Examples 

-------- 

>>> spark = SparkSession.builder \\ 

... .master("local") \\ 

... .appName("Word Count") \\ 

... .config("spark.some.config.option", "some-value") \\ 

... .getOrCreate() 

 

>>> from datetime import datetime 

>>> from pyspark.sql import Row 

>>> spark = SparkSession(sc) 

>>> allTypes = sc.parallelize([Row(i=1, s="string", d=1.0, l=1, 

... b=True, list=[1, 2, 3], dict={"s": 0}, row=Row(a=1), 

... time=datetime(2014, 8, 1, 14, 1, 5))]) 

>>> df = allTypes.toDF() 

>>> df.createOrReplaceTempView("allTypes") 

>>> spark.sql('select i+1, d+1, not b, list[1], dict["s"], time, row.a ' 

... 'from allTypes where b and i > 0').collect() 

[Row((i + 1)=2, (d + 1)=2.0, (NOT b)=False, list[1]=2, \ 

dict[s]=0, time=datetime.datetime(2014, 8, 1, 14, 1, 5), a=1)] 

>>> df.rdd.map(lambda x: (x.i, x.s, x.d, x.l, x.b, x.time, x.row.a, x.list)).collect() 

[(1, 'string', 1.0, 1, True, datetime.datetime(2014, 8, 1, 14, 1, 5), 1, [1, 2, 3])] 

""" 

 

class Builder(object): 

"""Builder for :class:`SparkSession`. 

""" 

 

_lock = RLock() 

_options = {} 

_sc = None 

 

def config(self, key=None, value=None, conf=None): 

"""Sets a config option. Options set using this method are automatically propagated to 

both :class:`SparkConf` and :class:`SparkSession`'s own configuration. 

 

.. versionadded:: 2.0.0 

 

Parameters 

---------- 

key : str, optional 

a key name string for configuration property 

value : str, optional 

a value for configuration property 

conf : :class:`SparkConf`, optional 

an instance of :class:`SparkConf` 

 

Examples 

-------- 

For an existing SparkConf, use `conf` parameter. 

 

>>> from pyspark.conf import SparkConf 

>>> SparkSession.builder.config(conf=SparkConf()) 

<pyspark.sql.session... 

 

For a (key, value) pair, you can omit parameter names. 

 

>>> SparkSession.builder.config("spark.some.config.option", "some-value") 

<pyspark.sql.session... 

 

""" 

with self._lock: 

if conf is None: 

self._options[key] = str(value) 

else: 

for (k, v) in conf.getAll(): 

self._options[k] = v 

return self 

 

def master(self, master): 

"""Sets the Spark master URL to connect to, such as "local" to run locally, "local[4]" 

to run locally with 4 cores, or "spark://master:7077" to run on a Spark standalone 

cluster. 

 

.. versionadded:: 2.0.0 

 

Parameters 

---------- 

master : str 

a url for spark master 

""" 

return self.config("spark.master", master) 

 

def appName(self, name): 

"""Sets a name for the application, which will be shown in the Spark web UI. 

 

If no application name is set, a randomly generated name will be used. 

 

.. versionadded:: 2.0.0 

 

Parameters 

---------- 

name : str 

an application name 

""" 

return self.config("spark.app.name", name) 

 

@since(2.0) 

def enableHiveSupport(self): 

"""Enables Hive support, including connectivity to a persistent Hive metastore, support 

for Hive SerDes, and Hive user-defined functions. 

""" 

return self.config("spark.sql.catalogImplementation", "hive") 

 

def _sparkContext(self, sc): 

with self._lock: 

self._sc = sc 

return self 

 

def getOrCreate(self): 

"""Gets an existing :class:`SparkSession` or, if there is no existing one, creates a 

new one based on the options set in this builder. 

 

.. versionadded:: 2.0.0 

 

Examples 

-------- 

This method first checks whether there is a valid global default SparkSession, and if 

yes, return that one. If no valid global default SparkSession exists, the method 

creates a new SparkSession and assigns the newly created SparkSession as the global 

default. 

 

>>> s1 = SparkSession.builder.config("k1", "v1").getOrCreate() 

>>> s1.conf.get("k1") == "v1" 

True 

 

In case an existing SparkSession is returned, the config options specified 

in this builder will be applied to the existing SparkSession. 

 

>>> s2 = SparkSession.builder.config("k2", "v2").getOrCreate() 

>>> s1.conf.get("k1") == s2.conf.get("k1") 

True 

>>> s1.conf.get("k2") == s2.conf.get("k2") 

True 

""" 

with self._lock: 

from pyspark.context import SparkContext 

from pyspark.conf import SparkConf 

session = SparkSession._instantiatedSession 

if session is None or session._sc._jsc is None: 

221 ↛ 222line 221 didn't jump to line 222, because the condition on line 221 was never true if self._sc is not None: 

sc = self._sc 

else: 

sparkConf = SparkConf() 

for key, value in self._options.items(): 

sparkConf.set(key, value) 

# This SparkContext may be an existing one. 

sc = SparkContext.getOrCreate(sparkConf) 

# Do not update `SparkConf` for existing `SparkContext`, as it's shared 

# by all sessions. 

session = SparkSession(sc) 

for key, value in self._options.items(): 

session._jsparkSession.sessionState().conf().setConfString(key, value) 

return session 

 

builder = Builder() 

"""A class attribute having a :class:`Builder` to construct :class:`SparkSession` instances.""" 

 

_instantiatedSession = None 

_activeSession = None 

 

def __init__(self, sparkContext, jsparkSession=None): 

from pyspark.sql.context import SQLContext 

self._sc = sparkContext 

self._jsc = self._sc._jsc 

self._jvm = self._sc._jvm 

if jsparkSession is None: 

if self._jvm.SparkSession.getDefaultSession().isDefined() \ 

and not self._jvm.SparkSession.getDefaultSession().get() \ 

.sparkContext().isStopped(): 

jsparkSession = self._jvm.SparkSession.getDefaultSession().get() 

else: 

jsparkSession = self._jvm.SparkSession(self._jsc.sc()) 

self._jsparkSession = jsparkSession 

self._jwrapped = self._jsparkSession.sqlContext() 

self._wrapped = SQLContext(self._sc, self, self._jwrapped) 

_monkey_patch_RDD(self) 

install_exception_handler() 

# If we had an instantiated SparkSession attached with a SparkContext 

# which is stopped now, we need to renew the instantiated SparkSession. 

# Otherwise, we will use invalid SparkSession when we call Builder.getOrCreate. 

if SparkSession._instantiatedSession is None \ 

or SparkSession._instantiatedSession._sc._jsc is None: 

SparkSession._instantiatedSession = self 

SparkSession._activeSession = self 

self._jvm.SparkSession.setDefaultSession(self._jsparkSession) 

self._jvm.SparkSession.setActiveSession(self._jsparkSession) 

 

def _repr_html_(self): 

return """ 

<div> 

<p><b>SparkSession - {catalogImplementation}</b></p> 

{sc_HTML} 

</div> 

""".format( 

catalogImplementation=self.conf.get("spark.sql.catalogImplementation"), 

sc_HTML=self.sparkContext._repr_html_() 

) 

 

@since(2.0) 

def newSession(self): 

""" 

Returns a new :class:`SparkSession` as new session, that has separate SQLConf, 

registered temporary views and UDFs, but shared :class:`SparkContext` and 

table cache. 

""" 

return self.__class__(self._sc, self._jsparkSession.newSession()) 

 

@classmethod 

def getActiveSession(cls): 

""" 

Returns the active :class:`SparkSession` for the current thread, returned by the builder 

 

.. versionadded:: 3.0.0 

 

Returns 

------- 

:class:`SparkSession` 

Spark session if an active session exists for the current thread 

 

Examples 

-------- 

>>> s = SparkSession.getActiveSession() 

>>> l = [('Alice', 1)] 

>>> rdd = s.sparkContext.parallelize(l) 

>>> df = s.createDataFrame(rdd, ['name', 'age']) 

>>> df.select("age").collect() 

[Row(age=1)] 

""" 

from pyspark import SparkContext 

sc = SparkContext._active_spark_context 

if sc is None: 

return None 

else: 

315 ↛ 319line 315 didn't jump to line 319, because the condition on line 315 was never false if sc._jvm.SparkSession.getActiveSession().isDefined(): 

SparkSession(sc, sc._jvm.SparkSession.getActiveSession().get()) 

return SparkSession._activeSession 

else: 

return None 

 

@property 

@since(2.0) 

def sparkContext(self): 

"""Returns the underlying :class:`SparkContext`.""" 

return self._sc 

 

@property 

@since(2.0) 

def version(self): 

"""The version of Spark on which this application is running.""" 

return self._jsparkSession.version() 

 

@property 

@since(2.0) 

def conf(self): 

"""Runtime configuration interface for Spark. 

 

This is the interface through which the user can get and set all Spark and Hadoop 

configurations that are relevant to Spark SQL. When getting the value of a config, 

this defaults to the value set in the underlying :class:`SparkContext`, if any. 

 

Returns 

------- 

:class:`pyspark.sql.conf.RuntimeConfig` 

""" 

if not hasattr(self, "_conf"): 

self._conf = RuntimeConfig(self._jsparkSession.conf()) 

return self._conf 

 

@property 

def catalog(self): 

"""Interface through which the user may create, drop, alter or query underlying 

databases, tables, functions, etc. 

 

.. versionadded:: 2.0.0 

 

Returns 

------- 

:class:`Catalog` 

""" 

from pyspark.sql.catalog import Catalog 

if not hasattr(self, "_catalog"): 

self._catalog = Catalog(self) 

return self._catalog 

 

@property 

def udf(self): 

"""Returns a :class:`UDFRegistration` for UDF registration. 

 

.. versionadded:: 2.0.0 

 

Returns 

------- 

:class:`UDFRegistration` 

""" 

from pyspark.sql.udf import UDFRegistration 

return UDFRegistration(self) 

 

def range(self, start, end=None, step=1, numPartitions=None): 

""" 

Create a :class:`DataFrame` with single :class:`pyspark.sql.types.LongType` column named 

``id``, containing elements in a range from ``start`` to ``end`` (exclusive) with 

step value ``step``. 

 

.. versionadded:: 2.0.0 

 

Parameters 

---------- 

start : int 

the start value 

end : int, optional 

the end value (exclusive) 

step : int, optional 

the incremental step (default: 1) 

numPartitions : int, optional 

the number of partitions of the DataFrame 

 

Returns 

------- 

:class:`DataFrame` 

 

Examples 

-------- 

>>> spark.range(1, 7, 2).collect() 

[Row(id=1), Row(id=3), Row(id=5)] 

 

If only one argument is specified, it will be used as the end value. 

 

>>> spark.range(3).collect() 

[Row(id=0), Row(id=1), Row(id=2)] 

""" 

if numPartitions is None: 

numPartitions = self._sc.defaultParallelism 

 

if end is None: 

jdf = self._jsparkSession.range(0, int(start), int(step), int(numPartitions)) 

else: 

jdf = self._jsparkSession.range(int(start), int(end), int(step), int(numPartitions)) 

 

return DataFrame(jdf, self._wrapped) 

 

def _inferSchemaFromList(self, data, names=None): 

""" 

Infer schema from list of Row, dict, or tuple. 

 

Parameters 

---------- 

data : iterable 

list of Row, dict, or tuple 

names : list, optional 

list of column names 

 

Returns 

------- 

:class:`pyspark.sql.types.StructType` 

""" 

437 ↛ 438line 437 didn't jump to line 438, because the condition on line 437 was never true if not data: 

raise ValueError("can not infer schema from empty dataset") 

infer_dict_as_struct = self._wrapped._conf.inferDictAsStruct() 

schema = reduce(_merge_type, (_infer_schema(row, names, infer_dict_as_struct) 

for row in data)) 

442 ↛ 443line 442 didn't jump to line 443, because the condition on line 442 was never true if _has_nulltype(schema): 

raise ValueError("Some of types cannot be determined after inferring") 

return schema 

 

def _inferSchema(self, rdd, samplingRatio=None, names=None): 

""" 

Infer schema from an RDD of Row, dict, or tuple. 

 

Parameters 

---------- 

rdd : :class:`RDD` 

an RDD of Row, dict, or tuple 

samplingRatio : float, optional 

sampling ratio, or no sampling (default) 

names : list, optional 

 

Returns 

------- 

:class:`pyspark.sql.types.StructType` 

""" 

first = rdd.first() 

463 ↛ 464line 463 didn't jump to line 464, because the condition on line 463 was never true if not first: 

raise ValueError("The first row in RDD is empty, " 

"can not infer schema") 

 

infer_dict_as_struct = self._wrapped._conf.inferDictAsStruct() 

if samplingRatio is None: 

schema = _infer_schema(first, names=names, infer_dict_as_struct=infer_dict_as_struct) 

if _has_nulltype(schema): 

471 ↛ 477line 471 didn't jump to line 477, because the loop on line 471 didn't complete for row in rdd.take(100)[1:]: 

schema = _merge_type(schema, _infer_schema( 

row, names=names, infer_dict_as_struct=infer_dict_as_struct)) 

474 ↛ 471line 474 didn't jump to line 471, because the condition on line 474 was never false if not _has_nulltype(schema): 

break 

else: 

raise ValueError("Some of types cannot be determined by the " 

"first 100 rows, please try again with sampling") 

else: 

480 ↛ 481line 480 didn't jump to line 481, because the condition on line 480 was never true if samplingRatio < 0.99: 

rdd = rdd.sample(False, float(samplingRatio)) 

schema = rdd.map(lambda row: _infer_schema( 

row, names, infer_dict_as_struct=infer_dict_as_struct)).reduce(_merge_type) 

return schema 

 

def _createFromRDD(self, rdd, schema, samplingRatio): 

""" 

Create an RDD for DataFrame from an existing RDD, returns the RDD and schema. 

""" 

if schema is None or isinstance(schema, (list, tuple)): 

struct = self._inferSchema(rdd, samplingRatio, names=schema) 

converter = _create_converter(struct) 

rdd = rdd.map(converter) 

if isinstance(schema, (list, tuple)): 

for i, name in enumerate(schema): 

struct.fields[i].name = name 

struct.names[i] = name 

schema = struct 

 

500 ↛ 501line 500 didn't jump to line 501, because the condition on line 500 was never true elif not isinstance(schema, StructType): 

raise TypeError("schema should be StructType or list or None, but got: %s" % schema) 

 

# convert python objects to sql data 

rdd = rdd.map(schema.toInternal) 

return rdd, schema 

 

def _createFromLocal(self, data, schema): 

""" 

Create an RDD for DataFrame from a list or pandas.DataFrame, returns 

the RDD and schema. 

""" 

# make sure data could consumed multiple times 

513 ↛ 516line 513 didn't jump to line 516, because the condition on line 513 was never false if not isinstance(data, list): 

data = list(data) 

 

if schema is None or isinstance(schema, (list, tuple)): 

struct = self._inferSchemaFromList(data, names=schema) 

converter = _create_converter(struct) 

data = map(converter, data) 

if isinstance(schema, (list, tuple)): 

for i, name in enumerate(schema): 

struct.fields[i].name = name 

struct.names[i] = name 

schema = struct 

 

526 ↛ 527line 526 didn't jump to line 527, because the condition on line 526 was never true elif not isinstance(schema, StructType): 

raise TypeError("schema should be StructType or list or None, but got: %s" % schema) 

 

# convert python objects to sql data 

data = [schema.toInternal(row) for row in data] 

return self._sc.parallelize(data), schema 

 

@staticmethod 

def _create_shell_session(): 

""" 

Initialize a :class:`SparkSession` for a pyspark shell session. This is called from 

shell.py to make error handling simpler without needing to declare local variables in 

that script, which would expose those to users. 

""" 

import py4j 

from pyspark.conf import SparkConf 

from pyspark.context import SparkContext 

try: 

# Try to access HiveConf, it will raise exception if Hive is not added 

conf = SparkConf() 

if conf.get('spark.sql.catalogImplementation', 'hive').lower() == 'hive': 

SparkContext._jvm.org.apache.hadoop.hive.conf.HiveConf() 

return SparkSession.builder\ 

.enableHiveSupport()\ 

.getOrCreate() 

else: 

return SparkSession.builder.getOrCreate() 

except (py4j.protocol.Py4JError, TypeError): 

if conf.get('spark.sql.catalogImplementation', '').lower() == 'hive': 

warnings.warn("Fall back to non-hive support because failing to access HiveConf, " 

"please make sure you build spark with hive") 

 

return SparkSession.builder.getOrCreate() 

 

def createDataFrame(self, data, schema=None, samplingRatio=None, verifySchema=True): 

""" 

Creates a :class:`DataFrame` from an :class:`RDD`, a list or a :class:`pandas.DataFrame`. 

 

When ``schema`` is a list of column names, the type of each column 

will be inferred from ``data``. 

 

When ``schema`` is ``None``, it will try to infer the schema (column names and types) 

from ``data``, which should be an RDD of either :class:`Row`, 

:class:`namedtuple`, or :class:`dict`. 

 

When ``schema`` is :class:`pyspark.sql.types.DataType` or a datatype string, it must match 

the real data, or an exception will be thrown at runtime. If the given schema is not 

:class:`pyspark.sql.types.StructType`, it will be wrapped into a 

:class:`pyspark.sql.types.StructType` as its only field, and the field name will be "value". 

Each record will also be wrapped into a tuple, which can be converted to row later. 

 

If schema inference is needed, ``samplingRatio`` is used to determined the ratio of 

rows used for schema inference. The first row will be used if ``samplingRatio`` is ``None``. 

 

.. versionadded:: 2.0.0 

 

.. versionchanged:: 2.1.0 

Added verifySchema. 

 

Parameters 

---------- 

data : :class:`RDD` or iterable 

an RDD of any kind of SQL data representation (:class:`Row`, 

:class:`tuple`, ``int``, ``boolean``, etc.), or :class:`list`, or 

:class:`pandas.DataFrame`. 

schema : :class:`pyspark.sql.types.DataType`, str or list, optional 

a :class:`pyspark.sql.types.DataType` or a datatype string or a list of 

column names, default is None. The data type string format equals to 

:class:`pyspark.sql.types.DataType.simpleString`, except that top level struct type can 

omit the ``struct<>`` and atomic types use ``typeName()`` as their format, e.g. use 

``byte`` instead of ``tinyint`` for :class:`pyspark.sql.types.ByteType`. 

We can also use ``int`` as a short name for :class:`pyspark.sql.types.IntegerType`. 

samplingRatio : float, optional 

the sample ratio of rows used for inferring 

verifySchema : bool, optional 

verify data types of every row against schema. Enabled by default. 

 

Returns 

------- 

:class:`DataFrame` 

 

Notes 

----- 

Usage with spark.sql.execution.arrow.pyspark.enabled=True is experimental. 

 

Examples 

-------- 

>>> l = [('Alice', 1)] 

>>> spark.createDataFrame(l).collect() 

[Row(_1='Alice', _2=1)] 

>>> spark.createDataFrame(l, ['name', 'age']).collect() 

[Row(name='Alice', age=1)] 

 

>>> d = [{'name': 'Alice', 'age': 1}] 

>>> spark.createDataFrame(d).collect() 

[Row(age=1, name='Alice')] 

 

>>> rdd = sc.parallelize(l) 

>>> spark.createDataFrame(rdd).collect() 

[Row(_1='Alice', _2=1)] 

>>> df = spark.createDataFrame(rdd, ['name', 'age']) 

>>> df.collect() 

[Row(name='Alice', age=1)] 

 

>>> from pyspark.sql import Row 

>>> Person = Row('name', 'age') 

>>> person = rdd.map(lambda r: Person(*r)) 

>>> df2 = spark.createDataFrame(person) 

>>> df2.collect() 

[Row(name='Alice', age=1)] 

 

>>> from pyspark.sql.types import * 

>>> schema = StructType([ 

... StructField("name", StringType(), True), 

... StructField("age", IntegerType(), True)]) 

>>> df3 = spark.createDataFrame(rdd, schema) 

>>> df3.collect() 

[Row(name='Alice', age=1)] 

 

>>> spark.createDataFrame(df.toPandas()).collect() # doctest: +SKIP 

[Row(name='Alice', age=1)] 

>>> spark.createDataFrame(pandas.DataFrame([[1, 2]])).collect() # doctest: +SKIP 

[Row(0=1, 1=2)] 

 

>>> spark.createDataFrame(rdd, "a: string, b: int").collect() 

[Row(a='Alice', b=1)] 

>>> rdd = rdd.map(lambda row: row[1]) 

>>> spark.createDataFrame(rdd, "int").collect() 

[Row(value=1)] 

>>> spark.createDataFrame(rdd, "boolean").collect() # doctest: +IGNORE_EXCEPTION_DETAIL 

Traceback (most recent call last): 

... 

Py4JJavaError: ... 

""" 

SparkSession._activeSession = self 

self._jvm.SparkSession.setActiveSession(self._jsparkSession) 

662 ↛ 663line 662 didn't jump to line 663, because the condition on line 662 was never true if isinstance(data, DataFrame): 

raise TypeError("data is already a DataFrame") 

 

if isinstance(schema, str): 

schema = _parse_datatype_string(schema) 

elif isinstance(schema, (list, tuple)): 

# Must re-encode any unicode strings to be consistent with StructField names 

schema = [x.encode('utf-8') if not isinstance(x, str) else x for x in schema] 

 

try: 

import pandas 

has_pandas = True 

except Exception: 

has_pandas = False 

if has_pandas and isinstance(data, pandas.DataFrame): 

# Create a DataFrame from pandas DataFrame. 

return super(SparkSession, self).createDataFrame( 

data, schema, samplingRatio, verifySchema) 

return self._create_dataframe(data, schema, samplingRatio, verifySchema) 

 

def _create_dataframe(self, data, schema, samplingRatio, verifySchema): 

if isinstance(schema, StructType): 

verify_func = _make_type_verifier(schema) if verifySchema else lambda _: True 

 

def prepare(obj): 

verify_func(obj) 

return obj 

elif isinstance(schema, DataType): 

dataType = schema 

schema = StructType().add("value", schema) 

 

693 ↛ exitline 694 didn't finish the lambda on line 694 verify_func = _make_type_verifier( 

dataType, name="field value") if verifySchema else lambda _: True 

 

def prepare(obj): 

verify_func(obj) 

return obj, 

else: 

prepare = lambda obj: obj 

 

if isinstance(data, RDD): 

rdd, schema = self._createFromRDD(data.map(prepare), schema, samplingRatio) 

else: 

rdd, schema = self._createFromLocal(map(prepare, data), schema) 

jrdd = self._jvm.SerDeUtil.toJavaArray(rdd._to_java_object_rdd()) 

jdf = self._jsparkSession.applySchemaToPythonRDD(jrdd.rdd(), schema.json()) 

df = DataFrame(jdf, self._wrapped) 

df._schema = schema 

return df 

 

def sql(self, sqlQuery): 

"""Returns a :class:`DataFrame` representing the result of the given query. 

 

.. versionadded:: 2.0.0 

 

Returns 

------- 

:class:`DataFrame` 

 

Examples 

-------- 

>>> df.createOrReplaceTempView("table1") 

>>> df2 = spark.sql("SELECT field1 AS f1, field2 as f2 from table1") 

>>> df2.collect() 

[Row(f1=1, f2='row1'), Row(f1=2, f2='row2'), Row(f1=3, f2='row3')] 

""" 

return DataFrame(self._jsparkSession.sql(sqlQuery), self._wrapped) 

 

def table(self, tableName): 

"""Returns the specified table as a :class:`DataFrame`. 

 

.. versionadded:: 2.0.0 

 

Returns 

------- 

:class:`DataFrame` 

 

Examples 

-------- 

>>> df.createOrReplaceTempView("table1") 

>>> df2 = spark.table("table1") 

>>> sorted(df.collect()) == sorted(df2.collect()) 

True 

""" 

return DataFrame(self._jsparkSession.table(tableName), self._wrapped) 

 

@property 

def read(self): 

""" 

Returns a :class:`DataFrameReader` that can be used to read data 

in as a :class:`DataFrame`. 

 

.. versionadded:: 2.0.0 

 

Returns 

------- 

:class:`DataFrameReader` 

""" 

return DataFrameReader(self._wrapped) 

 

@property 

def readStream(self): 

""" 

Returns a :class:`DataStreamReader` that can be used to read data streams 

as a streaming :class:`DataFrame`. 

 

.. versionadded:: 2.0.0 

 

Notes 

----- 

This API is evolving. 

 

Returns 

------- 

:class:`DataStreamReader` 

""" 

return DataStreamReader(self._wrapped) 

 

@property 

def streams(self): 

"""Returns a :class:`StreamingQueryManager` that allows managing all the 

:class:`StreamingQuery` instances active on `this` context. 

 

.. versionadded:: 2.0.0 

 

Notes 

----- 

This API is evolving. 

 

Returns 

------- 

:class:`StreamingQueryManager` 

""" 

from pyspark.sql.streaming import StreamingQueryManager 

return StreamingQueryManager(self._jsparkSession.streams()) 

 

@since(2.0) 

def stop(self): 

"""Stop the underlying :class:`SparkContext`. 

""" 

from pyspark.sql.context import SQLContext 

self._sc.stop() 

# We should clean the default session up. See SPARK-23228. 

self._jvm.SparkSession.clearDefaultSession() 

self._jvm.SparkSession.clearActiveSession() 

SparkSession._instantiatedSession = None 

SparkSession._activeSession = None 

SQLContext._instantiatedContext = None 

 

@since(2.0) 

def __enter__(self): 

""" 

Enable 'with SparkSession.builder.(...).getOrCreate() as session: app' syntax. 

""" 

return self 

 

@since(2.0) 

def __exit__(self, exc_type, exc_val, exc_tb): 

""" 

Enable 'with SparkSession.builder.(...).getOrCreate() as session: app' syntax. 

 

Specifically stop the SparkSession on exit of the with block. 

""" 

self.stop() 

 

 

def _test(): 

import os 

import doctest 

from pyspark.context import SparkContext 

from pyspark.sql import Row 

import pyspark.sql.session 

 

os.chdir(os.environ["SPARK_HOME"]) 

 

globs = pyspark.sql.session.__dict__.copy() 

sc = SparkContext('local[4]', 'PythonTest') 

globs['sc'] = sc 

globs['spark'] = SparkSession(sc) 

globs['rdd'] = rdd = sc.parallelize( 

[Row(field1=1, field2="row1"), 

Row(field1=2, field2="row2"), 

Row(field1=3, field2="row3")]) 

globs['df'] = rdd.toDF() 

(failure_count, test_count) = doctest.testmod( 

pyspark.sql.session, globs=globs, 

optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE) 

globs['sc'].stop() 

850 ↛ 851line 850 didn't jump to line 851, because the condition on line 850 was never true if failure_count: 

sys.exit(-1) 

 

if __name__ == "__main__": 

_test()