@@ -28,7 +28,6 @@ use mz_compute_client::protocol::command::PeekTarget;
2828use mz_compute_client:: protocol:: response:: PeekResponse ;
2929use mz_compute_types:: ComputeInstanceId ;
3030use mz_compute_types:: dataflows:: { DataflowDescription , IndexImport } ;
31- use mz_compute_types:: sinks:: ComputeSinkConnection ;
3231use mz_controller_types:: ClusterId ;
3332use mz_expr:: explain:: { HumanizedExplain , HumanizerMode , fmt_text_constant_rows} ;
3433use mz_expr:: row:: RowCollection ;
@@ -1255,12 +1254,14 @@ impl crate::coord::Coordinator {
12551254 . await
12561255 }
12571256
1258- /// Implements a COPY TO command by installing peek watch sets, validating S3 connection ,
1257+ /// Implements a ` COPY TO` command by installing peek watch sets,
12591258 /// shipping the dataflow, and spawning a background task to wait for completion.
12601259 /// This is called from the command handler for ExecuteCopyTo.
12611260 ///
1262- /// This method inlines the logic from peek_copy_to_preflight and peek_copy_to_dataflow
1263- /// to avoid the complexity of the staging mechanism for the frontend peek sequencing path.
1261+ /// (The S3 preflight check must be completed successfully via the
1262+ /// `CopyToPreflight` command _before_ calling this method. The preflight is
1263+ /// handled separately to avoid blocking the coordinator's main task with
1264+ /// slow S3 network operations.)
12641265 ///
12651266 /// This method does NOT block waiting for completion. Instead, it spawns a background task that
12661267 /// will send the response through the provided tx channel when the COPY TO completes.
@@ -1300,49 +1301,6 @@ impl crate::coord::Coordinator {
13001301
13011302 let sink_id = df_desc. sink_id ( ) ;
13021303
1303- // # Inlined from peek_copy_to_preflight
1304-
1305- let connection_context = self . connection_context ( ) . clone ( ) ;
1306- let sinks = & df_desc. sink_exports ;
1307-
1308- if sinks. len ( ) != 1 {
1309- send_err (
1310- tx,
1311- AdapterError :: Internal ( "expected exactly one copy to s3 sink" . into ( ) ) ,
1312- ) ;
1313- return ;
1314- }
1315- let ( sink_id_check, sink_desc) = sinks
1316- . first_key_value ( )
1317- . expect ( "known to be exactly one copy to s3 sink" ) ;
1318- assert_eq ! ( sink_id, * sink_id_check) ;
1319-
1320- match & sink_desc. connection {
1321- ComputeSinkConnection :: CopyToS3Oneshot ( conn) => {
1322- if let Err ( e) = mz_storage_types:: sinks:: s3_oneshot_sink:: preflight (
1323- connection_context,
1324- & conn. aws_connection ,
1325- & conn. upload_info ,
1326- conn. connection_id ,
1327- sink_id,
1328- )
1329- . await
1330- {
1331- send_err ( tx, e. into ( ) ) ;
1332- return ;
1333- }
1334- }
1335- _ => {
1336- send_err (
1337- tx,
1338- AdapterError :: Internal ( "expected copy to s3 oneshot sink" . into ( ) ) ,
1339- ) ;
1340- return ;
1341- }
1342- }
1343-
1344- // # Inlined from peek_copy_to_dataflow
1345-
13461304 // Create and register ActiveCopyTo.
13471305 // Note: sink_tx/sink_rx is the channel for the compute sink to notify completion
13481306 // This is different from the command's tx which sends the response to the client
0 commit comments