diff --git a/.circleci/config.yml b/.circleci/config.yml index 1c9f57a40df..a77e02757f0 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -21,7 +21,7 @@ jobs: pyenv install 3.7.4 pyenv global system 3.7.4 pipenv --python 3.7 install - pipenv run pip install https://docs.delta.io/spark3artifacts/rc1/distributions/pyspark-3.0.0.tar.gz + pipenv run pip install https://docs.delta.io/spark3artifacts/snapshot-fa608b94/distributions/pyspark-3.0.0.tar.gz pipenv run pip install flake8==3.5.0 pypandoc==1.3.3 - run: name: Run Scala/Java and Python tests diff --git a/Dockerfile b/Dockerfile index 74de6ac7266..aa27692c7a5 100644 --- a/Dockerfile +++ b/Dockerfile @@ -2,7 +2,7 @@ FROM python:3.7.3-stretch RUN apt-get update && apt-get -y install openjdk-8-jdk -RUN pip install https://docs.delta.io/spark3artifacts/rc1/distributions/pyspark-3.0.0.tar.gz +RUN pip install https://docs.delta.io/spark3artifacts/snapshot-fa608b94/distributions/pyspark-3.0.0.tar.gz COPY . /usr/src/delta diff --git a/PROTOCOL.md b/PROTOCOL.md index c42bcf32576..992a1608a9a 100644 --- a/PROTOCOL.md +++ b/PROTOCOL.md @@ -160,6 +160,8 @@ description|`String`| User-provided description for this table format|[Format Struct](#Format-Specification)| Specification of the encoding for the files stored in the table schemaString|[Schema Struct](#Schema-Serialization-Format)| Schema of the table partitionColumns|`Array[String]`| An array containing the names of columns by which the data should be partitioned +createdTime|`Option[Long]`| The time when this metadata action is created, in milliseconds since the Unix epoch +configuration|`Map[String, String]`| A map containing configuration options for the metadata action #### Format Specification Field Name | Data Type | Description @@ -274,6 +276,7 @@ Field Name | Data Type | Description -|-|- appId | String | A unique identifier for the application performing the transaction version | Long | An application-specific numeric identifier for this transaction +lastUpdated | Option[Long] | The time when this transaction action is created, in milliseconds since the Unix epoch The following is an example `txn` action: ``` @@ -433,6 +436,7 @@ A reference implementation can be found in [the catalyst package of the Apache S Type Name | Description -|- string| UTF-8 encoded string of characters +long| 8-byte signed integer. Rnage: -9223372036854775808 to 9223372036854775807 integer|4-byte signed integer. Range: -2147483648 to 2147483647 short| 2-byte signed integer numbers. Range: -32768 to 32767 byte| 1-byte signed integer number. Range: -128 to 127 diff --git a/README.md b/README.md index 19144eb6732..bf277759354 100644 --- a/README.md +++ b/README.md @@ -18,7 +18,7 @@ You include Delta Lake in your Maven project by adding it as a dependency in you io.delta delta-core_2.11 - 0.6.0 + 0.6.1 ``` @@ -27,7 +27,7 @@ You include Delta Lake in your Maven project by adding it as a dependency in you You include Delta Lake in your SBT project by adding the following line to your build.sbt file: ```scala -libraryDependencies += "io.delta" %% "delta-core" % "0.6.0" +libraryDependencies += "io.delta" %% "delta-core" % "0.6.1" ``` ## API Documentation diff --git a/build.sbt b/build.sbt index f0ec3a49f19..c845d9e0554 100644 --- a/build.sbt +++ b/build.sbt @@ -20,7 +20,7 @@ organization := "io.delta" scalaVersion := "2.12.10" -sparkVersion := "3.0.0" +sparkVersion := "3.0.1-SNAPSHOT" libraryDependencies ++= Seq( // Adding test classifier seems to break transitive resolution of the core dependencies @@ -36,13 +36,14 @@ libraryDependencies ++= Seq( "org.apache.spark" %% "spark-catalyst" % sparkVersion.value % "test" classifier "tests", "org.apache.spark" %% "spark-core" % sparkVersion.value % "test" classifier "tests", "org.apache.spark" %% "spark-sql" % sparkVersion.value % "test" classifier "tests", + "org.apache.spark" %% "spark-hive" % sparkVersion.value % "test" classifier "tests", // Compiler plugins // -- Bump up the genjavadoc version explicitly to 0.16 to work with Scala 2.12 compilerPlugin("com.typesafe.genjavadoc" %% "genjavadoc-plugin" % "0.16" cross CrossVersion.full) ) -resolvers += "Temporary Staging of Spark 3.0" at "https://docs.delta.io/spark3artifacts/rc1/maven/" +resolvers += "Temporary Staging of Spark 3.0" at "https://docs.delta.io/spark3artifacts/snapshot-fa608b94/maven/" antlr4Settings diff --git a/build/sbt-config/repositories b/build/sbt-config/repositories index 675bb45f43f..eaaab9180bb 100644 --- a/build/sbt-config/repositories +++ b/build/sbt-config/repositories @@ -10,4 +10,4 @@ bintray-typesafe-sbt-plugin-releases: https://dl.bintray.com/typesafe/sbt-plugins/, [organization]/[module]/(scala_[scalaVersion]/)(sbt_[sbtVersion]/)[revision]/[type]s/[artifact](-[classifier]).[ext] bintray-spark-packages: https://dl.bintray.com/spark-packages/maven/ typesafe-releases: http://repo.typesafe.com/typesafe/releases/ - temporary-staging-for-spark-3.0: https://docs.delta.io/spark3artifacts/rc1/maven/ + temporary-staging-for-spark-3.0: https://docs.delta.io/spark3artifacts/snapshot-fa608b94/maven/ diff --git a/docs/generate_api_docs.py b/docs/generate_api_docs.py index 7c81758c645..e66b438d30f 100755 --- a/docs/generate_api_docs.py +++ b/docs/generate_api_docs.py @@ -43,11 +43,6 @@ def main(): with WorkingDirectory(repo_root_dir): run_cmd(["build/sbt", ";clean;unidoc"], stream_output=verbose) - # Generate Python docs - print('## Generating Python(Sphinx) docs ...') - with WorkingDirectory(sphinx_gen_dir): - run_cmd(["make", "html"], stream_output=verbose) - # Update Scala docs print("## Patching ScalaDoc ...") with WorkingDirectory(scaladoc_gen_dir): @@ -78,7 +73,7 @@ def main(): # Create script elements to load new js files javadoc_jquery_script = \ - js_script_start + path_to_js_file + "lib/jquery.js" + js_script_end + js_script_start + path_to_js_file + "lib/jquery.min.js" + js_script_end javadoc_api_docs_script = \ js_script_start + path_to_js_file + "lib/api-javadocs.js" + js_script_end javadoc_script_elements = javadoc_jquery_script + javadoc_api_docs_script @@ -88,10 +83,15 @@ def main(): # Patch the js and css files run_cmd(["mkdir", "-p", "./lib"]) - run_cmd(["cp", scaladoc_gen_dir + "/lib/jquery.js", "./lib/"]) # copy jquery from ScalaDocs + run_cmd(["cp", scaladoc_gen_dir + "/lib/jquery.min.js", "./lib/"]) # copy from ScalaDocs run_cmd(["cp", docs_root_dir + "/api-javadocs.js", "./lib/"]) # copy new js file append(docs_root_dir + "/api-javadocs.css", "./stylesheet.css") # append new styles + # Generate Python docs + print('## Generating Python(Sphinx) docs ...') + with WorkingDirectory(sphinx_gen_dir): + run_cmd(["make", "html"], stream_output=verbose) + # Copy to final location log("Copying to API doc directory %s" % all_api_docs_final_dir) run_cmd(["rm", "-rf", all_api_docs_final_dir]) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 07375f32bd7..8d5085bb345 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -73,7 +73,10 @@ object MimaExcludes { ProblemFilters.exclude[DirectMissingMethodProblem]("io.delta.tables.DeltaMergeBuilder.recordOperation$default$5"), ProblemFilters.exclude[DirectMissingMethodProblem]("io.delta.tables.DeltaMergeBuilder.recordEvent$default$4"), ProblemFilters.exclude[DirectMissingMethodProblem]("io.delta.tables.DeltaMergeBuilder.recordOperation$default$8"), - ProblemFilters.exclude[DirectMissingMethodProblem]("io.delta.tables.DeltaMergeBuilder.initializeLogIfNecessary$default$2") + ProblemFilters.exclude[DirectMissingMethodProblem]("io.delta.tables.DeltaMergeBuilder.initializeLogIfNecessary$default$2"), + + // Changes in 0.7.0 + ProblemFilters.exclude[DirectMissingMethodProblem]("io.delta.tables.DeltaTable.makeUpdateTable") ) } diff --git a/python/delta/tables.py b/python/delta/tables.py index f97c5db7379..7c3eb911644 100644 --- a/python/delta/tables.py +++ b/python/delta/tables.py @@ -316,6 +316,28 @@ def forPath(cls, sparkSession, path): sparkSession._jsparkSession, path) return DeltaTable(sparkSession, jdt) + @classmethod + @since(0.7) + def forName(cls, sparkSession, tableOrViewName): + """ + Create a DeltaTable using the given table or view name using the given SparkSession. + + :param sparkSession: SparkSession to use for loading the table + :param tableOrViewName: name of the table or view + :return: loaded Delta table + :rtype: :py:class:`~delta.tables.DeltaTable` + + Example:: + + deltaTable = DeltaTable.forName(spark, "tblName") + + .. note:: Evolving + """ + assert sparkSession is not None + jdt = sparkSession._sc._jvm.io.delta.tables.DeltaTable.forName( + sparkSession._jsparkSession, tableOrViewName) + return DeltaTable(sparkSession, jdt) + @classmethod @since(0.4) def isDeltaTable(cls, sparkSession, identifier): diff --git a/python/delta/testing/utils.py b/python/delta/testing/utils.py index eff8993aa48..4e70c2086f9 100644 --- a/python/delta/testing/utils.py +++ b/python/delta/testing/utils.py @@ -14,31 +14,46 @@ # limitations under the License. # +import os +import shutil +import sys +import tempfile import unittest from pyspark import SparkContext, SparkConf -import sys +from pyspark.sql import SparkSession -class PySparkTestCase(unittest.TestCase): +class DeltaTestCase(unittest.TestCase): + """Test class base that sets up a correctly configured SparkSession for querying Delta tables. + """ def setUp(self): self._old_sys_path = list(sys.path) class_name = self.__class__.__name__ + self.warehouse_dir = "./spark-warehouse/" + if os.path.exists(self.warehouse_dir) and os.path.isdir(self.warehouse_dir): + shutil.rmtree(self.warehouse_dir) # Configurations to speed up tests and reduce memory footprint - conf = SparkConf() \ - .setAppName(class_name) \ - .setMaster('local[4]') \ - .set("spark.ui.enabled", "false") \ - .set("spark.databricks.delta.snapshotPartitions", "2") \ - .set("spark.sql.shuffle.partitions", "5") \ - .set("delta.log.cacheSize", "3") \ - .set("spark.sql.sources.parallelPartitionDiscovery.parallelism", "5") - # Enable Delta's SQL syntax for Spark 3.0+. Older versions require a hack to - # enable it. See "DeltaSqlTests.setUp" for details. - conf.set("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension") - self.sc = SparkContext(conf=conf) + self.spark = SparkSession.builder \ + .appName(class_name) \ + .master('local[4]') \ + .config("spark.ui.enabled", "false") \ + .config("spark.databricks.delta.snapshotPartitions", "2") \ + .config("spark.sql.shuffle.partitions", "5") \ + .config("delta.log.cacheSize", "3") \ + .config("spark.sql.sources.parallelPartitionDiscovery.parallelism", "5") \ + .config("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension") \ + .config("spark.sql.catalog.spark_catalog", + "org.apache.spark.sql.delta.catalog.DeltaCatalog") \ + .getOrCreate() + self.sc = self.spark.sparkContext + self.tempPath = tempfile.mkdtemp() + self.tempFile = os.path.join(self.tempPath, "tempFile") def tearDown(self): self.sc.stop() + shutil.rmtree(self.tempPath) + if os.path.exists(self.warehouse_dir) and os.path.isdir(self.warehouse_dir): + shutil.rmtree(self.warehouse_dir) sys.path = self._old_sys_path diff --git a/python/delta/tests/test_deltatable.py b/python/delta/tests/test_deltatable.py index b39cc5d7e87..63693110940 100644 --- a/python/delta/tests/test_deltatable.py +++ b/python/delta/tests/test_deltatable.py @@ -19,33 +19,26 @@ import shutil import os -from pyspark.sql import SQLContext, Row, SparkSession +from pyspark.sql import Row from pyspark.sql.functions import * from pyspark.sql.types import * from delta.tables import * -from delta.testing.utils import PySparkTestCase +from delta.testing.utils import DeltaTestCase -class DeltaTableTests(PySparkTestCase): - - def setUp(self): - super(DeltaTableTests, self).setUp() - self.sqlContext = SQLContext(self.sc) - self.spark = SparkSession(self.sc) - self.tempPath = tempfile.mkdtemp() - self.tempFile = os.path.join(self.tempPath, "tempFile") - - def tearDown(self): - self.spark.stop() - shutil.rmtree(self.tempPath) - super(DeltaTableTests, self).tearDown() +class DeltaTableTests(DeltaTestCase): def test_forPath(self): self.__writeDeltaTable([('a', 1), ('b', 2), ('c', 3)]) dt = DeltaTable.forPath(self.spark, self.tempFile).toDF() self.__checkAnswer(dt, [('a', 1), ('b', 2), ('c', 3)]) + def test_forName(self): + self.__writeAsTable([('a', 1), ('b', 2), ('c', 3)], "test") + df = DeltaTable.forName(self.spark, "test").toDF() + self.__checkAnswer(df, [('a', 1), ('b', 2), ('c', 3)]) + def test_alias_and_toDF(self): self.__writeDeltaTable([('a', 1), ('b', 2), ('c', 3)]) dt = DeltaTable.forPath(self.spark, self.tempFile).toDF() @@ -380,6 +373,10 @@ def __writeDeltaTable(self, datalist): df = self.spark.createDataFrame(datalist, ["key", "value"]) df.write.format("delta").save(self.tempFile) + def __writeAsTable(self, datalist, tblName): + df = self.spark.createDataFrame(datalist, ["key", "value"]) + df.write.format("delta").saveAsTable(tblName) + def __overwriteDeltaTable(self, datalist): df = self.spark.createDataFrame(datalist, ["key", "value"]) df.write.format("delta").mode("overwrite").save(self.tempFile) diff --git a/python/delta/tests/test_sql.py b/python/delta/tests/test_sql.py index fa20dfd23b8..01812b449d8 100644 --- a/python/delta/tests/test_sql.py +++ b/python/delta/tests/test_sql.py @@ -19,49 +19,29 @@ import shutil import os -from pyspark.sql import SQLContext, functions, Row, SparkSession -from pyspark import SparkContext, SparkConf +from delta.testing.utils import DeltaTestCase -from delta.tables import DeltaTable -from delta.testing.utils import PySparkTestCase - -class DeltaSqlTests(PySparkTestCase): +class DeltaSqlTests(DeltaTestCase): def setUp(self): super(DeltaSqlTests, self).setUp() - spark = SparkSession(self.sc) - if self.sc.version < "3.": - # Manually activate "DeltaSparkSessionExtension" in PySpark 2.4 in a cloned session - # because "spark.sql.extensions" is not picked up. (See SPARK-25003). - self.sc._jvm.io.delta.sql.DeltaSparkSessionExtension() \ - .apply(spark._jsparkSession.extensions()) - self.spark = SparkSession(self.sc, spark._jsparkSession.cloneSession()) - else: - self.spark = spark - self.temp_path = tempfile.mkdtemp() - self.temp_file = os.path.join(self.temp_path, "delta_sql_test_table") # Create a simple Delta table inside the temp directory to test SQL commands. df = self.spark.createDataFrame([('a', 1), ('b', 2), ('c', 3)], ["key", "value"]) - df.write.format("delta").save(self.temp_file) - df.write.mode("overwrite").format("delta").save(self.temp_file) - - def tearDown(self): - self.spark.stop() - shutil.rmtree(self.temp_path) - super(DeltaSqlTests, self).tearDown() + df.write.format("delta").save(self.tempFile) + df.write.mode("overwrite").format("delta").save(self.tempFile) def test_vacuum(self): self.spark.sql("set spark.databricks.delta.retentionDurationCheck.enabled = false") try: - deleted_files = self.spark.sql("VACUUM '%s' RETAIN 0 HOURS" % self.temp_file).collect() + deleted_files = self.spark.sql("VACUUM '%s' RETAIN 0 HOURS" % self.tempFile).collect() # Verify `VACUUM` did delete some data files - self.assertTrue(self.temp_file in deleted_files[0][0]) + self.assertTrue(self.tempFile in deleted_files[0][0]) finally: self.spark.sql("set spark.databricks.delta.retentionDurationCheck.enabled = true") def test_describe_history(self): - assert(len(self.spark.sql("desc history delta.`%s`" % (self.temp_file)).collect()) > 0) + assert(len(self.spark.sql("desc history delta.`%s`" % (self.tempFile)).collect()) > 0) def test_generate(self): # create a delta table @@ -109,6 +89,53 @@ def test_convert(self): shutil.rmtree(temp_path2) shutil.rmtree(temp_path3) + def test_ddls(self): + table = "deltaTable" + table2 = "deltaTable2" + try: + def read_table(): + return self.spark.sql(f"SELECT * FROM {table}") + + self.spark.sql(f"DROP TABLE IF EXISTS {table}") + self.spark.sql(f"DROP TABLE IF EXISTS {table2}") + + self.spark.sql(f"CREATE TABLE {table}(a LONG, b String NOT NULL) USING delta") + self.assertEqual(read_table().count(), 0) + + self.__checkAnswer( + self.spark.sql(f"DESCRIBE TABLE {table}").select("col_name", "data_type"), + [("a", "bigint"), ("b", "string"), ("", ""), ("# Partitioning", ""), + ("Not partitioned", "")], + schema=["col_name", "data_type"]) + + self.spark.sql(f"ALTER TABLE {table} CHANGE COLUMN a a LONG AFTER b") + self.assertSequenceEqual(["b", "a"], [f.name for f in read_table().schema.fields]) + + self.spark.sql(f"ALTER TABLE {table} ALTER COLUMN b DROP NOT NULL") + self.assertIn(True, [f.nullable for f in read_table().schema.fields if f.name == "b"]) + + self.spark.sql(f"ALTER TABLE {table} ADD COLUMNS (x LONG)") + self.assertIn("x", [f.name for f in read_table().schema.fields]) + + self.spark.sql(f"ALTER TABLE {table} SET TBLPROPERTIES ('k' = 'v')") + self.__checkAnswer(self.spark.sql(f"SHOW TBLPROPERTIES {table}"), [('k', 'v')]) + + self.spark.sql(f"ALTER TABLE {table} UNSET TBLPROPERTIES ('k')") + self.__checkAnswer(self.spark.sql(f"SHOW TBLPROPERTIES {table}"), []) + + self.spark.sql(f"ALTER TABLE {table} RENAME TO {table2}") + self.assertEqual(self.spark.sql(f"SELECT * FROM {table2}").count(), 0) + + test_dir = os.path.join(tempfile.mkdtemp(), table2) + self.spark.createDataFrame([("", 0, 0)], ["b", "a", "x"]) \ + .write.format("delta").save(test_dir) + + self.spark.sql(f"ALTER TABLE {table2} SET LOCATION '{test_dir}'") + self.assertEqual(self.spark.sql(f"SELECT * FROM {table2}").count(), 1) + finally: + self.spark.sql(f"DROP TABLE IF EXISTS {table}") + self.spark.sql(f"DROP TABLE IF EXISTS {table2}") + def __checkAnswer(self, df, expectedAnswer, schema=["key", "value"]): if not expectedAnswer: self.assertEqual(df.count(), 0) diff --git a/src/main/scala/io/delta/sql/DeltaSparkSessionExtension.scala b/src/main/scala/io/delta/sql/DeltaSparkSessionExtension.scala index 0ca1c7a468f..12cda706e69 100644 --- a/src/main/scala/io/delta/sql/DeltaSparkSessionExtension.scala +++ b/src/main/scala/io/delta/sql/DeltaSparkSessionExtension.scala @@ -16,10 +16,11 @@ package io.delta.sql -import org.apache.spark.sql.delta.{DeltaAnalysis, DeltaUnsupportedOperationsCheck, PreprocessTableMerge} +import org.apache.spark.sql.delta._ import io.delta.sql.parser.DeltaSqlParser import org.apache.spark.sql.SparkSessionExtensions +import org.apache.spark.sql.internal.SQLConf /** * An extension for Spark SQL to activate Delta SQL parser to support Delta SQL grammar. @@ -80,8 +81,14 @@ class DeltaSparkSessionExtension extends (SparkSessionExtensions => Unit) { extensions.injectCheckRule { session => new DeltaUnsupportedOperationsCheck(session) } + extensions.injectPostHocResolutionRule { session => + new PreprocessTableUpdate(session.sessionState.conf) + } extensions.injectPostHocResolutionRule { session => new PreprocessTableMerge(session.sessionState.conf) } + extensions.injectPostHocResolutionRule { session => + new PreprocessTableDelete(session.sessionState.conf) + } } } diff --git a/src/main/scala/io/delta/sql/parser/DeltaSqlParser.scala b/src/main/scala/io/delta/sql/parser/DeltaSqlParser.scala index 60617650830..24d5c82acce 100644 --- a/src/main/scala/io/delta/sql/parser/DeltaSqlParser.scala +++ b/src/main/scala/io/delta/sql/parser/DeltaSqlParser.scala @@ -42,9 +42,9 @@ import java.util.Locale import scala.collection.JavaConverters._ -import org.apache.spark.sql.delta.commands.DeltaGenerateCommand +import org.apache.spark.sql.delta.commands._ import io.delta.sql.parser.DeltaSqlBaseParser._ -import io.delta.tables.execution.{DescribeDeltaHistoryCommand, VacuumTableCommand} +import io.delta.tables.execution.VacuumTableCommand import org.antlr.v4.runtime._ import org.antlr.v4.runtime.atn.PredictionMode import org.antlr.v4.runtime.misc.{Interval, ParseCancellationException} @@ -57,8 +57,6 @@ import org.apache.spark.sql.catalyst.parser.{ParseErrorListener, ParseException, import org.apache.spark.sql.catalyst.parser.ParserUtils.{string, withOrigin} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.trees.Origin -import org.apache.spark.sql.delta.commands.DescribeDeltaDetailCommand -import org.apache.spark.sql.delta.commands.ConvertToDeltaCommand import org.apache.spark.sql.types._ /** diff --git a/src/main/scala/io/delta/tables/DeltaMergeBuilder.scala b/src/main/scala/io/delta/tables/DeltaMergeBuilder.scala index b6f9ab13fac..94701a32ff1 100644 --- a/src/main/scala/io/delta/tables/DeltaMergeBuilder.scala +++ b/src/main/scala/io/delta/tables/DeltaMergeBuilder.scala @@ -222,7 +222,7 @@ class DeltaMergeBuilder private( * @since 0.3.0 */ @Evolving - def execute(): Unit = { + def execute(): Unit = improveUnsupportedOpError { val sparkSession = targetTable.toDF.sparkSession val resolvedMergeInto = DeltaMergeInto.resolveReferences(mergePlan, sparkSession.sessionState.conf)( diff --git a/src/main/scala/io/delta/tables/DeltaTable.scala b/src/main/scala/io/delta/tables/DeltaTable.scala index 6dcddd8038e..38f66b11fa3 100644 --- a/src/main/scala/io/delta/tables/DeltaTable.scala +++ b/src/main/scala/io/delta/tables/DeltaTable.scala @@ -39,7 +39,7 @@ import org.apache.spark.sql.types.StructType * @since 0.3.0 */ @Evolving -class DeltaTable private[tables](df: => Dataset[Row], deltaLog: DeltaLog) +class DeltaTable private[tables](df: Dataset[Row], deltaLog: DeltaLog) extends DeltaTableOperations { /** diff --git a/src/main/scala/io/delta/tables/execution/DeltaTableOperations.scala b/src/main/scala/io/delta/tables/execution/DeltaTableOperations.scala index c8e5508cc22..25b01745e9a 100644 --- a/src/main/scala/io/delta/tables/execution/DeltaTableOperations.scala +++ b/src/main/scala/io/delta/tables/execution/DeltaTableOperations.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.delta.commands.{DeleteCommand, DeltaGenerateCommand, import org.apache.spark.sql.delta.util.AnalysisHelper import io.delta.tables.DeltaTable -import org.apache.spark.sql.{functions, Column, DataFrame} +import org.apache.spark.sql.{functions, Column, DataFrame, Dataset} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions.{Expression, SubqueryExpression} @@ -34,34 +34,9 @@ import org.apache.spark.sql.catalyst.plans.logical._ */ trait DeltaTableOperations extends AnalysisHelper { self: DeltaTable => - protected def executeDelete(condition: Option[Expression]): Unit = { - val delete = DeltaDelete(self.toDF.queryExecution.analyzed, condition) - - // current DELETE does not support subquery, - // and the reason why perform checking here is that - // we want to have more meaningful exception messages, - // instead of having some random msg generated by executePlan(). - subqueryNotSupportedCheck(condition, "DELETE") - - val qe = sparkSession.sessionState.executePlan(delete) - val resolvedDelete = qe.analyzed.asInstanceOf[DeltaDelete] - val deleteCommand = DeleteCommand(resolvedDelete) - deleteCommand.run(sparkSession) - } - - protected def toStrColumnMap(map: Map[String, String]): Map[String, Column] = { - map.toSeq.map { case (k, v) => k -> functions.expr(v) }.toMap - } - - protected def makeUpdateTable( - target: DeltaTable, - onCondition: Option[Column], - setColumns: Seq[(String, Column)]): DeltaUpdateTable = { - val updateColumns = setColumns.map { x => UnresolvedAttribute.quotedString(x._1) } - val updateExpressions = setColumns.map{ x => x._2.expr } - val condition = onCondition.map {_.expr} - DeltaUpdateTable( - target.toDF.queryExecution.analyzed, updateColumns, updateExpressions, condition) + protected def executeDelete(condition: Option[Expression]): Unit = improveUnsupportedOpError { + val delete = DeleteFromTable(self.toDF.queryExecution.analyzed, condition) + toDataset(sparkSession, delete) } protected def executeHistory(deltaLog: DeltaLog, limit: Option[Int]): DataFrame = { @@ -79,28 +54,14 @@ trait DeltaTableOperations extends AnalysisHelper { self: DeltaTable => generate.run(sparkSession) } - protected def executeUpdate(set: Map[String, Column], condition: Option[Column]): Unit = { - val setColumns = set.map{ case (col, expr) => (col, expr) }.toSeq - - // Current UPDATE does not support subquery, - // and the reason why perform checking here is that - // we want to have more meaningful exception messages, - // instead of having some random msg generated by executePlan(). - subqueryNotSupportedCheck(condition.map {_.expr}, "UPDATE") - - val update = makeUpdateTable(self, condition, setColumns) - val resolvedUpdate = - DeltaUpdateTable.resolveReferences(update, tryResolveReferences(sparkSession)(_, update)) - val updateCommand = PreprocessTableUpdate(sparkSession.sessionState.conf)(resolvedUpdate) - updateCommand.run(sparkSession) - } - - private def subqueryNotSupportedCheck(condition: Option[Expression], op: String): Unit = { - condition.foreach { cond => - if (SubqueryExpression.hasSubquery(cond)) { - throw DeltaErrors.subqueryNotSupportedException(op, cond) - } - } + protected def executeUpdate( + set: Map[String, Column], + condition: Option[Column]): Unit = improveUnsupportedOpError { + val assignments = set.map { case (targetColName, column) => + Assignment(UnresolvedAttribute.quotedString(targetColName), column.expr) + }.toSeq + val update = UpdateTable(self.toDF.queryExecution.analyzed, assignments, condition.map(_.expr)) + toDataset(sparkSession, update) } protected def executeVacuum( @@ -110,5 +71,9 @@ trait DeltaTableOperations extends AnalysisHelper { self: DeltaTable => sparkSession.emptyDataFrame } + protected def toStrColumnMap(map: Map[String, String]): Map[String, Column] = { + map.toSeq.map { case (k, v) => k -> functions.expr(v) }.toMap + } + protected def sparkSession = self.toDF.sparkSession } diff --git a/src/main/scala/io/delta/tables/execution/DescribeDeltaHistoryCommand.scala b/src/main/scala/io/delta/tables/execution/DescribeDeltaHistoryCommand.scala deleted file mode 100644 index fe3304c8f79..00000000000 --- a/src/main/scala/io/delta/tables/execution/DescribeDeltaHistoryCommand.scala +++ /dev/null @@ -1,63 +0,0 @@ -/* - * Copyright (2020) 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 io.delta.tables.execution - -import org.apache.hadoop.fs.Path -import org.apache.spark.sql.{Encoders, Row, SparkSession} -import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} -import org.apache.spark.sql.delta.actions.CommitInfo -import org.apache.spark.sql.delta.{DeltaErrors, DeltaLog, DeltaTableIdentifier} -import org.apache.spark.sql.execution.command.RunnableCommand - -case class DescribeDeltaHistoryCommand( - path: Option[String], - table: Option[TableIdentifier], - limit: Option[Int]) extends RunnableCommand { - - override val output: Seq[Attribute] = Encoders.product[CommitInfo].schema - .map(f => AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()) - - override def run(sparkSession: SparkSession): Seq[Row] = { - val basePath = - new Path(if (table.nonEmpty) { - DeltaTableIdentifier(sparkSession, table.get) match { - case Some(id) => id.path.getOrElse { - throw DeltaErrors.tableNotSupportedException("DESCRIBE HISTORY") - } - case None => throw DeltaErrors.notADeltaTableException("DESCRIBE HISTORY") - } - } else { - path.get - }) - - // Max array size - if (limit.exists(_ > Int.MaxValue - 8)) { - throw new IllegalArgumentException("Please use a limit less than Int.MaxValue - 8.") - } - - val deltaLog = DeltaLog.forTable(sparkSession, basePath) - if (deltaLog.snapshot.version == -1) { - throw DeltaErrors.notADeltaTableException( - "DESCRIBE HISTORY", - DeltaTableIdentifier(path = Some(basePath.toString))) - } - - import sparkSession.implicits._ - deltaLog.history.getHistory(limit).toDF().collect().toSeq - } -} diff --git a/src/main/scala/io/delta/tables/execution/VacuumTableCommand.scala b/src/main/scala/io/delta/tables/execution/VacuumTableCommand.scala index e1e3c90e669..670a2929ea2 100644 --- a/src/main/scala/io/delta/tables/execution/VacuumTableCommand.scala +++ b/src/main/scala/io/delta/tables/execution/VacuumTableCommand.scala @@ -42,16 +42,18 @@ case class VacuumTableCommand( override def run(sparkSession: SparkSession): Seq[Row] = { val pathToVacuum = - new Path(if (table.nonEmpty) { + if (path.nonEmpty) { + new Path(path.get) + } else if (table.nonEmpty) { DeltaTableIdentifier(sparkSession, table.get) match { - case Some(id) => id.path.getOrElse { - throw DeltaErrors.tableNotSupportedException("VACUUM") - } - case None => throw DeltaErrors.notADeltaTableException("VACUUM") + case Some(id) if id.path.nonEmpty => + new Path(id.path.get) + case _ => + new Path(sparkSession.sessionState.catalog.getTableMetadata(table.get).location) } } else { - path.get - }) + throw DeltaErrors.missingTableIdentifierException("VACUUM") + } val baseDeltaPath = DeltaTableUtils.findDeltaTableRoot(sparkSession, pathToVacuum) if (baseDeltaPath.isDefined) { if (baseDeltaPath.get != pathToVacuum) { diff --git a/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DeltaUpdateTable.scala b/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DeltaUpdateTable.scala index 46e1d985683..f4168f84449 100644 --- a/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DeltaUpdateTable.scala +++ b/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DeltaUpdateTable.scala @@ -42,30 +42,6 @@ case class DeltaUpdateTable( object DeltaUpdateTable { - /** Resolve all the references of target columns and condition using the given `resolver` */ - def resolveReferences( - update: DeltaUpdateTable, - resolver: Expression => Expression): DeltaUpdateTable = { - if (update.resolved) return update - assert(update.child.resolved) - - val DeltaUpdateTable(child, updateColumns, updateExpressions, condition) = update - - val cleanedUpAttributes = updateColumns.map { unresolvedExpr => - // Keep them unresolved but use the cleaned-up name parts from the resolved - val errMsg = s"Failed to resolve ${unresolvedExpr.sql} given columns " + - s"[${child.output.map(_.qualifiedName).mkString(", ")}]." - val resolveNameParts = - DeltaUpdateTable.getNameParts(resolver(unresolvedExpr), errMsg, update) - UnresolvedAttribute(resolveNameParts) - } - - update.copy( - updateColumns = cleanedUpAttributes, - updateExpressions = updateExpressions.map(resolver), - condition = condition.map(resolver)) - } - /** * Extracts name parts from a resolved expression referring to a nested or non-nested column * - For non-nested column, the resolved expression will be like `AttributeReference(...)`. @@ -84,18 +60,15 @@ object DeltaUpdateTable { * -> `AttributeReference(a)` ++ Seq(b, c) * -> [a, b, c] */ - def getNameParts( - resolvedTargetCol: Expression, - errMsg: String, - errNode: LogicalPlan): Seq[String] = { + def getTargetColNameParts(resolvedTargetCol: Expression, errMsg: String = null): Seq[String] = { def fail(extraMsg: String): Nothing = { - throw new AnalysisException( - s"$errMsg - $extraMsg", errNode.origin.line, errNode.origin.startPosition) + val msg = Option(errMsg).map(_ + " - ").getOrElse("") + extraMsg + throw new AnalysisException(msg) } def extractRecursively(expr: Expression): Seq[String] = expr match { - case attr: AttributeReference => Seq(attr.name) + case attr: Attribute => Seq(attr.name) case Alias(c, _) => extractRecursively(c) diff --git a/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/deltaMerge.scala b/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/deltaMerge.scala index 5c9a44f3c9c..ceb4beaf080 100644 --- a/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/deltaMerge.scala +++ b/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/deltaMerge.scala @@ -312,10 +312,10 @@ object DeltaMergeInto { // column name. The target columns do not need resolution. The right hand side // expression (i.e. sourceColumnBySameName) needs to be resolved only by the source // plan. - target.output.map(_.name).map { tgtColName => + fakeTargetPlan.output.map(_.name).map { tgtColName => val resolvedExpr = resolveOrFail( UnresolvedAttribute.quotedString(s"`$tgtColName`"), - source, s"$typ clause") + fakeSourcePlan, s"$typ clause") DeltaMergeAction(Seq(tgtColName), resolvedExpr) } case _: UnresolvedStar if shouldAutoMigrate => @@ -376,10 +376,8 @@ object DeltaMergeInto { // If clause allows nested field to be target, then this will return the all the // parts of the name (e.g., "a.b" -> Seq("a", "b")). Otherwise, this will // return only one string. - val resolvedNameParts = DeltaUpdateTable.getNameParts( - resolveOrFail(unresolvedAttrib, fakeTargetPlan, s"$typ clause"), - resolutionErrorMsg, - merge) + val resolvedNameParts = DeltaUpdateTable.getTargetColNameParts( + resolveOrFail(unresolvedAttrib, fakeTargetPlan, s"$typ clause"), resolutionErrorMsg) val resolvedExpr = resolveOrFail(expr, planToResolveAction, s"$typ clause") Seq(DeltaMergeAction(resolvedNameParts, resolvedExpr)) diff --git a/src/main/scala/org/apache/spark/sql/delta/Checkpoints.scala b/src/main/scala/org/apache/spark/sql/delta/Checkpoints.scala index e4f77095e05..58503d5ec4f 100644 --- a/src/main/scala/org/apache/spark/sql/delta/Checkpoints.scala +++ b/src/main/scala/org/apache/spark/sql/delta/Checkpoints.scala @@ -31,6 +31,7 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.mapred.{JobConf, TaskAttemptContextImpl, TaskAttemptID} import org.apache.hadoop.mapreduce.{Job, TaskType} +import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.util.SerializableConfiguration @@ -202,7 +203,7 @@ trait Checkpoints extends DeltaLogging { } } -object Checkpoints { +object Checkpoints extends DeltaLogging { /** * Writes out the contents of a [[Snapshot]] into a checkpoint file that * can be used to short-circuit future replays of the log. @@ -305,6 +306,11 @@ object Checkpoints { throw new IllegalStateException( "State of the checkpoint doesn't match that of the snapshot.") } + + // Attempting to write empty checkpoint + if (checkpointSize.value == 0) { + logWarning(DeltaErrors.EmptyCheckpointErrorMessage) + } CheckpointMetaData(snapshot.version, checkpointSize.value, None) } } diff --git a/src/main/scala/org/apache/spark/sql/delta/Checksum.scala b/src/main/scala/org/apache/spark/sql/delta/Checksum.scala index 7744559c39d..59c01147eaa 100644 --- a/src/main/scala/org/apache/spark/sql/delta/Checksum.scala +++ b/src/main/scala/org/apache/spark/sql/delta/Checksum.scala @@ -167,7 +167,7 @@ trait ValidateChecksum extends DeltaLogging { self: Snapshot => throw new IllegalStateException( "The transaction log has failed integrity checks. We recommend you contact " + s"Databricks support for assistance. To disable this check, set ${conf.key} to " + - s"false. Failed verification of:\n${mismatchStringOpt.get}" + s"false. Failed verification at version $version of:\n${mismatchStringOpt.get}" ) } } diff --git a/src/main/scala/org/apache/spark/sql/delta/DeltaAnalysis.scala b/src/main/scala/org/apache/spark/sql/delta/DeltaAnalysis.scala index 51ff4744c70..40eddb7d216 100644 --- a/src/main/scala/org/apache/spark/sql/delta/DeltaAnalysis.scala +++ b/src/main/scala/org/apache/spark/sql/delta/DeltaAnalysis.scala @@ -75,12 +75,35 @@ class DeltaAnalysis(session: SparkSession, conf: SQLConf) case dsv2 @ DataSourceV2Relation(d: DeltaTableV2, _, _, _, _) => DeltaRelation.fromV2Relation(d, dsv2) - // DML - TODO: Remove these and use stable public interfaces once they are in Spark - case u @ UpdateTable(table, assignments, condition) => + // DML - TODO: Remove these Delta-specific DML logical plans and use Spark's plans directly + + case d @ DeleteFromTable(table, condition) if d.childrenResolved => + // rewrites Delta from V2 to V1 + val newTarget = table.transformUp { case DeltaRelation(lr) => lr } + val indices = newTarget.collect { + case DeltaFullTable(index) => index + } + if (indices.isEmpty) { + // Not a Delta table at all, do not transform + d + } else if (indices.size == 1 && indices(0).deltaLog.snapshot.version > -1) { + // It is a well-defined Delta table with a schema + DeltaDelete(newTarget, condition) + } else { + // Not a well-defined Delta table + throw DeltaErrors.notADeltaSourceException("DELETE", Some(d)) + } + + case u @ UpdateTable(table, assignments, condition) if u.childrenResolved => val (cols, expressions) = assignments.map(a => a.key.asInstanceOf[NamedExpression] -> a.value).unzip // rewrites Delta from V2 to V1 val newTable = table.transformUp { case DeltaRelation(lr) => lr } + newTable.collectLeaves().headOption match { + case Some(DeltaFullTable(index)) => + case o => + throw DeltaErrors.notADeltaSourceException("UPDATE", o) + } DeltaUpdateTable(newTable, cols, expressions, condition) case m@MergeIntoTable(target, source, condition, matched, notMatched) if m.childrenResolved => @@ -101,8 +124,8 @@ class DeltaAnalysis(session: SparkSession, conf: SQLConf) insert.condition, DeltaMergeIntoClause.toActions(insert.assignments)) case other => - throw new AnalysisException( - s"${other.prettyName} clauses cannot be part of the WHEN MATCHED clause in MERGE INTO.") + throw new AnalysisException(s"${other.prettyName} clauses cannot be part of the " + + s"WHEN NOT MATCHED clause in MERGE INTO.") } // rewrites Delta from V2 to V1 val newTarget = target.transformUp { case DeltaRelation(lr) => lr } @@ -116,11 +139,6 @@ class DeltaAnalysis(session: SparkSession, conf: SQLConf) deltaMergeResolved - - case DeleteFromTable(table, condition) => - // rewrites Delta from V2 to V1 - val newTarget = table.transformUp { case DeltaRelation(lr) => lr } - DeltaDelete(newTarget, condition) } /** diff --git a/src/main/scala/org/apache/spark/sql/delta/DeltaErrors.scala b/src/main/scala/org/apache/spark/sql/delta/DeltaErrors.scala index b16c95e07e8..921a5a64e60 100644 --- a/src/main/scala/org/apache/spark/sql/delta/DeltaErrors.scala +++ b/src/main/scala/org/apache/spark/sql/delta/DeltaErrors.scala @@ -20,12 +20,14 @@ package org.apache.spark.sql.delta import java.io.{FileNotFoundException, IOException} import java.util.ConcurrentModificationException -import org.apache.spark.sql.delta.actions.{CommitInfo, Metadata} +import org.apache.spark.sql.delta.actions.{Action, CommitInfo, Metadata} +import org.apache.spark.sql.delta.catalog.DeltaCatalog import org.apache.spark.sql.delta.hooks.PostCommitHook import org.apache.spark.sql.delta.metering.DeltaLogging -import org.apache.spark.sql.delta.schema.{Invariant, InvariantViolationException} +import org.apache.spark.sql.delta.schema.{Invariant, InvariantViolationException, SchemaUtils} import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.delta.util.JsonUtils +import io.delta.sql.DeltaSparkSessionExtension import org.apache.hadoop.fs.Path import org.apache.spark.{SparkConf, SparkEnv} @@ -118,6 +120,13 @@ object DeltaErrors "like to ignore updates, set the option 'ignoreChanges' to 'true'. If you would like the " + "data update to be reflected, please restart this query with a fresh checkpoint directory." + val EmptyCheckpointErrorMessage = + s""" + |Attempted to write an empty checkpoint without any actions. This checkpoint will not be + |useful in recomputing the state of the table. However this might cause other checkpoints to + |get deleted based on retention settings. + """.stripMargin + /** * File not found hint for Delta, replacing the normal one which is inapplicable. * @@ -206,6 +215,18 @@ object DeltaErrors new AnalysisException(s"$command destination only supports Delta sources.\n$planName") } + def schemaChangedSinceAnalysis(atAnalysis: StructType, latestSchema: StructType): Throwable = { + val schemaDiff = SchemaUtils.reportDifferences(atAnalysis, latestSchema) + .map(_.replace("Specified", "Latest")) + new AnalysisException( + s"""The schema of your Delta table has changed in an incompatible way since your DataFrame or + |DeltaTable object was created. Please redefine your DataFrame or DeltaTable object. + |Changes:\n${schemaDiff.mkString("\n")} + |This check can be turned off by setting the session configuration key + |${DeltaSQLConf.DELTA_SCHEMA_ON_READ_CHECK_ENABLED.key} to false. + """.stripMargin) + } + def invalidColumnName(name: String): Throwable = { new AnalysisException( s"""Attribute name "$name" contains invalid character(s) among " ,;{}()\\n\\t=". @@ -431,8 +452,19 @@ object DeltaErrors s"Couldn't find all part files of the checkpoint version: $version", ae) } - def deltaVersionsNotContiguousException(deltaVersions: Seq[Long]): Throwable = { - new IllegalStateException(s"versions ($deltaVersions) are not contiguous") + def deltaVersionsNotContiguousException( + spark: SparkSession, deltaVersions: Seq[Long]): Throwable = { + new IllegalStateException(s"Versions ($deltaVersions) are not contiguous.") + } + + def actionNotFoundException(action: String, version: Long): Throwable = { + new IllegalStateException( + s""" + |The $action of your Delta table couldn't be recovered while Reconstructing + |version: ${version.toString}. Did you manually delete files in the _delta_log directory? + |Set ${DeltaSQLConf.DELTA_STATE_RECONSTRUCTION_VALIDATION_ENABLED.key} + |to "false" to skip validation. + """.stripMargin) } def schemaChangedException(oldSchema: StructType, newSchema: StructType): Throwable = { @@ -806,6 +838,31 @@ object DeltaErrors new AnalysisException("Cannot describe the history of a view.") } + def copyIntoEncryptionOnlyS3(scheme: String): Throwable = { + new IllegalArgumentException( + s"Invalid scheme $scheme. COPY INTO source encryption is only supported for S3 paths.") + } + + def copyIntoEncryptionSseCRequired(): Throwable = { + new IllegalArgumentException( + s"Invalid encryption type. COPY INTO source encryption must specify 'type' = 'SSE-C'.") + } + + def copyIntoEncryptionMasterKeyRequired(): Throwable = { + new IllegalArgumentException( + s"Invalid encryption arguments. COPY INTO source encryption must specify a masterKey.") + } + + def copyIntoCredentialsOnlyS3(scheme: String): Throwable = { + new IllegalArgumentException( + s"Invalid scheme $scheme. COPY INTO source credentials are only supported for S3 paths.") + } + + def copyIntoCredentialsAllRequired(cause: Throwable): Throwable = { + new IllegalArgumentException( + "COPY INTO credentials must include awsKeyId, awsSecretKey, and awsSessionToken.", cause) + } + def postCommitHookFailedException( failedHook: PostCommitHook, failedOnCommitVersion: Long, @@ -836,6 +893,26 @@ object DeltaErrors s"Couldn't find column $column in:\n${schema.treeString}") } + def metadataAbsentException(): Throwable = { + new IllegalStateException( + s""" + |Couldn't find Metadata while committing the first version of the Delta table. To disable + |this check set ${DeltaSQLConf.DELTA_COMMIT_VALIDATION_ENABLED.key} to "false" + """.stripMargin) + } + + def addFilePartitioningMismatchException( + addFilePartitions: Seq[String], + metadataPartitions: Seq[String]): Throwable = { + new IllegalStateException( + s""" + |The AddFile contains partitioning schema different from the table's partitioning schema + |expected: ${DeltaErrors.formatColumnList(metadataPartitions)} + |actual: ${DeltaErrors.formatColumnList(addFilePartitions)} + |To disable this check set ${DeltaSQLConf.DELTA_COMMIT_VALIDATION_ENABLED.key} to "false" + """.stripMargin) + } + def concurrentModificationExceptionMsg( sparkConf: SparkConf, baseMessage: String, @@ -855,6 +932,22 @@ object DeltaErrors |'ignoreDeletes' or 'ignoreChanges'. Refer to $docPage for details. """.stripMargin } + + def configureSparkSessionWithExtensionAndCatalog(originalException: Throwable): Throwable = { + val catalogImplConfig = SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION.key + new AnalysisException( + s"""This Delta operation requires the SparkSession to be configured with the + |DeltaSparkSessionExtension and the DeltaCatalog. Please set the necessary + |configurations when creating the SparkSession as shown below. + | + | SparkSession.builder() + | .option("spark.sql.extensions", "${classOf[DeltaSparkSessionExtension].getName}") + | .option("$catalogImplConfig", "${classOf[DeltaCatalog].getName}" + | ... + | .build() + """.stripMargin, + cause = Some(originalException)) + } } /** The basic class for all Tahoe commit conflict exceptions. */ diff --git a/src/main/scala/org/apache/spark/sql/delta/DeltaHistoryManager.scala b/src/main/scala/org/apache/spark/sql/delta/DeltaHistoryManager.scala index 3c24b1702cf..d9fd1ef30b8 100644 --- a/src/main/scala/org/apache/spark/sql/delta/DeltaHistoryManager.scala +++ b/src/main/scala/org/apache/spark/sql/delta/DeltaHistoryManager.scala @@ -189,7 +189,7 @@ class DeltaHistoryManager( * that way we can reconstruct the entire history of the table. This method assumes that the * commits are contiguous. */ - private def getEarliestReproducibleCommit: Long = { + private[delta] def getEarliestReproducibleCommit: Long = { val files = deltaLog.store.listFrom(FileNames.deltaFile(deltaLog.logPath, 0)) .filter(f => FileNames.isDeltaFile(f.getPath) || FileNames.isCheckpointFile(f.getPath)) @@ -222,7 +222,7 @@ class DeltaHistoryManager( // if we have a multi-part checkpoint, we need to check that all parts exist val numParts = parts.getOrElse(1) val preCount = checkpointMap.getOrElse(checkpointVersion -> numParts, 0) - if (numParts == preCount + 1 && smallestDeltaVersion <= checkpointVersion) { + if (numParts == preCount + 1) { lastCompleteCheckpoint = Some(checkpointVersion) } checkpointMap.put(checkpointVersion -> numParts, preCount + 1) diff --git a/src/main/scala/org/apache/spark/sql/delta/DeltaLog.scala b/src/main/scala/org/apache/spark/sql/delta/DeltaLog.scala index 8bbebf46bed..b96b3716bc2 100644 --- a/src/main/scala/org/apache/spark/sql/delta/DeltaLog.scala +++ b/src/main/scala/org/apache/spark/sql/delta/DeltaLog.scala @@ -376,8 +376,9 @@ class DeltaLog private( } /** Used to link the files present in the table into the query planner. */ - val fileIndex = TahoeLogFileIndex(spark, this, dataPath, partitionFilters, versionToUse) val snapshotToUse = versionToUse.map(getSnapshotAt(_)).getOrElse(snapshot) + val fileIndex = TahoeLogFileIndex( + spark, this, dataPath, snapshotToUse.metadata.schema, partitionFilters, versionToUse) new HadoopFsRelation( fileIndex, @@ -494,11 +495,15 @@ object DeltaLog extends DeltaLogging { // - Different `scheme` // - Different `authority` (e.g., different user tokens in the path) // - Different mount point. + // + // Whether the `cached` object is newly created because of case missing. + var newlyCreated = false val cached = try { deltaLogCache.get(path, new Callable[DeltaLog] { override def call(): DeltaLog = recordDeltaOperation( null, "delta.log.create", Map(TAG_TAHOE_PATH -> path.getParent.toString)) { AnalysisHelper.allowInvokingTransformsInAnalyzer { + newlyCreated = true new DeltaLog(path, path.getParent, clock) } } @@ -508,11 +513,17 @@ object DeltaLog extends DeltaLogging { throw e.getCause } - // Invalidate the cache if the reference is no longer valid as a result of the - // log being deleted. - if (cached.snapshot.version == -1 || cached.isValid) { + if (cached.snapshot.version == -1) { + cached + } else if (newlyCreated) { + // The DeltaLog object was not in the cache and we just created it. It should be valid. + cached + } else if (cached.isValid) { + // The DeltaLog object is got from the cache, we should check whether it's valid. cached } else { + // Invalidate the cache if the reference is no longer valid as a result of the + // log being deleted. deltaLogCache.invalidate(path) apply(spark, path) } diff --git a/src/main/scala/org/apache/spark/sql/delta/DeltaOperations.scala b/src/main/scala/org/apache/spark/sql/delta/DeltaOperations.scala index bae6954d19d..534194ddceb 100644 --- a/src/main/scala/org/apache/spark/sql/delta/DeltaOperations.scala +++ b/src/main/scala/org/apache/spark/sql/delta/DeltaOperations.scala @@ -47,13 +47,16 @@ object DeltaOperations { operationMetrics.contains(s) ).transform((_, v) => v.value.toString) } + + val userMetadata: Option[String] = None } /** Recorded during batch inserts. Predicates can be provided for overwrites. */ case class Write( mode: SaveMode, partitionBy: Option[Seq[String]] = None, - predicate: Option[String] = None) extends Operation("WRITE") { + predicate: Option[String] = None, + override val userMetadata: Option[String] = None) extends Operation("WRITE") { override val parameters: Map[String, Any] = Map("mode" -> mode.name()) ++ partitionBy.map("partitionBy" -> JsonUtils.toJson(_)) ++ predicate.map("predicate" -> _) @@ -64,7 +67,8 @@ object DeltaOperations { case class StreamingUpdate( outputMode: OutputMode, queryId: String, - epochId: Long) extends Operation("STREAMING UPDATE") { + epochId: Long, + override val userMetadata: Option[String] = None) extends Operation("STREAMING UPDATE") { override val parameters: Map[String, Any] = Map("outputMode" -> outputMode.toString, "queryId" -> queryId, "epochId" -> epochId.toString) override val operationMetrics: Set[String] = DeltaOperationMetrics.STREAMING_UPDATE @@ -159,7 +163,7 @@ object DeltaOperations { strMetrics += "numCopiedRows" -> "0" } else { strMetrics += "numCopiedRows" -> ( - numOutputRows - strMetrics("numUpdatedRows").toInt).toString + numOutputRows - strMetrics("numUpdatedRows").toLong).toString } strMetrics } diff --git a/src/main/scala/org/apache/spark/sql/delta/DeltaOptions.scala b/src/main/scala/org/apache/spark/sql/delta/DeltaOptions.scala index f7931a83cd0..4314bd97565 100644 --- a/src/main/scala/org/apache/spark/sql/delta/DeltaOptions.scala +++ b/src/main/scala/org/apache/spark/sql/delta/DeltaOptions.scala @@ -49,6 +49,7 @@ trait DeltaWriteOptions import DeltaOptions._ val replaceWhere: Option[String] = options.get(REPLACE_WHERE_OPTION) + val userMetadata: Option[String] = options.get(USER_METADATA_OPTION) /** * Whether to add an adaptive shuffle before writing out the files to break skew, and coalesce @@ -144,6 +145,8 @@ object DeltaOptions extends DeltaLogging { val MERGE_SCHEMA_OPTION = "mergeSchema" /** An option to allow overwriting schema and partitioning during an overwrite write operation. */ val OVERWRITE_SCHEMA_OPTION = "overwriteSchema" + /** An option to specify user-defined metadata in commitInfo */ + val USER_METADATA_OPTION = "userMetadata" val MAX_FILES_PER_TRIGGER_OPTION = "maxFilesPerTrigger" val MAX_FILES_PER_TRIGGER_OPTION_DEFAULT = 1000 @@ -160,6 +163,7 @@ object DeltaOptions extends DeltaLogging { MERGE_SCHEMA_OPTION, EXCLUDE_REGEX_OPTION, OVERWRITE_SCHEMA_OPTION, + USER_METADATA_OPTION, MAX_FILES_PER_TRIGGER_OPTION, IGNORE_FILE_DELETION_OPTION, IGNORE_CHANGES_OPTION, diff --git a/src/main/scala/org/apache/spark/sql/delta/DeltaTable.scala b/src/main/scala/org/apache/spark/sql/delta/DeltaTable.scala index 911303f87b6..62fe6b55a99 100644 --- a/src/main/scala/org/apache/spark/sql/delta/DeltaTable.scala +++ b/src/main/scala/org/apache/spark/sql/delta/DeltaTable.scala @@ -54,7 +54,8 @@ object DeltaTable { */ object DeltaFullTable { def unapply(a: LogicalPlan): Option[TahoeLogFileIndex] = a match { - case PhysicalOperation(_, filters, DeltaTable(index: TahoeLogFileIndex)) => + case PhysicalOperation(_, filters, lr @ DeltaTable(index: TahoeLogFileIndex)) => + if (index.deltaLog.snapshot.version < 0) return None if (index.partitionFilters.isEmpty && index.versionToUse.isEmpty && filters.isEmpty) { Some(index) } else if (index.versionToUse.nonEmpty) { @@ -65,6 +66,8 @@ object DeltaFullTable { throw new AnalysisException( s"Expect a full scan of Delta sources, but found a partial scan. path:${index.path}") } + // Convert V2 relations to V1 and perform the check + case DeltaRelation(lr) => unapply(lr) case _ => None } diff --git a/src/main/scala/org/apache/spark/sql/delta/DeltaUnsupportedOperationsCheck.scala b/src/main/scala/org/apache/spark/sql/delta/DeltaUnsupportedOperationsCheck.scala index 390e9d8a6a6..90befa07a22 100644 --- a/src/main/scala/org/apache/spark/sql/delta/DeltaUnsupportedOperationsCheck.scala +++ b/src/main/scala/org/apache/spark/sql/delta/DeltaUnsupportedOperationsCheck.scala @@ -16,17 +16,21 @@ package org.apache.spark.sql.delta +import org.apache.spark.sql.delta.catalog.DeltaTableV2 import org.apache.spark.sql.delta.metering.DeltaLogging import org.apache.spark.sql.delta.sources.DeltaSourceUtils import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.{AppendData, LogicalPlan, OverwriteByExpression, V2WriteCommand} import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.RefreshTable +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation /** - * A rule to check whether the functions are supported only when Hive support is enabled + * A rule to add helpful error messages when Delta is being used with unsupported Hive operations + * or if an unsupported operation is being made, e.g. a DML operation like + * INSERT/UPDATE/DELETE/MERGE when a table doesn't exist. */ case class DeltaUnsupportedOperationsCheck(spark: SparkSession) extends (LogicalPlan => Unit) @@ -44,41 +48,63 @@ case class DeltaUnsupportedOperationsCheck(spark: SparkSession) } } - def apply(plan: LogicalPlan): Unit = { - plan.foreach { - case c: CreateTableLikeCommand => - recordDeltaEvent(null, "delta.unsupported.createLike") - fail(operation = "CREATE TABLE LIKE", c.sourceTable) + def apply(plan: LogicalPlan): Unit = plan.foreach { + // Unsupported Hive commands + case c: CreateTableLikeCommand => + recordDeltaEvent(null, "delta.unsupported.createLike") + fail(operation = "CREATE TABLE LIKE", c.sourceTable) - case a: AnalyzePartitionCommand => - recordDeltaEvent(null, "delta.unsupported.analyzePartition") - fail(operation = "ANALYZE TABLE PARTITION", a.tableIdent) + case a: AnalyzePartitionCommand => + recordDeltaEvent(null, "delta.unsupported.analyzePartition") + fail(operation = "ANALYZE TABLE PARTITION", a.tableIdent) - case a: AlterTableAddPartitionCommand => - recordDeltaEvent(null, "delta.unsupported.addPartition") - fail(operation = "ALTER TABLE ADD PARTITION", a.tableName) + case a: AlterTableAddPartitionCommand => + recordDeltaEvent(null, "delta.unsupported.addPartition") + fail(operation = "ALTER TABLE ADD PARTITION", a.tableName) - case a: AlterTableDropPartitionCommand => - recordDeltaEvent(null, "delta.unsupported.dropPartition") - fail(operation = "ALTER TABLE DROP PARTITION", a.tableName) + case a: AlterTableDropPartitionCommand => + recordDeltaEvent(null, "delta.unsupported.dropPartition") + fail(operation = "ALTER TABLE DROP PARTITION", a.tableName) - case a: AlterTableRecoverPartitionsCommand => - recordDeltaEvent(null, "delta.unsupported.recoverPartitions") - fail(operation = "ALTER TABLE RECOVER PARTITIONS", a.tableName) + case a: AlterTableRecoverPartitionsCommand => + recordDeltaEvent(null, "delta.unsupported.recoverPartitions") + fail(operation = "ALTER TABLE RECOVER PARTITIONS", a.tableName) - case a: AlterTableSerDePropertiesCommand => - recordDeltaEvent(null, "delta.unsupported.alterSerDe") - fail(operation = "ALTER TABLE table SET SERDEPROPERTIES", a.tableName) + case a: AlterTableSerDePropertiesCommand => + recordDeltaEvent(null, "delta.unsupported.alterSerDe") + fail(operation = "ALTER TABLE table SET SERDEPROPERTIES", a.tableName) - case l: LoadDataCommand => - recordDeltaEvent(null, "delta.unsupported.loadData") - fail(operation = "LOAD DATA", l.table) + case l: LoadDataCommand => + recordDeltaEvent(null, "delta.unsupported.loadData") + fail(operation = "LOAD DATA", l.table) - case i: InsertIntoDataSourceDirCommand => - recordDeltaEvent(null, "delta.unsupported.insertDirectory") - fail(operation = "INSERT OVERWRITE DIRECTORY", i.provider) + case i: InsertIntoDataSourceDirCommand => + recordDeltaEvent(null, "delta.unsupported.insertDirectory") + fail(operation = "INSERT OVERWRITE DIRECTORY", i.provider) - case _ => // OK + // Delta table checks + case append: AppendData => + val op = if (append.isByName) "APPEND" else "INSERT" + checkDeltaTableExists(append, op) + + case overwrite: OverwriteByExpression => + checkDeltaTableExists(overwrite, "OVERWRITE") + + case _ => // OK + } + + /** + * Check that the given operation is being made on a full Delta table that exists. + */ + private def checkDeltaTableExists(command: V2WriteCommand, operation: String): Unit = { + command.table match { + case DeltaRelation(lr) => + // the extractor performs the check that we want if this is indeed being called on a Delta + // table. It should leave others unchanged + if (DeltaFullTable.unapply(lr).isEmpty) { + throw DeltaErrors.notADeltaTableException(operation) + } + case _ => } } } diff --git a/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala b/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala index a0b2b7b1623..ada4aa76db6 100644 --- a/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala +++ b/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala @@ -270,6 +270,17 @@ trait OptimisticTransactionImpl extends TransactionalWrite with SQLMetricsReport } } + /** + * Return the user-defined metadata for the operation. + */ + def getUserMetadata(op: Operation): Option[String] = { + // option wins over config if both are set + op.userMetadata match { + case data @ Some(_) => data + case None => spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_USER_METADATA) + } + } + /** * Modifies the state of the log by adding a new commit that is based on a read at * the given `lastVersion`. In the case of a conflict with a concurrent writer this @@ -282,6 +293,8 @@ trait OptimisticTransactionImpl extends TransactionalWrite with SQLMetricsReport def commit(actions: Seq[Action], op: DeltaOperations.Operation): Long = recordDeltaOperation( deltaLog, "delta.commit") { + commitStartNano = System.nanoTime() + val version = try { // Try to commit at the next version. var finalActions = prepareCommit(actions, op) @@ -313,7 +326,8 @@ trait OptimisticTransactionImpl extends TransactionalWrite with SQLMetricsReport Some(readVersion).filter(_ >= 0), None, Some(isBlindAppend), - getOperationMetrics(op)) + getOperationMetrics(op), + getUserMetadata(op)) finalActions = commitInfo +: finalActions } @@ -355,25 +369,59 @@ trait OptimisticTransactionImpl extends TransactionalWrite with SQLMetricsReport // If the metadata has changed, add that to the set of actions var finalActions = newMetadata.toSeq ++ actions val metadataChanges = finalActions.collect { case m: Metadata => m } - assert( - metadataChanges.length <= 1, - "Cannot change the metadata more than once in a transaction.") + if (metadataChanges.length > 1) { + recordDeltaEvent(deltaLog, "delta.metadataCheck.multipleMetadataActions", data = Map( + "metadataChanges" -> metadataChanges + )) + assert( + metadataChanges.length <= 1, "Cannot change the metadata more than once in a transaction.") + } metadataChanges.foreach(m => verifyNewMetadata(m)) - // If this is the first commit and no protocol is specified, initialize the protocol version. if (snapshot.version == -1) { deltaLog.ensureLogDirectoryExist() + // If this is the first commit and no protocol is specified, initialize the protocol version. if (!finalActions.exists(_.isInstanceOf[Protocol])) { finalActions = Protocol() +: finalActions } + // If this is the first commit and no metadata is specified, throw an exception + if (!finalActions.exists(_.isInstanceOf[Metadata])) { + recordDeltaEvent(deltaLog, "delta.metadataCheck.noMetadataInInitialCommit") + if (spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_COMMIT_VALIDATION_ENABLED)) { + throw DeltaErrors.metadataAbsentException() + } + logWarning( + s""" + |Detected no metadata in initial commit but commit validation was turned off. To turn + |it back on set ${DeltaSQLConf.DELTA_COMMIT_VALIDATION_ENABLED} to "true" + """.stripMargin) + } } + val partitionColumns = metadata.partitionColumns.toSet finalActions = finalActions.map { // Fetch global config defaults for the first commit case m: Metadata if snapshot.version == -1 => val updatedConf = DeltaConfigs.mergeGlobalConfigs( spark.sessionState.conf, m.configuration, Protocol()) m.copy(configuration = updatedConf) + case a: AddFile if partitionColumns != a.partitionValues.keySet => + // If the partitioning in metadata does not match the partitioning in the AddFile + recordDeltaEvent(deltaLog, "delta.metadataCheck.partitionMismatch", data = Map( + "tablePartitionColumns" -> metadata.partitionColumns, + "filePartitionValues" -> a.partitionValues + )) + if (spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_COMMIT_VALIDATION_ENABLED)) { + throw DeltaErrors.addFilePartitioningMismatchException( + a.partitionValues.keySet.toSeq, partitionColumns.toSeq) + } + logWarning( + s""" + |Detected mismatch in partition values between AddFile and table metadata but + |commit validation was turned off. + |To turn it back on set ${DeltaSQLConf.DELTA_COMMIT_VALIDATION_ENABLED} to "true" + """.stripMargin) + a case other => other } diff --git a/src/main/scala/org/apache/spark/sql/delta/PartitionFiltering.scala b/src/main/scala/org/apache/spark/sql/delta/PartitionFiltering.scala index 61f73a684de..99257c04c8e 100644 --- a/src/main/scala/org/apache/spark/sql/delta/PartitionFiltering.scala +++ b/src/main/scala/org/apache/spark/sql/delta/PartitionFiltering.scala @@ -21,13 +21,10 @@ import org.apache.spark.sql.delta.stats.DeltaScan import org.apache.spark.sql.catalyst.expressions._ -trait PartitionFiltering { - self: Snapshot => +/** Provides the capability for partition pruning when querying a Delta table. */ +trait PartitionFiltering { self: Snapshot => - def filesForScan( - projection: Seq[Attribute], - filters: Seq[Expression], - keepStats: Boolean = false): DeltaScan = { + def filesForScan(projection: Seq[Attribute], filters: Seq[Expression]): DeltaScan = { implicit val enc = SingleAction.addFileEncoder val partitionFilters = filters.flatMap { filter => diff --git a/src/main/scala/org/apache/spark/sql/delta/PreprocessTableDelete.scala b/src/main/scala/org/apache/spark/sql/delta/PreprocessTableDelete.scala new file mode 100644 index 00000000000..4a8e7621f0e --- /dev/null +++ b/src/main/scala/org/apache/spark/sql/delta/PreprocessTableDelete.scala @@ -0,0 +1,51 @@ +/* + * Copyright (2020) 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.commands.DeleteCommand + +import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases +import org.apache.spark.sql.catalyst.expressions.SubqueryExpression +import org.apache.spark.sql.catalyst.plans.logical.{DeltaDelete, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.internal.SQLConf + +/** + * Preprocess the [[DeltaDelete]] plan to convert to [[DeleteCommand]]. + */ +case class PreprocessTableDelete(conf: SQLConf) extends Rule[LogicalPlan] { + + override def apply(plan: LogicalPlan): LogicalPlan = { + plan.resolveOperators { + case d: DeltaDelete if d.resolved => + d.condition.foreach { cond => + if (SubqueryExpression.hasSubquery(cond)) { + throw DeltaErrors.subqueryNotSupportedException("DELETE", cond) + } + } + toCommand(d) + } + } + + def toCommand(d: DeltaDelete): DeleteCommand = EliminateSubqueryAliases(d.child) match { + case DeltaFullTable(tahoeFileIndex) => + DeleteCommand(tahoeFileIndex, d.child, d.condition) + + case o => + throw DeltaErrors.notADeltaSourceException("DELETE", Some(o)) + } +} diff --git a/src/main/scala/org/apache/spark/sql/delta/PreprocessTableUpdate.scala b/src/main/scala/org/apache/spark/sql/delta/PreprocessTableUpdate.scala index e8fee1d75c9..2de15c8b7f3 100644 --- a/src/main/scala/org/apache/spark/sql/delta/PreprocessTableUpdate.scala +++ b/src/main/scala/org/apache/spark/sql/delta/PreprocessTableUpdate.scala @@ -18,12 +18,33 @@ package org.apache.spark.sql.delta import org.apache.spark.sql.delta.commands.UpdateCommand -import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, UnresolvedAttribute} -import org.apache.spark.sql.catalyst.plans.logical.DeltaUpdateTable +import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases +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.internal.SQLConf -case class PreprocessTableUpdate(conf: SQLConf) extends UpdateExpressionsSupport { - def apply(update: DeltaUpdateTable): UpdateCommand = { +/** + * Preprocesses the [[DeltaUpdateTable]] logical plan before converting it to [[UpdateCommand]]. + * - Adjusts the column order, which could be out of order, based on the destination table + * - Generates expressions to compute the value of all target columns in Delta table, while taking + * into account that the specified SET clause may only update some columns or nested fields of + * columns. + */ +case class PreprocessTableUpdate(conf: SQLConf) + extends Rule[LogicalPlan] with UpdateExpressionsSupport { + + override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperators { + case u: DeltaUpdateTable if u.resolved => + u.condition.foreach { cond => + if (SubqueryExpression.hasSubquery(cond)) { + throw DeltaErrors.subqueryNotSupportedException("UPDATE", cond) + } + } + toCommand(u) + } + + def toCommand(update: DeltaUpdateTable): UpdateCommand = { val index = EliminateSubqueryAliases(update.child) match { case DeltaFullTable(tahoeFileIndex) => tahoeFileIndex @@ -31,9 +52,7 @@ case class PreprocessTableUpdate(conf: SQLConf) extends UpdateExpressionsSupport throw DeltaErrors.notADeltaSourceException("UPDATE", Some(o)) } - val targetColNameParts = - update.updateColumns.map{ col => UnresolvedAttribute(col.name).nameParts } - + val targetColNameParts = update.updateColumns.map(DeltaUpdateTable.getTargetColNameParts(_)) val alignedUpdateExprs = generateUpdateExpressions( update.child.output, targetColNameParts, update.updateExpressions, conf.resolver) UpdateCommand(index, update.child, alignedUpdateExprs, update.condition) diff --git a/src/main/scala/org/apache/spark/sql/delta/Snapshot.scala b/src/main/scala/org/apache/spark/sql/delta/Snapshot.scala index 068fd91845e..518cd6266f3 100644 --- a/src/main/scala/org/apache/spark/sql/delta/Snapshot.scala +++ b/src/main/scala/org/apache/spark/sql/delta/Snapshot.scala @@ -122,6 +122,16 @@ class Snapshot( /** The current set of actions in this [[Snapshot]]. */ def state: Dataset[SingleAction] = cachedState.getDS + /** Helper method to log missing actions when state reconstruction checks are not enabled */ + protected def logMissingActionWarning(action: String): Unit = { + logWarning( + s""" + |Found no $action in computed state, setting it to defaults. State reconstruction + |validation was turned off. To turn it back on set + |${DeltaSQLConf.DELTA_STATE_RECONSTRUCTION_VALIDATION_ENABLED.key} to "true" + """.stripMargin) + } + /** * Computes some statistics around the transaction log, therefore on the actions made on this * Delta table. @@ -129,9 +139,9 @@ class Snapshot( protected lazy val computedState: State = { val implicits = spark.implicits import implicits._ - state.select( - coalesce(last($"protocol", ignoreNulls = true), defaultProtocol()) as "protocol", - coalesce(last($"metaData", ignoreNulls = true), emptyMetadata()) as "metadata", + var _computedState = state.select( + last($"protocol", ignoreNulls = true) as "protocol", + last($"metaData", ignoreNulls = true) as "metadata", collect_set($"txn") as "setTransactions", // sum may return null for empty data set. coalesce(sum($"add.size"), lit(0L)) as "sizeInBytes", @@ -141,6 +151,31 @@ class Snapshot( count($"remove") as "numOfRemoves", count($"txn") as "numOfSetTransactions" ).as[State](stateEncoder).first() + val stateReconstructionCheck = spark.sessionState.conf.getConf( + DeltaSQLConf.DELTA_STATE_RECONSTRUCTION_VALIDATION_ENABLED) + if (_computedState.protocol == null) { + recordDeltaEvent( + deltaLog, + opType = "delta.assertions.missingAction", + data = Map("version" -> version.toString, "action" -> "Protocol", "source" -> "Snapshot")) + if (stateReconstructionCheck) { + throw DeltaErrors.actionNotFoundException("protocol", version) + } + logMissingActionWarning("protocol") + _computedState = _computedState.copy(protocol = Protocol()) + } + if (_computedState.metadata == null) { + recordDeltaEvent( + deltaLog, + opType = "delta.assertions.missingAction", + data = Map("version" -> version.toString, "action" -> "Metadata", "source" -> "Metadata")) + if (stateReconstructionCheck) { + throw DeltaErrors.actionNotFoundException("metadata", version) + } + logMissingActionWarning("metadata") + _computedState = _computedState.copy(metadata = Metadata()) + } + _computedState } def protocol: Protocol = computedState.protocol diff --git a/src/main/scala/org/apache/spark/sql/delta/SnapshotManagement.scala b/src/main/scala/org/apache/spark/sql/delta/SnapshotManagement.scala index 12e18f4ab65..daf27a23143 100644 --- a/src/main/scala/org/apache/spark/sql/delta/SnapshotManagement.scala +++ b/src/main/scala/org/apache/spark/sql/delta/SnapshotManagement.scala @@ -350,8 +350,8 @@ trait SnapshotManagement { self: DeltaLog => // Turn this to a vector so that we can compare it with a range. val deltaVersions = versions.toVector if (deltaVersions.nonEmpty && - (deltaVersions.head to deltaVersions.last) != deltaVersions) { - throw new IllegalStateException(s"versions ($deltaVersions) are not contiguous") + (deltaVersions.head to deltaVersions.last) != deltaVersions) { + throw DeltaErrors.deltaVersionsNotContiguousException(self.spark, deltaVersions) } } } diff --git a/src/main/scala/org/apache/spark/sql/delta/UpdateExpressionsSupport.scala b/src/main/scala/org/apache/spark/sql/delta/UpdateExpressionsSupport.scala index 2ad315f40df..dee88744585 100644 --- a/src/main/scala/org/apache/spark/sql/delta/UpdateExpressionsSupport.scala +++ b/src/main/scala/org/apache/spark/sql/delta/UpdateExpressionsSupport.scala @@ -138,6 +138,7 @@ trait UpdateExpressionsSupport extends CastSupport { } } + /** See docs on overloaded method. */ protected def generateUpdateExpressions( targetCols: Seq[NamedExpression], nameParts: Seq[Seq[String]], diff --git a/src/main/scala/org/apache/spark/sql/delta/actions/actions.scala b/src/main/scala/org/apache/spark/sql/delta/actions/actions.scala index 20956a0efd8..ad851769bc9 100644 --- a/src/main/scala/org/apache/spark/sql/delta/actions/actions.scala +++ b/src/main/scala/org/apache/spark/sql/delta/actions/actions.scala @@ -260,7 +260,8 @@ case class CommitInfo( isolationLevel: Option[String], /** Whether this commit has blindly appended without caring about existing files */ isBlindAppend: Option[Boolean], - operationMetrics: Option[Map[String, String]]) extends Action with CommitMarker { + operationMetrics: Option[Map[String, String]], + userMetadata: Option[String]) extends Action with CommitMarker { override def wrap: SingleAction = SingleAction(commitInfo = this) override def withTimestamp(timestamp: Long): CommitInfo = { @@ -302,7 +303,8 @@ object NotebookInfo { object CommitInfo { def empty(version: Option[Long] = None): CommitInfo = { - CommitInfo(version, null, None, None, null, null, None, None, None, None, None, None, None) + CommitInfo(version, null, None, None, null, null, None, None, + None, None, None, None, None, None) } def apply( @@ -313,7 +315,8 @@ object CommitInfo { readVersion: Option[Long], isolationLevel: Option[String], isBlindAppend: Option[Boolean], - operationMetrics: Option[Map[String, String]]): CommitInfo = { + operationMetrics: Option[Map[String, String]], + userMetadata: Option[String]): CommitInfo = { val getUserName = commandContext.get("user").flatMap { case "unknown" => None case other => Option(other) @@ -332,8 +335,8 @@ object CommitInfo { readVersion, isolationLevel, isBlindAppend, - operationMetrics - ) + operationMetrics, + userMetadata) } } diff --git a/src/main/scala/org/apache/spark/sql/delta/catalog/DeltaCatalog.scala b/src/main/scala/org/apache/spark/sql/delta/catalog/DeltaCatalog.scala index 05d035e82df..3471b896898 100644 --- a/src/main/scala/org/apache/spark/sql/delta/catalog/DeltaCatalog.scala +++ b/src/main/scala/org/apache/spark/sql/delta/catalog/DeltaCatalog.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.delta.catalog import java.util +// scalastyle:off import.ordering.noEmptyLine import scala.collection.JavaConverters._ import scala.collection.mutable @@ -106,7 +107,6 @@ class DeltaCatalog(val spark: SparkSession) extends DelegatingCatalogExtension partitionColumnNames = partitionColumns, bucketSpec = maybeBucketSpec, properties = tableProperties.toMap, - tracksPartitionsInCatalog = false, comment = Option(properties.get("comment"))) // END: copy-paste from the super method finished. @@ -127,26 +127,17 @@ class DeltaCatalog(val spark: SparkSession) extends DelegatingCatalogExtension try { super.loadTable(ident) match { case v1: V1Table if DeltaTableUtils.isDeltaTable(v1.catalogTable) => - if (isPathIdentifier(ident)) { - // TODO: Get rid of this hack when path based tables become a first class citizen. - DeltaTableV2( - spark, - new Path(v1.catalogTable.location)) - } else { - DeltaTableV2( - spark, - new Path(v1.catalogTable.location), - catalogTable = Some(v1.catalogTable), - tableIdentifier = Some(ident.toString)) - } + DeltaTableV2( + spark, + new Path(v1.catalogTable.location), + catalogTable = Some(v1.catalogTable), + tableIdentifier = Some(ident.toString)) case o => o } } catch { case _: NoSuchDatabaseException | _: NoSuchNamespaceException | _: NoSuchTableException if isPathIdentifier(ident) => - DeltaTableV2( - spark, - new Path(ident.name())) + DeltaTableV2(spark, new Path(ident.name())) } } diff --git a/src/main/scala/org/apache/spark/sql/delta/catalog/DeltaTableV2.scala b/src/main/scala/org/apache/spark/sql/delta/catalog/DeltaTableV2.scala index 921737d838b..94f2e29b55d 100644 --- a/src/main/scala/org/apache/spark/sql/delta/catalog/DeltaTableV2.scala +++ b/src/main/scala/org/apache/spark/sql/delta/catalog/DeltaTableV2.scala @@ -22,7 +22,7 @@ import java.{util => ju} import scala.collection.JavaConverters._ import scala.collection.mutable -import org.apache.spark.sql.delta.{DeltaErrors, DeltaLog, DeltaOptions, DeltaTableUtils, DeltaTimeTravelSpec, Snapshot} +import org.apache.spark.sql.delta.{DeltaErrors, DeltaLog, DeltaOptions, DeltaTableIdentifier, DeltaTableUtils, DeltaTimeTravelSpec, Snapshot} import org.apache.spark.sql.delta.commands.WriteIntoDelta import org.apache.spark.sql.delta.sources.{DeltaDataSource, DeltaSourceUtils} import org.apache.hadoop.fs.Path @@ -52,8 +52,14 @@ case class DeltaTableV2( tableIdentifier: Option[String] = None, timeTravelOpt: Option[DeltaTimeTravelSpec] = None) extends Table with SupportsWrite { - private lazy val (rootPath, partitionFilters, timeTravelByPath) = - DeltaDataSource.parsePathIdentifier(spark, path.toString) + private lazy val (rootPath, partitionFilters, timeTravelByPath) = { + if (catalogTable.isDefined) { + // Fast path for reducing path munging overhead + (new Path(catalogTable.get.location), Nil, None) + } else { + DeltaDataSource.parsePathIdentifier(spark, path.toString) + } + } // The loading of the DeltaLog is lazy in order to reduce the amount of FileSystem calls, // in cases where we will fallback to the V1 behavior. @@ -73,11 +79,11 @@ case class DeltaTableV2( timeTravelOpt.orElse(timeTravelByPath) } - private def snapshot: Snapshot = { + private lazy val snapshot: Snapshot = { timeTravelSpec.map { spec => val v = DeltaTableUtils.resolveTimeTravelVersion(spark.sessionState.conf, deltaLog, spec) deltaLog.getSnapshotAt(v._1) - }.getOrElse(deltaLog.snapshot) + }.getOrElse(deltaLog.update(stalenessAcceptable = true)) } override def schema(): StructType = snapshot.schema @@ -115,9 +121,15 @@ case class DeltaTableV2( * paths. */ def toBaseRelation: BaseRelation = { + if (deltaLog.snapshot.version == -1) { + val id = catalogTable.map(ct => DeltaTableIdentifier(table = Some(ct.identifier))) + .getOrElse(DeltaTableIdentifier(path = Some(path.toString))) + throw DeltaErrors.notADeltaTableException(id) + } val partitionPredicates = DeltaDataSource.verifyAndCreatePartitionFilters( - path.toString, deltaLog, partitionFilters) + path.toString, deltaLog.snapshot, partitionFilters) + // TODO(burak): We should pass in the snapshot here deltaLog.createRelation(partitionPredicates, timeTravelSpec) } } diff --git a/src/main/scala/org/apache/spark/sql/delta/commands/ConvertToDeltaCommand.scala b/src/main/scala/org/apache/spark/sql/delta/commands/ConvertToDeltaCommand.scala index ec761c17163..6da8cc85373 100644 --- a/src/main/scala/org/apache/spark/sql/delta/commands/ConvertToDeltaCommand.scala +++ b/src/main/scala/org/apache/spark/sql/delta/commands/ConvertToDeltaCommand.scala @@ -129,7 +129,7 @@ abstract class ConvertToDeltaCommandBase( val table = v1.catalogTable // Hive adds some transient table properties which should be ignored val props = table.properties.filterKeys(_ != "transient_lastDdlTime") - Some(ConvertTarget(Some(table), table.provider, table.location.getPath, props)) + Some(ConvertTarget(Some(table), table.provider, new Path(table.location).toString, props)) case _: DeltaTableV2 => // Already a Delta table None @@ -520,7 +520,8 @@ abstract class ConvertToDeltaCommandBase( readVersion = None, isolationLevel = None, isBlindAppend = None, - metrics) + metrics, + userMetadata = None) val extraActions = Seq(commitInfo, Protocol(), metadata) val actions = extraActions.toIterator ++ addFiles diff --git a/src/main/scala/org/apache/spark/sql/delta/commands/CreateDeltaTableCommand.scala b/src/main/scala/org/apache/spark/sql/delta/commands/CreateDeltaTableCommand.scala index 7402c248e55..aa0900af004 100644 --- a/src/main/scala/org/apache/spark/sql/delta/commands/CreateDeltaTableCommand.scala +++ b/src/main/scala/org/apache/spark/sql/delta/commands/CreateDeltaTableCommand.scala @@ -190,10 +190,7 @@ case class CreateDeltaTableCommand( val tableWithDefaultOptions = tableWithLocation.copy( schema = new StructType(), partitionColumnNames = Nil, - tracksPartitionsInCatalog = - tableWithLocation.partitionColumnNames.nonEmpty && - sparkSession.sessionState.conf.manageFilesourcePartitions) - + tracksPartitionsInCatalog = true) updateCatalog(sparkSession, tableWithDefaultOptions) Nil diff --git a/src/main/scala/org/apache/spark/sql/delta/commands/DeltaGenerateCommand.scala b/src/main/scala/org/apache/spark/sql/delta/commands/DeltaGenerateCommand.scala index 0dc627d3543..524ab7d1b19 100644 --- a/src/main/scala/org/apache/spark/sql/delta/commands/DeltaGenerateCommand.scala +++ b/src/main/scala/org/apache/spark/sql/delta/commands/DeltaGenerateCommand.scala @@ -20,28 +20,13 @@ import org.apache.spark.sql.delta.{DeltaErrors, DeltaLog, DeltaTableIdentifier} import org.apache.spark.sql.delta.hooks.GenerateSymlinkManifest import org.apache.hadoop.fs.Path -import org.apache.spark.sql.{AnalysisException, Row, SparkSession} +import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.execution.command.RunnableCommand -trait DeltaGenerateCommandBase extends RunnableCommand { - - protected def getPath(spark: SparkSession, tableId: TableIdentifier): Path = { - DeltaTableIdentifier(spark, tableId) match { - case Some(id) if id.path.isDefined => new Path(id.path.get) - case Some(id) => - throw DeltaErrors.tableNotSupportedException("DELTA GENERATE") - case None => - // This is not a Delta table. - val metadata = spark.sessionState.catalog.getTableMetadata(tableId) - new Path(metadata.location) - } - } -} - case class DeltaGenerateCommand(modeName: String, tableId: TableIdentifier) - extends DeltaGenerateCommandBase { + extends RunnableCommand { import DeltaGenerateCommand._ @@ -49,10 +34,17 @@ case class DeltaGenerateCommand(modeName: String, tableId: TableIdentifier) if (!modeNameToGenerationFunc.contains(modeName)) { throw DeltaErrors.unsupportedGenerateModeException(modeName) } - val tablePath = getPath(sparkSession, tableId) + + val tablePath = DeltaTableIdentifier(sparkSession, tableId) match { + case Some(id) if id.path.isDefined => + new Path(id.path.get) + case _ => + new Path(sparkSession.sessionState.catalog.getTableMetadata(tableId).location) + } + val deltaLog = DeltaLog.forTable(sparkSession, tablePath) if (deltaLog.snapshot.version < 0) { - throw new AnalysisException(s"Delta table not found at $tablePath.") + throw DeltaErrors.notADeltaTableException("GENERATE") } val generationFunc = modeNameToGenerationFunc(modeName) generationFunc(sparkSession, deltaLog) diff --git a/src/main/scala/org/apache/spark/sql/delta/commands/DescribeDeltaHistoryCommand.scala b/src/main/scala/org/apache/spark/sql/delta/commands/DescribeDeltaHistoryCommand.scala new file mode 100644 index 00000000000..3558f22e92c --- /dev/null +++ b/src/main/scala/org/apache/spark/sql/delta/commands/DescribeDeltaHistoryCommand.scala @@ -0,0 +1,97 @@ +/* + * Copyright (2020) 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.commands + +// scalastyle:off import.ordering.noEmptyLine +import org.apache.spark.sql.delta.{DeltaErrors, DeltaLog, DeltaTableIdentifier} +import org.apache.spark.sql.delta.actions.CommitInfo +import org.apache.spark.sql.delta.metering.DeltaLogging +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation +import org.apache.spark.sql.catalyst.catalog.CatalogTableType +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics} +import org.apache.spark.sql.execution.command.RunnableCommand + +/** + * A logical placeholder for describing a Delta table's history, so that the history can be + * leveraged in subqueries. Replaced with `DescribeDeltaHistoryCommand` during planning. + */ +case class DescribeDeltaHistory( + path: Option[String], + tableIdentifier: Option[TableIdentifier], + limit: Option[Int], + output: Seq[Attribute] = ExpressionEncoder[CommitInfo]().schema.toAttributes) + extends LeafNode with MultiInstanceRelation { + override def computeStats(): Statistics = Statistics(sizeInBytes = conf.defaultSizeInBytes) + + override def newInstance(): LogicalPlan = copy(output = output.map(_.newInstance())) +} + +/** + * A command for describing the history of a Delta table. + */ +case class DescribeDeltaHistoryCommand( + path: Option[String], + tableIdentifier: Option[TableIdentifier], + limit: Option[Int], + override val output: Seq[Attribute] = ExpressionEncoder[CommitInfo]().schema.toAttributes) + extends RunnableCommand with DeltaLogging { + + override def run(sparkSession: SparkSession): Seq[Row] = { + val basePath = + if (path.nonEmpty) { + new Path(path.get) + } else if (tableIdentifier.nonEmpty) { + val sessionCatalog = sparkSession.sessionState.catalog + lazy val metadata = sessionCatalog.getTableMetadata(tableIdentifier.get) + + DeltaTableIdentifier(sparkSession, tableIdentifier.get) match { + case Some(id) if id.path.nonEmpty => + new Path(id.path.get) + case Some(id) if id.table.nonEmpty => + new Path(metadata.location) + case _ => + if (metadata.tableType == CatalogTableType.VIEW) { + throw DeltaErrors.describeViewHistory + } + throw DeltaErrors.notADeltaTableException("DESCRIBE HISTORY") + } + } else { + throw DeltaErrors.missingTableIdentifierException("DESCRIBE HISTORY") + } + + // Max array size + if (limit.exists(_ > Int.MaxValue - 8)) { + throw new IllegalArgumentException("Please use a limit less than Int.MaxValue - 8.") + } + + val deltaLog = DeltaLog.forTable(sparkSession, basePath) + recordDeltaOperation(deltaLog, "delta.ddl.describeHistory") { + if (deltaLog.snapshot.version == -1) { + throw DeltaErrors.notADeltaTableException("DESCRIBE HISTORY") + } + + import sparkSession.implicits._ + deltaLog.history.getHistory(limit).toDF().collect().toSeq + } + } +} diff --git a/src/main/scala/org/apache/spark/sql/delta/commands/MergeIntoCommand.scala b/src/main/scala/org/apache/spark/sql/delta/commands/MergeIntoCommand.scala index c380f604ac0..c89776e587c 100644 --- a/src/main/scala/org/apache/spark/sql/delta/commands/MergeIntoCommand.scala +++ b/src/main/scala/org/apache/spark/sql/delta/commands/MergeIntoCommand.scala @@ -133,6 +133,11 @@ case class MergeIntoCommand( override def run( spark: SparkSession): Seq[Row] = recordDeltaOperation(targetDeltaLog, "delta.dml.merge") { targetDeltaLog.withNewTransaction { deltaTxn => + if (target.schema.size != deltaTxn.metadata.schema.size) { + throw DeltaErrors.schemaChangedSinceAnalysis( + atAnalysis = target.schema, latestSchema = deltaTxn.metadata.schema) + } + if (canMergeSchema) { updateMetadata( spark, deltaTxn, migratedSchema.getOrElse(target.schema), @@ -550,8 +555,8 @@ object MergeIntoCommand { } } - val toRow = joinedRowEncoder.toRow _ - val fromRow = outputRowEncoder.fromRow _ + val toRow = joinedRowEncoder.createSerializer() + val fromRow = outputRowEncoder.createDeserializer() rowIterator .map(toRow) .map(processRow) diff --git a/src/main/scala/org/apache/spark/sql/delta/commands/WriteIntoDelta.scala b/src/main/scala/org/apache/spark/sql/delta/commands/WriteIntoDelta.scala index 0e634063d4b..c76697562cf 100644 --- a/src/main/scala/org/apache/spark/sql/delta/commands/WriteIntoDelta.scala +++ b/src/main/scala/org/apache/spark/sql/delta/commands/WriteIntoDelta.scala @@ -63,7 +63,8 @@ case class WriteIntoDelta( override def run(sparkSession: SparkSession): Seq[Row] = { deltaLog.withNewTransaction { txn => val actions = write(txn, sparkSession) - val operation = DeltaOperations.Write(mode, Option(partitionColumns), options.replaceWhere) + val operation = DeltaOperations.Write(mode, Option(partitionColumns), + options.replaceWhere, options.userMetadata) txn.commit(actions, operation) } Seq.empty diff --git a/src/main/scala/org/apache/spark/sql/delta/files/TahoeFileIndex.scala b/src/main/scala/org/apache/spark/sql/delta/files/TahoeFileIndex.scala index 63beff2e5a5..622fb2a479e 100644 --- a/src/main/scala/org/apache/spark/sql/delta/files/TahoeFileIndex.scala +++ b/src/main/scala/org/apache/spark/sql/delta/files/TahoeFileIndex.scala @@ -18,9 +18,11 @@ package org.apache.spark.sql.delta.files import java.net.URI -import org.apache.spark.sql.delta.{DeltaLog, Snapshot} +import org.apache.spark.sql.delta.{DeltaErrors, DeltaLog, Snapshot} import org.apache.spark.sql.delta.actions.AddFile import org.apache.spark.sql.delta.actions.SingleAction.addFileEncoder +import org.apache.spark.sql.delta.schema.SchemaUtils +import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.hadoop.fs.FileStatus import org.apache.hadoop.fs.Path @@ -46,8 +48,7 @@ abstract class TahoeFileIndex( */ def matchingFiles( partitionFilters: Seq[Expression], - dataFilters: Seq[Expression], - keepStats: Boolean = false): Seq[AddFile] + dataFilters: Seq[Expression]): Seq[AddFile] override def listFiles( partitionFilters: Seq[Expression], @@ -119,6 +120,7 @@ case class TahoeLogFileIndex( override val spark: SparkSession, override val deltaLog: DeltaLog, override val path: Path, + schemaAtAnalysis: StructType, partitionFilters: Seq[Expression] = Nil, versionToUse: Option[Long] = None) extends TahoeFileIndex(spark, deltaLog, path) { @@ -128,16 +130,27 @@ case class TahoeLogFileIndex( private lazy val historicalSnapshotOpt: Option[Snapshot] = versionToUse.map(deltaLog.getSnapshotAt(_)) + private def checkSchemaOnRead: Boolean = { + spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_SCHEMA_ON_READ_CHECK_ENABLED) + } + def getSnapshot(stalenessAcceptable: Boolean): Snapshot = { - historicalSnapshotOpt.getOrElse(deltaLog.update(stalenessAcceptable)) + val snapshotToScan = historicalSnapshotOpt.getOrElse(deltaLog.update(stalenessAcceptable)) + if (checkSchemaOnRead) { + // Ensure that the schema hasn't changed in an incompatible manner since analysis time + val snapshotSchema = snapshotToScan.metadata.schema + if (!SchemaUtils.isReadCompatible(schemaAtAnalysis, snapshotSchema)) { + throw DeltaErrors.schemaChangedSinceAnalysis(schemaAtAnalysis, snapshotSchema) + } + } + snapshotToScan } override def matchingFiles( partitionFilters: Seq[Expression], - dataFilters: Seq[Expression], - keepStats: Boolean = false): Seq[AddFile] = { + dataFilters: Seq[Expression]): Seq[AddFile] = { getSnapshot(stalenessAcceptable = false).filesForScan( - projection = Nil, this.partitionFilters ++ partitionFilters ++ dataFilters, keepStats).files + projection = Nil, this.partitionFilters ++ partitionFilters ++ dataFilters).files } override def inputFiles: Array[String] = { @@ -163,6 +176,11 @@ case class TahoeLogFileIndex( .getOrElse(super.partitionSchema) } +object TahoeLogFileIndex { + def apply(spark: SparkSession, deltaLog: DeltaLog): TahoeLogFileIndex = + TahoeLogFileIndex(spark, deltaLog, deltaLog.dataPath, deltaLog.snapshot.metadata.schema) +} + /** * A [[TahoeFileIndex]] that generates the list of files from a given list of files * that are within a version range of DeltaLog. @@ -180,8 +198,7 @@ class TahoeBatchFileIndex( override def matchingFiles( partitionFilters: Seq[Expression], - dataFilters: Seq[Expression], - keepStats: Boolean = false): Seq[AddFile] = { + dataFilters: Seq[Expression]): Seq[AddFile] = { DeltaLog.filterFileList( snapshot.metadata.partitionSchema, spark.createDataset(addFiles)(addFileEncoder).toDF(), partitionFilters) diff --git a/src/main/scala/org/apache/spark/sql/delta/sources/DeltaDataSource.scala b/src/main/scala/org/apache/spark/sql/delta/sources/DeltaDataSource.scala index fc66c762392..81f2b0fd8af 100644 --- a/src/main/scala/org/apache/spark/sql/delta/sources/DeltaDataSource.scala +++ b/src/main/scala/org/apache/spark/sql/delta/sources/DeltaDataSource.scala @@ -165,21 +165,10 @@ class DeltaDataSource DeltaOptions.verifyOptions(CaseInsensitiveMap(parameters)) val timeTravelByParams = DeltaDataSource.getTimeTravelVersion(parameters) - // TODO(burak): Move all this logic into DeltaTableV2 when Spark 3.0 is ready - // Handle time travel - val (path, partitionFilters, timeTravelByPath) = - DeltaDataSource.parsePathIdentifier(sqlContext.sparkSession, maybePath) - - if (timeTravelByParams.isDefined && timeTravelByPath.isDefined) { - throw DeltaErrors.multipleTimeTravelSyntaxUsed - } - - val deltaLog = DeltaLog.forTable(sqlContext.sparkSession, path) - - val partitionPredicates = - DeltaDataSource.verifyAndCreatePartitionFilters(maybePath, deltaLog, partitionFilters) - - deltaLog.createRelation(partitionPredicates, timeTravelByParams.orElse(timeTravelByPath)) + DeltaTableV2( + sqlContext.sparkSession, + new Path(maybePath), + timeTravelOpt = timeTravelByParams).toBaseRelation } override def shortName(): String = { @@ -305,10 +294,9 @@ object DeltaDataSource extends DatabricksLogging { */ def verifyAndCreatePartitionFilters( userPath: String, - deltaLog: DeltaLog, + snapshot: Snapshot, partitionFilters: Seq[(String, String)]): Seq[Expression] = { if (partitionFilters.nonEmpty) { - val snapshot = deltaLog.update() val metadata = snapshot.metadata val badColumns = partitionFilters.map(_._1).filterNot(metadata.partitionColumns.contains) diff --git a/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala b/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala index 426db75f056..efa5809acd2 100644 --- a/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala +++ b/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala @@ -45,6 +45,12 @@ object DeltaSQLConf { .booleanConf .createWithDefault(true) + val DELTA_USER_METADATA = + buildConf("commitInfo.userMetadata") + .doc("Arbitrary user-defined metadata to include in CommitInfo. Requires commitInfo.enabled.") + .stringConf + .createOptional + val DELTA_SNAPSHOT_PARTITIONS = buildConf("snapshotPartitions") .internal() @@ -61,6 +67,31 @@ object DeltaSQLConf { .booleanConf .createWithDefault(true) + val DELTA_STATE_RECONSTRUCTION_VALIDATION_ENABLED = + buildConf("stateReconstructionValidation.enabled") + .internal() + .doc("Whether to perform validation checks on the reconstructed state.") + .booleanConf + .createWithDefault(true) + + val DELTA_COMMIT_VALIDATION_ENABLED = + buildConf("commitValidation.enabled") + .internal() + .doc("Whether to perform validation checks before commit or not.") + .booleanConf + .createWithDefault(true) + + val DELTA_SCHEMA_ON_READ_CHECK_ENABLED = + buildConf("checkLatestSchemaOnRead") + .doc("In Delta, we always try to give users the latest version of their data without " + + "having to call REFRESH TABLE or redefine their DataFrames when used in the context of " + + "streaming. There is a possibility that the schema of the latest version of the table " + + "may be incompatible with the schema at the time of DataFrame creation. This flag " + + "enables a check that ensures that users won't read corrupt data if the source schema " + + "changes in an incompatible way.") + .booleanConf + .createWithDefault(true) + val DELTA_COLLECT_STATS = buildConf("stats.collect") .internal() diff --git a/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSink.scala b/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSink.scala index 6688ffa55a6..9b715a5318e 100644 --- a/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSink.scala +++ b/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSink.scala @@ -98,7 +98,7 @@ class DeltaSink( } val newFiles = txn.writeFiles(data, Some(options)) val setTxn = SetTransaction(queryId, batchId, Some(deltaLog.clock.getTimeMillis())) :: Nil - val info = DeltaOperations.StreamingUpdate(outputMode, queryId, batchId) + val info = DeltaOperations.StreamingUpdate(outputMode, queryId, batchId, options.userMetadata) metrics("numRemovedFiles").set(deletedFiles.size) metrics("numAddedFiles").set(newFiles.size) txn.registerSQLMetrics(sqlContext.sparkSession, metrics) diff --git a/src/main/scala/org/apache/spark/sql/delta/util/AnalysisHelper.scala b/src/main/scala/org/apache/spark/sql/delta/util/AnalysisHelper.scala index 1043f4193b0..f28fb4b83a5 100644 --- a/src/main/scala/org/apache/spark/sql/delta/util/AnalysisHelper.scala +++ b/src/main/scala/org/apache/spark/sql/delta/util/AnalysisHelper.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.delta.util import org.apache.spark.sql.delta.DeltaErrors -import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.{Dataset, Row, SparkSession} import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan @@ -40,6 +40,28 @@ trait AnalysisHelper { s"Could not resolve expression $expr", plan = Option(planContainingExpr)) } } + + protected def toDataset(sparkSession: SparkSession, logicalPlan: LogicalPlan): Dataset[Row] = { + Dataset.ofRows(sparkSession, logicalPlan) + } + + protected def improveUnsupportedOpError(f: => Unit): Unit = { + val possibleErrorMsgs = Seq( + "is only supported with v2 table", // full error: DELETE is only supported with v2 tables + "is not supported temporarily", // full error: UPDATE TABLE is not supported temporarily + "Table does not support read", + "Table implementation does not support writes" + ).map(_.toLowerCase()) + + def isExtensionOrCatalogError(error: Exception): Boolean = { + possibleErrorMsgs.exists(m => error.getMessage().toLowerCase().contains(m)) + } + + try { f } catch { + case e: Exception if isExtensionOrCatalogError(e) => + throw DeltaErrors.configureSparkSessionWithExtensionAndCatalog(e) + } + } } object AnalysisHelper { diff --git a/src/test/java/org/apache/spark/sql/delta/DeleteJavaSuite.java b/src/test/java/org/apache/spark/sql/delta/DeleteJavaSuite.java index bcab6a2d6b4..03dfc4a813d 100644 --- a/src/test/java/org/apache/spark/sql/delta/DeleteJavaSuite.java +++ b/src/test/java/org/apache/spark/sql/delta/DeleteJavaSuite.java @@ -14,7 +14,7 @@ * limitations under the License. */ -package test.org.apache.spark.sql.delta; +package org.apache.spark.sql.delta; import java.util.ArrayList; import java.util.Arrays; @@ -28,17 +28,16 @@ import org.junit.Test; import org.apache.spark.sql.*; -import org.apache.spark.sql.test.TestSparkSession; import org.apache.spark.util.Utils; -public class DeleteJavaSuite { +public class DeleteJavaSuite implements DeltaSQLCommandJavaTest { - private transient TestSparkSession spark; + private transient SparkSession spark; private transient String tempPath; @Before public void setUp() { - spark = new TestSparkSession(); + spark = buildSparkSession(); tempPath = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "spark").toString(); } diff --git a/src/test/java/org/apache/spark/sql/delta/DeltaSQLCommandJavaTest.java b/src/test/java/org/apache/spark/sql/delta/DeltaSQLCommandJavaTest.java new file mode 100644 index 00000000000..be58e4ebc67 --- /dev/null +++ b/src/test/java/org/apache/spark/sql/delta/DeltaSQLCommandJavaTest.java @@ -0,0 +1,30 @@ +/* + * Copyright (2020) 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.SparkSession; + +interface DeltaSQLCommandJavaTest { + default SparkSession buildSparkSession() { + // Set the configurations as DeltaSQLCommandTest + return SparkSession.builder() + .appName("JavaDeltaSparkSessionExtensionSuiteUsingSQLConf") + .master("local[2]") + .config("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension") + .getOrCreate(); + } +} diff --git a/src/test/java/org/apache/spark/sql/delta/MergeIntoJavaSuite.java b/src/test/java/org/apache/spark/sql/delta/MergeIntoJavaSuite.java index 658db4f2072..589c6b5535a 100644 --- a/src/test/java/org/apache/spark/sql/delta/MergeIntoJavaSuite.java +++ b/src/test/java/org/apache/spark/sql/delta/MergeIntoJavaSuite.java @@ -14,7 +14,7 @@ * limitations under the License. */ -package test.org.apache.spark.sql.delta; +package org.apache.spark.sql.delta; import java.io.Serializable; import java.util.Arrays; diff --git a/src/test/java/org/apache/spark/sql/delta/UpdateJavaSuite.java b/src/test/java/org/apache/spark/sql/delta/UpdateJavaSuite.java index 1f6fae9b51d..61e85e7e18f 100644 --- a/src/test/java/org/apache/spark/sql/delta/UpdateJavaSuite.java +++ b/src/test/java/org/apache/spark/sql/delta/UpdateJavaSuite.java @@ -14,7 +14,7 @@ * limitations under the License. */ -package test.org.apache.spark.sql.delta; +package org.apache.spark.sql.delta; import java.util.*; @@ -24,16 +24,15 @@ import org.junit.*; import org.apache.spark.sql.*; -import org.apache.spark.sql.test.TestSparkSession; import org.apache.spark.util.Utils; -public class UpdateJavaSuite { - private transient TestSparkSession spark; +public class UpdateJavaSuite implements DeltaSQLCommandJavaTest { + private transient SparkSession spark; private transient String tempPath; @Before public void setUp() { - spark = new TestSparkSession(); + spark = buildSparkSession(); tempPath = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "spark").toString(); } diff --git a/src/test/scala/io/delta/sql/DeltaExtensionAndCatalogSuite.scala b/src/test/scala/io/delta/sql/DeltaExtensionAndCatalogSuite.scala new file mode 100644 index 00000000000..f9669497d77 --- /dev/null +++ b/src/test/scala/io/delta/sql/DeltaExtensionAndCatalogSuite.scala @@ -0,0 +1,194 @@ +/* + * Copyright (2020) 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 io.delta.sql + +import java.nio.file.Files + +import org.apache.spark.sql.delta.catalog.DeltaCatalog +import io.delta.tables.DeltaTable +import org.apache.commons.io.FileUtils + +import org.apache.spark.SparkFunSuite +import org.apache.spark.network.util.JavaUtils +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf + +class DeltaExtensionAndCatalogSuite extends SparkFunSuite { + + private def createTempDir(): String = { + val dir = Files.createTempDirectory("DeltaSparkSessionExtensionSuite").toFile + FileUtils.forceDeleteOnExit(dir) + dir.getCanonicalPath + } + + private def verifyDeltaSQLParserIsActivated(spark: SparkSession): Unit = { + val input = Files.createTempDirectory("DeltaSparkSessionExtensionSuite").toFile + try { + spark.range(1, 10).write.format("delta").save(input.getCanonicalPath) + spark.sql(s"vacuum delta.`${input.getCanonicalPath}`") + } finally { + JavaUtils.deleteRecursively(input) + } + } + + test("activate Delta SQL parser using SQL conf") { + val spark = SparkSession.builder() + .appName("DeltaSparkSessionExtensionSuiteUsingSQLConf") + .master("local[2]") + .config("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension") + .getOrCreate() + try { + verifyDeltaSQLParserIsActivated(spark) + } finally { + spark.close() + } + } + + test("activate Delta SQL parser using withExtensions") { + val spark = SparkSession.builder() + .appName("DeltaSparkSessionExtensionSuiteUsingWithExtensions") + .master("local[2]") + .withExtensions(new io.delta.sql.DeltaSparkSessionExtension) + .getOrCreate() + try { + verifyDeltaSQLParserIsActivated(spark) + } finally { + spark.close() + } + } + + private def withSparkSession(configs: (String, String)*)(f: SparkSession => Unit): Unit = { + var builder = SparkSession.builder() + .appName("DeltaSparkSessionExtensionSuite") + .master("local[2]") + .config("spark.sql.warehouse.dir", createTempDir()) + + configs.foreach { c => builder = builder.config(c._1, c._2) } + val spark = builder.getOrCreate() + try { + f(spark) + } finally { + spark.close() + } + } + + private def checkErrorMessage(f: => Unit): Unit = { + val e = intercept[AnalysisException](f) + val expectedStrs = Seq( + "Delta operation requires the SparkSession to be configured", + "spark.sql.extensions", + s"${classOf[DeltaSparkSessionExtension].getName}", + SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION.key, + s"${classOf[DeltaCatalog].getName}" + ) + expectedStrs.foreach { m => assert(e.getMessage().contains(m), "full exception: " + e) } + } + + test("behavior without Delta extension - scala API on path-based tables") { + withSparkSession( + SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION.key -> classOf[DeltaCatalog].getName + ) { spark => + import spark.implicits._ + + val tablePath = createTempDir() + spark.range(1, 10).toDF("key") + .withColumn("value", col("key")).write.format("delta").save(tablePath) + + val deltaTable = DeltaTable.forPath(spark, tablePath) + + checkErrorMessage { + deltaTable.delete("key = 0") + } + + checkErrorMessage { + deltaTable.updateExpr("key = 0", Map("key" -> "0")) + } + + // No error expected + deltaTable.merge(Seq((0, 0)).toDF("key", "value").as("s"), "s.key = t.key") + .whenMatched().updateAll() + .whenNotMatched().insertAll() + + deltaTable.history() + + deltaTable.vacuum() + + deltaTable.generate("symlink_format_manifest") + } + } + + test("behavior without Delta extension - scala API on name-based tables") { + withSparkSession( + SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION.key -> classOf[DeltaCatalog].getName + ) { spark => + import spark.implicits._ + + spark.range(1, 10).toDF("key") + .withColumn("value", col("key")).write.format("delta").saveAsTable("tbl") + + val deltaTable = DeltaTable.forName(spark, "tbl") + + checkErrorMessage { + deltaTable.delete("key = 0") + } + + checkErrorMessage { + deltaTable.updateExpr("key = 0", Map("key" -> "0")) + } + + deltaTable.merge(Seq((0, 0)).toDF("key", "value").as("s"), "s.key = t.key") + .whenMatched().updateAll() + .whenNotMatched().insertAll() + + deltaTable.history() + + deltaTable.vacuum() + + deltaTable.generate("symlink_format_manifest") + } + } + + test("behavior without DeltaCatalog configuration - scala API on path-based tables") { + withSparkSession( + "spark.sql.extensions" -> classOf[DeltaSparkSessionExtension].getName + ) { spark => + import spark.implicits._ + val tablePath = createTempDir() + + spark.range(1, 10).toDF("key").withColumn("value", col("key")) + .write.format("delta").save(tablePath) + + val deltaTable = DeltaTable.forPath(spark, tablePath).as("t") + + // No errors expected + deltaTable.delete("key = 0") + + deltaTable.updateExpr("key = 0", Map("key" -> "0")) + + deltaTable.merge(Seq((0, 0)).toDF("key", "value").as("s"), "s.key = t.key") + .whenMatched().updateAll() + .whenNotMatched().insertAll() + + deltaTable.history() + + deltaTable.vacuum() + + deltaTable.generate("symlink_format_manifest") + } + } +} diff --git a/src/test/scala/io/delta/sql/DeltaSparkSessionExtensionSuite.scala b/src/test/scala/io/delta/sql/DeltaSparkSessionExtensionSuite.scala deleted file mode 100644 index 5204a2166e5..00000000000 --- a/src/test/scala/io/delta/sql/DeltaSparkSessionExtensionSuite.scala +++ /dev/null @@ -1,62 +0,0 @@ -/* - * Copyright (2020) 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 io.delta.sql - -import java.nio.file.Files - -import org.apache.spark.SparkFunSuite -import org.apache.spark.network.util.JavaUtils -import org.apache.spark.sql.SparkSession - -class DeltaSparkSessionExtensionSuite extends SparkFunSuite { - - private def verifyDeltaSQLParserIsActivated(spark: SparkSession): Unit = { - val input = Files.createTempDirectory("DeltaSparkSessionExtensionSuite").toFile - try { - spark.range(1, 10).write.format("delta").save(input.getCanonicalPath) - spark.sql(s"vacuum delta.`${input.getCanonicalPath}`") - } finally { - JavaUtils.deleteRecursively(input) - } - } - - test("activate Delta SQL parser using SQL conf") { - val spark = SparkSession.builder() - .appName("DeltaSparkSessionExtensionSuiteUsingSQLConf") - .master("local[2]") - .config("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension") - .getOrCreate() - try { - verifyDeltaSQLParserIsActivated(spark) - } finally { - spark.close() - } - } - - test("activate Delta SQL parser using withExtensions") { - val spark = SparkSession.builder() - .appName("DeltaSparkSessionExtensionSuiteUsingWithExtensions") - .master("local[2]") - .withExtensions(new io.delta.sql.DeltaSparkSessionExtension) - .getOrCreate() - try { - verifyDeltaSQLParserIsActivated(spark) - } finally { - spark.close() - } - } -} diff --git a/src/test/scala/io/delta/tables/DeltaTableSuite.scala b/src/test/scala/io/delta/tables/DeltaTableSuite.scala index 83700ee42da..8b65959da12 100644 --- a/src/test/scala/io/delta/tables/DeltaTableSuite.scala +++ b/src/test/scala/io/delta/tables/DeltaTableSuite.scala @@ -121,14 +121,6 @@ class DeltaTableSuite extends QueryTest } } - test("toDF regenerated each time") { - withTempDir { dir => - testData.write.format("delta").save(dir.getAbsolutePath) - val table = DeltaTable.forPath(dir.getAbsolutePath) - assert(table.toDF != table.toDF) - } - } - def testError(expectedMsg: String)(thunk: => Unit): Unit = { val e = intercept[AnalysisException] { thunk } assert(e.getMessage.toLowerCase(Locale.ROOT).contains(expectedMsg.toLowerCase(Locale.ROOT))) diff --git a/src/test/scala/org/apache/spark/sql/delta/ConvertToDeltaSuiteBase.scala b/src/test/scala/org/apache/spark/sql/delta/ConvertToDeltaSuiteBase.scala index 5d8f236339e..0058e62bb1a 100644 --- a/src/test/scala/org/apache/spark/sql/delta/ConvertToDeltaSuiteBase.scala +++ b/src/test/scala/org/apache/spark/sql/delta/ConvertToDeltaSuiteBase.scala @@ -933,4 +933,26 @@ trait ConvertToDeltaHiveTableTests extends ConvertToDeltaTestUtils with SQLTestU simpleDF.union(simpleDF).filter("id % 2 == 1").select("id")) } } + + test("external tables use correct path scheme") { + withTempDir { dir => + withTable("externalTable") { + withSQLConf(("fs.s3.impl", classOf[S3LikeLocalFileSystem].getCanonicalName)) { + sql(s"CREATE TABLE externalTable USING parquet LOCATION 's3://$dir' AS SELECT 1") + + // Ideally we would test a successful conversion with a remote filesystem, but there's + // no good way to set one up in unit tests. So instead we delete the data, and let the + // FileNotFoundException tell us which scheme it was using to look for it. + Utils.deleteRecursively(dir) + + val ex = intercept[FileNotFoundException] { + convertToDelta("default.externalTable", None) + } + + // If the path incorrectly used the default scheme, this would be file: at the end. + assert(ex.getMessage.contains(s"No file found in the directory: s3:$dir")) + } + } + } + } } diff --git a/src/test/scala/org/apache/spark/sql/delta/DeleteSQLSuite.scala b/src/test/scala/org/apache/spark/sql/delta/DeleteSQLSuite.scala new file mode 100644 index 00000000000..936e01612d4 --- /dev/null +++ b/src/test/scala/org/apache/spark/sql/delta/DeleteSQLSuite.scala @@ -0,0 +1,42 @@ +/* + * Copyright (2020) 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.test.DeltaSQLCommandTest + +import org.apache.spark.sql.Row + +class DeleteSQLSuite extends DeleteSuiteBase with DeltaSQLCommandTest { + + import testImplicits._ + + override protected def executeDelete(target: String, where: String = null): Unit = { + val whereClause = Option(where).map(c => s"WHERE $c").getOrElse("") + sql(s"DELETE FROM $target $whereClause") + } + + // For EXPLAIN, which is not supported in OSS + test("explain") { + append(Seq((2, 2)).toDF("key", "value")) + val df = sql(s"EXPLAIN DELETE FROM delta.`$tempPath` WHERE key = 2") + val outputs = df.collect().map(_.mkString).mkString + assert(outputs.contains("Delta")) + assert(!outputs.contains("index") && !outputs.contains("ActionLog")) + // no change should be made by explain + checkAnswer(readDeltaTable(tempPath), Row(2, 2)) + } +} diff --git a/src/test/scala/org/apache/spark/sql/delta/DeleteScalaSuite.scala b/src/test/scala/org/apache/spark/sql/delta/DeleteScalaSuite.scala index 73dfd38ef9e..9285560f2cf 100644 --- a/src/test/scala/org/apache/spark/sql/delta/DeleteScalaSuite.scala +++ b/src/test/scala/org/apache/spark/sql/delta/DeleteScalaSuite.scala @@ -16,11 +16,12 @@ package org.apache.spark.sql.delta +import org.apache.spark.sql.delta.test.DeltaSQLCommandTest import io.delta.tables.{DeltaTable, DeltaTableTestUtils} import org.apache.spark.sql.{functions, Row} -class DeleteScalaSuite extends DeleteSuiteBase { +class DeleteScalaSuite extends DeleteSuiteBase with DeltaSQLCommandTest { import testImplicits._ diff --git a/src/test/scala/org/apache/spark/sql/delta/DeleteSuiteBase.scala b/src/test/scala/org/apache/spark/sql/delta/DeleteSuiteBase.scala index 02a930280ea..7a0d80a416b 100644 --- a/src/test/scala/org/apache/spark/sql/delta/DeleteSuiteBase.scala +++ b/src/test/scala/org/apache/spark/sql/delta/DeleteSuiteBase.scala @@ -41,7 +41,6 @@ abstract class DeleteSuiteBase extends QueryTest spark.read.format("delta").load(path) } - override def beforeEach() { super.beforeEach() tempDir = Utils.createTempDir() @@ -85,7 +84,7 @@ abstract class DeleteSuiteBase extends QueryTest } Seq(true, false).foreach { isPartitioned => - test(s"basic case - delete from a Delta table - Partition=$isPartitioned") { + test(s"basic case - delete from a Delta table by path - Partition=$isPartitioned") { withTable("deltaTable") { val partitions = if (isPartitioned) "key" :: Nil else Nil val input = Seq((2, 2), (1, 4), (1, 1), (0, 3)).toDF("key", "value") @@ -102,6 +101,37 @@ abstract class DeleteSuiteBase extends QueryTest } } + Seq(true, false).foreach { isPartitioned => + test(s"basic case - delete from a Delta table by name - Partition=$isPartitioned") { + withTable("delta_table") { + val partitionByClause = if (isPartitioned) "PARTITIONED BY (key)" else "" + sql( + s""" + |CREATE TABLE delta_table(key INT, value INT) + |USING delta + |OPTIONS('path'='$tempPath') + |$partitionByClause + """.stripMargin) + + val input = Seq((2, 2), (1, 4), (1, 1), (0, 3)).toDF("key", "value") + append(input) + + checkDelete(Some("value = 4 and key = 3"), + Row(2, 2) :: Row(1, 4) :: Row(1, 1) :: Row(0, 3) :: Nil, + Some("delta_table")) + checkDelete(Some("value = 4 and key = 1"), + Row(2, 2) :: Row(1, 1) :: Row(0, 3) :: Nil, + Some("delta_table")) + checkDelete(Some("value = 2 or key = 1"), + Row(0, 3) :: Nil, + Some("delta_table")) + checkDelete(Some("key = 0 or value = 99"), + Nil, + Some("delta_table")) + } + } + } + Seq(true, false).foreach { isPartitioned => test(s"basic key columns - Partition=$isPartitioned") { val input = Seq((2, 2), (1, 4), (1, 1), (0, 3)).toDF("key", "value") @@ -189,7 +219,27 @@ abstract class DeleteSuiteBase extends QueryTest assert(e.contains("nondeterministic expressions are only allowed in")) } - test("delete cached table") { + test("Negative case - DELETE the child directory") { + append(Seq((2, 2), (3, 2)).toDF("key", "value"), partitionBy = "key" :: Nil) + val e = intercept[AnalysisException] { + executeDelete(target = s"delta.`$tempPath/key=2`", where = "value = 2") + }.getMessage + assert(e.contains("Expect a full scan of Delta sources, but found a partial scan")) + } + + test("delete cached table by name") { + withTable("cached_delta_table") { + Seq((2, 2), (1, 4)).toDF("key", "value") + .write.format("delta").saveAsTable("cached_delta_table") + + spark.table("cached_delta_table").cache() + spark.table("cached_delta_table").collect() + executeDelete(target = "cached_delta_table", where = "key = 2") + checkAnswer(spark.table("cached_delta_table"), Row(1, 4) :: Nil) + } + } + + test("delete cached table by path") { Seq((2, 2), (1, 4)).toDF("key", "value") .write.mode("overwrite").format("delta").save(tempPath) spark.read.format("delta").load(tempPath).cache() diff --git a/src/test/scala/org/apache/spark/sql/delta/DeltaAlterTableTests.scala b/src/test/scala/org/apache/spark/sql/delta/DeltaAlterTableTests.scala index b1b19ca19ae..26a5944f41d 100644 --- a/src/test/scala/org/apache/spark/sql/delta/DeltaAlterTableTests.scala +++ b/src/test/scala/org/apache/spark/sql/delta/DeltaAlterTableTests.scala @@ -1359,14 +1359,14 @@ trait DeltaAlterTableByNameTests extends DeltaAlterTableTests { */ trait DeltaAlterTableByPathTests extends DeltaAlterTableTestBase { override protected def createTable(schema: String, tblProperties: Map[String, String]): String = { - val tmpDir = Utils.createTempDir().getCanonicalPath - val deltaLog = getDeltaLog(tmpDir) - val txn = deltaLog.startTransaction() - val metadata = Metadata( - schemaString = StructType.fromDDL(schema).json, - configuration = tblProperties) - txn.commit(metadata :: Nil, DeltaOperations.ManualUpdate) - s"delta.`$tmpDir`" + val tmpDir = Utils.createTempDir().getCanonicalPath + val deltaLog = getDeltaLog(tmpDir) + val txn = deltaLog.startTransaction() + val metadata = Metadata( + schemaString = StructType.fromDDL(schema).json, + configuration = tblProperties) + txn.commit(metadata :: Nil, DeltaOperations.ManualUpdate) + s"delta.`$tmpDir`" } override protected def createTable(df: DataFrame, partitionedBy: Seq[String]): String = { diff --git a/src/test/scala/org/apache/spark/sql/delta/DeltaDDLSuite.scala b/src/test/scala/org/apache/spark/sql/delta/DeltaDDLSuite.scala index d2f4103761d..e17c8c8eb83 100644 --- a/src/test/scala/org/apache/spark/sql/delta/DeltaDDLSuite.scala +++ b/src/test/scala/org/apache/spark/sql/delta/DeltaDDLSuite.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.delta.test.DeltaSQLCommandTest import org.apache.spark.{SparkEnv, SparkException} import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.NoSuchPartitionException import org.apache.spark.sql.catalyst.catalog.CatalogUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} @@ -52,7 +53,7 @@ abstract class DeltaDDLTestBase extends QueryTest with SQLTestUtils { protected def verifyNullabilityFailure(exception: AnalysisException): Unit protected def getDeltaLog(tableLocation: String): DeltaLog = { - DeltaLog.forTable(spark, tableLocation) + DeltaLog.forTable(spark, tableLocation) } testQuietly("create table with NOT NULL - check violation through file writing") { @@ -438,6 +439,8 @@ abstract class DeltaDDLTestBase extends QueryTest with SQLTestUtils { verifyDescribeTable("delta_test") verifyDescribeTable(s"delta.`$path`") + + assert(sql("DESCRIBE EXTENDED delta_test").collect().length > 0) } } } diff --git a/src/test/scala/org/apache/spark/sql/delta/DeltaGenerateSymlinkManifestSuite.scala b/src/test/scala/org/apache/spark/sql/delta/DeltaGenerateSymlinkManifestSuite.scala index f25078700a5..a8e19b850b5 100644 --- a/src/test/scala/org/apache/spark/sql/delta/DeltaGenerateSymlinkManifestSuite.scala +++ b/src/test/scala/org/apache/spark/sql/delta/DeltaGenerateSymlinkManifestSuite.scala @@ -29,6 +29,7 @@ import org.apache.hadoop.fs.permission.FsPermission import org.apache.hadoop.util.Progressable import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSparkSession @@ -41,6 +42,119 @@ trait DeltaGenerateSymlinkManifestSuiteBase extends QueryTest import testImplicits._ + test("basic case: SQL command - path-based table") { + withTempDir { tablePath => + tablePath.delete() + + spark.createDataset(spark.sparkContext.parallelize(1 to 100, 7)) + .write.format("delta").mode("overwrite").save(tablePath.toString) + + assertManifest(tablePath, expectSameFiles = false, expectedNumFiles = 0) + + // Create a Delta table and call the scala api for generating manifest files + spark.sql(s"GENERATE symlink_ForMat_Manifest FOR TABLE delta.`${tablePath.getAbsolutePath}`") + assertManifest(tablePath, expectSameFiles = true, expectedNumFiles = 7) + } + } + + test("basic case: SQL command - name-based table") { + withTable("deltaTable") { + spark.createDataset(spark.sparkContext.parallelize(1 to 100, 7)) + .write.format("delta").saveAsTable("deltaTable") + + val tableId = TableIdentifier("deltaTable") + val tablePath = new File(spark.sessionState.catalog.getTableMetadata(tableId).location) + assertManifest(tablePath, expectSameFiles = false, expectedNumFiles = 0) + + spark.sql(s"GENERATE symlink_ForMat_Manifest FOR TABLE deltaTable") + assertManifest(tablePath, expectSameFiles = true, expectedNumFiles = 7) + } + } + + test("basic case: SQL command - throw error on bad tables") { + var e: Exception = intercept[AnalysisException] { + spark.sql("GENERATE symlink_format_manifest FOR TABLE nonExistentTable") + } + assert(e.getMessage.contains("not found")) + + withTable("nonDeltaTable") { + spark.range(2).write.format("parquet").saveAsTable("nonDeltaTable") + e = intercept[AnalysisException] { + spark.sql("GENERATE symlink_format_manifest FOR TABLE nonDeltaTable") + } + assert(e.getMessage.contains("only supported for Delta")) + } + } + + test("basic case: SQL command - throw error on non delta table paths") { + withTempDir { dir => + var e = intercept[AnalysisException] { + spark.sql(s"GENERATE symlink_format_manifest FOR TABLE delta.`$dir`") + } + + assert(e.getMessage.contains("not found") || + e.getMessage.contains("only supported for Delta")) + + spark.range(2).write.format("parquet").mode("overwrite").save(dir.toString) + + e = intercept[AnalysisException] { + spark.sql(s"GENERATE symlink_format_manifest FOR TABLE delta.`$dir`") + } + assert(e.getMessage.contains("table not found") || + e.getMessage.contains("only supported for Delta")) + + e = intercept[AnalysisException] { + spark.sql(s"GENERATE symlink_format_manifest FOR TABLE parquet.`$dir`") + } + assert(e.getMessage.contains("not found")) + } + } + + test("basic case: SQL command - throw error on unsupported mode") { + withTempDir { tablePath => + spark.range(2).write.format("delta").save(tablePath.getAbsolutePath) + val e = intercept[IllegalArgumentException] { + spark.sql(s"GENERATE xyz FOR TABLE delta.`${tablePath.getAbsolutePath}`") + } + assert(e.toString.contains("not supported")) + DeltaGenerateCommand.modeNameToGenerationFunc.keys.foreach { modeName => + assert(e.toString.contains(modeName)) + } + } + } + + test("basic case: Scala API - path-based table") { + withTempDir { tablePath => + tablePath.delete() + + spark.createDataset(spark.sparkContext.parallelize(1 to 100, 7)) + .write.format("delta").mode("overwrite").save(tablePath.toString) + + assertManifest(tablePath, expectSameFiles = false, expectedNumFiles = 0) + + // Create a Delta table and call the scala api for generating manifest files + val deltaTable = io.delta.tables.DeltaTable.forPath(tablePath.getAbsolutePath) + deltaTable.generate("symlink_format_manifest") + assertManifest(tablePath, expectSameFiles = true, expectedNumFiles = 7) + } + } + + test("basic case: Scala API - name-based table") { + withTable("deltaTable") { + spark.createDataset(spark.sparkContext.parallelize(1 to 100, 7)) + .write.format("delta").saveAsTable("deltaTable") + + val tableId = TableIdentifier("deltaTable") + val tablePath = new File(spark.sessionState.catalog.getTableMetadata(tableId).location) + assertManifest(tablePath, expectSameFiles = false, expectedNumFiles = 0) + + val deltaTable = io.delta.tables.DeltaTable.forName("deltaTable") + deltaTable.generate("symlink_format_manifest") + assertManifest(tablePath, expectSameFiles = true, expectedNumFiles = 7) + } + } + + test ("full manifest: non-partitioned table") { withTempDir { tablePath => tablePath.delete() @@ -137,27 +251,6 @@ trait DeltaGenerateSymlinkManifestSuiteBase extends QueryTest } } - test("full manifest: throw error on non delta table paths") { - withTempDir { dir => - var e = intercept[AnalysisException] { - spark.sql(s"GENERATE symlink_format_manifest FOR TABLE delta.`$dir`") - } - assert(e.getMessage.contains("not found")) - - spark.range(2).write.format("parquet").mode("overwrite").save(dir.toString) - - e = intercept[AnalysisException] { - spark.sql(s"GENERATE symlink_format_manifest FOR TABLE delta.`$dir`") - } - assert(e.getMessage.contains("table not found")) - - e = intercept[AnalysisException] { - spark.sql(s"GENERATE symlink_format_manifest FOR TABLE parquet.`$dir`") - } - assert(e.getMessage.contains("not found")) - } - } - test("incremental manifest: table property controls post commit manifest generation") { withTempDir { tablePath => tablePath.delete() @@ -375,93 +468,13 @@ trait DeltaGenerateSymlinkManifestSuiteBase extends QueryTest } } - test("full manifest: scala api") { - withTempDir { tablePath => - tablePath.delete() - - def write(parallelism: Int): Unit = { - spark.createDataset(spark.sparkContext.parallelize(1 to 100, parallelism)) - .write.format("delta").mode("overwrite").save(tablePath.toString) - } - - write(7) - assertManifest(tablePath, expectSameFiles = false, expectedNumFiles = 0) - - // Create a Delta table and call the scala api for generating manifest files - val deltaTable = io.delta.tables.DeltaTable.forPath(tablePath.getAbsolutePath) - deltaTable.generate("symlink_format_manifest") - assertManifest(tablePath, expectSameFiles = true, expectedNumFiles = 7) - } - } - - test("full manifest: SQL command") { - withTable("deltaTable") { - withTempDir { tablePath => - tablePath.delete() - - def write(parallelism: Int, partitions1: Int, partitions2: Int): Unit = { - spark.createDataset(spark.sparkContext.parallelize(1 to 100, parallelism)).toDF("value") - .withColumn("part1", $"value" % partitions1) - .withColumn("part2", $"value" % partitions2) - .write.format("delta").partitionBy("part1", "part2") - .mode("overwrite") - .option("path", tablePath.toString) - .save(tablePath.getAbsolutePath) - } - - val path = tablePath.getAbsolutePath - write(10, 10, 10) - assertManifest(tablePath, expectSameFiles = false, expectedNumFiles = 0) - spark.sql(s"""GENERATE symlink_ForMat_Manifest FOR TABLE delta.`$path`""") - - // 10 files each in ../part1=X/part2=X/ for X = 0 to 9 - assertManifest(tablePath, expectSameFiles = true, expectedNumFiles = 100) - - // Reduce # partitions on both dimensions - write(1, 1, 1) - assertManifest(tablePath, expectSameFiles = false, expectedNumFiles = 100) - spark.sql(s"""GENERATE SYMLINK_FORMAT_MANIFEST FOR TABLE delta.`$path`""") - assertManifest(tablePath, expectSameFiles = true, expectedNumFiles = 1) - - // Increase # partitions on both dimensions - write(5, 5, 5) - assertManifest(tablePath, expectSameFiles = false, expectedNumFiles = 1) - spark.sql(s"GENERATE symlink_ForMat_Manifest FOR TABLE delta.`$path`") - assertManifest(tablePath, expectSameFiles = true, expectedNumFiles = 25) - - // Increase # partitions on only one dimension - write(5, 10, 5) - assertManifest(tablePath, expectSameFiles = false, expectedNumFiles = 25) - spark.sql(s"GENERATE symlink_format_manifest FOR TABLE delta.`$path`") - assertManifest(tablePath, expectSameFiles = true, expectedNumFiles = 50) - - // Remove all data - spark.emptyDataset[Int].toDF("value") - .withColumn("part1", $"value" % 10) - .withColumn("part2", $"value" % 10) - .write.format("delta").mode("overwrite").save(tablePath.toString) - assertManifest(tablePath, expectSameFiles = false, expectedNumFiles = 50) - spark.sql(s"GENERATE symlink_ForMat_Manifest FOR TABLE delta.`$path`") - assertManifest(tablePath, expectSameFiles = true, expectedNumFiles = 0) - assert(spark.read.format("delta") - .load(tablePath.getAbsolutePath).count() == 0) - } - } - } - - test("full manifest: SQL command - throw error on unsupported mode") { - withTempDir { tablePath => - spark.range(2).write.format("delta").save(tablePath.getAbsolutePath) - val e = intercept[IllegalArgumentException] { - spark.sql(s"GENERATE xyz FOR TABLE delta.`${tablePath.getAbsolutePath}`") - } - assert(e.toString.contains("not supported")) - DeltaGenerateCommand.modeNameToGenerationFunc.keys.foreach { modeName => - assert(e.toString.contains(modeName)) - } - } - } - + /** + * Assert that the manifest files in the table meet the expectations. + * @param tablePath Path of the Delta table + * @param expectSameFiles Expect that the manifest files contain the same data files + * as the latest version of the table + * @param expectedNumFiles Expected number of manifest files + */ def assertManifest(tablePath: File, expectSameFiles: Boolean, expectedNumFiles: Int): Unit = { val deltaSnapshot = DeltaLog.forTable(spark, tablePath.toString).update() val manifestPath = new File(tablePath, GenerateSymlinkManifest.MANIFEST_LOCATION) @@ -509,15 +522,10 @@ trait DeltaGenerateSymlinkManifestSuiteBase extends QueryTest protected def withIncrementalManifest(tablePath: File, enabled: Boolean)(func: => Unit): Unit = { if (tablePath.exists()) { - val deltaLog = DeltaLog.forTable(spark, tablePath) - val latestMetadata = deltaLog.update().metadata + val latestMetadata = DeltaLog.forTable(spark, tablePath).update().metadata if (DeltaConfigs.SYMLINK_FORMAT_MANIFEST_ENABLED.fromMetaData(latestMetadata) != enabled) { - // Update the metadata of the table - val config = Map(DeltaConfigs.SYMLINK_FORMAT_MANIFEST_ENABLED.key -> enabled.toString) - val txn = deltaLog.startTransaction() - val metadata = txn.metadata - val newMetadata = metadata.copy(configuration = metadata.configuration ++ config) - txn.commit(newMetadata :: Nil, DeltaOperations.SetTableProperties(config)) + spark.sql(s"ALTER TABLE delta.`$tablePath` " + + s"SET TBLPROPERTIES(${DeltaConfigs.SYMLINK_FORMAT_MANIFEST_ENABLED.key}=$enabled)") } } func diff --git a/src/test/scala/org/apache/spark/sql/delta/DeltaInsertIntoTableSuite.scala b/src/test/scala/org/apache/spark/sql/delta/DeltaInsertIntoTableSuite.scala index 5cdf32c56d3..8ea4946fdf8 100644 --- a/src/test/scala/org/apache/spark/sql/delta/DeltaInsertIntoTableSuite.scala +++ b/src/test/scala/org/apache/spark/sql/delta/DeltaInsertIntoTableSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.delta // scalastyle:off import.ordering.noEmptyLine +import java.io.File + import scala.collection.JavaConverters._ import org.apache.spark.sql.delta.schema.SchemaUtils @@ -26,6 +28,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row, SaveMode} +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.functions.{lit, struct} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.{PARTITION_OVERWRITE_MODE, PartitionOverwriteMode} @@ -56,6 +59,30 @@ class DeltaInsertIntoSQLByPathSuite extends DeltaInsertIntoTests(false, true) sql(s"INSERT $overwrite TABLE delta.`${catalogTable.location}` SELECT * FROM $tmpView") } } + + testQuietly("insertInto: cannot insert into a table that doesn't exist") { + import testImplicits._ + Seq(SaveMode.Append, SaveMode.Overwrite).foreach { mode => + withTempDir { dir => + val t1 = s"delta.`${dir.getCanonicalPath}`" + val tmpView = "tmp_view" + withTempView(tmpView) { + val overwrite = if (mode == SaveMode.Overwrite) "OVERWRITE" else "INTO" + val df = Seq((1L, "a"), (2L, "b"), (3L, "c")).toDF("id", "data") + df.createOrReplaceTempView(tmpView) + + intercept[AnalysisException] { + sql(s"INSERT $overwrite TABLE $t1 SELECT * FROM $tmpView") + } + + assert(new File(dir, "_delta_log").mkdirs(), "Failed to create a _delta_log directory") + intercept[AnalysisException] { + sql(s"INSERT $overwrite TABLE $t1 SELECT * FROM $tmpView") + } + } + } + } + } } class DeltaInsertIntoDataFrameSuite extends DeltaInsertIntoTests(false, false) @@ -80,6 +107,37 @@ class DeltaInsertIntoDataFrameByPathSuite extends DeltaInsertIntoTests(false, fa val catalogTable = spark.sessionState.catalog.getTableMetadata(ident) dfw.insertInto(s"delta.`${catalogTable.location}`") } + + testQuietly("insertInto: cannot insert into a table that doesn't exist") { + import testImplicits._ + Seq(SaveMode.Append, SaveMode.Overwrite).foreach { mode => + withTempDir { dir => + val t1 = s"delta.`${dir.getCanonicalPath}`" + val df = Seq((1L, "a"), (2L, "b"), (3L, "c")).toDF("id", "data") + + intercept[AnalysisException] { + df.write.mode(mode).insertInto(t1) + } + + assert(new File(dir, "_delta_log").mkdirs(), "Failed to create a _delta_log directory") + intercept[AnalysisException] { + df.write.mode(mode).insertInto(t1) + } + + // Test DataFrameWriterV2 as well + val dfW2 = df.writeTo(t1) + if (mode == SaveMode.Append) { + intercept[AnalysisException] { + dfW2.append() + } + } else { + intercept[AnalysisException] { + dfW2.overwrite(lit(true)) + } + } + } + } + } } /** These tests come from Apache Spark with some modifications to match Delta behavior. */ diff --git a/src/test/scala/org/apache/spark/sql/delta/DeltaLogSuite.scala b/src/test/scala/org/apache/spark/sql/delta/DeltaLogSuite.scala index a8eebfdc564..7d83f5aaef2 100644 --- a/src/test/scala/org/apache/spark/sql/delta/DeltaLogSuite.scala +++ b/src/test/scala/org/apache/spark/sql/delta/DeltaLogSuite.scala @@ -19,7 +19,9 @@ package org.apache.spark.sql.delta import java.io.{File, FileNotFoundException} import org.apache.spark.sql.delta.DeltaOperations.Truncate +import org.apache.spark.sql.delta.DeltaTestUtils.OptimisticTxnTestHelper import org.apache.spark.sql.delta.actions._ +import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.delta.util.{FileNames, JsonUtils} import org.apache.hadoop.fs.Path @@ -36,7 +38,7 @@ class DeltaLogSuite extends QueryTest testQuietly("checkpoint") { val tempDir = Utils.createTempDir() - val log1 = DeltaLog(spark, new Path(tempDir.getCanonicalPath)) + val log1 = DeltaLog.forTable(spark, new Path(tempDir.getCanonicalPath)) (1 to 15).foreach { i => val txn = log1.startTransaction() @@ -46,11 +48,11 @@ class DeltaLogSuite extends QueryTest } else { Nil } - txn.commit(delete ++ file, testOp) + txn.commitManually(delete ++ file: _*) } DeltaLog.clearCache() - val log2 = DeltaLog(spark, new Path(tempDir.getCanonicalPath)) + val log2 = DeltaLog.forTable(spark, new Path(tempDir.getCanonicalPath)) assert(log2.snapshot.version == log1.snapshot.version) assert(log2.snapshot.allFiles.count == 1) } @@ -58,12 +60,12 @@ class DeltaLogSuite extends QueryTest testQuietly("SC-8078: update deleted directory") { withTempDir { dir => val path = new Path(dir.getCanonicalPath) - val log = DeltaLog(spark, path) + val log = DeltaLog.forTable(spark, path) // Commit data so the in-memory state isn't consistent with an empty log. val txn = log.startTransaction() val files = (1 to 10).map(f => AddFile(f.toString, Map.empty, 1, 1, true)) - txn.commit(files, testOp) + txn.commitManually(files: _*) log.checkpoint() val fs = path.getFileSystem(spark.sessionState.newHadoopConf()) @@ -78,11 +80,11 @@ class DeltaLogSuite extends QueryTest testQuietly("update should pick up checkpoints") { withTempDir { tempDir => - val log = DeltaLog(spark, new Path(tempDir.getCanonicalPath)) + val log = DeltaLog.forTable(spark, new Path(tempDir.getCanonicalPath)) val checkpointInterval = log.checkpointInterval for (f <- 0 until (checkpointInterval * 2)) { val txn = log.startTransaction() - txn.commit(Seq(AddFile(f.toString, Map.empty, 1, 1, true)), testOp) + txn.commitManually(AddFile(f.toString, Map.empty, 1, 1, true)) } def collectReservoirStateRDD(rdd: RDD[_]): Seq[RDD[_]] = { @@ -101,7 +103,7 @@ class DeltaLogSuite extends QueryTest test("update shouldn't pick up delta files earlier than checkpoint") { val tempDir = Utils.createTempDir() - val log1 = DeltaLog(spark, new Path(tempDir.getCanonicalPath)) + val log1 = DeltaLog.forTable(spark, new Path(tempDir.getCanonicalPath)) (1 to 5).foreach { i => val txn = log1.startTransaction() @@ -111,11 +113,11 @@ class DeltaLogSuite extends QueryTest } else { Nil } - txn.commit(delete ++ file, testOp) + txn.commitManually(delete ++ file: _*) } DeltaLog.clearCache() - val log2 = DeltaLog(spark, new Path(tempDir.getCanonicalPath)) + val log2 = DeltaLog.forTable(spark, new Path(tempDir.getCanonicalPath)) (6 to 15).foreach { i => val txn = log1.startTransaction() @@ -125,7 +127,7 @@ class DeltaLogSuite extends QueryTest } else { Nil } - txn.commit(delete ++ file, testOp) + txn.commitManually(delete ++ file: _*) } // Since log2 is a separate instance, it shouldn't be updated to version 15 @@ -150,7 +152,7 @@ class DeltaLogSuite extends QueryTest new Path(fs.getScheme + ":" + dir + "/foo"), new Path(fs.getScheme + "://" + dir + "/foo") ) - val logs = samePaths.map(DeltaLog(spark, _)) + val logs = samePaths.map(DeltaLog.forTable(spark, _)) logs.foreach { log => assert(log eq logs.head) } @@ -159,12 +161,12 @@ class DeltaLogSuite extends QueryTest testQuietly("handle corrupted '_last_checkpoint' file") { withTempDir { tempDir => - val log = DeltaLog(spark, new Path(tempDir.getCanonicalPath)) + val log = DeltaLog.forTable(spark, new Path(tempDir.getCanonicalPath)) val checkpointInterval = log.checkpointInterval for (f <- 0 to checkpointInterval) { val txn = log.startTransaction() - txn.commit(Seq(AddFile(f.toString, Map.empty, 1, 1, true)), testOp) + txn.commitManually(AddFile(f.toString, Map.empty, 1, 1, true)) } assert(log.lastCheckpoint.isDefined) @@ -176,7 +178,7 @@ class DeltaLogSuite extends QueryTest // Create a new DeltaLog DeltaLog.clearCache() - val log2 = DeltaLog(spark, new Path(tempDir.getCanonicalPath)) + val log2 = DeltaLog.forTable(spark, new Path(tempDir.getCanonicalPath)) // Make sure we create a new DeltaLog in order to test the loading logic. assert(log ne log2) @@ -198,7 +200,7 @@ class DeltaLogSuite extends QueryTest log.store.write( FileNames.deltaFile(log.logPath, 0L), - Iterator(JsonUtils.toJson(add.wrap))) + Iterator(Protocol(), Metadata(), add).map(a => JsonUtils.toJson(a.wrap))) log.store.write( FileNames.deltaFile(log.logPath, 1L), Iterator(JsonUtils.toJson(rm.wrap))) @@ -222,7 +224,7 @@ class DeltaLogSuite extends QueryTest log.store.write( FileNames.deltaFile(log.logPath, 0L), - Iterator(JsonUtils.toJson(add.wrap))) + Iterator(Protocol(), Metadata(), add).map(a => JsonUtils.toJson(a.wrap))) log.store.write( FileNames.deltaFile(log.logPath, 1L), Iterator(JsonUtils.toJson(rm.wrap))) @@ -244,7 +246,7 @@ class DeltaLogSuite extends QueryTest assert(new File(log.logPath.toUri).mkdirs()) val path = new File(dir, "a/b/c").getCanonicalPath val rm = RemoveFile(path, Some(System.currentTimeMillis()), dataChange = true) - log.startTransaction().commit(rm :: Nil, DeltaOperations.ManualUpdate) + log.startTransaction().commitManually(rm) val committedRemove = log.update(stalenessAcceptable = false).tombstones.collect() assert(committedRemove.head.path === s"file://$path") @@ -257,7 +259,7 @@ class DeltaLogSuite extends QueryTest assert(new File(log.logPath.toUri).mkdirs()) val add1 = AddFile("foo", Map.empty, 1L, System.currentTimeMillis(), dataChange = true) - log.startTransaction().commit(add1 :: Nil, DeltaOperations.ManualUpdate) + log.startTransaction().commitManually(add1) val rm = add1.remove log.startTransaction().commit(rm :: Nil, DeltaOperations.ManualUpdate) @@ -275,4 +277,124 @@ class DeltaLogSuite extends QueryTest === Some(add2.copy(dataChange = false))) } } + + test("error - versions not contiguous") { + withTempDir { dir => + val log = DeltaLog.forTable(spark, dir) + assert(new File(log.logPath.toUri).mkdirs()) + + val metadata = Metadata() + val add1 = AddFile("foo", Map.empty, 1L, System.currentTimeMillis(), dataChange = true) + log.startTransaction().commit(metadata :: add1 :: Nil, DeltaOperations.ManualUpdate) + + val add2 = AddFile("foo", Map.empty, 1L, System.currentTimeMillis(), dataChange = true) + log.startTransaction().commit(add2 :: Nil, DeltaOperations.ManualUpdate) + + val add3 = AddFile("foo", Map.empty, 1L, System.currentTimeMillis(), dataChange = true) + log.startTransaction().commit(add3 :: Nil, DeltaOperations.ManualUpdate) + + new File(new Path(log.logPath, "00000000000000000001.json").toUri).delete() + + DeltaLog.clearCache() + val ex = intercept[IllegalStateException] { + DeltaLog.forTable(spark, dir) + } + assert(ex.getMessage === "Versions (Vector(0, 2)) are not contiguous.") + } + } + + Seq("protocol", "metadata").foreach { action => + test(s"state reconstruction without $action should fail") { + withTempDir { tempDir => + val log = DeltaLog.forTable(spark, new Path(tempDir.getCanonicalPath)) + assert(new File(log.logPath.toUri).mkdirs()) + val selectedAction = if (action == "metadata") { + Protocol() + } else { + Metadata() + } + val file = AddFile("abc", Map.empty, 1, 1, true) + log.store.write( + FileNames.deltaFile(log.logPath, 0L), + Iterator(selectedAction, file).map(a => JsonUtils.toJson(a.wrap))) + withSQLConf(DeltaSQLConf.DELTA_STATE_RECONSTRUCTION_VALIDATION_ENABLED.key -> "true") { + val e = intercept[IllegalStateException] { + log.update() + } + assert(e.getMessage === DeltaErrors.actionNotFoundException(action, 0).getMessage) + } + // Disable the validation check + withSQLConf(DeltaSQLConf.DELTA_STATE_RECONSTRUCTION_VALIDATION_ENABLED.key -> "false") { + assert(log.update().version === 0L) + } + } + } + } + + Seq("protocol", "metadata").foreach { action => + testQuietly(s"state reconstruction from checkpoint with missing $action should fail") { + withTempDir { tempDir => + import testImplicits._ + val log = DeltaLog.forTable(spark, tempDir) + val checkpointInterval = log.checkpointInterval + // Create a checkpoint regularly + for (f <- 0 to checkpointInterval) { + val txn = log.startTransaction() + if (f == 0) { + txn.commitManually(AddFile(f.toString, Map.empty, 1, 1, true)) + } else { + txn.commit(Seq(AddFile(f.toString, Map.empty, 1, 1, true)), testOp) + } + } + + { + // Create an incomplete checkpoint without the action and overwrite the + // original checkpoint + val checkpointPath = FileNames.checkpointFileSingular(log.logPath, log.snapshot.version) + withTempDir { tmpCheckpoint => + val takeAction = if (action == "metadata") { + "protocol" + } else { + "metadata" + } + val corruptedCheckpointData = spark.read.parquet(checkpointPath.toString) + .where(s"add is not null or $takeAction is not null") + .as[SingleAction].collect() + + // Keep the add files and also filter by the additional condition + corruptedCheckpointData.toSeq.toDS().coalesce(1).write + .mode("overwrite").parquet(tmpCheckpoint.toString) + val writtenCheckpoint = + tmpCheckpoint.listFiles().toSeq.filter(_.getName.startsWith("part")).head + val checkpointFile = new File(checkpointPath.toUri) + new File(log.logPath.toUri).listFiles().toSeq.foreach { file => + if (file.getName.startsWith(".0")) { + // we need to delete checksum files, otherwise trying to replace our incomplete + // checkpoint file fails due to the LocalFileSystem's checksum checks. + require(file.delete(), "Failed to delete checksum file") + } + } + require(checkpointFile.delete(), "Failed to delete old checkpoint") + require(writtenCheckpoint.renameTo(checkpointFile), + "Failed to rename corrupt checkpoint") + } + } + + DeltaLog.clearCache() + + // Verify if the state reconstruction from the checkpoint fails. + withSQLConf(DeltaSQLConf.DELTA_STATE_RECONSTRUCTION_VALIDATION_ENABLED.key -> "true") { + val e = intercept[IllegalStateException] { + DeltaLog.forTable(spark, tempDir).update() + } + assert(e.getMessage === DeltaErrors.actionNotFoundException(action, 10).getMessage) + } + + // Disable state reconstruction validation and try again + withSQLConf(DeltaSQLConf.DELTA_STATE_RECONSTRUCTION_VALIDATION_ENABLED.key -> "false") { + assert(DeltaLog.forTable(spark, tempDir).update().version === 10) + } + } + } + } } diff --git a/src/test/scala/org/apache/spark/sql/delta/DeltaProtocolVersionSuite.scala b/src/test/scala/org/apache/spark/sql/delta/DeltaProtocolVersionSuite.scala index e7a2cf491e2..b3c207ba79c 100644 --- a/src/test/scala/org/apache/spark/sql/delta/DeltaProtocolVersionSuite.scala +++ b/src/test/scala/org/apache/spark/sql/delta/DeltaProtocolVersionSuite.scala @@ -74,7 +74,7 @@ trait DeltaProtocolVersionSuiteBase extends QueryTest log.ensureLogDirectoryExist() log.store.write( deltaFile(log.logPath, 0), - Iterator(Protocol(Integer.MAX_VALUE, Integer.MAX_VALUE).json)) + Iterator(Metadata().json, Protocol(Integer.MAX_VALUE, Integer.MAX_VALUE).json)) intercept[InvalidProtocolVersionException] { spark.range(1).write.format("delta").save(path.getCanonicalPath) } diff --git a/src/test/scala/org/apache/spark/sql/delta/DeltaRetentionSuite.scala b/src/test/scala/org/apache/spark/sql/delta/DeltaRetentionSuite.scala index cd9e7fab32b..3cdb95817d6 100644 --- a/src/test/scala/org/apache/spark/sql/delta/DeltaRetentionSuite.scala +++ b/src/test/scala/org/apache/spark/sql/delta/DeltaRetentionSuite.scala @@ -38,7 +38,8 @@ class DeltaRetentionSuite extends QueryTest with DeltaRetentionSuiteBase with SQ test("delete expired logs") { withTempDir { tempDir => val clock = new ManualClock(System.currentTimeMillis()) - val log = DeltaLog(spark, new Path(tempDir.getCanonicalPath), clock) + val log = DeltaLog.forTable(spark, new Path(tempDir.getCanonicalPath), clock) + val logPath = new File(log.logPath.toUri) (1 to 5).foreach { i => val txn = if (i == 1) startTxnWithManualLogCleanup(log) else log.startTransaction() val file = AddFile(i.toString, Map.empty, 1, 1, true) :: Nil @@ -50,25 +51,25 @@ class DeltaRetentionSuite extends QueryTest with DeltaRetentionSuiteBase with SQ txn.commit(delete ++ file, testOp) } - val initialFiles = getLogFiles(tempDir) + val initialFiles = getLogFiles(logPath) // Shouldn't clean up, no checkpoint, no expired files log.cleanUpExpiredLogs() - assert(initialFiles === getLogFiles(tempDir)) + assert(initialFiles === getLogFiles(logPath)) clock.advance(intervalStringToMillis(DeltaConfigs.LOG_RETENTION.defaultValue) + intervalStringToMillis("interval 1 day")) // Shouldn't clean up, no checkpoint, although all files have expired log.cleanUpExpiredLogs() - assert(initialFiles === getLogFiles(tempDir)) + assert(initialFiles === getLogFiles(logPath)) log.checkpoint() val expectedFiles = Seq("04.json", "04.checkpoint.parquet") // after checkpointing, the files should be cleared log.cleanUpExpiredLogs() - val afterCleanup = getLogFiles(tempDir) + val afterCleanup = getLogFiles(logPath) assert(initialFiles !== afterCleanup) assert(expectedFiles.forall(suffix => afterCleanup.exists(_.getName.endsWith(suffix))), s"${afterCleanup.mkString("\n")}\n didn't contain files with suffixes: $expectedFiles") @@ -78,7 +79,8 @@ class DeltaRetentionSuite extends QueryTest with DeltaRetentionSuiteBase with SQ test("log files being already deleted shouldn't fail log deletion job") { withTempDir { tempDir => val clock = new ManualClock(System.currentTimeMillis()) - val log = DeltaLog(spark, new Path(tempDir.getCanonicalPath), clock) + val log = DeltaLog.forTable(spark, new Path(tempDir.getCanonicalPath), clock) + val logPath = new File(log.logPath.toUri) (1 to 25).foreach { i => val txn = if (i == 1) startTxnWithManualLogCleanup(log) else log.startTransaction() @@ -100,19 +102,19 @@ class DeltaRetentionSuite extends QueryTest with DeltaRetentionSuiteBase with SQ } // delete some files in the middle - getDeltaFiles(tempDir).sortBy(_.getName).slice(5, 15).foreach(_.delete()) + getDeltaFiles(logPath).sortBy(_.getName).slice(5, 15).foreach(_.delete()) clock.advance(intervalStringToMillis(DeltaConfigs.LOG_RETENTION.defaultValue) + intervalStringToMillis("interval 2 day")) log.cleanUpExpiredLogs() val minDeltaFile = - getDeltaFiles(tempDir).map(f => FileNames.deltaVersion(new Path(f.toString))).min - val maxChkFile = getCheckpointFiles(tempDir).map(f => + getDeltaFiles(logPath).map(f => FileNames.deltaVersion(new Path(f.toString))).min + val maxChkFile = getCheckpointFiles(logPath).map(f => FileNames.checkpointVersion(new Path(f.toString))).max assert(maxChkFile === minDeltaFile, "Delta files before the last checkpoint version should have been deleted") - assert(getCheckpointFiles(tempDir).length === 1, + assert(getCheckpointFiles(logPath).length === 1, "There should only be the last checkpoint version") } } @@ -121,7 +123,7 @@ class DeltaRetentionSuite extends QueryTest with DeltaRetentionSuiteBase with SQ "RemoveFiles persist across checkpoints as tombstones if retention time hasn't expired") { withTempDir { tempDir => val clock = new ManualClock(System.currentTimeMillis()) - val log1 = DeltaLog(spark, new Path(tempDir.getCanonicalPath), clock) + val log1 = DeltaLog.forTable(spark, new Path(tempDir.getCanonicalPath), clock) val txn = startTxnWithManualLogCleanup(log1) val files1 = (1 to 10).map(f => AddFile(f.toString, Map.empty, 1, 1, true)) @@ -132,7 +134,7 @@ class DeltaRetentionSuite extends QueryTest with DeltaRetentionSuiteBase with SQ log1.checkpoint() DeltaLog.clearCache() - val log2 = DeltaLog(spark, new Path(tempDir.getCanonicalPath), clock) + val log2 = DeltaLog.forTable(spark, new Path(tempDir.getCanonicalPath), clock) assert(log2.snapshot.tombstones.count() === 4) assert(log2.snapshot.allFiles.count() === 6) } @@ -141,7 +143,7 @@ class DeltaRetentionSuite extends QueryTest with DeltaRetentionSuiteBase with SQ testQuietly("RemoveFiles get deleted during checkpoint if retention time has passed") { withTempDir { tempDir => val clock = new ManualClock(System.currentTimeMillis()) - val log1 = DeltaLog(spark, new Path(tempDir.getCanonicalPath), clock) + val log1 = DeltaLog.forTable(spark, new Path(tempDir.getCanonicalPath), clock) val txn = startTxnWithManualLogCleanup(log1) val files1 = (1 to 10).map(f => AddFile(f.toString, Map.empty, 1, 1, true)) @@ -156,7 +158,7 @@ class DeltaRetentionSuite extends QueryTest with DeltaRetentionSuiteBase with SQ log1.checkpoint() DeltaLog.clearCache() - val log2 = DeltaLog(spark, new Path(tempDir.getCanonicalPath), clock) + val log2 = DeltaLog.forTable(spark, new Path(tempDir.getCanonicalPath), clock) assert(log2.snapshot.tombstones.count() === 0) assert(log2.snapshot.allFiles.count() === 6) } @@ -165,11 +167,12 @@ class DeltaRetentionSuite extends QueryTest with DeltaRetentionSuiteBase with SQ test("the checkpoint file for version 0 should be cleaned") { withTempDir { tempDir => val clock = new ManualClock(System.currentTimeMillis()) - val log = DeltaLog(spark, new Path(tempDir.getCanonicalPath), clock) + val log = DeltaLog.forTable(spark, new Path(tempDir.getCanonicalPath), clock) + val logPath = new File(log.logPath.toUri) startTxnWithManualLogCleanup(log).commit(AddFile("0", Map.empty, 1, 1, true) :: Nil, testOp) log.checkpoint() - val initialFiles = getLogFiles(tempDir) + val initialFiles = getLogFiles(logPath) clock.advance(intervalStringToMillis(DeltaConfigs.LOG_RETENTION.defaultValue) + intervalStringToMillis("interval 1 day")) @@ -178,7 +181,7 @@ class DeltaRetentionSuite extends QueryTest with DeltaRetentionSuiteBase with SQ log.checkpoint() log.cleanUpExpiredLogs() - val afterCleanup = getLogFiles(tempDir) + val afterCleanup = getLogFiles(logPath) initialFiles.foreach { file => assert(!afterCleanup.contains(file)) } diff --git a/src/test/scala/org/apache/spark/sql/delta/DeltaSinkSuite.scala b/src/test/scala/org/apache/spark/sql/delta/DeltaSinkSuite.scala index 2b9a8c0717b..8ed0e7a86ba 100644 --- a/src/test/scala/org/apache/spark/sql/delta/DeltaSinkSuite.scala +++ b/src/test/scala/org/apache/spark/sql/delta/DeltaSinkSuite.scala @@ -20,6 +20,7 @@ import java.io.File import java.util.Locale import org.apache.spark.sql.delta.actions.CommitInfo +import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.commons.io.FileUtils import org.scalatest.time.SpanSugar._ @@ -506,4 +507,28 @@ class DeltaSinkSuite extends StreamTest { Seq(Row("ss", Row("ss", null), null))) } } + + test("history includes user-defined metadata for DataFrame.writeStream API") { + failAfter(streamingTimeout) { + withTempDirs { (outputDir, checkpointDir) => + val inputData = MemoryStream[Int] + val df = inputData.toDF() + val query = df.writeStream + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .option("userMetadata", "testMeta!") + .format("delta") + .start(outputDir.getCanonicalPath) + val log = DeltaLog.forTable(spark, outputDir.getCanonicalPath) + + inputData.addData(1) + query.processAllAvailable() + + val lastCommitInfo = io.delta.tables.DeltaTable.forPath(spark, outputDir.getCanonicalPath) + .history(1).as[CommitInfo].head + + assert(lastCommitInfo.userMetadata === Some("testMeta!")) + query.stop() + } + } + } } diff --git a/src/test/scala/org/apache/spark/sql/delta/DeltaSuite.scala b/src/test/scala/org/apache/spark/sql/delta/DeltaSuite.scala index ceb4de8988c..f4dd9d91c69 100644 --- a/src/test/scala/org/apache/spark/sql/delta/DeltaSuite.scala +++ b/src/test/scala/org/apache/spark/sql/delta/DeltaSuite.scala @@ -19,8 +19,10 @@ package org.apache.spark.sql.delta import java.io.{File, FileNotFoundException} import java.util.concurrent.atomic.AtomicInteger +import org.apache.spark.sql.delta.actions.CommitInfo import org.apache.spark.sql.delta.files.TahoeLogFileIndex import org.apache.spark.sql.delta.sources.DeltaSQLConf +import org.apache.spark.sql.delta.test.DeltaSQLCommandTest import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.SparkException @@ -37,7 +39,8 @@ import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} import org.apache.spark.util.Utils class DeltaSuite extends QueryTest - with SharedSparkSession with SQLTestUtils { + with SharedSparkSession with SQLTestUtils + with DeltaSQLCommandTest { import testImplicits._ @@ -756,9 +759,7 @@ class DeltaSuite extends QueryTest // The file names are opaque. To identify which one we're deleting, we ensure that only one // append has 2 partitions, and give them the same value so we know what was deleted. - val inputFiles = - TahoeLogFileIndex(spark, deltaLog, new Path(tempDir.getCanonicalPath)) - .inputFiles.toSeq + val inputFiles = TahoeLogFileIndex(spark, deltaLog).inputFiles.toSeq assert(inputFiles.size == 5) val filesToDelete = inputFiles.filter(_.split("/").last.startsWith("part-00001")) @@ -790,9 +791,7 @@ class DeltaSuite extends QueryTest // The file names are opaque. To identify which one we're deleting, we ensure that only one // append has 2 partitions, and give them the same value so we know what was deleted. - val inputFiles = - TahoeLogFileIndex(spark, deltaLog, new Path(tempDir.getCanonicalPath)) - .inputFiles.toSeq + val inputFiles = TahoeLogFileIndex(spark, deltaLog).inputFiles.toSeq assert(inputFiles.size == 5) val filesToCorrupt = inputFiles.filter(_.split("/").last.startsWith("part-00001")) @@ -821,9 +820,7 @@ class DeltaSuite extends QueryTest Range(0, 10).foreach(n => Seq(n).toDF().write.format("delta").mode("append").save(tempDir.toString)) - val inputFiles = - TahoeLogFileIndex(spark, deltaLog, new Path(tempDir.getCanonicalPath)) - .inputFiles.toSeq + val inputFiles = TahoeLogFileIndex(spark, deltaLog).inputFiles.toSeq val filesToDelete = inputFiles.take(4) filesToDelete.foreach { f => @@ -850,9 +847,7 @@ class DeltaSuite extends QueryTest Range(0, 10).foreach(n => Seq(n).toDF().write.format("delta").mode("append").save(tempDir.toString)) - val inputFiles = - TahoeLogFileIndex(spark, deltaLog, new Path(tempDir.getCanonicalPath)) - .inputFiles.toSeq + val inputFiles = TahoeLogFileIndex(spark, deltaLog).inputFiles.toSeq val filesToDelete = inputFiles.take(4) filesToDelete.foreach { f => @@ -1013,4 +1008,78 @@ class DeltaSuite extends QueryTest sparkContext.removeSparkListener(listener) } } + + def lastCommitInfo(dir: String): CommitInfo = + io.delta.tables.DeltaTable.forPath(spark, dir).history(1).as[CommitInfo].head + + test("history includes user-defined metadata for DataFrame.Write API") { + val tempDir = Utils.createTempDir().toString + val df = Seq(2).toDF().write.format("delta").mode("overwrite") + + df.option("userMetadata", "meta1") + .save(tempDir) + + assert(lastCommitInfo(tempDir).userMetadata === Some("meta1")) + + df.option("userMetadata", "meta2") + .save(tempDir) + + assert(lastCommitInfo(tempDir).userMetadata === Some("meta2")) + } + + test("history includes user-defined metadata for SQL API") { + val tempDir = Utils.createTempDir().toString + val tblName = "tblName" + + withTable(tblName) { + withSQLConf(DeltaSQLConf.DELTA_USER_METADATA.key -> "meta1") { + spark.sql(s"CREATE TABLE $tblName (data STRING) USING delta LOCATION '$tempDir';") + } + assert(lastCommitInfo(tempDir).userMetadata === Some("meta1")) + + withSQLConf(DeltaSQLConf.DELTA_USER_METADATA.key -> "meta2") { + spark.sql(s"INSERT INTO $tblName VALUES ('test');") + } + assert(lastCommitInfo(tempDir).userMetadata === Some("meta2")) + + withSQLConf(DeltaSQLConf.DELTA_USER_METADATA.key -> "meta3") { + spark.sql(s"INSERT INTO $tblName VALUES ('test2');") + } + assert(lastCommitInfo(tempDir).userMetadata === Some("meta3")) + } + } + + test("history includes user-defined metadata for DF.Write API and config setting") { + val tempDir = Utils.createTempDir().toString + val df = Seq(2).toDF().write.format("delta").mode("overwrite") + + withSQLConf(DeltaSQLConf.DELTA_USER_METADATA.key -> "meta1") { + df.save(tempDir) + } + assert(lastCommitInfo(tempDir).userMetadata === Some("meta1")) + + withSQLConf(DeltaSQLConf.DELTA_USER_METADATA.key -> "meta2") { + df.option("userMetadata", "optionMeta2") + .save(tempDir) + } + assert(lastCommitInfo(tempDir).userMetadata === Some("optionMeta2")) + } + + test("history includes user-defined metadata for SQL + DF.Write API") { + val tempDir = Utils.createTempDir().toString + val df = Seq(2).toDF().write.format("delta").mode("overwrite") + + // metadata given in `option` should beat config + withSQLConf(DeltaSQLConf.DELTA_USER_METADATA.key -> "meta1") { + df.option("userMetadata", "optionMeta1") + .save(tempDir) + } + assert(lastCommitInfo(tempDir).userMetadata === Some("optionMeta1")) + + withSQLConf(DeltaSQLConf.DELTA_USER_METADATA.key -> "meta2") { + df.option("userMetadata", "optionMeta2") + .save(tempDir) + } + assert(lastCommitInfo(tempDir).userMetadata === Some("optionMeta2")) + } } diff --git a/src/test/scala/org/apache/spark/sql/delta/DeltaTestUtils.scala b/src/test/scala/org/apache/spark/sql/delta/DeltaTestUtils.scala new file mode 100644 index 00000000000..d335fe6bc71 --- /dev/null +++ b/src/test/scala/org/apache/spark/sql/delta/DeltaTestUtils.scala @@ -0,0 +1,38 @@ +/* + * Copyright (2020) 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.actions.{Action, AddFile, Metadata, Protocol, SingleAction} + +trait DeltaTestUtilsBase { + + /** + * Helper class for to ensure initial commits contain a Metadata action. + */ + implicit class OptimisticTxnTestHelper(txn: OptimisticTransaction) { + def commitManually(actions: Action*): Long = { + if (txn.readVersion == -1 && !actions.exists(_.isInstanceOf[Metadata])) { + txn.commit(Metadata() +: actions, ManualUpdate) + } else { + txn.commit(actions, ManualUpdate) + } + } + } +} + +object DeltaTestUtils extends DeltaTestUtilsBase diff --git a/src/test/scala/org/apache/spark/sql/delta/DeltaTimeTravelSuite.scala b/src/test/scala/org/apache/spark/sql/delta/DeltaTimeTravelSuite.scala index 8bf1e6f6f3e..a3ccf0312d3 100644 --- a/src/test/scala/org/apache/spark/sql/delta/DeltaTimeTravelSuite.scala +++ b/src/test/scala/org/apache/spark/sql/delta/DeltaTimeTravelSuite.scala @@ -25,6 +25,7 @@ import scala.concurrent.duration._ import scala.language.implicitConversions import org.apache.spark.sql.delta.DeltaHistoryManager.BufferingLogDeletionIterator +import org.apache.spark.sql.delta.DeltaTestUtils.OptimisticTxnTestHelper import org.apache.spark.sql.delta.actions.AddFile import org.apache.spark.sql.delta.util.FileNames import org.apache.commons.lang3.time.DateUtils @@ -66,7 +67,7 @@ class DeltaTimeTravelSuite extends QueryTest var startVersion = deltaLog.snapshot.version + 1 commits.foreach { ts => val action = AddFile(startVersion.toString, Map.empty, 10L, startVersion, dataChange = true) - deltaLog.startTransaction().commit(Seq(action), DeltaOperations.ManualUpdate) + deltaLog.startTransaction().commitManually(action) modifyCommitTimestamp(deltaLog, startVersion, ts) startVersion += 1 } diff --git a/src/test/scala/org/apache/spark/sql/delta/DeltaVacuumSuite.scala b/src/test/scala/org/apache/spark/sql/delta/DeltaVacuumSuite.scala index fff9a728e81..60fa725c0e5 100644 --- a/src/test/scala/org/apache/spark/sql/delta/DeltaVacuumSuite.scala +++ b/src/test/scala/org/apache/spark/sql/delta/DeltaVacuumSuite.scala @@ -18,9 +18,9 @@ package org.apache.spark.sql.delta import java.io.File import java.util.Locale -import java.util.concurrent.TimeUnit import org.apache.spark.sql.delta.DeltaOperations.{Delete, Write} +import org.apache.spark.sql.delta.DeltaTestUtils.OptimisticTxnTestHelper import org.apache.spark.sql.delta.actions.{AddFile, Metadata, RemoveFile} import org.apache.spark.sql.delta.commands.VacuumCommand import org.apache.spark.sql.delta.test.DeltaSQLCommandTest @@ -30,6 +30,7 @@ import org.apache.hadoop.fs.Path import org.scalatest.GivenWhenThen import org.apache.spark.sql.{AnalysisException, QueryTest, SaveMode} +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.execution.metric.SQLMetrics.createMetric @@ -43,6 +44,55 @@ trait DeltaVacuumSuiteBase extends QueryTest with GivenWhenThen with SQLTestUtils { + testQuietly("basic case - SQL command on path-based tables with direct 'path'") { + withEnvironment { (tempDir, _) => + vacuumSQLTest(tablePath = tempDir.getAbsolutePath, identifier = s"'$tempDir'") + } + } + + testQuietly("basic case - SQL command on path-based table with delta.`path`") { + withEnvironment { (tempDir, _) => + vacuumSQLTest(tablePath = tempDir.getAbsolutePath, identifier = s"delta.`$tempDir`") + } + } + + testQuietly("basic case - SQL command on name-based table") { + val tableName = "deltaTable" + withEnvironment { (_, _) => + withTable(tableName) { + import testImplicits._ + spark.emptyDataset[Int].write.format("delta").saveAsTable(tableName) + val tablePath = + new File(spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)).location) + vacuumSQLTest(tablePath, tableName) + } + } + } + + test("basic case - Scala on path-based table") { + withEnvironment { (tempDir, _) => + import testImplicits._ + spark.emptyDataset[Int].write.format("delta").save(tempDir.getAbsolutePath) + val deltaTable = io.delta.tables.DeltaTable.forPath(tempDir.getAbsolutePath) + vacuumScalaTest(deltaTable, tempDir.getAbsolutePath) + } + } + + test("basic case - Scala on name-based table") { + val tableName = "deltaTable" + withEnvironment { (tempDir, _) => + withTable(tableName) { + // Initialize the table so that we can create the DeltaTable object + import testImplicits._ + spark.emptyDataset[Int].write.format("delta").saveAsTable(tableName) + val deltaTable = io.delta.tables.DeltaTable.forName(tableName) + val tablePath = + new File(spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)).location) + vacuumScalaTest(deltaTable, tablePath) + } + } + } + test("don't delete data in a non-reservoir") { withEnvironment { (tempDir, clock) => val deltaLog = DeltaLog.forTable(spark, tempDir.getAbsolutePath, clock) @@ -81,8 +131,8 @@ trait DeltaVacuumSuiteBase extends QueryTest Metadata(schemaString = schema.json, partitionColumns = Seq("_underscore_col_")) txn.commit(metadata :: Nil, DeltaOperations.CreateTable(metadata, isManaged = true)) gcTest(deltaLog, clock)( - CreateFile("file1.txt", commitToActionLog = true), - CreateFile("_underscore_col_=10/test.txt", commitToActionLog = true), + CreateFile("file1.txt", commitToActionLog = true, Map("_underscore_col_" -> "10")), + CreateFile("_underscore_col_=10/test.txt", true, Map("_underscore_col_" -> "10")), CheckFiles(Seq("file1.txt", "_underscore_col_=10")), LogicallyDeleteFile("_underscore_col_=10/test.txt"), AdvanceClock(defaultTombstoneInterval + 1000), @@ -127,7 +177,7 @@ trait DeltaVacuumSuiteBase extends QueryTest } } - testQuietly("gc test") { + testQuietly("correctness test") { withEnvironment { (tempDir, clock) => val reservoirDir = new File(tempDir.getAbsolutePath, "reservoir") @@ -203,11 +253,12 @@ trait DeltaVacuumSuiteBase extends QueryTest classOf[IllegalArgumentException], Seq("Retention", "less than", "0")) ) + val deltaTable = io.delta.tables.DeltaTable.forPath(spark, tempDir.getAbsolutePath) gcTest(deltaLog, clock)( CreateFile("file1.txt", commitToActionLog = true), CheckFiles(Seq("file1.txt")), ExpectFailure( - GCScalaApi(Seq(), Some(-2)), + ExecuteVacuumInScala(deltaTable, Seq(), Some(-2)), classOf[IllegalArgumentException], Seq("Retention", "less than", "0")) ) @@ -268,24 +319,6 @@ trait DeltaVacuumSuiteBase extends QueryTest } } - test("scala api test") { - withEnvironment { (tempDir, clock) => - val deltaLog = DeltaLog.forTable(spark, tempDir.getAbsolutePath, clock) - gcTest(deltaLog, clock)( - CreateFile("file1.txt", commitToActionLog = true), - CreateFile("file2.txt", commitToActionLog = false), - GCScalaApi(expectedDf = Seq()), - CheckFiles(Seq("file1.txt")), - GCScalaApi(expectedDf = Seq(), retentionHours = Some(0)), - CheckFiles(Seq("file2.txt"), exist = false), - CreateFile("file2.txt", commitToActionLog = false), - CheckFiles(Seq("file2.txt")), - GCScalaApi(expectedDf = Seq(), retentionHours = Some(0)), - CheckFiles(Seq("file2.txt"), exist = false) - ) - } - } - protected def withEnvironment(f: (File, ManualClock) => Unit): Unit = { withTempDir { file => val clock = new ManualClock() @@ -308,7 +341,10 @@ trait DeltaVacuumSuiteBase extends QueryTest * Write a file to the given absolute or relative path. Could be inside or outside the Reservoir * base path. The file can be committed to the action log to be tracked, or left out for deletion. */ - case class CreateFile(path: String, commitToActionLog: Boolean) extends Action + case class CreateFile( + path: String, + commitToActionLog: Boolean, + partitionValues: Map[String, String] = Map.empty) extends Action /** Create a directory at the given path. */ case class CreateDirectory(path: String) extends Action /** @@ -324,13 +360,24 @@ trait DeltaVacuumSuiteBase extends QueryTest expectedDf: Seq[String], retentionHours: Option[Double] = None) extends Action /** Garbage collect the reservoir. */ - case class GCScalaApi( + case class ExecuteVacuumInScala( + deltaTable: io.delta.tables.DeltaTable, expectedDf: Seq[String], retentionHours: Option[Double] = None) extends Action /** Advance the time. */ case class AdvanceClock(timeToAdd: Long) extends Action /** Execute SQL command */ - case class ExecuteSQL(sql: String, expectedDf: Seq[String]) extends Action + case class ExecuteVacuumInSQL( + identifier: String, + expectedDf: Seq[String], + retentionHours: Option[Long] = None, + dryRun: Boolean = false) extends Action { + def sql: String = { + val retainStr = retentionHours.map { h => s"RETAIN $h HOURS"}.getOrElse("") + val dryRunStr = if (dryRun) "DRY RUN" else "" + s"VACUUM $identifier $retainStr $dryRunStr" + } + } /** * Expect a failure with the given exception type. Expect the given `msg` fragments as the error * message. @@ -344,10 +391,11 @@ trait DeltaVacuumSuiteBase extends QueryTest reservoirBase: String, filePath: String, file: File, - clock: ManualClock): AddFile = { + clock: ManualClock, + partitionValues: Map[String, String] = Map.empty): AddFile = { FileUtils.write(file, "gibberish") file.setLastModified(clock.getTimeMillis()) - AddFile(filePath, Map.empty, 10L, clock.getTimeMillis(), dataChange = true) + AddFile(filePath, partitionValues, 10L, clock.getTimeMillis(), dataChange = true) } protected def gcTest(deltaLog: DeltaLog, clock: ManualClock)(actions: Action*): Unit = { @@ -355,14 +403,18 @@ trait DeltaVacuumSuiteBase extends QueryTest val basePath = deltaLog.dataPath.toString val fs = new Path(basePath).getFileSystem(spark.sessionState.newHadoopConf()) actions.foreach { - case CreateFile(path, commit) => + case CreateFile(path, commit, partitionValues) => Given(s"*** Writing file to $path. Commit to log: $commit") val sanitizedPath = new Path(path).toUri.toString val file = new File( fs.makeQualified(DeltaFileOperations.absolutePath(basePath, sanitizedPath)).toUri) if (commit) { + if (!DeltaTableUtils.isDeltaTable(spark, new Path(basePath))) { + // initialize the table + deltaLog.startTransaction().commitManually() + } val txn = deltaLog.startTransaction() - val action = createFile(basePath, sanitizedPath, file, clock) + val action = createFile(basePath, sanitizedPath, file, clock, partitionValues) txn.commit(Seq(action), Write(SaveMode.Append)) } else { createFile(basePath, path, file, clock) @@ -385,10 +437,11 @@ trait DeltaVacuumSuiteBase extends QueryTest txn.registerSQLMetrics(spark, metrics) txn.commit(Seq(RemoveFile(path, Option(clock.getTimeMillis()))), Delete("true" :: Nil)) // scalastyle:on - case ExecuteSQL(statement, expectedDf) => - Given(s"*** Executing SQL: $statement") - val qualified = expectedDf.map(p => fs.makeQualified(new Path(p)).toString) - checkDatasetUnorderly(spark.sql(statement).as[String], qualified: _*) + case e: ExecuteVacuumInSQL => + Given(s"*** Executing SQL: ${e.sql}") + val qualified = e.expectedDf.map(p => fs.makeQualified(new Path(p)).toString) + val df = spark.sql(e.sql).as[String] + checkDatasetUnorderly(df, qualified: _*) case CheckFiles(paths, exist) => Given(s"*** Checking files exist=$exist") paths.foreach { p => @@ -402,9 +455,8 @@ trait DeltaVacuumSuiteBase extends QueryTest val result = VacuumCommand.gc(spark, deltaLog, dryRun, retention, clock = clock) val qualified = expectedDf.map(p => fs.makeQualified(new Path(p)).toString) checkDatasetUnorderly(result.as[String], qualified: _*) - case GCScalaApi(expectedDf, retention) => + case ExecuteVacuumInScala(deltaTable, expectedDf, retention) => Given("*** Garbage collecting Reservoir using Scala") - val deltaTable = io.delta.tables.DeltaTable.forPath(spark, deltaLog.dataPath.toString) val result = if (retention.isDefined) { deltaTable.vacuum(retention.get) } else { @@ -430,44 +482,63 @@ trait DeltaVacuumSuiteBase extends QueryTest } } - /** - * A basic vacuum test for different representations of a vacuum command. - * - * @param f A function to convert the file path to a representation in a vacuum command. - */ - def vacuumTest(f: File => String): Unit = { - withEnvironment { (tempDir, clock) => - val retention = defaultTombstoneInterval / 5 - val retentionHours = TimeUnit.MILLISECONDS.toHours(retention) - val deltaLog = DeltaLog.forTable(spark, tempDir.getAbsolutePath, clock) - gcTest(deltaLog, clock)( - CreateFile("file1.txt", commitToActionLog = true), - CheckFiles(Seq("file1.txt")), - AdvanceClock(2000), - LogicallyDeleteFile("file1.txt"), - AdvanceClock(retention + 10000), - // shouldn't delete because it's a dry run - ExecuteSQL( - s"vacuum ${f(tempDir)} retain $retentionHours hours DRY RUN", - Seq(new File(tempDir, "file1.txt").toString)), - CheckFiles(Seq("file1.txt")), - // shouldn't delete because default period hasn't passed - ExecuteSQL(s"vacuum ${f(tempDir)}", Seq(tempDir.toString)), - CheckFiles(Seq("file1.txt")), - ExecuteSQL(s"vacuum ${f(tempDir)} retain $retentionHours hours", - Seq(tempDir.toString)), - CheckFiles(Seq("file1.txt"), exist = false) - ) - } + protected def vacuumSQLTest(tablePath: String, identifier: String) { + val deltaLog = DeltaLog.forTable(spark, tablePath) + val committedFile = "committedFile.txt" + val notCommittedFile = "notCommittedFile.txt" + + gcTest(deltaLog, new ManualClock())( + // Prepare the table with files with timestamp of epoch-time 0 (i.e. 01-01-1970 00:00) + CreateFile(committedFile, commitToActionLog = true), + CreateFile(notCommittedFile, commitToActionLog = false), + CheckFiles(Seq(committedFile, notCommittedFile)), + + // Dry run should return the not committed file and but not delete files + ExecuteVacuumInSQL( + identifier, + expectedDf = Seq(new File(tablePath, notCommittedFile).toString), + dryRun = true), + CheckFiles(Seq(committedFile, notCommittedFile)), + + // Actual run should delete the not committed file but delete the not-committed file + ExecuteVacuumInSQL(identifier, Seq(tablePath)), + CheckFiles(Seq(committedFile)), + CheckFiles(Seq(notCommittedFile), exist = false), // file ts older than default retention + + // Logically delete the file. + LogicallyDeleteFile(committedFile), + CheckFiles(Seq(committedFile)), + + // Vacuum with 0 retention should actually delete the file. + ExecuteVacuumInSQL(identifier, Seq(tablePath), Some(0)), + CheckFiles(Seq(committedFile), exist = false)) } - testQuietly("vacuum command") { - vacuumTest(f => s"'${f.toString()}'") + protected def vacuumScalaTest(deltaTable: io.delta.tables.DeltaTable, tablePath: String) { + val deltaLog = DeltaLog.forTable(spark, tablePath) + val committedFile = "committedFile.txt" + val notCommittedFile = "notCommittedFile.txt" + + gcTest(deltaLog, new ManualClock())( + // Prepare the table with files with timestamp of epoch-time 0 (i.e. 01-01-1970 00:00) + CreateFile(committedFile, commitToActionLog = true), + CreateFile(notCommittedFile, commitToActionLog = false), + CheckFiles(Seq(committedFile, notCommittedFile)), + + // Actual run should delete the not committed file and but not delete files + ExecuteVacuumInScala(deltaTable, Seq()), + CheckFiles(Seq(committedFile)), + CheckFiles(Seq(notCommittedFile), exist = false), // file ts older than default retention + + // Logically delete the file. + LogicallyDeleteFile(committedFile), + CheckFiles(Seq(committedFile)), + + // Vacuum with 0 retention should actually delete the file. + ExecuteVacuumInScala(deltaTable, Seq(), Some(0)), + CheckFiles(Seq(committedFile), exist = false)) } - testQuietly("vacuum command with delta table identifier") { - vacuumTest(f => s"delta.`${f.toString()}`") - } test("vacuum for a partition path") { withEnvironment { (tempDir, _) => diff --git a/src/test/scala/org/apache/spark/sql/delta/DescribeDeltaHistorySuite.scala b/src/test/scala/org/apache/spark/sql/delta/DescribeDeltaHistorySuite.scala index dc009e289a8..502d1b8dcf1 100644 --- a/src/test/scala/org/apache/spark/sql/delta/DescribeDeltaHistorySuite.scala +++ b/src/test/scala/org/apache/spark/sql/delta/DescribeDeltaHistorySuite.scala @@ -17,27 +17,32 @@ package org.apache.spark.sql.delta // scalastyle:off import.ordering.noEmptyLine -import org.apache.spark.sql.delta.DeltaOperations.{Delete, Merge, Operation, Update} -import org.apache.spark.sql.delta.actions.CommitInfo +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 +import org.apache.spark.sql.delta.util.FileNames +import org.scalactic.source.Position import org.scalatest.Tag import org.apache.spark.sql.{AnalysisException, Column, DataFrame, QueryTest, Row} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.execution.streaming.MemoryStream import org.apache.spark.sql.functions._ -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.util.Utils trait DescribeDeltaHistorySuiteBase extends QueryTest - with SharedSparkSession { + with SharedSparkSession with DeltaSQLCommandTest { import testImplicits._ - protected val evolvabilityResource = "src/test/resources/delta/history/delta-0.2.0" + protected val evolvabilityResource = { + new File("src/test/resources/delta/history/delta-0.2.0").getAbsolutePath() + } protected val evolvabilityLastOp = Seq("STREAMING UPDATE", null, null) @@ -53,11 +58,10 @@ trait DescribeDeltaHistorySuiteBase basePath: String, expected: Seq[String], columns: Seq[Column] = Seq($"operation", $"operationParameters.mode")): Unit = { - val df = getHistory(basePath, Some(1)) - checkAnswer( - df.select(columns: _*), - Seq(Row(expected: _*)) - ) + val df = io.delta.tables.DeltaTable.forPath(spark, basePath).history(1) + checkAnswer(df.select(columns: _*), Seq(Row(expected: _*))) + val df2 = spark.sql(s"DESCRIBE HISTORY delta.`$basePath` LIMIT 1") + checkAnswer(df2.select(columns: _*), Seq(Row(expected: _*))) } protected def checkOperationMetrics( @@ -94,21 +98,243 @@ trait DescribeDeltaHistorySuiteBase .asInstanceOf[Map[String, String]] } - def getHistory(path: String, limit: Option[Int] = None): DataFrame = { - val deltaTable = io.delta.tables.DeltaTable.forPath(spark, path) - if (limit.isDefined) { - deltaTable.history(limit.get) - } else { - deltaTable.history() + testWithFlag("basic case - Scala history with path-based table") { + val tempDir = Utils.createTempDir().toString + Seq(1, 2, 3).toDF().write.format("delta").save(tempDir) + Seq(4, 5, 6).toDF().write.format("delta").mode("overwrite").save(tempDir) + + val deltaTable = io.delta.tables.DeltaTable.forPath(spark, tempDir) + // Full History + checkAnswer( + deltaTable.history().select("operation", "operationParameters.mode"), + Seq(Row("WRITE", "Overwrite"), Row("WRITE", "ErrorIfExists"))) + + // History with limit + checkAnswer( + deltaTable.history(1).select("operation", "operationParameters.mode"), + Seq(Row("WRITE", "Overwrite"))) + } + + test("basic case - Scala history with name-based table") { + withTable("delta_test") { + Seq(1, 2, 3).toDF().write.format("delta").saveAsTable("delta_test") + Seq(4, 5, 6).toDF().write.format("delta").mode("overwrite").saveAsTable("delta_test") + + val deltaTable = io.delta.tables.DeltaTable.forName(spark, "delta_test") + // Full History + checkAnswer( + deltaTable.history().select("operation"), + Seq(Row("CREATE OR REPLACE TABLE AS SELECT"), Row("CREATE TABLE AS SELECT"))) + + // History with limit + checkAnswer( + deltaTable.history(1).select("operation"), + Seq(Row("CREATE OR REPLACE TABLE AS SELECT"))) } } - testWithFlag("logging and limit") { + testWithFlag("basic case - SQL describe history with path-based table") { val tempDir = Utils.createTempDir().toString Seq(1, 2, 3).toDF().write.format("delta").save(tempDir) Seq(4, 5, 6).toDF().write.format("delta").mode("overwrite").save(tempDir) - assert(getHistory(tempDir).count === 2) - checkLastOperation(tempDir, Seq("WRITE", "Overwrite")) + + // With delta.`path` format + checkAnswer( + sql(s"DESCRIBE HISTORY delta.`$tempDir`").select("operation", "operationParameters.mode"), + Seq(Row("WRITE", "Overwrite"), Row("WRITE", "ErrorIfExists"))) + + checkAnswer( + sql(s"DESCRIBE HISTORY delta.`$tempDir` LIMIT 1") + .select("operation", "operationParameters.mode"), + Seq(Row("WRITE", "Overwrite"))) + + // With direct path format + checkAnswer( + sql(s"DESCRIBE HISTORY '$tempDir'").select("operation", "operationParameters.mode"), + Seq(Row("WRITE", "Overwrite"), Row("WRITE", "ErrorIfExists"))) + + checkAnswer( + sql(s"DESCRIBE HISTORY '$tempDir' LIMIT 1") + .select("operation", "operationParameters.mode"), + Seq(Row("WRITE", "Overwrite"))) + } + + testWithFlag("basic case - SQL describe history with name-based table") { + withTable("delta_test") { + Seq(1, 2, 3).toDF().write.format("delta").saveAsTable("delta_test") + Seq(4, 5, 6).toDF().write.format("delta").mode("overwrite").saveAsTable("delta_test") + + checkAnswer( + sql(s"DESCRIBE HISTORY delta_test").select("operation"), + Seq(Row("CREATE OR REPLACE TABLE AS SELECT"), Row("CREATE TABLE AS SELECT"))) + + checkAnswer( + sql(s"DESCRIBE HISTORY delta_test LIMIT 1").select("operation"), + Seq(Row("CREATE OR REPLACE TABLE AS SELECT"))) + } + } + + testWithFlag("describe history fails on views") { + val tempDir = Utils.createTempDir().toString + Seq(1, 2, 3).toDF().write.format("delta").save(tempDir) + val viewName = "delta_view" + withView(viewName) { + sql(s"create view $viewName as select * from delta.`$tempDir`") + + val e = intercept[AnalysisException] { + sql(s"DESCRIBE HISTORY $viewName").collect() + } + assert(e.getMessage.contains("history of a view")) + } + } + + testWithFlag("operations - create table") { + withTable("delta_test") { + sql( + s"""create table delta_test ( + | a int, + | b string + |) + |using delta + |partitioned by (b) + |comment 'this is my table' + |tblproperties (delta.appendOnly=true) + """.stripMargin) + checkLastOperation( + spark.sessionState.catalog.defaultTablePath(TableIdentifier("delta_test")).toString, + Seq( + "CREATE TABLE", + "true", + """["b"]""", + """{"delta.appendOnly":"true"}""", + "this is my table"), + Seq( + $"operation", $"operationParameters.isManaged", $"operationParameters.partitionBy", + $"operationParameters.properties", $"operationParameters.description")) + } + } + + testWithFlag("operations - ctas (saveAsTable)") { + val tempDir = Utils.createTempDir().toString + withTable("delta_test") { + Seq((1, "a"), (2, "3")).toDF("id", "data").write.format("delta") + .option("path", tempDir).saveAsTable("delta_test") + checkLastOperation( + tempDir, + Seq("CREATE TABLE AS SELECT", "false", """[]""", "{}", null), + Seq($"operation", $"operationParameters.isManaged", $"operationParameters.partitionBy", + $"operationParameters.properties", $"operationParameters.description")) + } + } + + testWithFlag("operations - ctas (sql)") { + val tempDir = Utils.createTempDir().toString + withTable("delta_test") { + sql( + s"""create table delta_test + |using delta + |location '$tempDir' + |tblproperties (delta.appendOnly=true) + |partitioned by (b) + |as select 1 as a, 'x' as b + """.stripMargin) + checkLastOperation( + tempDir, + Seq("CREATE TABLE AS SELECT", + "false", + """["b"]""", + """{"delta.appendOnly":"true"}""", null), + Seq($"operation", $"operationParameters.isManaged", $"operationParameters.partitionBy", + $"operationParameters.properties", $"operationParameters.description")) + } + val tempDir2 = Utils.createTempDir().toString + withTable("delta_test") { + sql( + s"""create table delta_test + |using delta + |location '$tempDir2' + |comment 'this is my table' + |as select 1 as a, 'x' as b + """.stripMargin) + // TODO(burak): Fix comments for CTAS + checkLastOperation( + tempDir2, + Seq("CREATE TABLE AS SELECT", + "false", """[]""", """{}""", null), + Seq($"operation", $"operationParameters.isManaged", $"operationParameters.partitionBy", + $"operationParameters.properties", $"operationParameters.description")) + } + } + + + testWithFlag("operations - [un]set tbproperties") { + withTable("delta_test") { + sql("CREATE TABLE delta_test (v1 int, v2 string) USING delta") + + sql(""" + |ALTER TABLE delta_test + |SET TBLPROPERTIES ( + | 'delta.checkpointInterval' = '20', + | 'key' = 'value' + |)""".stripMargin) + checkLastOperation( + spark.sessionState.catalog.defaultTablePath(TableIdentifier("delta_test")).toString, + Seq("SET TBLPROPERTIES", """{"delta.checkpointInterval":"20","key":"value"}"""), + Seq($"operation", $"operationParameters.properties")) + + sql("ALTER TABLE delta_test UNSET TBLPROPERTIES ('key')") + checkLastOperation( + spark.sessionState.catalog.defaultTablePath(TableIdentifier("delta_test")).toString, + Seq("UNSET TBLPROPERTIES", """["key"]""", "true"), + Seq($"operation", $"operationParameters.properties", $"operationParameters.ifExists")) + } + } + + testWithFlag("operations - add columns") { + withTable("delta_test") { + sql("CREATE TABLE delta_test (v1 int, v2 string) USING delta") + + sql("ALTER TABLE delta_test ADD COLUMNS (v3 long, v4 int AFTER v1)") + val column3 = """{"name":"v3","type":"long","nullable":true,"metadata":{}}""" + val column4 = """{"name":"v4","type":"integer","nullable":true,"metadata":{}}""" + checkLastOperation( + spark.sessionState.catalog.defaultTablePath(TableIdentifier("delta_test")).toString, + Seq("ADD COLUMNS", + s"""[{"column":$column3},{"column":$column4,"position":"AFTER v1"}]"""), + Seq($"operation", $"operationParameters.columns")) + } + } + + testWithFlag("operations - change column") { + withTable("delta_test") { + sql("CREATE TABLE delta_test (v1 int, v2 string) USING delta") + + sql("ALTER TABLE delta_test CHANGE COLUMN v1 v1 integer AFTER v2") + checkLastOperation( + spark.sessionState.catalog.defaultTablePath(TableIdentifier("delta_test")).toString, + Seq("CHANGE COLUMN", + s"""{"name":"v1","type":"integer","nullable":true,"metadata":{}}""", + "AFTER v2"), + Seq($"operation", $"operationParameters.column", $"operationParameters.position")) + } + } + + test("operations - upgrade protocol") { + withTempDir { path => + val log = DeltaLog.forTable(spark, path) + log.ensureLogDirectoryExist() + log.store.write( + FileNames.deltaFile(log.logPath, 0), + Iterator(Metadata(schemaString = spark.range(1).schema.json).json, Protocol(1, 1).json)) + log.update() + log.upgradeProtocol() + checkLastOperation( + path.toString, + Seq("UPGRADE PROTOCOL", + s"""{"minReaderVersion":${Action.readerVersion},""" + + s""""minWriterVersion":${Action.writerVersion}}"""), + Seq($"operation", $"operationParameters.newProtocol")) + } } testWithFlag("operations - insert append with partition columns") { @@ -222,7 +448,7 @@ trait DescribeDeltaHistorySuiteBase Seq(1, 2, 3).toDF().write.format("delta").mode("append").save(tempDir.toString) } - assert(getHistory(tempDir).count() === 2) + assert(spark.sql(s"DESCRIBE HISTORY delta.`$tempDir`").count() === 2) checkLastOperation(tempDir, Seq("WRITE", "Append")) } @@ -241,8 +467,11 @@ trait DescribeDeltaHistorySuiteBase Seq(4).toDF().write.format("delta").mode("overwrite").save(tempDir) } - val ans = getHistory(tempDir).as[CommitInfo].collect() + val ans = io.delta.tables.DeltaTable.forPath(spark, tempDir).history().as[CommitInfo].collect() assert(ans.map(_.version) === Seq(Some(4), Some(3), Some(2), Some(1), Some(0))) + + val ans2 = sql(s"DESCRIBE HISTORY delta.`$tempDir`").as[CommitInfo].collect() + assert(ans2.map(_.version) === Seq(Some(4), Some(3), Some(2), Some(1), Some(0))) } test("read version") { @@ -266,7 +495,7 @@ trait DescribeDeltaHistorySuiteBase withSQLConf(DeltaSQLConf.DELTA_COMMIT_INFO_ENABLED.key -> "false") { Seq(5).toDF().write.format("delta").mode("append").save(tempDir) // readVersion = None } - val ans = getHistory(tempDir).as[CommitInfo].collect() + val ans = sql(s"DESCRIBE HISTORY delta.`$tempDir`").as[CommitInfo].collect() assert(ans.map(x => x.version.get -> x.readVersion) === Seq(5 -> None, 4 -> Some(1), 3 -> Some(2), 2 -> Some(1), 1 -> Some(0), 0 -> None)) } @@ -278,15 +507,6 @@ trait DescribeDeltaHistorySuiteBase Seq($"operation", $"operationParameters.mode", $"operationParameters.partitionBy")) } - testWithFlag("describe history with delta table identifier") { - val tempDir = Utils.createTempDir().toString - Seq(1, 2, 3).toDF().write.format("delta").save(tempDir) - Seq(4, 5, 6).toDF().write.format("delta").mode("overwrite").save(tempDir) - val df = sql(s"DESCRIBE HISTORY delta.`$tempDir` LIMIT 1") - checkAnswer(df.select("operation", "operationParameters.mode"), - Seq(Row("WRITE", "Overwrite"))) - } - test("using on non delta") { withTempDir { basePath => val e = intercept[AnalysisException] { diff --git a/src/test/scala/org/apache/spark/sql/delta/HiveConvertToDeltaSuite.scala b/src/test/scala/org/apache/spark/sql/delta/HiveConvertToDeltaSuite.scala new file mode 100644 index 00000000000..a9d81f82451 --- /dev/null +++ b/src/test/scala/org/apache/spark/sql/delta/HiveConvertToDeltaSuite.scala @@ -0,0 +1,109 @@ +/* + * Copyright (2020) 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.test.DeltaHiveTest + +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.test.SQLTestUtils + +abstract class HiveConvertToDeltaSuiteBase + extends ConvertToDeltaHiveTableTests + with SQLTestUtils { + + override protected def convertToDelta( + identifier: String, + partitionSchema: Option[String] = None): Unit = { + if (partitionSchema.isEmpty) { + sql(s"convert to delta $identifier") + } else { + val stringSchema = partitionSchema.get + sql(s"convert to delta $identifier partitioned by ($stringSchema) ") + } + } + + override protected def verifyExternalCatalogMetadata(tableName: String): Unit = { + val catalogTable = spark.sessionState.catalog.externalCatalog.getTable("default", tableName) + // Hive automatically adds some properties + val cleanProps = catalogTable.properties.filterKeys(_ != "transient_lastDdlTime") + // We can't alter the schema in the catalog at the moment :( + assert(cleanProps.isEmpty, + s"Table properties weren't empty for table $tableName: $cleanProps") + } + + test("convert a Hive based parquet table") { + val tbl = "hive_parquet" + withTable(tbl) { + sql( + s""" + |CREATE TABLE $tbl (id int, str string) + |PARTITIONED BY (part string) + |STORED AS PARQUET + """.stripMargin) + + sql(s"insert into $tbl VALUES (1, 'a', 1)") + + val catalogTable = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tbl)) + assert(catalogTable.provider === Some("hive")) + assert(catalogTable.storage.serde.exists(_.contains("parquet"))) + + convertToDelta(tbl, Some("part string")) + + checkAnswer( + sql(s"select * from delta.`${getPathForTableName(tbl)}`"), + Row(1, "a", "1")) + + verifyExternalCatalogMetadata(tbl) + val updatedTable = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tbl)) + assert(updatedTable.provider === Some("delta")) + } + } + + test("convert a Hive based external parquet table") { + val tbl = "hive_parquet" + withTempDir { dir => + withTable(tbl) { + sql( + s""" + |CREATE EXTERNAL TABLE $tbl (id int, str string) + |PARTITIONED BY (part string) + |STORED AS PARQUET + |LOCATION '${dir.getCanonicalPath}' + """.stripMargin) + sql(s"insert into $tbl VALUES (1, 'a', 1)") + + val catalogTable = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tbl)) + assert(catalogTable.provider === Some("hive")) + assert(catalogTable.storage.serde.exists(_.contains("parquet"))) + + convertToDelta(tbl, Some("part string")) + + checkAnswer( + sql(s"select * from delta.`${dir.getCanonicalPath}`"), + Row(1, "a", "1")) + + verifyExternalCatalogMetadata(tbl) + val updatedTable = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tbl)) + assert(updatedTable.provider === Some("delta")) + } + } + } +} + +class HiveConvertToDeltaSuite extends HiveConvertToDeltaSuiteBase with DeltaHiveTest diff --git a/src/test/scala/org/apache/spark/sql/delta/HiveDeltaDDLSuite.scala b/src/test/scala/org/apache/spark/sql/delta/HiveDeltaDDLSuite.scala new file mode 100644 index 00000000000..25932323823 --- /dev/null +++ b/src/test/scala/org/apache/spark/sql/delta/HiveDeltaDDLSuite.scala @@ -0,0 +1,37 @@ +/* + * Copyright (2020) 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.test.DeltaHiveTest + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.hive.test.TestHiveSingleton + +abstract class HiveDeltaDDLSuiteBase + extends DeltaDDLTestBase { + + override protected def verifyDescribeTable(tblName: String): Unit = { + val res = sql(s"DESCRIBE TABLE $tblName").collect() + assert(res.takeRight(2).map(_.getString(1)) === Seq("name", "dept")) + } + + override protected def verifyNullabilityFailure(exception: AnalysisException): Unit = { + exception.getMessage.contains("not supported for changing column") + } +} + +class HiveDeltaDDLSuite extends HiveDeltaDDLSuiteBase with DeltaHiveTest diff --git a/src/test/scala/org/apache/spark/sql/delta/HiveDeltaNotSupportedDDLSuite.scala b/src/test/scala/org/apache/spark/sql/delta/HiveDeltaNotSupportedDDLSuite.scala new file mode 100644 index 00000000000..5d8e4d8107f --- /dev/null +++ b/src/test/scala/org/apache/spark/sql/delta/HiveDeltaNotSupportedDDLSuite.scala @@ -0,0 +1,23 @@ +/* + * Copyright (2020) 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.test.DeltaHiveTest + +import org.apache.spark.sql.hive.test.TestHiveSingleton + +class HiveDeltaNotSupportedDDLSuite extends DeltaNotSupportedDDLBase with DeltaHiveTest diff --git a/src/test/scala/org/apache/spark/sql/delta/LogStoreSuite.scala b/src/test/scala/org/apache/spark/sql/delta/LogStoreSuite.scala index 8050ef1b28d..7d1e2ec2433 100644 --- a/src/test/scala/org/apache/spark/sql/delta/LogStoreSuite.scala +++ b/src/test/scala/org/apache/spark/sql/delta/LogStoreSuite.scala @@ -22,6 +22,7 @@ import java.net.URI import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.delta.DeltaOperations.ManualUpdate +import org.apache.spark.sql.delta.DeltaTestUtils.OptimisticTxnTestHelper import org.apache.spark.sql.delta.actions.AddFile import org.apache.spark.sql.delta.storage._ import org.apache.hadoop.fs.{Path, RawLocalFileSystem} @@ -110,16 +111,16 @@ abstract class LogStoreSuiteBase extends QueryTest test("simple log store test") { val tempDir = Utils.createTempDir() - val log1 = DeltaLog(spark, new Path(tempDir.getCanonicalPath)) + val log1 = DeltaLog.forTable(spark, new Path(tempDir.getCanonicalPath)) assert(log1.store.getClass.getName == logStoreClassName) val txn = log1.startTransaction() val file = AddFile("1", Map.empty, 1, 1, true) :: Nil - txn.commit(file, ManualUpdate) + txn.commitManually(file: _*) log1.checkpoint() DeltaLog.clearCache() - val log2 = DeltaLog(spark, new Path(tempDir.getCanonicalPath)) + val log2 = DeltaLog.forTable(spark, new Path(tempDir.getCanonicalPath)) assert(log2.store.getClass.getName == logStoreClassName) assert(log2.lastCheckpoint.map(_.version) === Some(0L)) diff --git a/src/test/scala/org/apache/spark/sql/delta/MergeIntoAccumulatorSuite.scala b/src/test/scala/org/apache/spark/sql/delta/MergeIntoAccumulatorSuite.scala new file mode 100644 index 00000000000..64df71408fb --- /dev/null +++ b/src/test/scala/org/apache/spark/sql/delta/MergeIntoAccumulatorSuite.scala @@ -0,0 +1,79 @@ +/* + * Copyright (2020) 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.atomic.AtomicReference + +import scala.collection.JavaConverters._ + +import org.apache.spark.sql.delta.commands.MergeIntoCommand +import org.apache.spark.sql.delta.test.DeltaSQLCommandTest + +import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.status.TaskDataWrapper +import org.apache.spark.util.JsonProtocol + +/** + * Tests how the accumulator used by the MERGE command reacts with other Spark components such as + * Spark UI. These tests stay in a separated file so that we can use the package name + * `org.apache.spark.sql.delta` to access `private[spark]` APIs. + */ +class MergeIntoAccumulatorSuite extends QueryTest with SharedSparkSession with DeltaSQLCommandTest { + + import testImplicits._ + + private def runTestMergeCommand(): Unit = { + // Run a simple merge command + withTempView("source") { + withTempDir { tempDir => + val tempPath = tempDir.getCanonicalPath + Seq((1, 1), (0, 3)).toDF("key", "value").createOrReplaceTempView("source") + Seq((2, 2), (1, 4)).toDF("key", "value").write.format("delta").save(tempPath) + spark.sql(s""" + |MERGE INTO delta.`$tempPath` target + |USING source src + |ON src.key = target.key + |WHEN MATCHED THEN UPDATE SET * + |WHEN NOT MATCHED THEN INSERT * + |""".stripMargin) + } + } + } + + test("accumulators used by MERGE should not be tracked by Spark UI") { + runTestMergeCommand() + + // Make sure all Spark events generated by the above command have been processed + spark.sparkContext.listenerBus.waitUntilEmpty(30000) + + val store = spark.sparkContext.statusStore.store + val iter = store.view(classOf[TaskDataWrapper]).closeableIterator() + try { + // Collect all accumulator names tracked by Spark UI. + val accumNames = iter.asScala.toVector.flatMap { task => + task.accumulatorUpdates.map(_.name) + }.toSet + // Verify accumulators used by MergeIntoCommand are not tracked. + assert(!accumNames.contains(MergeIntoCommand.TOUCHED_FILES_ACCUM_NAME)) + } finally { + iter.close() + } + } + +} diff --git a/src/test/scala/org/apache/spark/sql/delta/MergeIntoSQLSuite.scala b/src/test/scala/org/apache/spark/sql/delta/MergeIntoSQLSuite.scala index b486e9ef171..01d52e2549b 100644 --- a/src/test/scala/org/apache/spark/sql/delta/MergeIntoSQLSuite.scala +++ b/src/test/scala/org/apache/spark/sql/delta/MergeIntoSQLSuite.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.types.StructType class MergeIntoSQLSuite extends MergeIntoSuiteBase with DeltaSQLCommandTest { import testImplicits._ + private def basicMergeStmt( target: String, source: String, diff --git a/src/test/scala/org/apache/spark/sql/delta/MergeIntoScalaSuite.scala b/src/test/scala/org/apache/spark/sql/delta/MergeIntoScalaSuite.scala index 98e346001b3..e70b14f0bfd 100644 --- a/src/test/scala/org/apache/spark/sql/delta/MergeIntoScalaSuite.scala +++ b/src/test/scala/org/apache/spark/sql/delta/MergeIntoScalaSuite.scala @@ -280,17 +280,55 @@ class MergeIntoScalaSuite extends MergeIntoSuiteBase with DeltaSQLCommandTest { .whenMatched().updateAll() .whenNotMatched().insertAll() .execute() - assert(t.toDF.schema == StructType.fromDDL("id LONG, newCol1 STRING")) + // assert(t.toDF.schema == StructType.fromDDL("id LONG, newCol1 STRING")) + + // SC-35564 - ideally this shouldn't throw an error, but right now we can't fix it without + // causing a regression. + val ex = intercept[Exception] { + t.merge(Seq((12L, "newVal12")).toDF("id", "newCol2").as("s"), "t.id = s.id") + .whenMatched().updateAll() + .whenNotMatched().insertAll() + .execute() + } + ex.getMessage.contains("schema of your Delta table has changed in an incompatible way") + } + } + } - // In order to work, the next merge will need to pick up the updated schema from above, - // rather than the schema from when `t` was originally created. - t.merge(Seq((12L, "newVal12")).toDF("id", "newCol2").as("s"), "t.id = s.id") - .whenMatched().updateAll() - .whenNotMatched().insertAll() - .execute() + test("merge without table alias") { + withTempDir { dir => + val location = dir.getAbsolutePath + Seq((1, 1, 1), (2, 2, 2)).toDF("part", "id", "n").write + .format("delta") + .partitionBy("part") + .save(location) + val table = io.delta.tables.DeltaTable.forPath(spark, location) + val data1 = Seq((2, 2, 4, 2), (9, 3, 6, 9), (3, 3, 9, 3)).toDF("part", "id", "n", "part2") + table.alias("t").merge( + data1, + "t.part = part2") + .whenMatched().updateAll() + .whenNotMatched().insertAll() + .execute() + } + } - assert(t.toDF.schema == StructType.fromDDL("id LONG, newCol1 STRING, newCol2 STRING")) - } + test("merge without table alias with pre-computed condition") { + withTempDir { dir => + val location = dir.getAbsolutePath + Seq((1, 1, 1), (2, 2, 2)).toDF("part", "id", "x").write + .format("delta") + .partitionBy("part") + .save(location) + val table = io.delta.tables.DeltaTable.forPath(spark, location) + val tableDf = table.toDF + val data1 = Seq((2, 2, 4), (2, 3, 6), (3, 3, 9)).toDF("part", "id", "x") + table.merge( + data1, + tableDf("part") === data1("part") && tableDf("id") === data1("id")) + .whenMatched().updateAll() + .whenNotMatched().insertAll() + .execute() } } diff --git a/src/test/scala/org/apache/spark/sql/delta/MergeIntoSuiteBase.scala b/src/test/scala/org/apache/spark/sql/delta/MergeIntoSuiteBase.scala index 56a1ceb0baf..abd561f77a6 100644 --- a/src/test/scala/org/apache/spark/sql/delta/MergeIntoSuiteBase.scala +++ b/src/test/scala/org/apache/spark/sql/delta/MergeIntoSuiteBase.scala @@ -20,9 +20,6 @@ import java.io.File import java.lang.{Integer => JInt} import java.util.Locale -import scala.collection.JavaConverters._ - -import org.apache.spark.sql.delta.commands.MergeIntoCommand import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.scalatest.BeforeAndAfterEach @@ -30,9 +27,7 @@ import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} import org.apache.spark.sql.types.{IntegerType, MapType, StringType, StructType} -import org.apache.spark.status.api.v1.AccumulableInfo import org.apache.spark.util.Utils -import org.apache.spark.util.kvstore.KVStore abstract class MergeIntoSuiteBase extends QueryTest @@ -1798,52 +1793,6 @@ abstract class MergeIntoSuiteBase insert("(id, part1, part2) VALUES (id, part1, part2)") ) - test("accumulators used by MERGE should not be tracked by Spark UI") { - // Run a simple merge command - withTable("source") { - append(Seq((2, 2), (1, 4)).toDF("key2", "value")) - Seq((1, 1), (0, 3)).toDF("key1", "value").createOrReplaceTempView("source") - - executeMerge( - target = s"delta.`$tempPath`", - source = "source src", - condition = "src.key1 = key2", - update = "key2 = 20 + key1, value = 20 + src.value", - insert = "(key2, value) VALUES (key1 - 10, src.value + 10)") - - checkAnswer(readDeltaTable(tempPath), - Row(2, 2) :: // No change - Row(21, 21) :: // Update - Row(-10, 13) :: // Insert - Nil) - } - - // Make sure all Spark events generated by the above command have been processed - spark.sparkContext.listenerBus.waitUntilEmpty(30000) - - // Use Reflection to get `SparkContext.statusStore.store` and access all accumulators stored - // with Spark tasks, so that this test can be run with Apache Spark. - val statusStore = - spark.sparkContext.getClass.getMethod("statusStore").invoke(spark.sparkContext) - val store = statusStore.getClass.getMethod("store").invoke(statusStore).asInstanceOf[KVStore] - // scalastyle:off classforname - val iter = store - .view(Class.forName("org.apache.spark.status.TaskDataWrapper")) - .closeableIterator() - // scalastyle:on classforname - try { - // Collect all accumulator names tracked by Spark UI. - val accumNames = iter.asScala.toVector.flatMap { task => - task.getClass.getMethod("accumulatorUpdates").invoke(task) - .asInstanceOf[Seq[AccumulableInfo]].map(_.name) - }.toSet - // Verify accumulators used by MergeIntoCommand are not tracked. - assert(!accumNames.contains(MergeIntoCommand.TOUCHED_FILES_ACCUM_NAME)) - } finally { - iter.close() - } - } - protected def testMatchedOnlyOptimization( name: String)( source: Seq[(Int, Int)], diff --git a/src/test/scala/org/apache/spark/sql/delta/OptimisticTransactionSuite.scala b/src/test/scala/org/apache/spark/sql/delta/OptimisticTransactionSuite.scala index 7d58ad441ad..d3afdf27d7d 100644 --- a/src/test/scala/org/apache/spark/sql/delta/OptimisticTransactionSuite.scala +++ b/src/test/scala/org/apache/spark/sql/delta/OptimisticTransactionSuite.scala @@ -17,7 +17,9 @@ 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, AddFile, FileAction, Metadata, RemoveFile, SetTransaction} +import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.hadoop.fs.Path import org.apache.spark.sql.{QueryTest, Row} @@ -34,8 +36,8 @@ class OptimisticTransactionSuite extends QueryTest with SharedSparkSession { test("block append against metadata change") { withTempDir { tempDir => val log = DeltaLog(spark, new Path(tempDir.getCanonicalPath)) - // Initialize the log. ManualUpdate is just a no-op placeholder. - log.startTransaction().commit(Nil, ManualUpdate) + // Initialize the log. + log.startTransaction().commitManually() val txn = log.startTransaction() val winningTxn = log.startTransaction() @@ -49,8 +51,8 @@ class OptimisticTransactionSuite extends QueryTest with SharedSparkSession { test("block read+append against append") { withTempDir { tempDir => val log = DeltaLog(spark, new Path(tempDir.getCanonicalPath)) - // Initialize the log. ManualUpdate is just a no-op placeholder. - log.startTransaction().commit(Metadata() :: Nil, ManualUpdate) + // Initialize the log. + log.startTransaction().commitManually() val txn = log.startTransaction() // reads the table @@ -67,8 +69,8 @@ class OptimisticTransactionSuite extends QueryTest with SharedSparkSession { test("allow blind-append against any 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().commit(addA :: Nil, ManualUpdate) + // Initialize the log and add data. + log.startTransaction().commitManually(addA) val txn = log.startTransaction() val winningTxn = log.startTransaction() @@ -82,7 +84,7 @@ class OptimisticTransactionSuite extends QueryTest with SharedSparkSession { 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(addA :: Nil, ManualUpdate) + log.startTransaction().commitManually(addA) val txn = log.startTransaction() txn.filterFiles() @@ -644,7 +646,8 @@ class OptimisticTransactionSuite extends QueryTest with SharedSparkSession { 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(actionWithMetaData, ManualUpdate) + log.startTransaction().commit(Seq(Metadata(partitionColumns = partitionCols)), ManualUpdate) + log.startTransaction().commitManually(actionWithMetaData: _*) test(log) } } @@ -652,8 +655,8 @@ class OptimisticTransactionSuite extends QueryTest with SharedSparkSession { test("allow concurrent set-txns with different app ids") { 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(Nil, ManualUpdate) + // Initialize the log. + log.startTransaction().commitManually() val txn = log.startTransaction() txn.txnVersion("t1") @@ -668,8 +671,8 @@ class OptimisticTransactionSuite extends QueryTest with SharedSparkSession { test("block concurrent set-txns with the same app id") { 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(Nil, ManualUpdate) + // Initialize the log. + log.startTransaction().commitManually() val txn = log.startTransaction() txn.txnVersion("t1") @@ -681,4 +684,57 @@ class OptimisticTransactionSuite extends QueryTest with SharedSparkSession { } } } + + test("initial commit without metadata should fail") { + withTempDir { tempDir => + val log = DeltaLog(spark, new Path(tempDir.getCanonicalPath)) + val txn = log.startTransaction() + withSQLConf(DeltaSQLConf.DELTA_COMMIT_VALIDATION_ENABLED.key -> "true") { + val e = intercept[IllegalStateException] { + txn.commit(Nil, ManualUpdate) + } + 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") { + txn.commit(Nil, ManualUpdate) + assert(log.update().version === 0) + } + } + } + + test("initial commit with multiple metadata actions should fail") { + withTempDir { tempDir => + val log = DeltaLog(spark, new Path(tempDir.getAbsolutePath)) + val txn = log.startTransaction() + val e = intercept[AssertionError] { + txn.commit(Seq(Metadata(), Metadata()), ManualUpdate) + } + assert(e.getMessage.contains("Cannot change the metadata more than once in a transaction.")) + } + } + + test("AddFile with different partition schema compared to metadata should fail") { + withTempDir { tempDir => + val log = DeltaLog(spark, new Path(tempDir.getAbsolutePath)) + log.startTransaction().commit(Seq(Metadata(partitionColumns = Seq("col2"))), ManualUpdate) + withSQLConf(DeltaSQLConf.DELTA_COMMIT_VALIDATION_ENABLED.key -> "true") { + val e = intercept[IllegalStateException] { + log.startTransaction().commit(Seq(AddFile( + log.dataPath.toString, Map("col3" -> "1"), 12322, 0L, true, null, null)), ManualUpdate) + } + assert(e.getMessage === DeltaErrors.addFilePartitioningMismatchException( + Seq("col3"), Seq("col2")).getMessage) + } + // Try with commit validation turned off + withSQLConf(DeltaSQLConf.DELTA_STATE_RECONSTRUCTION_VALIDATION_ENABLED.key -> "false", + DeltaSQLConf.DELTA_COMMIT_VALIDATION_ENABLED.key -> "false") { + log.startTransaction().commit(Seq(AddFile( + log.dataPath.toString, Map("col3" -> "1"), 12322, 0L, true, null, null)), ManualUpdate) + assert(log.update().version === 1) + } + } + } } diff --git a/src/test/scala/org/apache/spark/sql/delta/S3LikeLocalFileSystem.scala b/src/test/scala/org/apache/spark/sql/delta/S3LikeLocalFileSystem.scala new file mode 100644 index 00000000000..d9a88a9738e --- /dev/null +++ b/src/test/scala/org/apache/spark/sql/delta/S3LikeLocalFileSystem.scala @@ -0,0 +1,43 @@ +/* + * Copyright (2020) 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.net.URI + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.RawLocalFileSystem + +/** + * A local filesystem on scheme s3. Useful for testing paths on non-defualt schemes. + */ +class S3LikeLocalFileSystem extends RawLocalFileSystem { + private var uri: URI = _ + override def getScheme: String = "s3" + + override def initialize(name: URI, conf: Configuration): Unit = { + uri = URI.create(name.getScheme + ":///") + super.initialize(name, conf) + } + + override def getUri(): URI = if (uri == null) { + // RawLocalFileSystem's constructor will call this one before `initialize` is called. + // Just return the super's URI to avoid NPE. + super.getUri + } else { + uri + } +} diff --git a/src/test/scala/org/apache/spark/sql/delta/SchemaValidationSuite.scala b/src/test/scala/org/apache/spark/sql/delta/SchemaValidationSuite.scala new file mode 100644 index 00000000000..afd1d214c95 --- /dev/null +++ b/src/test/scala/org/apache/spark/sql/delta/SchemaValidationSuite.scala @@ -0,0 +1,408 @@ +/* + * Copyright (2020) 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.CountDownLatch + +import org.apache.spark.sql.delta.test.DeltaSQLCommandTest + +import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SparkSession} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.test.SharedSparkSession + +trait SchemaValidationSuiteBase extends QueryTest with SharedSparkSession with DeltaSQLCommandTest { + + def checkMergeException(e: Exception, col: String): Unit = { + assert(e.isInstanceOf[MetadataChangedException]) + assert(e.getMessage.contains( + "The metadata of the Delta table has been changed by a concurrent update")) + } +} + +/** + * This Suite tests the behavior of Delta commands when a schema altering commit is run after the + * command completes analysis but before the command starts the transaction. We want to make sure + * That we do not corrupt tables. + */ +class SchemaValidationSuite extends SchemaValidationSuiteBase { + + class BlockingRule( + blockActionLatch: CountDownLatch, + startConcurrentUpdateLatch: CountDownLatch) extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = { + startConcurrentUpdateLatch.countDown() + blockActionLatch.await() + plan + } + } + + /** + * Blocks the thread with the help of an optimizer rule until end of scope. + * We need two latches to ensure that the thread executing the query is blocked until + * the other thread concurrently updates the metadata. `blockActionLatch` blocks the action + * until it is counted down by the thread updating the metadata. `startConcurrentUpdateLatch` + * will block the concurrent update to happen until it is counted down by the action reaches the + * optimizer rule. + */ + private def withBlockedExecution( + t: Thread, + blockActionLatch: CountDownLatch, + startConcurrentUpdateLatch: CountDownLatch)(f: => Unit): Unit = { + t.start() + startConcurrentUpdateLatch.await() + try { + f + } finally { + blockActionLatch.countDown() + t.join() + } + } + + def cloneSession(spark: SparkSession): SparkSession = { + val cloneMethod = classOf[SparkSession].getDeclaredMethod("cloneSession") + cloneMethod.setAccessible(true) + val clonedSession = cloneMethod.invoke(spark).asInstanceOf[SparkSession] + clonedSession + } + + /** + * Common base method for both the path based and table name based tests. + */ + private def testConcurrentChangeBase(identifier: String)( + createTable: (SparkSession, String) => Unit, + actionToTest: (SparkSession, String) => Unit, + concurrentChange: (SparkSession, String) => Unit): Unit = { + createTable(spark, identifier) + + // Clone the session to run the query in a separate thread. + val newSession = cloneSession(spark) + val blockActionLatch = new CountDownLatch(1) + val startConcurrentUpdateLatch = new CountDownLatch(1) + val rule = new BlockingRule(blockActionLatch, startConcurrentUpdateLatch) + newSession.experimental.extraOptimizations :+= rule + + var actionException: Exception = null + val actionToTestThread = new Thread() { + override def run(): Unit = { + try { + actionToTest(newSession, identifier) + } catch { + case e: Exception => + actionException = e + } + } + } + withBlockedExecution(actionToTestThread, blockActionLatch, startConcurrentUpdateLatch) { + concurrentChange(spark, identifier) + } + if (actionException != null) { + throw actionException + } + } + + /** + * tests the behavior of concurrent changes to schema on a blocked command. + * @param testName - name of the test + * @param createTable - method that creates a table given an identifier and spark session. + * @param actionToTest - the method we want to test. + * @param concurrentChange - the concurrent query that updates the schema of the table + * + * All the above methods take SparkSession and the table path as parameters + */ + def testConcurrentChange(testName: String)( + createTable: (SparkSession, String) => Unit, + actionToTest: (SparkSession, String) => Unit, + concurrentChange: (SparkSession, String) => Unit): Unit = { + + test(testName) { + withTempDir { tempDir => + testConcurrentChangeBase(tempDir.getCanonicalPath)( + createTable, + actionToTest, + concurrentChange + ) + } + } + } + + /** + * tests the behavior of concurrent changes pf schema on a blocked command with metastore tables. + * @param testName - name of the test + * @param createTable - method that creates a table given an identifier and spark session. + * @param actionToTest - the method we want to test. + * @param concurrentChange - the concurrent query that updates the schema of the table + * + * All the above methods take SparkSession and the table name as parameters + */ + def testConcurrentChangeWithTable(testName: String)( + createTable: (SparkSession, String) => Unit, + actionToTest: (SparkSession, String) => Unit, + concurrentChange: (SparkSession, String) => Unit): Unit = { + + val tblName = "metastoreTable" + test(testName) { + withTable(tblName) { + testConcurrentChangeBase(tblName)( + createTable, + actionToTest, + concurrentChange + ) + } + } + } + + /** + * Creates a method to remove a column from the table by taking column as an argument. + */ + def dropColFromSampleTable(col: String): (SparkSession, String) => Unit = { + (spark: SparkSession, tblPath: String) => { + spark.read.format("delta").load(tblPath) + .drop(col) + .write + .format("delta") + .mode("overwrite") + .option("overwriteSchema", "true") + .save(tblPath) + } + } + + /** + * Adding a column to the schema will result in the blocked thread appending to the table + * with null values for the new column. + */ + testConcurrentChange("write - add a column concurrently")( + createTable = (spark: SparkSession, tblPath: String) => { + spark.range(10).write.format("delta").save(tblPath) + }, + actionToTest = (spark: SparkSession, tblPath: String) => { + spark.range(11, 20).write.format("delta") + .mode("append") + .save(tblPath) + + val appendedCol2Values = spark.read.format("delta") + .load(tblPath) + .filter(col("id") <= 20) + .select("col2") + .distinct() + .collect() + .toList + assert(appendedCol2Values == List(Row(null))) + }, + concurrentChange = (spark: SparkSession, tblPath: String) => { + spark.range(21, 30).withColumn("col2", lit(2)).write + .format("delta") + .mode("append") + .option("mergeSchema", "true") + .save(tblPath) + } + ) + + /** + * Removing a column while a query is in running should throw an analysis + * exception + */ + testConcurrentChange("write - remove a column concurrently")( + createTable = (spark: SparkSession, tblPath: String) => { + spark.range(10).withColumn("col2", lit(1)) + .write + .format("delta") + .save(tblPath) + }, + actionToTest = (spark: SparkSession, tblPath: String) => { + val e = intercept[AnalysisException] { + spark.range(11, 20) + .withColumn("col2", lit(1)).write.format("delta") + .mode("append") + .save(tblPath) + } + assert(e.getMessage.contains( + "A schema mismatch detected when writing to the Delta table")) + }, + concurrentChange = dropColFromSampleTable("col2") + ) + + /** + * Removing a column while performing a delete should be caught while + * writing the deleted files(i.e files with rows that were not deleted). + */ + testConcurrentChange("delete - remove a column concurrently")( + createTable = (spark: SparkSession, tblPath: String) => { + spark.range(10).withColumn("col2", lit(1)) + .write + .format("delta") + .save(tblPath) + }, + actionToTest = (spark: SparkSession, tblPath: String) => { + val deltaTable = io.delta.tables.DeltaTable.forPath(spark, tblPath) + val e = intercept[Exception] { + deltaTable.delete(col("id") === 1) + } + assert(e.getMessage.contains(s"Can't resolve column col2")) + }, + concurrentChange = dropColFromSampleTable("col2") + ) + + /** + * Removing a column(referenced in condition) while performing a delete will + * result in a no-op. + */ + testConcurrentChange("delete - remove condition column concurrently")( + createTable = (spark: SparkSession, tblPath: String) => { + spark.range(10).withColumn("col2", lit(1)) + .repartition(2) + .write + .format("delta") + .save(tblPath) + }, + actionToTest = (spark: SparkSession, tblPath: String) => { + val deltaTable = io.delta.tables.DeltaTable.forPath(spark, tblPath) + deltaTable.delete(col("id") === 1) + // check if delete is no-op + checkAnswer( + deltaTable.history.select("operation"), + Seq(Row("WRITE"), Row("WRITE"))) + }, + concurrentChange = dropColFromSampleTable("id") + ) + + /** + * An update command that has to rewrite files will have the old schema, + * we catch the outdated schema during the write. + */ + testConcurrentChange("update - remove a column concurrently")( + createTable = (spark: SparkSession, tblPath: String) => { + spark.range(10).withColumn("col2", lit(1)) + .write + .format("delta") + .save(tblPath) + }, + actionToTest = (spark: SparkSession, tblPath: String) => { + val deltaTable = io.delta.tables.DeltaTable.forPath(spark, tblPath) + val e = intercept[AnalysisException] { + deltaTable.update(col("id") =!= 1, Map("col2" -> lit(-1))) + } + assert(e.getMessage.contains(s"Can't resolve column col2")) + }, + concurrentChange = dropColFromSampleTable("col2") + ) + + /** + * Removing a column(referenced in condition) while performing a update will + * result in a no-op. + */ + testConcurrentChange("update - remove condition column concurrently")( + createTable = (spark: SparkSession, tblPath: String) => { + spark.range(10).withColumn("col2", lit(1)) + .repartition(2) + .write + .format("delta") + .save(tblPath) + }, + actionToTest = (spark: SparkSession, tblPath: String) => { + val deltaTable = io.delta.tables.DeltaTable.forPath(spark, tblPath) + deltaTable.update(col("id") === 1, Map("id" -> lit("2"))) + // check if update is no-op + checkAnswer( + deltaTable.history.select("operation"), + Seq(Row("WRITE"), Row("WRITE"))) + }, + concurrentChange = dropColFromSampleTable("id") + ) + + /** + * Concurrently drop column in merge condition. Merge command detects the schema change while + * resolving the target and throws an AnalysisException + */ + testConcurrentChange("merge - remove a column in merge condition concurrently")( + createTable = (spark: SparkSession, tblPath: String) => { + spark.range(10).withColumn("col2", lit(1)) + .write + .format("delta") + .save(tblPath) + }, + actionToTest = (spark: SparkSession, tblPath: String) => { + val deltaTable = io.delta.tables.DeltaTable.forPath(spark, tblPath) + val sourceDf = spark.range(10).withColumn("col2", lit(2)) + val e = intercept[Exception] { + deltaTable.as("t1") + .merge(sourceDf.as("t2"), "t1.id == t2.id") + .whenNotMatched() + .insertAll() + .whenMatched() + .updateAll() + .execute() + } + checkMergeException(e, "id") + }, + concurrentChange = dropColFromSampleTable("id") + ) + + /** + * Concurrently drop column not in merge condition but in target. Merge command detects the schema + * change while resolving the target and throws an AnalysisException + */ + testConcurrentChange("merge - remove a column not in merge condition concurrently")( + createTable = (spark: SparkSession, tblPath: String) => { + spark.range(10).withColumn("col2", lit(1)) + .write + .format("delta") + .save(tblPath) + }, + actionToTest = (spark: SparkSession, tblPath: String) => { + val deltaTable = io.delta.tables.DeltaTable.forPath(spark, tblPath) + val sourceDf = spark.range(10).withColumn("col2", lit(2)) + val e = intercept[Exception] { + deltaTable.as("t1") + .merge(sourceDf.as("t2"), "t1.id == t2.id") + .whenNotMatched() + .insertAll() + .whenMatched() + .updateAll() + .execute() + } + checkMergeException(e, "col2") + }, + concurrentChange = dropColFromSampleTable("col2") + ) + + /** + * Alter table to add a column and at the same time add a column concurrently. + */ + testConcurrentChangeWithTable("alter table add column - remove column and add same column")( + createTable = (spark: SparkSession, tblName: String) => { + spark.range(10).write.format("delta").saveAsTable(tblName) + }, + actionToTest = (spark: SparkSession, tblName: String) => { + val e = intercept[AnalysisException] { + spark.sql(s"ALTER TABLE `$tblName` ADD COLUMNS (col2 string)") + } + assert(e.getMessage.contains("Found duplicate column(s) in adding columns: col2")) + }, + concurrentChange = (spark: SparkSession, tblName: String) => { + spark.read.format("delta").table(tblName) + .withColumn("col2", lit(1)) + .write + .format("delta") + .option("overwriteSchema", "true") + .mode("overwrite") + .saveAsTable(tblName) + } + ) +} + diff --git a/src/test/scala/org/apache/spark/sql/delta/UpdateSQLSuite.scala b/src/test/scala/org/apache/spark/sql/delta/UpdateSQLSuite.scala new file mode 100644 index 00000000000..416bf34069e --- /dev/null +++ b/src/test/scala/org/apache/spark/sql/delta/UpdateSQLSuite.scala @@ -0,0 +1,62 @@ +/* + * Copyright (2020) 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.test.DeltaSQLCommandTest + +import org.apache.spark.sql.Row + +class UpdateSQLSuite extends UpdateSuiteBase with DeltaSQLCommandTest { + + import testImplicits._ + + test("explain") { + append(Seq((2, 2)).toDF("key", "value")) + val df = sql(s"EXPLAIN UPDATE delta.`$tempPath` SET key = 1, value = 2 WHERE key = 2") + val outputs = df.collect().map(_.mkString).mkString + assert(outputs.contains("Delta")) + assert(!outputs.contains("index") && !outputs.contains("ActionLog")) + // no change should be made by explain + checkAnswer(readDeltaTable(tempPath), Row(2, 2)) + } + + test("SC-11376: Update command should check target columns during analysis, same key") { + val targetDF = spark.read.json( + """ + {"a": {"c": {"d": 'random', "e": 'str'}, "g": 1}, "z": 10} + {"a": {"c": {"d": 'random2', "e": 'str2'}, "g": 2}, "z": 20}""" + .split("\n").toSeq.toDS()) + + testAnalysisException( + targetDF, + set = "z = 30" :: "z = 40" :: Nil, + errMsgs = "There is a conflict from these SET columns" :: Nil) + + testAnalysisException( + targetDF, + set = "a.c.d = 'rand'" :: "a.c.d = 'RANDOM2'" :: Nil, + errMsgs = "There is a conflict from these SET columns" :: Nil) + } + + override protected def executeUpdate( + target: String, + set: String, + where: String = null): Unit = { + val whereClause = Option(where).map(c => s"WHERE $c").getOrElse("") + sql(s"UPDATE $target SET $set $whereClause") + } +} diff --git a/src/test/scala/org/apache/spark/sql/delta/UpdateScalaSuite.scala b/src/test/scala/org/apache/spark/sql/delta/UpdateScalaSuite.scala index 0063778ee98..c96f4c5b1c3 100644 --- a/src/test/scala/org/apache/spark/sql/delta/UpdateScalaSuite.scala +++ b/src/test/scala/org/apache/spark/sql/delta/UpdateScalaSuite.scala @@ -18,11 +18,12 @@ package org.apache.spark.sql.delta import java.util.Locale +import org.apache.spark.sql.delta.test.DeltaSQLCommandTest import io.delta.tables.DeltaTableTestUtils import org.apache.spark.sql.{functions, Row} -class UpdateScalaSuite extends UpdateSuiteBase { +class UpdateScalaSuite extends UpdateSuiteBase with DeltaSQLCommandTest { import testImplicits._ diff --git a/src/test/scala/org/apache/spark/sql/delta/UpdateSuiteBase.scala b/src/test/scala/org/apache/spark/sql/delta/UpdateSuiteBase.scala index 36f06bc7907..b5bdc966c89 100644 --- a/src/test/scala/org/apache/spark/sql/delta/UpdateSuiteBase.scala +++ b/src/test/scala/org/apache/spark/sql/delta/UpdateSuiteBase.scala @@ -26,6 +26,7 @@ import org.apache.hadoop.fs.Path import org.scalatest.BeforeAndAfterEach import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -96,7 +97,7 @@ abstract class UpdateSuiteBase } Seq(true, false).foreach { isPartitioned => - test(s"basic update - delta table - Partition=$isPartitioned") { + test(s"basic update - Delta table by path - Partition=$isPartitioned") { withTable("deltaTable") { val partitions = if (isPartitioned) "key" :: Nil else Nil append(Seq((2, 2), (1, 4), (1, 1), (0, 3)).toDF("key", "value"), partitions) @@ -104,7 +105,30 @@ abstract class UpdateSuiteBase checkUpdate( condition = Some("key >= 1"), setClauses = "value = key + value, key = key + 1", - expectedResults = Row(0, 3) :: Row(2, 5) :: Row(2, 2) :: Row(3, 4) :: Nil) + expectedResults = Row(0, 3) :: Row(2, 5) :: Row(2, 2) :: Row(3, 4) :: Nil, + tableName = Some(s"delta.`$tempPath`")) + } + } + } + + Seq(true, false).foreach { isPartitioned => + test(s"basic update - Delta table by name - Partition=$isPartitioned") { + withTable("delta_table") { + val partitionByClause = if (isPartitioned) "PARTITIONED BY (key)" else "" + sql(s""" + |CREATE TABLE delta_table(key INT, value INT) + |USING delta + |OPTIONS('path'='$tempPath') + |$partitionByClause + """.stripMargin) + + append(Seq((2, 2), (1, 4), (1, 1), (0, 3)).toDF("key", "value")) + + checkUpdate( + condition = Some("key >= 1"), + setClauses = "value = key + value, key = key + 1", + expectedResults = Row(0, 3) :: Row(2, 5) :: Row(2, 2) :: Row(3, 4) :: Nil, + tableName = Some("delta_table")) } } } @@ -268,16 +292,6 @@ abstract class UpdateSuiteBase } } - test("Negative case - non-delta target") { - Seq((1, 1), (0, 3), (1, 5)).toDF("key1", "value") - .write.mode("overwrite").format("parquet").save(tempPath) - val e = intercept[AnalysisException] { - executeUpdate(target = s"delta.`$tempPath`", set = "key1 = 3") - }.getMessage - assert(e.contains("UPDATE destination only supports Delta sources") || - e.contains("is not a Delta table") || e.contains("Incompatible format")) - } - test("update cached table") { Seq((2, 2), (1, 4)).toDF("key", "value") .write.mode("overwrite").format("delta").save(tempPath) @@ -304,7 +318,84 @@ abstract class UpdateSuiteBase Row(99, -1) :: Row(100, -1) :: Row(101, -1) :: Row(102, -1) :: Nil, Some("tblName")) } - test("do not support subquery test") { + test("target columns can have db and table qualifiers") { + withTable("target") { + spark.read.json(""" + {"a": {"b.1": 1, "c.e": 'random'}, "d": 1} + {"a": {"b.1": 3, "c.e": 'string'}, "d": 2}""" + .split("\n").toSeq.toDS()).write.format("delta").saveAsTable("`target`") + + executeUpdate( + target = "target", + set = "`default`.`target`.a.`b.1` = -1, target.a.`c.e` = 'RANDOM'", + where = "d = 1") + + checkAnswer(spark.table("target"), + spark.read.json(""" + {"a": {"b.1": -1, "c.e": 'RANDOM'}, "d": 1} + {"a": {"b.1": 3, "c.e": 'string'}, "d": 2}""" + .split("\n").toSeq.toDS())) + } + } + + test("Negative case - non-delta target") { + Seq((1, 1), (0, 3), (1, 5)).toDF("key1", "value") + .write.mode("overwrite").format("parquet").save(tempPath) + val e = intercept[AnalysisException] { + executeUpdate(target = s"delta.`$tempPath`", set = "key1 = 3") + }.getMessage + assert(e.contains("UPDATE destination only supports Delta sources") || + e.contains("is not a Delta table") || e.contains("Incompatible format")) + } + + test("Negative case - check target columns during analysis") { + withTable("table") { + sql("CREATE TABLE table (s int, t string) USING delta PARTITIONED BY (s)") + var ae = intercept[AnalysisException] { + executeUpdate("table", set = "column_doesnt_exist = 'San Francisco'", where = "t = 'a'") + } + assert(ae.message.contains("cannot resolve")) + + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + executeUpdate(target = "table", set = "S = 1, T = 'b'", where = "T = 'a'") + ae = intercept[AnalysisException] { + executeUpdate(target = "table", set = "S = 1, s = 'b'", where = "s = 1") + } + assert(ae.message.contains("There is a conflict from these SET columns")) + } + + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + ae = intercept[AnalysisException] { + executeUpdate(target = "table", set = "S = 1", where = "t = 'a'") + } + assert(ae.message.contains("cannot resolve")) + + ae = intercept[AnalysisException] { + executeUpdate(target = "table", set = "S = 1, s = 'b'", where = "s = 1") + } + assert(ae.message.contains("cannot resolve")) + + // unresolved column in condition + ae = intercept[AnalysisException] { + executeUpdate(target = "table", set = "s = 1", where = "T = 'a'") + } + assert(ae.message.contains("cannot resolve")) + } + } + } + + test("Negative case - UPDATE the child directory") { + append(Seq((2, 2), (3, 2)).toDF("key", "value"), partitionBy = "key" :: Nil) + val e = intercept[AnalysisException] { + executeUpdate( + target = s"delta.`$tempPath/key=2`", + set = "key = 1, value = 2", + where = "value = 2") + }.getMessage + assert(e.contains("Expect a full scan of Delta sources, but found a partial scan")) + } + + test("Negative case - do not support subquery test") { append(Seq((2, 2), (1, 4), (1, 1), (0, 3)).toDF("key", "value")) Seq((2, 2), (1, 4), (1, 1), (0, 3)).toDF("c", "d").createOrReplaceTempView("source") diff --git a/src/test/scala/org/apache/spark/sql/delta/test/DeltaHiveTest.scala b/src/test/scala/org/apache/spark/sql/delta/test/DeltaHiveTest.scala new file mode 100644 index 00000000000..2541321afe1 --- /dev/null +++ b/src/test/scala/org/apache/spark/sql/delta/test/DeltaHiveTest.scala @@ -0,0 +1,62 @@ +/* + * Copyright (2020) 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.test + +import org.apache.spark.sql.delta.catalog.DeltaCatalog +import io.delta.sql.DeltaSparkSessionExtension +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark.{SparkContext, SparkFunSuite} +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.hive.test.{TestHive, TestHiveContext} +import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} +import org.apache.spark.sql.test.SQLTestUtils + +/** + * Test utility for initializing a SparkSession with a Hive Client and a Hive Catalog for testing + * DDL operations. Typical tests leverage an in-memory catalog with a mock catalog client. Here we + * use real Hive classes. + */ +trait DeltaHiveTest extends SparkFunSuite with BeforeAndAfterAll { self: SQLTestUtils => + + private var _session: SparkSession = _ + private var _hiveContext: TestHiveContext = _ + private var _sc: SparkContext = _ + + override def beforeAll(): Unit = { + val conf = TestHive.sparkSession.sparkContext.getConf.clone() + TestHive.sparkSession.stop() + conf.set(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION.key, classOf[DeltaCatalog].getName) + conf.set(StaticSQLConf.SPARK_SESSION_EXTENSIONS.key, + classOf[DeltaSparkSessionExtension].getName) + _sc = new SparkContext("local", this.getClass.getName, conf) + _hiveContext = new TestHiveContext(_sc) + _session = _hiveContext.sparkSession + SparkSession.setActiveSession(_session) + super.beforeAll() + } + + override protected def spark: SparkSession = _session + + override def afterAll(): Unit = { + try { + _hiveContext.reset() + } finally { + _sc.stop() + } + } +}