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 @@ -27,10 +27,18 @@
import java.nio.file.Files;
import java.util.List;
import java.util.Map;
import org.apache.avro.generic.GenericData;
import org.apache.iceberg.ManifestFile;
import org.apache.iceberg.ParameterizedTestExtension;
import org.apache.iceberg.Schema;
import org.apache.iceberg.Table;
import org.apache.iceberg.TableProperties;
import org.apache.iceberg.avro.Avro;
import org.apache.iceberg.avro.AvroIterable;
import org.apache.iceberg.avro.GenericAvroReader;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.types.Types;
import org.apache.spark.sql.AnalysisException;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.TestTemplate;
Expand Down Expand Up @@ -225,6 +233,14 @@ public void testInvalidSnapshotsCases() throws IOException {
.hasMessage("Cannot handle an empty identifier for argument table");
}

private static final Schema SNAPSHOT_ID_READ_SCHEMA =
new Schema(
Types.NestedField.required("snapshot_id")
.withId(1)
.ofType(Types.LongType.get())
.asOptional()
.build());

@TestTemplate
public void testSnapshotWithParallelism() throws IOException {
String location = Files.createTempDirectory(temp, "junit").toFile().toString();
Expand Down Expand Up @@ -280,4 +296,82 @@ public void testSnapshotPartitionedWithParallelism() throws IOException {
ImmutableList.of(row("a", 1L), row("b", 2L)),
sql("SELECT * FROM %s ORDER BY id", tableName));
}

@TestTemplate
public void testSnapshotPartitioned() throws IOException {
String location = Files.createTempDirectory(temp, "junit").toFile().toString();
sql(
"CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet PARTITIONED BY (id) LOCATION '%s'",
SOURCE_NAME, location);
sql("INSERT INTO TABLE %s (id, data) VALUES (1, 'a'), (2, 'b')", SOURCE_NAME);
assertThat(
sql(
"CALL %s.system.snapshot(source_table => '%s', table => '%s')",
catalogName, SOURCE_NAME, tableName))
.containsExactly(row(2L));
assertThat(sql("SELECT * FROM %s ORDER BY id", tableName))
.containsExactly(row("a", 1L), row("b", 2L));

assertEquals(
"Should have expected rows",
ImmutableList.of(row("a", 1L), row("b", 2L)),
sql("SELECT * FROM %s ORDER BY id", tableName));

Table createdTable = validationCatalog.loadTable(tableIdent);

for (ManifestFile manifest : createdTable.currentSnapshot().dataManifests(createdTable.io())) {
try (AvroIterable<GenericData.Record> reader =
Avro.read(org.apache.iceberg.Files.localInput(manifest.path()))
.project(SNAPSHOT_ID_READ_SCHEMA)
.createResolvingReader(GenericAvroReader::create)
.build()) {

assertThat(reader.getMetadata()).containsEntry("format-version", "2");

List<GenericData.Record> records = Lists.newArrayList(reader.iterator());
for (GenericData.Record row : records) {
assertThat(row.get(0)).as("Field-ID should be inherited").isNull();
}
}
}
}

@TestTemplate
public void testSnapshotPartitionedV1() throws IOException {
String location = Files.createTempDirectory(temp, "junit").toFile().toString();
sql(
"CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet PARTITIONED BY (id) LOCATION '%s'",
SOURCE_NAME, location);
sql("INSERT INTO TABLE %s (id, data) VALUES (1, 'a'), (2, 'b')", SOURCE_NAME);
assertThat(
sql(
"CALL %s.system.snapshot(source_table => '%s', table => '%s', properties => map('format-version', '1'))",
catalogName, SOURCE_NAME, tableName))
.containsExactly(row(2L));
assertThat(sql("SELECT * FROM %s ORDER BY id", tableName))
.containsExactly(row("a", 1L), row("b", 2L));

assertEquals(
"Should have expected rows",
ImmutableList.of(row("a", 1L), row("b", 2L)),
sql("SELECT * FROM %s ORDER BY id", tableName));

Table createdTable = validationCatalog.loadTable(tableIdent);

for (ManifestFile manifest : createdTable.currentSnapshot().dataManifests(createdTable.io())) {
try (AvroIterable<GenericData.Record> reader =
Avro.read(org.apache.iceberg.Files.localInput(manifest.path()))
.project(SNAPSHOT_ID_READ_SCHEMA)
.createResolvingReader(GenericAvroReader::create)
.build()) {

assertThat(reader.getMetadata()).containsEntry("format-version", "1");

List<GenericData.Record> records = Lists.newArrayList(reader.iterator());
for (GenericData.Record row : records) {
assertThat(row.get(0)).as("Field-ID should not be inherited").isNotNull();
}
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -362,6 +362,8 @@ public boolean isDefinedAt(Expression attr) {
}

private static Iterator<ManifestFile> buildManifest(
int formatVersion,
Long snapshotId,
SerializableConfiguration conf,
PartitionSpec spec,
String basePath,
Expand All @@ -379,7 +381,8 @@ private static Iterator<ManifestFile> buildManifest(
Path location = new Path(basePath, suffix);
String outputPath = FileFormat.AVRO.addExtension(location.toString());
OutputFile outputFile = io.newOutputFile(outputPath);
ManifestWriter<DataFile> writer = ManifestFiles.write(spec, outputFile);
ManifestWriter<DataFile> writer =
ManifestFiles.write(formatVersion, spec, outputFile, snapshotId);

try (ManifestWriter<DataFile> writerRef = writer) {
fileTuples.forEachRemaining(fileTuple -> writerRef.add(fileTuple._2));
Expand Down Expand Up @@ -867,6 +870,21 @@ public static void importSparkPartitions(
DUPLICATE_FILE_MESSAGE, Joiner.on(",").join((String[]) duplicates.take(10))));
}

TableOperations ops = ((HasTableOperations) targetTable).operations();
int formatVersion = ops.current().formatVersion();
boolean snapshotIdInheritanceEnabled =
PropertyUtil.propertyAsBoolean(
targetTable.properties(),
TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED,
TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED_DEFAULT);

final Long snapshotId;
if (formatVersion == 1 && !snapshotIdInheritanceEnabled) {
snapshotId = -1L;
} else {
snapshotId = null;
}

List<ManifestFile> manifests =
filesToImport
.repartition(numShufflePartitions)
Expand All @@ -877,19 +895,18 @@ public static void importSparkPartitions(
.orderBy(col("_1"))
.mapPartitions(
(MapPartitionsFunction<Tuple2<String, DataFile>, ManifestFile>)
fileTuple -> buildManifest(serializableConf, spec, stagingDir, fileTuple),
fileTuple ->
buildManifest(
formatVersion,
snapshotId,
serializableConf,
spec,
stagingDir,
fileTuple),
Encoders.javaSerialization(ManifestFile.class))
.collectAsList();

try {
TableOperations ops = ((HasTableOperations) targetTable).operations();
int formatVersion = ops.current().formatVersion();
boolean snapshotIdInheritanceEnabled =
PropertyUtil.propertyAsBoolean(
targetTable.properties(),
TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED,
TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED_DEFAULT);

AppendFiles append = targetTable.newAppend();
manifests.forEach(append::appendManifest);
append.commit();
Expand Down