aokolnychyi commented on a change in pull request #1473: URL: https://github.com/apache/iceberg/pull/1473#discussion_r492346875
########## File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala ########## @@ -0,0 +1,364 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.catalyst.parser.extensions + +import org.antlr.v4.runtime._ +import org.antlr.v4.runtime.atn.PredictionMode +import org.antlr.v4.runtime.misc.{Interval, ParseCancellationException} +import org.antlr.v4.runtime.tree.{ParseTree, TerminalNodeImpl} +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} +import org.apache.spark.sql.catalyst.expressions.{Expression, Literal} +import org.apache.spark.sql.catalyst.parser.{ParseErrorListener, ParseException, ParserInterface} +import org.apache.spark.sql.catalyst.parser.ParserUtils._ +import org.apache.spark.sql.catalyst.parser.extensions.IcebergSqlExtensionsParser._ +import org.apache.spark.sql.catalyst.plans.logical.{Call, CallArgument, LogicalPlan} +import org.apache.spark.sql.catalyst.trees.Origin +import org.apache.spark.sql.types.{ByteType, DataType, DoubleType, FloatType, LongType, ShortType, StructType} +import scala.collection.JavaConverters._ + +class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserInterface { Review comment: They are written in Scala in Spark so it is easier to copy the relevant code part (e.g. literal parsing). ########## File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala ########## @@ -0,0 +1,364 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.catalyst.parser.extensions + +import org.antlr.v4.runtime._ +import org.antlr.v4.runtime.atn.PredictionMode +import org.antlr.v4.runtime.misc.{Interval, ParseCancellationException} +import org.antlr.v4.runtime.tree.{ParseTree, TerminalNodeImpl} +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} +import org.apache.spark.sql.catalyst.expressions.{Expression, Literal} +import org.apache.spark.sql.catalyst.parser.{ParseErrorListener, ParseException, ParserInterface} +import org.apache.spark.sql.catalyst.parser.ParserUtils._ +import org.apache.spark.sql.catalyst.parser.extensions.IcebergSqlExtensionsParser._ +import org.apache.spark.sql.catalyst.plans.logical.{Call, CallArgument, LogicalPlan} +import org.apache.spark.sql.catalyst.trees.Origin +import org.apache.spark.sql.types.{ByteType, DataType, DoubleType, FloatType, LongType, ShortType, StructType} +import scala.collection.JavaConverters._ + +class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserInterface { + + private val astBuilder = new IcebergSqlExtensionsAstBuilder() + + /** + * Parse a string to a DataType. + */ + override def parseDataType(sqlText: String): DataType = { + delegate.parseDataType(sqlText) + } + + /** + * Parse a string to a raw DataType without CHAR/VARCHAR replacement. + */ + override def parseRawDataType(sqlText: String): DataType = { + delegate.parseRawDataType(sqlText) + } + + /** + * Parse a string to an Expression. + */ + override def parseExpression(sqlText: String): Expression = { + delegate.parseExpression(sqlText) + } + + /** + * Parse a string to a TableIdentifier. + */ + override def parseTableIdentifier(sqlText: String): TableIdentifier = { + delegate.parseTableIdentifier(sqlText) + } + + /** + * Parse a string to a FunctionIdentifier. + */ + override def parseFunctionIdentifier(sqlText: String): FunctionIdentifier = { + delegate.parseFunctionIdentifier(sqlText) + } + + /** + * Parse a string to a multi-part identifier. + */ + override def parseMultipartIdentifier(sqlText: String): Seq[String] = { + delegate.parseMultipartIdentifier(sqlText) + } + + /** + * Creates StructType for a given SQL string, which is a comma separated list of field + * definitions which will preserve the correct Hive metadata. + */ + override def parseTableSchema(sqlText: String): StructType = { + delegate.parseTableSchema(sqlText) + } + + /** + * Parse a string to a LogicalPlan. + */ + override def parsePlan(sqlText: String): LogicalPlan = parse(sqlText) { parser => + astBuilder.visit(parser.singleStatement()) match { + case plan: LogicalPlan => plan + case _ => delegate.parsePlan(sqlText) + } + } + + protected def parse[T](command: String)(toResult: IcebergSqlExtensionsParser => T): T = { + val lexer = new IcebergSqlExtensionsLexer(new UpperCaseCharStream(CharStreams.fromString(command))) + lexer.removeErrorListeners() + lexer.addErrorListener(ParseErrorListener) + + val tokenStream = new CommonTokenStream(lexer) + val parser = new IcebergSqlExtensionsParser(tokenStream) + parser.addParseListener(PostProcessor) + parser.removeErrorListeners() + parser.addErrorListener(ParseErrorListener) + + try { + try { + // first, try parsing with potentially faster SLL mode + parser.getInterpreter.setPredictionMode(PredictionMode.SLL) + toResult(parser) + } + catch { + case _: ParseCancellationException => + // if we fail, parse with LL mode + tokenStream.seek(0) // rewind input stream + parser.reset() + + // Try Again. + parser.getInterpreter.setPredictionMode(PredictionMode.LL) + toResult(parser) + } + } + catch { + case e: ParseException if e.command.isDefined => + throw e + case e: ParseException => + throw e.withCommand(command) + case e: AnalysisException => + val position = Origin(e.line, e.startPosition) + throw new ParseException(Option(command), e.message, position, position) + } + } +} + +// literal parsing is taken from Spark's AstBuilder +class IcebergSqlExtensionsAstBuilder extends IcebergSqlExtensionsBaseVisitor[AnyRef] { + + override def visitCall(ctx: CallContext): LogicalPlan = { + val name = ctx.multipartIdentifier.parts.asScala.map(_.getText) + val args = ctx.callArgument.asScala.map(typedVisit[CallArgument]) + Call(name, args) + } + + override def visitPositionalArgument(ctx: PositionalArgumentContext): CallArgument = { + val expr = typedVisit[Expression](ctx.expression) + CallArgument(None, expr) + } + + override def visitNamedArgument(ctx: NamedArgumentContext): CallArgument = { + val name = ctx.identifier.getText + val expr = typedVisit[Expression](ctx.expression) + CallArgument(Some(name), expr) + } + + override def visitNonIcebergCommand(ctx: NonIcebergCommandContext): LogicalPlan = null + + override def visitSingleStatement(ctx: SingleStatementContext): LogicalPlan = withOrigin(ctx) { + visit(ctx.statement).asInstanceOf[LogicalPlan] + } + + /** + * Create an integral literal expression. The code selects the most narrow integral type + * possible, either a BigDecimal, a Long or an Integer is returned. + */ + override def visitIntegerLiteral(ctx: IntegerLiteralContext): Literal = withOrigin(ctx) { Review comment: `IntegerLiteralContext` is generated class using antlr4 from the extensions grammar we have. That's why we cannot simply call Spark's parser on it. That being said, we will only intercept CALL statements and the rest will be delegated to the main parser in Spark. ########## File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveIcebergProcedures.scala ########## @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.catalyst.analysis + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.CatalystTypeConverters +import org.apache.spark.sql.catalyst.expressions.{Expression, Literal} +import org.apache.spark.sql.catalyst.plans.logical.{Call, CallArgument, LogicalPlan} +import org.apache.spark.sql.catalyst.procedures.{IcebergProcedureRegistry, OptionalParameter, Parameter, Procedure} +import org.apache.spark.sql.catalyst.rules.Rule +import scala.collection.mutable + +// TODO: case sensitivity? +object ResolveIcebergProcedures extends Rule[LogicalPlan] { + + override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { + case Call(nameParts, args) => + val procedure = IcebergProcedureRegistry.resolve(nameParts) + val values = buildProcedureValues(procedure, args) + procedure.createRunnableCommand(values) + } + + private def buildProcedureValues(procedure: Procedure, args: Seq[CallArgument]): Array[Any] = { + // build a map of declared parameter names to their positions + val params = procedure.parameters + val nameToPositionMap = params.map(_.name).zipWithIndex.toMap + + // verify named and positional args are not mixed + val containsNamedArg = args.exists(arg => arg.name.isDefined) + val containsPositionalArg = args.exists(arg => arg.name.isEmpty) + if (containsNamedArg && containsPositionalArg) { + throw new AnalysisException("Named and positional arguments cannot be mixed") + } + + // build a map of parameter names to args + val nameToArgMap = if (containsNamedArg) { + buildNameToArgMapUsingNames(args, nameToPositionMap) + } else { + buildNameToArgMapUsingPositions(args, params) + } + + // verify all required parameters are provided + params.filter(_.required) + .find(param => !nameToArgMap.contains(param.name)) + .foreach { missingArg => + throw new AnalysisException(s"Required procedure argument '${missingArg.name}' is missing") + } + + val values = new Array[Any](params.size) + + // convert provided args from internal Spark representation to Scala + nameToArgMap.foreach { case (name, arg) => + val position = nameToPositionMap(name) + val param = params(position) + val paramType = param.dataType + val argType = arg.expr.dataType + if (paramType != argType) { + throw new AnalysisException(s"Wrong arg type for '${param.name}': expected $paramType but got $argType") + } + values(position) = toScalaValue(arg.expr) + } + + // assign default values for optional params + params.foreach { + case p: OptionalParameter if !nameToArgMap.contains(p.name) => + val position = nameToPositionMap(p.name) + values(position) = p.defaultValue + case _ => + } + + values + } + + private def buildNameToArgMapUsingNames( + args: Seq[CallArgument], + nameToPositionMap: Map[String, Int]): Map[String, CallArgument] = { + + val nameToArgMap = mutable.LinkedHashMap.empty[String, CallArgument] + args.foreach { arg => Review comment: Will reconsider this next. ########## File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveIcebergProcedures.scala ########## @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.catalyst.analysis + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.CatalystTypeConverters +import org.apache.spark.sql.catalyst.expressions.{Expression, Literal} +import org.apache.spark.sql.catalyst.plans.logical.{Call, CallArgument, LogicalPlan} +import org.apache.spark.sql.catalyst.procedures.{IcebergProcedureRegistry, OptionalParameter, Parameter, Procedure} +import org.apache.spark.sql.catalyst.rules.Rule +import scala.collection.mutable + +// TODO: case sensitivity? +object ResolveIcebergProcedures extends Rule[LogicalPlan] { + + override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { + case Call(nameParts, args) => + val procedure = IcebergProcedureRegistry.resolve(nameParts) + val values = buildProcedureValues(procedure, args) + procedure.createRunnableCommand(values) + } + + private def buildProcedureValues(procedure: Procedure, args: Seq[CallArgument]): Array[Any] = { + // build a map of declared parameter names to their positions + val params = procedure.parameters + val nameToPositionMap = params.map(_.name).zipWithIndex.toMap + + // verify named and positional args are not mixed + val containsNamedArg = args.exists(arg => arg.name.isDefined) + val containsPositionalArg = args.exists(arg => arg.name.isEmpty) + if (containsNamedArg && containsPositionalArg) { + throw new AnalysisException("Named and positional arguments cannot be mixed") + } + + // build a map of parameter names to args + val nameToArgMap = if (containsNamedArg) { + buildNameToArgMapUsingNames(args, nameToPositionMap) + } else { + buildNameToArgMapUsingPositions(args, params) + } + + // verify all required parameters are provided + params.filter(_.required) + .find(param => !nameToArgMap.contains(param.name)) + .foreach { missingArg => + throw new AnalysisException(s"Required procedure argument '${missingArg.name}' is missing") + } + + val values = new Array[Any](params.size) + + // convert provided args from internal Spark representation to Scala + nameToArgMap.foreach { case (name, arg) => + val position = nameToPositionMap(name) + val param = params(position) + val paramType = param.dataType + val argType = arg.expr.dataType + if (paramType != argType) { + throw new AnalysisException(s"Wrong arg type for '${param.name}': expected $paramType but got $argType") + } + values(position) = toScalaValue(arg.expr) + } + + // assign default values for optional params + params.foreach { + case p: OptionalParameter if !nameToArgMap.contains(p.name) => + val position = nameToPositionMap(p.name) + values(position) = p.defaultValue + case _ => + } + + values + } + + private def buildNameToArgMapUsingNames( + args: Seq[CallArgument], + nameToPositionMap: Map[String, Int]): Map[String, CallArgument] = { + + val nameToArgMap = mutable.LinkedHashMap.empty[String, CallArgument] + args.foreach { arg => + val name = arg.name.get + + if (nameToArgMap.contains(name)) { + throw new AnalysisException(s"Duplicate procedure argument: '$name'") + } + + if (!nameToPositionMap.contains(name)) { + throw new AnalysisException(s"Unknown argument name: '$name'") + } + + nameToArgMap.put(name, arg) + } + nameToArgMap.toMap + } + + private def buildNameToArgMapUsingPositions( + args: Seq[CallArgument], + params: Seq[Parameter]): Map[String, CallArgument] = { + + val nameToArgMap = mutable.LinkedHashMap.empty[String, CallArgument] + args.zipWithIndex.foreach { case (arg, position) => Review comment: Will update. ########## File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveIcebergProcedures.scala ########## @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.catalyst.analysis + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.CatalystTypeConverters +import org.apache.spark.sql.catalyst.expressions.{Expression, Literal} +import org.apache.spark.sql.catalyst.plans.logical.{Call, CallArgument, LogicalPlan} +import org.apache.spark.sql.catalyst.procedures.{IcebergProcedureRegistry, OptionalParameter, Parameter, Procedure} +import org.apache.spark.sql.catalyst.rules.Rule +import scala.collection.mutable + +// TODO: case sensitivity? +object ResolveIcebergProcedures extends Rule[LogicalPlan] { + + override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { + case Call(nameParts, args) => + val procedure = IcebergProcedureRegistry.resolve(nameParts) + val values = buildProcedureValues(procedure, args) + procedure.createRunnableCommand(values) + } + + private def buildProcedureValues(procedure: Procedure, args: Seq[CallArgument]): Array[Any] = { + // build a map of declared parameter names to their positions + val params = procedure.parameters + val nameToPositionMap = params.map(_.name).zipWithIndex.toMap + + // verify named and positional args are not mixed + val containsNamedArg = args.exists(arg => arg.name.isDefined) + val containsPositionalArg = args.exists(arg => arg.name.isEmpty) + if (containsNamedArg && containsPositionalArg) { + throw new AnalysisException("Named and positional arguments cannot be mixed") + } + + // build a map of parameter names to args + val nameToArgMap = if (containsNamedArg) { + buildNameToArgMapUsingNames(args, nameToPositionMap) + } else { + buildNameToArgMapUsingPositions(args, params) + } + + // verify all required parameters are provided + params.filter(_.required) + .find(param => !nameToArgMap.contains(param.name)) + .foreach { missingArg => + throw new AnalysisException(s"Required procedure argument '${missingArg.name}' is missing") + } + + val values = new Array[Any](params.size) + + // convert provided args from internal Spark representation to Scala + nameToArgMap.foreach { case (name, arg) => + val position = nameToPositionMap(name) + val param = params(position) + val paramType = param.dataType + val argType = arg.expr.dataType + if (paramType != argType) { + throw new AnalysisException(s"Wrong arg type for '${param.name}': expected $paramType but got $argType") + } + values(position) = toScalaValue(arg.expr) + } + + // assign default values for optional params + params.foreach { + case p: OptionalParameter if !nameToArgMap.contains(p.name) => + val position = nameToPositionMap(p.name) + values(position) = p.defaultValue + case _ => + } + + values + } + + private def buildNameToArgMapUsingNames( + args: Seq[CallArgument], + nameToPositionMap: Map[String, Int]): Map[String, CallArgument] = { + + val nameToArgMap = mutable.LinkedHashMap.empty[String, CallArgument] + args.foreach { arg => + val name = arg.name.get + + if (nameToArgMap.contains(name)) { + throw new AnalysisException(s"Duplicate procedure argument: '$name'") + } + + if (!nameToPositionMap.contains(name)) { + throw new AnalysisException(s"Unknown argument name: '$name'") + } + + nameToArgMap.put(name, arg) + } + nameToArgMap.toMap + } + + private def buildNameToArgMapUsingPositions( + args: Seq[CallArgument], + params: Seq[Parameter]): Map[String, CallArgument] = { + + val nameToArgMap = mutable.LinkedHashMap.empty[String, CallArgument] + args.zipWithIndex.foreach { case (arg, position) => + if (position < params.size) { + val param = params(position) + nameToArgMap.put(param.name, arg) + } else { + throw new AnalysisException("Too many arguments for procedure") + } + } + nameToArgMap.toMap + } + + private def toScalaValue(expr: Expression): Any = expr match { + case literal: Literal => CatalystTypeConverters.convertToScala(literal.value, literal.dataType) + case _ => throw new AnalysisException(s"Cannot convert '$expr' to a Scala value") Review comment: Done. ########## File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/call.scala ########## @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.catalyst.plans.logical + +import org.apache.spark.sql.catalyst.expressions.Expression + +case class Call(name: Seq[String], args: Seq[CallArgument]) extends ParsedStatement + +case class CallArgument(name: Option[String], expr: Expression) Review comment: Done. ########## File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/procedures/BaseIcebergProcedureCommand.scala ########## @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.catalyst.procedures + +import org.apache.iceberg.Table +import org.apache.iceberg.spark.source.SparkTable +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} +import org.apache.spark.sql.execution.command.RunnableCommand + +trait BaseIcebergProcedureCommand extends RunnableCommand { + + protected def loadTable(spark: SparkSession, catalog: TableCatalog, tableName: String): Table = { Review comment: Updated. ########## File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/procedures/IcebergProcedureRegistry.scala ########## @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.catalyst.procedures + +import org.apache.iceberg.spark.{SparkCatalog, SparkSessionCatalog} +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogNotFoundException, CatalogPlugin, Identifier, TableCatalog} + +object IcebergProcedureRegistry { + + def resolve(nameParts: Seq[String]): Procedure = { + val catalogManager = SparkSession.active.sessionState.catalogManager + val (catalog, ident) = resolveCatalog(catalogManager, nameParts) + + validateCatalog(catalog) + validateProcedureIdentifier(ident) + + ident.name match { + case "rollback_to_snapshot" => RollbackToSnapshotProcedure(catalog.asInstanceOf[TableCatalog]) Review comment: We will want to support a rollback [to time as well as to snapshot](https://docs.google.com/document/d/1Nf8c16R2hj4lSc-4sQg4oiUUV_F4XqZKth1woEo6TN8/). If we want to have a single procedure, we can make `snapshot_id` and `time` as optional arguments and require at least one to be given. However, Presto already defines `rollback_to_snapshot` so I followed that. ########## File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/procedures/BaseIcebergProcedureCommand.scala ########## @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.catalyst.procedures + +import org.apache.iceberg.Table +import org.apache.iceberg.spark.source.SparkTable +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} +import org.apache.spark.sql.execution.command.RunnableCommand + +trait BaseIcebergProcedureCommand extends RunnableCommand { + + protected def loadTable(spark: SparkSession, catalog: TableCatalog, tableName: String): Table = { + val tableNameParts = spark.sessionState.sqlParser.parseMultipartIdentifier(tableName) Review comment: I think it is up to us to interpret that. ########## File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveIcebergProcedures.scala ########## @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.catalyst.analysis + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.CatalystTypeConverters +import org.apache.spark.sql.catalyst.expressions.{Expression, Literal} +import org.apache.spark.sql.catalyst.plans.logical.{Call, CallArgument, LogicalPlan} +import org.apache.spark.sql.catalyst.procedures.{IcebergProcedureRegistry, OptionalParameter, Parameter, Procedure} +import org.apache.spark.sql.catalyst.rules.Rule +import scala.collection.mutable + +// TODO: case sensitivity? +object ResolveIcebergProcedures extends Rule[LogicalPlan] { Review comment: I've updated to make it generic. It is still in a rough state but is reviewable. ########## File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/procedures/procedure.scala ########## @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.catalyst.procedures + +import org.apache.spark.sql.execution.command.RunnableCommand +import org.apache.spark.sql.types.DataType + +trait Procedure { + def name: String + def parameters: Seq[Parameter] + def createRunnableCommand(args: Array[Any]): RunnableCommand Review comment: `RunnableCommand` is a logical plan that gets wrapped into `ExecutedCommandExec` by a strategy. I used it because it does not need an extra logical plan node like `CallProcedure` and an extra strategy to convert into `V2CommandExec`. However, we can reconsider that. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org