@@ -666,7 +666,7 @@ Status VTabletWriterV2::close(Status exec_status) {
666666 // close_wait on all non-incremental streams, even if this is not the last sink.
667667 // because some per-instance data structures are now shared among all sinks
668668 // due to sharing delta writers and load stream stubs.
669- RETURN_IF_ERROR (_close_wait (false ));
669+ RETURN_IF_ERROR (_close_wait (_non_incremental_streams () ));
670670
671671 // send CLOSE_LOAD on all incremental streams if this is the last sink.
672672 // this must happen after all non-incremental streams are closed,
@@ -676,7 +676,7 @@ Status VTabletWriterV2::close(Status exec_status) {
676676 }
677677
678678 // close_wait on all incremental streams, even if this is not the last sink.
679- RETURN_IF_ERROR (_close_wait (true ));
679+ RETURN_IF_ERROR (_close_wait (_incremental_streams () ));
680680
681681 // calculate and submit commit info
682682 if (is_last_sink) {
@@ -721,32 +721,87 @@ Status VTabletWriterV2::close(Status exec_status) {
721721 return status;
722722}
723723
724- Status VTabletWriterV2::_close_wait (bool incremental) {
724+ std::unordered_set<std::shared_ptr<LoadStreamStub>> VTabletWriterV2::_incremental_streams () {
725+ std::unordered_set<std::shared_ptr<LoadStreamStub>> incremental_streams;
726+ auto streams_for_node = _load_stream_map->get_streams_for_node ();
727+ for (const auto & [dst_id, streams] : streams_for_node) {
728+ for (const auto & stream : streams->streams ()) {
729+ if (stream->is_incremental ()) {
730+ incremental_streams.insert (stream);
731+ }
732+ }
733+ }
734+ return incremental_streams;
735+ }
736+
737+ std::unordered_set<std::shared_ptr<LoadStreamStub>> VTabletWriterV2::_non_incremental_streams () {
738+ std::unordered_set<std::shared_ptr<LoadStreamStub>> non_incremental_streams;
739+ auto streams_for_node = _load_stream_map->get_streams_for_node ();
740+ for (const auto & [dst_id, streams] : streams_for_node) {
741+ for (const auto & stream : streams->streams ()) {
742+ if (!stream->is_incremental ()) {
743+ non_incremental_streams.insert (stream);
744+ }
745+ }
746+ }
747+ return non_incremental_streams;
748+ }
749+
750+ Status VTabletWriterV2::_close_wait (
751+ std::unordered_set<std::shared_ptr<LoadStreamStub>> unfinished_streams) {
725752 SCOPED_TIMER (_close_load_timer);
726- auto st = _load_stream_map->for_each_st (
727- [this , incremental](int64_t dst_id, LoadStreamStubs& streams) -> Status {
728- if (streams.is_incremental () != incremental) {
729- return Status::OK ();
730- }
731- int64_t remain_ms = static_cast <int64_t >(_state->execution_timeout ()) * 1000 -
732- _timeout_watch.elapsed_time () / 1000 / 1000 ;
733- DBUG_EXECUTE_IF (" VTabletWriterV2._close_wait.load_timeout" , { remain_ms = 0 ; });
734- if (remain_ms <= 0 ) {
735- LOG (WARNING) << " load timed out before close waiting, load_id="
736- << print_id (_load_id);
737- return Status::TimedOut (" load timed out before close waiting" );
738- }
739- auto st = streams.close_wait (_state, remain_ms);
740- if (!st.ok ()) {
741- LOG (WARNING) << " close_wait timeout on streams to dst_id=" << dst_id
742- << " , load_id=" << print_id (_load_id) << " : " << st;
743- }
744- return st;
745- });
746- if (!st.ok ()) {
747- LOG (WARNING) << " close_wait failed: " << st << " , load_id=" << print_id (_load_id);
753+ Status status;
754+ auto streams_for_node = _load_stream_map->get_streams_for_node ();
755+ while (true ) {
756+ RETURN_IF_ERROR (_check_timeout ());
757+ RETURN_IF_ERROR (_check_streams_finish (unfinished_streams, status, streams_for_node));
758+ if (!status.ok () || unfinished_streams.empty ()) {
759+ LOG (INFO) << " is all unfinished: " << unfinished_streams.empty ()
760+ << " , status: " << status << " , txn_id: " << _txn_id
761+ << " , load_id: " << print_id (_load_id);
762+ break ;
763+ }
764+ bthread_usleep (1000 * 10 );
748765 }
749- return st;
766+ if (!status.ok ()) {
767+ LOG (WARNING) << " close_wait failed: " << status << " , load_id=" << print_id (_load_id);
768+ }
769+ return status;
770+ }
771+
772+ Status VTabletWriterV2::_check_timeout () {
773+ int64_t remain_ms = static_cast <int64_t >(_state->execution_timeout ()) * 1000 -
774+ _timeout_watch.elapsed_time () / 1000 / 1000 ;
775+ DBUG_EXECUTE_IF (" VTabletWriterV2._close_wait.load_timeout" , { remain_ms = 0 ; });
776+ if (remain_ms <= 0 ) {
777+ LOG (WARNING) << " load timed out before close waiting, load_id=" << print_id (_load_id);
778+ return Status::TimedOut (" load timed out before close waiting" );
779+ }
780+ return Status::OK ();
781+ }
782+
783+ Status VTabletWriterV2::_check_streams_finish (
784+ std::unordered_set<std::shared_ptr<LoadStreamStub>>& unfinished_streams, Status& status,
785+ const std::unordered_map<int64_t , std::shared_ptr<LoadStreamStubs>>& streams_for_node) {
786+ for (const auto & [dst_id, streams] : streams_for_node) {
787+ for (const auto & stream : streams->streams ()) {
788+ if (!unfinished_streams.contains (stream)) {
789+ continue ;
790+ }
791+ bool is_closed = false ;
792+ auto stream_st = stream->close_finish_check (_state, &is_closed);
793+ if (!stream_st.ok ()) {
794+ status = stream_st;
795+ unfinished_streams.erase (stream);
796+ LOG (WARNING) << " close_wait failed: " << stream_st
797+ << " , load_id=" << print_id (_load_id);
798+ }
799+ if (is_closed) {
800+ unfinished_streams.erase (stream);
801+ }
802+ }
803+ }
804+ return status;
750805}
751806
752807void VTabletWriterV2::_calc_tablets_to_commit () {
0 commit comments