jackye1995 commented on a change in pull request #3132:
URL: https://github.com/apache/iceberg/pull/3132#discussion_r715224759



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java
##########
@@ -0,0 +1,170 @@
+/*
+ * 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 java.util.Locale;
+import java.util.Map;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.spark.sql.RuntimeConfig;
+import org.apache.spark.sql.SparkSession;
+
+/**
+ * A class for common Iceberg configs for Spark writes.
+ * <p>
+ * If a config is set at multiple levels, the following order of precedence is 
used (top to bottom):
+ * <ol>
+ *   <li>Write options</li>
+ *   <li>Session configuration</li>
+ *   <li>Table metadata</li>
+ * </ol>
+ * The most specific value is set in write options and takes precedence over 
all other configs.
+ * If no write option is provided, this class checks the session configuration 
for any overrides.
+ * If no applicable value is found in the session configuration, this classes 
uses the table metadata.
+ * <p>
+ * Note this class is NOT meant to be serialized and sent to executors.
+ */
+public class SparkWriteConf {
+
+  private final Table table;
+  private final RuntimeConfig sessionConf;
+  private final Map<String, String> writeOptions;
+
+  public SparkWriteConf(SparkSession spark, Table table, Map<String, String> 
writeOptions) {
+    this.table = table;
+    this.sessionConf = spark.conf();
+    this.writeOptions = writeOptions;
+  }
+
+  public boolean checkNullability() {
+    String writeOptionValue = 
writeOptions.get(SparkWriteOptions.CHECK_NULLABILITY);
+    if (writeOptionValue != null) {
+      return Boolean.parseBoolean(writeOptionValue);
+    }
+
+    String sessionConfValue = 
sessionConf.get(SparkSQLConfigs.CHECK_NULLABILITY, null);
+    if (sessionConfValue != null) {
+      return Boolean.parseBoolean(sessionConfValue);
+    }
+
+    return true;
+  }
+
+  public boolean checkOrdering() {
+    String writeOptionValue = 
writeOptions.get(SparkWriteOptions.CHECK_ORDERING);
+    if (writeOptionValue != null) {
+      return Boolean.parseBoolean(writeOptionValue);
+    }
+
+    String sessionConfValue = sessionConf.get(SparkSQLConfigs.CHECK_ORDERING, 
null);
+    if (sessionConfValue != null) {
+      return Boolean.parseBoolean(sessionConfValue);
+    }
+
+    return true;
+  }
+
+  /**
+   * Enables writing a timestamp with time zone as a timestamp without time 
zone.
+   * <p>
+   * Generally, this is not safe as a timestamp without time zone is supposed 
to represent the wall-clock time,
+   * i.e. no matter the reader/writer timezone 3PM should always be read as 
3PM,
+   * but a timestamp with time zone represents instant semantics, i.e. the 
timestamp
+   * is adjusted so that the corresponding time in the reader timezone is 
displayed.
+   * <p>
+   * When set to false (default), an exception must be thrown if the table 
contains a timestamp without time zone.
+   *
+   * @return boolean indicating if writing timestamps without timezone is 
allowed
+   */
+  public boolean handleTimestampWithoutZone() {
+    String writeOptionValue = 
writeOptions.get(SparkWriteOptions.HANDLE_TIMESTAMP_WITHOUT_TIMEZONE);
+    if (writeOptionValue != null) {
+      return Boolean.parseBoolean(writeOptionValue);
+    }
+
+    String sessionConfValue = 
sessionConf.get(SparkSQLConfigs.HANDLE_TIMESTAMP_WITHOUT_TIMEZONE, null);
+    if (sessionConfValue != null) {
+      return Boolean.parseBoolean(sessionConfValue);
+    }
+
+    return false;
+  }
+
+  public String overwriteMode() {
+    String overwriteMode = writeOptions.get("overwrite-mode");
+    return overwriteMode != null ? overwriteMode.toLowerCase(Locale.ROOT) : 
null;
+  }
+
+  public String wapId() {
+    return sessionConf.get("spark.wap.id", null);
+  }
+
+  public FileFormat dataFileFormat() {
+    String writeOptionValue = writeOptions.get(SparkWriteOptions.WRITE_FORMAT);

Review comment:
       But that will require us to read the default before reading the options 
value. 
   
   I am thinking maybe `Optional<String> getOption(String optionName)`, and 
then we can chain it with table property default, just a thought.




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



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

Reply via email to