diff --git a/build/sbt-launch-lib.bash b/build/sbt-launch-lib.bash index 44d37fbf3d0..644e7419f9e 100755 --- a/build/sbt-launch-lib.bash +++ b/build/sbt-launch-lib.bash @@ -38,7 +38,12 @@ dlog () { acquire_sbt_jar () { SBT_VERSION=`awk -F "=" '/sbt\.version/ {print $2}' ./project/build.properties` - URL1=https://dl.bintray.com/typesafe/ivy-releases/org.scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch.jar + # Download sbt from mirror URL if the environment variable is provided + if [[ "${SBT_VERSION}" == "0.13.18" ]] && [[ -n "${SBT_MIRROR_JAR_URL}" ]]; then + URL1="${SBT_MIRROR_JAR_URL}" + else + URL1="https://repo.typesafe.com/typesafe/ivy-releases/org.scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch.jar" + fi JAR=build/sbt-launch-${SBT_VERSION}.jar sbt_jar=$JAR @@ -47,7 +52,7 @@ acquire_sbt_jar () { # Download sbt launch jar if it hasn't been downloaded yet if [ ! -f "${JAR}" ]; then # Download - printf "Attempting to fetch sbt\n" + printf 'Attempting to fetch sbt from %s\n' "${URL1}" JAR_DL="${JAR}.part" if [ $(command -v curl) ]; then curl --fail --location --silent ${URL1} > "${JAR_DL}" &&\ diff --git a/contribs/src/main/scala/io/delta/storage/IBMCOSLogStore.scala b/contribs/src/main/scala/io/delta/storage/IBMCOSLogStore.scala index 8a8196a3d02..f3b088ead11 100644 --- a/contribs/src/main/scala/io/delta/storage/IBMCOSLogStore.scala +++ b/contribs/src/main/scala/io/delta/storage/IBMCOSLogStore.scala @@ -61,14 +61,10 @@ class IBMCOSLogStore(sparkConf: SparkConf, hadoopConf: Configuration) if (exists && overwrite == false) { throw new FileAlreadyExistsException(path.toString) } else { - // create is atomic + // write is atomic when overwrite == false val stream = fs.create(path, overwrite) try { - var writeSize = 0L - actions.map(_ + "\n").map(_.getBytes(UTF_8)).foreach(action => { - stream.write(action) - writeSize += action.length - }) + actions.map(_ + "\n").map(_.getBytes(UTF_8)).foreach(stream.write) stream.close() } catch { case e: IOException if isPreconditionFailure(e) => diff --git a/core/src/main/scala/io/delta/sql/DeltaSparkSessionExtension.scala b/core/src/main/scala/io/delta/sql/DeltaSparkSessionExtension.scala index 55509696a66..f7647ee16b7 100644 --- a/core/src/main/scala/io/delta/sql/DeltaSparkSessionExtension.scala +++ b/core/src/main/scala/io/delta/sql/DeltaSparkSessionExtension.scala @@ -76,7 +76,7 @@ class DeltaSparkSessionExtension extends (SparkSessionExtensions => Unit) { new DeltaSqlParser(parser) } extensions.injectResolutionRule { session => - new DeltaAnalysis(session, session.sessionState.conf) + new DeltaAnalysis(session) } extensions.injectCheckRule { session => new DeltaUnsupportedOperationsCheck(session) diff --git a/core/src/main/scala/io/delta/tables/DeltaMergeBuilder.scala b/core/src/main/scala/io/delta/tables/DeltaMergeBuilder.scala index f1255e851a1..e7ce9c189d5 100644 --- a/core/src/main/scala/io/delta/tables/DeltaMergeBuilder.scala +++ b/core/src/main/scala/io/delta/tables/DeltaMergeBuilder.scala @@ -20,6 +20,8 @@ import scala.collection.JavaConverters._ import scala.collection.Map import org.apache.spark.sql.delta.{DeltaErrors, PreprocessTableMerge} +import org.apache.spark.sql.delta.DeltaViewHelper +import org.apache.spark.sql.delta.commands.MergeIntoCommand import org.apache.spark.sql.delta.metering.DeltaLogging import org.apache.spark.sql.delta.util.AnalysisHelper @@ -30,6 +32,7 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions.{AttributeReference, NamedExpression} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.functions.expr +import org.apache.spark.sql.internal.SQLConf /** * Builder to specify how to merge data from source DataFrame into the target Delta table. @@ -217,8 +220,13 @@ class DeltaMergeBuilder private( if (!resolvedMergeInto.resolved) { throw DeltaErrors.analysisException("Failed to resolve\n", plan = Some(resolvedMergeInto)) } + val strippedMergeInto = resolvedMergeInto.copy( + target = DeltaViewHelper.stripTempViewForMerge(resolvedMergeInto.target, SQLConf.get) + ) // Preprocess the actions and verify - val mergeIntoCommand = PreprocessTableMerge(sparkSession.sessionState.conf)(resolvedMergeInto) + val mergeIntoCommand = + PreprocessTableMerge(sparkSession.sessionState.conf)(strippedMergeInto) + .asInstanceOf[MergeIntoCommand] sparkSession.sessionState.analyzer.checkAnalysis(mergeIntoCommand) mergeIntoCommand.run(sparkSession) } diff --git a/core/src/main/scala/io/delta/tables/DeltaTable.scala b/core/src/main/scala/io/delta/tables/DeltaTable.scala index b353e28b0a9..2fde7554b2b 100644 --- a/core/src/main/scala/io/delta/tables/DeltaTable.scala +++ b/core/src/main/scala/io/delta/tables/DeltaTable.scala @@ -618,6 +618,8 @@ object DeltaTable { new DeltaTable( sparkSession.table(tableName), DeltaTableV2(sparkSession, new Path(tbl.location), Some(tbl), Some(tableName))) + } else if (DeltaTableUtils.isValidPath(tableId)) { + forPath(sparkSession, tableId.table) } else { throw DeltaErrors.notADeltaTableException(DeltaTableIdentifier(table = Some(tableId))) } diff --git a/core/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/deltaMerge.scala b/core/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/deltaMerge.scala index d3a59715294..ed0f710c76d 100644 --- a/core/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/deltaMerge.scala +++ b/core/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/deltaMerge.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.plans.logical import java.util.Locale +import org.apache.spark.sql.delta.schema.SchemaMergingUtils import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.AnalysisException @@ -227,7 +228,9 @@ case class DeltaMergeInto( condition: Expression, matchedClauses: Seq[DeltaMergeIntoMatchedClause], notMatchedClauses: Seq[DeltaMergeIntoInsertClause], - migrateSchema: Boolean) extends Command with SupportsSubquery { + migrateSchema: Boolean, + finalSchema: Option[StructType]) + extends Command with SupportsSubquery { (matchedClauses ++ notMatchedClauses).foreach(_.verifyActions()) @@ -271,12 +274,13 @@ object DeltaMergeInto { condition, whenClauses.collect { case x: DeltaMergeIntoMatchedClause => x }, whenClauses.collect { case x: DeltaMergeIntoInsertClause => x }, - migrateSchema = false) + migrateSchema = false, + finalSchema = Some(target.schema)) } def resolveReferences(merge: DeltaMergeInto, conf: SQLConf)( resolveExpr: (Expression, LogicalPlan) => Expression): DeltaMergeInto = { - val DeltaMergeInto(target, source, condition, matchedClauses, notMatchedClause, _) = merge + val DeltaMergeInto(target, source, condition, matchedClauses, notMatchedClause, _, _) = merge // We must do manual resolution as the expressions in different clauses of the MERGE have // visibility of the source, the target or both. Additionally, the resolution logic operates @@ -403,10 +407,24 @@ object DeltaMergeInto { } val containsStarAction = (matchedClauses ++ notMatchedClause).flatMap(_.actions).exists(_.isInstanceOf[UnresolvedStar]) + + val migrateSchema = canAutoMigrate && containsStarAction + + val finalSchema = if (migrateSchema) { + // The implicit conversions flag allows any type to be merged from source to target if Spark + // SQL considers the source type implicitly castable to the target. Normally, mergeSchemas + // enforces Parquet-level write compatibility, which would mean an INT source can't be merged + // into a LONG target. + SchemaMergingUtils.mergeSchemas(target.schema, source.schema, allowImplicitConversions = true) + } else { + target.schema + } + val resolvedMerge = DeltaMergeInto( target, source, resolvedCond, resolvedMatchedClauses, resolvedNotMatchedClause, - migrateSchema = canAutoMigrate && containsStarAction) + migrateSchema = migrateSchema, + finalSchema = Some(finalSchema)) // Its possible that pre-resolved expressions (e.g. `sourceDF("key") = targetDF("key")`) have // attribute references that are not present in the output attributes of the children (i.e., diff --git a/core/src/main/scala/org/apache/spark/sql/delta/ConflictChecker.scala b/core/src/main/scala/org/apache/spark/sql/delta/ConflictChecker.scala new file mode 100644 index 00000000000..453a88c2972 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/sql/delta/ConflictChecker.scala @@ -0,0 +1,273 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed 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. + */ + +package org.apache.spark.sql.delta + +import java.util.concurrent.TimeUnit + +import scala.collection.mutable + +import org.apache.spark.sql.delta.actions._ +import org.apache.spark.sql.delta.metering.DeltaLogging +import org.apache.spark.sql.delta.util.FileNames + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionSet} + + +/** + * A class representing different attributes of current transaction needed for conflict detection. + * + * @param readPredicates - partition predicates by which files have been queried by the transaction + * @param readFiles - specific files that have been seen by the transaction + * @param readWholeTable - whether the whole table was read during the transaction + * @param readAppIds - appIds that have been seen by the transaction + * @param metadata - table metadata for the transaction + * @param actions - delta log actions that the transaction wants to commit + * @param deltaLog - [[DeltaLog]] corresponding to the target table + */ +private[delta] case class CurrentTransactionInfo( + readPredicates: Seq[Expression], + readFiles: Set[AddFile], + readWholeTable: Boolean, + readAppIds: Set[String], + metadata: Metadata, + actions: Seq[Action], + deltaLog: DeltaLog) { + +} + +/** + * Summary of the Winning commit against which we want to check the conflict + * @param actions - delta log actions committed by the winning commit + * @param commitVersion - winning commit version + */ +private[delta] case class WinningCommitSummary(actions: Seq[Action], commitVersion: Long) { + + val metadataUpdates: Seq[Metadata] = actions.collect { case a: Metadata => a } + val appLevelTransactions: Seq[SetTransaction] = actions.collect { case a: SetTransaction => a } + val protocol: Seq[Protocol] = actions.collect { case a: Protocol => a } + val commitInfo: Option[CommitInfo] = actions.collectFirst { case a: CommitInfo => a }.map( + ci => ci.copy(version = Some(commitVersion))) + val removedFiles: Seq[RemoveFile] = actions.collect { case a: RemoveFile => a } + val addedFiles: Seq[AddFile] = actions.collect { case a: AddFile => a } + val isBlindAppendOption: Option[Boolean] = commitInfo.flatMap(_.isBlindAppend) + val blindAppendAddedFiles: Seq[AddFile] = if (isBlindAppendOption.getOrElse(false)) { + addedFiles + } else { + Seq() + } + val changedDataAddedFiles: Seq[AddFile] = if (isBlindAppendOption.getOrElse(false)) { + Seq() + } else { + addedFiles + } + val onlyAddFiles: Boolean = actions.collect { case f: FileAction => f } + .forall(_.isInstanceOf[AddFile]) +} + +private[delta] class ConflictChecker( + spark: SparkSession, + currentTransactionInfo: CurrentTransactionInfo, + winningCommitVersion: Long, + isolationLevel: IsolationLevel, + logPrefixStr: String) extends DeltaLogging { + + protected val timingStats = mutable.HashMap[String, Long]() + protected val winningCommitSummary: WinningCommitSummary = createWinningCommitSummary() + + def checkConflicts(): Unit = { + checkProtocolCompatibility() + checkNoMetadataUpdates() + checkForAddedFilesThatShouldHaveBeenReadByCurrentTxn() + checkForDeletedFilesAgainstCurrentTxnReadFiles() + checkForDeletedFilesAgainstCurrentTxnDeletedFiles() + checkForUpdatedApplicationTransactionIdsThatCurrentTxnDependsOn() + reportMetrics() + } + + /** + * Initializes [[WinningCommitSummary]] for the already committed + * transaction (winning transaction). + */ + protected def createWinningCommitSummary(): WinningCommitSummary = { + recordTime("initialize-old-commit") { + val deltaLog = currentTransactionInfo.deltaLog + val winningCommitActions = deltaLog.store.read( + FileNames.deltaFile(deltaLog.logPath, winningCommitVersion)).map(Action.fromJson) + WinningCommitSummary(winningCommitActions, winningCommitVersion) + } + } + + /** + * Asserts that the client is up to date with the protocol and is allowed to read and write + * against the protocol set by the committed transaction. + */ + protected def checkProtocolCompatibility(): Unit = { + if (winningCommitSummary.protocol.nonEmpty) { + winningCommitSummary.protocol.foreach { p => + currentTransactionInfo.deltaLog.protocolRead(p) + currentTransactionInfo.deltaLog.protocolWrite(p) + } + currentTransactionInfo.actions.foreach { + case Protocol(_, _) => + throw DeltaErrors.protocolChangedException(winningCommitSummary.commitInfo) + case _ => + } + } + } + + /** + * Check if the committed transaction has changed metadata. + */ + protected def checkNoMetadataUpdates(): Unit = { + // Fail if the metadata is different than what the txn read. + if (winningCommitSummary.metadataUpdates.nonEmpty) { + throw DeltaErrors.metadataChangedException(winningCommitSummary.commitInfo) + } + } + + /** + * Check if the new files added by the already committed transactions should have been read by + * the current transaction. + */ + protected def checkForAddedFilesThatShouldHaveBeenReadByCurrentTxn(): Unit = { + recordTime("checked-appends") { + // Fail if new files have been added that the txn should have read. + val addedFilesToCheckForConflicts = isolationLevel match { + case Serializable => + winningCommitSummary.changedDataAddedFiles ++ winningCommitSummary.blindAppendAddedFiles + case WriteSerializable => + winningCommitSummary.changedDataAddedFiles // don't conflict with blind appends + case SnapshotIsolation => + Seq.empty + } + + import spark.implicits._ + val predicatesMatchingAddedFiles = ExpressionSet( + currentTransactionInfo.readPredicates).iterator.flatMap { p => + val conflictingFile = DeltaLog.filterFileList( + currentTransactionInfo.metadata.partitionSchema, + addedFilesToCheckForConflicts.toDF(), p :: Nil).as[AddFile].take(1) + + conflictingFile.headOption.map(f => getPrettyPartitionMessage(f.partitionValues)) + }.take(1).toArray + + if (predicatesMatchingAddedFiles.nonEmpty) { + val isWriteSerializable = isolationLevel == WriteSerializable + + val retryMsg = if (isWriteSerializable && winningCommitSummary.onlyAddFiles && + winningCommitSummary.isBlindAppendOption.isEmpty) { + // The transaction was made by an older version which did not set `isBlindAppend` flag + // So even if it looks like an append, we don't know for sure if it was a blind append + // or not. So we suggest them to upgrade all there workloads to latest version. + Some( + "Upgrading all your concurrent writers to use the latest Delta Lake may " + + "avoid this error. Please upgrade and then retry this operation again.") + } else None + throw DeltaErrors.concurrentAppendException( + winningCommitSummary.commitInfo, + predicatesMatchingAddedFiles.head, + retryMsg) + } + } + } + + /** + * Check if [[RemoveFile]] actions added by already committed transactions conflicts with files + * read by the current transaction. + */ + protected def checkForDeletedFilesAgainstCurrentTxnReadFiles(): Unit = { + recordTime("checked-deletes") { + // Fail if files have been deleted that the txn read. + val readFilePaths = currentTransactionInfo.readFiles.map( + f => f.path -> f.partitionValues).toMap + val deleteReadOverlap = winningCommitSummary.removedFiles + .find(r => readFilePaths.contains(r.path)) + if (deleteReadOverlap.nonEmpty) { + val filePath = deleteReadOverlap.get.path + val partition = getPrettyPartitionMessage(readFilePaths(filePath)) + throw DeltaErrors.concurrentDeleteReadException( + winningCommitSummary.commitInfo, s"$filePath in $partition") + } + if (winningCommitSummary.removedFiles.nonEmpty && currentTransactionInfo.readWholeTable) { + val filePath = winningCommitSummary.removedFiles.head.path + throw DeltaErrors.concurrentDeleteReadException( + winningCommitSummary.commitInfo, s"$filePath") + } + } + } + + /** + * Check if [[RemoveFile]] actions added by already committed transactions conflicts with + * [[RemoveFile]] actions this transaction is trying to add. + */ + protected def checkForDeletedFilesAgainstCurrentTxnDeletedFiles(): Unit = { + recordTime("checked-2x-deletes") { + // Fail if a file is deleted twice. + val txnDeletes = currentTransactionInfo.actions + .collect { case r: RemoveFile => r } + .map(_.path).toSet + val deleteOverlap = winningCommitSummary.removedFiles.map(_.path).toSet intersect txnDeletes + if (deleteOverlap.nonEmpty) { + throw DeltaErrors.concurrentDeleteDeleteException( + winningCommitSummary.commitInfo, deleteOverlap.head) + } + } + } + + /** + * Checks if the winning transaction corresponds to some AppId on which current transaction + * also depends. + */ + protected def checkForUpdatedApplicationTransactionIdsThatCurrentTxnDependsOn(): Unit = { + // Fail if the appIds seen by the current transaction has been updated by the winning + // transaction i.e. the winning transaction have [[SetTransaction]] corresponding to + // some appId on which current transaction depends on. Example - This can happen when + // multiple instances of the same streaming query are running at the same time. + val txnOverlap = winningCommitSummary.appLevelTransactions.map(_.appId).toSet intersect + currentTransactionInfo.readAppIds + if (txnOverlap.nonEmpty) { + throw DeltaErrors.concurrentTransactionException(winningCommitSummary.commitInfo) + } + } + + /** A helper function for pretty printing a specific partition directory. */ + protected def getPrettyPartitionMessage(partitionValues: Map[String, String]): String = { + val partitionColumns = currentTransactionInfo.metadata.partitionColumns + if (partitionColumns.isEmpty) { + "the root of the table" + } else { + val partition = partitionColumns.map { name => + s"$name=${partitionValues(name)}" + }.mkString("[", ", ", "]") + s"partition ${partition}" + } + } + + protected def recordTime[T](phase: String)(f: => T): T = { + val startTimeNs = System.nanoTime() + val ret = f + val timeTakenMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeNs) + timingStats += phase -> timeTakenMs + ret + } + + protected def reportMetrics(): Unit = { + val timingStr = timingStats.keys.toSeq.sorted.map(k => s"$k=${timingStats(k)}").mkString(",") + logInfo(s"[$logPrefixStr] Timing stats against $winningCommitVersion [$timingStr]") + } +} diff --git a/core/src/main/scala/org/apache/spark/sql/delta/DeltaAnalysis.scala b/core/src/main/scala/org/apache/spark/sql/delta/DeltaAnalysis.scala index 47a724d7024..2c88b6e4a89 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/DeltaAnalysis.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/DeltaAnalysis.scala @@ -18,13 +18,11 @@ package org.apache.spark.sql.delta import scala.collection.JavaConverters._ - // scalastyle:off import.ordering.noEmptyLine - import org.apache.spark.sql.delta.DeltaErrors.{TemporallyUnstableInputException, TimestampEarlierThanCommitRetentionException} import org.apache.spark.sql.delta.catalog.DeltaTableV2 import org.apache.spark.sql.delta.constraints.{AddConstraint, DropConstraint} -import org.apache.spark.sql.delta.files.TahoeLogFileIndex +import org.apache.spark.sql.delta.files.{TahoeFileIndex, TahoeLogFileIndex} import org.apache.spark.sql.delta.metering.DeltaLogging import org.apache.spark.sql.delta.schema.SchemaUtils import org.apache.spark.sql.delta.sources.DeltaDataSource @@ -54,7 +52,7 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap * Analysis rules for Delta. Currently, these rules enable schema enforcement / evolution with * INSERT INTO. */ -class DeltaAnalysis(session: SparkSession, conf: SQLConf) +class DeltaAnalysis(session: SparkSession) extends Rule[LogicalPlan] with AnalysisHelper with DeltaLogging { import session.sessionState.analyzer.SessionCatalogAndIdentifier @@ -156,7 +154,8 @@ class DeltaAnalysis(session: SparkSession, conf: SQLConf) s"WHEN NOT MATCHED clause in MERGE INTO.") } // rewrites Delta from V2 to V1 - val newTarget = stripTempViewWrapper(target).transformUp { case DeltaRelation(lr) => lr } + val newTarget = + stripTempViewForMergeWrapper(target).transformUp { case DeltaRelation(lr) => lr } // Even if we're merging into a non-Delta target, we will catch it later and throw an // exception. val deltaMerge = @@ -168,7 +167,7 @@ class DeltaAnalysis(session: SparkSession, conf: SQLConf) val d = if (deltaMerge.childrenResolved && !deltaMerge.resolved) { DeltaMergeInto.resolveReferences(deltaMerge, conf)(tryResolveReferences(session)) } else deltaMerge - d.copy(target = stripTempViewWrapper(d.target)) + d.copy(target = stripTempViewForMergeWrapper(d.target)) case AlterTableAddConstraintStatement( original @ SessionCatalogAndIdentifier(catalog, ident), constraintName, expr) => @@ -362,8 +361,11 @@ class DeltaAnalysis(session: SparkSession, conf: SQLConf) private def stripTempViewWrapper(plan: LogicalPlan): LogicalPlan = { DeltaViewHelper.stripTempView(plan, conf) } -} + private def stripTempViewForMergeWrapper(plan: LogicalPlan): LogicalPlan = { + DeltaViewHelper.stripTempViewForMerge(plan, conf) + } +} /** Matchers for dealing with a Delta table. */ object DeltaRelation { diff --git a/core/src/main/scala/org/apache/spark/sql/delta/DeltaHistoryManager.scala b/core/src/main/scala/org/apache/spark/sql/delta/DeltaHistoryManager.scala index 305981dcfe8..951b5cfbae8 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/DeltaHistoryManager.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/DeltaHistoryManager.scala @@ -218,8 +218,8 @@ class DeltaHistoryManager( smallestDeltaVersion = math.min(version, smallestDeltaVersion) // Note that we also check this condition at the end of the function - we check it - // here too to to try and avoid more file listing when it's unnecessary. - if (lastCompleteCheckpoint.exists(_ >= smallestDeltaVersion)) { + // here too to try and avoid more file listing when it's unnecessary. + if (lastCompleteCheckpoint.exists(_ >= smallestDeltaVersion - 1)) { return lastCompleteCheckpoint.get } } else if (FileNames.isCheckpointFile(nextFilePath)) { diff --git a/core/src/main/scala/org/apache/spark/sql/delta/DeltaLog.scala b/core/src/main/scala/org/apache/spark/sql/delta/DeltaLog.scala index 565efe53f4c..bcb6e0c0670 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/DeltaLog.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/DeltaLog.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.delta.actions._ import org.apache.spark.sql.delta.commands.WriteIntoDelta import org.apache.spark.sql.delta.files.{TahoeBatchFileIndex, TahoeLogFileIndex} import org.apache.spark.sql.delta.metering.DeltaLogging -import org.apache.spark.sql.delta.schema.SchemaUtils +import org.apache.spark.sql.delta.schema.{SchemaMergingUtils, SchemaUtils} import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.delta.storage.LogStoreProvider import com.google.common.cache.{CacheBuilder, RemovalNotification} @@ -205,7 +205,7 @@ class DeltaLog private( val txn = startTransaction() try { - SchemaUtils.checkColumnNameDuplication(txn.metadata.schema, "in the table schema") + SchemaMergingUtils.checkColumnNameDuplication(txn.metadata.schema, "in the table schema") } catch { case e: AnalysisException => throw new AnalysisException( diff --git a/core/src/main/scala/org/apache/spark/sql/delta/DeltaViewHelper.scala b/core/src/main/scala/org/apache/spark/sql/delta/DeltaViewHelper.scala index 41ae1c641ad..b383c5d710f 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/DeltaViewHelper.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/DeltaViewHelper.scala @@ -39,8 +39,9 @@ package org.apache.spark.sql.delta import org.apache.spark.sql.catalyst.catalog.CatalogTable +import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast, NamedExpression} -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, View} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, SubqueryAlias, View} import org.apache.spark.sql.internal.SQLConf object DeltaViewHelper { @@ -86,4 +87,10 @@ object DeltaViewHelper { case (_, originAttr) => originAttr } } + + def stripTempViewForMerge(plan: LogicalPlan, conf: SQLConf): LogicalPlan = { + // Check that the two expression lists have the same names and types in the same order, and + // are either attributes or direct casts of attributes. + stripTempView(plan, conf) + } } diff --git a/core/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala b/core/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala index 6a80b0b87a7..07a3a551353 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala @@ -18,7 +18,8 @@ package org.apache.spark.sql.delta import java.net.URI import java.nio.file.FileAlreadyExistsException -import java.util.{ConcurrentModificationException, Locale} +import java.util.{ConcurrentModificationException, Locale, UUID} +import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit.NANOSECONDS import scala.collection.mutable @@ -31,7 +32,7 @@ import org.apache.spark.sql.delta.actions._ import org.apache.spark.sql.delta.files._ import org.apache.spark.sql.delta.hooks.{GenerateSymlinkManifest, PostCommitHook} import org.apache.spark.sql.delta.metering.DeltaLogging -import org.apache.spark.sql.delta.schema.SchemaUtils +import org.apache.spark.sql.delta.schema.{SchemaMergingUtils, SchemaUtils} import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.delta.stats.FileSizeHistogram import org.apache.hadoop.fs.Path @@ -166,7 +167,8 @@ trait OptimisticTransactionImpl extends TransactionalWrite with SQLMetricsReport implicit val clock: Clock protected def spark = SparkSession.active - protected val _spark = spark + + private val txnId = UUID.randomUUID().toString /** Tracks the appIds that have been seen by this transaction. */ protected val readTxn = new ArrayBuffer[String] @@ -336,8 +338,8 @@ trait OptimisticTransactionImpl extends TransactionalWrite with SQLMetricsReport protected def verifyNewMetadata(metadata: Metadata): Unit = { assert(!CharVarcharUtils.hasCharVarchar(metadata.schema), "The schema in Delta log should not contain char/varchar type.") - SchemaUtils.checkColumnNameDuplication(metadata.schema, "in the metadata update") - SchemaUtils.checkFieldNames(SchemaUtils.explodeNestedFieldNames(metadata.dataSchema)) + SchemaMergingUtils.checkColumnNameDuplication(metadata.schema, "in the metadata update") + SchemaUtils.checkFieldNames(SchemaMergingUtils.explodeNestedFieldNames(metadata.dataSchema)) val partitionColCheckIsFatal = spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_PARTITION_COLUMN_CHECK_ENABLED) try { @@ -608,8 +610,13 @@ trait OptimisticTransactionImpl extends TransactionalWrite with SQLMetricsReport } } + private[delta] def isCommitLockEnabled: Boolean = { + spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_COMMIT_LOCK_ENABLED).getOrElse( + deltaLog.store.isPartialWriteVisible(deltaLog.logPath)) + } + private def lockCommitIfEnabled[T](body: => T): T = { - if (spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_COMMIT_LOCK_ENABLED)) { + if (isCommitLockEnabled) { deltaLog.lockInterruptibly(body) } else { body @@ -752,143 +759,67 @@ trait OptimisticTransactionImpl extends TransactionalWrite with SQLMetricsReport "delta.commit.retry.conflictCheck", tags = Map(TAG_LOG_STORE_CLASS -> deltaLog.store.getClass.getName)) { - import _spark.implicits._ - val nextAttemptVersion = getNextAttemptVersion(checkVersion) - (checkVersion until nextAttemptVersion).foreach { version => - val totalCheckAndRetryTime = clock.getTimeMillis() - commitAttemptStartTime - val baseLog = s" Version: $version Attempt: $attemptNumber Time: $totalCheckAndRetryTime ms" - logInfo("Checking for conflict" + baseLog) - - // Actions of a commit which went in before ours - val winningCommitActions = - deltaLog.store.read(deltaFile(deltaLog.logPath, version)).map(Action.fromJson) - - // Categorize all the actions that have happened since the transaction read. - val metadataUpdates = winningCommitActions.collect { case a: Metadata => a } - val removedFiles = winningCommitActions.collect { case a: RemoveFile => a } - val txns = winningCommitActions.collect { case a: SetTransaction => a } - val protocol = winningCommitActions.collect { case a: Protocol => a } - val commitInfo = winningCommitActions.collectFirst { case a: CommitInfo => a }.map( - ci => ci.copy(version = Some(version))) - - val blindAppendAddedFiles = mutable.ArrayBuffer[AddFile]() - val changedDataAddedFiles = mutable.ArrayBuffer[AddFile]() - - val isBlindAppendOption = commitInfo.flatMap(_.isBlindAppend) - if (isBlindAppendOption.getOrElse(false)) { - blindAppendAddedFiles ++= winningCommitActions.collect { case a: AddFile => a } - } else { - changedDataAddedFiles ++= winningCommitActions.collect { case a: AddFile => a } - } - val actionsCollectionCompleteLog = - s"Found ${metadataUpdates.length} metadata, ${removedFiles.length} removes, " + - s"${changedDataAddedFiles.length + blindAppendAddedFiles.length} adds" - logInfo(actionsCollectionCompleteLog + baseLog) - - // If the log protocol version was upgraded, make sure we are still okay. - // Fail the transaction if we're trying to upgrade protocol ourselves. - if (protocol.nonEmpty) { - protocol.foreach { p => - deltaLog.protocolRead(p) - deltaLog.protocolWrite(p) - } - actions.foreach { - case Protocol(_, _) => throw DeltaErrors.protocolChangedException(commitInfo) - case _ => - } - } - - // Fail if the metadata is different than what the txn read. - if (metadataUpdates.nonEmpty) { - throw DeltaErrors.metadataChangedException(commitInfo) - } - - // Fail if new files have been added that the txn should have read. - val addedFilesToCheckForConflicts = commitIsolationLevel match { - case Serializable => changedDataAddedFiles ++ blindAppendAddedFiles - case WriteSerializable => changedDataAddedFiles // don't conflict with blind appends - case SnapshotIsolation => Seq.empty - } - val predicatesMatchingAddedFiles = ExpressionSet(readPredicates).iterator.flatMap { p => - val conflictingFile = DeltaLog.filterFileList( - metadata.partitionSchema, - addedFilesToCheckForConflicts.toDF(), p :: Nil).as[AddFile].take(1) - - conflictingFile.headOption.map(f => getPrettyPartitionMessage(f.partitionValues)) - }.take(1).toArray - if (predicatesMatchingAddedFiles.nonEmpty) { - val isWriteSerializable = commitIsolationLevel == WriteSerializable - val onlyAddFiles = - winningCommitActions.collect { case f: FileAction => f }.forall(_.isInstanceOf[AddFile]) - - val retryMsg = - if (isWriteSerializable && onlyAddFiles && isBlindAppendOption.isEmpty) { - // The transaction was made by an older version which did not set `isBlindAppend` flag - // So even if it looks like an append, we don't know for sure if it was a blind append - // or not. So we suggest them to upgrade all there workloads to latest version. - Some( - "Upgrading all your concurrent writers to use the latest Delta Lake may " + - "avoid this error. Please upgrade and then retry this operation again.") - } else None - throw DeltaErrors.concurrentAppendException( - commitInfo, - predicatesMatchingAddedFiles.head, - retryMsg) - } - - // Fail if files have been deleted that the txn read. - val readFilePaths = readFiles.map(f => f.path -> f.partitionValues).toMap - val deleteReadOverlap = removedFiles.find(r => readFilePaths.contains(r.path)) - if (deleteReadOverlap.nonEmpty) { - val filePath = deleteReadOverlap.get.path - val partition = getPrettyPartitionMessage(readFilePaths(filePath)) - throw DeltaErrors.concurrentDeleteReadException(commitInfo, s"$filePath in $partition") - } - if (removedFiles.nonEmpty && readTheWholeTable) { - val filePath = removedFiles.head.path - throw DeltaErrors.concurrentDeleteReadException(commitInfo, s"$filePath") - } - - // Fail if a file is deleted twice. - val txnDeletes = actions.collect { case r: RemoveFile => r }.map(_.path).toSet - val deleteOverlap = removedFiles.map(_.path).toSet intersect txnDeletes - if (deleteOverlap.nonEmpty) { - throw DeltaErrors.concurrentDeleteDeleteException(commitInfo, deleteOverlap.head) - } - - // Fail if idempotent transactions have conflicted. - val txnOverlap = txns.map(_.appId).toSet intersect readTxn.toSet - if (txnOverlap.nonEmpty) { - throw DeltaErrors.concurrentTransactionException(commitInfo) + val logPrefixStr = s"[attempt $attemptNumber]" + val txnDetailsLogStr = { + var adds = 0L + var removes = 0L + actions.foreach { + case _: AddFile => adds += 1 + case _: RemoveFile => removes += 1 + case _ => } + s"$adds adds, $removes removes, ${readPredicates.size} read predicates, " + + s"${readFiles.size} read files" + } - logInfo("Completed checking for conflicts" + baseLog) + logInfo(s"$logPrefixStr Checking for conflicts with versions " + + s"[$checkVersion, $nextAttemptVersion) with current txn having $txnDetailsLogStr") + + val currentTransactionInfo = CurrentTransactionInfo( + readPredicates = readPredicates, + readFiles = readFiles.toSet, + readWholeTable = readTheWholeTable, + readAppIds = readTxn.toSet, + metadata = metadata, + actions = actions, + deltaLog = deltaLog) + + (checkVersion until nextAttemptVersion).foreach { otherCommitVersion => + checkForConflictsAgainstVersion( + currentTransactionInfo, + otherCommitVersion, + commitIsolationLevel) + logInfo(s"$logPrefixStr No conflicts in version $otherCommitVersion, " + + s"${clock.getTimeMillis() - commitAttemptStartTime} ms since start") } - logInfo(s"No logical conflicts with deltas [$checkVersion, $nextAttemptVersion), retrying.") + logInfo(s"$logPrefixStr No conflicts with versions [$checkVersion, $nextAttemptVersion) " + + s"with current txn having $txnDetailsLogStr, " + + s"${clock.getTimeMillis() - commitAttemptStartTime} ms since start") nextAttemptVersion } + protected def checkForConflictsAgainstVersion( + currentTransactionInfo: CurrentTransactionInfo, + otherCommitVersion: Long, + commitIsolationLevel: IsolationLevel): Unit = { + + val conflictChecker = new ConflictChecker( + spark, + currentTransactionInfo, + otherCommitVersion, + commitIsolationLevel, logPrefix) + conflictChecker.checkConflicts() + } + /** Returns the next attempt version given the last attempted version */ protected def getNextAttemptVersion(previousAttemptVersion: Long): Long = { deltaLog.update() deltaLog.snapshot.version + 1 } - /** A helper function for pretty printing a specific partition directory. */ - protected def getPrettyPartitionMessage(partitionValues: Map[String, String]): String = { - if (metadata.partitionColumns.isEmpty) { - "the root of the table" - } else { - val partition = metadata.partitionColumns.map { name => - s"$name=${partitionValues(name)}" - }.mkString("[", ", ", "]") - s"partition ${partition}" - } - } - /** Register a hook that will be executed once a commit is successful. */ def registerPostCommitHook(hook: PostCommitHook): Unit = { if (!postCommitHooks.contains(hook)) { @@ -927,23 +858,28 @@ trait OptimisticTransactionImpl extends TransactionalWrite with SQLMetricsReport } } + private lazy val logPrefix: String = { + def truncate(uuid: String): String = uuid.split("-").head + s"[tableId=${truncate(snapshot.metadata.id)},txnId=${truncate(txnId)}] " + } + override def logInfo(msg: => String): Unit = { - super.logInfo(s"[tableId=${snapshot.metadata.id}] " + msg) + super.logInfo(logPrefix + msg) } override def logWarning(msg: => String): Unit = { - super.logWarning(s"[tableId=${snapshot.metadata.id}] " + msg) + super.logWarning(logPrefix + msg) } override def logWarning(msg: => String, throwable: Throwable): Unit = { - super.logWarning(s"[tableId=${snapshot.metadata.id}] " + msg, throwable) + super.logWarning(logPrefix + msg, throwable) } override def logError(msg: => String): Unit = { - super.logError(s"[tableId=${snapshot.metadata.id}] " + msg) + super.logError(logPrefix + msg) } override def logError(msg: => String, throwable: Throwable): Unit = { - super.logError(s"[tableId=${snapshot.metadata.id}] " + msg, throwable) + super.logError(logPrefix + msg, throwable) } } diff --git a/core/src/main/scala/org/apache/spark/sql/delta/PreprocessTableMerge.scala b/core/src/main/scala/org/apache/spark/sql/delta/PreprocessTableMerge.scala index 3b247dbb51b..597af7130d6 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/PreprocessTableMerge.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/PreprocessTableMerge.scala @@ -18,30 +18,34 @@ package org.apache.spark.sql.delta import java.util.Locale -import scala.collection.mutable - import org.apache.spark.sql.delta.commands.MergeIntoCommand -import org.apache.spark.sql.delta.schema.SchemaUtils -import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, TypeCoercion, UnresolvedAttribute} -import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Cast, Expression, Literal, SubqueryExpression} +import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, Literal, SubqueryExpression} import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{StructField, StructType} case class PreprocessTableMerge(override val conf: SQLConf) extends Rule[LogicalPlan] with UpdateExpressionsSupport { override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperators { - case m: DeltaMergeInto if m.resolved => apply(m) + case m: DeltaMergeInto if m.resolved => apply(m, true) } - def apply(mergeInto: DeltaMergeInto): MergeIntoCommand = { - val DeltaMergeInto(target, source, condition, matched, notMatched, migrateSchema) = mergeInto + def apply(mergeInto: DeltaMergeInto, transformToCommand: Boolean): LogicalPlan = { + val DeltaMergeInto( + target, source, condition, matched, notMatched, migrateSchema, finalSchemaOpt) = mergeInto + + if (finalSchemaOpt.isEmpty) { + throw new AnalysisException("Target Table Final Schema is empty.") + } + + val finalSchema = finalSchemaOpt.get + def checkCondition(cond: Expression, conditionName: String): Unit = { if (!cond.deterministic) { throw DeltaErrors.nonDeterministicNotSupportedException( @@ -62,15 +66,10 @@ case class PreprocessTableMerge(override val conf: SQLConf) checkCondition(clause.condition.get, clause.clauseType.toUpperCase(Locale.ROOT)) } - val shouldAutoMigrate = conf.getConf(DeltaSQLConf.DELTA_SCHEMA_AUTO_MIGRATE) && migrateSchema - val finalSchema = if (shouldAutoMigrate) { - // The implicit conversions flag allows any type to be merged from source to target if Spark - // SQL considers the source type implicitly castable to the target. Normally, mergeSchemas - // enforces Parquet-level write compatibility, which would mean an INT source can't be merged - // into a LONG target. - SchemaUtils.mergeSchemas(target.schema, source.schema, allowImplicitConversions = true) - } else { - target.schema + val deltaLogicalPlan = EliminateSubqueryAliases(target) + val tahoeFileIndex = deltaLogicalPlan match { + case DeltaFullTable(index) => index + case o => throw DeltaErrors.notADeltaSourceException("MERGE", Some(o)) } val processedMatched = matched.map { @@ -94,15 +93,6 @@ case class PreprocessTableMerge(override val conf: SQLConf) UpdateOperation(a.targetColNameParts, a.expr) } - // The operations for new columns... - val newOpsFromTargetSchema = target.output.filterNot { col => - m.resolvedActions.exists { updateAct => - conf.resolver(updateAct.targetColNameParts.head, col.name) - } - }.map { col => - UpdateOperation(Seq(col.name), col) - } - // And construct operations for columns that the insert clause will add. val newOpsFromInsert = newColsFromInsert.map { col => UpdateOperation(Seq(col.name), Literal(null, col.dataType)) @@ -119,20 +109,30 @@ case class PreprocessTableMerge(override val conf: SQLConf) } } + val generatedColumns = GeneratedColumn.getGeneratedColumns( + tahoeFileIndex.snapshotAtAnalysis) + if (generatedColumns.nonEmpty && !deltaLogicalPlan.isInstanceOf[LogicalRelation]) { + throw DeltaErrors.updateOnTempViewWithGenerateColsNotSupported + } + // Use the helper methods for in UpdateExpressionsSupport to generate expressions such // that nested fields can be updated (only for existing columns). val alignedExprs = generateUpdateExpressions( finalSchemaExprs, - existingUpdateOps ++ newOpsFromTargetSchema ++ newOpsFromInsert, + existingUpdateOps ++ newOpsFromInsert, conf.resolver, - allowStructEvolution = shouldAutoMigrate, - generatedColumns = Nil) - .map(_.getOrElse { - // Should not happen - throw new IllegalStateException("Calling without generated columns should " + - "always return a update expression for each column") - }) - val alignedActions: Seq[DeltaMergeAction] = alignedExprs + allowStructEvolution = migrateSchema, + generatedColumns = generatedColumns) + + val alignedExprsWithGenerationExprs = + if (alignedExprs.forall(_.nonEmpty)) { + alignedExprs.map(_.get) + } else { + generateUpdateExprsForGeneratedColumns(target, generatedColumns, alignedExprs, + Some(finalSchemaExprs)) + } + + val alignedActions: Seq[DeltaMergeAction] = alignedExprsWithGenerationExprs .zip(finalSchemaExprs) .map { case (expr, attrib) => DeltaMergeAction(Seq(attrib.name), expr, targetColNameResolved = true) @@ -186,7 +186,7 @@ case class PreprocessTableMerge(override val conf: SQLConf) castIfNeeded( a.expr, targetAttrib.dataType, - allowStructEvolution = shouldAutoMigrate), + allowStructEvolution = migrateSchema), targetColNameResolved = true) }.getOrElse { // If a target table column was not found in the INSERT columns and expressions, @@ -201,13 +201,17 @@ case class PreprocessTableMerge(override val conf: SQLConf) m.copy(m.condition, alignedActions) } - val tahoeFileIndex = EliminateSubqueryAliases(target) match { - case DeltaFullTable(index) => index - case o => throw DeltaErrors.notADeltaSourceException("MERGE", Some(o)) + if (transformToCommand) { + val tahoeFileIndex = EliminateSubqueryAliases(target) match { + case DeltaFullTable(index) => index + case o => throw DeltaErrors.notADeltaSourceException("MERGE", Some(o)) + } + MergeIntoCommand( + source, target, tahoeFileIndex, condition, + processedMatched, processedNotMatched, finalSchemaOpt) + } else { + DeltaMergeInto(source, target, condition, + processedMatched, processedNotMatched, migrateSchema, finalSchemaOpt) } - - MergeIntoCommand( - source, target, tahoeFileIndex, condition, - processedMatched, processedNotMatched, Some(finalSchema)) } } diff --git a/core/src/main/scala/org/apache/spark/sql/delta/PreprocessTableUpdate.scala b/core/src/main/scala/org/apache/spark/sql/delta/PreprocessTableUpdate.scala index 1eafa4c3312..202d07a1592 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/PreprocessTableUpdate.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/PreprocessTableUpdate.scala @@ -18,14 +18,12 @@ package org.apache.spark.sql.delta import org.apache.spark.sql.delta.commands.UpdateCommand -import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases -import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Expression, NamedExpression, SubqueryExpression} +import org.apache.spark.sql.catalyst.expressions.SubqueryExpression import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.StructField /** * Preprocesses the [[DeltaUpdateTable]] logical plan before converting it to [[UpdateCommand]]. @@ -49,65 +47,6 @@ case class PreprocessTableUpdate(sqlConf: SQLConf) toCommand(u) } - /** - * Generate update expressions for generated columns that the user doesn't provide a update - * expression. For each item in `updateExprs` that's None, we will find its generation expression - * from `generatedColumns`. In order to resolve this generation expression, we will create a - * fake Project which contains all update expressions and resolve the generation expression with - * this project. Source columns of a generation expression will also be replaced with their - * corresponding update expressions. - * - * For example, given a table that has a generated column `g` defined as `c1 + 10`. For the - * following update command: - * - * UPDATE target SET c1 = c2 + 100, c2 = 1000 - * - * We will generate the update expression `(c2 + 100) + 10`` for column `g`. Note: in this update - * expression, we should use the old `c2` attribute rather than its new value 1000. - * - * @return a sequence of update expressions for all of columns in the table. - */ - private def generateUpdateExprsForGeneratedColumns( - update: DeltaUpdateTable, - generatedColumns: Seq[StructField], - updateExprs: Seq[Option[Expression]]): Seq[Expression] = { - assert( - update.child.output.size == updateExprs.length, - s"'generateUpdateExpressions' should return expressions that are aligned with the column " + - s"list. Expected size: ${update.child.output.size}, actual size: ${updateExprs.length}") - val attrsWithExprs = update.child.output.zip(updateExprs) - val exprsForProject = attrsWithExprs.flatMap { - case (attr, Some(expr)) => - // Create a named expression so that we can use it in Project - val exprForProject = Alias(expr, attr.name)() - Some(exprForProject.exprId -> exprForProject) - case (_, None) => None - }.toMap - // Create a fake Project to resolve the generation expressions - val fakePlan = Project(exprsForProject.values.toArray[NamedExpression], update.child) - attrsWithExprs.map { - case (_, Some(expr)) => expr - case (targetCol, None) => - // `targetCol` is a generated column and the user doesn't provide a update expression. - val resolvedExpr = - generatedColumns.find(f => conf.resolver(f.name, targetCol.name)) match { - case Some(field) => - val expr = GeneratedColumn.getGenerationExpression(field).get - resolveReferencesForExpressions(SparkSession.active, expr :: Nil, fakePlan).head - case None => - // Should not happen - throw new IllegalStateException(s"$targetCol is not a generated column " + - s"but is missing its update expression") - } - // As `resolvedExpr` will refer to attributes in `fakePlan`, we need to manually replace - // these attributes with their update expressions. - resolvedExpr.transform { - case a: AttributeReference if exprsForProject.contains(a.exprId) => - exprsForProject(a.exprId).child - } - } - } - def toCommand(update: DeltaUpdateTable): UpdateCommand = { val deltaLogicalNode = EliminateSubqueryAliases(update.child) val index = deltaLogicalNode match { @@ -140,7 +79,7 @@ case class PreprocessTableUpdate(sqlConf: SQLConf) } else { // Some expressions for generated columns are not specified by the user, so we need to // create them based on the generation expressions. - generateUpdateExprsForGeneratedColumns(update, generatedColumns, alignedUpdateExprs) + generateUpdateExprsForGeneratedColumns(update.child, generatedColumns, alignedUpdateExprs) } UpdateCommand( index, diff --git a/core/src/main/scala/org/apache/spark/sql/delta/UpdateExpressionsSupport.scala b/core/src/main/scala/org/apache/spark/sql/delta/UpdateExpressionsSupport.scala index 6e8ab0af850..ea5ffe1f069 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/UpdateExpressionsSupport.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/UpdateExpressionsSupport.scala @@ -20,9 +20,11 @@ import org.apache.spark.sql.delta.schema.SchemaUtils import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.delta.util.AnalysisHelper +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.SQLConfHelper import org.apache.spark.sql.catalyst.analysis.{CastSupport, Resolver} -import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Cast, CreateNamedStruct, Expression, ExtractValue, GetStructField, Literal, NamedExpression} +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, CreateNamedStruct, Expression, ExtractValue, GetStructField, Literal, NamedExpression} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} import org.apache.spark.sql.types._ /** @@ -203,7 +205,7 @@ trait UpdateExpressionsSupport extends CastSupport with SQLConfHelper with Analy Alias(GetStructField(fieldExpr, ordinal, Some(field.name)), field.name)() } // Recursively apply update operations to the children - val updatedChildExprs = generateUpdateExpressions( + val targetExprs = generateUpdateExpressions( childExprs, prefixMatchedOps.map(u => u.copy(targetColNameParts = u.targetColNameParts.tail)), resolver, @@ -218,7 +220,7 @@ trait UpdateExpressionsSupport extends CastSupport with SQLConfHelper with Analy }) // Reconstruct the expression for targetCol using its possibly updated children val namedStructExprs = fields - .zip(updatedChildExprs) + .zip(targetExprs) .flatMap { case (field, expr) => Seq(Literal(field.name), expr) } Some(CreateNamedStruct(namedStructExprs)) @@ -244,4 +246,72 @@ trait UpdateExpressionsSupport extends CastSupport with SQLConfHelper with Analy } generateUpdateExpressions(targetCols, updateOps, resolver, generatedColumns = generatedColumns) } + + /** + * Generate update expressions for generated columns that the user doesn't provide a update + * expression. For each item in `updateExprs` that's None, we will find its generation expression + * from `generatedColumns`. In order to resolve this generation expression, we will create a + * fake Project which contains all update expressions and resolve the generation expression with + * this project. Source columns of a generation expression will also be replaced with their + * corresponding update expressions. + * + * For example, given a table that has a generated column `g` defined as `c1 + 10`. For the + * following update command: + * + * UPDATE target SET c1 = c2 + 100, c2 = 1000 + * + * We will generate the update expression `(c2 + 100) + 10`` for column `g`. Note: in this update + * expression, we should use the old `c2` attribute rather than its new value 1000. + * + * @param updateTarget The logical plan of the table to be updated. + * @param generatedColumns A list of generated columns. + * @param updateExprs The aligned (with `finalSchemaExprs` if not None, or `updateTarget.output` + * otherwise) update actions. + * @param finalSchemaExprs In case of UPDATE in MERGE when schema evolution happened, this is + * the final schema of the target table. This might not be the same as + * the output of `updateTarget`. + * @return a sequence of update expressions for all of columns in the table. + */ + protected def generateUpdateExprsForGeneratedColumns( + updateTarget: LogicalPlan, + generatedColumns: Seq[StructField], + updateExprs: Seq[Option[Expression]], + finalSchemaExprs: Option[Seq[Attribute]] = None): Seq[Expression] = { + val targetExprs = finalSchemaExprs.getOrElse(updateTarget.output) + assert( + targetExprs.size == updateExprs.length, + s"'generateUpdateExpressions' should return expressions that are aligned with the column " + + s"list. Expected size: ${updateTarget.output.size}, actual size: ${updateExprs.length}") + val attrsWithExprs = targetExprs.zip(updateExprs) + val exprsForProject = attrsWithExprs.flatMap { + case (attr, Some(expr)) => + // Create a named expression so that we can use it in Project + val exprForProject = Alias(expr, attr.name)() + Some(exprForProject.exprId -> exprForProject) + case (_, None) => None + }.toMap + // Create a fake Project to resolve the generation expressions + val fakePlan = Project(exprsForProject.values.toArray[NamedExpression], updateTarget) + attrsWithExprs.map { + case (_, Some(expr)) => expr + case (targetCol, None) => + // `targetCol` is a generated column and the user doesn't provide a update expression. + val resolvedExpr = + generatedColumns.find(f => conf.resolver(f.name, targetCol.name)) match { + case Some(field) => + val expr = GeneratedColumn.getGenerationExpression(field).get + resolveReferencesForExpressions(SparkSession.active, expr :: Nil, fakePlan).head + case None => + // Should not happen + throw new IllegalStateException(s"$targetCol is not a generated column " + + s"but is missing its update expression") + } + // As `resolvedExpr` will refer to attributes in `fakePlan`, we need to manually replace + // these attributes with their update expressions. + resolvedExpr.transform { + case a: AttributeReference if exprsForProject.contains(a.exprId) => + exprsForProject(a.exprId).child + } + } + } } diff --git a/core/src/main/scala/org/apache/spark/sql/delta/actions/actions.scala b/core/src/main/scala/org/apache/spark/sql/delta/actions/actions.scala index cd0622d49f6..3823b76a866 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/actions/actions.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/actions/actions.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.delta.util.JsonUtils import com.fasterxml.jackson.annotation.{JsonIgnore, JsonInclude} import com.fasterxml.jackson.core.JsonGenerator -import com.fasterxml.jackson.databind.{JsonSerializer, SerializerProvider} +import com.fasterxml.jackson.databind.{JsonSerializer, ObjectMapper, SerializerProvider} import com.fasterxml.jackson.databind.annotation.{JsonDeserialize, JsonSerialize} import org.codehaus.jackson.annotate.JsonRawValue @@ -255,6 +255,7 @@ case class AddFile( def copyWithoutTag(tag: AddFile.Tags.KeyType): AddFile = copy(tags = Option(tags).getOrElse(Map.empty) - tag.name) + } object AddFile { diff --git a/core/src/main/scala/org/apache/spark/sql/delta/catalog/DeltaCatalog.scala b/core/src/main/scala/org/apache/spark/sql/delta/catalog/DeltaCatalog.scala index 634e64bc885..9c2fbf25394 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/catalog/DeltaCatalog.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/catalog/DeltaCatalog.scala @@ -113,13 +113,16 @@ class DeltaCatalog extends DelegatingCatalogExtension } else { Option(allTableProperties.get("location")) } - val locUriOpt = location.map(CatalogUtils.stringToURI) + val id = TableIdentifier(ident.name(), ident.namespace().lastOption) + var locUriOpt = location.map(CatalogUtils.stringToURI) + val existingTableOpt = getExistingTableIfExists(id) + val loc = locUriOpt + .orElse(existingTableOpt.flatMap(_.storage.locationUri)) + .getOrElse(spark.sessionState.catalog.defaultTablePath(id)) val storage = DataSource.buildStorageFormatFromOptions(writeOptions) - .copy(locationUri = locUriOpt) + .copy(locationUri = Option(loc)) val tableType = if (location.isDefined) CatalogTableType.EXTERNAL else CatalogTableType.MANAGED - val id = TableIdentifier(ident.name(), ident.namespace().lastOption) - val loc = new Path(locUriOpt.getOrElse(spark.sessionState.catalog.defaultTablePath(id))) val commentOpt = Option(allTableProperties.get("comment")) val tableDesc = new CatalogTable( @@ -138,7 +141,7 @@ class DeltaCatalog extends DelegatingCatalogExtension val writer = sourceQuery.map { df => WriteIntoDelta( - DeltaLog.forTable(spark, loc), + DeltaLog.forTable(spark, new Path(loc)), operation.mode, new DeltaOptions(withDb.storage.properties, spark.sessionState.conf), withDb.partitionColumnNames, @@ -149,7 +152,7 @@ class DeltaCatalog extends DelegatingCatalogExtension CreateDeltaTableCommand( withDb, - getExistingTableIfExists(tableDesc), + existingTableOpt, operation.mode, writer, operation, @@ -309,19 +312,19 @@ class DeltaCatalog extends DelegatingCatalogExtension } /** Checks if a table already exists for the provided identifier. */ - private def getExistingTableIfExists(table: CatalogTable): Option[CatalogTable] = { + private def getExistingTableIfExists(table: TableIdentifier): Option[CatalogTable] = { // If this is a path identifier, we cannot return an existing CatalogTable. The Create command // will check the file system itself if (isPathIdentifier(table)) return None - val tableExists = catalog.tableExists(table.identifier) + val tableExists = catalog.tableExists(table) if (tableExists) { - val oldTable = catalog.getTableMetadata(table.identifier) + val oldTable = catalog.getTableMetadata(table) if (oldTable.tableType == CatalogTableType.VIEW) { throw new AnalysisException( - s"${table.identifier} is a view. You may not write data into a view.") + s"$table is a view. You may not write data into a view.") } if (!DeltaSourceUtils.isDeltaTable(oldTable.provider)) { - throw new AnalysisException(s"${table.identifier} is not a Delta table. Please drop this " + + throw new AnalysisException(s"$table is not a Delta table. Please drop this " + "table first if you would like to recreate it with Delta Lake.") } Some(oldTable) @@ -589,7 +592,11 @@ trait SupportsPathIdentifier extends TableCatalog { self: DeltaCatalog => } protected def isPathIdentifier(table: CatalogTable): Boolean = { - isPathIdentifier(Identifier.of(table.identifier.database.toArray, table.identifier.table)) + isPathIdentifier(table.identifier) + } + + protected def isPathIdentifier(tableIdentifier: TableIdentifier) : Boolean = { + isPathIdentifier(Identifier.of(tableIdentifier.database.toArray, tableIdentifier.table)) } override def tableExists(ident: Identifier): Boolean = { diff --git a/core/src/main/scala/org/apache/spark/sql/delta/catalog/DeltaTableV2.scala b/core/src/main/scala/org/apache/spark/sql/delta/catalog/DeltaTableV2.scala index 3ff2b15a735..06c840fc703 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/catalog/DeltaTableV2.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/catalog/DeltaTableV2.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.delta.{DeltaErrors, DeltaLog, DeltaOperations, Delta import org.apache.spark.sql.delta.GeneratedColumn import org.apache.spark.sql.delta.commands.WriteIntoDelta import org.apache.spark.sql.delta.metering.DeltaLogging +import org.apache.spark.sql.delta.schema.SchemaUtils import org.apache.spark.sql.delta.sources.{DeltaDataSource, DeltaSourceUtils} import org.apache.hadoop.fs.Path @@ -101,7 +102,9 @@ case class DeltaTableV2( } private lazy val tableSchema: StructType = - GeneratedColumn.removeGenerationExpressions(snapshot.schema) + GeneratedColumn.removeGenerationExpressions( + SchemaUtils.dropNullTypeColumns(snapshot.schema)) + override def schema(): StructType = tableSchema @@ -136,6 +139,8 @@ case class DeltaTableV2( * paths. */ def toBaseRelation: BaseRelation = { + // force update() if necessary in DataFrameReader.load code + snapshot if (!deltaLog.tableExists) { val id = catalogTable.map(ct => DeltaTableIdentifier(table = Some(ct.identifier))) .getOrElse(DeltaTableIdentifier(path = Some(path.toString))) diff --git a/core/src/main/scala/org/apache/spark/sql/delta/commands/ConvertToDeltaCommand.scala b/core/src/main/scala/org/apache/spark/sql/delta/commands/ConvertToDeltaCommand.scala index 86e022e7a2b..25463d9b735 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/commands/ConvertToDeltaCommand.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/commands/ConvertToDeltaCommand.scala @@ -26,7 +26,7 @@ import scala.util.control.NonFatal import org.apache.spark.sql.delta._ import org.apache.spark.sql.delta.actions.{AddFile, CommitInfo, Metadata, Protocol} import org.apache.spark.sql.delta.catalog.DeltaTableV2 -import org.apache.spark.sql.delta.schema.SchemaUtils +import org.apache.spark.sql.delta.schema.SchemaMergingUtils import org.apache.spark.sql.delta.sources.{DeltaSourceUtils, DeltaSQLConf} import org.apache.spark.sql.delta.util.{DateFormatter, DeltaFileOperations, PartitionUtils, TimestampFormatter} import org.apache.spark.sql.delta.util.FileNames.deltaFile @@ -346,7 +346,7 @@ abstract class ConvertToDeltaCommandBase( // Here we explicitly mark the inferred schema nullable. This also means we don't // currently support specifying non-nullable columns after the table conversion. val batchSchema = getSchemaForBatch(spark, batch, serializableConfiguration).asNullable - dataSchema = SchemaUtils.mergeSchemas(dataSchema, batchSchema) + dataSchema = SchemaMergingUtils.mergeSchemas(dataSchema, batchSchema) } } @@ -500,8 +500,8 @@ abstract class ConvertToDeltaCommandBase( /** * This method is forked from [[ParquetFileFormat]]. The only change here is that we use - * our SchemaUtils.mergeSchemas() instead of StructType.merge(), where we allow upcast between - * ByteType, ShortType and IntegerType. + * our SchemaMergingUtils.mergeSchemas() instead of StructType.merge(), + * where we allow upcast between ByteType, ShortType and IntegerType. * * Figures out a merged Parquet schema with a distributed Spark job. * @@ -570,7 +570,7 @@ abstract class ConvertToDeltaCommandBase( footers.tail.foreach { footer => val schema = ParquetFileFormat.readSchemaFromFooter(footer, converter) try { - mergedSchema = SchemaUtils.mergeSchemas(mergedSchema, schema) + mergedSchema = SchemaMergingUtils.mergeSchemas(mergedSchema, schema) } catch { case cause: AnalysisException => throw new SparkException( s"Failed to merge schema of file ${footer.getFile}:\n${schema.treeString}", cause) @@ -585,7 +585,7 @@ abstract class ConvertToDeltaCommandBase( } else { var finalSchema = partiallyMergedSchemas.head partiallyMergedSchemas.tail.foreach { schema => - finalSchema = SchemaUtils.mergeSchemas(finalSchema, schema) + finalSchema = SchemaMergingUtils.mergeSchemas(finalSchema, schema) } Some(finalSchema) } diff --git a/core/src/main/scala/org/apache/spark/sql/delta/commands/CreateDeltaTableCommand.scala b/core/src/main/scala/org/apache/spark/sql/delta/commands/CreateDeltaTableCommand.scala index ce06ca55fcf..b9c16cc52da 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/commands/CreateDeltaTableCommand.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/commands/CreateDeltaTableCommand.scala @@ -57,6 +57,8 @@ case class CreateDeltaTableCommand( with DeltaLogging { override def run(sparkSession: SparkSession): Seq[Row] = { + var table = this.table + assert(table.tableType != CatalogTableType.VIEW) assert(table.identifier.database.isDefined, "Database should've been fixed at analysis") // There is a subtle race condition here, where the table can be created by someone else @@ -89,8 +91,9 @@ case class CreateDeltaTableCommand( val loc = sparkSession.sessionState.catalog.defaultTablePath(table.identifier) table.copy(storage = table.storage.copy(locationUri = Some(loc))) } else { - // We are defining a new external table - assert(table.tableType == CatalogTableType.EXTERNAL) + // 1. We are defining a new external table + // 2. It's a managed table which already has the location populated. This can happen in DSV2 + // CTAS flow. table } diff --git a/core/src/main/scala/org/apache/spark/sql/delta/commands/MergeIntoCommand.scala b/core/src/main/scala/org/apache/spark/sql/delta/commands/MergeIntoCommand.scala index 3c2e03e9247..2873fd31913 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/commands/MergeIntoCommand.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/commands/MergeIntoCommand.scala @@ -114,7 +114,8 @@ case class MergeStats( targetRowsCopied: Long, targetRowsUpdated: Long, targetRowsInserted: Long, - targetRowsDeleted: Long) + targetRowsDeleted: Long +) object MergeStats { @@ -168,8 +169,7 @@ object MergeStats { updateExprs = null, insertConditionExpr = null, insertExprs = null, - deleteConditionExpr = null - ) + deleteConditionExpr = null) } } diff --git a/core/src/main/scala/org/apache/spark/sql/delta/commands/alterDeltaTableCommands.scala b/core/src/main/scala/org/apache/spark/sql/delta/commands/alterDeltaTableCommands.scala index d417322e939..9924554b856 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/commands/alterDeltaTableCommands.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/commands/alterDeltaTableCommands.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.delta._ import org.apache.spark.sql.delta.actions.Protocol import org.apache.spark.sql.delta.catalog.DeltaTableV2 import org.apache.spark.sql.delta.constraints.Constraints -import org.apache.spark.sql.delta.schema.SchemaUtils +import org.apache.spark.sql.delta.schema.{SchemaMergingUtils, SchemaUtils} import org.apache.spark.sql.delta.schema.SchemaUtils.transformColumnsStructs import org.apache.spark.sql.delta.sources.DeltaSQLConf @@ -205,8 +205,8 @@ case class AlterTableAddColumnsDeltaCommand( SchemaUtils.addColumn(schema, column, position) } - SchemaUtils.checkColumnNameDuplication(newSchema, "in adding columns") - ParquetSchemaConverter.checkFieldNames(SchemaUtils.explodeNestedFieldNames(newSchema)) + SchemaMergingUtils.checkColumnNameDuplication(newSchema, "in adding columns") + ParquetSchemaConverter.checkFieldNames(SchemaMergingUtils.explodeNestedFieldNames(newSchema)) val newMetadata = metadata.copy(schemaString = newSchema.json) txn.updateMetadata(newMetadata) @@ -437,8 +437,8 @@ case class AlterTableReplaceColumnsDeltaCommand( val newSchema = SchemaUtils.changeDataType(existingSchema, changingSchema, resolver) .asInstanceOf[StructType] - SchemaUtils.checkColumnNameDuplication(newSchema, "in replacing columns") - ParquetSchemaConverter.checkFieldNames(SchemaUtils.explodeNestedFieldNames(newSchema)) + SchemaMergingUtils.checkColumnNameDuplication(newSchema, "in replacing columns") + ParquetSchemaConverter.checkFieldNames(SchemaMergingUtils.explodeNestedFieldNames(newSchema)) val newMetadata = metadata.copy(schemaString = newSchema.json) txn.updateMetadata(newMetadata) diff --git a/core/src/main/scala/org/apache/spark/sql/delta/constraints/DeltaInvariantCheckerExec.scala b/core/src/main/scala/org/apache/spark/sql/delta/constraints/DeltaInvariantCheckerExec.scala index 84edc50fd69..1118bb227eb 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/constraints/DeltaInvariantCheckerExec.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/constraints/DeltaInvariantCheckerExec.scala @@ -70,7 +70,9 @@ case class DeltaInvariantCheckerExec( child: SparkPlan, constraints: Seq[Constraint]) extends UnaryExecNode { - private def spark: SparkSession = sqlContext.sparkSession + // TODO: we can replace `SparkSession.active` with `session` once OSS Delta + // upgrades to Spark 3.2 + private def spark: SparkSession = SparkSession.active override def output: Seq[Attribute] = child.output diff --git a/core/src/main/scala/org/apache/spark/sql/delta/schema/ImplicitMetadataOperation.scala b/core/src/main/scala/org/apache/spark/sql/delta/schema/ImplicitMetadataOperation.scala index b68e83a9633..8ce57f7f4aa 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/schema/ImplicitMetadataOperation.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/schema/ImplicitMetadataOperation.scala @@ -65,7 +65,7 @@ trait ImplicitMetadataOperation extends DeltaLogging { } else { Set.empty[String] } - SchemaUtils.mergeSchemas( + SchemaMergingUtils.mergeSchemas( txn.metadata.schema, dataSchema, fixedTypeColumns = fixedTypeColumns) diff --git a/core/src/main/scala/org/apache/spark/sql/delta/schema/SchemaMergingUtils.scala b/core/src/main/scala/org/apache/spark/sql/delta/schema/SchemaMergingUtils.scala new file mode 100644 index 00000000000..b7693c8143a --- /dev/null +++ b/core/src/main/scala/org/apache/spark/sql/delta/schema/SchemaMergingUtils.scala @@ -0,0 +1,251 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed 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. + */ + +package org.apache.spark.sql.delta.schema + +import java.util.Locale + +import scala.util.control.NonFatal + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.analysis.{TypeCoercion, UnresolvedAttribute} +import org.apache.spark.sql.catalyst.expressions.Literal +import org.apache.spark.sql.catalyst.plans.logical.DeltaMergeInto +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.types.{ArrayType, ByteType, DataType, DecimalType, IntegerType, MapType, NullType, ShortType, StructField, StructType} + +/** + * Utils to merge table schema with data schema. + * This is split from SchemaUtils, because finalSchema is introduced into DeltaMergeInto, + * and resolving the final schema is now part of [[DeltaMergeInto.resolveReferences]]. + */ +object SchemaMergingUtils { + /** + * Returns all column names in this schema as a flat list. For example, a schema like: + * | - a + * | | - 1 + * | | - 2 + * | - b + * | - c + * | | - nest + * | | - 3 + * will get flattened to: "a", "a.1", "a.2", "b", "c", "c.nest", "c.nest.3" + */ + def explodeNestedFieldNames(schema: StructType): Seq[String] = { + def explode(schema: StructType): Seq[Seq[String]] = { + def recurseIntoComplexTypes(complexType: DataType): Seq[Seq[String]] = { + complexType match { + case s: StructType => explode(s) + case a: ArrayType => recurseIntoComplexTypes(a.elementType) + case m: MapType => + recurseIntoComplexTypes(m.keyType).map(Seq("key") ++ _) ++ + recurseIntoComplexTypes(m.valueType).map(Seq("value") ++ _) + case _ => Nil + } + } + + schema.flatMap { + case StructField(name, s: StructType, _, _) => + Seq(Seq(name)) ++ explode(s).map(nested => Seq(name) ++ nested) + case StructField(name, a: ArrayType, _, _) => + Seq(Seq(name)) ++ recurseIntoComplexTypes(a).map(nested => Seq(name) ++ nested) + case StructField(name, m: MapType, _, _) => + Seq(Seq(name)) ++ recurseIntoComplexTypes(m).map(nested => Seq(name) ++ nested) + case f => Seq(f.name) :: Nil + } + } + + explode(schema).map(UnresolvedAttribute.apply(_).name) + } + + /** + * Checks if input column names have duplicate identifiers. This throws an exception if + * the duplication exists. + * + * @param schema the schema to check for duplicates + * @param colType column type name, used in an exception message + */ + def checkColumnNameDuplication(schema: StructType, colType: String): Unit = { + val columnNames = explodeNestedFieldNames(schema) + // scalastyle:off caselocale + val names = columnNames.map(_.toLowerCase) + // scalastyle:on caselocale + if (names.distinct.length != names.length) { + val duplicateColumns = names.groupBy(identity).collect { + case (x, ys) if ys.length > 1 => s"$x" + } + throw new AnalysisException( + s"Found duplicate column(s) $colType: ${duplicateColumns.mkString(", ")}") + } + } + + /** + * Check whether we can write to the Delta table, which has `tableSchema`, using a query that has + * `dataSchema`. Our rules are that: + * - `dataSchema` may be missing columns or have additional columns + * - We don't trust the nullability in `dataSchema`. Assume fields are nullable. + * - We only allow nested StructType expansions. For all other complex types, we check for + * strict equality + * - `dataSchema` can't have duplicate column names. Columns that only differ by case are also + * not allowed. + * The following merging strategy is + * applied: + * - The name of the current field is used. + * - The data types are merged by calling this function. + * - We respect the current field's nullability. + * - The metadata is current field's metadata. + * + * Schema merging occurs in a case insensitive manner. Hence, column names that only differ + * by case are not accepted in the `dataSchema`. + * + * @param tableSchema The current schema of the table. + * @param dataSchema The schema of the new data being written. + * @param allowImplicitConversions Whether to allow Spark SQL implicit conversions. By default, + * we merge according to Parquet write compatibility - for + * example, an integer type data field will throw when merged to a + * string type table field, because int and string aren't stored + * the same way in Parquet files. With this flag enabled, the + * merge will succeed, because once we get to write time Spark SQL + * will support implicitly converting the int to a string. + * @param keepExistingType Whether to keep existing types instead of trying to merge types. + * @param fixedTypeColumns The set of columns whose type should not be changed in any case. + */ + def mergeSchemas( + tableSchema: StructType, + dataSchema: StructType, + allowImplicitConversions: Boolean = false, + keepExistingType: Boolean = false, + fixedTypeColumns: Set[String] = Set.empty): StructType = { + checkColumnNameDuplication(dataSchema, "in the data to save") + def merge( + current: DataType, + update: DataType, + fixedTypeColumnsSet: Set[String] = Set.empty): DataType = { + (current, update) match { + case (StructType(currentFields), StructType(updateFields)) => + // Merge existing fields. + val updateFieldMap = toFieldMap(updateFields) + val updatedCurrentFields = currentFields.map { currentField => + updateFieldMap.get(currentField.name) match { + case Some(updateField) => + if (fixedTypeColumnsSet.contains(currentField.name.toLowerCase(Locale.ROOT)) && + currentField.dataType != updateField.dataType) { + throw new AnalysisException( + s"Column ${currentField.name} is a generated column " + + "or a column used by a generated column. " + + s"The data type is ${currentField.dataType.sql}. " + + s"It doesn't accept data type ${updateField.dataType.sql}") + } + try { + StructField( + currentField.name, + merge(currentField.dataType, updateField.dataType), + currentField.nullable, + currentField.metadata) + } catch { + case NonFatal(e) => + throw new AnalysisException(s"Failed to merge fields '${currentField.name}' " + + s"and '${updateField.name}'. " + e.getMessage) + } + case None => + // Retain the old field. + currentField + } + } + + // Identify the newly added fields. + val nameToFieldMap = toFieldMap(currentFields) + val newFields = updateFields.filterNot(f => nameToFieldMap.contains(f.name)) + + // Create the merged struct, the new fields are appended at the end of the struct. + StructType(updatedCurrentFields ++ newFields) + case (ArrayType(currentElementType, currentContainsNull), + ArrayType(updateElementType, _)) => + ArrayType( + merge(currentElementType, updateElementType), + currentContainsNull) + case (MapType(currentKeyType, currentElementType, currentContainsNull), + MapType(updateKeyType, updateElementType, _)) => + MapType( + merge(currentKeyType, updateKeyType), + merge(currentElementType, updateElementType), + currentContainsNull) + + // Simply keeps the existing type for primitive types + case (current, update) if keepExistingType => current + + // If implicit conversions are allowed, that means we can use any valid implicit cast to + // perform the merge. + case (current, update) + if allowImplicitConversions && typeForImplicitCast(update, current).isDefined => + typeForImplicitCast(update, current).get + + case (DecimalType.Fixed(leftPrecision, leftScale), + DecimalType.Fixed(rightPrecision, rightScale)) => + if ((leftPrecision == rightPrecision) && (leftScale == rightScale)) { + current + } else if ((leftPrecision != rightPrecision) && (leftScale != rightScale)) { + throw new AnalysisException("Failed to merge decimal types with incompatible " + + s"precision $leftPrecision and $rightPrecision & scale $leftScale and $rightScale") + } else if (leftPrecision != rightPrecision) { + throw new AnalysisException("Failed to merge decimal types with incompatible " + + s"precision $leftPrecision and $rightPrecision") + } else { + throw new AnalysisException("Failed to merge decimal types with incompatible " + + s"scale $leftScale and $rightScale") + } + case _ if current == update => + current + + // Parquet physically stores ByteType, ShortType and IntType as IntType, so when a parquet + // column is of one of these three types, you can read this column as any of these three + // types. Since Parquet doesn't complain, we should also allow upcasting among these + // three types when merging schemas. + case (ByteType, ShortType) => ShortType + case (ByteType, IntegerType) => IntegerType + + case (ShortType, ByteType) => ShortType + case (ShortType, IntegerType) => IntegerType + + case (IntegerType, ShortType) => IntegerType + case (IntegerType, ByteType) => IntegerType + + case (NullType, _) => + update + case (_, NullType) => + current + case _ => + throw new AnalysisException( + s"Failed to merge incompatible data types $current and $update") + } + } + merge(tableSchema, dataSchema, fixedTypeColumns.map(_.toLowerCase(Locale.ROOT))) + .asInstanceOf[StructType] + } + + /** + * Try to cast the source data type to the target type, returning the final type or None if + * there's no valid cast. + */ + private def typeForImplicitCast(sourceType: DataType, targetType: DataType): Option[DataType] = { + TypeCoercion.ImplicitTypeCasts.implicitCast(Literal.default(sourceType), targetType) + .map(_.dataType) + } + + def toFieldMap(fields: Seq[StructField]): Map[String, StructField] = { + CaseInsensitiveMap(fields.map(field => field.name -> field).toMap) + } +} diff --git a/core/src/main/scala/org/apache/spark/sql/delta/schema/SchemaUtils.scala b/core/src/main/scala/org/apache/spark/sql/delta/schema/SchemaUtils.scala index d54664de0a7..aacd2d797ad 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/schema/SchemaUtils.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/schema/SchemaUtils.scala @@ -17,27 +17,23 @@ package org.apache.spark.sql.delta.schema // scalastyle:off import.ordering.noEmptyLine -import java.util.Locale - import scala.collection.Set._ import scala.collection.mutable -import scala.util.control.NonFatal import org.apache.spark.sql.delta.{DeltaErrors, GeneratedColumn} +import org.apache.spark.sql.delta.schema.SchemaMergingUtils._ import org.apache.spark.sql.delta.sources.DeltaSourceUtils.GENERATION_EXPRESSION_METADATA_KEY import org.apache.spark.sql.delta.sources.DeltaSQLConf +import org.apache.spark.internal.Logging import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.analysis.{Resolver, TypeCoercion, UnresolvedAttribute} -import org.apache.spark.sql.catalyst.expressions.{Expression, Literal} -import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap -import org.apache.spark.sql.connector.catalog.Identifier +import org.apache.spark.sql.catalyst.analysis.{Resolver, UnresolvedAttribute} import org.apache.spark.sql.execution.datasources.parquet.ParquetSchemaConverter import org.apache.spark.sql.functions.{col, struct} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -object SchemaUtils { +object SchemaUtils extends Logging { // We use case insensitive resolution while writing into Delta val DELTA_COL_RESOLVER: (String, String) => Boolean = org.apache.spark.sql.catalyst.analysis.caseInsensitiveResolution @@ -112,6 +108,8 @@ object SchemaUtils { case st: StructType => val nested = st.fields.flatMap { f => if (f.dataType.isInstanceOf[NullType]) { + logInfo( + s"Dropping column '${f.name}' from nested column because it is of NullType.") None } else { Some(generateSelectExpr(f, nameStack :+ sf.name)) @@ -134,7 +132,12 @@ object SchemaUtils { } val selectExprs = schema.flatMap { f => - if (f.dataType.isInstanceOf[NullType]) None else Some(generateSelectExpr(f, Nil)) + if (f.dataType.isInstanceOf[NullType]) { + logInfo(s"Dropping column '${f.name}' from ${schema.toString} because it is of NullType.") + None + } else { + Some(generateSelectExpr(f, Nil)) + } } df.select(selectExprs: _*) } @@ -149,72 +152,15 @@ object SchemaUtils { struct.flatMap { case sf @ StructField(_, s: StructType, _, _) => Some(sf.copy(dataType = StructType(recurseAndRemove(s)))) - case StructField(_, n: NullType, _, _) => None + case StructField(name, n: NullType, _, _) => + logInfo(s"Dropping column '$name' because it is of NullType.") + None case other => Some(other) } } StructType(recurseAndRemove(schema)) } - /** - * Returns all column names in this schema as a flat list. For example, a schema like: - * | - a - * | | - 1 - * | | - 2 - * | - b - * | - c - * | | - nest - * | | - 3 - * will get flattened to: "a", "a.1", "a.2", "b", "c", "c.nest", "c.nest.3" - */ - def explodeNestedFieldNames(schema: StructType): Seq[String] = { - def explode(schema: StructType): Seq[Seq[String]] = { - def recurseIntoComplexTypes(complexType: DataType): Seq[Seq[String]] = { - complexType match { - case s: StructType => explode(s) - case a: ArrayType => recurseIntoComplexTypes(a.elementType) - case m: MapType => - recurseIntoComplexTypes(m.keyType).map(Seq("key") ++ _) ++ - recurseIntoComplexTypes(m.valueType).map(Seq("value") ++ _) - case _ => Nil - } - } - - schema.flatMap { - case StructField(name, s: StructType, _, _) => - Seq(Seq(name)) ++ explode(s).map(nested => Seq(name) ++ nested) - case StructField(name, a: ArrayType, _, _) => - Seq(Seq(name)) ++ recurseIntoComplexTypes(a).map(nested => Seq(name) ++ nested) - case StructField(name, m: MapType, _, _) => - Seq(Seq(name)) ++ recurseIntoComplexTypes(m).map(nested => Seq(name) ++ nested) - case f => Seq(f.name) :: Nil - } - } - - explode(schema).map(UnresolvedAttribute.apply(_).name) - } - - /** - * Checks if input column names have duplicate identifiers. This throws an exception if - * the duplication exists. - * - * @param schema the schema to check for duplicates - * @param colType column type name, used in an exception message - */ - def checkColumnNameDuplication(schema: StructType, colType: String): Unit = { - val columnNames = explodeNestedFieldNames(schema) - // scalastyle:off caselocale - val names = columnNames.map(_.toLowerCase) - // scalastyle:on caselocale - if (names.distinct.length != names.length) { - val duplicateColumns = names.groupBy(identity).collect { - case (x, ys) if ys.length > 1 => s"$x" - } - throw new AnalysisException( - s"Found duplicate column(s) $colType: ${duplicateColumns.mkString(", ")}") - } - } - /** * Rewrite the query field names according to the table schema. This method assumes that all * schema validation checks have been made and this is the last operation before writing into @@ -783,159 +729,6 @@ object SchemaUtils { } } - /** - * Check whether we can write to the Delta table, which has `tableSchema`, using a query that has - * `dataSchema`. Our rules are that: - * - `dataSchema` may be missing columns or have additional columns - * - We don't trust the nullability in `dataSchema`. Assume fields are nullable. - * - We only allow nested StructType expansions. For all other complex types, we check for - * strict equality - * - `dataSchema` can't have duplicate column names. Columns that only differ by case are also - * not allowed. - * The following merging strategy is - * applied: - * - The name of the current field is used. - * - The data types are merged by calling this function. - * - We respect the current field's nullability. - * - The metadata is current field's metadata. - * - * Schema merging occurs in a case insensitive manner. Hence, column names that only differ - * by case are not accepted in the `dataSchema`. - * - * @param tableSchema The current schema of the table. - * @param dataSchema The schema of the new data being written. - * @param allowImplicitConversions Whether to allow Spark SQL implicit conversions. By default, - * we merge according to Parquet write compatibility - for - * example, an integer type data field will throw when merged to a - * string type table field, because int and string aren't stored - * the same way in Parquet files. With this flag enabled, the - * merge will succeed, because once we get to write time Spark SQL - * will support implicitly converting the int to a string. - * @param keepExistingType Whether to keep existing types instead of trying to merge types. - * @param fixedTypeColumns The set of columns whose type should not be changed in any case. - */ - def mergeSchemas( - tableSchema: StructType, - dataSchema: StructType, - allowImplicitConversions: Boolean = false, - keepExistingType: Boolean = false, - fixedTypeColumns: Set[String] = Set.empty): StructType = { - checkColumnNameDuplication(dataSchema, "in the data to save") - def merge( - current: DataType, - update: DataType, - fixedTypeColumnsSet: Set[String] = Set.empty): DataType = { - (current, update) match { - case (StructType(currentFields), StructType(updateFields)) => - // Merge existing fields. - val updateFieldMap = toFieldMap(updateFields) - val updatedCurrentFields = currentFields.map { currentField => - updateFieldMap.get(currentField.name) match { - case Some(updateField) => - if (fixedTypeColumnsSet.contains(currentField.name.toLowerCase(Locale.ROOT)) && - currentField.dataType != updateField.dataType) { - throw DeltaErrors.generatedColumnsUpdateColumnType(currentField, updateField) - } - try { - StructField( - currentField.name, - merge(currentField.dataType, updateField.dataType), - currentField.nullable, - currentField.metadata) - } catch { - case NonFatal(e) => - throw new AnalysisException(s"Failed to merge fields '${currentField.name}' " + - s"and '${updateField.name}'. " + e.getMessage) - } - case None => - // Retain the old field. - currentField - } - } - - // Identify the newly added fields. - val nameToFieldMap = toFieldMap(currentFields) - val newFields = updateFields.filterNot(f => nameToFieldMap.contains(f.name)) - - // Create the merged struct, the new fields are appended at the end of the struct. - StructType(updatedCurrentFields ++ newFields) - case (ArrayType(currentElementType, currentContainsNull), - ArrayType(updateElementType, _)) => - ArrayType( - merge(currentElementType, updateElementType), - currentContainsNull) - case (MapType(currentKeyType, currentElementType, currentContainsNull), - MapType(updateKeyType, updateElementType, _)) => - MapType( - merge(currentKeyType, updateKeyType), - merge(currentElementType, updateElementType), - currentContainsNull) - - // Simply keeps the existing type for primitive types - case (current, update) if keepExistingType => current - - // If implicit conversions are allowed, that means we can use any valid implicit cast to - // perform the merge. - case (current, update) - if allowImplicitConversions && typeForImplicitCast(update, current).isDefined => - typeForImplicitCast(update, current).get - - case (DecimalType.Fixed(leftPrecision, leftScale), - DecimalType.Fixed(rightPrecision, rightScale)) => - if ((leftPrecision == rightPrecision) && (leftScale == rightScale)) { - current - } else if ((leftPrecision != rightPrecision) && (leftScale != rightScale)) { - throw new AnalysisException("Failed to merge decimal types with incompatible " + - s"precision $leftPrecision and $rightPrecision & scale $leftScale and $rightScale") - } else if (leftPrecision != rightPrecision) { - throw new AnalysisException("Failed to merge decimal types with incompatible " + - s"precision $leftPrecision and $rightPrecision") - } else { - throw new AnalysisException("Failed to merge decimal types with incompatible " + - s"scale $leftScale and $rightScale") - } - case _ if current == update => - current - - // Parquet physically stores ByteType, ShortType and IntType as IntType, so when a parquet - // column is of one of these three types, you can read this column as any of these three - // types. Since Parquet doesn't complain, we should also allow upcasting among these - // three types when merging schemas. - case (ByteType, ShortType) => ShortType - case (ByteType, IntegerType) => IntegerType - - case (ShortType, ByteType) => ShortType - case (ShortType, IntegerType) => IntegerType - - case (IntegerType, ShortType) => IntegerType - case (IntegerType, ByteType) => IntegerType - - case (NullType, _) => - update - case (_, NullType) => - current - case _ => - throw new AnalysisException( - s"Failed to merge incompatible data types $current and $update") - } - } - merge(tableSchema, dataSchema, fixedTypeColumns.map(_.toLowerCase(Locale.ROOT))) - .asInstanceOf[StructType] - } - - /** - * Try to cast the source data type to the target type, returning the final type or None if - * there's no valid cast. - */ - private def typeForImplicitCast(sourceType: DataType, targetType: DataType): Option[DataType] = { - TypeCoercion.ImplicitTypeCasts.implicitCast(Literal.default(sourceType), targetType) - .map(_.dataType) - } - - private def toFieldMap(fields: Seq[StructField]): Map[String, StructField] = { - CaseInsensitiveMap(fields.map(field => field.name -> field).toMap) - } - /** * Transform (nested) columns in a schema. * diff --git a/core/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala b/core/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala index a5a280547e8..4c78ab04eda 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala @@ -49,7 +49,7 @@ trait DeltaSQLConfBase { .internal() .doc("Whether to lock a Delta table when doing a commit.") .booleanConf - .createWithDefault(false) + .createOptional val DELTA_USER_METADATA = buildConf("commitInfo.userMetadata") @@ -108,13 +108,6 @@ trait DeltaSQLConfBase { .booleanConf .createWithDefault(true) - val DELTA_COLLECT_STATS = - buildConf("stats.collect") - .internal() - .doc("When true, statistics are collected while writing files into a Delta table.") - .booleanConf - .createWithDefault(true) - val DELTA_IMPORT_BATCH_SIZE_STATS_COLLECTION = buildConf("import.batchSize.statsCollection") .internal() @@ -162,15 +155,6 @@ trait DeltaSQLConfBase { .booleanConf .createWithDefault(true) - val DELTA_STATS_SKIPPING_LOCAL_CACHE_MAX_NUM_FILES = - buildConf("stats.localCache.maxNumFiles") - .internal() - .doc("The maximum number of files for a table to be considered a 'delta small table'." + - "Some metadata operations (such as using data skipping) are optimized for small tables " + - "using driver local caching and local execution.") - .intConf - .createWithDefault(2000) - val DELTA_SNAPSHOT_ISOLATION = buildConf("snapshotIsolation.enabled") .internal() diff --git a/core/src/main/scala/org/apache/spark/sql/delta/stats/FileSizeHistogram.scala b/core/src/main/scala/org/apache/spark/sql/delta/stats/FileSizeHistogram.scala index bca1f66ea7b..741be2664ba 100644 --- a/core/src/main/scala/org/apache/spark/sql/delta/stats/FileSizeHistogram.scala +++ b/core/src/main/scala/org/apache/spark/sql/delta/stats/FileSizeHistogram.scala @@ -16,7 +16,7 @@ package org.apache.spark.sql.delta.stats -import java.util +import com.fasterxml.jackson.databind.annotation.JsonDeserialize import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.types.StructType @@ -30,8 +30,9 @@ import org.apache.spark.sql.types.StructType * @param totalBytes - an array of Long representing total number of bytes in different bins */ private[delta] case class FileSizeHistogram( - sortedBinBoundaries: Array[Long], - fileCounts: Array[Int], + @JsonDeserialize(contentAs = classOf[java.lang.Long]) + sortedBinBoundaries: IndexedSeq[Long], + fileCounts: Array[Long], totalBytes: Array[Long]) { require(sortedBinBoundaries.nonEmpty) @@ -45,16 +46,7 @@ private[delta] case class FileSizeHistogram( * Insert a given value into the appropriate histogram bin */ def insert(fileSize: Long): Unit = { - var index = util.Arrays.binarySearch(sortedBinBoundaries, fileSize) - // If the element is present in the array, it returns a non-negative result and that - // represents the bucket in which fileSize belongs. - // If the element is not present in the array, then it returns (-1 * insertion_point) - 1 - // where insertion_point is the index of the first element greater than the key. - if (index < 0) { - index = ((index + 1) * -1) - 1 - } - - // If fileSize is lesser than min bucket of histogram, then no need to update the histogram. + val index = FileSizeHistogram.getBinIndex(fileSize, sortedBinBoundaries) if (index >= 0) { fileCounts(index) += 1 totalBytes(index) += fileSize @@ -64,9 +56,27 @@ private[delta] case class FileSizeHistogram( private[delta] object FileSizeHistogram { - def apply(sortedBinBoundaries: Seq[Long]): FileSizeHistogram = { + /** + * Returns the index of the bin to which given fileSize belongs OR -1 if given fileSize doesn't + * belongs to any bin + */ + def getBinIndex(fileSize: Long, sortedBinBoundaries: IndexedSeq[Long]): Int = { + import scala.collection.Searching._ + // The search function on IndexedSeq uses binary search. + val searchResult = sortedBinBoundaries.search(fileSize) + searchResult match { + case Found(index) => + index + case InsertionPoint(insertionPoint) => + // insertionPoint=0 means that fileSize is lesser than min bucket of histogram + // return -1 in that case + insertionPoint - 1 + } + } + + def apply(sortedBinBoundaries: IndexedSeq[Long]): FileSizeHistogram = { new FileSizeHistogram( - sortedBinBoundaries.toArray, + sortedBinBoundaries, Array.fill(sortedBinBoundaries.size)(0), Array.fill(sortedBinBoundaries.size)(0) ) diff --git a/core/src/test/scala/io/delta/tables/DeltaTableSuite.scala b/core/src/test/scala/io/delta/tables/DeltaTableSuite.scala index b1cc5cafc3e..e8688701520 100644 --- a/core/src/test/scala/io/delta/tables/DeltaTableSuite.scala +++ b/core/src/test/scala/io/delta/tables/DeltaTableSuite.scala @@ -97,8 +97,22 @@ class DeltaTableSuite extends QueryTest } test("forName - with delta.`path`") { + // for name should work on Delta table paths withTempDir { dir => testData.write.format("delta").save(dir.getAbsolutePath) + checkAnswer( + DeltaTable.forName(spark, s"delta.`$dir`").toDF, + testData.collect().toSeq) + checkAnswer( + DeltaTable.forName(s"delta.`$dir`").toDF, + testData.collect().toSeq) + } + + // using forName on non Delta Table paths should fail + withTempDir { dir => + testForNameOnNonDeltaName(s"delta.`$dir`") + + testData.write.format("parquet").mode("overwrite").save(dir.getAbsolutePath) testForNameOnNonDeltaName(s"delta.`$dir`") } } diff --git a/core/src/test/scala/org/apache/spark/sql/delta/DeltaCommitLockSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/DeltaCommitLockSuite.scala new file mode 100644 index 00000000000..5d82f52b5a1 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/sql/delta/DeltaCommitLockSuite.scala @@ -0,0 +1,76 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed 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. + */ + +package org.apache.spark.sql.delta + +import java.io.File + +import org.apache.spark.sql.delta.sources.DeltaSQLConf +import org.apache.spark.sql.delta.storage.{AzureLogStore, S3SingleDriverLogStore} + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.{LocalSparkSession, SparkSession} +import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.util.Utils + +class DeltaCommitLockSuite extends SparkFunSuite with LocalSparkSession with SQLHelper { + + private def verifyIsCommitLockEnabled(path: File, expected: Boolean): Unit = { + val deltaLog = DeltaLog.forTable(spark, path) + val txn = deltaLog.startTransaction() + assert(txn.isCommitLockEnabled == expected) + } + + test("commit lock flag on Azure") { + spark = SparkSession.builder() + .config("spark.delta.logStore.class", classOf[AzureLogStore].getName) + .master("local[2]") + .getOrCreate() + val path = Utils.createTempDir() + try { + // Should lock by default on Azure + verifyIsCommitLockEnabled(path, expected = true) + // Should respect user config + for (enabled <- true :: false :: Nil) { + withSQLConf(DeltaSQLConf.DELTA_COMMIT_LOCK_ENABLED.key -> enabled.toString) { + verifyIsCommitLockEnabled(path, expected = enabled) + } + } + } finally { + Utils.deleteRecursively(path) + } + } + + test("commit lock flag on S3") { + spark = SparkSession.builder() + .config("spark.delta.logStore.class", classOf[S3SingleDriverLogStore].getName) + .master("local[2]") + .getOrCreate() + val path = Utils.createTempDir() + try { + // Should not lock by default on S3 + verifyIsCommitLockEnabled(path, expected = false) + // Should respect user config + for (enabled <- true :: false :: Nil) { + withSQLConf(DeltaSQLConf.DELTA_COMMIT_LOCK_ENABLED.key -> enabled.toString) { + verifyIsCommitLockEnabled(path, expected = enabled) + } + } + } finally { + Utils.deleteRecursively(path) + } + } +} diff --git a/core/src/test/scala/org/apache/spark/sql/delta/DeltaHistoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/DeltaHistoryManagerSuite.scala index 0113f9cb4a9..6174c714227 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/DeltaHistoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/DeltaHistoryManagerSuite.scala @@ -150,7 +150,7 @@ trait DeltaTimeTravelTests extends QueryTest } } -abstract class DeltaHistoryManagerBase extends DeltaTimeTravelTests // Edge // Edge +abstract class DeltaHistoryManagerBase extends DeltaTimeTravelTests { } diff --git a/core/src/test/scala/org/apache/spark/sql/delta/DeltaSourceSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/DeltaSourceSuite.scala index 6ec102c83fe..a3d001ca049 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/DeltaSourceSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/DeltaSourceSuite.scala @@ -913,8 +913,9 @@ class DeltaSourceSuite extends DeltaSourceSuiteBase with DeltaSQLCommandTest { ).map { name => new Path(inputDir.toString + "/_delta_log" + name) } for (logFilePath <- possibleFiles) { if (fs.exists(logFilePath)) { - // The cleanup logic has an edge case when files for higher versions don't have higher - // timestamps, so we set the timestamp to scale with version rather than just being 0. + // The cleanup logic has a corner case when files for higher versions don't have + // higher timestamps, so we set the timestamp to scale with version rather than just + // being 0. fs.setTimes(logFilePath, version * 1000, 0) } } @@ -958,8 +959,9 @@ class DeltaSourceSuite extends DeltaSourceSuiteBase with DeltaSQLCommandTest { ).map { name => new Path(inputDir.toString + "/_delta_log" + name) } for (logFilePath <- possibleFiles) { if (fs.exists(logFilePath)) { - // The cleanup logic has an edge case when files for higher versions don't have higher - // timestamps, so we set the timestamp to scale with version rather than just being 0. + // The cleanup logic has a corner case when files for higher versions don't have + // higher timestamps, so we set the timestamp to scale with version rather than just + // being 0. fs.setTimes(logFilePath, version * 1000, 0) } } diff --git a/core/src/test/scala/org/apache/spark/sql/delta/DeltaSourceTableAPISuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/DeltaSourceTableAPISuite.scala index 2fb1e76a6af..06652fe33e4 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/DeltaSourceTableAPISuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/DeltaSourceTableAPISuite.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.streaming.{StreamingQuery, StreamTest} import org.apache.spark.util.Utils -class DeltaSourceTableAPISuite extends StreamTest // Edge +class DeltaSourceTableAPISuite extends StreamTest with DeltaSQLCommandTest { override def beforeAll(): Unit = { diff --git a/core/src/test/scala/org/apache/spark/sql/delta/DeltaSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/DeltaSuite.scala index a63d92ed651..a6c72417965 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/DeltaSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/DeltaSuite.scala @@ -173,7 +173,7 @@ class DeltaSuite extends QueryTest val e = intercept[AnalysisException] { spark.read.format("delta").load(tempDir.toString).collect() }.getMessage - assert(e.contains("doesn't exist")) + assert(e.contains("is not a Delta table")) } test("append then read") { @@ -1360,4 +1360,23 @@ class DeltaSuite extends QueryTest checkAnswer(sql(s"SELECT `a.b`, `c.d`, a.b from $table"), Row("a", "b", "c") :: Nil) } } + + test("need to update DeltaLog on DataFrameReader.load() code path") { + // Due to possible race conditions (like in mounting/unmounting paths) there might be an initial + // snapshot that gets cached for a table that should have a valid (non-initial) snapshot. In + // such a case we need to call deltaLog.update() in the DataFrame read paths to update the + // initial snapshot to a valid one. + // + // We simulate a cached InitialSnapshot + valid delta table by creating an empty DeltaLog + // (which creates an InitialSnapshot cached for that path) then move an actual Delta table's + // transaction log into the path for the empty log. + val dir1 = Utils.createTempDir() + val dir2 = Utils.createTempDir() + val log = DeltaLog.forTable(spark, dir1) + assert(!log.tableExists) + spark.range(10).write.format("delta").save(dir2.getCanonicalPath) + // rename dir2 to dir1 then read + dir2.renameTo(dir1) + checkAnswer(spark.read.format("delta").load(dir1.getCanonicalPath), spark.range(10).toDF) + } } diff --git a/core/src/test/scala/org/apache/spark/sql/delta/DescribeDeltaHistorySuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/DescribeDeltaHistorySuite.scala index 2c21a9aea1b..09bc826dcf3 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/DescribeDeltaHistorySuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/DescribeDeltaHistorySuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.delta // scalastyle:off import.ordering.noEmptyLine import java.io.File - // Edge import org.apache.spark.sql.delta.actions.{Action, CommitInfo, Metadata, Protocol} import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.delta.test.DeltaSQLCommandTest diff --git a/core/src/test/scala/org/apache/spark/sql/delta/EvolvabilitySuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/EvolvabilitySuite.scala index e1b4efa029a..b17a3542f20 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/EvolvabilitySuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/EvolvabilitySuite.scala @@ -21,7 +21,6 @@ import org.apache.spark.sql.delta.util.{FileNames, JsonUtils} import org.apache.hadoop.fs.Path // scalastyle:off import.ordering.noEmptyLine -import org.apache.spark.DebugFilesystem // EDGE import org.apache.spark.sql.execution.streaming.MemoryStream import org.apache.spark.sql.functions.typedLit import org.apache.spark.sql.internal.SQLConf diff --git a/core/src/test/scala/org/apache/spark/sql/delta/EvolvabilitySuiteBase.scala b/core/src/test/scala/org/apache/spark/sql/delta/EvolvabilitySuiteBase.scala index 1d32e87c0fb..3bdf26d04e1 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/EvolvabilitySuiteBase.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/EvolvabilitySuiteBase.scala @@ -87,14 +87,14 @@ abstract class EvolvabilitySuiteBase extends QueryTest * * scalastyle:off * ``` - * build/sbt "test:runMain org.apache.spark.sql.delta.EvolvabilitySuite src/test/resources/delta/delta-0.1.0 generateData" + * build/sbt "core/test:runMain org.apache.spark.sql.delta.EvolvabilitySuite src/test/resources/delta/delta-0.1.0 generateData" * ``` * * You can also use this tool to generate DeltaLog that contains a checkpoint a json log with a new column. * * scalastyle:off * ``` - * build/sbt "test:runMain org.apache.spark.sql.delta.EvolvabilitySuite /path/src/test/resources/delta/transaction_log_schema_evolvability generateTransactionLogWithExtraColumn" + * build/sbt "core/test:runMain org.apache.spark.sql.delta.EvolvabilitySuite /path/src/test/resources/delta/transaction_log_schema_evolvability generateTransactionLogWithExtraColumn" * ``` */ // scalastyle:on @@ -266,11 +266,15 @@ object EvolvabilitySuiteBase { throw new AssertionError(s"${path.getCanonicalPath} exists. Please delete it and retry.") // scalastyle:on throwerror } - if (args(1) == "generateData") { - generateData(spark, path.toString) - validateData(spark, path.toString) - } else { - generateTransactionLogWithExtraColumn(spark, path.toString) + args(1) match { + case "generateData" => + generateData(spark, path.toString) + validateData(spark, path.toString) + case "generateTransactionLogWithExtraColumn" => + generateTransactionLogWithExtraColumn(spark, path.toString) + case _ => + throw new RuntimeException("Unrecognized (or omitted) argument. " + + "Please try again (no data generated).") } } } diff --git a/core/src/test/scala/org/apache/spark/sql/delta/GeneratedColumnSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/GeneratedColumnSuite.scala index af8430b894d..437687f12a4 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/GeneratedColumnSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/GeneratedColumnSuite.scala @@ -22,6 +22,7 @@ import org.apache.spark.sql.delta.sources.DeltaSourceUtils.GENERATION_EXPRESSION import org.apache.spark.sql.delta.test.DeltaSQLCommandTest import io.delta.tables.DeltaTableBuilder +import org.apache.spark.SparkConf import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, QueryTest, Row} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, stringToDate, stringToTimestamp, toJavaDate, toJavaTimestamp} @@ -35,6 +36,7 @@ import org.apache.spark.unsafe.types.UTF8String trait GeneratedColumnTest extends QueryTest with SharedSparkSession with DeltaSQLCommandTest { + protected def sqlDate(date: String): java.sql.Date = { toJavaDate(stringToDate( UTF8String.fromString(date), @@ -993,6 +995,199 @@ trait GeneratedColumnSuiteBase extends GeneratedColumnTest { assert("foo" :: Nil == comments) } } + + test("MERGE UPDATE basic") { + withTableName("source") { src => + withTableName("target") { tgt => + createTable(src, None, "c1 INT, c2 INT, c3 INT", Map.empty, Seq.empty) + sql(s"INSERT INTO ${src} values (1, 3, 4);") + createTable(tgt, None, "c1 INT, c2 INT, c3 INT", Map("c3" -> "c2 + 1"), Seq.empty) + sql(s"INSERT INTO ${tgt} values (1, 2, 3);") + sql(s""" + |MERGE INTO ${tgt} + |USING ${src} + |on ${tgt}.c1 = ${src}.c1 + |WHEN MATCHED THEN UPDATE SET ${tgt}.c2 = ${src}.c2 + |""".stripMargin) + checkAnswer( + sql(s"SELECT * FROM ${tgt}"), + Seq(Row(1, 3, 4)) + ) + } + } + } + + test("MERGE UPDATE set both generated column and its input") { + withTableName("source") { src => + withTableName("target") { tgt => + createTable(src, None, "c1 INT, c2 INT, c3 INT", Map.empty, Seq.empty) + sql(s"INSERT INTO ${src} values (1, 3, 4);") + createTable(tgt, None, "c1 INT, c2 INT, c3 INT", Map("c3" -> "c2 + 1"), Seq.empty) + sql(s"INSERT INTO ${tgt} values (1, 2, 3);") + sql(s""" + |MERGE INTO ${tgt} + |USING ${src} + |on ${tgt}.c1 = ${src}.c1 + |WHEN MATCHED THEN UPDATE SET ${tgt}.c2 = ${src}.c2, ${tgt}.c3 = ${src}.c3 + |""".stripMargin) + checkAnswer( + sql(s"SELECT * FROM ${tgt}"), + Seq(Row(1, 3, 4)) + ) + } + } + } + + test("MERGE UPDATE set star") { + withTableName("source") { src => + withTableName("target") { tgt => + createTable(src, None, "c1 INT, c2 INT, c3 INT", Map.empty, Seq.empty) + sql(s"INSERT INTO ${src} values (1, 4, 5);") + createTable(tgt, None, "c1 INT, c2 INT, c3 INT", Map("c3" -> "c2 + 1"), Seq.empty) + sql(s"INSERT INTO ${tgt} values (1, 2, 3);") + sql(s""" + |MERGE INTO ${tgt} + |USING ${src} + |on ${tgt}.c1 = ${src}.c1 + |WHEN MATCHED THEN UPDATE SET * + |""".stripMargin) + checkAnswer( + sql(s"SELECT * FROM ${tgt}"), + Seq(Row(1, 4, 5)) + ) + } + } + } + + + test("MERGE UPDATE using value from target") { + withTableName("source") { src => + withTableName("target") { tgt => + createTable(src, None, "c1 INT, c2 INT, c3 INT", Map.empty, Seq.empty) + sql(s"INSERT INTO ${src} values (1, 3, 4);") + createTable(tgt, None, "c1 INT, c2 INT, c3 INT", Map("c3" -> "c2 + 1"), Seq.empty) + sql(s"INSERT INTO ${tgt} values (1, 2, 3);") + sql(s""" + |MERGE INTO ${tgt} + |USING ${src} + |on ${tgt}.c1 = ${src}.c1 + |WHEN MATCHED THEN UPDATE SET ${tgt}.c2 = ${tgt}.c3 + |""".stripMargin) + checkAnswer( + sql(s"SELECT * FROM ${tgt}"), + Seq(Row(1, 3, 4)) + ) + } + } + } + + test("MERGE UPDATE using value from both target and source") { + withTableName("source") { src => + withTableName("target") { tgt => + createTable(src, None, "c1 INT, c2 INT, c3 INT", Map.empty, Seq.empty) + sql(s"INSERT INTO ${src} values (1, 3, 4);") + createTable(tgt, None, "c1 INT, c2 INT, c3 INT", Map("c3" -> "c2 + 1"), Seq.empty) + sql(s"INSERT INTO ${tgt} values (1, 2, 3);") + sql(s""" + |MERGE INTO ${tgt} + |USING ${src} + |on ${tgt}.c1 = ${src}.c1 + |WHEN MATCHED THEN UPDATE SET ${tgt}.c2 = ${tgt}.c3 + ${src}.c3 + |""".stripMargin) + checkAnswer( + sql(s"SELECT * FROM ${tgt}"), + Seq(Row(1, 7, 8)) + ) + } + } + } + + test("MERGE UPDATE set to null") { + withTableName("source") { src => + withTableName("target") { tgt => + createTable(src, None, "c1 INT, c2 INT, c3 INT", Map.empty, Seq.empty) + sql(s"INSERT INTO ${src} values (1, 3, 4);") + createTable(tgt, None, "c1 INT, c2 INT, c3 INT", Map("c3" -> "c2 + 1"), Seq.empty) + sql(s"INSERT INTO ${tgt} values (1, 2, 3);") + sql(s""" + |MERGE INTO ${tgt} + |USING ${src} + |on ${tgt}.c1 = ${src}.c1 + |WHEN MATCHED THEN UPDATE SET ${tgt}.c2 = null + |""".stripMargin) + checkAnswer( + sql(s"SELECT * FROM ${tgt}"), + Seq(Row(1, null, null)) + ) + } + } + } + + test("MERGE UPDATE multiple columns") { + withTableName("source") { src => + withTableName("target") { tgt => + createTable(src, None, "c1 INT, c2 INT, c3 INT", Map.empty, Seq.empty) + sql(s"INSERT INTO ${src} values (1, 3, 4);") + createTable(tgt, None, "c1 INT, c2 INT, c3 INT", Map("c3" -> "c2 + 1"), Seq.empty) + sql(s"INSERT INTO ${tgt} values (1, 2, 3);") + sql(s""" + |MERGE INTO ${tgt} + |USING ${src} + |on ${tgt}.c1 = ${src}.c1 + |WHEN MATCHED THEN UPDATE + | SET ${tgt}.c2 = ${src}.c1 * 10, ${tgt}.c1 = ${tgt}.c1 * 100 + |""".stripMargin) + checkAnswer( + sql(s"SELECT * FROM ${tgt}"), + Seq(Row(100, 10, 11)) + ) + } + } + } + + test("MERGE UPDATE source is a query") { + withTableName("source") { src => + withTableName("target") { tgt => + createTable(src, None, "c1 INT, c2 INT, c3 INT", Map.empty, Seq.empty) + sql(s"INSERT INTO ${src} values (1, 3, 4);") + createTable(tgt, None, "c1 INT, c2 INT, c3 INT", Map("c3" -> "c2 + 1"), Seq.empty) + sql(s"INSERT INTO ${tgt} values (1, 2, 3);") + sql(s""" + |MERGE INTO ${tgt} + |USING (SELECT c1, max(c3) + min(c2) AS m FROM ${src} GROUP BY c1) source + |on ${tgt}.c1 = source.c1 + |WHEN MATCHED THEN UPDATE SET ${tgt}.c2 = source.m + |""".stripMargin) + checkAnswer( + sql(s"SELECT * FROM ${tgt}"), + Seq(Row(1, 7, 8)) + ) + } + } + } + + test("MERGE UPDATE temp view is not supported") { + withTableName("source") { src => + withTableName("target") { tgt => + withTempView("test_temp_view") { + createTable(src, None, "c1 INT, c2 INT, c3 INT", Map.empty, Seq.empty) + sql(s"INSERT INTO ${src} values (1, 3, 4);") + createTable(tgt, None, "c1 INT, c2 INT, c3 INT", Map("c3" -> "c2 + 1"), Seq.empty) + sql(s"INSERT INTO ${tgt} values (1, 2, 3);") + sql(s"CREATE TEMP VIEW test_temp_view AS SELECT c1 as c2, c2 as c1, c3 FROM ${tgt}") + val e = intercept[AnalysisException] { + sql(s""" + |MERGE INTO test_temp_view + |USING ${src} + |on test_temp_view.c2 = ${src}.c1 + |WHEN MATCHED THEN UPDATE SET test_temp_view.c1 = ${src}.c2 + |""".stripMargin) + } + assert(e.getMessage.contains("Updating a temp view")) + } + } + } + } } class GeneratedColumnSuite extends GeneratedColumnSuiteBase diff --git a/core/src/test/scala/org/apache/spark/sql/delta/HiveDeltaDDLSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/HiveDeltaDDLSuite.scala index eb040f380c7..480b9c1f9e1 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/HiveDeltaDDLSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/HiveDeltaDDLSuite.scala @@ -16,6 +16,7 @@ package org.apache.spark.sql.delta +import org.apache.spark.sql.delta.hooks.GenerateSymlinkManifest import org.apache.spark.sql.delta.test.DeltaHiveTest import org.apache.spark.sql.AnalysisException @@ -23,6 +24,7 @@ import org.apache.spark.sql.hive.test.TestHiveSingleton abstract class HiveDeltaDDLSuiteBase extends DeltaDDLTestBase { + import testImplicits._ override protected def verifyDescribeTable(tblName: String): Unit = { val res = sql(s"DESCRIBE TABLE $tblName").collect() @@ -32,6 +34,7 @@ abstract class HiveDeltaDDLSuiteBase override protected def verifyNullabilityFailure(exception: AnalysisException): Unit = { exception.getMessage.contains("not supported for changing column") } + } class HiveDeltaDDLSuite extends HiveDeltaDDLSuiteBase with DeltaHiveTest diff --git a/core/src/test/scala/org/apache/spark/sql/delta/MergeIntoSQLSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/MergeIntoSQLSuite.scala index 489ca1084b4..59ca520f999 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/MergeIntoSQLSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/MergeIntoSQLSuite.scala @@ -21,9 +21,13 @@ import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.delta.test.DeltaSQLCommandTest import org.apache.spark.sql.{AnalysisException, Row} +import org.apache.spark.sql.catalyst.analysis.{Analyzer, ResolveSessionCatalog} import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.catalyst.plans.logical.{DeltaMergeInto, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.FileSourceScanExec -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{IntegerType, StructField, StructType} class MergeIntoSQLSuite extends MergeIntoSuiteBase with DeltaSQLCommandTest { @@ -230,7 +234,7 @@ class MergeIntoSQLSuite extends MergeIntoSuiteBase with DeltaSQLCommandTest { } } - ignore("merge into a SQL temp view") { + test("merge into a SQL temp view") { withTable("tab") { withTempView("v") { withTempView("src") { @@ -253,6 +257,40 @@ class MergeIntoSQLSuite extends MergeIntoSuiteBase with DeltaSQLCommandTest { } } + protected def testInvalidSqlTempView(name: String)(text: String, expectedError: String): Unit = { + test(s"can't merge into invalid SQL temp view - $name") { + withTable("tab") { + withTempView("v") { + withTempView("src") { + Seq((0, 3), (1, 2)).toDF("key", "value").write.format("delta").saveAsTable("tab") + sql(text) + sql("CREATE TEMP VIEW src AS SELECT * FROM VALUES (1, 2), (3, 4) AS t(a, b)") + val ex = intercept[AnalysisException] { + sql( + s""" + |MERGE INTO v + |USING src + |ON src.a = v.key AND src.b = v.value + |WHEN MATCHED THEN + | UPDATE SET v.value = src.b + 1 + |WHEN NOT MATCHED THEN + | INSERT (v.key, v.value) VALUES (src.a, src.b) + |""".stripMargin) + } + assert(ex.getMessage.contains(expectedError)) + } + } + } + } + } + + testInvalidSqlTempView("subset cols")( + text = "CREATE TEMP VIEW v AS SELECT key FROM tab", + expectedError = "cannot resolve" + ) + + + // This test is to capture the incorrect behavior caused by // https://github.com/delta-io/delta/issues/618 . // If this test fails then the issue has been fixed. Replace this test with a correct test @@ -275,4 +313,5 @@ class MergeIntoSQLSuite extends MergeIntoSuiteBase with DeltaSQLCommandTest { } } } + } diff --git a/core/src/test/scala/org/apache/spark/sql/delta/MergeIntoSuiteBase.scala b/core/src/test/scala/org/apache/spark/sql/delta/MergeIntoSuiteBase.scala index 52400d9d7bf..d6dd2a62caf 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/MergeIntoSuiteBase.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/MergeIntoSuiteBase.scala @@ -110,15 +110,14 @@ abstract class MergeIntoSuiteBase } Seq(true, false).foreach { skippingEnabled => - Seq(true, false).foreach { isPartitioned => - // TODO (SC-72770): enable test case when useSQLView = true - Seq(false).foreach { useSQLView => + Seq(true, false).foreach { partitioned => + Seq(true, false).foreach { useSQLView => test("basic case - merge to view on a Delta table by path, " + - s"isPartitioned: $isPartitioned skippingEnabled: $skippingEnabled") { + s"partitioned: $partitioned skippingEnabled: $skippingEnabled useSqlView: $useSQLView") { withTable("delta_target", "source") { withSQLConf(DeltaSQLConf.DELTA_STATS_SKIPPING.key -> skippingEnabled.toString) { Seq((1, 1), (0, 3), (1, 6)).toDF("key1", "value").createOrReplaceTempView("source") - val partitions = if (isPartitioned) "key2" :: Nil else Nil + val partitions = if (partitioned) "key2" :: Nil else Nil append(Seq((2, 2), (1, 4)).toDF("key2", "value"), partitions) if (useSQLView) { sql(s"CREATE OR REPLACE TEMP VIEW delta_target AS " + @@ -149,7 +148,7 @@ abstract class MergeIntoSuiteBase Seq(true, false).foreach { skippingEnabled => Seq(true, false).foreach { isPartitioned => - test("basic case edge - merge to Delta table by name, " + + test("basic case - merge to Delta table by name, " + s"isPartitioned: $isPartitioned skippingEnabled: $skippingEnabled") { withTable("delta_target", "source") { withSQLConf(DeltaSQLConf.DELTA_STATS_SKIPPING.key -> skippingEnabled.toString) { @@ -921,7 +920,7 @@ abstract class MergeIntoSuiteBase } } - test("merge into cached table edge") { + test("merge into cached table") { // Merge with a cached target only works in the join-based implementation right now withTable("source") { append(Seq((2, 2), (1, 4)).toDF("key2", "value")) diff --git a/core/src/test/scala/org/apache/spark/sql/delta/OptimisticTransactionLegacyTests.scala b/core/src/test/scala/org/apache/spark/sql/delta/OptimisticTransactionLegacyTests.scala new file mode 100644 index 00000000000..8f6d9179129 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/sql/delta/OptimisticTransactionLegacyTests.scala @@ -0,0 +1,709 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed 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. + */ + +package org.apache.spark.sql.delta + +import org.apache.spark.sql.delta.DeltaOperations.ManualUpdate +import org.apache.spark.sql.delta.DeltaTestUtils.OptimisticTxnTestHelper +import org.apache.spark.sql.delta.actions.{Action, AddCDCFile, AddFile, FileAction, Metadata, RemoveFile, SetTransaction} +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.{QueryTest, Row} +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.{StringType, StructField, StructType} + +// These tests are potentially a subset of the tests already in OptimisticTransactionSuite. +// These tests can potentially be removed but only after confirming that these tests are +// truly a subset of the tests in OptimisticTransactionSuite. +trait OptimisticTransactionLegacyTests + extends QueryTest + with SharedSparkSession { + + private val addA = AddFile("a", Map.empty, 1, 1, dataChange = true) + private val addB = AddFile("b", Map.empty, 1, 1, dataChange = true) + private val addC = AddFile("c", Map.empty, 1, 1, dataChange = true) + + import testImplicits._ + + test("block append against metadata change") { + withTempDir { tempDir => + val log = DeltaLog(spark, new Path(tempDir.getCanonicalPath)) + // Initialize the log. + log.startTransaction().commitManually() + + val txn = log.startTransaction() + val winningTxn = log.startTransaction() + winningTxn.commit(Metadata() :: Nil, ManualUpdate) + intercept[MetadataChangedException] { + txn.commit(addA :: Nil, ManualUpdate) + } + } + } + + test("block read+append against append") { + withTempDir { tempDir => + val log = DeltaLog(spark, new Path(tempDir.getCanonicalPath)) + // Initialize the log. + log.startTransaction().commitManually() + + val txn = log.startTransaction() + // reads the table + txn.filterFiles() + val winningTxn = log.startTransaction() + winningTxn.commit(addA :: Nil, ManualUpdate) + // TODO: intercept a more specific exception + intercept[DeltaConcurrentModificationException] { + txn.commit(addB :: Nil, ManualUpdate) + } + } + } + + test("allow blind-append against any data change") { + withTempDir { tempDir => + val log = DeltaLog(spark, new Path(tempDir.getCanonicalPath)) + // Initialize the log and add data. + log.startTransaction().commitManually(addA) + + val txn = log.startTransaction() + val winningTxn = log.startTransaction() + winningTxn.commit(addA.remove :: addB :: Nil, ManualUpdate) + txn.commit(addC :: Nil, ManualUpdate) + checkAnswer(log.update().allFiles.select("path"), Row("b") :: Row("c") :: Nil) + } + } + + test("allow read+append+delete against no data change") { + withTempDir { tempDir => + val log = DeltaLog(spark, new Path(tempDir.getCanonicalPath)) + // Initialize the log and add data. ManualUpdate is just a no-op placeholder. + log.startTransaction().commitManually(addA) + + val txn = log.startTransaction() + txn.filterFiles() + val winningTxn = log.startTransaction() + winningTxn.commit(Nil, ManualUpdate) + txn.commit(addA.remove :: addB :: Nil, ManualUpdate) + checkAnswer(log.update().allFiles.select("path"), Row("b") :: Nil) + } + } + + + val A_P1 = "part=1/a" + val B_P1 = "part=1/b" + val C_P1 = "part=1/c" + val C_P2 = "part=2/c" + val D_P2 = "part=2/d" + val E_P3 = "part=3/e" + val F_P3 = "part=3/f" + val G_P4 = "part=4/g" + + private val addA_P1 = AddFile(A_P1, Map("part" -> "1"), 1, 1, dataChange = true) + private val addB_P1 = AddFile(B_P1, Map("part" -> "1"), 1, 1, dataChange = true) + private val addC_P1 = AddFile(C_P1, Map("part" -> "1"), 1, 1, dataChange = true) + private val addC_P2 = AddFile(C_P2, Map("part" -> "2"), 1, 1, dataChange = true) + private val addD_P2 = AddFile(D_P2, Map("part" -> "2"), 1, 1, dataChange = true) + private val addE_P3 = AddFile(E_P3, Map("part" -> "3"), 1, 1, dataChange = true) + private val addF_P3 = AddFile(F_P3, Map("part" -> "3"), 1, 1, dataChange = true) + private val addG_P4 = AddFile(G_P4, Map("part" -> "4"), 1, 1, dataChange = true) + + test("allow concurrent commit on disjoint partitions") { + withLog(addA_P1 :: addE_P3 :: Nil) { log => + val tx1 = log.startTransaction() + // TX1 reads P3 (but not P1) + val tx1Read = tx1.filterFiles(('part === 3).expr :: Nil) + assert(tx1Read.map(_.path) == E_P3 :: Nil) + + val tx2 = log.startTransaction() + tx2.filterFiles() + // TX2 modifies only P1 + tx2.commit(addB_P1 :: Nil, ManualUpdate) + + // free to commit because P1 modified by TX2 was not read + tx1.commit(addC_P2 :: addE_P3.remove :: Nil, ManualUpdate) + checkAnswer( + log.update().allFiles.select("path"), + Row(A_P1) :: // start (E_P3 was removed by TX1) + Row(B_P1) :: // TX2 + Row(C_P2) :: Nil) // TX1 + } + } + + test("allow concurrent commit on disjoint partitions reading all partitions") { + withLog(addA_P1 :: addD_P2 :: Nil) { log => + val tx1 = log.startTransaction() + // TX1 read P1 + tx1.filterFiles(('part isin 1).expr :: Nil) + + val tx2 = log.startTransaction() + tx2.filterFiles() + tx2.commit(addC_P2 :: addD_P2.remove :: Nil, ManualUpdate) + + tx1.commit(addE_P3 :: addF_P3 :: Nil, ManualUpdate) + + checkAnswer( + log.update().allFiles.select("path"), + Row(A_P1) :: // start + Row(C_P2) :: // TX2 + Row(E_P3) :: Row(F_P3) :: Nil) // TX1 + } + } + + test("block concurrent commit when read partition was appended to by concurrent write") { + withLog(addA_P1 :: addD_P2 :: addE_P3 :: Nil) { log => + val tx1 = log.startTransaction() + // TX1 reads only P1 + val tx1Read = tx1.filterFiles(('part === 1).expr :: Nil) + assert(tx1Read.map(_.path) == A_P1 :: Nil) + + val tx2 = log.startTransaction() + tx2.filterFiles() + // TX2 modifies only P1 + tx2.commit(addB_P1 :: Nil, ManualUpdate) + + intercept[ConcurrentAppendException] { + // P1 was modified + tx1.commit(addC_P2 :: addE_P3 :: Nil, ManualUpdate) + } + } + } + + test("block concurrent commit on full table scan") { + withLog(addA_P1 :: addD_P2 :: Nil) { log => + val tx1 = log.startTransaction() + // TX1 full table scan + tx1.filterFiles() + tx1.filterFiles(('part === 1).expr :: Nil) + + val tx2 = log.startTransaction() + tx2.filterFiles() + tx2.commit(addC_P2 :: addD_P2.remove :: Nil, ManualUpdate) + + intercept[ConcurrentAppendException] { + tx1.commit(addE_P3 :: addF_P3 :: Nil, ManualUpdate) + } + } + } + + val A_1_1 = "a=1/b=1/a" + val B_1_2 = "a=1/b=2/b" + val C_2_1 = "a=2/b=1/c" + val D_3_1 = "a=3/b=1/d" + + val addA_1_1_nested = AddFile( + A_1_1, Map("a" -> "1", "b" -> "1"), + 1, 1, dataChange = true) + val addB_1_2_nested = AddFile( + B_1_2, Map("a" -> "1", "b" -> "2"), + 1, 1, dataChange = true) + val addC_2_1_nested = AddFile( + C_2_1, Map("a" -> "2", "b" -> "1"), + 1, 1, dataChange = true) + val addD_3_1_nested = AddFile( + D_3_1, Map("a" -> "3", "b" -> "1"), + 1, 1, dataChange = true) + + test("allow concurrent adds to disjoint nested partitions when read is disjoint from write") { + withLog(addA_1_1_nested :: Nil, partitionCols = "a" :: "b" :: Nil) { log => + val tx1 = log.startTransaction() + // TX1 reads a=1/b=1 + val tx1Read = tx1.filterFiles(('a === 1 and 'b === 1).expr :: Nil) + assert(tx1Read.map(_.path) == A_1_1 :: Nil) + + val tx2 = log.startTransaction() + tx2.filterFiles() + // TX2 reads all partitions and modifies only a=1/b=2 + tx2.commit(addB_1_2_nested :: Nil, ManualUpdate) + + // TX1 reads a=1/b=1 which was not modified by TX2, hence TX1 can write to a=2/b=1 + tx1.commit(addC_2_1_nested :: Nil, ManualUpdate) + checkAnswer( + log.update().allFiles.select("path"), + Row(A_1_1) :: // start + Row(B_1_2) :: // TX2 + Row(C_2_1) :: Nil) // TX1 + } + } + + test("allow concurrent adds to same nested partitions when read is disjoint from write") { + withLog(addA_1_1_nested :: Nil, partitionCols = "a" :: "b" :: Nil) { log => + val tx1 = log.startTransaction() + // TX1 reads a=1/b=1 + val tx1Read = tx1.filterFiles(('a === 1 and 'b === 1).expr :: Nil) + assert(tx1Read.map(_.path) == A_1_1 :: Nil) + + val tx2 = log.startTransaction() + tx2.filterFiles() + // TX2 modifies a=1/b=2 + tx2.commit(addB_1_2_nested :: Nil, ManualUpdate) + + // TX1 reads a=1/b=1 which was not modified by TX2, hence TX1 can write to a=2/b=1 + val add = AddFile( + "a=1/b=2/x", Map("a" -> "1", "b" -> "2"), + 1, 1, dataChange = true) + tx1.commit(add :: Nil, ManualUpdate) + checkAnswer( + log.update().allFiles.select("path"), + Row(A_1_1) :: // start + Row(B_1_2) :: // TX2 + Row("a=1/b=2/x") :: Nil) // TX1 + } + } + + test("allow concurrent add when read at lvl1 partition is disjoint from concur. write at lvl2") { + withLog( + addA_1_1_nested :: addB_1_2_nested :: Nil, + partitionCols = "a" :: "b" :: Nil) { log => + val tx1 = log.startTransaction() + // TX1 reads a=1 + val tx1Read = tx1.filterFiles(('a === 1).expr :: Nil) + assert(tx1Read.map(_.path).toSet == Set(A_1_1, B_1_2)) + + val tx2 = log.startTransaction() + tx2.filterFiles() + // TX2 modifies only a=2/b=1 + tx2.commit(addC_2_1_nested :: Nil, ManualUpdate) + + // free to commit a=2/b=1 + tx1.commit(addD_3_1_nested :: Nil, ManualUpdate) + checkAnswer( + log.update().allFiles.select("path"), + Row(A_1_1) :: Row(B_1_2) :: // start + Row(C_2_1) :: // TX2 + Row(D_3_1) :: Nil) // TX1 + } + } + + test("block commit when read at lvl1 partition reads lvl2 file concur. deleted") { + withLog( + addA_1_1_nested :: addB_1_2_nested :: Nil, + partitionCols = "a" :: "b" :: Nil) { log => + + val tx1 = log.startTransaction() + // TX1 reads a=1 + val tx1Read = tx1.filterFiles(('a === 1).expr :: Nil) + assert(tx1Read.map(_.path).toSet == Set(A_1_1, B_1_2)) + + val tx2 = log.startTransaction() + tx2.filterFiles() + // TX2 modifies a=1/b=1 + tx2.commit(addA_1_1_nested.remove :: Nil, ManualUpdate) + + intercept[ConcurrentDeleteReadException] { + // TX2 modified a=1, which was read by TX1 + tx1.commit(addD_3_1_nested :: Nil, ManualUpdate) + } + } + } + + test("block commit when full table read conflicts with concur. write in lvl2 nested partition") { + withLog(addA_1_1_nested :: Nil, partitionCols = "a" :: "b" :: Nil) { log => + val tx1 = log.startTransaction() + // TX1 full table scan + tx1.filterFiles() + + val tx2 = log.startTransaction() + tx2.filterFiles() + // TX2 modifies only a=1/b=2 + tx2.commit(addB_1_2_nested :: Nil, ManualUpdate) + + intercept[ConcurrentAppendException] { + // TX2 modified table all of which was read by TX1 + tx1.commit(addC_2_1_nested :: Nil, ManualUpdate) + } + } + } + + test("block commit when part. range read conflicts with concur. write in lvl2 nested partition") { + withLog( + addA_1_1_nested :: Nil, + partitionCols = "a" :: "b" :: Nil) { log => + + val tx1 = log.startTransaction() + // TX1 reads multiple nested partitions a >= 1 or b > 1 + val tx1Read = tx1.filterFiles(('a >= 1 or 'b > 1).expr :: Nil) + assert(tx1Read.map(_.path).toSet == Set(A_1_1)) + + val tx2 = log.startTransaction() + tx2.filterFiles() + // TX2 modifies a=1/b=2 + tx2.commit(addB_1_2_nested :: Nil, ManualUpdate) + + intercept[ConcurrentAppendException] { + // partition a=1/b=2 conflicts with our read a >= 1 or 'b > 1 + tx1.commit(addD_3_1_nested :: Nil, ManualUpdate) + } + } + } + + test("block commit with concurrent removes on same file") { + withLog(addB_1_2_nested :: Nil, partitionCols = "a" :: "b" :: Nil) { log => + val tx1 = log.startTransaction() + // TX1 reads a=2 so that read is disjoint with write partition. + tx1.filterFiles(('a === 2).expr :: Nil) + + val tx2 = log.startTransaction() + tx2.filterFiles() + // TX2 modifies a=1/b=2 + tx2.commit(addB_1_2_nested.remove :: Nil, ManualUpdate) + + intercept[ConcurrentDeleteDeleteException] { + // TX1 read does not conflict with TX2 as disjoint partitions + // But TX2 removed the same file that TX1 is trying to remove + tx1.commit(addB_1_2_nested.remove:: Nil, ManualUpdate) + } + } + } + + test("block commit when full table read conflicts with add in any partition") { + withLog(addA_P1 :: Nil) { log => + val tx1 = log.startTransaction() + tx1.filterFiles() + + val tx2 = log.startTransaction() + tx2.filterFiles() + tx2.commit(addC_P2.remove :: addB_P1 :: Nil, ManualUpdate) + + intercept[ConcurrentAppendException] { + // TX1 read whole table but TX2 concurrently modified partition P2 + tx1.commit(addD_P2 :: Nil, ManualUpdate) + } + } + } + + test("block commit when full table read conflicts with delete in any partition") { + withLog(addA_P1 :: addC_P2 :: Nil) { log => + val tx1 = log.startTransaction() + tx1.filterFiles() + + val tx2 = log.startTransaction() + tx2.filterFiles() + tx2.commit(addA_P1.remove :: Nil, ManualUpdate) + + intercept[ConcurrentDeleteReadException] { + // TX1 read whole table but TX2 concurrently modified partition P1 + tx1.commit(addB_P1.remove :: Nil, ManualUpdate) + } + } + } + + test("block concurrent replaceWhere initial empty") { + withLog(addA_P1 :: Nil) { log => + val tx1 = log.startTransaction() + // replaceWhere (part >= 2) -> empty read + val tx1Read = tx1.filterFiles(('part >= 2).expr :: Nil) + assert(tx1Read.isEmpty) + + val tx2 = log.startTransaction() + // replaceWhere (part >= 2) -> empty read + val tx2Read = tx2.filterFiles(('part >= 2).expr :: Nil) + assert(tx2Read.isEmpty) + tx2.commit(addE_P3 :: Nil, ManualUpdate) + + intercept[ConcurrentAppendException] { + // Tx2 have modified P2 which conflicts with our read (part >= 2) + tx1.commit(addC_P2 :: Nil, ManualUpdate) + } + } + } + + test("allow concurrent replaceWhere disjoint partitions initial empty") { + withLog(addA_P1 :: Nil) { log => + val tx1 = log.startTransaction() + // replaceWhere (part > 2 and part <= 3) -> empty read + val tx1Read = tx1.filterFiles(('part > 1 and 'part <= 3).expr :: Nil) + assert(tx1Read.isEmpty) + + val tx2 = log.startTransaction() + // replaceWhere (part > 3) -> empty read + val tx2Read = tx2.filterFiles(('part > 3).expr :: Nil) + assert(tx2Read.isEmpty) + + tx1.commit(addC_P2 :: Nil, ManualUpdate) + // P2 doesn't conflict with read predicate (part > 3) + tx2.commit(addG_P4 :: Nil, ManualUpdate) + checkAnswer( + log.update().allFiles.select("path"), + Row(A_P1) :: // start + Row(C_P2) :: // TX1 + Row(G_P4) :: Nil) // TX2 + } + } + + test("block concurrent replaceWhere NOT empty but conflicting predicate") { + withLog(addA_P1 :: addG_P4 :: Nil) { log => + val tx1 = log.startTransaction() + // replaceWhere (part <= 3) -> read P1 + val tx1Read = tx1.filterFiles(('part <= 3).expr :: Nil) + assert(tx1Read.map(_.path) == A_P1 :: Nil) + val tx2 = log.startTransaction() + // replaceWhere (part >= 2) -> read P4 + val tx2Read = tx2.filterFiles(('part >= 2).expr :: Nil) + assert(tx2Read.map(_.path) == G_P4 :: Nil) + + tx1.commit(addA_P1.remove :: addC_P2 :: Nil, ManualUpdate) + intercept[ConcurrentAppendException] { + // Tx1 have modified P2 which conflicts with our read (part >= 2) + tx2.commit(addG_P4.remove :: addE_P3 :: Nil, ManualUpdate) + } + } + } + + test("block concurrent commit on read & add conflicting partitions") { + withLog(addA_P1 :: Nil) { log => + val tx1 = log.startTransaction() + // read P1 + val tx1Read = tx1.filterFiles(('part === 1).expr :: Nil) + assert(tx1Read.map(_.path) == A_P1 :: Nil) + + // tx2 commits before tx1 + val tx2 = log.startTransaction() + tx2.filterFiles() + tx2.commit(addB_P1 :: Nil, ManualUpdate) + + intercept[ConcurrentAppendException] { + // P1 read by TX1 was modified by TX2 + tx1.commit(addE_P3 :: Nil, ManualUpdate) + } + } + } + + test("block concurrent commit on read & delete conflicting partitions") { + withLog(addA_P1 :: addB_P1 :: Nil) { log => + val tx1 = log.startTransaction() + // read P1 + tx1.filterFiles(('part === 1).expr :: Nil) + + // tx2 commits before tx1 + val tx2 = log.startTransaction() + tx2.filterFiles() + tx2.commit(addA_P1.remove :: Nil, ManualUpdate) + + intercept[ConcurrentDeleteReadException] { + // P1 read by TX1 was removed by TX2 + tx1.commit(addE_P3 :: Nil, ManualUpdate) + } + } + } + + test("block 2 concurrent replaceWhere transactions") { + withLog(addA_P1 :: Nil) { log => + val tx1 = log.startTransaction() + // read P1 + tx1.filterFiles(('part === 1).expr :: Nil) + + val tx2 = log.startTransaction() + // read P1 + tx2.filterFiles(('part === 1).expr :: Nil) + + // tx1 commits before tx2 + tx1.commit(addA_P1.remove :: addB_P1 :: Nil, ManualUpdate) + + intercept[ConcurrentAppendException] { + // P1 read & deleted by TX1 is being modified by TX2 + tx2.commit(addA_P1.remove :: addC_P1 :: Nil, ManualUpdate) + } + } + } + + test("block 2 concurrent replaceWhere transactions changing partitions") { + withLog(addA_P1 :: addC_P2 :: addE_P3 :: Nil) { log => + val tx1 = log.startTransaction() + // read P3 + tx1.filterFiles(('part === 3 or 'part === 1).expr :: Nil) + + val tx2 = log.startTransaction() + // read P3 + tx2.filterFiles(('part === 3 or 'part === 2).expr :: Nil) + + // tx1 commits before tx2 + tx1.commit(addA_P1.remove :: addE_P3.remove :: addB_P1 :: Nil, ManualUpdate) + + intercept[ConcurrentDeleteReadException] { + // P3 read & deleted by TX1 is being modified by TX2 + tx2.commit(addC_P2.remove :: addE_P3.remove :: addD_P2 :: Nil, ManualUpdate) + } + } + } + + test("block concurrent full table scan after concurrent write completes") { + withLog(addA_P1 :: addC_P2 :: addE_P3 :: Nil) { log => + val tx1 = log.startTransaction() + + val tx2 = log.startTransaction() + tx2.filterFiles() + tx2.commit(addC_P2 :: Nil, ManualUpdate) + + tx1.filterFiles(('part === 1).expr :: Nil) + // full table scan + tx1.filterFiles() + + intercept[ConcurrentAppendException] { + tx1.commit(addA_P1.remove :: Nil, ManualUpdate) + } + } + } + + test("block concurrent commit mixed metadata and data predicate") { + withLog(addA_P1 :: addC_P2 :: addE_P3 :: Nil) { log => + val tx1 = log.startTransaction() + + val tx2 = log.startTransaction() + tx2.filterFiles() + tx2.commit(addC_P2 :: Nil, ManualUpdate) + + // actually a full table scan + tx1.filterFiles(('part === 1 or 'year > 2019).expr :: Nil) + + intercept[ConcurrentAppendException] { + tx1.commit(addA_P1.remove :: Nil, ManualUpdate) + } + } + } + + test("block concurrent read (2 scans) and add when read partition was changed by concur. write") { + withLog(addA_P1 :: addE_P3 :: Nil) { log => + val tx1 = log.startTransaction() + tx1.filterFiles(('part === 1).expr :: Nil) + + val tx2 = log.startTransaction() + tx2.filterFiles() + tx2.commit(addC_P2 :: Nil, ManualUpdate) + + tx1.filterFiles(('part > 1 and 'part < 3).expr :: Nil) + + intercept[ConcurrentAppendException] { + // P2 added by TX2 conflicts with our read condition 'part > 1 and 'part < 3 + tx1.commit(addA_P1.remove :: Nil, ManualUpdate) + } + } + } + + def setDataChangeFalse(fileActions: Seq[FileAction]): Seq[FileAction] = { + fileActions.map { + case a: AddFile => a.copy(dataChange = false) + case r: RemoveFile => r.copy(dataChange = false) + case cdc: AddCDCFile => cdc // change files are always dataChange = false + } + } + + test("no data change: allow data rearrange when new files concurrently added") { + withLog(addA_P1 :: addB_P1 :: Nil) { log => + val tx1 = log.startTransaction() + tx1.filterFiles() + + val tx2 = log.startTransaction() + tx2.filterFiles() + tx2.commit( + addE_P3 :: Nil, + ManualUpdate) + + // tx1 rearranges files + tx1.commit( + setDataChangeFalse(addA_P1.remove :: addB_P1.remove :: addC_P1 :: Nil), + ManualUpdate) + + checkAnswer( + log.update().allFiles.select("path"), + Row(C_P1) :: Row(E_P3) :: Nil) + } + } + + test("no data change: block data rearrange when concurrently delete removes same file") { + withLog(addA_P1 :: addB_P1 :: Nil) { log => + val tx1 = log.startTransaction() + tx1.filterFiles() + + // tx2 removes file + val tx2 = log.startTransaction() + tx2.filterFiles() + tx2.commit(addA_P1.remove :: Nil, ManualUpdate) + + intercept[ConcurrentDeleteReadException] { + // tx1 reads to rearrange the same file that tx2 deleted + tx1.commit( + setDataChangeFalse(addA_P1.remove :: addB_P1.remove :: addC_P1 :: Nil), + ManualUpdate) + } + } + } + + test("readWholeTable should block concurrent delete") { + withLog(addA_P1 :: Nil) { log => + val tx1 = log.startTransaction() + tx1.readWholeTable() + + // tx2 removes file + val tx2 = log.startTransaction() + tx2.commit(addA_P1.remove :: Nil, ManualUpdate) + + intercept[ConcurrentDeleteReadException] { + // tx1 reads the whole table but tx2 removes files before tx1 commits + tx1.commit(addB_P1 :: Nil, ManualUpdate) + } + } + } + + def withLog( + actions: Seq[Action], + partitionCols: Seq[String] = "part" :: Nil)( + test: DeltaLog => Unit): Unit = { + + val schema = StructType(partitionCols.map(p => StructField(p, StringType)).toArray) + val actionWithMetaData = + actions :+ Metadata(partitionColumns = partitionCols, schemaString = schema.json) + + withTempDir { tempDir => + val log = DeltaLog(spark, new Path(tempDir.getCanonicalPath)) + // Initialize the log and add data. ManualUpdate is just a no-op placeholder. + log.startTransaction().commit(Seq(Metadata(partitionColumns = partitionCols)), ManualUpdate) + log.startTransaction().commitManually(actionWithMetaData: _*) + test(log) + } + } + + test("allow concurrent set-txns with different app ids") { + withTempDir { tempDir => + val log = DeltaLog(spark, new Path(tempDir.getCanonicalPath)) + // Initialize the log. + log.startTransaction().commitManually() + + val txn = log.startTransaction() + txn.txnVersion("t1") + val winningTxn = log.startTransaction() + winningTxn.commit(SetTransaction("t2", 1, Some(1234L)) :: Nil, ManualUpdate) + txn.commit(Nil, ManualUpdate) + + assert(log.update().transactions === Map("t2" -> 1)) + } + } + + test("block concurrent set-txns with the same app id") { + withTempDir { tempDir => + val log = DeltaLog(spark, new Path(tempDir.getCanonicalPath)) + // Initialize the log. + log.startTransaction().commitManually() + + val txn = log.startTransaction() + txn.txnVersion("t1") + val winningTxn = log.startTransaction() + winningTxn.commit(SetTransaction("t1", 1, Some(1234L)) :: Nil, ManualUpdate) + + intercept[ConcurrentTransactionException] { + txn.commit(Nil, ManualUpdate) + } + } + } +} diff --git a/core/src/test/scala/org/apache/spark/sql/delta/OptimisticTransactionSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/OptimisticTransactionSuite.scala index 47d34ac2bf4..5daf8049d88 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/OptimisticTransactionSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/OptimisticTransactionSuite.scala @@ -17,690 +17,208 @@ package org.apache.spark.sql.delta import org.apache.spark.sql.delta.DeltaOperations.ManualUpdate -import org.apache.spark.sql.delta.DeltaTestUtils.OptimisticTxnTestHelper -import org.apache.spark.sql.delta.actions.{Action, AddCDCFile, AddFile, FileAction, Metadata, RemoveFile, SetTransaction} +import org.apache.spark.sql.delta.actions.{AddFile, Metadata, Protocol, RemoveFile, SetTransaction} import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.hadoop.fs.Path -import org.apache.spark.sql.{QueryTest, Row} -import org.apache.spark.sql.test.SharedSparkSession -import org.apache.spark.sql.types.{StringType, StructField, StructType} +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions.{EqualTo, Literal} +import org.apache.spark.sql.types.{IntegerType, StructType} -class OptimisticTransactionSuite extends QueryTest with SharedSparkSession { - private val addA = AddFile("a", Map.empty, 1, 1, dataChange = true) - private val addB = AddFile("b", Map.empty, 1, 1, dataChange = true) - private val addC = AddFile("c", Map.empty, 1, 1, dataChange = true) - - import testImplicits._ - - test("block append against metadata change") { - withTempDir { tempDir => - val log = DeltaLog(spark, new Path(tempDir.getCanonicalPath)) - // Initialize the log. - log.startTransaction().commitManually() - - val txn = log.startTransaction() - val winningTxn = log.startTransaction() - winningTxn.commit(Metadata() :: Nil, ManualUpdate) - intercept[MetadataChangedException] { - txn.commit(addA :: Nil, ManualUpdate) - } - } - } - - test("block read+append against append") { - withTempDir { tempDir => - val log = DeltaLog(spark, new Path(tempDir.getCanonicalPath)) - // Initialize the log. - log.startTransaction().commitManually() - - val txn = log.startTransaction() - // reads the table - txn.filterFiles() - val winningTxn = log.startTransaction() - winningTxn.commit(addA :: Nil, ManualUpdate) - // TODO: intercept a more specific exception - intercept[DeltaConcurrentModificationException] { - txn.commit(addB :: Nil, ManualUpdate) - } - } - } - - test("allow blind-append against any data change") { - withTempDir { tempDir => - val log = DeltaLog(spark, new Path(tempDir.getCanonicalPath)) - // Initialize the log and add data. - log.startTransaction().commitManually(addA) - - val txn = log.startTransaction() - val winningTxn = log.startTransaction() - winningTxn.commit(addA.remove :: addB :: Nil, ManualUpdate) - txn.commit(addC :: Nil, ManualUpdate) - checkAnswer(log.update().allFiles.select("path"), Row("b") :: Row("c") :: Nil) - } - } - - test("allow read+append+delete against no data change") { - withTempDir { tempDir => - val log = DeltaLog(spark, new Path(tempDir.getCanonicalPath)) - // Initialize the log and add data. ManualUpdate is just a no-op placeholder. - log.startTransaction().commitManually(addA) - - val txn = log.startTransaction() - txn.filterFiles() - val winningTxn = log.startTransaction() - winningTxn.commit(Nil, ManualUpdate) - txn.commit(addA.remove :: addB :: Nil, ManualUpdate) - checkAnswer(log.update().allFiles.select("path"), Row("b") :: Nil) - } - } - - - val A_P1 = "part=1/a" - val B_P1 = "part=1/b" - val C_P1 = "part=1/c" - val C_P2 = "part=2/c" - val D_P2 = "part=2/d" - val E_P3 = "part=3/e" - val F_P3 = "part=3/f" - val G_P4 = "part=4/g" - - private val addA_P1 = AddFile(A_P1, Map("part" -> "1"), 1, 1, dataChange = true) - private val addB_P1 = AddFile(B_P1, Map("part" -> "1"), 1, 1, dataChange = true) - private val addC_P1 = AddFile(C_P1, Map("part" -> "1"), 1, 1, dataChange = true) - private val addC_P2 = AddFile(C_P2, Map("part" -> "2"), 1, 1, dataChange = true) - private val addD_P2 = AddFile(D_P2, Map("part" -> "2"), 1, 1, dataChange = true) - private val addE_P3 = AddFile(E_P3, Map("part" -> "3"), 1, 1, dataChange = true) - private val addF_P3 = AddFile(F_P3, Map("part" -> "3"), 1, 1, dataChange = true) - private val addG_P4 = AddFile(G_P4, Map("part" -> "4"), 1, 1, dataChange = true) - - test("allow concurrent commit on disjoint partitions") { - withLog(addA_P1 :: addE_P3 :: Nil) { log => - val tx1 = log.startTransaction() - // TX1 reads P3 (but not P1) - val tx1Read = tx1.filterFiles(('part === 3).expr :: Nil) - assert(tx1Read.map(_.path) == E_P3 :: Nil) - - val tx2 = log.startTransaction() - tx2.filterFiles() - // TX2 modifies only P1 - tx2.commit(addB_P1 :: Nil, ManualUpdate) - - // free to commit because P1 modified by TX2 was not read - tx1.commit(addC_P2 :: addE_P3.remove :: Nil, ManualUpdate) - checkAnswer( - log.update().allFiles.select("path"), - Row(A_P1) :: // start (E_P3 was removed by TX1) - Row(B_P1) :: // TX2 - Row(C_P2) :: Nil) // TX1 - } - } - - test("allow concurrent commit on disjoint partitions reading all partitions") { - withLog(addA_P1 :: addD_P2 :: Nil) { log => - val tx1 = log.startTransaction() - // TX1 read P1 - tx1.filterFiles(('part isin 1).expr :: Nil) - - val tx2 = log.startTransaction() - tx2.filterFiles() - tx2.commit(addC_P2 :: addD_P2.remove :: Nil, ManualUpdate) - - tx1.commit(addE_P3 :: addF_P3 :: Nil, ManualUpdate) - - checkAnswer( - log.update().allFiles.select("path"), - Row(A_P1) :: // start - Row(C_P2) :: // TX2 - Row(E_P3) :: Row(F_P3) :: Nil) // TX1 - } - } - - test("block concurrent commit when read partition was appended to by concurrent write") { - withLog(addA_P1 :: addD_P2 :: addE_P3 :: Nil) { log => - val tx1 = log.startTransaction() - // TX1 reads only P1 - val tx1Read = tx1.filterFiles(('part === 1).expr :: Nil) - assert(tx1Read.map(_.path) == A_P1 :: Nil) - - val tx2 = log.startTransaction() - tx2.filterFiles() - // TX2 modifies only P1 - tx2.commit(addB_P1 :: Nil, ManualUpdate) - - intercept[ConcurrentAppendException] { - // P1 was modified - tx1.commit(addC_P2 :: addE_P3 :: Nil, ManualUpdate) - } - } - } - - test("block concurrent commit on full table scan") { - withLog(addA_P1 :: addD_P2 :: Nil) { log => - val tx1 = log.startTransaction() - // TX1 full table scan - tx1.filterFiles() - tx1.filterFiles(('part === 1).expr :: Nil) - - val tx2 = log.startTransaction() - tx2.filterFiles() - tx2.commit(addC_P2 :: addD_P2.remove :: Nil, ManualUpdate) - - intercept[ConcurrentAppendException] { - tx1.commit(addE_P3 :: addF_P3 :: Nil, ManualUpdate) - } - } - } - - val A_1_1 = "a=1/b=1/a" - val B_1_2 = "a=1/b=2/b" - val C_2_1 = "a=2/b=1/c" - val D_3_1 = "a=3/b=1/d" - - val addA_1_1_nested = AddFile( - A_1_1, Map("a" -> "1", "b" -> "1"), - 1, 1, dataChange = true) - val addB_1_2_nested = AddFile( - B_1_2, Map("a" -> "1", "b" -> "2"), - 1, 1, dataChange = true) - val addC_2_1_nested = AddFile( - C_2_1, Map("a" -> "2", "b" -> "1"), - 1, 1, dataChange = true) - val addD_3_1_nested = AddFile( - D_3_1, Map("a" -> "3", "b" -> "1"), - 1, 1, dataChange = true) - - test("allow concurrent adds to disjoint nested partitions when read is disjoint from write") { - withLog(addA_1_1_nested :: Nil, partitionCols = "a" :: "b" :: Nil) { log => - val tx1 = log.startTransaction() - // TX1 reads a=1/b=1 - val tx1Read = tx1.filterFiles(('a === 1 and 'b === 1).expr :: Nil) - assert(tx1Read.map(_.path) == A_1_1 :: Nil) - - val tx2 = log.startTransaction() - tx2.filterFiles() - // TX2 reads all partitions and modifies only a=1/b=2 - tx2.commit(addB_1_2_nested :: Nil, ManualUpdate) - - // TX1 reads a=1/b=1 which was not modified by TX2, hence TX1 can write to a=2/b=1 - tx1.commit(addC_2_1_nested :: Nil, ManualUpdate) - checkAnswer( - log.update().allFiles.select("path"), - Row(A_1_1) :: // start - Row(B_1_2) :: // TX2 - Row(C_2_1) :: Nil) // TX1 - } - } - - test("allow concurrent adds to same nested partitions when read is disjoint from write") { - withLog(addA_1_1_nested :: Nil, partitionCols = "a" :: "b" :: Nil) { log => - val tx1 = log.startTransaction() - // TX1 reads a=1/b=1 - val tx1Read = tx1.filterFiles(('a === 1 and 'b === 1).expr :: Nil) - assert(tx1Read.map(_.path) == A_1_1 :: Nil) - - val tx2 = log.startTransaction() - tx2.filterFiles() - // TX2 modifies a=1/b=2 - tx2.commit(addB_1_2_nested :: Nil, ManualUpdate) - - // TX1 reads a=1/b=1 which was not modified by TX2, hence TX1 can write to a=2/b=1 - val add = AddFile( - "a=1/b=2/x", Map("a" -> "1", "b" -> "2"), - 1, 1, dataChange = true) - tx1.commit(add :: Nil, ManualUpdate) - checkAnswer( - log.update().allFiles.select("path"), - Row(A_1_1) :: // start - Row(B_1_2) :: // TX2 - Row("a=1/b=2/x") :: Nil) // TX1 - } - } - - test("allow concurrent add when read at lvl1 partition is disjoint from concur. write at lvl2") { - withLog( - addA_1_1_nested :: addB_1_2_nested :: Nil, - partitionCols = "a" :: "b" :: Nil) { log => - val tx1 = log.startTransaction() - // TX1 reads a=1 - val tx1Read = tx1.filterFiles(('a === 1).expr :: Nil) - assert(tx1Read.map(_.path).toSet == Set(A_1_1, B_1_2)) - - val tx2 = log.startTransaction() - tx2.filterFiles() - // TX2 modifies only a=2/b=1 - tx2.commit(addC_2_1_nested :: Nil, ManualUpdate) - - // free to commit a=2/b=1 - tx1.commit(addD_3_1_nested :: Nil, ManualUpdate) - checkAnswer( - log.update().allFiles.select("path"), - Row(A_1_1) :: Row(B_1_2) :: // start - Row(C_2_1) :: // TX2 - Row(D_3_1) :: Nil) // TX1 - } - } - - test("block commit when read at lvl1 partition reads lvl2 file concur. deleted") { - withLog( - addA_1_1_nested :: addB_1_2_nested :: Nil, - partitionCols = "a" :: "b" :: Nil) { log => - - val tx1 = log.startTransaction() - // TX1 reads a=1 - val tx1Read = tx1.filterFiles(('a === 1).expr :: Nil) - assert(tx1Read.map(_.path).toSet == Set(A_1_1, B_1_2)) - - val tx2 = log.startTransaction() - tx2.filterFiles() - // TX2 modifies a=1/b=1 - tx2.commit(addA_1_1_nested.remove :: Nil, ManualUpdate) - - intercept[ConcurrentDeleteReadException] { - // TX2 modified a=1, which was read by TX1 - tx1.commit(addD_3_1_nested :: Nil, ManualUpdate) - } - } - } - - test("block commit when full table read conflicts with concur. write in lvl2 nested partition") { - withLog(addA_1_1_nested :: Nil, partitionCols = "a" :: "b" :: Nil) { log => - val tx1 = log.startTransaction() - // TX1 full table scan - tx1.filterFiles() - val tx2 = log.startTransaction() - tx2.filterFiles() - // TX2 modifies only a=1/b=2 - tx2.commit(addB_1_2_nested :: Nil, ManualUpdate) +class OptimisticTransactionSuite + extends OptimisticTransactionLegacyTests + with OptimisticTransactionSuiteBase { - intercept[ConcurrentAppendException] { - // TX2 modified table all of which was read by TX1 - tx1.commit(addC_2_1_nested :: Nil, ManualUpdate) - } - } - } - - test("block commit when part. range read conflicts with concur. write in lvl2 nested partition") { - withLog( - addA_1_1_nested :: Nil, - partitionCols = "a" :: "b" :: Nil) { log => - - val tx1 = log.startTransaction() - // TX1 reads multiple nested partitions a >= 1 or b > 1 - val tx1Read = tx1.filterFiles(('a >= 1 or 'b > 1).expr :: Nil) - assert(tx1Read.map(_.path).toSet == Set(A_1_1)) - - val tx2 = log.startTransaction() - tx2.filterFiles() - // TX2 modifies a=1/b=2 - tx2.commit(addB_1_2_nested :: Nil, ManualUpdate) - - intercept[ConcurrentAppendException] { - // partition a=1/b=2 conflicts with our read a >= 1 or 'b > 1 - tx1.commit(addD_3_1_nested :: Nil, ManualUpdate) - } - } - } - - test("block commit with concurrent removes on same file") { - withLog(addB_1_2_nested :: Nil, partitionCols = "a" :: "b" :: Nil) { log => - val tx1 = log.startTransaction() - // TX1 reads a=2 so that read is disjoint with write partition. - tx1.filterFiles(('a === 2).expr :: Nil) - - val tx2 = log.startTransaction() - tx2.filterFiles() - // TX2 modifies a=1/b=2 - tx2.commit(addB_1_2_nested.remove :: Nil, ManualUpdate) - - intercept[ConcurrentDeleteDeleteException] { - // TX1 read does not conflict with TX2 as disjoint partitions - // But TX2 removed the same file that TX1 is trying to remove - tx1.commit(addB_1_2_nested.remove:: Nil, ManualUpdate) - } - } - } - - test("block commit when full table read conflicts with add in any partition") { - withLog(addA_P1 :: Nil) { log => - val tx1 = log.startTransaction() - tx1.filterFiles() - - val tx2 = log.startTransaction() - tx2.filterFiles() - tx2.commit(addC_P2.remove :: addB_P1 :: Nil, ManualUpdate) - - intercept[ConcurrentAppendException] { - // TX1 read whole table but TX2 concurrently modified partition P2 - tx1.commit(addD_P2 :: Nil, ManualUpdate) - } - } - } - - test("block commit when full table read conflicts with delete in any partition") { - withLog(addA_P1 :: addC_P2 :: Nil) { log => - val tx1 = log.startTransaction() - tx1.filterFiles() - - val tx2 = log.startTransaction() - tx2.filterFiles() - tx2.commit(addA_P1.remove :: Nil, ManualUpdate) - - intercept[ConcurrentDeleteReadException] { - // TX1 read whole table but TX2 concurrently modified partition P1 - tx1.commit(addB_P1.remove :: Nil, ManualUpdate) - } - } - } - - test("block concurrent replaceWhere initial empty") { - withLog(addA_P1 :: Nil) { log => - val tx1 = log.startTransaction() - // replaceWhere (part >= 2) -> empty read - val tx1Read = tx1.filterFiles(('part >= 2).expr :: Nil) - assert(tx1Read.isEmpty) - - val tx2 = log.startTransaction() - // replaceWhere (part >= 2) -> empty read - val tx2Read = tx2.filterFiles(('part >= 2).expr :: Nil) - assert(tx2Read.isEmpty) - tx2.commit(addE_P3 :: Nil, ManualUpdate) - - intercept[ConcurrentAppendException] { - // Tx2 have modified P2 which conflicts with our read (part >= 2) - tx1.commit(addC_P2 :: Nil, ManualUpdate) - } - } - } - - test("allow concurrent replaceWhere disjoint partitions initial empty") { - withLog(addA_P1 :: Nil) { log => - val tx1 = log.startTransaction() - // replaceWhere (part > 2 and part <= 3) -> empty read - val tx1Read = tx1.filterFiles(('part > 1 and 'part <= 3).expr :: Nil) - assert(tx1Read.isEmpty) - - val tx2 = log.startTransaction() - // replaceWhere (part > 3) -> empty read - val tx2Read = tx2.filterFiles(('part > 3).expr :: Nil) - assert(tx2Read.isEmpty) - - tx1.commit(addC_P2 :: Nil, ManualUpdate) - // P2 doesn't conflict with read predicate (part > 3) - tx2.commit(addG_P4 :: Nil, ManualUpdate) - checkAnswer( - log.update().allFiles.select("path"), - Row(A_P1) :: // start - Row(C_P2) :: // TX1 - Row(G_P4) :: Nil) // TX2 - } - } - - test("block concurrent replaceWhere NOT empty but conflicting predicate") { - withLog(addA_P1 :: addG_P4 :: Nil) { log => - val tx1 = log.startTransaction() - // replaceWhere (part <= 3) -> read P1 - val tx1Read = tx1.filterFiles(('part <= 3).expr :: Nil) - assert(tx1Read.map(_.path) == A_P1 :: Nil) - val tx2 = log.startTransaction() - // replaceWhere (part >= 2) -> read P4 - val tx2Read = tx2.filterFiles(('part >= 2).expr :: Nil) - assert(tx2Read.map(_.path) == G_P4 :: Nil) - - tx1.commit(addA_P1.remove :: addC_P2 :: Nil, ManualUpdate) - intercept[ConcurrentAppendException] { - // Tx1 have modified P2 which conflicts with our read (part >= 2) - tx2.commit(addG_P4.remove :: addE_P3 :: Nil, ManualUpdate) - } - } - } - - test("block concurrent commit on read & add conflicting partitions") { - withLog(addA_P1 :: Nil) { log => - val tx1 = log.startTransaction() - // read P1 - val tx1Read = tx1.filterFiles(('part === 1).expr :: Nil) - assert(tx1Read.map(_.path) == A_P1 :: Nil) - - // tx2 commits before tx1 - val tx2 = log.startTransaction() - tx2.filterFiles() - tx2.commit(addB_P1 :: Nil, ManualUpdate) - - intercept[ConcurrentAppendException] { - // P1 read by TX1 was modified by TX2 - tx1.commit(addE_P3 :: Nil, ManualUpdate) - } - } - } - - test("block concurrent commit on read & delete conflicting partitions") { - withLog(addA_P1 :: addB_P1 :: Nil) { log => - val tx1 = log.startTransaction() - // read P1 - tx1.filterFiles(('part === 1).expr :: Nil) - - // tx2 commits before tx1 - val tx2 = log.startTransaction() - tx2.filterFiles() - tx2.commit(addA_P1.remove :: Nil, ManualUpdate) - - intercept[ConcurrentDeleteReadException] { - // P1 read by TX1 was removed by TX2 - tx1.commit(addE_P3 :: Nil, ManualUpdate) - } - } - } - - test("block 2 concurrent replaceWhere transactions") { - withLog(addA_P1 :: Nil) { log => - val tx1 = log.startTransaction() - // read P1 - tx1.filterFiles(('part === 1).expr :: Nil) - - val tx2 = log.startTransaction() - // read P1 - tx2.filterFiles(('part === 1).expr :: Nil) - - // tx1 commits before tx2 - tx1.commit(addA_P1.remove :: addB_P1 :: Nil, ManualUpdate) - - intercept[ConcurrentAppendException] { - // P1 read & deleted by TX1 is being modified by TX2 - tx2.commit(addA_P1.remove :: addC_P1 :: Nil, ManualUpdate) - } - } - } - - test("block 2 concurrent replaceWhere transactions changing partitions") { - withLog(addA_P1 :: addC_P2 :: addE_P3 :: Nil) { log => - val tx1 = log.startTransaction() - // read P3 - tx1.filterFiles(('part === 3 or 'part === 1).expr :: Nil) - - val tx2 = log.startTransaction() - // read P3 - tx2.filterFiles(('part === 3 or 'part === 2).expr :: Nil) - - // tx1 commits before tx2 - tx1.commit(addA_P1.remove :: addE_P3.remove :: addB_P1 :: Nil, ManualUpdate) - - intercept[ConcurrentDeleteReadException] { - // P3 read & deleted by TX1 is being modified by TX2 - tx2.commit(addC_P2.remove :: addE_P3.remove :: addD_P2 :: Nil, ManualUpdate) - } - } - } - - test("block concurrent full table scan after concurrent write completes") { - withLog(addA_P1 :: addC_P2 :: addE_P3 :: Nil) { log => - val tx1 = log.startTransaction() - - val tx2 = log.startTransaction() - tx2.filterFiles() - tx2.commit(addC_P2 :: Nil, ManualUpdate) - - tx1.filterFiles(('part === 1).expr :: Nil) - // full table scan - tx1.filterFiles() - - intercept[ConcurrentAppendException] { - tx1.commit(addA_P1.remove :: Nil, ManualUpdate) - } - } - } - - test("block concurrent commit mixed metadata and data predicate") { - withLog(addA_P1 :: addC_P2 :: addE_P3 :: Nil) { log => - val tx1 = log.startTransaction() - - val tx2 = log.startTransaction() - tx2.filterFiles() - tx2.commit(addC_P2 :: Nil, ManualUpdate) - - // actually a full table scan - tx1.filterFiles(('part === 1 or 'year > 2019).expr :: Nil) - - intercept[ConcurrentAppendException] { - tx1.commit(addA_P1.remove :: Nil, ManualUpdate) - } - } - } - - test("block concurrent read (2 scans) and add when read partition was changed by concur. write") { - withLog(addA_P1 :: addE_P3 :: Nil) { log => - val tx1 = log.startTransaction() - tx1.filterFiles(('part === 1).expr :: Nil) - - val tx2 = log.startTransaction() - tx2.filterFiles() - tx2.commit(addC_P2 :: Nil, ManualUpdate) - - tx1.filterFiles(('part > 1 and 'part < 3).expr :: Nil) - - intercept[ConcurrentAppendException] { - // P2 added by TX2 conflicts with our read condition 'part > 1 and 'part < 3 - tx1.commit(addA_P1.remove :: Nil, ManualUpdate) - } - } - } - - def setDataChangeFalse(fileActions: Seq[FileAction]): Seq[FileAction] = { - fileActions.map { - case a: AddFile => a.copy(dataChange = false) - case r: RemoveFile => r.copy(dataChange = false) - case cdc: AddCDCFile => cdc // change files are always dataChange = false - } - } - - test("no data change: allow data rearrange when new files concurrently added") { - withLog(addA_P1 :: addB_P1 :: Nil) { log => - val tx1 = log.startTransaction() - tx1.filterFiles() - - val tx2 = log.startTransaction() - tx2.filterFiles() - tx2.commit( - addE_P3 :: Nil, - ManualUpdate) - - // tx1 rearranges files - tx1.commit( - setDataChangeFalse(addA_P1.remove :: addB_P1.remove :: addC_P1 :: Nil), - ManualUpdate) - - checkAnswer( - log.update().allFiles.select("path"), - Row(C_P1) :: Row(E_P3) :: Nil) - } - } - - test("no data change: block data rearrange when concurrently delete removes same file") { - withLog(addA_P1 :: addB_P1 :: Nil) { log => - val tx1 = log.startTransaction() - tx1.filterFiles() - - // tx2 removes file - val tx2 = log.startTransaction() - tx2.filterFiles() - tx2.commit(addA_P1.remove :: Nil, ManualUpdate) - - intercept[ConcurrentDeleteReadException] { - // tx1 reads to rearrange the same file that tx2 deleted - tx1.commit( - setDataChangeFalse(addA_P1.remove :: addB_P1.remove :: addC_P1 :: Nil), - ManualUpdate) - } - } - } - - test("readWholeTable should block concurrent delete") { - withLog(addA_P1 :: Nil) { log => - val tx1 = log.startTransaction() - tx1.readWholeTable() - - // tx2 removes file - val tx2 = log.startTransaction() - tx2.commit(addA_P1.remove :: Nil, ManualUpdate) - - intercept[ConcurrentDeleteReadException] { - // tx1 reads the whole table but tx2 removes files before tx1 commits - tx1.commit(addB_P1 :: Nil, ManualUpdate) - } - } - } - - def withLog( - actions: Seq[Action], - partitionCols: Seq[String] = "part" :: Nil)( - test: DeltaLog => Unit): Unit = { - - val schema = StructType(partitionCols.map(p => StructField(p, StringType)).toArray) - val actionWithMetaData = - actions :+ Metadata(partitionColumns = partitionCols, schemaString = schema.json) - - withTempDir { tempDir => - val log = DeltaLog(spark, new Path(tempDir.getCanonicalPath)) - // Initialize the log and add data. ManualUpdate is just a no-op placeholder. - log.startTransaction().commit(Seq(Metadata(partitionColumns = partitionCols)), ManualUpdate) - log.startTransaction().commitManually(actionWithMetaData: _*) - test(log) - } - } - - test("allow concurrent set-txns with different app ids") { - withTempDir { tempDir => - val log = DeltaLog(spark, new Path(tempDir.getCanonicalPath)) - // Initialize the log. - log.startTransaction().commitManually() - - val txn = log.startTransaction() - txn.txnVersion("t1") - val winningTxn = log.startTransaction() - winningTxn.commit(SetTransaction("t2", 1, Some(1234L)) :: Nil, ManualUpdate) - txn.commit(Nil, ManualUpdate) - - assert(log.update().transactions === Map("t2" -> 1)) - } - } - - test("block concurrent set-txns with the same app id") { - withTempDir { tempDir => - val log = DeltaLog(spark, new Path(tempDir.getCanonicalPath)) - // Initialize the log. - log.startTransaction().commitManually() - - val txn = log.startTransaction() - txn.txnVersion("t1") - val winningTxn = log.startTransaction() - winningTxn.commit(SetTransaction("t1", 1, Some(1234L)) :: Nil, ManualUpdate) + // scalastyle:off: removeFile + private val addA = AddFile("a", Map.empty, 1, 1, dataChange = true) + private val addB = AddFile("b", Map.empty, 1, 1, dataChange = true) - intercept[ConcurrentTransactionException] { - txn.commit(Nil, ManualUpdate) - } - } - } + /* ************************** * + * Allowed concurrent actions * + * ************************** */ + + check( + "append / append", + conflicts = false, + reads = Seq( + t => t.metadata + ), + concurrentWrites = Seq( + addA), + actions = Seq( + addB)) + + check( + "disjoint txns", + conflicts = false, + reads = Seq( + t => t.txnVersion("t1") + ), + concurrentWrites = Seq( + SetTransaction("t2", 0, Some(1234L))), + actions = Nil) + + check( + "disjoint delete / read", + conflicts = false, + setup = Seq( + Metadata( + schemaString = new StructType().add("x", IntegerType).json, + partitionColumns = Seq("x")), + AddFile("a", Map("x" -> "2"), 1, 1, dataChange = true) + ), + reads = Seq( + t => t.filterFiles(EqualTo('x, Literal(1)) :: Nil) + ), + concurrentWrites = Seq( + RemoveFile("a", Some(4))), + actions = Seq()) + + check( + "disjoint add / read", + conflicts = false, + setup = Seq( + Metadata( + schemaString = new StructType().add("x", IntegerType).json, + partitionColumns = Seq("x")) + ), + reads = Seq( + t => t.filterFiles(EqualTo('x, Literal(1)) :: Nil) + ), + concurrentWrites = Seq( + AddFile("a", Map("x" -> "2"), 1, 1, dataChange = true)), + actions = Seq()) + + /* ***************************** * + * Disallowed concurrent actions * + * ***************************** */ + + check( + "delete / delete", + conflicts = true, + reads = Nil, + concurrentWrites = Seq( + RemoveFile("a", Some(4))), + actions = Seq( + RemoveFile("a", Some(5)))) + + check( + "add / read + write", + conflicts = true, + setup = Seq( + Metadata( + schemaString = new StructType().add("x", IntegerType).json, + partitionColumns = Seq("x")) + ), + reads = Seq( + t => t.filterFiles(EqualTo('x, Literal(1)) :: Nil) + ), + concurrentWrites = Seq( + AddFile("a", Map("x" -> "1"), 1, 1, dataChange = true)), + actions = Seq(AddFile("b", Map("x" -> "1"), 1, 1, dataChange = true)), + // commit info should show operation as truncate, because that's the operation used by the + // harness + errorMessageHint = Some("[x=1]" :: "TRUNCATE" :: Nil)) + + check( + "add / read + no write", // no write = no real conflicting change even though data was added + conflicts = false, // so this should not conflict + setup = Seq( + Metadata( + schemaString = new StructType().add("x", IntegerType).json, + partitionColumns = Seq("x")) + ), + reads = Seq( + t => t.filterFiles(EqualTo('x, Literal(1)) :: Nil) + ), + concurrentWrites = Seq( + AddFile("a", Map("x" -> "1"), 1, 1, dataChange = true)), + actions = Seq()) + + check( + "delete / read", + conflicts = true, + setup = Seq( + Metadata( + schemaString = new StructType().add("x", IntegerType).json, + partitionColumns = Seq("x")), + AddFile("a", Map("x" -> "1"), 1, 1, dataChange = true) + ), + reads = Seq( + t => t.filterFiles(EqualTo('x, Literal(1)) :: Nil) + ), + concurrentWrites = Seq( + RemoveFile("a", Some(4))), + actions = Seq(), + errorMessageHint = Some("a in partition [x=1]" :: "TRUNCATE" :: Nil)) + + check( + "schema change", + conflicts = true, + reads = Seq( + t => t.metadata + ), + concurrentWrites = Seq( + Metadata()), + actions = Nil) + + check( + "conflicting txns", + conflicts = true, + reads = Seq( + t => t.txnVersion("t1") + ), + concurrentWrites = Seq( + SetTransaction("t1", 0, Some(1234L))), + actions = Nil) + + check( + "upgrade / upgrade", + conflicts = true, + reads = Seq( + t => t.metadata + ), + concurrentWrites = Seq( + Protocol()), + actions = Seq( + Protocol())) + + check( + "taint whole table", + conflicts = true, + setup = Seq( + Metadata( + schemaString = new StructType().add("x", IntegerType).json, + partitionColumns = Seq("x")), + AddFile("a", Map("x" -> "2"), 1, 1, dataChange = true) + ), + reads = Seq( + t => t.filterFiles(EqualTo('x, Literal(1)) :: Nil), + // `readWholeTable` should disallow any concurrent change, even if the change + // is disjoint with the earlier filter + t => t.readWholeTable() + ), + concurrentWrites = Seq( + AddFile("b", Map("x" -> "3"), 1, 1, dataChange = true)), + actions = Seq( + AddFile("c", Map("x" -> "4"), 1, 1, dataChange = true))) + + check( + "taint whole table + concurrent remove", + conflicts = true, + setup = Seq( + Metadata(schemaString = new StructType().add("x", IntegerType).json), + AddFile("a", Map.empty, 1, 1, dataChange = true) + ), + reads = Seq( + // `readWholeTable` should disallow any concurrent `RemoveFile`s. + t => t.readWholeTable() + ), + concurrentWrites = Seq( + RemoveFile("a", Some(4L))), + actions = Seq( + AddFile("b", Map.empty, 1, 1, dataChange = true))) test("initial commit without metadata should fail") { withTempDir { tempDir => @@ -710,12 +228,12 @@ class OptimisticTransactionSuite extends QueryTest with SharedSparkSession { val e = intercept[IllegalStateException] { txn.commit(Nil, ManualUpdate) } - assert(e.getMessage === DeltaErrors.metadataAbsentException().getMessage) + assert(e.getMessage == DeltaErrors.metadataAbsentException().getMessage) } // Try with commit validation turned off withSQLConf(DeltaSQLConf.DELTA_STATE_RECONSTRUCTION_VALIDATION_ENABLED.key -> "false", - DeltaSQLConf.DELTA_COMMIT_VALIDATION_ENABLED.key -> "false") { + DeltaSQLConf.DELTA_COMMIT_VALIDATION_ENABLED.key -> "false") { txn.commit(Nil, ManualUpdate) assert(log.update().version === 0) } @@ -742,7 +260,7 @@ class OptimisticTransactionSuite extends QueryTest with SharedSparkSession { log.startTransaction().commit(Seq(AddFile( log.dataPath.toString, Map("col3" -> "1"), 12322, 0L, true, null, null)), ManualUpdate) } - assert(e.getMessage === DeltaErrors.addFilePartitioningMismatchException( + assert(e.getMessage == DeltaErrors.addFilePartitioningMismatchException( Seq("col3"), Seq("col2")).getMessage) } // Try with commit validation turned off diff --git a/core/src/test/scala/org/apache/spark/sql/delta/OptimisticTransactionSuiteBase.scala b/core/src/test/scala/org/apache/spark/sql/delta/OptimisticTransactionSuiteBase.scala new file mode 100644 index 00000000000..f0639a668e7 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/sql/delta/OptimisticTransactionSuiteBase.scala @@ -0,0 +1,138 @@ +/* + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed 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. + */ + +package org.apache.spark.sql.delta + +import java.util.ConcurrentModificationException + +import org.apache.spark.sql.delta.DeltaOperations.{ManualUpdate, Truncate} +import org.apache.spark.sql.delta.actions.{Action, Metadata, Protocol} +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.util.Utils + +trait OptimisticTransactionSuiteBase + extends QueryTest + with SharedSparkSession { + + + /** + * Check whether the test transaction conflict with the concurrent writes by executing the + * given params in the following order: + * - setup (including setting table isolation level + * - reads + * - concurrentWrites + * - actions + * + * When `conflicts` == true, this function checks to make sure the commit of `actions` fails with + * [[ConcurrentModificationException]], otherwise checks that the commit is successful. + * + * @param name test name + * @param conflicts should test transaction is expected to conflict or not + * @param setup sets up the initial delta log state (set schema, partitioning, etc.) + * @param reads reads made in the test transaction + * @param concurrentWrites writes made by concurrent transactions after the test txn reads + * @param actions actions to be committed by the test transaction + * @param errorMessageHint What to expect in the error message + */ + protected def check( + name: String, + conflicts: Boolean, + setup: Seq[Action] = Seq(Metadata(), Protocol()), + reads: Seq[OptimisticTransaction => Unit], + concurrentWrites: Seq[Action], + actions: Seq[Action], + errorMessageHint: Option[Seq[String]] = None): Unit = { + + val concurrentTxn: OptimisticTransaction => Unit = + (opt: OptimisticTransaction) => opt.commit(concurrentWrites, Truncate()) + + def initialSetup(log: DeltaLog): Unit = { + // Setup the log + setup.foreach { action => + log.startTransaction().commit(Seq(action), ManualUpdate) + } + } + check( + name, + conflicts, + initialSetup _, + reads, + concurrentTxn, + actions, + errorMessageHint + ) + } + + /** + * Check whether the test transaction conflict with the concurrent writes by executing the + * given params in the following order: + * - sets up the initial delta log state using `initialSetup` (set schema, partitioning, etc.) + * - reads + * - concurrentWrites + * - actions + * + * When `conflicts` == true, this function checks to make sure the commit of `actions` fails with + * [[ConcurrentModificationException]], otherwise checks that the commit is successful. + * + * @param name test name + * @param conflicts should test transaction is expected to conflict or not + * @param initialSetup sets up the initial delta log state (set schema, partitioning, etc.) + * @param reads reads made in the test transaction + * @param concurrentTxn concurrent txn that may write data after the test txn reads + * @param actions actions to be committed by the test transaction + * @param errorMessageHint What to expect in the error message + */ + protected def check( + name: String, + conflicts: Boolean, + initialSetup: DeltaLog => Unit, + reads: Seq[OptimisticTransaction => Unit], + concurrentTxn: OptimisticTransaction => Unit, + actions: Seq[Action], + errorMessageHint: Option[Seq[String]]): Unit = { + + val conflict = if (conflicts) "should conflict" else "should not conflict" + test(s"$name - $conflict") { + val tempDir = Utils.createTempDir() + val log = DeltaLog(spark, new Path(tempDir.getCanonicalPath)) + + // Setup the log + initialSetup(log) + + // Perform reads + val txn = log.startTransaction() + reads.foreach(_(txn)) + + // Execute concurrent txn while current transaction is active + concurrentTxn(log.startTransaction()) + + // Try commit and check expected conflict behavior + if (conflicts) { + val e = intercept[ConcurrentModificationException] { + txn.commit(actions, Truncate()) + } + errorMessageHint.foreach { expectedParts => + assert(expectedParts.forall(part => e.getMessage.contains(part))) + } + } else { + txn.commit(actions, Truncate()) + } + } + } +} diff --git a/core/src/test/scala/org/apache/spark/sql/delta/schema/CheckConstraintsSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/schema/CheckConstraintsSuite.scala index 6b14bb2b3f4..fb465bab120 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/schema/CheckConstraintsSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/schema/CheckConstraintsSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.delta.schema import scala.collection.JavaConverters._ // scalastyle:off import.ordering.noEmptyLine - // Edge import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.delta.test.DeltaSQLCommandTest diff --git a/core/src/test/scala/org/apache/spark/sql/delta/schema/InvariantEnforcementSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/schema/InvariantEnforcementSuite.scala index 6c5f90eb547..ef4d9273d50 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/schema/InvariantEnforcementSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/schema/InvariantEnforcementSuite.scala @@ -21,7 +21,6 @@ import java.io.File import scala.collection.JavaConverters._ - // Edge import org.apache.spark.sql.delta.DeltaLog import org.apache.spark.sql.delta.DeltaOperations import org.apache.spark.sql.delta.actions.{Metadata, Protocol} diff --git a/core/src/test/scala/org/apache/spark/sql/delta/schema/SchemaUtilsSuite.scala b/core/src/test/scala/org/apache/spark/sql/delta/schema/SchemaUtilsSuite.scala index fa428f43785..de52179bebb 100644 --- a/core/src/test/scala/org/apache/spark/sql/delta/schema/SchemaUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/delta/schema/SchemaUtilsSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.delta.schema // scalastyle:off import.ordering.noEmptyLine import java.util.Locale +import org.apache.spark.sql.delta.schema.SchemaMergingUtils._ import org.apache.spark.sql.delta.sources.DeltaSourceUtils.GENERATION_EXPRESSION_METADATA_KEY import org.scalatest.GivenWhenThen @@ -1136,10 +1137,10 @@ class SchemaUtilsSuite extends QueryTest Seq(byteType, shortType, intType).foreach { sourceType => assert( - longType === SchemaUtils.mergeSchemas( + longType === mergeSchemas( longType, sourceType, allowImplicitConversions = true)) val e = intercept[AnalysisException] { - SchemaUtils.mergeSchemas(longType, sourceType) + mergeSchemas(longType, sourceType) } assert(e.getMessage.contains( s"Failed to merge incompatible data types LongType and ${sourceType.head.dataType}")) diff --git a/examples/scala/build/sbt-launch-lib.bash b/examples/scala/build/sbt-launch-lib.bash index 44d37fbf3d0..c0e8f06c7b0 100755 --- a/examples/scala/build/sbt-launch-lib.bash +++ b/examples/scala/build/sbt-launch-lib.bash @@ -38,7 +38,7 @@ dlog () { acquire_sbt_jar () { SBT_VERSION=`awk -F "=" '/sbt\.version/ {print $2}' ./project/build.properties` - URL1=https://dl.bintray.com/typesafe/ivy-releases/org.scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch.jar + URL1=https://repo.typesafe.com/typesafe/ivy-releases/org.scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch.jar JAR=build/sbt-launch-${SBT_VERSION}.jar sbt_jar=$JAR diff --git a/run-tests.py b/run-tests.py index e9ec1d0bee0..b20f10c133b 100755 --- a/run-tests.py +++ b/run-tests.py @@ -67,7 +67,8 @@ def run_cmd(cmd, throw_on_error=True, env=None, stream_output=False, **kwargs): run_cmd(prepare_docker_img, stream_output=True) # JENKINS_URL is passed here so that the Docker container # can be in line with Jenkins build behavior(usage of sbt sources) - cmd = ["docker", "run", "-e", "JENKINS_URL", "pydeltalake:latest"] + cmd = ["docker", "run", "-e", "JENKINS_URL", + "-e", "SBT_MIRROR_JAR_URL", "pydeltalake:latest"] run_cmd(cmd, stream_output=True) else: root_dir = os.path.dirname(os.path.dirname(__file__))