Skip to content

Commit

Permalink
[Iceberg] cleanup FileIO resources (#33509)
Browse files Browse the repository at this point in the history
* cleanup FileIO resources

* trigger integration tests

* cleanup
  • Loading branch information
ahmedabu98 authored Jan 8, 2025
1 parent b4c3a4f commit 5a3ddc4
Show file tree
Hide file tree
Showing 3 changed files with 11 additions and 7 deletions.
2 changes: 1 addition & 1 deletion .github/trigger_files/IO_Iceberg_Integration_Tests.json
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
{
"comment": "Modify this file in a trivial way to cause this test suite to run",
"modification": 4
"modification": 1
}
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,6 @@
import org.apache.iceberg.Table;
import org.apache.iceberg.catalog.Catalog;
import org.apache.iceberg.io.FileIO;
import org.apache.iceberg.io.OutputFile;
import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -187,8 +186,10 @@ private void appendManifestFiles(Table table, Iterable<FileWriteResult> fileWrit
int specId = entry.getKey();
List<DataFile> files = entry.getValue();
PartitionSpec spec = Preconditions.checkStateNotNull(specs.get(specId));
ManifestWriter<DataFile> writer =
createManifestWriter(table.location(), uuid, spec, table.io());
ManifestWriter<DataFile> writer;
try (FileIO io = table.io()) {
writer = createManifestWriter(table.location(), uuid, spec, io);
}
for (DataFile file : files) {
writer.add(file);
committedDataFileByteSize.update(file.fileSizeInBytes());
Expand All @@ -207,8 +208,7 @@ private ManifestWriter<DataFile> createManifestWriter(
String.format(
"%s/metadata/%s-%s-%s.manifest",
tableLocation, manifestFilePrefix, uuid, spec.specId()));
OutputFile outputFile = io.newOutputFile(location);
return ManifestFiles.write(spec, outputFile);
return ManifestFiles.write(spec, io.newOutputFile(location));
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import org.apache.iceberg.data.Record;
import org.apache.iceberg.data.parquet.GenericParquetWriter;
import org.apache.iceberg.io.DataWriter;
import org.apache.iceberg.io.FileIO;
import org.apache.iceberg.io.OutputFile;
import org.apache.iceberg.parquet.Parquet;
import org.slf4j.Logger;
Expand Down Expand Up @@ -66,7 +67,10 @@ class RecordWriter {
fileFormat.addExtension(
table.locationProvider().newDataLocation(table.spec(), partitionKey, filename));
}
OutputFile outputFile = table.io().newOutputFile(absoluteFilename);
OutputFile outputFile;
try (FileIO io = table.io()) {
outputFile = io.newOutputFile(absoluteFilename);
}

switch (fileFormat) {
case AVRO:
Expand Down

0 comments on commit 5a3ddc4

Please sign in to comment.