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]

Reply via email to