This is an automated email from the ASF dual-hosted git repository.
dataroaring pushed a commit to branch branch-3.0
in repository https://gitbox.apache.org/repos/asf/doris.git
The following commit(s) were added to refs/heads/branch-3.0 by this push:
new b4b6e9ef98a branch-3.0: [fix](tablet io error) fix tablet not increase
_io_error_times in some cases. (#42955)
b4b6e9ef98a is described below
commit b4b6e9ef98a01a6753f6cc0592e6af8384791aa8
Author: github-actions[bot]
<41898282+github-actions[bot]@users.noreply.github.com>
AuthorDate: Thu Nov 7 14:09:38 2024 +0800
branch-3.0: [fix](tablet io error) fix tablet not increase _io_error_times
in some cases. (#42955)
PR Body: When read/write a tablet, if meet io error, it will increase
this tablet's field `_io_error_times`. And if `_io_error_times` >=
config::max_tablet_io_errors, then be will report this tablet as bad,
then fe can drop it later.
But there are a lot of code that may met an io error! And they may
forget to increase this tablet's _io_error_times.
This PR only fix some, there should be others not fixed.
What's more, config::max_tablet_io_errors of be.conf default value is
-1. It means always no set tablet as bad. For an online cluster, it
shoud manually set this parameter > 0 in order to make it work.
Cherry-picked from #42208
Co-authored-by: yujun <[email protected]>
---
be/src/io/fs/local_file_reader.cpp | 11 ++
be/src/io/fs/local_file_system.cpp | 17 +++-
be/src/io/fs/local_file_writer.cpp | 9 ++
be/src/olap/rowset/beta_rowset.cpp | 9 +-
be/src/olap/rowset/beta_rowset_writer.cpp | 3 +-
be/src/olap/rowset/segment_v2/segment.cpp | 24 ++++-
be/src/olap/rowset/segment_v2/segment.h | 8 +-
be/src/olap/tablet.cpp | 13 +++
be/src/olap/tablet.h | 8 +-
be/src/runtime/exec_env.cpp | 5 +-
be/src/runtime/load_stream.cpp | 11 ++
be/test/olap/delete_bitmap_calculator_test.cpp | 3 +-
docker/runtime/doris-compose/command.py | 16 +++
.../doris/regression/suite/SuiteCluster.groovy | 5 +
.../suites/control_p0/test_tablet_io_error.groovy | 111 +++++++++++++++++++++
15 files changed, 234 insertions(+), 19 deletions(-)
diff --git a/be/src/io/fs/local_file_reader.cpp
b/be/src/io/fs/local_file_reader.cpp
index b4f144a6330..4a41fa479d9 100644
--- a/be/src/io/fs/local_file_reader.cpp
+++ b/be/src/io/fs/local_file_reader.cpp
@@ -34,11 +34,13 @@
#include "common/compiler_util.h" // IWYU pragma: keep
#include "cpp/sync_point.h"
#include "io/fs/err_utils.h"
+#include "olap/data_dir.h"
#include "olap/olap_common.h"
#include "olap/options.h"
#include "runtime/thread_context.h"
#include "runtime/workload_management/io_throttle.h"
#include "util/async_io.h"
+#include "util/debug_points.h"
#include "util/doris_metrics.h"
namespace doris {
@@ -139,6 +141,15 @@ Status LocalFileReader::read_at_impl(size_t offset, Slice
result, size_t* bytes_
while (bytes_req != 0) {
auto res = SYNC_POINT_HOOK_RETURN_VALUE(::pread(_fd, to, bytes_req,
offset),
"LocalFileReader::pread", _fd,
to);
+ DBUG_EXECUTE_IF("LocalFileReader::read_at_impl.io_error", {
+ auto sub_path = dp->param<std::string>("sub_path", "");
+ if ((sub_path.empty() && _path.filename().compare(kTestFilePath))
||
+ (!sub_path.empty() && _path.native().find(sub_path) !=
std::string::npos)) {
+ res = -1;
+ errno = EIO;
+ LOG(WARNING) << Status::IOError("debug read io error: {}",
_path.native());
+ }
+ });
if (UNLIKELY(-1 == res && errno != EINTR)) {
return localfs_error(errno, fmt::format("failed to read {}",
_path.native()));
}
diff --git a/be/src/io/fs/local_file_system.cpp
b/be/src/io/fs/local_file_system.cpp
index 4b44027abbb..0107ed57dc8 100644
--- a/be/src/io/fs/local_file_system.cpp
+++ b/be/src/io/fs/local_file_system.cpp
@@ -62,9 +62,13 @@ Status LocalFileSystem::create_file_impl(const Path& file,
FileWriterPtr* writer
int fd = ::open(file.c_str(), O_TRUNC | O_WRONLY | O_CREAT | O_CLOEXEC,
0666);
DBUG_EXECUTE_IF("LocalFileSystem.create_file_impl.open_file_failed", {
// spare '.testfile' to make bad disk checker happy
- if (file.filename().compare(kTestFilePath)) {
+ auto sub_path = dp->param<std::string>("sub_path", "");
+ if ((sub_path.empty() && file.filename().compare(kTestFilePath)) ||
+ (!sub_path.empty() && file.native().find(sub_path) !=
std::string::npos)) {
::close(fd);
fd = -1;
+ errno = EIO;
+ LOG(WARNING) << Status::IOError("debug open io error: {}",
file.native());
}
});
if (-1 == fd) {
@@ -85,6 +89,17 @@ Status LocalFileSystem::open_file_impl(const Path& file,
FileReaderSPtr* reader,
}
int fd = -1;
RETRY_ON_EINTR(fd, open(file.c_str(), O_RDONLY));
+ DBUG_EXECUTE_IF("LocalFileSystem.create_file_impl.open_file_failed", {
+ // spare '.testfile' to make bad disk checker happy
+ auto sub_path = dp->param<std::string>("sub_path", "");
+ if ((sub_path.empty() && file.filename().compare(kTestFilePath)) ||
+ (!sub_path.empty() && file.native().find(sub_path) !=
std::string::npos)) {
+ ::close(fd);
+ fd = -1;
+ errno = EIO;
+ LOG(WARNING) << Status::IOError("debug open io error: {}",
file.native());
+ }
+ });
if (fd < 0) {
return localfs_error(errno, fmt::format("failed to open {}",
file.native()));
}
diff --git a/be/src/io/fs/local_file_writer.cpp
b/be/src/io/fs/local_file_writer.cpp
index 7301ceae588..c65dee2535e 100644
--- a/be/src/io/fs/local_file_writer.cpp
+++ b/be/src/io/fs/local_file_writer.cpp
@@ -147,6 +147,15 @@ Status LocalFileWriter::appendv(const Slice* data, size_t
data_cnt) {
RETRY_ON_EINTR(res, SYNC_POINT_HOOK_RETURN_VALUE(
::writev(_fd, iov.data() + completed_iov,
iov_count),
"LocalFileWriter::writev", _fd));
+ DBUG_EXECUTE_IF("LocalFileWriter::appendv.io_error", {
+ auto sub_path = dp->param<std::string>("sub_path", "");
+ if ((sub_path.empty() && _path.filename().compare(kTestFilePath))
||
+ (!sub_path.empty() && _path.native().find(sub_path) !=
std::string::npos)) {
+ res = -1;
+ errno = EIO;
+ LOG(WARNING) << Status::IOError("debug write io error: {}",
_path.native());
+ }
+ });
if (UNLIKELY(res < 0)) {
return localfs_error(errno, fmt::format("failed to write {}",
_path.native()));
}
diff --git a/be/src/olap/rowset/beta_rowset.cpp
b/be/src/olap/rowset/beta_rowset.cpp
index 209aca7fb03..ee1605a3043 100644
--- a/be/src/olap/rowset/beta_rowset.cpp
+++ b/be/src/olap/rowset/beta_rowset.cpp
@@ -183,8 +183,9 @@ Status BetaRowset::load_segment(int64_t seg_id,
segment_v2::SegmentSharedPtr* se
.file_size = _rowset_meta->segment_file_size(seg_id),
};
- auto s = segment_v2::Segment::open(fs, seg_path, seg_id, rowset_id(),
_schema, reader_options,
- segment,
_rowset_meta->inverted_index_file_info(seg_id));
+ auto s = segment_v2::Segment::open(fs, seg_path,
_rowset_meta->tablet_id(), seg_id, rowset_id(),
+ _schema, reader_options, segment,
+
_rowset_meta->inverted_index_file_info(seg_id));
if (!s.ok()) {
LOG(WARNING) << "failed to open segment. " << seg_path << " under
rowset " << rowset_id()
<< " : " << s.to_string();
@@ -543,8 +544,8 @@ Status BetaRowset::check_current_rowset_segment() {
.file_size = _rowset_meta->segment_file_size(seg_id),
};
- auto s = segment_v2::Segment::open(fs, seg_path, seg_id, rowset_id(),
_schema,
- reader_options, &segment,
+ auto s = segment_v2::Segment::open(fs, seg_path,
_rowset_meta->tablet_id(), seg_id,
+ rowset_id(), _schema,
reader_options, &segment,
_rowset_meta->inverted_index_file_info(seg_id));
if (!s.ok()) {
LOG(WARNING) << "segment can not be opened. file=" << seg_path;
diff --git a/be/src/olap/rowset/beta_rowset_writer.cpp
b/be/src/olap/rowset/beta_rowset_writer.cpp
index 8b726b99c3b..ed659b601e8 100644
--- a/be/src/olap/rowset/beta_rowset_writer.cpp
+++ b/be/src/olap/rowset/beta_rowset_writer.cpp
@@ -307,7 +307,8 @@ Status
BetaRowsetWriter::_load_noncompacted_segment(segment_v2::SegmentSharedPtr
.is_doris_table = true,
.cache_base_path {},
};
- auto s = segment_v2::Segment::open(io::global_local_filesystem(), path,
segment_id, rowset_id(),
+ auto s = segment_v2::Segment::open(io::global_local_filesystem(), path,
+ _rowset_meta->tablet_id(), segment_id,
rowset_id(),
_context.tablet_schema, reader_options,
&segment);
if (!s.ok()) {
LOG(WARNING) << "failed to open segment. " << path << ":" << s;
diff --git a/be/src/olap/rowset/segment_v2/segment.cpp
b/be/src/olap/rowset/segment_v2/segment.cpp
index 06a458a75ca..2f43641b778 100644
--- a/be/src/olap/rowset/segment_v2/segment.cpp
+++ b/be/src/olap/rowset/segment_v2/segment.cpp
@@ -86,10 +86,30 @@ std::string file_cache_key_str(const std::string& seg_path)
{
return file_cache_key_from_path(seg_path).to_string();
}
-Status Segment::open(io::FileSystemSPtr fs, const std::string& path, uint32_t
segment_id,
- RowsetId rowset_id, TabletSchemaSPtr tablet_schema,
+Status Segment::open(io::FileSystemSPtr fs, const std::string& path, int64_t
tablet_id,
+ uint32_t segment_id, RowsetId rowset_id, TabletSchemaSPtr
tablet_schema,
const io::FileReaderOptions& reader_options,
std::shared_ptr<Segment>* output,
InvertedIndexFileInfo idx_file_info) {
+ auto s = _open(fs, path, segment_id, rowset_id, tablet_schema,
reader_options, output,
+ idx_file_info);
+ if (!s.ok()) {
+ if (!config::is_cloud_mode()) {
+ auto res = ExecEnv::get_tablet(tablet_id);
+ TabletSharedPtr tablet =
+ res.has_value() ?
std::dynamic_pointer_cast<Tablet>(res.value()) : nullptr;
+ if (tablet) {
+ tablet->report_error(s);
+ }
+ }
+ }
+
+ return s;
+}
+
+Status Segment::_open(io::FileSystemSPtr fs, const std::string& path, uint32_t
segment_id,
+ RowsetId rowset_id, TabletSchemaSPtr tablet_schema,
+ const io::FileReaderOptions& reader_options,
std::shared_ptr<Segment>* output,
+ InvertedIndexFileInfo idx_file_info) {
io::FileReaderSPtr file_reader;
RETURN_IF_ERROR(fs->open_file(path, &file_reader, &reader_options));
std::shared_ptr<Segment> segment(
diff --git a/be/src/olap/rowset/segment_v2/segment.h
b/be/src/olap/rowset/segment_v2/segment.h
index 322b5d8b238..103522e13bd 100644
--- a/be/src/olap/rowset/segment_v2/segment.h
+++ b/be/src/olap/rowset/segment_v2/segment.h
@@ -80,8 +80,8 @@ using SegmentSharedPtr = std::shared_ptr<Segment>;
// change finished, client should disable all cached Segment for old
TabletSchema.
class Segment : public std::enable_shared_from_this<Segment> {
public:
- static Status open(io::FileSystemSPtr fs, const std::string& path,
uint32_t segment_id,
- RowsetId rowset_id, TabletSchemaSPtr tablet_schema,
+ static Status open(io::FileSystemSPtr fs, const std::string& path, int64_t
tablet_id,
+ uint32_t segment_id, RowsetId rowset_id,
TabletSchemaSPtr tablet_schema,
const io::FileReaderOptions& reader_options,
std::shared_ptr<Segment>* output, InvertedIndexFileInfo
idx_file_info = {});
@@ -212,6 +212,10 @@ private:
DISALLOW_COPY_AND_ASSIGN(Segment);
Segment(uint32_t segment_id, RowsetId rowset_id, TabletSchemaSPtr
tablet_schema,
InvertedIndexFileInfo idx_file_info = InvertedIndexFileInfo());
+ static Status _open(io::FileSystemSPtr fs, const std::string& path,
uint32_t segment_id,
+ RowsetId rowset_id, TabletSchemaSPtr tablet_schema,
+ const io::FileReaderOptions& reader_options,
+ std::shared_ptr<Segment>* output,
InvertedIndexFileInfo idx_file_info);
// open segment file and read the minimum amount of necessary information
(footer)
Status _open();
Status _parse_footer(SegmentFooterPB* footer);
diff --git a/be/src/olap/tablet.cpp b/be/src/olap/tablet.cpp
index 30216e6013a..2f2acd8ae21 100644
--- a/be/src/olap/tablet.cpp
+++ b/be/src/olap/tablet.cpp
@@ -1652,6 +1652,19 @@ void Tablet::build_tablet_report_info(TTabletInfo*
tablet_info,
}
}
+void Tablet::report_error(const Status& st) {
+ if (st.is<ErrorCode::IO_ERROR>()) {
+ ++_io_error_times;
+ } else if (st.is<ErrorCode::CORRUPTION>()) {
+ _io_error_times = config::max_tablet_io_errors + 1;
+ } else if (st.is<ErrorCode::NOT_FOUND>()) {
+ check_tablet_path_exists();
+ if (!_is_tablet_path_exists.load(std::memory_order_relaxed)) {
+ _io_error_times = config::max_tablet_io_errors + 1;
+ }
+ }
+}
+
Status Tablet::prepare_compaction_and_calculate_permits(
CompactionType compaction_type, const TabletSharedPtr& tablet,
std::shared_ptr<CompactionMixin>& compaction, int64_t& permits) {
diff --git a/be/src/olap/tablet.h b/be/src/olap/tablet.h
index e88324af1b6..e1f6384fb51 100644
--- a/be/src/olap/tablet.h
+++ b/be/src/olap/tablet.h
@@ -446,13 +446,7 @@ public:
void gc_binlogs(int64_t version);
Status ingest_binlog_metas(RowsetBinlogMetasPB* metas_pb);
- inline void report_error(const Status& st) {
- if (st.is<ErrorCode::IO_ERROR>()) {
- ++_io_error_times;
- } else if (st.is<ErrorCode::CORRUPTION>()) {
- _io_error_times = config::max_tablet_io_errors + 1;
- }
- }
+ void report_error(const Status& st);
inline int64_t get_io_error_times() const { return _io_error_times; }
diff --git a/be/src/runtime/exec_env.cpp b/be/src/runtime/exec_env.cpp
index c714db2d5e4..e41cc982a74 100644
--- a/be/src/runtime/exec_env.cpp
+++ b/be/src/runtime/exec_env.cpp
@@ -54,7 +54,10 @@ void ExecEnv::set_write_cooldown_meta_executors() {
#endif // BE_TEST
Result<BaseTabletSPtr> ExecEnv::get_tablet(int64_t tablet_id) {
- return GetInstance()->storage_engine().get_tablet(tablet_id);
+ auto storage_engine = GetInstance()->_storage_engine.get();
+ return storage_engine != nullptr
+ ? storage_engine->get_tablet(tablet_id)
+ : ResultError(Status::InternalError("failed to get tablet
{}", tablet_id));
}
const std::string& ExecEnv::token() const {
diff --git a/be/src/runtime/load_stream.cpp b/be/src/runtime/load_stream.cpp
index 80cd167260c..88c64eb517c 100644
--- a/be/src/runtime/load_stream.cpp
+++ b/be/src/runtime/load_stream.cpp
@@ -31,11 +31,14 @@
#include <sstream>
#include "bvar/bvar.h"
+#include "cloud/config.h"
#include "common/signal_handler.h"
#include "exec/tablet_info.h"
#include "gutil/ref_counted.h"
+#include "olap/tablet.h"
#include "olap/tablet_fwd.h"
#include "olap/tablet_schema.h"
+#include "runtime/exec_env.h"
#include "runtime/fragment_mgr.h"
#include "runtime/load_channel.h"
#include "runtime/load_stream_mgr.h"
@@ -149,6 +152,14 @@ Status TabletStream::append_data(const PStreamHeader&
header, butil::IOBuf* data
signal::set_signal_task_id(_load_id);
g_load_stream_flush_running_threads << -1;
auto st = _load_stream_writer->append_data(new_segid, header.offset(),
buf, file_type);
+ if (!st.ok() && !config::is_cloud_mode()) {
+ auto res = ExecEnv::get_tablet(_id);
+ TabletSharedPtr tablet =
+ res.has_value() ?
std::dynamic_pointer_cast<Tablet>(res.value()) : nullptr;
+ if (tablet) {
+ tablet->report_error(st);
+ }
+ }
if (eos && st.ok()) {
DBUG_EXECUTE_IF("TabletStream.append_data.unknown_file_type",
{ file_type = static_cast<FileType>(-1); });
diff --git a/be/test/olap/delete_bitmap_calculator_test.cpp
b/be/test/olap/delete_bitmap_calculator_test.cpp
index 7e527078613..aec1b52087d 100644
--- a/be/test/olap/delete_bitmap_calculator_test.cpp
+++ b/be/test/olap/delete_bitmap_calculator_test.cpp
@@ -127,7 +127,8 @@ public:
EXPECT_NE("", writer.min_encoded_key().to_string());
EXPECT_NE("", writer.max_encoded_key().to_string());
- st = segment_v2::Segment::open(fs, path, segment_id, rowset_id,
query_schema,
+ int64_t tablet_id = 100;
+ st = segment_v2::Segment::open(fs, path, tablet_id, segment_id,
rowset_id, query_schema,
io::FileReaderOptions {}, res);
EXPECT_TRUE(st.ok());
EXPECT_EQ(nrows, (*res)->num_rows());
diff --git a/docker/runtime/doris-compose/command.py
b/docker/runtime/doris-compose/command.py
index 604eb8f206e..267c0de3ab0 100644
--- a/docker/runtime/doris-compose/command.py
+++ b/docker/runtime/doris-compose/command.py
@@ -211,6 +211,7 @@ class SimpleCommand(Command):
LOG.info(
utils.render_green("{} succ, total related node num {}".format(
show_cmd, related_node_num)))
+ return ""
if for_all:
related_nodes = cluster.get_all_nodes()
@@ -1255,6 +1256,20 @@ class GetCloudIniCommand(Command):
return self._handle_data(header, rows)
+class AddRWPermCommand(Command):
+
+ def add_parser(self, args_parsers):
+ parser = args_parsers.add_parser(
+ "add-rw-perm",
+ help="Add read and write permissions to the cluster files")
+ parser.add_argument("NAME", help="Specify cluster name.")
+ self._add_parser_common_args(parser)
+
+ def run(self, args):
+ utils.enable_dir_with_rw_perm(CLUSTER.get_cluster_path(args.NAME))
+ return ""
+
+
ALL_COMMANDS = [
UpCommand("up"),
DownCommand("down"),
@@ -1266,4 +1281,5 @@ ALL_COMMANDS = [
GetCloudIniCommand("get-cloud-ini"),
GenConfCommand("config"),
ListCommand("ls"),
+ AddRWPermCommand("add-rw-perm"),
]
diff --git
a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/SuiteCluster.groovy
b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/SuiteCluster.groovy
index 159e622f454..856b0e76956 100644
---
a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/SuiteCluster.groovy
+++
b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/SuiteCluster.groovy
@@ -615,6 +615,11 @@ class SuiteCluster {
}
}
+ void addRWPermToAllFiles() {
+ def cmd = 'add-rw-perm ' + name
+ runCmd(cmd)
+ }
+
private void waitHbChanged() {
// heart beat interval is 5s
Thread.sleep(7000)
diff --git a/regression-test/suites/control_p0/test_tablet_io_error.groovy
b/regression-test/suites/control_p0/test_tablet_io_error.groovy
new file mode 100644
index 00000000000..3ce3aa216f7
--- /dev/null
+++ b/regression-test/suites/control_p0/test_tablet_io_error.groovy
@@ -0,0 +1,111 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+import org.apache.doris.regression.suite.ClusterOptions
+
+import groovy.io.FileType
+
+suite('test_tablet_io_error', 'docker') {
+ def runTest = { isRead, debugPointName, isDropTabletDir ->
+ GetDebugPoint().clearDebugPointsForAllBEs()
+ def tbl = 'tbl_test_tablet_io_error'
+ sql "create table ${tbl} (k int) distributed by hash(k) buckets 1
properties('replication_num' = '2')"
+ sql "insert into ${tbl} values (1)"
+ sql "insert into ${tbl} values (2)"
+ sql "insert into ${tbl} values (3)"
+ def tablets = sql_return_maparray "SHOW TABLETS FROM ${tbl}"
+ assertEquals(2, tablets.size())
+ def tabletId = tablets[0].TabletId.toLong()
+ def injectBe = cluster.getBeByBackendId(tablets[0].BackendId.toLong())
+ assertNotNull(injectBe)
+
+ sql 'set use_fix_replica = 0'
+
+ def tabletOnInjectBe = sql_return_maparray("SHOW TABLETS FROM
${tbl}").find { it.BackendId.toLong() == injectBe.backendId }
+ assertNotNull(tabletOnInjectBe)
+
+ if (debugPointName != null) {
+ GetDebugPoint().enableDebugPoint(injectBe.host, injectBe.httpPort,
injectBe.getNodeType(),
+ debugPointName, [ sub_path : "/${tabletId}/" ])
+ }
+
+ if (isDropTabletDir) {
+ // the docker files owner is root, need change its permission
+ cluster.addRWPermToAllFiles()
+ def dataPath = new File("${injectBe.path}/storage/1.HDD/data")
+ dataPath.eachFile(FileType.DIRECTORIES) { shardPath ->
+ shardPath.eachFile(FileType.DIRECTORIES) { tabletPath ->
+ try {
+ if (tabletPath.getName().toLong() == tabletId) {
+ logger.info("delete tablet path: ${tabletPath}")
+ tabletPath.deleteDir()
+ }
+ } catch (Throwable t) {
+ logger.warn('delete tablet path exception: ', t)
+ }
+ }
+ }
+ }
+
+ boolean hasExcept = false
+ try {
+ if (isRead) {
+ sql "select * from ${tbl}"
+ } else {
+ sql "insert into ${tbl} values (1)"
+ }
+ } catch (Throwable e) {
+ logger.info("exec exeption: ${e.getMessage()}")
+ hasExcept = true
+ }
+ assertTrue(hasExcept)
+
+ sleep 8000
+
+ // be will report tablet as bad, then fe will drop it
+ tabletOnInjectBe = sql_return_maparray("SHOW TABLETS FROM
${tbl}").find { it.BackendId.toLong() == injectBe.backendId }
+ assertNull(tabletOnInjectBe)
+ sql "insert into ${tbl} values (1)"
+ sql "select * from ${tbl}"
+
+ sql "drop table ${tbl} force"
+ }
+
+ def options = new ClusterOptions()
+ options.cloudMode = false
+ options.enableDebugPoints()
+ options.feConfigs += [
+ 'disable_balance=true',
+ 'tablet_checker_interval_ms=500',
+ 'schedule_batch_size=1000',
+ 'schedule_slot_num_per_hdd_path=1000',
+ ]
+ options.beConfigs += [
+ 'report_tablet_interval_seconds=1',
+ 'max_tablet_io_errors=1',
+ 'disable_page_cache=true',
+ ]
+
+ docker(options) {
+ runTest(true, 'LocalFileReader::read_at_impl.io_error', false)
+ runTest(true, 'LocalFileSystem.create_file_impl.open_file_failed',
false)
+ runTest(true, null, true)
+ runTest(false, 'LocalFileWriter::appendv.io_error', false)
+ runTest(false, 'LocalFileSystem.create_file_impl.open_file_failed',
false)
+ runTest(false, null, true)
+ }
+}
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]