Skip to content

Commit

Permalink
move system tables related with vector index to vi folder and rename …
Browse files Browse the repository at this point in the history
…Vector...
  • Loading branch information
sw1228 authored and Shanfeng Pang committed Apr 16, 2024
1 parent 300e872 commit 484e551
Show file tree
Hide file tree
Showing 31 changed files with 249 additions and 251 deletions.
2 changes: 1 addition & 1 deletion src/Storages/AlterCommands.h
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,7 @@
#include <Storages/ColumnsDescription.h>
#include <Common/SettingsChanges.h>

#include <VectorIndex/Storages/VICommands.h>
#include <VectorIndex/Common/VICommands.h>

namespace DB
{
Expand Down
4 changes: 2 additions & 2 deletions src/Storages/MergeTree/DataPartStorageOnDiskBase.h
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,7 @@ namespace DB
class IVolume;
using VolumePtr = std::shared_ptr<IVolume>;
class VIBuilderUpdater;
class MergetreeDataPartVectorIndex;
class VIWithDataPart;
class IMergeTreeDataPart;
using MergeTreeDataPartPtr = std::shared_ptr<const IMergeTreeDataPart>;
using MergeTreeMutableDataPartPtr = std::shared_ptr<IMergeTreeDataPart>;
Expand Down Expand Up @@ -104,7 +104,7 @@ class DataPartStorageOnDiskBase : public IDataPartStorage
friend class VIBuilderUpdater;
friend class MergeTreeVSManager;
friend class IMergeTreeDataPart;
friend class MergetreeDataPartVectorIndex;
friend class VIWithDataPart;
friend VolumePtr getVolumeFromPartStorage(const DataPartStorageOnDiskBase & storage);
friend MergeTreeDataPartChecksums moveVectorIndexFiles(
const bool decouple,
Expand Down
10 changes: 5 additions & 5 deletions src/Storages/MergeTree/DataPartsExchange.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -198,12 +198,12 @@ void Service::processQuery(const HTMLForm & params, ReadBuffer & /*body*/, Write
}

auto column_index = column_index_opt.value();
if (column_index->isBuildCancelled() || column_index->getVectorIndexState() == VectorIndexState::ERROR)
if (column_index->isBuildCancelled() || column_index->getVectorIndexState() == VIState::ERROR)
{
LOG_DEBUG(log, "The vector index {} build in part {} was cancelled or failed with error, cannot send it", vec_index_name, part_name);
response.addCookie({"vector_index_build_status", "fail"});
}
else if (column_index->getVectorIndexState() <= VectorIndexState::BUILDING)
else if (column_index->getVectorIndexState() <= VIState::BUILDING)
{
LOG_WARNING(log, "The vector index {} in part {} was not ready, cannot send it", vec_index_name, part_name);
response.addCookie({"vector_index_build_status", "not_ready"});
Expand Down Expand Up @@ -399,7 +399,7 @@ MergeTreeData::DataPart::Checksums Service::sendPartFromDisk(
bool include_vector_indices = true;

/// Add the part name to currently_sending_vector_index_parts
if (part->vector_index.containAnyVectorIndexInReady())
if (part->vector_index.containAnyVIInReady())
{
std::lock_guard lock(data.currently_sending_vector_index_parts_mutex);
if (!data.currently_sending_vector_index_parts.insert(part->name).second)
Expand All @@ -410,7 +410,7 @@ MergeTreeData::DataPart::Checksums Service::sendPartFromDisk(

SCOPE_EXIT_MEMORY
({
if (part->vector_index.containAnyVectorIndexInReady())
if (part->vector_index.containAnyVIInReady())
{
std::lock_guard lock(data.currently_sending_vector_index_parts_mutex);
data.currently_sending_vector_index_parts.erase(part->name);
Expand Down Expand Up @@ -999,7 +999,7 @@ String Fetcher::fetchVectorIndex(
if (build_status == "fail")
{
LOG_DEBUG(log, "Unable to fetch vector index {} in part {} due to build status is fail", vec_index_name, future_part_name);
future_part->vector_index.onVectorIndexBuildError(vec_index_name, "Another replica failed to build the vector index.");
future_part->vector_index.onVIBuildError(vec_index_name, "Another replica failed to build the vector index.");
return {};
}
else if (build_status == "no_need")
Expand Down
2 changes: 1 addition & 1 deletion src/Storages/MergeTree/IMergeTreeDataPart.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -1842,7 +1842,7 @@ void IMergeTreeDataPart::onLightweightDelete(const String index_name) const

/// Support multiple vector indices. We may need to update specified vector index after build finished.
bool update_all_indices = index_name.empty() ? true : false;
if ((update_all_indices && !vector_index.containAnyVectorIndexInReady()) || (!update_all_indices && !vector_index.containDecoupleOrVPartIndexInReady(index_name)))
if ((update_all_indices && !vector_index.containAnyVIInReady()) || (!update_all_indices && !vector_index.containDecoupleOrVPartIndexInReady(index_name)))
return;

/// Store deleted row ids
Expand Down
6 changes: 3 additions & 3 deletions src/Storages/MergeTree/IMergeTreeDataPart.h
Original file line number Diff line number Diff line change
Expand Up @@ -22,8 +22,8 @@
#include <Storages/MergeTree/MergeTreeIndices.h>
#include <Storages/MergeTree/MergeTreePartInfo.h>
#include <Storages/MergeTree/MergeTreePartition.h>
#include <VectorIndex/Common/VIWithDataPart.h>
#include <VectorIndex/Storages/VIInfo.h>
#include <VectorIndex/Storages/VIWithDataPart.h>
#include <base/types.h>


Expand Down Expand Up @@ -81,7 +81,7 @@ class IMergeTreeDataPart : public std::enable_shared_from_this<IMergeTreeDataPar

using uint128 = IPartMetadataManager::uint128;

friend class MergetreeDataPartVectorIndex;
friend class VIWithDataPart;

IMergeTreeDataPart(
const MergeTreeData & storage_,
Expand Down Expand Up @@ -340,7 +340,7 @@ class IMergeTreeDataPart : public std::enable_shared_from_this<IMergeTreeDataPar

mutable VersionMetadata version;

mutable MergetreeDataPartVectorIndex vector_index;
mutable VIWithDataPart vector_index;

/// For data in RAM ('index')
UInt64 getIndexSizeInBytes() const;
Expand Down
4 changes: 2 additions & 2 deletions src/Storages/MergeTree/MergeTask.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@
#include <Storages/MergeTree/MergeTreeSequentialSource.h>
#include <Storages/MergeTree/MergeTreeSource.h>
#include <VectorIndex/Cache/VICacheManager.h>
#include <VectorIndex/Storages/VIWithDataPart.h>
#include <VectorIndex/Common/VIWithDataPart.h>
#include <VectorIndex/Utils/VIUtils.h>
#include <Common/ActionBlocker.h>
#include <Common/logger_useful.h>
Expand Down Expand Up @@ -338,7 +338,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare()
if (!column_index_opt.has_value())
continue;
auto column_index = column_index_opt.value();
if (column_index->getVectorIndexState() == VectorIndexState::BUILT)
if (column_index->getVectorIndexState() == VIState::BUILT)
num_parts_with_vector_index++;

if (part->rows_count == 0)
Expand Down
7 changes: 3 additions & 4 deletions src/Storages/MergeTree/MergeTreeData.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -2420,8 +2420,8 @@ std::pair<bool, bool> MergeTreeData::needClearVectorIndexCacheAndFile(
/// Here the vector index is valid.
/// When remove old parts, we can remove it from cache only when it is not used by future part (mutation or merge).
/// Further check the part status, decouple part or VPart with single vector index
if ((is_same_without_mutate && column_index->getVectorIndexState() == VectorIndexState::BUILT)
|| (!is_same_without_mutate && (is_decouple && column_index->getVectorIndexState() != VectorIndexState::BUILT)))
if ((is_same_without_mutate && column_index->getVectorIndexState() == VIState::BUILT)
|| (!is_same_without_mutate && (is_decouple && column_index->getVectorIndexState() != VIState::BUILT)))
{
existed = true;
break;
Expand Down Expand Up @@ -2518,8 +2518,7 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t
}
for (const DataPartPtr & part : parts_to_remove)
{
if (vec_event_log &&
part->vector_index.containAnyVectorIndexInReady() &&
if (vec_event_log && part->vector_index.containAnyVIInReady() &&
vec_elem.event_type != VIEventLogElement::DEFAULT)
{
vec_elem.part_name = part->name;
Expand Down
2 changes: 1 addition & 1 deletion src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -682,7 +682,7 @@ void MergeTreeDataMergerMutator::handleVectorIndicesForMergedPart(

/// Special handling for merge one single VPart. If new part has vector index, expire the index cache for old part.
/// TODO: Can use old part's index cache, just update cache key to avoid load it for new part?
if (new_part->vector_index.containAnyVectorIndexInReady() && old_parts.size() == 1)
if (new_part->vector_index.containAnyVIInReady() && old_parts.size() == 1)
{
auto old_part = old_parts[0];
for (const auto & vec_index : new_vector_indices)
Expand Down
2 changes: 1 addition & 1 deletion src/Storages/MergeTree/MutateFromLogEntryTask.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -250,7 +250,7 @@ bool MutateFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrit
/// Update vector index bitmap after mutations with lightweight delete.
if (new_part->lightweight_delete_mask_updated)
{
if (new_part->vector_index.containAnyVectorIndexInReady())
if (new_part->vector_index.containAnyVIInReady())
new_part->onLightweightDelete();
}

Expand Down
2 changes: 1 addition & 1 deletion src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -105,7 +105,7 @@ bool MutatePlainMergeTreeTask::executeStep()
/// resulting in insufficient topk returned during subsequent searches.
if (new_part->lightweight_delete_mask_updated)
{
if (new_part->vector_index.containAnyVectorIndexInReady())
if (new_part->vector_index.containAnyVIInReady())
new_part->onLightweightDelete();
}

Expand Down
2 changes: 1 addition & 1 deletion src/Storages/MergeTree/MutateTask.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -1359,7 +1359,7 @@ class MutateAllPartColumnsTask : public IExecutableTask

/// Create hardlinks for vector index files in simple built part or decoupled part when MutateAllPartColumns
/// Reuse vector index when no rows are deleted
if (!ctx->need_delete_rows && ctx->source_part->vector_index.containAnyVectorIndexInReady())
if (!ctx->need_delete_rows && ctx->source_part->vector_index.containAnyVIInReady())
{
/// get current decouple index set
[[maybe_unused]] bool vector_files_found = false;
Expand Down
2 changes: 1 addition & 1 deletion src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -2583,7 +2583,7 @@ bool ReplicatedMergeTreeMergePredicate::canMergeWithVectorIndex(
/// Check if two parts contain vector index files.
/// Two parts can be merged when both have built vector index or both not.
for (const auto & vec_desc : metadata_snapshot->getVectorIndices())
if (!MergeTreeDataPartColumnIndex::canMergeForColumnIndex(left, right, vec_desc.name))
if (!VIWithColumnInPart::canMergeForColumnIndex(left, right, vec_desc.name))
{
if (out_reason)
*out_reason = "source part " + left->name + " or " + right->name + " doesn't contain the same built vector index";
Expand Down
2 changes: 1 addition & 1 deletion src/Storages/StorageMergeTree.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -904,7 +904,7 @@ bool StorageMergeTree::canMergeForVectorIndex(const StorageMetadataPtr & metadat

/// Check if part contains merged vector index
for (const auto & vec_desc : metadata_snapshot->getVectorIndices())
if (!MergeTreeDataPartColumnIndex::canMergeForColumnIndex(left, right, vec_desc.name))
if (!VIWithColumnInPart::canMergeForColumnIndex(left, right, vec_desc.name))
return false;

return true;
Expand Down
2 changes: 1 addition & 1 deletion src/Storages/StorageReplicatedMergeTree.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -3669,7 +3669,7 @@ void StorageReplicatedMergeTree::removeVecIndexBuildStatusForPartsFromZK(zkutil:
continue;

auto column_index = column_index_opt.value();
if (!(column_index->getVectorIndexState() == VectorIndexState::BUILT) && !(column_index->getVectorIndexState() == VectorIndexState::ERROR))
if (!(column_index->getVectorIndexState() == VIState::BUILT) && !(column_index->getVectorIndexState() == VIState::ERROR))
continue;

String part_status_path = fs::path(replica_path) / "vidx_build_parts" / vec_index_name / part_name;
Expand Down
90 changes: 45 additions & 45 deletions src/Storages/System/attachSystemTables.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -6,83 +6,83 @@

#include <Interpreters/Context.h>
#include <Storages/System/StorageSystemAggregateFunctionCombinators.h>
#include <Storages/System/StorageSystemAsynchronousInserts.h>
#include <Storages/System/StorageSystemAsynchronousMetrics.h>
#include <Storages/System/StorageSystemBackups.h>
#include <Storages/System/StorageSystemBuildOptions.h>
#include <Storages/System/StorageSystemCollations.h>
#include <Storages/System/StorageSystemCertificates.h>
#include <Storages/System/StorageSystemClusters.h>
#include <Storages/System/StorageSystemCollations.h>
#include <Storages/System/StorageSystemColumns.h>
#include <Storages/System/StorageSystemDatabases.h>
#include <Storages/System/StorageSystemConnections.h>
#include <Storages/System/StorageSystemContributors.h>
#include <Storages/System/StorageSystemCurrentRoles.h>
#include <Storages/System/StorageSystemDDLWorkerQueue.h>
#include <Storages/System/StorageSystemDataSkippingIndices.h>
#include <Storages/System/StorageSystemVectorIndices.h>
#include <Storages/System/StorageSystemDataTypeFamilies.h>
#include <Storages/System/StorageSystemDatabases.h>
#include <Storages/System/StorageSystemDetachedParts.h>
#include <Storages/System/StorageSystemDictionaries.h>
#include <Storages/System/StorageSystemDisks.h>
#include <Storages/System/StorageSystemDistributionQueue.h>
#include <Storages/System/StorageSystemDroppedTables.h>
#include <Storages/System/StorageSystemEnabledRoles.h>
#include <Storages/System/StorageSystemErrors.h>
#include <Storages/System/StorageSystemEvents.h>
#include <Storages/System/StorageSystemFilesystemCache.h>
#include <Storages/System/StorageSystemFormats.h>
#include <Storages/System/StorageSystemFunctions.h>
#include <Storages/System/StorageSystemGrants.h>
#include <Storages/System/StorageSystemGraphite.h>
#include <Storages/System/StorageSystemLicenses.h>
#include <Storages/System/StorageSystemMacros.h>
#include <Storages/System/StorageSystemMergeTreeSettings.h>
#include <Storages/System/StorageSystemMerges.h>
#include <Storages/System/StorageSystemMoves.h>
#include <Storages/System/StorageSystemReplicatedFetches.h>
#include <Storages/System/StorageSystemMetrics.h>
#include <Storages/System/StorageSystemModels.h>
#include <Storages/System/StorageSystemMoves.h>
#include <Storages/System/StorageSystemMutations.h>
#include <Storages/System/StorageSystemNamedCollections.h>
#include <Storages/System/StorageSystemNumbers.h>
#include <Storages/System/StorageSystemOne.h>
#include <Storages/System/StorageSystemPartMovesBetweenShards.h>
#include <Storages/System/StorageSystemParts.h>
#include <Storages/System/StorageSystemProjectionParts.h>
#include <Storages/System/StorageSystemPartsColumns.h>
#include <Storages/System/StorageSystemProjectionPartsColumns.h>
#include <Storages/System/StorageSystemPrivileges.h>
#include <Storages/System/StorageSystemProcesses.h>
#include <Storages/System/StorageSystemProjectionParts.h>
#include <Storages/System/StorageSystemProjectionPartsColumns.h>
#include <Storages/System/StorageSystemQueryCache.h>
#include <Storages/System/StorageSystemQuotaLimits.h>
#include <Storages/System/StorageSystemQuotaUsage.h>
#include <Storages/System/StorageSystemQuotas.h>
#include <Storages/System/StorageSystemQuotasUsage.h>
#include <Storages/System/StorageSystemRemoteDataPaths.h>
#include <Storages/System/StorageSystemReplicas.h>
#include <Storages/System/StorageSystemReplicatedFetches.h>
#include <Storages/System/StorageSystemReplicationQueue.h>
#include <Storages/System/StorageSystemDistributionQueue.h>
#include <Storages/System/StorageSystemRoleGrants.h>
#include <Storages/System/StorageSystemRoles.h>
#include <Storages/System/StorageSystemRowPolicies.h>
#include <Storages/System/StorageSystemSchemaInferenceCache.h>
#include <Storages/System/StorageSystemServerSettings.h>
#include <Storages/System/StorageSystemSettings.h>
#include <Storages/System/StorageSystemSettingsChanges.h>
#include <Storages/System/StorageSystemMergeTreeSettings.h>
#include <Storages/System/StorageSystemSettingsProfileElements.h>
#include <Storages/System/StorageSystemSettingsProfiles.h>
#include <Storages/System/StorageSystemStoragePolicies.h>
#include <Storages/System/StorageSystemTableEngines.h>
#include <Storages/System/StorageSystemTableFunctions.h>
#include <Storages/System/StorageSystemTables.h>
#include <Storages/System/StorageSystemZooKeeper.h>
#include <Storages/System/StorageSystemContributors.h>
#include <Storages/System/StorageSystemErrors.h>
#include <Storages/System/StorageSystemWarnings.h>
#include <Storages/System/StorageSystemDDLWorkerQueue.h>
#include <Storages/System/StorageSystemLicenses.h>
#include <Storages/System/StorageSystemTimeZones.h>
#include <Storages/System/StorageSystemDisks.h>
#include <Storages/System/StorageSystemStoragePolicies.h>
#include <Storages/System/StorageSystemZeros.h>
#include <Storages/System/StorageSystemUsers.h>
#include <Storages/System/StorageSystemRoles.h>
#include <Storages/System/StorageSystemGrants.h>
#include <Storages/System/StorageSystemRoleGrants.h>
#include <Storages/System/StorageSystemCurrentRoles.h>
#include <Storages/System/StorageSystemEnabledRoles.h>
#include <Storages/System/StorageSystemSettingsProfiles.h>
#include <Storages/System/StorageSystemSettingsProfileElements.h>
#include <Storages/System/StorageSystemRowPolicies.h>
#include <Storages/System/StorageSystemQuotas.h>
#include <Storages/System/StorageSystemQuotaLimits.h>
#include <Storages/System/StorageSystemQuotaUsage.h>
#include <Storages/System/StorageSystemQuotasUsage.h>
#include <Storages/System/StorageSystemConnections.h>
#include <Storages/System/StorageSystemUserDirectories.h>
#include <Storages/System/StorageSystemPrivileges.h>
#include <Storages/System/StorageSystemAsynchronousInserts.h>
#include <Storages/System/StorageSystemTransactions.h>
#include <Storages/System/StorageSystemFilesystemCache.h>
#include <Storages/System/StorageSystemQueryCache.h>
#include <Storages/System/StorageSystemNamedCollections.h>
#include <Storages/System/StorageSystemRemoteDataPaths.h>
#include <Storages/System/StorageSystemCertificates.h>
#include <Storages/System/StorageSystemSchemaInferenceCache.h>
#include <Storages/System/StorageSystemDroppedTables.h>
#include <Storages/System/StorageSystemVectorIndexSegments.h>
#include <Storages/System/StorageSystemUserDirectories.h>
#include <Storages/System/StorageSystemUsers.h>
#include <Storages/System/StorageSystemWarnings.h>
#include <Storages/System/StorageSystemZeros.h>
#include <Storages/System/StorageSystemZooKeeper.h>
#include <VectorIndex/Storages/StorageSystemVIs.h>
#include <VectorIndex/Storages/StorageSystemVIsWithPart.h>

#ifdef OS_LINUX
#include <Storages/System/StorageSystemStackTrace.h>
Expand Down Expand Up @@ -141,8 +141,8 @@ void attachSystemTablesLocal(ContextPtr context, IDatabase & system_database)
attach<StorageSystemErrors>(context, system_database, "errors");
attach<StorageSystemWarnings>(context, system_database, "warnings");
attach<StorageSystemDataSkippingIndices>(context, system_database, "data_skipping_indices");
attach<StorageSystemVectorIndices>(context, system_database, "vector_indices");
attach<StorageSystemVectorIndexSegments>(context, system_database, "vector_index_segments");
attach<StorageSystemVIs>(context, system_database, "vector_indices");
attach<StorageSystemVIsWithPart>(context, system_database, "vector_index_segments");
attach<StorageSystemLicenses>(context, system_database, "licenses");
attach<StorageSystemTimeZones>(context, system_database, "time_zones");
attach<StorageSystemBackups>(context, system_database, "backups");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,8 +31,8 @@
#include <Parsers/queryToString.h>
#include <Common/logger_useful.h>

#include <VectorIndex/Common/VICommands.h>
#include <VectorIndex/Parsers/ASTVIDeclaration.h>
#include <VectorIndex/Storages/VICommands.h>

namespace DB
{
Expand Down
File renamed without changes.
Loading

0 comments on commit 484e551

Please sign in to comment.