diff --git a/README.md b/README.md index 757396f342e..317bd71c4ad 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.4.0 + 0.5.0 ``` @@ -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.4.0" +libraryDependencies += "io.delta" %% "delta-core" % "0.5.0" ``` ## API Documentation @@ -109,7 +109,7 @@ We welcome contributions to Delta Lake. We use [GitHub Pull Requests ](https://g There are two mediums of communication within the Delta Lake community. - Public Slack Channel - - [Register here](https://join.slack.com/t/delta-users/shared_invite/enQtNTY1NDg0ODcxOTI1LWJkZGU3ZmQ3MjkzNmY2ZDM0NjNlYjE4MWIzYjg2OWM1OTBmMWIxZTllMjg3ZmJkNjIwZmE1ZTZkMmQ0OTk5ZjA) + - [Register here](https://join.slack.com/t/delta-users/shared_invite/enQtODQ5ODM5OTAxMjAwLWY4NGI5ZmQ3Y2JmMjZjYjc1MDkwNTA5YTQ4MzhjOWY1MmVjNTM2OGZhNTExNmM5MzQ0YzEzZjIwMjc0OGI0OGM) - [Login here](https://delta-users.slack.com/) - Public [Mailing list](https://groups.google.com/forum/#!forum/delta-users) diff --git a/build.sbt b/build.sbt index 4c58083ac48..98c399ae8c1 100644 --- a/build.sbt +++ b/build.sbt @@ -166,8 +166,6 @@ spIncludeMaven := true spIgnoreProvided := true -packagedArtifacts in publishM2 <<= packagedArtifacts in spPublishLocal - packageBin in Compile := spPackage.value sparkComponents := Seq("sql") diff --git a/build/sbt-config/repositories b/build/sbt-config/repositories index dcac6f66c19..f398fda1cea 100644 --- a/build/sbt-config/repositories +++ b/build/sbt-config/repositories @@ -7,5 +7,6 @@ typesafe-ivy-releases: https://repo.typesafe.com/typesafe/ivy-releases/, [organization]/[module]/[revision]/[type]s/[artifact](-[classifier]).[ext], bootOnly sbt-ivy-snapshots: https://repo.scala-sbt.org/scalasbt/ivy-snapshots/, [organization]/[module]/[revision]/[type]s/[artifact](-[classifier]).[ext], bootOnly sbt-plugin-releases: https://repo.scala-sbt.org/scalasbt/sbt-plugin-releases/, [organization]/[module]/(scala_[scalaVersion]/)(sbt_[sbtVersion]/)[revision]/[type]s/[artifact](-[classifier]).[ext] + 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/ diff --git a/docs/generate_api_docs.py b/docs/generate_api_docs.py index 99f0a77cb7b..b6ca481e1a2 100755 --- a/docs/generate_api_docs.py +++ b/docs/generate_api_docs.py @@ -1,4 +1,4 @@ -# !/usr/bin/env python2 +# !/usr/bin/env python # # Copyright 2019 Databricks, Inc. # Licensed under the Apache License, Version 2.0 (the "License"); @@ -13,6 +13,7 @@ # import os +import sys import subprocess import argparse @@ -38,24 +39,24 @@ def main(): sphinx_docs_final_dir = all_api_docs_final_dir + "/python" # Generate Java and Scala docs - print "## Generating ScalaDoc and JavaDoc ..." + print("## Generating ScalaDoc and JavaDoc ...") with WorkingDirectory(repo_root_dir): run_cmd(["build/sbt", ";clean;unidoc"], stream_output=verbose) # Generate Python docs - print '## Generating Python(Sphinx) docs ...' + print('## Generating Python(Sphinx) docs ...') with WorkingDirectory(sphinx_gen_dir): run_cmd(["make", "html"], stream_output=verbose) # Update Scala docs - print "## Patching ScalaDoc ..." + print("## Patching ScalaDoc ...") with WorkingDirectory(scaladoc_gen_dir): # Patch the js and css files append(docs_root_dir + "/api-docs.js", "./lib/template.js") # append new js functions append(docs_root_dir + "/api-docs.css", "./lib/template.css") # append new styles # Update Java docs - print "## Patching JavaDoc ..." + print("## Patching JavaDoc ...") with WorkingDirectory(javadoc_gen_dir): # Find html files to patch (_, stdout, _) = run_cmd(["find", ".", "-name", "*.html", "-mindepth", "2"]) @@ -99,7 +100,7 @@ def main(): run_cmd(["cp", "-r", javadoc_gen_dir, java_api_docs_final_dir]) run_cmd(["cp", "-r", sphinx_cp_dir, sphinx_docs_final_dir]) - print "## API docs generated in " + all_api_docs_final_dir + print("## API docs generated in " + all_api_docs_final_dir) def run_cmd(cmd, throw_on_error=True, env=None, stream_output=False, **kwargs): @@ -136,6 +137,10 @@ def run_cmd(cmd, throw_on_error=True, env=None, stream_output=False, **kwargs): stderr=subprocess.PIPE, **kwargs) (stdout, stderr) = child.communicate() + if sys.version_info >= (3, 0): + stdout = stdout.decode("UTF-8") + stderr = stderr.decode("UTF-8") + exit_code = child.wait() if throw_on_error and exit_code is not 0: raise Exception( @@ -180,7 +185,7 @@ def __exit__(self, tpe, value, traceback): def log(str): if verbose: - print str + print(str) verbose = False diff --git a/examples/README.md b/examples/README.md index 4c002977eda..c0f066008f9 100644 --- a/examples/README.md +++ b/examples/README.md @@ -6,5 +6,5 @@ In this folder there are examples taken from the delta.io quickstart guide and d * PySpark is required for running python examples ### Instructions -* To run an example in Python run `spark-submit --packages io.delta:delta-core_2.11:0.4.0 PATH/TO/EXAMPLE` +* To run an example in Python run `spark-submit --packages io.delta:delta-core_2.11:0.5.0 PATH/TO/EXAMPLE` * To run the Scala examples, `cd examples/scala` and run `./build/sbt "runMain example.{Example class name}"` e.g. `./build/sbt "runMain example.Quickstart"` \ No newline at end of file diff --git a/examples/python/streaming.py b/examples/python/streaming.py index 52347758603..da8c1905c29 100644 --- a/examples/python/streaming.py +++ b/examples/python/streaming.py @@ -24,7 +24,8 @@ import threading # Clear previous run delta-tables -files = ["/tmp/delta-table", "/tmp/delta-table2", "/tmp/delta-table3"] +files = ["/tmp/delta-table", "/tmp/delta-table2", "/tmp/delta-table3", "/tmp/delta-table4", + "/tmp/delta-table5", "/tmp/checkpoint/tbl1"] for i in files: try: shutil.rmtree(i) @@ -94,6 +95,36 @@ def upsertToDelta(microBatchOutputDF, batchId): print("########### DeltaTable after streaming upsert #########") deltaTable.toDF().show() +# Streaming append and concurrent repartition using data change = false +# tbl1 is the sink and tbl2 is the source +print("############ Streaming appends with concurrent table repartition ##########") +tbl1 = "/tmp/delta-table4" +tbl2 = "/tmp/delta-table5" +numRows = 10 +spark.range(numRows).write.mode("overwrite").format("delta").save(tbl1) +spark.read.format("delta").load(tbl1).show() +spark.range(numRows, numRows * 10).write.mode("overwrite").format("delta").save(tbl2) + + +# Start reading tbl2 as a stream and do a streaming write to tbl1 +# Prior to Delta 0.5.0 this would throw StreamingQueryException: Detected a data update in the +# source table. This is currently not supported. +stream4 = spark.readStream.format("delta").load(tbl2).writeStream.format("delta")\ + .option("checkpointLocation", "/tmp/checkpoint/tbl1") \ + .outputMode("append") \ + .start(tbl1) + +# repartition table while streaming job is running +spark.read.format("delta").load(tbl2).repartition(10).write\ + .format("delta")\ + .mode("overwrite")\ + .option("dataChange", "false")\ + .save(tbl2) + +stream4.awaitTermination(10) +stream4.stop() +print("######### After streaming write #########") +spark.read.format("delta").load(tbl1).show() # cleanup for i in files: try: diff --git a/examples/python/utilities.py b/examples/python/utilities.py index 4f4876806bb..13d00429bbb 100644 --- a/examples/python/utilities.py +++ b/examples/python/utilities.py @@ -71,6 +71,10 @@ print("######## Describe history for the table ######") deltaTable.history().show() +# Generate manifest +print("######## Generating manifest ######") +deltaTable.generate("SYMLINK_FORMAT_MANIFEST") + # SQL Vacuum print("####### SQL Vacuum #######") spark.sql("VACUUM '%s' RETAIN 169 HOURS" % "/tmp/delta-table").collect() diff --git a/examples/scala/build.sbt b/examples/scala/build.sbt index 691ebaea08c..ea12d8f27f9 100644 --- a/examples/scala/build.sbt +++ b/examples/scala/build.sbt @@ -23,6 +23,6 @@ version := "0.1.0" lazy val root = (project in file(".")) .settings( name := "hello-world", - libraryDependencies += "io.delta" %% "delta-core" % "0.4.0", + libraryDependencies += "io.delta" %% "delta-core" % "0.5.0", libraryDependencies += "org.apache.spark" %% "spark-sql" % "2.4.3", resolvers += "Delta" at "https://dl.bintray.com/delta-io/delta/") diff --git a/examples/scala/src/main/scala/example/Streaming.scala b/examples/scala/src/main/scala/example/Streaming.scala index d90756f2fad..194c6cf559b 100644 --- a/examples/scala/src/main/scala/example/Streaming.scala +++ b/examples/scala/src/main/scala/example/Streaming.scala @@ -36,6 +36,7 @@ object Streaming { import spark.implicits._ + println("=== Section 1: write and read delta table using batch queries, and initialize table for later sections") // Create a table val data = spark.range(0, 5) val path = new File("/tmp/delta-table").getAbsolutePath @@ -45,20 +46,21 @@ object Streaming { val df = spark.read.format("delta").load(path) df.show() - println("Streaming write") + + println("=== Section 2: write and read delta using structured streaming") val streamingDf = spark.readStream.format("rate").load() val tablePath2 = new File("/tmp/delta-table2").getCanonicalPath + val checkpointPath = new File("/tmp/checkpoint").getCanonicalPath val stream = streamingDf .select($"value" as "id") .writeStream .format("delta") - .option("checkpointLocation", new File("/tmp/checkpoint").getCanonicalPath) + .option("checkpointLocation", checkpointPath) .start(tablePath2) stream.awaitTermination(10000) stream.stop() - println("Reading from stream") val stream2 = spark .readStream .format("delta") @@ -70,8 +72,9 @@ object Streaming { stream2.awaitTermination(10000) stream2.stop() + + println("=== Section 3: Streaming upserts using MERGE") // Function to upsert microBatchOutputDF into Delta Lake table using merge - println("Streaming upgrades in update mode") def upsertToDelta(microBatchOutputDF: DataFrame, batchId: Long) { val deltaTable = DeltaTable.forPath(path) deltaTable.as("t") @@ -101,15 +104,42 @@ object Streaming { .outputMode("update") .start() - stream3.awaitTermination(10000) + stream3.awaitTermination(20000) stream3.stop() println("Delta Table after streaming upsert") deltaTable.toDF.show() + // Streaming append and concurrent repartition using data change = false + // tbl1 is the sink and tbl2 is the source + println("############ Streaming appends with concurrent table repartition ##########") + val tbl1 = "/tmp/delta-table4" + val tbl2 = "/tmp/delta-table5" + val numRows = 10 + spark.range(numRows).write.mode("overwrite").format("delta").save(tbl1) + spark.read.format("delta").load(tbl1).show() + spark.range(numRows, numRows * 10).write.mode("overwrite").format("delta").save(tbl2) + + // Start reading tbl2 as a stream and do a streaming write to tbl1 + // Prior to Delta 0.5.0 this would throw StreamingQueryException: Detected a data update in the source table. This is currently not supported. + val stream4 = spark.readStream.format("delta").load(tbl2).writeStream.format("delta") + .option("checkpointLocation", new File("/tmp/checkpoint/tbl1").getCanonicalPath) + .outputMode("append") + .start(tbl1) + + // repartition table while streaming job is running + spark.read.format("delta").load(tbl2).repartition(10).write.format("delta").mode("overwrite").option("dataChange", "false").save(tbl2) + + stream4.awaitTermination(10) + stream4.stop() + println("######### After streaming write #########") + spark.read.format("delta").load(tbl1).show() + + println("=== In the end, clean all paths") // Cleanup - FileUtils.deleteDirectory(new File(path)) - FileUtils.deleteDirectory(new File(tablePath2)) + Seq(path, tbl1, tbl2, "/tmp/checkpoint/tbl1", tablePath2).foreach { path => + FileUtils.deleteDirectory(new File(path)) + } spark.stop() } } diff --git a/examples/scala/src/main/scala/example/Utilities.scala b/examples/scala/src/main/scala/example/Utilities.scala index 2dbe8ca7a4b..5209ca4d424 100644 --- a/examples/scala/src/main/scala/example/Utilities.scala +++ b/examples/scala/src/main/scala/example/Utilities.scala @@ -65,6 +65,10 @@ object Utilities { println("Describe History for the table") deltaTable.history().show() + // Generate manifest + println("Generate Manifest files") + deltaTable.generate("SYMLINK_FORMAT_MANIFEST") + // SQL utility commands println("SQL Vacuum") spark.sql(s"VACUUM '$path' RETAIN 169 HOURS") diff --git a/examples/tutorials/saiseu19/SAISEu19 - Delta Lake Python Tutorial.py b/examples/tutorials/saiseu19/SAISEu19 - Delta Lake Python Tutorial.py index fd493ee3406..d10f1f5b44d 100644 --- a/examples/tutorials/saiseu19/SAISEu19 - Delta Lake Python Tutorial.py +++ b/examples/tutorials/saiseu19/SAISEu19 - Delta Lake Python Tutorial.py @@ -216,7 +216,7 @@ def stop_all_streams(): # MAGIC Schema is not enforced when writing leading to dirty and often corrupted data. # MAGIC # MAGIC #### 2. No interoperatbility between batch and streaming workloads -# MAGIC Apache Spark's Parquet streaming sink does not maintain enough metadata such that batch workload can seamlessly interact with batch workloads. +# MAGIC Apache Spark's Parquet streaming sink does not maintain enough metadata such that streaming workloads can seamlessly interact with batch workloads. # COMMAND ---------- @@ -425,7 +425,7 @@ def generate_and_append_data_stream_fixed(table_format, table_path): # COMMAND ---------- -# MAGIC %md **Note**: Because we were able to easily `DELETE` the data, the above value should be `null`. +# MAGIC %md **Note**: Because we were able to easily `DELETE` the data, the above value should be `0`. # COMMAND ---------- diff --git a/examples/tutorials/saiseu19/SAISEu19 - Delta Lake Scala Tutorial.scala b/examples/tutorials/saiseu19/SAISEu19 - Delta Lake Scala Tutorial.scala index 0094f93f10a..b4002b7e338 100644 --- a/examples/tutorials/saiseu19/SAISEu19 - Delta Lake Scala Tutorial.scala +++ b/examples/tutorials/saiseu19/SAISEu19 - Delta Lake Scala Tutorial.scala @@ -222,7 +222,7 @@ stopAllStreams() // MAGIC Schema is not enforced when writing leading to dirty and often corrupted data. // MAGIC // MAGIC #### 2. No interoperatbility between batch and streaming workloads -// MAGIC Apache Spark's Parquet streaming sink does not maintain enough metadata such that batch workload can seamlessly interact with batch workloads. +// MAGIC Apache Spark's Parquet streaming sink does not maintain enough metadata such that streaming workloads can seamlessly interact with batch workloads. // COMMAND ---------- @@ -425,7 +425,7 @@ display(spark.sql("SELECT COUNT(*) FROM loans_delta WHERE funded_amnt = paid_amn // COMMAND ---------- -// MAGIC %md **Note**: Because we were able to easily `DELETE` the data, the above value should be `null`. +// MAGIC %md **Note**: Because we were able to easily `DELETE` the data, the above value should be `0`. // COMMAND ---------- diff --git a/examples/tutorials/saiseu19/img/Attach-Notebook.png b/examples/tutorials/saiseu19/img/Attach-Notebook.png new file mode 100644 index 00000000000..ac77816515d Binary files /dev/null and b/examples/tutorials/saiseu19/img/Attach-Notebook.png differ diff --git a/examples/tutorials/saiseu19/img/Video-Preview.png b/examples/tutorials/saiseu19/img/Video-Preview.png new file mode 100644 index 00000000000..c3b4bc4d3f5 Binary files /dev/null and b/examples/tutorials/saiseu19/img/Video-Preview.png differ diff --git a/examples/tutorials/saiseu19/readme.md b/examples/tutorials/saiseu19/readme.md index ebe57e70102..2409a4c31ee 100644 --- a/examples/tutorials/saiseu19/readme.md +++ b/examples/tutorials/saiseu19/readme.md @@ -2,6 +2,10 @@ ### Sign Up for Databricks Community Edition +This tutorial goes through many features of Delta Lake features including schema enforcement and schema evolution, interoperability between batch and streaming workloads, time travel, and DML commands like Delete and Merge. It was originally given at [Spark Summit 2019 Europe](https://databricks.com/sparkaisummit/europe) and is available in both Scala and Python. The instructions on this page explain how to run the examples on Databricks Community Edition, but all the pieces (except some of the Databricks filesystem bits) should work in any **Spark 2.4.2** or higher with **Delta Lake 0.4.0** or higher. If you'd prefer to watch the tutorial, along with some brief background information about the problems Delta Lake tries to solve, in video form, here's the recording from one of the Spark Summit sessions: + +[![ ](img/Video-Preview.png)](https://youtu.be/cpA3Ni8ZUPI) +
Expand to view more details about Databricks Community Edition @@ -51,7 +55,7 @@ Note, within DBCE, you can only create one cluster at a time. If one already ex
-### Import and Attach a Notebook +### Importing Notebooks
Expand to view more details about Importing Notebooks @@ -89,3 +93,14 @@ Once you have imported the notebook, your screen should similar to the view belo
+### Attaching Notebooks + +
+Expand to view more details about Attaching Notebooks + + 
  +Near the top left, click the *cluster dropdown* and choose the cluster you want to attach the notebook. + +![](img/Attach-Notebook.png) + +
diff --git a/project/plugins.sbt b/project/plugins.sbt index acc946c3552..b5afbf66725 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -38,3 +38,6 @@ addSbtPlugin("com.eed3si9n" % "sbt-unidoc" % "0.4.2") addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.3.0") addSbtPlugin("com.simplytyped" % "sbt-antlr4" % "0.7.13") + +resolvers += Resolver.url("typesafe sbt-plugins", + url("https://dl.bintray.com/typesafe/sbt-plugins"))(Resolver.ivyStylePatterns) diff --git a/python/delta/tables.py b/python/delta/tables.py index 67bccafefd4..75400f554a8 100644 --- a/python/delta/tables.py +++ b/python/delta/tables.py @@ -60,6 +60,21 @@ def alias(self, aliasName): jdt = self._jdt.alias(aliasName) return DeltaTable(self._spark, jdt) + @since(0.5) + def generate(self, mode): + """ + Generate manifest files for the given delta table. + + :param mode: mode for the type of manifest file to be generated + The valid modes are as follows (not case sensitive): + - "symlink_format_manifest" : This will generate manifests in symlink format + for Presto and Athena read support. + See the online documentation for more information. + + .. note:: Evolving + """ + self._jdt.generate(mode) + @since(0.4) def delete(self, condition=None): """ diff --git a/python/delta/tests/test_deltatable.py b/python/delta/tests/test_deltatable.py index 4071e7787e0..9fcac68f114 100644 --- a/python/delta/tests/test_deltatable.py +++ b/python/delta/tests/test_deltatable.py @@ -73,6 +73,26 @@ def test_delete(self): with self.assertRaises(TypeError): dt.delete(condition=1) + def test_generate(self): + # create a delta table + numFiles = 10 + self.spark.range(100).repartition(numFiles).write.format("delta").save(self.tempFile) + dt = DeltaTable.forPath(self.spark, self.tempFile) + + # Generate the symlink format manifest + dt.generate("symlink_format_manifest") + + # check the contents of the manifest + # NOTE: this is not a correctness test, we are testing correctness in the scala suite + manifestPath = os.path.join(self.tempFile, + os.path.join("_symlink_format_manifest", "manifest")) + files = [] + with open(manifestPath) as f: + files = f.readlines() + + # the number of files we write should equal the number of lines in the manifest + assert(len(files) == numFiles) + def test_update(self): self.__writeDeltaTable([('a', 1), ('b', 2), ('c', 3), ('d', 4)]) dt = DeltaTable.forPath(self.spark, self.tempFile) diff --git a/python/delta/tests/test_sql.py b/python/delta/tests/test_sql.py index 222f60e92ef..4873c86d95a 100644 --- a/python/delta/tests/test_sql.py +++ b/python/delta/tests/test_sql.py @@ -20,6 +20,7 @@ import os from pyspark.sql import SQLContext, functions, Row, SparkSession +from pyspark import SparkContext, SparkConf from delta.tables import DeltaTable from delta.testing.utils import PySparkTestCase @@ -62,6 +63,29 @@ def test_vacuum(self): def test_describe_history(self): assert(len(self.spark.sql("desc history delta.`%s`" % (self.temp_file)).collect()) > 0) + def test_generate(self): + # create a delta table + temp_path = tempfile.mkdtemp() + temp_file = os.path.join(temp_path, "delta_sql_test_table") + numFiles = 10 + self.spark.range(100).repartition(numFiles).write.format("delta").save(temp_file) + + # Generate the symlink format manifest + self.spark.sql("GENERATE SYMLINK_FORMAT_MANIFEST FOR TABLE delta.`{}`" + .format(temp_file)) + + # check the contents of the manifest + # NOTE: this is not a correctness test, we are testing correctness in the scala suite + manifestPath = os.path.join(temp_file, + os.path.join("_symlink_format_manifest", "manifest")) + files = [] + with open(manifestPath) as f: + files = f.readlines() + + shutil.rmtree(temp_path) + # the number of files we write should equal the number of lines in the manifest + assert(len(files) == numFiles) + def test_convert(self): df = self.spark.createDataFrame([('a', 1), ('b', 2), ('c', 3)], ["key", "value"]) temp_path2 = tempfile.mkdtemp() @@ -95,6 +119,7 @@ def __checkAnswer(self, df, expectedAnswer, schema=["key", "value"]): self.assertEqual([], df.subtract(expectedDF).take(1)) self.assertEqual([], expectedDF.subtract(df).take(1)) + if __name__ == "__main__": try: import xmlrunner diff --git a/src/main/antlr4/io/delta/sql/parser/DeltaSqlBase.g4 b/src/main/antlr4/io/delta/sql/parser/DeltaSqlBase.g4 index 093092c0830..7a60891530d 100644 --- a/src/main/antlr4/io/delta/sql/parser/DeltaSqlBase.g4 +++ b/src/main/antlr4/io/delta/sql/parser/DeltaSqlBase.g4 @@ -75,6 +75,7 @@ statement : VACUUM (path=STRING | table=qualifiedName) (RETAIN number HOURS)? (DRY RUN)? #vacuumTable | (DESC | DESCRIBE) DETAIL (path=STRING | table=qualifiedName) #describeDeltaDetail + | GENERATE modeName=identifier FOR TABLE table=qualifiedName #generate | (DESC | DESCRIBE) HISTORY (path=STRING | table=qualifiedName) (LIMIT limit=INTEGER_VALUE)? #describeDeltaHistory | CONVERT TO DELTA table=qualifiedName @@ -124,6 +125,7 @@ nonReserved : VACUUM | RETAIN | HOURS | DRY | RUN | CONVERT | TO | DELTA | PARTITIONED | BY | DESC | DESCRIBE | LIMIT | DETAIL + | GENERATE | FOR | TABLE ; // Define how the keywords above should appear in a user's SQL statement. @@ -134,6 +136,7 @@ DELTA: 'DELTA'; DESC: 'DESC'; DESCRIBE: 'DESCRIBE'; DETAIL: 'DETAIL'; +GENERATE: 'GENERATE'; DRY: 'DRY'; HISTORY: 'HISTORY'; HOURS: 'HOURS'; @@ -141,6 +144,8 @@ LIMIT: 'LIMIT'; MINUS: '-'; NOT: 'NOT' | '!'; NULL: 'NULL'; +FOR: 'FOR'; +TABLE: 'TABLE'; PARTITIONED: 'PARTITIONED'; RETAIN: 'RETAIN'; RUN: 'RUN'; diff --git a/src/main/scala/com/databricks/spark/util/DatabricksLogging.scala b/src/main/scala/com/databricks/spark/util/DatabricksLogging.scala index e59e47beba7..3bb3cfe875b 100644 --- a/src/main/scala/com/databricks/spark/util/DatabricksLogging.scala +++ b/src/main/scala/com/databricks/spark/util/DatabricksLogging.scala @@ -31,7 +31,7 @@ object TagDefinitions { object TAG_OP_TYPE extends TagDefinition } -class OpType(typeName: String, description: String) +case class OpType(typeName: String, description: String) class MetricDefinition diff --git a/src/main/scala/io/delta/sql/parser/DeltaSqlParser.scala b/src/main/scala/io/delta/sql/parser/DeltaSqlParser.scala index 117dab85cec..57b5aeaed77 100644 --- a/src/main/scala/io/delta/sql/parser/DeltaSqlParser.scala +++ b/src/main/scala/io/delta/sql/parser/DeltaSqlParser.scala @@ -42,6 +42,7 @@ import java.util.Locale import scala.collection.JavaConverters._ +import org.apache.spark.sql.delta.commands.DeltaGenerateCommand import io.delta.sql.parser.DeltaSqlBaseParser._ import io.delta.tables.execution.{DescribeDeltaHistoryCommand, VacuumTableCommand} import org.antlr.v4.runtime._ @@ -56,7 +57,7 @@ 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.DescribeDeltaDetailCommandOSS +import org.apache.spark.sql.delta.commands.DescribeDeltaDetailCommand import org.apache.spark.sql.delta.commands.ConvertToDeltaCommand import org.apache.spark.sql.types._ @@ -155,7 +156,7 @@ class DeltaSqlAstBuilder extends DeltaSqlBaseBaseVisitor[AnyRef] { override def visitDescribeDeltaDetail( ctx: DescribeDeltaDetailContext): LogicalPlan = withOrigin(ctx) { - DescribeDeltaDetailCommandOSS( + DescribeDeltaDetailCommand( Option(ctx.path).map(string), Option(ctx.table).map(visitTableIdentifier)) } @@ -168,6 +169,12 @@ class DeltaSqlAstBuilder extends DeltaSqlBaseBaseVisitor[AnyRef] { Option(ctx.limit).map(_.getText.toInt)) } + override def visitGenerate(ctx: GenerateContext): LogicalPlan = withOrigin(ctx) { + DeltaGenerateCommand( + modeName = ctx.modeName.getText, + tableId = visitTableIdentifier(ctx.table)) + } + override def visitConvert(ctx: ConvertContext): LogicalPlan = withOrigin(ctx) { ConvertToDeltaCommand( visitTableIdentifier(ctx.table), diff --git a/src/main/scala/io/delta/tables/DeltaTable.scala b/src/main/scala/io/delta/tables/DeltaTable.scala index 2514cbb9032..01db5d3160c 100644 --- a/src/main/scala/io/delta/tables/DeltaTable.scala +++ b/src/main/scala/io/delta/tables/DeltaTable.scala @@ -135,6 +135,24 @@ class DeltaTable private[tables](df: Dataset[Row], deltaLog: DeltaLog) executeHistory(deltaLog, None) } + /** + * :: Evolving :: + * + * Generate a manifest for the given Delta Table + * + * @param mode Specifies the mode for the generation of the manifest. + * The valid modes are as follows (not case sensitive): + * - "symlink_format_manifest" : This will generate manifests in symlink format + * for Presto and Athena read support. + * See the online documentation for more information. + * @since 0.5.0 + */ + @Evolving + def generate(mode: String): Unit = { + val path = deltaLog.dataPath.toString + executeGenerate(s"delta.`$path`", mode) + } + /** * :: Evolving :: * @@ -526,7 +544,6 @@ object DeltaTable { partitionSchema: StructType): DeltaTable = { val tableId: TableIdentifier = spark.sessionState.sqlParser.parseTableIdentifier(identifier) DeltaConvert.executeConvert(spark, tableId, Some(partitionSchema), None) - forPath(spark, tableId.table) } /** @@ -557,7 +574,6 @@ object DeltaTable { partitionSchema: String): DeltaTable = { val tableId: TableIdentifier = spark.sessionState.sqlParser.parseTableIdentifier(identifier) DeltaConvert.executeConvert(spark, tableId, Some(StructType.fromDDL(partitionSchema)), None) - forPath(spark, tableId.table) } /** @@ -585,7 +601,6 @@ object DeltaTable { identifier: String): DeltaTable = { val tableId: TableIdentifier = spark.sessionState.sqlParser.parseTableIdentifier(identifier) DeltaConvert.executeConvert(spark, tableId, None, None) - forPath(spark, tableId.table) } /** diff --git a/src/main/scala/io/delta/tables/execution/DeltaConvert.scala b/src/main/scala/io/delta/tables/execution/DeltaConvert.scala index 843b6016df9..70bd49b455f 100644 --- a/src/main/scala/io/delta/tables/execution/DeltaConvert.scala +++ b/src/main/scala/io/delta/tables/execution/DeltaConvert.scala @@ -17,6 +17,7 @@ package io.delta.tables.execution import org.apache.spark.sql.delta.commands.{ConvertToDeltaCommand, ConvertToDeltaCommandBase} +import io.delta.tables.DeltaTable import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.TableIdentifier @@ -27,9 +28,10 @@ trait DeltaConvertBase { spark: SparkSession, tableIdentifier: TableIdentifier, partitionSchema: Option[StructType], - deltaPath: Option[String]): Unit = { + deltaPath: Option[String]): DeltaTable = { val cvt = ConvertToDeltaCommand(tableIdentifier, partitionSchema, deltaPath) cvt.run(spark) + DeltaTable.forPath(spark, tableIdentifier.table) } } diff --git a/src/main/scala/io/delta/tables/execution/DeltaTableOperations.scala b/src/main/scala/io/delta/tables/execution/DeltaTableOperations.scala index fe3156162da..f92831e40b7 100644 --- a/src/main/scala/io/delta/tables/execution/DeltaTableOperations.scala +++ b/src/main/scala/io/delta/tables/execution/DeltaTableOperations.scala @@ -19,11 +19,12 @@ package io.delta.tables.execution import scala.collection.Map import org.apache.spark.sql.delta.{DeltaErrors, DeltaHistoryManager, DeltaLog, PreprocessTableUpdate} -import org.apache.spark.sql.delta.commands.{DeleteCommand, VacuumCommand} +import org.apache.spark.sql.delta.commands.{DeleteCommand, DeltaGenerateCommand, VacuumCommand} 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.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions.{Expression, SubqueryExpression} import org.apache.spark.sql.catalyst.plans.logical._ @@ -74,6 +75,15 @@ trait DeltaTableOperations extends AnalysisHelper { self: DeltaTable => spark.createDataFrame(history.getHistory(limit)) } + protected def executeGenerate(tblIdentifier: String, mode: String): Unit = { + val tableId: TableIdentifier = sparkSession + .sessionState + .sqlParser + .parseTableIdentifier(tblIdentifier) + val generate = DeltaGenerateCommand(mode, tableId) + generate.run(sparkSession) + } + protected def executeUpdate(set: Map[String, Column], condition: Option[Column]): Unit = { val setColumns = set.map{ case (col, expr) => (col, expr) }.toSeq diff --git a/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/merge.scala b/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/merge.scala index 4d12da8df1a..7b4a385a56c 100644 --- a/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/merge.scala +++ b/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/merge.scala @@ -265,8 +265,9 @@ object MergeInto { // expression (i.e. sourceColumnBySameName) needs to be resolved only by the source // plan. fakeTargetPlan.output.map(_.name).map { tgtColName => - val resolvedExpr = - resolveOrFail(UnresolvedAttribute(tgtColName), fakeSourcePlan, s"$typ clause") + val resolvedExpr = resolveOrFail( + UnresolvedAttribute.quotedString(s"`$tgtColName`"), + fakeSourcePlan, s"$typ clause") MergeAction(Seq(tgtColName), resolvedExpr) } diff --git a/src/main/scala/org/apache/spark/sql/delta/DeltaConfig.scala b/src/main/scala/org/apache/spark/sql/delta/DeltaConfig.scala index 2f1d8c01aa1..4c3c257622f 100644 --- a/src/main/scala/org/apache/spark/sql/delta/DeltaConfig.scala +++ b/src/main/scala/org/apache/spark/sql/delta/DeltaConfig.scala @@ -211,6 +211,14 @@ object DeltaConfigs extends DeltaLogging { } } + def getMilliSeconds(i: CalendarInterval): Long = { + i.milliseconds() + } + + private def isPositiveDayTimeInterval(i: CalendarInterval): Boolean = { + i.months == 0 && i.microseconds > 0 + } + /** * The shortest duration we have to keep delta files around before deleting them. We can only * delete delta files that are before a compaction. We may keep files beyond this duration until @@ -220,7 +228,7 @@ object DeltaConfigs extends DeltaLogging { "logRetentionDuration", "interval 30 days", parseCalendarInterval, - i => i.microseconds > 0 && i.months == 0, + isPositiveDayTimeInterval, "needs to be provided as a calendar interval such as '2 weeks'. Months " + "and years are not accepted. You may specify '365 days' for a year instead.") @@ -231,7 +239,7 @@ object DeltaConfigs extends DeltaLogging { "sampleRetentionDuration", "interval 7 days", parseCalendarInterval, - i => i.microseconds > 0 && i.months == 0, + isPositiveDayTimeInterval, "needs to be provided as a calendar interval such as '2 weeks'. Months " + "and years are not accepted. You may specify '365 days' for a year instead.") @@ -244,7 +252,7 @@ object DeltaConfigs extends DeltaLogging { "checkpointRetentionDuration", "interval 2 days", parseCalendarInterval, - i => i.microseconds > 0 && i.months == 0, + isPositiveDayTimeInterval, "needs to be provided as a calendar interval such as '2 weeks'. Months " + "and years are not accepted. You may specify '365 days' for a year instead.") @@ -294,7 +302,7 @@ object DeltaConfigs extends DeltaLogging { "deletedFileRetentionDuration", "interval 1 week", parseCalendarInterval, - i => i.microseconds > 0 && i.months == 0, + isPositiveDayTimeInterval, "needs to be provided as a calendar interval such as '2 weeks'. Months " + "and years are not accepted. You may specify '365 days' for a year instead.") @@ -355,4 +363,11 @@ object DeltaConfigs extends DeltaLogging { a => a >= -1, "needs to be larger than or equal to -1.") + val SYMLINK_FORMAT_MANIFEST_ENABLED = buildConfig[Boolean]( + s"${hooks.GenerateSymlinkManifest.CONFIG_NAME_ROOT}.enabled", + "false", + _.toBoolean, + _ => true, + "needs to be a boolean.") + } 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 8750627ddea..af7c86cf937 100644 --- a/src/main/scala/org/apache/spark/sql/delta/DeltaErrors.scala +++ b/src/main/scala/org/apache/spark/sql/delta/DeltaErrors.scala @@ -21,6 +21,7 @@ import java.io.FileNotFoundException import java.util.ConcurrentModificationException import org.apache.spark.sql.delta.actions.{CommitInfo, Metadata} +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.util.JsonUtils @@ -37,13 +38,22 @@ import org.apache.spark.sql.types.{DataType, StructField, StructType} trait DocsPath { /** - * The URL for the base path of Delta's docs. + * The URL for the base path of Delta's docs. When changing this path, ensure that the new path + * works with the error messages below. */ - def baseDocsPath(conf: SparkConf): String = "https://docs.delta.io" + protected def baseDocsPath(conf: SparkConf): String = "https://docs.delta.io/latest" + + def generateDocsLink(conf: SparkConf, path: String): String = { + baseDocsPath(conf) + path + } } /** * A holder object for Delta errors. + * + * IMPORTANT: Any time you add a test that references the docs, add to the Seq defined in + * DeltaErrorsSuite so that the doc links that are generated can be verified to work in Azure, + * docs.databricks.com and docs.delta.io */ object DeltaErrors extends DocsPath @@ -51,6 +61,8 @@ object DeltaErrors def baseDocsPath(spark: SparkSession): String = baseDocsPath(spark.sparkContext.getConf) + val faqRelativePath = "/delta-intro.html#frequently-asked-questions" + val DeltaSourceIgnoreDeleteErrorMessage = "Detected deleted data from streaming source. This is currently not supported. If you'd like " + "to ignore deletes, set the option 'ignoreDeletes' to 'true'." @@ -66,12 +78,11 @@ object DeltaErrors * Note that we must pass in the docAddress as a string, because the config is not available on * executors where this method is called. */ - def deltaFileNotFoundHint(path: String, docAddress: String): String = { + def deltaFileNotFoundHint(faqPath: String, path: String): String = { recordDeltaEvent(null, "delta.error.fileNotFound", data = path) - val faq = docAddress + "/delta/delta-intro.html#frequently-asked-questions" "A file referenced in the transaction log cannot be found. This occurs when data has been " + "manually deleted from the file system rather than using the table `DELETE` statement. " + - s"For more information, see $faq" + s"For more information, see $faqPath" } def formatColumn(colName: String): String = s"`$colName`" @@ -149,6 +160,12 @@ object DeltaErrors + newColumns) } + def notEnoughColumnsInInsert(table: String, query: Int, target: Int): Throwable = { + new AnalysisException(s"Cannot write to '$table', not enough data columns; " + + s"target table has ${target} column(s) but the inserted data has " + + s"${query} column(s)") + } + def alterTableReplaceColumnsException( oldSchema: StructType, newSchema: StructType, @@ -185,6 +202,11 @@ object DeltaErrors "Vacuuming specific partitions is currently not supported.") } + def unexpectedDataChangeException(op: String): Throwable = { + new AnalysisException(s"Attempting to change metadata when 'dataChange' option is set" + + s" to false during $op") + } + def unknownConfigurationKeyException(confKey: String): Throwable = { new AnalysisException(s"Unknown configuration was specified: $confKey") } @@ -199,7 +221,8 @@ object DeltaErrors |using format("delta") and that you are trying to $operation the table base path. | |To disable this check, SET spark.databricks.delta.formatCheck.enabled=false - |To learn more about Delta, see ${baseDocsPath(spark)}/delta/index.html + |To learn more about Delta, see ${generateDocsLink(spark.sparkContext.getConf, + "/index.html")} |""".stripMargin) } @@ -217,7 +240,8 @@ object DeltaErrors |'format("delta")' when reading and writing to a delta table. | |To disable this check, SET spark.databricks.delta.formatCheck.enabled=false - |To learn more about Delta, see ${baseDocsPath(spark)}/delta/index.html + |To learn more about Delta, see + |${generateDocsLink(spark.sparkContext.getConf, "/index.html")} |""".stripMargin) } @@ -310,7 +334,7 @@ object DeltaErrors s"(${DeltaConfigs.IS_APPEND_ONLY.key}=false)'.") } - def missingPartFilesException(c: CheckpointMetaData, ae: AnalysisException): Throwable = { + def missingPartFilesException(c: CheckpointMetaData, ae: Exception): Throwable = { new IllegalStateException( s"Couldn't find all part files of the checkpoint version: ${c.version}", ae) } @@ -416,9 +440,17 @@ object DeltaErrors + unknownColumns.mkString(", ")) } - def multipleSourceRowMatchingTargetRowInMergeException: Throwable = { - new UnsupportedOperationException("Cannot perform MERGE as multiple source rows " + - "matched and attempted to update the same target row in the Delta table.") + def multipleSourceRowMatchingTargetRowInMergeException(spark: SparkSession): Throwable = { + new UnsupportedOperationException( + s"""Cannot perform MERGE as multiple source rows matched and attempted to update the same + |target row in the Delta table. By SQL semantics of merge, when multiple source rows match + |on the same target row, the update operation is ambiguous as it is unclear which source + |should be used to update the matching target row. + |You can preprocess the source table to eliminate the possibility of multiple matches. + |Please refer to + |${baseDocsPath(spark)}/delta/delta-update.html#upsert-into-a-table-using-merge + """.stripMargin + ) } def subqueryNotSupportedException(op: String, cond: Expression): Throwable = { @@ -470,7 +502,8 @@ object DeltaErrors |`$path/_delta_log`. Check the upstream job to make sure that it is writing using |format("delta") and that the path is the root of the table. | - |To learn more about Delta, see ${baseDocsPath(spark)}/delta/index.html + |To learn more about Delta, see ${generateDocsLink(spark.sparkContext.getConf, + "/index.html")} """.stripMargin) } @@ -482,7 +515,8 @@ object DeltaErrors |from `$path` using Delta Lake, but the schema is not specified when the |input path is empty. | - |To learn more about Delta, see ${baseDocsPath(spark)}/delta/index.html + |To learn more about Delta, see ${generateDocsLink(spark.sparkContext.getConf, + "/index.html")} """.stripMargin) } @@ -493,7 +527,8 @@ object DeltaErrors |You are trying to create a managed table $tableName |using Delta Lake, but the schema is not specified. | - |To learn more about Delta, see ${baseDocsPath(spark)}/delta/index.html + |To learn more about Delta, see ${generateDocsLink(spark.sparkContext.getConf, + "/index.html")} """.stripMargin) } @@ -665,9 +700,33 @@ object DeltaErrors |%sql set spark.databricks.delta.alterLocation.bypassSchemaCheck = true""".stripMargin) } + def setLocationNotSupportedOnPathIdentifiers(): Throwable = { + throw new AnalysisException("Cannot change the location of a path based table.") + } + def describeViewHistory: Throwable = { new AnalysisException("Cannot describe the history of a view.") } + + def postCommitHookFailedException( + failedHook: PostCommitHook, + failedOnCommitVersion: Long, + extraErrorMessage: String, + error: Throwable): Throwable = { + var errorMessage = s"Committing to the Delta table version $failedOnCommitVersion succeeded" + + s" but error while executing post-commit hook ${failedHook.name}" + if (extraErrorMessage != null && extraErrorMessage.nonEmpty) { + errorMessage += s": $extraErrorMessage" + } + new RuntimeException(errorMessage, error) + } + + def unsupportedGenerateModeException(modeName: String): Throwable = { + import org.apache.spark.sql.delta.commands.DeltaGenerateCommand + val supportedModes = DeltaGenerateCommand.modeNameToGenerationFunc.keys.toSeq.mkString(", ") + new IllegalArgumentException( + s"Specified mode '$modeName' is not supported. Supported modes are: $supportedModes") + } } /** The basic class for all Tahoe commit conflict exceptions. */ @@ -677,8 +736,9 @@ abstract class DeltaConcurrentModificationException(message: String) def this(baseMessage: String, conflictingCommit: Option[CommitInfo]) = this( baseMessage + conflictingCommit.map(ci => s"\nConflicting commit: ${JsonUtils.toJson(ci)}").getOrElse("") + - s"\nRefer to ${DeltaErrors.baseDocsPath(SparkEnv.get.conf)}" + - "/delta/isolation-level.html#optimistic-concurrency-control for more details." + s"\nRefer to " + + s"${DeltaErrors.generateDocsLink(SparkEnv.get.conf, "/concurrency-control.html")} " + + "for more details." ) /** 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 6e34f0a97d1..196fb53d2e3 100644 --- a/src/main/scala/org/apache/spark/sql/delta/DeltaLog.scala +++ b/src/main/scala/org/apache/spark/sql/delta/DeltaLog.scala @@ -39,10 +39,13 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SparkContext import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.{Resolver, UnresolvedAttribute} -import org.apache.spark.sql.catalyst.expressions.{And, Attribute, Expression, In, InSet, Literal} +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, Cast, Expression, In, InSet, Literal} import org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.json.JsonFileFormat +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.sources.{BaseRelation, InsertableRelation} +import org.apache.spark.sql.types.{StructField, StructType} import org.apache.spark.util.{Clock, SystemClock, ThreadUtils} /** @@ -105,7 +108,7 @@ class DeltaLog private( /** How long to keep around logically deleted files before physically deleting them. */ private[delta] def tombstoneRetentionMillis: Long = - DeltaConfigs.TOMBSTONE_RETENTION.fromMetaData(metadata).milliseconds() + DeltaConfigs.getMilliSeconds(DeltaConfigs.TOMBSTONE_RETENTION.fromMetaData(metadata)) // TODO: There is a race here where files could get dropped when increasing the // retention interval... @@ -144,14 +147,15 @@ class DeltaLog private( val deltaVersions = deltas.map(f => deltaVersion(f.getPath)) verifyDeltaVersions(deltaVersions) val newVersion = deltaVersions.lastOption.getOrElse(c.version) - val deltaFiles = ((c.version + 1) to newVersion).map(deltaFile(logPath, _)) logInfo(s"Loading version $newVersion starting from checkpoint ${c.version}") try { + val deltaIndex = DeltaLogFileIndex(DeltaLog.COMMIT_FILE_FORMAT, deltas) + val checkpointIndex = DeltaLogFileIndex(DeltaLog.CHECKPOINT_FILE_FORMAT, fs, checkpointFiles) val snapshot = new Snapshot( logPath, newVersion, None, - checkpointFiles ++ deltaFiles, + checkpointIndex :: deltaIndex :: Nil, minFileRetentionTimestamp, this, // we don't want to make an additional RPC here to get commit timestamps when "deltas" is @@ -162,6 +166,10 @@ class DeltaLog private( lastUpdateTimestamp = clock.getTimeMillis() snapshot } catch { + case e: FileNotFoundException + if Option(e.getMessage).exists(_.contains("parquet does not exist")) => + recordDeltaEvent(this, "delta.checkpoint.error.partial") + throw DeltaErrors.missingPartFilesException(c, e) case e: AnalysisException if Option(e.getMessage).exists(_.contains("Path does not exist")) => recordDeltaEvent(this, "delta.checkpoint.error.partial") throw DeltaErrors.missingPartFilesException(c, e) @@ -300,8 +308,10 @@ class DeltaLog private( val newCheckpointFiles = newCheckpoint.get.getCorrespondingFiles(logPath) val newVersion = deltaVersions.last - val deltaFiles = - ((newCheckpointVersion + 1) to newVersion).map(deltaFile(logPath, _)) + + val deltaIndex = DeltaLogFileIndex(DeltaLog.COMMIT_FILE_FORMAT, deltas) + val checkpointIndex = + DeltaLogFileIndex(DeltaLog.CHECKPOINT_FILE_FORMAT, fs, newCheckpointFiles) logInfo(s"Loading version $newVersion starting from checkpoint $newCheckpointVersion") @@ -309,7 +319,7 @@ class DeltaLog private( logPath, newVersion, None, - newCheckpointFiles ++ deltaFiles, + checkpointIndex :: deltaIndex :: Nil, minFileRetentionTimestamp, this, deltas.last.getModificationTime) @@ -321,11 +331,12 @@ class DeltaLog private( // Load Snapshot from scratch to avoid StackOverflowError getSnapshotAt(deltaVersions.last, Some(deltas.last.getModificationTime)) } else { + val deltaIndex = DeltaLogFileIndex(DeltaLog.COMMIT_FILE_FORMAT, deltas) new Snapshot( logPath, deltaVersions.last, Some(currentSnapshot.state), - deltas.map(_.getPath), + deltaIndex :: Nil, minFileRetentionTimestamp, this, deltas.last.getModificationTime, @@ -526,13 +537,31 @@ class DeltaLog private( throw DeltaErrors.logFileNotFoundException(versionZeroFile, 0L, metadata) } } - val deltaData = - ((checkpointVersion.getOrElse(-1L) + 1) to version).map(deltaFile(logPath, _)) + val startVersion = checkpointVersion.getOrElse(-1L) + 1 + // Listing the files may be more efficient than getting the file status for each file + val deltaData = store.listFrom(deltaFile(logPath, startVersion)) + .filter(f => isDeltaFile(f.getPath)) + .takeWhile(f => deltaVersion(f.getPath) <= version) + .toArray + val deltaFileVersions = deltaData.map(f => deltaVersion(f.getPath)) + if (deltaFileVersions.nonEmpty) { + // deltaFileVersions can be empty if we're loading a version for which a checkpoint exists + verifyDeltaVersions(deltaFileVersions) + require(deltaFileVersions.head == startVersion, + s"Did not get the first delta file version: $startVersion to compute Snapshot") + require(deltaFileVersions.last == version, + s"Did not get the last delta file version: $version to compute Snapshot") + } + + val deltaIndex = DeltaLogFileIndex(DeltaLog.COMMIT_FILE_FORMAT, deltaData) + val checkpointIndex = + DeltaLogFileIndex(DeltaLog.CHECKPOINT_FILE_FORMAT, fs, lastCheckpointFiles) + new Snapshot( logPath, version, None, - lastCheckpointFiles ++ deltaData, + checkpointIndex :: deltaIndex :: Nil, minFileRetentionTimestamp, this, commitTimestamp.getOrElse(-1L)) @@ -756,12 +785,12 @@ object DeltaLog extends DeltaLogging { * @param partitionColumnPrefixes The path to the `partitionValues` column, if it's nested */ def filterFileList( - partitionColumns: Seq[String], + partitionSchema: StructType, files: DataFrame, partitionFilters: Seq[Expression], partitionColumnPrefixes: Seq[String] = Nil): DataFrame = { val rewrittenFilters = rewritePartitionFilters( - partitionColumns, + partitionSchema, files.sparkSession.sessionState.conf.resolver, partitionFilters, partitionColumnPrefixes) @@ -779,22 +808,30 @@ object DeltaLog extends DeltaLogging { * @param partitionColumnPrefixes The path to the `partitionValues` column, if it's nested */ def rewritePartitionFilters( - partitionColumns: Seq[String], + partitionSchema: StructType, resolver: Resolver, partitionFilters: Seq[Expression], partitionColumnPrefixes: Seq[String] = Nil): Seq[Expression] = { - partitionFilters.map(_.transform { + partitionFilters.map(_.transformUp { case a: Attribute => - val colName = partitionColumns.find(resolver(_, a.name)).getOrElse(a.name) - UnresolvedAttribute(partitionColumnPrefixes ++ Seq("partitionValues", colName)) - }.transform { - // TODO(SC-10573): This is a temporary fix. - // What we really need to do is ensure that the partition filters are evaluated against - // the actual partition values. Right now they're evaluated against a String-casted version - // of the partition value in AddFile. - // As a warmfixable change, we're just transforming the only operator we've seen cause - // problems. - case InSet(a, set) => In(a, set.toSeq.map(Literal(_))) + // If we have a special column name, e.g. `a.a`, then an UnresolvedAttribute returns + // the column name as '`a.a`' instead of 'a.a', therefore we need to strip the backticks. + val unquoted = a.name.stripPrefix("`").stripSuffix("`") + val partitionCol = partitionSchema.find { field => resolver(field.name, unquoted) } + partitionCol match { + case Some(StructField(name, dataType, _, _)) => + Cast( + UnresolvedAttribute(partitionColumnPrefixes ++ Seq("partitionValues", name)), + dataType) + case None => + // This should not be able to happen, but the case was present in the original code so + // we kept it to be safe. + log.error(s"Partition filter referenced column ${a.name} not in the partition schema") + UnresolvedAttribute(partitionColumnPrefixes ++ Seq("partitionValues", a.name)) + } }) } + + private lazy val COMMIT_FILE_FORMAT = new JsonFileFormat + private lazy val CHECKPOINT_FILE_FORMAT = new ParquetFileFormat } diff --git a/src/main/scala/org/apache/spark/sql/delta/DeltaLogFileIndex.scala b/src/main/scala/org/apache/spark/sql/delta/DeltaLogFileIndex.scala new file mode 100644 index 00000000000..b0ced21117b --- /dev/null +++ b/src/main/scala/org/apache/spark/sql/delta/DeltaLogFileIndex.scala @@ -0,0 +1,65 @@ +/* + * Copyright 2019 Databricks, Inc. + * + * 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.hadoop.fs._ + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.execution.datasources.{FileFormat, FileIndex, PartitionDirectory} +import org.apache.spark.sql.types.StructType + +/** + * A specialized file index for files found in the _delta_log directory. By using this file index, + * we avoid any additional file listing, partitioning inference, and file existence checks when + * computing the state of a Delta table. + * + * @param format The file format of the log files. Currently "parquet" or "json" + * @param files The files to read + */ +case class DeltaLogFileIndex(format: FileFormat, files: Array[FileStatus]) extends FileIndex { + + override lazy val rootPaths: Seq[Path] = files.map(_.getPath) + + override def listFiles( + partitionFilters: Seq[Expression], + dataFilters: Seq[Expression]): Seq[PartitionDirectory] = { + PartitionDirectory(InternalRow(), files) :: Nil + } + + override val inputFiles: Array[String] = files.map(_.getPath).map(_.toString) + + override def refresh(): Unit = {} + + override val sizeInBytes: Long = files.map(_.getLen).sum + + override def partitionSchema: StructType = new StructType() + + override def listFiles( + partitionFilters: Seq[Expression], + dynamicPartitionFilters: Seq[Expression], + dataFilters: Seq[Expression]): Seq[PartitionDirectory] = { + listFiles(partitionFilters, dataFilters) + } +} + +object DeltaLogFileIndex { + + def apply(format: FileFormat, fs: FileSystem, paths: Seq[Path]): DeltaLogFileIndex = { + DeltaLogFileIndex(format, paths.map(fs.getFileStatus).toArray) + } +} 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 024d09d55a1..3500ea2115a 100644 --- a/src/main/scala/org/apache/spark/sql/delta/DeltaOperations.scala +++ b/src/main/scala/org/apache/spark/sql/delta/DeltaOperations.scala @@ -38,6 +38,8 @@ object DeltaOperations { val parameters: Map[String, Any] lazy val jsonEncodedValues: Map[String, String] = parameters.mapValues(JsonUtils.toJson(_)) + + val metricParameters: Seq[String] = Seq() } /** Recorded during batch inserts. Predicates can be provided for overwrites. */ @@ -48,6 +50,11 @@ object DeltaOperations { override val parameters: Map[String, Any] = Map("mode" -> mode.name()) ++ partitionBy.map("partitionBy" -> JsonUtils.toJson(_)) ++ predicate.map("predicate" -> _) + + override val metricParameters: Seq[String] = Seq( + "numFiles", + "numOutputBytes", + "numOutputRows") } /** Recorded during streaming inserts. */ case class StreamingUpdate( @@ -107,6 +114,15 @@ object DeltaOperations { deletePredicate.map("deletePredicate" -> _).toMap ++ insertPredicate.map("insertPredicate" -> _).toMap } + override val metricParameters: Seq[String] = Seq( + "numSourceRows", + "numTargetRowsInserted", + "numTargetRowsUpdated", + "numTargetRowsDeleted", + "numTargetRowsCopied", + "numOutputRows", + "numTargetFilesAdded", + "numTargetFilesRemoved") } /** Recorded when an update operation is committed to the table. */ case class Update(predicate: Option[String]) extends Operation("UPDATE") { @@ -121,6 +137,20 @@ object DeltaOperations { "partitionBy" -> JsonUtils.toJson(metadata.partitionColumns), "properties" -> JsonUtils.toJson(metadata.configuration)) } + /** Recorded when the table is replaced. */ + case class ReplaceTable( + metadata: Metadata, + isManaged: Boolean, + orCreate: Boolean, + asSelect: Boolean = false) + extends Operation(s"${if (orCreate) "CREATE OR " else ""}REPLACE TABLE" + + s"${if (asSelect) " AS SELECT" else ""}") { + override val parameters: Map[String, Any] = Map( + "isManaged" -> isManaged.toString, + "description" -> Option(metadata.description), + "partitionBy" -> JsonUtils.toJson(metadata.partitionColumns), + "properties" -> JsonUtils.toJson(metadata.configuration)) + } /** Recorded when the table properties are set. */ case class SetTableProperties( properties: Map[String, String]) extends Operation("SET TBLPROPERTIES") { @@ -213,6 +243,7 @@ object DeltaOperations { "newSchema" -> JsonUtils.toJson(newSchema)) } + private def structFieldToMap(colPath: Seq[String], field: StructField): Map[String, Any] = { Map( "name" -> UnresolvedAttribute(colPath :+ field.name).name, 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 3f3e030513f..52c8fe302ea 100644 --- a/src/main/scala/org/apache/spark/sql/delta/DeltaOptions.scala +++ b/src/main/scala/org/apache/spark/sql/delta/DeltaOptions.scala @@ -22,7 +22,7 @@ import java.util.regex.PatternSyntaxException import scala.util.Try import scala.util.matching.Regex -import org.apache.spark.sql.delta.DeltaOptions.{MERGE_SCHEMA_OPTION, OVERWRITE_SCHEMA_OPTION} +import org.apache.spark.sql.delta.DeltaOptions.{DATA_CHANGE_OPTION, MERGE_SCHEMA_OPTION, OVERWRITE_SCHEMA_OPTION} import org.apache.spark.sql.delta.metering.DeltaLogging import org.apache.spark.sql.delta.sources.DeltaSQLConf @@ -77,6 +77,17 @@ trait DeltaWriteOptionsImpl extends DeltaOptionParser { def canOverwriteSchema: Boolean = { options.get(OVERWRITE_SCHEMA_OPTION).map(toBoolean(_, OVERWRITE_SCHEMA_OPTION)).getOrElse(false) } + + /** + * Whether to write new data to the table or just rearrange data that is already + * part of the table. This option declares that the data being written by this job + * does not change any data in the table and merely rearranges existing data. + * This makes sure streaming queries reading from this table will not see any new changes + */ + def rearrangeOnly: Boolean = { + options.get(DATA_CHANGE_OPTION).map(!toBoolean(_, DATA_CHANGE_OPTION)).getOrElse(false) + } + } trait DeltaReadOptions extends DeltaOptionParser { @@ -135,6 +146,7 @@ object DeltaOptions extends DeltaLogging { val IGNORE_CHANGES_OPTION = "ignoreChanges" val IGNORE_DELETES_OPTION = "ignoreDeletes" val OPTIMIZE_WRITE_OPTION = "optimizeWrite" + val DATA_CHANGE_OPTION = "dataChange" val validOptionKeys : Set[String] = Set( REPLACE_WHERE_OPTION, @@ -146,6 +158,7 @@ object DeltaOptions extends DeltaLogging { IGNORE_CHANGES_OPTION, IGNORE_DELETES_OPTION, OPTIMIZE_WRITE_OPTION, + DATA_CHANGE_OPTION, "queryName", "checkpointLocation", "path", 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 6d2e5508d61..ca0bd0becd1 100644 --- a/src/main/scala/org/apache/spark/sql/delta/DeltaTable.scala +++ b/src/main/scala/org/apache/spark/sql/delta/DeltaTable.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.delta // scalastyle:off import.ordering.noEmptyLine +import java.util.Locale + import scala.util.Try import org.apache.hadoop.fs.Path @@ -25,8 +27,10 @@ import org.apache.spark.sql.delta.metering.DeltaLogging import org.apache.spark.sql.delta.sources.DeltaSourceUtils import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.catalog.CatalogTable +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException +import org.apache.spark.sql.catalyst.catalog.{CatalogTable, SessionCatalog} import org.apache.spark.sql.catalyst.expressions.{Expression, PredicateHelper, SubqueryExpression} +import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.datasources.{FileIndex, HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.internal.SQLConf @@ -48,17 +52,17 @@ object DeltaTable { * Extractor Object for pulling out the full table scan of a Delta table. */ object DeltaFullTable { - def unapply(a: LogicalRelation): Option[TahoeLogFileIndex] = a match { - case LogicalRelation(HadoopFsRelation(index: TahoeLogFileIndex, _, _, _, _, _), _, _, _) => - if (index.partitionFilters.isEmpty && index.versionToUse.isEmpty) { + def unapply(a: LogicalPlan): Option[TahoeLogFileIndex] = a match { + case PhysicalOperation(_, filters, DeltaTable(index: TahoeLogFileIndex)) => + if (index.partitionFilters.isEmpty && index.versionToUse.isEmpty && filters.isEmpty) { Some(index) } else if (index.versionToUse.nonEmpty) { throw new AnalysisException( s"Expect a full scan of the latest version of the Delta source, but found a historical " + - s"scan of version ${index.versionToUse.get}") + s"scan of version ${index.versionToUse.get}") } else { throw new AnalysisException( - s"Expect a full scan of Delta sources, but found the partial scan. path:${index.path}") + s"Expect a full scan of Delta sources, but found a partial scan. path:${index.path}") } case _ => None @@ -88,6 +92,32 @@ object DeltaTableUtils extends PredicateHelper findDeltaTableRoot(spark, path).isDefined } + /** + * Checks whether TableIdentifier is a path or a table name + * We assume it is a path unless the table and database both exist in the catalog@param catalog + * + * @param tableIdent the provided table or path + * @return true if using table name, false if using path, error otherwise + */ + def isCatalogTable(catalog: SessionCatalog, tableIdent: TableIdentifier): Boolean = { + val dbExists = tableIdent.database.forall(catalog.databaseExists) + val dbNameIsAlsoValidFormatName = + tableIdent.database.getOrElse("").toLowerCase(Locale.ROOT) == "parquet" || + DeltaSourceUtils.isDeltaDataSourceName(tableIdent.database.getOrElse("")) + + // If db doesnt exist or db is called parquet/delta/tahoe then check if path exists + if ((!dbExists || dbNameIsAlsoValidFormatName) && new Path(tableIdent.table).isAbsolute) { + return false + } + + // check for dbexists otherwise catalog.tableExists may throw NoSuchDatabaseException + if ((dbExists || tableIdent.database.isEmpty) && catalog.tableExists(tableIdent)) { + true + } else { + throw new NoSuchTableException(tableIdent.database.getOrElse(""), tableIdent.table) + } + } + /** Find the root of a Delta table from the provided path. */ def findDeltaTableRoot( spark: SparkSession, diff --git a/src/main/scala/org/apache/spark/sql/delta/MetadataCleanup.scala b/src/main/scala/org/apache/spark/sql/delta/MetadataCleanup.scala index 5a13c0bbcb6..4ef33909475 100644 --- a/src/main/scala/org/apache/spark/sql/delta/MetadataCleanup.scala +++ b/src/main/scala/org/apache/spark/sql/delta/MetadataCleanup.scala @@ -35,8 +35,10 @@ trait MetadataCleanup extends DeltaLogging { * Returns the duration in millis for how long to keep around obsolete logs. We may keep logs * beyond this duration until the next calendar day to avoid constantly creating checkpoints. */ - def deltaRetentionMillis: Long = - DeltaConfigs.LOG_RETENTION.fromMetaData(metadata).milliseconds() + def deltaRetentionMillis: Long = { + val interval = DeltaConfigs.LOG_RETENTION.fromMetaData(metadata) + DeltaConfigs.getMilliSeconds(interval) + } override def doLogCleanup(): Unit = { if (enableExpiredLogCleanup) { 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 58646680806..426d4db1087 100644 --- a/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala +++ b/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala @@ -21,12 +21,14 @@ import java.util.ConcurrentModificationException import java.util.concurrent.TimeUnit.NANOSECONDS import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.{ArrayBuffer, HashSet} import scala.util.control.NonFatal import com.databricks.spark.util.TagDefinitions.TAG_LOG_STORE_CLASS +import org.apache.spark.sql.delta.DeltaOperations.Operation import org.apache.spark.sql.delta.actions._ import org.apache.spark.sql.delta.files._ +import org.apache.spark.sql.delta.hooks.{GenerateSymlinkManifest, PostCommitHook} import org.apache.spark.sql.delta.metering.DeltaLogging import org.apache.spark.sql.delta.schema.SchemaUtils import org.apache.spark.sql.delta.sources.DeltaSQLConf @@ -127,7 +129,7 @@ object OptimisticTransaction { * * This trait is not thread-safe. */ -trait OptimisticTransactionImpl extends TransactionalWrite { +trait OptimisticTransactionImpl extends TransactionalWrite with SQLMetricsReporting { import org.apache.spark.sql.delta.util.FileNames._ @@ -144,6 +146,15 @@ trait OptimisticTransactionImpl extends TransactionalWrite { /** Tracks the appIds that have been seen by this transaction. */ protected val readTxn = new ArrayBuffer[String] + /** + * Tracks the data that could have been seen by recording the partition + * predicates by which files have been queried by by this transaction. + */ + protected val readPredicates = new ArrayBuffer[Expression] + + /** Tracks specific files that have been seen by this transaction. */ + protected val readFiles = new HashSet[AddFile] + /** Tracks if this transaction has already committed. */ protected var committed = false @@ -154,12 +165,6 @@ trait OptimisticTransactionImpl extends TransactionalWrite { protected var commitStartNano = -1L protected var commitInfo: CommitInfo = _ - /** - * Tracks if this transaction depends on any data files. This flag must be set if this transaction - * reads any data explicitly or implicitly (e.g., delete, update and overwrite). - */ - protected var dependsOnFiles: Boolean = false - /** The version that this transaction is reading from. */ def readVersion: Long = snapshot.version @@ -172,6 +177,8 @@ trait OptimisticTransactionImpl extends TransactionalWrite { snapshot.metadata } + protected val postCommitHooks = new ArrayBuffer[PostCommitHook]() + /** Returns the metadata at the current point in the log. */ def metadata: Metadata = newMetadata.getOrElse(snapshotMetadata) @@ -210,13 +217,18 @@ trait OptimisticTransactionImpl extends TransactionalWrite { /** Returns files matching the given predicates. */ def filterFiles(filters: Seq[Expression]): Seq[AddFile] = { - dependsOnFiles = true - snapshot.filesForScan(Nil, filters).files + val scan = snapshot.filesForScan(Nil, filters) + val partitionFilters = filters.filter { f => + DeltaTableUtils.isPredicatePartitionColumnsOnly(f, metadata.partitionColumns, spark) + } + readPredicates += partitionFilters.reduceLeftOption(And).getOrElse(Literal(true)) + readFiles ++= scan.files + scan.files } /** Mark the entire table as tainted by this transaction. */ def readWholeTable(): Unit = { - dependsOnFiles = true + readPredicates += Literal(true) } /** @@ -227,6 +239,17 @@ trait OptimisticTransactionImpl extends TransactionalWrite { snapshot.transactions.getOrElse(id, -1L) } + /** + * Return the operation metrics for the operation if it is enabled + */ + def getOperationMetrics(op: Operation): Option[Map[String, String]] = { + if (spark.conf.get(DeltaSQLConf.DELTA_HISTORY_METRICS_ENABLED)) { + Some(getMetricsForOperation(op)) + } else { + None + } + } + /** * 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 @@ -243,7 +266,19 @@ trait OptimisticTransactionImpl extends TransactionalWrite { // Try to commit at the next version. var finalActions = prepareCommit(actions, op) + // Find the isolation level to use for this commit + val noDataChanged = actions.collect { case f: FileAction => f.dataChange }.forall(_ == false) + val isolationLevelToUse = if (noDataChanged) { + // If no data has changed (i.e. its is only being rearranged), then SnapshotIsolation + // provides Serializable guarantee. Hence, allow reduced conflict detection by using + // SnapshotIsolation of what the table isolation level is. + SnapshotIsolation + } else { + Serializable + } + val isBlindAppend = { + val dependsOnFiles = readPredicates.nonEmpty || readFiles.nonEmpty val onlyAddFiles = finalActions.collect { case f: FileAction => f }.forall(_.isInstanceOf[AddFile]) onlyAddFiles && !dependsOnFiles @@ -257,11 +292,18 @@ trait OptimisticTransactionImpl extends TransactionalWrite { Map.empty, Some(readVersion).filter(_ >= 0), None, - Some(isBlindAppend)) + Some(isBlindAppend), + getOperationMetrics(op)) finalActions = commitInfo +: finalActions } - val commitVersion = doCommit(snapshot.version + 1, finalActions, 0) + // Register post-commit hooks if any + lazy val hasFileActions = finalActions.collect { case f: FileAction => f }.nonEmpty + if (DeltaConfigs.SYMLINK_FORMAT_MANIFEST_ENABLED.fromMetaData(metadata) && hasFileActions) { + registerPostCommitHook(GenerateSymlinkManifest) + } + + val commitVersion = doCommit(snapshot.version + 1, finalActions, 0, isolationLevelToUse) logInfo(s"Committed delta #$commitVersion to ${deltaLog.logPath}") postCommit(commitVersion, finalActions) commitVersion @@ -274,6 +316,9 @@ trait OptimisticTransactionImpl extends TransactionalWrite { deltaLog, "delta.commit.failure", data = Map("exception" -> Utils.exceptionString(e))) throw e } + + runPostCommitHooks(version, actions) + version } @@ -346,13 +391,17 @@ trait OptimisticTransactionImpl extends TransactionalWrite { private def doCommit( attemptVersion: Long, actions: Seq[Action], - attemptNumber: Int): Long = deltaLog.lockInterruptibly { + attemptNumber: Int, + isolationLevel: IsolationLevel): Long = deltaLog.lockInterruptibly { try { - logDebug(s"Attempting to commit version $attemptVersion with ${actions.size} actions") + logDebug( + s"Attempting to commit version $attemptVersion with ${actions.size} actions with " + + s"$isolationLevel isolation level") deltaLog.store.write( deltaFile(deltaLog.logPath, attemptVersion), actions.map(_.json).toIterator) + val commitTime = System.nanoTime() val postCommitSnapshot = deltaLog.update() if (postCommitSnapshot.version < attemptVersion) { @@ -394,7 +443,7 @@ trait OptimisticTransactionImpl extends TransactionalWrite { attemptVersion } catch { case e: java.nio.file.FileAlreadyExistsException => - checkAndRetry(attemptVersion, actions, attemptNumber) + checkAndRetry(attemptVersion, actions, attemptNumber, isolationLevel) } } @@ -406,22 +455,38 @@ trait OptimisticTransactionImpl extends TransactionalWrite { protected def checkAndRetry( checkVersion: Long, actions: Seq[Action], - attemptNumber: Int): Long = recordDeltaOperation( + attemptNumber: Int, + commitIsolationLevel: IsolationLevel): Long = recordDeltaOperation( deltaLog, "delta.commit.retry", tags = Map(TAG_LOG_STORE_CLASS -> deltaLog.store.getClass.getName)) { - deltaLog.update() - val nextAttempt = deltaLog.snapshot.version + 1 - (checkVersion until nextAttempt).foreach { version => + import _spark.implicits._ + + val nextAttemptVersion = getNextAttemptVersion(checkVersion) + (checkVersion until nextAttemptVersion).foreach { version => + // Actions of a commit which went in before ours val winningCommitActions = deltaLog.store.read(deltaFile(deltaLog.logPath, version)).map(Action.fromJson) + + // Categorize all the actions that have happened since the transaction read. val metadataUpdates = winningCommitActions.collect { case a: Metadata => a } + val removedFiles = winningCommitActions.collect { case a: RemoveFile => a } val txns = winningCommitActions.collect { case a: SetTransaction => a } val protocol = winningCommitActions.collect { case a: Protocol => a } val commitInfo = winningCommitActions.collectFirst { case a: CommitInfo => a }.map( ci => ci.copy(version = Some(version))) - val fileActions = winningCommitActions.collect { case f: FileAction => f } + + val blindAppendAddedFiles = mutable.ArrayBuffer[AddFile]() + val changedDataAddedFiles = mutable.ArrayBuffer[AddFile]() + + val isBlindAppendOption = commitInfo.flatMap(_.isBlindAppend) + if (isBlindAppendOption.getOrElse(false)) { + blindAppendAddedFiles ++= winningCommitActions.collect { case a: AddFile => a } + } else { + changedDataAddedFiles ++= winningCommitActions.collect { case a: AddFile => a } + } + // If the log protocol version was upgraded, make sure we are still okay. // Fail the transaction if we're trying to upgrade protocol ourselves. if (protocol.nonEmpty) { @@ -434,21 +499,123 @@ trait OptimisticTransactionImpl extends TransactionalWrite { case _ => } } + // Fail if the metadata is different than what the txn read. if (metadataUpdates.nonEmpty) { throw new MetadataChangedException(commitInfo) } - // Fail if the data is different than what the txn read. - if (dependsOnFiles && fileActions.nonEmpty) { - throw new ConcurrentWriteException(commitInfo) + + // Fail if new files have been added that the txn should have read. + val addedFilesToCheckForConflicts = commitIsolationLevel match { + case Serializable => changedDataAddedFiles ++ blindAppendAddedFiles + case WriteSerializable => changedDataAddedFiles // don't conflict with blind appends + case SnapshotIsolation => Seq.empty + } + val predicatesMatchingAddedFiles = ExpressionSet(readPredicates).iterator.flatMap { p => + val conflictingFile = DeltaLog.filterFileList( + metadata.partitionSchema, + addedFilesToCheckForConflicts.toDF(), p :: Nil).as[AddFile].take(1) + + conflictingFile.headOption.map(f => getPrettyPartitionMessage(f.partitionValues)) + }.take(1).toArray + + if (predicatesMatchingAddedFiles.nonEmpty) { + val isWriteSerializable = commitIsolationLevel == WriteSerializable + val onlyAddFiles = + winningCommitActions.collect { case f: FileAction => f }.forall(_.isInstanceOf[AddFile]) + + val retryMsg = + if (isWriteSerializable && onlyAddFiles && isBlindAppendOption.isEmpty) { + // This transaction was made by an older version which did not set `isBlindAppend` flag. + // So even if it looks like an append, we don't know for sure if it was a blind append + // or not. So we suggest them to upgrade all there workloads to latest version. + Some( + "Upgrading all your concurrent writers to use the latest Delta Lake may " + + "avoid this error. Please upgrade and then retry this operation again.") + } else None + throw new ConcurrentAppendException(commitInfo, predicatesMatchingAddedFiles.head, retryMsg) + } + + // Fail if files have been deleted that the txn read. + val readFilePaths = readFiles.map(f => f.path -> f.partitionValues).toMap + val deleteReadOverlap = removedFiles.find(r => readFilePaths.contains(r.path)) + if (deleteReadOverlap.nonEmpty) { + val filePath = deleteReadOverlap.get.path + val partition = getPrettyPartitionMessage(readFilePaths(filePath)) + throw new ConcurrentDeleteReadException(commitInfo, s"$filePath in $partition") } + + // Fail if a file is deleted twice. + val txnDeletes = actions.collect { case r: RemoveFile => r }.map(_.path).toSet + val deleteOverlap = removedFiles.map(_.path).toSet intersect txnDeletes + if (deleteOverlap.nonEmpty) { + throw new ConcurrentDeleteDeleteException(commitInfo, deleteOverlap.head) + } + // Fail if idempotent transactions have conflicted. val txnOverlap = txns.map(_.appId).toSet intersect readTxn.toSet if (txnOverlap.nonEmpty) { throw new ConcurrentTransactionException(commitInfo) } } - logInfo(s"No logical conflicts with deltas [$checkVersion, $nextAttempt), retrying.") - doCommit(nextAttempt, actions, attemptNumber + 1) + + logInfo(s"No logical conflicts with deltas [$checkVersion, $nextAttemptVersion), retrying.") + doCommit(nextAttemptVersion, actions, attemptNumber + 1, commitIsolationLevel) + } + + /** Returns the next attempt version given the last attempted version */ + protected def getNextAttemptVersion(previousAttemptVersion: Long): Long = { + deltaLog.update() + deltaLog.snapshot.version + 1 + } + + /** A helper function for pretty printing a specific partition directory. */ + protected def getPrettyPartitionMessage(partitionValues: Map[String, String]): String = { + if (metadata.partitionColumns.isEmpty) { + "the root of the table" + } else { + val partition = metadata.partitionColumns.map { name => + s"$name=${partitionValues(name)}" + }.mkString("[", ", ", "]") + s"partition ${partition}" + } + } + + /** Register a hook that will be executed once a commit is successful. */ + def registerPostCommitHook(hook: PostCommitHook): Unit = { + if (!postCommitHooks.contains(hook)) { + postCommitHooks.append(hook) + } + } + + /** Executes the registered post commit hooks. */ + protected def runPostCommitHooks( + version: Long, + committedActions: Seq[Action]): Unit = { + assert(committed, "Can't call post commit hooks before committing") + + // Keep track of the active txn because hooks may create more txns and overwrite the active one. + val activeCommit = OptimisticTransaction.getActive() + OptimisticTransaction.clearActive() + + try { + postCommitHooks.foreach { hook => + try { + hook.run(spark, this, committedActions) + } catch { + case NonFatal(e) => + logWarning(s"Error when executing post-commit hook ${hook.name} " + + s"for commit $version", e) + recordDeltaEvent(deltaLog, "delta.commit.hook.failure", data = Map( + "hook" -> hook.name, + "version" -> version, + "exception" -> e.toString + )) + hook.handleError(e, version) + } + } + } finally { + activeCommit.foreach(OptimisticTransaction.setActive) + } } } 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 f75a89d1aa4..7ee21f0d803 100644 --- a/src/main/scala/org/apache/spark/sql/delta/PartitionFiltering.scala +++ b/src/main/scala/org/apache/spark/sql/delta/PartitionFiltering.scala @@ -35,7 +35,7 @@ trait PartitionFiltering { } val files = DeltaLog.filterFileList( - metadata.partitionColumns, + metadata.partitionSchema, allFiles.toDF(), partitionFilters).as[AddFile].collect() 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 1eff4c63e95..2cea6fddb9c 100644 --- a/src/main/scala/org/apache/spark/sql/delta/Snapshot.scala +++ b/src/main/scala/org/apache/spark/sql/delta/Snapshot.scala @@ -29,6 +29,8 @@ import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +import org.apache.spark.sql.catalyst.plans.logical.Union +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.expressions.UserDefinedFunction import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.StructType @@ -53,7 +55,7 @@ class Snapshot( val path: Path, val version: Long, previousSnapshot: Option[Dataset[SingleAction]], - files: Seq[Path], + files: Seq[DeltaLogFileIndex], val minFileRetentionTimestamp: Long, val deltaLog: DeltaLog, val timestamp: Long, @@ -168,17 +170,28 @@ class Snapshot( val numIndexedCols: Int = DeltaConfigs.DATA_SKIPPING_NUM_INDEXED_COLS.fromMetaData(metadata) /** - * Load the transaction logs from paths. The files here may have different file formats and the - * file format can be extracted from the file extensions. - * - * Here we are reading the transaction log, and we need to bypass the ACL checks - * for SELECT any file permissions. + * Load the transaction logs from file indices. The files here may have different file formats + * and the file format can be extracted from the file extensions. */ - private def load(paths: Seq[Path]): Dataset[SingleAction] = { - val pathAndFormats = paths.map(_.toString).map(path => path -> path.split("\\.").last) - pathAndFormats.groupBy(_._2).map { case (format, paths) => - spark.read.format(format).schema(logSchema).load(paths.map(_._1): _*).as[SingleAction] - }.reduceOption(_.union(_)).getOrElse(emptyActions) + private def load( + files: Seq[DeltaLogFileIndex]): Dataset[SingleAction] = { + val relations = files.map { index: DeltaLogFileIndex => + val fsRelation = HadoopFsRelation( + index, + index.partitionSchema, + logSchema, + None, + index.format, + Map.empty[String, String])(spark) + LogicalRelation(fsRelation) + } + if (relations.length == 1) { + Dataset[SingleAction](spark, relations.head) + } else if (relations.nonEmpty) { + Dataset[SingleAction](spark, Union(relations)) + } else { + emptyActions + } } private def emptyActions = 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 1876e8b8c31..363d9903f89 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 @@ -254,7 +254,8 @@ case class CommitInfo( readVersion: Option[Long], isolationLevel: Option[String], /** Whether this commit has blindly appended without caring about existing files */ - isBlindAppend: Option[Boolean]) extends Action with CommitMarker { + isBlindAppend: Option[Boolean], + operationMetrics: Option[Map[String, String]]) extends Action with CommitMarker { override def wrap: SingleAction = SingleAction(commitInfo = this) override def withTimestamp(timestamp: Long): CommitInfo = { @@ -296,7 +297,7 @@ object NotebookInfo { object CommitInfo { def empty(version: Option[Long] = None): CommitInfo = { - CommitInfo(version, null, None, None, null, null, None, None, None, None, None, None) + CommitInfo(version, null, None, None, null, null, None, None, None, None, None, None, None) } def apply( @@ -306,7 +307,8 @@ object CommitInfo { commandContext: Map[String, String], readVersion: Option[Long], isolationLevel: Option[String], - isBlindAppend: Option[Boolean]): CommitInfo = { + isBlindAppend: Option[Boolean], + operationMetrics: Option[Map[String, String]]): CommitInfo = { val getUserName = commandContext.get("user").flatMap { case "unknown" => None case other => Option(other) @@ -324,7 +326,8 @@ object CommitInfo { commandContext.get("clusterId"), readVersion, isolationLevel, - isBlindAppend + isBlindAppend, + operationMetrics ) } } 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 ec1b86aa669..231265b138b 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 @@ -39,6 +39,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.expressions.Cast import org.apache.spark.sql.execution.command.RunnableCommand import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, ParquetToSparkSchemaConverter} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{StringType, StructField, StructType} import org.apache.spark.util.{SerializableConfiguration, Utils} @@ -173,8 +174,7 @@ abstract class ConvertToDeltaCommandBase( spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_IMPORT_BATCH_SIZE_STATS_COLLECTION) val addFilesIter = fileListResult.asScala.grouped(statsBatchSize).flatMap { batch => - val resolver = spark.sessionState.conf.resolver - val adds = batch.map(createAddFile(_, txn.deltaLog.dataPath, fs, resolver)) + val adds = batch.map(createAddFile(_, txn.deltaLog.dataPath, fs, spark.sessionState.conf)) adds.toIterator } streamWrite( @@ -193,12 +193,16 @@ abstract class ConvertToDeltaCommandBase( } protected def createAddFile( - file: SerializableFileStatus, basePath: Path, fs: FileSystem, resolver: Resolver): AddFile = { + file: SerializableFileStatus, + basePath: Path, + fs: FileSystem, + conf: SQLConf): AddFile = { val path = file.getPath val pathStr = file.getPath.toUri.toString val dateFormatter = DateFormatter() val timestampFormatter = TimestampFormatter(timestampPartitionPattern, java.util.TimeZone.getDefault) + val resolver = conf.resolver val (partitionOpt, _) = PartitionUtils.parsePartition( path, typeInference = false, @@ -215,10 +219,11 @@ abstract class ConvertToDeltaCommandBase( pathStr, partValues.columnNames, partitionColNames) } + val tz = Option(conf.sessionLocalTimeZone) // Check if the partition value can be casted to the provided type partValues.literals.zip(partitionFields).foreach { case (literal, field) => - if (literal.eval() != null && Cast(literal, field.dataType).eval() == null) { - val partitionValue = Cast(literal, StringType).eval() + if (literal.eval() != null && Cast(literal, field.dataType, tz).eval() == null) { + val partitionValue = Cast(literal, StringType, tz).eval() val partitionValueStr = Option(partitionValue).map(_.toString).orNull throw DeltaErrors.castPartitionValueException(partitionValueStr, field.dataType) } @@ -226,7 +231,7 @@ abstract class ConvertToDeltaCommandBase( val values = partValues .literals - .map(l => Cast(l, StringType).eval()) + .map(l => Cast(l, StringType, tz).eval()) .map(Option(_).map(_.toString).orNull) partitionColNames.zip(partValues.columnNames).foreach { case (expected, parsed) => @@ -314,7 +319,8 @@ abstract class ConvertToDeltaCommandBase( context, readVersion = None, isolationLevel = None, - isBlindAppend = None) + isBlindAppend = None, + None) val extraActions = Seq(commitInfo, Protocol(), metadata) val actions = extraActions.toIterator ++ addFiles 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 new file mode 100644 index 00000000000..c4dacafafb0 --- /dev/null +++ b/src/main/scala/org/apache/spark/sql/delta/commands/DeltaGenerateCommand.scala @@ -0,0 +1,68 @@ +/* + * Copyright 2019 Databricks, Inc. + * + * 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 + +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.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 { + + import DeltaGenerateCommand._ + + override def run(sparkSession: SparkSession): Seq[Row] = { + if (!modeNameToGenerationFunc.contains(modeName)) { + throw DeltaErrors.unsupportedGenerateModeException(modeName) + } + val tablePath = getPath(sparkSession, tableId) + val deltaLog = DeltaLog.forTable(sparkSession, tablePath) + if (deltaLog.snapshot.version < 0) { + throw new AnalysisException(s"Delta table not found at $tablePath.") + } + val generationFunc = modeNameToGenerationFunc(modeName) + generationFunc(sparkSession, deltaLog) + Seq.empty + } +} + +object DeltaGenerateCommand { + val modeNameToGenerationFunc = CaseInsensitiveMap( + Map[String, (SparkSession, DeltaLog) => Unit]( + "symlink_format_manifest" -> GenerateSymlinkManifest.generateFullManifest + )) +} diff --git a/src/main/scala/org/apache/spark/sql/delta/commands/DescribeDeltaDetailsCommand.scala b/src/main/scala/org/apache/spark/sql/delta/commands/DescribeDeltaDetailsCommand.scala new file mode 100644 index 00000000000..86d1020b308 --- /dev/null +++ b/src/main/scala/org/apache/spark/sql/delta/commands/DescribeDeltaDetailsCommand.scala @@ -0,0 +1,24 @@ +/* + * Copyright 2019 Databricks, Inc. + * + * 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 + +import org.apache.spark.sql.catalyst.TableIdentifier + +case class DescribeDeltaDetailCommand( + override val path: Option[String], + override val tableIdentifier: Option[TableIdentifier]) + extends DescribeDeltaDetailCommandBase diff --git a/src/main/scala/org/apache/spark/sql/delta/commands/DescribeDeltaDetailsCommandBase.scala b/src/main/scala/org/apache/spark/sql/delta/commands/DescribeDeltaDetailsCommandBase.scala index 7d99d3e4967..404d4bfbe39 100644 --- a/src/main/scala/org/apache/spark/sql/delta/commands/DescribeDeltaDetailsCommandBase.scala +++ b/src/main/scala/org/apache/spark/sql/delta/commands/DescribeDeltaDetailsCommandBase.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.delta.commands import java.io.FileNotFoundException import java.sql.Timestamp -import org.apache.spark.sql.delta.{DeltaLog, Snapshot} +import org.apache.spark.sql.delta.{DeltaErrors, DeltaLog, DeltaTableIdentifier, Snapshot} import org.apache.spark.sql.delta.metering.DeltaLogging import org.apache.spark.sql.delta.util.FileNames import org.apache.hadoop.fs.Path @@ -51,9 +51,11 @@ case class TableDetail( /** * A command for describing the details of a table such as the format, name, and size. */ -abstract class DescribeDeltaDetailCommandBase( - path: Option[String], - tableIdentifier: Option[TableIdentifier]) extends RunnableCommand with DeltaLogging { +trait DescribeDeltaDetailCommandBase extends RunnableCommand with DeltaLogging { + + val path: Option[String] + + val tableIdentifier: Option[TableIdentifier] private val encoder = ExpressionEncoder[TableDetail]() @@ -88,11 +90,32 @@ abstract class DescribeDeltaDetailCommandBase( * Resolve `path` and `tableIdentifier` to get the underlying storage path, and its `CatalogTable` * if it's a table. The caller will make sure either `path` or `tableIdentifier` is set but not * both. + * + * If `path` is set, return it and an empty `CatalogTable` since it's a physical path. If + * `tableIdentifier` is set, we will try to see if it's a Delta data source path (such as + * `delta.`). If so, we will return the path and an empty `CatalogTable`. Otherwise, + * we will use `SessionCatalog` to resolve `tableIdentifier`. */ protected def getPathAndTableMetadata( spark: SparkSession, path: Option[String], - tableIdentifier: Option[TableIdentifier]): (Path, Option[CatalogTable]) + tableIdentifier: Option[TableIdentifier]): (Path, Option[CatalogTable]) = { + path.map(new Path(_) -> None).orElse { + tableIdentifier.map { i => + DeltaTableIdentifier(spark, tableIdentifier.get) match { + case Some(id) if id.path.isDefined => new Path(id.path.get) -> None + case Some(id) => + throw DeltaErrors.tableNotSupportedException("DESCRIBE DETAIL") + case None => + // This is not a Delta table. + val metadata = spark.sessionState.catalog.getTableMetadata(i) + new Path(metadata.location) -> Some(metadata) + } + } + }.getOrElse { + throw DeltaErrors.missingTableIdentifierException("DESCRIBE DETAIL") + } + } private def describeNonDeltaTable(table: CatalogTable): Seq[Row] = { Seq(rowEncoder.fromRow(encoder.toRow( diff --git a/src/main/scala/org/apache/spark/sql/delta/commands/DescribeDeltaDetailsCommandOSS.scala b/src/main/scala/org/apache/spark/sql/delta/commands/DescribeDeltaDetailsCommandOSS.scala deleted file mode 100644 index 8f3e6985f2c..00000000000 --- a/src/main/scala/org/apache/spark/sql/delta/commands/DescribeDeltaDetailsCommandOSS.scala +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Copyright 2019 Databricks, Inc. - * - * 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 - -import org.apache.spark.sql.delta.{DeltaErrors, DeltaTableIdentifier} -import org.apache.hadoop.fs.Path - -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.catalog.CatalogTable - -case class DescribeDeltaDetailCommandOSS( - path: Option[String], - tableIdentifier: Option[TableIdentifier]) - extends DescribeDeltaDetailCommandBase(path, tableIdentifier) { - - /** - * If `path` is set, return it and an empty `CatalogTable` since it's a physical path. If - * `tableIdentifier` is set, we will try to see if it's a Delta data source path (such as - * `delta.`). If so, we will return the path and an empty `CatalogTable`. Otherwise, - * we will use `SessionCatalog` to resolve `tableIdentifier`. - */ - override protected def getPathAndTableMetadata( - spark: SparkSession, - path: Option[String], - tableIdentifier: Option[TableIdentifier]): (Path, Option[CatalogTable]) = { - path.map(new Path(_) -> None).orElse { - tableIdentifier.map { i => - DeltaTableIdentifier(spark, tableIdentifier.get) match { - case Some(id) if id.path.isDefined => new Path(id.path.get) -> None - case Some(id) => - throw DeltaErrors.tableNotSupportedException("DESCRIBE DETAIL") - case None => - // This is not a Delta table. - val metadata = spark.sessionState.catalog.getTableMetadata(i) - new Path(metadata.location) -> Some(metadata) - } - } - }.getOrElse { - throw DeltaErrors.missingTableIdentifierException("DESCRIBE DETAIL") - } - } -} 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 4f964a475d6..daad6593c67 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 @@ -21,14 +21,15 @@ import scala.collection.JavaConverters._ import org.apache.spark.sql.delta._ import org.apache.spark.sql.delta.actions.AddFile import org.apache.spark.sql.delta.files._ +import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.delta.util.{AnalysisHelper, SetAccumulator} import org.apache.spark.SparkContext import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.{ExpressionEncoder, RowEncoder} -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.codegen.{GeneratePredicate, Predicate} +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, Expression, Literal, PredicateHelper, UnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.command.RunnableCommand @@ -100,6 +101,9 @@ case class MergeIntoCommand( @transient private lazy val sc: SparkContext = SparkContext.getOrCreate() @transient private lazy val targetDeltaLog: DeltaLog = targetFileIndex.deltaLog + /** Whether this merge statement only inserts new data. */ + private def isInsertOnly: Boolean = matchedClauses.isEmpty && notMatchedClause.isDefined + lazy val updateClause: Option[MergeIntoUpdateClause] = matchedClauses.collectFirst { case u: MergeIntoUpdateClause => u } lazy val deleteClause: Option[MergeIntoDeleteClause] = @@ -120,14 +124,18 @@ case class MergeIntoCommand( spark: SparkSession): Seq[Row] = recordDeltaOperation(targetDeltaLog, "delta.dml.merge") { targetDeltaLog.withNewTransaction { deltaTxn => val deltaActions = { - val filesToRewrite = - recordDeltaOperation(targetDeltaLog, "delta.dml.merge.findTouchedFiles") { - findTouchedFiles(spark, deltaTxn) - } - - val newWrittenFiles = writeAllChanges(spark, deltaTxn, filesToRewrite) - filesToRewrite.map(_.remove) ++ newWrittenFiles + if (isInsertOnly && spark.conf.get(DeltaSQLConf.MERGE_INSERT_ONLY_ENABLED)) { + writeInsertsOnlyWhenNoMatchedClauses(spark, deltaTxn) + } else { + val filesToRewrite = + recordDeltaOperation(targetDeltaLog, "delta.dml.merge.findTouchedFiles") { + findTouchedFiles(spark, deltaTxn) + } + val newWrittenFiles = writeAllChanges(spark, deltaTxn, filesToRewrite) + filesToRewrite.map(_.remove) ++ newWrittenFiles + } } + deltaTxn.registerSQLMetrics(spark, metrics) deltaTxn.commit( deltaActions, DeltaOperations.Merge( @@ -215,7 +223,7 @@ case class MergeIntoCommand( // Calculate frequency of matches per source row val matchedRowCounts = collectTouchedFiles.groupBy(ROW_ID_COL).agg(sum("one").as("count")) if (matchedRowCounts.filter("count > 1").count() != 0) { - throw DeltaErrors.multipleSourceRowMatchingTargetRowInMergeException + throw DeltaErrors.multipleSourceRowMatchingTargetRowInMergeException(spark) } // Get the AddFiles using the touched file names. @@ -232,6 +240,52 @@ case class MergeIntoCommand( touchedAddFiles } + /** + * This is an optimization of the case when there is no update clause for the merge. + * We perform an left anti join on the source data to find the rows to be inserted. + */ + private def writeInsertsOnlyWhenNoMatchedClauses( + spark: SparkSession, + deltaTxn: OptimisticTransaction + ): Seq[AddFile] = withStatusCode("DELTA", s"Writing new files " + + s"for insert-only MERGE operation") { + + // UDFs to update metrics + val incrSourceRowCountExpr = makeMetricUpdateUDF("numSourceRows") + val incrInsertedCountExpr = makeMetricUpdateUDF("numTargetRowsInserted") + + val outputColNames = target.output.map(_.name) + val outputExprs = notMatchedClause.get.resolvedActions.map(_.expr) :+ incrInsertedCountExpr + val outputCols = outputExprs.zip(outputColNames).map { case (expr, name) => + new Column(Alias(expr, name)()) + } + + // source DataFrame + val sourceDF = Dataset.ofRows(spark, source) + .filter(new Column(incrSourceRowCountExpr)) + .filter(new Column(notMatchedClause.get.condition.getOrElse(Literal(true)))) + + // Skip data based on the merge condition + val conjunctivePredicates = splitConjunctivePredicates(condition) + val targetOnlyPredicates = + conjunctivePredicates.filter(_.references.subsetOf(target.outputSet)) + val dataSkippedFiles = deltaTxn.filterFiles(targetOnlyPredicates) + + // target DataFrame + val targetDF = Dataset.ofRows( + spark, buildTargetPlanWithFiles(deltaTxn, dataSkippedFiles)) + + val insertDf = sourceDF.join(targetDF, new Column(condition), "leftanti") + .select(outputCols: _*) + + val newFiles = deltaTxn.writeFiles(insertDf) + metrics("numTargetFilesBeforeSkipping") += deltaTxn.snapshot.numOfFiles + metrics("numTargetFilesAfterSkipping") += dataSkippedFiles.size + metrics("numTargetFilesRemoved") += 0 + metrics("numTargetFilesAdded") += newFiles.size + newFiles + } + /** * Write new files by reading the touched files and updating/inserting data using the source * query/table. This is implemented using a full-outer-join using the merge condition. diff --git a/src/main/scala/org/apache/spark/sql/delta/commands/UpdateWithJoinCommand.scala b/src/main/scala/org/apache/spark/sql/delta/commands/UpdateWithJoinCommand.scala index a39afa6618e..9d053be8d2d 100644 --- a/src/main/scala/org/apache/spark/sql/delta/commands/UpdateWithJoinCommand.scala +++ b/src/main/scala/org/apache/spark/sql/delta/commands/UpdateWithJoinCommand.scala @@ -173,7 +173,7 @@ case class UpdateWithJoinCommand( // Calculate frequency of matches per source row val matchedRowCounts = collectTouchedFiles.groupBy(ROW_ID_COL).agg(sum("one").as("count")) if (matchedRowCounts.filter("count > 1").count() != 0) { - throw DeltaErrors.multipleSourceRowMatchingTargetRowInMergeException + throw DeltaErrors.multipleSourceRowMatchingTargetRowInMergeException(spark) } // Get the AddFiles using the touched file names. 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 9cc567e327d..aef3821526f 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 @@ -81,7 +81,8 @@ case class WriteIntoDelta( deltaLog.assertRemovable() } } - updateMetadata(txn, data, partitionColumns, configuration, isOverwriteOperation) + val rearrangeOnly = options.rearrangeOnly + updateMetadata(txn, data, partitionColumns, configuration, isOverwriteOperation, rearrangeOnly) // Validate partition predicates val replaceWhere = options.replaceWhere @@ -108,7 +109,7 @@ case class WriteIntoDelta( case (SaveMode.Overwrite, Some(predicates)) => // Check to make sure the files we wrote out were actually valid. val matchingFiles = DeltaLog.filterFileList( - txn.metadata.partitionColumns, newFiles.toDF(), predicates).as[AddFile].collect() + txn.metadata.partitionSchema, newFiles.toDF(), predicates).as[AddFile].collect() val invalidFiles = newFiles.toSet -- matchingFiles if (invalidFiles.nonEmpty) { val badPartitions = invalidFiles @@ -122,6 +123,11 @@ case class WriteIntoDelta( case _ => Nil } - newFiles ++ deletedFiles + if (rearrangeOnly) { + newFiles.map(_.copy(dataChange = !rearrangeOnly)) ++ + deletedFiles.map(_.copy(dataChange = !rearrangeOnly)) + } else { + newFiles ++ deletedFiles + } } } diff --git a/src/main/scala/org/apache/spark/sql/delta/files/DeltaSourceSnapshot.scala b/src/main/scala/org/apache/spark/sql/delta/files/DeltaSourceSnapshot.scala index 3881429f997..13e53cf4e46 100644 --- a/src/main/scala/org/apache/spark/sql/delta/files/DeltaSourceSnapshot.scala +++ b/src/main/scala/org/apache/spark/sql/delta/files/DeltaSourceSnapshot.scala @@ -80,7 +80,7 @@ trait SnapshotIterator { import spark.implicits._ if (result == null) { result = DeltaLog.filterFileList( - snapshot.metadata.partitionColumns, + snapshot.metadata.partitionSchema, initialFiles.toDF(), partitionFilters, Seq("add")).as[IndexedFile].collect().toIterable diff --git a/src/main/scala/org/apache/spark/sql/delta/files/SQLMetricsReporting.scala b/src/main/scala/org/apache/spark/sql/delta/files/SQLMetricsReporting.scala new file mode 100644 index 00000000000..7e9e02ff8a9 --- /dev/null +++ b/src/main/scala/org/apache/spark/sql/delta/files/SQLMetricsReporting.scala @@ -0,0 +1,54 @@ +/* + * Copyright 2019 Databricks, Inc. + * + * 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.files + +import org.apache.spark.sql.delta.DeltaOperations.Operation +import org.apache.spark.sql.delta.sources.DeltaSQLConf + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.metric.SQLMetric + +/** + * This trait is used to register SQL metrics for a Delta Operation. + * Registering will allow the metrics to be instrumented via the CommitInfo and is accessible via + * DescribeHistory + */ +trait SQLMetricsReporting { + + // Map of SQL Metrics + private var operationSQLMetrics = Map[String, SQLMetric]() + + /** + * Register SQL metrics for an operation by appending the supplied metrics map to the + * operationSQLMetrics map. + */ + def registerSQLMetrics(spark: SparkSession, metrics: Map[String, SQLMetric]): Unit = { + if (spark.conf.get(DeltaSQLConf.DELTA_HISTORY_METRICS_ENABLED)) { + operationSQLMetrics = operationSQLMetrics ++ metrics + } + } + + /** + * Get the metrics for an operation based on collected SQL Metrics and filtering out + * the ones based on the metric parameters for that operation. + */ + def getMetricsForOperation(operation: Operation): Map[String, String] = { + operationSQLMetrics = operationSQLMetrics.filterKeys(s => + operation.metricParameters.contains(s)) + operationSQLMetrics.transform((_, v) => v.value.toString) + } +} 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 1d8991622da..773236f9bf1 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 @@ -184,7 +184,7 @@ class TahoeBatchFileIndex( dataFilters: Seq[Expression], keepStats: Boolean = false): Seq[AddFile] = { DeltaLog.filterFileList( - snapshot.metadata.partitionColumns, + snapshot.metadata.partitionSchema, spark.createDataset(addFiles)(addFileEncoder).toDF(), partitionFilters) .as[AddFile](addFileEncoder) .collect() diff --git a/src/main/scala/org/apache/spark/sql/delta/files/TransactionalWrite.scala b/src/main/scala/org/apache/spark/sql/delta/files/TransactionalWrite.scala index 6fcd81a0b89..e1f33dbc712 100644 --- a/src/main/scala/org/apache/spark/sql/delta/files/TransactionalWrite.scala +++ b/src/main/scala/org/apache/spark/sql/delta/files/TransactionalWrite.scala @@ -16,17 +16,21 @@ package org.apache.spark.sql.delta.files +import scala.collection.mutable.ListBuffer + import org.apache.spark.sql.delta._ import org.apache.spark.sql.delta.actions._ import org.apache.spark.sql.delta.metering.DeltaLogging import org.apache.spark.sql.delta.schema._ +import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.hadoop.fs.Path import org.apache.spark.sql.Dataset import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.datasources.FileFormatWriter +import org.apache.spark.sql.execution.datasources.{BasicWriteJobStatsTracker, FileFormatWriter, WriteJobStatsTracker} import org.apache.spark.sql.types.{ArrayType, MapType, StructType} +import org.apache.spark.util.SerializableConfiguration /** * Adds the ability to write files out as part of a transaction. Checks @@ -135,6 +139,16 @@ trait TransactionalWrite extends DeltaLogging { self: OptimisticTransactionImpl val physicalPlan = DeltaInvariantCheckerExec(queryExecution.executedPlan, invariants) + val statsTrackers: ListBuffer[WriteJobStatsTracker] = ListBuffer() + + if (spark.conf.get(DeltaSQLConf.DELTA_HISTORY_METRICS_ENABLED)) { + val basicWriteJobStatsTracker = new BasicWriteJobStatsTracker( + new SerializableConfiguration(spark.sessionState.newHadoopConf()), + BasicWriteJobStatsTracker.metrics) + registerSQLMetrics(spark, basicWriteJobStatsTracker.metrics) + statsTrackers.append(basicWriteJobStatsTracker) + } + FileFormatWriter.write( sparkSession = spark, plan = physicalPlan, @@ -144,7 +158,7 @@ trait TransactionalWrite extends DeltaLogging { self: OptimisticTransactionImpl hadoopConf = spark.sessionState.newHadoopConfWithOptions(metadata.configuration), partitionColumns = partitioningColumns, bucketSpec = None, - statsTrackers = Nil, + statsTrackers = statsTrackers, options = Map.empty) } diff --git a/src/main/scala/org/apache/spark/sql/delta/hooks/GenerateSymlinkManifest.scala b/src/main/scala/org/apache/spark/sql/delta/hooks/GenerateSymlinkManifest.scala new file mode 100644 index 00000000000..f194d58b873 --- /dev/null +++ b/src/main/scala/org/apache/spark/sql/delta/hooks/GenerateSymlinkManifest.scala @@ -0,0 +1,359 @@ +/* + * Copyright 2019 Databricks, Inc. + * + * 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.hooks + +import org.apache.spark.sql.delta._ +import org.apache.spark.sql.delta.actions._ +import org.apache.spark.sql.delta.metering.DeltaLogging +import org.apache.spark.sql.delta.storage.LogStore +import org.apache.spark.sql.delta.util.DeltaFileOperations +import org.apache.hadoop.fs.Path +import org.apache.spark.SparkEnv +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils +import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Concat, Expression, Literal, ScalaUDF} +import org.apache.spark.sql.execution.datasources.InMemoryFileIndex +import org.apache.spark.sql.functions.col +import org.apache.spark.sql.types.{BooleanType, StringType} +import org.apache.spark.util.SerializableConfiguration + +/** + * Post commit hook to generate hive-style manifests for Delta table. This is useful for + * compatibility with Presto / Athena. + */ +object GenerateSymlinkManifest extends GenerateSymlinkManifestImpl + + +trait GenerateSymlinkManifestImpl extends PostCommitHook with DeltaLogging with Serializable { + val CONFIG_NAME_ROOT = "compatibility.symlinkFormatManifest" + + val MANIFEST_LOCATION = "_symlink_format_manifest" + + val OP_TYPE_ROOT = "delta.compatibility.symlinkFormatManifest" + val FULL_MANIFEST_OP_TYPE = s"$OP_TYPE_ROOT.full" + val INCREMENTAL_MANIFEST_OP_TYPE = s"$OP_TYPE_ROOT.incremental" + + override val name: String = "Generate Symlink Format Manifest" + + override def run( + spark: SparkSession, + txn: OptimisticTransactionImpl, + committedActions: Seq[Action]): Unit = { + generateIncrementalManifest(spark, txn.deltaLog, txn.snapshot, committedActions) + } + + override def handleError(error: Throwable, version: Long): Unit = { + throw DeltaErrors.postCommitHookFailedException(this, version, name, error) + } + + /** + * Generate manifest files incrementally, that is, only for the table partitions touched by the + * given actions. + */ + protected def generateIncrementalManifest( + spark: SparkSession, + deltaLog: DeltaLog, + txnReadSnapshot: Snapshot, + actions: Seq[Action]): Unit = recordManifestGeneration(deltaLog, full = false) { + + import spark.implicits._ + val currentSnapshot = deltaLog.snapshot + + val partitionCols = currentSnapshot.metadata.partitionColumns + val manifestRootDirPath = new Path(deltaLog.dataPath, MANIFEST_LOCATION) + val hadoopConf = new SerializableConfiguration(spark.sessionState.newHadoopConf()) + val fs = deltaLog.dataPath.getFileSystem(hadoopConf.value) + if (!fs.exists(manifestRootDirPath)) { + generateFullManifest(spark, deltaLog) + return + } + + // Find all the manifest partitions that need to updated or deleted + val (allFilesInUpdatedPartitions, nowEmptyPartitions) = if (partitionCols.nonEmpty) { + // Get the partitions where files were added + val partitionsOfAddedFiles = actions.collect { case a: AddFile => a.partitionValues }.toSet + + // Get the partitions where files were deleted + val removedFileNames = + spark.createDataset(actions.collect { case r: RemoveFile => r.path }).toDF("path") + val partitionValuesOfRemovedFiles = + txnReadSnapshot.allFiles.join(removedFileNames, "path").select("partitionValues").persist() + try { + val partitionsOfRemovedFiles = + partitionValuesOfRemovedFiles.as[Map[String, String]].collect().toSet + + // Get the files present in the updated partitions + val partitionsUpdated: Set[Map[String, String]] = + partitionsOfAddedFiles ++ partitionsOfRemovedFiles + val filesInUpdatedPartitions = currentSnapshot.allFiles.filter { a => + partitionsUpdated.contains(a.partitionValues) + } + + // Find the current partitions + val currentPartitionRelativeDirs = + withRelativePartitionDir(spark, partitionCols, currentSnapshot.allFiles) + .select("relativePartitionDir").distinct() + + // Find the partitions that became empty and delete their manifests + val partitionRelativeDirsOfRemovedFiles = + withRelativePartitionDir(spark, partitionCols, partitionValuesOfRemovedFiles) + .select("relativePartitionDir").distinct() + + val partitionsThatBecameEmpty = + partitionRelativeDirsOfRemovedFiles.join( + currentPartitionRelativeDirs, Seq("relativePartitionDir"), "leftanti") + .as[String].collect() + + (filesInUpdatedPartitions, partitionsThatBecameEmpty) + } finally { + partitionValuesOfRemovedFiles.unpersist() + } + } else { + (currentSnapshot.allFiles, Array.empty[String]) + } + + val manifestFilePartitionsWritten = writeManifestFiles( + deltaLog.dataPath, + manifestRootDirPath.toString, + allFilesInUpdatedPartitions, + partitionCols, + hadoopConf) + + if (nowEmptyPartitions.nonEmpty) { + deleteManifestFiles(manifestRootDirPath.toString, nowEmptyPartitions, hadoopConf) + } + + // Post stats + val stats = SymlinkManifestStats( + filesWritten = manifestFilePartitionsWritten.size, + filesDeleted = nowEmptyPartitions.length, + partitioned = partitionCols.nonEmpty) + recordDeltaEvent(deltaLog, s"$INCREMENTAL_MANIFEST_OP_TYPE.stats", data = stats) + } + + /** + * Generate manifest files for all the partitions in the table. Note, this will ensure that + * that stale and unnecessary files will be vacuumed. + */ + def generateFullManifest( + spark: SparkSession, + deltaLog: DeltaLog): Unit = recordManifestGeneration(deltaLog, full = true) { + + val snapshot = deltaLog.update(stalenessAcceptable = false) + val partitionCols = snapshot.metadata.partitionColumns + val manifestRootDirPath = new Path(deltaLog.dataPath, MANIFEST_LOCATION).toString + val hadoopConf = new SerializableConfiguration(spark.sessionState.newHadoopConf()) + + // Update manifest files of the current partitions + val newManifestPartitionRelativePaths = writeManifestFiles( + deltaLog.dataPath, + manifestRootDirPath, + snapshot.allFiles, + partitionCols, + hadoopConf) + + // Get the existing manifest files as relative partition paths, that is, + // [ "col1=0/col2=0", "col1=1/col2=1", "col1=2/col2=2" ] + val fs = deltaLog.dataPath.getFileSystem(hadoopConf.value) + val existingManifestPartitionRelativePaths = { + val manifestRootDirAbsPath = fs.makeQualified(new Path(manifestRootDirPath)) + if (fs.exists(manifestRootDirAbsPath)) { + val index = new InMemoryFileIndex(spark, Seq(manifestRootDirAbsPath), Map.empty, None) + val prefixToStrip = manifestRootDirAbsPath.toUri.getPath + index.inputFiles.map { p => + // Remove root directory "rootDir" path from the manifest file paths like + // "rootDir/col1=0/col2=0/manifest" to get the relative partition dir "col1=0/col2=0". + // Note: It important to compare only the "path" in the URI and not the user info in it. + // In s3a://access-key:secret-key@host/path, the access-key and secret-key may change + // unknowingly to `\` and `%` encoding between the root dir and file names generated + // by listing. + val relativeManifestFilePath = + new Path(p).toUri.getPath.stripPrefix(prefixToStrip).stripPrefix(Path.SEPARATOR) + new Path(relativeManifestFilePath).getParent.toString // returns "col1=0/col2=0" + }.filterNot(_.trim.isEmpty).toSet + } else Set.empty[String] + } + + // Delete manifest files for partitions that are not in current and so weren't overwritten + val manifestFilePartitionsToDelete = + existingManifestPartitionRelativePaths.diff(newManifestPartitionRelativePaths) + deleteManifestFiles(manifestRootDirPath, manifestFilePartitionsToDelete, hadoopConf) + + // Post stats + val stats = SymlinkManifestStats( + filesWritten = newManifestPartitionRelativePaths.size, + filesDeleted = manifestFilePartitionsToDelete.size, + partitioned = partitionCols.nonEmpty) + recordDeltaEvent(deltaLog, s"$FULL_MANIFEST_OP_TYPE.stats", data = stats) + } + + /** + * Write the manifest files and return the partition relative paths of the manifests written. + * + * @param deltaLogDataPath path of the table data (e.g., tablePath which has _delta_log in it) + * @param manifestRootDirPath root directory of the manifest files (e.g., tablePath/_manifest/) + * @param fileNamesForManifest relative paths or file names of data files for being written into + * the manifest (e.g., partition=1/xyz.parquet) + * @param partitionCols Table partition columns + * @param hadoopConf Hadoop configuration to use + * @return Set of partition relative paths of the written manifest files (e.g., part1=1/part2=2) + */ + private def writeManifestFiles( + deltaLogDataPath: Path, + manifestRootDirPath: String, + fileNamesForManifest: Dataset[AddFile], + partitionCols: Seq[String], + hadoopConf: SerializableConfiguration): Set[String] = { + + val spark = fileNamesForManifest.sparkSession + import spark.implicits._ + + val tableAbsPathForManifest = + LogStore(spark.sparkContext).resolvePathOnPhysicalStorage(deltaLogDataPath).toString + + /** Write the data file relative paths to manifestDirAbsPath/manifest as absolute paths */ + def writeSingleManifestFile( + manifestDirAbsPath: String, + dataFileRelativePaths: Iterator[String]): Unit = { + + val manifestFilePath = new Path(manifestDirAbsPath, "manifest") + val fs = manifestFilePath.getFileSystem(hadoopConf.value) + fs.mkdirs(manifestFilePath.getParent()) + + val manifestContent = dataFileRelativePaths.map { relativePath => + DeltaFileOperations.absolutePath(tableAbsPathForManifest, relativePath).toString + } + val logStore = LogStore(SparkEnv.get.conf, hadoopConf.value) + logStore.write(manifestFilePath, manifestContent, overwrite = true) + } + + val newManifestPartitionRelativePaths = + withRelativePartitionDir(spark, partitionCols, fileNamesForManifest) + .select("relativePartitionDir", "path").as[(String, String)] + .groupByKey(_._1).mapGroups { + (relativePartitionDir: String, relativeDataFilePath: Iterator[(String, String)]) => + val manifestPartitionDirAbsPath = { + if (relativePartitionDir == null || relativePartitionDir.isEmpty) manifestRootDirPath + else new Path(manifestRootDirPath, relativePartitionDir).toString + } + writeSingleManifestFile(manifestPartitionDirAbsPath, relativeDataFilePath.map(_._2)) + relativePartitionDir + }.collect().toSet + + logInfo(s"Generated manifest partitions for $deltaLogDataPath " + + s"[${newManifestPartitionRelativePaths.size}]:\n\t" + + newManifestPartitionRelativePaths.mkString("\n\t")) + + newManifestPartitionRelativePaths + } + + /** + * Delete manifest files in the given paths. + * + * @param manifestRootDirPath root directory of the manifest files (e.g., tablePath/_manifest/) + * @param partitionRelativePathsToDelete partitions to delete manifest files from + * (e.g., part1=1/part2=2/) + * @param hadoopConf Hadoop configuration to use + */ + private def deleteManifestFiles( + manifestRootDirPath: String, + partitionRelativePathsToDelete: Iterable[String], + hadoopConf: SerializableConfiguration): Unit = { + + val fs = new Path(manifestRootDirPath).getFileSystem(hadoopConf.value) + partitionRelativePathsToDelete.foreach { path => + val absPathToDelete = new Path(manifestRootDirPath, path) + fs.delete(absPathToDelete, true) + } + + logInfo(s"Deleted manifest partitions [${partitionRelativePathsToDelete.size}]:\n\t" + + partitionRelativePathsToDelete.mkString("\n\t")) + } + + /** + * Append a column `relativePartitionDir` to the given Dataset which has `partitionValues` as + * one of the columns. `partitionValues` is a map-type column that contains values of the + * given `partitionCols`. + */ + private def withRelativePartitionDir( + spark: SparkSession, + partitionCols: Seq[String], + datasetWithPartitionValues: Dataset[_]) = { + + require(datasetWithPartitionValues.schema.fieldNames.contains("partitionValues")) + val colNamePrefix = "_col_" + var df: Dataset[_] = datasetWithPartitionValues + + // Flatten out nested partition value columns while renaming them, so that the new columns do + // not conflict with existing columns in DF `pathsWithPartitionValues. + val colToRenamedCols = partitionCols.map { column => + val renamedColumn = s"$colNamePrefix$column" + df = df.withColumn(renamedColumn, col(s"partitionValues.`$column`")) + column -> renamedColumn + } + + // Mapping between original column names to use for generating partition path and + // attributes referring to corresponding columns added to DF `pathsWithPartitionValues`. + val colNameToAttribs = + colToRenamedCols.map { case (col, renamed) => col -> UnresolvedAttribute.quoted(renamed) } + + // Build an expression that can generate the path fragment col1=value/col2=value/ from the + // partition columns. Note: The session time zone maybe different from the time zone that was + // used to write the partition structure of the actual data files. This may lead to + // inconsistencies between the partition structure of metadata files and data files. + val relativePartitionDirExpression = generatePartitionPathExpression( + colNameToAttribs, + spark.sessionState.conf.sessionLocalTimeZone) + + df.withColumn("relativePartitionDir", new Column(relativePartitionDirExpression)) + .drop(colToRenamedCols.map(_._2): _*) + } + + /** Expression that given partition columns builds a path string like: col1=val/col2=val/... */ + protected def generatePartitionPathExpression( + partitionColNameToAttrib: Seq[(String, Attribute)], + timeZoneId: String): Expression = Concat( + + + partitionColNameToAttrib.zipWithIndex.flatMap { case ((colName, col), i) => + val partitionName = ScalaUDF( + ExternalCatalogUtils.getPartitionPathString _, + StringType, + Seq(Literal(colName), Cast(col, StringType, Option(timeZoneId))), + Seq(BooleanType, BooleanType)) + if (i == 0) Seq(partitionName) else Seq(Literal(Path.SEPARATOR), partitionName) + } + ) + + + private def recordManifestGeneration(deltaLog: DeltaLog, full: Boolean)(thunk: => Unit): Unit = { + val (opType, manifestType) = + if (full) FULL_MANIFEST_OP_TYPE -> "full" + else INCREMENTAL_MANIFEST_OP_TYPE -> "incremental" + recordDeltaOperation(deltaLog, opType) { + withStatusCode("DELTA", s"Updating $manifestType Hive manifest for the Delta table") { + thunk + } + } + } + + case class SymlinkManifestStats( + filesWritten: Int, + filesDeleted: Int, + partitioned: Boolean) +} diff --git a/src/main/scala/org/apache/spark/sql/delta/hooks/PostCommitHook.scala b/src/main/scala/org/apache/spark/sql/delta/hooks/PostCommitHook.scala new file mode 100644 index 00000000000..3309e2a9b2d --- /dev/null +++ b/src/main/scala/org/apache/spark/sql/delta/hooks/PostCommitHook.scala @@ -0,0 +1,41 @@ +/* + * Copyright 2019 Databricks, Inc. + * + * 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.hooks + +import org.apache.spark.sql.delta._ +import org.apache.spark.sql.delta.actions.Action + +import org.apache.spark.sql.SparkSession + +/** + * A hook which can be executed after a transaction. These hooks are registered to a + * [[OptimisticTransaction]], and are executed after a *successful* commit takes place. + */ +trait PostCommitHook { + + /** A user friendly name for the hook for error reporting purposes. */ + val name: String + + /** Executes the hook. */ + def run(spark: SparkSession, txn: OptimisticTransactionImpl, committedActions: Seq[Action]): Unit + + /** + * Handle any error caused while running the hook. By default, all errors are ignored as + * default policy should be to not let post-commit hooks to cause failures in the operation. + */ + def handleError(error: Throwable, version: Long): Unit = {} +} diff --git a/src/main/scala/org/apache/spark/sql/delta/isolationLevels.scala b/src/main/scala/org/apache/spark/sql/delta/isolationLevels.scala new file mode 100644 index 00000000000..a0f666f8f8b --- /dev/null +++ b/src/main/scala/org/apache/spark/sql/delta/isolationLevels.scala @@ -0,0 +1,91 @@ +/* + * Copyright 2019 Databricks, Inc. + * + * 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 + +/** + * Trait that defines the level consistency guarantee is going to be provided by + * `OptimisticTransaction.commit()`. [[Serializable]] is the most + * strict level and [[SnapshotIsolation]] is the least strict one. + * + * @see [[IsolationLevel.allLevelsInDescOrder]] for all the levels in the descending order + * of strictness and [[IsolationLevel.DEFAULT]] for the default table isolation level. + */ +sealed trait IsolationLevel { + override def toString: String = this.getClass.getSimpleName.stripSuffix("$") +} + +/** + * This isolation level will ensure serializability between all read and write operations. + * Specifically, for write operations, this mode will ensure that the result of + * the table will be perfectly consistent with the visible history of operations, that is, + * as if all the operations were executed sequentially one by one. + */ +case object Serializable extends IsolationLevel + +/** + * This isolation level will ensure snapshot isolation consistency guarantee between write + * operations only. In other words, if only the write operations are considered, then + * there exists a serializable sequence between them that would produce the same result + * as seen in the table. However, if both read and write operations are considered, then + * there may not exist a serializable sequence that would explain all the observed reads. + * + * This provides a lower consistency guarantee than [[Serializable]] but a higher + * availability than that. For example, unlike [[Serializable]], this level allows an UPDATE + * operation to be committed even if there was a concurrent INSERT operation that has already + * added data that should have been read by the UPDATE. It will be as if the UPDATE was executed + * before the INSERT even if the former was committed after the latter. As a side effect, + * the visible history of operations may not be consistent with the + * result expected if these operations were executed sequentially one by one. + */ +case object WriteSerializable extends IsolationLevel + +/** + * This isolation level will ensure that all reads will see a consistent + * snapshot of the table and any transactional write will successfully commit only + * if the values updated by the transaction have not been changed externally since + * the snapshot was read by the transaction. + * + * This provides a lower consistency guarantee than [[WriteSerializable]] but a higher + * availability than that. For example, unlike [[WriteSerializable]], this level allows two + * concurrent UPDATE operations reading the same data to be committed successfully as long as + * they don't modify the same data. + * + * Note that for operations that do not modify data in the table, Snapshot isolation is same + * as Serializablity. Hence such operations can be safely committed with Snapshot isolation level. + */ +case object SnapshotIsolation extends IsolationLevel + + +object IsolationLevel { + + val DEFAULT = WriteSerializable + + /** All possible isolation levels in descending order of guarantees provided */ + val allLevelsInDescOrder: Seq[IsolationLevel] = Seq( + Serializable, + WriteSerializable, + SnapshotIsolation) + + /** All the valid isolation levels that can be specified as the table isolation level */ + val validTableIsolationLevels = Set[IsolationLevel](Serializable, WriteSerializable) + + def fromString(s: String): IsolationLevel = { + allLevelsInDescOrder.find(_.toString.equalsIgnoreCase(s)).getOrElse { + throw new IllegalArgumentException(s"invalid isolation level '$s'") + } + } +} diff --git a/src/main/scala/org/apache/spark/sql/delta/schema/ImplicitMetadataOperation.scala b/src/main/scala/org/apache/spark/sql/delta/schema/ImplicitMetadataOperation.scala index 4f588feff91..e2e20342d35 100644 --- a/src/main/scala/org/apache/spark/sql/delta/schema/ImplicitMetadataOperation.scala +++ b/src/main/scala/org/apache/spark/sql/delta/schema/ImplicitMetadataOperation.scala @@ -52,7 +52,8 @@ trait ImplicitMetadataOperation extends DeltaLogging { data: Dataset[_], partitionColumns: Seq[String], configuration: Map[String, String], - isOverwriteMode: Boolean): Unit = { + isOverwriteMode: Boolean, + rearrangeOnly: Boolean = false): Unit = { val dataSchema = data.schema.asNullable val mergedSchema = if (isOverwriteMode && canOverwriteSchema) { dataSchema @@ -79,6 +80,9 @@ trait ImplicitMetadataOperation extends DeltaLogging { } recordDeltaEvent(txn.deltaLog, "delta.ddl.initializeSchema") // If this is the first write, configure the metadata of the table. + if (rearrangeOnly) { + throw DeltaErrors.unexpectedDataChangeException("Create a Delta table") + } txn.updateMetadata( Metadata( schemaString = dataSchema.json, @@ -91,10 +95,17 @@ trait ImplicitMetadataOperation extends DeltaLogging { partitionColumns = normalizedPartitionCols ) recordDeltaEvent(txn.deltaLog, "delta.ddl.overwriteSchema") + if (rearrangeOnly) { + throw DeltaErrors.unexpectedDataChangeException("Overwrite the Delta table schema or " + + "change the partition schema") + } txn.updateMetadata(newMetadata) } else if (isNewSchema && canMergeSchema && !isNewPartitioning) { logInfo(s"New merged schema: ${mergedSchema.treeString}") recordDeltaEvent(txn.deltaLog, "delta.ddl.mergeSchema") + if (rearrangeOnly) { + throw DeltaErrors.unexpectedDataChangeException("Change the Delta table schema") + } txn.updateMetadata(txn.metadata.copy(schemaString = mergedSchema.json)) } else if (isNewSchema || isNewPartitioning) { recordDeltaEvent(txn.deltaLog, "delta.schemaValidation.failure") 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 f28a39e6ef5..a8754497a52 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 @@ -18,6 +18,7 @@ package org.apache.spark.sql.delta.sources import scala.util.{Failure, Success, Try} +// scalastyle:off import.ordering.noEmptyLine import org.apache.spark.sql.delta._ import org.apache.spark.sql.delta.commands.WriteIntoDelta import org.apache.spark.sql.delta.metering.DeltaLogging @@ -51,6 +52,7 @@ class DeltaDataSource spark.conf.set("spark.sql.legacy.sources.write.passPartitionByAsOptions", "true") } + override def sourceSchema( sqlContext: SQLContext, schema: Option[StructType], @@ -193,10 +195,11 @@ class DeltaDataSource } val filters = partitions.map { case (key, value) => - EqualTo(UnresolvedAttribute(key), Literal(value)) + // Nested fields cannot be partitions, so we pass the key as a identifier + EqualTo(UnresolvedAttribute(Seq(key)), Literal(value)) } val files = DeltaLog.filterFileList( - metadata.partitionColumns, snapshot.allFiles.toDF(), filters) + metadata.partitionSchema, snapshot.allFiles.toDF(), filters) if (files.count() == 0) { throw DeltaErrors.pathNotExistsException(path) } 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 ba5cc9f2f51..be3be7f9be1 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 @@ -130,6 +130,13 @@ object DeltaSQLConf { .intConf .createWithDefault(1000) + val DELTA_HISTORY_METRICS_ENABLED = + buildConf("history.metricsEnabled") + .doc("Enables Metrics reporting in Describe History. CommitInfo will now record the " + + "Operation Metrics.") + .booleanConf + .createWithDefault(false) + val DELTA_VACUUM_RETENTION_CHECK_ENABLED = buildConf("retentionDurationCheck.enabled") .doc("Adds a check preventing users from running vacuum with a very short retention " + @@ -206,4 +213,15 @@ object DeltaSQLConf { .longConf .createWithDefault(10000L) + val MERGE_INSERT_ONLY_ENABLED = + buildConf("merge.optimizeInsertOnlyMerge.enabled") + .internal() + .doc( + """ + |If enabled, merge without any matched clause (i.e., insert-only merge) will be optimized + |by avoiding rewriting old files and just inserting new files. + """.stripMargin) + .booleanConf + .createWithDefault(true) + } diff --git a/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSource.scala b/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSource.scala index 5fc9772f8db..95920e2bbf4 100644 --- a/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSource.scala +++ b/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSource.scala @@ -72,8 +72,8 @@ case class DeltaSource( // Deprecated. Please use `ignoreDeletes` or `ignoreChanges` from now on. private val ignoreFileDeletion = { if (options.ignoreFileDeletion) { - val docPage = DeltaErrors.baseDocsPath(spark.sparkContext.getConf) + - "/delta/delta-streaming.html#ignoring-updates-and-deletes" + val docPage = DeltaErrors.generateDocsLink(spark.sparkContext.getConf, + "/delta/delta-streaming.html#ignoring-updates-and-deletes") logConsole( s"""WARNING: The 'ignoreFileDeletion' option is deprecated. Switch to using one of |'ignoreDeletes' or 'ignoreChanges'. Refer to $docPage for details. 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 8dbe099b35c..87de57ad165 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.com.databricks.sql.transaction.tahoe; +package test.org.apache.spark.sql.delta; import java.util.ArrayList; import java.util.Arrays; 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 2abeb9ea44e..bc2ce18566a 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.com.databricks.sql.transaction.tahoe; +package test.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 e0050012a0d..a2eddf3e42c 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.com.databricks.sql.transaction.tahoe; +package test.org.apache.spark.sql.delta; import java.util.*; diff --git a/src/test/scala/org/apache/spark/sql/delta/ConvertToDeltaSQLSuite.scala b/src/test/scala/org/apache/spark/sql/delta/ConvertToDeltaSQLSuite.scala index bb4bdd40fcd..2cecde35d86 100644 --- a/src/test/scala/org/apache/spark/sql/delta/ConvertToDeltaSQLSuite.scala +++ b/src/test/scala/org/apache/spark/sql/delta/ConvertToDeltaSQLSuite.scala @@ -16,14 +16,15 @@ package org.apache.spark.sql.delta +import org.apache.spark.sql.delta.test.DeltaSQLCommandTest + class ConvertToDeltaSQLSuite - extends ConvertToDeltaSuiteBase - with org.apache.spark.sql.delta.test.DeltaSQLCommandTest + extends ConvertToDeltaSuiteBase with DeltaSQLCommandTest { override protected def convertToDelta( identifier: String, partitionSchema: Option[String] = None): Unit = { - if (!partitionSchema.isDefined) { + if (partitionSchema.isEmpty) { sql(s"convert to delta $identifier") } else { val stringSchema = partitionSchema.get diff --git a/src/test/scala/org/apache/spark/sql/delta/ConvertToDeltaScalaSuite.scala b/src/test/scala/org/apache/spark/sql/delta/ConvertToDeltaScalaSuite.scala new file mode 100644 index 00000000000..c4f4a6920b6 --- /dev/null +++ b/src/test/scala/org/apache/spark/sql/delta/ConvertToDeltaScalaSuite.scala @@ -0,0 +1,38 @@ +/* + * Copyright 2019 Databricks, Inc. + * + * 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.types.StructType + +class ConvertToDeltaScalaSuite extends ConvertToDeltaSuiteBase { + override protected def convertToDelta( + identifier: String, + partitionSchema: Option[String] = None): Unit = { + if (partitionSchema.isDefined) { + io.delta.tables.DeltaTable.convertToDelta( + spark, + identifier, + StructType.fromDDL(partitionSchema.get) + ) + } else { + io.delta.tables.DeltaTable.convertToDelta( + spark, + identifier + ) + } + } +} diff --git a/src/test/scala/org/apache/spark/sql/delta/ConvertToDeltaSuite.scala b/src/test/scala/org/apache/spark/sql/delta/ConvertToDeltaSuiteBase.scala similarity index 96% rename from src/test/scala/org/apache/spark/sql/delta/ConvertToDeltaSuite.scala rename to src/test/scala/org/apache/spark/sql/delta/ConvertToDeltaSuiteBase.scala index 7ceafdd96db..82b2f623946 100644 --- a/src/test/scala/org/apache/spark/sql/delta/ConvertToDeltaSuite.scala +++ b/src/test/scala/org/apache/spark/sql/delta/ConvertToDeltaSuiteBase.scala @@ -27,10 +27,7 @@ import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ import org.apache.spark.util.Utils -class ConvertToDeltaSuite - extends ConvertToDeltaSuiteBase with org.apache.spark.sql.delta.test.DeltaSQLCommandTest - -trait ConvertToDeltaSuiteBase extends QueryTest +abstract class ConvertToDeltaSuiteBase extends QueryTest with SharedSparkSession { import org.apache.spark.sql.functions._ @@ -77,20 +74,7 @@ trait ConvertToDeltaSuiteBase extends QueryTest } } - protected def convertToDelta(identifier: String, partitionSchema: Option[String] = None): Unit = { - if (partitionSchema.isDefined) { - io.delta.tables.DeltaTable.convertToDelta( - spark, - identifier, - StructType.fromDDL(partitionSchema.get) - ) - } else { - io.delta.tables.DeltaTable.convertToDelta( - spark, - identifier - ) - } - } + protected def convertToDelta(identifier: String, partitionSchema: Option[String] = None): Unit test("negative case: convert a non-delta path falsely claimed as parquet") { Seq("orc", "json", "csv").foreach { format => @@ -396,6 +380,24 @@ trait ConvertToDeltaSuiteBase extends QueryTest } } + test("converting tables with dateType partition columns") { + withTempDir { dir => + val tempDir = dir.getCanonicalPath + val df1 = Seq(0).toDF("id").withColumn("key1", lit("2019-11-22").cast("date")) + + val df2 = Seq(1).toDF("id").withColumn("key1", lit(null)) + + writeFiles(tempDir, df1.union(df2), partCols = Seq("key1")) + convertToDelta(s"parquet.`$tempDir`", Some("key1 date")) + checkAnswer( + spark.read.format("delta").load(tempDir).where("key1 is null").select("id"), + Row(1)) + checkAnswer( + spark.read.format("delta").load(tempDir).where("key1 = '2019-11-22'").select("id"), + Row(0)) + } + } + test("empty string partition value will be read back as null") { withTempDir { dir => val tempDir = dir.getCanonicalPath diff --git a/src/test/scala/org/apache/spark/sql/delta/DeltaErrorsSuite.scala b/src/test/scala/org/apache/spark/sql/delta/DeltaErrorsSuite.scala new file mode 100644 index 00000000000..afb586d4a44 --- /dev/null +++ b/src/test/scala/org/apache/spark/sql/delta/DeltaErrorsSuite.scala @@ -0,0 +1,90 @@ +/* + * Copyright 2019 Databricks, Inc. + * + * 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 scala.sys.process.Process + +import org.apache.hadoop.fs.Path +import org.scalatest.GivenWhenThen + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} + +trait DeltaErrorsSuiteBase + extends QueryTest + with SharedSparkSession with GivenWhenThen + with SQLTestUtils { + + val MAX_URL_ACCESS_RETRIES = 3 + val path = "/sample/path" + // Map of error name to the actual error message it throws + def errorsToTest: Map[String, Throwable] = Map( + "useDeltaOnOtherFormatPathException" -> + DeltaErrors.useDeltaOnOtherFormatPathException("operation", path, spark), + "useOtherFormatOnDeltaPathException" -> + DeltaErrors.useOtherFormatOnDeltaPathException("operation", path, path, "format", spark), + "createExternalTableWithoutLogException" -> + DeltaErrors.createExternalTableWithoutLogException(new Path(path), "tableName", spark), + "createExternalTableWithoutSchemaException" -> + DeltaErrors.createExternalTableWithoutSchemaException(new Path(path), "tableName", spark), + "createManagedTableWithoutSchemaException" -> + DeltaErrors.createManagedTableWithoutSchemaException("tableName", spark)) + + def otherMessagesToTest: Map[String, String] = Map( + "deltaFileNotFoundHint" -> + DeltaErrors.deltaFileNotFoundHint( + DeltaErrors.generateDocsLink(sparkConf, DeltaErrors.faqRelativePath), path)) + + def errorMessagesToTest: Map[String, String] = + errorsToTest.mapValues(_.getMessage) ++ otherMessagesToTest + + def checkIfValidResponse(url: String, response: String): Boolean = { + response.contains("HTTP/1.1 200 OK") || response.contains("HTTP/2 200") + } + + def getUrlsFromMessage(message: String): List[String] = { + val regexToFindUrl = "https://[^\\s]+".r + regexToFindUrl.findAllIn(message).toList + } + + def testUrls() { + errorMessagesToTest.foreach { case (errName, message) => + getUrlsFromMessage(message).foreach { url => + Given(s"*** Checking response for url: $url") + var response = "" + (1 to MAX_URL_ACCESS_RETRIES).foreach { attempt => + if (attempt > 1) Thread.sleep(1000) + response = Process("curl -I " + url).!! + if (!checkIfValidResponse(url, response)) { + fail( + s""" + |A link to the URL: '$url' is broken in the error: $errName, accessing this URL + |does not result in a valid response, received the following response: $response + """.stripMargin) + } + } + } + } + } + + test("Validate that links to docs in DeltaErrors are correct") { + testUrls() + } +} + +class DeltaErrorsSuite + extends DeltaErrorsSuiteBase diff --git a/src/test/scala/org/apache/spark/sql/delta/DeltaGenerateSymlinkManifestSuite.scala b/src/test/scala/org/apache/spark/sql/delta/DeltaGenerateSymlinkManifestSuite.scala new file mode 100644 index 00000000000..8385fc5d805 --- /dev/null +++ b/src/test/scala/org/apache/spark/sql/delta/DeltaGenerateSymlinkManifestSuite.scala @@ -0,0 +1,559 @@ +/* + * Copyright 2019 Databricks, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.delta + +import java.io.File +import java.net.URI + +import org.apache.spark.sql.delta.commands.DeltaGenerateCommand +import org.apache.spark.sql.delta.hooks.GenerateSymlinkManifest +import org.apache.spark.sql.delta.test.DeltaSQLCommandTest +import org.apache.spark.sql.delta.util.DeltaFileOperations +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs._ +import org.apache.hadoop.util.Progressable + +import org.apache.spark.sql._ +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.test.SharedSparkSession + +class DeltaGenerateSymlinkManifestSuite + extends DeltaGenerateSymlinkManifestSuiteBase + with DeltaSQLCommandTest + +trait DeltaGenerateSymlinkManifestSuiteBase extends QueryTest + with SharedSparkSession { + + import testImplicits._ + + test ("full manifest: non-partitioned table") { + 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) + generateSymlinkManifest(tablePath.toString) + assertManifest(tablePath, expectSameFiles = true, expectedNumFiles = 7) + + // Reduce files + write(5) + assertManifest(tablePath, expectSameFiles = false, expectedNumFiles = 7) + generateSymlinkManifest(tablePath.toString) + assertManifest(tablePath, expectSameFiles = true, expectedNumFiles = 5) + + // Remove all data + spark.emptyDataset[Int].write.format("delta").mode("overwrite").save(tablePath.toString) + assertManifest(tablePath, expectSameFiles = false, expectedNumFiles = 5) + generateSymlinkManifest(tablePath.toString) + assertManifest(tablePath, expectSameFiles = true, expectedNumFiles = 1) + assert(spark.read.format("delta").load(tablePath.toString).count() == 0) + } + } + + test("full manifest: partitioned table") { + 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").save(tablePath.toString) + } + + write(10, 10, 10) + assertManifest(tablePath, expectSameFiles = false, expectedNumFiles = 0) + generateSymlinkManifest(tablePath.toString) + // 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) + generateSymlinkManifest(tablePath.toString) + assertManifest(tablePath, expectSameFiles = true, expectedNumFiles = 1) + + // Increase # partitions on both dimensions + write(5, 5, 5) + assertManifest(tablePath, expectSameFiles = false, expectedNumFiles = 1) + generateSymlinkManifest(tablePath.toString) + assertManifest(tablePath, expectSameFiles = true, expectedNumFiles = 25) + + // Increase # partitions on only one dimension + write(5, 10, 5) + assertManifest(tablePath, expectSameFiles = false, expectedNumFiles = 25) + generateSymlinkManifest(tablePath.toString) + 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) + generateSymlinkManifest(tablePath.toString) + assertManifest(tablePath, expectSameFiles = true, expectedNumFiles = 0) + assert(spark.read.format("delta").load(tablePath.toString).count() == 0) + } + } + + 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() + + def writeWithIncrementalManifest(enabled: Boolean, numFiles: Int): Unit = { + withIncrementalManifest(tablePath, enabled) { + spark.createDataset(spark.sparkContext.parallelize(1 to 100, numFiles)) + .write.format("delta").mode("overwrite").save(tablePath.toString) + } + } + + writeWithIncrementalManifest(enabled = false, numFiles = 1) + assertManifest(tablePath, expectSameFiles = false, expectedNumFiles = 0) + + // Enabling it should automatically generate manifest files + writeWithIncrementalManifest(enabled = true, numFiles = 2) + assertManifest(tablePath, expectSameFiles = true, expectedNumFiles = 2) + + // Disabling it should stop updating existing manifest files + writeWithIncrementalManifest(enabled = false, numFiles = 3) + assertManifest(tablePath, expectSameFiles = false, expectedNumFiles = 2) + } + } + + test("incremental manifest: unpartitioned table") { + withTempDir { tablePath => + tablePath.delete() + + def write(numFiles: Int): Unit = withIncrementalManifest(tablePath, enabled = true) { + spark.createDataset(spark.sparkContext.parallelize(1 to 100, numFiles)) + .write.format("delta").mode("overwrite").save(tablePath.toString) + } + + write(1) + // first write won't generate automatic manifest as mode enable after first write + assertManifest(tablePath, expectSameFiles = false, expectedNumFiles = 0) + + // Increase files + write(7) + assertManifest(tablePath, expectSameFiles = true, expectedNumFiles = 7) + + // Reduce files + write(5) + assertManifest(tablePath, expectSameFiles = true, expectedNumFiles = 5) + + // Remove all data + spark.emptyDataset[Int].write.format("delta").mode("overwrite").save(tablePath.toString) + assert(spark.read.format("delta").load(tablePath.toString).count() == 0) + assertManifest(tablePath, expectSameFiles = true, expectedNumFiles = 1) + } + } + + test("incremental manifest: partitioned table") { + withTempDir { tablePath => + tablePath.delete() + + def writePartitioned(parallelism: Int, numPartitions1: Int, numPartitions2: Int): Unit = { + withIncrementalManifest(tablePath, enabled = true) { + val input = + if (parallelism == 0) spark.emptyDataset[Int] + else spark.createDataset(spark.sparkContext.parallelize(1 to 100, parallelism)) + input.toDF("value") + .withColumn("part1", $"value" % numPartitions1) + .withColumn("part2", $"value" % numPartitions2) + .write.format("delta").partitionBy("part1", "part2") + .mode("overwrite").save(tablePath.toString) + } + } + + writePartitioned(1, 1, 1) + // Manifests wont be generated in the first write because `withIncrementalManifest` will + // enable manifest generation only after the first write defines the table log. + assertManifest(tablePath, expectSameFiles = false, expectedNumFiles = 0) + + writePartitioned(10, 10, 10) + // 10 files each in ../part1=X/part2=X/ for X = 0 to 9 (so only 10 subdirectories) + assertManifest(tablePath, expectSameFiles = true, expectedNumFiles = 100) + + // Update such that 1 file is removed and 1 file is added in another partition + val deltaTable = io.delta.tables.DeltaTable.forPath(spark, tablePath.toString) + deltaTable.updateExpr("value = 1", Map("part1" -> "0", "value" -> "-1")) + assertManifest(tablePath, expectSameFiles = true, expectedNumFiles = 100) + + // Delete such that 1 file is removed + deltaTable.delete("value = -1") + assertManifest(tablePath, expectSameFiles = true, expectedNumFiles = 99) + + // Reduce # partitions on both dimensions + writePartitioned(1, 1, 1) + assertManifest(tablePath, expectSameFiles = true, expectedNumFiles = 1) + + // Increase # partitions on both dimensions + writePartitioned(5, 5, 5) + assertManifest(tablePath, expectSameFiles = true, expectedNumFiles = 25) + + // Increase # partitions on only one dimension + writePartitioned(5, 10, 5) + assertManifest(tablePath, expectSameFiles = true, expectedNumFiles = 50) + + // Remove all data + writePartitioned(0, 1, 1) + assert(spark.read.format("delta").load(tablePath.toString).count() == 0) + assertManifest(tablePath, expectSameFiles = true, expectedNumFiles = 0) + } + } + + test("incremental manifest: generate full manifest if manifest did not exist") { + withTempDir { tablePath => + + def write(numPartitions: Int): Unit = { + spark.range(0, 100, 1, 1).toDF("value").withColumn("part", $"value" % numPartitions) + .write.format("delta").partitionBy("part").mode("append").save(tablePath.toString) + } + + write(10) + assertManifest(tablePath, expectSameFiles = false, expectedNumFiles = 0) + + withIncrementalManifest(tablePath, enabled = true) { + write(1) // update only one partition + } + // Manifests should be generated for all partitions + assertManifest(tablePath, expectSameFiles = true, expectedNumFiles = 11) + } + } + + test("incremental manifest: failure to generate manifest throws exception") { + withTempDir { tablePath => + tablePath.delete() + + import SymlinkManifestFailureTestFileSystem._ + + withSQLConf( + s"fs.$SCHEME.impl" -> classOf[SymlinkManifestFailureTestFileSystem].getName, + s"fs.$SCHEME.impl.disable.cache" -> "true", + s"fs.AbstractFileSystem.$SCHEME.impl" -> + classOf[SymlinkManifestFailureTestAbstractFileSystem].getName, + s"fs.AbstractFileSystem.$SCHEME.impl.disable.cache" -> "true") { + def write(numFiles: Int): Unit = withIncrementalManifest(tablePath, enabled = true) { + spark.createDataset(spark.sparkContext.parallelize(1 to 100, numFiles)) + .write.format("delta").mode("overwrite").save(s"$SCHEME://$tablePath") + } + + write(1) // first write enables the property + val ex = catalyst.util.quietly { + intercept[RuntimeException] { write(2) } + } + + assert(ex.getMessage().contains(GenerateSymlinkManifest.name)) + assert(ex.getCause().toString.contains("Test exception")) + } + } + } + + test("special partition column names") { + + def assertColNames(inputStr: String): Unit = withClue(s"input: $inputStr") { + withTempDir { tablePath => + tablePath.delete() + val inputLines = inputStr.trim.stripMargin.trim.split("\n").toSeq + require(inputLines.size > 0) + val input = spark.read.json(inputLines.toDS) + val partitionCols = input.schema.fieldNames + val inputWithValue = input.withColumn("value", lit(1)) + + inputWithValue.write.format("delta").partitionBy(partitionCols: _*).save(tablePath.toString) + generateSymlinkManifest(tablePath.toString) + assertManifest(tablePath, expectSameFiles = true, expectedNumFiles = inputLines.size) + } + } + + assertColNames("""{ " " : 0 }""") + assertColNames("""{ "%" : 0 }""") + assertColNames("""{ "a.b." : 0 }""") + assertColNames("""{ "a_b" : 0 }""") + assertColNames("""{ "a b" : 0 }""") + } + + test("special partition column values") { + withTempDir { tablePath => + tablePath.delete() + val inputStr = """ + |{ "part1" : 1, "part2": "$0$", "value" : 1 } + |{ "part1" : null, "part2": "_1_", "value" : 1 } + |{ "part1" : 1, "part2": "", "value" : 1 } + |{ "part1" : null, "part2": " ", "value" : 1 } + |{ "part1" : 1, "part2": " ", "value" : 1 } + |{ "part1" : null, "part2": "/", "value" : 1 } + |{ "part1" : 1, "part2": null, "value" : 1 } + |""" + val input = spark.read.json(inputStr.trim.stripMargin.trim.split("\n").toSeq.toDS) + input.write.format("delta").partitionBy("part1", "part2").save(tablePath.toString) + generateSymlinkManifest(tablePath.toString) + assertManifest(tablePath, expectSameFiles = true, expectedNumFiles = 7) + } + } + + test("root table path with escapable chars like space") { + withTempDir { p => + val tablePath = new File(p.toString, "path with space") + spark.createDataset(spark.sparkContext.parallelize(1 to 100, 1)).toDF("value") + .withColumn("part", $"value" % 2) + .write.format("delta").partitionBy("part").save(tablePath.toString) + + generateSymlinkManifest(tablePath.toString) + assertManifest(tablePath, expectSameFiles = true, expectedNumFiles = 2) + } + } + + 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)) + } + } + } + + 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) + + if (!manifestPath.exists) { + assert(expectedNumFiles == 0 && !expectSameFiles) + return + } + + // Validate the expected number of files are present in the manifest + val filesInManifest = spark.read.text(manifestPath.toString).select("value").as[String] + .map { _.stripPrefix("file:") }.toDF("file") + assert(filesInManifest.count() == expectedNumFiles) + + // Validate that files in the latest version of DeltaLog is same as those in the manifest + val filesInLog = deltaSnapshot.allFiles.map { addFile => + // Note: this unescapes the relative path in `addFile` + DeltaFileOperations.absolutePath(tablePath.toString, addFile.path).toString + }.toDF("file") + if (expectSameFiles) { + checkAnswer(filesInManifest, filesInLog.toDF()) + + // Validate that each file in the manifest is actually present in table. This mainly checks + // whether the file names in manifest are not escaped and therefore are readable directly + // by Hadoop APIs. + val fs = new Path(manifestPath.toString).getFileSystem(spark.sessionState.newHadoopConf()) + spark.read.text(manifestPath.toString).select("value").as[String].collect().foreach { p => + assert(fs.exists(new Path(p)), s"path $p in manifest not found in file system") + } + } else { + assert(filesInManifest.as[String].collect().toSet != filesInLog.as[String].collect().toSet) + } + + // If there are partitioned files, make sure the partitions values read from them are the + // same as those in the table. + val partitionCols = deltaSnapshot.metadata.partitionColumns.map(x => s"`$x`") + if (partitionCols.nonEmpty && expectSameFiles && expectedNumFiles > 0) { + val partitionsInManifest = spark.read.text(manifestPath.toString) + .selectExpr(partitionCols: _*).distinct() + val partitionsInData = spark.read.format("delta").load(tablePath.toString) + .selectExpr(partitionCols: _*).distinct() + checkAnswer(partitionsInManifest, partitionsInData) + } + } + + protected def withIncrementalManifest(tablePath: File, enabled: Boolean)(func: => Unit): Unit = { + if (tablePath.exists()) { + val deltaLog = DeltaLog.forTable(spark, tablePath) + val latestMetadata = deltaLog.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)) + } + } + func + } + + protected def generateSymlinkManifest(tablePath: String): Unit = { + val deltaLog = DeltaLog.forTable(spark, tablePath) + GenerateSymlinkManifest.generateFullManifest(spark, deltaLog) + } +} + +class SymlinkManifestFailureTestAbstractFileSystem( + uri: URI, + conf: org.apache.hadoop.conf.Configuration) + extends org.apache.hadoop.fs.DelegateToFileSystem( + uri, + new SymlinkManifestFailureTestFileSystem, + conf, + SymlinkManifestFailureTestFileSystem.SCHEME, + false) { + + // Implementation copied from RawLocalFs + import org.apache.hadoop.fs.local.LocalConfigKeys + import org.apache.hadoop.fs._ + + override def getUriDefaultPort(): Int = -1 + override def getServerDefaults(): FsServerDefaults = LocalConfigKeys.getServerDefaults() + override def isValidName(src: String): Boolean = true +} + + +class SymlinkManifestFailureTestFileSystem extends RawLocalFileSystem { + + private var uri: URI = _ + override def getScheme: String = SymlinkManifestFailureTestFileSystem.SCHEME + + 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 + } + + // Create function used by the parquet writer + override def create( + path: Path, + overwrite: Boolean, + bufferSize: Int, + replication: Short, + blockSize: Long, + progress: Progressable): FSDataOutputStream = { + if (path.toString.contains(GenerateSymlinkManifest.MANIFEST_LOCATION)) { + throw new RuntimeException("Test exception") + } + super.create(path, overwrite, bufferSize, replication, blockSize, null) + } +} + +object SymlinkManifestFailureTestFileSystem { + val SCHEME = "testScheme" +} + diff --git a/src/test/scala/org/apache/spark/sql/delta/DeltaOptionSuite.scala b/src/test/scala/org/apache/spark/sql/delta/DeltaOptionSuite.scala new file mode 100644 index 00000000000..5fee0c4c742 --- /dev/null +++ b/src/test/scala/org/apache/spark/sql/delta/DeltaOptionSuite.scala @@ -0,0 +1,133 @@ +/* + * Copyright 2019 Databricks, Inc. + * + * 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.actions.{Action, FileAction} +import org.apache.spark.sql.delta.util.FileNames + +import org.apache.spark.SparkConf +import org.apache.spark.sql.{AnalysisException, QueryTest} +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.util.Utils + +class DeltaOptionSuite extends QueryTest + with SharedSparkSession { + + import testImplicits._ + + + + test("support for setting dataChange to false") { + val tempDir = Utils.createTempDir() + + spark.range(100) + .write + .format("delta") + .save(tempDir.toString) + + val df = spark.read.format("delta").load(tempDir.toString) + + df + .write + .format("delta") + .mode("overwrite") + .option("dataChange", "false") + .save(tempDir.toString) + + val deltaLog = DeltaLog.forTable(spark, tempDir) + val version = deltaLog.snapshot.version + val commitActions = deltaLog.store.read(FileNames.deltaFile(deltaLog.logPath, version)) + .map(Action.fromJson) + val fileActions = commitActions.collect { case a: FileAction => a } + + assert(fileActions.forall(!_.dataChange)) + } + + test("dataChange is by default set to true") { + val tempDir = Utils.createTempDir() + + spark.range(100) + .write + .format("delta") + .save(tempDir.toString) + + val df = spark.read.format("delta").load(tempDir.toString) + + df + .write + .format("delta") + .mode("overwrite") + .save(tempDir.toString) + + val deltaLog = DeltaLog.forTable(spark, tempDir) + val version = deltaLog.snapshot.version + val commitActions = deltaLog.store.read(FileNames.deltaFile(deltaLog.logPath, version)) + .map(Action.fromJson) + val fileActions = commitActions.collect { case a: FileAction => a } + + assert(fileActions.forall(_.dataChange)) + } + + test("dataChange is set to false on metadata changing operation") { + withTempDir { tempDir => + // Initialize a table while having dataChange set to false. + val e = intercept[AnalysisException] { + spark.range(100) + .write + .format("delta") + .option("dataChange", "false") + .save(tempDir.getAbsolutePath) + } + assert(e.getMessage === + DeltaErrors.unexpectedDataChangeException("Create a Delta table").getMessage) + spark.range(100) + .write + .format("delta") + .save(tempDir.getAbsolutePath) + + // Adding a new column to the existing table while having dataChange set to false. + val e2 = intercept[AnalysisException] { + val df = spark.read.format("delta").load(tempDir.getAbsolutePath) + df.withColumn("id2", 'id + 1) + .write + .format("delta") + .mode("overwrite") + .option("mergeSchema", "true") + .option("dataChange", "false") + .save(tempDir.getAbsolutePath) + } + assert(e2.getMessage === + DeltaErrors.unexpectedDataChangeException("Change the Delta table schema").getMessage) + + // Overwriting the schema of the existing table while having dataChange as false. + val e3 = intercept[AnalysisException] { + val df = spark.read.format("delta").load(tempDir.getAbsolutePath) + spark.range(50) + .withColumn("id3", 'id + 1) + .write + .format("delta") + .mode("overwrite") + .option("dataChange", "false") + .option("overwriteSchema", "true") + .save(tempDir.getAbsolutePath) + } + assert(e3.getMessage === + DeltaErrors.unexpectedDataChangeException("Overwrite the Delta table schema or " + + "change the partition schema").getMessage) + } + } +} diff --git a/src/test/scala/org/apache/spark/sql/delta/DeltaRetentionSuiteBase.scala b/src/test/scala/org/apache/spark/sql/delta/DeltaRetentionSuiteBase.scala index 6c40e9361af..4a9a1ed680d 100644 --- a/src/test/scala/org/apache/spark/sql/delta/DeltaRetentionSuiteBase.scala +++ b/src/test/scala/org/apache/spark/sql/delta/DeltaRetentionSuiteBase.scala @@ -36,7 +36,7 @@ trait DeltaRetentionSuiteBase extends QueryTest .set("spark.databricks.delta.properties.defaults.enableExpiredLogCleanup", "false") protected def intervalStringToMillis(str: String): Long = { - CalendarInterval.fromString(str).milliseconds() + DeltaConfigs.getMilliSeconds(CalendarInterval.fromString(str)) } protected def getDeltaFiles(dir: File): Seq[File] = diff --git a/src/test/scala/org/apache/spark/sql/delta/DeltaSourceSuite.scala b/src/test/scala/org/apache/spark/sql/delta/DeltaSourceSuite.scala index d7d1c6aadfb..4f358e5501e 100644 --- a/src/test/scala/org/apache/spark/sql/delta/DeltaSourceSuite.scala +++ b/src/test/scala/org/apache/spark/sql/delta/DeltaSourceSuite.scala @@ -584,8 +584,8 @@ class DeltaSourceSuite extends DeltaSourceSuiteBase { CheckLastBatch("0", "1", "2"), Assert { clock.advance( - CalendarInterval.fromString( - DeltaConfigs.LOG_RETENTION.defaultValue).milliseconds() + 100000000L) + DeltaConfigs.getMilliSeconds(CalendarInterval.fromString( + DeltaConfigs.LOG_RETENTION.defaultValue)) + 100000000L) // Delete all logs before checkpoint writersLog.cleanUpExpiredLogs() 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 2744dfce260..83c9911dcd4 100644 --- a/src/test/scala/org/apache/spark/sql/delta/DeltaSuite.scala +++ b/src/test/scala/org/apache/spark/sql/delta/DeltaSuite.scala @@ -18,11 +18,13 @@ package org.apache.spark.sql.delta import java.io.{File, FileNotFoundException} +import org.apache.spark.sql.delta.actions.{Action, FileAction} import org.apache.spark.sql.delta.files.TahoeLogFileIndex import org.apache.spark.sql.delta.sources.DeltaSQLConf +import org.apache.spark.sql.delta.util.FileNames import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.spark.SparkException +import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions.InSet import org.apache.spark.sql.catalyst.plans.logical.Filter @@ -559,15 +561,15 @@ class DeltaSuite extends QueryTest test("metadataOnly query") { withSQLConf(OPTIMIZER_METADATA_ONLY.key -> "true") { - withTable("tahoe_test") { + withTable("delta_test") { Seq(1L -> "a").toDF("dataCol", "partCol") .write .mode(SaveMode.Overwrite) .partitionBy("partCol") .format("delta") - .saveAsTable("tahoe_test") + .saveAsTable("delta_test") checkAnswer( - sql("select count(distinct partCol) FROM tahoe_test"), + sql("select count(distinct partCol) FROM delta_test"), Row(1)) } } @@ -881,6 +883,23 @@ class DeltaSuite extends QueryTest } } + test("SC-24886: partition columns have correct datatype in metadata scans") { + withTempDir { inputDir => + Seq(("foo", 2019)).toDF("name", "y") + .write.format("delta").partitionBy("y").mode("overwrite") + .save(inputDir.getAbsolutePath) + + // Before the fix, this query would fail because it tried to read strings from the metadata + // partition values as the LONG type that the actual partition columns are. This works now + // because we added a cast. + val df = spark.read.format("delta") + .load(inputDir.getAbsolutePath) + .where( + """cast(format_string("%04d-01-01 12:00:00", y) as timestamp) is not null""".stripMargin) + assert(df.collect().length == 1) + } + } + test("SC-11332: session isolation for cached delta logs") { withTempDir { tempDir => val path = tempDir.getCanonicalPath @@ -913,12 +932,4 @@ class DeltaSuite extends QueryTest } } } - - test("SC-15200: SaveAsTable on empty dataframe should create table") { - withTable("sc15200test") { - spark.range(0).selectExpr("id", "id as id2") - .write.format("delta").partitionBy("id").saveAsTable("sc15200test") - checkAnswer(spark.table("sc15200test"), Seq.empty) - } - } } 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 490c31f12d2..2cf8505ff16 100644 --- a/src/test/scala/org/apache/spark/sql/delta/DeltaVacuumSuite.scala +++ b/src/test/scala/org/apache/spark/sql/delta/DeltaVacuumSuite.scala @@ -23,6 +23,7 @@ import java.util.concurrent.TimeUnit import org.apache.spark.sql.delta.DeltaOperations.{Delete, Write} 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 import org.apache.spark.sql.delta.util.DeltaFileOperations import org.apache.commons.io.FileUtils import org.apache.hadoop.fs.Path @@ -292,7 +293,8 @@ trait DeltaVacuumSuiteBase extends QueryTest } protected def defaultTombstoneInterval: Long = { - CalendarInterval.fromString(DeltaConfigs.TOMBSTONE_RETENTION.defaultValue).milliseconds() + DeltaConfigs.getMilliSeconds(CalendarInterval.fromString( + DeltaConfigs.TOMBSTONE_RETENTION.defaultValue)) } implicit def fileToPathString(f: File): String = new Path(f.getAbsolutePath).toString @@ -497,4 +499,4 @@ trait DeltaVacuumSuiteBase extends QueryTest } class DeltaVacuumSuite - extends DeltaVacuumSuiteBase with org.apache.spark.sql.delta.test.DeltaSQLCommandTest + extends DeltaVacuumSuiteBase with DeltaSQLCommandTest diff --git a/src/test/scala/org/apache/spark/sql/delta/DescribeDeltaDetailSuite.scala b/src/test/scala/org/apache/spark/sql/delta/DescribeDeltaDetailSuite.scala index 5e597c0171d..7722f523e8f 100644 --- a/src/test/scala/org/apache/spark/sql/delta/DescribeDeltaDetailSuite.scala +++ b/src/test/scala/org/apache/spark/sql/delta/DescribeDeltaDetailSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.delta import java.io.File import java.io.FileNotFoundException +import org.apache.spark.sql.delta.test.DeltaSQLCommandTest + import org.apache.spark.sql.{DataFrame, QueryTest, Row} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.util.Utils @@ -105,4 +107,4 @@ trait DescribeDeltaDetailSuiteBase extends QueryTest } class DescribeDeltaDetailSuite - extends DescribeDeltaDetailSuiteBase with org.apache.spark.sql.delta.test.DeltaSQLCommandTest + extends DescribeDeltaDetailSuiteBase with DeltaSQLCommandTest 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 20a0367bfa3..6dbeb919ca2 100644 --- a/src/test/scala/org/apache/spark/sql/delta/DescribeDeltaHistorySuite.scala +++ b/src/test/scala/org/apache/spark/sql/delta/DescribeDeltaHistorySuite.scala @@ -18,10 +18,12 @@ package org.apache.spark.sql.delta import org.apache.spark.sql.delta.actions.CommitInfo import org.apache.spark.sql.delta.sources.DeltaSQLConf +import org.apache.spark.sql.delta.test.DeltaSQLCommandTest import org.scalatest.Tag import org.apache.spark.sql.{AnalysisException, Column, DataFrame, QueryTest, Row} import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.util.Utils @@ -276,8 +278,63 @@ trait DescribeDeltaHistorySuiteBase assertNotADeltaTableException(tempDir.getCanonicalPath) } } + + test("operation metrics - write metrics") { + withSQLConf(DeltaSQLConf.DELTA_HISTORY_METRICS_ENABLED.key -> "true") { + withTempDir { tempDir => + // create table + spark.range(100).repartition(5).write.format("delta").save(tempDir.getAbsolutePath) + val deltaTable = io.delta.tables.DeltaTable.forPath(tempDir.getAbsolutePath) + + // get last command history + val lastCmd = deltaTable.history(1) + + // Check if operation metrics from history are accurate + assert(lastCmd.select("operationMetrics.numFiles").take(1).head.getString(0).toLong + == 5) + + assert(lastCmd.select("operationMetrics.numOutputBytes").take(1).head.getString(0).toLong + > 0) + + assert(lastCmd.select("operationMetrics.numOutputRows").take(1).head.getString(0).toLong + == 100) + } + } + } + + test("operation metrics - merge") { + withSQLConf(DeltaSQLConf.DELTA_HISTORY_METRICS_ENABLED.key -> "true") { + withTempDir { tempDir => + // create target + spark.range(100).write.format("delta").save(tempDir.getAbsolutePath) + val deltaTable = io.delta.tables.DeltaTable.forPath(tempDir.getAbsolutePath) + + // run merge + deltaTable.as("t") + .merge(spark.range(50, 150).toDF().as("s"), "s.id = t.id") + .whenMatched() + .updateAll() + .whenNotMatched() + .insertAll() + .execute() + + // Get operation metrics + val lastCommand = deltaTable.history(1) + val operationMetrics: Map[String, String] = lastCommand.select("operationMetrics") + .take(1) + .head + .getMap(0) + .asInstanceOf[Map[String, String]] + + assert(operationMetrics("numTargetRowsInserted") == "50") + assert(operationMetrics("numTargetRowsUpdated") == "50") + assert(operationMetrics("numOutputRows") == "100") + assert(operationMetrics("numSourceRows") == "100") + } + } + } } class DescribeDeltaHistorySuite - extends DescribeDeltaHistorySuiteBase with org.apache.spark.sql.delta.test.DeltaSQLCommandTest + extends DescribeDeltaHistorySuiteBase with DeltaSQLCommandTest 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 c1687d642b0..25b9cd18388 100644 --- a/src/test/scala/org/apache/spark/sql/delta/MergeIntoScalaSuite.scala +++ b/src/test/scala/org/apache/spark/sql/delta/MergeIntoScalaSuite.scala @@ -117,6 +117,29 @@ class MergeIntoScalaSuite extends MergeIntoSuiteBase { } } + test("updateAll and insertAll with columns containing dot") { + withTable("source") { + append(Seq((1, 10), (2, 20), (4, 40)).toDF("key", "the.value"), Nil) // target + val source = Seq((1, 100), (3, 30), (4, 41)).toDF("key", "the.value") // source + + io.delta.tables.DeltaTable.forPath(spark, tempPath).as("t") + .merge(source.as("s"), "t.key = s.key") + .whenMatched() + .updateAll() + .whenNotMatched() + .insertAll() + .execute() + + checkAnswer( + readDeltaTable(tempPath), + Row(1, 100) :: // Update + Row(2, 20) :: // No change + Row(4, 41) :: // Update + Row(3, 30) :: // Insert + Nil) + } + } + test("update with empty map should do nothing") { append(Seq((1, 10), (2, 20)).toDF("trgKey", "trgValue"), Nil) // target val source = Seq((1, 100), (3, 30)).toDF("srcKey", "srcValue") // source 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 f08f42afb66..5a9a5c1b9fb 100644 --- a/src/test/scala/org/apache/spark/sql/delta/MergeIntoSuiteBase.scala +++ b/src/test/scala/org/apache/spark/sql/delta/MergeIntoSuiteBase.scala @@ -487,7 +487,8 @@ abstract class MergeIntoSuiteBase insert = "(key2, value) VALUES (key1 - 10, src.value + 10)") }.toString - assert(e.contains(DeltaErrors.multipleSourceRowMatchingTargetRowInMergeException.getMessage)) + val expectedEx = DeltaErrors.multipleSourceRowMatchingTargetRowInMergeException(spark) + assert(e.contains(expectedEx.getMessage)) } } @@ -695,7 +696,7 @@ abstract class MergeIntoSuiteBase update = "key2 = 20 + key1, value = 20 + src.value", insert = "(key2, value) VALUES (key1 - 10, src.value + 10)") }.getMessage - errorContains(e, "MERGE destination only supports Delta sources") + errorContains(e, "Expect a full scan of Delta sources, but found a partial scan") } } @@ -712,7 +713,7 @@ abstract class MergeIntoSuiteBase update = "key2 = 20 + key1, value = 20 + src.value", insert = "(key2, value) VALUES (key1 - 10, src.value + 10)") }.getMessage - errorContains(e, "Expect a full scan of Delta sources, but found the partial scan") + errorContains(e, "Expect a full scan of Delta sources, but found a partial scan") } Seq(true, false).foreach { isPartitioned => @@ -1185,7 +1186,7 @@ abstract class MergeIntoSuiteBase errorStrs = Nil) // subqueries fail for unresolved reference to `t` - private def testExtendedMerge( + protected def testExtendedMerge( name: String)( source: Seq[(Int, Int)], target: Seq[(Int, Int)], @@ -1195,7 +1196,9 @@ abstract class MergeIntoSuiteBase Seq(true, false).foreach { isPartitioned => test(s"extended syntax - $name - isPartitioned: $isPartitioned ") { withKeyValueData(source, target, isPartitioned) { case (sourceName, targetName) => - executeMerge(s"$targetName t", s"$sourceName s", mergeOn, mergeClauses: _*) + withSQLConf(DeltaSQLConf.MERGE_INSERT_ONLY_ENABLED.key -> "true") { + executeMerge(s"$targetName t", s"$sourceName s", mergeOn, mergeClauses: _*) + } val deltaPath = if (targetName.startsWith("delta.`")) { targetName.stripPrefix("delta.`").stripSuffix("`") } else targetName @@ -1320,18 +1323,6 @@ abstract class MergeIntoSuiteBase (3, 30) // (3, 30) inserted )) - test(s"extended syntax - only insert with multiple matches") { - withKeyValueData( - source = (0, 0) :: (1, 10) :: (1, 100) :: (3, 30) :: (3, 300) :: Nil, - target = (1, 1) :: (2, 2) :: Nil - ) { case (sourceName, targetName) => - intercept[UnsupportedOperationException] { - executeMerge(s"$targetName t", s"$sourceName s", "s.key = t.key", - insert(values = "(key, value) VALUES (s.key, s.value)")) - } - } - } - testExtendedMerge("only conditional insert")( source = (0, 0) :: (1, 10) :: (3, 30) :: Nil, target = (1, 1) :: (2, 2) :: Nil, @@ -1560,4 +1551,141 @@ abstract class MergeIntoSuiteBase update(set = "value = someValue"), insert(values = "*"))( errorStrs = "INSERT clause" :: "value" :: Nil) + + testExtendedMerge("insert only merge")( + source = (0, 0) :: (1, 10) :: (3, 30) :: Nil, + target = (1, 1) :: (2, 2) :: Nil, + mergeOn = "s.key = t.key", + insert(values = "*"))( + result = Seq( + (0, 0), // inserted + (1, 1), // existed previously + (2, 2), // existed previously + (3, 30) // inserted + )) + + testExtendedMerge("insert only merge with insert condition on source")( + source = (0, 0) :: (1, 10) :: (3, 30) :: Nil, + target = (1, 1) :: (2, 2) :: Nil, + mergeOn = "s.key = t.key", + insert(values = "*", condition = "s.key = s.value"))( + result = Seq( + (0, 0), // inserted + (1, 1), // existed previously + (2, 2) // existed previously + )) + + testExtendedMerge("insert only merge with predicate insert")( + source = (0, 0) :: (1, 10) :: (3, 30) :: Nil, + target = (1, 1) :: (2, 2) :: Nil, + mergeOn = "s.key = t.key", + insert(values = "(t.key, t.value) VALUES (s.key + 10, s.value + 10)"))( + result = Seq( + (10, 10), // inserted + (1, 1), // existed previously + (2, 2), // existed previously + (13, 40) // inserted + )) + + testExtendedMerge(s"insert only merge with multiple matches") ( + source = (0, 0) :: (1, 10) :: (1, 100) :: (3, 30) :: (3, 300) :: Nil, + target = (1, 1) :: (2, 2) :: Nil, + mergeOn = "s.key = t.key", + insert(values = "(key, value) VALUES (s.key, s.value)")) ( + result = Seq( + (0, 0), // inserted + (1, 1), // existed previously + (2, 2), // existed previously + (3, 30), // inserted + (3, 300) // key exists but still inserted + ) + ) + + + protected def testNullCaseInsertOnly(name: String)( + target: Seq[(JInt, JInt)], + source: Seq[(JInt, JInt)], + condition: String, + expectedResults: Seq[(JInt, JInt)], + insertCondition: Option[String] = None) = { + Seq(true, false).foreach { isPartitioned => + test(s"basic case - null handling - $name, isPartitioned: $isPartitioned") { + withView("sourceView") { + val partitions = if (isPartitioned) "key" :: Nil else Nil + append(target.toDF("key", "value"), partitions) + source.toDF("key", "value").createOrReplaceTempView("sourceView") + withSQLConf(DeltaSQLConf.MERGE_INSERT_ONLY_ENABLED.key -> "true") { + if (insertCondition.isDefined) { + executeMerge( + s"delta.`$tempPath` as t", + "sourceView s", + condition, + insert("(t.key, t.value) VALUES (s.key, s.value)", + condition = insertCondition.get)) + } else { + executeMerge( + s"delta.`$tempPath` as t", + "sourceView s", + condition, + insert("(t.key, t.value) VALUES (s.key, s.value)")) + } + } + checkAnswer( + readDeltaTable(tempPath), + expectedResults.map { r => Row(r._1, r._2) } + ) + + Utils.deleteRecursively(new File(tempPath)) + } + } + } + } + + testNullCaseInsertOnly("insert only merge - null in source") ( + target = Seq((1, 1)), + source = Seq((1, 10), (2, 20), (null, null)), + condition = "s.key = t.key", + expectedResults = Seq( + (1, 1), // Existing value + (2, 20), // Insert + (null, null) // Insert + )) + + testNullCaseInsertOnly("insert only merge - null value in both source and target")( + target = Seq((1, 1), (null, null)), + source = Seq((1, 10), (2, 20), (null, 0)), + condition = "s.key = t.key", + expectedResults = Seq( + (null, null), // No change as null in source does not match null in target + (1, 1), // Existing value + (2, 20), // Insert + (null, 0) // Insert + )) + + testNullCaseInsertOnly("insert only merge - null in insert clause")( + target = Seq((1, 1), (2, 20)), + source = Seq((1, 10), (3, 30), (null, 0)), + condition = "s.key = t.key", + expectedResults = Seq( + (1, 1), // Existing value + (2, 20), // Existing value + (null, 0) // Insert + ), + insertCondition = Some("s.key IS NULL") + ) + + test("insert only merge - turn off feature flag") { + withSQLConf(DeltaSQLConf.MERGE_INSERT_ONLY_ENABLED.key -> "false") { + withKeyValueData( + source = (0, 0) :: (1, 10) :: (1, 100) :: (3, 30) :: (3, 300) :: Nil, + target = (1, 1) :: (2, 2) :: Nil + ) { case (sourceName, targetName) => + intercept[UnsupportedOperationException] { + // This is supposed to fail as the duplicated keys in source were not supported. + executeMerge(s"$targetName t", s"$sourceName s", "s.key = t.key", + insert(values = "(key, value) VALUES (s.key, s.value)")) + } + } + } + } } 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 97994439d91..bc2bfcaec16 100644 --- a/src/test/scala/org/apache/spark/sql/delta/OptimisticTransactionSuite.scala +++ b/src/test/scala/org/apache/spark/sql/delta/OptimisticTransactionSuite.scala @@ -16,13 +16,16 @@ package org.apache.spark.sql.delta -import org.apache.spark.sql.delta.DeltaOperations.Truncate -import org.apache.spark.sql.delta.actions.{AddFile, Metadata, SetTransaction} +import java.util.ConcurrentModificationException + +import org.apache.spark.sql.delta.DeltaOperations.{Delete, ManualUpdate, Truncate} +import org.apache.spark.sql.delta.actions.{Action, AddFile, FileAction, Metadata, RemoveFile, SetTransaction} import org.apache.hadoop.fs.Path import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.{StringType, StructField, StructType} class OptimisticTransactionSuite extends QueryTest with SharedSparkSession { private val addA = AddFile("a", Map.empty, 1, 1, dataChange = true) @@ -34,14 +37,14 @@ 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. Truncate() is just a no-op placeholder. - log.startTransaction().commit(Nil, Truncate()) + // Initialize the log. ManualUpdate is just a no-op placeholder. + log.startTransaction().commit(Nil, ManualUpdate) val txn = log.startTransaction() val winningTxn = log.startTransaction() - winningTxn.commit(Metadata() :: Nil, Truncate()) + winningTxn.commit(Metadata() :: Nil, ManualUpdate) intercept[MetadataChangedException] { - txn.commit(addA :: Nil, Truncate()) + txn.commit(addA :: Nil, ManualUpdate) } } } @@ -49,17 +52,17 @@ 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. Truncate() is just a no-op placeholder. - log.startTransaction().commit(Metadata() :: Nil, Truncate()) + // Initialize the log. ManualUpdate is just a no-op placeholder. + log.startTransaction().commit(Metadata() :: Nil, ManualUpdate) val txn = log.startTransaction() // reads the table txn.filterFiles() val winningTxn = log.startTransaction() - winningTxn.commit(addA :: Nil, Truncate()) + winningTxn.commit(addA :: Nil, ManualUpdate) // TODO: intercept a more specific exception intercept[DeltaConcurrentModificationException] { - txn.commit(addB :: Nil, Truncate()) + txn.commit(addB :: Nil, ManualUpdate) } } } @@ -67,13 +70,13 @@ 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. Truncate() is just a no-op placeholder. - log.startTransaction().commit(addA :: Nil, Truncate()) + // Initialize the log and add data. ManualUpdate is just a no-op placeholder. + log.startTransaction().commit(addA :: Nil, ManualUpdate) val txn = log.startTransaction() val winningTxn = log.startTransaction() - winningTxn.commit(addA.remove :: addB :: Nil, Truncate()) - txn.commit(addC :: Nil, Truncate()) + winningTxn.commit(addA.remove :: addB :: Nil, ManualUpdate) + txn.commit(addC :: Nil, ManualUpdate) checkAnswer(log.update().allFiles.select("path"), Row("b") :: Row("c") :: Nil) } } @@ -81,29 +84,585 @@ class OptimisticTransactionSuite extends QueryTest with SharedSparkSession { test("allow read+append+delete against no data change") { withTempDir { tempDir => val log = DeltaLog(spark, new Path(tempDir.getCanonicalPath)) - // Initialize the log and add data. Truncate() is just a no-op placeholder. - log.startTransaction().commit(addA :: Nil, Truncate()) + // Initialize the log and add data. ManualUpdate is just a no-op placeholder. + log.startTransaction().commit(addA :: Nil, ManualUpdate) val txn = log.startTransaction() txn.filterFiles() val winningTxn = log.startTransaction() - winningTxn.commit(Nil, Truncate()) - txn.commit(addA.remove :: addB :: Nil, Truncate()) + winningTxn.commit(Nil, ManualUpdate) + txn.commit(addA.remove :: addB :: Nil, ManualUpdate) checkAnswer(log.update().allFiles.select("path"), Row("b") :: Nil) } } + + val A_P1 = "part=1/a" + val B_P1 = "part=1/b" + val C_P1 = "part=1/c" + val C_P2 = "part=2/c" + val D_P2 = "part=2/d" + val E_P3 = "part=3/e" + val F_P3 = "part=3/f" + val G_P4 = "part=4/g" + + private val addA_P1 = AddFile(A_P1, Map("part" -> "1"), 1, 1, dataChange = true) + private val addB_P1 = AddFile(B_P1, Map("part" -> "1"), 1, 1, dataChange = true) + private val addC_P1 = AddFile(C_P1, Map("part" -> "1"), 1, 1, dataChange = true) + private val addC_P2 = AddFile(C_P2, Map("part" -> "2"), 1, 1, dataChange = true) + private val addD_P2 = AddFile(D_P2, Map("part" -> "2"), 1, 1, dataChange = true) + private val addE_P3 = AddFile(E_P3, Map("part" -> "3"), 1, 1, dataChange = true) + private val addF_P3 = AddFile(F_P3, Map("part" -> "3"), 1, 1, dataChange = true) + private val addG_P4 = AddFile(G_P4, Map("part" -> "4"), 1, 1, dataChange = true) + + test("allow concurrent commit on disjoint partitions") { + withLog(addA_P1 :: addE_P3 :: Nil) { log => + val tx1 = log.startTransaction() + // TX1 reads P3 (but not P1) + val tx1Read = tx1.filterFiles(('part === 3).expr :: Nil) + assert(tx1Read.map(_.path) == E_P3 :: Nil) + + val tx2 = log.startTransaction() + tx2.filterFiles() + // TX2 modifies only P1 + tx2.commit(addB_P1 :: Nil, ManualUpdate) + + // free to commit because P1 modified by TX2 was not read + tx1.commit(addC_P2 :: addE_P3.remove :: Nil, ManualUpdate) + checkAnswer( + log.update().allFiles.select("path"), + Row(A_P1) :: // start (E_P3 was removed by TX1) + Row(B_P1) :: // TX2 + Row(C_P2) :: Nil) // TX1 + } + } + + test("allow concurrent commit on disjoint partitions reading all partitions") { + withLog(addA_P1 :: addD_P2 :: Nil) { log => + val tx1 = log.startTransaction() + // TX1 read P1 + tx1.filterFiles(('part isin 1).expr :: Nil) + + val tx2 = log.startTransaction() + tx2.filterFiles() + tx2.commit(addC_P2 :: addD_P2.remove :: Nil, ManualUpdate) + + tx1.commit(addE_P3 :: addF_P3 :: Nil, ManualUpdate) + + checkAnswer( + log.update().allFiles.select("path"), + Row(A_P1) :: // start + Row(C_P2) :: // TX2 + Row(E_P3) :: Row(F_P3) :: Nil) // TX1 + } + } + + test("block concurrent commit when read partition was appended to by concurrent write") { + withLog(addA_P1 :: addD_P2 :: addE_P3 :: Nil) { log => + val tx1 = log.startTransaction() + // TX1 reads only P1 + val tx1Read = tx1.filterFiles(('part === 1).expr :: Nil) + assert(tx1Read.map(_.path) == A_P1 :: Nil) + + val tx2 = log.startTransaction() + tx2.filterFiles() + // TX2 modifies only P1 + tx2.commit(addB_P1 :: Nil, ManualUpdate) + + intercept[ConcurrentAppendException] { + // P1 was modified + tx1.commit(addC_P2 :: addE_P3 :: Nil, ManualUpdate) + } + } + } + + test("block concurrent commit on full table scan") { + withLog(addA_P1 :: addD_P2 :: Nil) { log => + val tx1 = log.startTransaction() + // TX1 full table scan + tx1.filterFiles() + tx1.filterFiles(('part === 1).expr :: Nil) + + val tx2 = log.startTransaction() + tx2.filterFiles() + tx2.commit(addC_P2 :: addD_P2.remove :: Nil, ManualUpdate) + + intercept[ConcurrentAppendException] { + tx1.commit(addE_P3 :: addF_P3 :: Nil, ManualUpdate) + } + } + } + + val A_1_1 = "a=1/b=1/a" + val B_1_2 = "a=1/b=2/b" + val C_2_1 = "a=2/b=1/c" + val D_3_1 = "a=3/b=1/d" + + val addA_1_1_nested = AddFile( + A_1_1, Map("a" -> "1", "b" -> "1"), + 1, 1, dataChange = true) + val addB_1_2_nested = AddFile( + B_1_2, Map("a" -> "1", "b" -> "2"), + 1, 1, dataChange = true) + val addC_2_1_nested = AddFile( + C_2_1, Map("a" -> "2", "b" -> "1"), + 1, 1, dataChange = true) + val addD_3_1_nested = AddFile( + D_3_1, Map("a" -> "3", "b" -> "1"), + 1, 1, dataChange = true) + + test("allow concurrent adds to disjoint nested partitions when read is disjoint from write") { + withLog(addA_1_1_nested :: Nil, partitionCols = "a" :: "b" :: Nil) { log => + val tx1 = log.startTransaction() + // TX1 reads a=1/b=1 + val tx1Read = tx1.filterFiles(('a === 1 and 'b === 1).expr :: Nil) + assert(tx1Read.map(_.path) == A_1_1 :: Nil) + + val tx2 = log.startTransaction() + tx2.filterFiles() + // TX2 reads all partitions and modifies only a=1/b=2 + tx2.commit(addB_1_2_nested :: Nil, ManualUpdate) + + // TX1 reads a=1/b=1 which was not modified by TX2, hence TX1 can write to a=2/b=1 + tx1.commit(addC_2_1_nested :: Nil, ManualUpdate) + checkAnswer( + log.update().allFiles.select("path"), + Row(A_1_1) :: // start + Row(B_1_2) :: // TX2 + Row(C_2_1) :: Nil) // TX1 + } + } + + test("allow concurrent adds to same nested partitions when read is disjoint from write") { + withLog(addA_1_1_nested :: Nil, partitionCols = "a" :: "b" :: Nil) { log => + val tx1 = log.startTransaction() + // TX1 reads a=1/b=1 + val tx1Read = tx1.filterFiles(('a === 1 and 'b === 1).expr :: Nil) + assert(tx1Read.map(_.path) == A_1_1 :: Nil) + + val tx2 = log.startTransaction() + tx2.filterFiles() + // TX2 modifies a=1/b=2 + tx2.commit(addB_1_2_nested :: Nil, ManualUpdate) + + // TX1 reads a=1/b=1 which was not modified by TX2, hence TX1 can write to a=2/b=1 + val add = AddFile( + "a=1/b=2/x", Map("a" -> "1", "b" -> "2"), + 1, 1, dataChange = true) + tx1.commit(add :: Nil, ManualUpdate) + checkAnswer( + log.update().allFiles.select("path"), + Row(A_1_1) :: // start + Row(B_1_2) :: // TX2 + Row("a=1/b=2/x") :: Nil) // TX1 + } + } + + test("allow concurrent add when read at lvl1 partition is disjoint from concur. write at lvl2") { + withLog( + addA_1_1_nested :: addB_1_2_nested :: Nil, + partitionCols = "a" :: "b" :: Nil) { log => + val tx1 = log.startTransaction() + // TX1 reads a=1 + val tx1Read = tx1.filterFiles(('a === 1).expr :: Nil) + assert(tx1Read.map(_.path).toSet == Set(A_1_1, B_1_2)) + + val tx2 = log.startTransaction() + tx2.filterFiles() + // TX2 modifies only a=2/b=1 + tx2.commit(addC_2_1_nested :: Nil, ManualUpdate) + + // free to commit a=2/b=1 + tx1.commit(addD_3_1_nested :: Nil, ManualUpdate) + checkAnswer( + log.update().allFiles.select("path"), + Row(A_1_1) :: Row(B_1_2) :: // start + Row(C_2_1) :: // TX2 + Row(D_3_1) :: Nil) // TX1 + } + } + + test("block commit when read at lvl1 partition reads lvl2 file concur. deleted") { + withLog( + addA_1_1_nested :: addB_1_2_nested :: Nil, + partitionCols = "a" :: "b" :: Nil) { log => + + val tx1 = log.startTransaction() + // TX1 reads a=1 + val tx1Read = tx1.filterFiles(('a === 1).expr :: Nil) + assert(tx1Read.map(_.path).toSet == Set(A_1_1, B_1_2)) + + val tx2 = log.startTransaction() + tx2.filterFiles() + // TX2 modifies a=1/b=1 + tx2.commit(addA_1_1_nested.remove :: Nil, ManualUpdate) + + intercept[ConcurrentDeleteReadException] { + // TX2 modified a=1, which was read by TX1 + tx1.commit(addD_3_1_nested :: Nil, ManualUpdate) + } + } + } + + test("block commit when full table read conflicts with concur. write in lvl2 nested partition") { + withLog(addA_1_1_nested :: Nil, partitionCols = "a" :: "b" :: Nil) { log => + val tx1 = log.startTransaction() + // TX1 full table scan + tx1.filterFiles() + + val tx2 = log.startTransaction() + tx2.filterFiles() + // TX2 modifies only a=1/b=2 + tx2.commit(addB_1_2_nested :: Nil, ManualUpdate) + + intercept[ConcurrentAppendException] { + // TX2 modified table all of which was read by TX1 + tx1.commit(addC_2_1_nested :: Nil, ManualUpdate) + } + } + } + + test("block commit when part. range read conflicts with concur. write in lvl2 nested partition") { + withLog( + addA_1_1_nested :: Nil, + partitionCols = "a" :: "b" :: Nil) { log => + + val tx1 = log.startTransaction() + // TX1 reads multiple nested partitions a >= 1 or b > 1 + val tx1Read = tx1.filterFiles(('a >= 1 or 'b > 1).expr :: Nil) + assert(tx1Read.map(_.path).toSet == Set(A_1_1)) + + val tx2 = log.startTransaction() + tx2.filterFiles() + // TX2 modifies a=1/b=2 + tx2.commit(addB_1_2_nested :: Nil, ManualUpdate) + + intercept[ConcurrentAppendException] { + // partition a=1/b=2 conflicts with our read a >= 1 or 'b > 1 + tx1.commit(addD_3_1_nested :: Nil, ManualUpdate) + } + } + } + + test("block commit with concurrent removes on same file") { + withLog(addB_1_2_nested :: Nil, partitionCols = "a" :: "b" :: Nil) { log => + val tx1 = log.startTransaction() + // TX1 reads a=2 so that read is disjoint with write partition. + tx1.filterFiles(('a === 2).expr :: Nil) + + val tx2 = log.startTransaction() + tx2.filterFiles() + // TX2 modifies a=1/b=2 + tx2.commit(addB_1_2_nested.remove :: Nil, ManualUpdate) + + intercept[ConcurrentDeleteDeleteException] { + // TX1 read does not conflict with TX2 as disjoint partitions + // But TX2 removed the same file that TX1 is trying to remove + tx1.commit(addB_1_2_nested.remove:: Nil, ManualUpdate) + } + } + } + + test("block commit when full table read conflicts with add in any partition") { + withLog(addA_P1 :: Nil) { log => + val tx1 = log.startTransaction() + tx1.filterFiles() + + val tx2 = log.startTransaction() + tx2.filterFiles() + tx2.commit(addC_P2.remove :: addB_P1 :: Nil, ManualUpdate) + + intercept[ConcurrentAppendException] { + // TX1 read whole table but TX2 concurrently modified partition P2 + tx1.commit(addD_P2 :: Nil, ManualUpdate) + } + } + } + + test("block commit when full table read conflicts with delete in any partition") { + withLog(addA_P1 :: addC_P2 :: Nil) { log => + val tx1 = log.startTransaction() + tx1.filterFiles() + + val tx2 = log.startTransaction() + tx2.filterFiles() + tx2.commit(addA_P1.remove :: Nil, ManualUpdate) + + intercept[ConcurrentDeleteReadException] { + // TX1 read whole table but TX2 concurrently modified partition P1 + tx1.commit(addB_P1.remove :: Nil, ManualUpdate) + } + } + } + + test("block concurrent replaceWhere initial empty") { + withLog(addA_P1 :: Nil) { log => + val tx1 = log.startTransaction() + // replaceWhere (part >= 2) -> empty read + val tx1Read = tx1.filterFiles(('part >= 2).expr :: Nil) + assert(tx1Read.isEmpty) + + val tx2 = log.startTransaction() + // replaceWhere (part >= 2) -> empty read + val tx2Read = tx2.filterFiles(('part >= 2).expr :: Nil) + assert(tx2Read.isEmpty) + tx2.commit(addE_P3 :: Nil, ManualUpdate) + + intercept[ConcurrentAppendException] { + // Tx2 have modified P2 which conflicts with our read (part >= 2) + tx1.commit(addC_P2 :: Nil, ManualUpdate) + } + } + } + + test("allow concurrent replaceWhere disjoint partitions initial empty") { + withLog(addA_P1 :: Nil) { log => + val tx1 = log.startTransaction() + // replaceWhere (part > 2 and part <= 3) -> empty read + val tx1Read = tx1.filterFiles(('part > 1 and 'part <= 3).expr :: Nil) + assert(tx1Read.isEmpty) + + val tx2 = log.startTransaction() + // replaceWhere (part > 3) -> empty read + val tx2Read = tx2.filterFiles(('part > 3).expr :: Nil) + assert(tx2Read.isEmpty) + + tx1.commit(addC_P2 :: Nil, ManualUpdate) + // P2 doesn't conflict with read predicate (part > 3) + tx2.commit(addG_P4 :: Nil, ManualUpdate) + checkAnswer( + log.update().allFiles.select("path"), + Row(A_P1) :: // start + Row(C_P2) :: // TX1 + Row(G_P4) :: Nil) // TX2 + } + } + + test("block concurrent replaceWhere NOT empty but conflicting predicate") { + withLog(addA_P1 :: addG_P4 :: Nil) { log => + val tx1 = log.startTransaction() + // replaceWhere (part <= 3) -> read P1 + val tx1Read = tx1.filterFiles(('part <= 3).expr :: Nil) + assert(tx1Read.map(_.path) == A_P1 :: Nil) + val tx2 = log.startTransaction() + // replaceWhere (part >= 2) -> read P4 + val tx2Read = tx2.filterFiles(('part >= 2).expr :: Nil) + assert(tx2Read.map(_.path) == G_P4 :: Nil) + + tx1.commit(addA_P1.remove :: addC_P2 :: Nil, ManualUpdate) + intercept[ConcurrentAppendException] { + // Tx1 have modified P2 which conflicts with our read (part >= 2) + tx2.commit(addG_P4.remove :: addE_P3 :: Nil, ManualUpdate) + } + } + } + + test("block concurrent commit on read & add conflicting partitions") { + withLog(addA_P1 :: Nil) { log => + val tx1 = log.startTransaction() + // read P1 + val tx1Read = tx1.filterFiles(('part === 1).expr :: Nil) + assert(tx1Read.map(_.path) == A_P1 :: Nil) + + // tx2 commits before tx1 + val tx2 = log.startTransaction() + tx2.filterFiles() + tx2.commit(addB_P1 :: Nil, ManualUpdate) + + intercept[ConcurrentAppendException] { + // P1 read by TX1 was modified by TX2 + tx1.commit(addE_P3 :: Nil, ManualUpdate) + } + } + } + + test("block concurrent commit on read & delete conflicting partitions") { + withLog(addA_P1 :: addB_P1 :: Nil) { log => + val tx1 = log.startTransaction() + // read P1 + tx1.filterFiles(('part === 1).expr :: Nil) + + // tx2 commits before tx1 + val tx2 = log.startTransaction() + tx2.filterFiles() + tx2.commit(addA_P1.remove :: Nil, ManualUpdate) + + intercept[ConcurrentDeleteReadException] { + // P1 read by TX1 was removed by TX2 + tx1.commit(addE_P3 :: Nil, ManualUpdate) + } + } + } + + test("block 2 concurrent replaceWhere transactions") { + withLog(addA_P1 :: Nil) { log => + val tx1 = log.startTransaction() + // read P1 + tx1.filterFiles(('part === 1).expr :: Nil) + + val tx2 = log.startTransaction() + // read P1 + tx2.filterFiles(('part === 1).expr :: Nil) + + // tx1 commits before tx2 + tx1.commit(addA_P1.remove :: addB_P1 :: Nil, ManualUpdate) + + intercept[ConcurrentAppendException] { + // P1 read & deleted by TX1 is being modified by TX2 + tx2.commit(addA_P1.remove :: addC_P1 :: Nil, ManualUpdate) + } + } + } + + test("block 2 concurrent replaceWhere transactions changing partitions") { + withLog(addA_P1 :: addC_P2 :: addE_P3 :: Nil) { log => + val tx1 = log.startTransaction() + // read P3 + tx1.filterFiles(('part === 3 or 'part === 1).expr :: Nil) + + val tx2 = log.startTransaction() + // read P3 + tx2.filterFiles(('part === 3 or 'part === 2).expr :: Nil) + + // tx1 commits before tx2 + tx1.commit(addA_P1.remove :: addE_P3.remove :: addB_P1 :: Nil, ManualUpdate) + + intercept[ConcurrentDeleteReadException] { + // P3 read & deleted by TX1 is being modified by TX2 + tx2.commit(addC_P2.remove :: addE_P3.remove :: addD_P2 :: Nil, ManualUpdate) + } + } + } + + test("block concurrent full table scan after concurrent write completes") { + withLog(addA_P1 :: addC_P2 :: addE_P3 :: Nil) { log => + val tx1 = log.startTransaction() + + val tx2 = log.startTransaction() + tx2.filterFiles() + tx2.commit(addC_P2 :: Nil, ManualUpdate) + + tx1.filterFiles(('part === 1).expr :: Nil) + // full table scan + tx1.filterFiles() + + intercept[ConcurrentAppendException] { + tx1.commit(addA_P1.remove :: Nil, ManualUpdate) + } + } + } + + test("block concurrent commit mixed metadata and data predicate") { + withLog(addA_P1 :: addC_P2 :: addE_P3 :: Nil) { log => + val tx1 = log.startTransaction() + + val tx2 = log.startTransaction() + tx2.filterFiles() + tx2.commit(addC_P2 :: Nil, ManualUpdate) + + // actually a full table scan + tx1.filterFiles(('part === 1 or 'year > 2019).expr :: Nil) + + intercept[ConcurrentAppendException] { + tx1.commit(addA_P1.remove :: Nil, ManualUpdate) + } + } + } + + test("block concurrent read (2 scans) and add when read partition was changed by concur. write") { + withLog(addA_P1 :: addE_P3 :: Nil) { log => + val tx1 = log.startTransaction() + tx1.filterFiles(('part === 1).expr :: Nil) + + val tx2 = log.startTransaction() + tx2.filterFiles() + tx2.commit(addC_P2 :: Nil, ManualUpdate) + + tx1.filterFiles(('part > 1 and 'part < 3).expr :: Nil) + + intercept[ConcurrentAppendException] { + // P2 added by TX2 conflicts with our read condition 'part > 1 and 'part < 3 + tx1.commit(addA_P1.remove :: Nil, ManualUpdate) + } + } + } + + def setDataChangeFalse(fileActions: Seq[FileAction]): Seq[FileAction] = { + fileActions.map { + case a: AddFile => a.copy(dataChange = false) + case r: RemoveFile => r.copy(dataChange = false) + } + } + + test("no data change: allow data rearrange when new files concurrently added") { + withLog(addA_P1 :: addB_P1 :: Nil) { log => + val tx1 = log.startTransaction() + tx1.filterFiles() + + val tx2 = log.startTransaction() + tx2.filterFiles() + tx2.commit( + addE_P3 :: Nil, + ManualUpdate) + + // tx1 rearranges files + tx1.commit( + setDataChangeFalse(addA_P1.remove :: addB_P1.remove :: addC_P1 :: Nil), + ManualUpdate) + + checkAnswer( + log.update().allFiles.select("path"), + Row(C_P1) :: Row(E_P3) :: Nil) + } + } + + test("no data change: block data rearrange when concurrently delete removes same file") { + withLog(addA_P1 :: addB_P1 :: Nil) { log => + val tx1 = log.startTransaction() + tx1.filterFiles() + + // tx2 removes file + val tx2 = log.startTransaction() + tx2.filterFiles() + tx2.commit(addA_P1.remove :: Nil, ManualUpdate) + + intercept[ConcurrentDeleteReadException] { + // tx1 reads to rearrange the same file that tx2 deleted + tx1.commit( + setDataChangeFalse(addA_P1.remove :: addB_P1.remove :: addC_P1 :: Nil), + ManualUpdate) + } + } + } + + def withLog( + actions: Seq[Action], + partitionCols: Seq[String] = "part" :: Nil)( + test: DeltaLog => Unit): Unit = { + + val schema = new StructType(partitionCols.map(p => new StructField(p, StringType)).toArray) + var actionWithMetaData = + actions :+ Metadata(partitionColumns = partitionCols, schemaString = schema.json) + + withTempDir { tempDir => + val log = DeltaLog(spark, new Path(tempDir.getCanonicalPath)) + // Initialize the log and add data. ManualUpdate is just a no-op placeholder. + log.startTransaction().commit(actionWithMetaData, ManualUpdate) + test(log) + } + } + 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. Truncate() is just a no-op placeholder. - log.startTransaction().commit(Nil, Truncate()) + // Initialize the log and add data. ManualUpdate is just a no-op placeholder. + log.startTransaction().commit(Nil, ManualUpdate) val txn = log.startTransaction() txn.txnVersion("t1") val winningTxn = log.startTransaction() - winningTxn.commit(SetTransaction("t2", 1, Some(1234L)) :: Nil, Truncate()) - txn.commit(Nil, Truncate()) + winningTxn.commit(SetTransaction("t2", 1, Some(1234L)) :: Nil, ManualUpdate) + txn.commit(Nil, ManualUpdate) assert(log.update().transactions === Map("t2" -> 1)) } @@ -112,16 +671,16 @@ 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. Truncate() is just a no-op placeholder. - log.startTransaction().commit(Nil, Truncate()) + // Initialize the log and add data. ManualUpdate is just a no-op placeholder. + log.startTransaction().commit(Nil, ManualUpdate) val txn = log.startTransaction() txn.txnVersion("t1") val winningTxn = log.startTransaction() - winningTxn.commit(SetTransaction("t1", 1, Some(1234L)) :: Nil, Truncate()) + winningTxn.commit(SetTransaction("t1", 1, Some(1234L)) :: Nil, ManualUpdate) intercept[ConcurrentTransactionException] { - txn.commit(Nil, Truncate()) + txn.commit(Nil, ManualUpdate) } } } diff --git a/src/test/scala/org/apache/spark/sql/delta/schema/CaseSensitivitySuite.scala b/src/test/scala/org/apache/spark/sql/delta/schema/CaseSensitivitySuite.scala new file mode 100644 index 00000000000..a0e34a11fa1 --- /dev/null +++ b/src/test/scala/org/apache/spark/sql/delta/schema/CaseSensitivitySuite.scala @@ -0,0 +1,215 @@ +/* + * Copyright 2019 Databricks, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.delta.schema + +import java.io.File + +import org.apache.spark.sql.delta.DeltaLog +import org.apache.spark.sql.delta.actions.AddFile + +import org.apache.spark.sql._ +import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.functions.col +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.streaming.{StreamingQuery, StreamingQueryException} +import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} + +class CaseSensitivitySuite extends QueryTest + with SharedSparkSession with SQLTestUtils { + + import testImplicits._ + + private def testWithCaseSensitivity(name: String)(f: => Unit): Unit = { + testQuietly(name) { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + f + } + + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + f + } + } + } + + private def getPartitionValues(allFiles: Dataset[AddFile], colName: String): Array[String] = { + allFiles.select(col(s"partitionValues.$colName")).where(col(colName).isNotNull) + .distinct().as[String].collect() + } + + testWithCaseSensitivity("case sensitivity of partition fields") { + withTempDir { tempDir => + val query = "SELECT id + 1 as Foo, id as Bar FROM RANGE(1)" + sql(query).write.partitionBy("foo").format("delta").save(tempDir.getAbsolutePath) + checkAnswer( + sql(query), + spark.read.format("delta").load(tempDir.getAbsolutePath) + ) + + val allFiles = DeltaLog.forTable(spark, tempDir.getAbsolutePath).snapshot.allFiles + assert(getPartitionValues(allFiles, "Foo") === Array("1")) + checkAnswer( + spark.read.format("delta").load(tempDir.getAbsolutePath), + Row(1L, 0L) + ) + } + } + + testQuietly("case sensitivity of partition fields (stream)") { + // DataStreamWriter auto normalizes partition columns, therefore we don't need to check + // case sensitive case + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + withTempDir { tempDir => + val memSource = MemoryStream[(Long, Long)] + val stream1 = startStream(memSource.toDF().toDF("Foo", "Bar"), tempDir) + try { + memSource.addData((1L, 0L)) + stream1.processAllAvailable() + } finally { + stream1.stop() + } + + checkAnswer( + spark.read.format("delta").load(tempDir.getAbsolutePath), + Row(1L, 0L) + ) + + val allFiles = DeltaLog.forTable(spark, tempDir.getAbsolutePath).snapshot.allFiles + assert(getPartitionValues(allFiles, "Foo") === Array("1")) + } + } + } + + testWithCaseSensitivity("two fields with same name") { + withTempDir { tempDir => + intercept[AnalysisException] { + val query = "SELECT id as Foo, id as foo FROM RANGE(1)" + sql(query).write.partitionBy("foo").format("delta").save(tempDir.getAbsolutePath) + } + } + } + + testWithCaseSensitivity("two fields with same name (stream)") { + withTempDir { tempDir => + val memSource = MemoryStream[(Long, Long)] + val stream1 = startStream(memSource.toDF().toDF("Foo", "foo"), tempDir) + try { + val e = intercept[StreamingQueryException] { + memSource.addData((0L, 0L)) + stream1.processAllAvailable() + } + assert(e.cause.isInstanceOf[AnalysisException]) + } finally { + stream1.stop() + } + } + } + + testWithCaseSensitivity("schema merging is case insenstive but preserves original case") { + withTempDir { tempDir => + val query1 = "SELECT id as foo, id as bar FROM RANGE(1)" + sql(query1).write.format("delta").save(tempDir.getAbsolutePath) + + val query2 = "SELECT id + 1 as Foo, id as bar FROM RANGE(1)" // notice how 'F' is capitalized + sql(query2).write.format("delta").mode("append").save(tempDir.getAbsolutePath) + + val query3 = "SELECT id as bAr, id + 2 as Foo FROM RANGE(1)" // changed order as well + sql(query3).write.format("delta").mode("append").save(tempDir.getAbsolutePath) + + val df = spark.read.format("delta").load(tempDir.getAbsolutePath) + checkAnswer( + df, + Row(0, 0) :: Row(1, 0) :: Row(2, 0) :: Nil + ) + assert(df.schema.fieldNames === Seq("foo", "bar")) + } + } + + testWithCaseSensitivity("schema merging preserving column case (stream)") { + withTempDir { tempDir => + val memSource = MemoryStream[(Long, Long)] + val stream1 = startStream(memSource.toDF().toDF("Foo", "Bar"), tempDir, None) + try { + memSource.addData((0L, 0L)) + stream1.processAllAvailable() + } finally { + stream1.stop() + } + val stream2 = startStream(memSource.toDF().toDF("foo", "Bar"), tempDir, None) + try { + memSource.addData((1L, 2L)) + stream2.processAllAvailable() + } finally { + stream2.stop() + } + + val df = spark.read.format("delta").load(tempDir.getAbsolutePath) + checkAnswer( + df, + Row(0L, 0L) :: Row(1L, 2L) :: Nil + ) + assert(df.schema.fieldNames === Seq("Foo", "Bar")) + } + } + + test("SC-12677: replaceWhere predicate should be case insensitive") { + withTempDir { tempDir => + val path = tempDir.getCanonicalPath + Seq((1, "a"), (2, "b")).toDF("Key", "val").write + .partitionBy("key").format("delta").mode("append").save(path) + + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + Seq((2, "c")).toDF("Key", "val").write + .format("delta") + .mode("overwrite") + .option("replaceWhere", "key = 2") // note the different case + .save(path) + } + + checkAnswer( + spark.read.format("delta").load(path), + Row(1, "a") :: Row(2, "c") :: Nil + ) + + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + val e = intercept[AnalysisException] { + Seq((2, "d")).toDF("Key", "val").write + .format("delta") + .mode("overwrite") + .option("replaceWhere", "key = 2") // note the different case + .save(path) + } + assert(e.getMessage.contains("Key")) + } + + checkAnswer( + spark.read.format("delta").load(path), + Row(1, "a") :: Row(2, "c") :: Nil + ) + } + } + + private def startStream( + df: Dataset[_], + tempDir: File, + partitionBy: Option[String] = Some("foo")): StreamingQuery = { + val writer = df.writeStream + .option("checkpointLocation", new File(tempDir, "_checkpoint").getAbsolutePath) + .format("delta") + partitionBy.foreach(writer.partitionBy(_)) + writer.start(tempDir.getAbsolutePath) + } +} diff --git a/src/test/scala/org/apache/spark/sql/delta/schema/InvariantEnforcementSuite.scala b/src/test/scala/org/apache/spark/sql/delta/schema/InvariantEnforcementSuite.scala new file mode 100644 index 00000000000..ddfdf015a8d --- /dev/null +++ b/src/test/scala/org/apache/spark/sql/delta/schema/InvariantEnforcementSuite.scala @@ -0,0 +1,424 @@ +/* + * Copyright 2019 Databricks, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.delta.schema + +import java.io.File + +import scala.collection.JavaConverters._ + +import org.apache.spark.sql.delta.{DeltaLog, DeltaOperations} +import org.apache.spark.sql.delta.actions.Metadata +import org.apache.spark.sql.delta.schema.Invariants.{ArbitraryExpression, NotNull, PersistedExpression} + +import org.apache.spark.SparkException +import org.apache.spark.sql._ +import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.streaming.StreamingQueryException +import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} +import org.apache.spark.sql.types._ + +class InvariantEnforcementSuite extends QueryTest + with SharedSparkSession with SQLTestUtils { + + import testImplicits._ + + private def tableWithSchema(schema: StructType)(f: String => Unit): Unit = { + withTempDir { tempDir => + val deltaLog = DeltaLog.forTable(spark, tempDir) + val txn = deltaLog.startTransaction() + txn.commit(Metadata(schemaString = schema.json) :: Nil, DeltaOperations.ManualUpdate) + spark.read.format("delta") + .load(tempDir.getAbsolutePath) + .write + .format("delta") + .mode("overwrite") + .save(tempDir.getAbsolutePath) + f(tempDir.getAbsolutePath) + } + } + + private def testBatchWriteRejection( + invariant: Invariants.Rule, + schema: StructType, + df: Dataset[_], + expectedErrors: String*): Unit = { + tableWithSchema(schema) { path => + val e = intercept[SparkException] { + df.write.mode("append").format("delta").save(path) + } + var violationException = e.getCause + while (violationException != null && + !violationException.isInstanceOf[InvariantViolationException]) { + violationException = violationException.getCause + } + if (violationException == null) { + fail("Didn't receive a InvariantViolationException.") + } + assert(violationException.isInstanceOf[InvariantViolationException]) + val error = violationException.getMessage + val allExpected = Seq(invariant.name) ++ expectedErrors + allExpected.foreach { expected => + assert(error.contains(expected), s"$error didn't contain $expected") + } + } + } + + private def testStreamingWriteRejection[T: Encoder]( + invariant: Invariants.Rule, + schema: StructType, + toDF: MemoryStream[T] => DataFrame, + data: Seq[T], + expectedErrors: String*): Unit = { + withTempDir { dir => + val deltaLog = DeltaLog.forTable(spark, dir) + val txn = deltaLog.startTransaction() + txn.commit(Metadata(schemaString = schema.json) :: Nil, DeltaOperations.ManualUpdate) + val memStream = MemoryStream[T] + val stream = toDF(memStream).writeStream + .outputMode("append") + .format("delta") + .option("checkpointLocation", new File(dir, "_checkpoint").getAbsolutePath) + .start(dir.getAbsolutePath) + try { + val e = intercept[StreamingQueryException] { + memStream.addData(data) + stream.processAllAvailable() + } + var violationException = e.getCause + while (violationException != null && + !violationException.isInstanceOf[InvariantViolationException]) { + violationException = violationException.getCause + } + if (violationException == null) { + fail("Didn't receive a InvariantViolationException.") + } + assert(violationException.isInstanceOf[InvariantViolationException]) + val error = violationException.getMessage + assert((Seq(invariant.name) ++ expectedErrors).forall(error.contains)) + } finally { + stream.stop() + } + } + } + + private def testStreamingWrite[T: Encoder]( + schema: StructType, + toDF: MemoryStream[T] => DataFrame, + data: Seq[T], + expected: DataFrame): Unit = { + withTempDir { dir => + val deltaLog = DeltaLog.forTable(spark, dir) + val txn = deltaLog.startTransaction() + txn.commit(Metadata(schemaString = schema.json) :: Nil, DeltaOperations.ManualUpdate) + val memStream = MemoryStream[T] + val stream = toDF(memStream).writeStream + .outputMode("append") + .format("delta") + .option("checkpointLocation", new File(dir, "_checkpoint").getAbsolutePath) + .start(dir.getAbsolutePath) + try { + memStream.addData(data) + stream.processAllAvailable() + + checkAnswer( + spark.read.format("delta").load(dir.getAbsolutePath), + expected + ) + } finally { + stream.stop() + } + } + } + + testQuietly("reject non-nullable top level column") { + val schema = new StructType() + .add("key", StringType, nullable = false) + .add("value", IntegerType) + testBatchWriteRejection( + NotNull, + schema, + Seq[(String, Int)](("a", 1), (null, 2)).toDF("key", "value"), + "key" + ) + testStreamingWriteRejection[(String, Int)]( + NotNull, + schema, + _.toDF().toDF("key", "value"), + Seq[(String, Int)](("a", 1), (null, 2)), + "key" + ) + } + + testQuietly("reject non-nullable top level column - column doesn't exist") { + val schema = new StructType() + .add("key", StringType, nullable = false) + .add("value", IntegerType) + testBatchWriteRejection( + NotNull, + schema, + Seq[Int](1, 2).toDF("value"), + "key" + ) + testStreamingWriteRejection[Int]( + NotNull, + schema, + _.toDF().toDF("value"), + Seq[Int](1, 2), + "key" + ) + } + + testQuietly("write empty DataFrame - zero rows") { + val schema = new StructType() + .add("key", StringType, nullable = false) + .add("value", IntegerType) + tableWithSchema(schema) { path => + spark.createDataFrame(Seq.empty[Row].asJava, schema.asNullable).write + .mode("append").format("delta").save(path) + } + } + + testQuietly("write empty DataFrame - zero columns") { + val schema = new StructType() + .add("key", StringType, nullable = false) + .add("value", IntegerType) + testBatchWriteRejection( + NotNull, + schema, + Seq[Int](1, 2).toDF("value").drop("value"), + "key" + ) + testStreamingWriteRejection[Int]( + NotNull, + schema, + _.toDF().toDF("value").drop("value"), + Seq[Int](1, 2), + "key" + ) + } + + testQuietly("reject non-nullable nested column") { + val schema = new StructType() + .add("top", new StructType() + .add("key", StringType, nullable = false) + .add("value", IntegerType)) + testBatchWriteRejection( + NotNull, + schema, + spark.createDataFrame(Seq(Row(Row("a", 1)), Row(Row(null, 2))).asJava, schema.asNullable), + "top.key" + ) + testBatchWriteRejection( + NotNull, + schema, + spark.createDataFrame(Seq(Row(Row("a", 1)), Row(null)).asJava, schema.asNullable), + "top.key" + ) + } + + testQuietly("complex type - children of array type can't be checked") { + val schema = new StructType() + .add("top", ArrayType(ArrayType(new StructType() + .add("key", StringType, nullable = false) + .add("value", IntegerType)))) + tableWithSchema(schema) { path => + spark.createDataFrame(Seq(Row(Seq(Seq(Row("a", 1)))), Row(Seq(Seq(Row(null, 2))))).asJava, + schema.asNullable).write.mode("append").format("delta").save(path) + spark.createDataFrame(Seq(Row(Seq(Seq(Row("a", 1)))), Row(null)).asJava, schema.asNullable) + .write.mode("append").format("delta").save(path) + } + } + + testQuietly("reject non-nullable array column") { + val schema = new StructType() + .add("top", ArrayType(ArrayType(new StructType() + .add("key", StringType) + .add("value", IntegerType))), nullable = false) + testBatchWriteRejection( + NotNull, + schema, + spark.createDataFrame(Seq(Row(Seq(Seq(Row("a", 1)))), Row(null)).asJava, schema.asNullable), + "top" + ) + } + + testQuietly("reject expression invariant on top level column") { + val expr = "value < 3" + val rule = ArbitraryExpression(spark, expr) + val metadata = new MetadataBuilder() + .putString(Invariants.INVARIANTS_FIELD, PersistedExpression(expr).json) + .build() + val schema = new StructType() + .add("key", StringType) + .add("value", IntegerType, nullable = true, metadata) + testBatchWriteRejection( + rule, + schema, + Seq[(String, Int)](("a", 1), (null, 5)).toDF("key", "value"), + "value", "5" + ) + testStreamingWriteRejection[(String, Int)]( + rule, + schema, + _.toDF().toDF("key", "value"), + Seq[(String, Int)](("a", 1), (null, 5)), + "value" + ) + } + + testQuietly("reject expression invariant on nested column") { + val expr = "top.key < 3" + val rule = ArbitraryExpression(spark, expr) + val metadata = new MetadataBuilder() + .putString(Invariants.INVARIANTS_FIELD, PersistedExpression(expr).json) + .build() + val schema = new StructType() + .add("top", new StructType() + .add("key", StringType) + .add("value", IntegerType, nullable = true, metadata)) + testBatchWriteRejection( + rule, + schema, + spark.createDataFrame(Seq(Row(Row("a", 1)), Row(Row(null, 5))).asJava, schema.asNullable), + "top.key", "5" + ) + } + + testQuietly("reject write on top level expression invariant when field is null") { + val expr = "value < 3" + val metadata = new MetadataBuilder() + .putString(Invariants.INVARIANTS_FIELD, PersistedExpression(expr).json) + .build() + val rule = ArbitraryExpression(spark, expr) + val schema = new StructType() + .add("key", StringType) + .add("value", IntegerType, nullable = true, metadata) + testBatchWriteRejection( + rule, + schema, + Seq[String]("a", "b").toDF("key"), + "value", "null" + ) + testBatchWriteRejection( + rule, + schema, + Seq[(String, Integer)](("a", 1), ("b", null)).toDF("key", "value"), + "value", "null" + ) + } + + testQuietly("reject write on nested expression invariant when field is null") { + val expr = "top.value < 3" + val metadata = new MetadataBuilder() + .putString(Invariants.INVARIANTS_FIELD, PersistedExpression(expr).json) + .build() + val rule = ArbitraryExpression(spark, expr) + val schema = new StructType() + .add("top", new StructType() + .add("key", StringType) + .add("value", IntegerType, nullable = true, metadata)) + testBatchWriteRejection( + rule, + schema, + spark.createDataFrame(Seq(Row(Row("a", 1)), Row(Row("b", null))).asJava, schema.asNullable), + "top.value", "null" + ) + val schema2 = new StructType() + .add("top", new StructType() + .add("key", StringType)) + testBatchWriteRejection( + rule, + schema, + spark.createDataFrame(Seq(Row(Row("a")), Row(Row("b"))).asJava, schema2.asNullable), + "top.value", "null" + ) + } + + testQuietly("is null on top level expression invariant when field is null") { + val expr = "value is null or value < 3" + val metadata = new MetadataBuilder() + .putString(Invariants.INVARIANTS_FIELD, PersistedExpression(expr).json) + .build() + val schema = new StructType() + .add("key", StringType) + .add("value", IntegerType, nullable = true, metadata) + tableWithSchema(schema) { path => + Seq[String]("a", "b").toDF("key").write + .mode("append").format("delta").save(path) + Seq[(String, Integer)](("a", 1), ("b", null)).toDF("key", "value").write + .mode("append").format("delta").save(path) + } + } + + testQuietly("is null on nested expression invariant when field is null") { + val expr = "top.value is null or top.value < 3" + val metadata = new MetadataBuilder() + .putString(Invariants.INVARIANTS_FIELD, PersistedExpression(expr).json) + .build() + val schema = new StructType() + .add("top", new StructType() + .add("key", StringType) + .add("value", IntegerType, nullable = true, metadata)) + val schema2 = new StructType() + .add("top", new StructType() + .add("key", StringType)) + tableWithSchema(schema) { path => + spark.createDataFrame(Seq(Row(Row("a", 1)), Row(Row("b", null))).asJava, schema.asNullable) + .write.mode("append").format("delta").save(path) + spark.createDataFrame(Seq(Row(Row("a")), Row(Row("b"))).asJava, schema2.asNullable) + .write.mode("append").format("delta").save(path) + } + } + + testQuietly("complex expressions - AND") { + val expr = "value < 3 AND value > 0" + val metadata = new MetadataBuilder() + .putString(Invariants.INVARIANTS_FIELD, PersistedExpression(expr).json) + .build() + val schema = new StructType() + .add("key", StringType) + .add("value", IntegerType, nullable = true, metadata) + tableWithSchema(schema) { path => + Seq(1, 2).toDF("value").write.mode("append").format("delta").save(path) + intercept[SparkException] { + Seq(1, 4).toDF("value").write.mode("append").format("delta").save(path) + } + intercept[SparkException] { + Seq(-1, 2).toDF("value").write.mode("append").format("delta").save(path) + } + } + } + + testQuietly("complex expressions - IN SET") { + val expr = "key in ('a', 'b', 'c')" + val metadata = new MetadataBuilder() + .putString(Invariants.INVARIANTS_FIELD, PersistedExpression(expr).json) + .build() + val schema = new StructType() + .add("key", StringType, nullable = true, metadata) + .add("value", IntegerType) + tableWithSchema(schema) { tempDir => + Seq("a", "b").toDF("key").write.mode("append").format("delta").save(tempDir) + intercept[SparkException] { + Seq("a", "d").toDF("key").write.mode("append").format("delta").save(tempDir) + } + intercept[SparkException] { + Seq("e").toDF("key").write.mode("append").format("delta").save(tempDir) + } + } + } +} diff --git a/src/test/scala/org/apache/spark/sql/delta/schema/SchemaEnforcementSuite.scala b/src/test/scala/org/apache/spark/sql/delta/schema/SchemaEnforcementSuite.scala new file mode 100644 index 00000000000..b67d117a8e2 --- /dev/null +++ b/src/test/scala/org/apache/spark/sql/delta/schema/SchemaEnforcementSuite.scala @@ -0,0 +1,857 @@ +/* + * Copyright 2019 Databricks, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.delta.schema + +import java.io.File + +// scalastyle:off import.ordering.noEmptyLine +import org.apache.spark.sql.delta.{DeltaLog, DeltaOptions} +import org.apache.spark.sql.delta.actions.SingleAction +import org.apache.spark.sql.delta.sources.DeltaSQLConf +import org.apache.spark.sql.delta.util.{FileNames, JsonUtils} + +import org.apache.spark.SparkConf +import org.apache.spark.sql._ +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.streaming.StreamingQueryException +import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} +import org.apache.spark.sql.types._ + +sealed trait SaveOperation { + def apply(dfw: DataFrameWriter[_]): Unit +} + +case class SaveWithPath(path: String = null) extends SaveOperation { + override def apply(dfw: DataFrameWriter[_]): Unit = { + if (path == null) dfw.save() else dfw.save(path) + } +} + +// Not supported yet +case class SaveAsTable(tableName: String) extends SaveOperation { + override def apply(dfw: DataFrameWriter[_]): Unit = dfw.saveAsTable(tableName) +} + +sealed trait SchemaEnforcementSuiteBase + extends QueryTest with SharedSparkSession { + protected def enableAutoMigration(f: => Unit): Unit = { + withSQLConf(DeltaSQLConf.DELTA_SCHEMA_AUTO_MIGRATE.key -> "true") { + f + } + } + + protected def disableAutoMigration(f: => Unit): Unit = { + withSQLConf(DeltaSQLConf.DELTA_SCHEMA_AUTO_MIGRATE.key -> "false") { + f + } + } +} + +sealed trait BatchWriterTest extends SchemaEnforcementSuiteBase with SharedSparkSession { + + def saveOperation: SaveOperation + + implicit class RichDataFrameWriter(dfw: DataFrameWriter[_]) { + def append(path: File): Unit = { + saveOperation(dfw.format("delta").mode("append").option("path", path.getAbsolutePath)) + } + + def overwrite(path: File): Unit = { + saveOperation(dfw.format("delta").mode("overwrite").option("path", path.getAbsolutePath)) + } + } + + def equivalenceTest(testName: String)(f: => Unit): Unit = { + test(s"batch: $testName") { + saveOperation match { + case _: SaveWithPath => f + case SaveAsTable(tbl) => withTable(tbl) { f } + } + } + } +} + +trait AppendSaveModeTests extends BatchWriterTest { + import testImplicits._ + + equivalenceTest("reject schema changes by default") { + disableAutoMigration { + withTempDir { dir => + spark.range(10).write.append(dir) + val e = intercept[AnalysisException] { + spark.range(10).withColumn("part", 'id + 1).write.append(dir) + } + assert(e.getMessage.contains(DeltaOptions.MERGE_SCHEMA_OPTION)) + } + } + } + + equivalenceTest("allow schema changes when autoMigrate is enabled") { + enableAutoMigration { + withTempDir { dir => + spark.range(10).write.append(dir) + spark.range(10).withColumn("part", 'id + 1).write.append(dir) + assert(spark.read.format("delta").load(dir.getAbsolutePath).schema.length == 2) + } + } + } + + equivalenceTest("disallow schema changes when autoMigrate enabled but writer config disabled") { + enableAutoMigration { + withTempDir { dir => + spark.range(10).write.append(dir) + val e = intercept[AnalysisException] { + spark.range(10).withColumn("part", 'id + 1).write + .option(DeltaOptions.MERGE_SCHEMA_OPTION, "false").append(dir) + } + assert(e.getMessage.contains(DeltaOptions.MERGE_SCHEMA_OPTION)) + } + } + } + + equivalenceTest("allow schema change with option") { + disableAutoMigration { + withTempDir { dir => + spark.range(10).write.append(dir) + spark.range(10).withColumn("part", 'id + 1).write + .option(DeltaOptions.MERGE_SCHEMA_OPTION, "true").append(dir) + assert(spark.read.format("delta").load(dir.getAbsolutePath).schema.length == 2) + } + } + } + + equivalenceTest("JSON ETL workflow, schema merging NullTypes") { + enableAutoMigration { + val row1 = """{"key":"abc","id":null,"extra":1}""" + val row2 = """{"key":"def","id":2,"extra":null}""" + val row3 = """{"key":"ghi","id":null,"extra":3}""" + withTempDir { dir => + val schema1 = new StructType() + .add("key", StringType).add("id", NullType).add("extra", IntegerType) + val schema2 = new StructType() + .add("key", StringType).add("id", IntegerType).add("extra", NullType) + spark.read.schema(schema1).json(Seq(row1).toDS()).write.append(dir) + spark.read.schema(schema2).json(Seq(row2).toDS()).write.append(dir) + spark.read.schema(schema1).json(Seq(row3).toDS()).write.append(dir) + + checkAnswer( + spark.read.format("delta").load(dir.getAbsolutePath), + Row("abc", null, 1) :: Row("def", 2, null) :: Row("ghi", null, 3) :: Nil + ) + } + } + } + + equivalenceTest("JSON ETL workflow, schema merging NullTypes - nested struct") { + enableAutoMigration { + val row1 = """{"key":"abc","top":{"id":null,"extra":1}}""" + val row2 = """{"key":"def","top":{"id":2,"extra":null}}""" + val row3 = """{"key":"ghi","top":{"id":null,"extra":3}}""" + withTempDir { dir => + val schema1 = new StructType().add("key", StringType) + .add("top", new StructType().add("id", NullType).add("extra", IntegerType)) + val schema2 = new StructType().add("key", StringType) + .add("top", new StructType().add("id", IntegerType).add("extra", NullType)) + val mergedSchema = new StructType().add("key", StringType) + .add("top", new StructType().add("id", IntegerType).add("extra", IntegerType)) + spark.read.schema(schema1).json(Seq(row1).toDS()).write.append(dir) + spark.read.schema(schema2).json(Seq(row2).toDS()).write.append(dir) + assert(spark.read.format("delta").load(dir.getAbsolutePath).schema === mergedSchema) + spark.read.schema(schema1).json(Seq(row3).toDS()).write.append(dir) + assert(spark.read.format("delta").load(dir.getAbsolutePath).schema === mergedSchema) + + checkAnswer( + spark.read.format("delta").load(dir.getAbsolutePath), + Row("abc", Row(null, 1)) :: Row("def", Row(2, null)) :: Row("ghi", Row(null, 3)) :: Nil + ) + } + } + } + + equivalenceTest("JSON ETL workflow, schema merging NullTypes - throw error on complex types") { + enableAutoMigration { + val row1 = """{"key":"abc","top":[]}""" + val row2 = """{"key":"abc","top":[{"id":null}]}""" + withTempDir { dir => + val schema1 = new StructType().add("key", StringType).add("top", ArrayType(NullType)) + val schema2 = new StructType().add("key", StringType) + .add("top", ArrayType(new StructType().add("id", NullType))) + val e1 = intercept[AnalysisException] { + spark.read.schema(schema1).json(Seq(row1).toDS()).write.append(dir) + } + assert(e1.getMessage.contains("NullType")) + val e2 = intercept[AnalysisException] { + spark.read.schema(schema2).json(Seq(row2).toDS()).write.append(dir) + } + assert(e2.getMessage.contains("NullType")) + } + } + } + + equivalenceTest("JSON ETL workflow, NullType being only data column") { + enableAutoMigration { + val row1 = """{"key":"abc","id":null}""" + withTempDir { dir => + val schema1 = new StructType().add("key", StringType).add("id", NullType) + val e = intercept[AnalysisException] { + spark.read.schema(schema1).json(Seq(row1).toDS()).write.partitionBy("key").append(dir) + } + assert(e.getMessage.contains("NullType have been dropped")) + } + } + } + + equivalenceTest("JSON ETL workflow, NullType partition column should fail") { + enableAutoMigration { + val row1 = """{"key":"abc","id":null}""" + withTempDir { dir => + val schema1 = new StructType().add("key", StringType).add("id", NullType) + intercept[AnalysisException] { + spark.read.schema(schema1).json(Seq(row1).toDS()).write.partitionBy("id").append(dir) + } + intercept[AnalysisException] { + // check case sensitivity with regards to column dropping + spark.read.schema(schema1).json(Seq(row1).toDS()).write.partitionBy("iD").append(dir) + } + } + } + } + + equivalenceTest("reject columns that only differ by case - append") { + withTempDir { dir => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + intercept[AnalysisException] { + spark.range(10).withColumn("ID", 'id + 1).write.append(dir) + } + + intercept[AnalysisException] { + spark.range(10).withColumn("ID", 'id + 1).write + .option(DeltaOptions.MERGE_SCHEMA_OPTION, "true").append(dir) + } + + intercept[AnalysisException] { + spark.range(10).withColumn("a", 'id + 1).write + .partitionBy("a", "A") + .option(DeltaOptions.MERGE_SCHEMA_OPTION, "true").append(dir) + } + } + } + } +} + +trait AppendOutputModeTests extends SchemaEnforcementSuiteBase with SharedSparkSession + with SQLTestUtils { + import testImplicits._ + + testQuietly("reject schema changes by default - streaming") { + withTempDir { dir => + spark.range(10).write.format("delta").save(dir.getAbsolutePath) + + val memStream = MemoryStream[Long] + val stream = memStream.toDS().toDF("value1234") // different column name + .writeStream + .option("checkpointLocation", new File(dir, "_checkpoint").getAbsolutePath) + .format("delta") + .start(dir.getAbsolutePath) + try { + disableAutoMigration { + val e = intercept[StreamingQueryException] { + memStream.addData(1L) + stream.processAllAvailable() + } + assert(e.cause.isInstanceOf[AnalysisException]) + assert(e.cause.getMessage.contains(DeltaOptions.MERGE_SCHEMA_OPTION)) + } + } finally { + stream.stop() + } + } + } + + testQuietly("reject schema changes when autoMigrate enabled but writer config disabled") { + withTempDir { dir => + spark.range(10).write.format("delta").save(dir.getAbsolutePath) + + val memStream = MemoryStream[Long] + val stream = memStream.toDS().toDF("value1234") // different column name + .writeStream + .option("checkpointLocation", new File(dir, "_checkpoint").getAbsolutePath) + .format("delta") + .option(DeltaOptions.MERGE_SCHEMA_OPTION, "false") + .start(dir.getAbsolutePath) + try { + enableAutoMigration { + val e = intercept[StreamingQueryException] { + memStream.addData(1L) + stream.processAllAvailable() + } + assert(e.cause.isInstanceOf[AnalysisException]) + assert(e.cause.getMessage.contains(DeltaOptions.MERGE_SCHEMA_OPTION)) + } + } finally { + stream.stop() + } + } + } + + test("allow schema changes when autoMigrate is enabled - streaming") { + withTempDir { dir => + spark.range(10).write.format("delta").save(dir.getAbsolutePath) + + enableAutoMigration { + val memStream = MemoryStream[Long] + val stream = memStream.toDS().toDF("value1234") // different column name + .writeStream + .option("checkpointLocation", new File(dir, "_checkpoint").getAbsolutePath) + .format("delta") + .start(dir.getAbsolutePath) + try { + memStream.addData(1L) + stream.processAllAvailable() + + assert(spark.read.format("delta").load(dir.getAbsolutePath).schema.length == 2) + } finally { + stream.stop() + } + } + } + } + + test("allow schema change with option - streaming") { + withTempDir { dir => + spark.range(10).write.format("delta").save(dir.getAbsolutePath) + + val memStream = MemoryStream[Long] + val stream = memStream.toDS().toDF("value1234") // different column name + .writeStream + .option("checkpointLocation", new File(dir, "_checkpoint").getAbsolutePath) + .option(DeltaOptions.MERGE_SCHEMA_OPTION, "true") + .format("delta") + .start(dir.getAbsolutePath) + try { + disableAutoMigration { + memStream.addData(1L) + stream.processAllAvailable() + + assert(spark.read.format("delta").load(dir.getAbsolutePath).schema.length == 2) + } + } finally { + stream.stop() + } + } + } + + testQuietly("JSON ETL workflow, reject NullTypes") { + enableAutoMigration { + val row1 = """{"key":"abc","id":null}""" + withTempDir { dir => + val schema = new StructType().add("key", StringType).add("id", NullType) + + val memStream = MemoryStream[String] + val stream = memStream.toDS().select(from_json('value, schema).as("value")) + .select($"value.*") + .writeStream + .option("checkpointLocation", new File(dir, "_checkpoint").getAbsolutePath) + .format("delta") + .start(dir.getAbsolutePath) + + try { + val e = intercept[StreamingQueryException] { + memStream.addData(row1) + stream.processAllAvailable() + } + assert(e.cause.isInstanceOf[AnalysisException]) + assert(e.cause.getMessage.contains("NullType")) + } finally { + stream.stop() + } + } + } + } + + testQuietly("JSON ETL workflow, reject NullTypes on nested column") { + enableAutoMigration { + val row1 = """{"key":"abc","id":{"a":null}}""" + withTempDir { dir => + val schema = new StructType().add("key", StringType) + .add("id", new StructType().add("a", NullType)) + + val memStream = MemoryStream[String] + val stream = memStream.toDS().select(from_json('value, schema).as("value")) + .select($"value.*") + .writeStream + .option("checkpointLocation", new File(dir, "_checkpoint").getAbsolutePath) + .format("delta") + .start(dir.getAbsolutePath) + + try { + val e = intercept[StreamingQueryException] { + memStream.addData(row1) + stream.processAllAvailable() + } + assert(e.cause.isInstanceOf[AnalysisException]) + assert(e.cause.getMessage.contains("NullType")) + } finally { + stream.stop() + } + } + } + } +} + +trait OverwriteSaveModeTests extends BatchWriterTest { + import testImplicits._ + + equivalenceTest("reject schema overwrites by default") { + disableAutoMigration { + withTempDir { dir => + spark.range(10).write.overwrite(dir) + val e = intercept[AnalysisException] { + spark.range(10).withColumn("part", 'id + 1).write.overwrite(dir) + } + assert(e.getMessage.contains(DeltaOptions.OVERWRITE_SCHEMA_OPTION)) + } + } + } + + equivalenceTest("can overwrite schema when using overwrite mode - option") { + disableAutoMigration { + withTempDir { dir => + spark.range(5).toDF("id").write.overwrite(dir) + spark.range(5).toDF("value").write.option(DeltaOptions.OVERWRITE_SCHEMA_OPTION, "true") + .overwrite(dir) + + val df = spark.read.format("delta").load(dir.getAbsolutePath) + assert(df.schema.fieldNames === Array("value")) + } + } + } + + equivalenceTest("when autoMerge sqlConf is enabled, we merge schemas") { + enableAutoMigration { + withTempDir { dir => + spark.range(5).toDF("id").write.overwrite(dir) + spark.range(5).toDF("value").write.overwrite(dir) + + val df = spark.read.format("delta").load(dir.getAbsolutePath) + assert(df.schema.fieldNames === Array("id", "value")) + } + } + } + + equivalenceTest("reject migration when autoMerge sqlConf is enabled and writer config disabled") { + enableAutoMigration { + withTempDir { dir => + spark.range(5).toDF("id").write.overwrite(dir) + intercept[AnalysisException] { + spark.range(5).toDF("value").write.option(DeltaOptions.MERGE_SCHEMA_OPTION, "false") + .overwrite(dir) + } + + val df = spark.read.format("delta").load(dir.getAbsolutePath) + assert(df.schema.fieldNames === Array("id")) + } + } + } + + equivalenceTest("schema merging with replaceWhere - sqlConf") { + enableAutoMigration { + withTempDir { dir => + spark.range(5).toDF("id").withColumn("part", 'id % 2).write + .partitionBy("part") + .overwrite(dir) + Seq((1L, 0L), (2L, 0L)).toDF("value", "part").write + .option(DeltaOptions.REPLACE_WHERE_OPTION, "part = 0") + .overwrite(dir) + + val df = spark.read.format("delta").load(dir.getAbsolutePath) + assert(df.schema.fieldNames === Array("id", "part", "value")) + } + } + } + + equivalenceTest("schema merging with replaceWhere - option") { + disableAutoMigration { + withTempDir { dir => + spark.range(5).toDF("id").withColumn("part", 'id % 2).write + .partitionBy("part") + .overwrite(dir) + Seq((1L, 0L), (2L, 0L)).toDF("value", "part").write + .option(DeltaOptions.REPLACE_WHERE_OPTION, "part = 0") + .option(DeltaOptions.MERGE_SCHEMA_OPTION, "true") + .overwrite(dir) + + val df = spark.read.format("delta").load(dir.getAbsolutePath) + assert(df.schema.fieldNames === Array("id", "part", "value")) + } + } + } + + equivalenceTest("schema merging with replaceWhere - option case insensitive") { + disableAutoMigration { + withTempDir { dir => + spark.range(5).toDF("id").withColumn("part", 'id % 2).write + .partitionBy("part") + .overwrite(dir) + Seq((1L, 0L), (2L, 0L)).toDF("value", "part").write + .option("RePlAcEwHeRe", "part = 0") + .option("mErGeScHeMa", "true") + .overwrite(dir) + + val df = spark.read.format("delta").load(dir.getAbsolutePath) + assert(df.schema.fieldNames === Array("id", "part", "value")) + } + } + } + + equivalenceTest("reject schema merging with replaceWhere - overwrite option") { + disableAutoMigration { + withTempDir { dir => + spark.range(5).toDF("id").withColumn("part", 'id % 2).write + .partitionBy("part") + .overwrite(dir) + val e = intercept[AnalysisException] { + Seq((1L, 0L), (2L, 0L)).toDF("value", "part").write + .option(DeltaOptions.REPLACE_WHERE_OPTION, "part = 0") + .option(DeltaOptions.OVERWRITE_SCHEMA_OPTION, "true") + .overwrite(dir) + } + assert(e.getMessage.contains(DeltaOptions.MERGE_SCHEMA_OPTION)) + } + } + } + + equivalenceTest("reject schema merging with replaceWhere - no option") { + disableAutoMigration { + withTempDir { dir => + spark.range(5).toDF("id").withColumn("part", 'id % 2).write + .partitionBy("part") + .overwrite(dir) + val e = intercept[AnalysisException] { + Seq((1L, 0L), (2L, 0L)).toDF("value", "part").write + .partitionBy("part") + .option(DeltaOptions.REPLACE_WHERE_OPTION, "part = 0") + .overwrite(dir) + } + assert(e.getMessage.contains(DeltaOptions.MERGE_SCHEMA_OPTION)) + } + } + } + + equivalenceTest("reject schema merging with replaceWhere - option set to false, config true") { + enableAutoMigration { + withTempDir { dir => + spark.range(5).toDF("id").withColumn("part", 'id % 2).write + .partitionBy("part") + .overwrite(dir) + val e = intercept[AnalysisException] { + Seq((1L, 0L), (2L, 0L)).toDF("value", "part").write + .partitionBy("part") + .option(DeltaOptions.REPLACE_WHERE_OPTION, "part = 0") + .option(DeltaOptions.MERGE_SCHEMA_OPTION, "false") + .overwrite(dir) + } + assert(e.getMessage.contains(DeltaOptions.MERGE_SCHEMA_OPTION)) + } + } + } + + equivalenceTest("reject change partitioning with overwrite - sqlConf") { + enableAutoMigration { + withTempDir { dir => + spark.range(5).toDF("id").write + .overwrite(dir) + val e = intercept[AnalysisException] { + spark.range(5).toDF("id").withColumn("part", 'id % 2).write + .partitionBy("part") + .overwrite(dir) + } + assert(e.getMessage.contains(DeltaOptions.OVERWRITE_SCHEMA_OPTION)) + + val deltaLog = DeltaLog.forTable(spark, dir) + assert(deltaLog.snapshot.metadata.partitionColumns === Nil) + assert(deltaLog.snapshot.metadata.schema.fieldNames === Array("id")) + } + } + } + + equivalenceTest("can change partitioning with overwrite - option") { + disableAutoMigration { + withTempDir { dir => + spark.range(5).toDF("id").write + .overwrite(dir) + spark.range(5).toDF("id").withColumn("part", 'id % 2).write + .partitionBy("part") + .option(DeltaOptions.OVERWRITE_SCHEMA_OPTION, "true") + .overwrite(dir) + + val deltaLog = DeltaLog.forTable(spark, dir) + assert(deltaLog.snapshot.metadata.partitionColumns === Seq("part")) + assert(deltaLog.snapshot.metadata.schema.fieldNames === Array("id", "part")) + } + } + } + + equivalenceTest("can't change partitioning with overwrite and replaceWhere - option") { + disableAutoMigration { + withTempDir { dir => + spark.range(5).toDF("id").withColumn("part", 'id % 2).write + .partitionBy("part") + .overwrite(dir) + + intercept[AnalysisException] { + spark.range(5).toDF("id").withColumn("part", lit(0L)).withColumn("test", 'id + 1).write + .partitionBy("part", "test") + .option(DeltaOptions.REPLACE_WHERE_OPTION, "part = 0") + .option(DeltaOptions.OVERWRITE_SCHEMA_OPTION, "true") + .overwrite(dir) + } + } + } + } + + equivalenceTest("can drop columns with overwriteSchema") { + disableAutoMigration { + withTempDir { dir => + spark.range(5).toDF("id").withColumn("part", 'id % 2).write + .overwrite(dir) + spark.range(5).toDF("id").write + .option(DeltaOptions.OVERWRITE_SCHEMA_OPTION, "true") + .overwrite(dir) + + val deltaLog = DeltaLog.forTable(spark, dir) + assert(deltaLog.snapshot.metadata.partitionColumns === Nil) + assert(deltaLog.snapshot.metadata.schema.fieldNames === Array("id")) + } + } + } + + equivalenceTest("can change column data type with overwriteSchema") { + disableAutoMigration { + withTempDir { dir => + val deltaLog = DeltaLog.forTable(spark, dir) + spark.range(5).toDF("id").write + .overwrite(dir) + assert(deltaLog.snapshot.metadata.schema.head === StructField("id", LongType)) + spark.range(5).toDF("id").selectExpr("cast(id as string) as id").write + .option(DeltaOptions.OVERWRITE_SCHEMA_OPTION, "true") + .overwrite(dir) + assert(deltaLog.snapshot.metadata.schema.head === StructField("id", StringType)) + } + } + } + + equivalenceTest("reject columns that only differ by case - overwrite") { + withTempDir { dir => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + intercept[AnalysisException] { + spark.range(10).withColumn("ID", 'id + 1).write.overwrite(dir) + } + + intercept[AnalysisException] { + spark.range(10).withColumn("ID", 'id + 1).write + .option(DeltaOptions.OVERWRITE_SCHEMA_OPTION, "true") + .overwrite(dir) + } + + intercept[AnalysisException] { + spark.range(10).withColumn("a", 'id + 1).write + .partitionBy("a", "A") + .option(DeltaOptions.OVERWRITE_SCHEMA_OPTION, "true") + .overwrite(dir) + } + } + } + } +} + +trait CompleteOutputModeTests extends SchemaEnforcementSuiteBase with SharedSparkSession + with SQLTestUtils { + import testImplicits._ + + testQuietly("reject complete mode with new schema by default") { + disableAutoMigration { + withTempDir { dir => + val memStream = MemoryStream[Long] + val query = memStream.toDS().toDF("id") + .withColumn("part", 'id % 3) + .groupBy("part") + .count() + + val stream1 = query.writeStream + .option("checkpointLocation", new File(dir, "_checkpoint").getAbsolutePath) + .outputMode("complete") + .format("delta") + .start(dir.getAbsolutePath) + try { + memStream.addData(1L) + stream1.processAllAvailable() + } finally { + stream1.stop() + } + + assert(spark.read.format("delta").load(dir.getAbsolutePath).schema.length == 2) + + val stream2 = query.withColumn("test", lit("abc")).writeStream + .option("checkpointLocation", new File(dir, "_checkpoint").getAbsolutePath) + .outputMode("complete") + .format("delta") + .start(dir.getAbsolutePath) + try { + val e = intercept[StreamingQueryException] { + memStream.addData(2L) + stream2.processAllAvailable() + } + assert(e.cause.isInstanceOf[AnalysisException]) + assert(e.cause.getMessage.contains(DeltaOptions.OVERWRITE_SCHEMA_OPTION)) + + } finally { + stream2.stop() + } + } + } + } + + test("complete mode can overwrite schema with option") { + disableAutoMigration { + withTempDir { dir => + val memStream = MemoryStream[Long] + val query = memStream.toDS().toDF("id") + .withColumn("part", 'id % 3) + .groupBy("part") + .count() + + val stream1 = query.writeStream + .option("checkpointLocation", new File(dir, "_checkpoint").getAbsolutePath) + .option(DeltaOptions.OVERWRITE_SCHEMA_OPTION, "true") + .outputMode("complete") + .format("delta") + .start(dir.getAbsolutePath) + try { + memStream.addData(1L) + stream1.processAllAvailable() + } finally { + stream1.stop() + } + + assert(spark.read.format("delta").load(dir.getAbsolutePath).schema.length == 2) + + val stream2 = query.withColumn("test", lit("abc")).writeStream + .option("checkpointLocation", new File(dir, "_checkpoint").getAbsolutePath) + .option(DeltaOptions.OVERWRITE_SCHEMA_OPTION, "true") + .outputMode("complete") + .format("delta") + .start(dir.getAbsolutePath) + try { + memStream.addData(2L) + stream2.processAllAvailable() + + memStream.addData(3L) + stream2.processAllAvailable() + } finally { + stream2.stop() + } + + val df = spark.read.format("delta").load(dir.getAbsolutePath) + assert(df.schema.length == 3) + + val deltaLog = DeltaLog.forTable(spark, dir) + val lastCommitFile = deltaLog.store.listFrom(FileNames.deltaFile(deltaLog.logPath, 0L)) + .map(_.getPath).filter(FileNames.isDeltaFile).toArray.last + val lastCommitContainsMetadata = deltaLog.store.read(lastCommitFile) + .exists(JsonUtils.mapper.readValue[SingleAction](_).metaData != null) + + assert(!lastCommitContainsMetadata, + "Metadata shouldn't be updated as long as schema doesn't change") + + checkAnswer( + df, + Row(0L, 1L, "abc") :: Row(1L, 1L, "abc") :: Row(2L, 1L, "abc") :: Nil) + } + } + } + + test("complete mode behavior with autoMigrate enabled is to migrate schema") { + enableAutoMigration { + withTempDir { dir => + val memStream = MemoryStream[Long] + val query = memStream.toDS().toDF("id") + .withColumn("part", 'id % 3) + .groupBy("part") + .count() + + val stream1 = query.writeStream + .option("checkpointLocation", new File(dir, "_checkpoint").getAbsolutePath) + .outputMode("complete") + .format("delta") + .start(dir.getAbsolutePath) + try { + memStream.addData(1L) + stream1.processAllAvailable() + } finally { + stream1.stop() + } + + assert(spark.read.format("delta").load(dir.getAbsolutePath).schema.length == 2) + + val stream2 = query.withColumn("test", lit("abc")).writeStream + .option("checkpointLocation", new File(dir, "_checkpoint").getAbsolutePath) + .outputMode("complete") + .format("delta") + .start(dir.getAbsolutePath) + try { + memStream.addData(2L) + stream2.processAllAvailable() + + memStream.addData(3L) + stream2.processAllAvailable() + } finally { + stream2.stop() + } + + val df = spark.read.format("delta").load(dir.getAbsolutePath) + assert(df.schema.length == 3) + + val deltaLog = DeltaLog.forTable(spark, dir) + val lastCommitFile = deltaLog.store.listFrom(FileNames.deltaFile(deltaLog.logPath, 0L)) + .map(_.getPath).filter(FileNames.isDeltaFile).toArray.last + val lastCommitContainsMetadata = deltaLog.store.read(lastCommitFile) + .exists(JsonUtils.mapper.readValue[SingleAction](_).metaData != null) + + assert(!lastCommitContainsMetadata, + "Metadata shouldn't be updated as long as schema doesn't change") + + checkAnswer( + df, + Row(0L, 1L, "abc") :: Row(1L, 1L, "abc") :: Row(2L, 1L, "abc") :: Nil) + } + } + } +} + +class SchemaEnforcementWithPathSuite extends AppendSaveModeTests with OverwriteSaveModeTests { + override val saveOperation = SaveWithPath() +} + + +class SchemaEnforcementStreamingSuite + extends AppendOutputModeTests + with CompleteOutputModeTests { +} + diff --git a/src/test/scala/org/apache/spark/sql/delta/schema/SchemaUtilsSuite.scala b/src/test/scala/org/apache/spark/sql/delta/schema/SchemaUtilsSuite.scala new file mode 100644 index 00000000000..64b4aa6a101 --- /dev/null +++ b/src/test/scala/org/apache/spark/sql/delta/schema/SchemaUtilsSuite.scala @@ -0,0 +1,1178 @@ +/* + * Copyright 2019 Databricks, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.delta.schema + +import java.util.Locale + + +import org.apache.spark.sql.{AnalysisException, QueryTest, Row} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} +import org.apache.spark.sql.types._ + +class SchemaUtilsSuite extends QueryTest + with SharedSparkSession + with SQLTestUtils { + import SchemaUtils._ + import testImplicits._ + + private def expectFailure(shouldContain: String*)(f: => Unit): Unit = { + val e = intercept[AnalysisException] { + f + } + val msg = e.getMessage.toLowerCase(Locale.ROOT) + assert(shouldContain.map(_.toLowerCase(Locale.ROOT)).forall(msg.contains), + s"Error message '$msg' didn't contain: $shouldContain") + } + + ///////////////////////////// + // Duplicate Column Checks + ///////////////////////////// + + test("duplicate column name in top level") { + val schema = new StructType() + .add("dupColName", IntegerType) + .add("b", IntegerType) + .add("dupColName", StringType) + expectFailure("dupColName") { checkColumnNameDuplication(schema, "") } + } + + test("duplicate column name in top level - case sensitivity") { + val schema = new StructType() + .add("dupColName", IntegerType) + .add("b", IntegerType) + .add("dupCOLNAME", StringType) + expectFailure("dupColName") { checkColumnNameDuplication(schema, "") } + } + + test("duplicate column name for nested column + non-nested column") { + val schema = new StructType() + .add("dupColName", new StructType() + .add("a", IntegerType) + .add("b", IntegerType)) + .add("dupColName", IntegerType) + expectFailure("dupColName") { checkColumnNameDuplication(schema, "") } + } + + test("duplicate column name for nested column + non-nested column - case sensitivity") { + val schema = new StructType() + .add("dupColName", new StructType() + .add("a", IntegerType) + .add("b", IntegerType)) + .add("dupCOLNAME", IntegerType) + expectFailure("dupCOLNAME") { checkColumnNameDuplication(schema, "") } + } + + test("duplicate column name in nested level") { + val schema = new StructType() + .add("top", new StructType() + .add("dupColName", IntegerType) + .add("b", IntegerType) + .add("dupColName", StringType) + ) + expectFailure("top.dupColName") { checkColumnNameDuplication(schema, "") } + } + + test("duplicate column name in nested level - case sensitivity") { + val schema = new StructType() + .add("top", new StructType() + .add("dupColName", IntegerType) + .add("b", IntegerType) + .add("dupCOLNAME", StringType) + ) + expectFailure("top.dupColName") { checkColumnNameDuplication(schema, "") } + } + + test("duplicate column name in double nested level") { + val schema = new StructType() + .add("top", new StructType() + .add("b", new StructType() + .add("dupColName", StringType) + .add("c", IntegerType) + .add("dupColName", StringType)) + .add("d", IntegerType) + ) + expectFailure("top.b.dupColName") { checkColumnNameDuplication(schema, "") } + } + + test("duplicate column name in double nested array") { + val schema = new StructType() + .add("top", new StructType() + .add("b", ArrayType(ArrayType(new StructType() + .add("dupColName", StringType) + .add("c", IntegerType) + .add("dupColName", StringType)))) + .add("d", IntegerType) + ) + expectFailure("top.b.dupColName") { checkColumnNameDuplication(schema, "") } + } + + test("duplicate column name in double nested map") { + val keyType = new StructType() + .add("dupColName", IntegerType) + .add("d", StringType) + expectFailure("top.b.key.dupColName") { + val schema = new StructType() + .add("top", new StructType() + .add("b", MapType(keyType.add("dupColName", StringType), keyType)) + ) + checkColumnNameDuplication(schema, "") + } + expectFailure("top.b.value.dupColName") { + val schema = new StructType() + .add("top", new StructType() + .add("b", MapType(keyType, keyType.add("dupColName", StringType))) + ) + checkColumnNameDuplication(schema, "") + } + // This is okay + val schema = new StructType() + .add("top", new StructType() + .add("b", MapType(keyType, keyType)) + ) + checkColumnNameDuplication(schema, "") + } + + test("duplicate column name in nested array") { + val schema = new StructType() + .add("top", ArrayType(new StructType() + .add("dupColName", IntegerType) + .add("b", IntegerType) + .add("dupColName", StringType)) + ) + expectFailure("top.dupColName") { checkColumnNameDuplication(schema, "") } + } + + test("duplicate column name in nested array - case sensitivity") { + val schema = new StructType() + .add("top", ArrayType(new StructType() + .add("dupColName", IntegerType) + .add("b", IntegerType) + .add("dupCOLNAME", StringType)) + ) + expectFailure("top.dupColName") { checkColumnNameDuplication(schema, "") } + } + + test("non duplicate column because of back tick") { + val schema = new StructType() + .add("top", new StructType() + .add("a", IntegerType) + .add("b", IntegerType)) + .add("top.a", IntegerType) + checkColumnNameDuplication(schema, "") + } + + test("non duplicate column because of back tick - nested") { + val schema = new StructType() + .add("first", new StructType() + .add("top", new StructType() + .add("a", IntegerType) + .add("b", IntegerType)) + .add("top.a", IntegerType)) + checkColumnNameDuplication(schema, "") + } + + test("duplicate column with back ticks - nested") { + val schema = new StructType() + .add("first", new StructType() + .add("top.a", StringType) + .add("b", IntegerType) + .add("top.a", IntegerType)) + expectFailure("first.`top.a`") { checkColumnNameDuplication(schema, "") } + } + + test("duplicate column with back ticks - nested and case sensitivity") { + val schema = new StructType() + .add("first", new StructType() + .add("TOP.a", StringType) + .add("b", IntegerType) + .add("top.a", IntegerType)) + expectFailure("first.`top.a`") { checkColumnNameDuplication(schema, "") } + } + + ///////////////////////////// + // Read Compatibility Checks + ///////////////////////////// + + /** + * Tests change of datatype within a schema. + * - the make() function is a "factory" function to create schemas that vary only by the + * given datatype in a specific position in the schema. + * - other tests will call this method with different make() functions to test datatype + * incompatibility in all the different places within a schema (in a top-level struct, + * in a nested struct, as the element type of an array, etc.) + */ + def testDatatypeChange(scenario: String)(make: DataType => StructType): Unit = { + val schemas = Map( + ("int", make(IntegerType)), + ("string", make(StringType)), + ("struct", make(new StructType().add("a", StringType))), + ("array", make(ArrayType(IntegerType))), + ("map", make(MapType(StringType, FloatType))) + ) + test(s"change of datatype should fail read compatibility - $scenario") { + for (a <- schemas.keys; b <- schemas.keys if a != b) { + assert(!isReadCompatible(schemas(a), schemas(b)), + s"isReadCompatible should have failed for: ${schemas(a)}, ${schemas(b)}") + } + } + } + + /** + * Tests change of nullability within a schema (making a field nullable is not allowed, + * but making a nullable field non-nullable is ok). + * - the make() function is a "factory" function to create schemas that vary only by the + * nullability (of a field, array elemnt, or map values) in a specific position in the schema. + * - other tests will call this method with different make() functions to test nullability + * incompatibility in all the different places within a schema (in a top-level struct, + * in a nested struct, for the element type of an array, etc.) + */ + def testNullability (scenario: String)(make: Boolean => StructType): Unit = { + val nullable = make(true) + val nonNullable = make(false) + test(s"relaxed nullability should fail read compatibility - $scenario") { + assert(!isReadCompatible(nonNullable, nullable)) + } + test(s"restricted nullability should not fail read compatibility - $scenario") { + assert(isReadCompatible(nullable, nonNullable)) + } + } + + /** + * Tests for fields of a struct: adding/dropping fields, changing nullability, case variation + * - The make() function is a "factory" method to produce schemas. It takes a function that + * mutates a struct (for example, but adding a column, or it could just not make any change). + * - Following tests will call this method with different factory methods, to mutate the + * various places where a struct can appear (at the top-level, nested in another struct, + * within an array, etc.) + * - This allows us to have one shared code to test compatibility of a struct field in all the + * different places where it may occur. + */ + def testColumnVariations(scenario: String) + (make: (StructType => StructType) => StructType): Unit = { + + // generate one schema without extra column, one with, one nullable, and one with mixed case + val withoutExtra = make(struct => struct) // produce struct WITHOUT extra field + val withExtraNullable = make(struct => struct.add("extra", StringType)) + val withExtraMixedCase = make(struct => struct.add("eXtRa", StringType)) + val withExtraNonNullable = make(struct => struct.add("extra", StringType, nullable = false)) + + test(s"dropping a field should fail read compatibility - $scenario") { + assert(!isReadCompatible(withExtraNullable, withoutExtra)) + } + test(s"adding a nullable field should not fail read compatibility - $scenario") { + assert(isReadCompatible(withoutExtra, withExtraNullable)) + } + test(s"adding a non-nullable field should not fail read compatibility - $scenario") { + assert(isReadCompatible(withoutExtra, withExtraNonNullable)) + } + test(s"case variation of field name should fail read compatibility - $scenario") { + assert(!isReadCompatible(withExtraNullable, withExtraMixedCase)) + } + testNullability(scenario)(b => make(struct => struct.add("extra", StringType, nullable = b))) + testDatatypeChange(scenario)(datatype => make(struct => struct.add("extra", datatype))) + } + + // -------------------------------------------------------------------- + // tests for all kinds of places where a field can appear in a struct + // -------------------------------------------------------------------- + + testColumnVariations("top level")( + f => f(new StructType().add("a", IntegerType))) + + testColumnVariations("nested struct")( + f => new StructType() + .add("a", f(new StructType().add("b", IntegerType)))) + + testColumnVariations("nested in array")( + f => new StructType() + .add("array", ArrayType( + f(new StructType().add("b", IntegerType))))) + + testColumnVariations("nested in map key")( + f => new StructType() + .add("map", MapType( + f(new StructType().add("b", IntegerType)), + StringType))) + + testColumnVariations("nested in map value")( + f => new StructType() + .add("map", MapType( + StringType, + f(new StructType().add("b", IntegerType))))) + + // -------------------------------------------------------------------- + // tests for data type change in places other than struct + // -------------------------------------------------------------------- + + testDatatypeChange("array element")( + datatype => new StructType() + .add("array", ArrayType(datatype))) + + testDatatypeChange("map key")( + datatype => new StructType() + .add("map", MapType(datatype, StringType))) + + testDatatypeChange("map value")( + datatype => new StructType() + .add("map", MapType(StringType, datatype))) + + // -------------------------------------------------------------------- + // tests for nullability change in places other than struct + // -------------------------------------------------------------------- + + testNullability("array contains null")( + b => new StructType() + .add("array", ArrayType(StringType, containsNull = b))) + + testNullability("map contains null values")( + b => new StructType() + .add("map", MapType(IntegerType, StringType, valueContainsNull = b))) + + testNullability("map nested in array")( + b => new StructType() + .add("map", ArrayType( + MapType(IntegerType, StringType, valueContainsNull = b)))) + + testNullability("array nested in map")( + b => new StructType() + .add("map", MapType( + IntegerType, + ArrayType(StringType, containsNull = b)))) + + //////////////////////////// + // reportDifference + //////////////////////////// + + /** + * @param existing the existing schema to compare to + * @param specified the new specified schema + * @param expected an expected list of messages, each describing a schema difference. + * Every expected message is actually a regex patterns that is matched + * against all diffs that are returned. This is necessary to tolerate + * variance in ordering of field names, for example in a message such as + * "Specified schema has additional field(s): x, y", we cannot predict + * the order of x and y. + */ + def testReportDifferences(testName: String) + (existing: StructType, specified: StructType, expected: String*): Unit = { + test(testName) { + val differences = SchemaUtils.reportDifferences(existing, specified) + // make sure every expected difference is reported + expected foreach ((exp: String) => + assert(differences.exists(message => exp.r.findFirstMatchIn(message).isDefined), + s"""Difference not reported. + |Expected: + |- $exp + |Reported: ${differences.mkString("\n- ", "\n- ", "")} + """.stripMargin)) + // make sure there are no extra differences reported + assert(expected.size == differences.size, + s"""Too many differences reported. + |Expected: ${expected.mkString("\n- ", "\n- ", "")} + |Reported: ${differences.mkString("\n- ", "\n- ", "")} + """.stripMargin) + } + } + + testReportDifferences("extra columns should be reported as a difference")( + existing = new StructType() + .add("a", IntegerType), + specified = new StructType() + .add("a", IntegerType) + .add("b", StringType), + expected = "additional field[(]s[)]: b" + ) + + testReportDifferences("missing columns should be reported as a difference")( + existing = new StructType() + .add("a", IntegerType) + .add("b", StringType), + specified = new StructType() + .add("a", IntegerType), + expected = "missing field[(]s[)]: b" + ) + + testReportDifferences("making a column nullable should be reported as a difference")( + existing = new StructType() + .add("a", IntegerType, nullable = false) + .add("b", StringType, nullable = true), + specified = new StructType() + .add("a", IntegerType, nullable = true) + .add("b", StringType, nullable = true), + expected = "a is nullable in specified schema but non-nullable in existing schema" + ) + + testReportDifferences("making a column non-nullable should be reported as a difference")( + existing = new StructType() + .add("a", IntegerType, nullable = false) + .add("b", StringType, nullable = true), + specified = new StructType() + .add("a", IntegerType, nullable = false) + .add("b", StringType, nullable = false), + expected = "b is non-nullable in specified schema but nullable in existing schema" + ) + + testReportDifferences("change in column metadata should be reported as a difference")( + existing = new StructType() + .add("a", IntegerType, nullable = true, new MetadataBuilder().putString("x", "1").build()) + .add("b", StringType), + specified = new StructType() + .add("a", IntegerType, nullable = true, new MetadataBuilder().putString("x", "2").build()) + .add("b", StringType), + expected = "metadata for field a is different" + ) + + testReportDifferences("change of column type should be reported as a difference")( + existing = new StructType() + .add("a", IntegerType) + .add("b", StringType), + specified = new StructType() + .add("a", IntegerType) + .add("b", new ArrayType( + StringType, containsNull = false)), + expected = "type for b is different" + ) + + testReportDifferences("change of array nullability should be reported as a difference")( + existing = new StructType() + .add("a", IntegerType) + .add("b", new ArrayType( + new StructType().add("x", LongType), containsNull = true)), + specified = new StructType() + .add("a", IntegerType) + .add("b", new ArrayType( + new StructType().add("x", LongType), containsNull = false)), + expected = "b\\[\\] can not contain null in specified schema but can in existing" + ) + + testReportDifferences("change of element type should be reported as a difference")( + existing = new StructType() + .add("a", IntegerType) + .add("b", new ArrayType(LongType, containsNull = true)), + specified = new StructType() + .add("a", IntegerType) + .add("b", new ArrayType(StringType, containsNull = true)), + expected = "type for b\\[\\] is different" + ) + + testReportDifferences("change of element struct type should be reported as a difference")( + existing = new StructType() + .add("a", IntegerType) + .add("b", new ArrayType( + new StructType() + .add("x", LongType), + containsNull = true)), + specified = new StructType() + .add("a", IntegerType) + .add("b", new ArrayType( + new StructType() + .add("x", StringType), + containsNull = true)), + expected = "type for b\\[\\].x is different" + ) + + testReportDifferences("change of map value nullability should be reported as a difference")( + existing = new StructType() + .add("a", IntegerType) + .add("b", new MapType( + StringType, + new StructType().add("x", LongType), valueContainsNull = true)), + specified = new StructType() + .add("a", IntegerType) + .add("b", new MapType( + StringType, + new StructType().add("x", LongType), valueContainsNull = false)), + expected = "b can not contain null values in specified schema but can in existing" + ) + + testReportDifferences("change of map key type should be reported as a difference")( + existing = new StructType() + .add("a", IntegerType) + .add("b", new MapType(LongType, StringType, valueContainsNull = true)), + specified = new StructType() + .add("a", IntegerType) + .add("b", new MapType(StringType, StringType, valueContainsNull = true)), + expected = "type for b\\[key\\] is different" + ) + + testReportDifferences("change of value struct type should be reported as a difference")( + existing = new StructType() + .add("a", IntegerType) + .add("b", new MapType( + StringType, + new StructType().add("x", LongType), + valueContainsNull = true)), + specified = new StructType() + .add("a", IntegerType) + .add("b", new MapType( + StringType, + new StructType().add("x", FloatType), + valueContainsNull = true)), + expected = "type for b\\[value\\].x is different" + ) + + testReportDifferences("nested extra columns should be reported as a difference")( + existing = new StructType() + .add("x", new StructType() + .add("a", IntegerType)), + specified = new StructType() + .add("x", new StructType() + .add("a", IntegerType) + .add("b", StringType) + .add("c", LongType)), + expected = "additional field[(]s[)]: (x.b, x.c|x.c, x.b)" + ) + + testReportDifferences("nested missing columns should be reported as a difference")( + existing = new StructType() + .add("x", new StructType() + .add("a", IntegerType) + .add("b", StringType) + .add("c", FloatType)), + specified = new StructType() + .add("x", new StructType() + .add("a", IntegerType)), + expected = "missing field[(]s[)]: (x.b, x.c|x.c, x.b)" + ) + + testReportDifferences("making a nested column nullable should be reported as a difference")( + existing = new StructType() + .add("x", new StructType() + .add("a", IntegerType, nullable = false) + .add("b", StringType, nullable = true)), + specified = new StructType() + .add("x", new StructType() + .add("a", IntegerType, nullable = true) + .add("b", StringType, nullable = true)), + expected = "x.a is nullable in specified schema but non-nullable in existing schema" + ) + + testReportDifferences("making a nested column non-nullable should be reported as a difference")( + existing = new StructType() + .add("x", new StructType() + .add("a", IntegerType, nullable = false) + .add("b", StringType, nullable = true)), + specified = new StructType() + .add("x", new StructType() + .add("a", IntegerType, nullable = false) + .add("b", StringType, nullable = false)), + expected = "x.b is non-nullable in specified schema but nullable in existing schema" + ) + + testReportDifferences("change in nested column metadata should be reported as a difference")( + existing = new StructType() + .add("x", new StructType() + .add("a", IntegerType, nullable = true, new MetadataBuilder().putString("x", "1").build()) + .add("b", StringType)), + specified = new StructType() + .add("x", new StructType() + .add("a", IntegerType, nullable = true, new MetadataBuilder().putString("x", "2").build()) + .add("b", StringType)), + expected = "metadata for field x.a is different" + ) + + testReportDifferences("change of nested column type should be reported as a difference")( + existing = new StructType() + .add("x", new StructType() + .add("a", IntegerType) + .add("b", StringType)), + specified = new StructType() + .add("x", new StructType() + .add("a", IntegerType) + .add("b", new ArrayType( + StringType, containsNull = false))), + expected = "type for x.b is different" + ) + + testReportDifferences("change of nested array nullability should be reported as a difference")( + existing = new StructType() + .add("x", new StructType() + .add("a", IntegerType) + .add("b", new ArrayType( + new StructType() + .add("x", LongType), + containsNull = true))), + specified = new StructType() + .add("x", new StructType() + .add("a", IntegerType) + .add("b", new ArrayType( + new StructType() + .add("x", LongType), + containsNull = false))), + expected = "x.b\\[\\] can not contain null in specified schema but can in existing" + ) + + testReportDifferences("change of nested element type should be reported as a difference")( + existing = new StructType() + .add("x", new StructType() + .add("a", IntegerType) + .add("b", new ArrayType(LongType, containsNull = true))), + specified = new StructType() + .add("x", new StructType() + .add("a", IntegerType) + .add("b", new ArrayType(StringType, containsNull = true))), + expected = "type for x.b\\[\\] is different" + ) + + testReportDifferences("change of nested element struct type should be reported as a difference")( + existing = new StructType() + .add("x", new StructType() + .add("a", IntegerType) + .add("b", new ArrayType( + new StructType() + .add("x", LongType), + containsNull = true))), + specified = new StructType() + .add("x", new StructType() + .add("a", IntegerType) + .add("b", new ArrayType( + new StructType() + .add("x", StringType), + containsNull = true))), + expected = "type for x.b\\[\\].x is different" + ) + + private val piiTrue = new MetadataBuilder().putBoolean("pii", value = true).build() + private val piiFalse = new MetadataBuilder().putBoolean("pii", value = false).build() + + testReportDifferences("multiple differences should be reported")( + existing = new StructType() + .add("a", IntegerType) + .add("b", StringType) + .add("c", BinaryType) + .add("f", LongType, nullable = true, piiTrue) + .add("g", new MapType( + IntegerType, + new StructType() + .add("a", IntegerType, nullable = false, piiFalse) + .add("b", StringType) + .add("d", new ArrayType( + LongType, + containsNull = false + )), + valueContainsNull = true)) + .add("h", new MapType( + LongType, + StringType, + valueContainsNull = true)), + specified = new StructType() + .add("a", FloatType) + .add("d", StringType) + .add("e", LongType) + .add("f", LongType, nullable = false, piiFalse) + .add("g", new MapType( + StringType, + new StructType() + .add("a", LongType, nullable = true) + .add("c", StringType) + .add("d", new ArrayType( + BooleanType, + containsNull = true + )), + valueContainsNull = false)) + .add("h", new MapType( + LongType, + new ArrayType(IntegerType, containsNull = false), + valueContainsNull = true)), + "type for a is different", + "additional field[(]s[)]: (d, e|e, d)", + "missing field[(]s[)]: (b, c|c, b)", + "f is non-nullable in specified schema but nullable", + "metadata for field f is different", + "type for g\\[key\\] is different", + "g can not contain null values in specified schema but can in existing", + "additional field[(]s[)]: g\\[value\\].c", + "missing field[(]s[)]: g\\[value\\].b", + "type for g\\[value\\].a is different", + "g\\[value\\].a is nullable in specified schema but non-nullable in existing", + "metadata for field g\\[value\\].a is different", + "field g\\[value\\].d\\[\\] can contain null in specified schema but can not in existing", + "type for g\\[value\\].d\\[\\] is different", + "type for h\\[value\\] is different" + ) + + //////////////////////////// + // findColumnPosition + //////////////////////////// + + test("findColumnPosition") { + val schema = new StructType() + .add("a", new StructType() + .add("b", IntegerType) + .add("c", IntegerType)) + .add("d", ArrayType(new StructType() + .add("b", IntegerType) + .add("c", IntegerType))) + .add("e", StringType) + assert(SchemaUtils.findColumnPosition(Seq("a"), schema) === ((Seq(0), 2))) + assert(SchemaUtils.findColumnPosition(Seq("A"), schema) === ((Seq(0), 2))) + expectFailure("Couldn't find", schema.treeString) { + SchemaUtils.findColumnPosition(Seq("a", "d"), schema) + } + assert(SchemaUtils.findColumnPosition(Seq("a", "b"), schema) === ((Seq(0, 0), 0))) + assert(SchemaUtils.findColumnPosition(Seq("A", "b"), schema) === ((Seq(0, 0), 0))) + assert(SchemaUtils.findColumnPosition(Seq("a", "B"), schema) === ((Seq(0, 0), 0))) + assert(SchemaUtils.findColumnPosition(Seq("A", "B"), schema) === ((Seq(0, 0), 0))) + assert(SchemaUtils.findColumnPosition(Seq("a", "c"), schema) === ((Seq(0, 1), 0))) + assert(SchemaUtils.findColumnPosition(Seq("d"), schema) === ((Seq(1), 2))) + assert(SchemaUtils.findColumnPosition(Seq("d", "b"), schema) === ((Seq(1, 0), 0))) + assert(SchemaUtils.findColumnPosition(Seq("d", "B"), schema) === ((Seq(1, 0), 0))) + assert(SchemaUtils.findColumnPosition(Seq("d", "c"), schema) === ((Seq(1, 1), 0))) + assert(SchemaUtils.findColumnPosition(Seq("e"), schema) === ((Seq(2), 0))) + + val resolver = org.apache.spark.sql.catalyst.analysis.caseSensitiveResolution + Seq(Seq("A", "b"), Seq("a", "B"), Seq("d", "B")).foreach { column => + expectFailure("Couldn't find", schema.treeString) { + SchemaUtils.findColumnPosition(column, schema, resolver) + } + } + } + + test("findColumnPosition that doesn't exist") { + val schema = new StructType() + .add("a", IntegerType) + .add("b", MapType(StringType, StringType)) + expectFailure("Couldn't find", schema.treeString) { + SchemaUtils.findColumnPosition(Seq("c"), schema) + } + expectFailure("b.c", "mapType", schema.treeString) { + SchemaUtils.findColumnPosition(Seq("b", "c"), schema) + } + } + + //////////////////////////// + // addColumn + //////////////////////////// + + test("addColumn - simple") { + val a = StructField("a", IntegerType) + val b = StructField("b", StringType) + val schema = new StructType().add(a).add(b) + + val x = StructField("x", LongType) + assert(SchemaUtils.addColumn(schema, x, Seq(0)) === new StructType().add(x).add(a).add(b)) + assert(SchemaUtils.addColumn(schema, x, Seq(1)) === new StructType().add(a).add(x).add(b)) + assert(SchemaUtils.addColumn(schema, x, Seq(2)) === new StructType().add(a).add(b).add(x)) + + expectFailure("Index -1", "lower than 0") { + SchemaUtils.addColumn(schema, x, Seq(-1)) + } + expectFailure("Index 3", "larger than struct length: 2") { + SchemaUtils.addColumn(schema, x, Seq(3)) + } + expectFailure("Can only add nested columns to StructType") { + SchemaUtils.addColumn(schema, x, Seq(0, 0)) + } + } + + test("addColumn - nested struct") { + val a = StructField("a", IntegerType) + val b = StructField("b", StringType) + val s = StructField("s", new StructType().add(a).add(b)) + val schema = new StructType().add(s) + + val x = StructField("x", LongType) + assert(SchemaUtils.addColumn(schema, x, Seq(0)) === new StructType().add(x).add(s)) + assert(SchemaUtils.addColumn(schema, x, Seq(0, 0)) === + new StructType().add("s", new StructType().add(x).add(a).add(b))) + assert(SchemaUtils.addColumn(schema, x, Seq(0, 2)) === + new StructType().add("s", new StructType().add(a).add(b).add(x))) + assert(SchemaUtils.addColumn(schema, x, Seq(1)) === new StructType().add(s).add(x)) + + expectFailure("Index -1", "lower than 0") { + SchemaUtils.addColumn(schema, x, Seq(0, -1)) + } + expectFailure("Index 3", "larger than struct length: 2") { + SchemaUtils.addColumn(schema, x, Seq(0, 3)) + } + expectFailure("Struct not found at position 2") { + SchemaUtils.addColumn(schema, x, Seq(0, 2, 0)) + } + expectFailure("Can only add nested columns to StructType") { + SchemaUtils.addColumn(schema, x, Seq(0, 0, 0)) + } + } + + //////////////////////////// + // dropColumn + //////////////////////////// + + test("dropColumn - simple") { + val a = StructField("a", IntegerType) + val b = StructField("b", StringType) + val schema = new StructType().add(a).add(b) + + assert(SchemaUtils.dropColumn(schema, Seq(0)) === ((new StructType().add(b), a))) + assert(SchemaUtils.dropColumn(schema, Seq(1)) === ((new StructType().add(a), b))) + + expectFailure("Index -1", "lower than 0") { + SchemaUtils.dropColumn(schema, Seq(-1)) + } + expectFailure("Index 2", "equals to or is larger than struct length: 2") { + SchemaUtils.dropColumn(schema, Seq(2)) + } + expectFailure("Can only drop nested columns from StructType") { + SchemaUtils.dropColumn(schema, Seq(0, 0)) + } + } + + test("dropColumn - nested struct") { + val a = StructField("a", IntegerType) + val b = StructField("b", StringType) + val s = StructField("s", new StructType().add(a).add(b)) + val schema = new StructType().add(s) + + assert(SchemaUtils.dropColumn(schema, Seq(0)) === ((new StructType(), s))) + assert(SchemaUtils.dropColumn(schema, Seq(0, 0)) === + ((new StructType().add("s", new StructType().add(b)), a))) + assert(SchemaUtils.dropColumn(schema, Seq(0, 1)) === + ((new StructType().add("s", new StructType().add(a)), b))) + + expectFailure("Index -1", "lower than 0") { + SchemaUtils.dropColumn(schema, Seq(0, -1)) + } + expectFailure("Index 2", "equals to or is larger than struct length: 2") { + SchemaUtils.dropColumn(schema, Seq(0, 2)) + } + expectFailure("Can only drop nested columns from StructType") { + SchemaUtils.dropColumn(schema, Seq(0, 0, 0)) + } + } + + //////////////////////////// + // normalizeColumnNames + //////////////////////////// + + test("normalize column names") { + val df = Seq((1, 2, 3)).toDF("Abc", "def", "gHi") + val schema = new StructType() + .add("abc", IntegerType) + .add("Def", IntegerType) + .add("ghi", IntegerType) + assert(normalizeColumnNames(schema, df).schema.fieldNames === schema.fieldNames) + } + + test("normalize column names - different ordering") { + val df = Seq((1, 2, 3)).toDF("def", "gHi", "abC") + val schema = new StructType() + .add("abc", IntegerType) + .add("Def", IntegerType) + .add("ghi", IntegerType) + assert(normalizeColumnNames(schema, df).schema.fieldNames === Seq("Def", "ghi", "abc")) + } + + test("throw error if nested column cases don't match") { + val df = spark.read.json(Seq("""{"a":1,"b":{"X":1,"y":2}}""").toDS()) + val schema = new StructType() + .add("a", IntegerType) + .add("b", new StructType() + .add("x", IntegerType) + .add("y", IntegerType)) + expectFailure("[b.X]", "b.x") { + normalizeColumnNames(schema, df) + } + } + + test("can rename top level nested column") { + val df = spark.read.json(Seq("""{"a":1,"B":{"x":1,"y":2}}""").toDS()).select('a, 'b) + val schema = new StructType() + .add("a", IntegerType) + .add("b", new StructType() + .add("x", IntegerType) + .add("y", IntegerType)) + assert(normalizeColumnNames(schema, df).schema.fieldNames === Seq("a", "b")) + } + + //////////////////////////// + // mergeSchemas + //////////////////////////// + + test("mergeSchemas: missing columns in df") { + val base = new StructType().add("a", IntegerType).add("b", IntegerType) + val write = new StructType().add("a", IntegerType) + assert(mergeSchemas(base, write) === base) + } + + test("mergeSchemas: missing columns in df - case sensitivity") { + val base = new StructType().add("a", IntegerType).add("b", IntegerType) + val write = new StructType().add("A", IntegerType) + assert(mergeSchemas(base, write) === base) + } + + test("new columns get added to the tail of the schema") { + val base = new StructType().add("a", IntegerType) + val write = new StructType().add("a", IntegerType).add("b", IntegerType) + val write2 = new StructType().add("b", IntegerType).add("a", IntegerType) + assert(mergeSchemas(base, write) === write) + assert(mergeSchemas(base, write2) === write) + } + + test("new columns get added to the tail of the schema - nested") { + val base = new StructType() + .add("regular", StringType) + .add("struct", new StructType() + .add("a", IntegerType)) + + val write = new StructType() + .add("other", StringType) + .add("struct", new StructType() + .add("b", DateType) + .add("a", IntegerType)) + .add("this", StringType) + + val expected = new StructType() + .add("regular", StringType) + .add("struct", new StructType() + .add("a", IntegerType) + .add("b", DateType)) + .add("other", StringType) + .add("this", StringType) + assert(mergeSchemas(base, write) === expected) + } + + test("schema merging of incompatible types") { + val base = new StructType() + .add("top", StringType) + .add("struct", new StructType() + .add("a", IntegerType)) + .add("array", ArrayType(new StructType() + .add("b", DecimalType(18, 10)))) + .add("map", MapType(StringType, StringType)) + + expectFailure("StringType", "IntegerType") { + mergeSchemas(base, new StructType().add("top", IntegerType)) + } + expectFailure("IntegerType", "DateType") { + mergeSchemas(base, new StructType() + .add("struct", new StructType().add("a", DateType))) + } + expectFailure("'struct'", "structType", "MapType") { + mergeSchemas(base, new StructType() + .add("struct", MapType(StringType, IntegerType))) + } + expectFailure("'array'", "DecimalType", "DoubleType") { + mergeSchemas(base, new StructType() + .add("array", ArrayType(new StructType().add("b", DoubleType)))) + } + expectFailure("'array'", "scale") { + mergeSchemas(base, new StructType() + .add("array", ArrayType(new StructType().add("b", DecimalType(18, 12))))) + } + expectFailure("'array'", "precision") { + mergeSchemas(base, new StructType() + .add("array", ArrayType(new StructType().add("b", DecimalType(16, 10))))) + } + expectFailure("'map'", "MapType", "StructType") { + mergeSchemas(base, new StructType() + .add("map", new StructType().add("b", StringType))) + } + expectFailure("'map'", "StringType", "IntegerType") { + mergeSchemas(base, new StructType() + .add("map", MapType(StringType, IntegerType))) + } + expectFailure("'map'", "StringType", "IntegerType") { + mergeSchemas(base, new StructType() + .add("map", MapType(IntegerType, StringType))) + } + } + + test("schema merging should pick current nullable and metadata") { + val m = new MetadataBuilder().putDouble("a", 0.2).build() + val base = new StructType() + .add("top", StringType, nullable = false, m) + .add("struct", new StructType() + .add("a", IntegerType, nullable = false, m)) + .add("array", ArrayType(new StructType() + .add("b", DecimalType(18, 10))), nullable = false, m) + .add("map", MapType(StringType, StringType), nullable = false, m) + + assert(mergeSchemas(base, new StructType().add("top", StringType)) === base) + assert(mergeSchemas(base, new StructType().add("struct", new StructType() + .add("a", IntegerType))) === base) + assert(mergeSchemas(base, new StructType().add("array", ArrayType(new StructType() + .add("b", DecimalType(18, 10))))) === base) + assert(mergeSchemas(base, new StructType() + .add("map", MapType(StringType, StringType))) === base) + } + + test("schema merging null type") { + val base = new StructType().add("top", NullType) + val update = new StructType().add("top", StringType) + + assert(mergeSchemas(base, update) === update) + assert(mergeSchemas(update, base) === update) + } + + test("schema merging performs upcast between ByteType, ShortType, and LongType") { + val byteType = new StructType().add("top", ByteType) + val shortType = new StructType().add("top", ShortType) + val intType = new StructType().add("top", IntegerType) + + assert(mergeSchemas(byteType, shortType) === shortType) + assert(mergeSchemas(byteType, intType) === intType) + assert(mergeSchemas(shortType, intType) === intType) + assert(mergeSchemas(shortType, byteType) === shortType) + assert(mergeSchemas(intType, shortType) === intType) + assert(mergeSchemas(intType, byteType) === intType) + + val structInt = new StructType().add("top", new StructType().add("leaf", IntegerType)) + val structShort = new StructType().add("top", new StructType().add("leaf", ShortType)) + assert(mergeSchemas(structInt, structShort) === structInt) + + val map1 = new StructType().add("top", new MapType(IntegerType, ShortType, true)) + val map2 = new StructType().add("top", new MapType(ShortType, IntegerType, true)) + val mapMerged = new StructType().add("top", new MapType(IntegerType, IntegerType, true)) + assert(mergeSchemas(map1, map2) === mapMerged) + + val arrInt = new StructType().add("top", new ArrayType(IntegerType, true)) + val arrShort = new StructType().add("top", new ArrayType(ShortType, true)) + assert(mergeSchemas(arrInt, arrShort) === arrInt) + } + + test("Upcast between ByteType, ShortType and IntegerType is OK for parquet") { + import org.apache.spark.sql.functions._ + def testParquetUpcast(): Unit = { + withTempDir { dir => + val tempDir = dir.getCanonicalPath + spark.range(1.toByte).select(col("id") cast ByteType).write.save(tempDir + "/byte") + spark.range(1.toShort).select(col("id") cast ShortType).write.save(tempDir + "/short") + spark.range(1).select(col("id") cast IntegerType).write.save(tempDir + "/int") + + val shortSchema = new StructType().add("id", ShortType) + val intSchema = new StructType().add("id", IntegerType) + + spark.read.schema(shortSchema).parquet(tempDir + "/byte").collect() === Seq(Row(1.toShort)) + spark.read.schema(intSchema).parquet(tempDir + "/short").collect() === Seq(Row(1)) + spark.read.schema(intSchema).parquet(tempDir + "/byte").collect() === Seq(Row(1)) + } + } + + testParquetUpcast() + + } + //////////////////////////// + // transformColumns + //////////////////////////// + + test("transform columns - simple") { + val base = new StructType() + .add("a", IntegerType) + .add("b", StringType) + val update = new StructType() + .add("c", IntegerType) + .add("b", StringType) + + // Identity. + var visitedFields = 0 + val res1 = transformColumns(base) { + case (Seq(), field, _) => + visitedFields += 1 + field + } + assert(visitedFields === 2) + assert(base === res1) + + // Rename a -> c + visitedFields = 0 + val res2 = transformColumns(base) { + case (Seq(), field, _) => + visitedFields += 1 + val name = field.name + field.copy(name = if (name == "a") "c" else name) + } + assert(visitedFields === 2) + assert(update === res2) + + // Rename a -> c; using input map. + visitedFields = 0 + val res3 = transformColumns(base, (Seq("A"), "c") :: Nil) { + case (Seq(), field, Seq((_, newName))) => + visitedFields += 1 + field.copy(name = newName) + } + assert(visitedFields === 1) + assert(update === res3) + } + + test("transform columns - nested") { + val nested = new StructType() + .add("s1", IntegerType) + .add("s2", LongType) + val base = new StructType() + .add("nested", nested) + .add("arr", ArrayType(nested)) + .add("kvs", MapType(nested, nested)) + val update = new StructType() + .add("nested", + new StructType() + .add("t1", IntegerType) + .add("s2", LongType)) + .add("arr", ArrayType( + new StructType() + .add("s1", IntegerType) + .add("a2", LongType))) + .add("kvs", MapType( + new StructType() + .add("k1", IntegerType) + .add("s2", LongType), + new StructType() + .add("s1", IntegerType) + .add("v2", LongType))) + + // Identity. + var visitedFields = 0 + val res1 = transformColumns(base) { + case (_, field, _) => + visitedFields += 1 + field + } + assert(visitedFields === 11) + assert(base === res1) + + // Rename + visitedFields = 0 + val res2 = transformColumns(base) { (path, field, _) => + visitedFields += 1 + val name = path :+ field.name match { + case Seq("nested", "s1") => "t1" + case Seq("arr", "s2") => "a2" + case Seq("kvs", "key", "s1") => "k1" + case Seq("kvs", "value", "s2") => "v2" + case _ => field.name + } + field.copy(name = name) + } + assert(visitedFields === 11) + assert(update === res2) + + // Rename; using map + visitedFields = 0 + val mapping = Seq( + Seq("nested", "s1") -> "t1", + Seq("arr", "s2") -> "a2", + Seq("kvs", "key", "S1") -> "k1", + Seq("kvs", "value", "s2") -> "v2") + val res3 = transformColumns(base, mapping) { + case (_, field, Seq((_, name))) => + visitedFields += 1 + field.copy(name = name) + } + assert(visitedFields === 4) + assert(update === res3) + } +} diff --git a/version.sbt b/version.sbt index 5bc7193ffae..999958d380c 100644 --- a/version.sbt +++ b/version.sbt @@ -1 +1 @@ -version in ThisBuild := "0.4.0.carmel0.1-SNAPSHOT" +version in ThisBuild := "0.5.0.carmel0.1-SNAPSHOT" \ No newline at end of file