From 0720bbaf14fcb5aee6f23670609d6ce2966b1091 Mon Sep 17 00:00:00 2001 From: Stas Pak Date: Sun, 6 Oct 2024 20:10:15 -0700 Subject: [PATCH 01/10] Extend OH SQL for snapshots retention --- .../SetSnapshotsRetentionPolicyTest.java | 66 +++++++++++++++++++ .../extensions/OpenhouseSqlExtensions.g4 | 27 +++++++- .../OpenhouseSqlExtensionsAstBuilder.scala | 51 +++++++++++--- .../logical/SetSnapshotsRetentionPolicy.scala | 9 +++ .../v2/OpenhouseDataSourceV2Strategy.scala | 4 +- .../v2/SetSnapshotsRetentionPolicyExec.scala | 45 +++++++++++++ 6 files changed, 192 insertions(+), 10 deletions(-) create mode 100644 integrations/spark/spark-3.1/openhouse-spark-itest/src/test/java/com/linkedin/openhouse/spark/e2e/extensions/SetSnapshotsRetentionPolicyTest.java create mode 100644 integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetSnapshotsRetentionPolicy.scala create mode 100644 integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetSnapshotsRetentionPolicyExec.scala diff --git a/integrations/spark/spark-3.1/openhouse-spark-itest/src/test/java/com/linkedin/openhouse/spark/e2e/extensions/SetSnapshotsRetentionPolicyTest.java b/integrations/spark/spark-3.1/openhouse-spark-itest/src/test/java/com/linkedin/openhouse/spark/e2e/extensions/SetSnapshotsRetentionPolicyTest.java new file mode 100644 index 00000000..643a9f98 --- /dev/null +++ b/integrations/spark/spark-3.1/openhouse-spark-itest/src/test/java/com/linkedin/openhouse/spark/e2e/extensions/SetSnapshotsRetentionPolicyTest.java @@ -0,0 +1,66 @@ +package com.linkedin.openhouse.spark.e2e.extensions; + +import static com.linkedin.openhouse.spark.MockHelpers.*; +import static com.linkedin.openhouse.spark.SparkTestBase.*; + +import com.linkedin.openhouse.spark.SparkTestBase; +import org.apache.iceberg.catalog.TableIdentifier; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(SparkTestBase.class) +public class SetSnapshotsRetentionPolicyTest { + @Test + public void testSetSnapshotsRetentionPolicy() { + Object existingTable = + mockGetTableResponseBody( + "dbSetSnapshotsRetention", + "t1", + "c1", + "dbSetSnapshotsRetention.t1", + "u1", + mockTableLocation( + TableIdentifier.of("dbSetSnapshotsRetention", "t1"), + convertSchemaToDDLComponent(baseSchema), + ""), + "V1", + baseSchema, + null, + null); + Object existingOpenhouseTable = + mockGetTableResponseBody( + "dbSetSnapshotsRetention", + "t1", + "c1", + "dbSetSnapshotsRetention.t1", + "u1", + mockTableLocationAfterOperation( + TableIdentifier.of("dbSetSnapshotsRetention", "t1"), + "ALTER TABLE %t SET TBLPROPERTIES ('openhouse.tableId'='t1')"), + "V1", + baseSchema, + null, + null); + Object tableAfterSetSnapshotsRetention = + mockGetTableResponseBody( + "dbSetSnapshotsRetention", + "t1", + "c1", + "dbSetSnapshotsRetention.t1", + "u1", + mockTableLocationAfterOperation( + TableIdentifier.of("dbSetSnapshotsRetention", "t1"), + "ALTER TABLE %t SET TBLPROPERTIES('policies'='{\"snapshotsRetention\":{\"timeCount\":24,\"granularity\":\"HOUR\",\"count\":10}}')"), + "V2", + baseSchema, + null, + null); + mockTableService.enqueue(mockResponse(200, existingOpenhouseTable)); // doRefresh() + mockTableService.enqueue(mockResponse(200, existingOpenhouseTable)); // doRefresh() + mockTableService.enqueue(mockResponse(201, tableAfterSetSnapshotsRetention)); // doCommit() + String ddlWithSchema = + "ALTER TABLE openhouse.dbSetSnapshotsRetention.t1 SET POLICY (SNAPSHOTS_RETENTION TTL=24H COUNT=10)"; + Assertions.assertDoesNotThrow(() -> spark.sql(ddlWithSchema)); + } +} diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/antlr/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensions.g4 b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/antlr/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensions.g4 index 435a77e6..f5557604 100644 --- a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/antlr/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensions.g4 +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/antlr/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensions.g4 @@ -26,6 +26,7 @@ statement : ALTER TABLE multipartIdentifier SET POLICY '(' retentionPolicy (columnRetentionPolicy)? ')' #setRetentionPolicy | ALTER TABLE multipartIdentifier SET POLICY '(' replicationPolicy ')' #setReplicationPolicy | ALTER TABLE multipartIdentifier SET POLICY '(' sharingPolicy ')' #setSharingPolicy + | ALTER TABLE multipartIdentifier SET POLICY '(' snapshotsRetentionPolicy ')' #setSnapshotsRetentionPolicy | ALTER TABLE multipartIdentifier MODIFY columnNameClause SET columnPolicy #setColumnPolicyTag | GRANT privilege ON grantableResource TO principal #grantStatement | REVOKE privilege ON grantableResource FROM principal #revokeStatement @@ -65,7 +66,7 @@ quotedIdentifier ; nonReserved - : ALTER | TABLE | SET | POLICY | RETENTION | SHARING | REPLICATION + : ALTER | TABLE | SET | POLICY | RETENTION | SHARING | REPLICATION | SNAPSHOTS_RETENTION | GRANT | REVOKE | ON | TO | SHOW | GRANTS | PATTERN | WHERE | COLUMN ; @@ -153,6 +154,27 @@ policyTag : PII | HC ; +snapshotsRetentionPolicy + : SNAPSHOTS_RETENTION snapshotsCombinedRetention + ; + +snapshotsCombinedRetention + : snapshotsTTL (snapshotsCount)? + ; + +snapshotsTTL + : TTL '=' snapshotsTTLValue + ; + +snapshotsCount + : COUNT '=' POSITIVE_INTEGER + ; + +snapshotsTTLValue + : RETENTION_DAY + | RETENTION_HOUR + ; + ALTER: 'ALTER'; TABLE: 'TABLE'; SET: 'SET'; @@ -182,6 +204,9 @@ HC: 'HC'; MODIFY: 'MODIFY'; TAG: 'TAG'; NONE: 'NONE'; +TTL: 'TTL'; +COUNT: 'COUNT'; +SNAPSHOTS_RETENTION : 'SNAPSHOTS_RETENTION'; POSITIVE_INTEGER : DIGIT+ diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensionsAstBuilder.scala b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensionsAstBuilder.scala index 0619f834..5c725a96 100644 --- a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensionsAstBuilder.scala +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensionsAstBuilder.scala @@ -2,7 +2,7 @@ package com.linkedin.openhouse.spark.sql.catalyst.parser.extensions import com.linkedin.openhouse.spark.sql.catalyst.enums.GrantableResourceTypes import com.linkedin.openhouse.spark.sql.catalyst.parser.extensions.OpenhouseSqlExtensionsParser._ -import com.linkedin.openhouse.spark.sql.catalyst.plans.logical.{GrantRevokeStatement, SetColumnPolicyTag, SetReplicationPolicy, SetRetentionPolicy, SetSharingPolicy, ShowGrantsStatement} +import com.linkedin.openhouse.spark.sql.catalyst.plans.logical.{GrantRevokeStatement, SetColumnPolicyTag, SetReplicationPolicy, SetRetentionPolicy, SetSharingPolicy, SetSnapshotsRetentionPolicy, ShowGrantsStatement} import com.linkedin.openhouse.spark.sql.catalyst.enums.GrantableResourceTypes.GrantableResourceType import com.linkedin.openhouse.gen.tables.client.model.TimePartitionSpec import org.antlr.v4.runtime.tree.ParseTree @@ -19,10 +19,11 @@ class OpenhouseSqlExtensionsAstBuilder (delegate: ParserInterface) extends Openh override def visitSetRetentionPolicy(ctx: SetRetentionPolicyContext): SetRetentionPolicy = { val tableName = typedVisit[Seq[String]](ctx.multipartIdentifier) - val (granularity, count) = typedVisit[(String, Int)](ctx.retentionPolicy()) + val retentionPolicy = ctx.retentionPolicy() + val (granularity, count) = typedVisit[(String, Int)](retentionPolicy) val (colName, colPattern) = if (ctx.columnRetentionPolicy() != null) - typedVisit[(String, String)](ctx.columnRetentionPolicy()) + typedVisit[(String, String)](ctx.columnRetentionPolicy()) else (null, null) SetRetentionPolicy(tableName, granularity, count, Option(colName), Option(colPattern)) } @@ -128,8 +129,8 @@ class OpenhouseSqlExtensionsAstBuilder (delegate: ParserInterface) extends Openh } } - override def visitColumnRetentionPolicyPatternClause(ctx: ColumnRetentionPolicyPatternClauseContext): (String) = { - (ctx.retentionColumnPatternClause().STRING().getText) + override def visitColumnRetentionPolicyPatternClause(ctx: ColumnRetentionPolicyPatternClauseContext): String = { + ctx.retentionColumnPatternClause().STRING().getText } override def visitSharingPolicy(ctx: SharingPolicyContext): String = { @@ -149,7 +150,7 @@ class OpenhouseSqlExtensionsAstBuilder (delegate: ParserInterface) extends Openh } override def visitDuration(ctx: DurationContext): (String, Int) = { - val granularity = if (ctx.RETENTION_DAY != null) { + val granularity: String = if (ctx.RETENTION_DAY != null) { TimePartitionSpec.GranularityEnum.DAY.getValue() } else if (ctx.RETENTION_YEAR() != null) { TimePartitionSpec.GranularityEnum.YEAR.getValue() @@ -158,13 +159,47 @@ class OpenhouseSqlExtensionsAstBuilder (delegate: ParserInterface) extends Openh } else { TimePartitionSpec.GranularityEnum.HOUR.getValue() } - val count = ctx.getText.substring(0, ctx.getText.length - 1).toInt (granularity, count) } + override def visitSetSnapshotsRetentionPolicy(ctx: SetSnapshotsRetentionPolicyContext): SetSnapshotsRetentionPolicy = { + val tableName = typedVisit[Seq[String]](ctx.multipartIdentifier) + val (granularity, timeCount, count) = typedVisit[(String, Int, Int)](ctx.snapshotsRetentionPolicy()) + SetSnapshotsRetentionPolicy(tableName, granularity, timeCount, count) + } + + override def visitSnapshotsRetentionPolicy(ctx: SnapshotsRetentionPolicyContext): (String, Int, Int) = { + typedVisit[(String, Int, Int)](ctx.snapshotsCombinedRetention()) + } + + override def visitSnapshotsCombinedRetention(ctx: SnapshotsCombinedRetentionContext): (String, Int, Int) = { + val snapshotsTTL = ctx.snapshotsTTL() + val (granularity, timeCount) = typedVisit[(String, Int)](snapshotsTTL) + val count = + if (ctx.snapshotsCount() != null) { + typedVisit[Int](ctx.snapshotsCount()) + } else 0 + (granularity, timeCount, count) + } + + override def visitSnapshotsTTL(ctx: SnapshotsTTLContext): (String, Int) = { + val ttl = ctx.snapshotsTTLValue() + val granularity: String = if (ttl.RETENTION_DAY() != null) { + TimePartitionSpec.GranularityEnum.DAY.getValue() + } else { + TimePartitionSpec.GranularityEnum.HOUR.getValue() + } + val count = ttl.getText.substring(0, ttl.getText.length - 1).toInt + (granularity, count) + } + + override def visitSnapshotsCount(ctx: SnapshotsCountContext): Integer = { + ctx.POSITIVE_INTEGER().getText.toInt + } + private def toBuffer[T](list: java.util.List[T]) = list.asScala - private def toSeq[T](list: java.util.List[T]): Seq[T] = toBuffer(list).toSeq + private def toSeq[T](list: java.util.List[T]) = toBuffer(list).toSeq private def typedVisit[T](ctx: ParseTree): T = { ctx.accept(this).asInstanceOf[T] diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetSnapshotsRetentionPolicy.scala b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetSnapshotsRetentionPolicy.scala new file mode 100644 index 00000000..46e16d92 --- /dev/null +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetSnapshotsRetentionPolicy.scala @@ -0,0 +1,9 @@ +package com.linkedin.openhouse.spark.sql.catalyst.plans.logical + +import org.apache.spark.sql.catalyst.plans.logical.Command + +case class SetSnapshotsRetentionPolicy (tableName: Seq[String], granularity: String, timeCount: Int, count: Int) extends Command { + override def simpleString(maxFields: Int): String = { + s"SetSnapshotsRetentionPolicy: ${tableName} ${timeCount} ${granularity} ${count}" + } +} diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/OpenhouseDataSourceV2Strategy.scala b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/OpenhouseDataSourceV2Strategy.scala index 8545a2bc..db8c6304 100644 --- a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/OpenhouseDataSourceV2Strategy.scala +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/OpenhouseDataSourceV2Strategy.scala @@ -1,6 +1,6 @@ package com.linkedin.openhouse.spark.sql.execution.datasources.v2 -import com.linkedin.openhouse.spark.sql.catalyst.plans.logical.{GrantRevokeStatement, SetColumnPolicyTag, SetReplicationPolicy, SetRetentionPolicy, SetSharingPolicy, ShowGrantsStatement} +import com.linkedin.openhouse.spark.sql.catalyst.plans.logical.{GrantRevokeStatement, SetColumnPolicyTag, SetReplicationPolicy, SetRetentionPolicy, SetSharingPolicy, SetSnapshotsRetentionPolicy, ShowGrantsStatement} import org.apache.iceberg.spark.{Spark3Util, SparkCatalog, SparkSessionCatalog} import org.apache.spark.sql.{SparkSession, Strategy} import org.apache.spark.sql.catalyst.expressions.PredicateHelper @@ -17,6 +17,8 @@ case class OpenhouseDataSourceV2Strategy(spark: SparkSession) extends Strategy w SetRetentionPolicyExec(catalog, ident, granularity, count, colName, colPattern) :: Nil case SetReplicationPolicy(CatalogAndIdentifierExtractor(catalog, ident), replicationPolicies) => SetReplicationPolicyExec(catalog, ident, replicationPolicies) :: Nil + case SetSnapshotsRetentionPolicy(CatalogAndIdentifierExtractor(catalog, ident), granularity, timeCount, count) => + SetSnapshotsRetentionPolicyExec(catalog, ident, granularity, timeCount, count) :: Nil case SetSharingPolicy(CatalogAndIdentifierExtractor(catalog, ident), sharing) => SetSharingPolicyExec(catalog, ident, sharing) :: Nil case SetColumnPolicyTag(CatalogAndIdentifierExtractor(catalog, ident), policyTag, cols) => diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetSnapshotsRetentionPolicyExec.scala b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetSnapshotsRetentionPolicyExec.scala new file mode 100644 index 00000000..183d8621 --- /dev/null +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetSnapshotsRetentionPolicyExec.scala @@ -0,0 +1,45 @@ +package com.linkedin.openhouse.spark.sql.execution.datasources.v2 + +import org.apache.iceberg.spark.source.SparkTable +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} +import org.apache.spark.sql.execution.datasources.v2.V2CommandExec + +case class SetSnapshotsRetentionPolicyExec( + catalog: TableCatalog, + ident: Identifier, + granularity: String, + timeCount: Int, + count: Int +) extends V2CommandExec { + + override lazy val output: Seq[Attribute] = Nil + + override protected def run(): Seq[InternalRow] = { + catalog.loadTable(ident) match { + case iceberg: SparkTable if iceberg.table().properties().containsKey("openhouse.tableId") => + val key = "updated.openhouse.policy" + val value = { + (count) match { + case (0) => s"""{"snapshotsRetention":{"timeCount":${timeCount},"granularity":"${granularity}"}}""" + case (_) => + s"""{"snapshotsRetention":{"timeCount":${timeCount}, "granularity":"${granularity}", "count":${count}}}""" + } + } + + iceberg.table().updateProperties() + .set(key, value) + .commit() + + case table => + throw new UnsupportedOperationException(s"Cannot set snapshots retention policy for non-Openhouse table: $table") + } + + Nil + } + + override def simpleString(maxFields: Int): String = { + s"SetSnapshotsRetentionPolicyExec: ${catalog} ${ident} ${timeCount} ${granularity} ${count}" + } +} From bc2ac040f5a50f6b67bcf1916713cdb43e513d8a Mon Sep 17 00:00:00 2001 From: Will-Lo Date: Sat, 23 Nov 2024 00:18:28 -0500 Subject: [PATCH 02/10] WIP updated SQL grammar for configurable table retention --- .../extensions/OpenhouseSqlExtensions.g4 | 31 ++++++----- .../sql/catalyst/enums/LogicalOperators.scala | 6 +++ .../OpenhouseSqlExtensionsAstBuilder.scala | 54 ++++++------------- .../logical/SetSnapshotsRetentionPolicy.scala | 5 +- .../v2/OpenhouseDataSourceV2Strategy.scala | 4 +- .../v2/SetSnapshotsRetentionPolicyExec.scala | 12 +++-- 6 files changed, 52 insertions(+), 60 deletions(-) create mode 100644 integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/enums/LogicalOperators.scala diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/antlr/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensions.g4 b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/antlr/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensions.g4 index f5557604..77d1ff32 100644 --- a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/antlr/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensions.g4 +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/antlr/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensions.g4 @@ -26,11 +26,11 @@ statement : ALTER TABLE multipartIdentifier SET POLICY '(' retentionPolicy (columnRetentionPolicy)? ')' #setRetentionPolicy | ALTER TABLE multipartIdentifier SET POLICY '(' replicationPolicy ')' #setReplicationPolicy | ALTER TABLE multipartIdentifier SET POLICY '(' sharingPolicy ')' #setSharingPolicy - | ALTER TABLE multipartIdentifier SET POLICY '(' snapshotsRetentionPolicy ')' #setSnapshotsRetentionPolicy + | ALTER TABLE multipartIdentifier SET POLICY snapshotsRetentionPolicy #setSnapshotsRetentionPolicy | ALTER TABLE multipartIdentifier MODIFY columnNameClause SET columnPolicy #setColumnPolicyTag | GRANT privilege ON grantableResource TO principal #grantStatement | REVOKE privilege ON grantableResource FROM principal #revokeStatement - | SHOW GRANTS ON grantableResource #showGrantsStatement + | SHOW GRANTS ON grantableResource #showGrantsStatement ; multipartIdentifier @@ -66,7 +66,7 @@ quotedIdentifier ; nonReserved - : ALTER | TABLE | SET | POLICY | RETENTION | SHARING | REPLICATION | SNAPSHOTS_RETENTION + : ALTER | TABLE | SET | POLICY | RETENTION | SHARING | REPLICATION | RETAIN_SNAPSHOTS | GRANT | REVOKE | ON | TO | SHOW | GRANTS | PATTERN | WHERE | COLUMN ; @@ -77,6 +77,11 @@ sharingPolicy BOOLEAN : 'TRUE' | 'FALSE' ; + +AND_OR_LOGICAL_OPERATOR + : 'AND' | 'OR' + ; + retentionPolicy : RETENTION '=' duration ; @@ -155,24 +160,22 @@ policyTag ; snapshotsRetentionPolicy - : SNAPSHOTS_RETENTION snapshotsCombinedRetention - ; - -snapshotsCombinedRetention - : snapshotsTTL (snapshotsCount)? + : RETAIN_SNAPSHOTS snapshotsTTL + | RETAIN_SNAPSHOTS snapshotsCount + | RETAIN_SNAPSHOTS snapshotsTTL AND_OR_LOGICAL_OPERATOR snapshotsCount ; snapshotsTTL - : TTL '=' snapshotsTTLValue + : dateGranularity'='POSITIVE_INTEGER ; snapshotsCount - : COUNT '=' POSITIVE_INTEGER + : COUNT'='POSITIVE_INTEGER ; -snapshotsTTLValue - : RETENTION_DAY - | RETENTION_HOUR +dateGranularity + : 'DAYS' + | 'HOURS' ; ALTER: 'ALTER'; @@ -204,9 +207,9 @@ HC: 'HC'; MODIFY: 'MODIFY'; TAG: 'TAG'; NONE: 'NONE'; +RETAIN_SNAPSHOTS: 'RETAIN SNAPSHOTS'; TTL: 'TTL'; COUNT: 'COUNT'; -SNAPSHOTS_RETENTION : 'SNAPSHOTS_RETENTION'; POSITIVE_INTEGER : DIGIT+ diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/enums/LogicalOperators.scala b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/enums/LogicalOperators.scala new file mode 100644 index 00000000..680a5cc9 --- /dev/null +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/enums/LogicalOperators.scala @@ -0,0 +1,6 @@ +package com.linkedin.openhouse.spark.sql.catalyst.enums + +private[sql] object LogicalOperators extends Enumeration { + type LogicalOperatorsType = Value + val AND, OR = Value +} diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensionsAstBuilder.scala b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensionsAstBuilder.scala index 5c725a96..6b5f15f3 100644 --- a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensionsAstBuilder.scala +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensionsAstBuilder.scala @@ -1,9 +1,10 @@ package com.linkedin.openhouse.spark.sql.catalyst.parser.extensions -import com.linkedin.openhouse.spark.sql.catalyst.enums.GrantableResourceTypes +import com.linkedin.openhouse.spark.sql.catalyst.enums.{GrantableResourceTypes, LogicalOperators} import com.linkedin.openhouse.spark.sql.catalyst.parser.extensions.OpenhouseSqlExtensionsParser._ import com.linkedin.openhouse.spark.sql.catalyst.plans.logical.{GrantRevokeStatement, SetColumnPolicyTag, SetReplicationPolicy, SetRetentionPolicy, SetSharingPolicy, SetSnapshotsRetentionPolicy, ShowGrantsStatement} import com.linkedin.openhouse.spark.sql.catalyst.enums.GrantableResourceTypes.GrantableResourceType +import com.linkedin.openhouse.spark.sql.catalyst.enums.LogicalOperators.LogicalOperatorsType import com.linkedin.openhouse.gen.tables.client.model.TimePartitionSpec import org.antlr.v4.runtime.tree.ParseTree import org.apache.spark.sql.catalyst.parser.ParserInterface @@ -149,48 +150,27 @@ class OpenhouseSqlExtensionsAstBuilder (delegate: ParserInterface) extends Openh toSeq(ctx.policyTag()).map(_.getText) } - override def visitDuration(ctx: DurationContext): (String, Int) = { - val granularity: String = if (ctx.RETENTION_DAY != null) { - TimePartitionSpec.GranularityEnum.DAY.getValue() - } else if (ctx.RETENTION_YEAR() != null) { - TimePartitionSpec.GranularityEnum.YEAR.getValue() - } else if (ctx.RETENTION_MONTH() != null) { - TimePartitionSpec.GranularityEnum.MONTH.getValue() - } else { - TimePartitionSpec.GranularityEnum.HOUR.getValue() - } - val count = ctx.getText.substring(0, ctx.getText.length - 1).toInt - (granularity, count) - } - override def visitSetSnapshotsRetentionPolicy(ctx: SetSnapshotsRetentionPolicyContext): SetSnapshotsRetentionPolicy = { val tableName = typedVisit[Seq[String]](ctx.multipartIdentifier) - val (granularity, timeCount, count) = typedVisit[(String, Int, Int)](ctx.snapshotsRetentionPolicy()) - SetSnapshotsRetentionPolicy(tableName, granularity, timeCount, count) - } - - override def visitSnapshotsRetentionPolicy(ctx: SnapshotsRetentionPolicyContext): (String, Int, Int) = { - typedVisit[(String, Int, Int)](ctx.snapshotsCombinedRetention()) - } - - override def visitSnapshotsCombinedRetention(ctx: SnapshotsCombinedRetentionContext): (String, Int, Int) = { - val snapshotsTTL = ctx.snapshotsTTL() - val (granularity, timeCount) = typedVisit[(String, Int)](snapshotsTTL) - val count = - if (ctx.snapshotsCount() != null) { + val (logicalOperator, granularity, timeCount, count) = typedVisit[(Option[LogicalOperatorsType], Option[String], Int, Int)](ctx.snapshotsRetentionPolicy()) + SetSnapshotsRetentionPolicy(tableName, logicalOperator, granularity, timeCount, count) + } + override def visitSnapshotsRetentionPolicy(ctx: SnapshotsRetentionPolicyContext): (Option[LogicalOperatorsType], Option[String], Int, Int) = { + val logicalOperator = if (ctx.AND_OR_LOGICAL_OPERATOR() != null) + LogicalOperators.withName(ctx.AND_OR_LOGICAL_OPERATOR().getText) + else null + val timePolicy = if (ctx.snapshotsTTL() != null) + typedVisit[(String, Int)](ctx.snapshotsTTL()) + else (null, -1) + val countPolicy = if (ctx.snapshotsCount() != null) typedVisit[Int](ctx.snapshotsCount()) - } else 0 - (granularity, timeCount, count) + else -1 + (Option(logicalOperator), Option(timePolicy._1), timePolicy._2, countPolicy) } override def visitSnapshotsTTL(ctx: SnapshotsTTLContext): (String, Int) = { - val ttl = ctx.snapshotsTTLValue() - val granularity: String = if (ttl.RETENTION_DAY() != null) { - TimePartitionSpec.GranularityEnum.DAY.getValue() - } else { - TimePartitionSpec.GranularityEnum.HOUR.getValue() - } - val count = ttl.getText.substring(0, ttl.getText.length - 1).toInt + val granularity = ctx.dateGranularity().getText + val count = ctx.POSITIVE_INTEGER().getText.toInt (granularity, count) } diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetSnapshotsRetentionPolicy.scala b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetSnapshotsRetentionPolicy.scala index 46e16d92..e38752be 100644 --- a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetSnapshotsRetentionPolicy.scala +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetSnapshotsRetentionPolicy.scala @@ -1,9 +1,10 @@ package com.linkedin.openhouse.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.Command +import com.linkedin.openhouse.spark.sql.catalyst.enums.LogicalOperators.LogicalOperatorsType -case class SetSnapshotsRetentionPolicy (tableName: Seq[String], granularity: String, timeCount: Int, count: Int) extends Command { +case class SetSnapshotsRetentionPolicy (tableName: Seq[String], logicalOperator: Option[LogicalOperatorsType], granularity: Option[String], timeCount: Int, count: Int) extends Command { override def simpleString(maxFields: Int): String = { - s"SetSnapshotsRetentionPolicy: ${tableName} ${timeCount} ${granularity} ${count}" + s"SetSnapshotsRetentionPolicy: ${tableName} ${if (timeCount > 0) timeCount else ""} ${granularity.getOrElse("")} ${logicalOperator.getOrElse("")} count ${if (count > 0) count else ""}" } } diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/OpenhouseDataSourceV2Strategy.scala b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/OpenhouseDataSourceV2Strategy.scala index db8c6304..5e36445f 100644 --- a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/OpenhouseDataSourceV2Strategy.scala +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/OpenhouseDataSourceV2Strategy.scala @@ -17,8 +17,8 @@ case class OpenhouseDataSourceV2Strategy(spark: SparkSession) extends Strategy w SetRetentionPolicyExec(catalog, ident, granularity, count, colName, colPattern) :: Nil case SetReplicationPolicy(CatalogAndIdentifierExtractor(catalog, ident), replicationPolicies) => SetReplicationPolicyExec(catalog, ident, replicationPolicies) :: Nil - case SetSnapshotsRetentionPolicy(CatalogAndIdentifierExtractor(catalog, ident), granularity, timeCount, count) => - SetSnapshotsRetentionPolicyExec(catalog, ident, granularity, timeCount, count) :: Nil + case SetSnapshotsRetentionPolicy(CatalogAndIdentifierExtractor(catalog, ident), logicalOperator, granularity, timeCount, count) => + SetSnapshotsRetentionPolicyExec(catalog, ident, logicalOperator, granularity, timeCount, count) :: Nil case SetSharingPolicy(CatalogAndIdentifierExtractor(catalog, ident), sharing) => SetSharingPolicyExec(catalog, ident, sharing) :: Nil case SetColumnPolicyTag(CatalogAndIdentifierExtractor(catalog, ident), policyTag, cols) => diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetSnapshotsRetentionPolicyExec.scala b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetSnapshotsRetentionPolicyExec.scala index 183d8621..02c32bcb 100644 --- a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetSnapshotsRetentionPolicyExec.scala +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetSnapshotsRetentionPolicyExec.scala @@ -1,5 +1,6 @@ package com.linkedin.openhouse.spark.sql.execution.datasources.v2 +import com.linkedin.openhouse.spark.sql.catalyst.enums.LogicalOperators.LogicalOperatorsType import org.apache.iceberg.spark.source.SparkTable import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute @@ -7,11 +8,12 @@ import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} import org.apache.spark.sql.execution.datasources.v2.V2CommandExec case class SetSnapshotsRetentionPolicyExec( - catalog: TableCatalog, - ident: Identifier, - granularity: String, - timeCount: Int, - count: Int + catalog: TableCatalog, + ident: Identifier, + logicalOperator: Option[LogicalOperatorsType], + granularity: Option[String], + timeCount: Int, + count: Int ) extends V2CommandExec { override lazy val output: Seq[Attribute] = Nil From 300db92e09657678ad301b837bb03ca3d7b94e35 Mon Sep 17 00:00:00 2001 From: Will-Lo Date: Mon, 25 Nov 2024 20:33:21 -0500 Subject: [PATCH 03/10] Formatting fix --- .../SetVersionsRetentionPolicyTest.java | 231 ++++++++++++++++++ .../extensions/OpenhouseSqlExtensions.g4 | 29 ++- .../OpenhouseSqlExtensionsAstBuilder.scala | 40 +-- .../logical/SetVersionsRetentionPolicy.scala | 10 + .../v2/OpenhouseDataSourceV2Strategy.scala | 6 +- .../v2/SetSnapshotsRetentionPolicyExec.scala | 14 +- .../v2/SetVersionsRetentionPolicyExec.scala | 47 ++++ 7 files changed, 336 insertions(+), 41 deletions(-) create mode 100644 integrations/spark/spark-3.1/openhouse-spark-itest/src/test/java/com/linkedin/openhouse/spark/e2e/extensions/SetVersionsRetentionPolicyTest.java create mode 100644 integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetVersionsRetentionPolicy.scala create mode 100644 integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetVersionsRetentionPolicyExec.scala diff --git a/integrations/spark/spark-3.1/openhouse-spark-itest/src/test/java/com/linkedin/openhouse/spark/e2e/extensions/SetVersionsRetentionPolicyTest.java b/integrations/spark/spark-3.1/openhouse-spark-itest/src/test/java/com/linkedin/openhouse/spark/e2e/extensions/SetVersionsRetentionPolicyTest.java new file mode 100644 index 00000000..df03093e --- /dev/null +++ b/integrations/spark/spark-3.1/openhouse-spark-itest/src/test/java/com/linkedin/openhouse/spark/e2e/extensions/SetVersionsRetentionPolicyTest.java @@ -0,0 +1,231 @@ +package com.linkedin.openhouse.spark.e2e.extensions; + +import static com.linkedin.openhouse.spark.MockHelpers.*; +import static com.linkedin.openhouse.spark.SparkTestBase.*; + +import com.linkedin.openhouse.spark.SparkTestBase; +import org.apache.iceberg.catalog.TableIdentifier; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(SparkTestBase.class) +public class SetVersionsRetentionPolicyTest { + @Test + public void testSetVersionsRetentionPolicyTimeAndCount() { + Object existingTable = + mockGetTableResponseBody( + "dbSetVersionsRetention", + "t1", + "c1", + "dbSetVersionsRetention.t1", + "u1", + mockTableLocation( + TableIdentifier.of("dbSetVersionsRetention", "t1"), + convertSchemaToDDLComponent(baseSchema), + ""), + "V1", + baseSchema, + null, + null); + Object existingOpenhouseTable = + mockGetTableResponseBody( + "dbSetVersionsRetention", + "t1", + "c1", + "dbSetVersionsRetention.t1", + "u1", + mockTableLocationAfterOperation( + TableIdentifier.of("dbSetVersionsRetention", "t1"), + "ALTER TABLE %t SET TBLPROPERTIES ('openhouse.tableId'='t1')"), + "V1", + baseSchema, + null, + null); + Object tableAfterSetversionsRetention = + mockGetTableResponseBody( + "dbSetVersionsRetention", + "t1", + "c1", + "dbSetVersionsRetention.t1", + "u1", + mockTableLocationAfterOperation( + TableIdentifier.of("dbSetVersionsRetention", "t1"), + "ALTER TABLE %t SET TBLPROPERTIES('policies'='{\"versionsRetention\":{\"timeCount\":24,\"granularity\":" + + "\"HOUR\",\"count\":10, \"logicalOperator\": " + + "\"AND\"}}')"), + "V2", + baseSchema, + null, + null); + mockTableService.enqueue(mockResponse(200, existingOpenhouseTable)); // doRefresh() + mockTableService.enqueue(mockResponse(200, existingOpenhouseTable)); // doRefresh() + mockTableService.enqueue(mockResponse(201, tableAfterSetversionsRetention)); // doCommit() + String ddlWithSchema = + "ALTER TABLE openhouse.dbSetversionsRetention.t1 SET POLICY (VERSIONS TIME=24H AND COUNT=10)"; + Assertions.assertDoesNotThrow(() -> spark.sql(ddlWithSchema)); + } + + @Test + public void testSetversionsRetentionPolicyTimeOrCount() { + Object existingTable = + mockGetTableResponseBody( + "dbSetVersionsRetention", + "t2", + "c1", + "dbSetVersionsRetention.t2", + "u1", + mockTableLocation( + TableIdentifier.of("dbSetVersionsRetention", "t2"), + convertSchemaToDDLComponent(baseSchema), + ""), + "V1", + baseSchema, + null, + null); + Object existingOpenhouseTable = + mockGetTableResponseBody( + "dbSetVersionsRetention", + "t2", + "c1", + "dbSetVersionsRetention.t2", + "u1", + mockTableLocationAfterOperation( + TableIdentifier.of("dbSetVersionsRetention", "t2"), + "ALTER TABLE %t SET TBLPROPERTIES ('openhouse.tableId'='t2')"), + "V1", + baseSchema, + null, + null); + Object tableAfterSetversionsRetention = + mockGetTableResponseBody( + "dbSetVersionsRetention", + "t1", + "c1", + "dbSetVersionsRetention.t2", + "u1", + mockTableLocationAfterOperation( + TableIdentifier.of("dbSetVersionsRetention", "t2"), + "ALTER TABLE %t SET TBLPROPERTIES('policies'='{\"versionsRetention\":{\"timeCount\":24,\"granularity\":" + + "\"HOUR\",\"count\":10, \"logicalOperator\": " + + "\"OR\"}}')"), + "V2", + baseSchema, + null, + null); + mockTableService.enqueue(mockResponse(200, existingOpenhouseTable)); // doRefresh() + mockTableService.enqueue(mockResponse(200, existingOpenhouseTable)); // doRefresh() + mockTableService.enqueue(mockResponse(201, tableAfterSetversionsRetention)); // doCommit() + String ddlWithSchema = + "ALTER TABLE openhouse.dbSetversionsRetention.t2 SET POLICY (VERSIONS TIME=24H OR COUNT=10)"; + Assertions.assertDoesNotThrow(() -> spark.sql(ddlWithSchema)); + } + + @Test + public void testSetversionsRetentionPolicyTimeOnly() { + Object existingTable = + mockGetTableResponseBody( + "dbSetVersionsRetention", + "t3", + "c1", + "dbSetVersionsRetention.t3", + "u1", + mockTableLocation( + TableIdentifier.of("dbSetVersionsRetention", "t3"), + convertSchemaToDDLComponent(baseSchema), + ""), + "V1", + baseSchema, + null, + null); + Object existingOpenhouseTable = + mockGetTableResponseBody( + "dbSetVersionsRetention", + "t3", + "c1", + "dbSetVersionsRetention.t3", + "u1", + mockTableLocationAfterOperation( + TableIdentifier.of("dbSetVersionsRetention", "t3"), + "ALTER TABLE %t SET TBLPROPERTIES ('openhouse.tableId'='t3')"), + "V1", + baseSchema, + null, + null); + Object tableAfterSetversionsRetention = + mockGetTableResponseBody( + "dbSetVersionsRetention", + "t3", + "c1", + "dbSetVersionsRetention.t3", + "u1", + mockTableLocationAfterOperation( + TableIdentifier.of("dbSetVersionsRetention", "t3"), + "ALTER TABLE %t SET TBLPROPERTIES('policies'='{\"versionsRetention\":{\"timeCount\":24,\"granularity\":" + + "\"HOUR\"}}')"), + "V2", + baseSchema, + null, + null); + mockTableService.enqueue(mockResponse(200, existingOpenhouseTable)); // doRefresh() + mockTableService.enqueue(mockResponse(200, existingOpenhouseTable)); // doRefresh() + mockTableService.enqueue(mockResponse(201, tableAfterSetversionsRetention)); // doCommit() + String ddlWithSchema = + "ALTER TABLE openhouse.dbSetversionsRetention.t3 SET POLICY (VERSIONS TIME=24H)"; + Assertions.assertDoesNotThrow(() -> spark.sql(ddlWithSchema)); + } + + @Test + public void testSetversionsRetentionPolicyCountOnly() { + Object existingTable = + mockGetTableResponseBody( + "dbSetVersionsRetention", + "t4", + "c1", + "dbSetVersionsRetention.t4", + "u1", + mockTableLocation( + TableIdentifier.of("dbSetVersionsRetention", "t4"), + convertSchemaToDDLComponent(baseSchema), + ""), + "V1", + baseSchema, + null, + null); + Object existingOpenhouseTable = + mockGetTableResponseBody( + "dbSetVersionsRetention", + "t4", + "c1", + "dbSetVersionsRetention.t4", + "u1", + mockTableLocationAfterOperation( + TableIdentifier.of("dbSetVersionsRetention", "t4"), + "ALTER TABLE %t SET TBLPROPERTIES ('openhouse.tableId'='t4')"), + "V1", + baseSchema, + null, + null); + Object tableAfterSetversionsRetention = + mockGetTableResponseBody( + "dbSetVersionsRetention", + "t4", + "c1", + "dbSetVersionsRetention.t4", + "u1", + mockTableLocationAfterOperation( + TableIdentifier.of("dbSetVersionsRetention", "t4"), + "ALTER TABLE %t SET TBLPROPERTIES('policies'='{\"versionsRetention\":{\"timeCount\":24,\"granularity\":" + + "\"HOUR\"}}')"), + "V2", + baseSchema, + null, + null); + mockTableService.enqueue(mockResponse(200, existingOpenhouseTable)); // doRefresh() + mockTableService.enqueue(mockResponse(200, existingOpenhouseTable)); // doRefresh() + mockTableService.enqueue(mockResponse(201, tableAfterSetversionsRetention)); // doCommit() + String ddlWithSchema = + "ALTER TABLE openhouse.dbSetversionsRetention.t4 SET POLICY (VERSIONS COUNT=10)"; + Assertions.assertDoesNotThrow(() -> spark.sql(ddlWithSchema)); + } +} diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/antlr/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensions.g4 b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/antlr/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensions.g4 index 77d1ff32..c1b03344 100644 --- a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/antlr/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensions.g4 +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/antlr/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensions.g4 @@ -26,7 +26,7 @@ statement : ALTER TABLE multipartIdentifier SET POLICY '(' retentionPolicy (columnRetentionPolicy)? ')' #setRetentionPolicy | ALTER TABLE multipartIdentifier SET POLICY '(' replicationPolicy ')' #setReplicationPolicy | ALTER TABLE multipartIdentifier SET POLICY '(' sharingPolicy ')' #setSharingPolicy - | ALTER TABLE multipartIdentifier SET POLICY snapshotsRetentionPolicy #setSnapshotsRetentionPolicy + | ALTER TABLE multipartIdentifier SET POLICY '(' versionsRetentionPolicy ')' #setVersionsRetentionPolicy | ALTER TABLE multipartIdentifier MODIFY columnNameClause SET columnPolicy #setColumnPolicyTag | GRANT privilege ON grantableResource TO principal #grantStatement | REVOKE privilege ON grantableResource FROM principal #revokeStatement @@ -66,7 +66,11 @@ quotedIdentifier ; nonReserved +<<<<<<< HEAD:integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/antlr/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensions.g4 : ALTER | TABLE | SET | POLICY | RETENTION | SHARING | REPLICATION | RETAIN_SNAPSHOTS +======= + : ALTER | TABLE | SET | POLICY | RETENTION | SHARING | VERSIONS +>>>>>>> 3b7388d (Formatting fix):integrations/spark/openhouse-spark-runtime/src/main/antlr/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensions.g4 | GRANT | REVOKE | ON | TO | SHOW | GRANTS | PATTERN | WHERE | COLUMN ; @@ -159,25 +163,20 @@ policyTag : PII | HC ; -snapshotsRetentionPolicy - : RETAIN_SNAPSHOTS snapshotsTTL - | RETAIN_SNAPSHOTS snapshotsCount - | RETAIN_SNAPSHOTS snapshotsTTL AND_OR_LOGICAL_OPERATOR snapshotsCount +versionsRetentionPolicy + : VERSIONS versionsTime + | VERSIONS versionsCount + | VERSIONS versionsTime AND_OR_LOGICAL_OPERATOR versionsCount ; -snapshotsTTL - : dateGranularity'='POSITIVE_INTEGER +versionsTime + : TIME'='duration ; -snapshotsCount +versionsCount : COUNT'='POSITIVE_INTEGER ; -dateGranularity - : 'DAYS' - | 'HOURS' - ; - ALTER: 'ALTER'; TABLE: 'TABLE'; SET: 'SET'; @@ -207,8 +206,8 @@ HC: 'HC'; MODIFY: 'MODIFY'; TAG: 'TAG'; NONE: 'NONE'; -RETAIN_SNAPSHOTS: 'RETAIN SNAPSHOTS'; -TTL: 'TTL'; +VERSIONS: 'VERSIONS'; +TIME: 'TIME'; COUNT: 'COUNT'; POSITIVE_INTEGER diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensionsAstBuilder.scala b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensionsAstBuilder.scala index 6b5f15f3..71aea4e5 100644 --- a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensionsAstBuilder.scala +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensionsAstBuilder.scala @@ -2,7 +2,7 @@ package com.linkedin.openhouse.spark.sql.catalyst.parser.extensions import com.linkedin.openhouse.spark.sql.catalyst.enums.{GrantableResourceTypes, LogicalOperators} import com.linkedin.openhouse.spark.sql.catalyst.parser.extensions.OpenhouseSqlExtensionsParser._ -import com.linkedin.openhouse.spark.sql.catalyst.plans.logical.{GrantRevokeStatement, SetColumnPolicyTag, SetReplicationPolicy, SetRetentionPolicy, SetSharingPolicy, SetSnapshotsRetentionPolicy, ShowGrantsStatement} +import com.linkedin.openhouse.spark.sql.catalyst.plans.logical.{GrantRevokeStatement, SetColumnPolicyTag, SetReplicationPolicy, SetRetentionPolicy, SetSharingPolicy, SetVersionsRetentionPolicy, ShowGrantsStatement} import com.linkedin.openhouse.spark.sql.catalyst.enums.GrantableResourceTypes.GrantableResourceType import com.linkedin.openhouse.spark.sql.catalyst.enums.LogicalOperators.LogicalOperatorsType import com.linkedin.openhouse.gen.tables.client.model.TimePartitionSpec @@ -150,31 +150,39 @@ class OpenhouseSqlExtensionsAstBuilder (delegate: ParserInterface) extends Openh toSeq(ctx.policyTag()).map(_.getText) } - override def visitSetSnapshotsRetentionPolicy(ctx: SetSnapshotsRetentionPolicyContext): SetSnapshotsRetentionPolicy = { + override def visitDuration(ctx: DurationContext): (String, Int) = { + val granularity: String = if (ctx.RETENTION_DAY != null) { + TimePartitionSpec.GranularityEnum.DAY.getValue() + } else if (ctx.RETENTION_YEAR() != null) { + TimePartitionSpec.GranularityEnum.YEAR.getValue() + } else if (ctx.RETENTION_MONTH() != null) { + TimePartitionSpec.GranularityEnum.MONTH.getValue() + } else { + TimePartitionSpec.GranularityEnum.HOUR.getValue() + } + val count = ctx.getText.substring(0, ctx.getText.length - 1).toInt + (granularity, count) + } + + override def visitSetVersionsRetentionPolicy(ctx: SetVersionsRetentionPolicyContext): SetVersionsRetentionPolicy = { val tableName = typedVisit[Seq[String]](ctx.multipartIdentifier) - val (logicalOperator, granularity, timeCount, count) = typedVisit[(Option[LogicalOperatorsType], Option[String], Int, Int)](ctx.snapshotsRetentionPolicy()) - SetSnapshotsRetentionPolicy(tableName, logicalOperator, granularity, timeCount, count) + val (logicalOperator, granularity, timeCount, count) = typedVisit[(Option[LogicalOperatorsType], Option[String], Int, Int)](ctx.versionsRetentionPolicy()) + SetVersionsRetentionPolicy(tableName, logicalOperator, granularity, timeCount, count) } - override def visitSnapshotsRetentionPolicy(ctx: SnapshotsRetentionPolicyContext): (Option[LogicalOperatorsType], Option[String], Int, Int) = { + override def visitVersionsRetentionPolicy(ctx: VersionsRetentionPolicyContext): (Option[LogicalOperatorsType], Option[String], Int, Int) = { val logicalOperator = if (ctx.AND_OR_LOGICAL_OPERATOR() != null) LogicalOperators.withName(ctx.AND_OR_LOGICAL_OPERATOR().getText) else null - val timePolicy = if (ctx.snapshotsTTL() != null) - typedVisit[(String, Int)](ctx.snapshotsTTL()) + val timePolicy = if (ctx.versionsTime() != null) + typedVisit[(String, Int)](ctx.versionsTime().duration()) else (null, -1) - val countPolicy = if (ctx.snapshotsCount() != null) - typedVisit[Int](ctx.snapshotsCount()) + val countPolicy = if (ctx.versionsCount() != null) + typedVisit[Int](ctx.versionsCount()) else -1 (Option(logicalOperator), Option(timePolicy._1), timePolicy._2, countPolicy) } - override def visitSnapshotsTTL(ctx: SnapshotsTTLContext): (String, Int) = { - val granularity = ctx.dateGranularity().getText - val count = ctx.POSITIVE_INTEGER().getText.toInt - (granularity, count) - } - - override def visitSnapshotsCount(ctx: SnapshotsCountContext): Integer = { + override def visitVersionsCount(ctx: VersionsCountContext): Integer = { ctx.POSITIVE_INTEGER().getText.toInt } diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetVersionsRetentionPolicy.scala b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetVersionsRetentionPolicy.scala new file mode 100644 index 00000000..715857e2 --- /dev/null +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetVersionsRetentionPolicy.scala @@ -0,0 +1,10 @@ +package com.linkedin.openhouse.spark.sql.catalyst.plans.logical + +import org.apache.spark.sql.catalyst.plans.logical.Command +import com.linkedin.openhouse.spark.sql.catalyst.enums.LogicalOperators.LogicalOperatorsType + +case class SetVersionsRetentionPolicy(tableName: Seq[String], logicalOperator: Option[LogicalOperatorsType], granularity: Option[String], timeCount: Int, count: Int) extends Command { + override def simpleString(maxFields: Int): String = { + s"SetVersionsRetentionPolicy: ${tableName} ${if (timeCount > 0) timeCount else ""} ${granularity.getOrElse("")} ${logicalOperator.getOrElse("")} count ${if (count > 0) count else ""}" + } +} diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/OpenhouseDataSourceV2Strategy.scala b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/OpenhouseDataSourceV2Strategy.scala index 5e36445f..ee172fc7 100644 --- a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/OpenhouseDataSourceV2Strategy.scala +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/OpenhouseDataSourceV2Strategy.scala @@ -1,6 +1,6 @@ package com.linkedin.openhouse.spark.sql.execution.datasources.v2 -import com.linkedin.openhouse.spark.sql.catalyst.plans.logical.{GrantRevokeStatement, SetColumnPolicyTag, SetReplicationPolicy, SetRetentionPolicy, SetSharingPolicy, SetSnapshotsRetentionPolicy, ShowGrantsStatement} +import com.linkedin.openhouse.spark.sql.catalyst.plans.logical.{GrantRevokeStatement, SetColumnPolicyTag, SetReplicationPolicy, SetRetentionPolicy, SetSharingPolicy, SetVersionsRetentionPolicy, ShowGrantsStatement} import org.apache.iceberg.spark.{Spark3Util, SparkCatalog, SparkSessionCatalog} import org.apache.spark.sql.{SparkSession, Strategy} import org.apache.spark.sql.catalyst.expressions.PredicateHelper @@ -17,8 +17,8 @@ case class OpenhouseDataSourceV2Strategy(spark: SparkSession) extends Strategy w SetRetentionPolicyExec(catalog, ident, granularity, count, colName, colPattern) :: Nil case SetReplicationPolicy(CatalogAndIdentifierExtractor(catalog, ident), replicationPolicies) => SetReplicationPolicyExec(catalog, ident, replicationPolicies) :: Nil - case SetSnapshotsRetentionPolicy(CatalogAndIdentifierExtractor(catalog, ident), logicalOperator, granularity, timeCount, count) => - SetSnapshotsRetentionPolicyExec(catalog, ident, logicalOperator, granularity, timeCount, count) :: Nil + case SetVersionsRetentionPolicy(CatalogAndIdentifierExtractor(catalog, ident), logicalOperator, granularity, timeCount, count) => + SetVersionsRetentionPolicyExec(catalog, ident, logicalOperator, granularity, timeCount, count) :: Nil case SetSharingPolicy(CatalogAndIdentifierExtractor(catalog, ident), sharing) => SetSharingPolicyExec(catalog, ident, sharing) :: Nil case SetColumnPolicyTag(CatalogAndIdentifierExtractor(catalog, ident), policyTag, cols) => diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetSnapshotsRetentionPolicyExec.scala b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetSnapshotsRetentionPolicyExec.scala index 02c32bcb..13958f6b 100644 --- a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetSnapshotsRetentionPolicyExec.scala +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetSnapshotsRetentionPolicyExec.scala @@ -1,5 +1,6 @@ package com.linkedin.openhouse.spark.sql.execution.datasources.v2 +import com.linkedin.openhouse.spark.sql.catalyst.enums.LogicalOperators import com.linkedin.openhouse.spark.sql.catalyst.enums.LogicalOperators.LogicalOperatorsType import org.apache.iceberg.spark.source.SparkTable import org.apache.spark.sql.catalyst.InternalRow @@ -7,7 +8,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} import org.apache.spark.sql.execution.datasources.v2.V2CommandExec -case class SetSnapshotsRetentionPolicyExec( +case class SetVersionsRetentionPolicyExec( catalog: TableCatalog, ident: Identifier, logicalOperator: Option[LogicalOperatorsType], @@ -23,11 +24,10 @@ case class SetSnapshotsRetentionPolicyExec( case iceberg: SparkTable if iceberg.table().properties().containsKey("openhouse.tableId") => val key = "updated.openhouse.policy" val value = { - (count) match { - case (0) => s"""{"snapshotsRetention":{"timeCount":${timeCount},"granularity":"${granularity}"}}""" - case (_) => - s"""{"snapshotsRetention":{"timeCount":${timeCount}, "granularity":"${granularity}", "count":${count}}}""" - } + (timeCount, count, logicalOperator) match { + case ttlOnly if count == -1 => s"""{"versionsRetention":{"timeCount":${timeCount},"granularity":"${granularity.get}"}}""" + case countOnly if timeCount == -1 => s"""{"versionsRetention":{"count":${count}}}""" + case _ => s"""{"versionsRetention":{"timeCount":${timeCount},"granularity":"${granularity.get}","count":${count},"logicalOperator":${logicalOperator.get.toString}}}"""} } iceberg.table().updateProperties() @@ -42,6 +42,6 @@ case class SetSnapshotsRetentionPolicyExec( } override def simpleString(maxFields: Int): String = { - s"SetSnapshotsRetentionPolicyExec: ${catalog} ${ident} ${timeCount} ${granularity} ${count}" + s"SetVersionsRetentionPolicyExec: ${catalog} ${ident} ${if (timeCount > 0) timeCount else ""} ${granularity.getOrElse("")} ${logicalOperator.getOrElse("")} count ${if (count > 0) count else ""}" } } diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetVersionsRetentionPolicyExec.scala b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetVersionsRetentionPolicyExec.scala new file mode 100644 index 00000000..13958f6b --- /dev/null +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetVersionsRetentionPolicyExec.scala @@ -0,0 +1,47 @@ +package com.linkedin.openhouse.spark.sql.execution.datasources.v2 + +import com.linkedin.openhouse.spark.sql.catalyst.enums.LogicalOperators +import com.linkedin.openhouse.spark.sql.catalyst.enums.LogicalOperators.LogicalOperatorsType +import org.apache.iceberg.spark.source.SparkTable +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} +import org.apache.spark.sql.execution.datasources.v2.V2CommandExec + +case class SetVersionsRetentionPolicyExec( + catalog: TableCatalog, + ident: Identifier, + logicalOperator: Option[LogicalOperatorsType], + granularity: Option[String], + timeCount: Int, + count: Int +) extends V2CommandExec { + + override lazy val output: Seq[Attribute] = Nil + + override protected def run(): Seq[InternalRow] = { + catalog.loadTable(ident) match { + case iceberg: SparkTable if iceberg.table().properties().containsKey("openhouse.tableId") => + val key = "updated.openhouse.policy" + val value = { + (timeCount, count, logicalOperator) match { + case ttlOnly if count == -1 => s"""{"versionsRetention":{"timeCount":${timeCount},"granularity":"${granularity.get}"}}""" + case countOnly if timeCount == -1 => s"""{"versionsRetention":{"count":${count}}}""" + case _ => s"""{"versionsRetention":{"timeCount":${timeCount},"granularity":"${granularity.get}","count":${count},"logicalOperator":${logicalOperator.get.toString}}}"""} + } + + iceberg.table().updateProperties() + .set(key, value) + .commit() + + case table => + throw new UnsupportedOperationException(s"Cannot set snapshots retention policy for non-Openhouse table: $table") + } + + Nil + } + + override def simpleString(maxFields: Int): String = { + s"SetVersionsRetentionPolicyExec: ${catalog} ${ident} ${if (timeCount > 0) timeCount else ""} ${granularity.getOrElse("")} ${logicalOperator.getOrElse("")} count ${if (count > 0) count else ""}" + } +} From fe18d99f04a3b0a9fcece2d741c1f80f89a0a193 Mon Sep 17 00:00:00 2001 From: Will-Lo Date: Tue, 3 Dec 2024 21:25:36 -0500 Subject: [PATCH 04/10] Fix compilation issues, use snapshot retention syntax --- .../SetSnapshotsRetentionPolicyTest.java | 169 ++++++++++++- .../SetVersionsRetentionPolicyTest.java | 231 ------------------ .../extensions/OpenhouseSqlExtensions.g4 | 21 +- .../sql/catalyst/enums/LogicalOperators.scala | 6 - .../OpenhouseSqlExtensionsAstBuilder.scala | 24 +- .../logical/SetSnapshotsRetentionPolicy.scala | 5 +- .../logical/SetVersionsRetentionPolicy.scala | 10 - .../v2/OpenhouseDataSourceV2Strategy.scala | 6 +- .../v2/SetSnapshotsRetentionPolicyExec.scala | 18 +- .../v2/SetVersionsRetentionPolicyExec.scala | 47 ---- 10 files changed, 195 insertions(+), 342 deletions(-) delete mode 100644 integrations/spark/spark-3.1/openhouse-spark-itest/src/test/java/com/linkedin/openhouse/spark/e2e/extensions/SetVersionsRetentionPolicyTest.java delete mode 100644 integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/enums/LogicalOperators.scala delete mode 100644 integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetVersionsRetentionPolicy.scala delete mode 100644 integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetVersionsRetentionPolicyExec.scala diff --git a/integrations/spark/spark-3.1/openhouse-spark-itest/src/test/java/com/linkedin/openhouse/spark/e2e/extensions/SetSnapshotsRetentionPolicyTest.java b/integrations/spark/spark-3.1/openhouse-spark-itest/src/test/java/com/linkedin/openhouse/spark/e2e/extensions/SetSnapshotsRetentionPolicyTest.java index 643a9f98..154baa1b 100644 --- a/integrations/spark/spark-3.1/openhouse-spark-itest/src/test/java/com/linkedin/openhouse/spark/e2e/extensions/SetSnapshotsRetentionPolicyTest.java +++ b/integrations/spark/spark-3.1/openhouse-spark-itest/src/test/java/com/linkedin/openhouse/spark/e2e/extensions/SetSnapshotsRetentionPolicyTest.java @@ -12,7 +12,7 @@ @ExtendWith(SparkTestBase.class) public class SetSnapshotsRetentionPolicyTest { @Test - public void testSetSnapshotsRetentionPolicy() { + public void testSetSnapshotsRetentionPolicyTimeAndCount() { Object existingTable = mockGetTableResponseBody( "dbSetSnapshotsRetention", @@ -51,7 +51,8 @@ public void testSetSnapshotsRetentionPolicy() { "u1", mockTableLocationAfterOperation( TableIdentifier.of("dbSetSnapshotsRetention", "t1"), - "ALTER TABLE %t SET TBLPROPERTIES('policies'='{\"snapshotsRetention\":{\"timeCount\":24,\"granularity\":\"HOUR\",\"count\":10}}')"), + "ALTER TABLE %t SET TBLPROPERTIES('policies'='{\"snapshotsRetention\":{\"timeCount\":24,\"granularity\":" + + "\"HOUR\",\"count\":10 }}')"), "V2", baseSchema, null, @@ -60,7 +61,169 @@ public void testSetSnapshotsRetentionPolicy() { mockTableService.enqueue(mockResponse(200, existingOpenhouseTable)); // doRefresh() mockTableService.enqueue(mockResponse(201, tableAfterSetSnapshotsRetention)); // doCommit() String ddlWithSchema = - "ALTER TABLE openhouse.dbSetSnapshotsRetention.t1 SET POLICY (SNAPSHOTS_RETENTION TTL=24H COUNT=10)"; + "ALTER TABLE openhouse.dbSetSnapshotsRetention.t1 SET POLICY (SNAPSHOT_RETENTION TIME=24H COUNT=10)"; + Assertions.assertDoesNotThrow(() -> spark.sql(ddlWithSchema)); + } + + @Test + public void testSetSnapshotsRetentionPolicyTimeOrCount() { + Object existingTable = + mockGetTableResponseBody( + "dbSetSnapshotsRetention", + "t2", + "c1", + "dbSetSnapshotsRetention.t2", + "u1", + mockTableLocation( + TableIdentifier.of("dbSetSnapshotsRetention", "t2"), + convertSchemaToDDLComponent(baseSchema), + ""), + "V1", + baseSchema, + null, + null); + Object existingOpenhouseTable = + mockGetTableResponseBody( + "dbSetSnapshotsRetention", + "t2", + "c1", + "dbSetSnapshotsRetention.t2", + "u1", + mockTableLocationAfterOperation( + TableIdentifier.of("dbSetSnapshotsRetention", "t2"), + "ALTER TABLE %t SET TBLPROPERTIES ('openhouse.tableId'='t2')"), + "V1", + baseSchema, + null, + null); + Object tableAfterSetSnapshotsRetention = + mockGetTableResponseBody( + "dbSetSnapshotsRetention", + "t1", + "c1", + "dbSetSnapshotsRetention.t2", + "u1", + mockTableLocationAfterOperation( + TableIdentifier.of("dbSetSnapshotsRetention", "t2"), + "ALTER TABLE %t SET TBLPROPERTIES('policies'='{\"snapshotsRetention\":{\"timeCount\":24,\"granularity\":" + + "\"HOUR\",\"count\":10 }}')"), + "V2", + baseSchema, + null, + null); + mockTableService.enqueue(mockResponse(200, existingOpenhouseTable)); // doRefresh() + mockTableService.enqueue(mockResponse(200, existingOpenhouseTable)); // doRefresh() + mockTableService.enqueue(mockResponse(201, tableAfterSetSnapshotsRetention)); // doCommit() + String ddlWithSchema = + "ALTER TABLE openhouse.dbSetSnapshotsRetention.t2 SET POLICY (VERSIONS TIME=24H COUNT=10)"; + Assertions.assertDoesNotThrow(() -> spark.sql(ddlWithSchema)); + } + + @Test + public void testSetSnapshotsRetentionPolicyTimeOnly() { + Object existingTable = + mockGetTableResponseBody( + "dbSetSnapshotsRetention", + "t3", + "c1", + "dbSetSnapshotsRetention.t3", + "u1", + mockTableLocation( + TableIdentifier.of("dbSetSnapshotsRetention", "t3"), + convertSchemaToDDLComponent(baseSchema), + ""), + "V1", + baseSchema, + null, + null); + Object existingOpenhouseTable = + mockGetTableResponseBody( + "dbSetSnapshotsRetention", + "t3", + "c1", + "dbSetSnapshotsRetention.t3", + "u1", + mockTableLocationAfterOperation( + TableIdentifier.of("dbSetSnapshotsRetention", "t3"), + "ALTER TABLE %t SET TBLPROPERTIES ('openhouse.tableId'='t3')"), + "V1", + baseSchema, + null, + null); + Object tableAfterSetSnapshotsRetention = + mockGetTableResponseBody( + "dbSetSnapshotsRetention", + "t3", + "c1", + "dbSetSnapshotsRetention.t3", + "u1", + mockTableLocationAfterOperation( + TableIdentifier.of("dbSetSnapshotsRetention", "t3"), + "ALTER TABLE %t SET TBLPROPERTIES('policies'='{\"snapshotsRetention\":{\"timeCount\":24,\"granularity\":" + + "\"HOUR\"}}')"), + "V2", + baseSchema, + null, + null); + mockTableService.enqueue(mockResponse(200, existingOpenhouseTable)); // doRefresh() + mockTableService.enqueue(mockResponse(200, existingOpenhouseTable)); // doRefresh() + mockTableService.enqueue(mockResponse(201, tableAfterSetSnapshotsRetention)); // doCommit() + String ddlWithSchema = + "ALTER TABLE openhouse.dbSetSnapshotsRetention.t3 SET POLICY (VERSIONS TIME=24H)"; + Assertions.assertDoesNotThrow(() -> spark.sql(ddlWithSchema)); + } + + @Test + public void testSetSnapshotsRetentionPolicyCountOnly() { + Object existingTable = + mockGetTableResponseBody( + "dbSetSnapshotsRetention", + "t4", + "c1", + "dbSetSnapshotsRetention.t4", + "u1", + mockTableLocation( + TableIdentifier.of("dbSetSnapshotsRetention", "t4"), + convertSchemaToDDLComponent(baseSchema), + ""), + "V1", + baseSchema, + null, + null); + Object existingOpenhouseTable = + mockGetTableResponseBody( + "dbSetSnapshotsRetention", + "t4", + "c1", + "dbSetSnapshotsRetention.t4", + "u1", + mockTableLocationAfterOperation( + TableIdentifier.of("dbSetSnapshotsRetention", "t4"), + "ALTER TABLE %t SET TBLPROPERTIES ('openhouse.tableId'='t4')"), + "V1", + baseSchema, + null, + null); + Object tableAfterSetSnapshotsRetention = + mockGetTableResponseBody( + "dbSetSnapshotsRetention", + "t4", + "c1", + "dbSetSnapshotsRetention.t4", + "u1", + mockTableLocationAfterOperation( + TableIdentifier.of("dbSetSnapshotsRetention", "t4"), + "ALTER TABLE %t SET TBLPROPERTIES('policies'='{\"snapshotsRetention\":{\"timeCount\":24,\"granularity\":" + + "\"HOUR\"}}')"), + "V2", + baseSchema, + null, + null); + mockTableService.enqueue(mockResponse(200, existingOpenhouseTable)); // doRefresh() + mockTableService.enqueue(mockResponse(200, existingOpenhouseTable)); // doRefresh() + mockTableService.enqueue(mockResponse(201, tableAfterSetSnapshotsRetention)); // doCommit() + String ddlWithSchema = + "ALTER TABLE openhouse.dbSetSnapshotsRetention.t4 SET POLICY (VERSIONS COUNT=10)"; Assertions.assertDoesNotThrow(() -> spark.sql(ddlWithSchema)); } } diff --git a/integrations/spark/spark-3.1/openhouse-spark-itest/src/test/java/com/linkedin/openhouse/spark/e2e/extensions/SetVersionsRetentionPolicyTest.java b/integrations/spark/spark-3.1/openhouse-spark-itest/src/test/java/com/linkedin/openhouse/spark/e2e/extensions/SetVersionsRetentionPolicyTest.java deleted file mode 100644 index df03093e..00000000 --- a/integrations/spark/spark-3.1/openhouse-spark-itest/src/test/java/com/linkedin/openhouse/spark/e2e/extensions/SetVersionsRetentionPolicyTest.java +++ /dev/null @@ -1,231 +0,0 @@ -package com.linkedin.openhouse.spark.e2e.extensions; - -import static com.linkedin.openhouse.spark.MockHelpers.*; -import static com.linkedin.openhouse.spark.SparkTestBase.*; - -import com.linkedin.openhouse.spark.SparkTestBase; -import org.apache.iceberg.catalog.TableIdentifier; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; - -@ExtendWith(SparkTestBase.class) -public class SetVersionsRetentionPolicyTest { - @Test - public void testSetVersionsRetentionPolicyTimeAndCount() { - Object existingTable = - mockGetTableResponseBody( - "dbSetVersionsRetention", - "t1", - "c1", - "dbSetVersionsRetention.t1", - "u1", - mockTableLocation( - TableIdentifier.of("dbSetVersionsRetention", "t1"), - convertSchemaToDDLComponent(baseSchema), - ""), - "V1", - baseSchema, - null, - null); - Object existingOpenhouseTable = - mockGetTableResponseBody( - "dbSetVersionsRetention", - "t1", - "c1", - "dbSetVersionsRetention.t1", - "u1", - mockTableLocationAfterOperation( - TableIdentifier.of("dbSetVersionsRetention", "t1"), - "ALTER TABLE %t SET TBLPROPERTIES ('openhouse.tableId'='t1')"), - "V1", - baseSchema, - null, - null); - Object tableAfterSetversionsRetention = - mockGetTableResponseBody( - "dbSetVersionsRetention", - "t1", - "c1", - "dbSetVersionsRetention.t1", - "u1", - mockTableLocationAfterOperation( - TableIdentifier.of("dbSetVersionsRetention", "t1"), - "ALTER TABLE %t SET TBLPROPERTIES('policies'='{\"versionsRetention\":{\"timeCount\":24,\"granularity\":" - + "\"HOUR\",\"count\":10, \"logicalOperator\": " - + "\"AND\"}}')"), - "V2", - baseSchema, - null, - null); - mockTableService.enqueue(mockResponse(200, existingOpenhouseTable)); // doRefresh() - mockTableService.enqueue(mockResponse(200, existingOpenhouseTable)); // doRefresh() - mockTableService.enqueue(mockResponse(201, tableAfterSetversionsRetention)); // doCommit() - String ddlWithSchema = - "ALTER TABLE openhouse.dbSetversionsRetention.t1 SET POLICY (VERSIONS TIME=24H AND COUNT=10)"; - Assertions.assertDoesNotThrow(() -> spark.sql(ddlWithSchema)); - } - - @Test - public void testSetversionsRetentionPolicyTimeOrCount() { - Object existingTable = - mockGetTableResponseBody( - "dbSetVersionsRetention", - "t2", - "c1", - "dbSetVersionsRetention.t2", - "u1", - mockTableLocation( - TableIdentifier.of("dbSetVersionsRetention", "t2"), - convertSchemaToDDLComponent(baseSchema), - ""), - "V1", - baseSchema, - null, - null); - Object existingOpenhouseTable = - mockGetTableResponseBody( - "dbSetVersionsRetention", - "t2", - "c1", - "dbSetVersionsRetention.t2", - "u1", - mockTableLocationAfterOperation( - TableIdentifier.of("dbSetVersionsRetention", "t2"), - "ALTER TABLE %t SET TBLPROPERTIES ('openhouse.tableId'='t2')"), - "V1", - baseSchema, - null, - null); - Object tableAfterSetversionsRetention = - mockGetTableResponseBody( - "dbSetVersionsRetention", - "t1", - "c1", - "dbSetVersionsRetention.t2", - "u1", - mockTableLocationAfterOperation( - TableIdentifier.of("dbSetVersionsRetention", "t2"), - "ALTER TABLE %t SET TBLPROPERTIES('policies'='{\"versionsRetention\":{\"timeCount\":24,\"granularity\":" - + "\"HOUR\",\"count\":10, \"logicalOperator\": " - + "\"OR\"}}')"), - "V2", - baseSchema, - null, - null); - mockTableService.enqueue(mockResponse(200, existingOpenhouseTable)); // doRefresh() - mockTableService.enqueue(mockResponse(200, existingOpenhouseTable)); // doRefresh() - mockTableService.enqueue(mockResponse(201, tableAfterSetversionsRetention)); // doCommit() - String ddlWithSchema = - "ALTER TABLE openhouse.dbSetversionsRetention.t2 SET POLICY (VERSIONS TIME=24H OR COUNT=10)"; - Assertions.assertDoesNotThrow(() -> spark.sql(ddlWithSchema)); - } - - @Test - public void testSetversionsRetentionPolicyTimeOnly() { - Object existingTable = - mockGetTableResponseBody( - "dbSetVersionsRetention", - "t3", - "c1", - "dbSetVersionsRetention.t3", - "u1", - mockTableLocation( - TableIdentifier.of("dbSetVersionsRetention", "t3"), - convertSchemaToDDLComponent(baseSchema), - ""), - "V1", - baseSchema, - null, - null); - Object existingOpenhouseTable = - mockGetTableResponseBody( - "dbSetVersionsRetention", - "t3", - "c1", - "dbSetVersionsRetention.t3", - "u1", - mockTableLocationAfterOperation( - TableIdentifier.of("dbSetVersionsRetention", "t3"), - "ALTER TABLE %t SET TBLPROPERTIES ('openhouse.tableId'='t3')"), - "V1", - baseSchema, - null, - null); - Object tableAfterSetversionsRetention = - mockGetTableResponseBody( - "dbSetVersionsRetention", - "t3", - "c1", - "dbSetVersionsRetention.t3", - "u1", - mockTableLocationAfterOperation( - TableIdentifier.of("dbSetVersionsRetention", "t3"), - "ALTER TABLE %t SET TBLPROPERTIES('policies'='{\"versionsRetention\":{\"timeCount\":24,\"granularity\":" - + "\"HOUR\"}}')"), - "V2", - baseSchema, - null, - null); - mockTableService.enqueue(mockResponse(200, existingOpenhouseTable)); // doRefresh() - mockTableService.enqueue(mockResponse(200, existingOpenhouseTable)); // doRefresh() - mockTableService.enqueue(mockResponse(201, tableAfterSetversionsRetention)); // doCommit() - String ddlWithSchema = - "ALTER TABLE openhouse.dbSetversionsRetention.t3 SET POLICY (VERSIONS TIME=24H)"; - Assertions.assertDoesNotThrow(() -> spark.sql(ddlWithSchema)); - } - - @Test - public void testSetversionsRetentionPolicyCountOnly() { - Object existingTable = - mockGetTableResponseBody( - "dbSetVersionsRetention", - "t4", - "c1", - "dbSetVersionsRetention.t4", - "u1", - mockTableLocation( - TableIdentifier.of("dbSetVersionsRetention", "t4"), - convertSchemaToDDLComponent(baseSchema), - ""), - "V1", - baseSchema, - null, - null); - Object existingOpenhouseTable = - mockGetTableResponseBody( - "dbSetVersionsRetention", - "t4", - "c1", - "dbSetVersionsRetention.t4", - "u1", - mockTableLocationAfterOperation( - TableIdentifier.of("dbSetVersionsRetention", "t4"), - "ALTER TABLE %t SET TBLPROPERTIES ('openhouse.tableId'='t4')"), - "V1", - baseSchema, - null, - null); - Object tableAfterSetversionsRetention = - mockGetTableResponseBody( - "dbSetVersionsRetention", - "t4", - "c1", - "dbSetVersionsRetention.t4", - "u1", - mockTableLocationAfterOperation( - TableIdentifier.of("dbSetVersionsRetention", "t4"), - "ALTER TABLE %t SET TBLPROPERTIES('policies'='{\"versionsRetention\":{\"timeCount\":24,\"granularity\":" - + "\"HOUR\"}}')"), - "V2", - baseSchema, - null, - null); - mockTableService.enqueue(mockResponse(200, existingOpenhouseTable)); // doRefresh() - mockTableService.enqueue(mockResponse(200, existingOpenhouseTable)); // doRefresh() - mockTableService.enqueue(mockResponse(201, tableAfterSetversionsRetention)); // doCommit() - String ddlWithSchema = - "ALTER TABLE openhouse.dbSetversionsRetention.t4 SET POLICY (VERSIONS COUNT=10)"; - Assertions.assertDoesNotThrow(() -> spark.sql(ddlWithSchema)); - } -} diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/antlr/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensions.g4 b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/antlr/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensions.g4 index c1b03344..fea71c47 100644 --- a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/antlr/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensions.g4 +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/antlr/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensions.g4 @@ -26,7 +26,7 @@ statement : ALTER TABLE multipartIdentifier SET POLICY '(' retentionPolicy (columnRetentionPolicy)? ')' #setRetentionPolicy | ALTER TABLE multipartIdentifier SET POLICY '(' replicationPolicy ')' #setReplicationPolicy | ALTER TABLE multipartIdentifier SET POLICY '(' sharingPolicy ')' #setSharingPolicy - | ALTER TABLE multipartIdentifier SET POLICY '(' versionsRetentionPolicy ')' #setVersionsRetentionPolicy + | ALTER TABLE multipartIdentifier SET POLICY '(' snapshotRetentionPolicy ')' #setSnapshotRetentionPolicy | ALTER TABLE multipartIdentifier MODIFY columnNameClause SET columnPolicy #setColumnPolicyTag | GRANT privilege ON grantableResource TO principal #grantStatement | REVOKE privilege ON grantableResource FROM principal #revokeStatement @@ -66,11 +66,7 @@ quotedIdentifier ; nonReserved -<<<<<<< HEAD:integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/antlr/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensions.g4 - : ALTER | TABLE | SET | POLICY | RETENTION | SHARING | REPLICATION | RETAIN_SNAPSHOTS -======= - : ALTER | TABLE | SET | POLICY | RETENTION | SHARING | VERSIONS ->>>>>>> 3b7388d (Formatting fix):integrations/spark/openhouse-spark-runtime/src/main/antlr/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensions.g4 + : ALTER | TABLE | SET | POLICY | RETENTION | SHARING | REPLICATION | SNAPSHOT_RETENTION | GRANT | REVOKE | ON | TO | SHOW | GRANTS | PATTERN | WHERE | COLUMN ; @@ -82,10 +78,6 @@ BOOLEAN : 'TRUE' | 'FALSE' ; -AND_OR_LOGICAL_OPERATOR - : 'AND' | 'OR' - ; - retentionPolicy : RETENTION '=' duration ; @@ -163,13 +155,11 @@ policyTag : PII | HC ; -versionsRetentionPolicy - : VERSIONS versionsTime - | VERSIONS versionsCount - | VERSIONS versionsTime AND_OR_LOGICAL_OPERATOR versionsCount +snapshotRetentionPolicy + : SNAPSHOT_RETENTION retainTime? versionsCount? ; -versionsTime +retainTime : TIME'='duration ; @@ -183,6 +173,7 @@ SET: 'SET'; POLICY: 'POLICY'; RETENTION: 'RETENTION'; REPLICATION: 'REPLICATION'; +SNAPSHOT_RETENTION: 'SNAPSHOT_RETENTION'; SHARING: 'SHARING'; GRANT: 'GRANT'; REVOKE: 'REVOKE'; diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/enums/LogicalOperators.scala b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/enums/LogicalOperators.scala deleted file mode 100644 index 680a5cc9..00000000 --- a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/enums/LogicalOperators.scala +++ /dev/null @@ -1,6 +0,0 @@ -package com.linkedin.openhouse.spark.sql.catalyst.enums - -private[sql] object LogicalOperators extends Enumeration { - type LogicalOperatorsType = Value - val AND, OR = Value -} diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensionsAstBuilder.scala b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensionsAstBuilder.scala index 71aea4e5..8dd98dc2 100644 --- a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensionsAstBuilder.scala +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensionsAstBuilder.scala @@ -1,10 +1,9 @@ package com.linkedin.openhouse.spark.sql.catalyst.parser.extensions -import com.linkedin.openhouse.spark.sql.catalyst.enums.{GrantableResourceTypes, LogicalOperators} +import com.linkedin.openhouse.spark.sql.catalyst.enums.GrantableResourceTypes import com.linkedin.openhouse.spark.sql.catalyst.parser.extensions.OpenhouseSqlExtensionsParser._ -import com.linkedin.openhouse.spark.sql.catalyst.plans.logical.{GrantRevokeStatement, SetColumnPolicyTag, SetReplicationPolicy, SetRetentionPolicy, SetSharingPolicy, SetVersionsRetentionPolicy, ShowGrantsStatement} +import com.linkedin.openhouse.spark.sql.catalyst.plans.logical.{GrantRevokeStatement, SetColumnPolicyTag, SetReplicationPolicy, SetRetentionPolicy, SetSharingPolicy, SetSnapshotsRetentionPolicy, ShowGrantsStatement} import com.linkedin.openhouse.spark.sql.catalyst.enums.GrantableResourceTypes.GrantableResourceType -import com.linkedin.openhouse.spark.sql.catalyst.enums.LogicalOperators.LogicalOperatorsType import com.linkedin.openhouse.gen.tables.client.model.TimePartitionSpec import org.antlr.v4.runtime.tree.ParseTree import org.apache.spark.sql.catalyst.parser.ParserInterface @@ -164,22 +163,19 @@ class OpenhouseSqlExtensionsAstBuilder (delegate: ParserInterface) extends Openh (granularity, count) } - override def visitSetVersionsRetentionPolicy(ctx: SetVersionsRetentionPolicyContext): SetVersionsRetentionPolicy = { + override def visitSetSnapshotRetentionPolicy(ctx: SetSnapshotRetentionPolicyContext): SetSnapshotsRetentionPolicy = { val tableName = typedVisit[Seq[String]](ctx.multipartIdentifier) - val (logicalOperator, granularity, timeCount, count) = typedVisit[(Option[LogicalOperatorsType], Option[String], Int, Int)](ctx.versionsRetentionPolicy()) - SetVersionsRetentionPolicy(tableName, logicalOperator, granularity, timeCount, count) - } - override def visitVersionsRetentionPolicy(ctx: VersionsRetentionPolicyContext): (Option[LogicalOperatorsType], Option[String], Int, Int) = { - val logicalOperator = if (ctx.AND_OR_LOGICAL_OPERATOR() != null) - LogicalOperators.withName(ctx.AND_OR_LOGICAL_OPERATOR().getText) - else null - val timePolicy = if (ctx.versionsTime() != null) - typedVisit[(String, Int)](ctx.versionsTime().duration()) + val (granularity, timeCount, count) = typedVisit[(Option[String], Int, Int)](ctx.snapshotRetentionPolicy()) + SetSnapshotsRetentionPolicy(tableName, granularity, timeCount, count) + } + override def visitSnapshotRetentionPolicy(ctx: SnapshotRetentionPolicyContext): (Option[String], Int, Int) = { + val timePolicy = if (ctx.retainTime() != null) + typedVisit[(String, Int)](ctx.retainTime().duration()) else (null, -1) val countPolicy = if (ctx.versionsCount() != null) typedVisit[Int](ctx.versionsCount()) else -1 - (Option(logicalOperator), Option(timePolicy._1), timePolicy._2, countPolicy) + (Option(timePolicy._1), timePolicy._2, countPolicy) } override def visitVersionsCount(ctx: VersionsCountContext): Integer = { diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetSnapshotsRetentionPolicy.scala b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetSnapshotsRetentionPolicy.scala index e38752be..ddc7acfa 100644 --- a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetSnapshotsRetentionPolicy.scala +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetSnapshotsRetentionPolicy.scala @@ -1,10 +1,9 @@ package com.linkedin.openhouse.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.Command -import com.linkedin.openhouse.spark.sql.catalyst.enums.LogicalOperators.LogicalOperatorsType -case class SetSnapshotsRetentionPolicy (tableName: Seq[String], logicalOperator: Option[LogicalOperatorsType], granularity: Option[String], timeCount: Int, count: Int) extends Command { +case class SetSnapshotsRetentionPolicy (tableName: Seq[String], granularity: Option[String], timeCount: Int, count: Int) extends Command { override def simpleString(maxFields: Int): String = { - s"SetSnapshotsRetentionPolicy: ${tableName} ${if (timeCount > 0) timeCount else ""} ${granularity.getOrElse("")} ${logicalOperator.getOrElse("")} count ${if (count > 0) count else ""}" + s"SetSnapshotsRetentionPolicy: ${tableName} ${if (timeCount > 0) timeCount else ""} ${granularity.getOrElse("")} count ${if (count > 0) count else ""}" } } diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetVersionsRetentionPolicy.scala b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetVersionsRetentionPolicy.scala deleted file mode 100644 index 715857e2..00000000 --- a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetVersionsRetentionPolicy.scala +++ /dev/null @@ -1,10 +0,0 @@ -package com.linkedin.openhouse.spark.sql.catalyst.plans.logical - -import org.apache.spark.sql.catalyst.plans.logical.Command -import com.linkedin.openhouse.spark.sql.catalyst.enums.LogicalOperators.LogicalOperatorsType - -case class SetVersionsRetentionPolicy(tableName: Seq[String], logicalOperator: Option[LogicalOperatorsType], granularity: Option[String], timeCount: Int, count: Int) extends Command { - override def simpleString(maxFields: Int): String = { - s"SetVersionsRetentionPolicy: ${tableName} ${if (timeCount > 0) timeCount else ""} ${granularity.getOrElse("")} ${logicalOperator.getOrElse("")} count ${if (count > 0) count else ""}" - } -} diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/OpenhouseDataSourceV2Strategy.scala b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/OpenhouseDataSourceV2Strategy.scala index ee172fc7..db8c6304 100644 --- a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/OpenhouseDataSourceV2Strategy.scala +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/OpenhouseDataSourceV2Strategy.scala @@ -1,6 +1,6 @@ package com.linkedin.openhouse.spark.sql.execution.datasources.v2 -import com.linkedin.openhouse.spark.sql.catalyst.plans.logical.{GrantRevokeStatement, SetColumnPolicyTag, SetReplicationPolicy, SetRetentionPolicy, SetSharingPolicy, SetVersionsRetentionPolicy, ShowGrantsStatement} +import com.linkedin.openhouse.spark.sql.catalyst.plans.logical.{GrantRevokeStatement, SetColumnPolicyTag, SetReplicationPolicy, SetRetentionPolicy, SetSharingPolicy, SetSnapshotsRetentionPolicy, ShowGrantsStatement} import org.apache.iceberg.spark.{Spark3Util, SparkCatalog, SparkSessionCatalog} import org.apache.spark.sql.{SparkSession, Strategy} import org.apache.spark.sql.catalyst.expressions.PredicateHelper @@ -17,8 +17,8 @@ case class OpenhouseDataSourceV2Strategy(spark: SparkSession) extends Strategy w SetRetentionPolicyExec(catalog, ident, granularity, count, colName, colPattern) :: Nil case SetReplicationPolicy(CatalogAndIdentifierExtractor(catalog, ident), replicationPolicies) => SetReplicationPolicyExec(catalog, ident, replicationPolicies) :: Nil - case SetVersionsRetentionPolicy(CatalogAndIdentifierExtractor(catalog, ident), logicalOperator, granularity, timeCount, count) => - SetVersionsRetentionPolicyExec(catalog, ident, logicalOperator, granularity, timeCount, count) :: Nil + case SetSnapshotsRetentionPolicy(CatalogAndIdentifierExtractor(catalog, ident), granularity, timeCount, count) => + SetSnapshotsRetentionPolicyExec(catalog, ident, granularity, timeCount, count) :: Nil case SetSharingPolicy(CatalogAndIdentifierExtractor(catalog, ident), sharing) => SetSharingPolicyExec(catalog, ident, sharing) :: Nil case SetColumnPolicyTag(CatalogAndIdentifierExtractor(catalog, ident), policyTag, cols) => diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetSnapshotsRetentionPolicyExec.scala b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetSnapshotsRetentionPolicyExec.scala index 13958f6b..6cd015ac 100644 --- a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetSnapshotsRetentionPolicyExec.scala +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetSnapshotsRetentionPolicyExec.scala @@ -1,17 +1,14 @@ package com.linkedin.openhouse.spark.sql.execution.datasources.v2 -import com.linkedin.openhouse.spark.sql.catalyst.enums.LogicalOperators -import com.linkedin.openhouse.spark.sql.catalyst.enums.LogicalOperators.LogicalOperatorsType import org.apache.iceberg.spark.source.SparkTable import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} import org.apache.spark.sql.execution.datasources.v2.V2CommandExec -case class SetVersionsRetentionPolicyExec( +case class SetSnapshotsRetentionPolicyExec( catalog: TableCatalog, ident: Identifier, - logicalOperator: Option[LogicalOperatorsType], granularity: Option[String], timeCount: Int, count: Int @@ -24,10 +21,11 @@ case class SetVersionsRetentionPolicyExec( case iceberg: SparkTable if iceberg.table().properties().containsKey("openhouse.tableId") => val key = "updated.openhouse.policy" val value = { - (timeCount, count, logicalOperator) match { - case ttlOnly if count == -1 => s"""{"versionsRetention":{"timeCount":${timeCount},"granularity":"${granularity.get}"}}""" - case countOnly if timeCount == -1 => s"""{"versionsRetention":{"count":${count}}}""" - case _ => s"""{"versionsRetention":{"timeCount":${timeCount},"granularity":"${granularity.get}","count":${count},"logicalOperator":${logicalOperator.get.toString}}}"""} + (timeCount, count) match { + case ttlOnly if count == -1 => s"""{"snapshotsRetention":{"timeCount":${timeCount},"granularity":"${granularity.get}"}}""" + case countOnly if timeCount == -1 => s"""{"snapshotsRetention":{"count":${count}}}""" + case _ => s"""{"snapshotsRetention":{"timeCount":${timeCount},"granularity":"${granularity.get}","count":${count}}}""" + } } iceberg.table().updateProperties() @@ -35,13 +33,13 @@ case class SetVersionsRetentionPolicyExec( .commit() case table => - throw new UnsupportedOperationException(s"Cannot set snapshots retention policy for non-Openhouse table: $table") + throw new UnsupportedOperationException(s"Cannot set snapshot retention policy for non-Openhouse table: $table") } Nil } override def simpleString(maxFields: Int): String = { - s"SetVersionsRetentionPolicyExec: ${catalog} ${ident} ${if (timeCount > 0) timeCount else ""} ${granularity.getOrElse("")} ${logicalOperator.getOrElse("")} count ${if (count > 0) count else ""}" + s"SetSnapshotsRetentionPolicyExec: ${catalog} ${ident} ${if (timeCount > 0) timeCount else ""} ${granularity.getOrElse("")} count ${if (count > 0) count else ""}" } } diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetVersionsRetentionPolicyExec.scala b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetVersionsRetentionPolicyExec.scala deleted file mode 100644 index 13958f6b..00000000 --- a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetVersionsRetentionPolicyExec.scala +++ /dev/null @@ -1,47 +0,0 @@ -package com.linkedin.openhouse.spark.sql.execution.datasources.v2 - -import com.linkedin.openhouse.spark.sql.catalyst.enums.LogicalOperators -import com.linkedin.openhouse.spark.sql.catalyst.enums.LogicalOperators.LogicalOperatorsType -import org.apache.iceberg.spark.source.SparkTable -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} -import org.apache.spark.sql.execution.datasources.v2.V2CommandExec - -case class SetVersionsRetentionPolicyExec( - catalog: TableCatalog, - ident: Identifier, - logicalOperator: Option[LogicalOperatorsType], - granularity: Option[String], - timeCount: Int, - count: Int -) extends V2CommandExec { - - override lazy val output: Seq[Attribute] = Nil - - override protected def run(): Seq[InternalRow] = { - catalog.loadTable(ident) match { - case iceberg: SparkTable if iceberg.table().properties().containsKey("openhouse.tableId") => - val key = "updated.openhouse.policy" - val value = { - (timeCount, count, logicalOperator) match { - case ttlOnly if count == -1 => s"""{"versionsRetention":{"timeCount":${timeCount},"granularity":"${granularity.get}"}}""" - case countOnly if timeCount == -1 => s"""{"versionsRetention":{"count":${count}}}""" - case _ => s"""{"versionsRetention":{"timeCount":${timeCount},"granularity":"${granularity.get}","count":${count},"logicalOperator":${logicalOperator.get.toString}}}"""} - } - - iceberg.table().updateProperties() - .set(key, value) - .commit() - - case table => - throw new UnsupportedOperationException(s"Cannot set snapshots retention policy for non-Openhouse table: $table") - } - - Nil - } - - override def simpleString(maxFields: Int): String = { - s"SetVersionsRetentionPolicyExec: ${catalog} ${ident} ${if (timeCount > 0) timeCount else ""} ${granularity.getOrElse("")} ${logicalOperator.getOrElse("")} count ${if (count > 0) count else ""}" - } -} From 284fd5faa9813571102eeeb5cda102e9e72a4a49 Mon Sep 17 00:00:00 2001 From: Will-Lo Date: Wed, 4 Dec 2024 22:01:28 -0500 Subject: [PATCH 05/10] Rename retention_snapshot to history, add negative tests --- .../SetSnapshotsRetentionPolicyTest.java | 229 ------------------ .../SetHistoryPolicyStatementTest.java | 112 +++++++++ .../extensions/OpenhouseSqlExtensions.g4 | 10 +- .../OpenhouseSqlExtensionsAstBuilder.scala | 13 +- .../plans/logical/SetHistoryPolicy.scala | 9 + .../logical/SetSnapshotsRetentionPolicy.scala | 9 - .../v2/OpenhouseDataSourceV2Strategy.scala | 6 +- ...yExec.scala => SetHistoryPolicyExec.scala} | 12 +- 8 files changed, 143 insertions(+), 257 deletions(-) delete mode 100644 integrations/spark/spark-3.1/openhouse-spark-itest/src/test/java/com/linkedin/openhouse/spark/e2e/extensions/SetSnapshotsRetentionPolicyTest.java create mode 100644 integrations/spark/spark-3.1/openhouse-spark-itest/src/test/java/com/linkedin/openhouse/spark/statementtest/SetHistoryPolicyStatementTest.java create mode 100644 integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetHistoryPolicy.scala delete mode 100644 integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetSnapshotsRetentionPolicy.scala rename integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/{SetSnapshotsRetentionPolicyExec.scala => SetHistoryPolicyExec.scala} (64%) diff --git a/integrations/spark/spark-3.1/openhouse-spark-itest/src/test/java/com/linkedin/openhouse/spark/e2e/extensions/SetSnapshotsRetentionPolicyTest.java b/integrations/spark/spark-3.1/openhouse-spark-itest/src/test/java/com/linkedin/openhouse/spark/e2e/extensions/SetSnapshotsRetentionPolicyTest.java deleted file mode 100644 index 154baa1b..00000000 --- a/integrations/spark/spark-3.1/openhouse-spark-itest/src/test/java/com/linkedin/openhouse/spark/e2e/extensions/SetSnapshotsRetentionPolicyTest.java +++ /dev/null @@ -1,229 +0,0 @@ -package com.linkedin.openhouse.spark.e2e.extensions; - -import static com.linkedin.openhouse.spark.MockHelpers.*; -import static com.linkedin.openhouse.spark.SparkTestBase.*; - -import com.linkedin.openhouse.spark.SparkTestBase; -import org.apache.iceberg.catalog.TableIdentifier; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; - -@ExtendWith(SparkTestBase.class) -public class SetSnapshotsRetentionPolicyTest { - @Test - public void testSetSnapshotsRetentionPolicyTimeAndCount() { - Object existingTable = - mockGetTableResponseBody( - "dbSetSnapshotsRetention", - "t1", - "c1", - "dbSetSnapshotsRetention.t1", - "u1", - mockTableLocation( - TableIdentifier.of("dbSetSnapshotsRetention", "t1"), - convertSchemaToDDLComponent(baseSchema), - ""), - "V1", - baseSchema, - null, - null); - Object existingOpenhouseTable = - mockGetTableResponseBody( - "dbSetSnapshotsRetention", - "t1", - "c1", - "dbSetSnapshotsRetention.t1", - "u1", - mockTableLocationAfterOperation( - TableIdentifier.of("dbSetSnapshotsRetention", "t1"), - "ALTER TABLE %t SET TBLPROPERTIES ('openhouse.tableId'='t1')"), - "V1", - baseSchema, - null, - null); - Object tableAfterSetSnapshotsRetention = - mockGetTableResponseBody( - "dbSetSnapshotsRetention", - "t1", - "c1", - "dbSetSnapshotsRetention.t1", - "u1", - mockTableLocationAfterOperation( - TableIdentifier.of("dbSetSnapshotsRetention", "t1"), - "ALTER TABLE %t SET TBLPROPERTIES('policies'='{\"snapshotsRetention\":{\"timeCount\":24,\"granularity\":" - + "\"HOUR\",\"count\":10 }}')"), - "V2", - baseSchema, - null, - null); - mockTableService.enqueue(mockResponse(200, existingOpenhouseTable)); // doRefresh() - mockTableService.enqueue(mockResponse(200, existingOpenhouseTable)); // doRefresh() - mockTableService.enqueue(mockResponse(201, tableAfterSetSnapshotsRetention)); // doCommit() - String ddlWithSchema = - "ALTER TABLE openhouse.dbSetSnapshotsRetention.t1 SET POLICY (SNAPSHOT_RETENTION TIME=24H COUNT=10)"; - Assertions.assertDoesNotThrow(() -> spark.sql(ddlWithSchema)); - } - - @Test - public void testSetSnapshotsRetentionPolicyTimeOrCount() { - Object existingTable = - mockGetTableResponseBody( - "dbSetSnapshotsRetention", - "t2", - "c1", - "dbSetSnapshotsRetention.t2", - "u1", - mockTableLocation( - TableIdentifier.of("dbSetSnapshotsRetention", "t2"), - convertSchemaToDDLComponent(baseSchema), - ""), - "V1", - baseSchema, - null, - null); - Object existingOpenhouseTable = - mockGetTableResponseBody( - "dbSetSnapshotsRetention", - "t2", - "c1", - "dbSetSnapshotsRetention.t2", - "u1", - mockTableLocationAfterOperation( - TableIdentifier.of("dbSetSnapshotsRetention", "t2"), - "ALTER TABLE %t SET TBLPROPERTIES ('openhouse.tableId'='t2')"), - "V1", - baseSchema, - null, - null); - Object tableAfterSetSnapshotsRetention = - mockGetTableResponseBody( - "dbSetSnapshotsRetention", - "t1", - "c1", - "dbSetSnapshotsRetention.t2", - "u1", - mockTableLocationAfterOperation( - TableIdentifier.of("dbSetSnapshotsRetention", "t2"), - "ALTER TABLE %t SET TBLPROPERTIES('policies'='{\"snapshotsRetention\":{\"timeCount\":24,\"granularity\":" - + "\"HOUR\",\"count\":10 }}')"), - "V2", - baseSchema, - null, - null); - mockTableService.enqueue(mockResponse(200, existingOpenhouseTable)); // doRefresh() - mockTableService.enqueue(mockResponse(200, existingOpenhouseTable)); // doRefresh() - mockTableService.enqueue(mockResponse(201, tableAfterSetSnapshotsRetention)); // doCommit() - String ddlWithSchema = - "ALTER TABLE openhouse.dbSetSnapshotsRetention.t2 SET POLICY (VERSIONS TIME=24H COUNT=10)"; - Assertions.assertDoesNotThrow(() -> spark.sql(ddlWithSchema)); - } - - @Test - public void testSetSnapshotsRetentionPolicyTimeOnly() { - Object existingTable = - mockGetTableResponseBody( - "dbSetSnapshotsRetention", - "t3", - "c1", - "dbSetSnapshotsRetention.t3", - "u1", - mockTableLocation( - TableIdentifier.of("dbSetSnapshotsRetention", "t3"), - convertSchemaToDDLComponent(baseSchema), - ""), - "V1", - baseSchema, - null, - null); - Object existingOpenhouseTable = - mockGetTableResponseBody( - "dbSetSnapshotsRetention", - "t3", - "c1", - "dbSetSnapshotsRetention.t3", - "u1", - mockTableLocationAfterOperation( - TableIdentifier.of("dbSetSnapshotsRetention", "t3"), - "ALTER TABLE %t SET TBLPROPERTIES ('openhouse.tableId'='t3')"), - "V1", - baseSchema, - null, - null); - Object tableAfterSetSnapshotsRetention = - mockGetTableResponseBody( - "dbSetSnapshotsRetention", - "t3", - "c1", - "dbSetSnapshotsRetention.t3", - "u1", - mockTableLocationAfterOperation( - TableIdentifier.of("dbSetSnapshotsRetention", "t3"), - "ALTER TABLE %t SET TBLPROPERTIES('policies'='{\"snapshotsRetention\":{\"timeCount\":24,\"granularity\":" - + "\"HOUR\"}}')"), - "V2", - baseSchema, - null, - null); - mockTableService.enqueue(mockResponse(200, existingOpenhouseTable)); // doRefresh() - mockTableService.enqueue(mockResponse(200, existingOpenhouseTable)); // doRefresh() - mockTableService.enqueue(mockResponse(201, tableAfterSetSnapshotsRetention)); // doCommit() - String ddlWithSchema = - "ALTER TABLE openhouse.dbSetSnapshotsRetention.t3 SET POLICY (VERSIONS TIME=24H)"; - Assertions.assertDoesNotThrow(() -> spark.sql(ddlWithSchema)); - } - - @Test - public void testSetSnapshotsRetentionPolicyCountOnly() { - Object existingTable = - mockGetTableResponseBody( - "dbSetSnapshotsRetention", - "t4", - "c1", - "dbSetSnapshotsRetention.t4", - "u1", - mockTableLocation( - TableIdentifier.of("dbSetSnapshotsRetention", "t4"), - convertSchemaToDDLComponent(baseSchema), - ""), - "V1", - baseSchema, - null, - null); - Object existingOpenhouseTable = - mockGetTableResponseBody( - "dbSetSnapshotsRetention", - "t4", - "c1", - "dbSetSnapshotsRetention.t4", - "u1", - mockTableLocationAfterOperation( - TableIdentifier.of("dbSetSnapshotsRetention", "t4"), - "ALTER TABLE %t SET TBLPROPERTIES ('openhouse.tableId'='t4')"), - "V1", - baseSchema, - null, - null); - Object tableAfterSetSnapshotsRetention = - mockGetTableResponseBody( - "dbSetSnapshotsRetention", - "t4", - "c1", - "dbSetSnapshotsRetention.t4", - "u1", - mockTableLocationAfterOperation( - TableIdentifier.of("dbSetSnapshotsRetention", "t4"), - "ALTER TABLE %t SET TBLPROPERTIES('policies'='{\"snapshotsRetention\":{\"timeCount\":24,\"granularity\":" - + "\"HOUR\"}}')"), - "V2", - baseSchema, - null, - null); - mockTableService.enqueue(mockResponse(200, existingOpenhouseTable)); // doRefresh() - mockTableService.enqueue(mockResponse(200, existingOpenhouseTable)); // doRefresh() - mockTableService.enqueue(mockResponse(201, tableAfterSetSnapshotsRetention)); // doCommit() - String ddlWithSchema = - "ALTER TABLE openhouse.dbSetSnapshotsRetention.t4 SET POLICY (VERSIONS COUNT=10)"; - Assertions.assertDoesNotThrow(() -> spark.sql(ddlWithSchema)); - } -} diff --git a/integrations/spark/spark-3.1/openhouse-spark-itest/src/test/java/com/linkedin/openhouse/spark/statementtest/SetHistoryPolicyStatementTest.java b/integrations/spark/spark-3.1/openhouse-spark-itest/src/test/java/com/linkedin/openhouse/spark/statementtest/SetHistoryPolicyStatementTest.java new file mode 100644 index 00000000..998ac8f6 --- /dev/null +++ b/integrations/spark/spark-3.1/openhouse-spark-itest/src/test/java/com/linkedin/openhouse/spark/statementtest/SetHistoryPolicyStatementTest.java @@ -0,0 +1,112 @@ +package com.linkedin.openhouse.spark.statementtest; + +import com.linkedin.openhouse.spark.sql.catalyst.parser.extensions.OpenhouseParseException; +import java.nio.file.Files; +import java.util.Optional; +import lombok.SneakyThrows; +import org.apache.hadoop.fs.Path; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.execution.ExplainMode; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; + +@TestInstance(TestInstance.Lifecycle.PER_CLASS) +public class SetHistoryPolicyStatementTest { + private static SparkSession spark = null; + + @SneakyThrows + @BeforeAll + public void setupSpark() { + Path unittest = new Path(Files.createTempDirectory("unittest_settablepolicy").toString()); + spark = + SparkSession.builder() + .master("local[2]") + .config( + "spark.sql.extensions", + ("org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions," + + "com.linkedin.openhouse.spark.extensions.OpenhouseSparkSessionExtensions")) + .config("spark.sql.catalog.openhouse", "org.apache.iceberg.spark.SparkCatalog") + .config("spark.sql.catalog.openhouse.type", "hadoop") + .config("spark.sql.catalog.openhouse.warehouse", unittest.toString()) + .getOrCreate(); + } + + @Test + public void testSetHistoryPolicyGood() { + // Validate setting only time setting + Dataset ds = spark.sql("ALTER TABLE openhouse.db.table SET POLICY (HISTORY TIME=24H)"); + assert isPlanValid(ds, "db.table", Optional.of("24"), Optional.of("HOUR"), Optional.empty()); + + ds = spark.sql("ALTER TABLE openhouse.db.table SET POLICY (HISTORY COUNT=10)"); + assert isPlanValid(ds, "db.table", Optional.empty(), Optional.empty(), Optional.of("10")); + + // Validate both time and count setting + ds = spark.sql("ALTER TABLE openhouse.db.table SET POLICY (HISTORY TIME=2D COUNT=20)"); + assert isPlanValid(ds, "db.table", Optional.of("2"), Optional.of("DAY"), Optional.of("20")); + } + + @Test + public void testSetHistoryPolicyIncorrectSyntax() { + // No time granularity + Assertions.assertThrows( + OpenhouseParseException.class, + () -> spark.sql("ALTER TABLE openhouse.db.table SET POLICY (HISTORY TIME=24)").show()); + + // Count before time + Assertions.assertThrows( + OpenhouseParseException.class, + () -> + spark + .sql("ALTER TABLE openhouse.db.table SET POLICY (HISTORY COUNT=10 TIME=24H)") + .show()); + + // No time or count + Assertions.assertThrows( + OpenhouseParseException.class, + () -> spark.sql("ALTER TABLE openhouse.db.table SET POLICY (HISTORY )").show()); + } + + @BeforeEach + public void setup() { + spark.sql("CREATE TABLE openhouse.db.table (id bigint, data string) USING iceberg").show(); + spark.sql("CREATE TABLE openhouse.0_.0_ (id bigint, data string) USING iceberg").show(); + spark + .sql("ALTER TABLE openhouse.db.table SET TBLPROPERTIES ('openhouse.tableId' = 'tableid')") + .show(); + spark + .sql("ALTER TABLE openhouse.0_.0_ SET TBLPROPERTIES ('openhouse.tableId' = 'tableid')") + .show(); + } + + @AfterEach + public void tearDown() { + spark.sql("DROP TABLE openhouse.db.table").show(); + spark.sql("DROP TABLE openhouse.0_.0_").show(); + } + + @AfterAll + public void tearDownSpark() { + spark.close(); + } + + @SneakyThrows + private boolean isPlanValid( + Dataset dataframe, + String dbTable, + Optional timeCount, + Optional granularity, + Optional versionCount) { + String queryStr = dataframe.queryExecution().explainString(ExplainMode.fromString("simple")); + return queryStr.contains(dbTable) + && (!timeCount.isPresent() || queryStr.contains(timeCount.get())) + && (!granularity.isPresent() || queryStr.contains(granularity.get())) + && (!versionCount.isPresent() || queryStr.contains(versionCount.get())); + } +} diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/antlr/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensions.g4 b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/antlr/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensions.g4 index fea71c47..df34bd88 100644 --- a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/antlr/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensions.g4 +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/antlr/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensions.g4 @@ -26,7 +26,7 @@ statement : ALTER TABLE multipartIdentifier SET POLICY '(' retentionPolicy (columnRetentionPolicy)? ')' #setRetentionPolicy | ALTER TABLE multipartIdentifier SET POLICY '(' replicationPolicy ')' #setReplicationPolicy | ALTER TABLE multipartIdentifier SET POLICY '(' sharingPolicy ')' #setSharingPolicy - | ALTER TABLE multipartIdentifier SET POLICY '(' snapshotRetentionPolicy ')' #setSnapshotRetentionPolicy + | ALTER TABLE multipartIdentifier SET POLICY '(' historyPolicy ')' #setHistoryPolicy | ALTER TABLE multipartIdentifier MODIFY columnNameClause SET columnPolicy #setColumnPolicyTag | GRANT privilege ON grantableResource TO principal #grantStatement | REVOKE privilege ON grantableResource FROM principal #revokeStatement @@ -66,7 +66,7 @@ quotedIdentifier ; nonReserved - : ALTER | TABLE | SET | POLICY | RETENTION | SHARING | REPLICATION | SNAPSHOT_RETENTION + : ALTER | TABLE | SET | POLICY | RETENTION | SHARING | REPLICATION | HISTORY | GRANT | REVOKE | ON | TO | SHOW | GRANTS | PATTERN | WHERE | COLUMN ; @@ -155,8 +155,8 @@ policyTag : PII | HC ; -snapshotRetentionPolicy - : SNAPSHOT_RETENTION retainTime? versionsCount? +historyPolicy + : HISTORY retainTime? versionsCount? ; retainTime @@ -173,7 +173,7 @@ SET: 'SET'; POLICY: 'POLICY'; RETENTION: 'RETENTION'; REPLICATION: 'REPLICATION'; -SNAPSHOT_RETENTION: 'SNAPSHOT_RETENTION'; +HISTORY: 'HISTORY'; SHARING: 'SHARING'; GRANT: 'GRANT'; REVOKE: 'REVOKE'; diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensionsAstBuilder.scala b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensionsAstBuilder.scala index 8dd98dc2..e52f08e3 100644 --- a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensionsAstBuilder.scala +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensionsAstBuilder.scala @@ -2,7 +2,7 @@ package com.linkedin.openhouse.spark.sql.catalyst.parser.extensions import com.linkedin.openhouse.spark.sql.catalyst.enums.GrantableResourceTypes import com.linkedin.openhouse.spark.sql.catalyst.parser.extensions.OpenhouseSqlExtensionsParser._ -import com.linkedin.openhouse.spark.sql.catalyst.plans.logical.{GrantRevokeStatement, SetColumnPolicyTag, SetReplicationPolicy, SetRetentionPolicy, SetSharingPolicy, SetSnapshotsRetentionPolicy, ShowGrantsStatement} +import com.linkedin.openhouse.spark.sql.catalyst.plans.logical.{GrantRevokeStatement, SetColumnPolicyTag, SetHistoryPolicy, SetReplicationPolicy, SetRetentionPolicy, SetSharingPolicy, ShowGrantsStatement} import com.linkedin.openhouse.spark.sql.catalyst.enums.GrantableResourceTypes.GrantableResourceType import com.linkedin.openhouse.gen.tables.client.model.TimePartitionSpec import org.antlr.v4.runtime.tree.ParseTree @@ -163,18 +163,21 @@ class OpenhouseSqlExtensionsAstBuilder (delegate: ParserInterface) extends Openh (granularity, count) } - override def visitSetSnapshotRetentionPolicy(ctx: SetSnapshotRetentionPolicyContext): SetSnapshotsRetentionPolicy = { + override def visitSetHistoryPolicy(ctx: SetHistoryPolicyContext): SetHistoryPolicy = { val tableName = typedVisit[Seq[String]](ctx.multipartIdentifier) - val (granularity, timeCount, count) = typedVisit[(Option[String], Int, Int)](ctx.snapshotRetentionPolicy()) - SetSnapshotsRetentionPolicy(tableName, granularity, timeCount, count) + val (granularity, timeCount, count) = typedVisit[(Option[String], Int, Int)](ctx.historyPolicy()) + SetHistoryPolicy(tableName, granularity, timeCount, count) } - override def visitSnapshotRetentionPolicy(ctx: SnapshotRetentionPolicyContext): (Option[String], Int, Int) = { + override def visitHistoryPolicy(ctx: HistoryPolicyContext): (Option[String], Int, Int) = { val timePolicy = if (ctx.retainTime() != null) typedVisit[(String, Int)](ctx.retainTime().duration()) else (null, -1) val countPolicy = if (ctx.versionsCount() != null) typedVisit[Int](ctx.versionsCount()) else -1 + if (timePolicy._2 == -1 && countPolicy == -1) { + throw new OpenhouseParseException("Either TIME or VERSIONS must be specified in HISTORY policy", ctx.start.getLine, ctx.start.getCharPositionInLine) + } (Option(timePolicy._1), timePolicy._2, countPolicy) } diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetHistoryPolicy.scala b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetHistoryPolicy.scala new file mode 100644 index 00000000..f2fa87d8 --- /dev/null +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetHistoryPolicy.scala @@ -0,0 +1,9 @@ +package com.linkedin.openhouse.spark.sql.catalyst.plans.logical + +import org.apache.spark.sql.catalyst.plans.logical.Command + +case class SetHistoryPolicy (tableName: Seq[String], granularity: Option[String], timeCount: Int, count: Int) extends Command { + override def simpleString(maxFields: Int): String = { + s"SetHistoryPolicy: ${tableName} ${if (timeCount > 0) "TIME=" + timeCount else ""}${granularity.getOrElse("")} ${if (count > 0) "VERSIONS=" + count else ""}" + } +} diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetSnapshotsRetentionPolicy.scala b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetSnapshotsRetentionPolicy.scala deleted file mode 100644 index ddc7acfa..00000000 --- a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetSnapshotsRetentionPolicy.scala +++ /dev/null @@ -1,9 +0,0 @@ -package com.linkedin.openhouse.spark.sql.catalyst.plans.logical - -import org.apache.spark.sql.catalyst.plans.logical.Command - -case class SetSnapshotsRetentionPolicy (tableName: Seq[String], granularity: Option[String], timeCount: Int, count: Int) extends Command { - override def simpleString(maxFields: Int): String = { - s"SetSnapshotsRetentionPolicy: ${tableName} ${if (timeCount > 0) timeCount else ""} ${granularity.getOrElse("")} count ${if (count > 0) count else ""}" - } -} diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/OpenhouseDataSourceV2Strategy.scala b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/OpenhouseDataSourceV2Strategy.scala index db8c6304..3d0d9c38 100644 --- a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/OpenhouseDataSourceV2Strategy.scala +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/OpenhouseDataSourceV2Strategy.scala @@ -1,6 +1,6 @@ package com.linkedin.openhouse.spark.sql.execution.datasources.v2 -import com.linkedin.openhouse.spark.sql.catalyst.plans.logical.{GrantRevokeStatement, SetColumnPolicyTag, SetReplicationPolicy, SetRetentionPolicy, SetSharingPolicy, SetSnapshotsRetentionPolicy, ShowGrantsStatement} +import com.linkedin.openhouse.spark.sql.catalyst.plans.logical.{GrantRevokeStatement, SetColumnPolicyTag, SetReplicationPolicy, SetRetentionPolicy, SetSharingPolicy, SetHistoryPolicy, ShowGrantsStatement} import org.apache.iceberg.spark.{Spark3Util, SparkCatalog, SparkSessionCatalog} import org.apache.spark.sql.{SparkSession, Strategy} import org.apache.spark.sql.catalyst.expressions.PredicateHelper @@ -17,8 +17,8 @@ case class OpenhouseDataSourceV2Strategy(spark: SparkSession) extends Strategy w SetRetentionPolicyExec(catalog, ident, granularity, count, colName, colPattern) :: Nil case SetReplicationPolicy(CatalogAndIdentifierExtractor(catalog, ident), replicationPolicies) => SetReplicationPolicyExec(catalog, ident, replicationPolicies) :: Nil - case SetSnapshotsRetentionPolicy(CatalogAndIdentifierExtractor(catalog, ident), granularity, timeCount, count) => - SetSnapshotsRetentionPolicyExec(catalog, ident, granularity, timeCount, count) :: Nil + case SetHistoryPolicy(CatalogAndIdentifierExtractor(catalog, ident), granularity, timeCount, count) => + SetHistoryPolicyExec(catalog, ident, granularity, timeCount, count) :: Nil case SetSharingPolicy(CatalogAndIdentifierExtractor(catalog, ident), sharing) => SetSharingPolicyExec(catalog, ident, sharing) :: Nil case SetColumnPolicyTag(CatalogAndIdentifierExtractor(catalog, ident), policyTag, cols) => diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetSnapshotsRetentionPolicyExec.scala b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetHistoryPolicyExec.scala similarity index 64% rename from integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetSnapshotsRetentionPolicyExec.scala rename to integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetHistoryPolicyExec.scala index 6cd015ac..ab999444 100644 --- a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetSnapshotsRetentionPolicyExec.scala +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetHistoryPolicyExec.scala @@ -6,7 +6,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} import org.apache.spark.sql.execution.datasources.v2.V2CommandExec -case class SetSnapshotsRetentionPolicyExec( +case class SetHistoryPolicyExec( catalog: TableCatalog, ident: Identifier, granularity: Option[String], @@ -22,9 +22,9 @@ case class SetSnapshotsRetentionPolicyExec( val key = "updated.openhouse.policy" val value = { (timeCount, count) match { - case ttlOnly if count == -1 => s"""{"snapshotsRetention":{"timeCount":${timeCount},"granularity":"${granularity.get}"}}""" - case countOnly if timeCount == -1 => s"""{"snapshotsRetention":{"count":${count}}}""" - case _ => s"""{"snapshotsRetention":{"timeCount":${timeCount},"granularity":"${granularity.get}","count":${count}}}""" + case ttlOnly if count == -1 => s"""{"history":{"timeCount":${timeCount},"granularity":"${granularity.get}"}}""" + case countOnly if timeCount == -1 => s"""{"history":{"count":${count}}}""" + case _ => s"""{"history":{"timeCount":${timeCount},"granularity":"${granularity.get}","count":${count}}}""" } } @@ -33,13 +33,13 @@ case class SetSnapshotsRetentionPolicyExec( .commit() case table => - throw new UnsupportedOperationException(s"Cannot set snapshot retention policy for non-Openhouse table: $table") + throw new UnsupportedOperationException(s"Cannot set history policy for non-Openhouse table: $table") } Nil } override def simpleString(maxFields: Int): String = { - s"SetSnapshotsRetentionPolicyExec: ${catalog} ${ident} ${if (timeCount > 0) timeCount else ""} ${granularity.getOrElse("")} count ${if (count > 0) count else ""}" + s"SetHistoryPolicyExec: ${catalog} ${ident} ${if (timeCount > 0) timeCount else ""} ${granularity.getOrElse("")} count ${if (count > 0) count else ""}" } } From ba5709ad9ac86f220a9533bcb0171d16b65a6b45 Mon Sep 17 00:00:00 2001 From: Will-Lo Date: Thu, 5 Dec 2024 14:18:59 -0500 Subject: [PATCH 06/10] Rerun tests From 6b7999a088e302df8949a7b3dd4181858f772372 Mon Sep 17 00:00:00 2001 From: Will-Lo Date: Fri, 6 Dec 2024 14:15:48 -0500 Subject: [PATCH 07/10] Update fields for history policy --- .../SetHistoryPolicyStatementTest.java | 20 ++++++++++--------- .../extensions/OpenhouseSqlExtensions.g4 | 15 +++++++------- .../OpenhouseSqlExtensionsAstBuilder.scala | 18 ++++++++--------- .../plans/logical/SetHistoryPolicy.scala | 4 ++-- .../v2/OpenhouseDataSourceV2Strategy.scala | 4 ++-- .../datasources/v2/SetHistoryPolicyExec.scala | 14 ++++++------- 6 files changed, 38 insertions(+), 37 deletions(-) diff --git a/integrations/spark/spark-3.1/openhouse-spark-itest/src/test/java/com/linkedin/openhouse/spark/statementtest/SetHistoryPolicyStatementTest.java b/integrations/spark/spark-3.1/openhouse-spark-itest/src/test/java/com/linkedin/openhouse/spark/statementtest/SetHistoryPolicyStatementTest.java index 998ac8f6..1bf412ef 100644 --- a/integrations/spark/spark-3.1/openhouse-spark-itest/src/test/java/com/linkedin/openhouse/spark/statementtest/SetHistoryPolicyStatementTest.java +++ b/integrations/spark/spark-3.1/openhouse-spark-itest/src/test/java/com/linkedin/openhouse/spark/statementtest/SetHistoryPolicyStatementTest.java @@ -41,14 +41,15 @@ public void setupSpark() { @Test public void testSetHistoryPolicyGood() { // Validate setting only time setting - Dataset ds = spark.sql("ALTER TABLE openhouse.db.table SET POLICY (HISTORY TIME=24H)"); + Dataset ds = spark.sql("ALTER TABLE openhouse.db.table SET POLICY (HISTORY MAX_AGE=24H)"); assert isPlanValid(ds, "db.table", Optional.of("24"), Optional.of("HOUR"), Optional.empty()); - ds = spark.sql("ALTER TABLE openhouse.db.table SET POLICY (HISTORY COUNT=10)"); + ds = spark.sql("ALTER TABLE openhouse.db.table SET POLICY (HISTORY MIN_VERSIONS=10)"); assert isPlanValid(ds, "db.table", Optional.empty(), Optional.empty(), Optional.of("10")); // Validate both time and count setting - ds = spark.sql("ALTER TABLE openhouse.db.table SET POLICY (HISTORY TIME=2D COUNT=20)"); + ds = + spark.sql("ALTER TABLE openhouse.db.table SET POLICY (HISTORY MAX_AGE=2D MIN_VERSIONS=20)"); assert isPlanValid(ds, "db.table", Optional.of("2"), Optional.of("DAY"), Optional.of("20")); } @@ -57,14 +58,15 @@ public void testSetHistoryPolicyIncorrectSyntax() { // No time granularity Assertions.assertThrows( OpenhouseParseException.class, - () -> spark.sql("ALTER TABLE openhouse.db.table SET POLICY (HISTORY TIME=24)").show()); + () -> spark.sql("ALTER TABLE openhouse.db.table SET POLICY (HISTORY MAX_AGE=24)").show()); // Count before time Assertions.assertThrows( OpenhouseParseException.class, () -> spark - .sql("ALTER TABLE openhouse.db.table SET POLICY (HISTORY COUNT=10 TIME=24H)") + .sql( + "ALTER TABLE openhouse.db.table SET POLICY (HISTORY MIN_VERSIONS=10 MAX_AGE=24H)") .show()); // No time or count @@ -100,13 +102,13 @@ public void tearDownSpark() { private boolean isPlanValid( Dataset dataframe, String dbTable, - Optional timeCount, + Optional maxAge, Optional granularity, - Optional versionCount) { + Optional minVersions) { String queryStr = dataframe.queryExecution().explainString(ExplainMode.fromString("simple")); return queryStr.contains(dbTable) - && (!timeCount.isPresent() || queryStr.contains(timeCount.get())) + && (!maxAge.isPresent() || queryStr.contains(maxAge.get())) && (!granularity.isPresent() || queryStr.contains(granularity.get())) - && (!versionCount.isPresent() || queryStr.contains(versionCount.get())); + && (!minVersions.isPresent() || queryStr.contains(minVersions.get())); } } diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/antlr/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensions.g4 b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/antlr/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensions.g4 index df34bd88..ca576211 100644 --- a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/antlr/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensions.g4 +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/antlr/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensions.g4 @@ -156,15 +156,15 @@ policyTag ; historyPolicy - : HISTORY retainTime? versionsCount? + : HISTORY maxAge? minVersions? ; -retainTime - : TIME'='duration +maxAge + : MAX_AGE'='duration ; -versionsCount - : COUNT'='POSITIVE_INTEGER +minVersions + : MIN_VERSIONS'='POSITIVE_INTEGER ; ALTER: 'ALTER'; @@ -197,9 +197,8 @@ HC: 'HC'; MODIFY: 'MODIFY'; TAG: 'TAG'; NONE: 'NONE'; -VERSIONS: 'VERSIONS'; -TIME: 'TIME'; -COUNT: 'COUNT'; +MIN_VERSIONS: 'MIN_VERSIONS'; +MAX_AGE: 'MAX_AGE'; POSITIVE_INTEGER : DIGIT+ diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensionsAstBuilder.scala b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensionsAstBuilder.scala index e52f08e3..141d2d04 100644 --- a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensionsAstBuilder.scala +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensionsAstBuilder.scala @@ -165,23 +165,23 @@ class OpenhouseSqlExtensionsAstBuilder (delegate: ParserInterface) extends Openh override def visitSetHistoryPolicy(ctx: SetHistoryPolicyContext): SetHistoryPolicy = { val tableName = typedVisit[Seq[String]](ctx.multipartIdentifier) - val (granularity, timeCount, count) = typedVisit[(Option[String], Int, Int)](ctx.historyPolicy()) - SetHistoryPolicy(tableName, granularity, timeCount, count) + val (granularity, maxAge, minVersions) = typedVisit[(Option[String], Int, Int)](ctx.historyPolicy()) + SetHistoryPolicy(tableName, granularity, maxAge, minVersions) } override def visitHistoryPolicy(ctx: HistoryPolicyContext): (Option[String], Int, Int) = { - val timePolicy = if (ctx.retainTime() != null) - typedVisit[(String, Int)](ctx.retainTime().duration()) + val maxAgePolicy = if (ctx.maxAge() != null) + typedVisit[(String, Int)](ctx.maxAge().duration()) else (null, -1) - val countPolicy = if (ctx.versionsCount() != null) - typedVisit[Int](ctx.versionsCount()) + val minVersionPolicy = if (ctx.minVersions() != null) + typedVisit[Int](ctx.minVersions()) else -1 - if (timePolicy._2 == -1 && countPolicy == -1) { + if (maxAgePolicy._2 == -1 && minVersionPolicy == -1) { throw new OpenhouseParseException("Either TIME or VERSIONS must be specified in HISTORY policy", ctx.start.getLine, ctx.start.getCharPositionInLine) } - (Option(timePolicy._1), timePolicy._2, countPolicy) + (Option(maxAgePolicy._1), maxAgePolicy._2, minVersionPolicy) } - override def visitVersionsCount(ctx: VersionsCountContext): Integer = { + override def visitMinVersions(ctx: MinVersionsContext): Integer = { ctx.POSITIVE_INTEGER().getText.toInt } diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetHistoryPolicy.scala b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetHistoryPolicy.scala index f2fa87d8..c064d12a 100644 --- a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetHistoryPolicy.scala +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetHistoryPolicy.scala @@ -2,8 +2,8 @@ package com.linkedin.openhouse.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.Command -case class SetHistoryPolicy (tableName: Seq[String], granularity: Option[String], timeCount: Int, count: Int) extends Command { +case class SetHistoryPolicy (tableName: Seq[String], granularity: Option[String], maxAge: Int, minVersions: Int) extends Command { override def simpleString(maxFields: Int): String = { - s"SetHistoryPolicy: ${tableName} ${if (timeCount > 0) "TIME=" + timeCount else ""}${granularity.getOrElse("")} ${if (count > 0) "VERSIONS=" + count else ""}" + s"SetHistoryPolicy: ${tableName} ${if (maxAge > 0) "MAX_AGE=" + maxAge else ""}${granularity.getOrElse("")} ${if (minVersions > 0) "MIN_VERSIONS=" + minVersions else ""}" } } diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/OpenhouseDataSourceV2Strategy.scala b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/OpenhouseDataSourceV2Strategy.scala index 3d0d9c38..d1478c42 100644 --- a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/OpenhouseDataSourceV2Strategy.scala +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/OpenhouseDataSourceV2Strategy.scala @@ -17,8 +17,8 @@ case class OpenhouseDataSourceV2Strategy(spark: SparkSession) extends Strategy w SetRetentionPolicyExec(catalog, ident, granularity, count, colName, colPattern) :: Nil case SetReplicationPolicy(CatalogAndIdentifierExtractor(catalog, ident), replicationPolicies) => SetReplicationPolicyExec(catalog, ident, replicationPolicies) :: Nil - case SetHistoryPolicy(CatalogAndIdentifierExtractor(catalog, ident), granularity, timeCount, count) => - SetHistoryPolicyExec(catalog, ident, granularity, timeCount, count) :: Nil + case SetHistoryPolicy(CatalogAndIdentifierExtractor(catalog, ident), granularity, maxAge, minVersions) => + SetHistoryPolicyExec(catalog, ident, granularity, maxAge, minVersions) :: Nil case SetSharingPolicy(CatalogAndIdentifierExtractor(catalog, ident), sharing) => SetSharingPolicyExec(catalog, ident, sharing) :: Nil case SetColumnPolicyTag(CatalogAndIdentifierExtractor(catalog, ident), policyTag, cols) => diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetHistoryPolicyExec.scala b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetHistoryPolicyExec.scala index ab999444..26da42a0 100644 --- a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetHistoryPolicyExec.scala +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetHistoryPolicyExec.scala @@ -10,8 +10,8 @@ case class SetHistoryPolicyExec( catalog: TableCatalog, ident: Identifier, granularity: Option[String], - timeCount: Int, - count: Int + maxAge: Int, + minVersions: Int ) extends V2CommandExec { override lazy val output: Seq[Attribute] = Nil @@ -21,10 +21,10 @@ case class SetHistoryPolicyExec( case iceberg: SparkTable if iceberg.table().properties().containsKey("openhouse.tableId") => val key = "updated.openhouse.policy" val value = { - (timeCount, count) match { - case ttlOnly if count == -1 => s"""{"history":{"timeCount":${timeCount},"granularity":"${granularity.get}"}}""" - case countOnly if timeCount == -1 => s"""{"history":{"count":${count}}}""" - case _ => s"""{"history":{"timeCount":${timeCount},"granularity":"${granularity.get}","count":${count}}}""" + (maxAge, minVersions) match { + case maxAgeOnly if minVersions == -1 => s"""{"history":{"maxAge":${maxAge},"granularity":"${granularity.get}"}}""" + case minVersionsOnly if maxAge == -1 => s"""{"history":{"minVersions":${minVersions}}}""" + case _ => s"""{"history":{"maxAge":${maxAge},"granularity":"${granularity.get}","minVersions":${minVersions}}}""" } } @@ -40,6 +40,6 @@ case class SetHistoryPolicyExec( } override def simpleString(maxFields: Int): String = { - s"SetHistoryPolicyExec: ${catalog} ${ident} ${if (timeCount > 0) timeCount else ""} ${granularity.getOrElse("")} count ${if (count > 0) count else ""}" + s"SetHistoryPolicyExec: ${catalog} ${ident} MAX_AGE=${if (maxAge > 0) maxAge else ""}${granularity.getOrElse("")} MIN_VERSIONS=${if (minVersions > 0) minVersions else ""}" } } From 244b02bde75c13c929e803caddbb08d88e457dd6 Mon Sep 17 00:00:00 2001 From: Will-Lo Date: Fri, 6 Dec 2024 15:51:46 -0500 Subject: [PATCH 08/10] Add spark 3.5 support for history policy --- .../plans/logical/SetHistoryPolicy.scala | 9 ++++ .../datasources/v2/SetHistoryPolicyExec.scala | 45 +++++++++++++++++++ 2 files changed, 54 insertions(+) create mode 100644 integrations/spark/spark-3.5/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetHistoryPolicy.scala create mode 100644 integrations/spark/spark-3.5/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetHistoryPolicyExec.scala diff --git a/integrations/spark/spark-3.5/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetHistoryPolicy.scala b/integrations/spark/spark-3.5/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetHistoryPolicy.scala new file mode 100644 index 00000000..1c4f2603 --- /dev/null +++ b/integrations/spark/spark-3.5/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetHistoryPolicy.scala @@ -0,0 +1,9 @@ +package com.linkedin.openhouse.spark.sql.catalyst.plans.logical + +import org.apache.spark.sql.catalyst.plans.logical.LeafCommand + +case class SetHistoryPolicy(tableName: Seq[String], granularity: Option[String], maxAge: Int, minVersions: Int) extends LeafCommand { + override def simpleString(maxFields: Int): String = { + s"SetHistoryPolicy: ${tableName} ${if (maxAge > 0) "MAX_AGE=" + maxAge else ""}${granularity.getOrElse("")} ${if (minVersions > 0) "MIN_VERSIONS=" + minVersions else ""}" + } +} diff --git a/integrations/spark/spark-3.5/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetHistoryPolicyExec.scala b/integrations/spark/spark-3.5/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetHistoryPolicyExec.scala new file mode 100644 index 00000000..9f6b7d7c --- /dev/null +++ b/integrations/spark/spark-3.5/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetHistoryPolicyExec.scala @@ -0,0 +1,45 @@ +package com.linkedin.openhouse.spark.sql.execution.datasources.v2 + +import org.apache.iceberg.spark.source.SparkTable +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} +import org.apache.spark.sql.execution.datasources.v2.LeafV2CommandExec + +case class SetHistoryPolicyExec( + catalog: TableCatalog, + ident: Identifier, + granularity: Option[String], + maxAge: Int, + minVersions: Int +) extends LeafV2CommandExec { + + override lazy val output: Seq[Attribute] = Nil + + override protected def run(): Seq[InternalRow] = { + catalog.loadTable(ident) match { + case iceberg: SparkTable if iceberg.table().properties().containsKey("openhouse.tableId") => + val key = "updated.openhouse.policy" + val value = { + (maxAge, minVersions) match { + case maxAgeOnly if minVersions == -1 => s"""{"history":{"maxAge":${maxAge},"granularity":"${granularity.get}"}}""" + case minVersionsOnly if maxAge == -1 => s"""{"history":{"minVersions":${minVersions}}}""" + case _ => s"""{"history":{"maxAge":${maxAge},"granularity":"${granularity.get}","minVersions":${minVersions}}}""" + } + } + + iceberg.table().updateProperties() + .set(key, value) + .commit() + + case table => + throw new UnsupportedOperationException(s"Cannot set history policy for non-Openhouse table: $table") + } + + Nil + } + + override def simpleString(maxFields: Int): String = { + s"SetHistoryPolicyExec: ${catalog} ${ident} MAX_AGE=${if (maxAge > 0) maxAge else ""}${granularity.getOrElse("")} MIN_VERSIONS=${if (minVersions > 0) minVersions else ""}" + } +} From fa105bbd3c7c901ea829ec184014e32e3300489a Mon Sep 17 00:00:00 2001 From: Will-Lo Date: Mon, 16 Dec 2024 14:48:41 -0500 Subject: [PATCH 09/10] Rename min versions to versions syntax --- .../SetHistoryPolicyStatementTest.java | 12 +++++------- .../parser/extensions/OpenhouseSqlExtensions.g4 | 8 ++++---- .../OpenhouseSqlExtensionsAstBuilder.scala | 16 ++++++++-------- .../plans/logical/SetHistoryPolicy.scala | 4 ++-- .../v2/OpenhouseDataSourceV2Strategy.scala | 4 ++-- .../datasources/v2/SetHistoryPolicyExec.scala | 12 ++++++------ .../plans/logical/SetHistoryPolicy.scala | 4 ++-- .../datasources/v2/SetHistoryPolicyExec.scala | 12 ++++++------ 8 files changed, 35 insertions(+), 37 deletions(-) diff --git a/integrations/spark/spark-3.1/openhouse-spark-itest/src/test/java/com/linkedin/openhouse/spark/statementtest/SetHistoryPolicyStatementTest.java b/integrations/spark/spark-3.1/openhouse-spark-itest/src/test/java/com/linkedin/openhouse/spark/statementtest/SetHistoryPolicyStatementTest.java index 1bf412ef..9b8d7526 100644 --- a/integrations/spark/spark-3.1/openhouse-spark-itest/src/test/java/com/linkedin/openhouse/spark/statementtest/SetHistoryPolicyStatementTest.java +++ b/integrations/spark/spark-3.1/openhouse-spark-itest/src/test/java/com/linkedin/openhouse/spark/statementtest/SetHistoryPolicyStatementTest.java @@ -44,12 +44,11 @@ public void testSetHistoryPolicyGood() { Dataset ds = spark.sql("ALTER TABLE openhouse.db.table SET POLICY (HISTORY MAX_AGE=24H)"); assert isPlanValid(ds, "db.table", Optional.of("24"), Optional.of("HOUR"), Optional.empty()); - ds = spark.sql("ALTER TABLE openhouse.db.table SET POLICY (HISTORY MIN_VERSIONS=10)"); + ds = spark.sql("ALTER TABLE openhouse.db.table SET POLICY (HISTORY VERSIONS=10)"); assert isPlanValid(ds, "db.table", Optional.empty(), Optional.empty(), Optional.of("10")); // Validate both time and count setting - ds = - spark.sql("ALTER TABLE openhouse.db.table SET POLICY (HISTORY MAX_AGE=2D MIN_VERSIONS=20)"); + ds = spark.sql("ALTER TABLE openhouse.db.table SET POLICY (HISTORY MAX_AGE=2D VERSIONS=20)"); assert isPlanValid(ds, "db.table", Optional.of("2"), Optional.of("DAY"), Optional.of("20")); } @@ -65,8 +64,7 @@ public void testSetHistoryPolicyIncorrectSyntax() { OpenhouseParseException.class, () -> spark - .sql( - "ALTER TABLE openhouse.db.table SET POLICY (HISTORY MIN_VERSIONS=10 MAX_AGE=24H)") + .sql("ALTER TABLE openhouse.db.table SET POLICY (HISTORY VERSIONS=10 MAX_AGE=24H)") .show()); // No time or count @@ -104,11 +102,11 @@ private boolean isPlanValid( String dbTable, Optional maxAge, Optional granularity, - Optional minVersions) { + Optional versions) { String queryStr = dataframe.queryExecution().explainString(ExplainMode.fromString("simple")); return queryStr.contains(dbTable) && (!maxAge.isPresent() || queryStr.contains(maxAge.get())) && (!granularity.isPresent() || queryStr.contains(granularity.get())) - && (!minVersions.isPresent() || queryStr.contains(minVersions.get())); + && (!versions.isPresent() || queryStr.contains(versions.get())); } } diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/antlr/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensions.g4 b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/antlr/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensions.g4 index ca576211..27cb980f 100644 --- a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/antlr/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensions.g4 +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/antlr/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensions.g4 @@ -156,15 +156,15 @@ policyTag ; historyPolicy - : HISTORY maxAge? minVersions? + : HISTORY maxAge? versions? ; maxAge : MAX_AGE'='duration ; -minVersions - : MIN_VERSIONS'='POSITIVE_INTEGER +versions + : VERSIONS'='POSITIVE_INTEGER ; ALTER: 'ALTER'; @@ -197,7 +197,7 @@ HC: 'HC'; MODIFY: 'MODIFY'; TAG: 'TAG'; NONE: 'NONE'; -MIN_VERSIONS: 'MIN_VERSIONS'; +VERSIONS: 'VERSIONS'; MAX_AGE: 'MAX_AGE'; POSITIVE_INTEGER diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensionsAstBuilder.scala b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensionsAstBuilder.scala index 141d2d04..9585af5f 100644 --- a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensionsAstBuilder.scala +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensionsAstBuilder.scala @@ -165,23 +165,23 @@ class OpenhouseSqlExtensionsAstBuilder (delegate: ParserInterface) extends Openh override def visitSetHistoryPolicy(ctx: SetHistoryPolicyContext): SetHistoryPolicy = { val tableName = typedVisit[Seq[String]](ctx.multipartIdentifier) - val (granularity, maxAge, minVersions) = typedVisit[(Option[String], Int, Int)](ctx.historyPolicy()) - SetHistoryPolicy(tableName, granularity, maxAge, minVersions) + val (granularity, maxAge, versions) = typedVisit[(Option[String], Int, Int)](ctx.historyPolicy()) + SetHistoryPolicy(tableName, granularity, maxAge, versions) } override def visitHistoryPolicy(ctx: HistoryPolicyContext): (Option[String], Int, Int) = { val maxAgePolicy = if (ctx.maxAge() != null) typedVisit[(String, Int)](ctx.maxAge().duration()) else (null, -1) - val minVersionPolicy = if (ctx.minVersions() != null) - typedVisit[Int](ctx.minVersions()) + val versionPolicy = if (ctx.versions() != null) + typedVisit[Int](ctx.versions()) else -1 - if (maxAgePolicy._2 == -1 && minVersionPolicy == -1) { - throw new OpenhouseParseException("Either TIME or VERSIONS must be specified in HISTORY policy", ctx.start.getLine, ctx.start.getCharPositionInLine) + if (maxAgePolicy._2 == -1 && versionPolicy == -1) { + throw new OpenhouseParseException("Either MAX_AGE or VERSIONS must be specified in HISTORY policy", ctx.start.getLine, ctx.start.getCharPositionInLine) } - (Option(maxAgePolicy._1), maxAgePolicy._2, minVersionPolicy) + (Option(maxAgePolicy._1), maxAgePolicy._2, versionPolicy) } - override def visitMinVersions(ctx: MinVersionsContext): Integer = { + override def visitVersions(ctx: VersionsContext): Integer = { ctx.POSITIVE_INTEGER().getText.toInt } diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetHistoryPolicy.scala b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetHistoryPolicy.scala index c064d12a..99b42c9b 100644 --- a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetHistoryPolicy.scala +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetHistoryPolicy.scala @@ -2,8 +2,8 @@ package com.linkedin.openhouse.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.Command -case class SetHistoryPolicy (tableName: Seq[String], granularity: Option[String], maxAge: Int, minVersions: Int) extends Command { +case class SetHistoryPolicy (tableName: Seq[String], granularity: Option[String], maxAge: Int, versions: Int) extends Command { override def simpleString(maxFields: Int): String = { - s"SetHistoryPolicy: ${tableName} ${if (maxAge > 0) "MAX_AGE=" + maxAge else ""}${granularity.getOrElse("")} ${if (minVersions > 0) "MIN_VERSIONS=" + minVersions else ""}" + s"SetHistoryPolicy: ${tableName} ${if (maxAge > 0) "MAX_AGE=" + maxAge else ""}${granularity.getOrElse("")} ${if (versions > 0) "VERSIONS=" + versions else ""}" } } diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/OpenhouseDataSourceV2Strategy.scala b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/OpenhouseDataSourceV2Strategy.scala index d1478c42..e93f8a5b 100644 --- a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/OpenhouseDataSourceV2Strategy.scala +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/OpenhouseDataSourceV2Strategy.scala @@ -17,8 +17,8 @@ case class OpenhouseDataSourceV2Strategy(spark: SparkSession) extends Strategy w SetRetentionPolicyExec(catalog, ident, granularity, count, colName, colPattern) :: Nil case SetReplicationPolicy(CatalogAndIdentifierExtractor(catalog, ident), replicationPolicies) => SetReplicationPolicyExec(catalog, ident, replicationPolicies) :: Nil - case SetHistoryPolicy(CatalogAndIdentifierExtractor(catalog, ident), granularity, maxAge, minVersions) => - SetHistoryPolicyExec(catalog, ident, granularity, maxAge, minVersions) :: Nil + case SetHistoryPolicy(CatalogAndIdentifierExtractor(catalog, ident), granularity, maxAge, versions) => + SetHistoryPolicyExec(catalog, ident, granularity, maxAge, versions) :: Nil case SetSharingPolicy(CatalogAndIdentifierExtractor(catalog, ident), sharing) => SetSharingPolicyExec(catalog, ident, sharing) :: Nil case SetColumnPolicyTag(CatalogAndIdentifierExtractor(catalog, ident), policyTag, cols) => diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetHistoryPolicyExec.scala b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetHistoryPolicyExec.scala index 26da42a0..271f9e19 100644 --- a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetHistoryPolicyExec.scala +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetHistoryPolicyExec.scala @@ -11,7 +11,7 @@ case class SetHistoryPolicyExec( ident: Identifier, granularity: Option[String], maxAge: Int, - minVersions: Int + versions: Int ) extends V2CommandExec { override lazy val output: Seq[Attribute] = Nil @@ -21,10 +21,10 @@ case class SetHistoryPolicyExec( case iceberg: SparkTable if iceberg.table().properties().containsKey("openhouse.tableId") => val key = "updated.openhouse.policy" val value = { - (maxAge, minVersions) match { - case maxAgeOnly if minVersions == -1 => s"""{"history":{"maxAge":${maxAge},"granularity":"${granularity.get}"}}""" - case minVersionsOnly if maxAge == -1 => s"""{"history":{"minVersions":${minVersions}}}""" - case _ => s"""{"history":{"maxAge":${maxAge},"granularity":"${granularity.get}","minVersions":${minVersions}}}""" + (maxAge, versions) match { + case maxAgeOnly if versions == -1 => s"""{"history":{"maxAge":${maxAge},"granularity":"${granularity.get}"}}""" + case versionsOnly if maxAge == -1 => s"""{"history":{"versions":${versions}}}""" + case _ => s"""{"history":{"maxAge":${maxAge},"granularity":"${granularity.get}","versions":${versions}}}""" } } @@ -40,6 +40,6 @@ case class SetHistoryPolicyExec( } override def simpleString(maxFields: Int): String = { - s"SetHistoryPolicyExec: ${catalog} ${ident} MAX_AGE=${if (maxAge > 0) maxAge else ""}${granularity.getOrElse("")} MIN_VERSIONS=${if (minVersions > 0) minVersions else ""}" + s"SetHistoryPolicyExec: ${catalog} ${ident} MAX_AGE=${if (maxAge > 0) maxAge else ""}${granularity.getOrElse("")} VERSIONS=${if (versions > 0) versions else ""}" } } diff --git a/integrations/spark/spark-3.5/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetHistoryPolicy.scala b/integrations/spark/spark-3.5/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetHistoryPolicy.scala index 1c4f2603..dc9899b9 100644 --- a/integrations/spark/spark-3.5/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetHistoryPolicy.scala +++ b/integrations/spark/spark-3.5/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/plans/logical/SetHistoryPolicy.scala @@ -2,8 +2,8 @@ package com.linkedin.openhouse.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.LeafCommand -case class SetHistoryPolicy(tableName: Seq[String], granularity: Option[String], maxAge: Int, minVersions: Int) extends LeafCommand { +case class SetHistoryPolicy(tableName: Seq[String], granularity: Option[String], maxAge: Int, versions: Int) extends LeafCommand { override def simpleString(maxFields: Int): String = { - s"SetHistoryPolicy: ${tableName} ${if (maxAge > 0) "MAX_AGE=" + maxAge else ""}${granularity.getOrElse("")} ${if (minVersions > 0) "MIN_VERSIONS=" + minVersions else ""}" + s"SetHistoryPolicy: ${tableName} ${if (maxAge > 0) "MAX_AGE=" + maxAge else ""}${granularity.getOrElse("")} ${if (versions > 0) "VERSIONS=" + versions else ""}" } } diff --git a/integrations/spark/spark-3.5/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetHistoryPolicyExec.scala b/integrations/spark/spark-3.5/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetHistoryPolicyExec.scala index 9f6b7d7c..c429d199 100644 --- a/integrations/spark/spark-3.5/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetHistoryPolicyExec.scala +++ b/integrations/spark/spark-3.5/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/execution/datasources/v2/SetHistoryPolicyExec.scala @@ -11,7 +11,7 @@ case class SetHistoryPolicyExec( ident: Identifier, granularity: Option[String], maxAge: Int, - minVersions: Int + versions: Int ) extends LeafV2CommandExec { override lazy val output: Seq[Attribute] = Nil @@ -21,10 +21,10 @@ case class SetHistoryPolicyExec( case iceberg: SparkTable if iceberg.table().properties().containsKey("openhouse.tableId") => val key = "updated.openhouse.policy" val value = { - (maxAge, minVersions) match { - case maxAgeOnly if minVersions == -1 => s"""{"history":{"maxAge":${maxAge},"granularity":"${granularity.get}"}}""" - case minVersionsOnly if maxAge == -1 => s"""{"history":{"minVersions":${minVersions}}}""" - case _ => s"""{"history":{"maxAge":${maxAge},"granularity":"${granularity.get}","minVersions":${minVersions}}}""" + (maxAge, versions) match { + case maxAgeOnly if versions == -1 => s"""{"history":{"maxAge":${maxAge},"granularity":"${granularity.get}"}}""" + case versionsOnly if maxAge == -1 => s"""{"history":{"versions":${versions}}}""" + case _ => s"""{"history":{"maxAge":${maxAge},"granularity":"${granularity.get}","versions":${versions}}}""" } } @@ -40,6 +40,6 @@ case class SetHistoryPolicyExec( } override def simpleString(maxFields: Int): String = { - s"SetHistoryPolicyExec: ${catalog} ${ident} MAX_AGE=${if (maxAge > 0) maxAge else ""}${granularity.getOrElse("")} MIN_VERSIONS=${if (minVersions > 0) minVersions else ""}" + s"SetHistoryPolicyExec: ${catalog} ${ident} MAX_AGE=${if (maxAge > 0) maxAge else ""}${granularity.getOrElse("")} VERSIONS=${if (versions > 0) versions else ""}" } } From 6039e0bc4188a506b7996abab7e210252da31c9f Mon Sep 17 00:00:00 2001 From: Will-Lo Date: Tue, 17 Dec 2024 14:53:12 -0500 Subject: [PATCH 10/10] Address comments --- .../extensions/OpenhouseSqlExtensionsAstBuilder.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensionsAstBuilder.scala b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensionsAstBuilder.scala index 9585af5f..8120d721 100644 --- a/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensionsAstBuilder.scala +++ b/integrations/spark/spark-3.1/openhouse-spark-runtime/src/main/scala/com/linkedin/openhouse/spark/sql/catalyst/parser/extensions/OpenhouseSqlExtensionsAstBuilder.scala @@ -19,8 +19,7 @@ class OpenhouseSqlExtensionsAstBuilder (delegate: ParserInterface) extends Openh override def visitSetRetentionPolicy(ctx: SetRetentionPolicyContext): SetRetentionPolicy = { val tableName = typedVisit[Seq[String]](ctx.multipartIdentifier) - val retentionPolicy = ctx.retentionPolicy() - val (granularity, count) = typedVisit[(String, Int)](retentionPolicy) + val (granularity, count) = typedVisit[(String, Int)](ctx.retentionPolicy()) val (colName, colPattern) = if (ctx.columnRetentionPolicy() != null) typedVisit[(String, String)](ctx.columnRetentionPolicy()) @@ -176,7 +175,9 @@ class OpenhouseSqlExtensionsAstBuilder (delegate: ParserInterface) extends Openh typedVisit[Int](ctx.versions()) else -1 if (maxAgePolicy._2 == -1 && versionPolicy == -1) { - throw new OpenhouseParseException("Either MAX_AGE or VERSIONS must be specified in HISTORY policy", ctx.start.getLine, ctx.start.getCharPositionInLine) + throw new OpenhouseParseException("At least one of MAX_AGE or VERSIONS must be specified in HISTORY policy, e.g. " + + "ALTER TABLE openhouse.db.table SET POLICY (HISTORY MAX_AGE=2D) or ALTER TABLE openhouse.db.table SET POLICY (HISTORY VERSIONS=3)", + ctx.start.getLine, ctx.start.getCharPositionInLine) } (Option(maxAgePolicy._1), maxAgePolicy._2, versionPolicy) }