Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

refine error message in mpptask #5304

Merged
merged 11 commits into from
Jul 7, 2022
2 changes: 1 addition & 1 deletion dbms/src/Flash/EstablishCall.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -143,7 +143,7 @@ void EstablishCallData::finishTunnelAndResponder()
state = FINISH;
if (mpp_tunnel)
{
mpp_tunnel->consumerFinish("grpc writes failed.", true); //trigger mpp tunnel finish work
mpp_tunnel->consumerFinish(fmt::format("{}: finishTunnelAndResponder called.", mpp_tunnel->id()), true); //trigger mpp tunnel finish work
}
grpc::Status status(static_cast<grpc::StatusCode>(GRPC_STATUS_UNKNOWN), "Consumer exits unexpected, grpc writes failed.");
responder.Finish(status, this);
Expand Down
4 changes: 3 additions & 1 deletion dbms/src/Flash/Mpp/MPPTask.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -379,7 +379,7 @@ void MPPTask::runImpl()
}
catch (...)
{
err_msg = getCurrentExceptionMessage(true);
err_msg = getCurrentExceptionMessage(true, true);
}

if (err_msg.empty())
Expand All @@ -405,6 +405,8 @@ void MPPTask::runImpl()
if (status == RUNNING)
{
LOG_FMT_ERROR(log, "task running meets error: {}", err_msg);
/// trim the stack trace to avoid too many useless information in log
trimStackTrace(err_msg);
try
{
handleError(err_msg);
Expand Down
4 changes: 4 additions & 0 deletions dbms/src/Flash/Mpp/MPPTunnel.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -220,7 +220,11 @@ void MPPTunnelBase<Writer>::sendJob(bool need_lock)
err_msg = "fatal error in sendJob()";
}
if (!err_msg.empty())
{
/// append tunnel id to error message
err_msg = fmt::format("{} meet error: {}", tunnel_id, err_msg);
LOG_ERROR(log, err_msg);
}
consumerFinish(err_msg, need_lock);
if (is_async)
writer->writeDone(grpc::Status::OK);
Expand Down
11 changes: 11 additions & 0 deletions dbms/src/Flash/Mpp/Utils.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@
// limitations under the License.

#include <Flash/Mpp/Utils.h>
#include <Poco/String.h>

#include <memory>

Expand All @@ -27,4 +28,14 @@ mpp::MPPDataPacket getPacketWithError(String reason)
return data;
}

void trimStackTrace(String & message)
{
auto stack_trace_pos = message.find("Stack trace");
if (stack_trace_pos != String::npos)
{
message.resize(stack_trace_pos);
Poco::trimRightInPlace(message);
}
}

} // namespace DB
1 change: 1 addition & 0 deletions dbms/src/Flash/Mpp/Utils.h
Original file line number Diff line number Diff line change
Expand Up @@ -23,5 +23,6 @@
namespace DB
{
mpp::MPPDataPacket getPacketWithError(String reason);
void trimStackTrace(String & message);

} // namespace DB
4 changes: 2 additions & 2 deletions dbms/src/Flash/Mpp/tests/gtest_mpptunnel.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -382,7 +382,7 @@ TEST_F(TestMPPTunnelBase, WriteError)
}
catch (Exception & e)
{
GTEST_ASSERT_EQ(e.message(), "Consumer exits unexpected, grpc writes failed.");
GTEST_ASSERT_EQ(e.message(), "Consumer exits unexpected, 0000_0001 meet error: grpc writes failed.");
}
}

Expand Down Expand Up @@ -631,7 +631,7 @@ TEST_F(TestMPPTunnelBase, AsyncWriteError)
}
catch (Exception & e)
{
GTEST_ASSERT_EQ(e.message(), "Consumer exits unexpected, grpc writes failed.");
GTEST_ASSERT_EQ(e.message(), "Consumer exits unexpected, 0000_0001 meet error: grpc writes failed.");
}
}

Expand Down
10 changes: 1 addition & 9 deletions tests/fullstack-test/mpp/issue_2471.test
Original file line number Diff line number Diff line change
Expand Up @@ -35,15 +35,7 @@ mysql> use test; set @@tidb_isolation_read_engines='tiflash'; set @@tidb_opt_bro
=> DBGInvoke __enable_fail_point(exception_in_creating_set_input_stream)

mysql> use test; set @@tidb_isolation_read_engines='tiflash'; set @@tidb_opt_broadcast_cartesian_join=2; select * from a as t1 left join a as t2 on t1.id = t2.id;
ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 10007, e.displayText() = DB::Exception: Fail point FailPoints::exception_in_creating_set_input_stream is triggered., e.what() = DB::Exception, Stack trace:
{#LINE}
{#LINE}
{#LINE}
{#LINE}
{#LINE}
{#LINE}
{#LINE}
{#LINE}
ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 10007, e.displayText() = DB::Exception: Fail point FailPoints::exception_in_creating_set_input_stream is triggered., e.what() = DB::Exception,

=> DBGInvoke __disable_fail_point(exception_in_creating_set_input_stream)

Expand Down
50 changes: 5 additions & 45 deletions tests/fullstack-test/mpp/mpp_fail.test
Original file line number Diff line number Diff line change
Expand Up @@ -71,59 +71,28 @@ ERROR 1105 (HY000) at line 1: DB::Exception: Fail point FailPoints::exception_be
## exception during mpp run non root task
=> DBGInvoke __enable_fail_point(exception_during_mpp_non_root_task_run)
mysql> use test; set @@tidb_isolation_read_engines='tiflash'; set @@tidb_allow_mpp=1; select count(value), id from t group by id;
ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 0, e.displayText() = DB::Exception: Exchange receiver meet error : Code: 10007, e.displayText() = DB::Exception: Fail point FailPoints::exception_during_mpp_non_root_task_run is triggered., e.what() = DB::Exception, Stack trace:
{#LINE}
{#LINE}
{#LINE}
{#LINE}
{#LINE}
{#LINE}
{#LINE}
ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 0, e.displayText() = DB::Exception: Exchange receiver meet error : Code: 10007, e.displayText() = DB::Exception: Fail point FailPoints::exception_during_mpp_non_root_task_run is triggered., e.what() = DB::Exception,, e.what() = DB::Exception,
=> DBGInvoke __disable_fail_point(exception_during_mpp_non_root_task_run)

## exception during mpp run root task
=> DBGInvoke __enable_fail_point(exception_during_mpp_root_task_run)
mysql> use test; set @@tidb_isolation_read_engines='tiflash'; set @@tidb_allow_mpp=1; select count(value), id from t group by id;
ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 10007, e.displayText() = DB::Exception: Fail point FailPoints::exception_during_mpp_root_task_run is triggered., e.what() = DB::Exception, Stack trace:
{#LINE}
{#LINE}
{#LINE}
{#LINE}
{#LINE}
{#LINE}
{#LINE}
{#LINE}
ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 10007, e.displayText() = DB::Exception: Fail point FailPoints::exception_during_mpp_root_task_run is triggered., e.what() = DB::Exception,
=> DBGInvoke __disable_fail_point(exception_during_mpp_root_task_run)

## exception during mpp write err to tunnel
=> DBGInvoke __enable_fail_point(exception_during_mpp_non_root_task_run)
=> DBGInvoke __enable_fail_point(exception_during_mpp_write_err_to_tunnel)
mysql> use test; set @@tidb_isolation_read_engines='tiflash'; set @@tidb_allow_mpp=1; select count(value), id from t group by id;
ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 0, e.displayText() = DB::Exception: Exchange receiver meet error : Failed to write error msg to tunnel, e.what() = DB::Exception, Stack trace:
{#LINE}
{#LINE}
{#LINE}
{#LINE}
{#LINE}
{#LINE}
{#LINE}
{#LINE}
{#LINE}
ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 0, e.displayText() = DB::Exception: Exchange receiver meet error : Failed to write error msg to tunnel, e.what() = DB::Exception,
=> DBGInvoke __disable_fail_point(exception_during_mpp_non_root_task_run)
=> DBGInvoke __disable_fail_point(exception_during_mpp_write_err_to_tunnel)

## exception during mpp write close tunnel
=> DBGInvoke __enable_fail_point(exception_during_mpp_non_root_task_run)
=> DBGInvoke __enable_fail_point(exception_during_mpp_close_tunnel)
mysql> use test; set @@tidb_isolation_read_engines='tiflash'; set @@tidb_allow_mpp=1; select count(value), id from t group by id;
ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 0, e.displayText() = DB::Exception: Exchange receiver meet error : Code: 10007, e.displayText() = DB::Exception: Fail point FailPoints::exception_during_mpp_non_root_task_run is triggered., e.what() = DB::Exception, Stack trace:
{#LINE}
{#LINE}
{#LINE}
{#LINE}
{#LINE}
{#LINE}
{#LINE}
ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 0, e.displayText() = DB::Exception: Exchange receiver meet error : Code: 10007, e.displayText() = DB::Exception: Fail point FailPoints::exception_during_mpp_non_root_task_run is triggered., e.what() = DB::Exception,, e.what() = DB::Exception,
=> DBGInvoke __disable_fail_point(exception_during_mpp_non_root_task_run)
=> DBGInvoke __disable_fail_point(exception_during_mpp_close_tunnel)

Expand Down Expand Up @@ -156,16 +125,7 @@ ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 0, e.displayText
## ensure build1, build2-probe1, probe2 in the CreatingSets, test the bug where build1 throw exception but not change the build state, thus block the build2-probe1, at last this query hangs.
=> DBGInvoke __enable_fail_point(exception_mpp_hash_build)
mysql> use test; set @@tidb_isolation_read_engines='tiflash'; set @@tidb_allow_mpp=1; set @@tidb_broadcast_join_threshold_count=0; set @@tidb_broadcast_join_threshold_size=0; select t1.id from test.t t1 join test.t t2 on t1.id = t2.id and t1.id <2 join (select id from test.t group by id) t3 on t2.id=t3.id;
ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 10007, e.displayText() = DB::Exception: Fail point FailPoints::exception_mpp_hash_build is triggered., e.what() = DB::Exception, Stack trace:
{#LINE}
{#LINE}
{#LINE}
{#LINE}
{#LINE}
{#LINE}
{#LINE}
{#LINE}
{#LINE}
ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 10007, e.displayText() = DB::Exception: Fail point FailPoints::exception_mpp_hash_build is triggered., e.what() = DB::Exception,
=> DBGInvoke __disable_fail_point(exception_mpp_hash_build)

# Clean up.
Expand Down