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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.Map;
+import java.util.Set;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+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 reads.
+ * <p>
+ * If a config is set at multiple levels, the following order of precedence is 
used (top to bottom):
+ * <ol>
+ *   <li>Read options</li>
+ *   <li>Session configuration</li>
+ *   <li>Table metadata</li>
+ * </ol>
+ * The most specific value is set in read options and takes precedence over 
all other configs.
+ * If no read 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 SparkReadConf {
+
+  private static final Set<String> LOCALITY_WHITELIST_FS = 
ImmutableSet.of("hdfs");
+
+  private final Table table;
+  private final RuntimeConfig sessionConf;
+  private final Map<String, String> readOptions;
+
+  public SparkReadConf(SparkSession spark, Table table, Map<String, String> 
readOptions) {
+    this.table = table;
+    this.sessionConf = spark.conf();
+    this.readOptions = readOptions;
+  }
+
+  public boolean localityEnabled() {
+    InputFile file = table.io().newInputFile(table.location());
+
+    if (file instanceof HadoopInputFile) {
+      String scheme = ((HadoopInputFile) file).getFileSystem().getScheme();
+      boolean defaultValue = LOCALITY_WHITELIST_FS.contains(scheme);
+      return PropertyUtil.propertyAsBoolean(
+          readOptions,
+          SparkReadOptions.LOCALITY,
+          defaultValue);
+    }
+
+    return false;
+  }
+
+  public boolean vectorizationEnabled(FileFormat fileFormat) {
+    String readOptionValue = 
readOptions.get(SparkReadOptions.VECTORIZATION_ENABLED);
+    if (readOptionValue != null) {
+      return Boolean.parseBoolean(readOptionValue);
+    }
+
+    String sessionConfValue = 
sessionConf.get(SparkSQLConfigs.VECTORIZATION_ENABLED, null);
+    if (sessionConfValue != null) {
+      return Boolean.parseBoolean(sessionConfValue);
+    }
+
+    switch (fileFormat) {
+      case PARQUET:
+        return PropertyUtil.propertyAsBoolean(
+            table.properties(),
+            TableProperties.PARQUET_VECTORIZATION_ENABLED,
+            TableProperties.PARQUET_VECTORIZATION_ENABLED_DEFAULT);
+
+      case ORC:
+        return PropertyUtil.propertyAsBoolean(
+            table.properties(),
+            TableProperties.ORC_VECTORIZATION_ENABLED,
+            TableProperties.ORC_VECTORIZATION_ENABLED_DEFAULT);
+
+      default:
+        return false;
+    }
+  }
+
+  public int batchSize(FileFormat fileFormat) {

Review comment:
       Yes, if you think that isn't a good idea we don't have to. I was just 
thinking that we should have a 
   batchSizeOrc
   batchSizeParquet
   
   In the conf, and then the reader would choose them in
   ```
     private int batchSize(boolean isParquetOnly, boolean isOrcOnly) {
       if (isParquetOnly) {
         return readConf.batchSizeParquet()
       } else if (isOrcOnly) {
         return readConf.batchSizeOrc()
       } else {
         return 0;
   ```
   
   Instead of the current code which passes through the file type.
   




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