You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2021/07/13 22:10:39 UTC

[GitHub] [iceberg] wypoon commented on a change in pull request #1508: Use schema at the time of the snapshot when reading a snapshot.

wypoon commented on a change in pull request #1508:
URL: https://github.com/apache/iceberg/pull/1508#discussion_r669143818



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/source/SparkTable.java
##########
@@ -76,24 +77,21 @@
       TableCapability.OVERWRITE_DYNAMIC);
 
   private final Table icebergTable;
-  private final StructType requestedSchema;
+  private final Long snapshotId;
+  private final Long asOfTimestamp;
   private final boolean refreshEagerly;
   private StructType lazyTableSchema = null;
   private SparkSession lazySpark = null;
 
   public SparkTable(Table icebergTable, boolean refreshEagerly) {
-    this(icebergTable, null, refreshEagerly);
+    this(icebergTable, null, null, refreshEagerly);
   }
 
-  public SparkTable(Table icebergTable, StructType requestedSchema, boolean refreshEagerly) {
+  public SparkTable(Table icebergTable, Long snapshotId, Long asOfTimestamp, boolean refreshEagerly) {
     this.icebergTable = icebergTable;
-    this.requestedSchema = requestedSchema;
+    this.snapshotId = snapshotId;
+    this.asOfTimestamp = asOfTimestamp;
     this.refreshEagerly = refreshEagerly;
-
-    if (requestedSchema != null) {
-      // convert the requested schema to throw an exception if any requested fields are unknown
-      SparkSchemaUtil.convert(icebergTable.schema(), requestedSchema);
-    }
   }

Review comment:
       I removed `requestedSchema` from `SparkTable` because with #1783, the spark3 `IcebergSource` changed to be a `SupportsCatalogOptions`, not just a `TableProvider`. Since `DataFrameReader` does not support specifying a schema when reading from an `IcebergSource`:
   ```
       DataSource.lookupDataSourceV2(source, sparkSession.sessionState.conf).map { provider =>
         ...
         val (table, catalog, ident) = provider match {
           case _: SupportsCatalogOptions if userSpecifiedSchema.nonEmpty =>
             throw new IllegalArgumentException(
               s"$source does not support user specified schema. Please don't specify the schema.")
   ```
   (see https://github.com/apache/spark/blob/v3.1.2/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala#L282-L285)
   there is no reason to have a `requestedSchema` field as we cannot make use of it.




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