Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -372,12 +372,6 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
if !r.hiveQlTable.isPartitioned && shouldConvertMetastoreParquet(r) =>
InsertIntoTable(convertToParquetRelation(r), partition, child, overwrite, ifNotExists)

// Write path
case InsertIntoHiveTable(r: MetastoreRelation, partition, child, overwrite, ifNotExists)
// Inserting into partitioned table is not supported in Parquet data source (yet).
if !r.hiveQlTable.isPartitioned && shouldConvertMetastoreParquet(r) =>
InsertIntoTable(convertToParquetRelation(r), partition, child, overwrite, ifNotExists)

// Read path
case relation: MetastoreRelation if shouldConvertMetastoreParquet(relation) =>
val parquetRelation = convertToParquetRelation(relation)
Expand Down Expand Up @@ -416,12 +410,6 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
if !r.hiveQlTable.isPartitioned && shouldConvertMetastoreOrc(r) =>
InsertIntoTable(convertToOrcRelation(r), partition, child, overwrite, ifNotExists)

// Write path
case InsertIntoHiveTable(r: MetastoreRelation, partition, child, overwrite, ifNotExists)
// Inserting into partitioned table is not supported in Orc data source (yet).
if !r.hiveQlTable.isPartitioned && shouldConvertMetastoreOrc(r) =>
InsertIntoTable(convertToOrcRelation(r), partition, child, overwrite, ifNotExists)

// Read path
case relation: MetastoreRelation if shouldConvertMetastoreOrc(relation) =>
val orcRelation = convertToOrcRelation(relation)
Expand Down Expand Up @@ -489,30 +477,3 @@ private[hive] object MetaStorePartitionedTableFileCatalog {
}
}
}

/**
* A logical plan representing insertion into Hive table.
* This plan ignores nullability of ArrayType, MapType, StructType unlike InsertIntoTable
* because Hive table doesn't have nullability for ARRAY, MAP, STRUCT types.
*/
private[hive] case class InsertIntoHiveTable(
table: MetastoreRelation,
partition: Map[String, Option[String]],
child: LogicalPlan,
overwrite: Boolean,
ifNotExists: Boolean)
extends LogicalPlan with Command {

override def children: Seq[LogicalPlan] = child :: Nil
override def output: Seq[Attribute] = Seq.empty

val numDynamicPartitions = partition.values.count(_.isEmpty)

// This is the expected schema of the table prepared to be inserted into,
// including dynamic partition columns.
val tableOutput = table.attributes ++ table.partitionKeys.takeRight(numDynamicPartitions)

override lazy val resolved: Boolean = childrenResolved && child.output.zip(tableOutput).forall {
case (childAttr, tableAttr) => childAttr.dataType.sameType(tableAttr.dataType)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -44,12 +44,7 @@ private[hive] trait HiveStrategies {
def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
case logical.InsertIntoTable(
table: MetastoreRelation, partition, child, overwrite, ifNotExists) =>
execution.InsertIntoHiveTable(
table, partition, planLater(child), overwrite, ifNotExists) :: Nil
case hive.InsertIntoHiveTable(
table: MetastoreRelation, partition, child, overwrite, ifNotExists) =>
execution.InsertIntoHiveTable(
table, partition, planLater(child), overwrite, ifNotExists) :: Nil
InsertIntoHiveTable(table, partition, planLater(child), overwrite, ifNotExists) :: Nil
case _ => Nil
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,6 @@ import org.apache.spark.sql.catalyst.planning.PhysicalOperation
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.util._
import org.apache.spark.sql.execution.command._
import org.apache.spark.sql.hive.{InsertIntoHiveTable => LogicalInsertIntoHiveTable}
import org.apache.spark.sql.hive.test.{TestHive, TestHiveQueryExecution}

/**
Expand Down Expand Up @@ -349,7 +348,6 @@ abstract class HiveComparisonTest
val containsCommands = originalQuery.analyzed.collectFirst {
case _: Command => ()
case _: InsertIntoTable => ()
case _: LogicalInsertIntoHiveTable => ()
}.nonEmpty

if (containsCommands) {
Expand Down