2323import static org .apache .beam .sdk .util .Preconditions .checkStateNotNull ;
2424import static org .apache .beam .sdk .values .PCollection .IsBounded .BOUNDED ;
2525
26+ import java .io .FileNotFoundException ;
2627import java .io .IOException ;
2728import java .nio .ByteBuffer ;
2829import java .nio .channels .SeekableByteChannel ;
8182import org .apache .iceberg .catalog .TableIdentifier ;
8283import org .apache .iceberg .data .Record ;
8384import org .apache .iceberg .exceptions .AlreadyExistsException ;
85+ import org .apache .iceberg .exceptions .NoSuchTableException ;
8486import org .apache .iceberg .io .CloseableIterable ;
8587import org .apache .iceberg .io .InputFile ;
8688import org .apache .iceberg .mapping .MappingUtil ;
@@ -240,7 +242,18 @@ public void process(@Element String filePath, MultiOutputReceiver output)
240242 }
241243
242244 if (table == null ) {
243- table = getOrCreateTable (getSchema (filePath , format ));
245+ try {
246+ table = getOrCreateTable (filePath , format );
247+ } catch (FileNotFoundException e ) {
248+ output
249+ .get (ERRORS )
250+ .output (
251+ Row .withSchema (ERROR_SCHEMA )
252+ .addValues (filePath , checkStateNotNull (e .getMessage ()))
253+ .build ());
254+ numErrorFiles .inc ();
255+ return ;
256+ }
244257 }
245258
246259 // Check if the file path contains the provided prefix
@@ -256,9 +269,24 @@ public void process(@Element String filePath, MultiOutputReceiver output)
256269
257270 InputFile inputFile = table .io ().newInputFile (filePath );
258271
259- Metrics metrics =
260- getFileMetrics (
261- inputFile , format , MetricsConfig .forTable (table ), MappingUtil .create (table .schema ()));
272+ Metrics metrics ;
273+ try {
274+ metrics =
275+ getFileMetrics (
276+ inputFile ,
277+ format ,
278+ MetricsConfig .forTable (table ),
279+ MappingUtil .create (table .schema ()));
280+ } catch (FileNotFoundException e ) {
281+ output
282+ .get (ERRORS )
283+ .output (
284+ Row .withSchema (ERROR_SCHEMA )
285+ .addValues (filePath , checkStateNotNull (e .getMessage ()))
286+ .build ());
287+ numErrorFiles .inc ();
288+ return ;
289+ }
262290
263291 // Figure out which partition this DataFile should go to
264292 String partitionPath ;
@@ -304,16 +332,23 @@ private static <W, T> T transformValue(Transform<W, T> transform, Type type, Obj
304332 return transform .bind (type ).apply ((W ) value );
305333 }
306334
307- private Table getOrCreateTable (org . apache . iceberg . Schema schema ) {
308- PartitionSpec spec = PartitionUtils . toPartitionSpec ( partitionFields , schema );
335+ private Table getOrCreateTable (String filePath , FileFormat format ) throws IOException {
336+ TableIdentifier tableId = TableIdentifier . parse ( identifier );
309337 try {
310- return tableProps == null
311- ? catalogConfig .catalog ().createTable (TableIdentifier .parse (identifier ), schema , spec )
312- : catalogConfig
313- .catalog ()
314- .createTable (TableIdentifier .parse (identifier ), schema , spec , tableProps );
315- } catch (AlreadyExistsException e ) { // if table already exists, just load it
316- return catalogConfig .catalog ().loadTable (TableIdentifier .parse (identifier ));
338+ return catalogConfig .catalog ().loadTable (tableId );
339+ } catch (NoSuchTableException e ) {
340+ try {
341+ org .apache .iceberg .Schema schema = getSchema (filePath , format );
342+ PartitionSpec spec = PartitionUtils .toPartitionSpec (partitionFields , schema );
343+
344+ return tableProps == null
345+ ? catalogConfig .catalog ().createTable (TableIdentifier .parse (identifier ), schema , spec )
346+ : catalogConfig
347+ .catalog ()
348+ .createTable (TableIdentifier .parse (identifier ), schema , spec , tableProps );
349+ } catch (AlreadyExistsException e2 ) { // if table already exists, just load it
350+ return catalogConfig .catalog ().loadTable (TableIdentifier .parse (identifier ));
351+ }
317352 }
318353 }
319354
0 commit comments