From 68c8e183e0e380caabde586fd23906c8110c2ea9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Helge=20Br=C3=BCgner?= Date: Thu, 10 Nov 2022 17:18:37 -0800 Subject: [PATCH] Pass the SparkSession when resolving the tables in the DeltaTableBuilder Signed-off-by: Helge Bruegner ## Description - Forward the SparkSession when resolving the tables in the DeltaTableBuilder - Remove some unused imports Resolves #1475 / ## Does this PR introduce _any_ user-facing changes? No. Closes delta-io/delta#1476 Signed-off-by: Shixiong Zhu GitOrigin-RevId: 880de0f55ee79289cecd19d74c4c177c76d30aeb --- .../main/scala/io/delta/tables/DeltaTableBuilder.scala | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/io/delta/tables/DeltaTableBuilder.scala b/core/src/main/scala/io/delta/tables/DeltaTableBuilder.scala index 74a2f2de8e..63a1ec1560 100644 --- a/core/src/main/scala/io/delta/tables/DeltaTableBuilder.scala +++ b/core/src/main/scala/io/delta/tables/DeltaTableBuilder.scala @@ -25,12 +25,10 @@ import io.delta.tables.execution._ import org.apache.spark.annotation._ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.logical.{CreateTable, LeafNode, LogicalPlan, ReplaceTable} +import org.apache.spark.sql.catalyst.plans.logical.{CreateTable, LogicalPlan, ReplaceTable} import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.execution.SQLExecution -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DataType, StructField, StructType} /** @@ -365,9 +363,9 @@ class DeltaTableBuilder private[tables]( // Return DeltaTable Object. if (DeltaTableUtils.isValidPath(tableId)) { - DeltaTable.forPath(location.get) + DeltaTable.forPath(spark, location.get) } else { - DeltaTable.forName(this.identifier) + DeltaTable.forName(spark, this.identifier) } } }