6565import org .apache .iceberg .catalog .Catalog ;
6666import org .apache .iceberg .catalog .Namespace ;
6767import org .apache .iceberg .catalog .TableIdentifier ;
68+ import org .apache .iceberg .data .Record ;
6869import org .apache .iceberg .hadoop .HadoopCatalog ;
6970import org .apache .iceberg .io .FileIO ;
7071import org .apache .iceberg .io .InputFile ;
@@ -957,7 +958,7 @@ public void testDefaultMetrics() throws IOException {
957958 }
958959
959960 @ Test
960- public void testRecordWriterKeepsFileIOOpenUntilClose () throws IOException {
961+ public void testRecordWriterDoesNotCloseSharedFileIO () throws IOException {
961962 TableIdentifier tableId =
962963 TableIdentifier .of (
963964 "default" ,
@@ -980,7 +981,104 @@ public void testRecordWriterKeepsFileIOOpenUntilClose() throws IOException {
980981 writer .write (IcebergUtils .beamRowToIcebergRecord (ICEBERG_SCHEMA , row ));
981982 writer .close ();
982983
983- assertTrue ("FileIO should be closed after writer close" , trackingFileIO .closed );
984+ // RecordWriter must NOT close FileIO — it may be a shared instance.
985+ assertFalse ("RecordWriter.close() must not close the shared FileIO" , trackingFileIO .closed );
986+ }
987+
988+ /**
989+ * Verifies that when multiple writers share the same FileIO, closing any writer does not close
990+ * the shared FileIO — that is the responsibility of RecordWriterManager.close().
991+ */
992+ @ Test
993+ public void testMultipleWritersSharingFileIOSurviveBatchClose () throws IOException {
994+ // Create two tables that share the same FileIO (simulating dynamic destinations
995+ // backed by the same catalog)
996+ TableIdentifier tableId1 =
997+ TableIdentifier .of (
998+ "default" ,
999+ "table_batch_close_a_" + UUID .randomUUID ().toString ().replace ("-" , "" ).substring (0 , 6 ));
1000+ TableIdentifier tableId2 =
1001+ TableIdentifier .of (
1002+ "default" ,
1003+ "table_batch_close_b_" + UUID .randomUUID ().toString ().replace ("-" , "" ).substring (0 , 6 ));
1004+ Table table1 = warehouse .createTable (tableId1 , ICEBERG_SCHEMA );
1005+ Table table2 = warehouse .createTable (tableId2 , ICEBERG_SCHEMA );
1006+
1007+ // Both tables share the same CloseTrackingFileIO — mirrors how some catalogs
1008+ // return a shared FileIO instance across tables
1009+ CloseTrackingFileIO sharedFileIO = new CloseTrackingFileIO (table1 .io ());
1010+ Table spyTable1 = Mockito .spy (table1 );
1011+ Table spyTable2 = Mockito .spy (table2 );
1012+ Mockito .doReturn (sharedFileIO ).when (spyTable1 ).io ();
1013+ Mockito .doReturn (sharedFileIO ).when (spyTable2 ).io ();
1014+
1015+ PartitionKey pk1 = new PartitionKey (spyTable1 .spec (), spyTable1 .schema ());
1016+ PartitionKey pk2 = new PartitionKey (spyTable2 .spec (), spyTable2 .schema ());
1017+
1018+ RecordWriter writer1 = new RecordWriter (spyTable1 , FileFormat .PARQUET , "file1.parquet" , pk1 );
1019+ RecordWriter writer2 = new RecordWriter (spyTable2 , FileFormat .PARQUET , "file2.parquet" , pk2 );
1020+
1021+ Row row = Row .withSchema (BEAM_SCHEMA ).addValues (1 , "aaa" , true ).build ();
1022+ Record record = IcebergUtils .beamRowToIcebergRecord (ICEBERG_SCHEMA , row );
1023+
1024+ writer1 .write (record );
1025+ writer2 .write (record );
1026+
1027+ writer1 .close ();
1028+ assertFalse ("FileIO must remain open between batch writer closes" , sharedFileIO .closed );
1029+
1030+ writer2 .close ();
1031+ assertFalse ("FileIO must remain open after all writers close" , sharedFileIO .closed );
1032+
1033+ // Both writers produced valid data files
1034+ assertNotNull (writer1 .getDataFile ());
1035+ assertNotNull (writer2 .getDataFile ());
1036+ }
1037+
1038+ /**
1039+ * Verifies that RecordWriterManager.close() flushes data files from multiple destinations and
1040+ * closes the shared FileIO.
1041+ */
1042+ @ Test
1043+ public void testRecordWriterManagerClosesSharedFileIOAfterFlush () throws IOException {
1044+ String tableName1 =
1045+ "table_mgr_io_a_" + UUID .randomUUID ().toString ().replace ("-" , "" ).substring (0 , 6 );
1046+ String tableName2 =
1047+ "table_mgr_io_b_" + UUID .randomUUID ().toString ().replace ("-" , "" ).substring (0 , 6 );
1048+ TableIdentifier tableId1 = TableIdentifier .of ("default" , tableName1 );
1049+ TableIdentifier tableId2 = TableIdentifier .of ("default" , tableName2 );
1050+
1051+ Table realTable1 = warehouse .createTable (tableId1 , ICEBERG_SCHEMA );
1052+ Table realTable2 = warehouse .createTable (tableId2 , ICEBERG_SCHEMA );
1053+
1054+ CloseTrackingFileIO sharedTrackingIO = new CloseTrackingFileIO (realTable1 .io ());
1055+ Table spyTable1 = Mockito .spy (realTable1 );
1056+ Table spyTable2 = Mockito .spy (realTable2 );
1057+ Mockito .doReturn (sharedTrackingIO ).when (spyTable1 ).io ();
1058+ Mockito .doReturn (sharedTrackingIO ).when (spyTable2 ).io ();
1059+
1060+ Catalog spyCatalog = Mockito .spy (catalog );
1061+ Mockito .doReturn (spyTable1 ).when (spyCatalog ).loadTable (tableId1 );
1062+ Mockito .doReturn (spyTable2 ).when (spyCatalog ).loadTable (tableId2 );
1063+
1064+ WindowedValue <IcebergDestination > dest1 = getWindowedDestination (tableName1 , null );
1065+ WindowedValue <IcebergDestination > dest2 = getWindowedDestination (tableName2 , null );
1066+
1067+ RecordWriterManager writerManager =
1068+ new RecordWriterManager (spyCatalog , "test_file_name" , 1000 , 3 );
1069+
1070+ Row row = Row .withSchema (BEAM_SCHEMA ).addValues (1 , "aaa" , true ).build ();
1071+ assertTrue (writerManager .write (dest1 , row ));
1072+ assertTrue (writerManager .write (dest2 , row ));
1073+ assertEquals (2 , writerManager .openWriters );
1074+
1075+ writerManager .close ();
1076+
1077+ Map <WindowedValue <IcebergDestination >, List <SerializableDataFile >> dataFiles =
1078+ writerManager .getSerializableDataFiles ();
1079+ assertTrue ("Should have data files for dest1" , dataFiles .containsKey (dest1 ));
1080+ assertTrue ("Should have data files for dest2" , dataFiles .containsKey (dest2 ));
1081+ assertTrue ("Shared FileIO should be closed" , sharedTrackingIO .closed );
9841082 }
9851083
9861084 private static final class CloseTrackingFileIO implements FileIO {
0 commit comments