Status SnapshotManager::_create_snapshot_files()

in be/src/olap/snapshot_manager.cpp [398:797]


Status SnapshotManager::_create_snapshot_files(const TabletSharedPtr& ref_tablet,
                                               const TabletSharedPtr& target_tablet,
                                               const TSnapshotRequest& request,
                                               string* snapshot_path,
                                               bool* allow_incremental_clone) {
    int32_t snapshot_version = request.preferred_snapshot_version;
    LOG(INFO) << "receive a make snapshot request"
              << ", request detail is " << apache::thrift::ThriftDebugString(request)
              << ", snapshot_version is " << snapshot_version;
    Status res = Status::OK();
    if (snapshot_path == nullptr) {
        return Status::Error<INVALID_ARGUMENT>("output parameter cannot be null");
    }

    // snapshot_id_path:
    //      /data/shard_id/tablet_id/snapshot/time_str/id.timeout/
    int64_t timeout_s = config::snapshot_expire_time_sec;
    if (request.__isset.timeout) {
        timeout_s = request.timeout;
    }
    std::string snapshot_id_path;
    res = _calc_snapshot_id_path(target_tablet, timeout_s, &snapshot_id_path);
    if (!res.ok()) {
        LOG(WARNING) << "failed to calc snapshot_id_path, tablet="
                     << target_tablet->data_dir()->path();
        return res;
    }

    bool is_copy_binlog = request.__isset.is_copy_binlog ? request.is_copy_binlog : false;

    // schema_full_path_desc.filepath:
    //      /snapshot_id_path/tablet_id/schema_hash/
    auto schema_full_path = get_schema_hash_full_path(target_tablet, snapshot_id_path);
    // header_path:
    //      /schema_full_path/tablet_id.hdr
    auto header_path = _get_header_full_path(target_tablet, schema_full_path);
    //      /schema_full_path/tablet_id.hdr.json
    auto json_header_path = _get_json_header_full_path(target_tablet, schema_full_path);
    bool exists = true;
    RETURN_IF_ERROR(io::global_local_filesystem()->exists(schema_full_path, &exists));
    if (exists) {
        VLOG_TRACE << "remove the old schema_full_path." << schema_full_path;
        RETURN_IF_ERROR(io::global_local_filesystem()->delete_directory(schema_full_path));
    }

    RETURN_IF_ERROR(io::global_local_filesystem()->create_directory(schema_full_path));
    string snapshot_id;
    RETURN_IF_ERROR(io::global_local_filesystem()->canonicalize(snapshot_id_path, &snapshot_id));

    std::vector<RowsetSharedPtr> consistent_rowsets;
    do {
        TabletMetaSharedPtr new_tablet_meta(new (nothrow) TabletMeta());
        if (new_tablet_meta == nullptr) {
            res = Status::Error<MEM_ALLOC_FAILED>("fail to malloc TabletMeta.");
            break;
        }
        DeleteBitmap delete_bitmap_snapshot(new_tablet_meta->tablet_id());

        /// If set missing_version, try to get all missing version.
        /// If some of them not exist in tablet, we will fall back to
        /// make the full snapshot of the tablet.
        {
            std::shared_lock rdlock(ref_tablet->get_header_lock());
            if (ref_tablet->tablet_state() == TABLET_SHUTDOWN) {
                return Status::Aborted("tablet has shutdown");
            }
            bool is_single_rowset_clone =
                    (request.__isset.start_version && request.__isset.end_version);
            if (is_single_rowset_clone) {
                LOG(INFO) << "handle compaction clone make snapshot, tablet_id: "
                          << ref_tablet->tablet_id();
                Version version(request.start_version, request.end_version);
                const RowsetSharedPtr rowset = ref_tablet->get_rowset_by_version(version, false);
                if (rowset && rowset->is_local()) {
                    consistent_rowsets.push_back(rowset);
                } else {
                    LOG(WARNING) << "failed to find local version when do compaction snapshot. "
                                 << " tablet=" << request.tablet_id
                                 << " schema_hash=" << request.schema_hash
                                 << " version=" << version;
                    res = Status::InternalError(
                            "failed to find version when do compaction snapshot");
                    break;
                }
            }
            // be would definitely set it as true no matter has missed version or not
            // but it would take no effects on the following range loop
            if (!is_single_rowset_clone && request.__isset.missing_version) {
                for (int64_t missed_version : request.missing_version) {
                    Version version = {missed_version, missed_version};
                    // find rowset in both rs_meta and stale_rs_meta
                    const RowsetSharedPtr rowset = ref_tablet->get_rowset_by_version(version, true);
                    if (rowset != nullptr) {
                        if (!rowset->is_local()) {
                            // MUST make full snapshot to ensure `cooldown_meta_id` is consistent with the cooldowned rowsets after clone.
                            res = Status::Error<ErrorCode::INTERNAL_ERROR>(
                                    "missed version is a cooldowned rowset, must make full "
                                    "snapshot. missed_version={}, tablet_id={}",
                                    missed_version, ref_tablet->tablet_id());
                            break;
                        }
                        consistent_rowsets.push_back(rowset);
                    } else {
                        res = Status::InternalError(
                                "failed to find missed version when snapshot. tablet={}, "
                                "schema_hash={}, version={}",
                                request.tablet_id, request.schema_hash, version.to_string());
                        break;
                    }
                }
            }

            DBUG_EXECUTE_IF("SnapshotManager.create_snapshot_files.allow_inc_clone", {
                auto tablet_id = dp->param("tablet_id", 0);
                auto is_full_clone = dp->param("is_full_clone", false);
                if (ref_tablet->tablet_id() == tablet_id && is_full_clone) {
                    LOG(INFO) << "injected full clone for tabelt: " << tablet_id;
                    res = Status::InternalError("fault injection error");
                }
            });

            // be would definitely set it as true no matter has missed version or not, we could
            // just check whether the missed version is empty or not
            int64_t version = -1;
            if (!is_single_rowset_clone && (!res.ok() || request.missing_version.empty())) {
                if (!request.__isset.missing_version &&
                    ref_tablet->tablet_meta()->cooldown_meta_id().initialized()) {
                    LOG(WARNING) << "currently not support backup tablet with cooldowned remote "
                                    "data. tablet="
                                 << request.tablet_id;
                    return Status::NotSupported(
                            "currently not support backup tablet with cooldowned remote data");
                }
                /// not all missing versions are found, fall back to full snapshot.
                res = Status::OK();         // reset res
                consistent_rowsets.clear(); // reset vector

                // get latest version
                const RowsetSharedPtr last_version = ref_tablet->get_rowset_with_max_version();
                if (last_version == nullptr) {
                    res = Status::InternalError("tablet has not any version. path={}",
                                                ref_tablet->tablet_id());
                    break;
                }
                // get snapshot version, use request.version if specified
                version = last_version->end_version();
                if (request.__isset.version) {
                    if (last_version->end_version() < request.version) {
                        res = Status::Error<INVALID_ARGUMENT>(
                                "invalid make snapshot request. version={}, req_version={}",
                                last_version->version().to_string(), request.version);
                        break;
                    }
                    version = request.version;
                }
                if (ref_tablet->tablet_meta()->cooldown_meta_id().initialized()) {
                    // Tablet has cooldowned data, MUST pick consistent rowsets with continuous cooldowned version
                    // Get max cooldowned version
                    int64_t max_cooldowned_version = -1;
                    for (auto& [v, rs] : ref_tablet->rowset_map()) {
                        if (rs->is_local()) {
                            continue;
                        }
                        consistent_rowsets.push_back(rs);
                        max_cooldowned_version = std::max(max_cooldowned_version, v.second);
                    }
                    DCHECK_GE(max_cooldowned_version, 1) << "tablet_id=" << ref_tablet->tablet_id();
                    std::sort(consistent_rowsets.begin(), consistent_rowsets.end(),
                              Rowset::comparator);
                    res = check_version_continuity(consistent_rowsets);
                    if (res.ok() && max_cooldowned_version < version) {
                        // Pick consistent rowsets of remaining required version
                        res = ref_tablet->capture_consistent_rowsets_unlocked(
                                {max_cooldowned_version + 1, version}, &consistent_rowsets);
                    }
                } else {
                    // get shortest version path
                    res = ref_tablet->capture_consistent_rowsets_unlocked(Version(0, version),
                                                                          &consistent_rowsets);
                }
                if (!res.ok()) {
                    LOG(WARNING) << "fail to select versions to span. res=" << res;
                    break;
                }
                *allow_incremental_clone = false;
            } else {
                version = ref_tablet->max_version_unlocked();
                *allow_incremental_clone = true;
            }

            // copy the tablet meta to new_tablet_meta inside header lock
            CHECK(res.ok()) << res;
            ref_tablet->generate_tablet_meta_copy_unlocked(*new_tablet_meta);
            // The delete bitmap update operation and the add_inc_rowset operation is not atomic,
            // so delete bitmap may contains some data generated by invisible rowset, we should
            // get rid of these useless bitmaps when doing snapshot.
            if (ref_tablet->keys_type() == UNIQUE_KEYS &&
                ref_tablet->enable_unique_key_merge_on_write()) {
                delete_bitmap_snapshot =
                        ref_tablet->tablet_meta()->delete_bitmap().snapshot(version);
            }
        }

        std::vector<RowsetMetaSharedPtr> rs_metas;
        for (auto& rs : consistent_rowsets) {
            if (rs->is_local()) {
                // local rowset
                res = rs->link_files_to(schema_full_path, rs->rowset_id());
                if (!res.ok()) {
                    break;
                }
            }
            rs_metas.push_back(rs->rowset_meta());
            VLOG_NOTICE << "add rowset meta to clone list. "
                        << " start version " << rs->rowset_meta()->start_version()
                        << " end version " << rs->rowset_meta()->end_version() << " empty "
                        << rs->rowset_meta()->empty();
        }
        if (!res.ok()) {
            LOG(WARNING) << "fail to create hard link. path=" << snapshot_id_path
                         << " tablet=" << target_tablet->tablet_id()
                         << " ref tablet=" << ref_tablet->tablet_id();
            break;
        }

        // The inc_rs_metas is deprecated since Doris version 0.13.
        // Clear it for safety reason.
        // Whether it is incremental or full snapshot, rowset information is stored in rs_meta.
        new_tablet_meta->revise_rs_metas(std::move(rs_metas));
        if (ref_tablet->keys_type() == UNIQUE_KEYS &&
            ref_tablet->enable_unique_key_merge_on_write()) {
            new_tablet_meta->revise_delete_bitmap_unlocked(delete_bitmap_snapshot);
        }

        if (snapshot_version == g_Types_constants.TSNAPSHOT_REQ_VERSION2) {
            res = new_tablet_meta->save(header_path);
            if (res.ok() && request.__isset.is_copy_tablet_task && request.is_copy_tablet_task) {
                res = new_tablet_meta->save_as_json(json_header_path);
            }
        } else {
            res = Status::Error<INVALID_SNAPSHOT_VERSION>(
                    "snapshot_version not equal to g_Types_constants.TSNAPSHOT_REQ_VERSION2");
        }

        if (!res.ok()) {
            LOG(WARNING) << "convert rowset failed, res:" << res
                         << ", tablet:" << new_tablet_meta->tablet_id()
                         << ", schema hash:" << new_tablet_meta->schema_hash()
                         << ", snapshot_version:" << snapshot_version
                         << ", is incremental:" << request.__isset.missing_version;
            break;
        }

    } while (false);

    // link all binlog files to snapshot path
    do {
        if (!res.ok()) {
            break;
        }

        if (!is_copy_binlog) {
            break;
        }

        RowsetBinlogMetasPB rowset_binlog_metas_pb;
        for (auto& rs : consistent_rowsets) {
            if (!rs->is_local()) {
                continue;
            }
            res = ref_tablet->get_rowset_binlog_metas(rs->version(), &rowset_binlog_metas_pb);
            if (!res.ok()) {
                break;
            }
        }
        if (!res.ok() || rowset_binlog_metas_pb.rowset_binlog_metas_size() == 0) {
            break;
        }

        // write to pb file
        auto rowset_binlog_metas_pb_filename =
                fmt::format("{}/rowset_binlog_metas.pb", schema_full_path);
        res = write_pb(rowset_binlog_metas_pb_filename, rowset_binlog_metas_pb);
        if (!res.ok()) {
            break;
        }

        for (const auto& rowset_binlog_meta : rowset_binlog_metas_pb.rowset_binlog_metas()) {
            std::string segment_file_path;
            auto num_segments = rowset_binlog_meta.num_segments();
            std::string_view rowset_id = rowset_binlog_meta.rowset_id();

            RowsetMetaPB rowset_meta_pb;
            if (!rowset_meta_pb.ParseFromString(rowset_binlog_meta.data())) {
                auto err_msg = fmt::format("fail to parse binlog meta data value:{}",
                                           rowset_binlog_meta.data());
                res = Status::InternalError(err_msg);
                LOG(WARNING) << err_msg;
                return res;
            }
            const auto& tablet_schema_pb = rowset_meta_pb.tablet_schema();
            TabletSchema tablet_schema;
            tablet_schema.init_from_pb(tablet_schema_pb);

            std::vector<string> linked_success_files;
            Defer remove_linked_files {[&]() { // clear linked files if errors happen
                if (!res.ok()) {
                    LOG(WARNING) << "will delete linked success files due to error " << res;
                    std::vector<io::Path> paths;
                    for (auto& file : linked_success_files) {
                        paths.emplace_back(file);
                        LOG(WARNING)
                                << "will delete linked success file " << file << " due to error";
                    }
                    static_cast<void>(io::global_local_filesystem()->batch_delete(paths));
                    LOG(WARNING) << "done delete linked success files due to error " << res;
                }
            }};

            // link segment files and index files
            for (int64_t segment_index = 0; segment_index < num_segments; ++segment_index) {
                segment_file_path = ref_tablet->get_segment_filepath(rowset_id, segment_index);
                auto snapshot_segment_file_path =
                        fmt::format("{}/{}_{}.binlog", schema_full_path, rowset_id, segment_index);

                res = io::global_local_filesystem()->link_file(segment_file_path,
                                                               snapshot_segment_file_path);
                if (!res.ok()) {
                    LOG(WARNING) << "fail to link binlog file. [src=" << segment_file_path
                                 << ", dest=" << snapshot_segment_file_path << "]";
                    break;
                }
                linked_success_files.push_back(snapshot_segment_file_path);

                if (tablet_schema.get_inverted_index_storage_format() ==
                    InvertedIndexStorageFormatPB::V1) {
                    for (const auto& index : tablet_schema.inverted_indexes()) {
                        auto index_id = index->index_id();
                        auto index_file = InvertedIndexDescriptor::get_index_file_path_v1(
                                InvertedIndexDescriptor::get_index_file_path_prefix(
                                        segment_file_path),
                                index_id, index->get_index_suffix());
                        auto snapshot_segment_index_file_path =
                                fmt::format("{}/{}_{}_{}.binlog-index", schema_full_path, rowset_id,
                                            segment_index, index_id);
                        VLOG_DEBUG << "link " << index_file << " to "
                                   << snapshot_segment_index_file_path;
                        res = io::global_local_filesystem()->link_file(
                                index_file, snapshot_segment_index_file_path);
                        if (!res.ok()) {
                            LOG(WARNING) << "fail to link binlog index file. [src=" << index_file
                                         << ", dest=" << snapshot_segment_index_file_path << "]";
                            break;
                        }
                        linked_success_files.push_back(snapshot_segment_index_file_path);
                    }
                } else {
                    if (tablet_schema.has_inverted_index()) {
                        auto index_file = InvertedIndexDescriptor::get_index_file_path_v2(
                                InvertedIndexDescriptor::get_index_file_path_prefix(
                                        segment_file_path));
                        auto snapshot_segment_index_file_path =
                                fmt::format("{}/{}_{}.binlog-index", schema_full_path, rowset_id,
                                            segment_index);
                        VLOG_DEBUG << "link " << index_file << " to "
                                   << snapshot_segment_index_file_path;
                        res = io::global_local_filesystem()->link_file(
                                index_file, snapshot_segment_index_file_path);
                        if (!res.ok()) {
                            LOG(WARNING) << "fail to link binlog index file. [src=" << index_file
                                         << ", dest=" << snapshot_segment_index_file_path << "]";
                            break;
                        }
                        linked_success_files.push_back(snapshot_segment_index_file_path);
                    }
                }
            }

            if (!res.ok()) {
                break;
            }
        }
    } while (false);

    if (!res.ok()) {
        LOG(WARNING) << "fail to make snapshot, try to delete the snapshot path. path="
                     << snapshot_id_path.c_str();

        bool exists = true;
        RETURN_IF_ERROR(io::global_local_filesystem()->exists(snapshot_id_path, &exists));
        if (exists) {
            VLOG_NOTICE << "remove snapshot path. [path=" << snapshot_id_path << "]";
            RETURN_IF_ERROR(io::global_local_filesystem()->delete_directory(snapshot_id_path));
        }
    } else {
        *snapshot_path = snapshot_id;
    }

    return res;
}