Kontinuation commented on PR #1701:
URL: https://github.com/apache/sedona/pull/1701#issuecomment-2500473956

   > so, SessionState Field("sqlParser") is always SedonaSqlParser, it doesn't 
matter the inject order
   
   From what I observed, `sparkSession.sessionState.sqlParser` will be 
`IcebergSparkSqlExtensionsParser` right after iceberg extension is applied. If 
Sedona is initialized afterwards, it will replace 
`sparkSession.sessionState.sqlParser` with `SedonaSqlParser`. The delegation 
hierarchy will be something like this:
   
   ```
   SedonaSqlParser
     └─ IcebergSparkSqlExtensionsParser
          └─ <delegated parser instance>
   ```
   
   If `SedonaSqlParser` cannot fallback to the delegated parser, the special 
SQL syntaxes such as `CALL ...` defined by iceberg won't work. Here is an 
example using iceberg 1.7.0 and a locally built Sedona with this patch applied:
   
   ```
   pyspark --packages org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:1.7.0 \
           --jars "$SEDONA_SHADED_JAR" \
           --conf 
spark.sql.extensions=org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions
 \
           --conf spark.sql.catalog.local=org.apache.iceberg.spark.SparkCatalog 
\
           --conf spark.sql.catalog.local.type=hadoop \
           --conf 
spark.sql.catalog.local.warehouse=$HOME/local/iceberg/warehouse
   
   Welcome to
         ____              __
        / __/__  ___ _____/ /__
       _\ \/ _ \/ _ `/ __/  '_/
      /__ / .__/\_,_/_/ /_/\_\   version 3.5.1
         /_/
   
   Using Python version 3.11.6 (main, Oct  2 2023 20:46:14)
   Spark context Web UI available at http://192.168.200.155:4040
   Spark context available as 'sc' (master = local[*], app id = 
local-1732620541890).
   SparkSession available as 'spark'.
   >>> from sedona.spark import *
   >>> sedona = SedonaContext.create(spark)
   24/11/26 19:33:39 WARN RasterRegistrator$: Geotools was not found on the 
classpath. Raster operations will not be available.
   >>> spark.sql("CALL local.system.remove_orphan_files (table => 
'test_db.test_table', dry_run => true)").show()
   Traceback (most recent call last):
     File "<stdin>", line 1, in <module>
     File 
"/workspace/local/spark/spark-3.5.1-bin-hadoop3/python/pyspark/sql/session.py", 
line 1631, in sql
       return DataFrame(self._jsparkSession.sql(sqlQuery, litArgs), self)
                        ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
     File 
"/workspace/local/spark/spark-3.5.1-bin-hadoop3/python/lib/py4j-0.10.9.7-src.zip/py4j/java_gateway.py",
 line 1322, in __call__
     File 
"/workspace/local/spark/spark-3.5.1-bin-hadoop3/python/pyspark/errors/exceptions/captured.py",
 line 185, in deco
       raise converted from None
   pyspark.errors.exceptions.captured.ParseException: 
   [PARSE_SYNTAX_ERROR] Syntax error at or near 'CALL'.(line 1, pos 0)
   
   == SQL ==
   CALL local.system.remove_orphan_files (table => 'test_db.test_table', 
dry_run => true)
   ^^^
   
   >>> 
   ```
   
   Even if we add `SedonaSqlExtensions` to `spark.sql.extensions` and get rid 
of manual initialization, there's still problem with SQL parsing:
   
   ```
   pyspark --packages org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:1.7.0 \
           --jars "$SEDONA_SHADED_JAR" \
           --conf 
spark.sql.extensions=org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions,org.apache.sedona.sql.SedonaSqlExtensions
 \
           --conf spark.sql.catalog.local=org.apache.iceberg.spark.SparkCatalog 
\
           --conf spark.sql.catalog.local.type=hadoop \
           --conf 
spark.sql.catalog.local.warehouse=$HOME/local/iceberg/warehouse
   
   Welcome to
         ____              __
        / __/__  ___ _____/ /__
       _\ \/ _ \/ _ `/ __/  '_/
      /__ / .__/\_,_/_/ /_/\_\   version 3.5.1
         /_/
   
   Using Python version 3.11.6 (main, Oct  2 2023 20:46:14)
   Spark context Web UI available at http://192.168.200.155:4040
   Spark context available as 'sc' (master = local[*], app id = 
local-1732621080857).
   SparkSession available as 'spark'.
   >>> spark.sql("CALL local.system.remove_orphan_files (table => 
'test_db.test_table', dry_run => true)").show()
   24/11/26 19:38:14 WARN RasterRegistrator$: Geotools was not found on the 
classpath. Raster operations will not be available.
   +--------------------+
   |orphan_file_location|
   +--------------------+
   +--------------------+
   
   >>> spark.sql("CALL local.system.remove_orphan_files (table => 
'test_db.test_table', dry_run => true)").show()
   Traceback (most recent call last):
     File "<stdin>", line 1, in <module>
     File 
"/workspace/local/spark/spark-3.5.1-bin-hadoop3/python/pyspark/sql/session.py", 
line 1631, in sql
       return DataFrame(self._jsparkSession.sql(sqlQuery, litArgs), self)
                        ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
     File 
"/workspace/local/spark/spark-3.5.1-bin-hadoop3/python/lib/py4j-0.10.9.7-src.zip/py4j/java_gateway.py",
 line 1322, in __call__
     File 
"/workspace/local/spark/spark-3.5.1-bin-hadoop3/python/pyspark/errors/exceptions/captured.py",
 line 185, in deco
       raise converted from None
   pyspark.errors.exceptions.captured.ParseException: 
   [PARSE_SYNTAX_ERROR] Syntax error at or near 'CALL'.(line 1, pos 0)
   
   == SQL ==
   CALL local.system.remove_orphan_files (table => 'test_db.test_table', 
dry_run => true)
   ^^^
   
   >>> 
   ```
   
   I like the idea of overriding `astBuilder`, as it enables better integration 
with iceberg. However I'm a bit worried about removing the fallback path, since 
it may break iceberg's sql parsing when sedona and iceberg coexists.


-- 
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.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to