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

855

856

857

858

859

860

861

862

863

864

865

866

867

868

869

870

871

872

873

874

875

876

877

878

879

880

881

882

883

884

885

886

887

888

889

890

891

892

893

894

895

896

897

898

899

900

901

902

903

904

905

906

907

908

909

910

911

912

913

914

915

916

917

918

919

920

921

922

923

924

925

926

927

928

929

930

931

932

933

934

935

936

937

938

939

# 

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

import warnings 

 

from pyspark import copy_func 

from pyspark.context import SparkContext 

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

 

__all__ = ["Column"] 

 

 

def _create_column_from_literal(literal): 

sc = SparkContext._active_spark_context 

return sc._jvm.functions.lit(literal) 

 

 

def _create_column_from_name(name): 

sc = SparkContext._active_spark_context 

return sc._jvm.functions.col(name) 

 

 

def _to_java_column(col): 

if isinstance(col, Column): 

jcol = col._jc 

elif isinstance(col, str): 

jcol = _create_column_from_name(col) 

else: 

raise TypeError( 

"Invalid argument, not a string or column: " 

"{0} of type {1}. " 

"For column literals, use 'lit', 'array', 'struct' or 'create_map' " 

"function.".format(col, type(col))) 

return jcol 

 

 

def _to_seq(sc, cols, converter=None): 

""" 

Convert a list of Column (or names) into a JVM Seq of Column. 

 

An optional `converter` could be used to convert items in `cols` 

into JVM Column objects. 

""" 

if converter: 

cols = [converter(c) for c in cols] 

return sc._jvm.PythonUtils.toSeq(cols) 

 

 

def _to_list(sc, cols, converter=None): 

""" 

Convert a list of Column (or names) into a JVM (Scala) List of Column. 

 

An optional `converter` could be used to convert items in `cols` 

into JVM Column objects. 

""" 

72 ↛ 73line 72 didn't jump to line 73, because the condition on line 72 was never true if converter: 

cols = [converter(c) for c in cols] 

return sc._jvm.PythonUtils.toList(cols) 

 

 

def _unary_op(name, doc="unary operator"): 

""" Create a method for given unary operator """ 

def _(self): 

jc = getattr(self._jc, name)() 

return Column(jc) 

_.__doc__ = doc 

return _ 

 

 

def _func_op(name, doc=''): 

def _(self): 

sc = SparkContext._active_spark_context 

jc = getattr(sc._jvm.functions, name)(self._jc) 

return Column(jc) 

_.__doc__ = doc 

return _ 

 

 

def _bin_func_op(name, reverse=False, doc="binary function"): 

def _(self, other): 

sc = SparkContext._active_spark_context 

fn = getattr(sc._jvm.functions, name) 

jc = other._jc if isinstance(other, Column) else _create_column_from_literal(other) 

njc = fn(self._jc, jc) if not reverse else fn(jc, self._jc) 

return Column(njc) 

_.__doc__ = doc 

return _ 

 

 

def _bin_op(name, doc="binary operator"): 

""" Create a method for given binary operator 

""" 

def _(self, other): 

jc = other._jc if isinstance(other, Column) else other 

njc = getattr(self._jc, name)(jc) 

return Column(njc) 

_.__doc__ = doc 

return _ 

 

 

def _reverse_op(name, doc="binary operator"): 

""" Create a method for binary operator (this object is on right side) 

""" 

def _(self, other): 

jother = _create_column_from_literal(other) 

jc = getattr(jother, name)(self._jc) 

return Column(jc) 

_.__doc__ = doc 

return _ 

 

 

class Column(object): 

 

""" 

A column in a DataFrame. 

 

:class:`Column` instances can be created by:: 

 

# 1. Select a column out of a DataFrame 

 

df.colName 

df["colName"] 

 

# 2. Create from an expression 

df.colName + 1 

1 / df.colName 

 

.. versionadded:: 1.3.0 

""" 

 

def __init__(self, jc): 

self._jc = jc 

 

# arithmetic operators 

__neg__ = _func_op("negate") 

__add__ = _bin_op("plus") 

__sub__ = _bin_op("minus") 

__mul__ = _bin_op("multiply") 

__div__ = _bin_op("divide") 

__truediv__ = _bin_op("divide") 

__mod__ = _bin_op("mod") 

__radd__ = _bin_op("plus") 

__rsub__ = _reverse_op("minus") 

__rmul__ = _bin_op("multiply") 

__rdiv__ = _reverse_op("divide") 

__rtruediv__ = _reverse_op("divide") 

__rmod__ = _reverse_op("mod") 

__pow__ = _bin_func_op("pow") 

__rpow__ = _bin_func_op("pow", reverse=True) 

 

# logistic operators 

__eq__ = _bin_op("equalTo") 

__ne__ = _bin_op("notEqual") 

__lt__ = _bin_op("lt") 

__le__ = _bin_op("leq") 

__ge__ = _bin_op("geq") 

__gt__ = _bin_op("gt") 

 

_eqNullSafe_doc = """ 

Equality test that is safe for null values. 

 

.. versionadded:: 2.3.0 

 

Parameters 

---------- 

other 

a value or :class:`Column` 

 

Examples 

-------- 

>>> from pyspark.sql import Row 

>>> df1 = spark.createDataFrame([ 

... Row(id=1, value='foo'), 

... Row(id=2, value=None) 

... ]) 

>>> df1.select( 

... df1['value'] == 'foo', 

... df1['value'].eqNullSafe('foo'), 

... df1['value'].eqNullSafe(None) 

... ).show() 

+-------------+---------------+----------------+ 

|(value = foo)|(value <=> foo)|(value <=> NULL)| 

+-------------+---------------+----------------+ 

| true| true| false| 

| null| false| true| 

+-------------+---------------+----------------+ 

>>> df2 = spark.createDataFrame([ 

... Row(value = 'bar'), 

... Row(value = None) 

... ]) 

>>> df1.join(df2, df1["value"] == df2["value"]).count() 

0 

>>> df1.join(df2, df1["value"].eqNullSafe(df2["value"])).count() 

1 

>>> df2 = spark.createDataFrame([ 

... Row(id=1, value=float('NaN')), 

... Row(id=2, value=42.0), 

... Row(id=3, value=None) 

... ]) 

>>> df2.select( 

... df2['value'].eqNullSafe(None), 

... df2['value'].eqNullSafe(float('NaN')), 

... df2['value'].eqNullSafe(42.0) 

... ).show() 

+----------------+---------------+----------------+ 

|(value <=> NULL)|(value <=> NaN)|(value <=> 42.0)| 

+----------------+---------------+----------------+ 

| false| true| false| 

| false| false| true| 

| true| false| false| 

+----------------+---------------+----------------+ 

 

Notes 

----- 

Unlike Pandas, PySpark doesn't consider NaN values to be NULL. See the 

`NaN Semantics <https://spark.apache.org/docs/latest/sql-ref-datatypes.html#nan-semantics>`_ 

for details. 

""" 

eqNullSafe = _bin_op("eqNullSafe", _eqNullSafe_doc) 

 

# `and`, `or`, `not` cannot be overloaded in Python, 

# so use bitwise operators as boolean operators 

__and__ = _bin_op('and') 

__or__ = _bin_op('or') 

__invert__ = _func_op('not') 

__rand__ = _bin_op("and") 

__ror__ = _bin_op("or") 

 

# container operators 

def __contains__(self, item): 

raise ValueError("Cannot apply 'in' operator against a column: please use 'contains' " 

"in a string column or 'array_contains' function for an array column.") 

 

# bitwise operators 

_bitwiseOR_doc = """ 

Compute bitwise OR of this expression with another expression. 

 

Parameters 

---------- 

other 

a value or :class:`Column` to calculate bitwise or(|) with 

this :class:`Column`. 

 

Examples 

-------- 

>>> from pyspark.sql import Row 

>>> df = spark.createDataFrame([Row(a=170, b=75)]) 

>>> df.select(df.a.bitwiseOR(df.b)).collect() 

[Row((a | b)=235)] 

""" 

_bitwiseAND_doc = """ 

Compute bitwise AND of this expression with another expression. 

 

Parameters 

---------- 

other 

a value or :class:`Column` to calculate bitwise and(&) with 

this :class:`Column`. 

 

Examples 

-------- 

>>> from pyspark.sql import Row 

>>> df = spark.createDataFrame([Row(a=170, b=75)]) 

>>> df.select(df.a.bitwiseAND(df.b)).collect() 

[Row((a & b)=10)] 

""" 

_bitwiseXOR_doc = """ 

Compute bitwise XOR of this expression with another expression. 

 

Parameters 

---------- 

other 

a value or :class:`Column` to calculate bitwise xor(^) with 

this :class:`Column`. 

 

Examples 

-------- 

>>> from pyspark.sql import Row 

>>> df = spark.createDataFrame([Row(a=170, b=75)]) 

>>> df.select(df.a.bitwiseXOR(df.b)).collect() 

[Row((a ^ b)=225)] 

""" 

 

bitwiseOR = _bin_op("bitwiseOR", _bitwiseOR_doc) 

bitwiseAND = _bin_op("bitwiseAND", _bitwiseAND_doc) 

bitwiseXOR = _bin_op("bitwiseXOR", _bitwiseXOR_doc) 

 

def getItem(self, key): 

""" 

An expression that gets an item at position ``ordinal`` out of a list, 

or gets an item by key out of a dict. 

 

.. versionadded:: 1.3.0 

 

Examples 

-------- 

>>> df = spark.createDataFrame([([1, 2], {"key": "value"})], ["l", "d"]) 

>>> df.select(df.l.getItem(0), df.d.getItem("key")).show() 

+----+------+ 

|l[0]|d[key]| 

+----+------+ 

| 1| value| 

+----+------+ 

""" 

321 ↛ 322line 321 didn't jump to line 322, because the condition on line 321 was never true if isinstance(key, Column): 

warnings.warn( 

"A column as 'key' in getItem is deprecated as of Spark 3.0, and will not " 

"be supported in the future release. Use `column[key]` or `column.key` syntax " 

"instead.", 

FutureWarning 

) 

return self[key] 

 

def getField(self, name): 

""" 

An expression that gets a field by name in a :class:`StructType`. 

 

.. versionadded:: 1.3.0 

 

Examples 

-------- 

>>> from pyspark.sql import Row 

>>> df = spark.createDataFrame([Row(r=Row(a=1, b="b"))]) 

>>> df.select(df.r.getField("b")).show() 

+---+ 

|r.b| 

+---+ 

| b| 

+---+ 

>>> df.select(df.r.a).show() 

+---+ 

|r.a| 

+---+ 

| 1| 

+---+ 

""" 

353 ↛ 354line 353 didn't jump to line 354, because the condition on line 353 was never true if isinstance(name, Column): 

warnings.warn( 

"A column as 'name' in getField is deprecated as of Spark 3.0, and will not " 

"be supported in the future release. Use `column[name]` or `column.name` syntax " 

"instead.", 

FutureWarning 

) 

return self[name] 

 

def withField(self, fieldName, col): 

""" 

An expression that adds/replaces a field in :class:`StructType` by name. 

 

.. versionadded:: 3.1.0 

 

Examples 

-------- 

>>> from pyspark.sql import Row 

>>> from pyspark.sql.functions import lit 

>>> df = spark.createDataFrame([Row(a=Row(b=1, c=2))]) 

>>> df.withColumn('a', df['a'].withField('b', lit(3))).select('a.b').show() 

+---+ 

| b| 

+---+ 

| 3| 

+---+ 

>>> df.withColumn('a', df['a'].withField('d', lit(4))).select('a.d').show() 

+---+ 

| d| 

+---+ 

| 4| 

+---+ 

""" 

if not isinstance(fieldName, str): 

raise TypeError("fieldName should be a string") 

 

if not isinstance(col, Column): 

raise TypeError("col should be a Column") 

 

return Column(self._jc.withField(fieldName, col._jc)) 

 

def dropFields(self, *fieldNames): 

""" 

An expression that drops fields in :class:`StructType` by name. 

This is a no-op if schema doesn't contain field name(s). 

 

.. versionadded:: 3.1.0 

 

Examples 

-------- 

>>> from pyspark.sql import Row 

>>> from pyspark.sql.functions import col, lit 

>>> df = spark.createDataFrame([ 

... Row(a=Row(b=1, c=2, d=3, e=Row(f=4, g=5, h=6)))]) 

>>> df.withColumn('a', df['a'].dropFields('b')).show() 

+-----------------+ 

| a| 

+-----------------+ 

|{2, 3, {4, 5, 6}}| 

+-----------------+ 

 

>>> df.withColumn('a', df['a'].dropFields('b', 'c')).show() 

+--------------+ 

| a| 

+--------------+ 

|{3, {4, 5, 6}}| 

+--------------+ 

 

This method supports dropping multiple nested fields directly e.g. 

 

>>> df.withColumn("a", col("a").dropFields("e.g", "e.h")).show() 

+--------------+ 

| a| 

+--------------+ 

|{1, 2, 3, {4}}| 

+--------------+ 

 

However, if you are going to add/replace multiple nested fields, 

it is preferred to extract out the nested struct before 

adding/replacing multiple fields e.g. 

 

>>> df.select(col("a").withField( 

... "e", col("a.e").dropFields("g", "h")).alias("a") 

... ).show() 

+--------------+ 

| a| 

+--------------+ 

|{1, 2, 3, {4}}| 

+--------------+ 

 

""" 

sc = SparkContext._active_spark_context 

 

jc = self._jc.dropFields(_to_seq(sc, fieldNames)) 

return Column(jc) 

 

def __getattr__(self, item): 

if item.startswith("__"): 

raise AttributeError(item) 

return self[item] 

 

def __getitem__(self, k): 

if isinstance(k, slice): 

if k.step is not None: 

raise ValueError("slice with step is not supported.") 

return self.substr(k.start, k.stop) 

else: 

return _bin_op("apply")(self, k) 

 

def __iter__(self): 

raise TypeError("Column is not iterable") 

 

# string methods 

_contains_doc = """ 

Contains the other element. Returns a boolean :class:`Column` based on a string match. 

 

Parameters 

---------- 

other 

string in line. A value as a literal or a :class:`Column`. 

 

Examples 

-------- 

>>> df.filter(df.name.contains('o')).collect() 

[Row(age=5, name='Bob')] 

""" 

_rlike_doc = """ 

SQL RLIKE expression (LIKE with Regex). Returns a boolean :class:`Column` based on a regex 

match. 

 

Parameters 

---------- 

other : str 

an extended regex expression 

 

Examples 

-------- 

>>> df.filter(df.name.rlike('ice$')).collect() 

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

""" 

_like_doc = """ 

SQL like expression. Returns a boolean :class:`Column` based on a SQL LIKE match. 

 

Parameters 

---------- 

other : str 

a SQL LIKE pattern 

 

See Also 

-------- 

pyspark.sql.Column.rlike 

 

Examples 

-------- 

>>> df.filter(df.name.like('Al%')).collect() 

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

""" 

_startswith_doc = """ 

String starts with. Returns a boolean :class:`Column` based on a string match. 

 

Parameters 

---------- 

other : :class:`Column` or str 

string at start of line (do not use a regex `^`) 

 

Examples 

-------- 

>>> df.filter(df.name.startswith('Al')).collect() 

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

>>> df.filter(df.name.startswith('^Al')).collect() 

[] 

""" 

_endswith_doc = """ 

String ends with. Returns a boolean :class:`Column` based on a string match. 

 

Parameters 

---------- 

other : :class:`Column` or str 

string at end of line (do not use a regex `$`) 

 

Examples 

-------- 

>>> df.filter(df.name.endswith('ice')).collect() 

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

>>> df.filter(df.name.endswith('ice$')).collect() 

[] 

""" 

 

contains = _bin_op("contains", _contains_doc) 

rlike = _bin_op("rlike", _rlike_doc) 

like = _bin_op("like", _like_doc) 

startswith = _bin_op("startsWith", _startswith_doc) 

endswith = _bin_op("endsWith", _endswith_doc) 

 

def substr(self, startPos, length): 

""" 

Return a :class:`Column` which is a substring of the column. 

 

.. versionadded:: 1.3.0 

 

Parameters 

---------- 

startPos : :class:`Column` or int 

start position 

length : :class:`Column` or int 

length of the substring 

 

Examples 

-------- 

>>> df.select(df.name.substr(1, 3).alias("col")).collect() 

[Row(col='Ali'), Row(col='Bob')] 

""" 

if type(startPos) != type(length): 

raise TypeError( 

"startPos and length must be the same type. " 

"Got {startPos_t} and {length_t}, respectively." 

.format( 

startPos_t=type(startPos), 

length_t=type(length), 

)) 

573 ↛ 575line 573 didn't jump to line 575, because the condition on line 573 was never false if isinstance(startPos, int): 

jc = self._jc.substr(startPos, length) 

elif isinstance(startPos, Column): 

jc = self._jc.substr(startPos._jc, length._jc) 

else: 

raise TypeError("Unexpected type: %s" % type(startPos)) 

return Column(jc) 

 

def isin(self, *cols): 

""" 

A boolean expression that is evaluated to true if the value of this 

expression is contained by the evaluated values of the arguments. 

 

.. versionadded:: 1.5.0 

 

Examples 

-------- 

>>> df[df.name.isin("Bob", "Mike")].collect() 

[Row(age=5, name='Bob')] 

>>> df[df.age.isin([1, 2, 3])].collect() 

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

""" 

if len(cols) == 1 and isinstance(cols[0], (list, set)): 

cols = cols[0] 

cols = [c._jc if isinstance(c, Column) else _create_column_from_literal(c) for c in cols] 

sc = SparkContext._active_spark_context 

jc = getattr(self._jc, "isin")(_to_seq(sc, cols)) 

return Column(jc) 

 

# order 

_asc_doc = """ 

Returns a sort expression based on ascending order of the column. 

 

Examples 

-------- 

>>> from pyspark.sql import Row 

>>> df = spark.createDataFrame([('Tom', 80), ('Alice', None)], ["name", "height"]) 

>>> df.select(df.name).orderBy(df.name.asc()).collect() 

[Row(name='Alice'), Row(name='Tom')] 

""" 

_asc_nulls_first_doc = """ 

Returns a sort expression based on ascending order of the column, and null values 

return before non-null values. 

 

.. versionadded:: 2.4.0 

 

Examples 

-------- 

>>> from pyspark.sql import Row 

>>> df = spark.createDataFrame([('Tom', 80), (None, 60), ('Alice', None)], ["name", "height"]) 

>>> df.select(df.name).orderBy(df.name.asc_nulls_first()).collect() 

[Row(name=None), Row(name='Alice'), Row(name='Tom')] 

 

""" 

_asc_nulls_last_doc = """ 

Returns a sort expression based on ascending order of the column, and null values 

appear after non-null values. 

 

.. versionadded:: 2.4.0 

 

Examples 

-------- 

>>> from pyspark.sql import Row 

>>> df = spark.createDataFrame([('Tom', 80), (None, 60), ('Alice', None)], ["name", "height"]) 

>>> df.select(df.name).orderBy(df.name.asc_nulls_last()).collect() 

[Row(name='Alice'), Row(name='Tom'), Row(name=None)] 

 

""" 

_desc_doc = """ 

Returns a sort expression based on the descending order of the column. 

 

.. versionadded:: 2.4.0 

 

Examples 

-------- 

>>> from pyspark.sql import Row 

>>> df = spark.createDataFrame([('Tom', 80), ('Alice', None)], ["name", "height"]) 

>>> df.select(df.name).orderBy(df.name.desc()).collect() 

[Row(name='Tom'), Row(name='Alice')] 

""" 

_desc_nulls_first_doc = """ 

Returns a sort expression based on the descending order of the column, and null values 

appear before non-null values. 

 

.. versionadded:: 2.4.0 

 

Examples 

-------- 

>>> from pyspark.sql import Row 

>>> df = spark.createDataFrame([('Tom', 80), (None, 60), ('Alice', None)], ["name", "height"]) 

>>> df.select(df.name).orderBy(df.name.desc_nulls_first()).collect() 

[Row(name=None), Row(name='Tom'), Row(name='Alice')] 

 

""" 

_desc_nulls_last_doc = """ 

Returns a sort expression based on the descending order of the column, and null values 

appear after non-null values. 

 

.. versionadded:: 2.4.0 

 

Examples 

-------- 

>>> from pyspark.sql import Row 

>>> df = spark.createDataFrame([('Tom', 80), (None, 60), ('Alice', None)], ["name", "height"]) 

>>> df.select(df.name).orderBy(df.name.desc_nulls_last()).collect() 

[Row(name='Tom'), Row(name='Alice'), Row(name=None)] 

""" 

 

asc = _unary_op("asc", _asc_doc) 

asc_nulls_first = _unary_op("asc_nulls_first", _asc_nulls_first_doc) 

asc_nulls_last = _unary_op("asc_nulls_last", _asc_nulls_last_doc) 

desc = _unary_op("desc", _desc_doc) 

desc_nulls_first = _unary_op("desc_nulls_first", _desc_nulls_first_doc) 

desc_nulls_last = _unary_op("desc_nulls_last", _desc_nulls_last_doc) 

 

_isNull_doc = """ 

True if the current expression is null. 

 

Examples 

-------- 

>>> from pyspark.sql import Row 

>>> df = spark.createDataFrame([Row(name='Tom', height=80), Row(name='Alice', height=None)]) 

>>> df.filter(df.height.isNull()).collect() 

[Row(name='Alice', height=None)] 

""" 

_isNotNull_doc = """ 

True if the current expression is NOT null. 

 

Examples 

-------- 

>>> from pyspark.sql import Row 

>>> df = spark.createDataFrame([Row(name='Tom', height=80), Row(name='Alice', height=None)]) 

>>> df.filter(df.height.isNotNull()).collect() 

[Row(name='Tom', height=80)] 

""" 

 

isNull = _unary_op("isNull", _isNull_doc) 

isNotNull = _unary_op("isNotNull", _isNotNull_doc) 

 

def alias(self, *alias, **kwargs): 

""" 

Returns this column aliased with a new name or names (in the case of expressions that 

return more than one column, such as explode). 

 

.. versionadded:: 1.3.0 

 

Parameters 

---------- 

alias : str 

desired column names (collects all positional arguments passed) 

 

Other Parameters 

---------------- 

metadata: dict 

a dict of information to be stored in ``metadata`` attribute of the 

corresponding :class:`StructField <pyspark.sql.types.StructField>` (optional, keyword 

only argument) 

 

.. versionchanged:: 2.2.0 

Added optional ``metadata`` argument. 

 

Examples 

-------- 

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

[Row(age2=2), Row(age2=5)] 

>>> df.select(df.age.alias("age3", metadata={'max': 99})).schema['age3'].metadata['max'] 

99 

""" 

 

metadata = kwargs.pop('metadata', None) 

assert not kwargs, 'Unexpected kwargs where passed: %s' % kwargs 

 

sc = SparkContext._active_spark_context 

if len(alias) == 1: 

if metadata: 

jmeta = sc._jvm.org.apache.spark.sql.types.Metadata.fromJson( 

json.dumps(metadata)) 

return Column(getattr(self._jc, "as")(alias[0], jmeta)) 

else: 

return Column(getattr(self._jc, "as")(alias[0])) 

else: 

754 ↛ 755line 754 didn't jump to line 755, because the condition on line 754 was never true if metadata: 

raise ValueError('metadata can only be provided for a single column') 

return Column(getattr(self._jc, "as")(_to_seq(sc, list(alias)))) 

 

name = copy_func(alias, sinceversion=2.0, doc=":func:`name` is an alias for :func:`alias`.") 

 

def cast(self, dataType): 

""" 

Casts the column into type ``dataType``. 

 

.. versionadded:: 1.3.0 

 

Examples 

-------- 

>>> df.select(df.age.cast("string").alias('ages')).collect() 

[Row(ages='2'), Row(ages='5')] 

>>> df.select(df.age.cast(StringType()).alias('ages')).collect() 

[Row(ages='2'), Row(ages='5')] 

""" 

if isinstance(dataType, str): 

jc = self._jc.cast(dataType) 

775 ↛ 781line 775 didn't jump to line 781, because the condition on line 775 was never false elif isinstance(dataType, DataType): 

from pyspark.sql import SparkSession 

spark = SparkSession.builder.getOrCreate() 

jdt = spark._jsparkSession.parseDataType(dataType.json()) 

jc = self._jc.cast(jdt) 

else: 

raise TypeError("unexpected type: %s" % type(dataType)) 

return Column(jc) 

 

astype = copy_func(cast, sinceversion=1.4, doc=":func:`astype` is an alias for :func:`cast`.") 

 

def between(self, lowerBound, upperBound): 

""" 

True if the current column is between the lower bound and upper bound, inclusive. 

 

.. versionadded:: 1.3.0 

 

Examples 

-------- 

>>> df.select(df.name, df.age.between(2, 4)).show() 

+-----+---------------------------+ 

| name|((age >= 2) AND (age <= 4))| 

+-----+---------------------------+ 

|Alice| true| 

| Bob| false| 

+-----+---------------------------+ 

""" 

return (self >= lowerBound) & (self <= upperBound) 

 

def when(self, condition, value): 

""" 

Evaluates a list of conditions and returns one of multiple possible result expressions. 

If :func:`Column.otherwise` is not invoked, None is returned for unmatched conditions. 

 

.. versionadded:: 1.4.0 

 

Parameters 

---------- 

condition : :class:`Column` 

a boolean :class:`Column` expression. 

value 

a literal value, or a :class:`Column` expression. 

 

Examples 

-------- 

>>> from pyspark.sql import functions as F 

>>> df.select(df.name, F.when(df.age > 4, 1).when(df.age < 3, -1).otherwise(0)).show() 

+-----+------------------------------------------------------------+ 

| name|CASE WHEN (age > 4) THEN 1 WHEN (age < 3) THEN -1 ELSE 0 END| 

+-----+------------------------------------------------------------+ 

|Alice| -1| 

| Bob| 1| 

+-----+------------------------------------------------------------+ 

 

See Also 

-------- 

pyspark.sql.functions.when 

""" 

833 ↛ 834line 833 didn't jump to line 834, because the condition on line 833 was never true if not isinstance(condition, Column): 

raise TypeError("condition should be a Column") 

v = value._jc if isinstance(value, Column) else value 

jc = self._jc.when(condition._jc, v) 

return Column(jc) 

 

def otherwise(self, value): 

""" 

Evaluates a list of conditions and returns one of multiple possible result expressions. 

If :func:`Column.otherwise` is not invoked, None is returned for unmatched conditions. 

 

.. versionadded:: 1.4.0 

 

Parameters 

---------- 

value 

a literal value, or a :class:`Column` expression. 

 

Examples 

-------- 

>>> from pyspark.sql import functions as F 

>>> df.select(df.name, F.when(df.age > 3, 1).otherwise(0)).show() 

+-----+-------------------------------------+ 

| name|CASE WHEN (age > 3) THEN 1 ELSE 0 END| 

+-----+-------------------------------------+ 

|Alice| 0| 

| Bob| 1| 

+-----+-------------------------------------+ 

 

See Also 

-------- 

pyspark.sql.functions.when 

""" 

v = value._jc if isinstance(value, Column) else value 

jc = self._jc.otherwise(v) 

return Column(jc) 

 

def over(self, window): 

""" 

Define a windowing column. 

 

.. versionadded:: 1.4.0 

 

Parameters 

---------- 

window : :class:`WindowSpec` 

 

Returns 

------- 

:class:`Column` 

 

Examples 

-------- 

>>> from pyspark.sql import Window 

>>> window = Window.partitionBy("name").orderBy("age") \ 

.rowsBetween(Window.unboundedPreceding, Window.currentRow) 

>>> from pyspark.sql.functions import rank, min 

>>> from pyspark.sql.functions import desc 

>>> df.withColumn("rank", rank().over(window)) \ 

.withColumn("min", min('age').over(window)).sort(desc("age")).show() 

+---+-----+----+---+ 

|age| name|rank|min| 

+---+-----+----+---+ 

| 5| Bob| 1| 5| 

| 2|Alice| 1| 2| 

+---+-----+----+---+ 

""" 

from pyspark.sql.window import WindowSpec 

901 ↛ 902line 901 didn't jump to line 902, because the condition on line 901 was never true if not isinstance(window, WindowSpec): 

raise TypeError("window should be WindowSpec") 

jc = self._jc.over(window._jspec) 

return Column(jc) 

 

def __nonzero__(self): 

raise ValueError("Cannot convert column into bool: please use '&' for 'and', '|' for 'or', " 

"'~' for 'not' when building DataFrame boolean expressions.") 

__bool__ = __nonzero__ 

 

def __repr__(self): 

return "Column<'%s'>" % self._jc.toString() 

 

 

def _test(): 

import doctest 

from pyspark.sql import SparkSession 

import pyspark.sql.column 

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

spark = SparkSession.builder\ 

.master("local[4]")\ 

.appName("sql.column tests")\ 

.getOrCreate() 

sc = spark.sparkContext 

globs['spark'] = spark 

globs['df'] = sc.parallelize([(2, 'Alice'), (5, 'Bob')]) \ 

.toDF(StructType([StructField('age', IntegerType()), 

StructField('name', StringType())])) 

 

(failure_count, test_count) = doctest.testmod( 

pyspark.sql.column, globs=globs, 

optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE | doctest.REPORT_NDIFF) 

spark.stop() 

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

sys.exit(-1) 

 

 

if __name__ == "__main__": 

_test()