You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by "dramaticlly (via GitHub)" <gi...@apache.org> on 2023/02/14 22:26:37 UTC

[GitHub] [iceberg] dramaticlly opened a new pull request, #6838: Spark 3.3: Add a new Spark SQLConf to influence the write distribution mode

dramaticlly opened a new pull request, #6838:
URL: https://github.com/apache/iceberg/pull/6838

   close #6741 
   
   I will need to rebase the change once #6679 is merged but now we will have a new SQL config `spark.sql.iceberg.write.distribution-mode` to influence how spark can distribute its write for delete (which currently can only be achieved via SQL)
   
   CC @aokolnychyi @szehon-ho @JunchengMa


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] amogh-jahagirdar commented on a diff in pull request #6838: Spark 3.3: Add a new Spark SQLConf to influence the write distribution mode

Posted by "amogh-jahagirdar (via GitHub)" <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6838:
URL: https://github.com/apache/iceberg/pull/6838#discussion_r1107848388


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConfDistributionMode.java:
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.iceberg.spark;
+
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE;
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_HASH;
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_NONE;
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_RANGE;
+
+import java.util.Map;
+import org.apache.iceberg.DistributionMode;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestSparkWriteConfDistributionMode extends SparkTestBaseWithCatalog {
+
+  @After
+  public void cleanUp() {
+    spark.conf().unset(SparkSQLProperties.SESSION_WRITE_DISTRIBUTION_MODE);
+    sql("DROP TABLE IF EXISTS %s", tableName);
+  }
+
+  @Test
+  public void testSparkWriteConfDistributionDefault() {
+    sql(
+        "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) "
+            + "USING iceberg "
+            + "PARTITIONED BY (date, days(ts))",
+        tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    SparkWriteConf writeConf = new SparkWriteConf(spark, table, ImmutableMap.of());
+    Assert.assertEquals(DistributionMode.NONE, writeConf.distributionMode());
+  }
+
+  @Test
+  public void testSparkWriteConfDistributionModeWithWriteOption() {
+    sql(
+        "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) "
+            + "USING iceberg "
+            + "PARTITIONED BY (date, days(ts))",
+        tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    Map<String, String> writeOptions =
+        ImmutableMap.of(SparkWriteOptions.DISTRIBUTION_MODE, DistributionMode.HASH.modeName());
+    SparkWriteConf writeConf = new SparkWriteConf(spark, table, writeOptions);
+
+    Assert.assertEquals(DistributionMode.HASH, writeConf.distributionMode());
+  }
+
+  @Test
+  public void testSparkWriteConfDistributionModeWithSessionConfig() {
+    sql(
+        "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) "
+            + "USING iceberg "
+            + "PARTITIONED BY (date, days(ts))",
+        tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    spark
+        .conf()
+        .set(SparkSQLProperties.SESSION_WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName());
+
+    SparkWriteConf writeConf = new SparkWriteConf(spark, table, ImmutableMap.of());
+    Assert.assertEquals(DistributionMode.HASH, writeConf.distributionMode());
+  }
+
+  @Test
+  public void testSparkWriteConfDistributionModeWithTableProperties() {
+    sql(
+        "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) "
+            + "USING iceberg "
+            + "PARTITIONED BY (date, days(ts))",
+        tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    table.updateProperties().set(WRITE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH).commit();
+
+    SparkWriteConf writeConf = new SparkWriteConf(spark, table, ImmutableMap.of());
+    Assert.assertEquals(DistributionMode.HASH, writeConf.distributionMode());
+  }
+
+  @Test
+  public void testSparkWriteConfDistributionModeWithTblPropAndSessionConfig() {
+    sql(
+        "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) "
+            + "USING iceberg "
+            + "PARTITIONED BY (date, days(ts))",
+        tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    table.updateProperties().set(WRITE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE).commit();
+    spark
+        .conf()
+        .set(SparkSQLProperties.SESSION_WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName());
+
+    SparkWriteConf writeConf = new SparkWriteConf(spark, table, ImmutableMap.of());
+    // session config overwrite the table properties
+    Assert.assertEquals(DistributionMode.HASH, writeConf.distributionMode());
+  }
+
+  @Test
+  public void testSparkWriteConfDistributionModeWithWriteOptionAndSessionConfig() {
+    sql(
+        "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) "
+            + "USING iceberg "
+            + "PARTITIONED BY (date, days(ts))",
+        tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    Map<String, String> writeOptions =
+        ImmutableMap.of(SparkWriteOptions.DISTRIBUTION_MODE, DistributionMode.RANGE.modeName());
+    spark
+        .conf()
+        .set(SparkSQLProperties.SESSION_WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName());
+
+    SparkWriteConf writeConf = new SparkWriteConf(spark, table, writeOptions);
+    // write options overwrite the session config
+    Assert.assertEquals(DistributionMode.RANGE, writeConf.distributionMode());
+  }
+
+  @Test
+  public void testSparkWriteConfDistributionModeWithEverything() {
+    sql(
+        "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) "
+            + "USING iceberg "
+            + "PARTITIONED BY (date, days(ts))",
+        tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    Map<String, String> writeOptions =
+        ImmutableMap.of(SparkWriteOptions.DISTRIBUTION_MODE, DistributionMode.RANGE.modeName());
+    spark
+        .conf()
+        .set(SparkSQLProperties.SESSION_WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName());

Review Comment:
   Nit could we have a newline after line 162 before the table.updateProperties(), makes it easier to read IMO.



##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConfDistributionMode.java:
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.iceberg.spark;
+
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE;
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_HASH;
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_NONE;
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_RANGE;
+
+import java.util.Map;
+import org.apache.iceberg.DistributionMode;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestSparkWriteConfDistributionMode extends SparkTestBaseWithCatalog {

Review Comment:
   Any reason we need a separate test class and not just add to `TestSparkDistributionAndOrderingUtil` ? 



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java:
##########
@@ -47,4 +47,8 @@ private SparkSQLProperties() {}
   public static final String PRESERVE_DATA_GROUPING =
       "spark.sql.iceberg.planning.preserve-data-grouping";
   public static final boolean PRESERVE_DATA_GROUPING_DEFAULT = false;
+
+  // Influence write distribution mode using spark SQL config

Review Comment:
   Nit could the comment just be `Control write distribution mode` similar pattern to the existing configs, it's explanatory from the class itself that it's for SparkSQL properties. 



##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConfDistributionMode.java:
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.iceberg.spark;
+
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE;
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_HASH;
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_NONE;
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_RANGE;
+
+import java.util.Map;
+import org.apache.iceberg.DistributionMode;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestSparkWriteConfDistributionMode extends SparkTestBaseWithCatalog {
+
+  @After
+  public void cleanUp() {
+    spark.conf().unset(SparkSQLProperties.SESSION_WRITE_DISTRIBUTION_MODE);
+    sql("DROP TABLE IF EXISTS %s", tableName);
+  }
+
+  @Test
+  public void testSparkWriteConfDistributionDefault() {
+    sql(
+        "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) "
+            + "USING iceberg "
+            + "PARTITIONED BY (date, days(ts))",
+        tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    SparkWriteConf writeConf = new SparkWriteConf(spark, table, ImmutableMap.of());
+    Assert.assertEquals(DistributionMode.NONE, writeConf.distributionMode());
+  }
+
+  @Test
+  public void testSparkWriteConfDistributionModeWithWriteOption() {
+    sql(
+        "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) "
+            + "USING iceberg "
+            + "PARTITIONED BY (date, days(ts))",
+        tableName);

Review Comment:
   I think this can be in an `@Before`



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi merged pull request #6838: Spark 3.3: Add a new Spark SQLConf to influence the write distribution mode

Posted by "aokolnychyi (via GitHub)" <gi...@apache.org>.
aokolnychyi merged PR #6838:
URL: https://github.com/apache/iceberg/pull/6838


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] amogh-jahagirdar commented on a diff in pull request #6838: Spark 3.3: Add a new Spark SQLConf to influence the write distribution mode

Posted by "amogh-jahagirdar (via GitHub)" <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #6838:
URL: https://github.com/apache/iceberg/pull/6838#discussion_r1109178928


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConf.java:
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.iceberg.spark;
+
+import static org.apache.iceberg.TableProperties.DELETE_DISTRIBUTION_MODE;
+import static org.apache.iceberg.TableProperties.MERGE_DISTRIBUTION_MODE;
+import static org.apache.iceberg.TableProperties.UPDATE_DISTRIBUTION_MODE;
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE;
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_HASH;
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_NONE;
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_RANGE;
+
+import java.util.Map;
+import org.apache.iceberg.DistributionMode;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestSparkWriteConf extends SparkTestBaseWithCatalog {
+
+  @Before
+  public void before() {
+    sql(
+        "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) "
+            + "USING iceberg "
+            + "PARTITIONED BY (date, days(ts))",
+        tableName);
+  }
+
+  @After
+  public void after() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+  }
+
+  @Test
+  public void testSparkWriteConfDistributionDefault() {
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    SparkWriteConf writeConf = new SparkWriteConf(spark, table, ImmutableMap.of());
+
+    Assert.assertEquals(DistributionMode.HASH, writeConf.distributionMode());
+    Assert.assertEquals(DistributionMode.HASH, writeConf.deleteDistributionMode());
+    Assert.assertEquals(DistributionMode.HASH, writeConf.updateDistributionMode());
+    Assert.assertEquals(DistributionMode.HASH, writeConf.copyOnWriteMergeDistributionMode());
+    Assert.assertEquals(DistributionMode.HASH, writeConf.positionDeltaMergeDistributionMode());
+  }
+
+  @Test
+  public void testSparkWriteConfDistributionModeWithWriteOption() {
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    Map<String, String> writeOptions =
+        ImmutableMap.of(SparkWriteOptions.DISTRIBUTION_MODE, DistributionMode.NONE.modeName());
+
+    SparkWriteConf writeConf = new SparkWriteConf(spark, table, writeOptions);
+    Assert.assertEquals(DistributionMode.NONE, writeConf.distributionMode());
+    Assert.assertEquals(DistributionMode.NONE, writeConf.deleteDistributionMode());
+    Assert.assertEquals(DistributionMode.NONE, writeConf.updateDistributionMode());
+    Assert.assertEquals(DistributionMode.NONE, writeConf.copyOnWriteMergeDistributionMode());
+    Assert.assertEquals(DistributionMode.NONE, writeConf.positionDeltaMergeDistributionMode());
+  }
+
+  @Test
+  public void testSparkWriteConfDistributionModeWithSessionConfig() {
+    withSQLConf(
+        ImmutableMap.of(SparkSQLProperties.DISTRIBUTION_MODE, DistributionMode.NONE.modeName()),
+        () -> {
+          Table table = validationCatalog.loadTable(tableIdent);
+
+          SparkWriteConf writeConf = new SparkWriteConf(spark, table, ImmutableMap.of());
+
+          Assert.assertEquals(DistributionMode.NONE, writeConf.distributionMode());
+          Assert.assertEquals(DistributionMode.NONE, writeConf.deleteDistributionMode());
+          Assert.assertEquals(DistributionMode.NONE, writeConf.updateDistributionMode());
+          Assert.assertEquals(DistributionMode.NONE, writeConf.copyOnWriteMergeDistributionMode());
+          Assert.assertEquals(
+              DistributionMode.NONE, writeConf.positionDeltaMergeDistributionMode());
+        });
+  }
+
+  @Test
+  public void testSparkWriteConfDistributionModeWithTableProperties() {
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    table
+        .updateProperties()
+        .set(WRITE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_NONE)
+        .set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_NONE)
+        .set(UPDATE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_NONE)
+        .set(MERGE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_NONE)
+        .commit();
+
+    SparkWriteConf writeConf = new SparkWriteConf(spark, table, ImmutableMap.of());
+    Assert.assertEquals(DistributionMode.NONE, writeConf.distributionMode());
+    Assert.assertEquals(DistributionMode.NONE, writeConf.deleteDistributionMode());
+    Assert.assertEquals(DistributionMode.NONE, writeConf.updateDistributionMode());
+    Assert.assertEquals(DistributionMode.NONE, writeConf.copyOnWriteMergeDistributionMode());
+    Assert.assertEquals(DistributionMode.NONE, writeConf.positionDeltaMergeDistributionMode());

Review Comment:
   Nonblocking nit: I guess this is a case where we can have some kind of helper function to validate that all of the write conf options match the given distribution mode.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on pull request #6838: Spark 3.3: Add a new Spark SQLConf to influence the write distribution mode

Posted by "aokolnychyi (via GitHub)" <gi...@apache.org>.
aokolnychyi commented on PR #6838:
URL: https://github.com/apache/iceberg/pull/6838#issuecomment-1432120227

   Will review today.


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] dramaticlly commented on a diff in pull request #6838: Spark 3.3: Add a new Spark SQLConf to influence the write distribution mode

Posted by "dramaticlly (via GitHub)" <gi...@apache.org>.
dramaticlly commented on code in PR #6838:
URL: https://github.com/apache/iceberg/pull/6838#discussion_r1107976878


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java:
##########
@@ -207,6 +207,7 @@ public DistributionMode distributionMode() {
         confParser
             .stringConf()
             .option(SparkWriteOptions.DISTRIBUTION_MODE)
+            .sessionConf(SparkSQLProperties.SESSION_WRITE_DISTRIBUTION_MODE)

Review Comment:
   thank you Anton, I will rebase after your merge and make sure corresponding sessionConf option for
   - distributionMode() 
   - deleteDistributionMode()
   - updateDistributionMode()
   - copyOnWriteMergeDistributionMode()
   - positionDeltaMergeDistributionMode()



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java:
##########
@@ -207,6 +207,7 @@ public DistributionMode distributionMode() {
         confParser
             .stringConf()
             .option(SparkWriteOptions.DISTRIBUTION_MODE)
+            .sessionConf(SparkSQLProperties.SESSION_WRITE_DISTRIBUTION_MODE)

Review Comment:
   thank you Anton, I will rebase after your PR is merged and make sure corresponding sessionConf option for
   - distributionMode() 
   - deleteDistributionMode()
   - updateDistributionMode()
   - copyOnWriteMergeDistributionMode()
   - positionDeltaMergeDistributionMode()



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] dramaticlly commented on a diff in pull request #6838: Spark 3.3: Add a new Spark SQLConf to influence the write distribution mode

Posted by "dramaticlly (via GitHub)" <gi...@apache.org>.
dramaticlly commented on code in PR #6838:
URL: https://github.com/apache/iceberg/pull/6838#discussion_r1107891891


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConfDistributionMode.java:
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.iceberg.spark;
+
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE;
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_HASH;
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_NONE;
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_RANGE;
+
+import java.util.Map;
+import org.apache.iceberg.DistributionMode;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestSparkWriteConfDistributionMode extends SparkTestBaseWithCatalog {

Review Comment:
   yes, good call. Initially I was thinking about putting the logic into `TestSparkDistributionAndOrderingUtil` but then I realized the change I made is only about influencing the precedence level in `SparkWriteConf`. 
   My change can be individually tested without any call to use method in `SparkDistributionAndOrderingUtil` class, so it does not seem make sense to add more complexity to 2000+ line of unit tests. 
   
   Looks like I dont find any existing tests focus on the SparkWriteConf alone, I just created a this new one and it seems neat.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on pull request #6838: Spark 3.3: Add a new Spark SQLConf to influence the write distribution mode

Posted by "aokolnychyi (via GitHub)" <gi...@apache.org>.
aokolnychyi commented on PR #6838:
URL: https://github.com/apache/iceberg/pull/6838#issuecomment-1432522831

   @dramaticlly, this one can be rebased now.


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6838: Spark 3.3: Add a new Spark SQLConf to influence the write distribution mode

Posted by "aokolnychyi (via GitHub)" <gi...@apache.org>.
aokolnychyi commented on code in PR #6838:
URL: https://github.com/apache/iceberg/pull/6838#discussion_r1107960743


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConfDistributionMode.java:
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.iceberg.spark;
+
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE;
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_HASH;
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_NONE;
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_RANGE;
+
+import java.util.Map;
+import org.apache.iceberg.DistributionMode;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestSparkWriteConfDistributionMode extends SparkTestBaseWithCatalog {

Review Comment:
   What about calling it `TestSparkWriteConf` so that it becomes generic? We may add more tests for other configs in the future. I think it is fine to purely test the config inheritance in this PR.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] dramaticlly commented on pull request #6838: Spark 3.3: Add a new Spark SQLConf to influence the write distribution mode

Posted by "dramaticlly (via GitHub)" <gi...@apache.org>.
dramaticlly commented on PR #6838:
URL: https://github.com/apache/iceberg/pull/6838#issuecomment-1433791990

   I rebased my changed, can you take another look? @amogh-jahagirdar and @aokolnychyi 


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on pull request #6838: Spark 3.3: Add a new Spark SQLConf to influence the write distribution mode

Posted by "aokolnychyi (via GitHub)" <gi...@apache.org>.
aokolnychyi commented on PR #6838:
URL: https://github.com/apache/iceberg/pull/6838#issuecomment-1435236861

   Thanks, @dramaticlly! Thanks for reviewing, @amogh-jahagirdar!


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6838: Spark 3.3: Add a new Spark SQLConf to influence the write distribution mode

Posted by "aokolnychyi (via GitHub)" <gi...@apache.org>.
aokolnychyi commented on code in PR #6838:
URL: https://github.com/apache/iceberg/pull/6838#discussion_r1107961291


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConfDistributionMode.java:
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.iceberg.spark;
+
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE;
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_HASH;
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_NONE;
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_RANGE;
+
+import java.util.Map;
+import org.apache.iceberg.DistributionMode;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestSparkWriteConfDistributionMode extends SparkTestBaseWithCatalog {
+
+  @Before
+  public void before() {
+    sql(
+        "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) "
+            + "USING iceberg "
+            + "PARTITIONED BY (date, days(ts))",
+        tableName);
+  }
+
+  @After
+  public void after() {
+    spark.conf().unset(SparkSQLProperties.SESSION_WRITE_DISTRIBUTION_MODE);

Review Comment:
   Instead of unsetting values in `after`, we normally use `withSQLConf()` method in tests.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java:
##########
@@ -47,4 +47,8 @@ private SparkSQLProperties() {}
   public static final String PRESERVE_DATA_GROUPING =
       "spark.sql.iceberg.planning.preserve-data-grouping";
   public static final boolean PRESERVE_DATA_GROUPING_DEFAULT = false;
+
+  // Controls write distribution mode
+  public static final String SESSION_WRITE_DISTRIBUTION_MODE =

Review Comment:
   I don't think we need the `SESSION_` prefix in the name. All SQL properties are session properties.



##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConfDistributionMode.java:
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.iceberg.spark;
+
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE;
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_HASH;
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_NONE;
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_RANGE;
+
+import java.util.Map;
+import org.apache.iceberg.DistributionMode;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestSparkWriteConfDistributionMode extends SparkTestBaseWithCatalog {
+
+  @Before
+  public void before() {
+    sql(
+        "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) "
+            + "USING iceberg "
+            + "PARTITIONED BY (date, days(ts))",
+        tableName);
+  }
+
+  @After
+  public void after() {
+    spark.conf().unset(SparkSQLProperties.SESSION_WRITE_DISTRIBUTION_MODE);
+    sql("DROP TABLE IF EXISTS %s", tableName);
+  }
+
+  @Test
+  public void testSparkWriteConfDistributionDefault() {
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    SparkWriteConf writeConf = new SparkWriteConf(spark, table, ImmutableMap.of());
+
+    Assert.assertEquals(DistributionMode.NONE, writeConf.distributionMode());
+  }
+
+  @Test
+  public void testSparkWriteConfDistributionModeWithWriteOption() {
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    Map<String, String> writeOptions =
+        ImmutableMap.of(SparkWriteOptions.DISTRIBUTION_MODE, DistributionMode.HASH.modeName());
+
+    SparkWriteConf writeConf = new SparkWriteConf(spark, table, writeOptions);
+    Assert.assertEquals(DistributionMode.HASH, writeConf.distributionMode());
+  }
+
+  @Test
+  public void testSparkWriteConfDistributionModeWithSessionConfig() {
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    spark
+        .conf()
+        .set(SparkSQLProperties.SESSION_WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName());
+
+    SparkWriteConf writeConf = new SparkWriteConf(spark, table, ImmutableMap.of());
+    Assert.assertEquals(DistributionMode.HASH, writeConf.distributionMode());
+  }
+
+  @Test
+  public void testSparkWriteConfDistributionModeWithTableProperties() {
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    table.updateProperties().set(WRITE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH).commit();
+
+    SparkWriteConf writeConf = new SparkWriteConf(spark, table, ImmutableMap.of());
+    Assert.assertEquals(DistributionMode.HASH, writeConf.distributionMode());
+  }
+
+  @Test
+  public void testSparkWriteConfDistributionModeWithTblPropAndSessionConfig() {
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    table.updateProperties().set(WRITE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE).commit();
+
+    spark

Review Comment:
   Such calls should be replaced with `withSQLConf()`.



##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConfDistributionMode.java:
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.iceberg.spark;
+
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE;
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_HASH;
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_NONE;
+import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_RANGE;
+
+import java.util.Map;
+import org.apache.iceberg.DistributionMode;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestSparkWriteConfDistributionMode extends SparkTestBaseWithCatalog {

Review Comment:
   What about calling it `TestSparkWriteConf` so that it becomes generic? We may add more tests for other configs in the future. I think it is fine to purely test the config.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java:
##########
@@ -207,6 +207,7 @@ public DistributionMode distributionMode() {
         confParser
             .stringConf()
             .option(SparkWriteOptions.DISTRIBUTION_MODE)
+            .sessionConf(SparkSQLProperties.SESSION_WRITE_DISTRIBUTION_MODE)

Review Comment:
   Once we merge #6828, this SQL conf must be also respected in other distribution modes wherever we respect the corresponding write option (e.g. DELETE, UPDATE, MERGE commands).



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java:
##########
@@ -47,4 +47,8 @@ private SparkSQLProperties() {}
   public static final String PRESERVE_DATA_GROUPING =
       "spark.sql.iceberg.planning.preserve-data-grouping";
   public static final boolean PRESERVE_DATA_GROUPING_DEFAULT = false;
+
+  // Controls write distribution mode
+  public static final String SESSION_WRITE_DISTRIBUTION_MODE =

Review Comment:
   I see our SQL configs rarely include read/write prefixes. What about this then?
   
   ```
   public static final String DISTRIBUTION_MODE = "spark.sql.iceberg.distribution-mode"
   ```



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] dramaticlly commented on a diff in pull request #6838: Spark 3.3: Add a new Spark SQLConf to influence the write distribution mode

Posted by "dramaticlly (via GitHub)" <gi...@apache.org>.
dramaticlly commented on code in PR #6838:
URL: https://github.com/apache/iceberg/pull/6838#discussion_r1107984679


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java:
##########
@@ -207,6 +207,7 @@ public DistributionMode distributionMode() {
         confParser
             .stringConf()
             .option(SparkWriteOptions.DISTRIBUTION_MODE)
+            .sessionConf(SparkSQLProperties.SESSION_WRITE_DISTRIBUTION_MODE)

Review Comment:
   Also needs to thinking of a best way to test all those methods. I guess the core is to verify the config inheritance to see  how its precedence level is being received, regardless of distribution mode is used for insert, delete, update or merge



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] dramaticlly commented on pull request #6838: Spark 3.3: Add a new Spark SQLConf to influence the write distribution mode

Posted by "dramaticlly (via GitHub)" <gi...@apache.org>.
dramaticlly commented on PR #6838:
URL: https://github.com/apache/iceberg/pull/6838#issuecomment-1435237338

   Thank you @aokolnychyi , I can refactor and also add more tests for SparkWirteConf in general if you want. 


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org