or run

npx @tessl/cli init
Log in

Version

Tile

Overview

Evals

Files

Files

docs

analysis.mdcode-generation.mddata-types.mdexpressions.mdindex.mdoptimization.mdparsing.mdquery-plans.mdutilities.md

analysis.mddocs/

0

# Analysis Framework and Rules

1

2

Catalyst's analysis framework transforms unresolved logical plans into fully resolved, type-checked plans. The analyzer resolves references, validates types, binds to catalog metadata, and ensures query correctness before optimization.

3

4

## Capabilities

5

6

### Analyzer Core

7

8

Main analysis engine that resolves logical plans through rule application:

9

10

```scala { .api }

11

/**

12

* Main analysis engine that resolves logical plans

13

* Applies resolution rules iteratively until fixed point or max iterations

14

*/

15

class Analyzer(

16

catalog: SessionCatalog,

17

conf: SQLConf,

18

maxIterations: Int = 100)

19

extends RuleExecutor[LogicalPlan] with CheckAnalysis {

20

21

/** Execute analysis on a logical plan */

22

def execute(plan: LogicalPlan): LogicalPlan = {

23

AnalysisContext.reset()

24

try {

25

executeSameContext(plan)

26

} finally {

27

AnalysisContext.reset()

28

}

29

}

30

31

/** Execute analysis and validate the result */

32

def executeAndCheck(plan: LogicalPlan): LogicalPlan = {

33

val analyzed = execute(plan)

34

try {

35

checkAnalysis(analyzed)

36

analyzed

37

} catch {

38

case e: AnalysisException =>

39

val ae = new AnalysisException(e.message, e.line, e.startPosition, Some(analyzed))

40

ae.setStackTrace(e.getStackTrace)

41

throw ae

42

}

43

}

44

45

/** Check if plan is fully analyzed */

46

def checkAnalysis(plan: LogicalPlan): Unit = {

47

def checkExpressions(exprs: Seq[Expression]): Unit = {

48

exprs.foreach { expr =>

49

expr.foreach {

50

case _: UnresolvedAttribute =>

51

throw new AnalysisException(s"Unresolved attribute: ${expr.prettyName}")

52

case _: UnresolvedFunction =>

53

throw new AnalysisException(s"Unresolved function: ${expr.prettyName}")

54

case _ =>

55

}

56

}

57

}

58

59

plan.foreach {

60

case p if !p.resolved =>

61

throw new AnalysisException(s"Unresolved plan: ${p.nodeName}")

62

case p =>

63

checkExpressions(p.expressions)

64

}

65

}

66

67

/** Rule batches for iterative analysis */

68

lazy val batches: Seq[Batch] = Seq(

69

Batch("Hints", Once,

70

new ResolveHints.ResolveBroadcastHints(conf)),

71

Batch("Simple Sanity Check", Once,

72

LookupFunctions),

73

Batch("Substitution", fixedPoint,

74

CTESubstitution,

75

WindowsSubstitution,

76

EliminateUnions),

77

Batch("Resolution", fixedPoint,

78

ResolveTableValuedFunctions ::

79

ResolveRelations :: ResolveReferences ::

80

ResolveCreateNamedStruct ::

81

ResolveDeserializer ::

82

ResolveNewInstance ::

83

ResolveUpCast ::

84

ResolveGroupingAnalytics ::

85

ResolvePivot ::

86

ResolveOrdinalInOrderByAndGroupBy ::

87

ResolveMissingReferences ::

88

ExtractGenerator ::

89

ResolveGenerate ::

90

ResolveFunctions ::

91

ResolveAliases ::

92

ResolveSubquery ::

93

ResolveWindowOrder ::

94

ResolveWindowFrame ::

95

ResolveNaturalAndUsingJoin ::

96

ExtractWindowExpressions ::

97

GlobalAggregates ::

98

ResolveAggregateFunctions ::

99

TimeWindowing ::

100

ResolveInlineTables ::

101

TypeCoercion.typeCoercionRules ++

102

extendedResolutionRules: _*),

103

Batch("Post-Hoc Resolution", Once, postHocResolutionRules: _*),

104

Batch("View", Once,

105

AliasViewChild),

106

Batch("Nondeterministic", Once,

107

PullOutNondeterministic),

108

Batch("UDF", Once,

109

HandleNullInputsForUDF),

110

Batch("Cleanup", fixedPoint,

111

CleanupAliases)

112

)

113

}

114

```

115

116

**Usage Examples:**

117

118

```scala

119

import org.apache.spark.sql.catalyst.analysis._

120

import org.apache.spark.sql.catalyst.plans.logical._

121

import org.apache.spark.sql.catalyst.parser.CatalystSqlParser

122

123

// Create analyzer with catalog and configuration

124

val catalog = new SessionCatalog(externalCatalog, globalTempViewManager,

125

functionResourceLoader, functionRegistry, conf, hadoopConf)

126

val analyzer = new Analyzer(catalog, conf, maxIterations = 100)

127

128

// Parse and analyze a query

129

val sqlText = "SELECT name, age FROM users WHERE age > 25"

130

val unresolvedPlan = CatalystSqlParser.parsePlan(sqlText)

131

val analyzedPlan = analyzer.execute(unresolvedPlan)

132

133

// Verify analysis succeeded

134

analyzer.checkAnalysis(analyzedPlan)

135

require(analyzedPlan.resolved, "Plan must be fully resolved after analysis")

136

```

137

138

### Resolution Rules

139

140

Core rules that resolve different aspects of logical plans:

141

142

```scala { .api }

143

/**

144

* Resolves relations (tables) against the session catalog

145

*/

146

object ResolveRelations extends Rule[LogicalPlan] {

147

def ruleName: String = "ResolveRelations"

148

149

def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {

150

case u @ UnresolvedRelation(tableIdentifier, alias) =>

151

try {

152

val table = lookupTableFromCatalog(tableIdentifier)

153

val relationPlan = UnresolvedCatalogRelation(table, alias)

154

relationPlan

155

} catch {

156

case _: NoSuchTableException =>

157

u

158

}

159

}

160

}

161

162

/**

163

* Resolves attribute references against available input attributes

164

*/

165

object ResolveReferences extends Rule[LogicalPlan] {

166

def ruleName: String = "ResolveReferences"

167

168

def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {

169

case p: LogicalPlan if !p.childrenResolved => p

170

case p: LogicalPlan => p.transformExpressionsUp {

171

case u @ UnresolvedAttribute(nameParts) =>

172

val resolved = resolveAttribute(u, p.inputSet)

173

resolved.getOrElse(u)

174

}

175

}

176

177

private def resolveAttribute(

178

u: UnresolvedAttribute,

179

input: AttributeSet): Option[NamedExpression] = {

180

val candidates = input.filter { attr =>

181

// Match attribute name (case-insensitive)

182

resolver(attr.name, u.name)

183

}

184

185

candidates.toSeq match {

186

case Seq(a) => Some(a)

187

case Nil => None

188

case ambiguous =>

189

throw new AnalysisException(s"Ambiguous reference to ${u.name}: $ambiguous")

190

}

191

}

192

}

193

194

/**

195

* Resolves function calls to registered functions

196

*/

197

object ResolveFunctions extends Rule[LogicalPlan] {

198

def ruleName: String = "ResolveFunctions"

199

200

def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {

201

case q: LogicalPlan => q.transformExpressionsUp {

202

case u @ UnresolvedFunction(name, children, isDistinct) =>

203

withPosition(u) {

204

try {

205

val info = catalog.lookupFunction(name, children)

206

val function = info.builder(children)

207

function

208

} catch {

209

case _: AnalysisException => u

210

}

211

}

212

}

213

}

214

}

215

216

/**

217

* Resolves aliases in SELECT lists and ensures proper naming

218

*/

219

object ResolveAliases extends Rule[LogicalPlan] {

220

def ruleName: String = "ResolveAliases"

221

222

def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {

223

case Project(projectList, child) if child.resolved =>

224

val resolvedProjectList = projectList.map {

225

case u @ UnresolvedAlias(child, aliasFunc) =>

226

aliasFunc.map(_(child)).getOrElse(Alias(child, toPrettySQL(child))())

227

case other => other

228

}

229

Project(resolvedProjectList, child)

230

}

231

}

232

233

/**

234

* Resolves aggregate functions and ensures proper grouping

235

*/

236

object ResolveAggregateFunctions extends Rule[LogicalPlan] {

237

def ruleName: String = "ResolveAggregateFunctions"

238

239

def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {

240

case a @ Aggregate(grouping, aggregates, child) if child.resolved =>

241

val resolvedAggregates = aggregates.map { expr =>

242

expr.transformUp {

243

case UnresolvedFunction(name, children, isDistinct) =>

244

// Resolve to appropriate aggregate function

245

catalog.lookupFunction(name, children) match {

246

case info if info.isAggregate =>

247

info.builder(children)

248

case _ =>

249

throw new AnalysisException(s"${name.unquotedString} is not an aggregate function")

250

}

251

}

252

}

253

a.copy(aggregateExpressions = resolvedAggregates)

254

}

255

}

256

```

257

258

### Type Checking and Coercion

259

260

Type validation and automatic type conversions:

261

262

```scala { .api }

263

/**

264

* Result of expression type checking

265

*/

266

sealed trait TypeCheckResult {

267

def isSuccess: Boolean

268

}

269

270

case object TypeCheckSuccess extends TypeCheckResult {

271

def isSuccess: Boolean = true

272

}

273

274

case class TypeCheckFailure(message: String) extends TypeCheckResult {

275

def isSuccess: Boolean = false

276

}

277

278

/**

279

* Trait for expressions that expect specific input types

280

*/

281

trait ExpectsInputTypes extends Expression {

282

/** Expected input data types in order */

283

def inputTypes: Seq[AbstractDataType]

284

285

/** Check if input types match expectations */

286

override def checkInputDataTypes(): TypeCheckResult = {

287

val expectedTypes = inputTypes

288

val actualTypes = children.map(_.dataType)

289

290

if (expectedTypes.length != actualTypes.length) {

291

return TypeCheckFailure(

292

s"${prettyName} requires ${expectedTypes.length} arguments, " +

293

s"but ${actualTypes.length} were provided"

294

)

295

}

296

297

expectedTypes.zip(actualTypes).zipWithIndex.foreach {

298

case ((expected, actual), index) =>

299

if (!expected.acceptsType(actual)) {

300

return TypeCheckFailure(

301

s"${prettyName} argument ${index + 1} requires ${expected.simpleString} type, " +

302

s"not ${actual.catalogString}"

303

)

304

}

305

}

306

307

TypeCheckSuccess

308

}

309

}

310

311

/**

312

* Type coercion rules that automatically convert compatible types

313

*/

314

object TypeCoercion {

315

/** All type coercion rules */

316

val typeCoercionRules: List[Rule[LogicalPlan]] = List(

317

InConversion,

318

WidenSetOperationTypes,

319

PromoteStrings,

320

DecimalPrecision,

321

BooleanEquality,

322

FunctionArgumentConversion,

323

ConcatCoercion,

324

EltCoercion,

325

CaseWhenCoercion,

326

IfCoercion,

327

StackCoercion,

328

Division,

329

ImplicitTypeCasts,

330

DateTimeOperations,

331

WindowFrameCoercion

332

)

333

}

334

335

/**

336

* Promotes string literals in numeric contexts

337

*/

338

object PromoteStrings extends Rule[LogicalPlan] {

339

def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {

340

case q: LogicalPlan => q.transformExpressionsUp {

341

case a @ Add(left @ StringType(), right) if right.dataType.isInstanceOf[NumericType] =>

342

a.copy(left = Cast(left, right.dataType))

343

case a @ Add(left, right @ StringType()) if left.dataType.isInstanceOf[NumericType] =>

344

a.copy(right = Cast(right, left.dataType))

345

// Similar rules for other arithmetic operations...

346

}

347

}

348

}

349

```

350

351

**Usage Examples:**

352

353

```scala

354

// Type checking example

355

val addExpr = Add(

356

AttributeReference("age", IntegerType)(),

357

Literal("25", StringType) // String literal

358

)

359

360

// Before type coercion - would fail type check

361

val typeCheckResult = addExpr.checkInputDataTypes()

362

// typeCheckResult: TypeCheckFailure

363

364

// After applying PromoteStrings rule

365

val coercedExpr = addExpr.transform {

366

case Add(left, right @ Literal(_, StringType)) =>

367

Add(left, Cast(right, IntegerType))

368

}

369

370

// Now type check passes

371

val newResult = coercedExpr.checkInputDataTypes()

372

// newResult: TypeCheckSuccess

373

```

374

375

### Function Registry

376

377

Registry system for SQL functions and user-defined functions:

378

379

```scala { .api }

380

/**

381

* Registry for SQL functions

382

*/

383

trait FunctionRegistry {

384

/** Register a function */

385

def registerFunction(

386

name: FunctionIdentifier,

387

info: ExpressionInfo,

388

builder: FunctionBuilder): Unit

389

390

/** Look up function metadata */

391

def lookupFunction(name: FunctionIdentifier): Option[ExpressionInfo]

392

393

/** Look up function and create expression */

394

def lookupFunction(name: FunctionIdentifier, children: Seq[Expression]): Expression

395

396

/** List all registered functions */

397

def listFunction(): Seq[FunctionIdentifier]

398

399

/** Drop a function */

400

def dropFunction(name: FunctionIdentifier): Boolean

401

402

/** Clear all functions */

403

def clear(): Unit

404

405

/** Create a copy of this registry */

406

def clone(): FunctionRegistry

407

}

408

409

/**

410

* Simple implementation of FunctionRegistry

411

*/

412

class SimpleFunctionRegistry extends FunctionRegistry {

413

private val functionBuilders =

414

new mutable.HashMap[FunctionIdentifier, (ExpressionInfo, FunctionBuilder)]

415

416

def registerFunction(

417

name: FunctionIdentifier,

418

info: ExpressionInfo,

419

builder: FunctionBuilder): Unit = {

420

functionBuilders(normalizeIdentifier(name)) = (info, builder)

421

}

422

423

def lookupFunction(name: FunctionIdentifier): Option[ExpressionInfo] = {

424

functionBuilders.get(normalizeIdentifier(name)).map(_._1)

425

}

426

427

def lookupFunction(name: FunctionIdentifier, children: Seq[Expression]): Expression = {

428

val normalizedName = normalizeIdentifier(name)

429

functionBuilders.get(normalizedName) match {

430

case Some((info, builder)) =>

431

try {

432

builder(children)

433

} catch {

434

case ex: Exception =>

435

throw new AnalysisException(s"Invalid function call: ${name.unquotedString}", cause = Some(ex))

436

}

437

case None =>

438

throw new AnalysisException(s"Undefined function: ${name.unquotedString}")

439

}

440

}

441

442

private def normalizeIdentifier(name: FunctionIdentifier): FunctionIdentifier = {

443

FunctionIdentifier(name.funcName.toLowerCase, name.database.map(_.toLowerCase))

444

}

445

}

446

447

/**

448

* Function builder type alias

449

*/

450

type FunctionBuilder = Seq[Expression] => Expression

451

452

/**

453

* Information about a function for the catalog

454

*/

455

case class ExpressionInfo(

456

className: String,

457

name: String,

458

usage: String,

459

extended: String,

460

note: String = "",

461

group: String = "",

462

since: String = "",

463

deprecated: String = "") {

464

465

def getUsage: String = if (usage.nonEmpty) usage else "N/A"

466

def getExtended: String = if (extended.nonEmpty) extended else "N/A"

467

}

468

469

/**

470

* Identifier for functions (name + optional database)

471

*/

472

case class FunctionIdentifier(funcName: String, database: Option[String] = None) {

473

def unquotedString: String =

474

database.map(db => s"$db.$funcName").getOrElse(funcName)

475

476

def quotedString: String =

477

database.map(db => s"`$db`.`$funcName`").getOrElse(s"`$funcName`")

478

479

override def toString: String = unquotedString

480

}

481

```

482

483

**Usage Examples:**

484

485

```scala

486

// Create function registry

487

val registry = new SimpleFunctionRegistry()

488

489

// Register built-in functions

490

registry.registerFunction(

491

FunctionIdentifier("upper"),

492

ExpressionInfo("org.apache.spark.sql.catalyst.expressions.Upper", "upper",

493

"upper(str) - Returns str with all characters changed to uppercase"),

494

(children: Seq[Expression]) => {

495

require(children.length == 1, "upper requires exactly 1 argument")

496

Upper(children.head)

497

}

498

)

499

500

registry.registerFunction(

501

FunctionIdentifier("length"),

502

ExpressionInfo("org.apache.spark.sql.catalyst.expressions.Length", "length",

503

"length(str) - Returns the character length of str"),

504

(children: Seq[Expression]) => {

505

require(children.length == 1, "length requires exactly 1 argument")

506

Length(children.head)

507

}

508

)

509

510

// Look up and use functions

511

val upperFunc = registry.lookupFunction(

512

FunctionIdentifier("upper"),

513

Seq(Literal("hello"))

514

)

515

// upperFunc: Upper(Literal("hello"))

516

517

val lengthFunc = registry.lookupFunction(

518

FunctionIdentifier("length"),

519

Seq(AttributeReference("name", StringType)())

520

)

521

// lengthFunc: Length(AttributeReference("name", StringType))

522

523

// List registered functions

524

val allFunctions = registry.listFunction()

525

// allFunctions: Seq(FunctionIdentifier("upper"), FunctionIdentifier("length"))

526

```

527

528

### Unresolved Expressions

529

530

Placeholder expressions used before analysis resolution:

531

532

```scala { .api }

533

/**

534

* Unresolved attribute reference (before resolution)

535

*/

536

case class UnresolvedAttribute(nameParts: Seq[String]) extends Attribute with Unevaluable {

537

def name: String = nameParts.mkString(".")

538

def exprId: ExprId = throw new UnresolvedException(this, "exprId")

539

def dataType: DataType = throw new UnresolvedException(this, "dataType")

540

def nullable: Boolean = throw new UnresolvedException(this, "nullable")

541

def qualifiers: Seq[String] = throw new UnresolvedException(this, "qualifiers")

542

def metadata: Metadata = throw new UnresolvedException(this, "metadata")

543

544

override def toString: String = s"'${nameParts.mkString(".")}"

545

546

def toAttribute: Attribute = throw new UnresolvedException(this, "toAttribute")

547

def newInstance(): NamedExpression = this

548

def withName(newName: String): Attribute = UnresolvedAttribute(Seq(newName))

549

def withQualifier(newQualifier: Seq[String]): Attribute = this

550

def withExprId(newExprId: ExprId): Attribute = this

551

def withDataType(newType: DataType): Attribute = this

552

def withNullability(newNullability: Boolean): Attribute = this

553

def withMetadata(newMetadata: Metadata): Attribute = this

554

}

555

556

/**

557

* Unresolved function call (before resolution)

558

*/

559

case class UnresolvedFunction(

560

name: FunctionIdentifier,

561

children: Seq[Expression],

562

isDistinct: Boolean) extends Expression with Unevaluable {

563

564

def dataType: DataType = throw new UnresolvedException(this, "dataType")

565

def nullable: Boolean = throw new UnresolvedException(this, "nullable")

566

567

override def toString: String = {

568

val distinct = if (isDistinct) "DISTINCT " else ""

569

s"'${name.unquotedString}($distinct${children.mkString(", ")})"

570

}

571

572

override def prettyName: String = name.unquotedString

573

}

574

575

/**

576

* Unresolved star expression (*)

577

*/

578

case class UnresolvedStar(target: Option[Seq[String]]) extends Star with Unevaluable {

579

override def toString: String = target match {

580

case None => "*"

581

case Some(prefix) => prefix.mkString(".") + ".*"

582

}

583

584

def expand(input: LogicalPlan, resolver: Resolver): Seq[NamedExpression] = {

585

val exprs = input.output

586

target match {

587

case Some(parts) if parts.length == 1 =>

588

// Qualified star like "table.*"

589

val qualifier = parts.head

590

exprs.filter(_.qualifiers.nonEmpty && resolver(_.qualifiers.last, qualifier))

591

case _ =>

592

// Unqualified star "*"

593

exprs

594

}

595

}

596

}

597

598

/**

599

* Unresolved alias (before name resolution)

600

*/

601

case class UnresolvedAlias(

602

child: Expression,

603

aliasFunc: Option[Expression => String] = None) extends UnaryExpression with NamedExpression with Unevaluable {

604

605

def name: String = throw new UnresolvedException(this, "name")

606

def exprId: ExprId = throw new UnresolvedException(this, "exprId")

607

def qualifiers: Seq[String] = throw new UnresolvedException(this, "qualifiers")

608

def toAttribute: Attribute = throw new UnresolvedException(this, "toAttribute")

609

def newInstance(): NamedExpression = throw new UnresolvedException(this, "newInstance")

610

def metadata: Metadata = Metadata.empty

611

612

override def dataType: DataType = throw new UnresolvedException(this, "dataType")

613

override def nullable: Boolean = throw new UnresolvedException(this, "nullable")

614

override def toString: String = s"unresolvedalias(${child})"

615

}

616

617

/**

618

* Exception thrown when accessing unresolved properties

619

*/

620

class UnresolvedException[TreeType <: TreeNode[_]](tree: TreeType, function: String)

621

extends AnalysisException(

622

s"Invalid call to $function on unresolved object, tree: $tree")

623

```

624

625

**Usage Examples:**

626

627

```scala

628

// Unresolved expressions before analysis

629

val unresolvedAttr = UnresolvedAttribute(Seq("users", "name"))

630

val unresolvedFunc = UnresolvedFunction(

631

FunctionIdentifier("upper"),

632

Seq(UnresolvedAttribute(Seq("name"))),

633

isDistinct = false

634

)

635

636

val unresolvedStar = UnresolvedStar(Some(Seq("users"))) // users.*

637

val globalStar = UnresolvedStar(None) // *

638

639

// These will be resolved during analysis

640

val project = Project(

641

Seq(

642

unresolvedStar.expand(inputPlan, resolver),

643

Alias(unresolvedFunc, "upper_name")()

644

),

645

inputPlan

646

)

647

648

// Accessing unresolved properties throws exceptions

649

try {

650

val dataType = unresolvedAttr.dataType // Throws UnresolvedException

651

} catch {

652

case _: UnresolvedException =>

653

println("Cannot access dataType on unresolved attribute")

654

}

655

```

656

657

### Analysis Exceptions

658

659

Exception types for analysis errors and validation failures:

660

661

```scala { .api }

662

/**

663

* Exception thrown when a query fails to analyze

664

*/

665

case class AnalysisException(

666

message: String,

667

line: Option[Int] = None,

668

startPosition: Option[Int] = None,

669

plan: Option[LogicalPlan] = None,

670

cause: Option[Throwable] = None) extends Exception(message, cause.orNull) with Serializable {

671

672

def withPosition(line: Option[Int], startPosition: Option[Int]): AnalysisException = {

673

copy(line = line, startPosition = startPosition)

674

}

675

676

override def getMessage: String = {

677

val planAnnotation = plan.map(p => s";\n$p").getOrElse("")

678

getSimpleMessage + planAnnotation

679

}

680

681

// Get message without plan annotation (for testing)

682

def getSimpleMessage: String = {

683

val positionInfo = (line, startPosition) match {

684

case (Some(l), Some(p)) => s" (line $l, pos $p)"

685

case (Some(l), None) => s" (line $l)"

686

case _ => ""

687

}

688

message + positionInfo

689

}

690

}

691

692

/**

693

* Base class for "not found" exceptions

694

*/

695

abstract class NoSuchItemException(protected val message: String) extends AnalysisException(message)

696

697

case class NoSuchDatabaseException(db: String) extends NoSuchItemException(s"Database '$db' not found")

698

699

case class NoSuchTableException(db: String, table: String)

700

extends NoSuchItemException(s"Table or view '$table' not found in database '$db'")

701

702

case class NoSuchFunctionException(db: String, func: String)

703

extends NoSuchItemException(s"Function '$func' not found in database '$db'")

704

705

case class NoSuchPartitionException(db: String, table: String, spec: TablePartitionSpec)

706

extends NoSuchItemException(s"Partition not found in table '$db.$table'")

707

708

case class TableAlreadyExistsException(db: String, table: String)

709

extends AnalysisException(s"Table '$table' already exists in database '$db'")

710

711

case class DatabaseAlreadyExistsException(db: String)

712

extends AnalysisException(s"Database '$db' already exists")

713

714

case class FunctionAlreadyExistsException(db: String, func: String)

715

extends AnalysisException(s"Function '$func' already exists in database '$db'")

716

```

717

718

**Usage Examples:**

719

720

```scala

721

// Analysis exception handling

722

try {

723

val analyzedPlan = analyzer.execute(unresolvedPlan)

724

} catch {

725

case ae: AnalysisException =>

726

println(s"Analysis failed: ${ae.getSimpleMessage}")

727

ae.plan.foreach(p => println(s"Problematic plan: $p"))

728

729

case nste: NoSuchTableException =>

730

println(s"Table not found: ${nste.getMessage}")

731

732

case nsde: NoSuchDatabaseException =>

733

println(s"Database not found: ${nsde.getMessage}")

734

}

735

736

// Create analysis exceptions with position info

737

val exception = AnalysisException(

738

"Column 'xyz' not found",

739

line = Some(5),

740

startPosition = Some(12),

741

plan = Some(problematicPlan)

742

)

743

744

throw exception.withPosition(Some(10), Some(5))

745

```