Skip to content

Commit

Permalink
Revert "[Iceberg] cleanup FileIO resources (apache#33509)"
Browse files Browse the repository at this point in the history
This reverts commit 5a3ddc4.
  • Loading branch information
Claude committed Jan 10, 2025
1 parent 0c9a46b commit bad9257
Show file tree
Hide file tree
Showing 3 changed files with 7 additions and 11 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": 1
"modification": 4
}
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@
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 @@ -186,10 +187,8 @@ 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;
try (FileIO io = table.io()) {
writer = createManifestWriter(table.location(), uuid, spec, io);
}
ManifestWriter<DataFile> writer =
createManifestWriter(table.location(), uuid, spec, table.io());
for (DataFile file : files) {
writer.add(file);
committedDataFileByteSize.update(file.fileSizeInBytes());
Expand All @@ -208,7 +207,8 @@ private ManifestWriter<DataFile> createManifestWriter(
String.format(
"%s/metadata/%s-%s-%s.manifest",
tableLocation, manifestFilePrefix, uuid, spec.specId()));
return ManifestFiles.write(spec, io.newOutputFile(location));
OutputFile outputFile = io.newOutputFile(location);
return ManifestFiles.write(spec, outputFile);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,6 @@
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 @@ -67,10 +66,7 @@ class RecordWriter {
fileFormat.addExtension(
table.locationProvider().newDataLocation(table.spec(), partitionKey, filename));
}
OutputFile outputFile;
try (FileIO io = table.io()) {
outputFile = io.newOutputFile(absoluteFilename);
}
OutputFile outputFile = table.io().newOutputFile(absoluteFilename);

switch (fileFormat) {
case AVRO:
Expand Down

0 comments on commit bad9257

Please sign in to comment.