From ad7d5f86fe0506b5d8b7d5cf2834ba4c4a63f4d4 Mon Sep 17 00:00:00 2001 From: vkorukanti Date: Mon, 8 Jan 2024 13:49:28 -0800 Subject: [PATCH] Squashed commits: - 09e2ec4a95be0c66204b80f36ed2f3f0e8f76831 scalastyle import by vkorukanti - babe0920ef1b507a94289b92fef3aec12570fd2c import ordering to match with the OSS one by vkorukanti - 659d6d3abd1ab9f2402d2d14059eb5f71ce32d87 Make the OptimizeMetadataOnlyDeltaQueryDeletionVectorSuit... by vkorukanti - 997b335960fea03d4a3ffb7accc5117da89b0c12 disable tests that don't run in DBR by vkorukanti - 1bca4b2d7792334dda3d636103109cf2bf9044b0 fix build by vkorukanti - 6280c6d5e65983d515501b6e25b72881c39db50c fix build by vkorukanti - 967f093594d745f8b37a5f4b8088e1bfdabae186 conflicts by vkorukanti - 87e81252a21bcf3039181f9df8d8d3d3e8908a1c [DELTA-OSS-EXTERNAL] Optimize Min/Max using Delta metadata by Felipe Pessoto GitOrigin-RevId: 09e2ec4a95be0c66204b80f36ed2f3f0e8f76831 --- PROTOCOL.md | 111 +- .../icebergShaded/IcebergSchemaUtils.scala | 42 +- .../icebergShaded/IcebergStatsConverter.scala | 211 ---- .../IcebergTransactionUtils.scala | 23 - python/delta/tables.py | 5 +- ...pache.spark.sql.sources.DataSourceRegister | 1 - .../DeltaFormatSharingLimitPushDown.scala | 53 - .../spark/DeltaFormatSharingSource.scala | 532 ---------- .../sharing/spark/DeltaSharingCDFUtils.scala | 112 -- .../spark/DeltaSharingDataSource.scala | 448 -------- .../spark/DeltaSharingLogFileSystem.scala | 360 ------- .../sharing/spark/DeltaSharingUtils.scala | 152 --- .../spark/DeltaFormatSharingSourceSuite.scala | 867 --------------- .../spark/DeltaSharingCDFUtilsSuite.scala | 243 ----- .../spark/DeltaSharingDataSourceCMSuite.scala | 984 ------------------ .../DeltaSharingDataSourceDeltaSuite.scala | 851 --------------- ...DeltaSharingDataSourceDeltaTestUtils.scala | 661 ------------ .../spark/DeltaSharingFileIndexSuite.scala | 17 +- .../TestClientForDeltaFormatSharing.scala | 270 ----- .../spark/TestDeltaSharingFileSystem.scala | 140 --- .../io/delta/sql/parser/DeltaSqlBase.g4 | 11 +- .../resources/error/delta-error-classes.json | 70 +- .../io/delta/sql/parser/DeltaSqlParser.scala | 16 +- .../scala/io/delta/tables/DeltaTable.scala | 14 +- .../spark/sql/delta/ConflictChecker.scala | 6 +- .../spark/sql/delta/DeltaAnalysis.scala | 6 +- .../spark/sql/delta/DeltaColumnMapping.scala | 138 +-- .../apache/spark/sql/delta/DeltaConfig.scala | 21 - .../apache/spark/sql/delta/DeltaErrors.scala | 113 -- .../org/apache/spark/sql/delta/DeltaLog.scala | 26 +- .../spark/sql/delta/DeltaOperations.scala | 38 +- .../sql/delta/DeltaParquetFileFormat.scala | 8 - .../sql/delta/DeltaParquetWriteSupport.scala | 149 --- .../sql/delta/DeltaTableIdentifier.scala | 4 +- ...bergCompat.scala => IcebergCompatV1.scala} | 15 - .../sql/delta/OptimisticTransaction.scala | 105 +- .../spark/sql/delta/SnapshotManagement.scala | 76 +- .../spark/sql/delta/actions/actions.scala | 3 - .../sql/delta/catalog/DeltaCatalog.scala | 68 +- .../clustering/ClusteringMetadataDomain.scala | 36 - .../commands/CreateDeltaTableCommand.scala | 84 +- .../commands/DeltaReorgTableCommand.scala | 45 +- .../delta/commands/MergeIntoCommandBase.scala | 5 +- .../delta/commands/OptimizeTableCommand.scala | 86 +- .../ReorgTableForUpgradeUniformHelper.scala | 246 ----- .../sql/delta/commands/WriteIntoDelta.scala | 30 +- .../delta/commands/WriteIntoDeltaLike.scala | 6 +- .../commands/alterDeltaTableCommands.scala | 7 - .../commands/merge/ClassicMergeExecutor.scala | 17 +- .../sql/delta/commands/merge/MergeStats.scala | 8 - .../sql/delta/files/TransactionalWrite.scala | 15 +- .../spark/sql/delta/hooks/AutoCompact.scala | 254 ----- .../sql/delta/hooks/AutoCompactUtils.scala | 399 ------- .../delta/hooks/IcebergConverterHook.scala | 5 +- .../spark/sql/delta/hooks/UpdateCatalog.scala | 370 ------- .../schema/ImplicitMetadataOperation.scala | 9 +- .../clustering/ClusteredTableUtils.scala | 298 +----- .../clustering/ClusteringColumn.scala | 95 -- .../clustering/temp/ClusterBySpec.scala | 5 - .../sql/delta/sources/DeltaSQLConf.scala | 166 +-- .../stats/AutoCompactPartitionStats.scala | 375 ------- .../apache/spark/sql/delta/util/Utils.scala | 12 - .../spark/sql/delta/AutoCompactSuite.scala | 332 ------ .../spark/sql/delta/DeltaErrorsSuite.scala | 8 +- .../spark/sql/delta/DeltaOptionSuite.scala | 95 -- .../sql/delta/DeltaUpdateCatalogSuite.scala | 544 ---------- .../delta/DeltaUpdateCatalogSuiteBase.scala | 313 ------ .../spark/sql/delta/MergeIntoDVsSuite.scala | 119 +-- .../sql/delta/MergeIntoMetricsBase.scala | 11 +- .../spark/sql/delta/MergeIntoSuiteBase.scala | 10 +- .../ClusteredTableClusteringSuite.scala | 82 -- .../ClusteringMetadataDomainSuite.scala | 33 - .../DeletionVectorsSuite.scala | 25 +- .../delta/optimize/CompactionTestHelper.scala | 109 -- .../skipping/ClusteredTableTestUtils.scala | 148 --- .../clustering/ClusteredTableDDLSuite.scala | 612 ----------- .../sql/delta/test/DeltaTestImplicits.scala | 14 - 77 files changed, 194 insertions(+), 11804 deletions(-) delete mode 100644 iceberg/src/main/scala/org/apache/spark/sql/delta/icebergShaded/IcebergStatsConverter.scala delete mode 100644 sharing/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister delete mode 100644 sharing/src/main/scala/io/delta/sharing/spark/DeltaFormatSharingLimitPushDown.scala delete mode 100644 sharing/src/main/scala/io/delta/sharing/spark/DeltaFormatSharingSource.scala delete mode 100644 sharing/src/main/scala/io/delta/sharing/spark/DeltaSharingCDFUtils.scala delete mode 100644 sharing/src/main/scala/io/delta/sharing/spark/DeltaSharingDataSource.scala delete mode 100644 sharing/src/test/scala/io/delta/sharing/spark/DeltaFormatSharingSourceSuite.scala delete mode 100644 sharing/src/test/scala/io/delta/sharing/spark/DeltaSharingCDFUtilsSuite.scala delete mode 100644 sharing/src/test/scala/io/delta/sharing/spark/DeltaSharingDataSourceCMSuite.scala delete mode 100644 sharing/src/test/scala/io/delta/sharing/spark/DeltaSharingDataSourceDeltaSuite.scala delete mode 100644 sharing/src/test/scala/io/delta/sharing/spark/DeltaSharingDataSourceDeltaTestUtils.scala delete mode 100644 sharing/src/test/scala/io/delta/sharing/spark/TestClientForDeltaFormatSharing.scala delete mode 100644 sharing/src/test/scala/io/delta/sharing/spark/TestDeltaSharingFileSystem.scala delete mode 100644 spark/src/main/scala/org/apache/spark/sql/delta/DeltaParquetWriteSupport.scala rename spark/src/main/scala/org/apache/spark/sql/delta/{IcebergCompat.scala => IcebergCompatV1.scala} (96%) delete mode 100644 spark/src/main/scala/org/apache/spark/sql/delta/clustering/ClusteringMetadataDomain.scala delete mode 100644 spark/src/main/scala/org/apache/spark/sql/delta/commands/ReorgTableForUpgradeUniformHelper.scala delete mode 100644 spark/src/main/scala/org/apache/spark/sql/delta/hooks/AutoCompact.scala delete mode 100644 spark/src/main/scala/org/apache/spark/sql/delta/hooks/AutoCompactUtils.scala delete mode 100644 spark/src/main/scala/org/apache/spark/sql/delta/hooks/UpdateCatalog.scala delete mode 100644 spark/src/main/scala/org/apache/spark/sql/delta/skipping/clustering/ClusteringColumn.scala delete mode 100644 spark/src/main/scala/org/apache/spark/sql/delta/stats/AutoCompactPartitionStats.scala delete mode 100644 spark/src/test/scala/org/apache/spark/sql/delta/AutoCompactSuite.scala delete mode 100644 spark/src/test/scala/org/apache/spark/sql/delta/DeltaUpdateCatalogSuite.scala delete mode 100644 spark/src/test/scala/org/apache/spark/sql/delta/DeltaUpdateCatalogSuiteBase.scala delete mode 100644 spark/src/test/scala/org/apache/spark/sql/delta/clustering/ClusteredTableClusteringSuite.scala delete mode 100644 spark/src/test/scala/org/apache/spark/sql/delta/clustering/ClusteringMetadataDomainSuite.scala delete mode 100644 spark/src/test/scala/org/apache/spark/sql/delta/optimize/CompactionTestHelper.scala delete mode 100644 spark/src/test/scala/org/apache/spark/sql/delta/skipping/ClusteredTableTestUtils.scala delete mode 100644 spark/src/test/scala/org/apache/spark/sql/delta/skipping/clustering/ClusteredTableDDLSuite.scala diff --git a/PROTOCOL.md b/PROTOCOL.md index d1d8f1c677f..68c750f7e63 100644 --- a/PROTOCOL.md +++ b/PROTOCOL.md @@ -46,8 +46,6 @@ - [Writer Requirement for Deletion Vectors](#writer-requirement-for-deletion-vectors) - [Iceberg Compatibility V1](#iceberg-compatibility-v1) - [Writer Requirements for IcebergCompatV1](#writer-requirements-for-icebergcompatv1) -- [Iceberg Compatibility V2](#iceberg-compatibility-v2) - - [Writer Requirement for IcebergCompatV2](#iceberg-compatibility-v2) - [Timestamp without timezone (TimestampNtz)](#timestamp-without-timezone-timestampntz) - [V2 Checkpoint Table Feature](#v2-checkpoint-table-feature) - [Row Tracking](#row-tracking) @@ -104,6 +102,8 @@ +THIS IS AN IN-PROGRESS DRAFT + # Overview This document is a specification for the Delta Transaction Protocol, which brings [ACID](https://en.wikipedia.org/wiki/ACID) properties to large collections of data, stored as files, in a distributed file system or object store. The protocol was designed with the following goals in mind: @@ -905,7 +905,9 @@ To support this feature: - The table must be on Writer Version 7. - The feature `icebergCompatV1` must exist in the table `protocol`'s `writerFeatures`. -This table feature is enabled when the table property `delta.enableIcebergCompatV1` is set to `true`. +Activation: Set table property `delta.enableIcebergCompatV1` to `true`. + +Deactivation: Unset table property `delta.enableIcebergCompatV1`, or set it to `false`. ## Writer Requirements for IcebergCompatV1 @@ -919,109 +921,6 @@ When supported and active, writers must: - e.g. replacing a table partitioned by `part_a INT` with partition spec `part_b INT` must be blocked - e.g. replacing a table partitioned by `part_a INT` with partition spec `part_a LONG` is allowed -# Iceberg Compatibility V2 - -This table feature (`icebergCompatV2`) ensures that Delta tables can be converted to Apache Iceberg™ format, though this table feature does not implement or specify that conversion. - -To support this feature: -- Since this table feature depends on Column Mapping, the table must be on Reader Version = 2, or it must be on Reader Version >= 3 and the feature `columnMapping` must exist in the `protocol`'s `readerFeatures`. -- The table must be on Writer Version 7. -- The feature `icebergCompatV2` must exist in the table protocol's `writerFeatures`. - -This table feature is enabled when the table property `delta.enableIcebergCompatV2` is set to `true`. - -## Writer Requirements for IcebergCompatV2 - -When this feature is supported and enabled, writers must: -- Require that Column Mapping be enabled and set to either `name` or `id` mode -- Require that the nested `element` field of ArrayTypes and the nested `key` and `value` fields of MapTypes be assigned 32 bit integer identifiers. These identifiers must be unique and different from those used in [Column Mapping](#column-mapping), and must be stored in the metadata of their nearest ancestor [StructField](#struct-field) of the Delta table schema. Identifiers belonging to the same `StructField` must be organized as a `Map[String, Long]` and stored in metadata with key `parquet.field.nested.ids`. The keys of the map are "element", "key", or "value", prefixed by the name of the nearest ancestor StructField, separated by dots. The values are the identifiers. The keys for fields in nested arrays or nested maps are prefixed by their parents' key, separated by dots. An [example](#example-of-storing-identifiers-for-nested-fields-in-arraytype-and-maptype) is provided below to demonstrate how the identifiers are stored. These identifiers must be also written to the `field_id` field of the `SchemaElement` struct in the [Parquet Thrift specification](https://github.com/apache/parquet-format/blob/master/src/main/thrift/parquet.thrift) when writing parquet files. -- Require that IcebergCompatV1 is not active, which means either the `icebergCompatV1` table feature is not present in the table protocol or the table property `delta.enableIcebergCompatV1` is not set to `true` -- Require that Deletion Vectors are not active, which means either the `deletionVectors` table feature is not present in the table protocol or the table property `delta.enableDeletionVectors` is not set to `true` -- Require that partition column values be materialized when writing Parquet data files -- Require that all new `AddFile`s committed to the table have the `numRecords` statistic populated in their `stats` field -- Require writing timestamp columns as int64 -- Require that the table schema contains only data types in the following allow-list: [`byte`, `short`, `integer`, `long`, `float`, `double`, `decimal`, `string`, `binary`, `boolean`, `timestamp`, `timestampNTZ`, `date`, `array`, `map`, `struct`]. -- Block replacing partitioned tables with a differently-named partition spec - - e.g. replacing a table partitioned by `part_a INT` with partition spec `part_b INT` must be blocked - - e.g. replacing a table partitioned by `part_a INT` with partition spec `part_a LONG` is allowed - -### Example of storing identifiers for nested fields in ArrayType and MapType -The following is an example of storing the identifiers for nested fields in `ArrayType` and `MapType`, of a table with the following schema, -``` -|-- col1: array[array[int]] -|-- col2: map[int, array[int]] -|-- col3: map[int, struct] - |-- subcol1: array[int] -``` -The identifiers for the nested fields are stored in the metadata as follows: -```json -[ - { - "name": "col1", - "type": { - "type": "array", - "elementType": { - "type": "array", - "elementType": "int" - } - }, - "metadata": { - "parquet.field.nested.ids": { - "col1.element": 100, - "col1.element.element": 101 - } - } - }, - { - "name": "col2", - "type": { - "type": "map", - "keyType": "int", - "valueType": { - "type": "array", - "elementType": "int" - } - }, - "metadata": { - "parquet.field.nested.ids": { - "col2.key": 102, - "col2.value": 103, - "col2.value.element": 104 - } - } - }, - { - "name": "col3", - "type": { - "type": "map", - "keyType": "int", - "valueType": { - "type": "struct", - "fields": [ - { - "name": "subcol1", - "type": { - "type": "array", - "elementType": "int" - }, - "metadata": { - "parquet.field.nested.ids": { - "subcol1.element": 107 - } - } - } - ] - } - }, - "metadata": { - "parquet.field.nested.ids": { - "col3.key": 105, - "col3.value": 106 - } - } - } -] -``` # Timestamp without timezone (TimestampNtz) This feature introduces a new data type to support timestamps without timezone information. For example: `1970-01-01 00:00:00`, or `1970-01-01 00:00:00.123456`. The serialization method is described in Sections [Partition Value Serialization](#partition-value-serialization) and [Schema Serialization Format](#schema-serialization-format). diff --git a/iceberg/src/main/scala/org/apache/spark/sql/delta/icebergShaded/IcebergSchemaUtils.scala b/iceberg/src/main/scala/org/apache/spark/sql/delta/icebergShaded/IcebergSchemaUtils.scala index 4e61231c64d..7e004cb5c1d 100644 --- a/iceberg/src/main/scala/org/apache/spark/sql/delta/icebergShaded/IcebergSchemaUtils.scala +++ b/iceberg/src/main/scala/org/apache/spark/sql/delta/icebergShaded/IcebergSchemaUtils.scala @@ -43,13 +43,6 @@ object IcebergSchemaUtils extends DeltaLogging { new IcebergSchema(icebergStruct.fields()) } - private[delta] def getNestedFieldId(field: Option[StructField], path: Seq[String]): Int = { - field.get.metadata - .getMetadata(DeltaColumnMapping.COLUMN_MAPPING_METADATA_NESTED_IDS_KEY) - .getLong(path.mkString(".")) - .toInt - } - //////////////////// // Helper Methods // //////////////////// @@ -65,8 +58,7 @@ object IcebergSchemaUtils extends DeltaLogging { * - MapType -> IcebergTypes.MapType * - primitive -> IcebergType.PrimitiveType */ - def transform[E <: DataType](elem: E, field: Option[StructField], name: Seq[String]) - : IcebergType = elem match { + def transform[E <: DataType](elem: E): IcebergType = elem match { case StructType(fields) => IcebergTypes.StructType.of(fields.map { f => if (!DeltaColumnMapping.hasColumnId(f)) { @@ -77,40 +69,16 @@ object IcebergSchemaUtils extends DeltaLogging { DeltaColumnMapping.getColumnId(f), f.nullable, f.name, - transform(f.dataType, Some(f), Seq(DeltaColumnMapping.getPhysicalName(f))), + transform(f.dataType), f.getComment().orNull ) }.toList.asJava) case ArrayType(elementType, containsNull) => - val currName = name :+ DeltaColumnMapping.PARQUET_LIST_ELEMENT_FIELD_NAME - val id = getNestedFieldId(field, currName) - if (containsNull) { - IcebergTypes.ListType.ofOptional(id, transform(elementType, field, currName)) - } else { - IcebergTypes.ListType.ofRequired(id, transform(elementType, field, currName)) - } + throw new UnsupportedOperationException("UniForm doesn't support Array columns") case MapType(keyType, valueType, valueContainsNull) => - val currKeyName = name :+ DeltaColumnMapping.PARQUET_MAP_KEY_FIELD_NAME - val currValName = name :+ DeltaColumnMapping.PARQUET_MAP_VALUE_FIELD_NAME - val keyId = getNestedFieldId(field, currKeyName) - val valId = getNestedFieldId(field, currValName) - if (valueContainsNull) { - IcebergTypes.MapType.ofOptional( - keyId, - valId, - transform(keyType, field, currKeyName), - transform(valueType, field, currValName) - ) - } else { - IcebergTypes.MapType.ofRequired( - keyId, - valId, - transform(keyType, field, currKeyName), - transform(valueType, field, currValName) - ) - } + throw new UnsupportedOperationException("UniForm doesn't support Map columns") case atomicType: AtomicType => convertAtomic(atomicType) @@ -118,7 +86,7 @@ object IcebergSchemaUtils extends DeltaLogging { throw new UnsupportedOperationException(s"Cannot convert Delta type $other to Iceberg") } - transform(deltaSchema, None, Seq.empty).asStructType() + transform(deltaSchema).asStructType() } /** diff --git a/iceberg/src/main/scala/org/apache/spark/sql/delta/icebergShaded/IcebergStatsConverter.scala b/iceberg/src/main/scala/org/apache/spark/sql/delta/icebergShaded/IcebergStatsConverter.scala deleted file mode 100644 index a43b5039c85..00000000000 --- a/iceberg/src/main/scala/org/apache/spark/sql/delta/icebergShaded/IcebergStatsConverter.scala +++ /dev/null @@ -1,211 +0,0 @@ -/* - * Copyright (2021) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.delta.icebergShaded - -import java.lang.{Long => JLong} -import java.nio.ByteBuffer - -import org.apache.spark.sql.delta.DeltaColumnMapping -import org.apache.spark.sql.delta.stats.{DeltaStatistics, SkippingEligibleDataType} -import shadedForDelta.org.apache.iceberg.types.Conversions - -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.UTF8String - -/** - * Converts Delta stats to Iceberg stats given an Internal Row representing Delta stats and the - * row's schema. - * - * Iceberg stores stats as a map from column ID to the statistic. For example, lower/upper bound - * statistics are represented as a map from column ID to byte buffer where the byte buffer stores - * any type. - * - * For example, given the following Delta stats schema with column IDs: - * | -- id(0): INT - * | -- person(1): STRUCT - * | name(2): STRUCT - * | -- first(3): STRING - * | -- last(4): STRING - * | height(5): LONG - * - * Iceberg's upper bound statistic map will be: - * {0 -> MAX_ID, 3 -> MAX_FIRST, 4 -> MAX_LAST, 5 -> MAX_HEIGHT} - * - * Iceberg requires the "record count" stat while the "upper bounds", "lower bounds", and - * "null value counts" are optional. See iceberg/DataFile.java. - * Iceberg's "record count" metric is set in `convertFileAction` before the stats conversion. - * If additional metrics are attached to the Iceberg data file, the "record count" metric must be - * left non-null. - */ -case class IcebergStatsConverter(statsRow: InternalRow, statsSchema: StructType) { - - val numRecordsStat: JLong = statsSchema.getFieldIndex(DeltaStatistics.NUM_RECORDS) match { - case Some(fieldIndex) => new JLong(statsRow.getLong(fieldIndex)) - case None => throw new IllegalArgumentException("Delta is missing the 'num records' stat. " + - "Iceberg requires this stat when attaching statistics to the output data file.") - } - - val lowerBoundsStat: Option[Map[Integer, ByteBuffer]] = - getByteBufferBackedColStats(DeltaStatistics.MIN) - - val upperBoundsStat: Option[Map[Integer, ByteBuffer]] = - getByteBufferBackedColStats(DeltaStatistics.MAX) - - val nullValueCountsStat: Option[Map[Integer, JLong]] = - statsSchema.getFieldIndex(DeltaStatistics.NULL_COUNT) match { - case Some(nullCountFieldIdx) => - val nullCountStatSchema = - statsSchema.fields(nullCountFieldIdx).dataType.asInstanceOf[StructType] - Some( - generateIcebergLongMetricMap( - statsRow.getStruct(nullCountFieldIdx, nullCountStatSchema.fields.length), - nullCountStatSchema - ) - ) - case None => None - } - - /** - * Generates Iceberg's metric representation by recursively flattening the Delta stat struct - * (represented as an internal row) and converts the column's physical name to its ID. - * - * Ignores null Delta stats. - * - * @param stats An internal row holding the `ByteBuffer`-based Delta column stats - * (i.e. lower bound). - * @param statsSchema The schema of the `stats` internal row. - * @return Iceberg's ByteBuffer-backed metric representation. - */ - private def generateIcebergByteBufferMetricMap( - stats: InternalRow, - statsSchema: StructType): Map[Integer, ByteBuffer] = { - statsSchema.fields.zipWithIndex.flatMap { case (field, idx) => - field.dataType match { - // Iceberg statistics cannot be null. - case _ if stats.isNullAt(idx) => Map[Integer, ByteBuffer]().empty - // If the stats schema contains a struct type, there is a corresponding struct in the data - // schema. The struct's per-field stats are also stored in the Delta stats struct. See the - // `StatisticsCollection` trait comment for more. - case st: StructType => - generateIcebergByteBufferMetricMap(stats.getStruct(idx, st.fields.length), st) - // Ignore the Delta statistic if the conversion doesn't support the given data type or the - // column ID for this field is missing. - case dt if !DeltaColumnMapping.hasColumnId(field) || - !IcebergStatsConverter.isMinMaxStatTypeSupported(dt) => Map[Integer, ByteBuffer]().empty - case b: ByteType => - // Iceberg stores bytes using integers. - val statVal = stats.getByte(idx).toInt - Map[Integer, ByteBuffer](Integer.valueOf(DeltaColumnMapping.getColumnId(field)) -> - Conversions.toByteBuffer(IcebergSchemaUtils.convertAtomic(b), statVal)) - case s: ShortType => - // Iceberg stores shorts using integers. - val statVal = stats.getShort(idx).toInt - Map[Integer, ByteBuffer](Integer.valueOf(DeltaColumnMapping.getColumnId(field)) -> - Conversions.toByteBuffer(IcebergSchemaUtils.convertAtomic(s), statVal)) - case dt if IcebergStatsConverter.isMinMaxStatTypeSupported(dt) => - val statVal = stats.get(idx, dt) - - // Iceberg's `Conversions.toByteBuffer` method expects the Java object representation - // for string and decimal types. - // Other types supported by Delta's min/max stat such as int, long, boolean, etc., do not - // require a different representation. - val compatibleStatsVal = statVal match { - case u: UTF8String => u.toString - case d: Decimal => d.toJavaBigDecimal - case _ => statVal - } - Map[Integer, ByteBuffer](Integer.valueOf(DeltaColumnMapping.getColumnId(field)) -> - Conversions.toByteBuffer(IcebergSchemaUtils.convertAtomic(dt), compatibleStatsVal)) - } - }.toMap - } - - /** - * Generates Iceberg's metric representation by recursively flattening the Delta stat struct - * (represented as an internal row) and converts the column's physical name to its ID. - * - * @param stats An internal row holding the long-backed Delta column stats (i.e. null counts). - * @param statsSchema The schema of the `stats` internal row. - * @return a map in Iceberg's metric representation. - */ - private def generateIcebergLongMetricMap( - stats: InternalRow, - statsSchema: StructType): Map[Integer, JLong] = { - statsSchema.fields.zipWithIndex.flatMap { case (field, idx) => - field.dataType match { - // If the stats schema contains a struct type, there is a corresponding struct in the data - // schema. The struct's per-field stats are also stored in the Delta stats struct. See the - // `StatisticsCollection` trait comment for more. - case st: StructType => - generateIcebergLongMetricMap(stats.getStruct(idx, st.fields.length), st) - case lt: LongType => - if (DeltaColumnMapping.hasColumnId(field)) { - Map[Integer, JLong](Integer.valueOf(DeltaColumnMapping.getColumnId(field)) -> - new JLong(stats.getLong(idx))) - } else { - Map[Integer, JLong]().empty - } - case _ => throw new UnsupportedOperationException("Expected metric to be a long type.") - } - }.toMap - } - - /** - * @param statName The name of the Delta stat that is being converted. Must be one of the field - * names in the `DeltaStatistics` object. - * @return An option holding Iceberg's statistic representation. Returns `None` if the output - * would otherwise be empty. - */ - private def getByteBufferBackedColStats(statName: String): Option[Map[Integer, ByteBuffer]] = { - statsSchema.getFieldIndex(statName) match { - case Some(statFieldIdx) => - val colStatSchema = statsSchema.fields(statFieldIdx).dataType.asInstanceOf[StructType] - val icebergMetricsMap = generateIcebergByteBufferMetricMap( - statsRow.getStruct(statFieldIdx, colStatSchema.fields.length), - colStatSchema - ) - if (icebergMetricsMap.nonEmpty) { - Some(icebergMetricsMap) - } else { - // The iceberg metrics map may be empty when all Delta stats are null. - None - } - case None => None - } - } -} - -object IcebergStatsConverter { - /** - * Returns true if a min/max statistic of the given Delta data type can be converted into an - * Iceberg metric of equivalent data type. - * - * Currently, nested types and null types are unsupported. - */ - def isMinMaxStatTypeSupported(dt: DataType): Boolean = { - if (!SkippingEligibleDataType(dt)) return false - - dt match { - case _: StringType | _: IntegerType | _: FloatType | _: DoubleType | - _: DoubleType | _: DecimalType | _: BooleanType | _: DateType | _: TimestampType | - // _: LongType TODO: enable after https://github.com/apache/spark/pull/42083 is released - _: TimestampNTZType | _: ByteType | _: ShortType => true - case _ => false - } - } -} diff --git a/iceberg/src/main/scala/org/apache/spark/sql/delta/icebergShaded/IcebergTransactionUtils.scala b/iceberg/src/main/scala/org/apache/spark/sql/delta/icebergShaded/IcebergTransactionUtils.scala index 9d77cccb269..bb19347e6b4 100644 --- a/iceberg/src/main/scala/org/apache/spark/sql/delta/icebergShaded/IcebergTransactionUtils.scala +++ b/iceberg/src/main/scala/org/apache/spark/sql/delta/icebergShaded/IcebergTransactionUtils.scala @@ -25,7 +25,6 @@ import org.apache.spark.sql.delta.metering.DeltaLogging import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import shadedForDelta.org.apache.iceberg.{DataFile, DataFiles, FileFormat, PartitionSpec, Schema => IcebergSchema} -import shadedForDelta.org.apache.iceberg.Metrics // scalastyle:off import.ordering.noEmptyLine import shadedForDelta.org.apache.iceberg.catalog.{Namespace, TableIdentifier => IcebergTableIdentifier} // scalastyle:on import.ordering.noEmptyLine @@ -76,28 +75,6 @@ object IcebergTransactionUtils // throw an exception when building the data file. .withRecordCount(add.numLogicalRecords.getOrElse(-1L)) - if (add.stats != null && add.stats.nonEmpty) { - try { - val statsRow = statsParser(add.stats) - - val metricsConverter = IcebergStatsConverter(statsRow, statsSchema) - val metrics = new Metrics( - metricsConverter.numRecordsStat, // rowCount - null, // columnSizes - null, // valueCounts - metricsConverter.nullValueCountsStat.getOrElse(null).asJava, // nullValueCounts - null, // nanValueCounts - metricsConverter.lowerBoundsStat.getOrElse(null).asJava, // lowerBounds - metricsConverter.upperBoundsStat.getOrElse(null).asJava // upperBounds - ) - - dataFileBuilder = dataFileBuilder.withMetrics(metrics) - } catch { - case NonFatal(e) => - logWarning("Failed to convert Delta stats to Iceberg stats. Iceberg conversion will " + - "attempt to proceed without stats.", e) - } - } dataFileBuilder.build() } diff --git a/python/delta/tables.py b/python/delta/tables.py index 6ee6fa83c2e..f3b4412f33d 100644 --- a/python/delta/tables.py +++ b/python/delta/tables.py @@ -393,10 +393,9 @@ def forName( cls, sparkSession: SparkSession, tableOrViewName: str ) -> "DeltaTable": """ - Instantiate a :class:`DeltaTable` object using the given table name. If the given + Instantiate a :class:`DeltaTable` object using the given table or view name. If the given tableOrViewName is invalid (i.e. either no table exists or an existing table is not a - Delta table), it throws a `not a Delta table` error. Note: Passing a view name will - also result in this error as views are not supported. + Delta table), it throws a `not a Delta table` error. The given tableOrViewName can also be the absolute path of a delta datasource (i.e. delta.`path`), If so, instantiate a :class:`DeltaTable` object representing the data at diff --git a/sharing/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/sharing/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister deleted file mode 100644 index c81a708f931..00000000000 --- a/sharing/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister +++ /dev/null @@ -1 +0,0 @@ -io.delta.sharing.spark.DeltaSharingDataSource \ No newline at end of file diff --git a/sharing/src/main/scala/io/delta/sharing/spark/DeltaFormatSharingLimitPushDown.scala b/sharing/src/main/scala/io/delta/sharing/spark/DeltaFormatSharingLimitPushDown.scala deleted file mode 100644 index 589df97057a..00000000000 --- a/sharing/src/main/scala/io/delta/sharing/spark/DeltaFormatSharingLimitPushDown.scala +++ /dev/null @@ -1,53 +0,0 @@ -/* - * Copyright (2021) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.delta.sharing.spark - -import io.delta.sharing.client.util.ConfUtils - -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.expressions.IntegerLiteral -import org.apache.spark.sql.catalyst.plans.logical.{LocalLimit, LogicalPlan} -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} - -// A spark rule that applies limit pushdown to DeltaSharingFileIndex, when the config is enabled. -// To allow only fetching needed files from delta sharing server. -object DeltaFormatSharingLimitPushDown extends Rule[LogicalPlan] { - - def setup(spark: SparkSession): Unit = synchronized { - if (!spark.experimental.extraOptimizations.contains(DeltaFormatSharingLimitPushDown)) { - spark.experimental.extraOptimizations ++= Seq(DeltaFormatSharingLimitPushDown) - } - } - - def apply(p: LogicalPlan): LogicalPlan = { - p transform { - case localLimit @ LocalLimit( - literalExpr @ IntegerLiteral(limit), - l @ LogicalRelation( - r @ HadoopFsRelation(remoteIndex: DeltaSharingFileIndex, _, _, _, _, _), - _, - _, - _ - ) - ) if (ConfUtils.limitPushdownEnabled(p.conf) && remoteIndex.limitHint.isEmpty) => - val spark = SparkSession.active - val newRel = r.copy(location = remoteIndex.copy(limitHint = Some(limit)))(spark) - LocalLimit(literalExpr, l.copy(relation = newRel)) - } - } -} diff --git a/sharing/src/main/scala/io/delta/sharing/spark/DeltaFormatSharingSource.scala b/sharing/src/main/scala/io/delta/sharing/spark/DeltaFormatSharingSource.scala deleted file mode 100644 index c631cb8bf62..00000000000 --- a/sharing/src/main/scala/io/delta/sharing/spark/DeltaFormatSharingSource.scala +++ /dev/null @@ -1,532 +0,0 @@ -/* - * Copyright (2021) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.delta.sharing.spark - -import java.lang.ref.WeakReference -import java.util.concurrent.TimeUnit - -import org.apache.spark.sql.delta.{ - DeltaErrors, - DeltaLog, - DeltaOptions, - SnapshotDescriptor -} -import org.apache.spark.sql.delta.actions.{Metadata, Protocol} -import org.apache.spark.sql.delta.commands.cdc.CDCReader -import org.apache.spark.sql.delta.metering.DeltaLogging -import org.apache.spark.sql.delta.sources.{ - DeltaDataSource, - DeltaSource, - DeltaSourceOffset -} -import io.delta.sharing.client.DeltaSharingClient -import io.delta.sharing.client.model.{Table => DeltaSharingTable} - -import org.apache.spark.delta.sharing.CachedTableManager -import org.apache.spark.sql.{DataFrame, SparkSession} -import org.apache.spark.sql.connector.read.streaming -import org.apache.spark.sql.connector.read.streaming.{ReadLimit, SupportsAdmissionControl} -import org.apache.spark.sql.execution.streaming.{Offset, Source} -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.StructType - -/** - * A streaming source for a Delta Sharing table. - * - * This class wraps a DeltaSource to read data out of locally constructed delta log. - * When a new stream is started, delta sharing starts by fetching delta log from the server side, - * constructing a local delta log, and call delta source apis to compute offset or read data. - * - * TODO: Support CDC Streaming, SupportsTriggerAvailableNow and SupportsConcurrentExecution. - */ -case class DeltaFormatSharingSource( - spark: SparkSession, - client: DeltaSharingClient, - table: DeltaSharingTable, - options: DeltaSharingOptions, - parameters: Map[String, String], - sqlConf: SQLConf, - metadataPath: String) - extends Source - with SupportsAdmissionControl - with DeltaLogging { - - private var tableId: String = "unset_table_id" - - private val tablePath = options.options.getOrElse( - "path", - throw DeltaSharingErrors.pathNotSpecifiedException - ) - - // A unique string composed of a formatted timestamp and an uuid. - // Used as a suffix for the table name and its delta log path of a delta sharing table in a - // streaming job, to avoid overwriting the delta log from multiple references of the same delta - // sharing table in one streaming job. - private val timestampWithUUID = DeltaSharingUtils.getFormattedTimestampWithUUID() - private val customTablePathWithUUIDSuffix = DeltaSharingUtils.getTablePathWithIdSuffix( - client.getProfileProvider.getCustomTablePath(tablePath), - timestampWithUUID - ) - private val deltaLogPath = - s"${DeltaSharingLogFileSystem.encode(customTablePathWithUUIDSuffix).toString}/_delta_log" - - // The latest metadata of the shared table, fetched at the initialization time of the - // DeltaFormatSharingSource, used to initialize the wrapped DeltaSource. - private lazy val deltaSharingTableMetadata = - DeltaSharingUtils.getDeltaSharingTableMetadata(client, table) - - private lazy val deltaSource = initDeltaSource() - - private def initDeltaSource(): DeltaSource = { - val (localDeltaLog, snapshotDescriptor) = DeltaSharingUtils.getDeltaLogAndSnapshotDescriptor( - spark, - deltaSharingTableMetadata, - customTablePathWithUUIDSuffix - ) - val schemaTrackingLogOpt = - DeltaDataSource.getMetadataTrackingLogForDeltaSource( - spark, - snapshotDescriptor, - parameters, - // Pass in the metadata path opt so we can use it for validation - sourceMetadataPathOpt = Some(metadataPath) - ) - - val readSchema = schemaTrackingLogOpt - .flatMap(_.getCurrentTrackedMetadata.map(_.dataSchema)) - .getOrElse(snapshotDescriptor.schema) - - if (readSchema.isEmpty) { - throw DeltaErrors.schemaNotSetException - } - - DeltaSource( - spark = spark, - deltaLog = localDeltaLog, - options = new DeltaOptions(parameters, sqlConf), - snapshotAtSourceInit = snapshotDescriptor, - metadataPath = metadataPath, - metadataTrackingLog = schemaTrackingLogOpt - ) - } - - // schema of the streaming source, based on the latest metadata of the shared table. - override val schema: StructType = { - val schemaWithoutCDC = deltaSharingTableMetadata.metadata.schema - tableId = deltaSharingTableMetadata.metadata.deltaMetadata.id - if (options.readChangeFeed) { - CDCReader.cdcReadSchema(schemaWithoutCDC) - } else { - schemaWithoutCDC - } - } - - // Latest endOffset of the getBatch call, used to compute startingOffset which will then be used - // to compare with the the latest table version on server to decide whether to fetch new data. - private var latestProcessedEndOffsetOption: Option[DeltaSourceOffset] = None - - // Latest table version for the data fetched from the delta sharing server, and stored in the - // local delta log. Used to check whether all fetched files are processed by the DeltaSource. - private var latestTableVersionInLocalDeltaLogOpt: Option[Long] = None - - // This is needed because DeltaSource is not advancing the offset to the next version - // automatically when scanning through a snapshot, so DeltaFormatSharingSource needs to count the - // number of files in the min version and advance the offset to the next version when the offset - // is at the last index of the version. - private var numFileActionsInStartingSnapshotOpt: Option[Int] = None - - // Latest timestamp for getTableVersion rpc from the server, used to compare with the current - // timestamp, to ensure the gap QUERY_TABLE_VERSION_INTERVAL_MILLIS between two rpcs, to avoid - // a high traffic load to the server. - private var lastTimestampForGetVersionFromServer: Long = -1 - - // The minimum gap between two getTableVersion rpcs, to avoid a high traffic load to the server. - private val QUERY_TABLE_VERSION_INTERVAL_MILLIS = TimeUnit.SECONDS.toMillis(30) - - // Maximum number of versions of getFiles() rpc when fetching files from the server. Used to - // reduce the number of files returned to avoid timeout of the rpc on the server. - private val maxVersionsPerRpc: Int = options.maxVersionsPerRpc.getOrElse( - DeltaSharingOptions.MAX_VERSIONS_PER_RPC_DEFAULT - ) - - // A variable to store the latest table version on server, returned from the getTableVersion rpc. - // Used to store the latest table version for getOrUpdateLatestTableVersion when not getting - // updates from the server. - // For all other callers, please use getOrUpdateLatestTableVersion instead of this variable. - private var latestTableVersionOnServer: Long = -1 - - /** - * Check the latest table version from the delta sharing server through the client.getTableVersion - * RPC. Adding a minimum interval of QUERY_TABLE_VERSION_INTERVAL_MILLIS between two consecutive - * rpcs to avoid traffic jam on the delta sharing server. - * - * @return the latest table version on the server. - */ - private def getOrUpdateLatestTableVersion: Long = { - val currentTimeMillis = System.currentTimeMillis() - if ((currentTimeMillis - lastTimestampForGetVersionFromServer) >= - QUERY_TABLE_VERSION_INTERVAL_MILLIS) { - latestTableVersionOnServer = client.getTableVersion(table) - lastTimestampForGetVersionFromServer = currentTimeMillis - } - latestTableVersionOnServer - } - - /** - * NOTE: need to match with the logic in DeltaSource.extractStartingState(). - * - * Get the starting offset used to send rpc to delta sharing server, to fetch needed files. - * Use input startOffset when it's defined, otherwise use user defined starting version, otherwise - * use input endOffset if it's defined, the least option is the latest table version returned from - * the delta sharing server (which is usually used when a streaming query starts from scratch). - * - * @param startOffsetOption optional start offset, return it if defined. It's empty when the - * streaming query starts from scratch. It's set for following calls. - * @param endOffsetOption optional end offset. It's set when the function is called from - * getBatch and is empty when called from latestOffset. - * @return The starting offset. - */ - private def getStartingOffset( - startOffsetOption: Option[DeltaSourceOffset], - endOffsetOption: Option[DeltaSourceOffset]): DeltaSourceOffset = { - if (startOffsetOption.isEmpty) { - val (version, isInitialSnapshot) = getStartingVersion match { - case Some(v) => (v, false) - case None => - if (endOffsetOption.isDefined) { - if (endOffsetOption.get.isInitialSnapshot) { - (endOffsetOption.get.reservoirVersion, true) - } else { - assert( - endOffsetOption.get.reservoirVersion > 0, - s"invalid reservoirVersion in endOffset: ${endOffsetOption.get}" - ) - // Load from snapshot `endOffset.reservoirVersion - 1L` so that `index` in `endOffset` - // is still valid. - // It's OK to use the previous version as the updated initial snapshot, even if the - // initial snapshot might have been different from the last time when this starting - // offset was computed. - (endOffsetOption.get.reservoirVersion - 1L, true) - } - } else { - (getOrUpdateLatestTableVersion, true) - } - } - // Constructed the same way as DeltaSource.buildOffsetFromIndexedFile - DeltaSourceOffset( - reservoirId = tableId, - reservoirVersion = version, - index = DeltaSourceOffset.BASE_INDEX, - isInitialSnapshot = isInitialSnapshot - ) - } else { - startOffsetOption.get - } - } - - /** - * The ending version used in rpc is restricted by both the latest table version and - * maxVersionsPerRpc, to avoid loading too many files from the server to cause a timeout. - * @param startingOffset The start offset used in the rpc. - * @param latestTableVersion The latest table version at the server. - * @return the ending version used in the rpc. - */ - private def getEndingVersionForRpc( - startingOffset: DeltaSourceOffset, - latestTableVersion: Long): Long = { - if (startingOffset.isInitialSnapshot) { - // ending version is the same as starting version for snapshot query. - return startingOffset.reservoirVersion - } - // using "startVersion + maxVersionsPerRpc - 1" because the endingVersion is inclusive. - val endingVersionForQuery = latestTableVersion.min( - startingOffset.reservoirVersion + maxVersionsPerRpc - 1 - ) - if (endingVersionForQuery < latestTableVersion) { - logInfo( - s"Reducing ending version for delta sharing rpc from latestTableVersion(" + - s"$latestTableVersion) to endingVersionForQuery($endingVersionForQuery), " + - s"startVersion:${startingOffset.reservoirVersion}, maxVersionsPerRpc:$maxVersionsPerRpc, " + - s"for table(id:$tableId, name:${table.toString})." - ) - } - endingVersionForQuery - } - - override def getDefaultReadLimit: ReadLimit = { - deltaSource.getDefaultReadLimit - } - - override def latestOffset(startOffset: streaming.Offset, limit: ReadLimit): streaming.Offset = { - val deltaSourceOffset = getStartingOffset(latestProcessedEndOffsetOption, None) - - if (deltaSourceOffset.reservoirVersion < 0) { - return null - } - - maybeGetLatestFileChangesFromServer(deltaSourceOffset) - - maybeMoveToNextVersion(deltaSource.latestOffset(startOffset, limit)) - } - - // Advance the DeltaSourceOffset to the next version when the offset is at the last index of the - // version. - // This is because DeltaSource is not advancing the offset automatically when processing a - // snapshot (isStartingVersion = true), and advancing the offset is necessary for delta sharing - // streaming to fetch new files from the delta sharing server. - private def maybeMoveToNextVersion( - latestOffsetFromDeltaSource: streaming.Offset): DeltaSourceOffset = { - val deltaLatestOffset = deltaSource.toDeltaSourceOffset(latestOffsetFromDeltaSource) - if (deltaLatestOffset.isInitialSnapshot && - (numFileActionsInStartingSnapshotOpt.exists(_ == deltaLatestOffset.index + 1))) { - DeltaSourceOffset( - reservoirId = deltaLatestOffset.reservoirId, - reservoirVersion = deltaLatestOffset.reservoirVersion + 1, - index = DeltaSourceOffset.BASE_INDEX, - isInitialSnapshot = false - ) - } else { - deltaLatestOffset - } - } - - /** - * Whether need to fetch new files from the delta sharing server. - * @param startingOffset the startingOffset of the next batch asked by spark streaming engine. - * @param latestTableVersion the latest table version on the delta sharing server. - * @return whether need to fetch new files from the delta sharing server, this is needed when all - * files are processed in the local delta log, and there are new files on the delta - * sharing server. - * And we avoid fetching new files when files in the delta log are not fully processed. - */ - private def needNewFilesFromServer( - startingOffset: DeltaSourceOffset, - latestTableVersion: Long): Boolean = { - if (latestTableVersionInLocalDeltaLogOpt.isEmpty) { - return true - } - - val allLocalFilesProcessed = latestTableVersionInLocalDeltaLogOpt.exists( - _ < startingOffset.reservoirVersion - ) - val newChangesOnServer = latestTableVersionInLocalDeltaLogOpt.exists(_ < latestTableVersion) - allLocalFilesProcessed && newChangesOnServer - } - - /** - * Check whether we need to fetch new files from the server and calls getTableFileChanges if true. - * - * @param startingOffset the starting offset used to fetch files, the 3 parameters will be useful: - * - reservoirVersion: initially would be the startingVersion or the latest - * table version. - * - index: index of a file within the same version. - * - isInitialSnapshot: If true, will load fromVersion as a table snapshot( - * including files from previous versions). If false, will only load files - * since fromVersion. - * 2 usages: 1) used to compare with latestTableVersionInLocalDeltaLogOpt to - * check whether new files are needed. 2) used for getTableFileChanges, - * check more details in the function header. - */ - private def maybeGetLatestFileChangesFromServer(startingOffset: DeltaSourceOffset): Unit = { - // Use a local variable to avoid a difference in the two usages below. - val latestTableVersion = getOrUpdateLatestTableVersion - - if (needNewFilesFromServer(startingOffset, latestTableVersion)) { - val endingVersionForQuery = - getEndingVersionForRpc(startingOffset, latestTableVersion) - - if (startingOffset.isInitialSnapshot || !options.readChangeFeed) { - getTableFileChanges(startingOffset, endingVersionForQuery) - } else { - throw new UnsupportedOperationException("CDF Streaming is not supported yet.") - } - } - } - - /** - * Fetch the table changes from delta sharing server starting from (version, index) of the - * startingOffset, and store them in locally constructed delta log. - * - * @param startingOffset Includes a reservoirVersion, an index of a file within the same version, - * and an isInitialSnapshot. - * If isInitialSnapshot is true, will load startingOffset.reservoirVersion - * as a table snapshot (including files from previous versions). If false, - * it will only load files since startingOffset.reservoirVersion. - * @param endingVersionForQuery The ending version used for the query, always smaller than - * the latest table version on server. - */ - private def getTableFileChanges( - startingOffset: DeltaSourceOffset, - endingVersionForQuery: Long): Unit = { - logInfo( - s"Fetching files with table version(${startingOffset.reservoirVersion}), " + - s"index(${startingOffset.index}), isInitialSnapshot(${startingOffset.isInitialSnapshot})," + - s" endingVersionForQuery($endingVersionForQuery), for table(id:$tableId, " + - s"name:${table.toString}) with latest version on server($latestTableVersionOnServer)." - ) - - val (tableFiles, refreshFunc) = if (startingOffset.isInitialSnapshot) { - // If isInitialSnapshot is true, it means to fetch the snapshot at the fromVersion, which may - // include table changes from previous versions. - val tableFiles = client.getFiles( - table = table, - predicates = Nil, - limit = None, - versionAsOf = Some(startingOffset.reservoirVersion), - timestampAsOf = None, - jsonPredicateHints = None, - refreshToken = None - ) - val refreshFunc = DeltaSharingUtils.getRefresherForGetFiles( - client = client, - table = table, - predicates = Nil, - limit = None, - versionAsOf = Some(startingOffset.reservoirVersion), - timestampAsOf = None, - jsonPredicateHints = None, - refreshToken = None - ) - logInfo( - s"Fetched ${tableFiles.lines.size} lines for table version ${tableFiles.version} from" + - " delta sharing server." - ) - (tableFiles, refreshFunc) - } else { - // If isStartingVersion is false, it means to fetch files for data changes since fromVersion, - // not including files from previous versions. - val tableFiles = client.getFiles( - table = table, - startingVersion = startingOffset.reservoirVersion, - endingVersion = Some(endingVersionForQuery) - ) - val refreshFunc = DeltaSharingUtils.getRefresherForGetFilesWithStartingVersion( - client = client, - table = table, - startingVersion = startingOffset.reservoirVersion, - endingVersion = Some(endingVersionForQuery) - ) - logInfo( - s"Fetched ${tableFiles.lines.size} lines from startingVersion " + - s"${startingOffset.reservoirVersion} to enedingVersion ${endingVersionForQuery} from " + - "delta sharing server." - ) - (tableFiles, refreshFunc) - } - - val deltaLogMetadata = DeltaSharingLogFileSystem.constructLocalDeltaLogAcrossVersions( - lines = tableFiles.lines, - customTablePath = customTablePathWithUUIDSuffix, - startingVersionOpt = Some(startingOffset.reservoirVersion), - endingVersionOpt = Some(endingVersionForQuery) - ) - assert( - deltaLogMetadata.maxVersion > 0, - s"Invalid table version in delta sharing response: ${tableFiles.lines}." - ) - latestTableVersionInLocalDeltaLogOpt = Some(deltaLogMetadata.maxVersion) - assert( - deltaLogMetadata.numFileActionsInMinVersionOpt.isDefined, - "numFileActionsInMinVersionOpt missing after constructed delta log." - ) - if (startingOffset.isInitialSnapshot) { - numFileActionsInStartingSnapshotOpt = deltaLogMetadata.numFileActionsInMinVersionOpt - } - - CachedTableManager.INSTANCE.register( - tablePath = DeltaSharingUtils.getTablePathWithIdSuffix(tablePath, timestampWithUUID), - idToUrl = deltaLogMetadata.idToUrl, - refs = Seq(new WeakReference(this)), - profileProvider = client.getProfileProvider, - refresher = refreshFunc, - expirationTimestamp = - if (CachedTableManager.INSTANCE - .isValidUrlExpirationTime(deltaLogMetadata.minUrlExpirationTimestamp)) { - deltaLogMetadata.minUrlExpirationTimestamp.get - } else { - System.currentTimeMillis() + CachedTableManager.INSTANCE.preSignedUrlExpirationMs - }, - refreshToken = tableFiles.refreshToken - ) - } - - override def getBatch(startOffsetOption: Option[Offset], end: Offset): DataFrame = { - val endOffset = deltaSource.toDeltaSourceOffset(end) - val startDeltaOffsetOption = startOffsetOption.map(deltaSource.toDeltaSourceOffset) - val startingOffset = getStartingOffset(startDeltaOffsetOption, Some(endOffset)) - - maybeGetLatestFileChangesFromServer(startingOffset = startingOffset) - // Reset latestProcessedEndOffsetOption only when endOffset is larger. - // Because with microbatch pipelining, we may get getBatch requests out of order. - if (latestProcessedEndOffsetOption.isEmpty || - endOffset.reservoirVersion > latestProcessedEndOffsetOption.get.reservoirVersion || - (endOffset.reservoirVersion == latestProcessedEndOffsetOption.get.reservoirVersion && - endOffset.index > latestProcessedEndOffsetOption.get.index)) { - latestProcessedEndOffsetOption = Some(endOffset) - } - - deltaSource.getBatch(startOffsetOption, end) - } - - override def getOffset: Option[Offset] = { - throw new UnsupportedOperationException( - "latestOffset(Offset, ReadLimit) should be called instead of this method." - ) - } - - /** - * Extracts whether users provided the option to time travel a relation. If a query restarts from - * a checkpoint and the checkpoint has recorded the offset, this method should never been called. - */ - private lazy val getStartingVersion: Option[Long] = { - - /** DeltaOption validates input and ensures that only one is provided. */ - if (options.startingVersion.isDefined) { - val v = options.startingVersion.get match { - case StartingVersionLatest => - getOrUpdateLatestTableVersion + 1 - case StartingVersion(version) => - version - } - Some(v) - } else if (options.startingTimestamp.isDefined) { - Some(client.getTableVersion(table, options.startingTimestamp)) - } else { - None - } - } - - override def stop(): Unit = { - deltaSource.stop() - - DeltaSharingLogFileSystem.tryToCleanUpDeltaLog(deltaLogPath) - } - - // Calls deltaSource.commit for checks related to column mapping. - override def commit(end: Offset): Unit = { - deltaSource.commit(end) - - // Clean up previous blocks after commit. - val endOffset = deltaSource.toDeltaSourceOffset(end) - DeltaSharingLogFileSystem.tryToCleanUpPreviousBlocks( - deltaLogPath, - endOffset.reservoirVersion - 1 - ) - } - - override def toString(): String = s"DeltaFormatSharingSource[${table.toString}]" -} diff --git a/sharing/src/main/scala/io/delta/sharing/spark/DeltaSharingCDFUtils.scala b/sharing/src/main/scala/io/delta/sharing/spark/DeltaSharingCDFUtils.scala deleted file mode 100644 index 7b8a8294c67..00000000000 --- a/sharing/src/main/scala/io/delta/sharing/spark/DeltaSharingCDFUtils.scala +++ /dev/null @@ -1,112 +0,0 @@ -/* - * Copyright (2021) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.delta.sharing.spark - -import java.lang.ref.WeakReference -import java.nio.charset.StandardCharsets.UTF_8 - -import org.apache.spark.sql.delta.catalog.DeltaTableV2 -import com.google.common.hash.Hashing -import io.delta.sharing.client.DeltaSharingClient -import io.delta.sharing.client.model.{Table => DeltaSharingTable} -import org.apache.hadoop.fs.Path - -import org.apache.spark.delta.sharing.CachedTableManager -import org.apache.spark.internal.Logging -import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.sources.BaseRelation - -object DeltaSharingCDFUtils extends Logging { - - private def getDuration(start: Long): Double = { - (System.currentTimeMillis() - start) / 1000.0 - } - - /** - * Prepares the BaseRelation for cdf queries on a delta sharing table. Since there's no limit - * pushdown or filter pushdown involved, it wiill firatly fetch all the files from the delta - * sharing server, prepare the local delta log, and leverage DeltaTableV2 to produce the relation. - */ - private[sharing] def prepareCDFRelation( - sqlContext: SQLContext, - options: DeltaSharingOptions, - table: DeltaSharingTable, - client: DeltaSharingClient): BaseRelation = { - val startTime = System.currentTimeMillis() - // 1. Get all files with DeltaSharingClient. - // includeHistoricalMetadata is always set to true, to get the metadata at the startingVersion - // and also any metadata changes between [startingVersion, endingVersion], to put them in the - // delta log. This is to allow delta library to check the metadata change and handle it - // properly -- currently it throws error for column mapping changes. - val deltaTableFiles = - client.getCDFFiles(table, options.cdfOptions, includeHistoricalMetadata = true) - logInfo( - s"Fetched ${deltaTableFiles.lines.size} lines with cdf options ${options.cdfOptions} " + - s"for table ${table} from delta sharing server, took ${getDuration(startTime)}s." - ) - - val path = options.options.getOrElse("path", throw DeltaSharingErrors.pathNotSpecifiedException) - // 2. Prepare local delta log - val queryCustomTablePath = client.getProfileProvider.getCustomTablePath(path) - val queryParamsHashId = DeltaSharingUtils.getQueryParamsHashId(options.cdfOptions) - val tablePathWithHashIdSuffix = - DeltaSharingUtils.getTablePathWithIdSuffix(queryCustomTablePath, queryParamsHashId) - val deltaLogMetadata = DeltaSharingLogFileSystem.constructLocalDeltaLogAcrossVersions( - lines = deltaTableFiles.lines, - customTablePath = tablePathWithHashIdSuffix, - startingVersionOpt = None, - endingVersionOpt = None - ) - - // 3. Register parquet file id to url mapping - CachedTableManager.INSTANCE.register( - // Using path instead of queryCustomTablePath because it will be customized within - // CachedTableManager. - tablePath = DeltaSharingUtils.getTablePathWithIdSuffix(path, queryParamsHashId), - idToUrl = deltaLogMetadata.idToUrl, - // A weak reference is needed by the CachedTableManager to decide whether the query is done - // and it's ok to clean up the id to url mapping for this table. - refs = Seq(new WeakReference(this)), - profileProvider = client.getProfileProvider, - refresher = DeltaSharingUtils.getRefresherForGetCDFFiles( - client = client, - table = table, - cdfOptions = options.cdfOptions - ), - expirationTimestamp = - if (CachedTableManager.INSTANCE - .isValidUrlExpirationTime(deltaLogMetadata.minUrlExpirationTimestamp)) { - deltaLogMetadata.minUrlExpirationTimestamp.get - } else { - System.currentTimeMillis() + CachedTableManager.INSTANCE.preSignedUrlExpirationMs - }, - refreshToken = None - ) - - // 4. return Delta - val localDeltaCdfOptions = Map( - DeltaSharingOptions.CDF_START_VERSION -> deltaLogMetadata.minVersion.toString, - DeltaSharingOptions.CDF_END_VERSION -> deltaLogMetadata.maxVersion.toString, - DeltaSharingOptions.CDF_READ_OPTION -> "true" - ) - DeltaTableV2( - spark = sqlContext.sparkSession, - path = DeltaSharingLogFileSystem.encode(tablePathWithHashIdSuffix), - options = localDeltaCdfOptions - ).toBaseRelation - } -} diff --git a/sharing/src/main/scala/io/delta/sharing/spark/DeltaSharingDataSource.scala b/sharing/src/main/scala/io/delta/sharing/spark/DeltaSharingDataSource.scala deleted file mode 100644 index a7f0eade084..00000000000 --- a/sharing/src/main/scala/io/delta/sharing/spark/DeltaSharingDataSource.scala +++ /dev/null @@ -1,448 +0,0 @@ -/* - * Copyright (2021) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.delta.sharing.spark - -import scala.collection.JavaConverters._ - -import org.apache.spark.sql.delta.{ - DeltaColumnMapping, - DeltaErrors, - DeltaTableUtils => TahoeDeltaTableUtils -} -import org.apache.spark.sql.delta.commands.cdc.CDCReader -import org.apache.spark.sql.delta.metering.DeltaLogging -import org.apache.spark.sql.delta.schema.SchemaUtils -import org.apache.spark.sql.delta.sources.{DeltaDataSource, DeltaSQLConf} -import io.delta.sharing.client.{DeltaSharingClient, DeltaSharingRestClient} -import io.delta.sharing.client.model.{Table => DeltaSharingTable} -import io.delta.sharing.client.util.{ConfUtils, JsonUtils} -import org.apache.hadoop.fs.Path - -import org.apache.spark.SparkEnv -import org.apache.spark.delta.sharing.PreSignedUrlCache -import org.apache.spark.sql.{SparkSession, SQLContext} -import org.apache.spark.sql.execution.datasources.HadoopFsRelation -import org.apache.spark.sql.execution.streaming.Source -import org.apache.spark.sql.sources.{ - BaseRelation, - DataSourceRegister, - RelationProvider, - StreamSourceProvider -} -import org.apache.spark.sql.types.StructType - -/** - * A DataSource for Delta Sharing, used to support all types of queries on a delta sharing table: - * batch, cdf, streaming, time travel, filters, etc. - */ -private[sharing] class DeltaSharingDataSource - extends RelationProvider - with StreamSourceProvider - with DataSourceRegister - with DeltaLogging { - - override def sourceSchema( - sqlContext: SQLContext, - schema: Option[StructType], - providerName: String, - parameters: Map[String, String]): (String, StructType) = { - DeltaSharingDataSource.setupFileSystem(sqlContext) - if (schema.nonEmpty && schema.get.nonEmpty) { - throw DeltaErrors.specifySchemaAtReadTimeException - } - val options = new DeltaSharingOptions(parameters) - if (options.isTimeTravel) { - throw DeltaErrors.timeTravelNotSupportedException - } - val path = options.options.getOrElse("path", throw DeltaSharingErrors.pathNotSpecifiedException) - - if (options.responseFormat == DeltaSharingOptions.RESPONSE_FORMAT_PARQUET) { - logInfo(s"sourceSchema with parquet format for table path:$path, parameters:$parameters") - val deltaLog = RemoteDeltaLog( - path, - forStreaming = true, - responseFormat = options.responseFormat - ) - val schemaToUse = deltaLog.snapshot().schema - if (schemaToUse.isEmpty) { - throw DeltaSharingErrors.schemaNotSetException - } - - if (options.readChangeFeed) { - (shortName(), DeltaTableUtils.addCdcSchema(schemaToUse)) - } else { - (shortName(), schemaToUse) - } - } else if (options.responseFormat == DeltaSharingOptions.RESPONSE_FORMAT_DELTA) { - logInfo(s"sourceSchema with delta format for table path:$path, parameters:$parameters") - if (options.readChangeFeed) { - throw new UnsupportedOperationException( - s"Delta sharing cdc streaming is not supported when responseforma=delta." - ) - } - // 1. create delta sharing client - val parsedPath = DeltaSharingRestClient.parsePath(path) - val client = DeltaSharingRestClient( - profileFile = parsedPath.profileFile, - forStreaming = true, - responseFormat = options.responseFormat, - // comma separated delta reader features, used to tell delta sharing server what delta - // reader features the client is able to process. - readerFeatures = DeltaSharingUtils.STREAMING_SUPPORTED_READER_FEATURES.mkString(",") - ) - val dsTable = DeltaSharingTable( - share = parsedPath.share, - schema = parsedPath.schema, - name = parsedPath.table - ) - - // 2. getMetadata for schema to be used in the file index. - val deltaSharingTableMetadata = DeltaSharingUtils.getDeltaSharingTableMetadata( - client = client, - table = dsTable - ) - val customTablePathWithUUIDSuffix = DeltaSharingUtils.getTablePathWithIdSuffix( - client.getProfileProvider.getCustomTablePath(path), - DeltaSharingUtils.getFormattedTimestampWithUUID() - ) - val deltaLogPath = - s"${DeltaSharingLogFileSystem.encode(customTablePathWithUUIDSuffix).toString}/_delta_log" - val (_, snapshotDescriptor) = DeltaSharingUtils.getDeltaLogAndSnapshotDescriptor( - sqlContext.sparkSession, - deltaSharingTableMetadata, - customTablePathWithUUIDSuffix - ) - - // This is the analyzed schema for Delta streaming - val readSchema = { - // Check if we would like to merge consecutive schema changes, this would allow customers - // to write queries based on their latest changes instead of an arbitrary schema in the - // past. - val shouldMergeConsecutiveSchemas = sqlContext.sparkSession.sessionState.conf.getConf( - DeltaSQLConf.DELTA_STREAMING_ENABLE_SCHEMA_TRACKING_MERGE_CONSECUTIVE_CHANGES - ) - - // This method is invoked during the analysis phase and would determine the schema for the - // streaming dataframe. We only need to merge consecutive schema changes here because the - // process would create a new entry in the schema log such that when the schema log is - // looked up again in the execution phase, we would use the correct schema. - DeltaDataSource - .getMetadataTrackingLogForDeltaSource( - sqlContext.sparkSession, - snapshotDescriptor, - parameters, - mergeConsecutiveSchemaChanges = shouldMergeConsecutiveSchemas - ) - .flatMap(_.getCurrentTrackedMetadata.map(_.dataSchema)) - .getOrElse(snapshotDescriptor.schema) - } - - val schemaToUse = TahoeDeltaTableUtils.removeInternalMetadata( - sqlContext.sparkSession, - readSchema - ) - if (schemaToUse.isEmpty) { - throw DeltaErrors.schemaNotSetException - } - - DeltaSharingLogFileSystem.tryToCleanUpDeltaLog(deltaLogPath) - (shortName(), schemaToUse) - } else { - throw new UnsupportedOperationException( - s"responseformat(${options.responseFormat}) is not " + - s"supported in delta sharing." - ) - } - } - - override def createSource( - sqlContext: SQLContext, - metadataPath: String, - schema: Option[StructType], - providerName: String, - parameters: Map[String, String]): Source = { - DeltaSharingDataSource.setupFileSystem(sqlContext) - if (schema.nonEmpty && schema.get.nonEmpty) { - throw DeltaSharingErrors.specifySchemaAtReadTimeException - } - val options = new DeltaSharingOptions(parameters) - val path = options.options.getOrElse("path", throw DeltaSharingErrors.pathNotSpecifiedException) - - if (options.responseFormat == DeltaSharingOptions.RESPONSE_FORMAT_PARQUET) { - logInfo(s"createSource with parquet format for table path:$path, parameters:$parameters") - val deltaLog = RemoteDeltaLog(path, forStreaming = true, options.responseFormat) - DeltaSharingSource(SparkSession.active, deltaLog, options) - } else if (options.responseFormat == DeltaSharingOptions.RESPONSE_FORMAT_DELTA) { - logInfo(s"createSource with delta format for table path:$path, parameters:$parameters") - if (options.readChangeFeed) { - throw new UnsupportedOperationException( - s"Delta sharing cdc streaming is not supported when responseforma=delta." - ) - } - // 1. create delta sharing client - val parsedPath = DeltaSharingRestClient.parsePath(path) - val client = DeltaSharingRestClient( - profileFile = parsedPath.profileFile, - forStreaming = true, - responseFormat = options.responseFormat, - // comma separated delta reader features, used to tell delta sharing server what delta - // reader features the client is able to process. - readerFeatures = DeltaSharingUtils.STREAMING_SUPPORTED_READER_FEATURES.mkString(",") - ) - val dsTable = DeltaSharingTable( - share = parsedPath.share, - schema = parsedPath.schema, - name = parsedPath.table - ) - - DeltaFormatSharingSource( - spark = sqlContext.sparkSession, - client = client, - table = dsTable, - options = options, - parameters = parameters, - sqlConf = sqlContext.sparkSession.sessionState.conf, - metadataPath = metadataPath - ) - } else { - throw new UnsupportedOperationException( - s"responseformat(${options.responseFormat}) is not " + - s"supported in delta sharing." - ) - } - } - - override def createRelation( - sqlContext: SQLContext, - parameters: Map[String, String]): BaseRelation = { - DeltaSharingDataSource.setupFileSystem(sqlContext) - val options = new DeltaSharingOptions(parameters) - - val userInputResponseFormat = options.options.get(DeltaSharingOptions.RESPONSE_FORMAT) - if (userInputResponseFormat.isEmpty && !options.readChangeFeed) { - return autoResolveBaseRelationForSnapshotQuery(options) - } - - val path = options.options.getOrElse("path", throw DeltaSharingErrors.pathNotSpecifiedException) - if (options.responseFormat == DeltaSharingOptions.RESPONSE_FORMAT_PARQUET) { - // When user explicitly set responseFormat=parquet, to query shared tables without advanced - // delta features. - logInfo(s"createRelation with parquet format for table path:$path, parameters:$parameters") - val deltaLog = RemoteDeltaLog( - path, - forStreaming = false, - responseFormat = options.responseFormat - ) - deltaLog.createRelation( - options.versionAsOf, - options.timestampAsOf, - options.cdfOptions - ) - } else if (options.responseFormat == DeltaSharingOptions.RESPONSE_FORMAT_DELTA) { - // When user explicitly set responseFormat=delta, to query shared tables with advanced - // delta features. - logInfo(s"createRelation with delta format for table path:$path, parameters:$parameters") - // 1. create delta sharing client - val parsedPath = DeltaSharingRestClient.parsePath(path) - val client = DeltaSharingRestClient( - profileFile = parsedPath.profileFile, - forStreaming = false, - responseFormat = options.responseFormat, - // comma separated delta reader features, used to tell delta sharing server what delta - // reader features the client is able to process. - readerFeatures = DeltaSharingUtils.SUPPORTED_READER_FEATURES.mkString(",") - ) - val dsTable = DeltaSharingTable( - share = parsedPath.share, - schema = parsedPath.schema, - name = parsedPath.table - ) - - if (options.readChangeFeed) { - return DeltaSharingCDFUtils.prepareCDFRelation(sqlContext, options, dsTable, client) - } - // 2. getMetadata for schema to be used in the file index. - val deltaTableMetadata = DeltaSharingUtils.queryDeltaTableMetadata( - client = client, - table = dsTable, - versionAsOf = options.versionAsOf, - timestampAsOf = options.timestampAsOf - ) - val deltaSharingTableMetadata = DeltaSharingUtils.getDeltaSharingTableMetadata( - table = dsTable, - deltaTableMetadata = deltaTableMetadata - ) - - // 3. Prepare HadoopFsRelation - getHadoopFsRelationForDeltaSnapshotQuery( - path = path, - options = options, - dsTable = dsTable, - client = client, - deltaSharingTableMetadata = deltaSharingTableMetadata - ) - } else { - throw new UnsupportedOperationException( - s"responseformat(${options.responseFormat}) is not supported in delta sharing." - ) - } - } - - /** - * "parquet format sharing" leverages the existing set of remote classes to directly handle the - * list of presigned urls and read data. - * "delta format sharing" instead constructs a local delta log and leverages the delta library to - * read data. - * Firstly we sends a getMetadata call to the delta sharing server the suggested response format - * of the shared table by the server (based on whether there are advanced delta features in the - * shared table), and then decide the code path on the client side. - */ - private def autoResolveBaseRelationForSnapshotQuery( - options: DeltaSharingOptions): BaseRelation = { - val path = options.options.getOrElse("path", throw DeltaSharingErrors.pathNotSpecifiedException) - val parsedPath = DeltaSharingRestClient.parsePath(path) - - val client = DeltaSharingRestClient( - profileFile = parsedPath.profileFile, - forStreaming = false, - // Indicating that the client is able to process response format in both parquet and delta. - responseFormat = s"${DeltaSharingOptions.RESPONSE_FORMAT_PARQUET}," + - s"${DeltaSharingOptions.RESPONSE_FORMAT_DELTA}", - // comma separated delta reader features, used to tell delta sharing server what delta - // reader features the client is able to process. - readerFeatures = DeltaSharingUtils.SUPPORTED_READER_FEATURES.mkString(",") - ) - val dsTable = DeltaSharingTable( - name = parsedPath.table, - schema = parsedPath.schema, - share = parsedPath.share - ) - - val deltaTableMetadata = DeltaSharingUtils.queryDeltaTableMetadata( - client = client, - table = dsTable, - versionAsOf = options.versionAsOf, - timestampAsOf = options.timestampAsOf - ) - - if (deltaTableMetadata.respondedFormat == DeltaSharingOptions.RESPONSE_FORMAT_PARQUET) { - val deltaLog = RemoteDeltaLog( - path = path, - forStreaming = false, - responseFormat = DeltaSharingOptions.RESPONSE_FORMAT_PARQUET, - initDeltaTableMetadata = Some(deltaTableMetadata) - ) - deltaLog.createRelation(options.versionAsOf, options.timestampAsOf, options.cdfOptions) - } else if (deltaTableMetadata.respondedFormat == DeltaSharingOptions.RESPONSE_FORMAT_DELTA) { - val deltaSharingTableMetadata = DeltaSharingUtils.getDeltaSharingTableMetadata( - table = dsTable, - deltaTableMetadata = deltaTableMetadata - ) - val deltaOnlyClient = DeltaSharingRestClient( - profileFile = parsedPath.profileFile, - forStreaming = false, - // Indicating that the client request delta format in response. - responseFormat = DeltaSharingOptions.RESPONSE_FORMAT_DELTA, - // comma separated delta reader features, used to tell delta sharing server what delta - // reader features the client is able to process. - readerFeatures = DeltaSharingUtils.SUPPORTED_READER_FEATURES.mkString(",") - ) - getHadoopFsRelationForDeltaSnapshotQuery( - path = path, - options = options, - dsTable = dsTable, - client = deltaOnlyClient, - deltaSharingTableMetadata = deltaSharingTableMetadata - ) - } else { - throw new UnsupportedOperationException( - s"Unexpected respondedFormat for getMetadata rpc:${deltaTableMetadata.respondedFormat}." - ) - } - } - - /** - * Prepare a HadoopFsRelation for the snapshot query on a delta sharing table. It will contain a - * DeltaSharingFileIndex which is used to handle delta sharing rpc, and construct the local delta - * log, and then build a TahoeFileIndex on top of the delta log. - */ - private def getHadoopFsRelationForDeltaSnapshotQuery( - path: String, - options: DeltaSharingOptions, - dsTable: DeltaSharingTable, - client: DeltaSharingClient, - deltaSharingTableMetadata: DeltaSharingUtils.DeltaSharingTableMetadata): BaseRelation = { - // Prepare DeltaSharingFileIndex - val spark = SparkSession.active - val params = new DeltaSharingFileIndexParams( - new Path(path), - spark, - deltaSharingTableMetadata.metadata, - options - ) - if (ConfUtils.limitPushdownEnabled(spark.sessionState.conf)) { - DeltaFormatSharingLimitPushDown.setup(spark) - } - // limitHint is always None here and will be overridden in DeltaFormatSharingLimitPushDown. - val fileIndex = DeltaSharingFileIndex( - params = params, - table = dsTable, - client = client, - limitHint = None - ) - - // return HadoopFsRelation with the DeltaSharingFileIndex. - HadoopFsRelation( - location = fileIndex, - // This is copied from DeltaLog.buildHadoopFsRelationWithFileIndex. - // Dropping column mapping metadata because it is not relevant for partition schema. - partitionSchema = DeltaColumnMapping.dropColumnMappingMetadata(fileIndex.partitionSchema), - // This is copied from DeltaLog.buildHadoopFsRelationWithFileIndex, original comment: - // We pass all table columns as `dataSchema` so that Spark will preserve the partition - // column locations. Otherwise, for any partition columns not in `dataSchema`, Spark would - // just append them to the end of `dataSchema`. - dataSchema = DeltaColumnMapping.dropColumnMappingMetadata( - TahoeDeltaTableUtils.removeInternalMetadata( - spark, - SchemaUtils.dropNullTypeColumns(deltaSharingTableMetadata.metadata.schema) - ) - ), - bucketSpec = None, - // Handle column mapping metadata in schema. - fileFormat = fileIndex.fileFormat( - deltaSharingTableMetadata.protocol.deltaProtocol, - deltaSharingTableMetadata.metadata.deltaMetadata - ), - options = Map.empty - )(spark) - } - - override def shortName(): String = "deltaSharing" -} - -private[sharing] object DeltaSharingDataSource { - def setupFileSystem(sqlContext: SQLContext): Unit = { - sqlContext.sparkContext.hadoopConfiguration - .setIfUnset("fs.delta-sharing.impl", "io.delta.sharing.client.DeltaSharingFileSystem") - sqlContext.sparkContext.hadoopConfiguration - .setIfUnset( - "fs.delta-sharing-log.impl", - "io.delta.sharing.spark.DeltaSharingLogFileSystem" - ) - PreSignedUrlCache.registerIfNeeded(SparkEnv.get) - } -} diff --git a/sharing/src/main/scala/io/delta/sharing/spark/DeltaSharingLogFileSystem.scala b/sharing/src/main/scala/io/delta/sharing/spark/DeltaSharingLogFileSystem.scala index 9010e9487d2..7214f6368b5 100644 --- a/sharing/src/main/scala/io/delta/sharing/spark/DeltaSharingLogFileSystem.scala +++ b/sharing/src/main/scala/io/delta/sharing/spark/DeltaSharingLogFileSystem.scala @@ -352,333 +352,6 @@ private[sharing] object DeltaSharingLogFileSystem extends Logging { f1.id < f2.id } - /** - * Cleanup the delta log upon explicit stop of a query on a delta sharing table. - * - * @param deltaLogPath deltaLogPath is constructed per query with credential scope id as prefix - * and a uuid as suffix, which is very unique to the query and won't interfere - * with other queries. - */ - def tryToCleanUpDeltaLog(deltaLogPath: String): Unit = { - def shouldCleanUp(blockId: BlockId): Boolean = { - if (!blockId.name.startsWith(DELTA_SHARING_LOG_BLOCK_ID_PREFIX)) { - return false - } - val blockName = blockId.name - // deltaLogPath is constructed per query with credential scope id as prefix and a uuid as - // suffix, which is very unique to the query and won't interfere with other queries. - blockName.startsWith(BLOCK_ID_TEST_PREFIX + deltaLogPath) - } - - val blockManager = SparkEnv.get.blockManager - val matchingBlockIds = blockManager.getMatchingBlockIds(shouldCleanUp(_)) - logInfo( - s"Trying to clean up ${matchingBlockIds.size} blocks for $deltaLogPath." - ) - - val problematicBlockIds = Seq.newBuilder[BlockId] - matchingBlockIds.foreach { b => - try { - blockManager.removeBlock(b) - } catch { - case _: Throwable => problematicBlockIds += b - } - } - - val problematicBlockIdsSeq = problematicBlockIds.result().toSeq - if (problematicBlockIdsSeq.size > 0) { - logWarning( - s"Done cleaning up ${matchingBlockIds.size} blocks for $deltaLogPath, but " + - s"failed to remove: ${problematicBlockIdsSeq}." - ) - } else { - logInfo( - s"Done cleaning up ${matchingBlockIds.size} blocks for $deltaLogPath." - ) - } - } - - /** - * @param deltaLogPath The delta log directory to clean up. It is constructed per query with - * credential scope id as prefix and a uuid as suffix, which is very unique - * to the query and won't interfere with other queries. - * @param maxVersion maxVersion of any checkpoint or delta file that needs clean up, inclusive. - */ - def tryToCleanUpPreviousBlocks(deltaLogPath: String, maxVersion: Long): Unit = { - if (maxVersion < 0) { - logInfo( - s"Skipping clean up previous blocks for $deltaLogPath because maxVersion(" + - s"$maxVersion) < 0." - ) - return - } - - def shouldCleanUp(blockId: BlockId): Boolean = { - if (!blockId.name.startsWith(DELTA_SHARING_LOG_BLOCK_ID_PREFIX)) { - return false - } - val blockName = blockId.name - blockName.startsWith(BLOCK_ID_TEST_PREFIX + deltaLogPath) && FileNames - .getFileVersionOpt(new Path(blockName.stripPrefix(BLOCK_ID_TEST_PREFIX))) - .exists(_ <= maxVersion) - } - - val blockManager = SparkEnv.get.blockManager - val matchingBlockIds = blockManager.getMatchingBlockIds(shouldCleanUp(_)) - logInfo( - s"Trying to clean up ${matchingBlockIds.size} previous blocks for $deltaLogPath " + - s"before version: $maxVersion." - ) - - val problematicBlockIds = Seq.newBuilder[BlockId] - matchingBlockIds.foreach { b => - try { - blockManager.removeBlock(b) - } catch { - case _: Throwable => problematicBlockIds += b - } - } - - val problematicBlockIdsSeq = problematicBlockIds.result().toSeq - if (problematicBlockIdsSeq.size > 0) { - logWarning( - s"Done cleaning up ${matchingBlockIds.size} previous blocks for $deltaLogPath " + - s"before version: $maxVersion, but failed to remove: ${problematicBlockIdsSeq}." - ) - } else { - logInfo( - s"Done cleaning up ${matchingBlockIds.size} previous blocks for $deltaLogPath " + - s"before version: $maxVersion." - ) - } - } - - /** - * Construct local delta log based on delta log actions returned from delta sharing server. - * - * @param lines a list of delta actions, to be processed and put in the local delta log, - * each action contains a version field to indicate the version of log to - * put it in. - * @param customTablePath query customized table path, used to construct action.path field for - * DeltaSharingFileSystem - * @param startingVersionOpt If set, used to construct the delta file (.json log file) from the - * given startingVersion. This is needed by DeltaSharingSource to - * construct the delta log for the rpc no matter if there are files in - * that version or not, so DeltaSource can read delta actions from the - * starting version (instead from checkpoint). - * @param endingVersionOpt If set, used to construct the delta file (.json log file) until the - * given endingVersion. This is needed by DeltaSharingSource to construct - * the delta log for the rpc no matter if there are files in that version - * or not. - * NOTE: DeltaSource will not advance the offset if there are no files in - * a version of the delta log, but we still create the delta log file for - * that version to avoid missing delta log (json) files. - * @return ConstructedDeltaLogMetadata, which contains 3 fields: - * - idToUrl: mapping from file id to pre-signed url - * - minUrlExpirationTimestamp timestamp indicating the when to refresh pre-signed urls. - * Both are used to register to CachedTableManager. - * - maxVersion: the max version returned in the http response, used by - * DeltaSharingSource to quickly understand the progress of rpcs from the server. - */ - def constructLocalDeltaLogAcrossVersions( - lines: Seq[String], - customTablePath: String, - startingVersionOpt: Option[Long], - endingVersionOpt: Option[Long]): ConstructedDeltaLogMetadata = { - val startTime = System.currentTimeMillis() - assert( - startingVersionOpt.isDefined == endingVersionOpt.isDefined, - s"startingVersionOpt($startingVersionOpt) and endingVersionOpt($endingVersionOpt) should be" + - " both defined or not." - ) - if (startingVersionOpt.isDefined) { - assert( - startingVersionOpt.get <= endingVersionOpt.get, - s"startingVersionOpt($startingVersionOpt) must be smaller than " + - s"endingVersionOpt($endingVersionOpt)." - ) - } - var minVersion = Long.MaxValue - var maxVersion = 0L - var minUrlExpirationTimestamp: Option[Long] = None - val idToUrl = scala.collection.mutable.Map[String, String]() - val versionToDeltaSharingFileActions = - scala.collection.mutable.Map[Long, ArrayBuffer[model.DeltaSharingFileAction]]() - val versionToMetadata = scala.collection.mutable.Map[Long, model.DeltaSharingMetadata]() - val versionToJsonLogBuilderMap = scala.collection.mutable.Map[Long, ArrayBuffer[String]]() - val versionToJsonLogSize = scala.collection.mutable.Map[Long, Long]().withDefaultValue(0L) - var numFileActionsInMinVersion = 0 - val versionToTimestampMap = scala.collection.mutable.Map[Long, Long]() - var startingMetadataLineOpt: Option[String] = None - var startingProtocolLineOpt: Option[String] = None - - lines.foreach { line => - val action = JsonUtils.fromJson[model.DeltaSharingSingleAction](line).unwrap - action match { - case fileAction: model.DeltaSharingFileAction => - minVersion = minVersion.min(fileAction.version) - maxVersion = maxVersion.max(fileAction.version) - // Store file actions in an array to sort them based on id later. - versionToDeltaSharingFileActions.getOrElseUpdate( - fileAction.version, - ArrayBuffer[model.DeltaSharingFileAction]() - ) += fileAction - case metadata: model.DeltaSharingMetadata => - if (metadata.version != null) { - // This is to handle the cdf and streaming query result. - minVersion = minVersion.min(metadata.version) - maxVersion = maxVersion.max(metadata.version) - versionToMetadata(metadata.version) = metadata - if (metadata.version == minVersion) { - startingMetadataLineOpt = Some(metadata.deltaMetadata.json + "\n") - } - } else { - // This is to handle the snapshot query result from DeltaSharingSource. - startingMetadataLineOpt = Some(metadata.deltaMetadata.json + "\n") - } - case protocol: model.DeltaSharingProtocol => - startingProtocolLineOpt = Some(protocol.deltaProtocol.json + "\n") - case _ => // do nothing, ignore the line. - } - } - - if (startingVersionOpt.isDefined) { - minVersion = minVersion.min(startingVersionOpt.get) - } else if (minVersion == Long.MaxValue) { - // This means there are no files returned from server for this cdf request. - // A 0.json file will be prepared with metadata and protocol only. - minVersion = 0 - } - if (endingVersionOpt.isDefined) { - maxVersion = maxVersion.max(endingVersionOpt.get) - } - // Store the starting protocol and metadata in the minVersion.json. - val protocolAndMetadataStr = startingMetadataLineOpt.getOrElse("") + startingProtocolLineOpt - .getOrElse("") - versionToJsonLogBuilderMap.getOrElseUpdate( - minVersion, - ArrayBuffer[String]() - ) += protocolAndMetadataStr - versionToJsonLogSize(minVersion) += protocolAndMetadataStr.length - numFileActionsInMinVersion = versionToDeltaSharingFileActions - .getOrElseUpdate(minVersion, ArrayBuffer[model.DeltaSharingFileAction]()) - .size - - // Write metadata to the delta log json file. - versionToMetadata.foreach { - case (version, metadata) => - if (version != minVersion) { - val metadataStr = metadata.deltaMetadata.json + "\n" - versionToJsonLogBuilderMap.getOrElseUpdate( - version, - ArrayBuffer[String]() - ) += metadataStr - versionToJsonLogSize(version) += metadataStr.length - } - } - // Write file actions to the delta log json file. - var previousIdOpt: Option[String] = None - versionToDeltaSharingFileActions.foreach { - case (version, actions) => - previousIdOpt = None - actions.toSeq.sortWith(deltaSharingFileActionIncreaseOrderFunc).foreach { fileAction => - assert( - // Using > instead of >= because there can be a removeFile and addFile pointing to the - // same parquet file which result in the same file id, since id is a hash of file path. - // This is ok because eventually it can read data out of the correct parquet file. - !previousIdOpt.exists(_ > fileAction.id), - s"fileActions must be in increasing order by id: ${previousIdOpt} is not smaller than" + - s" ${fileAction.id}, in version:$version." - ) - previousIdOpt = Some(fileAction.id) - - // 1. build it to url mapping - idToUrl(fileAction.id) = fileAction.path - if (requiresIdToUrlForDV(fileAction.getDeletionVectorOpt)) { - idToUrl(fileAction.deletionVectorFileId) = - fileAction.getDeletionVectorOpt.get.pathOrInlineDv - } - - // 2. prepare json log content. - versionToTimestampMap.getOrElseUpdate(version, fileAction.timestamp) - val actionJsonStr = getActionWithDeltaSharingPath(fileAction, customTablePath) + "\n" - versionToJsonLogBuilderMap.getOrElseUpdate( - version, - ArrayBuffer[String]() - ) += actionJsonStr - versionToJsonLogSize(version) += actionJsonStr.length - - // 3. process expiration timestamp - if (fileAction.expirationTimestamp != null) { - minUrlExpirationTimestamp = minUrlExpirationTimestamp - .filter(_ < fileAction.expirationTimestamp) - .orElse(Some(fileAction.expirationTimestamp)) - } - } - } - - val encodedTablePath = DeltaSharingLogFileSystem.encode(customTablePath) - val deltaLogPath = s"${encodedTablePath.toString}/_delta_log" - val fileSizeTsSeq = Seq.newBuilder[DeltaSharingLogFileStatus] - - if (minVersion > 0) { - // If the minVersion is not 0 in the response, then prepare checkpoint at minVersion - 1: - // need to prepare two files: 1) (minVersion-1).checkpoint.parquet 2) _last_checkpoint - val checkpointVersion = minVersion - 1 - - // 1) store the checkpoint byte array in BlockManager for future read. - val checkpointParquetFileName = - FileNames.checkpointFileSingular(new Path(deltaLogPath), checkpointVersion).toString - fileSizeTsSeq += DeltaSharingLogFileStatus( - path = checkpointParquetFileName, - size = FAKE_CHECKPOINT_BYTE_ARRAY.size, - modificationTime = 0L - ) - - // 2) Prepare the content for _last_checkpoint - val lastCheckpointContent = - s"""{"version":${checkpointVersion},"size":${FAKE_CHECKPOINT_BYTE_ARRAY.size}}""" - val lastCheckpointPath = new Path(deltaLogPath, "_last_checkpoint").toString - fileSizeTsSeq += DeltaSharingLogFileStatus( - path = lastCheckpointPath, - size = lastCheckpointContent.length, - modificationTime = 0L - ) - DeltaSharingUtils.overrideSingleBlock[String]( - blockId = getDeltaSharingLogBlockId(lastCheckpointPath), - value = lastCheckpointContent - ) - } - - for (version <- minVersion to maxVersion) { - val jsonFilePath = FileNames.deltaFile(new Path(deltaLogPath), version).toString - DeltaSharingUtils.overrideIteratorBlock[String]( - getDeltaSharingLogBlockId(jsonFilePath), - versionToJsonLogBuilderMap.getOrElse(version, Seq.empty).toIterator - ) - fileSizeTsSeq += DeltaSharingLogFileStatus( - path = jsonFilePath, - size = versionToJsonLogSize.getOrElse(version, 0), - modificationTime = versionToTimestampMap.get(version).getOrElse(0L) - ) - } - - DeltaSharingUtils.overrideIteratorBlock[DeltaSharingLogFileStatus]( - getDeltaSharingLogBlockId(deltaLogPath), - fileSizeTsSeq.result().toIterator - ) - logInfo( - s"It takes ${(System.currentTimeMillis() - startTime) / 1000.0}s to construct delta log" + - s"for $customTablePath from $minVersion to $maxVersion, with ${idToUrl.toMap.size} urls." - ) - ConstructedDeltaLogMetadata( - idToUrl = idToUrl.toMap, - minUrlExpirationTimestamp = minUrlExpirationTimestamp, - numFileActionsInMinVersionOpt = Some(numFileActionsInMinVersion), - minVersion = minVersion, - maxVersion = maxVersion - ) - } /** Set the modificationTime to zero, this is to align with the time returned from * DeltaSharingFileSystem.getFileStatus @@ -804,39 +477,6 @@ private[sharing] object DeltaSharingLogFileSystem extends Logging { ) } - // Create a delta log directory with protocol and metadata at version 0. - // Used by DeltaSharingSource to initialize a DeltaLog class, which is then used to initialize - // a DeltaSource class, also the metadata id will be used for schemaTrackingLocation. - // There are no data files in the delta log because the DeltaSource class is initialized before - // any rpcs to the delta sharing server, so no data files are available yet. - def constructDeltaLogWithMetadataAtVersionZero( - customTablePath: String, - deltaSharingTableMetadata: DeltaSharingTableMetadata): Unit = { - val encodedTablePath = DeltaSharingLogFileSystem.encode(customTablePath) - val deltaLogPath = s"${encodedTablePath.toString}/_delta_log" - - // Always use 0.json for snapshot queries. - val jsonLogStr = deltaSharingTableMetadata.protocol.deltaProtocol.json + "\n" + - deltaSharingTableMetadata.metadata.deltaMetadata.json + "\n" - - val jsonFilePath = FileNames.deltaFile(new Path(deltaLogPath), 0).toString - DeltaSharingUtils.overrideIteratorBlock[String]( - getDeltaSharingLogBlockId(jsonFilePath), - Seq(jsonLogStr).toIterator - ) - - val fileStatusSeq = Seq( - DeltaSharingLogFileStatus( - path = jsonFilePath, - size = jsonLogStr.length, - modificationTime = 0L - ) - ) - DeltaSharingUtils.overrideIteratorBlock[DeltaSharingLogFileStatus]( - getDeltaSharingLogBlockId(deltaLogPath), - fileStatusSeq.toIterator - ) - } } /** diff --git a/sharing/src/main/scala/io/delta/sharing/spark/DeltaSharingUtils.scala b/sharing/src/main/scala/io/delta/sharing/spark/DeltaSharingUtils.scala index 1974289c298..4e01408d294 100644 --- a/sharing/src/main/scala/io/delta/sharing/spark/DeltaSharingUtils.scala +++ b/sharing/src/main/scala/io/delta/sharing/spark/DeltaSharingUtils.scala @@ -44,11 +44,6 @@ import org.apache.spark.storage.{BlockId, StorageLevel} object DeltaSharingUtils extends Logging { - val STREAMING_SUPPORTED_READER_FEATURES: Seq[String] = - Seq(DeletionVectorsTableFeature.name, ColumnMappingTableFeature.name) - val SUPPORTED_READER_FEATURES: Seq[String] = - Seq(DeletionVectorsTableFeature.name, ColumnMappingTableFeature.name) - // The prefix will be used for block ids of all blocks that store the delta log in BlockManager. // It's used to ensure delta sharing queries don't mess up with blocks with other applications. val DELTA_SHARING_BLOCK_ID_PREFIX = "test_delta-sharing" @@ -63,66 +58,6 @@ object DeltaSharingUtils extends Logging { metadata: model.DeltaSharingMetadata ) - // A wrapper function for streaming query to get the latest version/protocol/metadata of the - // shared table. - def getDeltaSharingTableMetadata( - client: DeltaSharingClient, - table: Table): DeltaSharingTableMetadata = { - val deltaTableMetadata = client.getMetadata(table) - getDeltaSharingTableMetadata(table, deltaTableMetadata) - } - - def queryDeltaTableMetadata( - client: DeltaSharingClient, - table: Table, - versionAsOf: Option[Long] = None, - timestampAsOf: Option[String] = None): DeltaTableMetadata = { - val deltaTableMetadata = client.getMetadata(table, versionAsOf, timestampAsOf) - logInfo( - s"getMetadata returned in ${deltaTableMetadata.respondedFormat} format for table " + - s"$table with v_${versionAsOf.map(_.toString).getOrElse("None")} " + - s"t_${timestampAsOf.getOrElse("None")} from delta sharing server." - ) - deltaTableMetadata - } - - /** - * parse the protocol and metadata from rpc response for getMetadata. - */ - def getDeltaSharingTableMetadata( - table: Table, - deltaTableMetadata: DeltaTableMetadata): DeltaSharingTableMetadata = { - - var metadataOption: Option[model.DeltaSharingMetadata] = None - var protocolOption: Option[model.DeltaSharingProtocol] = None - - deltaTableMetadata.lines - .map( - JsonUtils.fromJson[model.DeltaSharingSingleAction](_).unwrap - ) - .foreach { - case m: model.DeltaSharingMetadata => metadataOption = Some(m) - case p: model.DeltaSharingProtocol => protocolOption = Some(p) - case _ => // ignore other lines - } - - DeltaSharingTableMetadata( - version = deltaTableMetadata.version, - protocol = protocolOption.getOrElse { - throw new IllegalStateException( - s"Failed to get Protocol for ${table.toString}, " + - s"response from server:${deltaTableMetadata.lines}." - ) - }, - metadata = metadataOption.getOrElse { - throw new IllegalStateException( - s"Failed to get Metadata for ${table.toString}, " + - s"response from server:${deltaTableMetadata.lines}." - ) - } - ) - } - private def getTableRefreshResult(tableFiles: DeltaTableFiles): TableRefreshResult = { var minUrlExpiration: Option[Long] = None val idToUrl = tableFiles.lines @@ -173,44 +108,6 @@ object DeltaSharingUtils extends Logging { } } - /** - * Get the refresher function for a delta sharing table who calls client.getCDFFiles with the - * provided parameters. - * - * @return A refresher function used by the CachedTableManager to refresh urls. - */ - def getRefresherForGetCDFFiles( - client: DeltaSharingClient, - table: Table, - cdfOptions: Map[String, String]): RefresherFunction = { (_: Option[String]) => - { - val tableFiles = client.getCDFFiles( - table = table, - cdfOptions = cdfOptions, - includeHistoricalMetadata = true - ) - getTableRefreshResult(tableFiles) - } - } - - /** - * Get the refresher function for a delta sharing table who calls client.getFiles with the - * provided parameters. - * - * @return A refresher function used by the CachedTableManager to refresh urls. - */ - def getRefresherForGetFilesWithStartingVersion( - client: DeltaSharingClient, - table: Table, - startingVersion: Long, - endingVersion: Option[Long]): RefresherFunction = { (_: Option[String]) => - { - val tableFiles = client - .getFiles(table = table, startingVersion = startingVersion, endingVersion = endingVersion) - getTableRefreshResult(tableFiles) - } - } - def overrideSingleBlock[T: ClassTag](blockId: BlockId, value: T): Unit = { assert( blockId.name.startsWith(DELTA_SHARING_BLOCK_ID_PREFIX), @@ -239,34 +136,6 @@ object DeltaSharingUtils extends Logging { ) } - // A helper function used by DeltaSharingSource and DeltaSharingDataSource to get - // SnapshotDescriptor used for delta sharing streaming. - def getDeltaLogAndSnapshotDescriptor( - spark: SparkSession, - deltaSharingTableMetadata: DeltaSharingTableMetadata, - customTablePathWithUUIDSuffix: String): (DeltaLog, SnapshotDescriptor) = { - // Create a delta log with metadata at version 0. - // Used by DeltaSharingSource to initialize a DeltaLog class, which is then used to initialize - // a DeltaSource class, also the metadata id will be used for schemaTrackingLocation. - DeltaSharingLogFileSystem.constructDeltaLogWithMetadataAtVersionZero( - customTablePathWithUUIDSuffix, - deltaSharingTableMetadata - ) - val tablePath = DeltaSharingLogFileSystem.encode(customTablePathWithUUIDSuffix).toString - val localDeltaLog = DeltaLog.forTable(spark, tablePath) - ( - localDeltaLog, - new SnapshotDescriptor { - val deltaLog: DeltaLog = localDeltaLog - val metadata: Metadata = deltaSharingTableMetadata.metadata.deltaMetadata - val protocol: Protocol = deltaSharingTableMetadata.protocol.deltaProtocol - val version = deltaSharingTableMetadata.version - val numOfFilesIfKnown = None - val sizeInBytesIfKnown = None - } - ) - } - // Get a query hash id based on the query parameters: time travel options and filters. // The id concatenated with table name and used in local DeltaLog and CachedTableManager. // This is to uniquely identify the delta sharing table used twice in the same query but with @@ -282,33 +151,12 @@ object DeltaSharingUtils extends Logging { Hashing.sha256().hashString(fullQueryString, UTF_8).toString } - // Get a query hash id based on the query parameters: cdfOptions. - // The id concatenated with table name and used in local DeltaLoc and CachedTableManager. - // This is to uniquely identify the delta sharing table used twice in the same query but with - // different query parameters, so we can differentiate their delta log and entries in the - // CachedTableManager. - private[sharing] def getQueryParamsHashId(cdfOptions: Map[String, String]): String = { - Hashing.sha256().hashString(cdfOptions.toString, UTF_8).toString - } - // Concatenate table path with an id as a suffix, to uniquely identify a delta sharing table and // its corresponding delta log in a query. private[sharing] def getTablePathWithIdSuffix(customTablePath: String, id: String): String = { s"${customTablePath}_${id}" } - // Get a unique string composed of a formatted timestamp and an uuid. - // Used as a suffix for the table name and its delta log path of a delta sharing table in a - // streaming job, to avoid overwriting the delta log from multiple references of the same delta - // sharing table in one streaming job. - private[sharing] def getFormattedTimestampWithUUID(): String = { - val dateFormat = new SimpleDateFormat("yyyyMMdd_HHmmss") - dateFormat.setTimeZone(TimeZone.getTimeZone("UTC")) - val formattedDateTime = dateFormat.format(System.currentTimeMillis()) - val uuid = UUID.randomUUID().toString().split('-').head - s"${formattedDateTime}_${uuid}" - } - private def removeBlockForJsonLogIfExists(blockId: BlockId): Unit = { val blockManager = SparkEnv.get.blockManager blockManager.getMatchingBlockIds(_.name == blockId.name).foreach { b => diff --git a/sharing/src/test/scala/io/delta/sharing/spark/DeltaFormatSharingSourceSuite.scala b/sharing/src/test/scala/io/delta/sharing/spark/DeltaFormatSharingSourceSuite.scala deleted file mode 100644 index 12860b29c1d..00000000000 --- a/sharing/src/test/scala/io/delta/sharing/spark/DeltaFormatSharingSourceSuite.scala +++ /dev/null @@ -1,867 +0,0 @@ -/* - * Copyright (2021) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.delta.sharing.spark - -import org.apache.spark.sql.delta.DeltaLog -import org.apache.spark.sql.delta.DeltaOptions.{ - IGNORE_CHANGES_OPTION, - IGNORE_DELETES_OPTION, - SKIP_CHANGE_COMMITS_OPTION -} -import org.apache.spark.sql.delta.test.DeltaSQLCommandTest -import io.delta.sharing.client.DeltaSharingRestClient -import io.delta.sharing.client.model.{Table => DeltaSharingTable} -import org.apache.hadoop.fs.Path - -import org.apache.spark.SparkEnv -import org.apache.spark.sql.Row -import org.apache.spark.sql.delta.sharing.DeltaSharingTestSparkUtils -import org.apache.spark.sql.functions.col -import org.apache.spark.sql.streaming.StreamTest -import org.apache.spark.sql.types.{ - DateType, - IntegerType, - LongType, - StringType, - StructType, - TimestampType -} - -class DeltaFormatSharingSourceSuite - extends StreamTest - with DeltaSQLCommandTest - with DeltaSharingTestSparkUtils - with DeltaSharingDataSourceDeltaTestUtils { - - import testImplicits._ - - private def getSource(parameters: Map[String, String]): DeltaFormatSharingSource = { - val options = new DeltaSharingOptions(parameters) - val path = options.options.getOrElse( - "path", - throw DeltaSharingErrors.pathNotSpecifiedException - ) - val parsedPath = DeltaSharingRestClient.parsePath(path) - val client = DeltaSharingRestClient( - profileFile = parsedPath.profileFile, - forStreaming = true, - responseFormat = "delta", - readerFeatures = DeltaSharingUtils.STREAMING_SUPPORTED_READER_FEATURES.mkString(",") - ) - val dsTable = DeltaSharingTable( - share = parsedPath.share, - schema = parsedPath.schema, - name = parsedPath.table - ) - DeltaFormatSharingSource( - spark = spark, - client = client, - table = dsTable, - options = options, - parameters = parameters, - sqlConf = sqlContext.sparkSession.sessionState.conf, - metadataPath = "" - ) - } - - private def assertBlocksAreCleanedUp(): Unit = { - val blockManager = SparkEnv.get.blockManager - val matchingBlockIds = blockManager.getMatchingBlockIds( - _.name.startsWith(DeltaSharingLogFileSystem.DELTA_SHARING_LOG_BLOCK_ID_PREFIX) - ) - assert(matchingBlockIds.isEmpty, "delta sharing blocks are not cleaned up.") - } - - test("DeltaFormatSharingSource able to get schema") { - withTempDir { tempDir => - val deltaTableName = "delta_table_schema" - withTable(deltaTableName) { - createTable(deltaTableName) - val sharedTableName = "shared_table_schema" - prepareMockedClientMetadata(deltaTableName, sharedTableName) - prepareMockedClientGetTableVersion(deltaTableName, sharedTableName) - - val profileFile = prepareProfileFile(tempDir) - withSQLConf(getDeltaSharingClassesSQLConf.toSeq: _*) { - val deltaSharingSource = getSource( - Map("path" -> s"${profileFile.getCanonicalPath}#share1.default.$sharedTableName") - ) - val expectedSchema: StructType = new StructType() - .add("c1", IntegerType) - .add("c2", StringType) - .add("c3", DateType) - .add("c4", TimestampType) - assert(deltaSharingSource.schema == expectedSchema) - - // CDF schema - val cdfDeltaSharingSource = getSource( - Map( - "path" -> s"${profileFile.getCanonicalPath}#share1.default.$sharedTableName", - "readChangeFeed" -> "true" - ) - ) - val expectedCdfSchema: StructType = expectedSchema - .copy() - .add("_change_type", StringType) - .add("_commit_version", LongType) - .add("_commit_timestamp", TimestampType) - assert(cdfDeltaSharingSource.schema == expectedCdfSchema) - } - } - } - } - - test("DeltaFormatSharingSource do not support cdc") { - withTempDir { tempDir => - val sharedTableName = "shared_streaming_table_nocdc" - val profileFile = prepareProfileFile(tempDir) - withSQLConf(getDeltaSharingClassesSQLConf.toSeq: _*) { - val tablePath = profileFile.getCanonicalPath + s"#share1.default.$sharedTableName" - val e = intercept[Exception] { - val df = spark.readStream - .format("deltaSharing") - .option("responseFormat", "delta") - .option("readChangeFeed", "true") - .load(tablePath) - testStream(df)( - AssertOnQuery { q => - q.processAllAvailable(); true - } - ) - } - assert(e.getMessage.contains("Delta sharing cdc streaming is not supported")) - } - } - } - - test("DeltaFormatSharingSource simple query works") { - withTempDir { tempDir => - val deltaTableName = "delta_table_simple" - withTable(deltaTableName) { - sql(s""" - |CREATE TABLE $deltaTableName (value STRING) - |USING DELTA - |""".stripMargin) - - val sharedTableName = "shared_streaming_table_simple" - prepareMockedClientMetadata(deltaTableName, sharedTableName) - - val profileFile = prepareProfileFile(tempDir) - withSQLConf(getDeltaSharingClassesSQLConf.toSeq: _*) { - val tablePath = profileFile.getCanonicalPath + s"#share1.default.$sharedTableName" - - def InsertToDeltaTable(values: String): Unit = { - sql(s"INSERT INTO $deltaTableName VALUES $values") - } - - InsertToDeltaTable("""("keep1"), ("keep2"), ("drop3")""") - prepareMockedClientAndFileSystemResult(deltaTableName, sharedTableName, Some(1L)) - prepareMockedClientGetTableVersion(deltaTableName, sharedTableName) - - val df = spark.readStream - .format("deltaSharing") - .option("responseFormat", "delta") - .load(tablePath) - .filter($"value" contains "keep") - - testStream(df)( - AssertOnQuery { q => - q.processAllAvailable(); true - }, - CheckAnswer("keep1", "keep2"), - StopStream - ) - } - } - } - } - - test("restart works sharing") { - withTempDirs { (inputDir, outputDir, checkpointDir) => - val deltaTableName = "delta_table_restart" - withTable(deltaTableName) { - createTableForStreaming(deltaTableName) - val sharedTableName = "shared_streaming_table_restart" - prepareMockedClientMetadata(deltaTableName, sharedTableName) - val profileFile = prepareProfileFile(inputDir) - val tablePath = profileFile.getCanonicalPath + s"#share1.default.$sharedTableName" - - withSQLConf(getDeltaSharingClassesSQLConf.toSeq: _*) { - def InsertToDeltaTable(values: String): Unit = { - sql(s"INSERT INTO $deltaTableName VALUES $values") - } - - // TODO: check testStream() function helper - def processAllAvailableInStream(): Unit = { - val q = spark.readStream - .format("deltaSharing") - .option("responseFormat", "delta") - .load(tablePath) - .filter($"value" contains "keep") - .writeStream - .format("delta") - .option("checkpointLocation", checkpointDir.toString) - .start(outputDir.toString) - - try { - q.processAllAvailable() - } finally { - q.stop() - } - } - - // Able to stream snapshot at version 1. - InsertToDeltaTable("""("keep1"), ("keep2"), ("drop1")""") - prepareMockedClientAndFileSystemResult( - deltaTable = deltaTableName, - sharedTable = sharedTableName, - versionAsOf = Some(1L) - ) - prepareMockedClientGetTableVersion(deltaTableName, sharedTableName) - processAllAvailableInStream() - checkAnswer( - spark.read.format("delta").load(outputDir.getCanonicalPath), - Seq("keep1", "keep2").toDF() - ) - - // No new data, so restart will not process any new data. - processAllAvailableInStream() - checkAnswer( - spark.read.format("delta").load(outputDir.getCanonicalPath), - Seq("keep1", "keep2").toDF() - ) - - // Able to stream new data at version 2. - InsertToDeltaTable("""("keep3"), ("keep4"), ("drop2")""") - prepareMockedClientGetTableVersion(deltaTableName, sharedTableName) - prepareMockedClientAndFileSystemResultForStreaming( - deltaTableName, - sharedTableName, - 2, - 2 - ) - processAllAvailableInStream() - checkAnswer( - spark.read.format("delta").load(outputDir.getCanonicalPath), - Seq("keep1", "keep2", "keep3", "keep4").toDF() - ) - - sql(s"""OPTIMIZE $deltaTableName""") - prepareMockedClientGetTableVersion(deltaTableName, sharedTableName) - prepareMockedClientAndFileSystemResultForStreaming( - deltaTableName, - sharedTableName, - 2, - 3 - ) - // Optimize doesn't produce new data, so restart will not process any new data. - processAllAvailableInStream() - checkAnswer( - spark.read.format("delta").load(outputDir.getCanonicalPath), - Seq("keep1", "keep2", "keep3", "keep4").toDF() - ) - - // Able to stream new data at version 3. - InsertToDeltaTable("""("keep5"), ("keep6"), ("drop3")""") - prepareMockedClientGetTableVersion(deltaTableName, sharedTableName) - prepareMockedClientAndFileSystemResultForStreaming( - deltaTableName, - sharedTableName, - 3, - 4 - ) - - processAllAvailableInStream() - checkAnswer( - spark.read.format("delta").load(outputDir.getCanonicalPath), - Seq("keep1", "keep2", "keep3", "keep4", "keep5", "keep6").toDF() - ) - assertBlocksAreCleanedUp() - } - } - } - } - - test("streaming works with deletes on basic table") { - withTempDir { inputDir => - val deltaTableName = "delta_table_deletes" - withTable(deltaTableName) { - createTableForStreaming(deltaTableName) - val sharedTableName = "shared_streaming_table_deletes" - prepareMockedClientMetadata(deltaTableName, sharedTableName) - val profileFile = prepareProfileFile(inputDir) - val tablePath = profileFile.getCanonicalPath + s"#share1.default.$sharedTableName" - - withSQLConf(getDeltaSharingClassesSQLConf.toSeq: _*) { - def InsertToDeltaTable(values: String): Unit = { - sql(s"INSERT INTO $deltaTableName VALUES $values") - } - - def processAllAvailableInStream( - sourceOptions: Map[String, String], - expectations: StreamAction*): Unit = { - val df = spark.readStream - .format("deltaSharing") - .options(sourceOptions) - .load(tablePath) - - val base = Seq(StartStream(), ProcessAllAvailable()) - testStream(df)((base ++ expectations): _*) - } - - // Insert at version 1 and 2. - InsertToDeltaTable("""("keep1")""") - InsertToDeltaTable("""("keep2")""") - // delete at version 3. - sql(s"""DELETE FROM $deltaTableName WHERE value = "keep1" """) - // update at version 4. - sql(s"""UPDATE $deltaTableName SET value = "keep3" WHERE value = "keep2" """) - - prepareMockedClientAndFileSystemResult( - deltaTable = deltaTableName, - sharedTable = sharedTableName, - versionAsOf = Some(4L) - ) - prepareMockedClientGetTableVersion(deltaTableName, sharedTableName) - processAllAvailableInStream( - Map("responseFormat" -> "delta"), - CheckAnswer("keep3") - ) - - prepareMockedClientAndFileSystemResultForStreaming( - deltaTableName, - sharedTableName, - 0, - 4 - ) - - // The streaming query will fail because changes detected in version 4. - // This is the original delta behavior. - val e = intercept[Exception] { - processAllAvailableInStream( - Map("responseFormat" -> "delta", "startingVersion" -> "0") - ) - } - for (msg <- Seq( - "Detected", - "not supported", - "true" - )) { - assert(e.getMessage.contains(msg)) - } - - // The streaming query will fail because changes detected in version 4. - // This is the original delta behavior. - val e2 = intercept[Exception] { - processAllAvailableInStream( - Map( - "responseFormat" -> "delta", - "startingVersion" -> "0", - IGNORE_DELETES_OPTION -> "true" - ) - ) - } - for (msg <- Seq( - "Detected", - "not supported", - "true" - )) { - assert(e2.getMessage.contains(msg)) - } - - // The streaming query will succeed because ignoreChanges helps to ignore the updates, but - // added updated data "keep3". - processAllAvailableInStream( - Map( - "responseFormat" -> "delta", - "startingVersion" -> "0", - IGNORE_CHANGES_OPTION -> "true" - ), - CheckAnswer("keep1", "keep2", "keep3") - ) - - // The streaming query will succeed because skipChangeCommits helps to ignore the whole - // commit with data update, so updated data is not produced either. - processAllAvailableInStream( - Map( - "responseFormat" -> "delta", - "startingVersion" -> "0", - SKIP_CHANGE_COMMITS_OPTION -> "true" - ), - CheckAnswer("keep1", "keep2") - ) - assertBlocksAreCleanedUp() - } - } - } - } - - test("streaming works with DV") { - withTempDir { inputDir => - val deltaTableName = "delta_table_dv" - withTable(deltaTableName) { - createSimpleTable(deltaTableName, enableCdf = false) - spark.sql( - s"ALTER TABLE $deltaTableName SET TBLPROPERTIES('delta.enableDeletionVectors' = true)" - ) - val sharedTableName = "shared_streaming_table_dv" - prepareMockedClientMetadata(deltaTableName, sharedTableName) - val profileFile = prepareProfileFile(inputDir) - val tablePath = profileFile.getCanonicalPath + s"#share1.default.$sharedTableName" - - withSQLConf(getDeltaSharingClassesSQLConf.toSeq: _*) { - def InsertToDeltaTable(values: String): Unit = { - sql(s"INSERT INTO $deltaTableName VALUES $values") - } - - def processAllAvailableInStream( - sourceOptions: Map[String, String], - expectations: StreamAction*): Unit = { - val df = spark.readStream - .format("deltaSharing") - .options(sourceOptions) - .load(tablePath) - .filter($"c2" contains "keep") - .select("c1") - - val base = Seq(StartStream(), ProcessAllAvailable()) - testStream(df)((base ++ expectations): _*) - } - - // Insert at version 2. - InsertToDeltaTable("""(1, "keep1"),(2, "keep1"),(3, "keep1"),(1,"drop1")""") - // delete at version 3. - sql(s"""DELETE FROM $deltaTableName WHERE c1 >= 2 """) - - prepareMockedClientAndFileSystemResult( - deltaTable = deltaTableName, - sharedTable = sharedTableName, - versionAsOf = Some(3L) - ) - prepareMockedClientGetTableVersion(deltaTableName, sharedTableName) - processAllAvailableInStream( - Map("responseFormat" -> "delta"), - CheckAnswer(1) - ) - - prepareMockedClientAndFileSystemResultForStreaming( - deltaTableName, - sharedTableName, - startingVersion = 0, - endingVersion = 3, - assertDVExists = true - ) - - // The streaming query will fail because deletes detected in version 3. And there are no - // options provided to ignore the deletion. - val e = intercept[Exception] { - processAllAvailableInStream( - Map("responseFormat" -> "delta", "startingVersion" -> "0") - ) - } - for (msg <- Seq( - "Detected a data update", - "not supported", - SKIP_CHANGE_COMMITS_OPTION, - "true" - )) { - assert(e.getMessage.contains(msg)) - } - - // The streaming query will fail because deletes detected in version 3, and it's - // recognized as updates and ignoreDeletes doesn't help. This is the original delta - // behavior. - val e2 = intercept[Exception] { - processAllAvailableInStream( - Map( - "responseFormat" -> "delta", - "startingVersion" -> "0", - IGNORE_DELETES_OPTION -> "true" - ) - ) - } - for (msg <- Seq( - "Detected a data update", - "not supported", - SKIP_CHANGE_COMMITS_OPTION, - "true" - )) { - assert(e2.getMessage.contains(msg)) - } - - // The streaming query will succeed because ignoreChanges helps to ignore the delete, but - // added duplicated data 1. - processAllAvailableInStream( - Map( - "responseFormat" -> "delta", - "startingVersion" -> "0", - IGNORE_CHANGES_OPTION -> "true" - ), - CheckAnswer(1, 2, 3, 1) - ) - - // The streaming query will succeed because skipChangeCommits helps to ignore the whole - // commit with data update, so no duplicated data is produced either. - processAllAvailableInStream( - Map( - "responseFormat" -> "delta", - "startingVersion" -> "0", - SKIP_CHANGE_COMMITS_OPTION -> "true" - ), - CheckAnswer(1, 2, 3) - ) - assertBlocksAreCleanedUp() - } - } - } - } - - test("startingVersion works") { - withTempDirs { (inputDir, outputDir, checkpointDir) => - val deltaTableName = "delta_table_startVersion" - withTable(deltaTableName) { - createTableForStreaming(deltaTableName) - val sharedTableName = "shared_streaming_table_startVersion" - prepareMockedClientMetadata(deltaTableName, sharedTableName) - val profileFile = prepareProfileFile(inputDir) - val tablePath = profileFile.getCanonicalPath + s"#share1.default.$sharedTableName" - - withSQLConf(getDeltaSharingClassesSQLConf.toSeq: _*) { - def InsertToDeltaTable(values: String): Unit = { - sql(s"INSERT INTO $deltaTableName VALUES $values") - } - - def processAllAvailableInStream(): Unit = { - val q = spark.readStream - .format("deltaSharing") - .option("responseFormat", "delta") - .option("startingVersion", 0) - .load(tablePath) - .filter($"value" contains "keep") - .writeStream - .format("delta") - .option("checkpointLocation", checkpointDir.toString) - .start(outputDir.toString) - - try { - q.processAllAvailable() - } finally { - q.stop() - } - } - - // Able to stream snapshot at version 1. - InsertToDeltaTable("""("keep1"), ("keep2"), ("drop1")""") - prepareMockedClientAndFileSystemResultForStreaming( - deltaTable = deltaTableName, - sharedTable = sharedTableName, - startingVersion = 0L, - endingVersion = 1L - ) - prepareMockedClientGetTableVersion(deltaTableName, sharedTableName) - processAllAvailableInStream() - checkAnswer( - spark.read.format("delta").load(outputDir.getCanonicalPath), - Seq("keep1", "keep2").toDF() - ) - - // No new data, so restart will not process any new data. - processAllAvailableInStream() - checkAnswer( - spark.read.format("delta").load(outputDir.getCanonicalPath), - Seq("keep1", "keep2").toDF() - ) - - // Able to stream new data at version 2. - InsertToDeltaTable("""("keep3"), ("keep4"), ("drop2")""") - prepareMockedClientGetTableVersion(deltaTableName, sharedTableName) - prepareMockedClientAndFileSystemResultForStreaming( - deltaTableName, - sharedTableName, - 0, - 2 - ) - processAllAvailableInStream() - checkAnswer( - spark.read.format("delta").load(outputDir.getCanonicalPath), - Seq("keep1", "keep2", "keep3", "keep4").toDF() - ) - - sql(s"""OPTIMIZE $deltaTableName""") - prepareMockedClientGetTableVersion(deltaTableName, sharedTableName) - prepareMockedClientAndFileSystemResultForStreaming( - deltaTableName, - sharedTableName, - 2, - 3 - ) - // Optimize doesn't produce new data, so restart will not process any new data. - processAllAvailableInStream() - checkAnswer( - spark.read.format("delta").load(outputDir.getCanonicalPath), - Seq("keep1", "keep2", "keep3", "keep4").toDF() - ) - - // No new data, so restart will not process any new data. It will ask for the last commit - // so that it can figure out that there's nothing to do. - prepareMockedClientAndFileSystemResultForStreaming( - deltaTableName, - sharedTableName, - 3, - 3 - ) - processAllAvailableInStream() - checkAnswer( - spark.read.format("delta").load(outputDir.getCanonicalPath), - Seq("keep1", "keep2", "keep3", "keep4").toDF() - ) - - // Able to stream new data at version 3. - InsertToDeltaTable("""("keep5"), ("keep6"), ("drop3")""") - prepareMockedClientGetTableVersion(deltaTableName, sharedTableName) - prepareMockedClientAndFileSystemResultForStreaming( - deltaTableName, - sharedTableName, - 3, - 4 - ) - processAllAvailableInStream() - checkAnswer( - spark.read.format("delta").load(outputDir.getCanonicalPath), - Seq("keep1", "keep2", "keep3", "keep4", "keep5", "keep6").toDF() - ) - - // No new data, so restart will not process any new data. It will ask for the last commit - // so that it can figure out that there's nothing to do. - prepareMockedClientAndFileSystemResultForStreaming( - deltaTableName, - sharedTableName, - 4, - 4 - ) - processAllAvailableInStream() - checkAnswer( - spark.read.format("delta").load(outputDir.getCanonicalPath), - Seq("keep1", "keep2", "keep3", "keep4", "keep5", "keep6").toDF() - ) - assertBlocksAreCleanedUp() - } - } - } - } - - test("files are in a stable order for streaming") { - // This test function is to check that DeltaSharingLogFileSystem puts the files in the delta log - // in a stable order for each commit, regardless of the returning order from the server, so that - // the DeltaSource can produce a stable file index. - // We are using maxBytesPerTrigger which causes the streaming to stop in the middle of a commit - // to be able to test this behavior. - withTempDirs { (inputDir, outputDir, checkpointDir) => - withTempDirs { (_, outputDir2, checkpointDir2) => - val deltaTableName = "delta_table_order" - withTable(deltaTableName) { - createSimpleTable(deltaTableName, enableCdf = false) - val sharedTableName = "shared_streaming_table_order" - prepareMockedClientMetadata(deltaTableName, sharedTableName) - val profileFile = prepareProfileFile(inputDir) - val tablePath = profileFile.getCanonicalPath + s"#share1.default.$sharedTableName" - - def InsertToDeltaTable(values: String): Unit = { - sql(s"INSERT INTO $deltaTableName VALUES $values") - } - - // Able to stream snapshot at version 1. - InsertToDeltaTable("""(1, "one"), (2, "two"), (3, "three")""") - - withSQLConf(getDeltaSharingClassesSQLConf.toSeq: _*) { - def processAllAvailableInStream( - outputDirStr: String, - checkpointDirStr: String): Unit = { - val q = spark.readStream - .format("deltaSharing") - .option("responseFormat", "delta") - .option("maxBytesPerTrigger", "1b") - .load(tablePath) - .writeStream - .format("delta") - .option("checkpointLocation", checkpointDirStr) - .start(outputDirStr) - - try { - q.processAllAvailable() - val progress = q.recentProgress.filter(_.numInputRows != 0) - assert(progress.length === 3) - progress.foreach { p => - assert(p.numInputRows === 1) - } - } finally { - q.stop() - } - } - - // First output, without reverseFileOrder - prepareMockedClientAndFileSystemResult( - deltaTable = deltaTableName, - sharedTable = sharedTableName, - versionAsOf = Some(1L) - ) - prepareMockedClientGetTableVersion(deltaTableName, sharedTableName) - processAllAvailableInStream(outputDir.toString, checkpointDir.toString) - checkAnswer( - spark.read.format("delta").load(outputDir.getCanonicalPath), - Seq((1, "one"), (2, "two"), (3, "three")).toDF() - ) - - // Second output, with reverseFileOrder = true - prepareMockedClientAndFileSystemResult( - deltaTable = deltaTableName, - sharedTable = sharedTableName, - versionAsOf = Some(1L), - reverseFileOrder = true - ) - prepareMockedClientGetTableVersion(deltaTableName, sharedTableName) - processAllAvailableInStream(outputDir2.toString, checkpointDir2.toString) - checkAnswer( - spark.read.format("delta").load(outputDir2.getCanonicalPath), - Seq((1, "one"), (2, "two"), (3, "three")).toDF() - ) - - // Check each version of the two output are the same, which means the files are sorted - // by DeltaSharingLogFileSystem, and are processed in a deterministic order by the - // DeltaSource. - val deltaLog = DeltaLog.forTable(spark, new Path(outputDir.toString)) - Seq(0, 1, 2).foreach { v => - val version = deltaLog.snapshot.version - v - val df1 = spark.read - .format("delta") - .option("versionAsOf", version) - .load(outputDir.getCanonicalPath) - val df2 = spark.read - .format("delta") - .option("versionAsOf", version) - .load(outputDir2.getCanonicalPath) - checkAnswer(df1, df2) - assert(df1.count() == (3 - v)) - } - assertBlocksAreCleanedUp() - } - } - } - } - } - - test("DeltaFormatSharingSource query with two delta sharing tables works") { - withTempDirs { (inputDir, outputDir, checkpointDir) => - val deltaTableName = "delta_table_two" - - def InsertToDeltaTable(values: String): Unit = { - sql(s"INSERT INTO $deltaTableName VALUES $values") - } - - withTable(deltaTableName) { - createSimpleTable(deltaTableName, enableCdf = false) - val sharedTableName = "shared_streaming_table_two" - prepareMockedClientMetadata(deltaTableName, sharedTableName) - - val profileFile = prepareProfileFile(inputDir) - val tablePath = profileFile.getCanonicalPath + s"#share1.default.$sharedTableName" - withSQLConf(getDeltaSharingClassesSQLConf.toSeq: _*) { - InsertToDeltaTable("""(1, "one"), (2, "one")""") - InsertToDeltaTable("""(1, "two"), (2, "two")""") - InsertToDeltaTable("""(1, "three"), (2, "three")""") - prepareMockedClientGetTableVersion(deltaTableName, sharedTableName) - prepareMockedClientAndFileSystemResult( - deltaTableName, - sharedTableName, - Some(3L) - ) - prepareMockedClientAndFileSystemResultForStreaming( - deltaTableName, - sharedTableName, - startingVersion = 1, - endingVersion = 3 - ) - - def processAllAvailableInStream(): Unit = { - val dfLatest = spark.readStream - .format("deltaSharing") - .option("responseFormat", "delta") - .load(tablePath) - val dfV1 = spark.readStream - .format("deltaSharing") - .option("responseFormat", "delta") - .option("startingVersion", 1) - .load(tablePath) - .select(col("c2"), col("c1").as("v1c1")) - .filter(col("v1c1") === 1) - - val q = dfLatest - .join(dfV1, "c2") - .writeStream - .format("delta") - .option("checkpointLocation", checkpointDir.toString) - .start(outputDir.toString) - - try { - q.processAllAvailable() - } finally { - q.stop() - } - } - - // c1 from dfLatest, c2 from dfLatest, c1 from dfV1 - var expected = Seq( - Row("one", 1, 1), - Row("one", 2, 1), - Row("two", 1, 1), - Row("two", 2, 1), - Row("three", 1, 1), - Row("three", 2, 1) - ) - processAllAvailableInStream() - checkAnswer( - spark.read.format("delta").load(outputDir.getCanonicalPath), - expected - ) - - InsertToDeltaTable("""(1, "four"), (2, "four")""") - prepareMockedClientGetTableVersion(deltaTableName, sharedTableName) - prepareMockedClientAndFileSystemResultForStreaming( - deltaTableName, - sharedTableName, - startingVersion = 4, - endingVersion = 4 - ) - prepareMockedClientAndFileSystemResultForStreaming( - deltaTableName, - sharedTableName, - startingVersion = 1, - endingVersion = 4 - ) - - expected = expected ++ Seq(Row("four", 1, 1), Row("four", 2, 1)) - processAllAvailableInStream() - checkAnswer( - spark.read.format("delta").load(outputDir.getCanonicalPath), - expected - ) - assertBlocksAreCleanedUp() - } - } - } - } -} diff --git a/sharing/src/test/scala/io/delta/sharing/spark/DeltaSharingCDFUtilsSuite.scala b/sharing/src/test/scala/io/delta/sharing/spark/DeltaSharingCDFUtilsSuite.scala deleted file mode 100644 index 687d9d10e94..00000000000 --- a/sharing/src/test/scala/io/delta/sharing/spark/DeltaSharingCDFUtilsSuite.scala +++ /dev/null @@ -1,243 +0,0 @@ -/* - * Copyright (2021) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.delta.sharing.spark - -import java.io.File - -import org.apache.spark.sql.delta.test.DeltaSQLCommandTest -import io.delta.sharing.client.{ - DeltaSharingClient, - DeltaSharingProfileProvider, - DeltaSharingRestClient -} -import io.delta.sharing.client.model.{DeltaTableFiles, DeltaTableMetadata, Table} -import org.apache.commons.io.FileUtils -import org.apache.hadoop.fs.Path - -import org.apache.spark.{SparkConf, SparkEnv} -import org.apache.spark.delta.sharing.{PreSignedUrlCache, PreSignedUrlFetcher} -import org.apache.spark.sql.{QueryTest, SparkSession} -import org.apache.spark.sql.delta.sharing.DeltaSharingTestSparkUtils -import org.apache.spark.sql.test.{SharedSparkSession} - -private object CDFTesTUtils { - val paths = Seq("http://path1", "http://path2") - - val SparkConfForReturnExpTime = "spark.delta.sharing.fileindexsuite.returnexptime" - - // 10 seconds - val expirationTimeMs = 10000 - - def getExpirationTimestampStr(returnExpTime: Boolean): String = { - if (returnExpTime) { - s""""expirationTimestamp":${System.currentTimeMillis() + expirationTimeMs},""" - } else { - "" - } - } - - // scalastyle:off line.size.limit - val fileStr1Id = "11d9b72771a72f178a6f2839f7f08528" - val metaDataStr = - """{"metaData":{"size":809,"deltaMetadata":{"id":"testId","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"c1\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"c2\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":["c2"],"configuration":{"delta.enableChangeDataFeed":"true"},"createdTime":1691734718560}}}""" - def getAddFileStr1(path: String, returnExpTime: Boolean = false): String = { - s"""{"file":{"id":"11d9b72771a72f178a6f2839f7f08528",${getExpirationTimestampStr( - returnExpTime - )}"deltaSingleAction":{"add":{"path":"${path}",""" + """"partitionValues":{"c2":"one"},"size":809,"modificationTime":1691734726073,"dataChange":true,"stats":"{\"numRecords\":2,\"minValues\":{\"c1\":1,\"c2\":\"one\"},\"maxValues\":{\"c1\":2,\"c2\":\"one\"},\"nullCount\":{\"c1\":0,\"c2\":0}}","tags":{"INSERTION_TIME":"1691734726073000","MIN_INSERTION_TIME":"1691734726073000","MAX_INSERTION_TIME":"1691734726073000","OPTIMIZE_TARGET_SIZE":"268435456"}}}}}""" - } - def getAddFileStr2(returnExpTime: Boolean = false): String = { - s"""{"file":{"id":"22d9b72771a72f178a6f2839f7f08529",${getExpirationTimestampStr( - returnExpTime - )}""" + """"deltaSingleAction":{"add":{"path":"http://path2","partitionValues":{"c2":"two"},"size":809,"modificationTime":1691734726073,"dataChange":true,"stats":"{\"numRecords\":2,\"minValues\":{\"c1\":1,\"c2\":\"two\"},\"maxValues\":{\"c1\":2,\"c2\":\"two\"},\"nullCount\":{\"c1\":0,\"c2\":0}}","tags":{"INSERTION_TIME":"1691734726073000","MIN_INSERTION_TIME":"1691734726073000","MAX_INSERTION_TIME":"1691734726073000","OPTIMIZE_TARGET_SIZE":"268435456"}}}}}""" - } - // scalastyle:on line.size.limit -} - -/** - * A mocked delta sharing client for unit tests. - */ -class TestDeltaSharingClientForCDFUtils( - profileProvider: DeltaSharingProfileProvider, - timeoutInSeconds: Int = 120, - numRetries: Int = 10, - maxRetryDuration: Long = Long.MaxValue, - sslTrustAll: Boolean = false, - forStreaming: Boolean = false, - responseFormat: String = DeltaSharingRestClient.RESPONSE_FORMAT_DELTA, - readerFeatures: String = "", - queryTablePaginationEnabled: Boolean = false, - maxFilesPerReq: Int = 100000) - extends DeltaSharingClient { - - import CDFTesTUtils._ - - private lazy val returnExpirationTimestamp = SparkSession.active.sessionState.conf - .getConfString( - SparkConfForReturnExpTime - ) - .toBoolean - - var numGetFileCalls: Int = -1 - - override def listAllTables(): Seq[Table] = throw new UnsupportedOperationException("not needed") - - override def getMetadata( - table: Table, - versionAsOf: Option[Long], - timestampAsOf: Option[String]): DeltaTableMetadata = { - throw new UnsupportedOperationException("getMetadata is not supported now.") - } - - override def getTableVersion(table: Table, startingTimestamp: Option[String] = None): Long = { - throw new UnsupportedOperationException("getTableVersion is not supported now.") - } - - override def getFiles( - table: Table, - predicates: Seq[String], - limit: Option[Long], - versionAsOf: Option[Long], - timestampAsOf: Option[String], - jsonPredicateHints: Option[String], - refreshToken: Option[String] - ): DeltaTableFiles = { - throw new UnsupportedOperationException("getFiles is not supported now.") - } - - override def getFiles( - table: Table, - startingVersion: Long, - endingVersion: Option[Long] - ): DeltaTableFiles = { - throw new UnsupportedOperationException(s"getFiles with startingVersion($startingVersion)") - } - - override def getCDFFiles( - table: Table, - cdfOptions: Map[String, String], - includeHistoricalMetadata: Boolean - ): DeltaTableFiles = { - numGetFileCalls += 1 - DeltaTableFiles( - version = 0, - lines = Seq[String]( - """{"protocol":{"deltaProtocol":{"minReaderVersion": 1, "minWriterVersion": 1}}}""", - metaDataStr, - getAddFileStr1(paths(numGetFileCalls.min(1)), returnExpirationTimestamp), - getAddFileStr2(returnExpirationTimestamp) - ), - respondedFormat = DeltaSharingRestClient.RESPONSE_FORMAT_DELTA - ) - } - - override def getForStreaming(): Boolean = forStreaming - - override def getProfileProvider: DeltaSharingProfileProvider = profileProvider -} - -class DeltaSharingCDFUtilsSuite - extends QueryTest - with DeltaSQLCommandTest - with SharedSparkSession - with DeltaSharingTestSparkUtils { - - import CDFTesTUtils._ - - private val shareName = "share" - private val schemaName = "default" - private val sharedTableName = "table" - - override protected def sparkConf: SparkConf = { - super.sparkConf - .set("spark.delta.sharing.preSignedUrl.expirationMs", expirationTimeMs.toString) - .set("spark.delta.sharing.driver.refreshCheckIntervalMs", "1000") - .set("spark.delta.sharing.driver.refreshThresholdMs", "2000") - .set("spark.delta.sharing.driver.accessThresholdToExpireMs", "60000") - } - - test("refresh works") { - PreSignedUrlCache.registerIfNeeded(SparkEnv.get) - - withTempDir { tempDir => - val profileFile = new File(tempDir, "foo.share") - FileUtils.writeStringToFile( - profileFile, - s"""{ - | "shareCredentialsVersion": 1, - | "endpoint": "https://localhost:12345/not-used-endpoint", - | "bearerToken": "mock" - |}""".stripMargin, - "utf-8" - ) - - def test(): Unit = { - val profilePath = profileFile.getCanonicalPath - val tablePath = new Path(s"$profilePath#$shareName.$schemaName.$sharedTableName") - val client = DeltaSharingRestClient(profilePath, false, "delta") - val dsTable = Table(share = shareName, schema = schemaName, name = sharedTableName) - - val options = new DeltaSharingOptions(Map("path" -> tablePath.toString)) - DeltaSharingCDFUtils.prepareCDFRelation( - SparkSession.active.sqlContext, - options, - dsTable, - client - ) - - val preSignedUrlCacheRef = PreSignedUrlCache.getEndpointRefInExecutor(SparkEnv.get) - val path = options.options.getOrElse( - "path", - throw DeltaSharingErrors.pathNotSpecifiedException - ) - val fetcher = new PreSignedUrlFetcher( - preSignedUrlCacheRef, - DeltaSharingUtils.getTablePathWithIdSuffix( - path, - DeltaSharingUtils.getQueryParamsHashId(options.cdfOptions) - ), - fileStr1Id, - 1000 - ) - // sleep for 25000ms to ensure that the urls are refreshed. - Thread.sleep(25000) - - // Verify that the url is refreshed as paths(1), not paths(0) anymore. - assert(fetcher.getUrl == paths(1)) - } - - withSQLConf( - "spark.delta.sharing.client.class" -> classOf[TestDeltaSharingClientForCDFUtils].getName, - "fs.delta-sharing-log.impl" -> classOf[DeltaSharingLogFileSystem].getName, - "spark.delta.sharing.profile.provider.class" -> - "io.delta.sharing.client.DeltaSharingFileProfileProvider", - SparkConfForReturnExpTime -> "true" - ) { - test() - } - - withSQLConf( - "spark.delta.sharing.client.class" -> classOf[TestDeltaSharingClientForCDFUtils].getName, - "fs.delta-sharing-log.impl" -> classOf[DeltaSharingLogFileSystem].getName, - "spark.delta.sharing.profile.provider.class" -> - "io.delta.sharing.client.DeltaSharingFileProfileProvider", - SparkConfForReturnExpTime -> "false" - ) { - test() - } - } - } -} diff --git a/sharing/src/test/scala/io/delta/sharing/spark/DeltaSharingDataSourceCMSuite.scala b/sharing/src/test/scala/io/delta/sharing/spark/DeltaSharingDataSourceCMSuite.scala deleted file mode 100644 index ee4cc01454f..00000000000 --- a/sharing/src/test/scala/io/delta/sharing/spark/DeltaSharingDataSourceCMSuite.scala +++ /dev/null @@ -1,984 +0,0 @@ -/* - * Copyright (2021) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.delta.sharing.spark - -import java.io.File - -import org.apache.spark.sql.delta.{ - BatchCDFSchemaEndVersion, - BatchCDFSchemaLatest, - BatchCDFSchemaLegacy, - DeltaUnsupportedOperationException -} -import org.apache.spark.sql.delta.sources.DeltaSQLConf -import org.apache.spark.sql.delta.test.DeltaSQLCommandTest - -import org.apache.spark.sql.delta.sharing.DeltaSharingTestSparkUtils -import org.apache.spark.sql.functions.col -import org.apache.spark.sql.streaming.{StreamingQueryException, StreamTest, Trigger} -import org.apache.spark.sql.types.{IntegerType, StringType, StructType} - -// Unit tests to verify that delta format sharing support column mapping (CM). -class DeltaSharingDataSourceCMSuite - extends StreamTest - with DeltaSQLCommandTest - with DeltaSharingTestSparkUtils - with DeltaSharingDataSourceDeltaTestUtils { - - import testImplicits._ - - override def beforeEach(): Unit = { - super.beforeEach() - spark.conf.set("spark.databricks.delta.streaming.allowSourceColumnRenameAndDrop", "false") - } - - - private def testReadCMTable( - deltaTableName: String, - sharedTablePath: String, - dropC1: Boolean = false): Unit = { - val expectedSchema: StructType = if (deltaTableName == "cm_id_table") { - spark.read.format("delta").table(deltaTableName).schema - } else { - if (dropC1) { - new StructType() - .add("c2rename", StringType) - } else { - new StructType() - .add("c1", IntegerType) - .add("c2rename", StringType) - } - } - assert( - expectedSchema == spark.read - .format("deltaSharing") - .option("responseFormat", "delta") - .load(sharedTablePath) - .schema - ) - - val sharingDf = - spark.read.format("deltaSharing").option("responseFormat", "delta").load(sharedTablePath) - val deltaDf = spark.read.format("delta").table(deltaTableName) - checkAnswer(sharingDf, deltaDf) - assert(sharingDf.count() > 0) - - val filteredSharingDf = - spark.read - .format("deltaSharing") - .option("responseFormat", "delta") - .load(sharedTablePath) - .filter(col("c2rename") === "one") - val filteredDeltaDf = - spark.read - .format("delta") - .table(deltaTableName) - .filter(col("c2rename") === "one") - checkAnswer(filteredSharingDf, filteredDeltaDf) - assert(filteredSharingDf.count() > 0) - } - - private def testReadCMCdf( - deltaTableName: String, - sharedTablePath: String, - startingVersion: Int): Unit = { - val schema = spark.read - .format("deltaSharing") - .option("responseFormat", "delta") - .option("readChangeFeed", "true") - .option("startingVersion", startingVersion) - .load(sharedTablePath) - .schema - val expectedSchema = spark.read - .format("delta") - .option("readChangeFeed", "true") - .option("startingVersion", startingVersion) - .table(deltaTableName) - .schema - assert(expectedSchema == schema) - - val deltaDf = spark.read - .format("delta") - .option("readChangeFeed", "true") - .option("startingVersion", startingVersion) - .table(deltaTableName) - val sharingDf = spark.read - .format("deltaSharing") - .option("responseFormat", "delta") - .option("readChangeFeed", "true") - .option("startingVersion", startingVersion) - .load(sharedTablePath) - if (startingVersion <= 2) { - Seq(BatchCDFSchemaEndVersion, BatchCDFSchemaLatest, BatchCDFSchemaLegacy).foreach { m => - withSQLConf( - DeltaSQLConf.DELTA_CDF_DEFAULT_SCHEMA_MODE_FOR_COLUMN_MAPPING_TABLE.key -> - m.name - ) { - val deltaException = intercept[DeltaUnsupportedOperationException] { - deltaDf.collect() - } - assert( - deltaException.getMessage.contains("Retrieving table changes between") && - deltaException.getMessage.contains("failed because of an incompatible") - ) - val sharingException = intercept[DeltaUnsupportedOperationException] { - sharingDf.collect() - } - assert( - sharingException.getMessage.contains("Retrieving table changes between") && - sharingException.getMessage.contains("failed because of an incompatible") - ) - } - } - } else { - checkAnswer(sharingDf, deltaDf) - assert(sharingDf.count() > 0) - } - } - - private def testReadingSharedCMTable( - tempDir: File, - deltaTableName: String, - sharedTableNameBase: String): Unit = { - val sharedTableNameBasic = sharedTableNameBase + "_one" - prepareMockedClientAndFileSystemResult( - deltaTable = deltaTableName, - sharedTable = sharedTableNameBasic - ) - prepareMockedClientGetTableVersion(deltaTableName, sharedTableNameBasic) - - withSQLConf(getDeltaSharingClassesSQLConf.toSeq: _*) { - val profileFile = prepareProfileFile(tempDir) - testReadCMTable( - deltaTableName = deltaTableName, - sharedTablePath = s"${profileFile.getCanonicalPath}#share1.default.$sharedTableNameBasic" - ) - } - - val sharedTableNameCdf = sharedTableNameBase + "_cdf" - // Test CM and CDF - // Error when reading cdf with startingVersion <= 2, matches delta behavior. - prepareMockedClientGetTableVersion(deltaTableName, sharedTableNameCdf) - prepareMockedClientAndFileSystemResultForCdf( - deltaTableName, - sharedTableNameCdf, - startingVersion = 0 - ) - prepareMockedClientAndFileSystemResultForCdf( - deltaTableName, - sharedTableNameCdf, - startingVersion = 2 - ) - prepareMockedClientAndFileSystemResultForCdf( - deltaTableName, - sharedTableNameCdf, - startingVersion = 3 - ) - - withSQLConf(getDeltaSharingClassesSQLConf.toSeq: _*) { - val profileFile = prepareProfileFile(tempDir) - testReadCMCdf( - deltaTableName, - s"${profileFile.getCanonicalPath}#share1.default.$sharedTableNameCdf", - 0 - ) - testReadCMCdf( - deltaTableName, - s"${profileFile.getCanonicalPath}#share1.default.$sharedTableNameCdf", - 2 - ) - testReadCMCdf( - deltaTableName, - s"${profileFile.getCanonicalPath}#share1.default.$sharedTableNameCdf", - 3 - ) - } - - val sharedTableNameDrop = sharedTableNameBase + "_drop" - // DROP COLUMN - sql(s"ALTER TABLE $deltaTableName DROP COLUMN c1") - prepareMockedClientGetTableVersion(deltaTableName, sharedTableNameDrop) - prepareMockedClientAndFileSystemResult( - deltaTable = deltaTableName, - sharedTable = sharedTableNameDrop - ) - prepareMockedClientGetTableVersion(deltaTableName, sharedTableNameDrop) - withSQLConf(getDeltaSharingClassesSQLConf.toSeq: _*) { - val profileFile = prepareProfileFile(tempDir) - testReadCMTable( - deltaTableName = deltaTableName, - sharedTablePath = s"${profileFile.getCanonicalPath}#share1.default.$sharedTableNameDrop", - dropC1 = true - ) - } - } - - /** - * column mapping tests - */ - test( - "DeltaSharingDataSource able to read data for cm name mode" - ) { - withTempDir { tempDir => - val deltaTableName = "delta_table_cm_name" - withTable(deltaTableName) { - createSimpleTable(deltaTableName, enableCdf = true) - sql(s"""INSERT INTO $deltaTableName VALUES (1, "one"), (2, "one")""") - spark.sql( - s"""ALTER TABLE $deltaTableName SET TBLPROPERTIES('delta.minReaderVersion' = '2', - |'delta.minWriterVersion' = '5', - |'delta.columnMapping.mode' = 'name')""".stripMargin - ) - sql(s"""ALTER TABLE $deltaTableName RENAME COLUMN c2 TO c2rename""") - sql(s"""INSERT INTO $deltaTableName VALUES (1, "two"), (2, "two")""") - - sql(s"""DELETE FROM $deltaTableName where c1=1""") - sql(s"""UPDATE $deltaTableName set c1="3" where c2rename="one"""") - - val sharedTableName = "shared_table_cm_name" - testReadingSharedCMTable(tempDir, deltaTableName, sharedTableName) - } - } - } - - test("DeltaSharingDataSource able to read data for cm id mode") { - withTempDir { tempDir => - val deltaTableName = "delta_table_cm_id" - withTable(deltaTableName) { - createCMIdTableWithCdf(deltaTableName) - sql(s"""INSERT INTO $deltaTableName VALUES (1, "one"), (2, "one")""") - sql(s"""INSERT INTO $deltaTableName VALUES (1, "two"), (2, "two")""") - - sql(s"""ALTER TABLE $deltaTableName RENAME COLUMN c2 TO c2rename""") - sql(s"""INSERT INTO $deltaTableName VALUES (1, "two"), (2, "two")""") - - sql(s"""DELETE FROM $deltaTableName where c1=1""") - sql(s"""UPDATE $deltaTableName set c1="3" where c2rename="one"""") - - val sharedTableName = "shared_table_cm_id" - testReadingSharedCMTable(tempDir, deltaTableName, sharedTableName) - } - } - } - - /** - * Streaming Test - */ - private def InsertToDeltaTable(tableName: String, values: String): Unit = { - sql(s"INSERT INTO $tableName VALUES $values") - } - - private def processAllAvailableInStream( - tablePath: String, - checkpointDirStr: String, - outputDirStr: String): Unit = { - val q = spark.readStream - .format("deltaSharing") - .option("responseFormat", "delta") - .load(tablePath) - .writeStream - .format("delta") - .option("checkpointLocation", checkpointDirStr) - .option("mergeSchema", "true") - .start(outputDirStr) - - try { - q.processAllAvailable() - } finally { - q.stop() - } - } - - private def processStreamWithSchemaTracking( - tablePath: String, - checkpointDirStr: String, - outputDirStr: String, - trigger: Option[Trigger] = None, - maxFilesPerTrigger: Option[Int] = None): Unit = { - var dataStreamReader = spark.readStream - .format("deltaSharing") - .option("schemaTrackingLocation", checkpointDirStr) - .option("responseFormat", "delta") - if (maxFilesPerTrigger.isDefined || trigger.isDefined) { - // When trigger.Once is defined, maxFilesPerTrigger is ignored -- this is the - // behavior of the streaming engine. And AvailableNow is converted as Once for delta sharing. - dataStreamReader = - dataStreamReader.option("maxFilesPerTrigger", maxFilesPerTrigger.getOrElse(1)) - } - var dataStreamWriter = dataStreamReader - .load(tablePath) - .writeStream - .format("delta") - .option("checkpointLocation", checkpointDirStr) - .option("mergeSchema", "true") - if (trigger.isDefined) { - dataStreamWriter = dataStreamWriter.trigger(trigger.get) - } - - val q = dataStreamWriter.start(outputDirStr) - - try { - q.processAllAvailable() - if (maxFilesPerTrigger.isDefined && trigger.isEmpty) { - val progress = q.recentProgress.filter(_.numInputRows != 0) - // 2 batches -- 2 files are processed, this is how the delta table is constructed. - assert(progress.length === 2) - progress.foreach { p => - assert(p.numInputRows === 2) // 2 rows per batch -- 2 rows in each file. - } - } - } finally { - q.stop() - } - } - - private def prepareProcessAndCheckInitSnapshot( - deltaTableName: String, - sharedTableName: String, - sharedTablePath: String, - checkpointDirStr: String, - outputDir: File, - useSchemaTracking: Boolean, - trigger: Option[Trigger] = None - ): Unit = { - InsertToDeltaTable(deltaTableName, """(1, "one"), (2, "one"), (1, "two")""") - prepareMockedClientAndFileSystemResult( - deltaTable = deltaTableName, - sharedTable = sharedTableName, - versionAsOf = Some(1L) - ) - prepareMockedClientGetTableVersion(deltaTableName, sharedTableName) - prepareMockedClientMetadata(deltaTableName, sharedTableName) - if (useSchemaTracking) { - processStreamWithSchemaTracking( - sharedTablePath, - checkpointDirStr, - outputDir.toString, - trigger - ) - } else { - processAllAvailableInStream( - sharedTablePath, - checkpointDirStr, - outputDir.toString - ) - } - - checkAnswer( - spark.read.format("delta").load(outputDir.getCanonicalPath), - Seq((1, "one"), (2, "one"), (1, "two")).toDF() - ) - } - - def prepareNewInsert( - deltaTableName: String, - sharedTableName: String, - values: String, - startingVersion: Long, - endingVersion: Long): Unit = { - InsertToDeltaTable(deltaTableName, values) - prepareMockedClientGetTableVersion(deltaTableName, sharedTableName) - prepareMockedClientAndFileSystemResultForStreaming( - deltaTableName, - sharedTableName, - startingVersion, - endingVersion - ) - } - - private def renameColumnAndPrepareRpcResponse( - deltaTableName: String, - sharedTableName: String, - startingVersion: Long, - endingVersion: Long, - insertAfterRename: Boolean): Unit = { - // Rename on the original delta table. - sql(s"""ALTER TABLE $deltaTableName RENAME COLUMN c2 TO c2rename""") - if (insertAfterRename) { - InsertToDeltaTable(deltaTableName, """(1, "three")""") - InsertToDeltaTable(deltaTableName, """(2, "three")""") - } - // Prepare all the delta sharing rpcs. - prepareMockedClientGetTableVersion(deltaTableName, sharedTableName) - prepareMockedClientMetadata(deltaTableName, sharedTableName) - prepareMockedClientAndFileSystemResultForStreaming( - deltaTableName, - sharedTableName, - startingVersion, - endingVersion - ) - } - - private def expectUseSchemaLogException( - tablePath: String, - checkpointDirStr: String, - outputDirStr: String): Unit = { - val error = intercept[StreamingQueryException] { - processAllAvailableInStream( - tablePath, - checkpointDirStr, - outputDirStr - ) - }.toString - assert(error.contains("DELTA_STREAMING_INCOMPATIBLE_SCHEMA_CHANGE_USE_SCHEMA_LOG")) - assert(error.contains("Please provide a 'schemaTrackingLocation'")) - } - - private def expectMetadataEvolutionException( - tablePath: String, - checkpointDirStr: String, - outputDirStr: String, - trigger: Option[Trigger] = None, - maxFilesPerTrigger: Option[Int] = None): Unit = { - val error = intercept[StreamingQueryException] { - processStreamWithSchemaTracking( - tablePath, - checkpointDirStr, - outputDirStr, - trigger, - maxFilesPerTrigger - ) - }.toString - assert(error.contains("DELTA_STREAMING_METADATA_EVOLUTION")) - assert(error.contains("Please restart the stream to continue")) - } - - private def expectSqlConfException( - tablePath: String, - checkpointDirStr: String, - outputDirStr: String, - trigger: Option[Trigger] = None, - maxFilesPerTrigger: Option[Int] = None): Unit = { - val error = intercept[StreamingQueryException] { - processStreamWithSchemaTracking( - tablePath, - checkpointDirStr, - outputDirStr, - trigger, - maxFilesPerTrigger - ) - }.toString - assert(error.contains("DELTA_STREAMING_CANNOT_CONTINUE_PROCESSING_POST_SCHEMA_EVOLUTION")) - assert(error.contains("delta.streaming.allowSourceColumnRenameAndDrop")) - } - - private def processWithSqlConf( - tablePath: String, - checkpointDirStr: String, - outputDirStr: String, - trigger: Option[Trigger] = None, - maxFilesPerTrigger: Option[Int] = None): Unit = { - // Using allowSourceColumnRenameAndDrop instead of - // allowSourceColumnRenameAndDrop.[checkpoint_hash] because the checkpointDir changes - // every test. - spark.conf - .set("spark.databricks.delta.streaming.allowSourceColumnRenameAndDrop", "always") - processStreamWithSchemaTracking( - tablePath, - checkpointDirStr, - outputDirStr, - trigger, - maxFilesPerTrigger - ) - } - - private def testRestartStreamingFourTimes( - tablePath: String, - checkpointDir: java.io.File, - outputDirStr: String): Unit = { - val checkpointDirStr = checkpointDir.toString - - // 1. Followed the previous error message to use schemaTrackingLocation, but received - // error suggesting restart. - expectMetadataEvolutionException(tablePath, checkpointDirStr, outputDirStr) - - // 2. Followed the previous error message to restart, but need to restart again for - // DeltaSource to handle offset movement, this is the SAME behavior as stream reading from - // the delta table directly. - expectMetadataEvolutionException(tablePath, checkpointDirStr, outputDirStr) - - // 3. Followed the previous error message to restart, but cannot write to the dest table. - expectSqlConfException(tablePath, checkpointDirStr, outputDirStr) - - // 4. Restart with new sqlConf, able to process new data and writing to a new column. - // Not using allowSourceColumnRenameAndDrop.[checkpoint_hash] because the checkpointDir - // changes every test, using allowSourceColumnRenameAndDrop=always instead. - processWithSqlConf(tablePath, checkpointDirStr, outputDirStr) - } - - test("cm streaming works with newly added schemaTrackingLocation") { - withTempDirs { (inputDir, outputDir, checkpointDir) => - val deltaTableName = "delta_table_cm_streaming_basic" - withTable(deltaTableName) { - createCMIdTableWithCdf(deltaTableName) - val sharedTableName = "shared_table_cm_streaming_basic" - val profileFile = prepareProfileFile(inputDir) - val tablePath = profileFile.getCanonicalPath + s"#share1.default.$sharedTableName" - - withSQLConf(getDeltaSharingClassesSQLConf.toSeq: _*) { - // 1. Able to stream snapshot at version 1. - // The streaming is started without schemaTrackingLocation. - prepareProcessAndCheckInitSnapshot( - deltaTableName = deltaTableName, - sharedTableName = sharedTableName, - sharedTablePath = tablePath, - checkpointDirStr = checkpointDir.toString, - outputDir = outputDir, - useSchemaTracking = false - ) - - // 2. Able to stream new data at version 2. - // The streaming is continued without schemaTrackingLocation. - prepareNewInsert( - deltaTableName = deltaTableName, - sharedTableName = sharedTableName, - values = """(2, "two")""", - startingVersion = 2, - endingVersion = 2 - ) - processAllAvailableInStream( - tablePath, - checkpointDir.toString, - outputDir.toString - ) - checkAnswer( - spark.read.format("delta").load(outputDir.getCanonicalPath), - Seq((1, "one"), (2, "one"), (1, "two"), (2, "two")).toDF() - ) - - // 3. column renaming at version 3, and expect exception. - renameColumnAndPrepareRpcResponse( - deltaTableName = deltaTableName, - sharedTableName = sharedTableName, - startingVersion = 2, - endingVersion = 3, - insertAfterRename = false - ) - expectUseSchemaLogException(tablePath, checkpointDir.toString, outputDir.toString) - - // 4. insert new data at version 4. - prepareNewInsert( - deltaTableName = deltaTableName, - sharedTableName = sharedTableName, - values = """(1, "three"), (2, "three")""", - startingVersion = 2, - endingVersion = 4 - ) - // Additional preparation for rpc because deltaSource moved the offset to (3, -20) and - // (3, -19) after restart. - prepareMockedClientAndFileSystemResultForStreaming( - deltaTableName, - sharedTableName, - 3, - 4 - ) - - // 5. with 4 restarts, able to continue the streaming - // The streaming is re-started WITH schemaTrackingLocation, and it's able to capture the - // schema used in previous version, based on the initial call of getBatch for the latest - // offset, which pulls the metadata from the server. - testRestartStreamingFourTimes(tablePath, checkpointDir, outputDir.toString) - - // An additional column is added to the output table. - checkAnswer( - spark.read.format("delta").load(outputDir.getCanonicalPath), - Seq( - (1, "one", null), - (2, "one", null), - (1, "two", null), - (2, "two", null), - (1, null, "three"), - (2, null, "three") - ).toDF() - ) - } - } - } - } - - test("cm streaming works with restart on snapshot query") { - // The main difference in this test is the rename happens after processing the initial snapshot, - // (instead of after making continuous progress), to test that the restart could fetch the - // latest metadata and the metadata from lastest offset. - withTempDirs { (inputDir, outputDir, checkpointDir) => - val deltaTableName = "delta_table_streaming_restart" - withTable(deltaTableName) { - createCMIdTableWithCdf(deltaTableName) - val sharedTableName = "shared_table_streaming_restart" - val profileFile = prepareProfileFile(inputDir) - val tablePath = profileFile.getCanonicalPath + s"#share1.default.$sharedTableName" - - withSQLConf(getDeltaSharingClassesSQLConf.toSeq: _*) { - // 1. Able to stream snapshot at version 1. - prepareProcessAndCheckInitSnapshot( - deltaTableName = deltaTableName, - sharedTableName = sharedTableName, - sharedTablePath = tablePath, - checkpointDirStr = checkpointDir.toString, - outputDir = outputDir, - useSchemaTracking = false - ) - - // 2. column renaming at version 2, and expect exception. - renameColumnAndPrepareRpcResponse( - deltaTableName = deltaTableName, - sharedTableName = sharedTableName, - startingVersion = 2, - endingVersion = 2, - insertAfterRename = false - ) - expectUseSchemaLogException(tablePath, checkpointDir.toString, outputDir.toString) - - // 3. insert new data at version 3. - prepareNewInsert( - deltaTableName = deltaTableName, - sharedTableName = sharedTableName, - values = """(1, "three"), (2, "three")""", - startingVersion = 2, - endingVersion = 3 - ) - - // 4. with 4 restarts, able to continue the streaming - testRestartStreamingFourTimes(tablePath, checkpointDir, outputDir.toString) - - // An additional column is added to the output table. - checkAnswer( - spark.read.format("delta").load(outputDir.getCanonicalPath), - Seq( - (1, "one", null), - (2, "one", null), - (1, "two", null), - (1, null, "three"), - (2, null, "three") - ).toDF() - ) - } - } - } - } - - test("cm streaming works with schemaTracking used at start") { - withTempDirs { (inputDir, outputDir, checkpointDir) => - val deltaTableName = "delta_table_streaming_schematracking" - withTable(deltaTableName) { - createCMIdTableWithCdf(deltaTableName) - val sharedTableName = "shared_table_streaming_schematracking" - val profileFile = prepareProfileFile(inputDir) - val tablePath = profileFile.getCanonicalPath + s"#share1.default.$sharedTableName" - - withSQLConf(getDeltaSharingClassesSQLConf.toSeq: _*) { - // 1. Able to stream snapshot at version 1. - prepareProcessAndCheckInitSnapshot( - deltaTableName = deltaTableName, - sharedTableName = sharedTableName, - sharedTablePath = tablePath, - checkpointDirStr = checkpointDir.toString, - outputDir = outputDir, - useSchemaTracking = true - ) - - // 2. Able to stream new data at version 2. - prepareNewInsert( - deltaTableName = deltaTableName, - sharedTableName = sharedTableName, - values = """(2, "two")""", - startingVersion = 2, - endingVersion = 2 - ) - processStreamWithSchemaTracking( - tablePath, - checkpointDir.toString, - outputDir.toString - ) - checkAnswer( - spark.read.format("delta").load(outputDir.getCanonicalPath), - Seq((1, "one"), (2, "one"), (1, "two"), (2, "two")).toDF() - ) - - // 3. column renaming at version 3, and expect exception. - renameColumnAndPrepareRpcResponse( - deltaTableName = deltaTableName, - sharedTableName = sharedTableName, - startingVersion = 2, - endingVersion = 3, - insertAfterRename = false - ) - expectMetadataEvolutionException(tablePath, checkpointDir.toString, outputDir.toString) - - // 4. First see exception, then with sql conf, able to stream new data at version 4. - prepareNewInsert( - deltaTableName = deltaTableName, - sharedTableName = sharedTableName, - values = """(1, "three"), (2, "three")""", - startingVersion = 3, - endingVersion = 4 - ) - expectSqlConfException(tablePath, checkpointDir.toString, outputDir.toString) - processWithSqlConf(tablePath, checkpointDir.toString, outputDir.toString) - - checkAnswer( - spark.read.format("delta").load(outputDir.getCanonicalPath), - Seq( - (1, "one", null), - (2, "one", null), - (1, "two", null), - (2, "two", null), - (1, null, "three"), - (2, null, "three") - ).toDF() - ) - } - } - } - } - - test("cm streaming works with restart with accumulated inserts after rename") { - withTempDirs { (inputDir, outputDir, checkpointDir) => - val deltaTableName = "delta_table_streaming_accumulate" - withTable(deltaTableName) { - createCMIdTableWithCdf(deltaTableName) - val sharedTableName = "shared_table_streaming_accumulate" - val profileFile = prepareProfileFile(inputDir) - val tablePath = profileFile.getCanonicalPath + s"#share1.default.$sharedTableName" - - withSQLConf(getDeltaSharingClassesSQLConf.toSeq: _*) { - // 1. Able to stream snapshot at version 1. - prepareProcessAndCheckInitSnapshot( - deltaTableName = deltaTableName, - sharedTableName = sharedTableName, - sharedTablePath = tablePath, - checkpointDirStr = checkpointDir.toString, - outputDir = outputDir, - useSchemaTracking = false - ) - - // 2. column renaming at version 2, and expect exception. - renameColumnAndPrepareRpcResponse( - deltaTableName = deltaTableName, - sharedTableName = sharedTableName, - startingVersion = 2, - endingVersion = 4, - insertAfterRename = true - ) - expectUseSchemaLogException(tablePath, checkpointDir.toString, outputDir.toString) - - // 4. with 4 restarts, able to continue the streaming - testRestartStreamingFourTimes(tablePath, checkpointDir, outputDir.toString) - - // An additional column is added to the output table. - checkAnswer( - spark.read.format("delta").load(outputDir.getCanonicalPath), - Seq( - (1, "one", null), - (2, "one", null), - (1, "two", null), - (1, null, "three"), - (2, null, "three") - ).toDF() - ) - } - } - } - } - - test("cm streaming works with column drop and add") { - withTempDirs { (inputDir, outputDir, checkpointDir) => - val deltaTableName = "delta_table_column_drop" - withTable(deltaTableName) { - createCMIdTableWithCdf(deltaTableName) - val sharedTableName = "shared_table_column_drop" - val profileFile = prepareProfileFile(inputDir) - val tablePath = profileFile.getCanonicalPath + s"#share1.default.$sharedTableName" - - withSQLConf(getDeltaSharingClassesSQLConf.toSeq: _*) { - // 1. Able to stream snapshot at version 1. - prepareProcessAndCheckInitSnapshot( - deltaTableName = deltaTableName, - sharedTableName = sharedTableName, - sharedTablePath = tablePath, - checkpointDirStr = checkpointDir.toString, - outputDir = outputDir, - useSchemaTracking = true - ) - - // 2. drop column c1 at version 2 - sql(s"ALTER TABLE $deltaTableName DROP COLUMN c1") - // 3. add column c3 at version 3 - sql(s"ALTER TABLE $deltaTableName ADD COLUMN (c3 int)") - prepareMockedClientGetTableVersion(deltaTableName, sharedTableName) - prepareMockedClientMetadata(deltaTableName, sharedTableName) - prepareMockedClientAndFileSystemResultForStreaming( - deltaTableName, - sharedTableName, - 2, - 3 - ) - prepareMockedClientAndFileSystemResultForStreaming( - deltaTableName, - sharedTableName, - 3, - 3 - ) - - // Needs a 3 restarts for deltaSource to catch up. - expectMetadataEvolutionException(tablePath, checkpointDir.toString, outputDir.toString) - expectSqlConfException(tablePath, checkpointDir.toString, outputDir.toString) - spark.conf - .set("spark.databricks.delta.streaming.allowSourceColumnRenameAndDrop", "always") - expectMetadataEvolutionException(tablePath, checkpointDir.toString, outputDir.toString) - processWithSqlConf(tablePath, checkpointDir.toString, outputDir.toString) - - // 4. insert at version 4 - InsertToDeltaTable(deltaTableName, """("four", 4)""") - // 5. insert at version 5 - InsertToDeltaTable(deltaTableName, """("five", 5)""") - prepareMockedClientGetTableVersion(deltaTableName, sharedTableName) - prepareMockedClientAndFileSystemResultForStreaming( - deltaTableName, - sharedTableName, - 3, - 5 - ) - - processStreamWithSchemaTracking( - tablePath, - checkpointDir.toString, - outputDir.toString - ) - checkAnswer( - spark.read.format("delta").load(outputDir.getCanonicalPath), - Seq[(java.lang.Integer, String, java.lang.Integer)]( - (1, "one", null), - (2, "one", null), - (1, "two", null), - (null, "four", 4), - (null, "five", 5) - ).toDF() - ) - } - } - } - } - - - test("cm streaming works with MaxFilesPerTrigger") { - withTempDirs { (inputDir, outputDir, checkpointDir) => - val deltaTableName = "delta_table_maxfiles" - withTable(deltaTableName) { - createCMIdTableWithCdf(deltaTableName) - val sharedTableName = "shared_table_maxfiles" - val profileFile = prepareProfileFile(inputDir) - val tablePath = profileFile.getCanonicalPath + s"#share1.default.$sharedTableName" - - withSQLConf(getDeltaSharingClassesSQLConf.toSeq: _*) { - // 1. Able to stream snapshot at version 1. - InsertToDeltaTable(deltaTableName, """(1, "one"), (2, "one"), (1, "two"), (2, "two")""") - prepareMockedClientAndFileSystemResult( - deltaTable = deltaTableName, - sharedTable = sharedTableName, - versionAsOf = Some(1L) - ) - prepareMockedClientGetTableVersion(deltaTableName, sharedTableName) - prepareMockedClientMetadata(deltaTableName, sharedTableName) - - // process with maxFilesPerTrigger. - processStreamWithSchemaTracking( - tablePath, - checkpointDir.toString, - outputDir.toString, - trigger = None, - maxFilesPerTrigger = Some(1) - ) - checkAnswer( - spark.read.format("delta").load(outputDir.getCanonicalPath), - Seq((1, "one"), (2, "one"), (1, "two"), (2, "two")).toDF() - ) - - // 2. column renaming at version 2, no exception because of Trigger.Once. - sql(s"""ALTER TABLE $deltaTableName RENAME COLUMN c2 TO c2rename""") - - // Prepare all the delta sharing rpcs. - prepareMockedClientGetTableVersion(deltaTableName, sharedTableName) - prepareMockedClientMetadata(deltaTableName, sharedTableName) - prepareMockedClientAndFileSystemResultForStreaming( - deltaTableName, - sharedTableName, - startingVersion = 1, - endingVersion = 2 - ) - prepareMockedClientAndFileSystemResultForStreaming( - deltaTableName, - sharedTableName, - startingVersion = 2, - endingVersion = 2 - ) - - // maxFilesPerTrigger doesn't change whether exception is thrown or not. - expectMetadataEvolutionException( - tablePath, - checkpointDir.toString, - outputDir.toString, - trigger = None, - maxFilesPerTrigger = Some(1) - ) - - // 4. First see exception, then with sql conf, able to stream new data at version 4 and 5. - InsertToDeltaTable( - deltaTableName, - """(1, "three"), (2, "three"), (1, "four"), (2, "four")""" - ) - prepareMockedClientGetTableVersion(deltaTableName, sharedTableName) - prepareMockedClientAndFileSystemResultForStreaming( - deltaTableName, - sharedTableName, - 2, - 3 - ) - - expectSqlConfException( - tablePath, - checkpointDir.toString, - outputDir.toString, - trigger = None, - maxFilesPerTrigger = Some(1) - ) - processWithSqlConf( - tablePath, - checkpointDir.toString, - outputDir.toString, - trigger = None, - maxFilesPerTrigger = Some(1) - ) - - checkAnswer( - spark.read.format("delta").load(outputDir.getCanonicalPath), - Seq( - (1, "one", null), - (2, "one", null), - (1, "two", null), - (2, "two", null), - (1, null, "three"), - (2, null, "three"), - (1, null, "four"), - (2, null, "four") - ).toDF() - ) - } - } - } - } -} diff --git a/sharing/src/test/scala/io/delta/sharing/spark/DeltaSharingDataSourceDeltaSuite.scala b/sharing/src/test/scala/io/delta/sharing/spark/DeltaSharingDataSourceDeltaSuite.scala deleted file mode 100644 index 13d0fca3fcf..00000000000 --- a/sharing/src/test/scala/io/delta/sharing/spark/DeltaSharingDataSourceDeltaSuite.scala +++ /dev/null @@ -1,851 +0,0 @@ -/* - * Copyright (2021) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.delta.sharing.spark - -import org.apache.spark.sql.delta.test.DeltaSQLCommandTest -import org.apache.spark.sql.{DataFrame, QueryTest, Row} -import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.delta.sharing.DeltaSharingTestSparkUtils -import org.apache.spark.sql.functions.col -import org.apache.spark.sql.types.{ - DateType, - IntegerType, - LongType, - StringType, - StructType, - TimestampType -} - -trait DeltaSharingDataSourceDeltaSuiteBase - extends QueryTest - with DeltaSQLCommandTest - with DeltaSharingTestSparkUtils - with DeltaSharingDataSourceDeltaTestUtils { - - /** - * metadata tests - */ - test("failed to getMetadata") { - withTempDir { tempDir => - val sharedTableName = "shared_table_broken_json" - - def test(tablePath: String, tableFullName: String): Unit = { - DeltaSharingUtils.overrideIteratorBlock[String]( - blockId = TestClientForDeltaFormatSharing.getBlockId(sharedTableName, "getMetadata"), - values = Seq("bad protocol string", "bad metadata string").toIterator - ) - DeltaSharingUtils.overrideSingleBlock[Long]( - blockId = TestClientForDeltaFormatSharing.getBlockId(sharedTableName, "getTableVersion"), - value = 1 - ) - // JsonParseException on "bad protocol string" - val exception = intercept[com.fasterxml.jackson.core.JsonParseException] { - spark.read.format("deltaSharing").option("responseFormat", "delta").load(tablePath).schema - } - assert(exception.getMessage.contains("Unrecognized token 'bad'")) - - // table_with_broken_protocol - // able to parse as a DeltaSharingSingleAction, but it's an addFile, not metadata. - DeltaSharingUtils.overrideIteratorBlock[String]( - blockId = TestClientForDeltaFormatSharing.getBlockId(sharedTableName, "getMetadata"), - // scalastyle:off line.size.limit - values = Seq( - """{"add": {"path":"random","id":"random","partitionValues":{},"size":1,"motificationTime":1,"dataChange":false}}""" - ).toIterator - ) - val exception2 = intercept[IllegalStateException] { - spark.read.format("deltaSharing").option("responseFormat", "delta").load(tablePath).schema - } - assert( - exception2.getMessage - .contains(s"Failed to get Protocol for $tableFullName") - ) - - // table_with_broken_metadata - // able to parse as a DeltaSharingSingleAction, but it's an addFile, not metadata. - DeltaSharingUtils.overrideIteratorBlock[String]( - blockId = TestClientForDeltaFormatSharing.getBlockId(sharedTableName, "getMetadata"), - values = Seq( - """{"protocol":{"minReaderVersion":1}}""" - ).toIterator - ) - val exception3 = intercept[IllegalStateException] { - spark.read.format("deltaSharing").option("responseFormat", "delta").load(tablePath).schema - } - assert( - exception3.getMessage - .contains(s"Failed to get Metadata for $tableFullName") - ) - } - - withSQLConf(getDeltaSharingClassesSQLConf.toSeq: _*) { - val profileFile = prepareProfileFile(tempDir) - val tableFullName = s"share1.default.$sharedTableName" - test(s"${profileFile.getCanonicalPath}#$tableFullName", tableFullName) - } - } - } - - def assertLimit(tablePath: String, expectedLimit: Seq[Long]): Unit = { - assert(expectedLimit == - TestClientForDeltaFormatSharing.limits.filter(_._1.contains(tablePath)).map(_._2)) - } - - def assertRequestedFormat(tablePath: String, expectedFormat: Seq[String]): Unit = { - assert(expectedFormat == - TestClientForDeltaFormatSharing.requestedFormat.filter(_._1.contains(tablePath)).map(_._2)) - } - /** - * snapshot queries - */ - test("DeltaSharingDataSource able to read simple data") { - withTempDir { tempDir => - val deltaTableName = "delta_table_simple" - withTable(deltaTableName) { - createTable(deltaTableName) - sql( - s"INSERT INTO $deltaTableName" + - """ VALUES (1, "one", "2023-01-01", "2023-01-01 00:00:00"), - |(2, "two", "2023-02-02", "2023-02-02 00:00:00")""".stripMargin - ) - - val expectedSchema: StructType = new StructType() - .add("c1", IntegerType) - .add("c2", StringType) - .add("c3", DateType) - .add("c4", TimestampType) - val expected = Seq( - Row(1, "one", sqlDate("2023-01-01"), sqlTimestamp("2023-01-01 00:00:00")), - Row(2, "two", sqlDate("2023-02-02"), sqlTimestamp("2023-02-02 00:00:00")) - ) - - Seq(true, false).foreach { enableLimitPushdown => - val sharedTableName = s"shared_table_simple_$enableLimitPushdown" - prepareMockedClientAndFileSystemResult(deltaTableName, sharedTableName) - prepareMockedClientGetTableVersion(deltaTableName, sharedTableName) - - def test(tablePath: String, tableName: String): Unit = { - assert( - expectedSchema == spark.read - .format("deltaSharing") - .option("responseFormat", "delta") - .load(tablePath) - .schema - ) - val df = - spark.read.format("deltaSharing").option("responseFormat", "delta").load(tablePath) - checkAnswer(df, expected) - assert(df.count() > 0) - assertLimit(tableName, Seq.empty[Long]) - val limitDf = spark.read - .format("deltaSharing") - .option("responseFormat", "delta") - .load(tablePath) - .limit(1) - assert(limitDf.collect().size == 1) - assertLimit(tableName, Some(1L).filter(_ => enableLimitPushdown).toSeq) - } - - val limitPushdownConfig = Map( - "spark.delta.sharing.limitPushdown.enabled" -> enableLimitPushdown.toString - ) - withSQLConf((limitPushdownConfig ++ getDeltaSharingClassesSQLConf).toSeq: _*) { - val profileFile = prepareProfileFile(tempDir) - val tableName = s"share1.default.$sharedTableName" - test(s"${profileFile.getCanonicalPath}#$tableName", tableName) - } - } - } - } - } - - test("DeltaSharingDataSource able to auto resolve responseFormat") { - withTempDir { tempDir => - val deltaTableName = "delta_table_auto" - withTable(deltaTableName) { - createSimpleTable(deltaTableName, enableCdf = false) - sql( - s"""INSERT INTO $deltaTableName VALUES (1, "one"), (2, "one")""".stripMargin - ) - sql( - s"""INSERT INTO $deltaTableName VALUES (1, "two"), (2, "two")""".stripMargin - ) - - val expectedSchema: StructType = new StructType() - .add("c1", IntegerType) - .add("c2", StringType) - - def testAutoResolve(tablePath: String, tableName: String, expectedFormat: String): Unit = { - assert( - expectedSchema == spark.read - .format("deltaSharing") - .load(tablePath) - .schema - ) - - val deltaDf = spark.read.format("delta").table(deltaTableName) - val sharingDf = spark.read.format("deltaSharing").load(tablePath) - checkAnswer(deltaDf, sharingDf) - assert(sharingDf.count() > 0) - assertLimit(tableName, Seq.empty[Long]) - assertRequestedFormat(tableName, Seq(expectedFormat)) - - val limitDf = spark.read - .format("deltaSharing") - .load(tablePath) - .limit(1) - assert(limitDf.collect().size == 1) - assertLimit(tableName, Seq(1L)) - - val deltaDfV1 = spark.read.format("delta").option("versionAsOf", 1).table(deltaTableName) - val sharingDfV1 = - spark.read.format("deltaSharing").option("versionAsOf", 1).load(tablePath) - checkAnswer(deltaDfV1, sharingDfV1) - assert(sharingDfV1.count() > 0) - assertRequestedFormat(tableName, Seq(expectedFormat)) - } - - // Test for delta format response - val sharedDeltaTable = "shared_delta_table" - prepareMockedClientAndFileSystemResult(deltaTableName, sharedDeltaTable) - prepareMockedClientAndFileSystemResult( - deltaTableName, - sharedDeltaTable, - versionAsOf = Some(1) - ) - prepareMockedClientGetTableVersion(deltaTableName, sharedDeltaTable) - - withSQLConf(getDeltaSharingClassesSQLConf.toSeq: _*) { - val profileFile = prepareProfileFile(tempDir) - testAutoResolve( - s"${profileFile.getCanonicalPath}#share1.default.$sharedDeltaTable", - s"share1.default.$sharedDeltaTable", - "delta" - ) - } - - // Test for parquet format response - val sharedParquetTable = "shared_parquet_table" - prepareMockedClientAndFileSystemResultForParquet( - deltaTableName, - sharedParquetTable - ) - prepareMockedClientAndFileSystemResultForParquet( - deltaTableName, - sharedParquetTable, - versionAsOf = Some(1) - ) - prepareMockedClientGetTableVersion(deltaTableName, sharedParquetTable) - - withSQLConf(getDeltaSharingClassesSQLConf.toSeq: _*) { - val profileFile = prepareProfileFile(tempDir) - testAutoResolve( - s"${profileFile.getCanonicalPath}#share1.default.$sharedParquetTable", - s"share1.default.$sharedParquetTable", - "parquet" - ) - } - } - } - } - - test("DeltaSharingDataSource able to read data with filters and select") { - withTempDir { tempDir => - val deltaTableName = "delta_table_filters" - withTable(deltaTableName) { - createSimpleTable(deltaTableName, enableCdf = false) - sql(s"""INSERT INTO $deltaTableName VALUES (1, "first"), (2, "first")""") - sql(s"""INSERT INTO $deltaTableName VALUES (1, "second"), (2, "second")""") - sql(s"""INSERT INTO $deltaTableName VALUES (1, "third"), (2, "third")""") - - val sharedTableName = "shared_table_filters" - prepareMockedClientAndFileSystemResult(deltaTableName, sharedTableName) - prepareMockedClientGetTableVersion(deltaTableName, sharedTableName) - - // The files returned from delta sharing client are the same for these queries. - // This is to test the filters are passed correctly to TahoeLogFileIndex for the local delta - // log. - def testFiltersAndSelect(tablePath: String): Unit = { - var expected = Seq(Row(1, "first"), Row(1, "second"), Row(1, "third"), Row(2, "second")) - var df = spark.read - .format("deltaSharing") - .option("responseFormat", "delta") - .load(tablePath) - .filter(col("c1") === 1 || col("c2") === "second") - checkAnswer(df, expected) - - expected = Seq(Row(1, "first"), Row(1, "second"), Row(1, "third")) - df = spark.read - .format("deltaSharing") - .option("responseFormat", "delta") - .load(tablePath) - .filter(col("c1") === 1) - checkAnswer(df, expected) - - expected = Seq(Row(1, "second"), Row(2, "second")) - df = spark.read - .format("deltaSharing") - .option("responseFormat", "delta") - .load(tablePath) - .filter(col("c2") === "second") - checkAnswer(df, expected) - - // with select as well - expected = Seq(Row(1), Row(1), Row(1), Row(2), Row(2), Row(2)) - df = spark.read - .format("deltaSharing") - .option("responseFormat", "delta") - .load(tablePath) - .select("c1") - checkAnswer(df, expected) - - expected = Seq( - Row("first"), - Row("first"), - Row("second"), - Row("second"), - Row("third"), - Row("third") - ) - df = spark.read - .format("deltaSharing") - .option("responseFormat", "delta") - .load(tablePath) - .select("c2") - checkAnswer(df, expected) - - expected = Seq(Row(1), Row(2)) - df = spark.read - .format("deltaSharing") - .option("responseFormat", "delta") - .load(tablePath) - .filter(col("c2") === "second") - .select("c1") - checkAnswer(df, expected) - } - - withSQLConf(getDeltaSharingClassesSQLConf.toSeq: _*) { - val profileFile = prepareProfileFile(tempDir) - testFiltersAndSelect(s"${profileFile.getCanonicalPath}#share1.default.$sharedTableName") - } - } - } - } - - test("DeltaSharingDataSource able to read data for time travel queries") { - withTempDir { tempDir => - val deltaTableName = "delta_table_time_travel" - withTable(deltaTableName) { - createTable(deltaTableName) - - sql( - s"INSERT INTO $deltaTableName" + - """ VALUES (1, "one", "2023-01-01", "2023-01-01 00:00:00")""".stripMargin - ) - sql( - s"INSERT INTO $deltaTableName" + - """ VALUES (2, "two", "2023-02-02", "2023-02-02 00:00:00")""".stripMargin - ) - sql( - s"INSERT INTO $deltaTableName" + - """ VALUES (3, "three", "2023-03-03", "2023-03-03 00:00:00")""".stripMargin - ) - - val sharedTableNameV1 = "shared_table_v1" - prepareMockedClientAndFileSystemResult( - deltaTable = deltaTableName, - sharedTable = sharedTableNameV1, - versionAsOf = Some(1L) - ) - - def testVersionAsOf1(tablePath: String): Unit = { - val dfV1 = spark.read - .format("deltaSharing") - .option("responseFormat", "delta") - .option("versionAsOf", 1) - .load(tablePath) - val expectedV1 = Seq( - Row(1, "one", sqlDate("2023-01-01"), sqlTimestamp("2023-01-01 00:00:00")) - ) - checkAnswer(dfV1, expectedV1) - } - withSQLConf(getDeltaSharingClassesSQLConf.toSeq: _*) { - val profileFile = prepareProfileFile(tempDir) - testVersionAsOf1(s"${profileFile.getCanonicalPath}#share1.default.$sharedTableNameV1") - } - - // using different table name because spark caches the content read from a file, i.e., - // the delta log from 0.json. - // TODO: figure out how to get a per query id and use it in getCustomTablePath to - // differentiate the same table used in different queries. - // TODO: Also check if it's possible to disable the file cache. - val sharedTableNameV3 = "shared_table_v3" - prepareMockedClientAndFileSystemResult( - deltaTable = deltaTableName, - sharedTable = sharedTableNameV3, - versionAsOf = Some(3L) - ) - - def testVersionAsOf3(tablePath: String): Unit = { - val dfV3 = spark.read - .format("deltaSharing") - .option("responseFormat", "delta") - .option("versionAsOf", 3) - .load(tablePath) - val expectedV3 = Seq( - Row(1, "one", sqlDate("2023-01-01"), sqlTimestamp("2023-01-01 00:00:00")), - Row(2, "two", sqlDate("2023-02-02"), sqlTimestamp("2023-02-02 00:00:00")), - Row(3, "three", sqlDate("2023-03-03"), sqlTimestamp("2023-03-03 00:00:00")) - ) - checkAnswer(dfV3, expectedV3) - } - withSQLConf(getDeltaSharingClassesSQLConf.toSeq: _*) { - val profileFile = prepareProfileFile(tempDir) - testVersionAsOf3(s"${profileFile.getCanonicalPath}#share1.default.$sharedTableNameV3") - } - - val sharedTableNameTs = "shared_table_ts" - // Given the result of delta sharing rpc is mocked, the actual value of the timestampStr - // can be any thing that's valid for DeltaSharingOptions, and formattedTimestamp is the - // parsed result and will be sent in the delta sharing rpc. - val timestampStr = "2023-01-01 00:00:00" - val formattedTimestamp = "2023-01-01T08:00:00Z" - - prepareMockedClientGetTableVersion(deltaTableName, sharedTableNameTs) - prepareMockedClientAndFileSystemResult( - deltaTable = deltaTableName, - sharedTable = sharedTableNameTs, - versionAsOf = None, - timestampAsOf = Some(formattedTimestamp) - ) - - def testTimestampQuery(tablePath: String): Unit = { - val dfTs = spark.read - .format("deltaSharing") - .option("responseFormat", "delta") - .option("timestampAsOf", timestampStr) - .load(tablePath) - val expectedTs = Seq( - Row(1, "one", sqlDate("2023-01-01"), sqlTimestamp("2023-01-01 00:00:00")), - Row(2, "two", sqlDate("2023-02-02"), sqlTimestamp("2023-02-02 00:00:00")), - Row(3, "three", sqlDate("2023-03-03"), sqlTimestamp("2023-03-03 00:00:00")) - ) - checkAnswer(dfTs, expectedTs) - } - withSQLConf(getDeltaSharingClassesSQLConf.toSeq: _*) { - val profileFile = prepareProfileFile(tempDir) - testTimestampQuery(s"${profileFile.getCanonicalPath}#share1.default.$sharedTableNameTs") - } - } - } - } - - test("DeltaSharingDataSource able to read data with more entries") { - withTempDir { tempDir => - val deltaTableName = "delta_table_more" - withTable(deltaTableName) { - createSimpleTable(deltaTableName, enableCdf = false) - // The table operations take about 6~10 seconds. - for (i <- 0 to 9) { - val iteration = s"iteration $i" - val valuesBuilder = Seq.newBuilder[String] - for (j <- 0 to 49) { - valuesBuilder += s"""(${i * 10 + j}, "$iteration")""" - } - sql(s"INSERT INTO $deltaTableName VALUES ${valuesBuilder.result().mkString(",")}") - } - - val sharedTableName = "shared_table_more" - prepareMockedClientAndFileSystemResult(deltaTableName, sharedTableName) - prepareMockedClientGetTableVersion(deltaTableName, sharedTableName) - - val expectedSchema: StructType = new StructType() - .add("c1", IntegerType) - .add("c2", StringType) - val expected = spark.read.format("delta").table(deltaTableName) - - def test(tablePath: String): Unit = { - assert( - expectedSchema == spark.read - .format("deltaSharing") - .option("responseFormat", "delta") - .load(tablePath) - .schema - ) - val df = - spark.read.format("deltaSharing").option("responseFormat", "delta").load(tablePath) - checkAnswer(df, expected) - } - - withSQLConf(getDeltaSharingClassesSQLConf.toSeq: _*) { - val profileFile = prepareProfileFile(tempDir) - test(s"${profileFile.getCanonicalPath}#share1.default.$sharedTableName") - } - } - } - } - - test("DeltaSharingDataSource able to read data with join on the same table") { - withTempDir { tempDir => - val deltaTableName = "delta_table_join" - withTable(deltaTableName) { - createSimpleTable(deltaTableName, enableCdf = false) - sql(s"""INSERT INTO $deltaTableName VALUES (1, "first"), (2, "first")""") - sql(s"""INSERT INTO $deltaTableName VALUES (1, "second"), (2, "second")""") - sql(s"""INSERT INTO $deltaTableName VALUES (1, "third"), (2, "third")""") - - val sharedTableName = "shared_table_join" - prepareMockedClientAndFileSystemResult(deltaTableName, sharedTableName) - prepareMockedClientGetTableVersion(deltaTableName, sharedTableName) - prepareMockedClientAndFileSystemResult( - deltaTableName, - sharedTableName, - versionAsOf = Some(1L) - ) - - def testJoin(tablePath: String): Unit = { - // Query the same latest version - val deltaDfLatest = spark.read.format("delta").table(deltaTableName) - val deltaDfV1 = spark.read.format("delta").option("versionAsOf", 1).table(deltaTableName) - val sharingDfLatest = - spark.read.format("deltaSharing").option("responseFormat", "delta").load(tablePath) - val sharingDfV1 = - spark.read - .format("deltaSharing") - .option("responseFormat", "delta") - .option("versionAsOf", 1) - .load(tablePath) - - var deltaDfJoined = deltaDfLatest.join(deltaDfLatest, "c1") - var sharingDfJoined = sharingDfLatest.join(sharingDfLatest, "c1") - // CheckAnswer ensures that delta sharing produces the same result as delta. - // The check on the size is used to double check that a valid dataframe is generated. - checkAnswer(deltaDfJoined, sharingDfJoined) - assert(sharingDfJoined.count() > 0) - - // Query the same versionAsOf - deltaDfJoined = deltaDfV1.join(deltaDfV1, "c1") - sharingDfJoined = sharingDfV1.join(sharingDfV1, "c1") - checkAnswer(deltaDfJoined, sharingDfJoined) - assert(sharingDfJoined.count() > 0) - - // Query with different versions - deltaDfJoined = deltaDfLatest.join(deltaDfV1, "c1") - sharingDfJoined = sharingDfLatest.join(sharingDfV1, "c1") - checkAnswer(deltaDfJoined, sharingDfJoined) - // Size is 6 because for each of the 6 rows in latest, there is 1 row with the same c1 - // value in v1. - assert(sharingDfJoined.count() > 0) - } - - withSQLConf(getDeltaSharingClassesSQLConf.toSeq: _*) { - val profileFile = prepareProfileFile(tempDir) - testJoin(s"${profileFile.getCanonicalPath}#share1.default.$sharedTableName") - } - } - } - } - - test("DeltaSharingDataSource able to read empty data") { - withTempDir { tempDir => - val deltaTableName = "delta_table_empty" - withTable(deltaTableName) { - createSimpleTable(deltaTableName, enableCdf = true) - sql(s"""INSERT INTO $deltaTableName VALUES (1, "first"), (2, "first")""") - sql(s"""INSERT INTO $deltaTableName VALUES (1, "second"), (2, "second")""") - sql(s"DELETE FROM $deltaTableName WHERE c1 <= 2") - // This command is just to create an empty table version at version 4. - spark.sql(s"ALTER TABLE $deltaTableName SET TBLPROPERTIES('delta.minReaderVersion' = 1)") - - val sharedTableName = "shared_table_empty" - prepareMockedClientAndFileSystemResult(deltaTableName, sharedTableName) - prepareMockedClientGetTableVersion(deltaTableName, sharedTableName) - - def testEmpty(tablePath: String): Unit = { - // linzhou - val deltaDf = spark.read.format("delta").table(deltaTableName) - val sharingDf = - spark.read.format("deltaSharing").option("responseFormat", "delta").load(tablePath) - checkAnswer(deltaDf, sharingDf) - assert(sharingDf.count() == 0) - - val deltaCdfDf = spark.read - .format("delta") - .option("readChangeFeed", "true") - .option("startingVersion", 4) - .table(deltaTableName) - val sharingCdfDf = spark.read - .format("deltaSharing") - .option("responseFormat", "delta") - .option("readChangeFeed", "true") - .option("startingVersion", 4) - .load(tablePath) - checkAnswer(deltaCdfDf, sharingCdfDf) - assert(sharingCdfDf.count() == 0) - } - - // There's only metadata change but not actual files in version 4. - prepareMockedClientAndFileSystemResultForCdf( - deltaTableName, - sharedTableName, - startingVersion = 4 - ) - - withSQLConf(getDeltaSharingClassesSQLConf.toSeq: _*) { - val profileFile = prepareProfileFile(tempDir) - testEmpty(s"${profileFile.getCanonicalPath}#share1.default.$sharedTableName") - } - } - } - } - - /** - * cdf queries - */ - test("DeltaSharingDataSource able to read data for simple cdf query") { - withTempDir { tempDir => - val deltaTableName = "delta_table_cdf" - withTable(deltaTableName) { - sql(s""" - |CREATE TABLE $deltaTableName (c1 INT, c2 STRING) USING DELTA PARTITIONED BY (c2) - |TBLPROPERTIES (delta.enableChangeDataFeed = true) - |""".stripMargin) - // 2 inserts in version 1, 1 with c1=2 - sql(s"""INSERT INTO $deltaTableName VALUES (1, "one"), (2, "two")""") - // 1 insert in version 2, 0 with c1=2 - sql(s"""INSERT INTO $deltaTableName VALUES (3, "two")""") - // 0 operations in version 3 - sql(s"""OPTIMIZE $deltaTableName""") - // 2 updates in version 4, 2 with c1=2 - sql(s"""UPDATE $deltaTableName SET c2="new two" where c1=2""") - // 1 delete in version 5, 1 with c1=2 - sql(s"""DELETE FROM $deltaTableName WHERE c1 = 2""") - - val sharedTableName = "shard_table_cdf" - prepareMockedClientGetTableVersion(deltaTableName, sharedTableName) - - Seq(0, 1, 2, 3, 4, 5).foreach { startingVersion => - val ts = getTimeStampForVersion(deltaTableName, startingVersion) - val startingTimestamp = DateTimeUtils.toJavaTimestamp(ts * 1000).toInstant.toString - prepareMockedClientAndFileSystemResultForCdf( - deltaTableName, - sharedTableName, - startingVersion, - Some(startingTimestamp) - ) - - def test(tablePath: String): Unit = { - val expectedSchema: StructType = new StructType() - .add("c1", IntegerType) - .add("c2", StringType) - .add("_change_type", StringType) - .add("_commit_version", LongType) - .add("_commit_timestamp", TimestampType) - val schema = spark.read - .format("deltaSharing") - .option("responseFormat", "delta") - .option("readChangeFeed", "true") - .option("startingVersion", startingVersion) - .load(tablePath) - .schema - assert(expectedSchema == schema) - - val expected = spark.read - .format("delta") - .option("readChangeFeed", "true") - .option("startingVersion", startingVersion) - .table(deltaTableName) - val df = spark.read - .format("deltaSharing") - .option("responseFormat", "delta") - .option("readChangeFeed", "true") - .option("startingVersion", startingVersion) - .load(tablePath) - checkAnswer(df, expected) - assert(df.count() > 0) - } - - def testFiltersAndSelect(tablePath: String): Unit = { - val expectedSchema: StructType = new StructType() - .add("c2", StringType) - .add("_change_type", StringType) - .add("_commit_version", LongType) - val schema = spark.read - .format("deltaSharing") - .option("responseFormat", "delta") - .option("readChangeFeed", "true") - .option("startingVersion", startingVersion) - .load(tablePath) - .select("c2", "_change_type", "_commit_version") - .schema - assert(expectedSchema == schema) - - val expected = spark.read - .format("delta") - .option("readChangeFeed", "true") - .option("startingVersion", startingVersion) - .table(deltaTableName) - .select("c2", "_change_type", "_commit_version") - val dfVersion = spark.read - .format("deltaSharing") - .option("responseFormat", "delta") - .option("readChangeFeed", "true") - .option("startingVersion", startingVersion) - .load(tablePath) - .select("c2", "_change_type", "_commit_version") - checkAnswer(dfVersion, expected) - val dfTime = spark.read - .format("deltaSharing") - .option("responseFormat", "delta") - .option("readChangeFeed", "true") - .option("startingTimestamp", startingTimestamp) - .load(tablePath) - .select("c2", "_change_type", "_commit_version") - checkAnswer(dfTime, expected) - assert(dfTime.count() > 0) - - val expectedFiltered = spark.read - .format("delta") - .option("readChangeFeed", "true") - .option("startingVersion", startingVersion) - .table(deltaTableName) - .select("c2", "_change_type", "_commit_version") - .filter(col("c1") === 2) - val dfFiltered = spark.read - .format("deltaSharing") - .option("responseFormat", "delta") - .option("readChangeFeed", "true") - .option("startingVersion", startingVersion) - .load(tablePath) - .select("c2", "_change_type", "_commit_version") - .filter(col("c1") === 2) - checkAnswer(dfFiltered, expectedFiltered) - assert(dfFiltered.count() > 0) - } - - withSQLConf(getDeltaSharingClassesSQLConf.toSeq: _*) { - val profileFile = prepareProfileFile(tempDir) - test(profileFile.getCanonicalPath + s"#share1.default.$sharedTableName") - testFiltersAndSelect( - profileFile.getCanonicalPath + s"#share1.default.$sharedTableName" - ) - } - } - - // test join on the same table in cdf query - def testJoin(tablePath: String): Unit = { - val deltaV0 = spark.read - .format("delta") - .option("readChangeFeed", "true") - .option("startingVersion", 0) - .table(deltaTableName) - val deltaV3 = spark.read - .format("delta") - .option("readChangeFeed", "true") - .option("startingVersion", 3) - .table(deltaTableName) - val sharingV0 = spark.read - .format("deltaSharing") - .option("responseFormat", "delta") - .option("readChangeFeed", "true") - .option("startingVersion", 0) - .load(tablePath) - val sharingV3 = spark.read - .format("deltaSharing") - .option("responseFormat", "delta") - .option("readChangeFeed", "true") - .option("startingVersion", 3) - .load(tablePath) - - def testJoinedDf( - deltaLeft: DataFrame, - deltaRight: DataFrame, - sharingLeft: DataFrame, - sharingRight: DataFrame, - expectedSize: Int): Unit = { - val deltaJoined = deltaLeft.join(deltaRight, usingColumns = Seq("c1", "c2")) - val sharingJoined = sharingLeft.join(sharingRight, usingColumns = Seq("c1", "c2")) - checkAnswer(deltaJoined, sharingJoined) - assert(sharingJoined.count() > 0) - } - testJoinedDf(deltaV0, deltaV0, sharingV0, sharingV0, 10) - testJoinedDf(deltaV3, deltaV3, sharingV3, sharingV3, 5) - testJoinedDf(deltaV0, deltaV3, sharingV0, sharingV3, 6) - } - - withSQLConf(getDeltaSharingClassesSQLConf.toSeq: _*) { - val profileFile = prepareProfileFile(tempDir) - testJoin(profileFile.getCanonicalPath + s"#share1.default.$sharedTableName") - } - } - } - } - - test("DeltaSharingDataSource able to read data for cdf query with more entries") { - withTempDir { tempDir => - val deltaTableName = "delta_table_cdf_more" - withTable(deltaTableName) { - createSimpleTable(deltaTableName, enableCdf = true) - // The table operations take about 20~30 seconds. - for (i <- 0 to 9) { - val iteration = s"iteration $i" - val valuesBuilder = Seq.newBuilder[String] - for (j <- 0 to 49) { - valuesBuilder += s"""(${i * 10 + j}, "$iteration")""" - } - sql(s"INSERT INTO $deltaTableName VALUES ${valuesBuilder.result().mkString(",")}") - sql(s"""UPDATE $deltaTableName SET c1 = c1 + 100 where c2 = "${iteration}"""") - sql(s"""DELETE FROM $deltaTableName where c2 = "${iteration}"""") - } - - val sharedTableName = "shard_table_cdf_more" - prepareMockedClientGetTableVersion(deltaTableName, sharedTableName) - Seq(0, 10, 20, 30).foreach { startingVersion => - prepareMockedClientAndFileSystemResultForCdf( - deltaTableName, - sharedTableName, - startingVersion - ) - - val expected = spark.read - .format("delta") - .option("readChangeFeed", "true") - .option("startingVersion", startingVersion) - .table(deltaTableName) - - def test(tablePath: String): Unit = { - val df = spark.read - .format("deltaSharing") - .option("responseFormat", "delta") - .option("readChangeFeed", "true") - .option("startingVersion", startingVersion) - .load(tablePath) - checkAnswer(df, expected) - assert(df.count() > 0) - } - - withSQLConf(getDeltaSharingClassesSQLConf.toSeq: _*) { - val profileFile = prepareProfileFile(tempDir) - test(profileFile.getCanonicalPath + s"#share1.default.$sharedTableName") - } - } - } - } - } -} - -class DeltaSharingDataSourceDeltaSuite extends DeltaSharingDataSourceDeltaSuiteBase {} diff --git a/sharing/src/test/scala/io/delta/sharing/spark/DeltaSharingDataSourceDeltaTestUtils.scala b/sharing/src/test/scala/io/delta/sharing/spark/DeltaSharingDataSourceDeltaTestUtils.scala deleted file mode 100644 index 0b268936d2b..00000000000 --- a/sharing/src/test/scala/io/delta/sharing/spark/DeltaSharingDataSourceDeltaTestUtils.scala +++ /dev/null @@ -1,661 +0,0 @@ -/* - * Copyright (2021) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.delta.sharing.spark - -import java.io.File -import java.nio.charset.StandardCharsets.UTF_8 - -import scala.collection.JavaConverters._ -import scala.collection.mutable.ArrayBuffer - -import org.apache.spark.sql.delta.{DeltaLog, Snapshot} -import org.apache.spark.sql.delta.actions.{ - Action, - AddCDCFile, - AddFile, - DeletionVectorDescriptor, - Metadata, - RemoveFile -} -import org.apache.spark.sql.delta.deletionvectors.{ - RoaringBitmapArray, - RoaringBitmapArrayFormat -} -import org.apache.spark.sql.delta.util.{FileNames, JsonUtils} -import com.google.common.hash.Hashing -import io.delta.sharing.client.model.{ - AddFile => ClientAddFile, - Metadata => ClientMetadata, - Protocol => ClientProtocol -} -import io.delta.sharing.spark.model.{ - DeltaSharingFileAction, - DeltaSharingMetadata, - DeltaSharingProtocol -} -import org.apache.commons.io.FileUtils -import org.apache.hadoop.fs.{FileSystem, Path} - -import org.apache.spark.SparkConf -import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.test.SharedSparkSession - -trait DeltaSharingDataSourceDeltaTestUtils extends SharedSparkSession { - - override def beforeAll(): Unit = { - super.beforeAll() - // close DeltaSharingFileSystem to avoid impact from other unit tests. - FileSystem.closeAll() - } - - override protected def sparkConf: SparkConf = { - super.sparkConf - .set("spark.delta.sharing.preSignedUrl.expirationMs", "30000") - .set("spark.delta.sharing.driver.refreshCheckIntervalMs", "3000") - .set("spark.delta.sharing.driver.refreshThresholdMs", "10000") - .set("spark.delta.sharing.driver.accessThresholdToExpireMs", "300000") - } - - private[spark] def removePartitionPrefix(filePath: String): String = { - filePath.split("/").last - } - - private def getResponseDVAndId( - sharedTable: String, - deletionVector: DeletionVectorDescriptor): (DeletionVectorDescriptor, String) = { - if (deletionVector != null) { - if (deletionVector.storageType == DeletionVectorDescriptor.INLINE_DV_MARKER) { - (deletionVector, Hashing.sha256().hashString(deletionVector.uniqueId, UTF_8).toString) - } else { - val dvPath = deletionVector.absolutePath(new Path("not-used")) - ( - deletionVector.copy( - pathOrInlineDv = TestDeltaSharingFileSystem.encode(sharedTable, dvPath.getName), - storageType = DeletionVectorDescriptor.PATH_DV_MARKER - ), - Hashing.sha256().hashString(deletionVector.uniqueId, UTF_8).toString - ) - } - } else { - (null, null) - } - } - - private def isDataFile(filePath: String): Boolean = { - filePath.endsWith(".parquet") || filePath.endsWith(".bin") - } - - // Convert from delta AddFile to DeltaSharingFileAction to serialize to json. - private def getDeltaSharingFileActionForAddFile( - addFile: AddFile, - sharedTable: String, - version: Long, - timestamp: Long): DeltaSharingFileAction = { - val parquetFile = removePartitionPrefix(addFile.path) - - val (responseDV, dvFileId) = getResponseDVAndId(sharedTable, addFile.deletionVector) - - DeltaSharingFileAction( - id = Hashing.sha256().hashString(parquetFile, UTF_8).toString, - version = version, - timestamp = timestamp, - deletionVectorFileId = dvFileId, - deltaSingleAction = addFile - .copy( - path = TestDeltaSharingFileSystem.encode(sharedTable, parquetFile), - deletionVector = responseDV - ) - .wrap - ) - } - - // Convert from delta RemoveFile to DeltaSharingFileAction to serialize to json. - // scalastyle:off removeFile - private def getDeltaSharingFileActionForRemoveFile( - removeFile: RemoveFile, - sharedTable: String, - version: Long, - timestamp: Long): DeltaSharingFileAction = { - val parquetFile = removePartitionPrefix(removeFile.path) - - val (responseDV, dvFileId) = getResponseDVAndId(sharedTable, removeFile.deletionVector) - - DeltaSharingFileAction( - id = Hashing.sha256().hashString(parquetFile, UTF_8).toString, - version = version, - timestamp = timestamp, - deletionVectorFileId = dvFileId, - deltaSingleAction = removeFile - .copy( - path = TestDeltaSharingFileSystem.encode(sharedTable, parquetFile), - deletionVector = responseDV - ) - .wrap - ) - // scalastyle:on removeFile - } - - // Reset the result for client.GetTableVersion for the sharedTable based on the latest table - // version of the deltaTable, use BlockManager to store the result. - private[spark] def prepareMockedClientGetTableVersion( - deltaTable: String, - sharedTable: String): Unit = { - val snapshotToUse = getSnapshotToUse(deltaTable, None) - DeltaSharingUtils.overrideSingleBlock[Long]( - blockId = TestClientForDeltaFormatSharing.getBlockId(sharedTable, "getTableVersion"), - value = snapshotToUse.version - ) - } - - def getTimeStampForVersion(deltaTable: String, version: Long): Long = { - val snapshotToUse = getSnapshotToUse(deltaTable, None) - FileUtils - .listFiles(new File(snapshotToUse.deltaLog.logPath.toUri()), null, true) - .asScala - .foreach { f => - if (FileNames.isDeltaFile(new Path(f.getName))) { - if (FileNames.getFileVersion(new Path(f.getName)) == version) { - return f.lastModified - } - } - } - 0 - } - - // Prepare the result(Protocol and Metadata) for client.GetMetadata for the sharedTable based on - // the latest table info of the deltaTable, store them in BlockManager. - private[spark] def prepareMockedClientMetadata(deltaTable: String, sharedTable: String): Unit = { - val snapshotToUse = getSnapshotToUse(deltaTable, None) - val dsProtocol: DeltaSharingProtocol = DeltaSharingProtocol(snapshotToUse.protocol) - val dsMetadata: DeltaSharingMetadata = DeltaSharingMetadata( - deltaMetadata = snapshotToUse.metadata - ) - - // Put the metadata in blockManager for DeltaSharingClient to return for getMetadata. - DeltaSharingUtils.overrideIteratorBlock[String]( - blockId = TestClientForDeltaFormatSharing.getBlockId(sharedTable, "getMetadata"), - values = Seq(dsProtocol.json, dsMetadata.json).toIterator - ) - } - - private def updateAddFileWithInlineDV( - addFile: AddFile, - inlineDvFormat: RoaringBitmapArrayFormat.Value, - bitmap: RoaringBitmapArray): AddFile = { - val dv = DeletionVectorDescriptor.inlineInLog( - bitmap.serializeAsByteArray(inlineDvFormat), - bitmap.cardinality - ) - addFile - .removeRows( - deletionVector = dv, - updateStats = true - ) - ._1 - } - - private def updateDvPathToCount( - addFile: AddFile, - pathToCount: scala.collection.mutable.Map[String, Int]): Unit = { - if (addFile.deletionVector != null && - addFile.deletionVector.storageType != DeletionVectorDescriptor.INLINE_DV_MARKER) { - val dvPath = addFile.deletionVector.pathOrInlineDv - pathToCount.put(dvPath, pathToCount.getOrElse(dvPath, 0) + 1) - } - } - - // Sort by id in decreasing order. - private def deltaSharingFileActionDecreaseOrderFunc( - f1: model.DeltaSharingFileAction, - f2: model.DeltaSharingFileAction): Boolean = { - f1.id > f2.id - } - - // Sort by id in increasing order. - private def deltaSharingFileActionIncreaseOrderFunc( - f1: model.DeltaSharingFileAction, - f2: model.DeltaSharingFileAction): Boolean = { - f1.id < f2.id - } - - private def getSnapshotToUse(deltaTable: String, versionAsOf: Option[Long]): Snapshot = { - val deltaLog = DeltaLog.forTable(spark, new TableIdentifier(deltaTable)) - if (versionAsOf.isDefined) { - deltaLog.getSnapshotAt(versionAsOf.get) - } else { - deltaLog.update() - } - } - - // This function does 2 jobs: - // 1. Prepare the result for functions of delta sharing rest client, i.e., (Protocol, Metadata) - // for getMetadata, (Protocol, Metadata, and list of lines from delta actions) for getFiles, use - // BlockManager to store the data to make them available across different classes. All the lines - // are for responseFormat=parquet. - // 2. Put the parquet file in blockManager for DeltaSharingFileSystem to load bytes out of it. - private[spark] def prepareMockedClientAndFileSystemResultForParquet( - deltaTable: String, - sharedTable: String, - versionAsOf: Option[Long] = None): Unit = { - val lines = Seq.newBuilder[String] - var totalSize = 0L - val clientAddFilesArrayBuffer = ArrayBuffer[ClientAddFile]() - - // To prepare faked delta sharing responses with needed files for DeltaSharingClient. - val snapshotToUse = getSnapshotToUse(deltaTable, versionAsOf) - - snapshotToUse.allFiles.collect().foreach { addFile => - val parquetFile = removePartitionPrefix(addFile.path) - val clientAddFile = ClientAddFile( - url = TestDeltaSharingFileSystem.encode(sharedTable, parquetFile), - id = Hashing.md5().hashString(parquetFile, UTF_8).toString, - partitionValues = addFile.partitionValues, - size = addFile.size, - stats = null, - version = snapshotToUse.version, - timestamp = snapshotToUse.timestamp - ) - totalSize = totalSize + addFile.size - clientAddFilesArrayBuffer += clientAddFile - } - - // Scan through the parquet files of the local delta table, and prepare the data of parquet file - // reading in DeltaSharingFileSystem. - val files = - FileUtils.listFiles(new File(snapshotToUse.deltaLog.dataPath.toUri()), null, true).asScala - files.foreach { f => - val filePath = f.getCanonicalPath - if (isDataFile(filePath)) { - // Put the parquet file in blockManager for DeltaSharingFileSystem to load bytes out of it. - DeltaSharingUtils.overrideIteratorBlock[Byte]( - blockId = TestDeltaSharingFileSystem.getBlockId(sharedTable, f.getName), - values = FileUtils.readFileToByteArray(f).toIterator - ) - } - } - - val clientProtocol = ClientProtocol(minReaderVersion = 1) - // This is specifically to set the size of the metadata. - val deltaMetadata = snapshotToUse.metadata - val clientMetadata = ClientMetadata( - id = deltaMetadata.id, - name = deltaMetadata.name, - description = deltaMetadata.description, - schemaString = deltaMetadata.schemaString, - configuration = deltaMetadata.configuration, - partitionColumns = deltaMetadata.partitionColumns, - size = totalSize - ) - lines += JsonUtils.toJson(clientProtocol.wrap) - lines += JsonUtils.toJson(clientMetadata.wrap) - clientAddFilesArrayBuffer.toSeq.foreach { clientAddFile => - lines += JsonUtils.toJson(clientAddFile.wrap) - } - - // Put the metadata in blockManager for DeltaSharingClient to return metadata when being asked. - DeltaSharingUtils.overrideIteratorBlock[String]( - blockId = TestClientForDeltaFormatSharing.getBlockId( - sharedTableName = sharedTable, - queryType = "getMetadata", - versionAsOf = versionAsOf - ), - values = Seq( - JsonUtils.toJson(clientProtocol.wrap), - JsonUtils.toJson(clientMetadata.wrap) - ).toIterator - ) - - // Put the delta log (list of actions) in blockManager for DeltaSharingClient to return as the - // http response when getFiles is called. - DeltaSharingUtils.overrideIteratorBlock[String]( - blockId = TestClientForDeltaFormatSharing.getBlockId( - sharedTableName = sharedTable, - queryType = "getFiles", - versionAsOf = versionAsOf - ), - values = lines.result().toIterator - ) - } - - // This function does 2 jobs: - // 1. Prepare the result for functions of delta sharing rest client, i.e., (Protocol, Metadata) - // for getMetadata, (Protocol, Metadata, and list of lines from delta actions) for getFiles, use - // BlockManager to store the data to make them available across different classes. - // 2. Put the parquet file in blockManager for DeltaSharingFileSystem to load bytes out of it. - private[spark] def prepareMockedClientAndFileSystemResult( - deltaTable: String, - sharedTable: String, - versionAsOf: Option[Long] = None, - timestampAsOf: Option[String] = None, - inlineDvFormat: Option[RoaringBitmapArrayFormat.Value] = None, - assertMultipleDvsInOneFile: Boolean = false, - reverseFileOrder: Boolean = false): Unit = { - val lines = Seq.newBuilder[String] - var totalSize = 0L - - // To prepare faked delta sharing responses with needed files for DeltaSharingClient. - val snapshotToUse = getSnapshotToUse(deltaTable, versionAsOf) - val fileActionsArrayBuffer = ArrayBuffer[model.DeltaSharingFileAction]() - val dvPathToCount = scala.collection.mutable.Map[String, Int]() - snapshotToUse.allFiles.collect().foreach { addFile => - if (assertMultipleDvsInOneFile) { - updateDvPathToCount(addFile, dvPathToCount) - } - - val updatedAdd = if (inlineDvFormat.isDefined) { - // Remove row 0 and 2 in the AddFile. - updateAddFileWithInlineDV(addFile, inlineDvFormat.get, RoaringBitmapArray(0L, 2L)) - } else { - addFile - } - - val dsAddFile = getDeltaSharingFileActionForAddFile( - updatedAdd, - sharedTable, - snapshotToUse.version, - snapshotToUse.timestamp - ) - totalSize = totalSize + addFile.size - fileActionsArrayBuffer += dsAddFile - } - val fileActionSeq = if (reverseFileOrder) { - fileActionsArrayBuffer.toSeq.sortWith(deltaSharingFileActionDecreaseOrderFunc) - } else { - fileActionsArrayBuffer.toSeq.sortWith(deltaSharingFileActionIncreaseOrderFunc) - } - var previousIdOpt: Option[String] = None - fileActionSeq.foreach { fileAction => - if (reverseFileOrder) { - assert( - // Using < instead of <= because there can be a removeFile and addFile pointing to the - // same parquet file which result in the same file id, since id is a hash of file path. - // This is ok because eventually it can read data out of the correct parquet file. - !previousIdOpt.exists(_ < fileAction.id), - s"fileActions must be in decreasing order by id: ${previousIdOpt} is not smaller than" + - s" ${fileAction.id}." - ) - previousIdOpt = Some(fileAction.id) - } - lines += fileAction.json - } - if (assertMultipleDvsInOneFile) { - assert(dvPathToCount.max._2 > 1) - } - - // Scan through the parquet files of the local delta table, and prepare the data of parquet file - // reading in DeltaSharingFileSystem. - val files = - FileUtils.listFiles(new File(snapshotToUse.deltaLog.dataPath.toUri()), null, true).asScala - files.foreach { f => - val filePath = f.getCanonicalPath - if (isDataFile(filePath)) { - // Put the parquet file in blockManager for DeltaSharingFileSystem to load bytes out of it. - DeltaSharingUtils.overrideIteratorBlock[Byte]( - blockId = TestDeltaSharingFileSystem.getBlockId(sharedTable, f.getName), - values = FileUtils.readFileToByteArray(f).toIterator - ) - } - } - - // This is specifically to set the size of the metadata. - val dsMetadata = DeltaSharingMetadata( - deltaMetadata = snapshotToUse.metadata, - size = totalSize - ) - val dsProtocol = DeltaSharingProtocol(deltaProtocol = snapshotToUse.protocol) - // Put the metadata in blockManager for DeltaSharingClient to return metadata when being asked. - DeltaSharingUtils.overrideIteratorBlock[String]( - blockId = TestClientForDeltaFormatSharing.getBlockId( - sharedTableName = sharedTable, - queryType = "getMetadata", - versionAsOf = versionAsOf, - timestampAsOf = timestampAsOf - ), - values = Seq(dsProtocol.json, dsMetadata.json).toIterator - ) - - lines += dsProtocol.json - lines += dsMetadata.json - // Put the delta log (list of actions) in blockManager for DeltaSharingClient to return as the - // http response when getFiles is called. - DeltaSharingUtils.overrideIteratorBlock[String]( - blockId = TestClientForDeltaFormatSharing.getBlockId( - sharedTableName = sharedTable, - queryType = "getFiles", - versionAsOf = versionAsOf, - timestampAsOf = timestampAsOf - ), - values = lines.result().toIterator - ) - } - - private[spark] def prepareMockedClientAndFileSystemResultForStreaming( - deltaTable: String, - sharedTable: String, - startingVersion: Long, - endingVersion: Long, - assertDVExists: Boolean = false): Unit = { - val actionLines = Seq.newBuilder[String] - - var maxVersion = -1L - var totalSize = 0L - - val deltaLog = DeltaLog.forTable(spark, new TableIdentifier(deltaTable)) - val startingSnapshot = deltaLog.getSnapshotAt(startingVersion) - actionLines += DeltaSharingProtocol(deltaProtocol = startingSnapshot.protocol).json - actionLines += DeltaSharingMetadata( - deltaMetadata = startingSnapshot.metadata, - version = startingVersion - ).json - - val logFiles = - FileUtils.listFiles(new File(deltaLog.logPath.toUri()), null, true).asScala - var dvExists = false - logFiles.foreach { f => - if (FileNames.isDeltaFile(new Path(f.getName))) { - val version = FileNames.getFileVersion(new Path(f.getName)) - if (version >= startingVersion && version <= endingVersion) { - // protocol/metadata are processed from startingSnapshot, only process versions greater - // than startingVersion for real actions and possible metadata changes. - maxVersion = maxVersion.max(version) - val timestamp = f.lastModified - - FileUtils.readLines(f).asScala.foreach { l => - val action = Action.fromJson(l) - action match { - case m: Metadata => - actionLines += DeltaSharingMetadata( - deltaMetadata = m, - version = version - ).json - case addFile: AddFile if addFile.dataChange => - // Convert from delta AddFile to DeltaSharingAddFile to serialize to json. - val dsAddFile = - getDeltaSharingFileActionForAddFile(addFile, sharedTable, version, timestamp) - dvExists = dvExists || (dsAddFile.deletionVectorFileId != null) - totalSize = totalSize + addFile.size - actionLines += dsAddFile.json - case removeFile: RemoveFile if removeFile.dataChange => - // scalastyle:off removeFile - val dsRemoveFile = getDeltaSharingFileActionForRemoveFile( - removeFile, - sharedTable, - version, - timestamp - ) - // scalastyle:on removeFile - dvExists = dvExists || (dsRemoveFile.deletionVectorFileId != null) - totalSize = totalSize + removeFile.size.getOrElse(0L) - actionLines += dsRemoveFile.json - case _ => // ignore all other actions such as CommitInfo. - } - } - } - } - } - val dataFiles = - FileUtils.listFiles(new File(deltaLog.dataPath.toUri()), null, true).asScala - dataFiles.foreach { f => - if (isDataFile(f.getCanonicalPath)) { - DeltaSharingUtils.overrideIteratorBlock[Byte]( - blockId = TestDeltaSharingFileSystem.getBlockId(sharedTable, f.getName), - values = FileUtils.readFileToByteArray(f).toIterator - ) - } - } - - if (assertDVExists) { - assert(dvExists, "There should be DV in the files returned from server.") - } - - DeltaSharingUtils.overrideIteratorBlock[String]( - blockId = TestClientForDeltaFormatSharing.getBlockId( - sharedTable, - s"getFiles_${startingVersion}_$endingVersion" - ), - values = actionLines.result().toIterator - ) - } - - private[spark] def prepareMockedClientAndFileSystemResultForCdf( - deltaTable: String, - sharedTable: String, - startingVersion: Long, - startingTimestamp: Option[String] = None, - inlineDvFormat: Option[RoaringBitmapArrayFormat.Value] = None, - assertMultipleDvsInOneFile: Boolean = false): Unit = { - val actionLines = Seq.newBuilder[String] - - var maxVersion = -1L - var totalSize = 0L - - val deltaLog = DeltaLog.forTable(spark, new TableIdentifier(deltaTable)) - val startingSnapshot = deltaLog.getSnapshotAt(startingVersion) - actionLines += DeltaSharingProtocol(deltaProtocol = startingSnapshot.protocol).json - actionLines += DeltaSharingMetadata( - deltaMetadata = startingSnapshot.metadata, - version = startingVersion - ).json - - val dvPathToCount = scala.collection.mutable.Map[String, Int]() - val files = - FileUtils.listFiles(new File(deltaLog.logPath.toUri()), null, true).asScala - files.foreach { f => - if (FileNames.isDeltaFile(new Path(f.getName))) { - val version = FileNames.getFileVersion(new Path(f.getName)) - if (version >= startingVersion) { - // protocol/metadata are processed from startingSnapshot, only process versions greater - // than startingVersion for real actions and possible metadata changes. - maxVersion = maxVersion.max(version) - val timestamp = f.lastModified - FileUtils.readLines(f).asScala.foreach { l => - val action = Action.fromJson(l) - action match { - case m: Metadata => - actionLines += DeltaSharingMetadata( - deltaMetadata = m, - version = version - ).json - case addFile: AddFile if addFile.dataChange => - if (assertMultipleDvsInOneFile) { - updateDvPathToCount(addFile, dvPathToCount) - } - val updatedAdd = if (inlineDvFormat.isDefined) { - // Remove row 0 and 1 in the AddFile. - updateAddFileWithInlineDV(addFile, inlineDvFormat.get, RoaringBitmapArray(0L, 1L)) - } else { - addFile - } - val dsAddFile = - getDeltaSharingFileActionForAddFile(updatedAdd, sharedTable, version, timestamp) - totalSize = totalSize + updatedAdd.size - actionLines += dsAddFile.json - case removeFile: RemoveFile if removeFile.dataChange => - // scalastyle:off removeFile - val dsRemoveFile = getDeltaSharingFileActionForRemoveFile( - removeFile, - sharedTable, - version, - timestamp - ) - // scalastyle:on removeFile - totalSize = totalSize + removeFile.size.getOrElse(0L) - actionLines += dsRemoveFile.json - case cdcFile: AddCDCFile => - val parquetFile = removePartitionPrefix(cdcFile.path) - - // Convert from delta AddCDCFile to DeltaSharingFileAction to serialize to json. - val dsCDCFile = DeltaSharingFileAction( - id = Hashing.sha256().hashString(parquetFile, UTF_8).toString, - version = version, - timestamp = timestamp, - deltaSingleAction = cdcFile - .copy( - path = TestDeltaSharingFileSystem.encode(sharedTable, parquetFile) - ) - .wrap - ) - totalSize = totalSize + cdcFile.size - actionLines += dsCDCFile.json - case _ => // ignore other lines - } - } - } - } - } - val dataFiles = - FileUtils.listFiles(new File(deltaLog.dataPath.toUri()), null, true).asScala - dataFiles.foreach { f => - val filePath = f.getCanonicalPath - if (isDataFile(filePath)) { - DeltaSharingUtils.overrideIteratorBlock[Byte]( - blockId = TestDeltaSharingFileSystem.getBlockId(sharedTable, f.getName), - values = FileUtils.readFileToByteArray(f).toIterator - ) - } - } - - if (assertMultipleDvsInOneFile) { - assert(dvPathToCount.max._2 > 1) - } - - DeltaSharingUtils.overrideIteratorBlock[String]( - blockId = - TestClientForDeltaFormatSharing.getBlockId(sharedTable, s"getCDFFiles_$startingVersion"), - values = actionLines.result().toIterator - ) - if (startingTimestamp.isDefined) { - DeltaSharingUtils.overrideIteratorBlock[String]( - blockId = TestClientForDeltaFormatSharing.getBlockId( - sharedTable, - s"getCDFFiles_${startingTimestamp.get}" - ), - values = actionLines.result().toIterator - ) - } - } - - protected def getDeltaSharingClassesSQLConf: Map[String, String] = { - Map( - "fs.delta-sharing.impl" -> classOf[TestDeltaSharingFileSystem].getName, - "spark.delta.sharing.client.class" -> - classOf[TestClientForDeltaFormatSharing].getName, - "spark.delta.sharing.profile.provider.class" -> - "io.delta.sharing.client.DeltaSharingFileProfileProvider" - ) - } -} diff --git a/sharing/src/test/scala/io/delta/sharing/spark/DeltaSharingFileIndexSuite.scala b/sharing/src/test/scala/io/delta/sharing/spark/DeltaSharingFileIndexSuite.scala index 7366bbf952e..9c1b4036fac 100644 --- a/sharing/src/test/scala/io/delta/sharing/spark/DeltaSharingFileIndexSuite.scala +++ b/sharing/src/test/scala/io/delta/sharing/spark/DeltaSharingFileIndexSuite.scala @@ -30,7 +30,7 @@ import io.delta.sharing.client.util.JsonUtils import org.apache.commons.io.FileUtils import org.apache.hadoop.fs.Path -import org.apache.spark.SparkEnv +import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.delta.sharing.{PreSignedUrlCache, PreSignedUrlFetcher} import org.apache.spark.sql.{QueryTest, SparkSession} import org.apache.spark.sql.catalyst.expressions.{ @@ -39,6 +39,7 @@ import org.apache.spark.sql.catalyst.expressions.{ Literal => SqlLiteral } import org.apache.spark.sql.delta.sharing.DeltaSharingTestSparkUtils +import org.apache.spark.sql.test.{SharedSparkSession} import org.apache.spark.sql.types.{FloatType, IntegerType} private object TestUtils { @@ -189,7 +190,7 @@ class TestDeltaSharingClientForFileIndex( class DeltaSharingFileIndexSuite extends QueryTest with DeltaSQLCommandTest - with DeltaSharingDataSourceDeltaTestUtils + with SharedSparkSession with DeltaSharingTestSparkUtils { import TestUtils._ @@ -291,6 +292,14 @@ class DeltaSharingFileIndexSuite } } + override protected def sparkConf: SparkConf = { + super.sparkConf + .set("spark.delta.sharing.preSignedUrl.expirationMs", defaultUrlExpirationMs.toString) + .set("spark.delta.sharing.driver.refreshCheckIntervalMs", "1000") + .set("spark.delta.sharing.driver.refreshThresholdMs", "2000") + .set("spark.delta.sharing.driver.accessThresholdToExpireMs", "60000") + } + test("refresh works") { PreSignedUrlCache.registerIfNeeded(SparkEnv.get) @@ -323,8 +332,8 @@ class DeltaSharingFileIndexSuite decodedPath.fileId, 1000 ) - // sleep for 25000ms to ensure that the urls are refreshed. - Thread.sleep(25000) + // sleep for expirationTimeMs to ensure that the urls are refreshed. + Thread.sleep(defaultUrlExpirationMs) // Verify that the url is refreshed as paths(1), not paths(0) anymore. assert(fetcher.getUrl == paths(1)) diff --git a/sharing/src/test/scala/io/delta/sharing/spark/TestClientForDeltaFormatSharing.scala b/sharing/src/test/scala/io/delta/sharing/spark/TestClientForDeltaFormatSharing.scala deleted file mode 100644 index 06f8d8776eb..00000000000 --- a/sharing/src/test/scala/io/delta/sharing/spark/TestClientForDeltaFormatSharing.scala +++ /dev/null @@ -1,270 +0,0 @@ -/* - * Copyright (2021) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.delta.sharing.spark - -import scala.collection.mutable.ArrayBuffer - -import org.apache.spark.sql.delta.util.JsonUtils -import io.delta.sharing.client.{ - DeltaSharingClient, - DeltaSharingProfileProvider, - DeltaSharingRestClient -} -import io.delta.sharing.client.model.{ - AddFile => ClientAddFile, - DeltaTableFiles, - DeltaTableMetadata, - SingleAction, - Table -} - -import org.apache.spark.SparkEnv -import org.apache.spark.storage.BlockId - -/** - * A mocked delta sharing client for DeltaFormatSharing. - * The test suite need to prepare the mocked delta sharing rpc response and store them in - * BlockManager. Then this client will just load the response of return upon rpc call. - */ -private[spark] class TestClientForDeltaFormatSharing( - profileProvider: DeltaSharingProfileProvider, - timeoutInSeconds: Int = 120, - numRetries: Int = 10, - maxRetryDuration: Long = Long.MaxValue, - sslTrustAll: Boolean = false, - forStreaming: Boolean = false, - responseFormat: String = DeltaSharingRestClient.RESPONSE_FORMAT_DELTA, - readerFeatures: String = "", - queryTablePaginationEnabled: Boolean = false, - maxFilesPerReq: Int = 100000) - extends DeltaSharingClient { - - assert( - responseFormat == DeltaSharingRestClient.RESPONSE_FORMAT_PARQUET || - (readerFeatures.contains("deletionVectors") && readerFeatures.contains("columnMapping")), - "deletionVectors and columnMapping should be supported in all types of queries." - ) - - import TestClientForDeltaFormatSharing._ - - override def listAllTables(): Seq[Table] = throw new UnsupportedOperationException("not needed") - - override def getMetadata( - table: Table, - versionAsOf: Option[Long] = None, - timestampAsOf: Option[String] = None): DeltaTableMetadata = { - val iterator = SparkEnv.get.blockManager - .get[String](getBlockId(table.name, "getMetadata", versionAsOf, timestampAsOf)) - .map(_.data.asInstanceOf[Iterator[String]]) - .getOrElse { - throw new IllegalStateException( - s"getMetadata is missing for: ${table.name}, versionAsOf:$versionAsOf, " + - s"timestampAsOf:$timestampAsOf. This shouldn't happen in the unit test." - ) - } - // iterator.toSeq doesn't trigger CompletionIterator in BlockManager which releases the reader - // lock on the underlying block. iterator hasNext does trigger it. - val linesBuilder = Seq.newBuilder[String] - while (iterator.hasNext) { - linesBuilder += iterator.next() - } - if (table.name.contains("shared_parquet_table")) { - val lines = linesBuilder.result() - val protocol = JsonUtils.fromJson[SingleAction](lines(0)).protocol - val metadata = JsonUtils.fromJson[SingleAction](lines(1)).metaData - DeltaTableMetadata( - version = versionAsOf.getOrElse(getTableVersion(table)), - protocol = protocol, - metadata = metadata, - respondedFormat = DeltaSharingRestClient.RESPONSE_FORMAT_PARQUET - ) - } else { - DeltaTableMetadata( - version = versionAsOf.getOrElse(getTableVersion(table)), - lines = linesBuilder.result(), - respondedFormat = DeltaSharingRestClient.RESPONSE_FORMAT_DELTA - ) - } - } - - override def getTableVersion(table: Table, startingTimestamp: Option[String] = None): Long = { - val versionOpt = SparkEnv.get.blockManager.getSingle[Long]( - getBlockId(table.name, "getTableVersion") - ) - val version = versionOpt.getOrElse { - throw new IllegalStateException( - s"getTableVersion is missing for: ${table.name}. This shouldn't happen in the unit test." - ) - } - SparkEnv.get.blockManager.releaseLock(getBlockId(table.name, "getTableVersion")) - version - } - - override def getFiles( - table: Table, - predicates: Seq[String], - limit: Option[Long], - versionAsOf: Option[Long], - timestampAsOf: Option[String], - jsonPredicateHints: Option[String], - refreshToken: Option[String] - ): DeltaTableFiles = { - limit.foreach(lim => TestClientForDeltaFormatSharing.limits.put( - s"${table.share}.${table.schema}.${table.name}", lim)) - TestClientForDeltaFormatSharing.requestedFormat.put( - s"${table.share}.${table.schema}.${table.name}", responseFormat) - val iterator = SparkEnv.get.blockManager - .get[String](getBlockId(table.name, "getFiles", versionAsOf, timestampAsOf)) - .map(_.data.asInstanceOf[Iterator[String]]) - .getOrElse { - throw new IllegalStateException( - s"getFiles is missing for: ${table.name} versionAsOf:$versionAsOf, " + - s"timestampAsOf:$timestampAsOf. This shouldn't happen in the unit test." - ) - } - // iterator.toSeq doesn't trigger CompletionIterator in BlockManager which releases the reader - // lock on the underlying block. iterator hasNext does trigger it. - val linesBuilder = Seq.newBuilder[String] - while (iterator.hasNext) { - linesBuilder += iterator.next() - } - if (table.name.contains("shared_parquet_table")) { - val lines = linesBuilder.result() - val protocol = JsonUtils.fromJson[SingleAction](lines(0)).protocol - val metadata = JsonUtils.fromJson[SingleAction](lines(1)).metaData - val files = ArrayBuffer[ClientAddFile]() - lines.drop(2).foreach { line => - val action = JsonUtils.fromJson[SingleAction](line) - if (action.file != null) { - files.append(action.file) - } else { - throw new IllegalStateException(s"Unexpected Line:${line}") - } - } - DeltaTableFiles( - versionAsOf.getOrElse(getTableVersion(table)), - protocol, - metadata, - files.toSeq, - respondedFormat = DeltaSharingRestClient.RESPONSE_FORMAT_PARQUET - ) - } else { - DeltaTableFiles( - version = versionAsOf.getOrElse(getTableVersion(table)), - lines = linesBuilder.result(), - respondedFormat = DeltaSharingRestClient.RESPONSE_FORMAT_DELTA - ) - } - } - - override def getFiles( - table: Table, - startingVersion: Long, - endingVersion: Option[Long] - ): DeltaTableFiles = { - assert( - endingVersion.isDefined, - "endingVersion is not defined. This shouldn't happen in unit test." - ) - val iterator = SparkEnv.get.blockManager - .get[String](getBlockId(table.name, s"getFiles_${startingVersion}_${endingVersion.get}")) - .map(_.data.asInstanceOf[Iterator[String]]) - .getOrElse { - throw new IllegalStateException( - s"getFiles is missing for: ${table.name} with [${startingVersion}, " + - s"${endingVersion.get}]. This shouldn't happen in the unit test." - ) - } - // iterator.toSeq doesn't trigger CompletionIterator in BlockManager which releases the reader - // lock on the underlying block. iterator hasNext does trigger it. - val linesBuilder = Seq.newBuilder[String] - while (iterator.hasNext) { - linesBuilder += iterator.next() - } - DeltaTableFiles( - version = getTableVersion(table), - lines = linesBuilder.result(), - respondedFormat = DeltaSharingRestClient.RESPONSE_FORMAT_DELTA - ) - } - - override def getCDFFiles( - table: Table, - cdfOptions: Map[String, String], - includeHistoricalMetadata: Boolean - ): DeltaTableFiles = { - val suffix = cdfOptions - .get(DeltaSharingOptions.CDF_START_VERSION) - .getOrElse( - cdfOptions.get(DeltaSharingOptions.CDF_START_TIMESTAMP).get - ) - val iterator = SparkEnv.get.blockManager - .get[String]( - getBlockId( - table.name, - s"getCDFFiles_$suffix" - ) - ) - .map( - _.data.asInstanceOf[Iterator[String]] - ) - .getOrElse { - throw new IllegalStateException( - s"getCDFFiles is missing for: ${table.name}. This shouldn't happen in the unit test." - ) - } - // iterator.toSeq doesn't trigger CompletionIterator in BlockManager which releases the reader - // lock on the underlying block. iterator hasNext does trigger it. - val linesBuilder = Seq.newBuilder[String] - while (iterator.hasNext) { - linesBuilder += iterator.next() - } - DeltaTableFiles( - version = getTableVersion(table), - lines = linesBuilder.result(), - respondedFormat = DeltaSharingRestClient.RESPONSE_FORMAT_DELTA - ) - } - - override def getForStreaming(): Boolean = forStreaming - - override def getProfileProvider: DeltaSharingProfileProvider = profileProvider -} - -object TestClientForDeltaFormatSharing { - def getBlockId( - sharedTableName: String, - queryType: String, - versionAsOf: Option[Long] = None, - timestampAsOf: Option[String] = None): BlockId = { - assert(!(versionAsOf.isDefined && timestampAsOf.isDefined)) - val suffix = if (versionAsOf.isDefined) { - s"_v${versionAsOf.get}" - } else if (timestampAsOf.isDefined) { - s"_t${timestampAsOf.get}" - } else { - "" - } - BlockId( - s"${DeltaSharingUtils.DELTA_SHARING_BLOCK_ID_PREFIX}" + - s"_${sharedTableName}_$queryType$suffix" - ) - } - - val limits = scala.collection.mutable.Map[String, Long]() - val requestedFormat = scala.collection.mutable.Map[String, String]() -} diff --git a/sharing/src/test/scala/io/delta/sharing/spark/TestDeltaSharingFileSystem.scala b/sharing/src/test/scala/io/delta/sharing/spark/TestDeltaSharingFileSystem.scala deleted file mode 100644 index 2d372afba75..00000000000 --- a/sharing/src/test/scala/io/delta/sharing/spark/TestDeltaSharingFileSystem.scala +++ /dev/null @@ -1,140 +0,0 @@ -/* - * Copyright (2021) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.delta.sharing.spark - -import java.io.FileNotFoundException -import java.net.{URI, URLDecoder, URLEncoder} -import java.util.concurrent.TimeUnit - -import io.delta.sharing.client.DeltaSharingFileSystem -import org.apache.hadoop.fs._ -import org.apache.hadoop.fs.permission.FsPermission -import org.apache.hadoop.util.Progressable - -import org.apache.spark.SparkEnv -import org.apache.spark.delta.sharing.{PreSignedUrlCache, PreSignedUrlFetcher} -import org.apache.spark.storage.BlockId - -/** - * Read-only file system for DeltaSharingDataSourceDeltaSuite. - * To replace DeltaSharingFileSystem and return the content for parquet files. - */ -private[spark] class TestDeltaSharingFileSystem extends FileSystem { - import TestDeltaSharingFileSystem._ - - private lazy val preSignedUrlCacheRef = PreSignedUrlCache.getEndpointRefInExecutor(SparkEnv.get) - - override def getScheme: String = SCHEME - - override def getUri(): URI = URI.create(s"$SCHEME:///") - - override def open(f: Path, bufferSize: Int): FSDataInputStream = { - val path = DeltaSharingFileSystem.decode(f) - val fetcher = - new PreSignedUrlFetcher( - preSignedUrlCacheRef, - path.tablePath, - path.fileId, - TimeUnit.MINUTES.toMillis(10) - ) - val (tableName, parquetFilePath) = decode(fetcher.getUrl()) - val arrayBuilder = Array.newBuilder[Byte] - val iterator = SparkEnv.get.blockManager - .get[Byte](getBlockId(tableName, parquetFilePath)) - .map( - _.data.asInstanceOf[Iterator[Byte]] - ) - .getOrElse { - throw new FileNotFoundException(f.toString) - } - while (iterator.hasNext) { - arrayBuilder += iterator.next() - } - new FSDataInputStream(new SeekableByteArrayInputStream(arrayBuilder.result())) - } - - override def create( - f: Path, - permission: FsPermission, - overwrite: Boolean, - bufferSize: Int, - replication: Short, - blockSize: Long, - progress: Progressable): FSDataOutputStream = - throw new UnsupportedOperationException("create") - - override def append(f: Path, bufferSize: Int, progress: Progressable): FSDataOutputStream = - throw new UnsupportedOperationException("append") - - override def rename(src: Path, dst: Path): Boolean = - throw new UnsupportedOperationException("rename") - - override def delete(f: Path, recursive: Boolean): Boolean = - throw new UnsupportedOperationException("delete") - - override def listStatus(f: Path): Array[FileStatus] = - throw new UnsupportedOperationException("listStatus") - - override def setWorkingDirectory(new_dir: Path): Unit = - throw new UnsupportedOperationException("setWorkingDirectory") - - override def getWorkingDirectory: Path = new Path(getUri) - - override def mkdirs(f: Path, permission: FsPermission): Boolean = - throw new UnsupportedOperationException("mkdirs") - - override def getFileStatus(f: Path): FileStatus = { - val resolved = makeQualified(f) - new FileStatus(DeltaSharingFileSystem.decode(resolved).fileSize, false, 0, 1, 0, f) - } - - override def close(): Unit = { - super.close() - } -} - -private[spark] object TestDeltaSharingFileSystem { - val SCHEME = "delta-sharing" - - def getBlockId(tableName: String, parquetFilePath: String): BlockId = { - BlockId( - s"${DeltaSharingUtils.DELTA_SHARING_BLOCK_ID_PREFIX}_" + - s"{$tableName}_$parquetFilePath" - ) - } - - // The encoded string is purely for testing purpose to contain the table name and file path, - // which will be decoded and used to find block in block manager. - // In real traffic, it will be a pre-signed url. - def encode(tableName: String, parquetFilePath: String): String = { - val encodedTableName = URLEncoder.encode(tableName, "UTF-8") - val encodedParquetFilePath = URLEncoder.encode(parquetFilePath, "UTF-8") - // SCHEME:/// is needed for making this path an absolute path - s"$SCHEME:///$encodedTableName/$encodedParquetFilePath" - } - - def decode(encodedPath: String): (String, String) = { - val Array(tableName, parquetFilePath) = encodedPath - .stripPrefix(s"$SCHEME:///") - .stripPrefix(s"$SCHEME:/") - .split("/") - .map( - URLDecoder.decode(_, "UTF-8") - ) - (tableName, parquetFilePath) - } -} diff --git a/spark/src/main/antlr4/io/delta/sql/parser/DeltaSqlBase.g4 b/spark/src/main/antlr4/io/delta/sql/parser/DeltaSqlBase.g4 index 5a16b863588..8a1c386440b 100644 --- a/spark/src/main/antlr4/io/delta/sql/parser/DeltaSqlBase.g4 +++ b/spark/src/main/antlr4/io/delta/sql/parser/DeltaSqlBase.g4 @@ -92,10 +92,8 @@ statement (WHERE partitionPredicate=predicateToken)? (zorderSpec)? #optimizeTable | REORG TABLE table=qualifiedName - ( - (WHERE partitionPredicate=predicateToken)? APPLY LEFT_PAREN PURGE RIGHT_PAREN | - APPLY LEFT_PAREN UPGRADE UNIFORM LEFT_PAREN ICEBERG_COMPAT_VERSION EQ version=INTEGER_VALUE RIGHT_PAREN RIGHT_PAREN - ) #reorgTable + (WHERE partitionPredicate=predicateToken)? + APPLY LEFT_PAREN PURGE RIGHT_PAREN #reorgTable | cloneTableHeader SHALLOW CLONE source=qualifiedName clause=temporalClause? (TBLPROPERTIES tableProps=propertyList)? (LOCATION location=stringLit)? #clone @@ -225,7 +223,7 @@ nonReserved | CONVERT | TO | DELTA | PARTITIONED | BY | DESC | DESCRIBE | LIMIT | DETAIL | GENERATE | FOR | TABLE | CHECK | EXISTS | OPTIMIZE - | REORG | APPLY | PURGE | UPGRADE | UNIFORM | ICEBERG_COMPAT_VERSION + | REORG | APPLY | PURGE | RESTORE | AS | OF | ZORDER | LEFT_PAREN | RIGHT_PAREN | NO | STATISTICS @@ -262,7 +260,6 @@ FOR: 'FOR'; GENERATE: 'GENERATE'; HISTORY: 'HISTORY'; HOURS: 'HOURS'; -ICEBERG_COMPAT_VERSION: 'ICEBERG_COMPAT_VERSION'; IF: 'IF'; LEFT_PAREN: '('; LIMIT: 'LIMIT'; @@ -291,8 +288,6 @@ TIMESTAMP: 'TIMESTAMP'; TRUNCATE: 'TRUNCATE'; TO: 'TO'; TRUE: 'TRUE'; -UNIFORM: 'UNIFORM'; -UPGRADE: 'UPGRADE'; VACUUM: 'VACUUM'; VERSION: 'VERSION'; WHERE: 'WHERE'; diff --git a/spark/src/main/resources/error/delta-error-classes.json b/spark/src/main/resources/error/delta-error-classes.json index a7d0cb13c3f..23cdd547aa4 100644 --- a/spark/src/main/resources/error/delta-error-classes.json +++ b/spark/src/main/resources/error/delta-error-classes.json @@ -342,44 +342,6 @@ ], "sqlState" : "0AKDC" }, - "DELTA_CLUSTERING_COLUMNS_MISMATCH" : { - "message" : [ - "The provided clustering columns do not match the existing table's.", - "- provided: ", - "- existing: " - ], - "sqlState" : "42P10" - }, - "DELTA_CLUSTERING_COLUMN_MISSING_STATS" : { - "message" : [ - "Clustering requires clustering columns to have stats. Couldn't find clustering column(s) '' in stats schema:\n" - ], - "sqlState" : "22000" - }, - "DELTA_CLUSTERING_REPLACE_TABLE_WITH_PARTITIONED_TABLE" : { - "message" : [ - "Replacing a clustered Delta table with a partitioned table is not allowed." - ], - "sqlState" : "42000" - }, - "DELTA_CLUSTERING_WITH_PARTITION_PREDICATE" : { - "message" : [ - "OPTIMIZE command for Delta table with clustering doesn't support partition predicates. Please remove the predicates: ." - ], - "sqlState" : "0A000" - }, - "DELTA_CLUSTERING_WITH_ZORDER_BY" : { - "message" : [ - "OPTIMIZE command for Delta table with clustering cannot specify ZORDER BY. Please remove ZORDER BY ()." - ], - "sqlState" : "42613" - }, - "DELTA_CLUSTER_BY_INVALID_NUM_COLUMNS" : { - "message" : [ - "CLUSTER BY supports up to clustering columns, but the table has clustering columns. Please remove the extra clustering columns." - ], - "sqlState" : "54000" - }, "DELTA_COLUMN_DATA_SKIPPING_NOT_SUPPORTED_PARTITIONED_COLUMN" : { "message" : [ "Data skipping is not supported for partition column ''." @@ -931,15 +893,10 @@ "subClass" : { "CHANGE_VERSION_NEED_REWRITE" : { "message" : [ - "Changing to IcebergCompatV requires rewriting the table. Please run REORG TABLE APPLY (UPGRADE UNIFORM ('ICEBERG_COMPAT_VERSION = '));", + "Changing to IcebergCompatV requires rewriting the table. Please run REORG TABLE APPLY (UPGRADE UNIFORM ('IcebergCompatVersion = '));", "Note that REORG enables table feature IcebergCompatV and other Delta lake clients without that table feature support may not be able to write to the table." ] }, - "COMPAT_VERSION_NOT_SUPPORTED" : { - "message" : [ - "IcebergCompatVersion = is not supported. Supported versions are between 1 and " - ] - }, "DELETION_VECTORS_NOT_PURGED" : { "message" : [ "IcebergCompatV requires Deletion Vectors to be completely purged from the table. Please run the REORG TABLE APPLY (PURGE) command." @@ -955,13 +912,6 @@ "IcebergCompatV requires feature to be supported and enabled. You cannot drop it from the table. Instead, please disable IcebergCompatV first." ] }, - "FILES_NOT_ICEBERG_COMPAT" : { - "message" : [ - "Enabling Uniform Iceberg with IcebergCompatV requires all files to be iceberg compatible.", - "There are files in table version and files are not iceberg compatible, which is usually a result of concurrent write.", - "Please run the REORG TABLE table APPLY (UPGRADE UNIFORM (ICEBERG_COMPAT_VERSION=) command again." - ] - }, "INCOMPATIBLE_TABLE_FEATURE" : { "message" : [ "IcebergCompatV is incompatible with feature ." @@ -979,12 +929,6 @@ "New Partition Spec: " ] }, - "REWRITE_DATA_FAILED" : { - "message" : [ - "Rewriting data to IcebergCompatV failed.", - "Please run the REORG TABLE table APPLY (UPGRADE UNIFORM (ICEBERG_COMPAT_VERSION=) command again." - ] - }, "UNSUPPORTED_DATA_TYPE" : { "message" : [ "IcebergCompatV does not support the data type in your schema. Your schema:", @@ -1079,12 +1023,6 @@ ], "sqlState" : "42802" }, - "DELTA_INVALID_AUTO_COMPACT_TYPE" : { - "message" : [ - "Invalid auto-compact type: . Allowed values are: ." - ], - "sqlState" : "22023" - }, "DELTA_INVALID_CALENDAR_INTERVAL_EMPTY" : { "message" : [ "Interval cannot be null or blank." @@ -2313,12 +2251,6 @@ ], "sqlState" : "42809" }, - "DELTA_UNSUPPORTED_DROP_CLUSTERING_COLUMN" : { - "message" : [ - "Dropping clustering columns () is not allowed." - ], - "sqlState" : "0AKDC" - }, "DELTA_UNSUPPORTED_DROP_COLUMN" : { "message" : [ "DROP COLUMN is not supported for your Delta table. " diff --git a/spark/src/main/scala/io/delta/sql/parser/DeltaSqlParser.scala b/spark/src/main/scala/io/delta/sql/parser/DeltaSqlParser.scala index 3fc0cb4c0eb..32a9b63dd39 100644 --- a/spark/src/main/scala/io/delta/sql/parser/DeltaSqlParser.scala +++ b/spark/src/main/scala/io/delta/sql/parser/DeltaSqlParser.scala @@ -371,10 +371,6 @@ class DeltaSqlAstBuilder extends DeltaSqlBaseBaseVisitor[AnyRef] { * -- Physically delete dropped rows and columns of target table * REORG TABLE (delta.`/path/to/table` | delta_table_name) * [WHERE partition_predicate] APPLY (PURGE) - * - * -- Rewrite the files in UNIFORM(ICEBERG) compliant way. - * REORG TABLE table_name (delta.`/path/to/table` | catalog.db.table) - * APPLY (UPGRADE UNIFORM(ICEBERG_COMPAT_VERSION=version)) * }}} */ override def visitReorgTable(ctx: ReorgTableContext): AnyRef = withOrigin(ctx) { @@ -390,17 +386,7 @@ class DeltaSqlAstBuilder extends DeltaSqlBaseBaseVisitor[AnyRef] { val tableNameParts = targetIdentifier.database.toSeq :+ targetIdentifier.table val targetTable = createUnresolvedTable(tableNameParts, "REORG") - val reorgTableSpec = if (ctx.PURGE != null) { - DeltaReorgTableSpec(DeltaReorgTableMode.PURGE, None) - } else if (ctx.ICEBERG_COMPAT_VERSION != null) { - DeltaReorgTableSpec(DeltaReorgTableMode.UNIFORM_ICEBERG, Option(ctx.version).map(_.getText.toInt)) - } else { - throw new ParseException( - "Invalid syntax: REORG TABLE only support PURGE/UPGRADE UNIFORM.", - ctx) - } - - DeltaReorgTable(targetTable, reorgTableSpec)(Option(ctx.partitionPredicate).map(extractRawText(_)).toSeq) + DeltaReorgTable(targetTable)(Option(ctx.partitionPredicate).map(extractRawText(_)).toSeq) } override def visitDescribeDeltaDetail( diff --git a/spark/src/main/scala/io/delta/tables/DeltaTable.scala b/spark/src/main/scala/io/delta/tables/DeltaTable.scala index 17a1f348e2f..40ee84227d9 100644 --- a/spark/src/main/scala/io/delta/tables/DeltaTable.scala +++ b/spark/src/main/scala/io/delta/tables/DeltaTable.scala @@ -751,10 +751,9 @@ object DeltaTable { } /** - * Instantiate a [[DeltaTable]] object using the given table name. If the given + * Instantiate a [[DeltaTable]] object using the given table or view name. If the given * tableOrViewName is invalid (i.e. either no table exists or an existing table is not a - * Delta table), it throws a `not a Delta table` error. Note: Passing a view name will also - * result in this error as views are not supported. + * Delta table), it throws a `not a Delta table` error. * * The given tableOrViewName can also be the absolute path of a delta datasource (i.e. * delta.`path`), If so, instantiate a [[DeltaTable]] object representing the data at @@ -772,12 +771,11 @@ object DeltaTable { } /** - * Instantiate a [[DeltaTable]] object using the given table name using the given - * SparkSession. If the given tableName is invalid (i.e. either no table exists or an - * existing table is not a Delta table), it throws a `not a Delta table` error. Note: - * Passing a view name will also result in this error as views are not supported. + * Instantiate a [[DeltaTable]] object using the given table or view name using the given + * SparkSession. If the given tableOrViewName is invalid (i.e. either no table exists or an + * existing table is not a Delta table), it throws a `not a Delta table` error. * - * The given tableName can also be the absolute path of a delta datasource (i.e. + * The given tableOrViewName can also be the absolute path of a delta datasource (i.e. * delta.`path`), If so, instantiate a [[DeltaTable]] object representing the data at * the given path (consistent with the [[forPath]]). */ diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/ConflictChecker.scala b/spark/src/main/scala/org/apache/spark/sql/delta/ConflictChecker.scala index 5f24ed250cd..b6af2e274b2 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/ConflictChecker.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/ConflictChecker.scala @@ -25,7 +25,6 @@ import org.apache.spark.sql.delta.RowId.RowTrackingMetadataDomain import org.apache.spark.sql.delta.actions._ import org.apache.spark.sql.delta.metering.DeltaLogging import org.apache.spark.sql.delta.util.FileNames -import org.apache.hadoop.fs.FileStatus import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionSet, Or} @@ -119,10 +118,9 @@ private[delta] class WinningCommitSummary(val actions: Seq[Action], val commitVe private[delta] class ConflictChecker( spark: SparkSession, initialCurrentTransactionInfo: CurrentTransactionInfo, - winningCommitFileStatus: FileStatus, + winningCommitVersion: Long, isolationLevel: IsolationLevel) extends DeltaLogging { - protected val winningCommitVersion = FileNames.deltaVersion(winningCommitFileStatus) protected val startTimeMs = System.currentTimeMillis() protected val timingStats = mutable.HashMap[String, Long]() protected val deltaLog = initialCurrentTransactionInfo.readSnapshot.deltaLog @@ -157,7 +155,7 @@ private[delta] class ConflictChecker( protected def createWinningCommitSummary(): WinningCommitSummary = { recordTime("initialize-old-commit") { val winningCommitActions = deltaLog.store.read( - winningCommitFileStatus, + FileNames.deltaFile(deltaLog.logPath, winningCommitVersion), deltaLog.newDeltaHadoopConf() ).map(Action.fromJson) new WinningCommitSummary(winningCommitActions, winningCommitVersion) diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaAnalysis.scala b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaAnalysis.scala index 940c16d21ad..33e8d423858 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaAnalysis.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaAnalysis.scala @@ -557,10 +557,10 @@ class DeltaAnalysis(session: SparkSession) } merge - case reorg @ DeltaReorgTable(resolved @ ResolvedTable(_, _, _: DeltaTableV2, _), spec) => - DeltaReorgTableCommand(resolved, spec)(reorg.predicates) + case reorg @ DeltaReorgTable(resolved @ ResolvedTable(_, _, _: DeltaTableV2, _)) => + DeltaReorgTableCommand(resolved)(reorg.predicates) - case DeltaReorgTable(ResolvedTable(_, _, t, _), _) => + case DeltaReorgTable(ResolvedTable(_, _, t, _)) => throw DeltaErrors.notADeltaTable(t.name()) case cmd @ ShowColumns(child @ ResolvedTable(_, _, table: DeltaTableV2, _), namespace, _) => diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaColumnMapping.scala b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaColumnMapping.scala index 30850d9f94e..e658dc986c0 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaColumnMapping.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaColumnMapping.scala @@ -32,18 +32,13 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{ArrayType, DataType, MapType, Metadata => SparkMetadata, MetadataBuilder, StructField, StructType} +import org.apache.spark.sql.types.{DataType, Metadata => SparkMetadata, MetadataBuilder, StructField, StructType} trait DeltaColumnMappingBase extends DeltaLogging { val PARQUET_FIELD_ID_METADATA_KEY = "parquet.field.id" - val PARQUET_FIELD_NESTED_IDS_METADATA_KEY = "parquet.field.nested.ids" val COLUMN_MAPPING_METADATA_PREFIX = "delta.columnMapping." val COLUMN_MAPPING_METADATA_ID_KEY = COLUMN_MAPPING_METADATA_PREFIX + "id" val COLUMN_MAPPING_PHYSICAL_NAME_KEY = COLUMN_MAPPING_METADATA_PREFIX + "physicalName" - val COLUMN_MAPPING_METADATA_NESTED_IDS_KEY = COLUMN_MAPPING_METADATA_PREFIX + "nested.ids" - val PARQUET_LIST_ELEMENT_FIELD_NAME = "element" - val PARQUET_MAP_KEY_FIELD_NAME = "key" - val PARQUET_MAP_VALUE_FIELD_NAME = "value" /** * This list of internal columns (and only this list) is allowed to have missing @@ -173,12 +168,6 @@ trait DeltaColumnMappingBase extends DeltaLogging { def getColumnId(field: StructField): Int = field.metadata.getLong(COLUMN_MAPPING_METADATA_ID_KEY).toInt - def hasNestedColumnIds(field: StructField): Boolean = - field.metadata.contains(COLUMN_MAPPING_METADATA_NESTED_IDS_KEY) - - def getNestedColumnIds(field: StructField): SparkMetadata = - field.metadata.getMetadata(COLUMN_MAPPING_METADATA_NESTED_IDS_KEY) - def hasPhysicalName(field: StructField): Boolean = field.metadata.contains(COLUMN_MAPPING_PHYSICAL_NAME_KEY) @@ -192,9 +181,7 @@ trait DeltaColumnMappingBase extends DeltaLogging { new MetadataBuilder() .withMetadata(field.metadata) .remove(COLUMN_MAPPING_METADATA_ID_KEY) - .remove(COLUMN_MAPPING_METADATA_NESTED_IDS_KEY) .remove(PARQUET_FIELD_ID_METADATA_KEY) - .remove(PARQUET_FIELD_NESTED_IDS_METADATA_KEY) .remove(COLUMN_MAPPING_PHYSICAL_NAME_KEY) .build() @@ -208,17 +195,10 @@ trait DeltaColumnMappingBase extends DeltaLogging { // Delta spec requires writer to always write field_id in parquet schema for column mapping // Reader strips PARQUET_FIELD_ID_METADATA_KEY in // DeltaParquetFileFormat:prepareSchemaForRead - val builder = new MetadataBuilder() + new MetadataBuilder() .withMetadata(field.metadata) .putLong(PARQUET_FIELD_ID_METADATA_KEY, getColumnId(field)) - - // Nested field IDs for the 'element' and 'key'/'value' fields of Arrays - // and Maps are written when Uniform with IcebergCompatV2 is enabled on a table. - if (hasNestedColumnIds(field)) { - builder.putMetadata(PARQUET_FIELD_NESTED_IDS_METADATA_KEY, getNestedColumnIds(field)) - } - - builder.build() + .build() case mode => throw DeltaErrors.unsupportedColumnMappingMode(mode.name) @@ -371,7 +351,6 @@ trait DeltaColumnMappingBase extends DeltaLogging { val rawSchema = newMetadata.schema var maxId = DeltaConfigs.COLUMN_MAPPING_MAX_ID.fromMetaData(newMetadata) max findMaxColumnId(rawSchema) - val startId = maxId val newSchema = SchemaMergingUtils.transformColumns(rawSchema)((path, field, _) => { val builder = new MetadataBuilder().withMetadata(field.metadata) @@ -439,16 +418,10 @@ trait DeltaColumnMappingBase extends DeltaLogging { field.copy(metadata = builder.build()) }) - val (finalSchema, newMaxId) = if (IcebergCompatV2.isEnabled(newMetadata)) { - rewriteFieldIdsForIceberg(newSchema, maxId) - } else { - (newSchema, maxId) - } - newMetadata.copy( - schemaString = finalSchema.json, - configuration = newMetadata.configuration - ++ Map(DeltaConfigs.COLUMN_MAPPING_MAX_ID.key -> newMaxId.toString) + schemaString = newSchema.json, + configuration = + newMetadata.configuration ++ Map(DeltaConfigs.COLUMN_MAPPING_MAX_ID.key -> maxId.toString) ) } @@ -458,10 +431,8 @@ trait DeltaColumnMappingBase extends DeltaLogging { metadata = new MetadataBuilder() .withMetadata(field.metadata) .remove(COLUMN_MAPPING_METADATA_ID_KEY) - .remove(COLUMN_MAPPING_METADATA_NESTED_IDS_KEY) .remove(COLUMN_MAPPING_PHYSICAL_NAME_KEY) .remove(PARQUET_FIELD_ID_METADATA_KEY) - .remove(PARQUET_FIELD_NESTED_IDS_METADATA_KEY) .build() ) } @@ -666,103 +637,6 @@ trait DeltaColumnMappingBase extends DeltaLogging { true } } - - /** - * Adds the nested field IDs required by Iceberg. - * - * In parquet, list-type columns have a nested, implicitly defined [[element]] field and - * map-type columns have implicitly defined [[key]] and [[value]] fields. By default, - * Spark does not write field IDs for these fields in the parquet files. However, Iceberg - * requires these *nested* field IDs to be present. This method rewrites the specified - * Spark schema to add those nested field IDs. - * - * As list and map types are not [[StructField]]s themselves, nested field IDs are stored in - * a map as part of the metadata of the *nearest* parent [[StructField]]. For example, consider - * the following schema: - * - * col1 ARRAY(INT) - * col2 MAP(INT, INT) - * col3 STRUCT(a INT, b ARRAY(STRUCT(c INT, d MAP(INT, INT)))) - * - * col1 is a list and so requires one nested field ID for the [[element]] field in parquet. - * This nested field ID will be stored in a map that is part of col1's [[StructField.metadata]]. - * The same applies to the nested field IDs for col2's implicit [[key]] and [[value]] fields. - * col3 itself is a Struct, consisting of an integer field and a list field named 'b'. The - * nested field ID for the list of 'b' is stored in b's StructField metadata. Finally, the - * list type itself is again a struct consisting of an integer field and a map field named 'd'. - * The nested field IDs for the map of 'd' are stored in d's StructField metadata. - * - * @param schema The schema to which nested field IDs should be added - * @param startId The first field ID to use for the nested field IDs - */ - def rewriteFieldIdsForIceberg(schema: StructType, startId: Long): (StructType, Long) = { - var currFieldId = startId - - def initNestedIdsMetadata(field: StructField): MetadataBuilder = { - if (hasNestedColumnIds(field)) { - new MetadataBuilder().withMetadata(getNestedColumnIds(field)) - } else { - new MetadataBuilder() - } - } - - /* - * Helper to add the next field ID to the specified [[MetadataBuilder]] under - * the specified key. This method first checks whether this is an existing nested - * field or a newly added nested field. New field IDs are only assigned to newly - * added nested fields. - */ - def updateFieldId(metadata: MetadataBuilder, key: String): Unit = { - if (!metadata.build().contains(key)) { - currFieldId += 1 - metadata.putLong(key, currFieldId) - } - } - - /* - * Recursively adds nested field IDs for the passed data type in pre-order, - * ensuring uniqueness of field IDs. - * - * @param dt The data type that should be transformed - * @param nestedIds A MetadataBuilder that keeps track of the nested field ID - * assignment. This metadata is added to the parent field. - * @param path The current field path relative to the parent field - */ - def transform[E <: DataType](dt: E, nestedIds: MetadataBuilder, path: Seq[String]): E = { - val newDt = dt match { - case StructType(fields) => - StructType(fields.map { field => - val newNestedIds = initNestedIdsMetadata(field) - val newDt = transform(field.dataType, newNestedIds, Seq(getPhysicalName(field))) - val newFieldMetadata = new MetadataBuilder().withMetadata(field.metadata).putMetadata( - COLUMN_MAPPING_METADATA_NESTED_IDS_KEY, newNestedIds.build()).build() - field.copy(dataType = newDt, metadata = newFieldMetadata) - }) - case ArrayType(elementType, containsNull) => - // update element type metadata and recurse into element type - val elemPath = path :+ PARQUET_LIST_ELEMENT_FIELD_NAME - updateFieldId(nestedIds, elemPath.mkString(".")) - val elementDt = transform(elementType, nestedIds, elemPath) - // return new array type with updated metadata - ArrayType(elementDt, containsNull) - case MapType(keyType, valType, valueContainsNull) => - // update key type metadata and recurse into key type - val keyPath = path :+ PARQUET_MAP_KEY_FIELD_NAME - updateFieldId(nestedIds, keyPath.mkString(".")) - val keyDt = transform(keyType, nestedIds, keyPath) - // update value type metadata and recurse into value type - val valPath = path :+ PARQUET_MAP_VALUE_FIELD_NAME - updateFieldId(nestedIds, valPath.mkString(".")) - val valDt = transform(valType, nestedIds, valPath) - // return new map type with updated metadata - MapType(keyDt, valDt, valueContainsNull) - case other => other - } - newDt.asInstanceOf[E] - } - - (transform(schema, new MetadataBuilder(), Seq.empty), currFieldId) - } } object DeltaColumnMapping extends DeltaColumnMappingBase diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaConfig.scala b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaConfig.scala index b30e09f7035..d95ee9d4fa3 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaConfig.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaConfig.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.delta import java.util.{HashMap, Locale} import org.apache.spark.sql.delta.actions.{Action, Metadata, Protocol, TableFeatureProtocolUtils} -import org.apache.spark.sql.delta.hooks.AutoCompactType import org.apache.spark.sql.delta.metering.DeltaLogging import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.delta.stats.{DataSkippingReader, StatisticsCollection} @@ -82,11 +81,6 @@ case class DeltaConfig[T]( */ trait DeltaConfigsBase extends DeltaLogging { - // Special properties stored in the Hive MetaStore that specifies which version last updated - // the entry in the MetaStore with the latest schema and table property information - val METASTORE_LAST_UPDATE_VERSION = "delta.lastUpdateVersion" - val METASTORE_LAST_COMMIT_TIMESTAMP = "delta.lastCommitTimestamp" - /** * Convert a string to [[CalendarInterval]]. This method is case-insensitive and will throw * [[IllegalArgumentException]] when the input string is not a valid interval. @@ -394,21 +388,6 @@ trait DeltaConfigsBase extends DeltaLogging { _ > 0, "needs to be a positive integer.") - /** - * Enable auto compaction for a Delta table. When enabled, we will check if files already - * written to a Delta table can leverage compaction after a commit. If so, we run a post-commit - * hook to compact the files. - * It can be enabled by setting the property to `true` - * Note that the behavior from table property can be overridden by the config: - * [[org.apache.spark.sql.delta.sources.DeltaSQLConf.DELTA_AUTO_COMPACT_ENABLED]] - */ - val AUTO_COMPACT = buildConfig[Option[String]]( - "autoOptimize.autoCompact", - null, - v => Option(v).map(_.toLowerCase(Locale.ROOT)), - v => v.isEmpty || AutoCompactType.ALLOWED_VALUES.contains(v.get), - s""""needs to be one of: ${AutoCompactType.ALLOWED_VALUES.mkString(",")}""") - /** Whether to clean up expired checkpoints and delta logs. */ val ENABLE_EXPIRED_LOG_CLEANUP = buildConfig[Boolean]( "enableExpiredLogCleanup", diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaErrors.scala b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaErrors.scala index 8ac0ba009c4..89956b6b29b 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaErrors.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaErrors.scala @@ -27,7 +27,6 @@ import org.apache.spark.sql.delta.actions.{CommitInfo, Metadata, Protocol, Table import org.apache.spark.sql.delta.catalog.DeltaCatalog import org.apache.spark.sql.delta.commands.AlterTableDropFeatureDeltaCommand import org.apache.spark.sql.delta.constraints.Constraints -import org.apache.spark.sql.delta.hooks.AutoCompactType import org.apache.spark.sql.delta.hooks.PostCommitHook import org.apache.spark.sql.delta.metering.DeltaLogging import org.apache.spark.sql.delta.schema.{DeltaInvariantViolationException, InvariantViolationException, SchemaUtils, UnsupportedDataTypeInfo} @@ -3014,50 +3013,6 @@ trait DeltaErrorsBase ) } - def icebergCompatVersionNotSupportedException( - currVersion: Int, - maxVersion: Int): Throwable = { - new DeltaUnsupportedOperationException( - errorClass = "DELTA_ICEBERG_COMPAT_VIOLATION.COMPAT_VERSION_NOT_SUPPORTED", - messageParameters = Array( - currVersion.toString, - currVersion.toString, - maxVersion.toString - ) - ) - } - - def icebergCompatReorgAddFileTagsMissingException( - tableVersion: Long, - icebergCompatVersion: Int, - addFilesCount: Long, - addFilesWithTagsCount: Long): Throwable = { - new DeltaIllegalStateException( - errorClass = "DELTA_ICEBERG_COMPAT_VIOLATION.FILES_NOT_ICEBERG_COMPAT", - messageParameters = Array( - icebergCompatVersion.toString, - icebergCompatVersion.toString, - addFilesCount.toString, - tableVersion.toString, - (addFilesCount - addFilesWithTagsCount).toString, - icebergCompatVersion.toString - ) - ) - } - - def icebergCompatDataFileRewriteFailedException( - icebergCompatVersion: Int, - cause: Throwable): Throwable = { - new DeltaIllegalStateException( - errorClass = "", - messageParameters = Array( - icebergCompatVersion.toString, - icebergCompatVersion.toString - ), - cause - ) - } - def icebergCompatReplacePartitionedTableException( version: Int, prevPartitionCols: Seq[String], @@ -3130,74 +3085,6 @@ trait DeltaErrorsBase messageParameters = Array(version.toString, version.toString, key, requiredValue, actualValue) ) } - - def invalidAutoCompactType(value: String): Throwable = { - new DeltaIllegalArgumentException( - errorClass = "DELTA_INVALID_AUTO_COMPACT_TYPE", - messageParameters = Array(value, AutoCompactType.ALLOWED_VALUES.mkString("(", ",", ")")) - ) - } - - def clusterByInvalidNumColumnsException( - numColumnsLimit: Int, - actualNumColumns: Int): Throwable = { - new DeltaAnalysisException( - errorClass = "DELTA_CLUSTER_BY_INVALID_NUM_COLUMNS", - messageParameters = Array(numColumnsLimit.toString, actualNumColumns.toString) - ) - } - - def clusteringColumnMissingStats( - clusteringColumnWithoutStats: String, - statsSchema: String): Throwable = { - new DeltaAnalysisException( - errorClass = "DELTA_CLUSTERING_COLUMN_MISSING_STATS", - messageParameters = Array(clusteringColumnWithoutStats, statsSchema) - ) - } - - def clusteringColumnsMismatchException( - providedClusteringColumns: String, - existingClusteringColumns: String): Throwable = { - new DeltaAnalysisException( - "DELTA_CLUSTERING_COLUMNS_MISMATCH", - Array(providedClusteringColumns, existingClusteringColumns) - ) - } - - def dropClusteringColumnNotSupported(droppingClusteringCols: Seq[String]): Throwable = { - new DeltaAnalysisException( - "DELTA_UNSUPPORTED_DROP_CLUSTERING_COLUMN", - Array(droppingClusteringCols.mkString(","))) - } - - def replacingClusteredTableWithPartitionedTableNotAllowed(): Throwable = { - new DeltaAnalysisException( - errorClass = "DELTA_CLUSTERING_REPLACE_TABLE_WITH_PARTITIONED_TABLE", - messageParameters = Array.empty) - } - - def clusteringWithPartitionPredicatesException(predicates: Seq[String]): Throwable = { - new DeltaUnsupportedOperationException( - errorClass = "DELTA_CLUSTERING_WITH_PARTITION_PREDICATE", - messageParameters = Array(s"${predicates.mkString(" ")}")) - } - - def clusteringWithZOrderByException(zOrderBy: Seq[UnresolvedAttribute]): Throwable = { - new DeltaAnalysisException( - errorClass = "DELTA_CLUSTERING_WITH_ZORDER_BY", - messageParameters = Array(s"${zOrderBy.map(_.name).mkString(", ")}")) - } - - def clusteringTablePreviewDisabledException(): Throwable = { - val msg = s""" - |A clustered table is currently in preview and is disabled by default. Please set - |${DeltaSQLConf.DELTA_CLUSTERING_TABLE_PREVIEW_ENABLED.key} to true to enable it. - |Note that a clustered table is not recommended for production use (e.g., unsupported - |incremental clustering). - |""".stripMargin.replace("\n", " ") - new UnsupportedOperationException(msg) - } } object DeltaErrors extends DeltaErrorsBase diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaLog.scala b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaLog.scala index 8324b3e1dd6..8ae29a04fd8 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaLog.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaLog.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.delta import java.lang.ref.WeakReference import java.net.URI import java.util.concurrent.TimeUnit +import java.util.concurrent.locks.ReentrantLock import scala.collection.JavaConverters._ import scala.collection.mutable @@ -119,6 +120,9 @@ class DeltaLog private( /** Used to read and write physical log files and checkpoints. */ lazy val store = createLogStore(spark) + /** Use ReentrantLock to allow us to call `lockInterruptibly` */ + protected val deltaLogLock = new ReentrantLock() + /** Delta History Manager containing version and commit history. */ lazy val history = new DeltaHistoryManager( this, spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_HISTORY_PAR_SEARCH_THRESHOLD)) @@ -151,6 +155,19 @@ class DeltaLog private( */ private[delta] def compositeId: (String, Path) = tableId -> dataPath + /** + * Run `body` inside `deltaLogLock` lock using `lockInterruptibly` so that the thread can be + * interrupted when waiting for the lock. + */ + def lockInterruptibly[T](body: => T): T = { + deltaLogLock.lockInterruptibly() + try { + body + } finally { + deltaLogLock.unlock() + } + } + /** * Creates a [[LogicalRelation]] for a given [[DeltaLogFileIndex]], with all necessary file source * options taken from the Delta Log. All reads of Delta metadata files should use this method. @@ -437,14 +454,11 @@ class DeltaLog private( /** Creates the log directory if it does not exist. */ def ensureLogDirectoryExist(): Unit = { val fs = logPath.getFileSystem(newDeltaHadoopConf()) - def createDirIfNotExists(path: Path): Unit = { - if (!fs.exists(path)) { - if (!fs.mkdirs(path)) { - throw DeltaErrors.cannotCreateLogPathException(logPath.toString) - } + if (!fs.exists(logPath)) { + if (!fs.mkdirs(logPath)) { + throw DeltaErrors.cannotCreateLogPathException(logPath.toString) } } - createDirIfNotExists(logPath) } /** diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaOperations.scala b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaOperations.scala index 5887e052712..f9d54a948d3 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaOperations.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaOperations.scala @@ -250,10 +250,6 @@ object DeltaOperations { strMetrics += "numOutputRows" -> actualNumOutputRows.toString } - val dvMetrics = transformDeletionVectorMetrics( - metrics, dvMetrics = DeltaOperationMetrics.MERGE_DELETION_VECTORS) - strMetrics ++= dvMetrics - strMetrics } @@ -475,8 +471,6 @@ object DeltaOperations { sealed abstract class OptimizeOrReorg(override val name: String, predicates: Seq[Expression]) extends OperationWithPredicates(name, predicates) - /** parameter key to indicate whether it's an Auto Compaction */ - val AUTO_COMPACTION_PARAMETER_KEY = "auto" /** operation name for REORG command */ val REORG_OPERATION_NAME = "REORG" @@ -488,12 +482,10 @@ object DeltaOperations { /** Recorded when optimizing the table. */ case class Optimize( predicate: Seq[Expression], - zOrderBy: Seq[String] = Seq.empty, - auto: Boolean = false + zOrderBy: Seq[String] = Seq.empty ) extends OptimizeOrReorg(OPTIMIZE_OPERATION_NAME, predicate) { override val parameters: Map[String, Any] = super.parameters ++ Map( - ZORDER_PARAMETER_KEY -> JsonUtils.toJson(zOrderBy), - AUTO_COMPACTION_PARAMETER_KEY -> auto + ZORDER_PARAMETER_KEY -> JsonUtils.toJson(zOrderBy) ) override val operationMetrics: Set[String] = DeltaOperationMetrics.OPTIMIZE @@ -584,17 +576,6 @@ object DeltaOperations { val maxToStringFields = SQLConf.get.maxToStringFields predicates.map(_.simpleString(maxToStringFields)) } - - /** Recorded when the table properties are set. */ - private val OP_UPGRADE_UNIFORM_BY_REORG = "REORG TABLE UPGRADE UNIFORM" - - /** - * recorded when upgrading a table set uniform properties by REORG TABLE ... UPGRADE UNIFORM - */ - case class UpgradeUniformProperties(properties: Map[String, String]) extends Operation( - OP_UPGRADE_UNIFORM_BY_REORG) { - override val parameters: Map[String, Any] = Map("properties" -> JsonUtils.toJson(properties)) - } } private[delta] object DeltaOperationMetrics { @@ -734,16 +715,6 @@ private[delta] object DeltaOperationMetrics { SumMetrics("numDeletionVectorsRemoved", "numDeletionVectorsUpdated") ) - // The same as [[DELETION_VECTORS]] but with the "Target" prefix that is used by MERGE. - val MERGE_DELETION_VECTORS = Map( - // Adding "numDeletionVectorsUpdated" here makes the values line up with how - // "numFilesAdded"/"numFilesRemoved" behave. - "numTargetDeletionVectorsAdded" -> - SumMetrics("numTargetDeletionVectorsAdded", "numTargetDeletionVectorsUpdated"), - "numTargetDeletionVectorsRemoved" -> - SumMetrics("numTargetDeletionVectorsRemoved", "numTargetDeletionVectorsUpdated") - ) - val TRUNCATE = Set( "numRemovedFiles", // number of files removed "executionTimeMs" // time taken to execute the entire operation @@ -773,10 +744,7 @@ private[delta] object DeltaOperationMetrics { "numTargetChangeFilesAdded", // number of CDC files "executionTimeMs", // time taken to execute the entire operation "scanTimeMs", // time taken to scan the files for matches - "rewriteTimeMs", // time taken to rewrite the matched files - "numTargetDeletionVectorsAdded", // number of deletion vectors added - "numTargetDeletionVectorsRemoved", // number of deletion vectors removed - "numTargetDeletionVectorsUpdated" // number of deletion vectors updated + "rewriteTimeMs" // time taken to rewrite the matched files ) val UPDATE = Set( diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaParquetFileFormat.scala b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaParquetFileFormat.scala index 6ed4a117fd5..c762c417646 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaParquetFileFormat.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaParquetFileFormat.scala @@ -29,7 +29,6 @@ import org.apache.spark.sql.delta.schema.SchemaMergingUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce.Job -import org.apache.parquet.hadoop.ParquetOutputFormat import org.apache.parquet.hadoop.util.ContextUtil import org.apache.spark.broadcast.Broadcast @@ -92,7 +91,6 @@ case class DeltaParquetFileFormat( field.copy(metadata = new MetadataBuilder() .withMetadata(field.metadata) .remove(DeltaColumnMapping.PARQUET_FIELD_ID_METADATA_KEY) - .remove(DeltaColumnMapping.PARQUET_FIELD_NESTED_IDS_METADATA_KEY) .build()) } } else schema @@ -215,12 +213,6 @@ case class DeltaParquetFileFormat( conf.set(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key, SQLConf.ParquetOutputTimestampType.TIMESTAMP_MICROS.toString) } - if (IcebergCompatV2.isEnabled(metadata)) { - // For Uniform with IcebergCompatV2, we need to write nested field IDs for list and map - // types to the parquet schema. Spark currently does not support it so we hook in our - // own write support class. - ParquetOutputFormat.setWriteSupportClass(job, classOf[DeltaParquetWriteSupport]) - } factory } diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaParquetWriteSupport.scala b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaParquetWriteSupport.scala deleted file mode 100644 index 41038246982..00000000000 --- a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaParquetWriteSupport.scala +++ /dev/null @@ -1,149 +0,0 @@ -/* - * Copyright (2021) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.delta - -import scala.util.Try - -import org.apache.spark.sql.delta.DeltaColumnMapping._ -import org.apache.hadoop.conf.Configuration -import org.apache.parquet.hadoop.api.WriteSupport.WriteContext -import org.apache.parquet.schema.{LogicalTypeAnnotation, Type, Types} -import org.apache.parquet.schema.LogicalTypeAnnotation.{ListLogicalTypeAnnotation, MapLogicalTypeAnnotation} - -import org.apache.spark.SparkRuntimeException -import org.apache.spark.sql.catalyst.parser.LegacyTypeStringParser -import org.apache.spark.sql.catalyst.trees.Origin -import org.apache.spark.sql.errors.QueryCompilationErrors -import org.apache.spark.sql.execution.datasources.parquet.{ParquetSchemaConverter, ParquetWriteSupport} -import org.apache.spark.sql.types.{DataType, StructField, StructType} - -class DeltaParquetWriteSupport extends ParquetWriteSupport { - - private def getNestedFieldId(field: StructField, path: Seq[String]): Int = { - field.metadata - .getMetadata(PARQUET_FIELD_NESTED_IDS_METADATA_KEY) - .getLong(path.mkString(".")) - .toInt - } - - private def findFieldInSparkSchema(schema: StructType, path: Seq[String]): StructField = { - schema.findNestedField(path, true) match { - case Some((_, field)) => field - case None => throw QueryCompilationErrors.invalidFieldName(Seq(path.head), path, Origin()) - } - } - - override def init(configuration: Configuration): WriteContext = { - val writeContext = super.init(configuration) - // Parse the Spark schema. This is the same as is done in super.init, however, the - // parsed schema is stored in [[ParquetWriteSupport.schema]], which is private so - // we can't access it here and need to parse it again. - val schemaString = configuration.get(ParquetWriteSupport.SPARK_ROW_SCHEMA) - // This code is copied from Spark StructType.fromString because it is not accessible here - val parsedSchema = Try(DataType.fromJson(schemaString)).getOrElse( - LegacyTypeStringParser.parseString(schemaString)) match { - case t: StructType => t - case _ => - // This code is copied from DataTypeErrors.failedParsingStructTypeError because - // it is not accessible here - throw new SparkRuntimeException( - errorClass = "FAILED_PARSE_STRUCT_TYPE", - messageParameters = Map("raw" -> s"'$schemaString'")) - } - - val messageType = writeContext.getSchema - val newMessageTypeBuilder = Types.buildMessage() - messageType.getFields.forEach { field => - val parentField = findFieldInSparkSchema(parsedSchema, Seq(field.getName)) - newMessageTypeBuilder.addField(convert( - field, parentField, parsedSchema, Seq(field.getName), Seq(field.getName))) - } - val newMessageType = newMessageTypeBuilder.named( - ParquetSchemaConverter.SPARK_PARQUET_SCHEMA_NAME) - new WriteContext(newMessageType, writeContext.getExtraMetaData) - } - - /** - * Recursively rewrites the parquet [[Type]] by adding the nested field - * IDs to list and map subtypes as defined in the schema. The - * recursion needs to keep track of the absolute field path in order - * to correctly identify the StructField in the spark schema for a - * corresponding parquet field. As nested field IDs are referenced - * by their relative path in a field's metadata, the recursion also needs - * to keep track of the relative path. - * - * For example, consider the following column type - * col1 STRUCT(a INT, b STRUCT(c INT, d ARRAY(INT))) - * - * The absolute path to the nested [[element]] field of the list is - * col1.b.d.element whereas the relative path is d.element, i.e. relative - * to the parent struct field. - */ - private def convert( - field: Type, - parentField: StructField, - sparkSchema: StructType, - absolutePath: Seq[String], - relativePath: Seq[String]): Type = { - field.getLogicalTypeAnnotation match { - case _: ListLogicalTypeAnnotation => - val relElemFieldPath = relativePath :+ PARQUET_LIST_ELEMENT_FIELD_NAME - val id = getNestedFieldId(parentField, relElemFieldPath) - val elementField = - field.asGroupType().getFields.get(0).asGroupType().getFields.get(0).withId(id) - Types - .buildGroup(field.getRepetition).as(LogicalTypeAnnotation.listType()) - .addField( - Types.repeatedGroup() - .addField(convert(elementField, parentField, sparkSchema, - absolutePath :+ PARQUET_LIST_ELEMENT_FIELD_NAME, relElemFieldPath)) - .named("list")) - .id(field.getId.intValue()) - .named(field.getName) - case _: MapLogicalTypeAnnotation => - val relKeyFieldPath = relativePath :+ PARQUET_MAP_KEY_FIELD_NAME - val relValFieldPath = relativePath :+ PARQUET_MAP_VALUE_FIELD_NAME - val keyId = getNestedFieldId(parentField, relKeyFieldPath) - val valId = getNestedFieldId(parentField, relValFieldPath) - val keyField = - field.asGroupType().getFields.get(0).asGroupType().getFields.get(0).withId(keyId) - val valueField = - field.asGroupType().getFields.get(0).asGroupType().getFields.get(1).withId(valId) - Types - .buildGroup(field.getRepetition).as(LogicalTypeAnnotation.mapType()) - .addField( - Types - .repeatedGroup() - .addField(convert(keyField, parentField, sparkSchema, - absolutePath :+ PARQUET_MAP_KEY_FIELD_NAME, relKeyFieldPath)) - .addField(convert(valueField, parentField, sparkSchema, - absolutePath :+ PARQUET_MAP_VALUE_FIELD_NAME, relValFieldPath)) - .named("key_value")) - .id(field.getId.intValue()) - .named(field.getName) - case _ if field.isPrimitive => field - case _ => - val builder = Types.buildGroup(field.getRepetition) - field.asGroupType().getFields.forEach { field => - val absPath = absolutePath :+ field.getName - val parentField = findFieldInSparkSchema(sparkSchema, absPath) - builder.addField(convert(field, parentField, sparkSchema, absPath, Seq(field.getName))) - } - builder.id(field.getId.intValue()).named(field.getName) - } - } -} diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaTableIdentifier.scala b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaTableIdentifier.scala index cc96b07e4bb..40e189bfd29 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/DeltaTableIdentifier.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/DeltaTableIdentifier.scala @@ -89,10 +89,10 @@ object DeltaTableIdentifier extends DeltaLogging { } spark.sessionState.conf.runSQLonFile && - new Path(identifier.table).isAbsolute && DeltaSourceUtils.isDeltaTable(identifier.database) && !tableIsTemporaryTable && - !tableExists + !tableExists && + new Path(identifier.table).isAbsolute } /** diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/IcebergCompat.scala b/spark/src/main/scala/org/apache/spark/sql/delta/IcebergCompatV1.scala similarity index 96% rename from spark/src/main/scala/org/apache/spark/sql/delta/IcebergCompat.scala rename to spark/src/main/scala/org/apache/spark/sql/delta/IcebergCompatV1.scala index 3910e37bcea..66088631f73 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/IcebergCompat.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/IcebergCompatV1.scala @@ -212,21 +212,6 @@ object IcebergCompat extends DeltaLogging { .find{ case (config, _) => config.fromMetaData(metadata).getOrElse(false) } .map{ case (_, version) => version } - /** - * Get the DeltaConfig for the given IcebergCompat version. If version is not valid, - * throw an exception. - * @return the DeltaConfig for the given version. E.g., - * [[DeltaConfigs.ICEBERG_COMPAT_V1_ENABLED]] for version 1. - */ - def getIcebergCompatVersionConfigForValidVersion(version: Int): DeltaConfig[Option[Boolean]] = { - if (version <= 0 || version > knownVersions.length) { - throw DeltaErrors.icebergCompatVersionNotSupportedException( - version, knownVersions.length - ) - } - knownVersions(version - 1)._1 - } - /** * @return true if any version of IcebergCompat is enabled */ diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala b/spark/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala index 18cc7f132d5..3283f03e14e 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/OptimisticTransaction.scala @@ -33,13 +33,13 @@ import org.apache.spark.sql.delta.catalog.DeltaTableV2 import org.apache.spark.sql.delta.commands.DeletionVectorUtils import org.apache.spark.sql.delta.commands.cdc.CDCReader import org.apache.spark.sql.delta.files._ -import org.apache.spark.sql.delta.hooks.{CheckpointHook, GenerateSymlinkManifest, IcebergConverterHook, PostCommitHook, UpdateCatalogFactory} +import org.apache.spark.sql.delta.hooks.{CheckpointHook, GenerateSymlinkManifest, IcebergConverterHook, PostCommitHook} import org.apache.spark.sql.delta.implicits.addFileEncoder import org.apache.spark.sql.delta.metering.DeltaLogging import org.apache.spark.sql.delta.schema.{SchemaMergingUtils, SchemaUtils} import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.delta.stats._ -import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.fs.Path import org.apache.spark.SparkException import org.apache.spark.sql.{AnalysisException, Column, DataFrame, SparkSession} @@ -321,9 +321,6 @@ trait OptimisticTransactionImpl extends TransactionalWrite } protected val postCommitHooks = new ArrayBuffer[PostCommitHook]() - catalogTable.foreach { ct => - registerPostCommitHook(UpdateCatalogFactory.getUpdateCatalogHook(ct, spark)) - } // The CheckpointHook will only checkpoint if necessary, so always register it to run. registerPostCommitHook(CheckpointHook) registerPostCommitHook(IcebergConverterHook) @@ -353,11 +350,6 @@ trait OptimisticTransactionImpl extends TransactionalWrite checkDeletionVectorFilesHaveWideBounds = false } - /** The set of distinct partitions that contain added files by current transaction. */ - protected[delta] var partitionsAddedToOpt: Option[mutable.HashSet[Map[String, String]]] = None - - /** True if this transaction is a blind append. This is only valid after commit. */ - protected[delta] var isBlindAppend: Boolean = false /** * The logSegment of the snapshot prior to the commit. @@ -868,47 +860,6 @@ trait OptimisticTransactionImpl extends TransactionalWrite } } - def reportAutoCompactStatsError(e: Throwable): Unit = { - recordDeltaEvent(deltaLog, "delta.collectStats", data = Map("message" -> e.getMessage)) - logError(e.getMessage) - } - - def collectAutoOptimizeStats(numAdd: Long, numRemove: Long, actions: Iterator[Action]): Unit = { - // Early exit if no files were added or removed. - if (numAdd == 0 && numRemove == 0) return - val collector = createAutoCompactStatsCollector() - if (collector.isInstanceOf[DisabledAutoCompactPartitionStatsCollector]) return - AutoCompactPartitionStats.instance(spark) - .collectPartitionStats(collector, deltaLog.tableId, actions) - } - - /** - * A subclass of AutoCompactPartitionStatsCollector that's to be used if the config to collect - * auto compaction stats is turned off. This subclass intentionally does nothing. - */ - class DisabledAutoCompactPartitionStatsCollector extends AutoCompactPartitionStatsCollector { - override def collectPartitionStatsForAdd(file: AddFile): Unit = {} - override def collectPartitionStatsForRemove(file: RemoveFile): Unit = {} - override def finalizeStats(tableId: String): Unit = {} - } - - def createAutoCompactStatsCollector(): AutoCompactPartitionStatsCollector = { - try { - if (spark.conf.get(DeltaSQLConf.DELTA_AUTO_COMPACT_RECORD_PARTITION_STATS_ENABLED)) { - val minFileSize = spark.conf - .get(DeltaSQLConf.DELTA_AUTO_COMPACT_MIN_FILE_SIZE) - .getOrElse(Long.MaxValue) - return AutoCompactPartitionStats.instance(spark) - .createStatsCollector(minFileSize, reportAutoCompactStatsError) - } - } catch { - case NonFatal(e) => reportAutoCompactStatsError(e) - } - - // If config-disabled, or error caught, fall though and use a no-op stats collector. - new DisabledAutoCompactPartitionStatsCollector - } - /** * Checks if the new schema contains any CDC columns (which is invalid) and throws the appropriate * error @@ -1071,7 +1022,7 @@ trait OptimisticTransactionImpl extends TransactionalWrite val domainMetadata = DomainMetadataUtils.validateDomainMetadataSupportedAndNoDuplicate(finalActions, protocol) - isBlindAppend = { + val isBlindAppend = { val dependsOnFiles = readPredicates.nonEmpty || readFiles.nonEmpty val onlyAddFiles = preparedActions.collect { case f: FileAction => f }.forall(_.isInstanceOf[AddFile]) @@ -1201,9 +1152,6 @@ trait OptimisticTransactionImpl extends TransactionalWrite var addFilesHistogram: Option[FileSizeHistogram] = None var removeFilesHistogram: Option[FileSizeHistogram] = None val assertDeletionVectorWellFormed = getAssertDeletionVectorWellFormedFunc(spark, op) - // Initialize everything needed to maintain auto-compaction stats. - partitionsAddedToOpt = Some(new mutable.HashSet[Map[String, String]]) - val acStatsCollector = createAutoCompactStatsCollector() var allActions = (extraActions.toIterator ++ actions).map { action => commitSize += 1 action match { @@ -1211,13 +1159,10 @@ trait OptimisticTransactionImpl extends TransactionalWrite numAddFiles += 1 if (a.pathAsUri.isAbsolute) numAbsolutePaths += 1 assertDeletionVectorWellFormed(a) - partitionsAddedToOpt.get += a.partitionValues - acStatsCollector.collectPartitionStatsForAdd(a) if (a.dataChange) bytesNew += a.size addFilesHistogram.foreach(_.insert(a.size)) case r: RemoveFile => numRemoveFiles += 1 - acStatsCollector.collectPartitionStatsForRemove(r) removeFilesHistogram.foreach(_.insert(r.getFileSize)) case _: SetTransaction => numSetTransaction += 1 @@ -1259,7 +1204,6 @@ trait OptimisticTransactionImpl extends TransactionalWrite overwrite = false, deltaLog.newDeltaHadoopConf()) - acStatsCollector.finalizeStats(deltaLog.tableId) spark.sessionState.conf.setConf( DeltaSQLConf.DELTA_LAST_COMMIT_VERSION_IN_SESSION, Some(attemptVersion)) @@ -1418,7 +1362,7 @@ trait OptimisticTransactionImpl extends TransactionalWrite snapshot, newestProtocol = protocol, // Note: this will try to use `newProtocol` newestMetadata = metadata, // Note: this will try to use `newMetadata` - isCreatingNewTable || op.isInstanceOf[DeltaOperations.UpgradeUniformProperties], + isCreatingNewTable, otherActions ) newProtocol = protocolUpdate1.orElse(newProtocol) @@ -1618,10 +1562,7 @@ trait OptimisticTransactionImpl extends TransactionalWrite private def lockCommitIfEnabled[T](body: => T): T = { if (isCommitLockEnabled) { - // We are borrowing the `snapshotLock` even for commits. Ideally we should be - // using a separate lock for this purpose, because multiple threads fighting over - // a commit shouldn't interfere with normal snapshot updates by readers. - deltaLog.withSnapshotLockInterruptibly(body) + deltaLog.lockInterruptibly(body) } else { body } @@ -1754,7 +1695,6 @@ trait OptimisticTransactionImpl extends TransactionalWrite numOfDomainMetadatas += 1 case _ => } - collectAutoOptimizeStats(numAdd, numRemove, actions.iterator) val info = currentTransactionInfo.commitInfo .map(_.copy(readVersion = None, isolationLevel = None)).orNull setNeedsCheckpoint(attemptVersion, postCommitSnapshot) @@ -1790,7 +1730,6 @@ trait OptimisticTransactionImpl extends TransactionalWrite txnId = Some(txnId)) recordDeltaEvent(deltaLog, DeltaLogging.DELTA_COMMIT_STATS_OPTYPE, data = stats) - partitionsAddedToOpt = Some(distinctPartitions) postCommitSnapshot } @@ -1823,16 +1762,8 @@ trait OptimisticTransactionImpl extends TransactionalWrite tags = Map(TAG_LOG_STORE_CLASS -> deltaLog.store.getClass.getName)) { DeltaTableV2.withEnrichedUnsupportedTableException(catalogTable) { - val fileStatuses = getConflictingVersions(checkVersion) - val nextAttemptVersion = checkVersion + fileStatuses.size - // validate that information about conflicting winning commit files is continuous and in the - // right order. - val expected = (checkVersion until nextAttemptVersion) - val found = fileStatuses.map(deltaVersion) - val mismatch = expected.zip(found).dropWhile{ case (v1, v2) => v1 == v2 }.take(10) - assert(mismatch.isEmpty, - s"Expected ${mismatch.map(_._1).mkString(",")} but got ${mismatch.map(_._2).mkString(",")}") + val nextAttemptVersion = getNextAttemptVersion(checkVersion) val logPrefixStr = s"[attempt $attemptNumber]" val txnDetailsLogStr = { @@ -1851,12 +1782,10 @@ trait OptimisticTransactionImpl extends TransactionalWrite s"[$checkVersion, $nextAttemptVersion) with current txn having $txnDetailsLogStr") var updatedCurrentTransactionInfo = currentTransactionInfo - (checkVersion until nextAttemptVersion) - .zip(fileStatuses) - .foreach { case (otherCommitVersion, otherCommitFileStatus) => + (checkVersion until nextAttemptVersion).foreach { otherCommitVersion => updatedCurrentTransactionInfo = checkForConflictsAgainstVersion( updatedCurrentTransactionInfo, - otherCommitFileStatus, + otherCommitVersion, commitIsolationLevel) logInfo(s"$logPrefixStr No conflicts in version $otherCommitVersion, " + s"${clock.getTimeMillis() - commitAttemptStartTime} ms since start") @@ -1871,13 +1800,13 @@ trait OptimisticTransactionImpl extends TransactionalWrite protected def checkForConflictsAgainstVersion( currentTransactionInfo: CurrentTransactionInfo, - otherCommitFileStatus: FileStatus, + otherCommitVersion: Long, commitIsolationLevel: IsolationLevel): CurrentTransactionInfo = { val conflictChecker = new ConflictChecker( spark, currentTransactionInfo, - otherCommitFileStatus, + otherCommitVersion, commitIsolationLevel) conflictChecker.checkConflicts() } @@ -1885,15 +1814,11 @@ trait OptimisticTransactionImpl extends TransactionalWrite /** Returns the version that the first attempt will try to commit at. */ protected def getFirstAttemptVersion: Long = readVersion + 1L - /** Returns the conflicting commit information */ - protected def getConflictingVersions(previousAttemptVersion: Long): Seq[FileStatus] = { - assert(previousAttemptVersion == preCommitLogSegment.version + 1) - val (newPreCommitLogSegment, newCommitFileStatuses) = - deltaLog.getUpdatedLogSegment(preCommitLogSegment) - assert(preCommitLogSegment.version + newCommitFileStatuses.size == - newPreCommitLogSegment.version) - preCommitLogSegment = newPreCommitLogSegment - newCommitFileStatuses + /** Returns the next attempt version given the last attempted version */ + protected def getNextAttemptVersion(previousAttemptVersion: Long): Long = { + val latestSnapshot = deltaLog.update() + preCommitLogSegment = latestSnapshot.logSegment + latestSnapshot.version + 1 } /** Register a hook that will be executed once a commit is successful. */ diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/SnapshotManagement.scala b/spark/src/main/scala/org/apache/spark/sql/delta/SnapshotManagement.scala index 577e6fa6aed..ea47dd08750 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/SnapshotManagement.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/SnapshotManagement.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.delta import java.io.FileNotFoundException import java.util.Objects import java.util.concurrent.Future -import java.util.concurrent.locks.ReentrantLock import scala.collection.mutable import scala.util.control.NonFatal @@ -59,22 +58,6 @@ trait SnapshotManagement { self: DeltaLog => @volatile protected var currentSnapshot: CapturedSnapshot = getSnapshotAtInit - /** Use ReentrantLock to allow us to call `lockInterruptibly` */ - protected val snapshotLock = new ReentrantLock() - - /** - * Run `body` inside `snapshotLock` lock using `lockInterruptibly` so that the thread - * can be interrupted when waiting for the lock. - */ - def withSnapshotLockInterruptibly[T](body: => T): T = { - snapshotLock.lockInterruptibly() - try { - body - } finally { - snapshotLock.unlock() - } - } - /** * Get the LogSegment that will help in computing the Snapshot of the table at DeltaLog * initialization, or None if the directory was empty/missing. @@ -179,7 +162,6 @@ trait SnapshotManagement { self: DeltaLog => getLogSegmentForVersion( versionToLoad, newFiles, - validateLogSegmentWithoutCompactedDeltas = true, oldCheckpointProviderOpt = oldCheckpointProviderOpt, lastCheckpointInfo = lastCheckpointInfo ) @@ -241,7 +223,6 @@ trait SnapshotManagement { self: DeltaLog => protected def getLogSegmentForVersion( versionToLoad: Option[Long], files: Option[Array[FileStatus]], - validateLogSegmentWithoutCompactedDeltas: Boolean, oldCheckpointProviderOpt: Option[UninitializedCheckpointProvider], lastCheckpointInfo: Option[LastCheckpointInfo]): Option[LogSegment] = { recordFrameProfile("Delta", "SnapshotManagement.getLogSegmentForVersion") { @@ -305,16 +286,7 @@ trait SnapshotManagement { self: DeltaLog => // Here we validate that we are able to create a valid LogSegment by just using commit deltas // and without considering minor-compacted deltas. We want to fail early if log is messed up // i.e. some commit deltas are missing (although compacted-deltas are present). - // We should not do this validation when we want to update the logSegment after a conflict - // via the [[SnapshotManagement.getUpdatedLogSegment]] method. In that specific flow, we just - // list from the committed version and reuse existing pre-commit logsegment together with - // listing result to create the new pre-commit logsegment. Because of this, we don't have info - // about all the delta files (e.g. when minor compactions are used in existing preCommit log - // segment) and hence the validation if attempted will fail. So we need to set - // `validateLogSegmentWithoutCompactedDeltas` to false in that case. - if (validateLogSegmentWithoutCompactedDeltas) { - validateDeltaVersions(deltasAfterCheckpoint, newCheckpointVersion, versionToLoad) - } + validateDeltaVersions(deltasAfterCheckpoint, newCheckpointVersion, versionToLoad) val newVersion = deltasAfterCheckpoint.lastOption.map(deltaVersion).getOrElse(newCheckpoint.get.version) @@ -656,40 +628,6 @@ trait SnapshotManagement { self: DeltaLog => timestamp => cutoffOpt.forall(timestamp < _) } - /** - * Get the newest logSegment, using the previous logSegment as a hint. This is faster than - * doing a full update, but it won't work if the table's log directory was replaced. - */ - def getUpdatedLogSegment(oldLogSegment: LogSegment): (LogSegment, Seq[FileStatus]) = { - val newFilesOpt = listDeltaCompactedDeltaAndCheckpointFiles( - startVersion = oldLogSegment.version + 1, - versionToLoad = None, - includeMinorCompactions = spark.conf.get(DeltaSQLConf.DELTALOG_MINOR_COMPACTION_USE_FOR_READS) - ) - val newFiles = newFilesOpt.getOrElse { - // An empty listing likely implies a list-after-write inconsistency or that somebody clobbered - // the Delta log. - return (oldLogSegment, Nil) - } - val allFiles = ( - oldLogSegment.checkpointProvider.topLevelFiles ++ - oldLogSegment.deltas ++ - newFiles - ).toArray - val lastCheckpointInfo = Option.empty[LastCheckpointInfo] - val newLogSegment = getLogSegmentForVersion( - versionToLoad = None, - files = Some(allFiles), - validateLogSegmentWithoutCompactedDeltas = false, - lastCheckpointInfo = lastCheckpointInfo, - oldCheckpointProviderOpt = Some(oldLogSegment.checkpointProvider) - ).getOrElse(oldLogSegment) - val fileStatusesOfConflictingCommits = newFiles.collect { - case DeltaFile(f, v) if v <= newLogSegment.version => f - } - (newLogSegment, fileStatusesOfConflictingCommits) - } - /** * Returns the snapshot, if it has been updated since the specified timestamp. * @@ -755,7 +693,7 @@ trait SnapshotManagement { self: DeltaLog => val doAsync = stalenessAcceptable && !isCurrentlyStale(capturedSnapshot.updateTimestamp) if (!doAsync) { recordFrameProfile("Delta", "SnapshotManagement.update") { - withSnapshotLockInterruptibly { + lockInterruptibly { val newSnapshot = updateInternal(isAsync = false) sendEvent(newSnapshot = capturedSnapshot.snapshot) newSnapshot @@ -789,11 +727,11 @@ trait SnapshotManagement { self: DeltaLog => * at once and return the current snapshot. The return snapshot may be stale. */ private def tryUpdate(isAsync: Boolean): Snapshot = { - if (snapshotLock.tryLock()) { + if (deltaLogLock.tryLock()) { try { updateInternal(isAsync) } finally { - snapshotLock.unlock() + deltaLogLock.unlock() } } else { currentSnapshot.snapshot @@ -802,7 +740,7 @@ trait SnapshotManagement { self: DeltaLog => /** * Queries the store for new delta files and applies them to the current state. - * Note: the caller should hold `snapshotLock` before calling this method. + * Note: the caller should hold `deltaLogLock` before calling this method. */ protected def updateInternal(isAsync: Boolean): Snapshot = recordDeltaOperation(this, "delta.log.update", Map(TAG_ASYNC -> isAsync.toString)) { @@ -841,7 +779,7 @@ trait SnapshotManagement { self: DeltaLog => /** Replace the given snapshot with the provided one. */ protected def replaceSnapshot(newSnapshot: Snapshot, updateTimestamp: Long): Unit = { - if (!snapshotLock.isHeldByCurrentThread) { + if (!deltaLogLock.isHeldByCurrentThread) { recordDeltaEvent(this, "delta.update.unsafeReplace") } val oldSnapshot = currentSnapshot.snapshot @@ -889,7 +827,7 @@ trait SnapshotManagement { self: DeltaLog => def updateAfterCommit( committedVersion: Long, newChecksumOpt: Option[VersionChecksum], - preCommitLogSegment: LogSegment): Snapshot = withSnapshotLockInterruptibly { + preCommitLogSegment: LogSegment): Snapshot = lockInterruptibly { recordDeltaOperation(this, "delta.log.updateAfterCommit") { val updateTimestamp = clock.getTimeMillis() val previousSnapshot = currentSnapshot.snapshot diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/actions/actions.scala b/spark/src/main/scala/org/apache/spark/sql/delta/actions/actions.scala index a5d017ffedf..9be45904ad8 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/actions/actions.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/actions/actions.scala @@ -844,9 +844,6 @@ object AddFile { /** [[OPTIMIZE_TARGET_SIZE]]: target file size the file was optimized to. */ object OPTIMIZE_TARGET_SIZE extends AddFile.Tags.KeyType("OPTIMIZE_TARGET_SIZE") - - /** [[ICEBERG_COMPAT_VERSION]]: IcebergCompat version */ - object ICEBERG_COMPAT_VERSION extends AddFile.Tags.KeyType("ICEBERG_COMPAT_VERSION") } /** Convert a [[Tags.KeyType]] to a string to be used in the AddMap.tags Map[String, String]. */ diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/catalog/DeltaCatalog.scala b/spark/src/main/scala/org/apache/spark/sql/delta/catalog/DeltaCatalog.scala index 4c1ea821823..d7143e7c469 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/catalog/DeltaCatalog.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/catalog/DeltaCatalog.scala @@ -24,23 +24,17 @@ import java.util.Locale import scala.collection.JavaConverters._ import scala.collection.mutable -import org.apache.spark.sql.delta.skipping.clustering.ClusteredTableUtils -import org.apache.spark.sql.delta.skipping.clustering.temp.ClusterBySpec -import org.apache.spark.sql.delta.skipping.clustering.temp.{ClusterByTransform => TempClusterByTransform} import org.apache.spark.sql.delta.{DeltaConfigs, DeltaErrors, DeltaTableUtils} import org.apache.spark.sql.delta.{DeltaLog, DeltaOptions} import org.apache.spark.sql.delta.DeltaTableIdentifier.gluePermissionError import org.apache.spark.sql.delta.commands._ import org.apache.spark.sql.delta.constraints.{AddConstraint, DropConstraint} import org.apache.spark.sql.delta.metering.DeltaLogging -import org.apache.spark.sql.delta.schema.SchemaUtils import org.apache.spark.sql.delta.sources.{DeltaDataSource, DeltaSourceUtils, DeltaSQLConf} import org.apache.spark.sql.delta.stats.StatisticsCollection import org.apache.spark.sql.delta.tablefeatures.DropFeature -import org.apache.spark.sql.delta.util.PartitionUtils import org.apache.hadoop.fs.Path -import org.apache.spark.SparkException import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, DataFrame, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier @@ -105,8 +99,9 @@ class DeltaCatalog extends DelegatingCatalogExtension case "option.path" => false case _ => true }.toMap - val (partitionColumns, maybeBucketSpec, maybeClusterBySpec) = convertTransforms(partitions) - validateClusterBySpec(maybeClusterBySpec, schema) + val ( + partitionColumns, maybeBucketSpec + ) = convertTransforms(partitions) var newSchema = schema var newPartitionColumns = partitionColumns var newBucketSpec = maybeBucketSpec @@ -159,8 +154,7 @@ class DeltaCatalog extends DelegatingCatalogExtension val withDb = verifyTableAndSolidify( tableDesc, - None, - maybeClusterBySpec + None ) val writer = sourceQuery.map { df => @@ -254,26 +248,6 @@ class DeltaCatalog extends DelegatingCatalogExtension } } - // Perform checks on ClusterBySpec. - def validateClusterBySpec( - maybeClusterBySpec: Option[ClusterBySpec], schema: StructType): Unit = { - // Validate that the preview is enabled if we are creating a clustered table. - ClusteredTableUtils.validatePreviewEnabled(maybeClusterBySpec) - maybeClusterBySpec.foreach { clusterBy => - // Check if the specified cluster by columns exists in the table. - val resolver = spark.sessionState.conf.resolver - clusterBy.columnNames.foreach { column => - // This is the same check as in rules.scala, to keep the behaviour consistent. - SchemaUtils.findColumnPosition(column.fieldNames(), schema, resolver) - } - // Check that columns are not duplicated in the cluster by statement. - PartitionUtils.checkColumnNameDuplication( - clusterBy.columnNames.map(_.toString), "in CLUSTER BY", resolver) - // Check number of clustering columns is within allowed range. - ClusteredTableUtils.validateNumClusteringColumns( - clusterBy.columnNames.map(_.fieldNames.toSeq)) - } - } protected def newDeltaPathTable(ident: Identifier): DeltaTableV2 = { DeltaTableV2(spark, new Path(ident.name())) @@ -398,11 +372,11 @@ class DeltaCatalog extends DelegatingCatalogExtension } // Copy of V2SessionCatalog.convertTransforms, which is private. - private def convertTransforms( - partitions: Seq[Transform]): (Seq[String], Option[BucketSpec], Option[ClusterBySpec]) = { + private def convertTransforms(partitions: Seq[Transform]): ( + Seq[String], Option[BucketSpec] + ) = { val identityCols = new mutable.ArrayBuffer[String] var bucketSpec = Option.empty[BucketSpec] - var clusterBySpec = Option.empty[ClusterBySpec] partitions.map { case IdentityTransform(FieldReference(Seq(col))) => @@ -411,29 +385,22 @@ class DeltaCatalog extends DelegatingCatalogExtension case BucketTransform(numBuckets, bucketCols, sortCols) => bucketSpec = Some(BucketSpec( numBuckets, bucketCols.map(_.fieldNames.head), sortCols.map(_.fieldNames.head))) - case TempClusterByTransform(columnNames) => - if (clusterBySpec.nonEmpty) { - // Parser guarantees that it only passes down one TempClusterByTransform. - throw SparkException.internalError("Cannot have multiple cluster by transforms.") - } - clusterBySpec = Some(ClusterBySpec(columnNames)) case transform => throw DeltaErrors.operationNotSupportedException(s"Partitioning by expressions") } - // Parser guarantees that partition and cluster by can't both exist. - assert(!(identityCols.toSeq.nonEmpty && clusterBySpec.nonEmpty)) - // Parser guarantees that bucketing and cluster by can't both exist. - assert(!(bucketSpec.nonEmpty && clusterBySpec.nonEmpty)) - (identityCols.toSeq, bucketSpec, clusterBySpec) + ( + identityCols.toSeq, bucketSpec + ) } /** Performs checks on the parameters provided for table creation for a Delta table. */ def verifyTableAndSolidify( tableDesc: CatalogTable, - query: Option[LogicalPlan], - maybeClusterBySpec: Option[ClusterBySpec] = None): CatalogTable = { + query: Option[LogicalPlan] + ): CatalogTable = { + if (tableDesc.bucketSpec.isDefined) { throw DeltaErrors.operationNotSupportedException("Bucketing", tableDesc.identifier) } @@ -448,15 +415,8 @@ class DeltaCatalog extends DelegatingCatalogExtension tableDesc.partitionColumnNames, caseSensitive = false) // Delta is case insensitive - var validatedConfigurations = + val validatedConfigurations = DeltaConfigs.validateConfigurations(tableDesc.properties) - // Add needed configs for Clustered table. - if (maybeClusterBySpec.nonEmpty) { - validatedConfigurations = - validatedConfigurations ++ - ClusteredTableUtils.getClusteringColumnsAsProperty(maybeClusterBySpec) ++ - ClusteredTableUtils.getTableFeatureProperties(validatedConfigurations) - } val db = tableDesc.identifier.database.getOrElse(catalog.getCurrentDatabase) val tableIdentWithDB = tableDesc.identifier.copy(database = Some(db)) diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/clustering/ClusteringMetadataDomain.scala b/spark/src/main/scala/org/apache/spark/sql/delta/clustering/ClusteringMetadataDomain.scala deleted file mode 100644 index 01102ff5190..00000000000 --- a/spark/src/main/scala/org/apache/spark/sql/delta/clustering/ClusteringMetadataDomain.scala +++ /dev/null @@ -1,36 +0,0 @@ -/* - * Copyright (2021) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.delta.clustering - -import org.apache.spark.sql.delta.skipping.clustering.ClusteringColumn -import org.apache.spark.sql.delta.{JsonMetadataDomain, JsonMetadataDomainUtils} - -/** - * Metadata domain for Clustered table which tracks clustering columns. - */ -case class ClusteringMetadataDomain(clusteringColumns: Seq[Seq[String]]) - extends JsonMetadataDomain[ClusteringMetadataDomain] { - override val domainName: String = ClusteringMetadataDomain.domainName -} - -object ClusteringMetadataDomain extends JsonMetadataDomainUtils[ClusteringMetadataDomain] { - override val domainName = "delta.clustering" - - def fromClusteringColumns(clusteringColumns: Seq[ClusteringColumn]): ClusteringMetadataDomain = { - ClusteringMetadataDomain(clusteringColumns.map(_.physicalName)) - } -} diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/commands/CreateDeltaTableCommand.scala b/spark/src/main/scala/org/apache/spark/sql/delta/commands/CreateDeltaTableCommand.scala index 0775597d178..f626545b6eb 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/commands/CreateDeltaTableCommand.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/commands/CreateDeltaTableCommand.scala @@ -19,12 +19,10 @@ package org.apache.spark.sql.delta.commands // scalastyle:off import.ordering.noEmptyLine import java.util.concurrent.TimeUnit -import org.apache.spark.sql.delta.skipping.clustering.ClusteredTableUtils import org.apache.spark.sql.delta._ import org.apache.spark.sql.delta.DeltaColumnMapping.{dropColumnMappingMetadata, filterColumnMappingProperties} import org.apache.spark.sql.delta.actions.{Action, Metadata, Protocol} import org.apache.spark.sql.delta.actions.DomainMetadata -import org.apache.spark.sql.delta.hooks.{UpdateCatalog, UpdateCatalogFactory} import org.apache.spark.sql.delta.hooks.IcebergConverterHook import org.apache.spark.sql.delta.metering.DeltaLogging import org.apache.spark.sql.delta.schema.SchemaUtils @@ -234,24 +232,14 @@ case class CreateDeltaTableCommand( } var actions = deltaWriter.write( txn, - sparkSession, - ClusteredTableUtils.getClusterBySpecOptional(table), - // Pass this option to the writer so that it can differentiate between an INSERT and a - // REPLACE command. This is needed because the writer is shared between the two commands. - // But some options, such as dynamic partition overwrite, are only valid for INSERT. - // Only allow createOrReplace command which is not a V1 writer. - // saveAsTable() command uses this same code path and is marked as a V1 writer. - // We do not want saveAsTable() to be treated as a REPLACE command wrt dynamic partition - // overwrite. - isTableReplace = isReplace && !isV1Writer + sparkSession ) // Metadata updates for creating table (with any writer) and replacing table // (only with V1 writer) will be handled inside WriteIntoDelta. // For createOrReplace operation, metadata updates are handled here if the table already // exists (replacing table), otherwise it is handled inside WriteIntoDelta (creating table). if (!isV1Writer && isReplace && txn.readVersion > -1L) { - val newDomainMetadata = Seq.empty[DomainMetadata] ++ - ClusteredTableUtils.getDomainMetadataOptional(table, txn) + val newDomainMetadata = Seq.empty[DomainMetadata] // Ensure to remove any domain metadata for REPLACE TABLE. actions = actions ++ DomainMetadataUtils.handleDomainMetadataForReplaceTable( txn.snapshot.domainMetadata, newDomainMetadata) @@ -325,7 +313,7 @@ case class CreateDeltaTableCommand( protocol.foreach { protocol => txn.updateProtocol(protocol) } - ClusteredTableUtils.getDomainMetadataOptional(table, txn).toSeq + Nil } else { verifyTableMetadata(txn, tableWithLocation) Nil @@ -360,10 +348,7 @@ case class CreateDeltaTableCommand( val operationTimestamp = System.currentTimeMillis() var actionsToCommit = Seq.empty[Action] val removes = txn.filterFiles().map(_.removeWithTimestamp(operationTimestamp)) - actionsToCommit = removes ++ - DomainMetadataUtils.handleDomainMetadataForReplaceTable( - txn.snapshot.domainMetadata, - ClusteredTableUtils.getDomainMetadataOptional(table, txn).toSeq) + actionsToCommit = removes actionsToCommit } @@ -381,9 +366,7 @@ case class CreateDeltaTableCommand( description = table.comment.orNull, schemaString = schemaString, partitionColumns = table.partitionColumnNames, - // Filter out ephemeral clustering columns config because we don't want to persist - // it in delta log. This will be persisted in CatalogTable's table properties instead. - configuration = ClusteredTableUtils.removeClusteringColumnsProperty(table.properties), + configuration = table.properties, createdTime = Some(System.currentTimeMillis())) } @@ -551,7 +534,7 @@ case class CreateDeltaTableCommand( validateLocation = false) case TableCreationModes.Replace | TableCreationModes.CreateOrReplace if existingTableOpt.isDefined => - UpdateCatalogFactory.getUpdateCatalogHook(table, spark).updateSchema(spark, snapshot) + spark.sessionState.catalog.alterTable(table) case TableCreationModes.Replace => val ident = Identifier.of(table.identifier.database.toArray, table.identifier.table) throw DeltaErrors.cannotReplaceMissingTableException(ident) @@ -575,37 +558,13 @@ case class CreateDeltaTableCommand( table.storage.copy(properties = Map.empty) } - // If we have to update the catalog, use the correct schema and table properties, otherwise - // empty out the schema and property information - if (conf.getConf(DeltaSQLConf.DELTA_UPDATE_CATALOG_ENABLED)) { - // In the case we're creating a Delta table on an existing path and adopting the schema - val schema = if (table.schema.isEmpty) snapshot.schema else table.schema - val truncatedSchema = UpdateCatalog.truncateSchemaIfNecessary(schema) - val additionalProperties = if (truncatedSchema.isEmpty) { - Map(UpdateCatalog.ERROR_KEY -> UpdateCatalog.LONG_SCHEMA_ERROR) - } else { - Map.empty - } - - table.copy( - schema = truncatedSchema, - // Hive does not allow for the removal of partition columns once stored. - // To avoid returning the incorrect schema when the partition columns change, - // we store the partition columns as regular data columns. - partitionColumnNames = Nil, - properties = UpdateCatalog.updatedProperties(snapshot) - ++ additionalProperties, - storage = storageProps, - tracksPartitionsInCatalog = true) - } else { - table.copy( - schema = new StructType(), - properties = Map.empty, - partitionColumnNames = Nil, - // Remove write specific options when updating the catalog - storage = storageProps, - tracksPartitionsInCatalog = true) - } + table.copy( + schema = new StructType(), + properties = Map.empty, + partitionColumnNames = Nil, + // Remove write specific options when updating the catalog + storage = storageProps, + tracksPartitionsInCatalog = true) } /** @@ -659,7 +618,6 @@ case class CreateDeltaTableCommand( tableWithLocation: CatalogTable, snapshotOpt: Option[Snapshot] = None): OptimisticTransaction = { val txn = deltaLog.startTransaction(None, snapshotOpt) - validatePrerequisitesForClusteredTable(txn.snapshot.protocol, txn.deltaLog) // During CREATE/REPLACE, we synchronously run conversion (if Uniform is enabled) so // we always remove the post commit hook here. @@ -667,22 +625,6 @@ case class CreateDeltaTableCommand( txn } - - /** - * Validate pre-requisites for clustered tables for CREATE/REPLACE operations. - * @param protocol Protocol used for validations. This protocol should - * be used during the CREATE/REPLACE commit. - * @param deltaLog Delta log used for logging purposes. - */ - private def validatePrerequisitesForClusteredTable( - protocol: Protocol, - deltaLog: DeltaLog): Unit = { - // Validate a clustered table is not replaced by a partitioned table. - if (table.partitionColumnNames.nonEmpty && - ClusteredTableUtils.isSupported(protocol)) { - throw DeltaErrors.replacingClusteredTableWithPartitionedTableNotAllowed() - } - } } // isCreate is true for Create and CreateOrReplace modes. It is false for Replace mode. diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/commands/DeltaReorgTableCommand.scala b/spark/src/main/scala/org/apache/spark/sql/delta/commands/DeltaReorgTableCommand.scala index 59016b3ff77..4e3398db0ff 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/commands/DeltaReorgTableCommand.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/commands/DeltaReorgTableCommand.scala @@ -24,19 +24,7 @@ import org.apache.spark.sql.catalyst.analysis.ResolvedTable import org.apache.spark.sql.catalyst.plans.logical.{IgnoreCachedData, LeafCommand, LogicalPlan, UnaryCommand} import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} -object DeltaReorgTableMode extends Enumeration { - val PURGE, UNIFORM_ICEBERG = Value -} - -case class DeltaReorgTableSpec( - reorgTableMode: DeltaReorgTableMode.Value, - icebergCompatVersionOpt: Option[Int] -) - -case class DeltaReorgTable( - target: LogicalPlan, - reorgTableSpec: DeltaReorgTableSpec = DeltaReorgTableSpec(DeltaReorgTableMode.PURGE, None))( - val predicates: Seq[String]) extends UnaryCommand { +case class DeltaReorgTable(target: LogicalPlan)(val predicates: Seq[String]) extends UnaryCommand { def child: LogicalPlan = target @@ -49,41 +37,20 @@ case class DeltaReorgTable( /** * The PURGE command. */ -case class DeltaReorgTableCommand( - target: LogicalPlan, - reorgTableSpec: DeltaReorgTableSpec = DeltaReorgTableSpec(DeltaReorgTableMode.PURGE, None))( - val predicates: Seq[String]) - extends OptimizeTableCommandBase - with ReorgTableForUpgradeUniformHelper - with LeafCommand - with IgnoreCachedData { +case class DeltaReorgTableCommand(target: LogicalPlan)(val predicates: Seq[String]) + extends OptimizeTableCommandBase with LeafCommand with IgnoreCachedData { override val otherCopyArgs: Seq[AnyRef] = predicates :: Nil - override def optimizeByReorg( - sparkSession: SparkSession, - isPurge: Boolean, - icebergCompatVersion: Option[Int]): Seq[Row] = { + override def run(sparkSession: SparkSession): Seq[Row] = { val command = OptimizeTableCommand( target, predicates, optimizeContext = DeltaOptimizeContext( - isPurge = isPurge, + isPurge = true, minFileSize = Some(0L), - maxDeletedRowsRatio = Some(0d), - icebergCompatVersion = icebergCompatVersion - ) + maxDeletedRowsRatio = Some(0d)) )(zOrderBy = Nil) command.run(sparkSession) } - - override def run(sparkSession: SparkSession): Seq[Row] = { - reorgTableSpec match { - case DeltaReorgTableSpec(DeltaReorgTableMode.PURGE, None) => - optimizeByReorg(sparkSession, isPurge = true, icebergCompatVersion = None) - case DeltaReorgTableSpec(DeltaReorgTableMode.UNIFORM_ICEBERG, Some(icebergCompatVersion)) => - val table = getDeltaTable(target, "REORG") - upgradeUniformIcebergCompatVersion(table, sparkSession, icebergCompatVersion) - } - } } diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/commands/MergeIntoCommandBase.scala b/spark/src/main/scala/org/apache/spark/sql/delta/commands/MergeIntoCommandBase.scala index 307e1008e4c..3266cc362fe 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/commands/MergeIntoCommandBase.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/commands/MergeIntoCommandBase.scala @@ -180,10 +180,7 @@ trait MergeIntoCommandBase extends LeafRunnableCommand "scanTimeMs" -> createTimingMetric(sc, "time taken to scan the files for matches"), "rewriteTimeMs" -> - createTimingMetric(sc, "time taken to rewrite the matched files"), - "numTargetDeletionVectorsAdded" -> createMetric(sc, "number of deletion vectors added"), - "numTargetDeletionVectorsRemoved" -> createMetric(sc, "number of deletion vectors removed"), - "numTargetDeletionVectorsUpdated" -> createMetric(sc, "number of deletion vectors updated") + createTimingMetric(sc, "time taken to rewrite the matched files") ) /** diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/commands/OptimizeTableCommand.scala b/spark/src/main/scala/org/apache/spark/sql/delta/commands/OptimizeTableCommand.scala index 2e84ec52460..e053ddce4d4 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/commands/OptimizeTableCommand.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/commands/OptimizeTableCommand.scala @@ -19,9 +19,10 @@ package org.apache.spark.sql.delta.commands import java.util.ConcurrentModificationException import scala.collection.mutable.ArrayBuffer +import scala.collection.parallel.ForkJoinTaskSupport +import scala.collection.parallel.immutable.ParVector import org.apache.spark.sql.delta.skipping.MultiDimClustering -import org.apache.spark.sql.delta.skipping.clustering.{ClusteredTableUtils, ClusteringColumnInfo} import org.apache.spark.sql.delta._ import org.apache.spark.sql.delta.DeltaOperations.Operation import org.apache.spark.sql.delta.actions.{Action, AddFile, DeletionVectorDescriptor, FileAction, RemoveFile} @@ -146,17 +147,6 @@ case class OptimizeTableCommand( throw DeltaErrors.notADeltaTableException(table.deltaLog.dataPath.toString) } - if (ClusteredTableUtils.isSupported(txn.protocol)) { - // Validate that the preview is enabled if we are optimizing a clustered table. - ClusteredTableUtils.validatePreviewEnabled(txn.snapshot.protocol) - if (userPartitionPredicates.nonEmpty) { - throw DeltaErrors.clusteringWithPartitionPredicatesException(userPartitionPredicates) - } - if (zOrderBy.nonEmpty) { - throw DeltaErrors.clusteringWithZOrderByException(zOrderBy) - } - } - val partitionColumns = txn.snapshot.metadata.partitionColumns // Parse the predicate expression into Catalyst expression and verify only simple filters // on partition columns are present @@ -173,14 +163,8 @@ case class OptimizeTableCommand( validateZorderByColumns(sparkSession, txn, zOrderBy) val zOrderByColumns = zOrderBy.map(_.name).toSeq - new OptimizeExecutor( - sparkSession, - txn, - partitionPredicates, - zOrderByColumns, - isAutoCompact = false, - optimizeContext - ).optimize() + new OptimizeExecutor(sparkSession, txn, partitionPredicates, zOrderByColumns, optimizeContext) + .optimize() } } @@ -198,16 +182,12 @@ case class OptimizeTableCommand( * specified, [[DeltaSQLConf.DELTA_OPTIMIZE_MAX_DELETED_ROWS_RATIO]] * will be used. This parameter must be set to `0` when [[isPurge]] is * true. - * @param icebergCompatVersion The iceberg compatibility version used to rewrite data for - * uniform tables. */ case class DeltaOptimizeContext( isPurge: Boolean = false, minFileSize: Option[Long] = None, - maxFileSize: Option[Long] = None, - maxDeletedRowsRatio: Option[Double] = None, - icebergCompatVersion: Option[Int] = None) { - if (isPurge || icebergCompatVersion.isDefined) { + maxDeletedRowsRatio: Option[Double] = None) { + if (isPurge) { require( minFileSize.contains(0L) && maxDeletedRowsRatio.contains(0d), "minFileSize and maxDeletedRowsRatio must be 0 when running PURGE.") @@ -227,42 +207,20 @@ class OptimizeExecutor( txn: OptimisticTransaction, partitionPredicate: Seq[Expression], zOrderByColumns: Seq[String], - isAutoCompact: Boolean, optimizeContext: DeltaOptimizeContext) extends DeltaCommand with SQLMetricsReporting with Serializable { /** Timestamp to use in [[FileAction]] */ private val operationTimestamp = new SystemClock().getTimeMillis() - private val isClusteredTable = ClusteredTableUtils.isSupported(txn.snapshot.protocol) - - private val isMultiDimClustering = isClusteredTable || zOrderByColumns.nonEmpty - - private val clusteringColumns: Seq[String] = { - if (zOrderByColumns.nonEmpty) { - zOrderByColumns - } else if (isClusteredTable) { - ClusteringColumnInfo.extractLogicalNames(txn.snapshot) - } else { - Nil - } - } - - private lazy val curve: String = { - if (zOrderByColumns.nonEmpty) { - "zorder" - } else { - assert(isClusteredTable) - "hilbert" - } - } + private val isMultiDimClustering = zOrderByColumns.nonEmpty def optimize(): Seq[Row] = { recordDeltaOperation(txn.deltaLog, "delta.optimize") { val minFileSize = optimizeContext.minFileSize.getOrElse( sparkSession.sessionState.conf.getConf(DeltaSQLConf.DELTA_OPTIMIZE_MIN_FILE_SIZE)) - val maxFileSize = optimizeContext.maxFileSize.getOrElse( - sparkSession.sessionState.conf.getConf(DeltaSQLConf.DELTA_OPTIMIZE_MAX_FILE_SIZE)) + val maxFileSize = + sparkSession.sessionState.conf.getConf(DeltaSQLConf.DELTA_OPTIMIZE_MAX_FILE_SIZE) val maxDeletedRowsRatio = optimizeContext.maxDeletedRowsRatio.getOrElse( sparkSession.sessionState.conf.getConf(DeltaSQLConf.DELTA_OPTIMIZE_MAX_DELETED_ROWS_RATIO)) @@ -358,18 +316,9 @@ class OptimizeExecutor( file.deletedToPhysicalRecordsRatio.getOrElse(0d) > maxDeletedRowsRatio } - def shouldRewriteToBeIcebergCompatible(file: AddFile): Boolean = { - if (optimizeContext.icebergCompatVersion.isEmpty) return false - if (file.tags == null) return true - val icebergCompatVersion = file.tags.getOrElse(AddFile.Tags.ICEBERG_COMPAT_VERSION.name, "0") - !optimizeContext.icebergCompatVersion.exists(_.toString == icebergCompatVersion) - } - - // Select files that are small, have too many deleted rows, - // or need to be made iceberg compatible + // Select files that are small or have too many deleted rows files.filter( - addFile => addFile.size < minFileSize || shouldCompactBecauseOfDeletedRows(addFile) || - shouldRewriteToBeIcebergCompatible(addFile)) + addFile => addFile.size < minFileSize || shouldCompactBecauseOfDeletedRows(addFile)) } /** @@ -415,7 +364,6 @@ class OptimizeExecutor( bins.filter { bin => bin.size > 1 || // bin has more than one file or (bin.size == 1 && bin(0).deletionVector != null) || // single file in the bin has a DV or - (bin.size == 1 && optimizeContext.icebergCompatVersion.isDefined) || // uniform reorg isMultiDimClustering // multi-clustering }.map(b => (partition, b)) } @@ -443,8 +391,8 @@ class OptimizeExecutor( MultiDimClustering.cluster( input, approxNumFiles, - clusteringColumns, - curve) + zOrderByColumns, + "zorder") } else { val useRepartition = sparkSession.sessionState.conf.getConf( DeltaSQLConf.DELTA_OPTIMIZE_REPARTITION_ENABLED) @@ -465,11 +413,7 @@ class OptimizeExecutor( val addFiles = txn.writeFiles(repartitionDF, None, isOptimize = true, Nil).collect { case a: AddFile => - (if (isClusteredTable) { - a.copy(clusteringProvider = Some(ClusteredTableUtils.clusteringProvider)) - } else { - a - }).copy(dataChange = false) + a.copy(dataChange = false) case other => throw new IllegalStateException( s"Unexpected action $other with type ${other.getClass}. File compaction job output" + @@ -514,7 +458,7 @@ class OptimizeExecutor( if (optimizeContext.isPurge) { DeltaOperations.Reorg(partitionPredicate) } else { - DeltaOperations.Optimize(partitionPredicate, clusteringColumns, auto = isAutoCompact) + DeltaOperations.Optimize(partitionPredicate, zOrderByColumns) } } diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/commands/ReorgTableForUpgradeUniformHelper.scala b/spark/src/main/scala/org/apache/spark/sql/delta/commands/ReorgTableForUpgradeUniformHelper.scala deleted file mode 100644 index c15831b761a..00000000000 --- a/spark/src/main/scala/org/apache/spark/sql/delta/commands/ReorgTableForUpgradeUniformHelper.scala +++ /dev/null @@ -1,246 +0,0 @@ -/* - * Copyright (2021) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.delta.commands - -// scalastyle:off import.ordering.noEmptyLine -import scala.util.control.NonFatal - -import org.apache.spark.sql.delta.{DeltaConfig, DeltaConfigs, DeltaErrors, DeltaOperations, Snapshot} -import org.apache.spark.sql.delta.IcebergCompat.{getEnabledVersion, getIcebergCompatVersionConfigForValidVersion} -import org.apache.spark.sql.delta.UniversalFormat.{icebergEnabled, ICEBERG_FORMAT} -import org.apache.spark.sql.delta.actions.{AddFile, Protocol} -import org.apache.spark.sql.delta.catalog.DeltaTableV2 -import org.apache.spark.sql.delta.metering.DeltaLogging -import org.apache.spark.sql.delta.util.Utils.try_element_at - -import org.apache.spark.sql.{Row, SparkSession} -import org.apache.spark.sql.functions.col - -/** - * Helper trait for ReorgTableCommand to rewrite the table to be Iceberg compatible. - */ -trait ReorgTableForUpgradeUniformHelper extends DeltaLogging { - - private val versionChangesRequireRewrite: Map[Int, Set[Int]] = - Map(0 -> Set(2), 1 -> Set(2), 2 -> Set(2)) - - /** - * Helper function to check if the table data may need to be rewritten to be iceberg compatible. - * Only if not all addFiles has the tag, Rewriting would be performed. - */ - private def reorgMayNeedRewrite(oldVersion: Int, newVersion: Int): Boolean = { - versionChangesRequireRewrite.getOrElse(oldVersion, Set.empty[Int]).contains(newVersion) - } - - /** - * Helper function to rewrite the table. Implemented by Reorg Table Command. - */ - def optimizeByReorg( - sparkSession: SparkSession, - isPurge: Boolean, - icebergCompatVersion: Option[Int]): Seq[Row] - - /** - * Helper function to update the table icebergCompat properties. - * We can not use AlterTableSetPropertiesDeltaCommand here because we don't allow customer to - * change icebergCompatVersion by using Alter Table command. - */ - private def enableIcebergCompat( - target: DeltaTableV2, - currIcebergCompatVersionOpt: Option[Int], - targetVersionDeltaConfig: DeltaConfig[Option[Boolean]]): Unit = { - var enableIcebergCompatConf = Map( - targetVersionDeltaConfig.key -> "true", - DeltaConfigs.ENABLE_DELETION_VECTORS_CREATION.key -> "false", - DeltaConfigs.COLUMN_MAPPING_MODE.key -> "name" - ) - if (currIcebergCompatVersionOpt.nonEmpty) { - val currIcebergCompatVersionDeltaConfig = getIcebergCompatVersionConfigForValidVersion( - currIcebergCompatVersionOpt.get) - enableIcebergCompatConf ++= Map(currIcebergCompatVersionDeltaConfig.key -> "false") - } - - val alterConfTxn = target.startTransaction() - - if (alterConfTxn.protocol.minWriterVersion < 7) { - enableIcebergCompatConf += Protocol.MIN_WRITER_VERSION_PROP -> "7" - } - if (alterConfTxn.protocol.minReaderVersion < 3) { - enableIcebergCompatConf += Protocol.MIN_READER_VERSION_PROP -> "3" - } - - val metadata = alterConfTxn.metadata - val newMetadata = metadata.copy( - description = metadata.description, - configuration = metadata.configuration ++ enableIcebergCompatConf) - alterConfTxn.updateMetadata(newMetadata) - alterConfTxn.commit( - Nil, - DeltaOperations.UpgradeUniformProperties(enableIcebergCompatConf) - ) - } - - /** - * Helper function to get the num of addFiles as well as - * num of addFiles with ICEBERG_COMPAT_VERSION tag. - * @param icebergCompatVersion target iceberg compat version - * @param snapshot current snapshot - * @return (NumOfAddFiles, NumOfAddFilesWithIcebergCompatTag) - */ - private def getNumOfAddFiles( - icebergCompatVersion: Int, - table: DeltaTableV2, - snapshot: Snapshot): (Long, Long) = { - val numOfAddFilesWithTag = snapshot.allFiles - .select("tags") - .where(try_element_at(col("tags"), AddFile.Tags.ICEBERG_COMPAT_VERSION.name) - === icebergCompatVersion.toString) - .count() - val numOfAddFiles = snapshot.numOfFiles - logInfo(s"For table ${table.tableIdentifier} at version ${snapshot.version}, there are " + - s"$numOfAddFiles addFiles, and $numOfAddFilesWithTag addFiles with ICEBERG_COMPAT_VERSION=" + - s"$icebergCompatVersion tag.") - (numOfAddFiles, numOfAddFilesWithTag) - } - - /** - * Helper function to rewrite the table data files in Iceberg compatible way. - * This method would do following things: - * 1. Update the table properties to enable the target iceberg compat version and disable the - * existing iceberg compat version. - * 2. If target iceberg compat version require rewriting and not all addFiles has - * ICEBERG_COMPAT_VERSION=version tag, rewrite the table data files to be iceberg compatible - * and adding tag to all addFiles. - * 3. If universal format not enabled, alter the table properties to enable - * universalFormat = Iceberg. - * - * * There are six possible write combinations: - * | CurrentIcebergCompatVersion | TargetIcebergCompatVersion | Required steps| - * | --------------- | --------------- | --------------- | - * | None | 1 | 1, 3 | - * | None | 2 | 1, 2, 3 | - * | 1 | 1 | 3 | - * | 1 | 2 | 1, 2, 3 | - * | 2 | 1 | 1, 3 | - * | 2 | 2 | 2, 3 | - */ - private def doRewrite( - target: DeltaTableV2, - sparkSession: SparkSession, - targetIcebergCompatVersion: Int): Seq[Row] = { - - val snapshot = target.deltaLog.update() - val currIcebergCompatVersionOpt = getEnabledVersion(snapshot.metadata) - val targetVersionDeltaConfig = getIcebergCompatVersionConfigForValidVersion( - targetIcebergCompatVersion) - val versionChangeMayNeedRewrite = reorgMayNeedRewrite( - currIcebergCompatVersionOpt.getOrElse(0), targetIcebergCompatVersion) - - // Step 1: Update the table properties to enable the target iceberg compat version - val didUpdateIcebergCompatVersion = - if (!currIcebergCompatVersionOpt.contains(targetIcebergCompatVersion)) { - enableIcebergCompat(target, currIcebergCompatVersionOpt, targetVersionDeltaConfig) - logInfo(s"Update table ${target.tableIdentifier} to iceberg compat version = " + - s"$targetIcebergCompatVersion successfully.") - true - } else { - false - } - - // Step 2: Rewrite the table data files to be Iceberg compatible. - val (numOfAddFilesBefore, numOfAddFilesWithTagBefore) = getNumOfAddFiles( - targetIcebergCompatVersion, target, snapshot) - val allAddFilesHaveTag = numOfAddFilesWithTagBefore == numOfAddFilesBefore - // The table needs to be rewritten if: - // 1. The target iceberg compat version requires rewrite. - // 2. Not all addFile have ICEBERG_COMPAT_VERSION=targetVersion tag - val (metricsOpt, didRewrite) = if (versionChangeMayNeedRewrite && !allAddFilesHaveTag) { - logInfo(s"Reorg Table ${target.tableIdentifier} to iceberg compat version = " + - s"$targetIcebergCompatVersion need rewrite data files.") - val metrics = try { - optimizeByReorg( - sparkSession, - isPurge = false, - icebergCompatVersion = Some(targetIcebergCompatVersion) - ) - } catch { - case NonFatal(e) => - throw DeltaErrors.icebergCompatDataFileRewriteFailedException( - targetIcebergCompatVersion, e) - } - logInfo(s"Rewrite table ${target.tableIdentifier} to iceberg compat version = " + - s"$targetIcebergCompatVersion successfully.") - (Some(metrics), true) - } else { - (None, false) - } - val updatedSnapshot = target.deltaLog.update() - val (numOfAddFiles, numOfAddFilesWithIcebergCompatTag) = getNumOfAddFiles( - targetIcebergCompatVersion, target, updatedSnapshot) - if (versionChangeMayNeedRewrite && numOfAddFilesWithIcebergCompatTag != numOfAddFiles) { - throw DeltaErrors.icebergCompatReorgAddFileTagsMissingException( - updatedSnapshot.version, - targetIcebergCompatVersion, - numOfAddFiles, - numOfAddFilesWithIcebergCompatTag - ) - } - - // Step 3: Update the table properties to enable the universalFormat = Iceberg. - if (!icebergEnabled(updatedSnapshot.metadata)) { - val enableUniformConf = Map( - DeltaConfigs.UNIVERSAL_FORMAT_ENABLED_FORMATS.key -> ICEBERG_FORMAT) - AlterTableSetPropertiesDeltaCommand(target, enableUniformConf).run(sparkSession) - logInfo(s"Enabling universal format with iceberg compat version = " + - s"$targetIcebergCompatVersion for table ${target.tableIdentifier} succeeded.") - } - - recordDeltaEvent(updatedSnapshot.deltaLog, "delta.upgradeUniform.success", data = Map( - "currIcebergCompatVersion" -> currIcebergCompatVersionOpt.toString, - "targetIcebergCompatVersion" -> targetIcebergCompatVersion.toString, - "metrics" -> metricsOpt.toString, - "didUpdateIcebergCompatVersion" -> didUpdateIcebergCompatVersion.toString, - "needRewrite" -> versionChangeMayNeedRewrite.toString, - "didRewrite" -> didRewrite.toString, - "numOfAddFilesBefore" -> numOfAddFilesBefore.toString, - "numOfAddFilesWithIcebergCompatTagBefore" -> numOfAddFilesWithTagBefore.toString, - "numOfAddFilesAfter" -> numOfAddFiles.toString, - "numOfAddFilesWithIcebergCompatTagAfter" -> numOfAddFilesWithIcebergCompatTag.toString, - "universalFormatIcebergEnabled" -> icebergEnabled(target.deltaLog.update().metadata).toString - )) - metricsOpt.getOrElse(Seq.empty[Row]) - } - - /** - * Helper function to upgrade the table to uniform iceberg compat version. - */ - protected def upgradeUniformIcebergCompatVersion( - target: DeltaTableV2, - sparkSession: SparkSession, - targetIcebergCompatVersion: Int): Seq[Row] = { - try { - doRewrite(target, sparkSession, targetIcebergCompatVersion) - } catch { - case NonFatal(e) => - recordDeltaEvent(target.deltaLog, "delta.upgradeUniform.exception", data = Map( - "targetIcebergCompatVersion" -> targetIcebergCompatVersion.toString, - "exception" -> e.toString - )) - throw e - } - } -} diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/commands/WriteIntoDelta.scala b/spark/src/main/scala/org/apache/spark/sql/delta/commands/WriteIntoDelta.scala index eb70e96c6fa..f743cec5c3d 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/commands/WriteIntoDelta.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/commands/WriteIntoDelta.scala @@ -24,8 +24,6 @@ import org.apache.spark.sql.delta.constraints.Constraint import org.apache.spark.sql.delta.constraints.Constraints.Check import org.apache.spark.sql.delta.constraints.Invariants.ArbitraryExpression import org.apache.spark.sql.delta.schema.{ImplicitMetadataOperation, InvariantViolationException, SchemaUtils} -import org.apache.spark.sql.delta.skipping.clustering.ClusteredTableUtils -import org.apache.spark.sql.delta.skipping.clustering.temp.ClusterBySpec import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql._ @@ -116,9 +114,8 @@ case class WriteIntoDelta( override def write( txn: OptimisticTransaction, - sparkSession: SparkSession, - clusterBySpecOpt: Option[ClusterBySpec] = None, - isTableReplace: Boolean = false): Seq[Action] = { + sparkSession: SparkSession + ): Seq[Action] = { import org.apache.spark.sql.delta.implicits._ if (txn.readVersion > -1) { // This table already exists, check if the insert is valid. @@ -130,19 +127,6 @@ case class WriteIntoDelta( DeltaLog.assertRemovable(txn.snapshot) } } - val isReplaceWhere = mode == SaveMode.Overwrite && options.replaceWhere.nonEmpty - // Validate that the preview is enabled if we are writing to a clustered table. - ClusteredTableUtils.validatePreviewEnabled(txn.snapshot.protocol) - val finalClusterBySpecOpt = - if (mode == SaveMode.Append || isReplaceWhere) { - clusterBySpecOpt.foreach { clusterBySpec => - ClusteredTableUtils.validateClusteringColumnsInSnapshot(txn.snapshot, clusterBySpec) - } - // Append mode and replaceWhere cannot update the clustering columns. - None - } else { - clusterBySpecOpt - } val rearrangeOnly = options.rearrangeOnly val charPadding = sparkSession.conf.get(SQLConf.READ_SIDE_CHAR_PADDING.key, "false") == "true" val charAsVarchar = sparkSession.conf.get(SQLConf.CHAR_AS_VARCHAR) @@ -165,8 +149,7 @@ case class WriteIntoDelta( val newDomainMetadata = getNewDomainMetadata( txn, canUpdateMetadata, - isReplacingTable = isOverwriteOperation && options.replaceWhere.isEmpty, - finalClusterBySpecOpt + isReplacingTable = isOverwriteOperation && options.replaceWhere.isEmpty ) val replaceOnDataColsEnabled = @@ -176,9 +159,6 @@ case class WriteIntoDelta( if (txn.metadata.partitionColumns.isEmpty) { // We ignore dynamic partition overwrite mode for non-partitioned tables false - } else if (isTableReplace) { - // A replace table command should always replace the table, not just some partitions. - false } else if (options.replaceWhere.nonEmpty) { if (options.partitionOverwriteModeInOptions && options.isDynamicPartitionOverwriteMode) { // replaceWhere and dynamic partition overwrite conflict because they both specify which @@ -192,9 +172,7 @@ case class WriteIntoDelta( // precedence over session configs false } - } else { - options.isDynamicPartitionOverwriteMode - } + } else options.isDynamicPartitionOverwriteMode } if (useDynamicPartitionOverwriteMode && canOverwriteSchema) { diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/commands/WriteIntoDeltaLike.scala b/spark/src/main/scala/org/apache/spark/sql/delta/commands/WriteIntoDeltaLike.scala index 5e21d22ada8..1b141ed6bb0 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/commands/WriteIntoDeltaLike.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/commands/WriteIntoDeltaLike.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.delta.commands // scalastyle:off import.ordering.noEmptyLine -import org.apache.spark.sql.delta.skipping.clustering.temp.ClusterBySpec import org.apache.spark.sql.delta.DeltaLog import org.apache.spark.sql.delta.OptimisticTransaction import org.apache.spark.sql.delta.actions.Action @@ -60,9 +59,8 @@ trait WriteIntoDeltaLike { */ def write( txn: OptimisticTransaction, - sparkSession: SparkSession, - clusterBySpecOpt: Option[ClusterBySpec] = None, - isTableReplace: Boolean = false): Seq[Action] + sparkSession: SparkSession + ): Seq[Action] val deltaLog: DeltaLog diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/commands/alterDeltaTableCommands.scala b/spark/src/main/scala/org/apache/spark/sql/delta/commands/alterDeltaTableCommands.scala index 5a97d90df02..9c02da901d1 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/commands/alterDeltaTableCommands.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/commands/alterDeltaTableCommands.scala @@ -22,7 +22,6 @@ import java.util.concurrent.TimeUnit import scala.util.control.NonFatal -import org.apache.spark.sql.delta.skipping.clustering.ClusteringColumnInfo import org.apache.spark.sql.delta._ import org.apache.spark.sql.delta.actions.Protocol import org.apache.spark.sql.delta.catalog.DeltaTableV2 @@ -467,12 +466,6 @@ case class AlterTableDropColumnsDeltaCommand( if (droppingPartitionCols.nonEmpty) { throw DeltaErrors.dropPartitionColumnNotSupported(droppingPartitionCols) } - // Disallow dropping clustering columns. - val clusteringCols = ClusteringColumnInfo.extractLogicalNames(txn.snapshot) - val droppingClusteringCols = clusteringCols.filter(droppedColumnSet.contains(_)) - if (droppingClusteringCols.nonEmpty) { - throw DeltaErrors.dropClusteringColumnNotSupported(droppingClusteringCols) - } // Updates the delta statistics column list by removing the dropped columns from it. val newConfiguration = metadata.configuration ++ StatisticsCollection.dropDeltaStatsColumns(metadata, columnsToDrop) diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/commands/merge/ClassicMergeExecutor.scala b/spark/src/main/scala/org/apache/spark/sql/delta/commands/merge/ClassicMergeExecutor.scala index dfdd4d76af5..16736c821a1 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/commands/merge/ClassicMergeExecutor.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/commands/merge/ClassicMergeExecutor.scala @@ -514,26 +514,11 @@ trait ClassicMergeExecutor extends MergeOutputGeneration { val touchedFilesWithDVs = DMLWithDeletionVectorsHelper .findFilesWithMatchingRows(deltaTxn, nameToAddFileMap, matchedDVResult) - val (dvActions, metricsMap) = DMLWithDeletionVectorsHelper.processUnmodifiedData( + val (dvActions, _) = DMLWithDeletionVectorsHelper.processUnmodifiedData( spark, touchedFilesWithDVs, deltaTxn.snapshot) - metrics("numTargetDeletionVectorsAdded") - .set(metricsMap.getOrElse("numDeletionVectorsAdded", 0L)) - metrics("numTargetDeletionVectorsRemoved") - .set(metricsMap.getOrElse("numDeletionVectorsRemoved", 0L)) - metrics("numTargetDeletionVectorsUpdated") - .set(metricsMap.getOrElse("numDeletionVectorsUpdated", 0L)) - - // When DVs are enabled we override metrics related to removed files. - metrics("numTargetFilesRemoved").set(metricsMap.getOrElse("numRemovedFiles", 0L)) - - val fullyRemovedFiles = touchedFilesWithDVs.filter(_.isFullyReplaced()).map(_.fileLogEntry) - val (removedBytes, removedPartitions) = totalBytesAndDistinctPartitionValues(fullyRemovedFiles) - metrics("numTargetBytesRemoved").set(removedBytes) - metrics("numTargetPartitionsRemovedFrom").set(removedPartitions) - dvActions } } diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/commands/merge/MergeStats.scala b/spark/src/main/scala/org/apache/spark/sql/delta/commands/merge/MergeStats.scala index 09bc69a3189..eeef2753754 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/commands/merge/MergeStats.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/commands/merge/MergeStats.scala @@ -105,9 +105,6 @@ case class MergeStats( targetRowsDeleted: Long, targetRowsMatchedDeleted: Long, targetRowsNotMatchedBySourceDeleted: Long, - numTargetDeletionVectorsAdded: Long, - numTargetDeletionVectorsRemoved: Long, - numTargetDeletionVectorsUpdated: Long, // MergeMaterializeSource stats materializeSourceReason: Option[String] = None, @@ -177,11 +174,6 @@ object MergeStats { targetRowsMatchedDeleted = metrics("numTargetRowsMatchedDeleted").value, targetRowsNotMatchedBySourceDeleted = metrics("numTargetRowsNotMatchedBySourceDeleted").value, - // Deletion Vector metrics. - numTargetDeletionVectorsAdded = metrics("numTargetDeletionVectorsAdded").value, - numTargetDeletionVectorsRemoved = metrics("numTargetDeletionVectorsRemoved").value, - numTargetDeletionVectorsUpdated = metrics("numTargetDeletionVectorsUpdated").value, - // Deprecated fields updateConditionExpr = null, updateExprs = null, diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/files/TransactionalWrite.scala b/spark/src/main/scala/org/apache/spark/sql/delta/files/TransactionalWrite.scala index 09a9bf5764c..33506412c42 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/files/TransactionalWrite.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/files/TransactionalWrite.scala @@ -22,7 +22,6 @@ import org.apache.spark.sql.delta._ import org.apache.spark.sql.delta.actions._ import org.apache.spark.sql.delta.commands.cdc.CDCReader import org.apache.spark.sql.delta.constraints.{Constraint, Constraints, DeltaInvariantCheckerExec} -import org.apache.spark.sql.delta.hooks.AutoCompact import org.apache.spark.sql.delta.metering.DeltaLogging import org.apache.spark.sql.delta.perf.DeltaOptimizedWriterExec import org.apache.spark.sql.delta.schema._ @@ -417,7 +416,7 @@ trait TransactionalWrite extends DeltaLogging { self: OptimisticTransactionImpl } // Iceberg spec requires partition columns in data files - val writePartitionColumns = IcebergCompat.isAnyEnabled(metadata) + val writePartitionColumns = IcebergCompatV1.isEnabled(metadata) // Retain only a minimal selection of Spark writer options to avoid any potential // compatibility issues val options = (writeOptions match { @@ -452,7 +451,7 @@ trait TransactionalWrite extends DeltaLogging { self: OptimisticTransactionImpl } } - var resultFiles = + val resultFiles = (if (optionalStatsTracker.isDefined) { committer.addedStatuses.map { a => a.copy(stats = optionalStatsTracker.map( @@ -473,16 +472,6 @@ trait TransactionalWrite extends DeltaLogging { self: OptimisticTransactionImpl case _ => true } - // add [[AddFile.Tags.ICEBERG_COMPAT_VERSION.name]] tags to addFiles - if (IcebergCompatV2.isEnabled(metadata)) { - resultFiles = resultFiles.map { addFile => - val tags = if (addFile.tags != null) addFile.tags else Map.empty[String, String] - addFile.copy(tags = tags + (AddFile.Tags.ICEBERG_COMPAT_VERSION.name -> "2")) - } - } - - - if (resultFiles.nonEmpty && !isOptimize) registerPostCommitHook(AutoCompact) resultFiles.toSeq ++ committer.changeFiles } diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/hooks/AutoCompact.scala b/spark/src/main/scala/org/apache/spark/sql/delta/hooks/AutoCompact.scala deleted file mode 100644 index 9cd1ed5f179..00000000000 --- a/spark/src/main/scala/org/apache/spark/sql/delta/hooks/AutoCompact.scala +++ /dev/null @@ -1,254 +0,0 @@ -/* - * Copyright (2021) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.delta.hooks - -import org.apache.spark.sql.delta.skipping.clustering.ClusteredTableUtils -import org.apache.spark.sql.delta._ -import org.apache.spark.sql.delta.actions._ -import org.apache.spark.sql.delta.commands.{DeltaOptimizeContext, OptimizeExecutor} -import org.apache.spark.sql.delta.commands.optimize._ -import org.apache.spark.sql.delta.metering.DeltaLogging -import org.apache.spark.sql.delta.sources.DeltaSQLConf -import org.apache.spark.sql.delta.stats.AutoCompactPartitionStats - -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.catalog.CatalogTable -import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.internal.SQLConf - -/** - * A trait for post commit hook which compacts files in a Delta table. This hook acts as a cheaper - * version of the OPTIMIZE command, by attempting to compact small files together into fewer bigger - * files. - * - * Auto Compact chooses files to compact greedily by looking at partition directories which - * have the largest number of files that are under a certain size threshold and launches a bounded - * number of optimize tasks based on the capacity of the cluster. - */ -trait AutoCompactBase extends PostCommitHook with DeltaLogging { - - override val name: String = "Auto Compact" - - private[delta] val OP_TYPE = "delta.commit.hooks.autoOptimize" - - /** - * This method returns the type of Auto Compaction to use on a delta table or returns None - * if Auto Compaction is disabled. - * Prioritization: - * 1. The highest priority is given to [[DeltaSQLConf.DELTA_AUTO_COMPACT_ENABLED]] config. - * 2. Then we check if the deprecated property `DeltaConfigs.AUTO_OPTIMIZE` is set. If yes, then - * we return [[AutoCompactType.Enabled]] type. - * 3. Then we check the table property [[DeltaConfigs.AUTO_COMPACT]]. - * 4. If none of 1/2/3 are set explicitly, then we return None - */ - def getAutoCompactType(conf: SQLConf, metadata: Metadata): Option[AutoCompactType] = { - // If user-facing conf is set to something, use that value. - val autoCompactTypeFromConf = - conf.getConf(DeltaSQLConf.DELTA_AUTO_COMPACT_ENABLED).map(AutoCompactType(_)) - if (autoCompactTypeFromConf.nonEmpty) return autoCompactTypeFromConf.get - - // If user-facing conf is not set, use what table property says. - val deprecatedFlag = DeltaConfigs.AUTO_OPTIMIZE.fromMetaData(metadata) - val autoCompactTypeFromPropertyOrDefaultValue = deprecatedFlag match { - case Some(true) => - Some(AutoCompactType.Enabled) - case _ => - // If the legacy property `DeltaConfigs.AUTO_OPTIMIZE` is false or not set, then check - // the new table property `DeltaConfigs.AUTO_COMPACT`. - val confValueFromTableProperty = DeltaConfigs.AUTO_COMPACT.fromMetaData(metadata) - confValueFromTableProperty match { - case Some(v) => - // Table property is set to something explicitly by user. - AutoCompactType(v) - case None => - AutoCompactType(AutoCompactType.DISABLED) // Default to disabled - } - } - autoCompactTypeFromPropertyOrDefaultValue - } - - private[hooks] def shouldSkipAutoCompact( - autoCompactTypeOpt: Option[AutoCompactType], - spark: SparkSession, - txn: OptimisticTransactionImpl): Boolean = { - // If auto compact type is empty, then skip compaction - if (autoCompactTypeOpt.isEmpty) return true - - // Skip Auto Compaction, if one of the following conditions is satisfied: - // -- Auto Compaction is not enabled. - // -- Transaction execution time is empty, which means the parent transaction is not committed. - !AutoCompactUtils.isQualifiedForAutoCompact(spark, txn) - - } - - override def run( - spark: SparkSession, - txn: OptimisticTransactionImpl, - committedVersion: Long, - postCommitSnapshot: Snapshot, - actions: Seq[Action]): Unit = { - val conf = spark.sessionState.conf - val autoCompactTypeOpt = getAutoCompactType(conf, postCommitSnapshot.metadata) - // Skip Auto Compact if current transaction is not qualified or the table is not qualified - // based on the value of autoCompactTypeOpt. - if (shouldSkipAutoCompact(autoCompactTypeOpt, spark, txn)) return - compactIfNecessary( - spark, - txn, - postCommitSnapshot, - OP_TYPE, - maxDeletedRowsRatio = None) - } - - /** - * Compact the target table of write transaction `txn` only when there are sufficient amount of - * small size files. - */ - private[delta] def compactIfNecessary( - spark: SparkSession, - txn: OptimisticTransactionImpl, - postCommitSnapshot: Snapshot, - opType: String, - maxDeletedRowsRatio: Option[Double] - ): Seq[OptimizeMetrics] = { - val tableId = txn.deltaLog.tableId - val autoCompactRequest = AutoCompactUtils.prepareAutoCompactRequest( - spark, - txn, - postCommitSnapshot, - txn.partitionsAddedToOpt.map(_.toSet), - opType, - maxDeletedRowsRatio) - if (autoCompactRequest.shouldCompact) { - try { - val metrics = AutoCompact - .compact( - spark, - txn.deltaLog, - txn.catalogTable, - autoCompactRequest.targetPartitionsPredicate, - opType, - maxDeletedRowsRatio - ) - val partitionsStats = AutoCompactPartitionStats.instance(spark) - // Mark partitions as compacted before releasing them. - // Otherwise an already compacted partition might get picked up by a concurrent thread. - // But only marks it as compacted, if no exception was thrown by auto compaction so that the - // partitions stay eligible for subsequent auto compactions. - partitionsStats.markPartitionsAsCompacted( - tableId, - autoCompactRequest.allowedPartitions - ) - metrics - } catch { - case e: Throwable => - logError("Auto Compaction failed with: " + e.getMessage) - throw e - } finally { - if (AutoCompactUtils.reservePartitionEnabled(spark)) { - AutoCompactPartitionReserve.releasePartitions( - tableId, - autoCompactRequest.allowedPartitions - ) - } - } - } else { - Seq.empty[OptimizeMetrics] - } - } - - - /** - * Launch Auto Compaction jobs if there is sufficient capacity. - * @param spark The spark session of the parent transaction that triggers this Auto Compaction. - * @param deltaLog The delta log of the parent transaction. - * @return the optimize metrics of this compaction job. - */ - private[delta] def compact( - spark: SparkSession, - deltaLog: DeltaLog, - catalogTable: Option[CatalogTable], - partitionPredicates: Seq[Expression] = Nil, - opType: String = OP_TYPE, - maxDeletedRowsRatio: Option[Double] = None) - : Seq[OptimizeMetrics] = recordDeltaOperation(deltaLog, opType) { - val maxFileSize = spark.conf.get(DeltaSQLConf.DELTA_AUTO_COMPACT_MAX_FILE_SIZE) - val minFileSizeOpt = Some(spark.conf.get(DeltaSQLConf.DELTA_AUTO_COMPACT_MIN_FILE_SIZE) - .getOrElse(maxFileSize / 2)) - val maxFileSizeOpt = Some(maxFileSize) - recordDeltaOperation(deltaLog, s"$opType.execute") { - val txn = deltaLog.startTransaction(catalogTable) - val optimizeContext = DeltaOptimizeContext( - isPurge = false, - minFileSizeOpt, - maxFileSizeOpt, - maxDeletedRowsRatio = maxDeletedRowsRatio - ) - val rows = new OptimizeExecutor(spark, txn, partitionPredicates, Seq(), true, optimizeContext) - .optimize() - val metrics = rows.map(_.getAs[OptimizeMetrics](1)) - recordDeltaEvent(deltaLog, s"$opType.execute.metrics", data = metrics.head) - metrics - } - } - -} - -/** - * Post commit hook for Auto Compaction. - */ -case object AutoCompact extends AutoCompactBase -/** - * A trait describing the type of Auto Compaction. - */ -sealed trait AutoCompactType { - val configValueStrings: Seq[String] -} - -object AutoCompactType { - - private[hooks] val DISABLED = "false" - - /** - * Enable auto compact. - * 1. MAX_FILE_SIZE is configurable and defaults to 128 MB unless overridden. - * 2. MIN_FILE_SIZE is configurable and defaults to MAX_FILE_SIZE / 2 unless overridden. - * Note: User can use DELTA_AUTO_COMPACT_MAX_FILE_SIZE to override this value. - */ - case object Enabled extends AutoCompactType { - override val configValueStrings = Seq( - "true" - ) - } - - - /** - * Converts the config value String (coming from [[DeltaSQLConf.DELTA_AUTO_COMPACT_ENABLED]] conf - * or [[DeltaConfigs.AUTO_COMPACT]] table property) and translates into the [[AutoCompactType]]. - */ - def apply(value: String): Option[AutoCompactType] = { - if (Enabled.configValueStrings.contains(value)) return Some(Enabled) - if (value == DISABLED) return None - throw DeltaErrors.invalidAutoCompactType(value) - } - - // All allowed values for [[DeltaSQLConf.DELTA_AUTO_COMPACT_ENABLED]] and - // [[DeltaConfigs.AUTO_COMPACT]]. - val ALLOWED_VALUES = - Enabled.configValueStrings ++ - Seq(DISABLED) -} diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/hooks/AutoCompactUtils.scala b/spark/src/main/scala/org/apache/spark/sql/delta/hooks/AutoCompactUtils.scala deleted file mode 100644 index c3ed3f84397..00000000000 --- a/spark/src/main/scala/org/apache/spark/sql/delta/hooks/AutoCompactUtils.scala +++ /dev/null @@ -1,399 +0,0 @@ -/* - * Copyright (2021) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.delta.hooks - -import scala.collection.mutable - -// scalastyle:off import.ordering.noEmptyLine -import org.apache.spark.sql.delta.{DeltaLog, OptimisticTransactionImpl, Snapshot} -import org.apache.spark.sql.delta.metering.DeltaLogging -import org.apache.spark.sql.delta.sources.DeltaSQLConf -import org.apache.spark.sql.delta.sources.DeltaSQLConf._ -import org.apache.spark.sql.delta.stats.AutoCompactPartitionStats - -import org.apache.spark.internal.config.ConfigEntry -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute -import org.apache.spark.sql.catalyst.expressions.{And, Cast, EqualNullSafe, Expression, Literal, Or} -import org.apache.spark.sql.functions.collect_list - -/** - * The request class that contains all information needed for Auto Compaction. - * @param shouldCompact True if Auto Compact should start. - * @param optimizeContext The context that control execution of optimize command. - * @param targetPartitionsPredicate The predicate of the target partitions of this Auto Compact - * request. - */ -case class AutoCompactRequest( - shouldCompact: Boolean, - allowedPartitions: AutoCompactUtils.PartitionKeySet, - targetPartitionsPredicate: Seq[Expression] = Nil) { -} - -object AutoCompactRequest { - /** Return a default AutoCompactRequest object that doesn't trigger Auto Compact. */ - def noopRequest: AutoCompactRequest = - AutoCompactRequest( - shouldCompact = false, - allowedPartitions = Set.empty - ) -} - -object AutoCompactUtils extends DeltaLogging { - type PartitionKey = Map[String, String] - type PartitionKeySet = Set[PartitionKey] - - val STATUS_NAME = { - "status" - } - - /** Create partition predicate from a partition key. */ - private def createPartitionPredicate( - postCommitSnapshot: Snapshot, - partitions: PartitionKeySet): Seq[Expression] = { - val schema = postCommitSnapshot.metadata.physicalPartitionSchema - val partitionBranches = partitions.filterNot(_.isEmpty).map { partition => - partition.toSeq - .map { case (key, value) => - val field = schema(key) - EqualNullSafe(UnresolvedAttribute.quoted(key), Cast(Literal(value), field.dataType)) - } - .reduceLeft[Expression]((l, r) => And(l, r)) - } - if (partitionBranches.size > 1) { - Seq(partitionBranches.reduceLeft[Expression]((l, r) => Or(l, r))) - } else if (partitionBranches.size == 1) { - partitionBranches.toList - } else { - Seq.empty - } - } - - /** True if Auto Compaction only runs on modified partitions. */ - def isModifiedPartitionsOnlyAutoCompactEnabled(spark: SparkSession): Boolean = - spark.sessionState.conf.getConf(DELTA_AUTO_COMPACT_MODIFIED_PARTITIONS_ONLY_ENABLED) - - def reservePartitionEnabled(spark: SparkSession): Boolean = - spark.sessionState.conf.getConf(DELTA_AUTO_COMPACT_RESERVE_PARTITIONS_ENABLED) - - /** - * Get the minimum number of files to trigger Auto Compact. - */ - def minNumFilesForAutoCompact(spark: SparkSession): Int = { - spark.sessionState.conf.getConf(DELTA_AUTO_COMPACT_MIN_NUM_FILES) - } - - - /** - * Try to reserve partitions inside `partitionsAddedToOpt` for Auto Compaction. - * @return (shouldCompact, finalPartitions) The value of needCompaction is True if Auto - * Compaction needs to run. `finalPartitions` is the set of target partitions that were - * reserved for compaction. If finalPartitions is empty, then all partitions need to be - * considered. - */ - private def reserveTablePartitions( - spark: SparkSession, - deltaLog: DeltaLog, - postCommitSnapshot: Snapshot, - partitionsAddedToOpt: Option[PartitionKeySet], - opType: String, - maxDeletedRowsRatio: Option[Double]): (Boolean, PartitionKeySet) = { - import AutoCompactPartitionReserve._ - if (partitionsAddedToOpt.isEmpty) { - recordDeltaEvent(deltaLog, opType, data = Map(STATUS_NAME -> "skipEmptyIngestion")) - // If partitionsAddedToOpt is empty, then just skip compact since it means there is no file - // added in parent transaction and we do not want to hook AC on empty commits. - return (false, Set.empty[PartitionKey]) - } - - // Reserve partitions as following: - // 1) First check if any partitions are free, i.e. no concurrent auto-compact thread is running. - // 2) From free partitions check if any are eligible based on the number of small files. - // 3) From free partitions check if any are eligible based on the deletion vectors. - // 4) Try and reserve the union of the two lists. - // All concurrent accesses to partitions reservation and partition stats are managed by the - // [[AutoCompactPartitionReserve]] and [[AutoCompactPartitionStats]] singletons. - val shouldReservePartitions = - isModifiedPartitionsOnlyAutoCompactEnabled(spark) && reservePartitionEnabled(spark) - val freePartitions = - if (shouldReservePartitions) { - filterFreePartitions(deltaLog.tableId, partitionsAddedToOpt.get) - } else { - partitionsAddedToOpt.get - } - - // Early abort if all partitions are reserved. - if (freePartitions.isEmpty) { - recordDeltaEvent(deltaLog, opType, - data = Map(STATUS_NAME -> "skipAllPartitionsAlreadyReserved")) - return (false, Set.empty[PartitionKey]) - } - - // Check min number of files criteria. - val ChosenPartitionsResult(shouldCompactBasedOnNumFiles, - chosenPartitionsBasedOnNumFiles, minNumFilesLogMsg) = - choosePartitionsBasedOnMinNumSmallFiles( - spark, - deltaLog, - postCommitSnapshot, - freePartitions - ) - if (shouldCompactBasedOnNumFiles && chosenPartitionsBasedOnNumFiles.isEmpty) { - // Run on all partitions, no need to check other criteria. - // Note: this outcome of [choosePartitionsBasedOnMinNumSmallFiles] - // is also only possible if partitions reservation is turned off, - // so we do not need to reserve partitions. - recordDeltaEvent(deltaLog, opType, data = Map(STATUS_NAME -> "runOnAllPartitions")) - return (shouldCompactBasedOnNumFiles, chosenPartitionsBasedOnNumFiles) - } - - // Check files with DVs criteria. - val (shouldCompactBasedOnDVs, chosenPartitionsBasedOnDVs) = - choosePartitionsBasedOnDVs(freePartitions, postCommitSnapshot, maxDeletedRowsRatio) - - var finalPartitions = chosenPartitionsBasedOnNumFiles ++ chosenPartitionsBasedOnDVs - if (isModifiedPartitionsOnlyAutoCompactEnabled(spark)) { - val maxNumPartitions = spark.conf.get(DELTA_AUTO_COMPACT_MAX_NUM_MODIFIED_PARTITIONS) - finalPartitions = if (finalPartitions.size > maxNumPartitions) { - // Choose maxNumPartitions at random. - scala.util.Random.shuffle(finalPartitions.toIndexedSeq).take(maxNumPartitions).toSet - } else { - finalPartitions - } - } - - val numChosenPartitions = finalPartitions.size - if (shouldReservePartitions) { - finalPartitions = tryReservePartitions(deltaLog.tableId, finalPartitions) - } - // Abort if all chosen partitions were reserved by a concurrent thread. - if (numChosenPartitions > 0 && finalPartitions.isEmpty) { - recordDeltaEvent(deltaLog, opType, - data = Map(STATUS_NAME -> "skipAllPartitionsAlreadyReserved")) - return (false, Set.empty[PartitionKey]) - } - - val shouldCompact = shouldCompactBasedOnNumFiles || shouldCompactBasedOnDVs - val statusLogMessage = - if (!shouldCompact) { - "skip" + minNumFilesLogMsg - } else if (shouldCompactBasedOnNumFiles && !shouldCompactBasedOnDVs) { - "run" + minNumFilesLogMsg - } else if (shouldCompactBasedOnNumFiles && shouldCompactBasedOnDVs) { - "run" + minNumFilesLogMsg + "AndPartitionsWithDVs" - } else if (!shouldCompactBasedOnNumFiles && shouldCompactBasedOnDVs) { - "runOnPartitionsWithDVs" - } - val logData = scala.collection.mutable.Map(STATUS_NAME -> statusLogMessage) - if (finalPartitions.nonEmpty) { - logData += ("partitions" -> finalPartitions.size.toString) - } - recordDeltaEvent(deltaLog, opType, data = logData) - - (shouldCompactBasedOnNumFiles || shouldCompactBasedOnDVs, finalPartitions) - } - - private case class ChosenPartitionsResult( - shouldRunAC: Boolean, - chosenPartitions: PartitionKeySet, - logMessage: String) - - private def choosePartitionsBasedOnMinNumSmallFiles( - spark: SparkSession, - deltaLog: DeltaLog, - postCommitSnapshot: Snapshot, - freePartitionsAddedTo: PartitionKeySet - ) = { - def getConf[T](entry: ConfigEntry[T]): T = spark.sessionState.conf.getConf(entry) - - val minNumFiles = minNumFilesForAutoCompact(spark) - val partitionEarlySkippingEnabled = - getConf(DELTA_AUTO_COMPACT_EARLY_SKIP_PARTITION_TABLE_ENABLED) - val tablePartitionStats = AutoCompactPartitionStats.instance(spark) - if (isModifiedPartitionsOnlyAutoCompactEnabled(spark)) { - // If modified partition only Auto Compact is enabled, pick the partitions that have more - // number of files than minNumFiles. - // If table partition early skipping feature is enabled, use the current minimum number of - // files threshold; otherwise, use 0 to indicate that any partition is qualified. - val minNumFilesPerPartition = if (partitionEarlySkippingEnabled) minNumFiles else 0L - val pickedPartitions = tablePartitionStats.filterPartitionsWithSmallFiles( - deltaLog.tableId, - freePartitionsAddedTo, - minNumFilesPerPartition) - if (pickedPartitions.isEmpty) { - ChosenPartitionsResult(shouldRunAC = false, - chosenPartitions = pickedPartitions, - logMessage = "InsufficientFilesInModifiedPartitions") - } else { - ChosenPartitionsResult(shouldRunAC = true, - chosenPartitions = pickedPartitions, - logMessage = "OnModifiedPartitions") - } - } else if (partitionEarlySkippingEnabled) { - // If only early skipping is enabled, then check whether there is any partition with more - // files than minNumFiles. - val maxNumFiles = tablePartitionStats.maxNumFilesInTable(deltaLog.tableId) - val shouldCompact = maxNumFiles >= minNumFiles - if (shouldCompact) { - ChosenPartitionsResult(shouldRunAC = true, - chosenPartitions = Set.empty[PartitionKey], - logMessage = "OnAllPartitions") - } else { - ChosenPartitionsResult(shouldRunAC = false, - chosenPartitions = Set.empty[PartitionKey], - logMessage = "InsufficientInAllPartitions") - } - } else { - // If both are disabled, then Auto Compaction should search all partitions of the target - // table. - ChosenPartitionsResult(shouldRunAC = true, - chosenPartitions = Set.empty[PartitionKey], - logMessage = "OnAllPartitions") - } - } - - private def choosePartitionsBasedOnDVs( - freePartitionsAddedTo: PartitionKeySet, - postCommitSnapshot: Snapshot, - maxDeletedRowsRatio: Option[Double]) = { - var partitionsWithDVs = if (maxDeletedRowsRatio.nonEmpty) { - postCommitSnapshot.allFiles - .where("deletionVector IS NOT NULL") - .where( - s""" - |(deletionVector.cardinality / stats:`numRecords`) > ${maxDeletedRowsRatio.get} - |""".stripMargin) - // Cast map to string so we can group by it. - // The string representation might not be deterministic. - // Still, there is only a limited number of representations we could get for a given map, - // Which should sufficiently reduce the data collected on the driver. - // We then make sure the partitions are distinct on the driver. - .selectExpr("CAST(partitionValues AS STRING) as partitionValuesStr", "partitionValues") - .groupBy("partitionValuesStr") - .agg(collect_list("partitionValues").as("partitionValues")) - .selectExpr("partitionValues[0] as partitionValues") - .collect() - .map(_.getAs[Map[String, String]]("partitionValues")).toSet - } else { - Set.empty[PartitionKey] - } - partitionsWithDVs = partitionsWithDVs.intersect(freePartitionsAddedTo) - (partitionsWithDVs.nonEmpty, partitionsWithDVs) - } - - /** - * Prepare an [[AutoCompactRequest]] object based on the statistics of partitions inside - * `partitionsAddedToOpt`. - * - * @param partitionsAddedToOpt The partitions that contain AddFile objects created by parent - * transaction. - * @param maxDeletedRowsRatio If set, signals to Auto Compaction to rewrite files with - * DVs with maxDeletedRowsRatio above this threshold. - */ - def prepareAutoCompactRequest( - spark: SparkSession, - txn: OptimisticTransactionImpl, - postCommitSnapshot: Snapshot, - partitionsAddedToOpt: Option[PartitionKeySet], - opType: String, - maxDeletedRowsRatio: Option[Double]): AutoCompactRequest = { - val (needAutoCompact, reservedPartitions) = reserveTablePartitions( - spark, - txn.deltaLog, - postCommitSnapshot, - partitionsAddedToOpt, - opType, - maxDeletedRowsRatio) - AutoCompactRequest( - needAutoCompact, - reservedPartitions, - createPartitionPredicate(postCommitSnapshot, reservedPartitions)) - } - - /** - * True if this transaction is qualified for Auto Compaction. - * - When current transaction is not blind append, it is safe to enable Auto Compaction when - * DELTA_AUTO_COMPACT_MODIFIED_PARTITIONS_ONLY_ENABLED is true, or it's an un-partitioned table, - * because then we cannot introduce _additional_ conflicts with concurrent write transactions. - */ - def isQualifiedForAutoCompact( - spark: SparkSession, - txn: OptimisticTransactionImpl): Boolean = { - // If txnExecutionTimeMs is empty, there is no transaction commit. - if (txn.txnExecutionTimeMs.isEmpty) return false - // If modified partitions only mode is not enabled, return true to avoid subsequent checking. - if (!isModifiedPartitionsOnlyAutoCompactEnabled(spark)) return true - - val nonBlindAppendAutoCompactEnabled = - spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_AUTO_COMPACT_NON_BLIND_APPEND_ENABLED) - !(nonBlindAppendAutoCompactEnabled && txn.isBlindAppend) - } - -} - -/** - * Thread-safe singleton to keep track of partitions reserved for auto-compaction. - */ -object AutoCompactPartitionReserve { - - import org.apache.spark.sql.delta.hooks.AutoCompactUtils.PartitionKey - - // Key is table id and the value the set of currently reserved partition hashes. - private val reservedTablesPartitions = new mutable.LinkedHashMap[String, Set[Int]] - - /** - * @return Partitions from targetPartitions that are not reserved. - */ - def filterFreePartitions(tableId: String, targetPartitions: Set[PartitionKey]) - : Set[PartitionKey] = synchronized { - val reservedPartitionKeys = reservedTablesPartitions.getOrElse(tableId, Set.empty) - targetPartitions.filter(partition => !reservedPartitionKeys.contains(partition.##)) - } - - /** - * Try to reserve partitions from [[targetPartitions]] which are not yet reserved. - * @return partitions from targetPartitions which were not previously reserved. - */ - def tryReservePartitions(tableId: String, targetPartitions: Set[PartitionKey]) - : Set[PartitionKey] = synchronized { - val allReservedPartitions = reservedTablesPartitions.getOrElse(tableId, Set.empty) - val unReservedPartitionsFromTarget = targetPartitions - .filter(targetPartition => !allReservedPartitions.contains(targetPartition.##)) - val newAllReservedPartitions = allReservedPartitions ++ unReservedPartitionsFromTarget.map(_.##) - reservedTablesPartitions.update(tableId, newAllReservedPartitions) - unReservedPartitionsFromTarget - } - - - /** - * Releases the reserved table partitions to allow other threads to reserve them. - * @param tableId The identity of the target table of Auto Compaction. - * @param reservedPartitions The set of partitions, which were reserved and which need releasing. - */ - def releasePartitions( - tableId: String, - reservedPartitions: Set[PartitionKey]): Unit = synchronized { - val allReservedPartitions = reservedTablesPartitions.getOrElse(tableId, Set.empty) - val newPartitions = allReservedPartitions -- reservedPartitions.map(_.##) - reservedTablesPartitions.update(tableId, newPartitions) - } - - /** This is test only code to reset the state of table partition reservations. */ - private[delta] def resetTestOnly(): Unit = synchronized { - reservedTablesPartitions.clear() - } -} diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/hooks/IcebergConverterHook.scala b/spark/src/main/scala/org/apache/spark/sql/delta/hooks/IcebergConverterHook.scala index a2dbc5409e5..9cae8940f46 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/hooks/IcebergConverterHook.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/hooks/IcebergConverterHook.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.delta.hooks import org.apache.spark.sql.delta.{OptimisticTransactionImpl, Snapshot, UniversalFormat} -import org.apache.spark.sql.delta.actions.{Action, Metadata} +import org.apache.spark.sql.delta.actions.Action import org.apache.spark.sql.delta.metering.DeltaLogging import org.apache.spark.sql.delta.sources.DeltaSQLConf.DELTA_UNIFORM_ICEBERG_SYNC_CONVERT_ENABLED @@ -45,8 +45,7 @@ object IcebergConverterHook extends PostCommitHook with DeltaLogging { val converter = postCommitSnapshot.deltaLog.icebergConverter - if (spark.sessionState.conf.getConf(DELTA_UNIFORM_ICEBERG_SYNC_CONVERT_ENABLED) || - !UniversalFormat.icebergEnabled(txn.snapshot.metadata)) { // UniForm was not enabled + if (spark.sessionState.conf.getConf(DELTA_UNIFORM_ICEBERG_SYNC_CONVERT_ENABLED)) { converter.convertSnapshot(postCommitSnapshot, txn) } else { converter.enqueueSnapshotForConversion(postCommitSnapshot, txn) diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/hooks/UpdateCatalog.scala b/spark/src/main/scala/org/apache/spark/sql/delta/hooks/UpdateCatalog.scala deleted file mode 100644 index d4e5c6162d7..00000000000 --- a/spark/src/main/scala/org/apache/spark/sql/delta/hooks/UpdateCatalog.scala +++ /dev/null @@ -1,370 +0,0 @@ -/* - * Copyright (2021) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.delta.hooks - -import java.nio.charset.Charset -import java.util.concurrent.atomic.AtomicInteger - -import scala.collection.JavaConverters._ -import scala.concurrent.{ExecutionContext, Future, TimeoutException} -import scala.util.Try -import scala.util.control.NonFatal - -import org.apache.spark.sql.delta.{DeltaConfigs, DeltaTableIdentifier, OptimisticTransactionImpl, Snapshot} -import org.apache.spark.sql.delta.actions.{Action, Metadata} -import org.apache.spark.sql.delta.metering.DeltaLogging -import org.apache.spark.sql.delta.sources.DeltaSQLConf -import org.apache.spark.sql.delta.util.threads.DeltaThreadPool -import org.apache.commons.lang3.exception.ExceptionUtils - -import org.apache.spark.internal.config.ConfigEntry -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.catalog.CatalogTable -import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{StructField, StructType} -import org.apache.spark.util.ThreadUtils - -/** - * Factory object to create an UpdateCatalog post commit hook. This should always be used - * instead of directly creating a specific hook. - */ -object UpdateCatalogFactory { - def getUpdateCatalogHook(table: CatalogTable, spark: SparkSession): UpdateCatalogBase = { - UpdateCatalog(table) - } -} - -/** - * Base trait for post commit hooks that want to update the catalog with the - * latest table schema and properties. - */ -trait UpdateCatalogBase extends PostCommitHook with DeltaLogging { - - protected val table: CatalogTable - - override def run( - spark: SparkSession, - txn: OptimisticTransactionImpl, - committedVersion: Long, - postCommitSnapshot: Snapshot, - actions: Seq[Action]): Unit = { - // There's a potential race condition here, where a newer commit has already triggered - // this to run. That's fine. - executeOnWrite(spark, postCommitSnapshot) - } - - /** - * Used to manually execute an UpdateCatalog hook during a write. - */ - def executeOnWrite( - spark: SparkSession, - snapshot: Snapshot - ): Unit - - - /** - * Update the schema in the catalog based on the provided snapshot. - */ - def updateSchema(spark: SparkSession, snapshot: Snapshot): Unit - - /** - * Update the properties in the catalog based on the provided snapshot. - */ - protected def updateProperties(spark: SparkSession, snapshot: Snapshot): Unit - - /** - * Checks if the table schema has changed in the Snapshot with respect to what's stored in - * the catalog. - */ - protected def schemaHasChanged(snapshot: Snapshot, spark: SparkSession): Boolean - - /** - * Checks if the table properties have changed in the Snapshot with respect to what's stored in - * the catalog. - * - * Visible for testing. - */ - protected[sql] def propertiesHaveChanged( - properties: Map[String, String], - metadata: Metadata, - spark: SparkSession): Boolean - - protected def shouldRun( - spark: SparkSession, - snapshot: Snapshot - ): Boolean = { - if (!spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_UPDATE_CATALOG_ENABLED)) { - return false - } - // Do not execute for path based tables, because they don't exist in the MetaStore - if (isPathBasedDeltaTable(table, spark)) return false - // Only execute if this is a Delta table - if (snapshot.version < 0) return false - true - } - - private def isPathBasedDeltaTable(table: CatalogTable, spark: SparkSession): Boolean = { - return DeltaTableIdentifier.isDeltaPath(spark, table.identifier) - } - - - /** Update the entry in the Catalog to reflect the latest schema and table properties. */ - protected def execute( - spark: SparkSession, - snapshot: Snapshot): Unit = { - recordDeltaOperation(snapshot.deltaLog, "delta.catalog.update") { - val properties = snapshot.getProperties.toMap - val v = table.properties.get(DeltaConfigs.METASTORE_LAST_UPDATE_VERSION) - .flatMap(v => Try(v.toLong).toOption) - .getOrElse(-1L) - val lastCommitTimestamp = table.properties.get(DeltaConfigs.METASTORE_LAST_COMMIT_TIMESTAMP) - .flatMap(v => Try(v.toLong).toOption) - .getOrElse(-1L) - // If the metastore entry is at an older version and not the timestamp of that version, e.g. - // a table can be rm -rf'd and get the same version number with a different timestamp - if (v <= snapshot.version || lastCommitTimestamp < snapshot.timestamp) { - try { - val loggingData = Map( - "identifier" -> table.identifier, - "snapshotVersion" -> snapshot.version, - "snapshotTimestamp" -> snapshot.timestamp, - "catalogVersion" -> v, - "catalogTimestamp" -> lastCommitTimestamp - ) - if (schemaHasChanged(snapshot, spark)) { - updateSchema(spark, snapshot) - recordDeltaEvent( - snapshot.deltaLog, - "delta.catalog.update.schema", - data = loggingData - ) - } else if (propertiesHaveChanged(properties, snapshot.metadata, spark)) { - updateProperties(spark, snapshot) - recordDeltaEvent( - snapshot.deltaLog, - "delta.catalog.update.properties", - data = loggingData - ) - } - } catch { - case NonFatal(e) => - recordDeltaEvent( - snapshot.deltaLog, - "delta.catalog.update.error", - data = Map( - "exceptionMsg" -> ExceptionUtils.getMessage(e), - "stackTrace" -> ExceptionUtils.getStackTrace(e)) - ) - logWarning(s"Failed to update the catalog for ${table.identifier} with the latest " + - s"table information.", e) - } - } - } - } -} - -/** - * A post-commit hook that allows us to cache the most recent schema and table properties of a Delta - * table in an External Catalog. In addition to the schema and table properties, we also store the - * last commit timestamp and version for which we updated the catalog. This prevents us from - * updating the MetaStore with potentially stale information. - */ -case class UpdateCatalog(table: CatalogTable) extends UpdateCatalogBase { - - override val name: String = "Update Catalog" - - override def executeOnWrite( - spark: SparkSession, - snapshot: Snapshot - ): Unit = { - executeAsync(spark, snapshot) - } - - - override protected def schemaHasChanged(snapshot: Snapshot, spark: SparkSession): Boolean = { - // We need to check whether the schema in the catalog matches the current schema. If a - // field in the schema is very long, we cannot store the schema in the catalog, therefore - // here we have to compare what's in the catalog with what we actually can store in the - // catalog - val schemaChanged = UpdateCatalog.truncateSchemaIfNecessary(snapshot.schema) != table.schema - // The table may have been dropped as we're just about to update the information. There is - // unfortunately no great way to avoid a race condition, but we do one last check here as - // updates may have been queued for some time. - schemaChanged && spark.sessionState.catalog.tableExists(table.identifier) - } - - /** - * Checks if the table properties have changed in the Snapshot with respect to what's stored in - * the catalog. We check to see if our table properties are a subset of what is in the MetaStore - * to avoid flip-flopping the information between older and newer versions of Delta. The - * assumption here is that newer Delta releases will only add newer table properties and not - * remove them. - */ - override protected[sql] def propertiesHaveChanged( - properties: Map[String, String], - metadata: Metadata, - spark: SparkSession): Boolean = { - val propertiesChanged = !properties.forall { case (k, v) => - table.properties.get(k) == Some(v) - } - // The table may have been dropped as we're just about to update the information. There is - // unfortunately no great way to avoid a race condition, but we do one last check here as - // updates may have been queued for some time. - propertiesChanged && spark.sessionState.catalog.tableExists(table.identifier) - } - - override def updateSchema(spark: SparkSession, snapshot: Snapshot): Unit = { - UpdateCatalog.replaceTable(spark, snapshot, table) - } - - override protected def updateProperties(spark: SparkSession, snapshot: Snapshot): Unit = { - spark.sessionState.catalog.alterTable( - table.copy(properties = UpdateCatalog.updatedProperties(snapshot))) - } - - /** - * Update the entry in the Catalog to reflect the latest schema and table properties - * asynchronously. - */ - private def executeAsync( - spark: SparkSession, - snapshot: Snapshot): Unit = { - if (!shouldRun(spark, snapshot)) return - Future[Unit] { - UpdateCatalog.activeAsyncRequests.incrementAndGet() - execute(spark, snapshot) - }(UpdateCatalog.getOrCreateExecutionContext(spark.sessionState.conf)).onComplete { _ => - UpdateCatalog.activeAsyncRequests.decrementAndGet() - }(UpdateCatalog.getOrCreateExecutionContext(spark.sessionState.conf)) - } -} - -object UpdateCatalog { - private var tp: ExecutionContext = _ - - // This is the encoding of the database for the Hive MetaStore - private val latin1 = Charset.forName("ISO-8859-1") - - // Maximum number of characters that a catalog can store. - val MAX_CATALOG_TYPE_DDL_LENGTH = 4000 - val ERROR_KEY = "delta.catalogUpdateError" - val LONG_SCHEMA_ERROR: String = "The schema contains a very long nested field and cannot be " + - "stored in the catalog." - val HIVE_METASTORE_NAME = "hive_metastore" - - private def getOrCreateExecutionContext(conf: SQLConf): ExecutionContext = synchronized { - if (tp == null) { - tp = ExecutionContext.fromExecutorService(DeltaThreadPool.newDaemonCachedThreadPool( - "delta-catalog-update", - conf.getConf(DeltaSQLConf.DELTA_UPDATE_CATALOG_THREAD_POOL_SIZE) - ) - ) - } - tp - } - - /** Keeps track of active or queued async requests. */ - private val activeAsyncRequests = new AtomicInteger(0) - - /** - * Waits for all active and queued updates to finish until the given timeout. Will return true - * if all async threads have completed execution. Will return false if not. Exposed for tests. - */ - def awaitCompletion(timeoutMillis: Long): Boolean = { - try { - ThreadUtils.runInNewThread("UpdateCatalog-awaitCompletion") { - val startTime = System.currentTimeMillis() - while (activeAsyncRequests.get() > 0) { - Thread.sleep(100) - val currentTime = System.currentTimeMillis() - if (currentTime - startTime > timeoutMillis) { - throw new TimeoutException( - s"Timed out waiting for catalog updates to complete after $currentTime ms") - } - } - } - true - } catch { - case _: TimeoutException => - false - } - } - - /** Replace the table definition in the MetaStore. */ - private def replaceTable(spark: SparkSession, snapshot: Snapshot, table: CatalogTable): Unit = { - val catalog = spark.sessionState.catalog - val qualifiedIdentifier = - catalog.qualifyIdentifier(TableIdentifier(table.identifier.table, Some(table.database))) - val db = qualifiedIdentifier.database.get - val tblName = qualifiedIdentifier.table - val schema = truncateSchemaIfNecessary(snapshot.schema) - val additionalProperties = if (schema.isEmpty) { - Map(ERROR_KEY -> LONG_SCHEMA_ERROR) - } else { - Map.empty - } - - // We call the lower level API so that we can actually drop columns. We also assume that - // all columns are data columns so that we don't have to deal with partition columns - // having to be at the end of the schema, which Hive follows. - val catalogName = table.identifier.catalog.getOrElse( - spark.sessionState.catalogManager.currentCatalog.name()) - if ( - (catalogName == UpdateCatalog.HIVE_METASTORE_NAME - || catalogName == SESSION_CATALOG_NAME) && - catalog.externalCatalog.tableExists(db, tblName)) { - catalog.externalCatalog.alterTableDataSchema(db, tblName, schema) - } - - // We have to update the properties anyway with the latest version/timestamp information - catalog.alterTable(table.copy(properties = updatedProperties(snapshot) ++ additionalProperties)) - } - - /** Updates our properties map with the version and timestamp information of the snapshot. */ - def updatedProperties(snapshot: Snapshot): Map[String, String] = { - var newProperties = - snapshot.getProperties.toMap ++ Map( - DeltaConfigs.METASTORE_LAST_UPDATE_VERSION -> snapshot.version.toString, - DeltaConfigs.METASTORE_LAST_COMMIT_TIMESTAMP -> snapshot.timestamp.toString) - newProperties - } - - /** - * If a field in the schema has a very long string representation, then the schema will be - * truncated to an empty schema to avoid corruption. - * Also, if the schema contains non-latin encoding characters, the schema will be garbled. In - * this case we also truncate the schema. - */ - def truncateSchemaIfNecessary(schema: StructType): StructType = { - // Encoders are not threadsafe - val encoder = latin1.newEncoder() - def isColumnValid(f: StructField): Boolean = { - val typeString = f.dataType.catalogString - encoder.canEncode(f.name) && - typeString.length <= MAX_CATALOG_TYPE_DDL_LENGTH && - encoder.canEncode(typeString) - } - - if (schema.exists(f => !isColumnValid(f))) { - new StructType() - } else { - schema - } - } -} diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/schema/ImplicitMetadataOperation.scala b/spark/src/main/scala/org/apache/spark/sql/delta/schema/ImplicitMetadataOperation.scala index 9b108f9715e..dfd238e40c9 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/schema/ImplicitMetadataOperation.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/schema/ImplicitMetadataOperation.scala @@ -16,8 +16,6 @@ package org.apache.spark.sql.delta.schema -import org.apache.spark.sql.delta.skipping.clustering.ClusteredTableUtils -import org.apache.spark.sql.delta.skipping.clustering.temp.ClusterBySpec import org.apache.spark.sql.delta._ import org.apache.spark.sql.delta.actions.DomainMetadata import org.apache.spark.sql.delta.actions.Metadata @@ -156,11 +154,10 @@ trait ImplicitMetadataOperation extends DeltaLogging { protected final def getNewDomainMetadata( txn: OptimisticTransaction, canUpdateMetadata: Boolean, - isReplacingTable: Boolean, - clusterBySpecOpt: Option[ClusterBySpec] = None): Seq[DomainMetadata] = { + isReplacingTable: Boolean + ): Seq[DomainMetadata] = { if (canUpdateMetadata && (!txn.deltaLog.tableExists || isReplacingTable)) { - val newDomainMetadata = Seq.empty[DomainMetadata] ++ - ClusteredTableUtils.getDomainMetadataOptional(clusterBySpecOpt, txn) + val newDomainMetadata = Seq.empty[DomainMetadata] if (!txn.deltaLog.tableExists) { newDomainMetadata } else { diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/skipping/clustering/ClusteredTableUtils.scala b/spark/src/main/scala/org/apache/spark/sql/delta/skipping/clustering/ClusteredTableUtils.scala index a8ddce543cc..0b503d34e4a 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/skipping/clustering/ClusteredTableUtils.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/skipping/clustering/ClusteredTableUtils.scala @@ -16,20 +16,12 @@ package org.apache.spark.sql.delta.skipping.clustering -import org.apache.spark.sql.delta.skipping.clustering.temp.ClusterBySpec -import org.apache.spark.sql.delta.{ClusteringTableFeature, DeltaColumnMappingMode, DeltaErrors, DeltaLog, OptimisticTransaction, Snapshot} -import org.apache.spark.sql.delta.actions.{DomainMetadata, Metadata, Protocol, TableFeatureProtocolUtils} -import org.apache.spark.sql.delta.clustering.ClusteringMetadataDomain +import org.apache.spark.sql.delta.ClusteringTableFeature +import org.apache.spark.sql.delta.actions.Protocol import org.apache.spark.sql.delta.metering.DeltaLogging -import org.apache.spark.sql.delta.schema.SchemaUtils import org.apache.spark.sql.delta.sources.DeltaSQLConf -import org.apache.spark.sql.delta.stats.{DeltaStatistics, StatisticsCollection} -import org.apache.spark.sql.delta.util.{Utils => DeltaUtils} -import org.apache.spark.sql.{AnalysisException, SparkSession} -import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{StructField, StructType} /** * Clustered table utility functions. @@ -45,289 +37,15 @@ trait ClusteredTableUtilsBase extends DeltaLogging { */ def isSupported(protocol: Protocol): Boolean = protocol.isFeatureSupported(ClusteringTableFeature) - /** The clustering implementation name for [[AddFile.clusteringProvider]] */ - def clusteringProvider: String = "liquid" - - /** - * Validate the clustering table preview is enabled. If not, throw an exception. - * This version is used when checking existing tables with updated metadata / protocol. - */ - def validatePreviewEnabled(protocol: Protocol): Unit = { - if (isSupported(protocol) && - !SQLConf.get.getConf(DeltaSQLConf.DELTA_CLUSTERING_TABLE_PREVIEW_ENABLED) && - !DeltaUtils.isTesting) { - throw DeltaErrors.clusteringTablePreviewDisabledException() - } - } - - /** - * Validate the clustering table preview is enabled. If not, throw an exception. - * This version is used for `CREATE TABLE...` where the initial snapshot doesn't have - * updated metadata / protocol yet. - */ - def validatePreviewEnabled(maybeClusterBySpec: Option[ClusterBySpec]): Unit = { - maybeClusterBySpec.foreach { _ => - if (!SQLConf.get.getConf(DeltaSQLConf.DELTA_CLUSTERING_TABLE_PREVIEW_ENABLED) && - !DeltaUtils.isTesting) { - throw DeltaErrors.clusteringTablePreviewDisabledException() - } - } - } - - /** - * Returns an optional [[ClusterBySpec]] from the given CatalogTable. - */ - def getClusterBySpecOptional(table: CatalogTable): Option[ClusterBySpec] = { - table.properties.get(PROP_CLUSTERING_COLUMNS).map(ClusterBySpec.fromProperty) - } - - /** - * Extract clustering columns from ClusterBySpec. - * - * @param maybeClusterBySpec optional ClusterBySpec. If it's empty, will return the - * original properties. - * @return an optional pair with clustering columns. - */ - def getClusteringColumnsAsProperty( - maybeClusterBySpec: Option[ClusterBySpec]): Option[(String, String)] = { - maybeClusterBySpec.map(ClusterBySpec.toProperty) - } - - /** - * Returns table feature properties that's required to create a clustered table. - * - * @param existingProperties Table properties set by the user when creating a clustered table. - */ - def getTableFeatureProperties(existingProperties: Map[String, String]): Map[String, String] = { - val properties = collection.mutable.Map.empty[String, String] - properties += TableFeatureProtocolUtils.propertyKey(ClusteringTableFeature) -> - TableFeatureProtocolUtils.FEATURE_PROP_SUPPORTED - - properties.toMap - } - - /** - * Validate the number of clustering columns doesn't exceed the limit. - * - * @param clusteringColumns clustering columns for the table. - * @param deltaLogOpt optional delta log. If present, will be used to record a delta event. - */ - def validateNumClusteringColumns( - clusteringColumns: Seq[Seq[String]], - deltaLogOpt: Option[DeltaLog] = None): Unit = { - val numColumnsLimit = - SQLConf.get.getConf(DeltaSQLConf.DELTA_NUM_CLUSTERING_COLUMNS_LIMIT) - val actualNumColumns = clusteringColumns.size - if (actualNumColumns > numColumnsLimit) { - deltaLogOpt.foreach { deltaLog => - recordDeltaEvent( - deltaLog, - opType = "delta.clusteredTable.invalidNumClusteringColumns", - data = Map( - "numCols" -> clusteringColumns.size, - "numColsLimit" -> numColumnsLimit)) - } - throw DeltaErrors.clusterByInvalidNumColumnsException(numColumnsLimit, actualNumColumns) - } - } - - /** - * Remove PROP_CLUSTERING_COLUMNS from metadata action. - * Clustering columns should only exist in: - * 1. CatalogTable.properties(PROP_CLUSTERING_COLUMNS) - * 2. Clustering metadata domain. - * @param configuration original configuration. - * @return new configuration without clustering columns property - */ - def removeClusteringColumnsProperty(configuration: Map[String, String]): Map[String, String] = { - configuration - PROP_CLUSTERING_COLUMNS - } - - /** - * Create an optional [[DomainMetadata]] action to store clustering columns. - */ - def getDomainMetadataOptional( - clusterBySpecOpt: Option[ClusterBySpec], - txn: OptimisticTransaction): Option[DomainMetadata] = { - clusterBySpecOpt.map { clusterBy => - ClusteredTableUtils.validateClusteringColumnsInStatsSchema( - txn.protocol, txn.metadata, clusterBy) - val clusteringColumns = - clusterBy.columnNames.map(_.toString).map(ClusteringColumn(txn.metadata.schema, _)) - createDomainMetadata(clusteringColumns) - } - } - - /** - * Create a [[DomainMetadata]] action to store clustering columns. - */ - def createDomainMetadata(clusteringColumns: Seq[ClusteringColumn]): DomainMetadata = { - ClusteringMetadataDomain.fromClusteringColumns(clusteringColumns).toDomainMetadata - } - - /** - * Create a [[ClusteringMetadataDomain]] with the given CatalogTable's clustering column property. - */ - def getDomainMetadataOptional( - table: CatalogTable, - txn: OptimisticTransaction): Option[DomainMetadata] = { - getDomainMetadataOptional(getClusterBySpecOptional(table), txn) - } - - /** - * Extract [[ClusteringColumn]]s from a given snapshot. Return None if the clustering domain - * metadata is missing. - */ - def getClusteringColumnsOptional(snapshot: Snapshot): Option[Seq[ClusteringColumn]] = { - ClusteringMetadataDomain - .fromSnapshot(snapshot) - .map(_.clusteringColumns.map(ClusteringColumn.apply)) - } - - /** - * Extract [[DomainMetadata]] for storing clustering columns from a given snapshot. - * It returns clustering domain metadata if exists. - * Return empty if the clustering domain metadata is missing. - */ - def getClusteringDomainMetadata(snapshot: Snapshot): Seq[DomainMetadata] = { - ClusteringMetadataDomain.fromSnapshot(snapshot).map(_.toDomainMetadata).toSeq - } - - /** - * Validate stats will be collected for all clustering columns. - */ - def validateClusteringColumnsInStatsSchema( - snapshot: Snapshot, - logicalClusteringColumns: Seq[String]): Unit = { - validateClusteringColumnsInStatsSchema( - snapshot, - logicalClusteringColumns.map { name => - ClusteringColumnInfo(snapshot.schema, ClusteringColumn(snapshot.schema, name)) - }) - } - - /** - * Returns true if stats will be collected for all clustering columns. - */ - def areClusteringColumnsInStatsSchema( - snapshot: Snapshot, - logicalClusteringColumns: Seq[String]): Boolean = { - getClusteringColumnsNotInStatsSchema( - snapshot, - logicalClusteringColumns.map { name => - ClusteringColumnInfo(snapshot.schema, ClusteringColumn(snapshot.schema, name)) - }).isEmpty - } - - /** - * Validate stats will be collected for all clustering columns. - * - * This version is used when [[Snapshot]] doesn't have latest stats column information such as - * `CREATE TABLE...` where the initial snapshot doesn't have updated metadata / protocol yet. - */ - def validateClusteringColumnsInStatsSchema( - protocol: Protocol, - metadata: Metadata, - clusterBy: ClusterBySpec): Unit = { - validateClusteringColumnsInStatsSchema( - statisticsCollectionFromMetadata(protocol, metadata), - clusterBy.columnNames.map { column => - ClusteringColumnInfo(metadata.schema, ClusteringColumn(metadata.schema, column.toString)) - }) - } - - /** - * Build a [[StatisticsCollection]] with minimal requirements that can be used to find stats - * columns. + /** Returns true to enable clustering table and currently it is only enabled for testing. * - * We can not use [[Snapshot]] as in a normal case during table creation such as `CREATE TABLE` - * because the initial snapshot doesn't have the updated metadata / protocol to find latest stats - * columns. + * Note this function is going to be removed when clustering table is fully developed. */ - private def statisticsCollectionFromMetadata( - p: Protocol, - metadata: Metadata): StatisticsCollection = { - new StatisticsCollection { - override val tableSchema: StructType = metadata.schema - override val outputAttributeSchema: StructType = tableSchema - // [[outputTableStatsSchema]] is the candidate schema to find statistics columns. - override val outputTableStatsSchema: StructType = tableSchema - override val statsColumnSpec = StatisticsCollection.configuredDeltaStatsColumnSpec(metadata) - override val columnMappingMode: DeltaColumnMappingMode = metadata.columnMappingMode - override val protocol: Protocol = p + def clusteringTableFeatureEnabled: Boolean = + SQLConf.get.getConf(DeltaSQLConf.DELTA_ENABLE_CLUSTERING_TABLE_FEATURE) - override def spark: SparkSession = { - throw new Exception("Method not used in statisticsCollectionFromMetadata") - } - } - } - - /** - * Validate physical clustering columns can be found in the latest stats columns. - * - * @param statsCollection Provides latest stats columns. - * @param clusteringColumnInfos Clustering columns in physical names. - * - * A [[AnalysisException]] is thrown if the clustering column can not be found in the latest - * stats columns. The error message contains logical names only for better user experience. - */ - private def validateClusteringColumnsInStatsSchema( - statsCollection: StatisticsCollection, - clusteringColumnInfos: Seq[ClusteringColumnInfo]): Unit = { - val missingColumn = getClusteringColumnsNotInStatsSchema(statsCollection, clusteringColumnInfos) - if (missingColumn.nonEmpty) { - // Convert back to logical names. - throw DeltaErrors.clusteringColumnMissingStats( - missingColumn.mkString(", "), - statsCollection.statCollectionLogicalSchema.treeString) - } - } - - /** - * Validate that the given clusterBySpec matches the existing table's in the given snapshot. - * This is used for append mode and replaceWhere. - */ - def validateClusteringColumnsInSnapshot( - snapshot: Snapshot, - clusterBySpec: ClusterBySpec): Unit = { - // This uses physical column names to compare. - val providedClusteringColumns = - Some(clusterBySpec.columnNames.map(col => ClusteringColumn(snapshot.schema, col.toString))) - val existingClusteringColumns = ClusteredTableUtils.getClusteringColumnsOptional(snapshot) - if (providedClusteringColumns != existingClusteringColumns) { - throw DeltaErrors.clusteringColumnsMismatchException( - clusterBySpec.columnNames.map(_.toString).mkString(","), - existingClusteringColumns.map(_.map( - ClusteringColumnInfo(snapshot.schema, _).logicalName).mkString(",")).getOrElse("") - ) - } - } - - /** - * Returns empty if all physical clustering columns can be found in the latest stats columns. - * Otherwise, returns the logical names of the all clustering columns that are not found. - * - * [[StatisticsCollection.statsSchema]] has converted field's name to physical name and also it - * filters out any columns that are NOT qualified as a stats data type - * through [[SkippingEligibleDataType]]. - * - * @param statsCollection Provides latest stats columns. - * @param clusteringColumnInfos Clustering columns in physical names. - */ - private def getClusteringColumnsNotInStatsSchema( - statsCollection: StatisticsCollection, - clusteringColumnInfos: Seq[ClusteringColumnInfo]): Seq[String] = { - clusteringColumnInfos.flatMap { info => - val path = DeltaStatistics.MIN +: info.physicalName - SchemaUtils.findNestedFieldIgnoreCase(statsCollection.statsSchema, path) match { - // Validate that the column exists in the stats schema and is not a struct - // in the stats schema (to catch CLUSTER BY an entire struct). - case None | Some(StructField(_, _: StructType, _, _)) => - Some(info.logicalName) - case _ => None - } - } - } + /** The clustering implementation name for [[AddFile.clusteringProvider]] */ + def clusteringProvider: String = "liquid" } object ClusteredTableUtils extends ClusteredTableUtilsBase diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/skipping/clustering/ClusteringColumn.scala b/spark/src/main/scala/org/apache/spark/sql/delta/skipping/clustering/ClusteringColumn.scala deleted file mode 100644 index 7faa3533311..00000000000 --- a/spark/src/main/scala/org/apache/spark/sql/delta/skipping/clustering/ClusteringColumn.scala +++ /dev/null @@ -1,95 +0,0 @@ -/* - * Copyright (2021) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.delta.skipping.clustering - -import org.apache.spark.sql.delta.{DeltaColumnMapping, Snapshot} -import org.apache.spark.sql.delta.metering.DeltaLogging -import org.apache.spark.sql.delta.schema.SchemaUtils - -import org.apache.spark.sql.connector.expressions.FieldReference -import org.apache.spark.sql.types.{DataType, StructType} - -/** - * A wrapper class that stores a clustering column's physical name parts. - */ -case class ClusteringColumn(physicalName: Seq[String]) - -object ClusteringColumn { - /** - * Note: `logicalName` must be validated to exist in the given `schema`. - */ - def apply(schema: StructType, logicalName: String): ClusteringColumn = { - val resolver = SchemaUtils.DELTA_COL_RESOLVER - // Note that we use AttributeNameParser instead of CatalystSqlParser to account for the case - // where the column name is a backquoted string with spaces. - val logicalNameParts = FieldReference(logicalName).fieldNames - val physicalNameParts = logicalNameParts.foldLeft[(DataType, Seq[String])]((schema, Nil)) { - (partial, namePart) => - val (currStructType, currPhysicalNameSeq) = partial - val field = - currStructType.asInstanceOf[StructType].find(field => resolver(field.name, namePart)).get - (field.dataType, currPhysicalNameSeq :+ DeltaColumnMapping.getPhysicalName(field)) - }._2 - ClusteringColumn(physicalNameParts) - } -} - -/** - * A wrapper class that stores a clustering column's physical name parts and data type. - */ -case class ClusteringColumnInfo( - physicalName: Seq[String], dataType: DataType, schema: StructType) { - lazy val logicalName: String = { - val reversePhysicalNameParts = physicalName.reverse - val resolver = SchemaUtils.DELTA_COL_RESOLVER - val logicalNameParts = - reversePhysicalNameParts - .foldRight[(Seq[String], DataType)]((Nil, schema)) { - (namePart, state) => - val (logicalNameParts, parentRawDataType) = state - val parentDataType = parentRawDataType.asInstanceOf[StructType] - val nextField = - parentDataType - .find(field => resolver(DeltaColumnMapping.getPhysicalName(field), namePart)) - .get - (nextField.name +: logicalNameParts, nextField.dataType) - }._1.reverse - FieldReference(logicalNameParts).toString - } -} - -object ClusteringColumnInfo extends DeltaLogging { - def apply(schema: StructType, clusteringColumn: ClusteringColumn): ClusteringColumnInfo = - apply(schema, clusteringColumn.physicalName) - - def apply(schema: StructType, physicalName: Seq[String]): ClusteringColumnInfo = { - val resolver = SchemaUtils.DELTA_COL_RESOLVER - val dataType = physicalName.foldLeft[DataType](schema) { - (currStructType, namePart) => - currStructType.asInstanceOf[StructType].find { field => - resolver(DeltaColumnMapping.getPhysicalName(field), namePart) - }.get.dataType - } - ClusteringColumnInfo(physicalName, dataType, schema) - } - - def extractLogicalNames(snapshot: Snapshot): Seq[String] = { - ClusteredTableUtils.getClusteringColumnsOptional(snapshot).map { clusteringColumns => - clusteringColumns.map(ClusteringColumnInfo(snapshot.schema, _).logicalName) - }.getOrElse(Seq.empty) - } -} diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/skipping/clustering/temp/ClusterBySpec.scala b/spark/src/main/scala/org/apache/spark/sql/delta/skipping/clustering/temp/ClusterBySpec.scala index 72510c514bb..69eb54be8d3 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/skipping/clustering/temp/ClusterBySpec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/skipping/clustering/temp/ClusterBySpec.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.delta.skipping.clustering.temp import scala.reflect.ClassTag -import org.apache.spark.sql.delta.skipping.clustering.ClusteredTableUtils import com.fasterxml.jackson.annotation.JsonInclude.Include import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper} import com.fasterxml.jackson.module.scala.{ClassTagExtensions, DefaultScalaModule} @@ -62,10 +61,6 @@ object ClusterBySpec { def fromProperty(columns: String): ClusterBySpec = { ClusterBySpec(mapper.readValue[Seq[Seq[String]]](columns).map(FieldReference(_))) } - - def toProperty(clusterBySpec: ClusterBySpec): (String, String) = { - ClusteredTableUtils.PROP_CLUSTERING_COLUMNS -> clusterBySpec.toJson - } } /** diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala b/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala index 93fb3cccfe1..cb1a4c74bc0 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSQLConf.scala @@ -181,7 +181,7 @@ trait DeltaSQLConfBase { .booleanConf .createWithDefault(true) - val DELTA_ALLOW_CREATE_EMPTY_SCHEMA_TABLE = + val DELTA_ALLOW_CREATE_EMPTY_SCHEMA_TABLE = buildConf("createEmptySchemaTable.enabled") .internal() .doc( @@ -196,130 +196,6 @@ trait DeltaSQLConfBase { .booleanConf .createWithDefault(true) - val AUTO_COMPACT_ALLOWED_VALUES = Seq( - "false", - "true" - ) - - val DELTA_AUTO_COMPACT_ENABLED = - buildConf("autoCompact.enabled") - .doc(s"""Whether to compact files after writes made into Delta tables from this session. This - | conf can be set to "true" to enable Auto Compaction, OR "false" to disable Auto Compaction - | on all writes across all delta tables in this session. - | """.stripMargin) - .stringConf - .transform(_.toLowerCase(Locale.ROOT)) - .checkValue(AUTO_COMPACT_ALLOWED_VALUES.contains(_), - """"spark.databricks.delta.autoCompact.enabled" must be one of: """ + - s"""${AUTO_COMPACT_ALLOWED_VALUES.mkString("(", ",", ")")}""") - .createOptional - - val DELTA_AUTO_COMPACT_RECORD_PARTITION_STATS_ENABLED = - buildConf("autoCompact.recordPartitionStats.enabled") - .internal() - .doc(s"""When enabled, each committed write delta transaction records the number of qualified - |files of each partition of the target table for Auto Compact in driver's - |memory.""".stripMargin) - .booleanConf - .createWithDefault(true) - - val DELTA_AUTO_COMPACT_EARLY_SKIP_PARTITION_TABLE_ENABLED = - buildConf("autoCompact.earlySkipPartitionTable.enabled") - .internal() - .doc(s"""Auto Compaction will be skipped if there is no partition with - |sufficient number of small files.""".stripMargin) - .booleanConf - .createWithDefault(true) - - val DELTA_AUTO_COMPACT_MAX_TABLE_PARTITION_STATS = - buildConf("autoCompact.maxTablePartitionStats") - .internal() - .doc( - s"""The maximum number of Auto Compaction partition statistics of each table. This controls - |the maximum number of partitions statistics each delta table can have. Increasing - |this value reduces the hash conflict and makes partitions statistics more accurate with - |the cost of more memory consumption. - |""".stripMargin) - .intConf - .checkValue(_ > 0, "The value of maxTablePartitionStats should be positive.") - .createWithDefault(16 * 1024) - - val DELTA_AUTO_COMPACT_PARTITION_STATS_SIZE = - buildConf("autoCompact.partitionStatsSize") - .internal() - .doc( - s"""The total number of partitions statistics entries can be kept in memory for all - |tables in each driver. If this threshold is reached, the partitions statistics of - |least recently accessed tables will be evicted out.""".stripMargin) - .intConf - .checkValue(_ > 0, "The value of partitionStatsSize should be positive.") - .createWithDefault(64 * 1024) - - val DELTA_AUTO_COMPACT_MAX_FILE_SIZE = - buildConf("autoCompact.maxFileSize") - .internal() - .doc(s"Target file size produced by auto compaction. The default value of this config" + - " is 128 MB.") - .longConf - .checkValue(_ >= 0, "maxFileSize has to be positive") - .createWithDefault(128 * 1024 * 1024) - - val DELTA_AUTO_COMPACT_MIN_NUM_FILES = - buildConf("autoCompact.minNumFiles") - .internal() - .doc("Number of small files that need to be in a directory before it can be optimized.") - .intConf - .checkValue(_ >= 0, "minNumFiles has to be positive") - .createWithDefault(50) - - val DELTA_AUTO_COMPACT_MIN_FILE_SIZE = - buildConf("autoCompact.minFileSize") - .internal() - .doc("Files which are smaller than this threshold (in bytes) will be grouped together and " + - "rewritten as larger files by the Auto Compaction. The default value of this config " + - s"is set to half of the config ${DELTA_AUTO_COMPACT_MAX_FILE_SIZE.key}") - .longConf - .checkValue(_ >= 0, "minFileSize has to be positive") - .createOptional - - val DELTA_AUTO_COMPACT_MODIFIED_PARTITIONS_ONLY_ENABLED = - buildConf("autoCompact.modifiedPartitionsOnly.enabled") - .internal() - .doc( - s"""When enabled, Auto Compaction only works on the modified partitions of the delta - |transaction that triggers compaction.""".stripMargin) - .booleanConf - .createWithDefault(true) - - val DELTA_AUTO_COMPACT_NON_BLIND_APPEND_ENABLED = - buildConf("autoCompact.nonBlindAppend.enabled") - .internal() - .doc( - s"""When enabled, Auto Compaction is only triggered by non-blind-append write - |transaction.""".stripMargin) - .booleanConf - .createWithDefault(false) - - val DELTA_AUTO_COMPACT_MAX_NUM_MODIFIED_PARTITIONS = - buildConf("autoCompact.maxNumModifiedPartitions") - .internal() - .doc( - s"""The maximum number of partition can be selected for Auto Compaction when - | Auto Compaction runs on modified partition is enabled.""".stripMargin) - .intConf - .checkValue(_ > 0, "The value of maxNumModifiedPartitions should be positive.") - .createWithDefault(128) - - val DELTA_AUTO_COMPACT_RESERVE_PARTITIONS_ENABLED = - buildConf("autoCompact.reservePartitions.enabled") - .internal() - .doc( - s"""When enabled, each Auto Compact thread reserves its target partitions and skips the - |partitions that are under Auto Compaction by another thread - |concurrently.""".stripMargin) - .booleanConf - .createWithDefault(true) - val DELTA_IMPORT_BATCH_SIZE_STATS_COLLECTION = buildConf("import.batchSize.statsCollection") .internal() @@ -481,22 +357,6 @@ trait DeltaSQLConfBase { .booleanConf .createWithDefault(true) - val DELTA_UPDATE_CATALOG_ENABLED = - buildConf("catalog.update.enabled") - .internal() - .doc("When enabled, we will cache the schema of the Delta table and the table properties " + - "in the external catalog, e.g. the Hive MetaStore.") - .booleanConf - .createWithDefault(false) - - val DELTA_UPDATE_CATALOG_THREAD_POOL_SIZE = - buildStaticConf("catalog.update.threadPoolSize") - .internal() - .doc("The size of the thread pool for updating the external catalog.") - .intConf - .checkValue(_ > 0, "threadPoolSize must be positive") - .createWithDefault(20) - val DELTA_ASSUMES_DROP_CONSTRAINT_IF_EXISTS = buildConf("constraints.assumesDropIfExists.enabled") .doc("""If true, DROP CONSTRAINT quietly drops nonexistent constraints even without @@ -1421,7 +1281,7 @@ trait DeltaSQLConfBase { .internal() .doc("Enable persistent Deletion Vectors in the Update command.") .booleanConf - .createWithDefault(true) + .createWithDefault(false) val DELETION_VECTOR_PACKING_TARGET_SIZE = buildConf("deletionVectors.packing.targetSize") @@ -1564,24 +1424,12 @@ trait DeltaSQLConfBase { // Clustered Table ////////////////// - val DELTA_CLUSTERING_TABLE_PREVIEW_ENABLED = - buildConf("clusteredTable.enableClusteringTablePreview") + val DELTA_ENABLE_CLUSTERING_TABLE_FEATURE = + buildConf("clusteringTable.enableClusteringTableFeature") .internal() - .doc("Whether to enable the clustering table preview.") - .booleanConf - .createWithDefault(false) - - val DELTA_NUM_CLUSTERING_COLUMNS_LIMIT = - buildStaticConf("clusteredTable.numClusteringColumnsLimit") - .internal() - .doc("""The maximum number of clustering columns allowed for a clustered table. - """.stripMargin) - .intConf - .checkValue( - _ > 0, - "'clusteredTable.numClusteringColumnsLimit' must be positive." - ) - .createWithDefault(4) + .doc("If true, enable ClusteringTableFeature when the table is a clustered table.") + .booleanConf + .createWithDefault(false) } object DeltaSQLConf extends DeltaSQLConfBase diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/stats/AutoCompactPartitionStats.scala b/spark/src/main/scala/org/apache/spark/sql/delta/stats/AutoCompactPartitionStats.scala deleted file mode 100644 index d9e24d89a45..00000000000 --- a/spark/src/main/scala/org/apache/spark/sql/delta/stats/AutoCompactPartitionStats.scala +++ /dev/null @@ -1,375 +0,0 @@ -/* - * Copyright (2021) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.delta.stats - -import scala.collection.mutable -import scala.util.control.NonFatal - -import org.apache.spark.sql.delta.actions.{Action, AddFile, FileAction, RemoveFile} -import org.apache.spark.sql.delta.hooks.AutoCompactPartitionReserve -import org.apache.spark.sql.delta.sources.DeltaSQLConf - -import org.apache.spark.sql.SparkSession - -/** - * A collector used to aggregate auto-compaction stats for a single commit. The expectation - * is to spin this up for a commit and then merging those local stats with the global stats. - */ -trait AutoCompactPartitionStatsCollector { - def collectPartitionStatsForAdd(file: AddFile): Unit - def collectPartitionStatsForRemove(file: RemoveFile): Unit - def finalizeStats(tableId: String): Unit -} - -/** - * This singleton object collect the table partition statistic for each commit that creates - * AddFile or RemoveFile objects. - * To control the memory usage, there are `maxNumTablePartitions` per table and 'maxNumPartitions' - * partition entries across all tables. - * Note: - * 1. Since the partition of each table is limited, if this limitation is reached, the least - * recently used table partitions will be evicted. - * 2. If all 'maxNumPartitions' are occupied, the partition stats of least recently used tables - * will be evicted until the used partitions fall back below to 'maxNumPartitions'. - * 3. The un-partitioned tables are treated as tables with single partition. - * @param maxNumTablePartitions The hash space of partition key to reduce memory usage per table. - * @param maxNumPartitions The maximum number of partition that can be occupied. - */ -class AutoCompactPartitionStats( - private var maxNumTablePartitions: Int, - private var maxNumPartitions: Int -) { - - /** - * This class to store the states of one table partition. These state includes: - * -- the number of small files, - * -- the thread that assigned to compact this partition, and - * -- whether the partition was compacted. - * - * Note: Since this class keeps tracking of the statistics of the table partition and the state of - * the auto compaction thread that works on the table partition, any method that accesses any - * attribute of this class needs to be protected by synchronized context. - */ - class PartitionStat( - var numFiles: Long, - var wasAutoCompacted: Boolean = false) { - - /** - * Determine whether this partition can be autocompacted based on the number of small files or - * if this [[AutoCompactPartitionStats]] instance has not auto compacted it yet. - * @param minNumFiles The minimum number of files this table-partition should have to trigger - * Auto Compaction in case it has already been compacted once. - */ - def hasSufficientSmallFilesOrHasNotBeenCompacted(minNumFiles: Long): Boolean = - !wasAutoCompacted || hasSufficientFiles(minNumFiles) - - def hasSufficientFiles(minNumFiles: Long): Boolean = numFiles >= minNumFiles - } - - /** - * This hashtable is used to store all table partition states of a table, the key is the hashcode - * of the partition, the value is [[PartitionStat]] object. - */ - type TablePartitionStats = mutable.LinkedHashMap[Int, PartitionStat] - - // The hash map to store the number of small files in each partition. - // -- Key is the hash code of the partition value. - // -- Values is the number of small files inside the corresponding partition. - type PartitionFilesMap = mutable.LinkedHashMap[Int, Long] - - type PartitionKey = Map[String, String] - - type PartitionKeySet = Set[Map[String, String]] - - // This is a simple LRU to store the table partition statistics. - // Workspace private to enable testing. - private[delta] val tablePartitionStatsCache = - new mutable.LinkedHashMap[String, TablePartitionStats]() - - // The number of partitions in this cache. - private[delta] var numUsedPartitions = 0 - - /** - * Helper class used to keep state regarding tracking auto-compaction stats of AddFile and - * RemoveFile actions in a single run that are greater than a passed-in minimum file size. - * If the collector runs into any non-fatal errors, it will invoke the error reporter on the error - * and then skip further execution. - * - * @param minFileSize Minimum file size for files we track auto-compact stats - * @param errorReporter Function that reports the first error, if any - * @return A collector object that tracks the Add/Remove file actions of the current commit. - */ - def createStatsCollector( - minFileSize: Long, - errorReporter: Throwable => Unit): - AutoCompactPartitionStatsCollector = new AutoCompactPartitionStatsCollector { - private val inputPartitionFiles = new PartitionFilesMap() - private var shouldCollect = true - - /** - * If the file is less than the specified min file size, updates the partition file map - * of stats with add or remove actions. If we encounter an error during stats collection, - * the remainder of the files will not be collected as well. - */ - private def collectPartitionStatsForFile(file: FileAction, addSub: Int): Unit = { - try { - val minSizeThreshold = minFileSize - if (shouldCollect && - file.estLogicalFileSize.getOrElse(file.getFileSize) <= minSizeThreshold - ) { - updatePartitionFileCounter(inputPartitionFiles, file.partitionValues, addSub) - } - } catch { - case NonFatal(e) => - errorReporter(e) - shouldCollect = false - } - } - /** - * Adds one file to all the appropriate partition counters. - */ - override def collectPartitionStatsForAdd(file: AddFile): Unit = { - collectPartitionStatsForFile(file, addSub = 1) - } - /** - * Removes one file from all the appropriate partition counters. - */ - override def collectPartitionStatsForRemove(file: RemoveFile): Unit = { - collectPartitionStatsForFile(file, addSub = -1) - } - - /** - * Merges the current collector's stats with the global one. - */ - override def finalizeStats(tableId: String): Unit = { - try { - if (shouldCollect) merge(tableId, inputPartitionFiles.filter(_._2 != 0)) - } catch { - case NonFatal(e) => errorReporter(e) - } - } - } - - /** - * This method merges the `inputPartitionFiles` of current committed transaction to the - * global cache of table partition stats. After merge is completed, tablePath will be moved - * to most recently used position. If the number of occupied partitions exceeds - * MAX_NUM_PARTITIONS, the least recently used tables will be evicted out. - * - * @param tableId The path of the table that contains `inputPartitionFiles`. - * @param inputPartitionFiles The number of files, which are qualified for Auto Compaction, in - * each partition. - */ - def merge(tableId: String, inputPartitionFiles: PartitionFilesMap): Unit = { - if (inputPartitionFiles.isEmpty) return - synchronized { - tablePartitionStatsCache.get(tableId) match { - case Some(cachedPartitionStates) => - // If the table is already stored, merges inputPartitionFiles' content to - // existing PartitionFilesMap. - for ((partitionHashCode, numFilesDelta) <- inputPartitionFiles) { - assert(numFilesDelta != 0) - cachedPartitionStates.get(partitionHashCode) match { - case Some(partitionState) => - // If there is an entry of partitionHashCode, updates its number of files - // and moves it to the most recently used slot. - partitionState.numFiles += numFilesDelta - moveAccessedPartitionToMru(cachedPartitionStates, partitionHashCode, partitionState) - case None => - if (numFilesDelta > 0) { - // New table partition is always in the most recently used entry. - cachedPartitionStates.put(partitionHashCode, new PartitionStat(numFilesDelta)) - numUsedPartitions += 1 - } - } - } - // Move the accessed table to MRU position and evicts the LRU partitions from it - // if necessary. - moveAccessedTableToMru(tableId, cachedPartitionStates) - case None => - // If it is new table, just create new entry. - val newPartitionStates = inputPartitionFiles - .filter { case (_, numFiles) => numFiles > 0 } - .map { case (partitionHashCode, numFiles) => - (partitionHashCode, new PartitionStat(numFiles)) - } - tablePartitionStatsCache.put(tableId, newPartitionStates) - numUsedPartitions += newPartitionStates.size - moveAccessedTableToMru(tableId, newPartitionStates) - } - evictLruTablesIfNecessary() - } - } - - /** Move the accessed table partition to the most recently used position. */ - private def moveAccessedPartitionToMru( - cachedPartitionFiles: TablePartitionStats, - partitionHashCode: Int, - partitionState: PartitionStat): Unit = { - cachedPartitionFiles.remove(partitionHashCode) - if (partitionState.numFiles <= 0) { - numUsedPartitions -= 1 - } else { - // If the newNumFiles is not empty, add it back and make it to be the - // most recently used entry. - cachedPartitionFiles.put(partitionHashCode, partitionState) - } - } - - /** Move the accessed table to the most recently used position. */ - private def moveAccessedTableToMru( - tableId: String, - cachedPartitionFiles: TablePartitionStats): Unit = { - // The tablePartitionStatsCache is insertion order preserved hash table. Thus, - // removing and adding back the entry make this to be most recently used entry. - // If cachedPartitionFiles's size is empty, no need to add it back to LRU. - tablePartitionStatsCache.remove(tableId) - numUsedPartitions -= cachedPartitionFiles.size - if (cachedPartitionFiles.nonEmpty) { - // Evict the least recently used partitions' statistics from table if necessary - val numExceededPartitions = cachedPartitionFiles.size - maxNumTablePartitions - if (numExceededPartitions > 0) { - val newPartitionStats = cachedPartitionFiles.drop(numExceededPartitions) - tablePartitionStatsCache.put(tableId, newPartitionStats) - numUsedPartitions += newPartitionStats.size - } else { - tablePartitionStatsCache.put(tableId, cachedPartitionFiles) - numUsedPartitions += cachedPartitionFiles.size - } - } - } - - /** - * Evicts the Lru tables from 'tablePartitionStatsCache' until the total number of partitions - * is less than maxNumPartitions. - */ - private def evictLruTablesIfNecessary(): Unit = { - // Keep removing the least recently used table until the used partition is lower than - // threshold. - while (numUsedPartitions > maxNumPartitions && tablePartitionStatsCache.nonEmpty) { - // Pick the least recently accessed table and remove it. - val (lruTable, tablePartitionStat) = tablePartitionStatsCache.head - numUsedPartitions -= tablePartitionStat.size - tablePartitionStatsCache.remove(lruTable) - } - } - - /** Update the file count of `PartitionFilesMap` according to the hash value of `partition`. */ - private def updatePartitionFileCounter( - partitionFileCounter: PartitionFilesMap, - partition: PartitionKey, - addSub: Int): Unit = { - partitionFileCounter.get(partition.##) match { - case Some(numFiles) => - partitionFileCounter.update(partition.##, numFiles + addSub) - case None => - partitionFileCounter.put(partition.##, addSub) - } - } - - /** Get the maximum number of files among all partitions inside table `tableId`. */ - def maxNumFilesInTable(tableId: String): Long = { - synchronized { - tablePartitionStatsCache.get(tableId) match { - case Some(partitionFileCounter) => - if (partitionFileCounter.isEmpty) { - 0 - } else { - partitionFileCounter.map(_._2.numFiles).max - } - case None => 0 - } - } - } - - /** - * @return Filter partitions from targetPartitions that have not been auto-compacted or - * that have enough small files. - */ - def filterPartitionsWithSmallFiles(tableId: String, targetPartitions: Set[PartitionKey], - minNumFiles: Long): Set[PartitionKey] = synchronized { - tablePartitionStatsCache.get(tableId).map { tablePartitionStates => - targetPartitions.filter { partitionKey => - tablePartitionStates.get(partitionKey.##).exists { partitionState => - partitionState.hasSufficientSmallFilesOrHasNotBeenCompacted(minNumFiles) - } - } - }.getOrElse(Set.empty) - } - - def markPartitionsAsCompacted(tableId: String, compactedPartitions: Set[PartitionKey]) - : Unit = synchronized { - tablePartitionStatsCache.get(tableId).foreach { tablePartitionStats => - compactedPartitions - .foreach(partitionKey => tablePartitionStats.get(partitionKey.##) - .foreach(_.wasAutoCompacted = true)) - } - } - - /** - * Collect the number of files, which are less than minFileSize, added to or removed from each - * partition from `actions`. - */ - def collectPartitionStats( - collector: AutoCompactPartitionStatsCollector, - tableId: String, - actions: Iterator[Action]): Unit = { - val acts = actions.toVector - acts.foreach { - case addFile: AddFile => collector.collectPartitionStatsForAdd(addFile) - case removeFile: RemoveFile => collector.collectPartitionStatsForRemove(removeFile) - case _ => // do nothing - } - collector.finalizeStats(tableId) - } - - /** This is test only code to reset the state of table partition statistics. */ - private[delta] def resetTestOnly(newHashSpace: Int, newMaxNumPartitions: Int): Unit = { - synchronized { - tablePartitionStatsCache.clear() - maxNumTablePartitions = newHashSpace - maxNumPartitions = newMaxNumPartitions - numUsedPartitions = 0 - AutoCompactPartitionReserve.resetTestOnly() - } - } - - /** - * This is test only code to reset all partition statistic information and keep current - * configuration. - */ - private[delta] def resetTestOnly(): Unit = resetTestOnly(maxNumTablePartitions, maxNumPartitions) -} - -object AutoCompactPartitionStats { - private var _instance: AutoCompactPartitionStats = null - - /** The thread safe constructor of singleton. */ - def instance(spark: SparkSession): AutoCompactPartitionStats = { - synchronized { - if (_instance == null) { - val config = spark.conf - val hashSpaceSize = config.get(DeltaSQLConf.DELTA_AUTO_COMPACT_MAX_TABLE_PARTITION_STATS) - val maxNumPartitions = config.get(DeltaSQLConf.DELTA_AUTO_COMPACT_PARTITION_STATS_SIZE) - _instance = new AutoCompactPartitionStats( - hashSpaceSize, maxNumPartitions - ) - } - } - _instance - } -} diff --git a/spark/src/main/scala/org/apache/spark/sql/delta/util/Utils.scala b/spark/src/main/scala/org/apache/spark/sql/delta/util/Utils.scala index df5649adb24..8e381937343 100644 --- a/spark/src/main/scala/org/apache/spark/sql/delta/util/Utils.scala +++ b/spark/src/main/scala/org/apache/spark/sql/delta/util/Utils.scala @@ -21,9 +21,6 @@ import scala.util.Random import org.apache.spark.sql.delta.DeltaConfigs import org.apache.spark.sql.delta.actions.Metadata -import org.apache.spark.sql.{Column, Dataset} -import org.apache.spark.sql.catalyst.expressions.ElementAt -import org.apache.spark.sql.functions.lit /** * Various utility methods used by Delta. @@ -59,13 +56,4 @@ object Utils { System.getenv("DELTA_TESTING") != null } - /** - * Returns value for the given key in value if column is a map and the key is present, NULL - * otherwise. - */ - def try_element_at(mapColumn: Column, key: Any): Column = { - Column { - ElementAt(mapColumn.expr, lit(key).expr, failOnError = false) - } - } } diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/AutoCompactSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/AutoCompactSuite.scala deleted file mode 100644 index 01467883054..00000000000 --- a/spark/src/test/scala/org/apache/spark/sql/delta/AutoCompactSuite.scala +++ /dev/null @@ -1,332 +0,0 @@ -/* - * Copyright (2021) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.delta - -import java.io.File - - -// scalastyle:off import.ordering.noEmptyLine -import com.databricks.spark.util.{Log4jUsageLogger, UsageRecord} -import org.apache.spark.sql.delta.actions.AddFile -import org.apache.spark.sql.delta.commands.optimize._ -import org.apache.spark.sql.delta.hooks.{AutoCompact, AutoCompactType} -import org.apache.spark.sql.delta.optimize.CompactionTestHelperForAutoCompaction -import org.apache.spark.sql.delta.sources.DeltaSQLConf -import org.apache.spark.sql.delta.stats.AutoCompactPartitionStats -import org.apache.spark.sql.delta.test.DeltaSQLCommandTest -import org.apache.spark.sql.delta.test.DeltaTestImplicits._ -import org.apache.spark.sql.delta.util.JsonUtils -import org.apache.hadoop.fs.Path - -import org.apache.spark.sql.Column -import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.expressions.Literal -import org.apache.spark.sql.test.SharedSparkSession -import org.apache.spark.sql.types.StringType -import org.apache.spark.unsafe.types.UTF8String - -trait AutoCompactTestUtils { - def captureOptimizeLogs(metrics: String)(f: => Unit): Seq[UsageRecord] = { - val usageLogs = Log4jUsageLogger.track(f) - usageLogs.filter { usageLog => - usageLog.tags.get("opType") == Some(metrics) - } - } - -} - - -/** - * This class extends the [[CompactionSuiteBase]] and runs all the [[CompactionSuiteBase]] tests - * with AutoCompaction. - * - * It also tests any AutoCompaction specific behavior. - */ -class AutoCompactSuite extends - CompactionTestHelperForAutoCompaction - with DeltaSQLCommandTest - with SharedSparkSession - with AutoCompactTestUtils { - - test("auto-compact-type: test table properties") { - withTempDir { tempDir => - val dir = tempDir.getCanonicalPath - spark.range(0, 1).write.format("delta").mode("append").save(dir) - val deltaLog = DeltaLog.forTable(spark, dir) - val defaultAutoCompactType = AutoCompact.getAutoCompactType(conf, deltaLog.snapshot.metadata) - Map( - "true" -> Some(AutoCompactType.Enabled), - "tRue" -> Some(AutoCompactType.Enabled), - "'true'" -> Some(AutoCompactType.Enabled), - "false" -> None, - "fALse" -> None, - "'false'" -> None - ).foreach { case (propertyValue, expectedAutoCompactType) => - setTableProperty(deltaLog, "delta.autoOptimize.autoCompact", propertyValue) - assert(AutoCompact.getAutoCompactType(conf, deltaLog.snapshot.metadata) == - expectedAutoCompactType) - } - } - } - - test("auto-compact-type: test confs") { - withTempDir { tempDir => - val dir = tempDir.getCanonicalPath - spark.range(0, 1).write.format("delta").mode("append").save(dir) - val deltaLog = DeltaLog.forTable(spark, dir) - val defaultAutoCompactType = AutoCompact.getAutoCompactType(conf, deltaLog.snapshot.metadata) - - Map( - "true" -> Some(AutoCompactType.Enabled), - "TrUE" -> Some(AutoCompactType.Enabled), - "false" -> None, - "FalsE" -> None - ).foreach { case (confValue, expectedAutoCompactType) => - withSQLConf(DeltaSQLConf.DELTA_AUTO_COMPACT_ENABLED.key -> confValue) { - assert(AutoCompact.getAutoCompactType(conf, deltaLog.snapshot.metadata) == - expectedAutoCompactType) - } - } - } - } - - private def testBothModesViaProperty(testName: String)(f: String => Unit): Unit = { - def runTest(autoCompactConfValue: String): Unit = { - withTempDir { dir => - withSQLConf( - "spark.databricks.delta.properties.defaults.autoOptimize.autoCompact" -> - s"$autoCompactConfValue", - DeltaSQLConf.DELTA_AUTO_COMPACT_MIN_NUM_FILES.key -> "0", - DeltaSQLConf.DELTA_AUTO_COMPACT_MODIFIED_PARTITIONS_ONLY_ENABLED.key -> "false") { - f(dir.getCanonicalPath) - } - } - } - - test(s"auto-compact-enabled-property: $testName") { runTest(autoCompactConfValue = "true") } - } - - private def testBothModesViaConf(testName: String)(f: String => Unit): Unit = { - def runTest(autoCompactConfValue: String): Unit = { - withTempDir { dir => - withSQLConf( - DeltaSQLConf.DELTA_AUTO_COMPACT_ENABLED.key -> s"$autoCompactConfValue", - DeltaSQLConf.DELTA_AUTO_COMPACT_MIN_NUM_FILES.key -> "0") { - f(dir.getCanonicalPath) - } - } - } - - test(s"auto-compact-enabled-conf: $testName") { runTest(autoCompactConfValue = "true") } - } - - private def checkAutoOptimizeLogging(f: => Unit): Boolean = { - val logs = Log4jUsageLogger.track { - f - } - logs.exists(_.opType.map(_.typeName) === Some("delta.commit.hooks.autoOptimize")) - } - - import testImplicits._ - - test("auto compact event log: inline AC") { - withTempDir { dir => - withSQLConf( - DeltaSQLConf.DELTA_AUTO_COMPACT_ENABLED.key -> s"true", - DeltaSQLConf.DELTA_AUTO_COMPACT_MIN_NUM_FILES.key -> "30") { - val path = dir.getCanonicalPath - // Append 1 file to each partition: record runOnModifiedPartitions event, as is first write - var usageLogs = captureOptimizeLogs(AutoCompact.OP_TYPE) { - createFilesToPartitions(numFilePartitions = 3, numFilesPerPartition = 1, path) - } - var log = JsonUtils.mapper.readValue[Map[String, String]](usageLogs.head.blob) - assert(log("status") == "runOnModifiedPartitions" && log("partitions") == "3") - // Append 10 more file to each partition: record skipInsufficientFilesInModifiedPartitions - // event. - usageLogs = captureOptimizeLogs(AutoCompact.OP_TYPE) { - createFilesToPartitions(numFilePartitions = 3, numFilesPerPartition = 10, path) - } - log = JsonUtils.mapper.readValue[Map[String, String]](usageLogs.head.blob) - assert(log("status") == "skipInsufficientFilesInModifiedPartitions") - // Append 20 more files to each partition: record runOnModifiedPartitions on all 3 - // partitions. - usageLogs = captureOptimizeLogs(AutoCompact.OP_TYPE) { - createFilesToPartitions(numFilePartitions = 3, numFilesPerPartition = 20, path) - } - log = JsonUtils.mapper.readValue[Map[String, String]](usageLogs.head.blob) - assert(log("status") == "runOnModifiedPartitions" && log("partitions") == "3") - // Append 30 more file to each partition and check OptimizeMetrics. - usageLogs = captureOptimizeLogs(metrics = s"${AutoCompact.OP_TYPE}.execute.metrics") { - createFilesToPartitions(numFilePartitions = 3, numFilesPerPartition = 30, path) - } - val metricsLog = JsonUtils.mapper.readValue[OptimizeMetrics](usageLogs.head.blob) - assert(metricsLog.numBytesSkippedToReduceWriteAmplification === 0) - assert(metricsLog.numFilesSkippedToReduceWriteAmplification === 0) - assert(metricsLog.totalConsideredFiles === 93) - assert(metricsLog.numFilesAdded == 3) - assert(metricsLog.numFilesRemoved == 93) - assert(metricsLog.numBatches === 3) - } - } - } - - private def checkAutoCompactionWorks(dir: String): Unit = { - spark.range(10).write.format("delta").mode("append").save(dir) - val deltaLog = DeltaLog.forTable(spark, dir) - val newSnapshot = deltaLog.update() - assert(newSnapshot.version === 1) // 0 is the first commit, 1 is optimize - assert(deltaLog.update().numOfFiles === 1) - - val isLogged = checkAutoOptimizeLogging { - spark.range(10).write.format("delta").mode("append").save(dir) - } - - assert(isLogged) - val lastEvent = deltaLog.history.getHistory(Some(1)).head - assert(lastEvent.operation === "OPTIMIZE") - assert(lastEvent.operationParameters("auto") === "true") - - assert(deltaLog.update().numOfFiles === 1, "Files should be optimized into a single one") - checkAnswer( - spark.range(10).union(spark.range(10)).toDF(), - spark.read.format("delta").load(dir) - ) - } - - testBothModesViaProperty("auto compact should kick in when enabled - table config") { dir => - checkAutoCompactionWorks(dir) - } - - testBothModesViaConf("auto compact should kick in when enabled - session config") { dir => - checkAutoCompactionWorks(dir) - } - - testBothModesViaProperty("auto compact should not kick in when session config is off") { dir => - withSQLConf(DeltaSQLConf.DELTA_AUTO_COMPACT_ENABLED.key -> "false") { - val isLogged = checkAutoOptimizeLogging { - spark.range(10).write.format("delta").mode("append").save(dir) - } - - val deltaLog = DeltaLog.forTable(spark, dir) - val newSnapshot = deltaLog.update() - assert(newSnapshot.version === 0) // 0 is the first commit - assert(deltaLog.update().numOfFiles > 1) - assert(!isLogged) - } - } - - test("auto compact should not kick in after optimize") { - withTempDir { tempDir => - val dir = tempDir.getCanonicalPath - spark.range(0, 12, 1, 4).write.format("delta").mode("append").save(dir) - val deltaLog = DeltaLog.forTable(spark, dir) - val newSnapshot = deltaLog.update() - assert(newSnapshot.version === 0) - assert(deltaLog.update().numOfFiles === 4) - spark.sql(s"ALTER TABLE delta.`${tempDir.getCanonicalPath}` SET TBLPROPERTIES " + - "(delta.autoOptimize.autoCompact = true)") - - val isLogged = checkAutoOptimizeLogging { - sql(s"optimize delta.`$dir`") - } - - assert(!isLogged) - val lastEvent = deltaLog.history.getHistory(Some(1)).head - assert(lastEvent.operation === "OPTIMIZE") - assert(lastEvent.operationParameters("auto") === "false") - } - } - - testBothModesViaProperty("auto compact should not kick in when there aren't " + - "enough files") { dir => - withSQLConf(DeltaSQLConf.DELTA_AUTO_COMPACT_MIN_NUM_FILES.key -> "5") { - AutoCompactPartitionStats.instance(spark).resetTestOnly() - spark.range(10).repartition(4).write.format("delta").mode("append").save(dir) - - val deltaLog = DeltaLog.forTable(spark, dir) - val newSnapshot = deltaLog.update() - assert(newSnapshot.version === 0) - assert(deltaLog.update().numOfFiles === 4) - - val isLogged2 = checkAutoOptimizeLogging { - spark.range(10).repartition(4).write.format("delta").mode("append").save(dir) - } - - assert(isLogged2) - val lastEvent = deltaLog.history.getHistory(Some(1)).head - assert(lastEvent.operation === "OPTIMIZE") - assert(lastEvent.operationParameters("auto") === "true") - - assert(deltaLog.update().numOfFiles === 1, "Files should be optimized into a single one") - - checkAnswer( - spark.read.format("delta").load(dir), - spark.range(10).union(spark.range(10)).toDF() - ) - } - } - - testBothModesViaProperty("ensure no NPE in auto compact UDF with null " + - "partition values") { dir => - Seq(null, "", " ").map(UTF8String.fromString).zipWithIndex.foreach { case (partValue, i) => - val path = new File(dir, i.toString).getCanonicalPath - val df1 = spark.range(5).withColumn("part", new Column(Literal(partValue, StringType))) - val df2 = spark.range(5, 10).withColumn("part", new Column(Literal("1"))) - val isLogged = checkAutoOptimizeLogging { - // repartition to increase number of files written - df1.union(df2).repartition(4) - .write.format("delta").partitionBy("part").mode("append").save(path) - } - val deltaLog = DeltaLog.forTable(spark, path) - val newSnapshot = deltaLog.update() - assert(newSnapshot.version === 1) // 0 is the first commit, 1 and 2 are optimizes - assert(newSnapshot.numOfFiles === 2) - - assert(isLogged) - val lastEvent = deltaLog.history.getHistory(Some(1)).head - assert(lastEvent.operation === "OPTIMIZE") - assert(lastEvent.operationParameters("auto") === "true") - } - } - - testBothModesViaProperty("check auto compact recorded metrics") { dir => - val logs = Log4jUsageLogger.track { - spark.range(30).repartition(3).write.format("delta").save(dir) - } - val metrics = JsonUtils.mapper.readValue[OptimizeMetrics](logs.filter( - _.tags.get("opType") == Some(s"${AutoCompact.OP_TYPE}.execute.metrics")).head.blob) - - assert(metrics.numFilesRemoved == 3) - assert(metrics.numFilesAdded == 1) - } - - private def setTableProperty(log: DeltaLog, key: String, value: String): Unit = { - spark.sql(s"ALTER TABLE delta.`${log.dataPath}` SET TBLPROPERTIES " + - s"($key = $value)") - } -} - -class AutoCompactIdColumnMappingSuite extends AutoCompactSuite - with DeltaColumnMappingEnableIdMode { - override def runAllTests: Boolean = true -} - -class AutoCompactNameColumnMappingSuite extends AutoCompactSuite - with DeltaColumnMappingEnableNameMode { - override def runAllTests: Boolean = true -} - diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaErrorsSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaErrorsSuite.scala index 67dc4c0a209..a7bd739ba7d 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaErrorsSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaErrorsSuite.scala @@ -32,7 +32,6 @@ import org.apache.spark.sql.delta.catalog.DeltaCatalog import org.apache.spark.sql.delta.constraints.CharVarcharConstraint import org.apache.spark.sql.delta.constraints.Constraints import org.apache.spark.sql.delta.constraints.Constraints.NotNull -import org.apache.spark.sql.delta.hooks.AutoCompactType import org.apache.spark.sql.delta.hooks.PostCommitHook import org.apache.spark.sql.delta.schema.{DeltaInvariantViolationException, InvariantViolationException, SchemaMergingUtils, SchemaUtils, UnsupportedDataTypeInfo} import org.apache.spark.sql.delta.sources.DeltaSQLConf @@ -445,12 +444,11 @@ trait DeltaErrorsSuiteBase Some("Column c is not specified in INSERT")) } { - val e = intercept[DeltaIllegalArgumentException] { - throw DeltaErrors.invalidAutoCompactType("invalid") + val e = intercept[DeltaAnalysisException] { + throw DeltaErrors.missingColumnsInInsertInto("c") } - val allowed = AutoCompactType.ALLOWED_VALUES.mkString("(", ",", ")") checkErrorMessage(e, None, None, - Some(s"Invalid auto-compact type: invalid. Allowed values are: $allowed.")) + Some("Column c is not specified in INSERT")) } { val table = DeltaTableIdentifier(Some("path")) diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaOptionSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaOptionSuite.scala index 580c86f6660..2d22707aefe 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaOptionSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaOptionSuite.scala @@ -29,7 +29,6 @@ import org.apache.commons.io.FileUtils import org.apache.parquet.format.CompressionCodec import org.apache.spark.sql.{AnalysisException, QueryTest} -import org.apache.spark.sql.functions.lit import org.apache.spark.sql.internal.SQLConf.PARTITION_OVERWRITE_MODE import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.util.Utils @@ -307,26 +306,6 @@ class DeltaOptionSuite extends QueryTest } } - test("overwriteSchema=true should be invalid with partitionOverwriteMode=dynamic, " + - "saveAsTable") { - withTable("temp") { - val e = intercept[DeltaIllegalArgumentException] { - withSQLConf(DeltaSQLConf.DYNAMIC_PARTITION_OVERWRITE_ENABLED.key -> "true") { - Seq(1, 2, 3).toDF - .withColumn("part", $"value" % 2) - .write - .mode("overwrite") - .format("delta") - .partitionBy("part") - .option(OVERWRITE_SCHEMA_OPTION, "true") - .option(PARTITION_OVERWRITE_MODE_OPTION, "dynamic") - .saveAsTable("temp") - } - } - assert(e.getErrorClass == "DELTA_OVERWRITE_SCHEMA_WITH_DYNAMIC_PARTITION_OVERWRITE") - } - } - test("Prohibit spark.databricks.delta.dynamicPartitionOverwrite.enabled=false in " + "dynamic partition overwrite mode") { withTempDir { tempDir => @@ -355,78 +334,4 @@ class DeltaOptionSuite extends QueryTest } } } - - for (createOrReplace <- Seq("CREATE OR REPLACE", "REPLACE")) { - test(s"$createOrReplace table command should not respect " + - "dynamic partition overwrite mode") { - withTempDir { tempDir => - Seq(0, 1).toDF - .withColumn("key", $"value" % 2) - .withColumn("stringColumn", lit("string")) - .withColumn("part", $"value" % 2) - .write - .format("delta") - .partitionBy("part") - .save(tempDir.getAbsolutePath) - withSQLConf(PARTITION_OVERWRITE_MODE.key -> "dynamic") { - // Write only to one partition with a different schema type of stringColumn. - sql( - s""" - |$createOrReplace TABLE delta.`${tempDir.getAbsolutePath}` - |USING delta - |PARTITIONED BY (part) - |LOCATION '${tempDir.getAbsolutePath}' - |AS SELECT -1 as value, 0 as part, 0 as stringColumn - |""".stripMargin) - assert(spark.read.format("delta").load(tempDir.getAbsolutePath).count() == 1, - "Table should be fully replaced even with DPO mode enabled") - } - } - } - } - - // Same test as above but using DeltaWriter V2. - test("create or replace table V2 should not respect dynamic partition overwrite mode") { - withTable("temp") { - Seq(0, 1).toDF - .withColumn("part", $"value" % 2) - .write - .format("delta") - .partitionBy("part") - .saveAsTable("temp") - withSQLConf(PARTITION_OVERWRITE_MODE.key -> "dynamic") { - // Write to one partition only. - Seq(0).toDF - .withColumn("part", $"value" % 2) - .writeTo("temp") - .using("delta") - .createOrReplace() - assert(spark.read.format("delta").table("temp").count() == 1, - "Table should be fully replaced even with DPO mode enabled") - } - } - } - - // Same test as above but using saveAsTable. - test("saveAsTable with overwrite should respect dynamic partition overwrite mode") { - withTable("temp") { - Seq(0, 1).toDF - .withColumn("part", $"value" % 2) - .write - .format("delta") - .partitionBy("part") - .saveAsTable("temp") - // Write to one partition only. - Seq(0).toDF - .withColumn("part", $"value" % 2) - .write - .mode("overwrite") - .option(PARTITION_OVERWRITE_MODE_OPTION, "dynamic") - .partitionBy("part") - .format("delta") - .saveAsTable("temp") - assert(spark.read.format("delta").table("temp").count() == 2, - "Table should keep the original partition with DPO mode enabled.") - } - } } diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaUpdateCatalogSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaUpdateCatalogSuite.scala deleted file mode 100644 index c4f9ab969bf..00000000000 --- a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaUpdateCatalogSuite.scala +++ /dev/null @@ -1,544 +0,0 @@ -/* - * Copyright (2021) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.delta - -import java.io.File - -import scala.util.control.NonFatal - -import com.databricks.spark.util.Log4jUsageLogger -import org.apache.spark.sql.delta.hooks.UpdateCatalog -import org.apache.spark.sql.delta.hooks.UpdateCatalog.MAX_CATALOG_TYPE_DDL_LENGTH -import org.apache.spark.sql.delta.sources.DeltaSQLConf -import org.apache.spark.sql.delta.test.DeltaHiveTest -import com.fasterxml.jackson.core.JsonParseException - -import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, SparkSession} -import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.catalog.ExternalCatalogWithListener -import org.apache.spark.sql.execution.streaming.MemoryStream -import org.apache.spark.sql.functions.{lit, struct} -import org.apache.spark.sql.hive.HiveExternalCatalog -import org.apache.spark.sql.types.{ArrayType, DoubleType, IntegerType, LongType, MapType, StringType, StructField, StructType} -import org.apache.spark.util.{ThreadUtils, Utils} - -class DeltaUpdateCatalogSuite - extends DeltaUpdateCatalogSuiteBase - with DeltaHiveTest { - - import testImplicits._ - - override def beforeAll(): Unit = { - super.beforeAll() - spark.conf.set(DeltaSQLConf.DELTA_UPDATE_CATALOG_ENABLED.key, "true") - } - - override def beforeEach(): Unit = { - super.beforeEach() - cleanupDefaultTable() - } - - override def afterEach(): Unit = { - if (!UpdateCatalog.awaitCompletion(10000)) { - logWarning(s"There are active catalog udpate requests after 10 seconds") - } - cleanupDefaultTable() - super.afterEach() - } - - /** Remove Hive specific table properties. */ - override protected def filterProperties(properties: Map[String, String]): Map[String, String] = { - properties.filterKeys(_ != "transient_lastDdlTime").toMap - } - - - test("streaming") { - withTable(tbl) { - implicit val _sqlContext = spark.sqlContext - val stream = MemoryStream[Long] - val df1 = stream.toDF().toDF("id") - - withTempDir { dir => - try { - val q = df1.writeStream - .option("checkpointLocation", dir.getCanonicalPath) - .format("delta") - .toTable(tbl) - - verifyTableMetadata(expectedSchema = df1.schema.asNullable) - - stream.addData(1, 2, 3) - q.processAllAvailable() - q.stop() - - val q2 = df1.withColumn("id2", 'id) - .writeStream - .format("delta") - .option("mergeSchema", "true") - .option("checkpointLocation", dir.getCanonicalPath) - .toTable(tbl) - - stream.addData(4, 5, 6) - q2.processAllAvailable() - - verifyTableMetadataAsync(expectedSchema = df1.schema.asNullable.add("id2", LongType)) - } finally { - spark.streams.active.foreach(_.stop()) - } - } - } - } - - test("streaming - external location") { - withTempDir { dir => - withTable(tbl) { - implicit val _sqlContext = spark.sqlContext - val stream = MemoryStream[Long] - val df1 = stream.toDF().toDF("id") - - val chk = new File(dir, "chkpoint").getCanonicalPath - val data = new File(dir, "data").getCanonicalPath - try { - val q = df1.writeStream - .option("checkpointLocation", chk) - .format("delta") - .option("path", data) - .toTable(tbl) - - verifyTableMetadata(expectedSchema = df1.schema.asNullable) - - stream.addData(1, 2, 3) - q.processAllAvailable() - q.stop() - - val q2 = df1.withColumn("id2", 'id) - .writeStream - .format("delta") - .option("mergeSchema", "true") - .option("checkpointLocation", chk) - .toTable(tbl) - - stream.addData(4, 5, 6) - q2.processAllAvailable() - - verifyTableMetadataAsync(expectedSchema = df1.schema.add("id2", LongType).asNullable) - } finally { - spark.streams.active.foreach(_.stop()) - } - } - } - } - - test("streaming - external table that already exists") { - withTable(tbl) { - implicit val _sqlContext = spark.sqlContext - val stream = MemoryStream[Long] - val df1 = stream.toDF().toDF("id") - - withTempDir { dir => - val chk = new File(dir, "chkpoint").getCanonicalPath - val data = new File(dir, "data").getCanonicalPath - - spark.range(10).write.format("delta").save(data) - try { - val q = df1.writeStream - .option("checkpointLocation", chk) - .format("delta") - .option("path", data) - .toTable(tbl) - - verifyTableMetadataAsync(expectedSchema = df1.schema.asNullable) - - stream.addData(1, 2, 3) - q.processAllAvailable() - q.stop() - - val q2 = df1.withColumn("id2", 'id) - .writeStream - .format("delta") - .option("mergeSchema", "true") - .option("checkpointLocation", chk) - .toTable(tbl) - - stream.addData(4, 5, 6) - q2.processAllAvailable() - - verifyTableMetadataAsync(expectedSchema = df1.schema.add("id2", LongType).asNullable) - } finally { - spark.streams.active.foreach(_.stop()) - } - } - } - } - - - test("convert to delta with partitioning change") { - withTable(tbl) { - val df = spark.range(10).withColumn("part", 'id / 2).withColumn("id2", 'id) - df.writeTo(tbl) - .partitionedBy('part) - .using("parquet") - .create() - - // Partitioning columns go to the end for parquet tables - val tableSchema = - new StructType().add("id", LongType).add("id2", LongType).add("part", DoubleType) - verifyTableMetadata( - expectedSchema = tableSchema, - expectedProperties = Map.empty, - partitioningCols = Seq("part") - ) - - sql(s"CONVERT TO DELTA $tbl PARTITIONED BY (part double)") - // Information is duplicated for now - verifyTableMetadata( - expectedSchema = tableSchema, - expectedProperties = Map.empty, - partitioningCols = Seq("part") - ) - - // Remove partitioning of table - df.writeTo(tbl).using("delta").replace() - - assert(snapshot.metadata.partitionColumns === Nil, "Table is unpartitioned") - - // Hive does not allow for the removal of the partition column once it has - // been added. Spark keeps the partition columns towards the end if it - // finds them in Hive. So, for converted tables with partitions, - // Hive schema != df.schema - val expectedSchema = tableSchema - - // Schema converts to Delta's format - verifyTableMetadata( - expectedSchema = expectedSchema, - expectedProperties = getBaseProperties(snapshot), - partitioningCols = Seq("part") // The partitioning information cannot be removed... - ) - - // table is still usable - checkAnswer(spark.table(tbl), df) - - val df2 = spark.range(10).withColumn("id2", 'id) - // Gets rid of partition column "part" from the schema - df2.writeTo(tbl).using("delta").replace() - - val expectedSchema2 = new StructType() - .add("id", LongType).add("id2", LongType).add("part", DoubleType) - verifyTableMetadataAsync( - expectedSchema = expectedSchema2, - expectedProperties = getBaseProperties(snapshot), - partitioningCols = Seq("part") // The partitioning information cannot be removed... - ) - - // table is still usable - checkAnswer(spark.table(tbl), df2) - } - } - - test("partitioned table + add column") { - withTable(tbl) { - val df = spark.range(10).withColumn("part", 'id / 2).withColumn("id2", 'id) - df.writeTo(tbl) - .partitionedBy('part) - .using("delta") - .create() - - val tableSchema = - new StructType().add("id", LongType).add("part", DoubleType).add("id2", LongType) - verifyTableMetadata( - expectedSchema = tableSchema, - expectedProperties = getBaseProperties(snapshot), - partitioningCols = Seq()) - - sql(s"ALTER TABLE $tbl ADD COLUMNS (id3 bigint)") - verifyTableMetadataAsync( - expectedSchema = tableSchema.add("id3", LongType), - expectedProperties = getBaseProperties(snapshot), - partitioningCols = Seq()) - } - } - - test("partitioned convert to delta with schema change") { - withTable(tbl) { - val df = spark.range(10).withColumn("part", 'id / 2).withColumn("id2", 'id) - df.writeTo(tbl) - .partitionedBy('part) - .using("parquet") - .create() - - // Partitioning columns go to the end - val tableSchema = - new StructType().add("id", LongType).add("id2", LongType).add("part", DoubleType) - verifyTableMetadata( - expectedSchema = tableSchema, - expectedProperties = Map.empty, - partitioningCols = Seq("part") - ) - - sql(s"CONVERT TO DELTA $tbl PARTITIONED BY (part double)") - // Information is duplicated for now - verifyTableMetadata( - expectedSchema = tableSchema, - expectedProperties = Map.empty, - partitioningCols = Seq("part") - ) - - sql(s"ALTER TABLE $tbl ADD COLUMNS (id3 bigint)") - - // Hive does not allow for the removal of the partition column once it has - // been added. Spark keeps the partition columns towards the end if it - // finds them in Hive. So, for converted tables with partitions, - // Hive schema != df.schema - val expectedSchema = new StructType() - .add("id", LongType) - .add("id2", LongType) - .add("id3", LongType) - .add("part", DoubleType) - - verifyTableMetadataAsync( - expectedSchema = expectedSchema, - partitioningCols = Seq("part") - ) - - // Table is still queryable - checkAnswer( - spark.table(tbl), - // Ordering of columns are different than df due to Hive semantics - spark.range(10).withColumn("id2", 'id) - .withColumn("part", 'id / 2) - .withColumn("id3", lit(null))) - } - } - - - import UpdateCatalog.MAX_CATALOG_TYPE_DDL_LENGTH - - test("Very long schemas can be stored in the catalog") { - withTable(tbl) { - val schema = StructType(Seq.tabulate(1000)(i => StructField(s"col$i", StringType))) - require(schema.toDDL.length >= MAX_CATALOG_TYPE_DDL_LENGTH, - s"The length of the schema should be over $MAX_CATALOG_TYPE_DDL_LENGTH " + - "characters for this test") - - sql(s"CREATE TABLE $tbl (${schema.toDDL}) USING delta") - verifyTableMetadata(expectedSchema = schema) - } - } - - test("Schemas that contain very long fields cannot be stored in the catalog") { - withTable(tbl) { - val schema = new StructType() - .add("i", StringType) - .add("struct", StructType(Seq.tabulate(1000)(i => StructField(s"col$i", StringType)))) - require(schema.toDDL.length >= MAX_CATALOG_TYPE_DDL_LENGTH, - s"The length of the schema should be over $MAX_CATALOG_TYPE_DDL_LENGTH " + - s"characters for this test") - - sql(s"CREATE TABLE $tbl (${schema.toDDL}) USING delta") - verifySchemaInCatalog() - } - } - - test("Schemas that contain very long fields cannot be stored in the catalog - array") { - withTable(tbl) { - val struct = StructType(Seq.tabulate(1000)(i => StructField(s"col$i", StringType))) - val schema = new StructType() - .add("i", StringType) - .add("array", ArrayType(struct)) - require(schema.toDDL.length >= MAX_CATALOG_TYPE_DDL_LENGTH, - s"The length of the schema should be over $MAX_CATALOG_TYPE_DDL_LENGTH " + - s"characters for this test") - - sql(s"CREATE TABLE $tbl (${schema.toDDL}) USING delta") - verifySchemaInCatalog() - } - } - - test("Schemas that contain very long fields cannot be stored in the catalog - map") { - withTable(tbl) { - val struct = StructType(Seq.tabulate(1000)(i => StructField(s"col$i", StringType))) - val schema = new StructType() - .add("i", StringType) - .add("map", MapType(StringType, struct)) - require(schema.toDDL.length >= MAX_CATALOG_TYPE_DDL_LENGTH, - s"The length of the schema should be over $MAX_CATALOG_TYPE_DDL_LENGTH " + - s"characters for this test") - - sql(s"CREATE TABLE $tbl (${schema.toDDL}) USING delta") - verifySchemaInCatalog() - } - } - - test("Very long schemas can be stored in the catalog - partitioned") { - withTable(tbl) { - val schema = StructType(Seq.tabulate(1000)(i => StructField(s"col$i", StringType))) - .add("part", StringType) - require(schema.toDDL.length >= MAX_CATALOG_TYPE_DDL_LENGTH, - "The length of the schema should be over 4000 characters for this test") - - sql(s"CREATE TABLE $tbl (${schema.toDDL}) USING delta PARTITIONED BY (part)") - verifyTableMetadata(expectedSchema = schema) - } - } - - test("Very long nested fields cannot be stored in the catalog - partitioned") { - withTable(tbl) { - val schema = new StructType() - .add("i", StringType) - .add("part", StringType) - .add("struct", StructType(Seq.tabulate(1000)(i => StructField(s"col$i", StringType)))) - require(schema.toDDL.length >= MAX_CATALOG_TYPE_DDL_LENGTH, - "The length of the schema should be over 4000 characters for this test") - - sql(s"CREATE TABLE $tbl (${schema.toDDL}) USING delta PARTITIONED BY (part)") - verifySchemaInCatalog() - } - } - - // scalastyle:off nonascii - test("Schema containing non-latin characters cannot be stored - top-level") { - withTable(tbl) { - val schema = new StructType().add("今天", "string") - sql(s"CREATE TABLE $tbl (${schema.toDDL}) USING delta") - verifySchemaInCatalog() - } - } - - test("Schema containing non-latin characters cannot be stored - struct") { - withTable(tbl) { - val schema = new StructType().add("struct", new StructType().add("今天", "string")) - sql(s"CREATE TABLE $tbl (${schema.toDDL}) USING delta") - verifySchemaInCatalog() - } - } - - test("Schema containing non-latin characters cannot be stored - array") { - withTable(tbl) { - val schema = new StructType() - .add("i", StringType) - .add("array", ArrayType(new StructType().add("今天", "string"))) - - sql(s"CREATE TABLE $tbl (${schema.toDDL}) USING delta") - verifySchemaInCatalog() - } - } - - test("Schema containing non-latin characters cannot be stored - map") { - withTable(tbl) { - val schema = new StructType() - .add("i", StringType) - .add("map", MapType(StringType, new StructType().add("今天", "string"))) - - sql(s"CREATE TABLE $tbl (${schema.toDDL}) USING delta") - verifySchemaInCatalog() - } - } - // scalastyle:on nonascii - - /** - * Verifies that the schema stored in the catalog explicitly is empty, however the getTablesByName - * method still correctly returns the actual schema. - */ - private def verifySchemaInCatalog( - table: String = tbl, - catalogPartitionCols: Seq[String] = Nil): Unit = { - val cat = spark.sessionState.catalog.externalCatalog.getTable("default", table) - assert(cat.schema.isEmpty, s"Schema wasn't empty") - assert(cat.partitionColumnNames === catalogPartitionCols) - getBaseProperties(snapshot).foreach { case (k, v) => - assert(cat.properties.get(k) === Some(v), - s"Properties didn't match for table: $table. Expected: ${getBaseProperties(snapshot)}, " + - s"Got: ${cat.properties}") - } - assert(cat.properties(UpdateCatalog.ERROR_KEY) === UpdateCatalog.LONG_SCHEMA_ERROR) - - // Make sure table is readable - checkAnswer(spark.table(table), Nil) - } - - def testAddRemoveProperties(): Unit = { - withTable(tbl) { - val df = spark.range(10).toDF("id") - df.writeTo(tbl) - .using("delta") - .create() - - var initialProperties: Map[String, String] = Map.empty - val logs = Log4jUsageLogger.track { - sql(s"ALTER TABLE $tbl SET TBLPROPERTIES(some.key = 1, another.key = 2)") - - initialProperties = getBaseProperties(snapshot) - verifyTableMetadataAsync( - expectedSchema = df.schema.asNullable, - expectedProperties = Map("some.key" -> "1", "another.key" -> "2") ++ - initialProperties - ) - } - val updateLogged = logs.filter(_.metric == "tahoeEvent") - .filter(_.tags.get("opType").exists(_.startsWith("delta.catalog.update.properties"))) - assert(updateLogged.nonEmpty, "Ensure that the schema update in the MetaStore is logged") - - // The UpdateCatalog hook only checks if new properties have been - // added. If properties have been removed only, no metadata update will be triggered. - val logs2 = Log4jUsageLogger.track { - sql(s"ALTER TABLE $tbl UNSET TBLPROPERTIES(another.key)") - verifyTableMetadataAsync( - expectedSchema = df.schema.asNullable, - expectedProperties = Map("some.key" -> "1", "another.key" -> "2") ++ - initialProperties - ) - } - val updateLogged2 = logs2.filter(_.metric == "tahoeEvent") - .filter(_.tags.get("opType").exists(_.startsWith("delta.catalog.update.properties"))) - assert(updateLogged2.size == 0, "Ensure that the schema update in the MetaStore is logged") - - // Adding a new property will trigger an update - val logs3 = Log4jUsageLogger.track { - sql(s"ALTER TABLE $tbl SET TBLPROPERTIES(a.third.key = 3)") - verifyTableMetadataAsync( - expectedSchema = df.schema.asNullable, - expectedProperties = Map("some.key" -> "1", "a.third.key" -> "3") ++ - getBaseProperties(snapshot) - ) - } - val updateLogged3 = logs3.filter(_.metric == "tahoeEvent") - .filter(_.tags.get("opType").exists(_.startsWith("delta.catalog.update.properties"))) - assert(updateLogged3.nonEmpty, "Ensure that the schema update in the MetaStore is logged") - } - } - - test("add and remove properties") { - testAddRemoveProperties() - } - - test("alter table commands update the catalog") { - runAlterTableTests { (tableName, expectedSchema) => - verifyTableMetadataAsync( - expectedSchema = expectedSchema, - // The ALTER TABLE statements in runAlterTableTests create table version 7. - // However, version 7 is created by dropping a CHECK constraint, which currently - // *does not* trigger a catalog update. For Hive tables, only *adding* properties - // causes a catalog update, not *removing*. Hence, the metadata in the catalog should - // still be at version 6. - expectedProperties = getBaseProperties(snapshotAt(6)) ++ - Map("some" -> "thing", "delta.constraints.id_3" -> "id3 > 10"), - table = tableName - ) - } - } -} diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaUpdateCatalogSuiteBase.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaUpdateCatalogSuiteBase.scala deleted file mode 100644 index 4f9f25acef7..00000000000 --- a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaUpdateCatalogSuiteBase.scala +++ /dev/null @@ -1,313 +0,0 @@ -/* - * Copyright (2021) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.delta - -import java.io.File - -import scala.util.control.NonFatal - -import org.apache.spark.sql.delta.actions.TableFeatureProtocolUtils -import org.apache.spark.sql.delta.hooks.UpdateCatalog -import org.apache.spark.sql.delta.sources.DeltaSQLConf -import org.apache.spark.sql.delta.test.DeltaTestImplicits._ -import org.scalatest.time.SpanSugar - -import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest} -import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.functions.{lit, struct} -import org.apache.spark.sql.test.SQLTestUtils -import org.apache.spark.sql.types.{BooleanType, DoubleType, IntegerType, LongType, StringType, StructField, StructType} -import org.apache.spark.util.{ThreadUtils, Utils} - -abstract class DeltaUpdateCatalogSuiteBase - extends QueryTest - with SQLTestUtils - with SpanSugar { - - protected val tbl = "delta_table" - - import testImplicits._ - - protected def cleanupDefaultTable(): Unit = disableUpdates { - spark.sql(s"DROP TABLE IF EXISTS $tbl") - val path = spark.sessionState.catalog.defaultTablePath(TableIdentifier(tbl)) - try Utils.deleteRecursively(new File(path)) catch { - case NonFatal(e) => // do nothing - } - } - - /** Turns off the storing of metadata (schema + properties) in the catalog. */ - protected def disableUpdates(f: => Unit): Unit = { - withSQLConf(DeltaSQLConf.DELTA_UPDATE_CATALOG_ENABLED.key -> "false") { - f - } - } - - protected def deltaLog: DeltaLog = DeltaLog.forTable(spark, TableIdentifier(tbl)) - protected def snapshot: Snapshot = deltaLog.unsafeVolatileSnapshot - protected def snapshotAt(v: Long): Snapshot = deltaLog.getSnapshotAt(v) - - protected def getBaseProperties(snapshot: Snapshot): Map[String, String] = { - Map( - DeltaConfigs.METASTORE_LAST_UPDATE_VERSION -> snapshot.version.toString, - DeltaConfigs.METASTORE_LAST_COMMIT_TIMESTAMP -> snapshot.timestamp.toString, - DeltaConfigs.MIN_READER_VERSION.key -> snapshot.protocol.minReaderVersion.toString, - DeltaConfigs.MIN_WRITER_VERSION.key -> snapshot.protocol.minWriterVersion.toString) ++ - snapshot.protocol.readerAndWriterFeatureNames.map { name => - s"${TableFeatureProtocolUtils.FEATURE_PROP_PREFIX}$name" -> - TableFeatureProtocolUtils.FEATURE_PROP_SUPPORTED - } ++ snapshot.metadata.configuration.get("delta.enableDeletionVectors") - .map("delta.enableDeletionVectors" -> _).toMap - } - - /** - * Verifies that the table metadata in the catalog are eventually up-to-date. Updates to the - * catalog are generally asynchronous, except explicit DDL operations, e.g. CREATE/REPLACE. - */ - protected def verifyTableMetadataAsync( - expectedSchema: StructType, - expectedProperties: Map[String, String] = getBaseProperties(snapshot), - table: String = tbl, - partitioningCols: Seq[String] = Nil): Unit = { - // We unfortunately need an eventually, because the updates can be async - eventually(timeout(10.seconds)) { - verifyTableMetadata(expectedSchema, expectedProperties, table, partitioningCols) - } - // Ensure that no other threads will later revert us back to the state we just checked - if (!UpdateCatalog.awaitCompletion(10000)) { - logWarning(s"There are active catalog udpate requests after 10 seconds") - } - } - - protected def filterProperties(properties: Map[String, String]): Map[String, String] - - /** Verifies that the table metadata in the catalog are up-to-date. */ - protected def verifyTableMetadata( - expectedSchema: StructType, - expectedProperties: Map[String, String] = getBaseProperties(snapshot), - table: String = tbl, - partitioningCols: Seq[String] = Nil): Unit = { - DeltaLog.clearCache() - val cat = spark.sessionState.catalog.externalCatalog.getTable("default", table) - assert(cat.schema === expectedSchema, s"Schema didn't match for table: $table") - assert(cat.partitionColumnNames === partitioningCols) - assert(filterProperties(cat.properties) === expectedProperties, - s"Properties didn't match for table: $table") - - val tables = spark.sessionState.catalog.getTablesByName(Seq(TableIdentifier(table))) - - assert(tables.head.schema === expectedSchema) - assert(tables.head.partitionColumnNames === partitioningCols) - assert(filterProperties(tables.head.properties) === expectedProperties) - } - - - test("mergeSchema") { - withTable(tbl) { - val df = spark.range(10).withColumn("part", 'id / 2) - df.writeTo(tbl).using("delta").create() - - verifyTableMetadata(expectedSchema = df.schema.asNullable) - - val df2 = spark.range(10).withColumn("part", 'id / 2).withColumn("id2", 'id) - df2.writeTo(tbl) - .option("mergeSchema", "true") - .append() - - verifyTableMetadataAsync(expectedSchema = df2.schema.asNullable) - } - } - - test("mergeSchema - nested data types") { - withTable(tbl) { - val df = spark.range(10).withColumn("part", 'id / 2) - .withColumn("str", struct('id.cast("int") as "int")) - df.writeTo(tbl).using("delta").create() - - verifyTableMetadata(expectedSchema = df.schema.asNullable) - - val df2 = spark.range(10).withColumn("part", 'id / 2) - .withColumn("str", struct('id as "id2", 'id.cast("int") as "int")) - df2.writeTo(tbl) - .option("mergeSchema", "true") - .append() - - val schema = new StructType() - .add("id", LongType) - .add("part", DoubleType) - .add("str", new StructType() - .add("int", IntegerType) - .add("id2", LongType)) // New columns go to the end - verifyTableMetadataAsync(expectedSchema = schema) - } - } - - - test("merge") { - val tmp = "tmpView" - withDeltaTable { df => - withTempView(tmp) { - withSQLConf(DeltaSQLConf.DELTA_SCHEMA_AUTO_MIGRATE.key -> "true") { - df.withColumn("id2", 'id).createOrReplaceTempView(tmp) - sql( - s"""MERGE INTO $tbl t - |USING $tmp s - |ON t.id = s.id - |WHEN NOT MATCHED THEN INSERT * - """.stripMargin) - - verifyTableMetadataAsync(df.withColumn("id2", 'id).schema.asNullable) - } - } - } - } - - test("creating and replacing a table puts the schema and table properties in the metastore") { - withTable(tbl) { - val df = spark.range(10).withColumn("part", 'id / 2).withColumn("id2", 'id) - df.writeTo(tbl) - .tableProperty("delta.checkpointInterval", "5") - .tableProperty("some", "thing") - .partitionedBy('part) - .using("delta") - .create() - - verifyTableMetadata( - expectedSchema = df.schema.asNullable, - expectedProperties = getBaseProperties(snapshot) ++ Map( - "delta.checkpointInterval" -> "5", - "some" -> "thing") - ) - - val df2 = spark.range(10).withColumn("part", 'id / 2) - df2.writeTo(tbl) - .tableProperty("other", "thing") - .using("delta") - .replace() - - verifyTableMetadata( - expectedSchema = df2.schema.asNullable, - expectedProperties = getBaseProperties(snapshot) ++ Map("other" -> "thing") - ) - } - } - - test("creating table in metastore over existing path") { - withTempDir { dir => - withTable(tbl) { - val df = spark.range(10).withColumn("part", 'id % 2).withColumn("id2", 'id) - df.write.format("delta").partitionBy("part").save(dir.getCanonicalPath) - - sql(s"CREATE TABLE $tbl USING delta LOCATION '${dir.getCanonicalPath}'") - verifyTableMetadata(df.schema.asNullable) - } - } - } - - test("replacing non-Delta table") { - withTable(tbl) { - val df = spark.range(10).withColumn("part", 'id / 2).withColumn("id2", 'id) - df.writeTo(tbl) - .tableProperty("delta.checkpointInterval", "5") - .tableProperty("some", "thing") - .partitionedBy('part) - .using("parquet") - .create() - - val e = intercept[AnalysisException] { - df.writeTo(tbl).using("delta").replace() - } - - assert(e.getMessage.contains("not a Delta table")) - } - } - - test("alter table add columns") { - withDeltaTable { df => - sql(s"ALTER TABLE $tbl ADD COLUMNS (id2 bigint)") - verifyTableMetadataAsync(df.withColumn("id2", 'id).schema.asNullable) - } - } - - protected def runAlterTableTests(f: (String, StructType) => Unit): Unit = { - // We set the default minWriterVersion to the version required to ADD/DROP CHECK constraints - // to prevent an automatic protocol upgrade (i.e. an implicit property change) when adding - // the CHECK constraint below. - withSQLConf( - "spark.databricks.delta.properties.defaults.minReaderVersion" -> "1", - "spark.databricks.delta.properties.defaults.minWriterVersion" -> "3") { - withDeltaTable { _ => - sql(s"ALTER TABLE $tbl SET TBLPROPERTIES ('some' = 'thing', 'other' = 'thing')") - sql(s"ALTER TABLE $tbl UNSET TBLPROPERTIES ('other')") - sql(s"ALTER TABLE $tbl ADD COLUMNS (id2 bigint, id3 bigint)") - sql(s"ALTER TABLE $tbl CHANGE COLUMN id2 id2 bigint FIRST") - sql(s"ALTER TABLE $tbl REPLACE COLUMNS (id3 bigint, id2 bigint, id bigint)") - sql(s"ALTER TABLE $tbl ADD CONSTRAINT id_3 CHECK (id3 > 10)") - sql(s"ALTER TABLE $tbl DROP CONSTRAINT id_3") - - val expectedSchema = StructType(Seq( - StructField("id3", LongType, true), - StructField("id2", LongType, true), - StructField("id", LongType, true)) - ) - - f(tbl, expectedSchema) - } - } - } - - /** - * Creates a table with the name `tbl` and executes a function that takes a representative - * DataFrame with the schema of the table. Performs cleanup of the table afterwards. - */ - protected def withDeltaTable(f: DataFrame => Unit): Unit = { - // Turn off async updates so that we don't update the catalog during table cleanup - disableUpdates { - withTable(tbl) { - withSQLConf(DeltaSQLConf.DELTA_UPDATE_CATALOG_ENABLED.key -> "true") { - sql(s"CREATE TABLE $tbl (id bigint) USING delta") - val df = spark.range(10) - verifyTableMetadata(df.schema.asNullable) - - f(df.toDF()) - } - } - } - } - - test("skip update when flag is not set") { - withDeltaTable(df => { - withSQLConf(DeltaSQLConf.DELTA_UPDATE_CATALOG_ENABLED.key -> "false") { - val propertiesAtV1 = getBaseProperties(snapshot) - sql(s"ALTER TABLE $tbl SET TBLPROPERTIES(some.key = 1)") - verifyTableMetadataAsync( - expectedSchema = df.schema.asNullable, - expectedProperties = propertiesAtV1) - } - }) - } - - - test(s"REORG TABLE does not perform catalog update") { - val tableName = "myTargetTable" - withDeltaTable { df => - sql(s"REORG TABLE $tbl APPLY (PURGE)") - verifyTableMetadataAsync(df.schema.asNullable) - } - } -} diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/MergeIntoDVsSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/MergeIntoDVsSuite.scala index b2448c2b441..ca18e330b78 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/MergeIntoDVsSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/MergeIntoDVsSuite.scala @@ -55,124 +55,7 @@ trait MergeIntoDVsTests extends MergeIntoSQLSuite with DeletionVectorsTestUtils "delta.dml.merge") } -class MergeIntoDVsSuite extends MergeIntoDVsTests { - import testImplicits._ - - def assertOperationalDVMetrics( - tablePath: String, - numDeletedRows: Long, - numUpdatedRows: Long, - numCopiedRows: Long, - numTargetFilesRemoved: Long, - numDeletionVectorsAdded: Long, - numDeletionVectorsRemoved: Long, - numDeletionVectorsUpdated: Long): Unit = { - val table = io.delta.tables.DeltaTable.forPath(tablePath) - val mergeMetrics = DeltaMetricsUtils.getLastOperationMetrics(table) - assert(mergeMetrics.getOrElse("numTargetRowsDeleted", -1) === numDeletedRows) - assert(mergeMetrics.getOrElse("numTargetRowsUpdated", -1) === numUpdatedRows) - assert(mergeMetrics.getOrElse("numTargetRowsCopied", -1) === numCopiedRows) - assert(mergeMetrics.getOrElse("numTargetFilesRemoved", -1) === numTargetFilesRemoved) - assert(mergeMetrics.getOrElse("numTargetDeletionVectorsAdded", -1) === numDeletionVectorsAdded) - assert( - mergeMetrics.getOrElse("numTargetDeletionVectorsRemoved", -1) === numDeletionVectorsRemoved) - assert( - mergeMetrics.getOrElse("numTargetDeletionVectorsUpdated", -1) === numDeletionVectorsUpdated) - } - - test(s"Merge with DVs metrics - Incremental Updates") { - withTempDir { dir => - val sourcePath = s"$dir/source" - val targetPath = s"$dir/target" - - spark.range(0, 10, 2).write.format("delta").save(sourcePath) - spark.range(10).write.format("delta").save(targetPath) - - executeMerge( - tgt = s"delta.`$targetPath` t", - src = s"delta.`$sourcePath` s", - cond = "t.id = s.id", - clauses = updateNotMatched(set = "id = t.id * 10")) - - checkAnswer(readDeltaTable(targetPath), Seq(0, 10, 2, 30, 4, 50, 6, 70, 8, 90).toDF("id")) - - assertOperationalDVMetrics( - targetPath, - numDeletedRows = 0, - numUpdatedRows = 5, - numCopiedRows = 0, - numTargetFilesRemoved = 0, // No files were fully deleted. - numDeletionVectorsAdded = 2, - numDeletionVectorsRemoved = 0, - numDeletionVectorsUpdated = 0) - - executeMerge( - tgt = s"delta.`$targetPath` t", - src = s"delta.`$sourcePath` s", - cond = "t.id = s.id", - clauses = delete(condition = "t.id = 2")) - - checkAnswer(readDeltaTable(targetPath), Seq(0, 10, 30, 4, 50, 6, 70, 8, 90).toDF("id")) - - assertOperationalDVMetrics( - targetPath, - numDeletedRows = 1, - numUpdatedRows = 0, - numCopiedRows = 0, - numTargetFilesRemoved = 0, - numDeletionVectorsAdded = 1, // Updating a DV equals removing and adding. - numDeletionVectorsRemoved = 1, // Updating a DV equals removing and adding. - numDeletionVectorsUpdated = 1) - - // Delete all rows from a file. - executeMerge( - tgt = s"delta.`$targetPath` t", - src = s"delta.`$sourcePath` s", - cond = "t.id = s.id", - clauses = delete(condition = "t.id < 5")) - - checkAnswer(readDeltaTable(targetPath), Seq(10, 30, 50, 6, 70, 8, 90).toDF("id")) - - assertOperationalDVMetrics( - targetPath, - numDeletedRows = 2, - numUpdatedRows = 0, - numCopiedRows = 0, - numTargetFilesRemoved = 1, - numDeletionVectorsAdded = 0, - numDeletionVectorsRemoved = 1, - numDeletionVectorsUpdated = 0) - } - } - - test(s"Merge with DVs metrics - delete entire file") { - withTempDir { dir => - val sourcePath = s"$dir/source" - val targetPath = s"$dir/target" - - spark.range(0, 7).write.format("delta").save(sourcePath) - spark.range(10).write.format("delta").save(targetPath) - - executeMerge( - tgt = s"delta.`$targetPath` t", - src = s"delta.`$sourcePath` s", - cond = "t.id = s.id", - clauses = update(set = "id = t.id * 10")) - - checkAnswer(readDeltaTable(targetPath), Seq(0, 10, 20, 30, 40, 50, 60, 7, 8, 9).toDF("id")) - - assertOperationalDVMetrics( - targetPath, - numDeletedRows = 0, - numUpdatedRows = 7, - numCopiedRows = 0, // No rows were copied. - numTargetFilesRemoved = 1, // 1 file was removed entirely. - numDeletionVectorsAdded = 1, // 1 file was deleted partially. - numDeletionVectorsRemoved = 0, - numDeletionVectorsUpdated = 0) - } - } -} +class MergeIntoDVsSuite extends MergeIntoDVsTests trait MergeCDCWithDVsTests extends MergeCDCTests with DeletionVectorsTestUtils { override def beforeAll(): Unit = { diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/MergeIntoMetricsBase.scala b/spark/src/test/scala/org/apache/spark/sql/delta/MergeIntoMetricsBase.scala index 46e676b6d86..73bb0550463 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/MergeIntoMetricsBase.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/MergeIntoMetricsBase.scala @@ -1339,20 +1339,11 @@ object MergeIntoMetricsBase extends QueryTest with SharedSparkSession { val mergeTimeMetrics = Set("executionTimeMs", "scanTimeMs", "rewriteTimeMs") // Metrics related with CDF. Available only when CDF is available. val mergeCdfMetrics = Set("numTargetChangeFilesAdded") - // DV Metrics. - val mergeDVMetrics = Set( - "numTargetDeletionVectorsAdded", - "numTargetDeletionVectorsUpdated", - "numTargetDeletionVectorsRemoved") // Ensure that all metrics are properly copied here. assert( DeltaOperationMetrics.MERGE.size == - mergeRowMetrics.size + - mergeFileMetrics.size + - mergeTimeMetrics.size + - mergeCdfMetrics.size + - mergeDVMetrics.size + mergeRowMetrics.size + mergeFileMetrics.size + mergeTimeMetrics.size + mergeCdfMetrics.size ) /////////////////// diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/MergeIntoSuiteBase.scala b/spark/src/test/scala/org/apache/spark/sql/delta/MergeIntoSuiteBase.scala index cdead65864e..ed48a9b8485 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/MergeIntoSuiteBase.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/MergeIntoSuiteBase.scala @@ -2236,10 +2236,8 @@ abstract class MergeIntoSuiteBase .collect().head.getMap(0).asInstanceOf[Map[String, String]] assert(metrics.contains("numTargetFilesRemoved")) // If insert-only code path is not used, then the general code path will rewrite existing - // target files when DVs are not enabled. - if (!spark.conf.get(DeltaSQLConf.MERGE_USE_PERSISTENT_DELETION_VECTORS)) { - assert(metrics("numTargetFilesRemoved").toInt > 0) - } + // target files. + assert(metrics("numTargetFilesRemoved").toInt > 0) } test("insert only merge - multiple matches when feature flag off") { @@ -2579,9 +2577,7 @@ abstract class MergeIntoSuiteBase assert(stats.targetBeforeSkipping.files.get > stats.targetAfterSkipping.files.get) } } else { - if (!spark.conf.get(DeltaSQLConf.MERGE_USE_PERSISTENT_DELETION_VECTORS)) { - assert(stats.targetFilesRemoved > 0) - } + assert(stats.targetFilesRemoved > 0) // If there is no insert clause and the flag is enabled, data skipping should be // performed on targetOnly predicates. // However, with insert clauses, it's expected that no additional data skipping diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/clustering/ClusteredTableClusteringSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/clustering/ClusteredTableClusteringSuite.scala deleted file mode 100644 index d1b2e0da346..00000000000 --- a/spark/src/test/scala/org/apache/spark/sql/delta/clustering/ClusteredTableClusteringSuite.scala +++ /dev/null @@ -1,82 +0,0 @@ -/* - * Copyright (2021) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.delta.clustering - -import org.apache.spark.sql.delta.skipping.ClusteredTableTestUtils -import org.apache.spark.sql.delta.skipping.clustering.ClusteredTableUtils -import org.apache.spark.sql.delta.DeltaLog -import org.apache.spark.sql.delta.actions.AddFile -import org.apache.spark.sql.delta.test.DeltaSQLCommandTest - -import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSparkSession - -class ClusteredTableClusteringSuite extends SparkFunSuite - with SharedSparkSession - with ClusteredTableTestUtils - with DeltaSQLCommandTest { - import testImplicits._ - - private val table: String = "test_table" - - // Ingest data to create numFiles files with one row in each file. - private def addFiles(table: String, numFiles: Int): Unit = { - val df = (1 to numFiles).map(i => (i, i)).toDF("col1", "col2") - withSQLConf(SQLConf.MAX_RECORDS_PER_FILE.key -> "1") { - df.write.format("delta").mode("append").saveAsTable(table) - } - } - - private def getFiles(table: String): Set[AddFile] = { - val deltaLog = DeltaLog.forTable(spark, TableIdentifier(table)) - deltaLog.update().allFiles.collect().toSet - } - - private def assertClustered(files: Set[AddFile]): Unit = { - assert(files.forall(_.clusteringProvider.contains(ClusteredTableUtils.clusteringProvider))) - } - - private def assertNotClustered(files: Set[AddFile]): Unit = { - assert(files.forall(_.clusteringProvider.isEmpty)) - } - - test("optimize clustered table") { - withSQLConf(SQLConf.MAX_RECORDS_PER_FILE.key -> "2") { - withClusteredTable( - table = table, - schema = "col1 int, col2 int", - clusterBy = "col1, col2") { - addFiles(table, numFiles = 4) - val files0 = getFiles(table) - assert(files0.size === 4) - assertNotClustered(files0) - - // Optimize should cluster the data into two 2 files since MAX_RECORDS_PER_FILE is 2. - runOptimize(table) { metrics => - assert(metrics.numFilesRemoved == 4) - assert(metrics.numFilesAdded == 2) - } - - val files1 = getFiles(table) - assert(files1.size == 2) - assertClustered(files1) - } - } - } -} diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/clustering/ClusteringMetadataDomainSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/clustering/ClusteringMetadataDomainSuite.scala deleted file mode 100644 index ce7027a25ba..00000000000 --- a/spark/src/test/scala/org/apache/spark/sql/delta/clustering/ClusteringMetadataDomainSuite.scala +++ /dev/null @@ -1,33 +0,0 @@ -/* - * Copyright (2021) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.delta.clustering - -import org.apache.spark.sql.delta.skipping.clustering.ClusteringColumn - -import org.apache.spark.SparkFunSuite - -class ClusteringMetadataDomainSuite extends SparkFunSuite { - test("serialized string follows the spec") { - val clusteringColumns = Seq(ClusteringColumn(Seq("col1", "`col2,col3`", "`col4.col5`,col6"))) - val clusteringMetadataDomain = ClusteringMetadataDomain.fromClusteringColumns(clusteringColumns) - val serializedString = clusteringMetadataDomain.toDomainMetadata.json - assert(serializedString === - """|{"domainMetadata":{"domain":"delta.clustering","configuration": - |"{\"clusteringColumns\":[[\"col1\",\"`col2,col3`\",\"`col4.col5`,col6\"]], - |\"domainName\":\"delta.clustering\"}","removed":false}}""".stripMargin.replace("\n", "")) - } -} diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/deletionvectors/DeletionVectorsSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/deletionvectors/DeletionVectorsSuite.scala index 23798151994..04eacb36d81 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/deletionvectors/DeletionVectorsSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/deletionvectors/DeletionVectorsSuite.scala @@ -501,9 +501,11 @@ class DeletionVectorsSuite extends QueryTest for (ts <- tombstones) { assert(ts.deletionVector != null) } - // target log should contain two files, one with and one without DV - assert(allFiles.count(_.deletionVector != null) === 1) - assert(allFiles.count(_.deletionVector == null) === 1) + // target log should not contain DVs + for (f <- allFiles) { + assert(f.deletionVector == null) + assert(f.tightBounds.get) + } } } @@ -571,12 +573,13 @@ class DeletionVectorsSuite extends QueryTest { sql(s"UPDATE delta.`$path` SET id = -1 WHERE id = 0") val (added, removed) = getFileActionsInLastVersion(deltaLog) - assert(added.length === 2) + assert(added.length === 1) assert(removed.length === 1) - // Added files must be two, one containing DV and one not - assert(added.count(_.deletionVector != null) === 1) - assert(added.count(_.deletionVector == null) === 1) - // Removed files must contain DV + // Removed files must contain DV, added files must not + for (a <- added) { + assert(a.deletionVector === null) + assert(a.tightBounds.get) + } for (r <- removed) { assert(r.deletionVector !== null) } @@ -593,13 +596,13 @@ class DeletionVectorsSuite extends QueryTest .updateExpr(Map("id" -> "source.value")) .whenNotMatchedBySource().delete().execute() val (added, removed) = getFileActionsInLastVersion(deltaLog) - assert(removed.length === 3) + assert(removed.length === 2) for (a <- added) { assert(a.deletionVector === null) assert(a.tightBounds.get) } - // Two of three removed files have DV - assert(removed.count(_.deletionVector != null) === 2) + // One of two removed files has DV + assert(removed.count(_.deletionVector != null) === 1) // -1 and 9 are deleted by "when not matched by source" checkTableContents(Seq(2, 3, 4, 5, 6, 7).toDF()) diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/optimize/CompactionTestHelper.scala b/spark/src/test/scala/org/apache/spark/sql/delta/optimize/CompactionTestHelper.scala deleted file mode 100644 index bf1458ff0bc..00000000000 --- a/spark/src/test/scala/org/apache/spark/sql/delta/optimize/CompactionTestHelper.scala +++ /dev/null @@ -1,109 +0,0 @@ -/* - * Copyright (2021) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.delta.optimize - -// scalastyle:off import.ordering.noEmptyLine -import org.apache.spark.sql.delta.DeltaLog -import org.apache.spark.sql.delta.commands.optimize.OptimizeMetrics -import org.apache.spark.sql.delta.hooks.AutoCompact -import org.apache.spark.sql.delta.sources.DeltaSQLConf._ -import org.apache.spark.sql.delta.test.DeltaTestImplicits._ - -import org.apache.spark.sql.QueryTest -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.test.SQLTestUtils - -/** - * A trait used by unit tests to trigger compaction over a table. - */ -private[delta] trait CompactionTestHelper extends QueryTest with SQLTestUtils { - - /** - * Compact files under the given `tablePath` using AutoCompaction/OPTIMIZE and - * returns the [[OptimizeMetrics]] - */ - def compactAndGetMetrics(tablePath: String, where: String = ""): OptimizeMetrics - - /** config controlling the min file size required for compaction */ - val minFileSizeConf: String - - /** config controlling the target file size for compaction */ - val maxFileSizeConf: String - - /** Create `numFilePartitions` partitions and each partition has `numFilesPerPartition` files. */ - def createFilesToPartitions( - numFilePartitions: Int, numFilesPerPartition: Int, dir: String) - (implicit spark: SparkSession): Unit = { - val totalNumFiles = numFilePartitions * numFilesPerPartition - spark.range(start = 0, end = totalNumFiles, step = 1, numPartitions = totalNumFiles) - .selectExpr(s"id % $numFilePartitions as c0", "id as c1") - .write - .format("delta") - .partitionBy("c0") - .mode("append") - .save(dir) - } - - /** Create `numFiles` files without any partition. */ - def createFilesWithoutPartitions( - numFiles: Int, dir: String)(implicit spark: SparkSession): Unit = { - spark.range(start = 0, end = numFiles, step = 1, numPartitions = numFiles) - .selectExpr("id as c0", "id as c1", "id as c2") - .write - .format("delta") - .mode("append") - .save(dir) - } -} - -private[delta] trait CompactionTestHelperForOptimize extends CompactionTestHelper { - - override def compactAndGetMetrics(tablePath: String, where: String = ""): OptimizeMetrics = { - import testImplicits._ - val whereClause = if (where != "") s"WHERE $where" else "" - val res = spark.sql(s"OPTIMIZE tahoe.`$tablePath` $whereClause") - val metrics: OptimizeMetrics = res.select($"metrics.*").as[OptimizeMetrics].head() - metrics - } - - override val minFileSizeConf: String = DELTA_OPTIMIZE_MIN_FILE_SIZE.key - - override val maxFileSizeConf: String = DELTA_OPTIMIZE_MAX_FILE_SIZE.key -} - -private[delta] trait CompactionTestHelperForAutoCompaction extends CompactionTestHelper { - - override def compactAndGetMetrics(tablePath: String, where: String = ""): OptimizeMetrics = { - // Set min num files to 2 - so that even if two small files are present in a partition, then - // also they are compacted. - var metrics: Option[OptimizeMetrics] = None - withSQLConf(DELTA_AUTO_COMPACT_MIN_NUM_FILES.key -> "2") { - metrics = - Some( - AutoCompact.compact( - spark, - DeltaLog.forTable(spark, tablePath) - ).head - ) - } - metrics.get - } - - override val minFileSizeConf: String = DELTA_AUTO_COMPACT_MIN_FILE_SIZE.key - - override val maxFileSizeConf: String = DELTA_AUTO_COMPACT_MAX_FILE_SIZE.key -} diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/skipping/ClusteredTableTestUtils.scala b/spark/src/test/scala/org/apache/spark/sql/delta/skipping/ClusteredTableTestUtils.scala deleted file mode 100644 index 5045a2a9ba2..00000000000 --- a/spark/src/test/scala/org/apache/spark/sql/delta/skipping/ClusteredTableTestUtils.scala +++ /dev/null @@ -1,148 +0,0 @@ -/* - * Copyright (2021) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.delta.skipping - -import org.apache.spark.sql.delta.skipping.clustering.{ClusteredTableUtils, ClusteringColumn} -import org.apache.spark.sql.delta.{DeltaLog, Snapshot} -import org.apache.spark.sql.delta.commands.optimize.OptimizeMetrics - -import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.test.SharedSparkSession -import org.apache.spark.util.Utils - -trait ClusteredTableTestUtilsBase extends SparkFunSuite with SharedSparkSession { - import testImplicits._ - - /** - * Helper for running optimize on the table with different APIs. - * @param table the name of table - */ - def optimizeTable(table: String): DataFrame = { - sql(s"OPTIMIZE $table") - } - - /** - * Runs optimize on the table and calls postHook on the metrics. - * @param table the name of table - * @param postHook callback triggered with OptimizeMetrics returned by the OPTIMIZE command - */ - def runOptimize(table: String)(postHook: OptimizeMetrics => Unit): Unit = { - postHook(optimizeTable(table).select($"metrics.*").as[OptimizeMetrics].head()) - } - - def verifyClusteringColumnsInDomainMetadata( - snapshot: Snapshot, - expectedLogicalClusteringColumns: String): Unit = { - val logicalColumnNames = if (expectedLogicalClusteringColumns.trim.isEmpty) { - Seq.empty[String] - } else { - expectedLogicalClusteringColumns.split(",").map(_.trim).toSeq - } - val expectedClusteringColumns = logicalColumnNames.map(ClusteringColumn(snapshot.schema, _)) - val actualClusteringColumns = - ClusteredTableUtils.getClusteringColumnsOptional(snapshot).getOrElse(Seq.empty) - assert(expectedClusteringColumns == actualClusteringColumns) - } - - def withClusteredTable[T]( - table: String, - schema: String, - clusterBy: String, - tableProperties: Map[String, String] = Map.empty, - location: Option[String] = None)(f: => T): T = { - createOrReplaceClusteredTable("CREATE", table, schema, clusterBy, tableProperties, location) - - Utils.tryWithSafeFinally(f) { - spark.sql(s"DROP TABLE IF EXISTS $table") - } - } - - /** - * Helper for creating or replacing table with different APIs. - * @param clause clause for SQL API ('CREATE', 'REPLACE', 'CREATE OR REPLACE') - * @param table the name of table - * @param schema comma separated list of "colName dataType" - * @param clusterBy comma separated list of clustering columns - */ - def createOrReplaceClusteredTable( - clause: String, - table: String, - schema: String, - clusterBy: String, - tableProperties: Map[String, String] = Map.empty, - location: Option[String] = None): Unit = { - val locationClause = if (location.isEmpty) "" else s"LOCATION '${location.get}'" - val tablePropertiesClause = if (!tableProperties.isEmpty) { - val tablePropertiesString = tableProperties.map { - case (key, value) => s"'$key' = '$value'" - }.mkString(",") - s"TBLPROPERTIES($tablePropertiesString)" - } else { - "" - } - spark.sql(s"$clause TABLE $table ($schema) USING delta CLUSTER BY ($clusterBy) " + - s"$tablePropertiesClause $locationClause") - } - - protected def createOrReplaceAsSelectClusteredTable( - clause: String, - table: String, - srcTable: String, - clusterBy: String, - location: Option[String] = None): Unit = { - val locationClause = if (location.isEmpty) "" else s"LOCATION '${location.get}'" - spark.sql(s"$clause TABLE $table USING delta CLUSTER BY ($clusterBy) " + - s"$locationClause AS SELECT * FROM $srcTable") - } - - def verifyClusteringColumns( - tableIdentifier: TableIdentifier, - expectedLogicalClusteringColumns: String - ): Unit = { - val (_, snapshot) = DeltaLog.forTableWithSnapshot(spark, tableIdentifier) - verifyClusteringColumnsInternal( - snapshot, - tableIdentifier.table, - expectedLogicalClusteringColumns - ) - } - - def verifyClusteringColumns( - dataPath: String, - expectedLogicalClusteringColumns: String - ): Unit = { - val (_, snapshot) = DeltaLog.forTableWithSnapshot(spark, dataPath) - verifyClusteringColumnsInternal( - snapshot, - s"delta.`$dataPath`", - expectedLogicalClusteringColumns - ) - } - - def verifyClusteringColumnsInternal( - snapshot: Snapshot, - tableNameOrPath: String, - expectedLogicalClusteringColumns: String - ): Unit = { - assert(ClusteredTableUtils.isSupported(snapshot.protocol) === true) - verifyClusteringColumnsInDomainMetadata(snapshot, expectedLogicalClusteringColumns) - } -} - -trait ClusteredTableTestUtils extends ClusteredTableTestUtilsBase diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/skipping/clustering/ClusteredTableDDLSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/skipping/clustering/ClusteredTableDDLSuite.scala deleted file mode 100644 index 92c52a626d2..00000000000 --- a/spark/src/test/scala/org/apache/spark/sql/delta/skipping/clustering/ClusteredTableDDLSuite.scala +++ /dev/null @@ -1,612 +0,0 @@ -/* - * Copyright (2021) The Delta Lake Project Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.delta.skipping.clustering - -import java.io.File - -import org.apache.spark.sql.delta.skipping.ClusteredTableTestUtils -import org.apache.spark.sql.delta.{DeltaAnalysisException, DeltaColumnMappingEnableIdMode, DeltaColumnMappingEnableNameMode, DeltaConfigs, DeltaLog, DeltaUnsupportedOperationException} -import org.apache.spark.sql.delta.sources.DeltaSQLConf -import org.apache.spark.sql.delta.stats.SkippingEligibleDataType -import org.apache.spark.sql.delta.test.{DeltaColumnMappingSelectedTestMixin, DeltaSQLCommandTest} - -import org.apache.spark.SparkConf -import org.apache.spark.sql.{QueryTest, Row} -import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.test.SharedSparkSession -import org.apache.spark.sql.types.{ArrayType, IntegerType, StructField, StructType} - -trait ClusteredTableCreateOrReplaceDDLSuiteBase - extends QueryTest with SharedSparkSession with ClusteredTableTestUtils { - - protected val testTable: String = "test_ddl_table" - protected val sourceTable: String = "test_ddl_source" - protected val targetTable: String = "test_ddl_target" - - protected def isPathBased: Boolean = false - - protected def supportedClauses: Seq[String] - - testCtasRtasHelper(supportedClauses) - testClusteringColumnsPartOfStatsColumn(supportedClauses) - testColTypeValidation("CREATE") - - def testCtasRtasHelper(clauses: Seq[String]): Unit = { - Seq( - ("", - "a INT, b STRING, ts TIMESTAMP", - "a, b"), - (" multipart name", - "a STRUCT, ts TIMESTAMP", - "a.b, ts") - ).foreach { case (testSuffix, columns, clusteringColumns) => - test(s"create/replace table$testSuffix") { - withTable(testTable) { - clauses.foreach { clause => - createOrReplaceClusteredTable(clause, testTable, columns, clusteringColumns) - verifyClusteringColumns(TableIdentifier(testTable), clusteringColumns) - } - } - } - - test(s"ctas/rtas$testSuffix") { - withTable(sourceTable, targetTable) { - sql(s"CREATE TABLE $sourceTable($columns) USING delta") - withTempDirIfNecessary { location => - clauses.foreach { clause => - createOrReplaceAsSelectClusteredTable( - clause, targetTable, sourceTable, clusteringColumns, location = location) - verifyClusteringColumns(targetTable, clusteringColumns, location) - } - } - } - } - - if (clauses.contains("REPLACE")) { - test(s"Replace from non clustered table$testSuffix") { - withTable(targetTable) { - sql(s"CREATE TABLE $targetTable($columns) USING delta") - createOrReplaceClusteredTable("REPLACE", targetTable, columns, clusteringColumns) - verifyClusteringColumns(TableIdentifier(targetTable), clusteringColumns) - } - } - } - } - } - - protected def createTableWithStatsColumns( - clause: String, - table: String, - clusterColumns: Seq[String], - numIndexedColumns: Int, - tableSchema: Option[String], - statsColumns: Seq[String] = Seq.empty, - location: Option[String] = None): Unit = { - val clusterSpec = clusterColumns.mkString(",") - val updatedTableProperties = - collection.mutable.Map("delta.dataSkippingNumIndexedCols" -> s"$numIndexedColumns") - if (statsColumns.nonEmpty) { - updatedTableProperties(DeltaConfigs.DATA_SKIPPING_STATS_COLUMNS.key) = - statsColumns.mkString(",") - } - val tablePropertiesString = updatedTableProperties.map { - case (key, value) => s"'$key' = '$value'" - }.mkString(",") - val locationClause = if (location.isEmpty) "" else s"LOCATION '${location.get}'" - if (clause == "REPLACE") { - // Create the default before it can be replaced. - sql(s"CREATE TABLE IF NOT EXISTS $table USING DELTA $locationClause") - } - if (tableSchema.isEmpty) { - sql( - s""" - |$clause TABLE $table USING DELTA CLUSTER BY ($clusterSpec) - |TBLPROPERTIES($tablePropertiesString) - |$locationClause - |AS SELECT * FROM $sourceTable - |""".stripMargin) - } else { - createOrReplaceClusteredTable( - clause, table, tableSchema.get, clusterSpec, updatedTableProperties.toMap, location) - } - } - - protected def testStatsCollectionHelper( - tableSchema: String, - numberOfIndexedCols: Int)(cb: => Unit): Unit = { - withTable(sourceTable) { - // Create a source table for CTAS. - sql( - s""" - | CREATE TABLE $sourceTable($tableSchema) USING DELTA - | TBLPROPERTIES('delta.dataSkippingNumIndexedCols' = '$numberOfIndexedCols') - |""".stripMargin) - // Run additional steps. - cb - } - } - - protected def testColTypeValidation(clause: String): Unit = { - test(s"validate column datatype checking on $clause table") { - withTable("srcTbl", "dstTbl") { - // Create reference table for CTAS/RTAS. - sql(s"CREATE table srcTbl (a STRUCT, d BOOLEAN, e MAP) USING delta") - - val data = (0 to 1000).map(i => Row(Row(i + 1, i * 10), i % 2 == 0, Map(i -> i))) - val schema = StructType(List( - StructField("a", StructType( - Array( - StructField("b", IntegerType), - StructField("c", IntegerType) - ) - )))) - spark.createDataFrame(spark.sparkContext.parallelize(data), StructType(schema)) - .write.mode("append").format("delta").saveAsTable("srcTbl") - - val (_, snapshot) = DeltaLog.forTableWithSnapshot(spark, new TableIdentifier("srcTbl")) - val schemaStr = snapshot.statCollectionLogicalSchema.treeString - // Test multiple data types. - Seq("a", "d", "e").foreach { colName => - withTempDir { tmpDir => - // Since validation happens both on create and replace, validate for both cases to - // ensure that datatype validation behaves consistently between the two. - if (clause == "REPLACE") { - sql("DROP TABLE IF EXISTS dstTbl") - sql(s"CREATE TABLE dstTbl LIKE srcTbl LOCATION '${tmpDir.getAbsolutePath}'") - } - - Seq( - // Scenario 1: Standard CREATE/REPLACE TABLE. - () => { - val schema = "a STRUCT, d BOOLEAN, e MAP" - createOrReplaceClusteredTable( - clause, "dstTbl", schema, colName, location = Some(tmpDir.getAbsolutePath)) - }, - // Scenario 2: CTAS/RTAS. - () => - createOrReplaceAsSelectClusteredTable( - clause, "dstTbl", "srcTbl", colName, location = Some(tmpDir.getAbsolutePath))) - .foreach { f => - val e = intercept[DeltaAnalysisException] { - f() - } - checkError( - e, - "DELTA_CLUSTERING_COLUMN_MISSING_STATS" - ) - } - } - } - } - } - } - - test("cluster by with more than 4 columns - create table") { - val testTable = "test_table" - withTable(testTable) { - val e = intercept[DeltaAnalysisException] { - createOrReplaceClusteredTable( - "CREATE", testTable, "a INT, b INT, c INT, d INT, e INT", "a, b, c, d, e") - } - checkError( - e, - errorClass = "DELTA_CLUSTER_BY_INVALID_NUM_COLUMNS" - ) - } - } - - test("cluster by with more than 4 columns - ctas") { - val testTable = "test_table" - val schema = "a INT, b INT, c INT, d INT, e INT" - withTempDirIfNecessary { location => - withTable(sourceTable, testTable) { - sql(s"CREATE TABLE $sourceTable($schema) USING delta") - val e = intercept[DeltaAnalysisException] { - createOrReplaceAsSelectClusteredTable( - "CREATE", testTable, sourceTable, "a, b, c, d, e", location = location) - } - checkError( - e, - errorClass = "DELTA_CLUSTER_BY_INVALID_NUM_COLUMNS" - ) - } - } - } - - protected def verifyPartitionColumns( - tableIdentifier: TableIdentifier, - expectedPartitionColumns: Seq[String]): Unit = { - val (_, snapshot) = DeltaLog.forTableWithSnapshot(spark, tableIdentifier) - assert(snapshot.metadata.partitionColumns === expectedPartitionColumns) - } - - protected def verifyClusteringColumns( - table: String, - expectedLogicalClusteringColumns: String, - locationOpt: Option[String]): Unit = { - locationOpt.map { location => - verifyClusteringColumns( - location, expectedLogicalClusteringColumns - ) - }.getOrElse { - verifyClusteringColumns(TableIdentifier(table), expectedLogicalClusteringColumns) - } - } - - def testClusteringColumnsPartOfStatsColumn(clauses: Seq[String]): Unit = { - clauses.foreach { clause => - val mode = if (clause == "CREATE") "create table" else "replace table" - test(s"Validate clustering columns part of stats columns - $mode") { - val tableSchema = "col0 int, col1 STRUCT, col2 int" - val indexedColumns = 2 - testStatsCollectionHelper( - tableSchema = tableSchema, - numberOfIndexedCols = indexedColumns) { - withTable(targetTable) { - val deltaLogSrc = DeltaLog.forTable(spark, TableIdentifier(sourceTable)) - // Validate the 3rd column `col1.col12` and 4th column `col2` can not be - // clustering columns. - val e = intercept[DeltaAnalysisException]( - createTableWithStatsColumns( - clause, - targetTable, - "col0" :: "col1.col11" :: "col1.col12" :: "col2" :: Nil, - indexedColumns, - Some(tableSchema))) - checkError( - e, - "DELTA_CLUSTERING_COLUMN_MISSING_STATS" - ) - // Validate the first two columns can be clustering columns. - createTableWithStatsColumns( - clause, - targetTable, - "col0" :: "col1.col11" :: Nil, - indexedColumns, - Some(tableSchema)) - } - } - } - } - - clauses.foreach { clause => - val mode = if (clause == "CREATE") "ctas" else "rtas" - test(s"Validate clustering columns part of stats columns - $mode") { - // Add a suffix for the target table name to work around the issue that delta table's - // location isn't removed by the DROP TABLE from ctas/rtas test cases. - val table = targetTable + "_" + clause - - val tableSchema = "col0 int, col1 STRUCT, col2 int" - val indexedColumns = 2 - testStatsCollectionHelper( - tableSchema = tableSchema, - numberOfIndexedCols = indexedColumns) { - withTable(table) { - withTempDir { dir => - val deltaLogSrc = DeltaLog.forTable(spark, TableIdentifier(sourceTable)) - val targetLog = DeltaLog.forTable(spark, s"${dir.getPath}") - val dataPath = new File(targetLog.dataPath.toString.replace("file:", "")) - val initialNumFiles = - if (dataPath.listFiles() != null) { // Returns null if directory doesn't exist -> 0 - dataPath.listFiles().size - } - else { - 0 - } - // Validate the 3rd column `col1.col12` and 4th column `col2` can not be - // clustering columns. - val e = intercept[DeltaAnalysisException]( - createTableWithStatsColumns( - clause, - table, - "col0" :: "col1.col11" :: "col1.col12" :: "col2" :: Nil, - indexedColumns, - None, - location = Some(dir.getPath))) - checkError( - e, - "DELTA_CLUSTERING_COLUMN_MISSING_STATS" - ) - - // Validate the first two columns can be clustering columns. - createTableWithStatsColumns( - clause, - table, - "col0" :: "col1.col11" :: Nil, - indexedColumns, - None) - } - } - } - } - } - } - - test("Validate clustering columns cannot be non-eligible data types") { - val indexedColumns = 3 - // Validate non-eligible column stat data type. - val nonEligibleType = ArrayType(IntegerType) - assert(!SkippingEligibleDataType(nonEligibleType)) - val nonEligibleTableSchema = s"col0 int, col1 STRUCT, col12: string>" - testStatsCollectionHelper( - tableSchema = nonEligibleTableSchema, - numberOfIndexedCols = indexedColumns) { - withTable(targetTable) { - val deltaLogSrc = DeltaLog.forTable(spark, TableIdentifier(sourceTable)) - // Validate the 2nd column `col1.col11` cannot be clustering column. - val e = intercept[DeltaAnalysisException]( - createTableWithStatsColumns( - "CREATE", - targetTable, - "col0" :: "col1.col11" :: Nil, - indexedColumns, - Some(nonEligibleTableSchema))) - checkError( - e, - "DELTA_CLUSTERING_COLUMN_MISSING_STATS" - ) - } - } - } - - protected def withTempDirIfNecessary(f: Option[String] => Unit): Unit = { - if (isPathBased) { - withTempDir { dir => - f(Some(dir.getAbsolutePath)) - } - } else { - f(None) - } - } -} - -trait ClusteredTableDDLWithColumnMapping - extends ClusteredTableCreateOrReplaceDDLSuite - with DeltaColumnMappingSelectedTestMixin { - - override protected def runOnlyTests: Seq[String] = Seq( - "validate dropping clustering column is not allowed: single clustering column", - "validate dropping clustering column is not allowed: multiple clustering columns", - "validate dropping clustering column is not allowed: clustering column + " + - "non-clustering column" - ) - - test("validate dropping clustering column is not allowed: single clustering column") { - withClusteredTable(testTable, "col1 INT, col2 STRING, col3 LONG", "col1") { - val e = intercept[DeltaAnalysisException] { - sql(s"ALTER TABLE $testTable DROP COLUMNS (col1)") - } - checkError( - e, - "DELTA_UNSUPPORTED_DROP_CLUSTERING_COLUMN" - ) - // Drop non-clustering columns are allowed. - sql(s"ALTER TABLE $testTable DROP COLUMNS (col2)") - } - } - - test("validate dropping clustering column is not allowed: multiple clustering columns") { - withClusteredTable(testTable, "col1 INT, col2 STRING, col3 LONG", "col1, col2") { - val e = intercept[DeltaAnalysisException] { - sql(s"ALTER TABLE $testTable DROP COLUMNS (col1, col2)") - } - checkError( - e, - "DELTA_UNSUPPORTED_DROP_CLUSTERING_COLUMN" - ) - } - } - - test("validate dropping clustering column is not allowed: clustering column + " + - "non-clustering column") { - withClusteredTable(testTable, "col1 INT, col2 STRING, col3 LONG", "col1, col2") { - val e = intercept[DeltaAnalysisException] { - sql(s"ALTER TABLE $testTable DROP COLUMNS (col1, col3)") - } - checkError( - e, - "DELTA_UNSUPPORTED_DROP_CLUSTERING_COLUMN" - ) - } - } -} - -trait ClusteredTableDDLWithColumnMappingV2Base extends ClusteredTableDDLWithColumnMapping - -trait ClusteredTableDDLWithColumnMappingV2 - extends ClusteredTableDDLWithColumnMappingV2Base - -trait ClusteredTableCreateOrReplaceDDLSuite - extends ClusteredTableCreateOrReplaceDDLSuiteBase - -trait ClusteredTableDDLSuiteBase - extends ClusteredTableCreateOrReplaceDDLSuite - with DeltaSQLCommandTest { - - test("optimize clustered table - error scenarios") { - withClusteredTable(testTable, "a INT, b STRING", "a") { - // Specify partition predicate. - val e = intercept[DeltaUnsupportedOperationException] { - sql(s"OPTIMIZE $testTable WHERE a > 0 and b = foo") - } - checkError( - e, - "DELTA_CLUSTERING_WITH_PARTITION_PREDICATE", - parameters = Map("predicates" -> "a > 0 and b = foo") - ) - - // Specify ZORDER BY. - val e2 = intercept[DeltaAnalysisException] { - sql(s"OPTIMIZE $testTable ZORDER BY (a)") - } - checkError( - e2, - "DELTA_CLUSTERING_WITH_ZORDER_BY" - ) - } - } -} - -trait ClusteredTableDDLSuite extends ClusteredTableDDLSuiteBase -trait ClusteredTableDDLWithNameColumnMapping - extends ClusteredTableCreateOrReplaceDDLSuite with DeltaColumnMappingEnableNameMode - -trait ClusteredTableDDLWithIdColumnMapping - extends ClusteredTableCreateOrReplaceDDLSuite with DeltaColumnMappingEnableIdMode - -trait ClusteredTableDDLWithV2Base - extends ClusteredTableCreateOrReplaceDDLSuite - with SharedSparkSession { - override protected def supportedClauses: Seq[String] = Seq("CREATE", "REPLACE") - - testColTypeValidation("REPLACE") - - test("replace with different clustering columns") { - withTable(sourceTable) { - sql(s"CREATE TABLE $sourceTable(i int, s string) USING delta") - // Validate REPLACE TABLE (AS SELECT). - Seq("REPLACE", "CREATE OR REPLACE").foreach { clause => - Seq(true, false).foreach { isRTAS => - withTempDirIfNecessary { location => - withClusteredTable(testTable, "a int", "a", location = location) { - if (isRTAS) { - createOrReplaceAsSelectClusteredTable( - clause, testTable, sourceTable, "i", location = location) - } else { - createOrReplaceClusteredTable( - clause, testTable, "i int, b string", "i", location = location) - } - verifyClusteringColumns(testTable, "i", location) - } - } - } - } - } - } - - test("Validate replacing clustered tables with partitioned tables is not allowed") { - withTable(sourceTable) { - sql(s"CREATE TABLE $sourceTable(i int, s string) USING delta") - - // Validate REPLACE TABLE (AS SELECT). - Seq("REPLACE", "CREATE OR REPLACE").foreach { clause => - withClusteredTable(testTable, "a int", "a") { - verifyClusteringColumns(TableIdentifier(testTable), "a") - - Seq(true, false).foreach { isRTAS => - val e = intercept[DeltaAnalysisException] { - if (isRTAS) { - sql(s"$clause TABLE $testTable USING delta PARTITIONED BY (i) " + - s"AS SELECT * FROM $sourceTable") - } else { - sql(s"$clause TABLE $testTable (i int, b string) USING delta PARTITIONED BY (i)") - } - } - checkError( - e, - "DELTA_CLUSTERING_REPLACE_TABLE_WITH_PARTITIONED_TABLE" - ) - } - } - } - } - } - - test("Validate replacing partitioned tables with clustered tables is allowed") { - withTable(sourceTable) { - sql(s"CREATE TABLE $sourceTable(i int, s string) USING delta") - - // Validate REPLACE TABLE (AS SELECT). - Seq("REPLACE", "CREATE OR REPLACE").foreach { clause => - Seq(true, false).foreach { isRTAS => - withTable(testTable) { - withTempDirIfNecessary { location => - val locationClause = if (location.isEmpty) "" else s"LOCATION '${location.get}'" - sql(s"CREATE TABLE $testTable USING delta PARTITIONED BY (i) $locationClause" + - s" SELECT 1 i, 'a' s") - verifyPartitionColumns(TableIdentifier(testTable), Seq("i")) - if (isRTAS) { - createOrReplaceAsSelectClusteredTable( - clause, testTable, sourceTable, "i", location = location) - } else { - createOrReplaceClusteredTable( - clause, testTable, "i int, b string", "i", location = location) - } - verifyClusteringColumns(testTable, "i", location) - verifyPartitionColumns(TableIdentifier(testTable), Seq()) - } - } - } - } - } - } - - Seq( - ("", - "a INT, b STRING, ts TIMESTAMP", - "a, b"), - (" multipart name", - "a STRUCT, ts TIMESTAMP", - "a.b, ts") - ).foreach { case (testSuffix, columns, clusteringColumns) => - test(s"create/replace table createOrReplace$testSuffix") { - withTable(testTable) { - // Repeat two times to test both create and replace cases. - (1 to 2).foreach { _ => - createOrReplaceClusteredTable("CREATE OR REPLACE", testTable, columns, clusteringColumns) - verifyClusteringColumns(TableIdentifier(testTable), clusteringColumns) - } - } - } - - test(s"ctas/rtas createOrReplace$testSuffix") { - withTable(sourceTable, targetTable) { - sql(s"CREATE TABLE $sourceTable($columns) USING delta") - withTempDirIfNecessary { location => - // Repeat two times to test both create and replace cases. - (1 to 2).foreach { _ => - createOrReplaceAsSelectClusteredTable( - "CREATE OR REPLACE", targetTable, sourceTable, clusteringColumns, location = location) - verifyClusteringColumns(targetTable, clusteringColumns, location) - } - } - } - } - } -} - -trait ClusteredTableDDLWithV2 - extends ClusteredTableDDLWithV2Base - -trait ClusteredTableDDLDataSourceV2SuiteBase - extends ClusteredTableDDLWithV2 - with ClusteredTableDDLSuite - -class ClusteredTableDDLDataSourceV2Suite - extends ClusteredTableDDLDataSourceV2SuiteBase - -class ClusteredTableDDLDataSourceV2IdColumnMappingSuite - extends ClusteredTableDDLWithIdColumnMapping - with ClusteredTableDDLWithV2 - with ClusteredTableDDLWithColumnMappingV2 - with ClusteredTableDDLSuite - -class ClusteredTableDDLDataSourceV2NameColumnMappingSuite - extends ClusteredTableDDLWithNameColumnMapping - with ClusteredTableDDLWithV2 - with ClusteredTableDDLWithColumnMappingV2 - with ClusteredTableDDLSuite diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/test/DeltaTestImplicits.scala b/spark/src/test/scala/org/apache/spark/sql/delta/test/DeltaTestImplicits.scala index 4363ec6368e..46794c25a68 100644 --- a/spark/src/test/scala/org/apache/spark/sql/delta/test/DeltaTestImplicits.scala +++ b/spark/src/test/scala/org/apache/spark/sql/delta/test/DeltaTestImplicits.scala @@ -22,8 +22,6 @@ import org.apache.spark.sql.delta.{DeltaLog, OptimisticTransaction, Snapshot} import org.apache.spark.sql.delta.DeltaOperations.{ManualUpdate, Operation, Write} import org.apache.spark.sql.delta.actions.{Action, AddFile, Metadata, Protocol} import org.apache.spark.sql.delta.catalog.DeltaTableV2 -import org.apache.spark.sql.delta.commands.optimize.OptimizeMetrics -import org.apache.spark.sql.delta.hooks.AutoCompact import org.apache.spark.sql.delta.stats.StatisticsCollection import org.apache.hadoop.fs.Path @@ -135,18 +133,6 @@ object DeltaTestImplicits { def snapshot: Snapshot = deltaTable.initialSnapshot } - implicit class AutoCompactObjectTestHelper(ac: AutoCompact.type) { - private[delta] def compact( - spark: SparkSession, - deltaLog: DeltaLog, - partitionPredicates: Seq[Expression] = Nil, - opType: String = AutoCompact.OP_TYPE): Seq[OptimizeMetrics] = { - AutoCompact.compact( - spark, deltaLog, catalogTable = None, - partitionPredicates, opType) - } - } - implicit class StatisticsCollectionObjectTestHelper(sc: StatisticsCollection.type) { /**