rdblue commented on a change in pull request #1473:
URL: https://github.com/apache/iceberg/pull/1473#discussion_r491090625



##########
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:
       When extending a grammar, we can't use rules from the other grammar? It 
looks like we try to parse with Iceberg's parser and fall back to Spark if it 
is unsupported?




----------------------------------------------------------------
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:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to