Merge remote-tracking branch 'upstream/master' into HEAD

This commit is contained in:
Anton Popov 2024-08-15 10:42:23 +00:00
commit a552747082
30 changed files with 223 additions and 133 deletions

View File

@ -101,6 +101,7 @@ jobs:
--volume=".:/wd" --workdir="/wd" \
clickhouse/style-test \
./tests/ci/changelog.py -v --debug-helpers \
--gh-user-or-token ${{ secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN }} \
--jobs=5 \
--output="./docs/changelogs/${{ env.RELEASE_TAG }}.md" ${{ env.RELEASE_TAG }}
git add ./docs/changelogs/${{ env.RELEASE_TAG }}.md

3
.gitmodules vendored
View File

@ -345,9 +345,6 @@
[submodule "contrib/FP16"]
path = contrib/FP16
url = https://github.com/Maratyszcza/FP16.git
[submodule "contrib/robin-map"]
path = contrib/robin-map
url = https://github.com/Tessil/robin-map.git
[submodule "contrib/aklomp-base64"]
path = contrib/aklomp-base64
url = https://github.com/aklomp/base64.git

View File

@ -322,17 +322,21 @@ if (DISABLE_OMIT_FRAME_POINTER)
set (CMAKE_ASM_FLAGS_ADD "${CMAKE_ASM_FLAGS_ADD} -fno-omit-frame-pointer -mno-omit-leaf-frame-pointer")
endif()
# Before you start hating your debugger because it refuses to show variables ('<optimized out>'), try building with -DDEBUG_O_LEVEL="0"
# https://stackoverflow.com/questions/63386189/whats-the-difference-between-a-compilers-o0-option-and-og-option/63386263#63386263
set(DEBUG_O_LEVEL "g" CACHE STRING "The -Ox level used for debug builds")
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${COMPILER_FLAGS} ${CMAKE_CXX_FLAGS_ADD}")
set (CMAKE_CXX_FLAGS_RELWITHDEBINFO "${CMAKE_CXX_FLAGS_RELWITHDEBINFO} -O3 ${DEBUG_INFO_FLAGS} ${CMAKE_CXX_FLAGS_ADD}")
set (CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS_DEBUG} -Og ${DEBUG_INFO_FLAGS} ${CMAKE_CXX_FLAGS_ADD}")
set (CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS_DEBUG} -O${DEBUG_O_LEVEL} ${DEBUG_INFO_FLAGS} ${CMAKE_CXX_FLAGS_ADD}")
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${COMPILER_FLAGS} ${CMAKE_C_FLAGS_ADD}")
set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -O3 ${DEBUG_INFO_FLAGS} ${CMAKE_C_FLAGS_ADD}")
set (CMAKE_C_FLAGS_DEBUG "${CMAKE_C_FLAGS_DEBUG} -Og ${DEBUG_INFO_FLAGS} ${CMAKE_C_FLAGS_ADD}")
set (CMAKE_C_FLAGS_DEBUG "${CMAKE_C_FLAGS_DEBUG} -O${DEBUG_O_LEVEL} ${DEBUG_INFO_FLAGS} ${CMAKE_C_FLAGS_ADD}")
set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} ${COMPILER_FLAGS} ${CMAKE_ASM_FLAGS_ADD}")
set (CMAKE_ASM_FLAGS_RELWITHDEBINFO "${CMAKE_ASM_FLAGS_RELWITHDEBINFO} -O3 ${DEBUG_INFO_FLAGS} ${CMAKE_ASM_FLAGS_ADD}")
set (CMAKE_ASM_FLAGS_DEBUG "${CMAKE_ASM_FLAGS_DEBUG} -Og ${DEBUG_INFO_FLAGS} ${CMAKE_ASM_FLAGS_ADD}")
set (CMAKE_ASM_FLAGS_DEBUG "${CMAKE_ASM_FLAGS_DEBUG} -O${DEBUG_O_LEVEL} ${DEBUG_INFO_FLAGS} ${CMAKE_ASM_FLAGS_ADD}")
if (OS_DARWIN)
set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -stdlib=libc++")

View File

@ -209,9 +209,8 @@ endif()
option(ENABLE_USEARCH "Enable USearch" ${ENABLE_LIBRARIES})
if (ENABLE_USEARCH)
add_contrib (FP16-cmake FP16)
add_contrib (robin-map-cmake robin-map)
add_contrib (SimSIMD-cmake SimSIMD)
add_contrib (usearch-cmake usearch) # requires: FP16, robin-map, SimdSIMD
add_contrib (usearch-cmake usearch) # requires: FP16, SimdSIMD
else ()
message(STATUS "Not using USearch")
endif ()

2
contrib/SimSIMD vendored

@ -1 +1 @@
Subproject commit de2cb75b9e9e3389d5e1e51fd9f8ed151f3c17cf
Subproject commit 91a76d1ac519b3b9dc8957734a3dabd985f00c26

1
contrib/robin-map vendored

@ -1 +0,0 @@
Subproject commit 851a59e0e3063ee0e23089062090a73fd3de482d

View File

@ -1 +0,0 @@
# See contrib/usearch-cmake/CMakeLists.txt

2
contrib/usearch vendored

@ -1 +1 @@
Subproject commit 30810452bec5d3d3aa0931bb5d761e2f09aa6356
Subproject commit e21a5778a0d4469ddaf38c94b7be0196bb701ee4

View File

@ -1,5 +1,4 @@
set(FP16_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/FP16")
set(ROBIN_MAP_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/robin-map")
set(SIMSIMD_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/SimSIMD")
set(USEARCH_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/usearch")
@ -7,7 +6,6 @@ add_library(_usearch INTERFACE)
target_include_directories(_usearch SYSTEM INTERFACE
${FP16_PROJECT_DIR}/include
${ROBIN_MAP_PROJECT_DIR}/include
${SIMSIMD_PROJECT_DIR}/include
${USEARCH_PROJECT_DIR}/include)

View File

@ -59,6 +59,8 @@ Parameters:
- `ef_construction`: (optional, default: 128)
- `ef_search`: (optional, default: 64)
Value 0 for parameters `m`, `ef_construction`, and `ef_search` refers to the default value.
Example:
```sql

View File

@ -307,7 +307,7 @@
M(FilteringMarksWithPrimaryKey, "Number of threads currently doing filtering of mark ranges by the primary key") \
M(FilteringMarksWithSecondaryKeys, "Number of threads currently doing filtering of mark ranges by secondary keys") \
\
M(S3DiskNoKeyErrors, "The number of `NoSuchKey` errors that occur when reading data from S3 cloud storage through ClickHouse disks.") \
M(DiskS3NoSuchKeyErrors, "The number of `NoSuchKey` errors that occur when reading data from S3 cloud storage through ClickHouse disks.") \
#ifdef APPLY_FOR_EXTERNAL_METRICS
#define APPLY_FOR_METRICS(M) APPLY_FOR_BUILTIN_METRICS(M) APPLY_FOR_EXTERNAL_METRICS(M)

View File

@ -184,14 +184,20 @@ void DynamicResourceManager::updateConfiguration(const Poco::Util::AbstractConfi
// Resource update leads to loss of runtime data of nodes and may lead to temporary violation of constraints (e.g. limits)
// Try to minimise this by reusing "equal" resources (initialized with the same configuration).
std::vector<State::ResourcePtr> resources_to_attach;
for (auto & [name, new_resource] : new_state->resources)
{
if (auto iter = state->resources.find(name); iter != state->resources.end()) // Resource update
{
State::ResourcePtr old_resource = iter->second;
if (old_resource->equals(*new_resource))
{
new_resource = old_resource; // Rewrite with older version to avoid loss of runtime data
continue;
}
}
// It is new or updated resource
resources_to_attach.emplace_back(new_resource);
}
// Commit new state
@ -199,17 +205,14 @@ void DynamicResourceManager::updateConfiguration(const Poco::Util::AbstractConfi
state = new_state;
// Attach new and updated resources to the scheduler
for (auto & [name, resource] : new_state->resources)
for (auto & resource : resources_to_attach)
{
const SchedulerNodePtr & root = resource->nodes.find("/")->second.ptr;
if (root->parent == nullptr)
resource->attached_to = &scheduler;
scheduler.event_queue->enqueue([this, root]
{
resource->attached_to = &scheduler;
scheduler.event_queue->enqueue([this, root]
{
scheduler.attachChild(root);
});
}
scheduler.attachChild(root);
});
}
// NOTE: after mutex unlock `state` became available for Classifier(s) and must be immutable

View File

@ -46,7 +46,7 @@ namespace ProfileEvents
namespace CurrentMetrics
{
extern const Metric S3DiskNoKeyErrors;
extern const Metric DiskS3NoSuchKeyErrors;
}
namespace DB
@ -701,7 +701,7 @@ RequestResult Client::processRequestResult(RequestResult && outcome) const
return std::forward<RequestResult>(outcome);
if (outcome.GetError().GetErrorType() == Aws::S3::S3Errors::NO_SUCH_KEY)
CurrentMetrics::add(CurrentMetrics::S3DiskNoKeyErrors);
CurrentMetrics::add(CurrentMetrics::DiskS3NoSuchKeyErrors);
String enriched_message = fmt::format(
"{} {}",

View File

@ -354,8 +354,8 @@ size_t HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::joinRightColumns(
{
if (unlikely(current_offset >= max_joined_block_rows))
{
added_columns.offsets_to_replicate->resize_assume_reserved(i);
added_columns.filter.resize_assume_reserved(i);
added_columns.offsets_to_replicate->resize(i);
added_columns.filter.resize(i);
break;
}
}

View File

@ -18,11 +18,25 @@ struct JoinFeatures
static constexpr bool inner = KIND == JoinKind::Inner;
static constexpr bool full = KIND == JoinKind::Full;
/** Whether we may need duplicate rows from the left table.
* For example, when we have row (key1, attr1) in left table
* and rows (key1, attr2), (key1, attr3) in right table,
* then we need to duplicate row (key1, attr1) for each of joined rows from right table, so result will be
* (key1, attr1, key1, attr2)
* (key1, attr1, key1, attr3)
*/
static constexpr bool need_replication = is_all_join || (is_any_join && right) || (is_semi_join && right);
/// Whether we need to filter rows from the left table that do not have matches in the right table.
static constexpr bool need_filter = !need_replication && (inner || right || (is_semi_join && left) || (is_anti_join && left));
/// Whether we need to add default values for columns from the left table.
static constexpr bool add_missing = (left || full) && !is_semi_join;
/// Whether we need to store flags for rows from the right table table
/// that indicates if they have matches in the left table.
static constexpr bool need_flags = MapGetter<KIND, STRICTNESS, std::is_same_v<std::decay_t<Map>, HashJoin::MapsAll>>::flagged;
static constexpr bool is_maps_all = std::is_same_v<std::decay_t<Map>, HashJoin::MapsAll>;
};

View File

@ -198,6 +198,29 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState & s
print_database_table();
}
if (sync_replica_mode != SyncReplicaMode::DEFAULT)
{
settings.ostr << ' ';
print_keyword(magic_enum::enum_name(sync_replica_mode));
// If the mode is LIGHTWEIGHT and specific source replicas are specified
if (sync_replica_mode == SyncReplicaMode::LIGHTWEIGHT && !src_replicas.empty())
{
settings.ostr << ' ';
print_keyword("FROM");
settings.ostr << ' ';
bool first = true;
for (const auto & src : src_replicas)
{
if (!first)
settings.ostr << ", ";
first = false;
settings.ostr << quoteString(src);
}
}
}
if (query_settings)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << settings.nl_or_ws << "SETTINGS " << (settings.hilite ? hilite_none : "");
@ -233,28 +256,6 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState & s
print_identifier(disk);
}
if (sync_replica_mode != SyncReplicaMode::DEFAULT)
{
settings.ostr << ' ';
print_keyword(magic_enum::enum_name(sync_replica_mode));
// If the mode is LIGHTWEIGHT and specific source replicas are specified
if (sync_replica_mode == SyncReplicaMode::LIGHTWEIGHT && !src_replicas.empty())
{
settings.ostr << ' ';
print_keyword("FROM");
settings.ostr << ' ';
bool first = true;
for (const auto & src : src_replicas)
{
if (!first)
settings.ostr << ", ";
first = false;
settings.ostr << quoteString(src);
}
}
}
break;
}
case Type::SYNC_DATABASE_REPLICA:

View File

@ -1157,8 +1157,7 @@ void WindowTransform::appendChunk(Chunk & chunk)
// Initialize output columns.
for (auto & ws : workspaces)
{
if (ws.window_function_impl)
block.casted_columns.push_back(ws.window_function_impl->castColumn(block.input_columns, ws.argument_column_indices));
block.casted_columns.push_back(ws.window_function_impl ? ws.window_function_impl->castColumn(block.input_columns, ws.argument_column_indices) : nullptr);
block.output_columns.push_back(ws.aggregate_function->getResultType()
->createColumn());

View File

@ -2,9 +2,6 @@
#if USE_USEARCH
#pragma clang diagnostic push
#pragma clang diagnostic ignored "-Wpass-failed"
#include <Columns/ColumnArray.h>
#include <Common/BitHelpers.h>
#include <Common/formatReadable.h>
@ -46,15 +43,15 @@ namespace
{
/// The only indexing method currently supported by USearch
std::set<String> methods = {"hnsw"};
const std::set<String> methods = {"hnsw"};
/// Maps from user-facing name to internal name
std::unordered_map<String, unum::usearch::metric_kind_t> distanceFunctionToMetricKind = {
const std::unordered_map<String, unum::usearch::metric_kind_t> distanceFunctionToMetricKind = {
{"L2Distance", unum::usearch::metric_kind_t::l2sq_k},
{"cosineDistance", unum::usearch::metric_kind_t::cos_k}};
/// Maps from user-facing name to internal name
std::unordered_map<String, unum::usearch::scalar_kind_t> quantizationToScalarKind = {
const std::unordered_map<String, unum::usearch::scalar_kind_t> quantizationToScalarKind = {
{"f32", unum::usearch::scalar_kind_t::f32_k},
{"f16", unum::usearch::scalar_kind_t::f16_k},
{"i8", unum::usearch::scalar_kind_t::i8_k}};
@ -95,9 +92,19 @@ USearchIndexWithSerialization::USearchIndexWithSerialization(
unum::usearch::metric_kind_t metric_kind,
unum::usearch::scalar_kind_t scalar_kind,
UsearchHnswParams usearch_hnsw_params)
: Base(Base::make(unum::usearch::metric_punned_t(dimensions, metric_kind, scalar_kind),
unum::usearch::index_dense_config_t(usearch_hnsw_params.m, usearch_hnsw_params.ef_construction, usearch_hnsw_params.ef_search)))
{
USearchIndex::metric_t metric(dimensions, metric_kind, scalar_kind);
unum::usearch::index_dense_config_t config(usearch_hnsw_params.m, usearch_hnsw_params.ef_construction, usearch_hnsw_params.ef_search);
config.enable_key_lookups = false; /// we don't do row-to-vector lookups
if (auto error = config.validate(); error) /// already called in vectorSimilarityIndexValidator, call again because usearch may change the config in-place
throw Exception(ErrorCodes::INCORRECT_DATA, "Invalid parameters passed to vector similarity index. Error: {}", String(error.release()));
if (auto result = USearchIndex::make(metric, config); !result)
throw Exception(ErrorCodes::INCORRECT_DATA, "Could not create vector similarity index. Error: {}", String(result.error.release()));
else
swap(result.index);
}
void USearchIndexWithSerialization::serialize(WriteBuffer & ostr) const
@ -108,9 +115,8 @@ void USearchIndexWithSerialization::serialize(WriteBuffer & ostr) const
return true;
};
auto result = Base::save_to_stream(callback);
if (result.error)
throw Exception::createRuntime(ErrorCodes::INCORRECT_DATA, "Could not save vector similarity index, error: " + String(result.error.release()));
if (auto result = Base::save_to_stream(callback); !result)
throw Exception(ErrorCodes::INCORRECT_DATA, "Could not save vector similarity index. Error: {}", String(result.error.release()));
}
void USearchIndexWithSerialization::deserialize(ReadBuffer & istr)
@ -121,26 +127,43 @@ void USearchIndexWithSerialization::deserialize(ReadBuffer & istr)
return true;
};
auto result = Base::load_from_stream(callback);
if (result.error)
if (auto result = Base::load_from_stream(callback); !result)
/// See the comment in MergeTreeIndexGranuleVectorSimilarity::deserializeBinary why we throw here
throw Exception::createRuntime(ErrorCodes::INCORRECT_DATA, "Could not load vector similarity index, error: " + String(result.error.release()) + " Please drop the index and create it again.");
throw Exception(ErrorCodes::INCORRECT_DATA, "Could not load vector similarity index. Please drop the index and create it again. Error: {}", String(result.error.release()));
if (!try_reserve(limits()))
throw Exception(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Could not reserve memory for usearch index");
}
USearchIndexWithSerialization::Statistics USearchIndexWithSerialization::getStatistics() const
{
USearchIndex::stats_t global_stats = Base::stats();
Statistics statistics = {
.max_level = max_level(),
.connectivity = connectivity(),
.size = size(), /// number of vectors
.capacity = capacity(), /// number of vectors reserved
.memory_usage = memory_usage(), /// in bytes, the value is not exact
.size = size(),
.capacity = capacity(),
.memory_usage = memory_usage(),
.bytes_per_vector = bytes_per_vector(),
.scalar_words = scalar_words(),
.statistics = stats()};
.nodes = global_stats.nodes,
.edges = global_stats.edges,
.max_edges = global_stats.max_edges,
.level_stats = {}};
for (size_t i = 0; i < statistics.max_level; ++i)
statistics.level_stats.push_back(Base::stats(i));
return statistics;
}
String USearchIndexWithSerialization::Statistics::toString() const
{
return fmt::format("max_level = {}, connectivity = {}, size = {}, capacity = {}, memory_usage = {}, bytes_per_vector = {}, scalar_words = {}, nodes = {}, edges = {}, max_edges = {}",
max_level, connectivity, size, capacity, ReadableSize(memory_usage), bytes_per_vector, scalar_words, nodes, edges, max_edges);
}
MergeTreeIndexGranuleVectorSimilarity::MergeTreeIndexGranuleVectorSimilarity(
const String & index_name_,
const Block & index_sample_block_,
@ -181,8 +204,7 @@ void MergeTreeIndexGranuleVectorSimilarity::serializeBinary(WriteBuffer & ostr)
index->serialize(ostr);
auto statistics = index->getStatistics();
LOG_TRACE(logger, "Wrote vector similarity index: max_level = {}, connectivity = {}, size = {}, capacity = {}, memory_usage = {}",
statistics.max_level, statistics.connectivity, statistics.size, statistics.capacity, ReadableSize(statistics.memory_usage));
LOG_TRACE(logger, "Wrote vector similarity index: {}", statistics.toString());
}
void MergeTreeIndexGranuleVectorSimilarity::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion /*version*/)
@ -204,8 +226,7 @@ void MergeTreeIndexGranuleVectorSimilarity::deserializeBinary(ReadBuffer & istr,
index->deserialize(istr);
auto statistics = index->getStatistics();
LOG_TRACE(logger, "Loaded vector similarity index: max_level = {}, connectivity = {}, size = {}, capacity = {}, memory_usage = {}",
statistics.max_level, statistics.connectivity, statistics.size, statistics.capacity, ReadableSize(statistics.memory_usage));
LOG_TRACE(logger, "Loaded vector similarity index: {}", statistics.toString());
}
MergeTreeIndexAggregatorVectorSimilarity::MergeTreeIndexAggregatorVectorSimilarity(
@ -285,19 +306,24 @@ void MergeTreeIndexAggregatorVectorSimilarity::update(const Block & block, size_
if (!index)
index = std::make_shared<USearchIndexWithSerialization>(dimensions, metric_kind, scalar_kind, usearch_hnsw_params);
/// We use Usearch's index_dense_t as index type which supports only 4 bio entries according to https://github.com/unum-cloud/usearch/tree/main/cpp
if (index->size() + num_rows > std::numeric_limits<UInt32>::max())
throw Exception(ErrorCodes::INCORRECT_DATA, "Size of vector similarity index in column {} would exceed 4 billion entries", index_column_name);
/// Reserving space is mandatory
if (!index->reserve(roundUpToPowerOfTwoOrZero(index->size() + num_rows)))
if (!index->try_reserve(roundUpToPowerOfTwoOrZero(index->size() + num_rows)))
throw Exception(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Could not reserve memory for vector similarity index");
for (size_t row = 0; row < num_rows; ++row)
{
auto rc = index->add(static_cast<UInt32>(index->size()), &column_array_data_float_data[column_array_offsets[row - 1]]);
if (!rc)
throw Exception::createRuntime(ErrorCodes::INCORRECT_DATA, "Could not add data to vector similarity index, error: " + String(rc.error.release()));
ProfileEvents::increment(ProfileEvents::USearchAddCount);
ProfileEvents::increment(ProfileEvents::USearchAddVisitedMembers, rc.visited_members);
ProfileEvents::increment(ProfileEvents::USearchAddComputedDistances, rc.computed_distances);
if (auto result = index->add(static_cast<UInt32>(index->size()), &column_array_data_float_data[column_array_offsets[row - 1]]); !result)
throw Exception(ErrorCodes::INCORRECT_DATA, "Could not add data to vector similarity index. Error: {}", String(result.error.release()));
else
{
ProfileEvents::increment(ProfileEvents::USearchAddCount);
ProfileEvents::increment(ProfileEvents::USearchAddVisitedMembers, result.visited_members);
ProfileEvents::increment(ProfileEvents::USearchAddComputedDistances, result.computed_distances);
}
}
}
else
@ -351,17 +377,16 @@ std::vector<size_t> MergeTreeIndexConditionVectorSimilarity::getUsefulRanges(Mer
const std::vector<float> reference_vector = vector_similarity_condition.getReferenceVector();
auto result = index->search(reference_vector.data(), limit);
if (result.error)
throw Exception::createRuntime(ErrorCodes::INCORRECT_DATA, "Could not search in vector similarity index, error: " + String(result.error.release()));
auto search_result = index->search(reference_vector.data(), limit);
if (!search_result)
throw Exception(ErrorCodes::INCORRECT_DATA, "Could not search in vector similarity index. Error: {}", String(search_result.error.release()));
ProfileEvents::increment(ProfileEvents::USearchSearchCount);
ProfileEvents::increment(ProfileEvents::USearchSearchVisitedMembers, result.visited_members);
ProfileEvents::increment(ProfileEvents::USearchSearchComputedDistances, result.computed_distances);
ProfileEvents::increment(ProfileEvents::USearchSearchVisitedMembers, search_result.visited_members);
ProfileEvents::increment(ProfileEvents::USearchSearchComputedDistances, search_result.computed_distances);
std::vector<USearchIndex::key_t> neighbors(result.size()); /// indexes of dots which were closest to the reference vector
std::vector<USearchIndex::distance_t> distances(result.size());
result.dump_to(neighbors.data(), distances.data());
std::vector<USearchIndex::vector_key_t> neighbors(search_result.size()); /// indexes of vectors which were closest to the reference vector
search_result.dump_to(neighbors.data());
std::vector<size_t> granules;
granules.reserve(neighbors.size());
@ -409,14 +434,13 @@ MergeTreeIndexConditionPtr MergeTreeIndexVectorSimilarity::createIndexCondition(
MergeTreeIndexPtr vectorSimilarityIndexCreator(const IndexDescription & index)
{
const bool has_six_args = (index.arguments.size() == 6);
/// Default parameters:
unum::usearch::metric_kind_t metric_kind = distanceFunctionToMetricKind.at(index.arguments[1].safeGet<String>());
/// use defaults for the other parameters
unum::usearch::scalar_kind_t scalar_kind = unum::usearch::scalar_kind_t::f32_k;
UsearchHnswParams usearch_hnsw_params;
/// Optional parameters:
const bool has_six_args = (index.arguments.size() == 6);
if (has_six_args)
{
scalar_kind = quantizationToScalarKind.at(index.arguments[2].safeGet<String>());
@ -461,12 +485,16 @@ void vectorSimilarityIndexValidator(const IndexDescription & index, bool /* atta
{
if (!quantizationToScalarKind.contains(index.arguments[2].safeGet<String>()))
throw Exception(ErrorCodes::INCORRECT_DATA, "Third argument (quantization) of vector similarity index is not supported. Supported quantizations are: {}", joinByComma(quantizationToScalarKind));
if (index.arguments[3].safeGet<UInt64>() < 2)
throw Exception(ErrorCodes::INCORRECT_DATA, "Fourth argument (M) of vector similarity index must be > 1");
if (index.arguments[4].safeGet<UInt64>() < 1)
throw Exception(ErrorCodes::INCORRECT_DATA, "Fifth argument (ef_construction) of vector similarity index must be > 0");
if (index.arguments[5].safeGet<UInt64>() < 1)
throw Exception(ErrorCodes::INCORRECT_DATA, "Sixth argument (ef_search) of vector similarity index must be > 0");
/// Call Usearche's own parameter validation method for HNSW-specific parameters
UInt64 m = index.arguments[3].safeGet<UInt64>();
UInt64 ef_construction = index.arguments[4].safeGet<UInt64>();
UInt64 ef_search = index.arguments[5].safeGet<UInt64>();
unum::usearch::index_dense_config_t config(m, ef_construction, ef_search);
if (auto error = config.validate(); error)
throw Exception(ErrorCodes::INCORRECT_DATA, "Invalid parameters passed to vector similarity index. Error: {}", String(error.release()));
}
/// Check that the index is created on a single column

View File

@ -4,12 +4,9 @@
#if USE_USEARCH
#pragma clang diagnostic push
#pragma clang diagnostic ignored "-Wpass-failed"
# include <Storages/MergeTree/VectorSimilarityCondition.h>
# include <Common/Logger.h>
# include <usearch/index_dense.hpp>
#pragma clang diagnostic pop
#include <Storages/MergeTree/VectorSimilarityCondition.h>
#include <Common/Logger.h>
#include <usearch/index_dense.hpp>
namespace DB
{
@ -21,7 +18,7 @@ struct UsearchHnswParams
size_t ef_search = unum::usearch::default_expansion_search();
};
using USearchIndex = unum::usearch::index_dense_gt</*key_at*/ uint32_t, /*compressed_slot_at*/ uint32_t>;
using USearchIndex = unum::usearch::index_dense_t;
class USearchIndexWithSerialization : public USearchIndex
{
@ -41,13 +38,18 @@ public:
{
size_t max_level;
size_t connectivity;
size_t size;
size_t capacity;
size_t memory_usage;
/// advanced stats:
size_t size; /// number of indexed vectors
size_t capacity; /// reserved number of indexed vectors
size_t memory_usage; /// byte size (not exact)
size_t bytes_per_vector;
size_t scalar_words;
Base::stats_t statistics;
size_t nodes;
size_t edges;
size_t max_edges;
std::vector<USearchIndex::stats_t> level_stats; /// for debugging, excluded from getStatistics()
String toString() const;
};
Statistics getStatistics() const;

View File

@ -19,7 +19,6 @@ from env_helper import TEMP_PATH
from git_helper import git_runner, is_shallow
from github_helper import GitHub, PullRequest, PullRequests, Repository
from s3_helper import S3Helper
from get_robot_token import get_best_robot_token
from ci_utils import Shell
from version_helper import (
FILE_WITH_VERSION_PATH,
@ -172,7 +171,6 @@ def parse_args() -> argparse.Namespace:
parser.add_argument(
"--gh-user-or-token",
help="user name or GH token to authenticate",
default=get_best_robot_token(),
)
parser.add_argument(
"--gh-password",

View File

@ -484,7 +484,7 @@ class ReleaseInfo:
)
else:
if not dry_run:
assert not self.changelog_pr
assert not self.version_bump_pr
self.prs_merged = res

View File

@ -708,7 +708,7 @@ def test_no_key_found_disk(cluster, broken_s3):
"""
SELECT value
FROM system.metrics
WHERE metric = 'S3DiskNoKeyErrors'
WHERE metric = 'DiskS3NoSuchKeyErrors'
"""
).strip()
)

View File

@ -20,21 +20,30 @@ node_1_2 = cluster.add_instance("node_1_2", with_zookeeper=True)
node_2_1 = cluster.add_instance("node_2_1", with_zookeeper=True)
node_2_2 = cluster.add_instance("node_2_2", with_zookeeper=True)
# For test to be runnable multiple times
seqno = 0
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
@pytest.fixture(scope="function", autouse=True)
def create_tables():
global seqno
try:
seqno += 1
for shard in (1, 2):
for replica in (1, 2):
node = cluster.instances["node_{}_{}".format(shard, replica)]
node.query(
"""
CREATE TABLE replicated (d Date, x UInt32) ENGINE =
ReplicatedMergeTree('/clickhouse/tables/{shard}/replicated', '{instance}') PARTITION BY toYYYYMM(d) ORDER BY d""".format(
shard=shard, instance=node.name
)
f"CREATE TABLE replicated (d Date, x UInt32) ENGINE = "
f"ReplicatedMergeTree('/clickhouse/tables/{shard}/replicated_{seqno}', '{node.name}') PARTITION BY toYYYYMM(d) ORDER BY d"
)
node_1_1.query(
@ -42,10 +51,15 @@ CREATE TABLE replicated (d Date, x UInt32) ENGINE =
"Distributed('test_cluster', 'default', 'replicated')"
)
yield cluster
yield
finally:
cluster.shutdown()
node_1_1.query("DROP TABLE distributed")
node_1_1.query("DROP TABLE replicated")
node_1_2.query("DROP TABLE replicated")
node_2_1.query("DROP TABLE replicated")
node_2_2.query("DROP TABLE replicated")
def test(started_cluster):
@ -101,7 +115,9 @@ SELECT sum(x) FROM distributed WITH TOTALS SETTINGS
# allow pings to zookeeper to timeout (must be greater than ZK session timeout).
for _ in range(30):
try:
node_2_2.query("SELECT * FROM system.zookeeper where path = '/'")
node_2_2.query(
"SELECT * FROM system.zookeeper where path = '/' SETTINGS insert_keeper_max_retries = 0"
)
time.sleep(0.5)
except:
break
@ -120,7 +136,7 @@ SELECT sum(x) FROM distributed SETTINGS
== "3"
)
# Regression for skip_unavailable_shards in conjunction with skip_unavailable_shards
# Prefer fallback_to_stale_replicas over skip_unavailable_shards
assert (
instance_with_dist_table.query(
"""

View File

@ -464,7 +464,7 @@ def test_restart_broken(started_cluster):
"""
SELECT value
FROM system.metrics
WHERE metric = 'S3DiskNoKeyErrors'
WHERE metric = 'DiskS3NoSuchKeyErrors'
"""
).strip()
)

View File

@ -3,8 +3,6 @@ Two or six index arguments
2nd argument (distance function) must be String and L2Distance or cosineDistance
3nd argument (quantization), if given, must be String and f32, f16, ...
4nd argument (M), if given, must be UInt64 and > 1
5nd argument (ef_construction), if given, must be UInt64 and > 0
6nd argument (ef_search), if given, must be UInt64 and > 0
Must be created on single column
Must be created on Array(Float32) columns
Rejects INSERTs of Arrays with different sizes

View File

@ -27,12 +27,6 @@ CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similar
SELECT '4nd argument (M), if given, must be UInt64 and > 1';
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 'invalid', 1, 1)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY }
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 1, 1, 1)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA }
SELECT '5nd argument (ef_construction), if given, must be UInt64 and > 0';
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 2, 'invalid', 1)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY }
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 2, 0, 1)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA }
SELECT '6nd argument (ef_search), if given, must be UInt64 and > 0';
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 2, 1, 'invalid')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY }
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 2, 1, 0)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA }
SELECT 'Must be created on single column';
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx (vec, id) TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_NUMBER_OF_COLUMNS }

View File

@ -0,0 +1 @@
SYSTEM SYNC REPLICA db.`table` LIGHTWEIGHT

View File

@ -0,0 +1 @@
SELECT formatQuery('SYSTEM SYNC REPLICA db.table LIGHTWEIGHT');

View File

@ -38,3 +38,19 @@
7
8
9
15 \N 3 15 15 15 15
14 \N 2 10 10 10 154
13 \N 2 10 10 10 143
12 \N 2 10 10 10 14
11 \N 2 10 10 10 12
10 \N 2 10 10 10 10
9 \N 1 5 5 5 99
8 \N 1 5 5 5 88
7 \N 1 5 5 5 9
6 \N 1 5 5 5 7
5 \N 1 5 5 5 5
4 \N 0 0 0 0 44
3 \N 0 0 0 0 33
2 \N 0 0 0 0 4
1 \N 0 0 0 0 2
0 \N 0 0 0 0 0

View File

@ -2,3 +2,23 @@ SELECT lagInFrame(2::UInt128, 2, number) OVER w FROM numbers(10) WINDOW w AS (OR
SELECT leadInFrame(2::UInt128, 2, number) OVER w FROM numbers(10) WINDOW w AS (ORDER BY number);
SELECT lagInFrame(2::UInt64, 2, number) OVER w FROM numbers(10) WINDOW w AS (ORDER BY number);
SELECT leadInFrame(2::UInt64, 2, number) OVER w FROM numbers(10) WINDOW w AS (ORDER BY number);
SELECT
number,
YYYYMMDDToDate(1, toLowCardinality(11), max(YYYYMMDDToDate(YYYYMMDDToDate(toLowCardinality(1), 11, materialize(NULL), 19700101.1, 1, 27, 7, materialize(toUInt256(37)), 9, 19, 9), 1, toUInt128(11), NULL, 19700101.1, 1, 27, 7, 37, 9, 19, 9), toUInt256(30)) IGNORE NULLS OVER w, NULL, 19700101.1, toNullable(1), 27, materialize(7), 37, 9, 19, 9),
p,
pp,
lagInFrame(number, number - pp) OVER w AS lag2,
lagInFrame(number, number - pp, number * 11) OVER w AS lag,
leadInFrame(number, number - pp, number * 11) OVER w AS lead
FROM
(
SELECT
number,
intDiv(number, 5) AS p,
p * 5 AS pp
FROM numbers(16)
)
WHERE toLowCardinality(1)
WINDOW w AS (PARTITION BY p ORDER BY number ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)
ORDER BY number DESC NULLS LAST;