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

940

941

942

943

944

945

946

947

948

949

950

951

952

953

954

955

956

957

958

959

960

961

962

963

964

965

966

967

968

969

970

971

972

973

974

975

976

977

978

979

980

981

982

983

984

985

986

987

988

989

990

991

992

993

994

995

996

997

998

999

1000

1001

1002

1003

1004

1005

1006

1007

1008

1009

1010

1011

1012

1013

1014

1015

1016

1017

1018

1019

1020

1021

1022

1023

1024

1025

1026

1027

1028

1029

1030

1031

1032

1033

1034

1035

1036

1037

1038

1039

1040

1041

1042

1043

1044

1045

1046

1047

1048

1049

1050

1051

1052

1053

1054

1055

1056

1057

1058

1059

1060

1061

1062

1063

1064

1065

1066

1067

1068

1069

1070

1071

1072

1073

1074

1075

1076

1077

1078

1079

1080

1081

1082

1083

1084

1085

1086

1087

1088

1089

1090

1091

1092

1093

1094

1095

1096

1097

1098

1099

1100

1101

1102

1103

1104

1105

1106

1107

1108

1109

1110

1111

1112

1113

1114

1115

1116

1117

1118

1119

1120

1121

1122

1123

1124

1125

1126

1127

1128

1129

1130

1131

1132

1133

1134

1135

1136

1137

1138

1139

1140

1141

1142

1143

1144

1145

1146

1147

1148

1149

1150

1151

1152

1153

1154

1155

1156

1157

1158

1159

1160

1161

1162

1163

1164

1165

1166

1167

1168

1169

1170

1171

1172

1173

1174

1175

1176

1177

1178

1179

1180

1181

1182

1183

1184

1185

1186

1187

1188

1189

1190

1191

1192

1193

1194

1195

1196

1197

1198

1199

1200

1201

1202

1203

1204

1205

1206

1207

1208

1209

1210

1211

1212

1213

1214

1215

1216

1217

1218

1219

1220

1221

1222

1223

1224

1225

1226

1227

1228

1229

1230

1231

1232

1233

1234

1235

1236

1237

1238

1239

1240

1241

1242

1243

1244

1245

1246

1247

1248

1249

1250

1251

1252

1253

1254

1255

1256

1257

1258

1259

1260

1261

1262

1263

1264

1265

1266

1267

1268

1269

1270

1271

1272

1273

1274

1275

1276

1277

1278

1279

1280

1281

1282

1283

1284

1285

1286

1287

1288

1289

1290

1291

1292

1293

1294

1295

1296

1297

1298

1299

1300

1301

1302

1303

1304

1305

1306

1307

1308

1309

1310

1311

1312

# 

# 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 

 

from py4j.java_gateway import java_import 

 

from pyspark import since, keyword_only 

from pyspark.sql.column import _to_seq 

from pyspark.sql.readwriter import OptionUtils, to_str 

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

from pyspark.sql.utils import ForeachBatchFunction, StreamingQueryException 

 

__all__ = ["StreamingQuery", "StreamingQueryManager", "DataStreamReader", "DataStreamWriter"] 

 

 

class StreamingQuery(object): 

""" 

A handle to a query that is executing continuously in the background as new data arrives. 

All these methods are thread-safe. 

 

.. versionadded:: 2.0.0 

 

Notes 

----- 

This API is evolving. 

""" 

 

def __init__(self, jsq): 

self._jsq = jsq 

 

@property 

@since(2.0) 

def id(self): 

"""Returns the unique id of this query that persists across restarts from checkpoint data. 

That is, this id is generated when a query is started for the first time, and 

will be the same every time it is restarted from checkpoint data. 

There can only be one query with the same id active in a Spark cluster. 

Also see, `runId`. 

""" 

return self._jsq.id().toString() 

 

@property 

@since(2.1) 

def runId(self): 

"""Returns the unique id of this query that does not persist across restarts. That is, every 

query that is started (or restarted from checkpoint) will have a different runId. 

""" 

return self._jsq.runId().toString() 

 

@property 

@since(2.0) 

def name(self): 

"""Returns the user-specified name of the query, or null if not specified. 

This name can be specified in the `org.apache.spark.sql.streaming.DataStreamWriter` 

as `dataframe.writeStream.queryName("query").start()`. 

This name, if set, must be unique across all active queries. 

""" 

return self._jsq.name() 

 

@property 

@since(2.0) 

def isActive(self): 

"""Whether this streaming query is currently active or not. 

""" 

return self._jsq.isActive() 

 

@since(2.0) 

def awaitTermination(self, timeout=None): 

"""Waits for the termination of `this` query, either by :func:`query.stop()` or by an 

exception. If the query has terminated with an exception, then the exception will be thrown. 

If `timeout` is set, it returns whether the query has terminated or not within the 

`timeout` seconds. 

 

If the query has terminated, then all subsequent calls to this method will either return 

immediately (if the query was terminated by :func:`stop()`), or throw the exception 

immediately (if the query has terminated with exception). 

 

throws :class:`StreamingQueryException`, if `this` query has terminated with an exception 

""" 

96 ↛ 101line 96 didn't jump to line 101, because the condition on line 96 was never false if timeout is not None: 

if not isinstance(timeout, (int, float)) or timeout < 0: 

raise ValueError("timeout must be a positive integer or float. Got %s" % timeout) 

return self._jsq.awaitTermination(int(timeout * 1000)) 

else: 

return self._jsq.awaitTermination() 

 

@property 

@since(2.1) 

def status(self): 

""" 

Returns the current status of the query. 

""" 

return json.loads(self._jsq.status().json()) 

 

@property 

@since(2.1) 

def recentProgress(self): 

"""Returns an array of the most recent [[StreamingQueryProgress]] updates for this query. 

The number of progress updates retained for each stream is configured by Spark session 

configuration `spark.sql.streaming.numRecentProgressUpdates`. 

""" 

return [json.loads(p.json()) for p in self._jsq.recentProgress()] 

 

@property 

def lastProgress(self): 

""" 

Returns the most recent :class:`StreamingQueryProgress` update of this streaming query or 

None if there were no progress updates 

 

.. versionadded:: 2.1.0 

 

Returns 

------- 

dict 

""" 

lastProgress = self._jsq.lastProgress() 

if lastProgress: 

return json.loads(lastProgress.json()) 

else: 

return None 

 

def processAllAvailable(self): 

"""Blocks until all available data in the source has been processed and committed to the 

sink. This method is intended for testing. 

 

.. versionadded:: 2.0.0 

 

Notes 

----- 

In the case of continually arriving data, this method may block forever. 

Additionally, this method is only guaranteed to block until data that has been 

synchronously appended data to a stream source prior to invocation. 

(i.e. `getOffset` must immediately reflect the addition). 

""" 

return self._jsq.processAllAvailable() 

 

@since(2.0) 

def stop(self): 

"""Stop this streaming query. 

""" 

self._jsq.stop() 

 

def explain(self, extended=False): 

"""Prints the (logical and physical) plans to the console for debugging purpose. 

 

.. versionadded:: 2.1.0 

 

Parameters 

---------- 

extended : bool, optional 

default ``False``. If ``False``, prints only the physical plan. 

 

Examples 

-------- 

>>> sq = sdf.writeStream.format('memory').queryName('query_explain').start() 

>>> sq.processAllAvailable() # Wait a bit to generate the runtime plans. 

>>> sq.explain() 

== Physical Plan == 

... 

>>> sq.explain(True) 

== Parsed Logical Plan == 

... 

== Analyzed Logical Plan == 

... 

== Optimized Logical Plan == 

... 

== Physical Plan == 

... 

>>> sq.stop() 

""" 

# Cannot call `_jsq.explain(...)` because it will print in the JVM process. 

# We should print it in the Python process. 

print(self._jsq.explainInternal(extended)) 

 

def exception(self): 

""" 

.. versionadded:: 2.1.0 

 

Returns 

------- 

:class:`StreamingQueryException` 

the StreamingQueryException if the query was terminated by an exception, or None. 

""" 

if self._jsq.exception().isDefined(): 

je = self._jsq.exception().get() 

msg = je.toString().split(': ', 1)[1] # Drop the Java StreamingQueryException type info 

stackTrace = '\n\t at '.join(map(lambda x: x.toString(), je.getStackTrace())) 

return StreamingQueryException(msg, stackTrace, je.getCause()) 

else: 

return None 

 

 

class StreamingQueryManager(object): 

"""A class to manage all the :class:`StreamingQuery` StreamingQueries active. 

 

.. versionadded:: 2.0.0 

 

Notes 

----- 

This API is evolving. 

""" 

 

def __init__(self, jsqm): 

self._jsqm = jsqm 

 

@property 

def active(self): 

"""Returns a list of active queries associated with this SQLContext 

 

.. versionadded:: 2.0.0 

 

Examples 

-------- 

>>> sq = sdf.writeStream.format('memory').queryName('this_query').start() 

>>> sqm = spark.streams 

>>> # get the list of active streaming queries 

>>> [q.name for q in sqm.active] 

['this_query'] 

>>> sq.stop() 

""" 

return [StreamingQuery(jsq) for jsq in self._jsqm.active()] 

 

def get(self, id): 

"""Returns an active query from this SQLContext or throws exception if an active query 

with this name doesn't exist. 

 

.. versionadded:: 2.0.0 

 

Examples 

-------- 

>>> sq = sdf.writeStream.format('memory').queryName('this_query').start() 

>>> sq.name 

'this_query' 

>>> sq = spark.streams.get(sq.id) 

>>> sq.isActive 

True 

>>> sq = sqlContext.streams.get(sq.id) 

>>> sq.isActive 

True 

>>> sq.stop() 

""" 

return StreamingQuery(self._jsqm.get(id)) 

 

@since(2.0) 

def awaitAnyTermination(self, timeout=None): 

"""Wait until any of the queries on the associated SQLContext has terminated since the 

creation of the context, or since :func:`resetTerminated()` was called. If any query was 

terminated with an exception, then the exception will be thrown. 

If `timeout` is set, it returns whether the query has terminated or not within the 

`timeout` seconds. 

 

If a query has terminated, then subsequent calls to :func:`awaitAnyTermination()` will 

either return immediately (if the query was terminated by :func:`query.stop()`), 

or throw the exception immediately (if the query was terminated with exception). Use 

:func:`resetTerminated()` to clear past terminations and wait for new terminations. 

 

In the case where multiple queries have terminated since :func:`resetTermination()` 

was called, if any query has terminated with exception, then :func:`awaitAnyTermination()` 

will throw any of the exception. For correctly documenting exceptions across multiple 

queries, users need to stop all of them after any of them terminates with exception, and 

then check the `query.exception()` for each query. 

 

throws :class:`StreamingQueryException`, if `this` query has terminated with an exception 

""" 

281 ↛ 286line 281 didn't jump to line 286, because the condition on line 281 was never false if timeout is not None: 

if not isinstance(timeout, (int, float)) or timeout < 0: 

raise ValueError("timeout must be a positive integer or float. Got %s" % timeout) 

return self._jsqm.awaitAnyTermination(int(timeout * 1000)) 

else: 

return self._jsqm.awaitAnyTermination() 

 

def resetTerminated(self): 

"""Forget about past terminated queries so that :func:`awaitAnyTermination()` can be used 

again to wait for new terminations. 

 

.. versionadded:: 2.0.0 

 

Examples 

-------- 

>>> spark.streams.resetTerminated() 

""" 

self._jsqm.resetTerminated() 

 

 

class DataStreamReader(OptionUtils): 

""" 

Interface used to load a streaming :class:`DataFrame <pyspark.sql.DataFrame>` from external 

storage systems (e.g. file systems, key-value stores, etc). 

Use :attr:`SparkSession.readStream <pyspark.sql.SparkSession.readStream>` to access this. 

 

.. versionadded:: 2.0.0 

 

Notes 

----- 

This API is evolving. 

""" 

 

def __init__(self, spark): 

self._jreader = spark._ssql_ctx.readStream() 

self._spark = spark 

 

def _df(self, jdf): 

from pyspark.sql.dataframe import DataFrame 

return DataFrame(jdf, self._spark) 

 

def format(self, source): 

"""Specifies the input data source format. 

 

.. versionadded:: 2.0.0 

 

Parameters 

---------- 

source : str 

name of the data source, e.g. 'json', 'parquet'. 

 

Notes 

----- 

This API is evolving. 

 

Examples 

-------- 

>>> s = spark.readStream.format("text") 

""" 

self._jreader = self._jreader.format(source) 

return self 

 

def schema(self, schema): 

"""Specifies the input schema. 

 

Some data sources (e.g. JSON) can infer the input schema automatically from data. 

By specifying the schema here, the underlying data source can skip the schema 

inference step, and thus speed up data loading. 

 

.. versionadded:: 2.0.0 

 

Parameters 

---------- 

schema : :class:`pyspark.sql.types.StructType` or str 

a :class:`pyspark.sql.types.StructType` object or a DDL-formatted string 

(For example ``col0 INT, col1 DOUBLE``). 

 

Notes 

----- 

This API is evolving. 

 

Examples 

-------- 

>>> s = spark.readStream.schema(sdf_schema) 

>>> s = spark.readStream.schema("col0 INT, col1 DOUBLE") 

""" 

from pyspark.sql import SparkSession 

spark = SparkSession.builder.getOrCreate() 

if isinstance(schema, StructType): 

jschema = spark._jsparkSession.parseDataType(schema.json()) 

self._jreader = self._jreader.schema(jschema) 

372 ↛ 375line 372 didn't jump to line 375, because the condition on line 372 was never false elif isinstance(schema, str): 

self._jreader = self._jreader.schema(schema) 

else: 

raise TypeError("schema should be StructType or string") 

return self 

 

def option(self, key, value): 

"""Adds an input option for the underlying data source. 

 

.. versionadded:: 2.0.0 

 

Notes 

----- 

This API is evolving. 

 

Examples 

-------- 

>>> s = spark.readStream.option("x", 1) 

""" 

self._jreader = self._jreader.option(key, to_str(value)) 

return self 

 

def options(self, **options): 

"""Adds input options for the underlying data source. 

 

.. versionadded:: 2.0.0 

 

Notes 

----- 

This API is evolving. 

 

Examples 

-------- 

>>> s = spark.readStream.options(x="1", y=2) 

""" 

for k in options: 

self._jreader = self._jreader.option(k, to_str(options[k])) 

return self 

 

def load(self, path=None, format=None, schema=None, **options): 

"""Loads a data stream from a data source and returns it as a 

:class:`DataFrame <pyspark.sql.DataFrame>`. 

 

.. versionadded:: 2.0.0 

 

Parameters 

---------- 

path : str, optional 

optional string for file-system backed data sources. 

format : str, optional 

optional string for format of the data source. Default to 'parquet'. 

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

optional :class:`pyspark.sql.types.StructType` for the input schema 

or a DDL-formatted string (For example ``col0 INT, col1 DOUBLE``). 

**options : dict 

all other string options 

 

Notes 

----- 

This API is evolving. 

 

Examples 

-------- 

>>> json_sdf = spark.readStream.format("json") \\ 

... .schema(sdf_schema) \\ 

... .load(tempfile.mkdtemp()) 

>>> json_sdf.isStreaming 

True 

>>> json_sdf.schema == sdf_schema 

True 

""" 

if format is not None: 

self.format(format) 

if schema is not None: 

self.schema(schema) 

self.options(**options) 

if path is not None: 

449 ↛ 450line 449 didn't jump to line 450, because the condition on line 449 was never true if type(path) != str or len(path.strip()) == 0: 

raise ValueError("If the path is provided for stream, it needs to be a " + 

"non-empty string. List of paths are not supported.") 

return self._df(self._jreader.load(path)) 

else: 

return self._df(self._jreader.load()) 

 

def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, 

allowComments=None, allowUnquotedFieldNames=None, allowSingleQuotes=None, 

allowNumericLeadingZero=None, allowBackslashEscapingAnyCharacter=None, 

mode=None, columnNameOfCorruptRecord=None, dateFormat=None, timestampFormat=None, 

multiLine=None, allowUnquotedControlChars=None, lineSep=None, locale=None, 

dropFieldIfAllNull=None, encoding=None, pathGlobFilter=None, 

recursiveFileLookup=None, allowNonNumericNumbers=None): 

""" 

Loads a JSON file stream and returns the results as a :class:`DataFrame`. 

 

`JSON Lines <http://jsonlines.org/>`_ (newline-delimited JSON) is supported by default. 

For JSON (one record per file), set the ``multiLine`` parameter to ``true``. 

 

If the ``schema`` parameter is not specified, this function goes 

through the input once to determine the input schema. 

 

.. versionadded:: 2.0.0 

 

Parameters 

---------- 

path : str 

string represents path to the JSON dataset, 

or RDD of Strings storing JSON objects. 

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

an optional :class:`pyspark.sql.types.StructType` for the input schema 

or a DDL-formatted string (For example ``col0 INT, col1 DOUBLE``). 

 

Other Parameters 

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

Extra options 

For the extra options, refer to 

`Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-json.html#data-source-option>`_ 

in the version you use. 

 

.. # noqa 

 

Notes 

----- 

This API is evolving. 

 

Examples 

-------- 

>>> json_sdf = spark.readStream.json(tempfile.mkdtemp(), schema = sdf_schema) 

>>> json_sdf.isStreaming 

True 

>>> json_sdf.schema == sdf_schema 

True 

""" 

self._set_opts( 

schema=schema, primitivesAsString=primitivesAsString, prefersDecimal=prefersDecimal, 

allowComments=allowComments, allowUnquotedFieldNames=allowUnquotedFieldNames, 

allowSingleQuotes=allowSingleQuotes, allowNumericLeadingZero=allowNumericLeadingZero, 

allowBackslashEscapingAnyCharacter=allowBackslashEscapingAnyCharacter, 

mode=mode, columnNameOfCorruptRecord=columnNameOfCorruptRecord, dateFormat=dateFormat, 

timestampFormat=timestampFormat, multiLine=multiLine, 

allowUnquotedControlChars=allowUnquotedControlChars, lineSep=lineSep, locale=locale, 

dropFieldIfAllNull=dropFieldIfAllNull, encoding=encoding, 

pathGlobFilter=pathGlobFilter, recursiveFileLookup=recursiveFileLookup, 

allowNonNumericNumbers=allowNonNumericNumbers) 

515 ↛ 518line 515 didn't jump to line 518, because the condition on line 515 was never false if isinstance(path, str): 

return self._df(self._jreader.json(path)) 

else: 

raise TypeError("path can be only a single string") 

 

def orc(self, path, mergeSchema=None, pathGlobFilter=None, recursiveFileLookup=None): 

"""Loads a ORC file stream, returning the result as a :class:`DataFrame`. 

 

.. versionadded:: 2.3.0 

 

Other Parameters 

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

Extra options 

For the extra options, refer to 

`Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-orc.html#data-source-option>`_ 

in the version you use. 

 

.. # noqa 

 

Examples 

-------- 

>>> orc_sdf = spark.readStream.schema(sdf_schema).orc(tempfile.mkdtemp()) 

>>> orc_sdf.isStreaming 

True 

>>> orc_sdf.schema == sdf_schema 

True 

""" 

self._set_opts(mergeSchema=mergeSchema, pathGlobFilter=pathGlobFilter, 

recursiveFileLookup=recursiveFileLookup) 

544 ↛ 547line 544 didn't jump to line 547, because the condition on line 544 was never false if isinstance(path, str): 

return self._df(self._jreader.orc(path)) 

else: 

raise TypeError("path can be only a single string") 

 

def parquet(self, path, mergeSchema=None, pathGlobFilter=None, recursiveFileLookup=None, 

datetimeRebaseMode=None, int96RebaseMode=None): 

""" 

Loads a Parquet file stream, returning the result as a :class:`DataFrame`. 

 

.. versionadded:: 2.0.0 

 

Parameters 

---------- 

path : str 

the path in any Hadoop supported file system 

 

Other Parameters 

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

Extra options 

For the extra options, refer to 

`Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#data-source-option>`_. 

in the version you use. 

 

.. # noqa 

 

Examples 

-------- 

>>> parquet_sdf = spark.readStream.schema(sdf_schema).parquet(tempfile.mkdtemp()) 

>>> parquet_sdf.isStreaming 

True 

>>> parquet_sdf.schema == sdf_schema 

True 

""" 

self._set_opts(mergeSchema=mergeSchema, pathGlobFilter=pathGlobFilter, 

recursiveFileLookup=recursiveFileLookup, 

datetimeRebaseMode=datetimeRebaseMode, int96RebaseMode=int96RebaseMode) 

581 ↛ 584line 581 didn't jump to line 584, because the condition on line 581 was never false if isinstance(path, str): 

return self._df(self._jreader.parquet(path)) 

else: 

raise TypeError("path can be only a single string") 

 

def text(self, path, wholetext=False, lineSep=None, pathGlobFilter=None, 

recursiveFileLookup=None): 

""" 

Loads a text file stream and returns a :class:`DataFrame` whose schema starts with a 

string column named "value", and followed by partitioned columns if there 

are any. 

The text files must be encoded as UTF-8. 

 

By default, each line in the text file is a new row in the resulting DataFrame. 

 

.. versionadded:: 2.0.0 

 

Parameters 

---------- 

paths : str or list 

string, or list of strings, for input path(s). 

 

Other Parameters 

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

Extra options 

For the extra options, refer to 

`Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-text.html#data-source-option>`_ 

in the version you use. 

 

.. # noqa 

 

Notes 

----- 

This API is evolving. 

 

Examples 

-------- 

>>> text_sdf = spark.readStream.text(tempfile.mkdtemp()) 

>>> text_sdf.isStreaming 

True 

>>> "value" in str(text_sdf.schema) 

True 

""" 

self._set_opts( 

wholetext=wholetext, lineSep=lineSep, pathGlobFilter=pathGlobFilter, 

recursiveFileLookup=recursiveFileLookup) 

627 ↛ 630line 627 didn't jump to line 630, because the condition on line 627 was never false if isinstance(path, str): 

return self._df(self._jreader.text(path)) 

else: 

raise TypeError("path can be only a single string") 

 

def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=None, 

comment=None, header=None, inferSchema=None, ignoreLeadingWhiteSpace=None, 

ignoreTrailingWhiteSpace=None, nullValue=None, nanValue=None, positiveInf=None, 

negativeInf=None, dateFormat=None, timestampFormat=None, maxColumns=None, 

maxCharsPerColumn=None, maxMalformedLogPerPartition=None, mode=None, 

columnNameOfCorruptRecord=None, multiLine=None, charToEscapeQuoteEscaping=None, 

enforceSchema=None, emptyValue=None, locale=None, lineSep=None, 

pathGlobFilter=None, recursiveFileLookup=None, unescapedQuoteHandling=None): 

r"""Loads a CSV file stream and returns the result as a :class:`DataFrame`. 

 

This function will go through the input once to determine the input schema if 

``inferSchema`` is enabled. To avoid going through the entire data once, disable 

``inferSchema`` option or specify the schema explicitly using ``schema``. 

 

Parameters 

---------- 

path : str or list 

string, or list of strings, for input path(s). 

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

an optional :class:`pyspark.sql.types.StructType` for the input schema 

or a DDL-formatted string (For example ``col0 INT, col1 DOUBLE``). 

 

.. versionadded:: 2.0.0 

 

Other Parameters 

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

Extra options 

For the extra options, refer to 

`Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-csv.html#data-source-option>`_ 

in the version you use. 

 

.. # noqa 

 

Notes 

----- 

This API is evolving. 

 

Examples 

-------- 

>>> csv_sdf = spark.readStream.csv(tempfile.mkdtemp(), schema = sdf_schema) 

>>> csv_sdf.isStreaming 

True 

>>> csv_sdf.schema == sdf_schema 

True 

""" 

self._set_opts( 

schema=schema, sep=sep, encoding=encoding, quote=quote, escape=escape, comment=comment, 

header=header, inferSchema=inferSchema, ignoreLeadingWhiteSpace=ignoreLeadingWhiteSpace, 

ignoreTrailingWhiteSpace=ignoreTrailingWhiteSpace, nullValue=nullValue, 

nanValue=nanValue, positiveInf=positiveInf, negativeInf=negativeInf, 

dateFormat=dateFormat, timestampFormat=timestampFormat, maxColumns=maxColumns, 

maxCharsPerColumn=maxCharsPerColumn, 

maxMalformedLogPerPartition=maxMalformedLogPerPartition, mode=mode, 

columnNameOfCorruptRecord=columnNameOfCorruptRecord, multiLine=multiLine, 

charToEscapeQuoteEscaping=charToEscapeQuoteEscaping, enforceSchema=enforceSchema, 

emptyValue=emptyValue, locale=locale, lineSep=lineSep, 

pathGlobFilter=pathGlobFilter, recursiveFileLookup=recursiveFileLookup, 

unescapedQuoteHandling=unescapedQuoteHandling) 

690 ↛ 693line 690 didn't jump to line 693, because the condition on line 690 was never false if isinstance(path, str): 

return self._df(self._jreader.csv(path)) 

else: 

raise TypeError("path can be only a single string") 

 

def table(self, tableName): 

"""Define a Streaming DataFrame on a Table. The DataSource corresponding to the table should 

support streaming mode. 

 

.. versionadded:: 3.1.0 

 

Parameters 

---------- 

tableName : str 

string, for the name of the table. 

 

Returns 

-------- 

:class:`DataFrame` 

 

Notes 

----- 

This API is evolving. 

 

Examples 

-------- 

>>> spark.readStream.table('input_table') # doctest: +SKIP 

""" 

718 ↛ 721line 718 didn't jump to line 721, because the condition on line 718 was never false if isinstance(tableName, str): 

return self._df(self._jreader.table(tableName)) 

else: 

raise TypeError("tableName can be only a single string") 

 

 

class DataStreamWriter(object): 

""" 

Interface used to write a streaming :class:`DataFrame <pyspark.sql.DataFrame>` to external 

storage systems (e.g. file systems, key-value stores, etc). 

Use :attr:`DataFrame.writeStream <pyspark.sql.DataFrame.writeStream>` 

to access this. 

 

.. versionadded:: 2.0.0 

 

Notes 

----- 

This API is evolving. 

""" 

 

def __init__(self, df): 

self._df = df 

self._spark = df.sql_ctx 

self._jwrite = df._jdf.writeStream() 

 

def _sq(self, jsq): 

from pyspark.sql.streaming import StreamingQuery 

return StreamingQuery(jsq) 

 

def outputMode(self, outputMode): 

"""Specifies how data of a streaming DataFrame/Dataset is written to a streaming sink. 

 

.. versionadded:: 2.0.0 

 

Options include: 

 

* `append`: Only the new rows in the streaming DataFrame/Dataset will be written to 

the sink 

* `complete`: All the rows in the streaming DataFrame/Dataset will be written to the sink 

every time these are some updates 

* `update`: only the rows that were updated in the streaming DataFrame/Dataset will be 

written to the sink every time there are some updates. If the query doesn't contain 

aggregations, it will be equivalent to `append` mode. 

 

Notes 

----- 

This API is evolving. 

 

Examples 

-------- 

>>> writer = sdf.writeStream.outputMode('append') 

""" 

770 ↛ 771line 770 didn't jump to line 771, because the condition on line 770 was never true if not outputMode or type(outputMode) != str or len(outputMode.strip()) == 0: 

raise ValueError('The output mode must be a non-empty string. Got: %s' % outputMode) 

self._jwrite = self._jwrite.outputMode(outputMode) 

return self 

 

def format(self, source): 

"""Specifies the underlying output data source. 

 

.. versionadded:: 2.0.0 

 

Parameters 

---------- 

source : str 

string, name of the data source, which for now can be 'parquet'. 

 

Notes 

----- 

This API is evolving. 

 

Examples 

-------- 

>>> writer = sdf.writeStream.format('json') 

""" 

self._jwrite = self._jwrite.format(source) 

return self 

 

def option(self, key, value): 

"""Adds an output option for the underlying data source. 

 

.. versionadded:: 2.0.0 

 

Notes 

----- 

This API is evolving. 

""" 

self._jwrite = self._jwrite.option(key, to_str(value)) 

return self 

 

def options(self, **options): 

"""Adds output options for the underlying data source. 

 

.. versionadded:: 2.0.0 

 

Notes 

----- 

This API is evolving. 

""" 

for k in options: 

self._jwrite = self._jwrite.option(k, to_str(options[k])) 

return self 

 

def partitionBy(self, *cols): 

"""Partitions the output by the given columns on the file system. 

 

If specified, the output is laid out on the file system similar 

to Hive's partitioning scheme. 

 

.. versionadded:: 2.0.0 

 

Parameters 

---------- 

cols : str or list 

name of columns 

 

Notes 

----- 

This API is evolving. 

""" 

838 ↛ 839line 838 didn't jump to line 839, because the condition on line 838 was never true if len(cols) == 1 and isinstance(cols[0], (list, tuple)): 

cols = cols[0] 

self._jwrite = self._jwrite.partitionBy(_to_seq(self._spark._sc, cols)) 

return self 

 

def queryName(self, queryName): 

"""Specifies the name of the :class:`StreamingQuery` that can be started with 

:func:`start`. This name must be unique among all the currently active queries 

in the associated SparkSession. 

 

.. versionadded:: 2.0.0 

 

Parameters 

---------- 

queryName : str 

unique name for the query 

 

Notes 

----- 

This API is evolving. 

 

Examples 

-------- 

>>> writer = sdf.writeStream.queryName('streaming_query') 

""" 

863 ↛ 864line 863 didn't jump to line 864, because the condition on line 863 was never true if not queryName or type(queryName) != str or len(queryName.strip()) == 0: 

raise ValueError('The queryName must be a non-empty string. Got: %s' % queryName) 

self._jwrite = self._jwrite.queryName(queryName) 

return self 

 

@keyword_only 

def trigger(self, *, processingTime=None, once=None, continuous=None): 

"""Set the trigger for the stream query. If this is not set it will run the query as fast 

as possible, which is equivalent to setting the trigger to ``processingTime='0 seconds'``. 

 

.. versionadded:: 2.0.0 

 

Parameters 

---------- 

processingTime : str, optional 

a processing time interval as a string, e.g. '5 seconds', '1 minute'. 

Set a trigger that runs a microbatch query periodically based on the 

processing time. Only one trigger can be set. 

once : bool, optional 

if set to True, set a trigger that processes only one batch of data in a 

streaming query then terminates the query. Only one trigger can be set. 

continuous : str, optional 

a time interval as a string, e.g. '5 seconds', '1 minute'. 

Set a trigger that runs a continuous query with a given checkpoint 

interval. Only one trigger can be set. 

 

Notes 

----- 

This API is evolving. 

 

Examples 

-------- 

>>> # trigger the query for execution every 5 seconds 

>>> writer = sdf.writeStream.trigger(processingTime='5 seconds') 

>>> # trigger the query for just once batch of data 

>>> writer = sdf.writeStream.trigger(once=True) 

>>> # trigger the query for execution every 5 seconds 

>>> writer = sdf.writeStream.trigger(continuous='5 seconds') 

""" 

params = [processingTime, once, continuous] 

 

if params.count(None) == 3: 

raise ValueError('No trigger provided') 

elif params.count(None) < 2: 

raise ValueError('Multiple triggers not allowed.') 

 

jTrigger = None 

if processingTime is not None: 

911 ↛ 912line 911 didn't jump to line 912, because the condition on line 911 was never true if type(processingTime) != str or len(processingTime.strip()) == 0: 

raise ValueError('Value for processingTime must be a non empty string. Got: %s' % 

processingTime) 

interval = processingTime.strip() 

jTrigger = self._spark._sc._jvm.org.apache.spark.sql.streaming.Trigger.ProcessingTime( 

interval) 

 

elif once is not None: 

919 ↛ 920line 919 didn't jump to line 920, because the condition on line 919 was never true if once is not True: 

raise ValueError('Value for once must be True. Got: %s' % once) 

jTrigger = self._spark._sc._jvm.org.apache.spark.sql.streaming.Trigger.Once() 

 

else: 

924 ↛ 925line 924 didn't jump to line 925, because the condition on line 924 was never true if type(continuous) != str or len(continuous.strip()) == 0: 

raise ValueError('Value for continuous must be a non empty string. Got: %s' % 

continuous) 

interval = continuous.strip() 

jTrigger = self._spark._sc._jvm.org.apache.spark.sql.streaming.Trigger.Continuous( 

interval) 

 

self._jwrite = self._jwrite.trigger(jTrigger) 

return self 

 

def foreach(self, f): 

""" 

Sets the output of the streaming query to be processed using the provided writer ``f``. 

This is often used to write the output of a streaming query to arbitrary storage systems. 

The processing logic can be specified in two ways. 

 

#. A **function** that takes a row as input. 

This is a simple way to express your processing logic. Note that this does 

not allow you to deduplicate generated data when failures cause reprocessing of 

some input data. That would require you to specify the processing logic in the next 

way. 

 

#. An **object** with a ``process`` method and optional ``open`` and ``close`` methods. 

The object can have the following methods. 

 

* ``open(partition_id, epoch_id)``: *Optional* method that initializes the processing 

(for example, open a connection, start a transaction, etc). Additionally, you can 

use the `partition_id` and `epoch_id` to deduplicate regenerated data 

(discussed later). 

 

* ``process(row)``: *Non-optional* method that processes each :class:`Row`. 

 

* ``close(error)``: *Optional* method that finalizes and cleans up (for example, 

close connection, commit transaction, etc.) after all rows have been processed. 

 

The object will be used by Spark in the following way. 

 

* A single copy of this object is responsible of all the data generated by a 

single task in a query. In other words, one instance is responsible for 

processing one partition of the data generated in a distributed manner. 

 

* This object must be serializable because each task will get a fresh 

serialized-deserialized copy of the provided object. Hence, it is strongly 

recommended that any initialization for writing data (e.g. opening a 

connection or starting a transaction) is done after the `open(...)` 

method has been called, which signifies that the task is ready to generate data. 

 

* The lifecycle of the methods are as follows. 

 

For each partition with ``partition_id``: 

 

... For each batch/epoch of streaming data with ``epoch_id``: 

 

....... Method ``open(partitionId, epochId)`` is called. 

 

....... If ``open(...)`` returns true, for each row in the partition and 

batch/epoch, method ``process(row)`` is called. 

 

....... Method ``close(errorOrNull)`` is called with error (if any) seen while 

processing rows. 

 

Important points to note: 

 

* The `partitionId` and `epochId` can be used to deduplicate generated data when 

failures cause reprocessing of some input data. This depends on the execution 

mode of the query. If the streaming query is being executed in the micro-batch 

mode, then every partition represented by a unique tuple (partition_id, epoch_id) 

is guaranteed to have the same data. Hence, (partition_id, epoch_id) can be used 

to deduplicate and/or transactionally commit data and achieve exactly-once 

guarantees. However, if the streaming query is being executed in the continuous 

mode, then this guarantee does not hold and therefore should not be used for 

deduplication. 

 

* The ``close()`` method (if exists) will be called if `open()` method exists and 

returns successfully (irrespective of the return value), except if the Python 

crashes in the middle. 

 

.. versionadded:: 2.4.0 

 

Notes 

----- 

This API is evolving. 

 

Examples 

-------- 

>>> # Print every row using a function 

>>> def print_row(row): 

... print(row) 

... 

>>> writer = sdf.writeStream.foreach(print_row) 

>>> # Print every row using a object with process() method 

>>> class RowPrinter: 

... def open(self, partition_id, epoch_id): 

... print("Opened %d, %d" % (partition_id, epoch_id)) 

... return True 

... def process(self, row): 

... print(row) 

... def close(self, error): 

... print("Closed with error: %s" % str(error)) 

... 

>>> writer = sdf.writeStream.foreach(RowPrinter()) 

""" 

 

from pyspark.rdd import _wrap_function 

from pyspark.serializers import PickleSerializer, AutoBatchedSerializer 

from pyspark.taskcontext import TaskContext 

 

if callable(f): 

# The provided object is a callable function that is supposed to be called on each row. 

# Construct a function that takes an iterator and calls the provided function on each 

# row. 

def func_without_process(_, iterator): 

for x in iterator: 

f(x) 

return iter([]) 

 

func = func_without_process 

 

else: 

# The provided object is not a callable function. Then it is expected to have a 

# 'process(row)' method, and optional 'open(partition_id, epoch_id)' and 

# 'close(error)' methods. 

 

if not hasattr(f, 'process'): 

raise AttributeError("Provided object does not have a 'process' method") 

 

if not callable(getattr(f, 'process')): 

raise TypeError("Attribute 'process' in provided object is not callable") 

 

def doesMethodExist(method_name): 

exists = hasattr(f, method_name) 

if exists and not callable(getattr(f, method_name)): 

raise TypeError( 

"Attribute '%s' in provided object is not callable" % method_name) 

return exists 

 

open_exists = doesMethodExist('open') 

close_exists = doesMethodExist('close') 

 

def func_with_open_process_close(partition_id, iterator): 

epoch_id = TaskContext.get().getLocalProperty('streaming.sql.batchId') 

1065 ↛ 1068line 1065 didn't jump to line 1068, because the condition on line 1065 was never false if epoch_id: 

epoch_id = int(epoch_id) 

else: 

raise RuntimeError("Could not get batch id from TaskContext") 

 

# Check if the data should be processed 

should_process = True 

if open_exists: 

should_process = f.open(partition_id, epoch_id) 

 

error = None 

 

try: 

if should_process: 

for x in iterator: 

f.process(x) 

except Exception as ex: 

error = ex 

finally: 

if close_exists: 

f.close(error) 

if error: 

raise error 

 

return iter([]) 

 

func = func_with_open_process_close 

 

serializer = AutoBatchedSerializer(PickleSerializer()) 

wrapped_func = _wrap_function(self._spark._sc, func, serializer, serializer) 

jForeachWriter = \ 

self._spark._sc._jvm.org.apache.spark.sql.execution.python.PythonForeachWriter( 

wrapped_func, self._df._jdf.schema()) 

self._jwrite.foreach(jForeachWriter) 

return self 

 

def foreachBatch(self, func): 

""" 

Sets the output of the streaming query to be processed using the provided 

function. This is supported only the in the micro-batch execution modes (that is, when the 

trigger is not continuous). In every micro-batch, the provided function will be called in 

every micro-batch with (i) the output rows as a DataFrame and (ii) the batch identifier. 

The batchId can be used deduplicate and transactionally write the output 

(that is, the provided Dataset) to external systems. The output DataFrame is guaranteed 

to exactly same for the same batchId (assuming all operations are deterministic in the 

query). 

 

.. versionadded:: 2.4.0 

 

Notes 

----- 

This API is evolving. 

 

Examples 

-------- 

>>> def func(batch_df, batch_id): 

... batch_df.collect() 

... 

>>> writer = sdf.writeStream.foreachBatch(func) 

""" 

 

from pyspark.java_gateway import ensure_callback_server_started 

gw = self._spark._sc._gateway 

java_import(gw.jvm, "org.apache.spark.sql.execution.streaming.sources.*") 

 

wrapped_func = ForeachBatchFunction(self._spark, func) 

gw.jvm.PythonForeachBatchHelper.callForeachBatch(self._jwrite, wrapped_func) 

ensure_callback_server_started(gw) 

return self 

 

def start(self, path=None, format=None, outputMode=None, partitionBy=None, queryName=None, 

**options): 

"""Streams the contents of the :class:`DataFrame` to a data source. 

 

The data source is specified by the ``format`` and a set of ``options``. 

If ``format`` is not specified, the default data source configured by 

``spark.sql.sources.default`` will be used. 

 

.. versionadded:: 2.0.0 

 

Parameters 

---------- 

path : str, optional 

the path in a Hadoop supported file system 

format : str, optional 

the format used to save 

outputMode : str, optional 

specifies how data of a streaming DataFrame/Dataset is written to a 

streaming sink. 

 

* `append`: Only the new rows in the streaming DataFrame/Dataset will be written to the 

sink 

* `complete`: All the rows in the streaming DataFrame/Dataset will be written to the 

sink every time these are some updates 

* `update`: only the rows that were updated in the streaming DataFrame/Dataset will be 

written to the sink every time there are some updates. If the query doesn't contain 

aggregations, it will be equivalent to `append` mode. 

partitionBy : str or list, optional 

names of partitioning columns 

queryName : str, optional 

unique name for the query 

**options : dict 

All other string options. You may want to provide a `checkpointLocation` 

for most streams, however it is not required for a `memory` stream. 

 

Notes 

----- 

This API is evolving. 

 

Examples 

-------- 

>>> sq = sdf.writeStream.format('memory').queryName('this_query').start() 

>>> sq.isActive 

True 

>>> sq.name 

'this_query' 

>>> sq.stop() 

>>> sq.isActive 

False 

>>> sq = sdf.writeStream.trigger(processingTime='5 seconds').start( 

... queryName='that_query', outputMode="append", format='memory') 

>>> sq.name 

'that_query' 

>>> sq.isActive 

True 

>>> sq.stop() 

""" 

self.options(**options) 

if outputMode is not None: 

self.outputMode(outputMode) 

1195 ↛ 1196line 1195 didn't jump to line 1196, because the condition on line 1195 was never true if partitionBy is not None: 

self.partitionBy(partitionBy) 

if format is not None: 

self.format(format) 

if queryName is not None: 

self.queryName(queryName) 

if path is None: 

return self._sq(self._jwrite.start()) 

else: 

return self._sq(self._jwrite.start(path)) 

 

def toTable(self, tableName, format=None, outputMode=None, partitionBy=None, queryName=None, 

**options): 

""" 

Starts the execution of the streaming query, which will continually output results to the 

given table as new data arrives. 

 

The returned :class:`StreamingQuery` object can be used to interact with the stream. 

 

.. versionadded:: 3.1.0 

 

Parameters 

---------- 

tableName : str 

string, for the name of the table. 

format : str, optional 

the format used to save. 

outputMode : str, optional 

specifies how data of a streaming DataFrame/Dataset is written to a 

streaming sink. 

 

* `append`: Only the new rows in the streaming DataFrame/Dataset will be written to the 

sink 

* `complete`: All the rows in the streaming DataFrame/Dataset will be written to the 

sink every time these are some updates 

* `update`: only the rows that were updated in the streaming DataFrame/Dataset will be 

written to the sink every time there are some updates. If the query doesn't contain 

aggregations, it will be equivalent to `append` mode. 

partitionBy : str or list, optional 

names of partitioning columns 

queryName : str, optional 

unique name for the query 

**options : dict 

All other string options. You may want to provide a `checkpointLocation`. 

 

Notes 

----- 

This API is evolving. 

 

For v1 table, partitioning columns provided by `partitionBy` will be respected no matter 

the table exists or not. A new table will be created if the table not exists. 

 

For v2 table, `partitionBy` will be ignored if the table already exists. `partitionBy` will 

be respected only if the v2 table does not exist. Besides, the v2 table created by this API 

lacks some functionalities (e.g., customized properties, options, and serde info). If you 

need them, please create the v2 table manually before the execution to avoid creating a 

table with incomplete information. 

 

Examples 

-------- 

>>> sdf.writeStream.format('parquet').queryName('query').toTable('output_table') 

... # doctest: +SKIP 

 

>>> sdf.writeStream.trigger(processingTime='5 seconds').toTable( 

... 'output_table', 

... queryName='that_query', 

... outputMode="append", 

... format='parquet', 

... checkpointLocation='/tmp/checkpoint') # doctest: +SKIP 

""" 

self.options(**options) 

1266 ↛ 1267line 1266 didn't jump to line 1267, because the condition on line 1266 was never true if outputMode is not None: 

self.outputMode(outputMode) 

1268 ↛ 1269line 1268 didn't jump to line 1269, because the condition on line 1268 was never true if partitionBy is not None: 

self.partitionBy(partitionBy) 

1270 ↛ 1272line 1270 didn't jump to line 1272, because the condition on line 1270 was never false if format is not None: 

self.format(format) 

1272 ↛ 1273line 1272 didn't jump to line 1273, because the condition on line 1272 was never true if queryName is not None: 

self.queryName(queryName) 

return self._sq(self._jwrite.toTable(tableName)) 

 

 

def _test(): 

import doctest 

import os 

import tempfile 

from pyspark.sql import SparkSession, SQLContext 

import pyspark.sql.streaming 

 

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

 

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

try: 

spark = SparkSession.builder.getOrCreate() 

except py4j.protocol.Py4JError: # noqa: F821 

spark = SparkSession(sc) # noqa: F821 

 

globs['tempfile'] = tempfile 

globs['os'] = os 

globs['spark'] = spark 

globs['sqlContext'] = SQLContext.getOrCreate(spark.sparkContext) 

globs['sdf'] = \ 

spark.readStream.format('text').load('python/test_support/sql/streaming') 

globs['sdf_schema'] = StructType([StructField("data", StringType(), True)]) 

globs['df'] = \ 

globs['spark'].readStream.format('text').load('python/test_support/sql/streaming') 

 

(failure_count, test_count) = doctest.testmod( 

pyspark.sql.streaming, globs=globs, 

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

globs['spark'].stop() 

 

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

sys.exit(-1) 

 

 

if __name__ == "__main__": 

_test()