rdblue commented on a change in pull request #1508:
URL: https://github.com/apache/iceberg/pull/1508#discussion_r698063278
##########
File path:
spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java
##########
@@ -901,6 +915,199 @@ public void testPartitionsTable() {
}
}
+ @Test
+ public synchronized void testSnapshotReadAfterAddColumn() {
+ TableIdentifier tableIdentifier = TableIdentifier.of("db", "table");
+ Table table = createTable(tableIdentifier, SCHEMA,
PartitionSpec.unpartitioned());
+
+ List<Row> expectedRecords = Lists.newArrayList(
+ RowFactory.create(1, "x"),
+ RowFactory.create(2, "y"),
+ RowFactory.create(3, "z"));
+
+ Dataset<Row> inputDf = spark.createDataFrame(expectedRecords,
SparkSchemaUtil.convert(SCHEMA));
+ inputDf.select("id", "data").write()
+ .format("iceberg")
+ .mode(SaveMode.Append)
+ .save(loadLocation(tableIdentifier));
+
+ table.refresh();
+
+ Dataset<Row> resultDf = spark.read()
+ .format("iceberg")
+ .load(loadLocation(tableIdentifier));
+ List<Row> actualRecords = resultDf.orderBy("id")
+ .collectAsList();
+
+ Assert.assertEquals("Records should match", expectedRecords,
actualRecords);
+ Snapshot snapshot1 = table.currentSnapshot();
+
+ table.updateSchema().addColumn("category",
Types.StringType.get()).commit();
+
+ List<Row> newRecords = Lists.newArrayList(
+ RowFactory.create(4, "xy", "B"),
+ RowFactory.create(5, "xyz", "C"));
+
+ Dataset<Row> inputDf2 = spark.createDataFrame(newRecords,
SparkSchemaUtil.convert(SCHEMA2));
+ inputDf2.select("id", "data", "category").write()
+ .format("iceberg")
+ .mode(SaveMode.Append)
+ .save(loadLocation(tableIdentifier));
+
+ table.refresh();
+
+ Dataset<Row> resultDf2 = spark.read()
+ .format("iceberg")
+ .load(loadLocation(tableIdentifier));
+ List<Row> actualRecords2 = resultDf2.orderBy("id")
+ .collectAsList();
+
+ List<Row> expectedRecords2 = Lists.newArrayList(
+ RowFactory.create(1, "x", null),
+ RowFactory.create(2, "y", null),
+ RowFactory.create(3, "z", null),
+ RowFactory.create(4, "xy", "B"),
+ RowFactory.create(5, "xyz", "C"));
+ Assert.assertEquals("Records should match", expectedRecords2,
actualRecords2);
+
+ Dataset<Row> resultDf3 = spark.read()
+ .format("iceberg")
+ .option(SparkReadOptions.SNAPSHOT_ID, snapshot1.snapshotId())
+ .load(loadLocation(tableIdentifier));
+ List<Row> actualRecords3 = resultDf3.orderBy("id")
+ .collectAsList();
+
+ Assert.assertEquals("Records should match", expectedRecords,
actualRecords3);
+ }
+
+ @Test
+ public synchronized void testSnapshotReadAfterDropColumn() {
+ TableIdentifier tableIdentifier = TableIdentifier.of("db", "table");
+ Table table = createTable(tableIdentifier, SCHEMA2,
PartitionSpec.unpartitioned());
+
+ List<Row> expectedRecords = Lists.newArrayList(
+ RowFactory.create(1, "x", "A"),
+ RowFactory.create(2, "y", "A"),
+ RowFactory.create(3, "z", "B"));
+
+ Dataset<Row> inputDf = spark.createDataFrame(expectedRecords,
SparkSchemaUtil.convert(SCHEMA2));
+ inputDf.select("id", "data", "category").write()
+ .format("iceberg")
+ .mode(SaveMode.Append)
+ .save(loadLocation(tableIdentifier));
+
+ table.refresh();
+
+ Dataset<Row> resultDf = spark.read()
+ .format("iceberg")
+ .load(loadLocation(tableIdentifier));
+ List<Row> actualRecords = resultDf.orderBy("id")
+ .collectAsList();
+
+ Assert.assertEquals("Records should match", expectedRecords,
actualRecords);
+
+ long ts1 = System.currentTimeMillis();
Review comment:
This is likely to make the tests flaky because of different behavior
with time. To avoid the problem, we added the method
`SparkTestBase.waitUntilAfter` that accepts a current timestamp and waits until
the timestamp returned by `currentTimeMillis` is strictly after the given
timestamp. To fix this test, just change these lines to:
```java
long ts1 = waitUntilAfter(System.currentTimeMillis());
```
I also recommend using more readable names, like `beforeDropColumnMs`.
--
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]