This is an automated email from the ASF dual-hosted git repository.

dongjoon pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/spark-kubernetes-operator.git


The following commit(s) were added to refs/heads/main by this push:
     new 67da720  [SPARK-53933] Add `Apache Iceberg` example
67da720 is described below

commit 67da720eaf7df2298858747a4aa3bae87528141d
Author: Dongjoon Hyun <[email protected]>
AuthorDate: Wed Oct 15 23:27:47 2025 -0700

    [SPARK-53933] Add `Apache Iceberg` example
    
    ### What changes were proposed in this pull request?
    
    This PR aims to add `Apache Iceberg` example.
    
    ### Why are the changes needed?
    
    To provide an working example of `Apache Spark 4.0.1` and `Apache Iceberg 
1.10.0`.
    
    1. Prepare the storage
    
    ```
    $ kubectl apply -f examples/localstack.ym
    ```
    
    2. Launch `Spark Connect Server` with `Apache Iceberg` setting.
    
    ```
    $ kubectl apply -f examples/spark-connect-server-iceberg.yaml
    ```
    
    3. Setup port-forwarding to test
    
    ```
    $ kubectl port-forward spark-connect-server-iceberg-0-driver 15002
    ```
    
    4. Test with `Apache Iceberg Spark Quickstart` guideline.
    - https://iceberg.apache.org/spark-quickstart/
    
    ```scala
    $ bin/spark-connect-shell --remote sc://localhost:15002
    
    scala> sql("""CREATE TABLE taxis(vendor_id bigint, trip_id bigint, 
trip_distance float, fare_amount double, store_and_fwd_flag string) PARTITIONED 
BY (vendor_id);""").show()
    
    scala> sql("""INSERT INTO taxis VALUES (1, 1000371, 1.8, 15.32, 'N'), (2, 
1000372, 2.5, 22.15, 'N'), (2, 1000373, 0.9, 9.01, 'N'), (1, 1000374, 8.4, 
42.13, 'Y');""").show()
    
    scala> sql("SELECT * FROM taxis").show(false)
    +---------+-------+-------------+-----------+------------------+
    |vendor_id|trip_id|trip_distance|fare_amount|store_and_fwd_flag|
    +---------+-------+-------------+-----------+------------------+
    |1        |1000374|8.4          |42.13      |Y                 |
    |1        |1000371|1.8          |15.32      |N                 |
    |2        |1000372|2.5          |22.15      |N                 |
    |2        |1000373|0.9          |9.01       |N                 |
    +---------+-------+-------------+-----------+------------------+
    
    scala> sql("SELECT * FROM taxis.history").show(false)
    +-----------------------+-------------------+---------+-------------------+
    |made_current_at        |snapshot_id        |parent_id|is_current_ancestor|
    +-----------------------+-------------------+---------+-------------------+
    |2025-10-16 03:53:04.063|6463217948421571140|NULL     |true               |
    +-----------------------+-------------------+---------+-------------------+
    
    scala> sql("SELECT * FROM taxis VERSION AS OF 
6463217948421571140").show(false)
    +---------+-------+-------------+-----------+------------------+
    |vendor_id|trip_id|trip_distance|fare_amount|store_and_fwd_flag|
    +---------+-------+-------------+-----------+------------------+
    |1        |1000374|8.4          |42.13      |Y                 |
    |1        |1000371|1.8          |15.32      |N                 |
    |2        |1000372|2.5          |22.15      |N                 |
    |2        |1000373|0.9          |9.01       |N                 |
    +---------+-------+-------------+-----------+------------------+
    ```
    
    5. Check the data in the storage.
    
    ```
    rootlocalstack:/opt/code/localstack# awslocal s3 ls s3://warehouse/ 
--recursive
    2025-10-16 03:53:03       1545 
taxis/data/vendor_id=1/00000-3-749fe2e5-bbe3-4f0e-b976-a21749550705-0-00002.parquet
    2025-10-16 03:53:03       1590 
taxis/data/vendor_id=2/00000-3-749fe2e5-bbe3-4f0e-b976-a21749550705-0-00001.parquet
    2025-10-16 03:53:04       7559 
taxis/metadata/9f629d40-ce91-4822-aeee-283d53ec5ef6-m0.avro
    2025-10-16 03:53:04       4446 
taxis/metadata/snap-6463217948421571140-1-9f629d40-ce91-4822-aeee-283d53ec5ef6.avro
    2025-10-16 03:52:47       1006 taxis/metadata/v1.metadata.json
    2025-10-16 03:53:04       1970 taxis/metadata/v2.metadata.json
    2025-10-16 03:53:04          1 taxis/metadata/version-hint.text
    ```
    
    ### Does this PR introduce _any_ user-facing change?
    
    No.
    
    ### How was this patch tested?
    
    Manually tested.
    
    ### Was this patch authored or co-authored using generative AI tooling?
    
    No.
    
    Closes #394 from dongjoon-hyun/SPARK-53933.
    
    Authored-by: Dongjoon Hyun <[email protected]>
    Signed-off-by: Dongjoon Hyun <[email protected]>
---
 examples/localstack.yml                    |  1 +
 examples/spark-connect-server-iceberg.yaml | 45 ++++++++++++++++++++++++++++++
 2 files changed, 46 insertions(+)

diff --git a/examples/localstack.yml b/examples/localstack.yml
index 1c3088b..841ff45 100644
--- a/examples/localstack.yml
+++ b/examples/localstack.yml
@@ -41,6 +41,7 @@ spec:
             awslocal s3 mb s3://spark-events;
             awslocal s3 mb s3://ingest;
             awslocal s3 mb s3://data;
+            awslocal s3 mb s3://warehouse;
             awslocal s3 cp /opt/code/localstack/Makefile s3://data/
 ---
 apiVersion: v1
diff --git a/examples/spark-connect-server-iceberg.yaml 
b/examples/spark-connect-server-iceberg.yaml
new file mode 100644
index 0000000..6f54c63
--- /dev/null
+++ b/examples/spark-connect-server-iceberg.yaml
@@ -0,0 +1,45 @@
+# 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.
+apiVersion: spark.apache.org/v1
+kind: SparkApplication
+metadata:
+  name: spark-connect-server-iceberg
+spec:
+  mainClass: "org.apache.spark.sql.connect.service.SparkConnectServer"
+  sparkConf:
+    spark.dynamicAllocation.enabled: "true"
+    spark.dynamicAllocation.maxExecutors: "3"
+    spark.dynamicAllocation.minExecutors: "3"
+    spark.dynamicAllocation.shuffleTracking.enabled: "true"
+    spark.hadoop.fs.s3a.access.key: "test"
+    spark.hadoop.fs.s3a.endpoint: "http://localstack:4566";
+    spark.hadoop.fs.s3a.path.style.access: "true"
+    spark.hadoop.fs.s3a.secret.key: "test"
+    spark.jars.ivy: "/tmp/.ivy2.5.2"
+    spark.jars.packages: 
"org.apache.hadoop:hadoop-aws:3.4.1,org.apache.iceberg:iceberg-spark-runtime-4.0_2.13:1.10.0"
+    spark.kubernetes.authenticate.driver.serviceAccountName: "spark"
+    spark.kubernetes.container.image: "apache/spark:4.0.1"
+    spark.kubernetes.driver.pod.excludedFeatureSteps: 
"org.apache.spark.deploy.k8s.features.KerberosConfDriverFeatureStep"
+    spark.kubernetes.executor.podNamePrefix: "spark-connect-server-iceberg"
+    spark.scheduler.mode: "FAIR"
+    spark.sql.catalog.s3.type: "hadoop"
+    spark.sql.catalog.s3.warehouse: "s3a://warehouse"
+    spark.sql.catalog.s3: "org.apache.iceberg.spark.SparkCatalog"
+    spark.sql.defaultCatalog: "s3"
+    spark.sql.extensions: 
"org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions"
+  applicationTolerations:
+    resourceRetainPolicy: OnFailure
+  runtimeVersions:
+    sparkVersion: "4.0.1"


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

Reply via email to