Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,8 @@
*/
package org.apache.iceberg.spark.extensions;

import static org.apache.iceberg.types.Types.NestedField.optional;

import java.io.IOException;
import java.util.Comparator;
import java.util.List;
Expand All @@ -42,10 +44,13 @@
import org.apache.iceberg.spark.SparkSchemaUtil;
import org.apache.iceberg.spark.data.TestHelpers;
import org.apache.iceberg.spark.source.SimpleRecord;
import org.apache.iceberg.types.Types;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.RowFactory;
import org.apache.spark.sql.catalyst.util.DateTimeUtils;
import org.apache.spark.sql.types.StructType;
import org.junit.After;
import org.junit.Assert;
import org.junit.Test;
Expand Down Expand Up @@ -488,6 +493,70 @@ public void testMetadataLogEntries() throws Exception {
metadataLogWithProjection);
}

@Test
public void testFilesTableTimeTravelWithSchemaEvolution() throws Exception {
// Create table and insert data
sql(
"CREATE TABLE %s (id bigint, data string) "
+ "USING iceberg "
+ "PARTITIONED BY (data) "
+ "TBLPROPERTIES"
+ "('format-version'='2', 'write.delete.mode'='merge-on-read')",
tableName);

List<SimpleRecord> recordsA =
Lists.newArrayList(new SimpleRecord(1, "a"), new SimpleRecord(2, "a"));
spark
.createDataset(recordsA, Encoders.bean(SimpleRecord.class))
.coalesce(1)
.writeTo(tableName)
.append();

Table table = Spark3Util.loadIcebergTable(spark, tableName);

table.updateSchema().addColumn("category", Types.StringType.get()).commit();

List<Row> newRecords =
Lists.newArrayList(RowFactory.create(3, "b", "c"), RowFactory.create(4, "b", "c"));

StructType newSparkSchema =
SparkSchemaUtil.convert(
new Schema(
optional(1, "id", Types.IntegerType.get()),
optional(2, "data", Types.StringType.get()),
optional(3, "category", Types.StringType.get())));

spark.createDataFrame(newRecords, newSparkSchema).coalesce(1).writeTo(tableName).append();

Long currentSnapshotId = table.currentSnapshot().snapshotId();

Dataset<Row> actualFilesDs =
spark
.read()
.format("iceberg")
.option("snapshot-id", currentSnapshotId)
.load(tableName + ".files")
.orderBy("content");
List<Row> actualFiles = TestHelpers.selectNonDerived(actualFilesDs).collectAsList();
Schema entriesTableSchema = Spark3Util.loadIcebergTable(spark, tableName + ".entries").schema();
List<ManifestFile> expectedDataManifests = TestHelpers.dataManifests(table);
List<Record> expectedFiles =
expectedEntries(table, FileContent.DATA, entriesTableSchema, expectedDataManifests, null);

Assert.assertEquals("actualFiles size should be 2", 2, actualFiles.size());

TestHelpers.assertEqualsSafe(
TestHelpers.nonDerivedSchema(actualFilesDs), expectedFiles.get(0), actualFiles.get(0));

TestHelpers.assertEqualsSafe(
TestHelpers.nonDerivedSchema(actualFilesDs), expectedFiles.get(1), actualFiles.get(1));

Assert.assertEquals(
"expectedFiles and actualFiles size should be the same",
actualFiles.size(),
expectedFiles.size());
}

@Test
public void testSnapshotReferencesMetatable() throws Exception {
// Create table and insert data
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import java.io.IOException;
import java.util.Map;
import java.util.Set;
import org.apache.iceberg.BaseMetadataTable;
import org.apache.iceberg.BaseTable;
import org.apache.iceberg.DataFile;
import org.apache.iceberg.FileScanTask;
Expand Down Expand Up @@ -151,7 +152,11 @@ public String name() {
}

private Schema snapshotSchema() {
return SnapshotUtil.schemaFor(icebergTable, snapshotId, null);
if (icebergTable instanceof BaseMetadataTable) {
return icebergTable.schema();
} else {
return SnapshotUtil.schemaFor(icebergTable, snapshotId, null);
}
}

@Override
Expand Down