From b8dfc5b0f35a20f5049b1981d2885da2bae58242 Mon Sep 17 00:00:00 2001 From: Hammad Bashir Date: Tue, 26 Mar 2024 15:44:39 -0700 Subject: [PATCH 1/3] [CLN] Remove pulsar from python codebase (#1932) ## Description of changes *Summarize the changes made by this PR.* - Improvements & Bug fixes - Removes pulsar from the python codebase - New functionality - None ## Test plan *How are these changes tested?* - [x] Tests pass locally with `pytest` for python, `yarn test` for js, `cargo test` for rust ## Documentation Changes None --- chromadb/config.py | 4 - chromadb/ingest/impl/pulsar.py | 317 ------------------ chromadb/ingest/impl/pulsar_admin.py | 81 ----- chromadb/ingest/impl/utils.py | 4 - chromadb/segment/impl/distributed/server.py | 11 - chromadb/test/db/test_system.py | 10 +- .../test/ingest/test_producer_consumer.py | 22 +- chromadb/test/utils/test_messagid.py | 86 +---- chromadb/types.py | 6 +- chromadb/utils/messageid.py | 72 ---- pyproject.toml | 1 - requirements.txt | 3 +- 12 files changed, 17 insertions(+), 600 deletions(-) delete mode 100644 chromadb/ingest/impl/pulsar.py delete mode 100644 chromadb/ingest/impl/pulsar_admin.py diff --git a/chromadb/config.py b/chromadb/config.py index d0e6e45a00f7..597a338c814b 100644 --- a/chromadb/config.py +++ b/chromadb/config.py @@ -146,10 +146,6 @@ def empty_str_to_none(cls, v: str) -> Optional[str]: chroma_server_nofile: Optional[int] = None - pulsar_broker_url: Optional[str] = None - pulsar_admin_port: Optional[int] = 8080 - pulsar_broker_port: Optional[int] = 6650 - chroma_server_auth_provider: Optional[str] = None @validator("chroma_server_auth_provider", pre=True, always=True, allow_reuse=True) diff --git a/chromadb/ingest/impl/pulsar.py b/chromadb/ingest/impl/pulsar.py deleted file mode 100644 index d84cadfa01ea..000000000000 --- a/chromadb/ingest/impl/pulsar.py +++ /dev/null @@ -1,317 +0,0 @@ -from __future__ import annotations -from collections import defaultdict -from typing import Any, Callable, Dict, List, Optional, Sequence, Set, Tuple -import uuid -from chromadb.config import Settings, System -from chromadb.ingest import Consumer, ConsumerCallbackFn, Producer -from overrides import overrides, EnforceOverrides -from uuid import UUID -from chromadb.ingest.impl.pulsar_admin import PulsarAdmin -from chromadb.ingest.impl.utils import create_pulsar_connection_str -from chromadb.proto.convert import from_proto_submit, to_proto_submit -import chromadb.proto.chroma_pb2 as proto -from chromadb.telemetry.opentelemetry import ( - OpenTelemetryClient, - OpenTelemetryGranularity, - trace_method, -) -from chromadb.types import SeqId, SubmitEmbeddingRecord -import pulsar -from concurrent.futures import wait, Future - -from chromadb.utils.messageid import int_to_pulsar, pulsar_to_int - - -class PulsarProducer(Producer, EnforceOverrides): - # TODO: ensure trace context propagates - _connection_str: str - _topic_to_producer: Dict[str, pulsar.Producer] - _opentelemetry_client: OpenTelemetryClient - _client: pulsar.Client - _admin: PulsarAdmin - _settings: Settings - - def __init__(self, system: System) -> None: - pulsar_host = system.settings.require("pulsar_broker_url") - pulsar_port = system.settings.require("pulsar_broker_port") - self._connection_str = create_pulsar_connection_str(pulsar_host, pulsar_port) - self._topic_to_producer = {} - self._settings = system.settings - self._admin = PulsarAdmin(system) - self._opentelemetry_client = system.require(OpenTelemetryClient) - super().__init__(system) - - @overrides - def start(self) -> None: - self._client = pulsar.Client(self._connection_str) - super().start() - - @overrides - def stop(self) -> None: - self._client.close() - super().stop() - - @overrides - def create_topic(self, topic_name: str) -> None: - self._admin.create_topic(topic_name) - - @overrides - def delete_topic(self, topic_name: str) -> None: - self._admin.delete_topic(topic_name) - - @trace_method("PulsarProducer.submit_embedding", OpenTelemetryGranularity.ALL) - @overrides - def submit_embedding( - self, topic_name: str, embedding: SubmitEmbeddingRecord - ) -> SeqId: - """Add an embedding record to the given topic. Returns the SeqID of the record.""" - producer = self._get_or_create_producer(topic_name) - proto_submit: proto.SubmitEmbeddingRecord = to_proto_submit(embedding) - # TODO: batch performance / async - msg_id: pulsar.MessageId = producer.send(proto_submit.SerializeToString()) - return pulsar_to_int(msg_id) - - @trace_method("PulsarProducer.submit_embeddings", OpenTelemetryGranularity.ALL) - @overrides - def submit_embeddings( - self, topic_name: str, embeddings: Sequence[SubmitEmbeddingRecord] - ) -> Sequence[SeqId]: - if not self._running: - raise RuntimeError("Component not running") - - if len(embeddings) == 0: - return [] - - if len(embeddings) > self.max_batch_size: - raise ValueError( - f""" - Cannot submit more than {self.max_batch_size:,} embeddings at once. - Please submit your embeddings in batches of size - {self.max_batch_size:,} or less. - """ - ) - - producer = self._get_or_create_producer(topic_name) - protos_to_submit = [to_proto_submit(embedding) for embedding in embeddings] - - def create_producer_callback( - future: Future[int], - ) -> Callable[[Any, pulsar.MessageId], None]: - def producer_callback(res: Any, msg_id: pulsar.MessageId) -> None: - if msg_id: - future.set_result(pulsar_to_int(msg_id)) - else: - future.set_exception( - Exception( - "Unknown error while submitting embedding in producer_callback" - ) - ) - - return producer_callback - - futures = [] - for proto_to_submit in protos_to_submit: - future: Future[int] = Future() - producer.send_async( - proto_to_submit.SerializeToString(), - callback=create_producer_callback(future), - ) - futures.append(future) - - wait(futures) - - results: List[SeqId] = [] - for future in futures: - exception = future.exception() - if exception is not None: - raise exception - results.append(future.result()) - - return results - - @property - @overrides - def max_batch_size(self) -> int: - # For now, we use 1,000 - # TODO: tune this to a reasonable value by default - return 1000 - - def _get_or_create_producer(self, topic_name: str) -> pulsar.Producer: - if topic_name not in self._topic_to_producer: - producer = self._client.create_producer(topic_name) - self._topic_to_producer[topic_name] = producer - return self._topic_to_producer[topic_name] - - @overrides - def reset_state(self) -> None: - if not self._settings.require("allow_reset"): - raise ValueError( - "Resetting the database is not allowed. Set `allow_reset` to true in the config in tests or other non-production environments where reset should be permitted." - ) - for topic_name in self._topic_to_producer: - self._admin.delete_topic(topic_name) - self._topic_to_producer = {} - super().reset_state() - - -class PulsarConsumer(Consumer, EnforceOverrides): - class PulsarSubscription: - id: UUID - topic_name: str - start: int - end: int - callback: ConsumerCallbackFn - consumer: pulsar.Consumer - - def __init__( - self, - id: UUID, - topic_name: str, - start: int, - end: int, - callback: ConsumerCallbackFn, - consumer: pulsar.Consumer, - ): - self.id = id - self.topic_name = topic_name - self.start = start - self.end = end - self.callback = callback - self.consumer = consumer - - _connection_str: str - _client: pulsar.Client - _opentelemetry_client: OpenTelemetryClient - _subscriptions: Dict[str, Set[PulsarSubscription]] - _settings: Settings - - def __init__(self, system: System) -> None: - pulsar_host = system.settings.require("pulsar_broker_url") - pulsar_port = system.settings.require("pulsar_broker_port") - self._connection_str = create_pulsar_connection_str(pulsar_host, pulsar_port) - self._subscriptions = defaultdict(set) - self._settings = system.settings - self._opentelemetry_client = system.require(OpenTelemetryClient) - super().__init__(system) - - @overrides - def start(self) -> None: - self._client = pulsar.Client(self._connection_str) - super().start() - - @overrides - def stop(self) -> None: - self._client.close() - super().stop() - - @trace_method("PulsarConsumer.subscribe", OpenTelemetryGranularity.ALL) - @overrides - def subscribe( - self, - topic_name: str, - consume_fn: ConsumerCallbackFn, - start: Optional[SeqId] = None, - end: Optional[SeqId] = None, - id: Optional[UUID] = None, - ) -> UUID: - """Register a function that will be called to recieve embeddings for a given - topic. The given function may be called any number of times, with any number of - records, and may be called concurrently. - - Only records between start (exclusive) and end (inclusive) SeqIDs will be - returned. If start is None, the first record returned will be the next record - generated, not including those generated before creating the subscription. If - end is None, the consumer will consume indefinitely, otherwise it will - automatically be unsubscribed when the end SeqID is reached. - - If the function throws an exception, the function may be called again with the - same or different records. - - Takes an optional UUID as a unique subscription ID. If no ID is provided, a new - ID will be generated and returned.""" - if not self._running: - raise RuntimeError("Consumer must be started before subscribing") - - subscription_id = ( - id or uuid.uuid4() - ) # TODO: this should really be created by the coordinator and stored in sysdb - - start, end = self._validate_range(start, end) - - def wrap_callback(consumer: pulsar.Consumer, message: pulsar.Message) -> None: - msg_data = message.data() - msg_id = pulsar_to_int(message.message_id()) - submit_embedding_record = proto.SubmitEmbeddingRecord() - proto.SubmitEmbeddingRecord.ParseFromString( - submit_embedding_record, msg_data - ) - embedding_record = from_proto_submit(submit_embedding_record, msg_id) - consume_fn([embedding_record]) - consumer.acknowledge(message) - if msg_id == end: - self.unsubscribe(subscription_id) - - consumer = self._client.subscribe( - topic_name, - subscription_id.hex, - message_listener=wrap_callback, - ) - - subscription = self.PulsarSubscription( - subscription_id, topic_name, start, end, consume_fn, consumer - ) - self._subscriptions[topic_name].add(subscription) - - # NOTE: For some reason the seek() method expects a shadowed MessageId type - # which resides in _msg_id. - consumer.seek(int_to_pulsar(start)._msg_id) - - return subscription_id - - def _validate_range( - self, start: Optional[SeqId], end: Optional[SeqId] - ) -> Tuple[int, int]: - """Validate and normalize the start and end SeqIDs for a subscription using this - impl.""" - start = start or pulsar_to_int(pulsar.MessageId.latest) - end = end or self.max_seqid() - if not isinstance(start, int) or not isinstance(end, int): - raise TypeError("SeqIDs must be integers") - if start >= end: - raise ValueError(f"Invalid SeqID range: {start} to {end}") - return start, end - - @overrides - def unsubscribe(self, subscription_id: UUID) -> None: - """Unregister a subscription. The consume function will no longer be invoked, - and resources associated with the subscription will be released.""" - for topic_name, subscriptions in self._subscriptions.items(): - for subscription in subscriptions: - if subscription.id == subscription_id: - subscription.consumer.close() - subscriptions.remove(subscription) - if len(subscriptions) == 0: - del self._subscriptions[topic_name] - return - - @overrides - def min_seqid(self) -> SeqId: - """Return the minimum possible SeqID in this implementation.""" - return pulsar_to_int(pulsar.MessageId.earliest) - - @overrides - def max_seqid(self) -> SeqId: - """Return the maximum possible SeqID in this implementation.""" - return 2**192 - 1 - - @overrides - def reset_state(self) -> None: - if not self._settings.require("allow_reset"): - raise ValueError( - "Resetting the database is not allowed. Set `allow_reset` to true in the config in tests or other non-production environments where reset should be permitted." - ) - for topic_name, subscriptions in self._subscriptions.items(): - for subscription in subscriptions: - subscription.consumer.close() - self._subscriptions = defaultdict(set) - super().reset_state() diff --git a/chromadb/ingest/impl/pulsar_admin.py b/chromadb/ingest/impl/pulsar_admin.py deleted file mode 100644 index e031e4a238ba..000000000000 --- a/chromadb/ingest/impl/pulsar_admin.py +++ /dev/null @@ -1,81 +0,0 @@ -# A thin wrapper around the pulsar admin api -import requests -from chromadb.config import System -from chromadb.ingest.impl.utils import parse_topic_name - - -class PulsarAdmin: - """A thin wrapper around the pulsar admin api, only used for interim development towards distributed chroma. - This functionality will be moved to the chroma coordinator.""" - - _connection_str: str - - def __init__(self, system: System): - pulsar_host = system.settings.require("pulsar_broker_url") - pulsar_port = system.settings.require("pulsar_admin_port") - self._connection_str = f"http://{pulsar_host}:{pulsar_port}" - - # Create the default tenant and namespace - # This is a temporary workaround until we have a proper tenant/namespace management system - self.create_tenant("default") - self.create_namespace("default", "default") - - def create_tenant(self, tenant: str) -> None: - """Make a PUT request to the admin api to create the tenant""" - - path = f"/admin/v2/tenants/{tenant}" - url = self._connection_str + path - response = requests.put( - url, json={"allowedClusters": ["standalone"], "adminRoles": []} - ) # TODO: how to manage clusters? - - if response.status_code != 204 and response.status_code != 409: - raise RuntimeError(f"Failed to create tenant {tenant}") - - def create_namespace(self, tenant: str, namespace: str) -> None: - """Make a PUT request to the admin api to create the namespace""" - - path = f"/admin/v2/namespaces/{tenant}/{namespace}" - url = self._connection_str + path - response = requests.put(url) - - if response.status_code != 204 and response.status_code != 409: - raise RuntimeError(f"Failed to create namespace {namespace}") - - def create_topic(self, topic: str) -> None: - # TODO: support non-persistent topics? - tenant, namespace, topic_name = parse_topic_name(topic) - - if tenant != "default": - raise ValueError(f"Only the default tenant is supported, got {tenant}") - if namespace != "default": - raise ValueError( - f"Only the default namespace is supported, got {namespace}" - ) - - # Make a PUT request to the admin api to create the topic - path = f"/admin/v2/persistent/{tenant}/{namespace}/{topic_name}" - url = self._connection_str + path - response = requests.put(url) - - if response.status_code != 204 and response.status_code != 409: - raise RuntimeError(f"Failed to create topic {topic_name}") - - def delete_topic(self, topic: str) -> None: - tenant, namespace, topic_name = parse_topic_name(topic) - - if tenant != "default": - raise ValueError(f"Only the default tenant is supported, got {tenant}") - if namespace != "default": - raise ValueError( - f"Only the default namespace is supported, got {namespace}" - ) - - # Make a PUT request to the admin api to delete the topic - path = f"/admin/v2/persistent/{tenant}/{namespace}/{topic_name}" - # Force delete the topic - path += "?force=true" - url = self._connection_str + path - response = requests.delete(url) - if response.status_code != 204 and response.status_code != 409: - raise RuntimeError(f"Failed to delete topic {topic_name}") diff --git a/chromadb/ingest/impl/utils.py b/chromadb/ingest/impl/utils.py index 144384d75db5..34b46d3899a7 100644 --- a/chromadb/ingest/impl/utils.py +++ b/chromadb/ingest/impl/utils.py @@ -12,9 +12,5 @@ def parse_topic_name(topic_name: str) -> Tuple[str, str, str]: return match.group("tenant"), match.group("namespace"), match.group("topic") -def create_pulsar_connection_str(host: str, port: str) -> str: - return f"pulsar://{host}:{port}" - - def create_topic_name(tenant: str, namespace: str, topic: str) -> str: return f"persistent://{tenant}/{namespace}/{topic}" diff --git a/chromadb/segment/impl/distributed/server.py b/chromadb/segment/impl/distributed/server.py index 32bd1f67cfda..7b08e1e5d667 100644 --- a/chromadb/segment/impl/distributed/server.py +++ b/chromadb/segment/impl/distributed/server.py @@ -16,7 +16,6 @@ from chromadb.types import EmbeddingRecord from chromadb.segment.distributed import MemberlistProvider, Memberlist from chromadb.utils.rendezvous_hash import assign, murmur3hasher -from chromadb.ingest.impl.pulsar_admin import PulsarAdmin import logging import os @@ -51,7 +50,6 @@ def __init__(self, system: System) -> None: self._memberlist_provider = system.require(MemberlistProvider) self._memberlist_provider.set_memberlist_name("query-service-memberlist") self._assignment_policy = system.require(CollectionAssignmentPolicy) - self._create_pulsar_topics() self._consumer = system.require(Consumer) # Init data @@ -113,15 +111,6 @@ def _on_message(self, embedding_records: Sequence[EmbeddingRecord]) -> None: ) return None - def _create_pulsar_topics(self) -> None: - """This creates the pulsar topics used by the system. - HACK: THIS IS COMPLETELY A HACK AND WILL BE REPLACED - BY A PROPER TOPIC MANAGEMENT SYSTEM IN THE COORDINATOR""" - topics = self._assignment_policy.get_topics() - admin = PulsarAdmin(self._system) - for topic in topics: - admin.create_topic(topic) - def QueryVectors( self, request: proto.QueryVectorsRequest, context: Any ) -> proto.QueryVectorsResponse: diff --git a/chromadb/test/db/test_system.py b/chromadb/test/db/test_system.py index e899ac0b204a..e3a8a966bb0c 100644 --- a/chromadb/test/db/test_system.py +++ b/chromadb/test/db/test_system.py @@ -20,8 +20,8 @@ from pytest import FixtureRequest import uuid -PULSAR_TENANT = "default" -PULSAR_NAMESPACE = "default" +TENANT = "default" +NAMESPACE = "default" # These are the sample collections that are used in the tests below. Tests can override # the fields as needed. @@ -35,7 +35,7 @@ Collection( id=uuid.UUID(int=1), name="test_collection_1", - topic=f"persistent://{PULSAR_TENANT}/{PULSAR_NAMESPACE}/chroma_log_1", + topic=f"persistent://{TENANT}/{NAMESPACE}/chroma_log_1", metadata={"test_str": "str1", "test_int": 1, "test_float": 1.3}, dimension=128, database=DEFAULT_DATABASE, @@ -44,7 +44,7 @@ Collection( id=uuid.UUID(int=2), name="test_collection_2", - topic=f"persistent://{PULSAR_TENANT}/{PULSAR_NAMESPACE}/chroma_log_14", + topic=f"persistent://{TENANT}/{NAMESPACE}/chroma_log_14", metadata={"test_str": "str2", "test_int": 2, "test_float": 2.3}, dimension=None, database=DEFAULT_DATABASE, @@ -53,7 +53,7 @@ Collection( id=uuid.UUID(int=3), name="test_collection_3", - topic=f"persistent://{PULSAR_TENANT}/{PULSAR_NAMESPACE}/chroma_log_14", + topic=f"persistent://{TENANT}/{NAMESPACE}/chroma_log_14", metadata={"test_str": "str3", "test_int": 3, "test_float": 3.3}, dimension=None, database=DEFAULT_DATABASE, diff --git a/chromadb/test/ingest/test_producer_consumer.py b/chromadb/test/ingest/test_producer_consumer.py index 199afde60de3..31450cb7dfea 100644 --- a/chromadb/test/ingest/test_producer_consumer.py +++ b/chromadb/test/ingest/test_producer_consumer.py @@ -54,29 +54,11 @@ def sqlite_persistent() -> Generator[Tuple[Producer, Consumer], None, None]: shutil.rmtree(save_path) -def pulsar() -> Generator[Tuple[Producer, Consumer], None, None]: - """Fixture generator for pulsar Producer + Consumer. This fixture requires a running - pulsar cluster. You can use bin/cluster-test.sh to start a standalone pulsar and run this test. - Assumes pulsar_broker_url etc is set from the environment variables like PULSAR_BROKER_URL. - """ - system = System( - Settings( - allow_reset=True, - chroma_producer_impl="chromadb.ingest.impl.pulsar.PulsarProducer", - chroma_consumer_impl="chromadb.ingest.impl.pulsar.PulsarConsumer", - ) - ) - producer = system.require(Producer) - consumer = system.require(Consumer) - system.start() - yield producer, consumer - system.stop() - - def fixtures() -> List[Callable[[], Generator[Tuple[Producer, Consumer], None, None]]]: fixtures = [sqlite, sqlite_persistent] if "CHROMA_CLUSTER_TEST_ONLY" in os.environ: - fixtures = [pulsar] + # TODO: We should add the new log service here + fixtures = [] return fixtures diff --git a/chromadb/test/utils/test_messagid.py b/chromadb/test/utils/test_messagid.py index eff20a1b6fed..64d80e9b6b06 100644 --- a/chromadb/test/utils/test_messagid.py +++ b/chromadb/test/utils/test_messagid.py @@ -1,93 +1,19 @@ import chromadb.utils.messageid as mid -import pulsar import hypothesis.strategies as st -from hypothesis import given, settings, note -from typing import Any, Tuple +from hypothesis import given, settings @st.composite -def message_id(draw: st.DrawFn) -> pulsar.MessageId: - ledger_id = draw(st.integers(min_value=0, max_value=2**63 - 1)) - entry_id = draw(st.integers(min_value=0, max_value=2**63 - 1)) - batch_index = draw(st.integers(min_value=(2**31 - 1) * -1, max_value=2**31 - 1)) - partition = draw(st.integers(min_value=(2**31 - 1) * -1, max_value=2**31 - 1)) - return pulsar.MessageId(partition, ledger_id, entry_id, batch_index) +def message_id(draw: st.DrawFn) -> int: + offset_id = draw(st.integers(min_value=0, max_value=2**63 - 1)) + return offset_id @given(message_id=message_id()) @settings(max_examples=10000) # these are very fast and we want good coverage -def test_roundtrip_formats(message_id: pulsar.MessageId) -> None: - int1 = mid.pulsar_to_int(message_id) - - # Roundtrip int->string and back - str1 = mid.int_to_str(int1) - assert int1 == mid.str_to_int(str1) +def test_roundtrip_formats(message_id: int) -> None: + int1 = message_id # Roundtrip int->bytes and back b1 = mid.int_to_bytes(int1) assert int1 == mid.bytes_to_int(b1) - - # Roundtrip int -> MessageId and back - message_id_result = mid.int_to_pulsar(int1) - assert message_id_result.partition() == message_id.partition() - assert message_id_result.ledger_id() == message_id.ledger_id() - assert message_id_result.entry_id() == message_id.entry_id() - assert message_id_result.batch_index() == message_id.batch_index() - - -def assert_compare(pair1: Tuple[Any, Any], pair2: Tuple[Any, Any]) -> None: - """Helper function: assert that the two pairs of values always compare in the same - way across all comparisons and orderings.""" - - a, b = pair1 - c, d = pair2 - - try: - assert (a > b) == (c > d) - assert (a >= b) == (c >= d) - assert (a < b) == (c < d) - assert (a <= b) == (c <= d) - assert (a == b) == (c == d) - except AssertionError: - note(f"Failed to compare {a} and {b} with {c} and {d}") - note(f"type: {type(a)}") - raise - - -@given(m1=message_id(), m2=message_id()) -@settings(max_examples=10000) # these are very fast and we want good coverage -def test_messageid_comparison(m1: pulsar.MessageId, m2: pulsar.MessageId) -> None: - # MessageID comparison is broken in the Pulsar Python & CPP libraries: - # The partition field is not taken into account, and two MessageIDs with different - # partitions will compare inconsistently (m1 > m2 AND m2 > m1) - # To avoid this, we zero-out the partition field before testing. - m1 = pulsar.MessageId(0, m1.ledger_id(), m1.entry_id(), m1.batch_index()) - m2 = pulsar.MessageId(0, m2.ledger_id(), m2.entry_id(), m2.batch_index()) - - i1 = mid.pulsar_to_int(m1) - i2 = mid.pulsar_to_int(m2) - - # In python, MessageId objects are not comparable directory, but the - # internal generated native object is. - internal1 = m1._msg_id - internal2 = m2._msg_id - - s1 = mid.int_to_str(i1) - s2 = mid.int_to_str(i2) - - # assert that all strings, all ints, and all native objects compare the same - assert_compare((internal1, internal2), (i1, i2)) - assert_compare((internal1, internal2), (s1, s2)) - - -def test_max_values() -> None: - pulsar.MessageId(2**31 - 1, 2**63 - 1, 2**63 - 1, 2**31 - 1) - - -@given( - i1=st.integers(min_value=0, max_value=2**192 - 1), - i2=st.integers(min_value=0, max_value=2**192 - 1), -) -@settings(max_examples=10000) # these are very fast and we want good coverage -def test_string_comparison(i1: int, i2: int) -> None: - assert_compare((i1, i2), (mid.int_to_str(i1), mid.int_to_str(i2))) diff --git a/chromadb/types.py b/chromadb/types.py index fd66f12af6ca..96597e18033d 100644 --- a/chromadb/types.py +++ b/chromadb/types.py @@ -57,9 +57,9 @@ class Segment(TypedDict): # SeqID can be one of three types of value in our current and future plans: -# 1. A Pulsar MessageID encoded as a 192-bit integer -# 2. A Pulsar MessageIndex (a 64-bit integer) -# 3. A SQL RowID (a 64-bit integer) +# 1. A Pulsar MessageID encoded as a 192-bit integer - This is no longer used as we removed pulsar +# 2. A Pulsar MessageIndex (a 64-bit integer) - This is no longer used as we removed pulsar +# 3. A SQL RowID (a 64-bit integer) - This is used by both sqlite and the new log-service # All three of these types can be expressed as a Python int, so that is the type we # use in the internal Python API. However, care should be taken that the larger 192-bit diff --git a/chromadb/utils/messageid.py b/chromadb/utils/messageid.py index 9501f36c7598..2583a7b420c1 100644 --- a/chromadb/utils/messageid.py +++ b/chromadb/utils/messageid.py @@ -1,36 +1,3 @@ -import pulsar - - -def pulsar_to_int(message_id: pulsar.MessageId) -> int: - ledger_id: int = message_id.ledger_id() - entry_id: int = message_id.entry_id() - batch_index: int = message_id.batch_index() - partition: int = message_id.partition() - - # Convert to offset binary encoding to preserve ordering semantics when encoded - # see https://en.wikipedia.org/wiki/Offset_binary - ledger_id = ledger_id + 2**63 - entry_id = entry_id + 2**63 - batch_index = batch_index + 2**31 - partition = partition + 2**31 - - return ledger_id << 128 | entry_id << 64 | batch_index << 32 | partition - - -def int_to_pulsar(message_id: int) -> pulsar.MessageId: - partition = message_id & 0xFFFFFFFF - batch_index = message_id >> 32 & 0xFFFFFFFF - entry_id = message_id >> 64 & 0xFFFFFFFFFFFFFFFF - ledger_id = message_id >> 128 & 0xFFFFFFFFFFFFFFFF - - partition = partition - 2**31 - batch_index = batch_index - 2**31 - entry_id = entry_id - 2**63 - ledger_id = ledger_id - 2**63 - - return pulsar.MessageId(partition, ledger_id, entry_id, batch_index) - - def int_to_bytes(int: int) -> bytes: """Convert int to a 24 byte big endian byte string""" return int.to_bytes(24, "big") @@ -39,42 +6,3 @@ def int_to_bytes(int: int) -> bytes: def bytes_to_int(bytes: bytes) -> int: """Convert a 24 byte big endian byte string to an int""" return int.from_bytes(bytes, "big") - - -# Sorted in lexographic order -base85 = ( - "!#$%&()*+-0123456789;<=>?@ABCDEFGHIJKLMNOP" - + "QRSTUVWXYZ^_`abcdefghijklmnopqrstuvwxyz{|}~" -) - - -# not the most efficient way to do this, see benchmark function below -def _int_to_str(n: int) -> str: - if n < 85: - return base85[n] - else: - return _int_to_str(n // 85) + base85[n % 85] - - -def int_to_str(n: int) -> str: - return _int_to_str(n).rjust(36, "!") # left pad with '!' to 36 chars - - -def str_to_int(s: str) -> int: - return sum(base85.index(c) * 85**i for i, c in enumerate(s[::-1])) - - -# 1m in 5 seconds on a M1 Pro -# Not fast, but not likely to be a bottleneck either -def _benchmark() -> None: - import random - import time - - t0 = time.time() - for i in range(1000000): - x = random.randint(0, 2**192 - 1) - s = int_to_str(x) - if s == "!": # prevent compiler from optimizing out - print("oops") - t1 = time.time() - print(t1 - t0) diff --git a/pyproject.toml b/pyproject.toml index d425e77952d7..8e5c29527e24 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -24,7 +24,6 @@ dependencies = [ 'numpy >= 1.22.5', 'posthog >= 2.4.0', 'typing_extensions >= 4.5.0', - 'pulsar-client>=3.1.0', 'onnxruntime >= 1.14.1', 'opentelemetry-api>=1.2.0', 'opentelemetry-exporter-otlp-proto-grpc>=1.2.0', diff --git a/requirements.txt b/requirements.txt index 0ed94e5033ba..02e7c2a62bbb 100644 --- a/requirements.txt +++ b/requirements.txt @@ -12,9 +12,9 @@ opentelemetry-api>=1.2.0 opentelemetry-exporter-otlp-proto-grpc>=1.2.0 opentelemetry-instrumentation-fastapi>=0.41b0 opentelemetry-sdk>=1.2.0 +orjson>=3.9.12 overrides>=7.3.1 posthog>=2.4.0 -pulsar-client>=3.1.0 pydantic>=1.9 pypika>=0.48.9 PyYAML>=6.0.0 @@ -25,4 +25,3 @@ tqdm>=4.65.0 typer>=0.9.0 typing_extensions>=4.5.0 uvicorn[standard]>=0.18.3 -orjson>=3.9.12 From 739e942b165dd7fae7888ea34656d79e584e045f Mon Sep 17 00:00:00 2001 From: Liquan Pei Date: Tue, 26 Mar 2024 17:01:24 -0700 Subject: [PATCH 2/3] [ENH] DataChunk abstraction with PullLog, Group and BF (#1929) ## Description of changes *Summarize the changes made by this PR.* - Improvements & Bug fixes - ... - New functionality - This PR adds DataChunk abstraction - PullLog, Group and BF are reimplemented with DataChunk - Compaction orchestrator is implemented with DataChunk ## Test plan *How are these changes tested?* - [ ] Tests pass locally with `pytest` for python, `yarn test` for js, `cargo test` for rust ## Documentation Changes *Are all docstrings for user-facing APIs updated if required? Do we need to make documentation changes in the [docs repository](https://github.com/chroma-core/docs)?* --------- Co-authored-by: Hammad Bashir --- .vscode/settings.json | 2 +- rust/worker/src/compactor/scheduler.rs | 8 +- rust/worker/src/execution/data/data_chunk.rs | 170 ++++++++++++++ rust/worker/src/execution/data/mod.rs | 1 + rust/worker/src/execution/mod.rs | 1 + .../execution/operators/brute_force_knn.rs | 186 +++++++++++++-- rust/worker/src/execution/operators/mod.rs | 1 + .../src/execution/operators/partition.rs | 221 ++++++++++++++++++ .../src/execution/operators/pull_log.rs | 29 +-- .../src/execution/orchestration/compact.rs | 221 ++++++++++++++++++ .../src/execution/orchestration/hnsw.rs | 21 +- .../worker/src/execution/orchestration/mod.rs | 2 +- rust/worker/src/log/log.rs | 8 +- rust/worker/src/types/embedding_record.rs | 8 +- 14 files changed, 816 insertions(+), 63 deletions(-) create mode 100644 rust/worker/src/execution/data/data_chunk.rs create mode 100644 rust/worker/src/execution/data/mod.rs create mode 100644 rust/worker/src/execution/operators/partition.rs create mode 100644 rust/worker/src/execution/orchestration/compact.rs diff --git a/.vscode/settings.json b/.vscode/settings.json index ccddc8d4c8c1..a5def08ba630 100644 --- a/.vscode/settings.json +++ b/.vscode/settings.json @@ -128,4 +128,4 @@ "unordered_set": "cpp", "algorithm": "cpp" }, -} +} \ No newline at end of file diff --git a/rust/worker/src/compactor/scheduler.rs b/rust/worker/src/compactor/scheduler.rs index bd51ed0320b0..1ec961a077f9 100644 --- a/rust/worker/src/compactor/scheduler.rs +++ b/rust/worker/src/compactor/scheduler.rs @@ -281,7 +281,7 @@ mod tests { collection_id: collection_id_1.clone(), log_id: 1, log_id_ts: 1, - record: Box::new(EmbeddingRecord { + record: EmbeddingRecord { id: "embedding_id_1".to_string(), seq_id: BigInt::from(1), embedding: None, @@ -289,7 +289,7 @@ mod tests { metadata: None, operation: Operation::Add, collection_id: collection_uuid_1, - }), + }, }), ); @@ -301,7 +301,7 @@ mod tests { collection_id: collection_id_2.clone(), log_id: 2, log_id_ts: 2, - record: Box::new(EmbeddingRecord { + record: EmbeddingRecord { id: "embedding_id_2".to_string(), seq_id: BigInt::from(2), embedding: None, @@ -309,7 +309,7 @@ mod tests { metadata: None, operation: Operation::Add, collection_id: collection_uuid_2, - }), + }, }), ); diff --git a/rust/worker/src/execution/data/data_chunk.rs b/rust/worker/src/execution/data/data_chunk.rs new file mode 100644 index 000000000000..5f13d57cb2a1 --- /dev/null +++ b/rust/worker/src/execution/data/data_chunk.rs @@ -0,0 +1,170 @@ +use std::sync::Arc; + +use crate::types::EmbeddingRecord; + +#[derive(Clone, Debug)] +pub(crate) struct DataChunk { + data: Arc<[EmbeddingRecord]>, + visibility: Arc<[bool]>, +} + +impl DataChunk { + pub fn new(data: Arc<[EmbeddingRecord]>) -> Self { + let len = data.len(); + DataChunk { + data, + visibility: vec![true; len].into(), + } + } + + /// Returns the total length of the data chunk + pub fn total_len(&self) -> usize { + self.data.len() + } + + /// Returns the number of visible elements in the data chunk + pub fn len(&self) -> usize { + self.visibility.iter().filter(|&v| *v).count() + } + + /// Returns the element at the given index + /// if the index is out of bounds, it returns None + /// # Arguments + /// * `index` - The index of the element + pub fn get(&self, index: usize) -> Option<&EmbeddingRecord> { + if index < self.data.len() { + Some(&self.data[index]) + } else { + None + } + } + + /// Returns the visibility of the element at the given index + /// if the index is out of bounds, it returns None + /// # Arguments + /// * `index` - The index of the element + pub fn get_visibility(&self, index: usize) -> Option { + if index < self.visibility.len() { + Some(self.visibility[index]) + } else { + None + } + } + + /// Sets the visibility of the elements in the data chunk. + /// Note that the length of the visibility vector should be + /// equal to the length of the data chunk. + /// + /// Note that this is the only way to change the visibility of the elements in the data chunk, + /// the data chunk does not provide a way to change the visibility of individual elements. + /// This is to ensure that the visibility of the elements is always in sync with the data. + /// If you want to change the visibility of individual elements, you should create a new data chunk. + /// + /// # Arguments + /// * `visibility` - A vector of boolean values indicating the visibility of the elements + pub fn set_visibility(&mut self, visibility: Vec) { + self.visibility = visibility.into(); + } + + /// Returns an iterator over the visible elements in the data chunk + /// The iterator returns a tuple of the element and its index + /// # Returns + /// An iterator over the visible elements in the data chunk + pub fn iter(&self) -> DataChunkIteraror<'_> { + DataChunkIteraror { + chunk: self, + index: 0, + } + } +} + +pub(crate) struct DataChunkIteraror<'a> { + chunk: &'a DataChunk, + index: usize, +} + +impl<'a> Iterator for DataChunkIteraror<'a> { + type Item = (&'a EmbeddingRecord, usize); + + fn next(&mut self) -> Option { + while self.index < self.chunk.total_len() { + let index = self.index; + match self.chunk.get_visibility(index) { + Some(true) => { + self.index += 1; + return self.chunk.get(index).map(|record| (record, index)); + } + Some(false) => { + self.index += 1; + } + None => { + break; + } + } + } + None + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::types::EmbeddingRecord; + use crate::types::Operation; + use num_bigint::BigInt; + use std::str::FromStr; + use uuid::Uuid; + + #[test] + fn test_data_chunk() { + let collection_uuid_1 = Uuid::from_str("00000000-0000-0000-0000-000000000001").unwrap(); + let data = vec![ + EmbeddingRecord { + id: "embedding_id_1".to_string(), + seq_id: BigInt::from(1), + embedding: None, + encoding: None, + metadata: None, + operation: Operation::Add, + collection_id: collection_uuid_1, + }, + EmbeddingRecord { + id: "embedding_id_2".to_string(), + seq_id: BigInt::from(2), + embedding: None, + encoding: None, + metadata: None, + operation: Operation::Add, + collection_id: collection_uuid_1, + }, + ]; + let data = data.into(); + let mut chunk = DataChunk::new(data); + assert_eq!(chunk.len(), 2); + let mut iter = chunk.iter(); + let elem = iter.next(); + assert_eq!(elem.is_some(), true); + let (record, index) = elem.unwrap(); + assert_eq!(record.id, "embedding_id_1"); + assert_eq!(index, 0); + let elem = iter.next(); + assert_eq!(elem.is_some(), true); + let (record, index) = elem.unwrap(); + assert_eq!(record.id, "embedding_id_2"); + assert_eq!(index, 1); + let elem = iter.next(); + assert_eq!(elem.is_none(), true); + + let visibility = vec![true, false].into(); + chunk.set_visibility(visibility); + assert_eq!(chunk.len(), 1); + let mut iter = chunk.iter(); + let elem = iter.next(); + assert_eq!(elem.is_some(), true); + let (record, index) = elem.unwrap(); + assert_eq!(record.id, "embedding_id_1"); + assert_eq!(index, 0); + let elem = iter.next(); + assert_eq!(elem.is_none(), true); + } +} diff --git a/rust/worker/src/execution/data/mod.rs b/rust/worker/src/execution/data/mod.rs new file mode 100644 index 000000000000..ecbe39f34453 --- /dev/null +++ b/rust/worker/src/execution/data/mod.rs @@ -0,0 +1 @@ +pub(crate) mod data_chunk; diff --git a/rust/worker/src/execution/mod.rs b/rust/worker/src/execution/mod.rs index 0000e23f3a3c..1d361780d77f 100644 --- a/rust/worker/src/execution/mod.rs +++ b/rust/worker/src/execution/mod.rs @@ -1,4 +1,5 @@ pub(crate) mod config; +mod data; pub(crate) mod dispatcher; pub(crate) mod operator; mod operators; diff --git a/rust/worker/src/execution/operators/brute_force_knn.rs b/rust/worker/src/execution/operators/brute_force_knn.rs index 3a9d05c84260..13e02dc9af92 100644 --- a/rust/worker/src/execution/operators/brute_force_knn.rs +++ b/rust/worker/src/execution/operators/brute_force_knn.rs @@ -1,6 +1,10 @@ +use crate::execution::data::data_chunk::DataChunk; use crate::{distance::DistanceFunction, execution::operator::Operator}; use async_trait::async_trait; -use std::cmp; +use std::cmp::Ord; +use std::cmp::Ordering; +use std::cmp::PartialOrd; +use std::collections::BinaryHeap; /// The brute force k-nearest neighbors operator is responsible for computing the k-nearest neighbors /// of a given query vector against a set of vectors using brute force calculation. @@ -17,7 +21,7 @@ pub struct BruteForceKnnOperator {} /// * `distance_metric` - The distance metric to use. #[derive(Debug)] pub struct BruteForceKnnOperatorInput { - pub data: Vec>, + pub data: DataChunk, pub query: Vec, pub k: usize, pub distance_metric: DistanceFunction, @@ -25,37 +29,95 @@ pub struct BruteForceKnnOperatorInput { /// The output of the brute force k-nearest neighbors operator. /// # Parameters +/// * `data` - The vectors to query against. Only the vectors that are nearest neighbors are visible. /// * `indices` - The indices of the nearest neighbors. This is a mask against the `query_vecs` input. /// One row for each query vector. /// * `distances` - The distances of the nearest neighbors. /// One row for each query vector. #[derive(Debug)] pub struct BruteForceKnnOperatorOutput { + pub data: DataChunk, pub indices: Vec, pub distances: Vec, } pub type BruteForceKnnOperatorResult = Result; +#[derive(Debug)] +struct Entry { + index: usize, + distance: f32, +} + +impl Ord for Entry { + fn cmp(&self, other: &Self) -> Ordering { + if self.distance == other.distance { + Ordering::Equal + } else if self.distance > other.distance { + // This is a min heap, so we need to reverse the ordering. + Ordering::Less + } else { + // This is a min heap, so we need to reverse the ordering. + Ordering::Greater + } + } +} + +impl PartialOrd for Entry { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.cmp(other)) + } +} + +impl PartialEq for Entry { + fn eq(&self, other: &Self) -> bool { + self.distance == other.distance + } +} + +impl Eq for Entry {} + #[async_trait] impl Operator for BruteForceKnnOperator { type Error = (); async fn run(&self, input: &BruteForceKnnOperatorInput) -> BruteForceKnnOperatorResult { - // We could use a heap approach here, but for now we just sort the distances and take the - // first k. - let mut sorted_indices_distances = input - .data - .iter() - .map(|data| input.distance_metric.distance(&input.query, data)) - .enumerate() - .collect::>(); - sorted_indices_distances.sort_by(|a, b| a.1.partial_cmp(&b.1).unwrap()); - let (sorted_indices, sorted_distances) = sorted_indices_distances - .drain(..cmp::min(input.k, input.data.len())) - .unzip(); + let mut heap = BinaryHeap::with_capacity(input.k); + let data_chunk = &input.data; + for data in data_chunk.iter() { + let embedding_record = data.0; + let index = data.1; + let embedding = match &embedding_record.embedding { + Some(embedding) => embedding, + None => { + continue; + } + }; + let distance = input.distance_metric.distance(&embedding[..], &input.query); + heap.push(Entry { index, distance }); + } + + let mut visibility = vec![false; data_chunk.total_len()]; + let mut sorted_indices = Vec::with_capacity(input.k); + let mut sorted_distances = Vec::with_capacity(input.k); + let mut i = 0; + while i < input.k { + let entry = match heap.pop() { + Some(entry) => entry, + None => { + break; + } + }; + sorted_indices.push(entry.index); + sorted_distances.push(entry.distance); + visibility[entry.index] = true; + i += 1; + } + let mut data_chunk = data_chunk.clone(); + data_chunk.set_visibility(visibility); Ok(BruteForceKnnOperatorOutput { + data: data_chunk, indices: sorted_indices, distances: sorted_distances, }) @@ -64,17 +126,49 @@ impl Operator for Brute #[cfg(test)] mod tests { + use crate::types::EmbeddingRecord; + use crate::types::Operation; + use num_bigint::BigInt; + use uuid::Uuid; + use super::*; #[tokio::test] async fn test_brute_force_knn_l2sqr() { let operator = BruteForceKnnOperator {}; + let data = vec![ + EmbeddingRecord { + id: "embedding_id_1".to_string(), + seq_id: BigInt::from(0), + embedding: Some(vec![0.0, 0.0, 0.0]), + encoding: None, + metadata: None, + operation: Operation::Add, + collection_id: Uuid::new_v4(), + }, + EmbeddingRecord { + id: "embedding_id_2".to_string(), + seq_id: BigInt::from(1), + embedding: Some(vec![0.0, 1.0, 1.0]), + encoding: None, + metadata: None, + operation: Operation::Add, + collection_id: Uuid::new_v4(), + }, + EmbeddingRecord { + id: "embedding_id_3".to_string(), + seq_id: BigInt::from(2), + embedding: Some(vec![7.0, 8.0, 9.0]), + encoding: None, + metadata: None, + operation: Operation::Add, + collection_id: Uuid::new_v4(), + }, + ]; + let data_chunk = DataChunk::new(data.into()); + let input = BruteForceKnnOperatorInput { - data: vec![ - vec![0.0, 0.0, 0.0], - vec![0.0, 1.0, 1.0], - vec![7.0, 8.0, 9.0], - ], + data: data_chunk, query: vec![0.0, 0.0, 0.0], k: 2, distance_metric: DistanceFunction::Euclidean, @@ -83,6 +177,9 @@ mod tests { assert_eq!(output.indices, vec![0, 1]); let distance_1 = 0.0_f32.powi(2) + 1.0_f32.powi(2) + 1.0_f32.powi(2); assert_eq!(output.distances, vec![0.0, distance_1]); + assert_eq!(output.data.get_visibility(0), Some(true)); + assert_eq!(output.data.get_visibility(1), Some(true)); + assert_eq!(output.data.get_visibility(2), Some(false)); } #[tokio::test] @@ -95,8 +192,39 @@ mod tests { let norm_2 = (0.0_f32.powi(2) + -1.0_f32.powi(2) + 6.0_f32.powi(2)).sqrt(); let data_2 = vec![0.0 / norm_2, -1.0 / norm_2, 6.0 / norm_2]; + let data = vec![ + EmbeddingRecord { + id: "embedding_id_1".to_string(), + seq_id: BigInt::from(0), + embedding: Some(vec![0.0, 1.0, 0.0]), + encoding: None, + metadata: None, + operation: Operation::Add, + collection_id: Uuid::new_v4(), + }, + EmbeddingRecord { + id: "embedding_id_2".to_string(), + seq_id: BigInt::from(1), + embedding: Some(data_1.clone()), + encoding: None, + metadata: None, + operation: Operation::Add, + collection_id: Uuid::new_v4(), + }, + EmbeddingRecord { + id: "embedding_id_3".to_string(), + seq_id: BigInt::from(2), + embedding: Some(data_2.clone()), + encoding: None, + metadata: None, + operation: Operation::Add, + collection_id: Uuid::new_v4(), + }, + ]; + let data_chunk = DataChunk::new(data.into()); + let input = BruteForceKnnOperatorInput { - data: vec![vec![0.0, 1.0, 0.0], data_1.clone(), data_2.clone()], + data: data_chunk, query: vec![0.0, 1.0, 0.0], k: 2, distance_metric: DistanceFunction::InnerProduct, @@ -107,14 +235,29 @@ mod tests { let expected_distance_1 = 1.0f32 - ((data_1[0] * 0.0) + (data_1[1] * 1.0) + (data_1[2] * 0.0)); assert_eq!(output.distances, vec![0.0, expected_distance_1]); + assert_eq!(output.data.get_visibility(0), Some(true)); + assert_eq!(output.data.get_visibility(1), Some(true)); + assert_eq!(output.data.get_visibility(2), Some(false)); } #[tokio::test] async fn test_data_less_than_k() { // If we have less data than k, we should return all the data, sorted by distance. let operator = BruteForceKnnOperator {}; + let data = vec![EmbeddingRecord { + id: "embedding_id_1".to_string(), + seq_id: BigInt::from(0), + embedding: Some(vec![0.0, 0.0, 0.0]), + encoding: None, + metadata: None, + operation: Operation::Add, + collection_id: Uuid::new_v4(), + }]; + + let data_chunk = DataChunk::new(data.into()); + let input = BruteForceKnnOperatorInput { - data: vec![vec![0.0, 0.0, 0.0]], + data: data_chunk, query: vec![0.0, 0.0, 0.0], k: 2, distance_metric: DistanceFunction::Euclidean, @@ -122,5 +265,6 @@ mod tests { let output = operator.run(&input).await.unwrap(); assert_eq!(output.indices, vec![0]); assert_eq!(output.distances, vec![0.0]); + assert_eq!(output.data.get_visibility(0), Some(true)); } } diff --git a/rust/worker/src/execution/operators/mod.rs b/rust/worker/src/execution/operators/mod.rs index 60198481545e..ed31dca33db7 100644 --- a/rust/worker/src/execution/operators/mod.rs +++ b/rust/worker/src/execution/operators/mod.rs @@ -1,3 +1,4 @@ pub(super) mod brute_force_knn; pub(super) mod normalize_vectors; +pub(super) mod partition; pub(super) mod pull_log; diff --git a/rust/worker/src/execution/operators/partition.rs b/rust/worker/src/execution/operators/partition.rs new file mode 100644 index 000000000000..e32b693ff73b --- /dev/null +++ b/rust/worker/src/execution/operators/partition.rs @@ -0,0 +1,221 @@ +use crate::errors::{ChromaError, ErrorCodes}; +use crate::execution::data::data_chunk::DataChunk; +use crate::execution::operator::Operator; +use async_trait::async_trait; +use std::collections::HashMap; +use thiserror::Error; + +#[derive(Debug)] +/// The partition Operator takes a DataChunk and presents a copy-free +/// view of N partitions by breaking the data into partitions by max_partition_size. It will group operations +/// on the same key into the same partition. Due to this, the max_partition_size is a +/// soft-limit, since if there are more operations to a key than max_partition_size we cannot +/// partition the data. +pub struct PartitionOperator {} + +/// The input to the partition operator. +/// # Parameters +/// * `records` - The records to partition. +#[derive(Debug)] +pub struct PartitionInput { + pub(crate) records: DataChunk, + pub(crate) max_partition_size: usize, +} + +impl PartitionInput { + /// Create a new partition input. + /// # Parameters + /// * `records` - The records to partition. + /// * `max_partition_size` - The maximum size of a partition. Since we are trying to + /// partition the records by id, which can casue the partition size to be larger than this + /// value. + pub fn new(records: DataChunk, max_partition_size: usize) -> Self { + PartitionInput { + records, + max_partition_size, + } + } +} + +/// The output of the partition operator. +/// # Parameters +/// * `records` - The partitioned records. +#[derive(Debug)] +pub struct PartitionOutput { + pub(crate) records: Vec, +} + +#[derive(Debug, Error)] +pub enum PartitionError { + #[error("Failed to partition records.")] + PartitionError, +} + +impl ChromaError for PartitionError { + fn code(&self) -> ErrorCodes { + match self { + PartitionError::PartitionError => ErrorCodes::Internal, + } + } +} + +pub type PartitionResult = Result; + +impl PartitionOperator { + pub fn new() -> Box { + Box::new(PartitionOperator {}) + } + + pub fn partition(&self, records: &DataChunk, partition_size: usize) -> Vec { + let mut map = HashMap::new(); + for data in records.iter() { + let record = data.0; + let index = data.1; + let key = record.id.clone(); + map.entry(key).or_insert_with(Vec::new).push(index); + } + let mut result = Vec::new(); + // Create a new DataChunk for each parition of records with partition_size without + // data copying. + let mut current_batch_size = 0; + let mut new_partition = true; + let mut visibility = vec![false; records.total_len()]; + for (_, v) in map.iter() { + // create DataChunk with partition_size by masking the visibility of the records + // in the partition. + if new_partition { + visibility = vec![false; records.total_len()]; + new_partition = false; + } + for i in v.iter() { + visibility[*i] = true; + } + current_batch_size += v.len(); + if current_batch_size >= partition_size { + let mut new_data_chunk = records.clone(); + new_data_chunk.set_visibility(visibility.clone()); + result.push(new_data_chunk); + new_partition = true; + current_batch_size = 0; + } + } + // handle the case that the last group is smaller than the group_size. + if !new_partition { + let mut new_data_chunk = records.clone(); + new_data_chunk.set_visibility(visibility.clone()); + result.push(new_data_chunk); + } + result + } + + fn determine_partition_size(&self, num_records: usize, threshold: usize) -> usize { + if num_records < threshold { + return num_records; + } else { + return threshold; + } + } +} + +#[async_trait] +impl Operator for PartitionOperator { + type Error = PartitionError; + + async fn run(&self, input: &PartitionInput) -> PartitionResult { + let records = &input.records; + let partition_size = self.determine_partition_size(records.len(), input.max_partition_size); + let deduped_records = self.partition(records, partition_size); + return Ok(PartitionOutput { + records: deduped_records, + }); + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::types::EmbeddingRecord; + use crate::types::Operation; + use num_bigint::BigInt; + use std::str::FromStr; + use std::sync::Arc; + use uuid::Uuid; + + #[tokio::test] + async fn test_partition_operator() { + let collection_uuid_1 = Uuid::from_str("00000000-0000-0000-0000-000000000001").unwrap(); + let collection_uuid_2 = Uuid::from_str("00000000-0000-0000-0000-000000000002").unwrap(); + let data = vec![ + EmbeddingRecord { + id: "embedding_id_1".to_string(), + seq_id: BigInt::from(1), + embedding: None, + encoding: None, + metadata: None, + operation: Operation::Add, + collection_id: collection_uuid_1, + }, + EmbeddingRecord { + id: "embedding_id_2".to_string(), + seq_id: BigInt::from(2), + embedding: None, + encoding: None, + metadata: None, + operation: Operation::Add, + collection_id: collection_uuid_1, + }, + EmbeddingRecord { + id: "embedding_id_1".to_string(), + seq_id: BigInt::from(3), + embedding: None, + encoding: None, + metadata: None, + operation: Operation::Add, + collection_id: collection_uuid_2, + }, + ]; + let data: Arc<[EmbeddingRecord]> = data.into(); + + // Test group size is larger than the number of records + let chunk = DataChunk::new(data.clone()); + let operator = PartitionOperator::new(); + let input = PartitionInput::new(chunk, 4); + let result = operator.run(&input).await.unwrap(); + assert_eq!(result.records.len(), 1); + assert_eq!(result.records[0].len(), 3); + + // Test group size is the same as the number of records + let chunk = DataChunk::new(data.clone()); + let operator = PartitionOperator::new(); + let input = PartitionInput::new(chunk, 3); + let result = operator.run(&input).await.unwrap(); + assert_eq!(result.records.len(), 1); + assert_eq!(result.records[0].len(), 3); + + // Test group size is smaller than the number of records + let chunk = DataChunk::new(data.clone()); + let operator = PartitionOperator::new(); + let input = PartitionInput::new(chunk, 2); + let mut result = operator.run(&input).await.unwrap(); + + // The result can be 1 or 2 groups depending on the order of the records. + assert!(result.records.len() == 2 || result.records.len() == 1); + if result.records.len() == 2 { + result.records.sort_by(|a, b| a.len().cmp(&b.len())); + assert_eq!(result.records[0].len(), 1); + assert_eq!(result.records[1].len(), 2); + } else { + assert_eq!(result.records[0].len(), 3); + } + + // Test group size is smaller than the number of records + let chunk = DataChunk::new(data.clone()); + let operator = PartitionOperator::new(); + let input = PartitionInput::new(chunk, 1); + let mut result = operator.run(&input).await.unwrap(); + assert_eq!(result.records.len(), 2); + result.records.sort_by(|a, b| a.len().cmp(&b.len())); + assert_eq!(result.records[0].len(), 1); + assert_eq!(result.records[1].len(), 2); + } +} diff --git a/rust/worker/src/execution/operators/pull_log.rs b/rust/worker/src/execution/operators/pull_log.rs index ad5646626548..d0a9cc7ae61d 100644 --- a/rust/worker/src/execution/operators/pull_log.rs +++ b/rust/worker/src/execution/operators/pull_log.rs @@ -1,8 +1,7 @@ -use crate::{ - execution::operator::Operator, - log::log::{Log, PullLogsError}, - types::EmbeddingRecord, -}; +use crate::execution::data::data_chunk::DataChunk; +use crate::execution::operator::Operator; +use crate::log::log::Log; +use crate::log::log::PullLogsError; use async_trait::async_trait; use uuid::Uuid; @@ -65,22 +64,22 @@ impl PullLogsInput { /// The output of the pull logs operator. #[derive(Debug)] pub struct PullLogsOutput { - logs: Vec>, + logs: DataChunk, } impl PullLogsOutput { /// Create a new pull logs output. /// # Parameters /// * `logs` - The logs that were read. - pub fn new(logs: Vec>) -> Self { + pub fn new(logs: DataChunk) -> Self { PullLogsOutput { logs } } /// Get the log entries that were read by an invocation of the pull logs operator. /// # Returns /// The log entries that were read. - pub fn logs(&self) -> &Vec> { - &self.logs + pub fn logs(&self) -> DataChunk { + self.logs.clone() } } @@ -139,7 +138,9 @@ impl Operator for PullLogsOperator { if input.num_records.is_some() && result.len() > input.num_records.unwrap() as usize { result.truncate(input.num_records.unwrap() as usize); } - Ok(PullLogsOutput::new(result)) + // Convert to DataChunk + let data_chunk = DataChunk::new(result.into()); + Ok(PullLogsOutput::new(data_chunk)) } } @@ -166,7 +167,7 @@ mod tests { collection_id: collection_id_1.clone(), log_id: 1, log_id_ts: 1, - record: Box::new(EmbeddingRecord { + record: EmbeddingRecord { id: "embedding_id_1".to_string(), seq_id: BigInt::from(1), embedding: None, @@ -174,7 +175,7 @@ mod tests { metadata: None, operation: Operation::Add, collection_id: collection_uuid_1, - }), + }, }), ); log.add_log( @@ -183,7 +184,7 @@ mod tests { collection_id: collection_id_1.clone(), log_id: 2, log_id_ts: 2, - record: Box::new(EmbeddingRecord { + record: EmbeddingRecord { id: "embedding_id_2".to_string(), seq_id: BigInt::from(2), embedding: None, @@ -191,7 +192,7 @@ mod tests { metadata: None, operation: Operation::Add, collection_id: collection_uuid_1, - }), + }, }), ); diff --git a/rust/worker/src/execution/orchestration/compact.rs b/rust/worker/src/execution/orchestration/compact.rs new file mode 100644 index 000000000000..bce69aa59e5a --- /dev/null +++ b/rust/worker/src/execution/orchestration/compact.rs @@ -0,0 +1,221 @@ +use super::super::operator::{wrap, TaskMessage}; +use crate::errors::ChromaError; +use crate::execution::data::data_chunk::DataChunk; +use crate::execution::operators::partition::PartitionInput; +use crate::execution::operators::partition::PartitionOperator; +use crate::execution::operators::partition::PartitionResult; +use crate::execution::operators::pull_log::PullLogsInput; +use crate::execution::operators::pull_log::PullLogsOperator; +use crate::execution::operators::pull_log::PullLogsResult; +use crate::log::log::Log; +use crate::sysdb::sysdb::SysDb; +use crate::system::Component; +use crate::system::Handler; +use crate::system::Receiver; +use crate::system::System; +use async_trait::async_trait; +use std::time::SystemTime; +use std::time::UNIX_EPOCH; +use uuid::Uuid; + +/** The state of the orchestrator. +In chroma, we have a relatively fixed number of query plans that we can execute. Rather +than a flexible state machine abstraction, we just manually define the states that we +expect to encounter for a given query plan. This is a bit more rigid, but it's also simpler and easier to +understand. We can always add more abstraction later if we need it. +```plaintext + + ┌───► Write─────-------┐ + │ │ + Pending ─► PullLogs ─► Group │ ├─► Flush ─► Finished + │ │ + └───► Write ───────────┘ + +``` +*/ +#[derive(Debug)] +enum ExecutionState { + Pending, + PullLogs, + Partition, + Write, + Flush, + Finished, +} + +#[derive(Debug)] +pub struct CompactOrchestrator { + state: ExecutionState, + // Component Execution + system: System, + segment_id: Uuid, + // Dependencies + log: Box, + sysdb: Box, + // Dispatcher + dispatcher: Box>, + // Result Channel + result_channel: Option>>>, +} + +impl CompactOrchestrator { + pub fn new( + system: System, + segment_id: Uuid, + log: Box, + sysdb: Box, + dispatcher: Box>, + result_channel: Option>>>, + ) -> Self { + CompactOrchestrator { + state: ExecutionState::Pending, + system, + segment_id, + log, + sysdb, + dispatcher, + result_channel, + } + } + + /// Get the collection id for a segment id. + /// TODO: This can be cached + async fn get_collection_id_for_segment_id(&mut self, segment_id: Uuid) -> Option { + let segments = self + .sysdb + .get_segments(Some(segment_id), None, None, None, None) + .await; + match segments { + Ok(segments) => match segments.get(0) { + Some(segment) => segment.collection, + None => None, + }, + Err(e) => { + // Log an error and return + return None; + } + } + } + + async fn pull_logs(&mut self, self_address: Box>) { + self.state = ExecutionState::PullLogs; + let operator = PullLogsOperator::new(self.log.clone()); + let collection_id = match self.get_collection_id_for_segment_id(self.segment_id).await { + Some(collection_id) => collection_id, + None => { + // Log an error and reply + return + return; + } + }; + let end_timestamp = SystemTime::now().duration_since(UNIX_EPOCH); + let end_timestamp = match end_timestamp { + // TODO: change protobuf definition to use u64 instead of i64 + Ok(end_timestamp) => end_timestamp.as_secs() as i64, + Err(e) => { + // Log an error and reply + return + return; + } + }; + let input = PullLogsInput::new(collection_id, 0, 100, None, Some(end_timestamp)); + let task = wrap(operator, input, self_address); + match self.dispatcher.send(task).await { + Ok(_) => (), + Err(e) => { + // TODO: log an error and reply to caller + } + } + } + + async fn group( + &mut self, + records: DataChunk, + self_address: Box>, + ) { + self.state = ExecutionState::Partition; + // TODO: make this configurable + let max_partition_size = 100; + let operator = PartitionOperator::new(); + let input = PartitionInput::new(records, max_partition_size); + let task = wrap(operator, input, self_address); + match self.dispatcher.send(task).await { + Ok(_) => (), + Err(e) => { + // TODO: log an error and reply to caller + } + } + } + + async fn write(&mut self, records: Vec) { + self.state = ExecutionState::Write; + + for record in records { + // TODO: implement write + } + } +} + +// ============== Component Implementation ============== + +#[async_trait] +impl Component for CompactOrchestrator { + fn queue_size(&self) -> usize { + 1000 // TODO: make configurable + } + + async fn on_start(&mut self, ctx: &crate::system::ComponentContext) -> () { + self.pull_logs(ctx.sender.as_receiver()).await; + } +} + +// ============== Handlers ============== +#[async_trait] +impl Handler for CompactOrchestrator { + async fn handle( + &mut self, + message: PullLogsResult, + ctx: &crate::system::ComponentContext, + ) { + let records = match message { + Ok(result) => result.logs(), + Err(e) => { + // Log an error and return + let result_channel = match self.result_channel.take() { + Some(tx) => tx, + None => { + // Log an error + return; + } + }; + let _ = result_channel.send(Err(Box::new(e))); + return; + } + }; + self.group(records, ctx.sender.as_receiver()).await; + } +} + +#[async_trait] +impl Handler for CompactOrchestrator { + async fn handle( + &mut self, + message: PartitionResult, + ctx: &crate::system::ComponentContext, + ) { + let records = match message { + Ok(result) => result.records, + Err(e) => { + // Log an error and return + let result_channel = match self.result_channel.take() { + Some(tx) => tx, + None => { + // Log an error + return; + } + }; + let _ = result_channel.send(Err(Box::new(e))); + return; + } + }; + // TODO: implement write records + } +} diff --git a/rust/worker/src/execution/orchestration/hnsw.rs b/rust/worker/src/execution/orchestration/hnsw.rs index 699bfde4b3f0..579497cc03b7 100644 --- a/rust/worker/src/execution/orchestration/hnsw.rs +++ b/rust/worker/src/execution/orchestration/hnsw.rs @@ -1,11 +1,9 @@ use super::super::operator::{wrap, TaskMessage}; -use super::super::operators::pull_log::{PullLogsInput, PullLogsOperator, PullLogsOutput}; -use crate::distance; +use super::super::operators::pull_log::{PullLogsInput, PullLogsOperator}; use crate::distance::DistanceFunction; use crate::errors::ChromaError; use crate::execution::operators::brute_force_knn::{ - BruteForceKnnOperator, BruteForceKnnOperatorInput, BruteForceKnnOperatorOutput, - BruteForceKnnOperatorResult, + BruteForceKnnOperator, BruteForceKnnOperatorInput, BruteForceKnnOperatorResult, }; use crate::execution::operators::pull_log::PullLogsResult; use crate::sysdb::sysdb::SysDb; @@ -30,7 +28,7 @@ understand. We can always add more abstraction later if we need it. ┌───► Brute Force ─────┐ │ │ - Pending ─► PullLogs ─► Dedupe│ ├─► MergeResults ─► Finished + Pending ─► PullLogs ─► Group│ ├─► MergeResults ─► Finished │ │ └───► HNSW ────────────┘ @@ -40,7 +38,7 @@ understand. We can always add more abstraction later if we need it. enum ExecutionState { Pending, PullLogs, - Dedupe, + Partition, QueryKnn, MergeResults, Finished, @@ -142,7 +140,7 @@ impl HnswQueryOrchestrator { /// Run the orchestrator and return the result. /// # Note /// Use this over spawning the component directly. This method will start the component and - /// wait for it to finish before returning the result. + /// wait for it to finish before returning the result. pub(crate) async fn run(mut self) -> Result>, Box> { let (tx, rx) = tokio::sync::oneshot::channel(); self.result_channel = Some(tx); @@ -175,19 +173,14 @@ impl Handler for HnswQueryOrchestrator { message: PullLogsResult, ctx: &crate::system::ComponentContext, ) { - self.state = ExecutionState::Dedupe; + self.state = ExecutionState::Partition; // TODO: implement the remaining state transitions and operators // TODO: don't need all this cloning and data shuffling, once we land the chunk abstraction - let mut dataset = Vec::new(); match message { Ok(logs) => { - for log in logs.logs().iter() { - // TODO: only adds have embeddings, unwrap is fine for now - dataset.push(log.embedding.clone().unwrap()); - } let bf_input = BruteForceKnnOperatorInput { - data: dataset, + data: logs.logs(), query: self.query_vectors[0].clone(), k: self.k as usize, distance_metric: DistanceFunction::Euclidean, diff --git a/rust/worker/src/execution/orchestration/mod.rs b/rust/worker/src/execution/orchestration/mod.rs index 902c3eaf84dd..2828cfd365a9 100644 --- a/rust/worker/src/execution/orchestration/mod.rs +++ b/rust/worker/src/execution/orchestration/mod.rs @@ -1,3 +1,3 @@ +mod compact; mod hnsw; - pub(crate) use hnsw::*; diff --git a/rust/worker/src/log/log.rs b/rust/worker/src/log/log.rs index 5dae688e6d35..0f1c5c6c16c7 100644 --- a/rust/worker/src/log/log.rs +++ b/rust/worker/src/log/log.rs @@ -38,7 +38,7 @@ pub(crate) trait Log: Send + Sync + LogClone + Debug { offset: i64, batch_size: i32, end_timestamp: Option, - ) -> Result>, PullLogsError>; + ) -> Result, PullLogsError>; async fn get_collections_with_new_data( &mut self, @@ -121,7 +121,7 @@ impl Log for GrpcLog { offset: i64, batch_size: i32, end_timestamp: Option, - ) -> Result>, PullLogsError> { + ) -> Result, PullLogsError> { let end_timestamp = match end_timestamp { Some(end_timestamp) => end_timestamp, None => -1, @@ -227,7 +227,7 @@ pub(crate) struct LogRecord { pub(crate) collection_id: String, pub(crate) log_id: i64, pub(crate) log_id_ts: i64, - pub(crate) record: Box, + pub(crate) record: EmbeddingRecord, } impl Debug for LogRecord { @@ -268,7 +268,7 @@ impl Log for InMemoryLog { offset: i64, batch_size: i32, end_timestamp: Option, - ) -> Result>, PullLogsError> { + ) -> Result, PullLogsError> { let end_timestamp = match end_timestamp { Some(end_timestamp) => end_timestamp, None => i64::MAX, diff --git a/rust/worker/src/types/embedding_record.rs b/rust/worker/src/types/embedding_record.rs index cc53631d4bf4..6ba9bdf255ac 100644 --- a/rust/worker/src/types/embedding_record.rs +++ b/rust/worker/src/types/embedding_record.rs @@ -101,7 +101,7 @@ impl TryFrom for EmbeddingRecord { } } -impl TryFrom for Box { +impl TryFrom for EmbeddingRecord { type Error = EmbeddingRecordConversionError; fn try_from(record_log: RecordLog) -> Result { @@ -143,7 +143,7 @@ impl TryFrom for Box { None => None, }; - Ok(Box::new(EmbeddingRecord { + Ok(EmbeddingRecord { id: proto_submit.id, seq_id: seq_id, embedding: embedding, @@ -151,7 +151,7 @@ impl TryFrom for Box { metadata: metadata, operation: op, collection_id: collection_uuid, - })) + }) } } @@ -364,7 +364,7 @@ mod tests { log_id: 42, record: Some(proto_submit), }; - let converted_embedding_record = Box::::try_from(record_log).unwrap(); + let converted_embedding_record = EmbeddingRecord::try_from(record_log).unwrap(); assert_eq!(converted_embedding_record.id, Uuid::nil().to_string()); assert_eq!(converted_embedding_record.seq_id, BigInt::from(42)); assert_eq!( From 1ce93c7e41344446dc18af7b27474b5d5bcd4a68 Mon Sep 17 00:00:00 2001 From: Hammad Bashir Date: Wed, 27 Mar 2024 10:48:48 -0700 Subject: [PATCH 3/3] [CLN] Rename SubmitEmbeddingRecord to OperationRecord. Remove Topic concept from Segment/Collection. (#1933) ## Description of changes This PR has two high-level aims - Clean up artifacts of our design that were implicitly coupled to pulsar. Namely topics and multiplexing. - Begin a rename of SubmitEmbeddingRecord, EmbeddingRecord to names that more accurately reflect their intents. I apologize for how large this PR is, but in practice, breaking something up like this is not really feasible AFAICT, unless we allow test-breaking stacked PRs... *Summarize the changes made by this PR.* - Improvements & Bug fixes - Renames SubmitEmbeddingRecord to OperationRecord in order to more correctly identify what it is - a record of an Operation (future PRs will rename EmbeddingRecord as well to LogRecord to make its intent clearer). - An OperationRecord does not need to store collection_id. This was an artifact of the pulsar log when we needed to demux data. We now improve the Ingest interface by presenting producers/consumers over logical log streams by collection. - Remove the concept of topic from the Producer/Consumer interfaces - it is no longer needed in a post pulsar-world. This also means Collection/Segment don't need to store Topic. - Removed the AssignmentPolicy concept. This only existed for multiplexing - which is not a concept without pulsar. - Update the Rust code with the topic field removed and with the OperationRecord naming. - Update Go code with the SysDB changes (No assignment policy + no log) no as well as the OperationRecord naming. - New functionality - None ## Test plan *How are these changes tested?* - [ ] Tests pass locally with `pytest` for python, `yarn test` for js, `cargo test` for rust ## Documentation Changes *Are all docstrings for user-facing APIs updated if required? Do we need to make documentation changes in the [docs repository](https://github.com/chroma-core/docs)?* --- chromadb/api/segment.py | 27 +- chromadb/config.py | 9 +- chromadb/db/impl/grpc/client.py | 21 - chromadb/db/impl/grpc/server.py | 18 +- chromadb/db/mixins/embeddings_queue.py | 47 +- chromadb/db/mixins/sysdb.py | 54 +- chromadb/db/system.py | 14 +- chromadb/ingest/__init__.py | 34 +- chromadb/ingest/impl/simple_policy.py | 61 --- chromadb/ingest/impl/utils.py | 5 +- chromadb/logservice/logservice.py | 54 +- .../sysdb/00005-remove-topic.sqlite.sql | 4 + chromadb/proto/chroma_pb2.py | 80 +-- chromadb/proto/chroma_pb2.pyi | 20 +- chromadb/proto/convert.py | 15 +- chromadb/proto/logservice_pb2.py | 36 +- chromadb/proto/logservice_pb2.pyi | 10 +- chromadb/segment/impl/distributed/server.py | 171 ------ chromadb/segment/impl/manager/distributed.py | 3 +- chromadb/segment/impl/manager/local.py | 46 +- chromadb/segment/impl/metadata/sqlite.py | 13 +- chromadb/segment/impl/vector/local_hnsw.py | 8 +- chromadb/test/conftest.py | 25 +- chromadb/test/db/test_system.py | 55 +- .../test/ingest/test_producer_consumer.py | 115 ++-- chromadb/test/property/strategies.py | 6 +- chromadb/test/segment/test_metadata.py | 175 +++--- chromadb/test/segment/test_vector.py | 146 +++-- chromadb/types.py | 7 +- go/cmd/coordinator/cmd.go | 1 - go/go.mod | 22 +- go/go.sum | 23 + go/migrations/20240327075032.sql | 4 + go/migrations/atlas.sum | 3 +- go/pkg/common/errors.go | 2 - go/pkg/coordinator/apis.go | 18 +- go/pkg/coordinator/apis_test.go | 175 ++---- go/pkg/coordinator/assignment_policy.go | 77 --- go/pkg/coordinator/coordinator.go | 17 +- go/pkg/coordinator/grpc/collection_service.go | 5 +- .../grpc/collection_service_test.go | 20 +- .../coordinator/grpc/proto_model_convert.go | 3 - .../grpc/proto_model_convert_test.go | 5 - go/pkg/coordinator/grpc/segment_service.go | 11 +- go/pkg/coordinator/grpc/server.go | 21 +- .../grpc/tenant_database_service_test.go | 6 +- go/pkg/logservice/grpc/record_log_service.go | 7 +- .../grpc/record_log_service_test.go | 41 +- .../testutils/record_log_test_util.go | 5 +- go/pkg/metastore/catalog.go | 5 +- .../metastore/coordinator/model_db_convert.go | 2 - .../coordinator/model_db_convert_test.go | 6 - go/pkg/metastore/coordinator/table_catalog.go | 35 +- .../coordinator/table_catalog_test.go | 18 +- go/pkg/metastore/db/dao/collection.go | 18 +- go/pkg/metastore/db/dao/collection_test.go | 26 +- go/pkg/metastore/db/dao/segment.go | 32 +- go/pkg/metastore/db/dao/segment_test.go | 30 +- go/pkg/metastore/db/dao/test_utils.go | 12 +- go/pkg/metastore/db/dbmodel/collection.go | 3 +- .../db/dbmodel/mocks/ICollectionDb.go | 18 +- .../metastore/db/dbmodel/mocks/ISegmentDb.go | 18 +- go/pkg/metastore/db/dbmodel/segment.go | 8 +- go/pkg/metastore/mocks/Catalog.go | 36 +- go/pkg/model/collection.go | 8 +- go/pkg/model/segment.go | 8 - go/pkg/proto/coordinatorpb/chroma.pb.go | 373 ++++++------- go/pkg/proto/coordinatorpb/coordinator.pb.go | 508 ++++++++---------- go/pkg/proto/logservicepb/logservice.pb.go | 153 +++--- idl/chromadb/proto/chroma.proto | 9 +- idl/chromadb/proto/coordinator.proto | 7 - idl/chromadb/proto/logservice.proto | 4 +- rust/worker/src/compactor/scheduler.rs | 11 +- .../src/execution/operators/pull_log.rs | 7 +- .../src/execution/orchestration/compact.rs | 2 +- .../src/execution/orchestration/hnsw.rs | 2 +- rust/worker/src/ingest/ingest.rs | 418 -------------- rust/worker/src/ingest/mod.rs | 6 - rust/worker/src/ingest/scheduler.rs | 213 -------- rust/worker/src/lib.rs | 30 +- rust/worker/src/log/log.rs | 13 +- rust/worker/src/segment/segment_manager.rs | 1 - rust/worker/src/sysdb/sysdb.rs | 6 - rust/worker/src/types/collection.rs | 4 - rust/worker/src/types/embedding_record.rs | 49 +- rust/worker/src/types/segment.rs | 4 - 86 files changed, 1196 insertions(+), 2652 deletions(-) delete mode 100644 chromadb/ingest/impl/simple_policy.py create mode 100644 chromadb/migrations/sysdb/00005-remove-topic.sqlite.sql delete mode 100644 chromadb/segment/impl/distributed/server.py create mode 100644 go/migrations/20240327075032.sql delete mode 100644 go/pkg/coordinator/assignment_policy.go delete mode 100644 rust/worker/src/ingest/ingest.rs delete mode 100644 rust/worker/src/ingest/scheduler.rs diff --git a/chromadb/api/segment.py b/chromadb/api/segment.py index f92e7607d6d9..ce19f0a379c3 100644 --- a/chromadb/api/segment.py +++ b/chromadb/api/segment.py @@ -52,7 +52,6 @@ ) import chromadb.types as t - from typing import Any, Optional, Sequence, Generator, List, cast, Set, Dict from overrides import override from uuid import UUID, uuid4 @@ -123,10 +122,12 @@ def create_database(self, name: str, tenant: str = DEFAULT_TENANT) -> None: name=name, tenant=tenant, ) + @trace_method("SegmentAPI.get_database", OpenTelemetryGranularity.OPERATION) @override def get_database(self, name: str, tenant: str = DEFAULT_TENANT) -> t.Database: return self._sysdb.get_database(name=name, tenant=tenant) + @trace_method("SegmentAPI.create_tenant", OpenTelemetryGranularity.OPERATION) @override def create_tenant(self, name: str) -> None: @@ -136,6 +137,7 @@ def create_tenant(self, name: str) -> None: self._sysdb.create_tenant( name=name, ) + @trace_method("SegmentAPI.get_tenant", OpenTelemetryGranularity.OPERATION) @override def get_tenant(self, name: str) -> t.Tenant: @@ -374,7 +376,6 @@ def _add( for r in _records( t.Operation.ADD, ids=ids, - collection_id=collection_id, embeddings=embeddings, metadatas=metadatas, documents=documents, @@ -382,7 +383,7 @@ def _add( ): self._validate_embedding_record(coll, r) records_to_submit.append(r) - self._producer.submit_embeddings(coll["topic"], records_to_submit) + self._producer.submit_embeddings(collection_id, records_to_submit) self._product_telemetry_client.capture( CollectionAddEvent( @@ -417,7 +418,6 @@ def _update( for r in _records( t.Operation.UPDATE, ids=ids, - collection_id=collection_id, embeddings=embeddings, metadatas=metadatas, documents=documents, @@ -425,7 +425,7 @@ def _update( ): self._validate_embedding_record(coll, r) records_to_submit.append(r) - self._producer.submit_embeddings(coll["topic"], records_to_submit) + self._producer.submit_embeddings(collection_id, records_to_submit) self._product_telemetry_client.capture( CollectionUpdateEvent( @@ -462,7 +462,6 @@ def _upsert( for r in _records( t.Operation.UPSERT, ids=ids, - collection_id=collection_id, embeddings=embeddings, metadatas=metadatas, documents=documents, @@ -470,7 +469,7 @@ def _upsert( ): self._validate_embedding_record(coll, r) records_to_submit.append(r) - self._producer.submit_embeddings(coll["topic"], records_to_submit) + self._producer.submit_embeddings(collection_id, records_to_submit) return True @@ -632,12 +631,10 @@ def _delete( return [] records_to_submit = [] - for r in _records( - operation=t.Operation.DELETE, ids=ids_to_delete, collection_id=collection_id - ): + for r in _records(operation=t.Operation.DELETE, ids=ids_to_delete): self._validate_embedding_record(coll, r) records_to_submit.append(r) - self._producer.submit_embeddings(coll["topic"], records_to_submit) + self._producer.submit_embeddings(collection_id, records_to_submit) self._product_telemetry_client.capture( CollectionDeleteEvent( @@ -803,7 +800,7 @@ def max_batch_size(self) -> int: # used for channel assignment in the distributed version of the system. @trace_method("SegmentAPI._validate_embedding_record", OpenTelemetryGranularity.ALL) def _validate_embedding_record( - self, collection: t.Collection, record: t.SubmitEmbeddingRecord + self, collection: t.Collection, record: t.OperationRecord ) -> None: """Validate the dimension of an embedding record before submitting it to the system.""" add_attributes_to_current_span({"collection_id": str(collection["id"])}) @@ -845,12 +842,11 @@ def _get_collection(self, collection_id: UUID) -> t.Collection: def _records( operation: t.Operation, ids: IDs, - collection_id: UUID, embeddings: Optional[Embeddings] = None, metadatas: Optional[Metadatas] = None, documents: Optional[Documents] = None, uris: Optional[URIs] = None, -) -> Generator[t.SubmitEmbeddingRecord, None, None]: +) -> Generator[t.OperationRecord, None, None]: """Convert parallel lists of embeddings, metadatas and documents to a sequence of SubmitEmbeddingRecords""" @@ -877,13 +873,12 @@ def _records( else: metadata = {"chroma:uri": uri} - record = t.SubmitEmbeddingRecord( + record = t.OperationRecord( id=id, embedding=embeddings[i] if embeddings else None, encoding=t.ScalarEncoding.FLOAT32, # Hardcode for now metadata=metadata, operation=operation, - collection_id=collection_id, ) yield record diff --git a/chromadb/config.py b/chromadb/config.py index 597a338c814b..7fd8e6d89812 100644 --- a/chromadb/config.py +++ b/chromadb/config.py @@ -71,7 +71,6 @@ "chromadb.ingest.Producer": "chroma_producer_impl", "chromadb.ingest.Consumer": "chroma_consumer_impl", "chromadb.quota.QuotaProvider": "chroma_quota_provider_impl", - "chromadb.ingest.CollectionAssignmentPolicy": "chroma_collection_assignment_policy_impl", # noqa "chromadb.db.system.SysDB": "chroma_sysdb_impl", "chromadb.segment.SegmentManager": "chroma_segment_manager_impl", "chromadb.segment.distributed.SegmentDirectory": "chroma_segment_directory_impl", @@ -86,9 +85,12 @@ class Settings(BaseSettings): # type: ignore environment: str = "" - # Legacy config has to be kept around because pydantic will error + # Legacy config that has to be kept around because pydantic will error # on nonexisting keys chroma_db_impl: Optional[str] = None + chroma_collection_assignment_policy_impl: str = ( + "chromadb.ingest.impl.simple_policy.SimpleAssignmentPolicy" + ) # Can be "chromadb.api.segment.SegmentAPI" or "chromadb.api.fastapi.FastAPI" chroma_api_impl: str = "chromadb.api.segment.SegmentAPI" chroma_product_telemetry_impl: str = "chromadb.telemetry.product.posthog.Posthog" @@ -109,9 +111,6 @@ class Settings(BaseSettings): # type: ignore # Distributed architecture specific components chroma_segment_directory_impl: str = "chromadb.segment.impl.distributed.segment_directory.RendezvousHashSegmentDirectory" chroma_memberlist_provider_impl: str = "chromadb.segment.impl.distributed.segment_directory.CustomResourceMemberlistProvider" - chroma_collection_assignment_policy_impl: str = ( - "chromadb.ingest.impl.simple_policy.SimpleAssignmentPolicy" - ) worker_memberlist_name: str = "query-service-memberlist" chroma_coordinator_host = "localhost" diff --git a/chromadb/db/impl/grpc/client.py b/chromadb/db/impl/grpc/client.py index f51249eab0ff..ebdf59445e8d 100644 --- a/chromadb/db/impl/grpc/client.py +++ b/chromadb/db/impl/grpc/client.py @@ -28,7 +28,6 @@ UpdateSegmentRequest, ) from chromadb.proto.coordinator_pb2_grpc import SysDBStub -from chromadb.telemetry.opentelemetry import OpenTelemetryClient from chromadb.telemetry.opentelemetry.grpc import OtelInterceptor from chromadb.types import ( Collection, @@ -145,14 +144,12 @@ def get_segments( id: Optional[UUID] = None, type: Optional[str] = None, scope: Optional[SegmentScope] = None, - topic: Optional[str] = None, collection: Optional[UUID] = None, ) -> Sequence[Segment]: request = GetSegmentsRequest( id=id.hex if id else None, type=type, scope=to_proto_segment_scope(scope) if scope else None, - topic=topic, collection=collection.hex if collection else None, ) response = self._sys_db_stub.GetSegments(request) @@ -166,14 +163,9 @@ def get_segments( def update_segment( self, id: UUID, - topic: OptionalArgument[Optional[str]] = Unspecified(), collection: OptionalArgument[Optional[UUID]] = Unspecified(), metadata: OptionalArgument[Optional[UpdateMetadata]] = Unspecified(), ) -> None: - write_topic = None - if topic != Unspecified(): - write_topic = cast(Union[str, None], topic) - write_collection = None if collection != Unspecified(): write_collection = cast(Union[UUID, None], collection) @@ -184,17 +176,12 @@ def update_segment( request = UpdateSegmentRequest( id=id.hex, - topic=write_topic, collection=write_collection.hex if write_collection else None, metadata=to_proto_update_metadata(write_metadata) if write_metadata else None, ) - if topic is None: - request.ClearField("topic") - request.reset_topic = True - if collection is None: request.ClearField("collection") request.reset_collection = True @@ -252,7 +239,6 @@ def delete_collection( def get_collections( self, id: Optional[UUID] = None, - topic: Optional[str] = None, name: Optional[str] = None, tenant: str = DEFAULT_TENANT, database: str = DEFAULT_DATABASE, @@ -262,7 +248,6 @@ def get_collections( # TODO: implement limit and offset in the gRPC service request = GetCollectionsRequest( id=id.hex if id else None, - topic=topic, name=name, tenant=tenant, database=database, @@ -277,15 +262,10 @@ def get_collections( def update_collection( self, id: UUID, - topic: OptionalArgument[str] = Unspecified(), name: OptionalArgument[str] = Unspecified(), dimension: OptionalArgument[Optional[int]] = Unspecified(), metadata: OptionalArgument[Optional[UpdateMetadata]] = Unspecified(), ) -> None: - write_topic = None - if topic != Unspecified(): - write_topic = cast(str, topic) - write_name = None if name != Unspecified(): write_name = cast(str, name) @@ -300,7 +280,6 @@ def update_collection( request = UpdateCollectionRequest( id=id.hex, - topic=write_topic, name=write_name, dimension=write_dimension, metadata=to_proto_update_metadata(write_metadata) diff --git a/chromadb/db/impl/grpc/server.py b/chromadb/db/impl/grpc/server.py index 13344eba8c80..cdec22eff0d5 100644 --- a/chromadb/db/impl/grpc/server.py +++ b/chromadb/db/impl/grpc/server.py @@ -2,7 +2,6 @@ from typing import Any, Dict, cast from uuid import UUID from overrides import overrides -from chromadb.ingest import CollectionAssignmentPolicy from chromadb.config import DEFAULT_DATABASE, DEFAULT_TENANT, Component, System from chromadb.proto.convert import ( from_proto_metadata, @@ -38,7 +37,7 @@ UpdateCollectionRequest, UpdateCollectionResponse, UpdateSegmentRequest, - UpdateSegmentResponse + UpdateSegmentResponse, ) from chromadb.proto.coordinator_pb2_grpc import ( SysDBServicer, @@ -55,7 +54,6 @@ class GrpcMockSysDB(SysDBServicer, Component): _server: grpc.Server _server_port: int - _assignment_policy: CollectionAssignmentPolicy _segments: Dict[str, Segment] = {} _tenants_to_databases_to_collections: Dict[ str, Dict[str, Dict[str, Collection]] @@ -64,7 +62,6 @@ class GrpcMockSysDB(SysDBServicer, Component): def __init__(self, system: System): self._server_port = system.settings.require("chroma_server_grpc_port") - self._assignment_policy = system.instance(CollectionAssignmentPolicy) return super().__init__(system) @overrides @@ -203,7 +200,6 @@ def GetSegments( if request.HasField("scope") else None ) - target_topic = request.topic if request.HasField("topic") else None target_collection = ( UUID(hex=request.collection) if request.HasField("collection") else None ) @@ -216,8 +212,6 @@ def GetSegments( continue if target_scope and segment["scope"] != target_scope: continue - if target_topic and segment["topic"] != target_topic: - continue if target_collection and segment["collection"] != target_collection: continue found_segments.append(segment) @@ -238,10 +232,6 @@ def UpdateSegment( ) else: segment = self._segments[id_to_update.hex] - if request.HasField("topic"): - segment["topic"] = request.topic - if request.HasField("reset_topic") and request.reset_topic: - segment["topic"] = None if request.HasField("collection"): segment["collection"] = UUID(hex=request.collection) if request.HasField("reset_collection") and request.reset_collection: @@ -326,7 +316,6 @@ def CreateCollection( name=request.name, metadata=from_proto_metadata(request.metadata), dimension=request.dimension, - topic=self._assignment_policy.assign_collection(id), database=database, tenant=tenant, ) @@ -368,7 +357,6 @@ def GetCollections( self, request: GetCollectionsRequest, context: grpc.ServicerContext ) -> GetCollectionsResponse: target_id = UUID(hex=request.id) if request.HasField("id") else None - target_topic = request.topic if request.HasField("topic") else None target_name = request.name if request.HasField("name") else None tenant = request.tenant @@ -387,8 +375,6 @@ def GetCollections( for collection in collections.values(): if target_id and collection["id"] != target_id: continue - if target_topic and collection["topic"] != target_topic: - continue if target_name and collection["name"] != target_name: continue found_collections.append(collection) @@ -418,8 +404,6 @@ def UpdateCollection( ) else: collection = collections[id_to_update.hex] - if request.HasField("topic"): - collection["topic"] = request.topic if request.HasField("name"): collection["name"] = request.name if request.HasField("dimension"): diff --git a/chromadb/db/mixins/embeddings_queue.py b/chromadb/db/mixins/embeddings_queue.py index b5d745b92865..8a55209b70eb 100644 --- a/chromadb/db/mixins/embeddings_queue.py +++ b/chromadb/db/mixins/embeddings_queue.py @@ -1,16 +1,17 @@ +import json from chromadb.db.base import SqlDB, ParameterValue, get_sql from chromadb.ingest import ( Producer, Consumer, - encode_vector, - decode_vector, ConsumerCallbackFn, + decode_vector, + encode_vector, ) from chromadb.types import ( - SubmitEmbeddingRecord, + OperationRecord, EmbeddingRecord, - SeqId, ScalarEncoding, + SeqId, Operation, ) from chromadb.config import System @@ -21,12 +22,13 @@ ) from overrides import override from collections import defaultdict -from typing import Sequence, Tuple, Optional, Dict, Set, cast +from typing import Sequence, Optional, Dict, Set, Tuple, cast from uuid import UUID from pypika import Table, functions import uuid -import json import logging +from chromadb.ingest.impl.utils import create_topic_name + logger = logging.getLogger(__name__) @@ -78,6 +80,8 @@ def __init__( _subscriptions: Dict[str, Set[Subscription]] _max_batch_size: Optional[int] + _tenant: str + _topic_namespace: str # How many variables are in the insert statement for a single record VARIABLES_PER_RECORD = 6 @@ -85,6 +89,8 @@ def __init__(self, system: System): self._subscriptions = defaultdict(set) self._max_batch_size = None self._opentelemetry_client = system.require(OpenTelemetryClient) + self._tenant = system.settings.require("tenant_id") + self._topic_namespace = system.settings.require("topic_namespace") super().__init__(system) @trace_method("SqlEmbeddingsQueue.reset_state", OpenTelemetryGranularity.ALL) @@ -93,14 +99,12 @@ def reset_state(self) -> None: super().reset_state() self._subscriptions = defaultdict(set) - @override - def create_topic(self, topic_name: str) -> None: - # Topic creation is implicit for this impl - pass - @trace_method("SqlEmbeddingsQueue.delete_topic", OpenTelemetryGranularity.ALL) @override - def delete_topic(self, topic_name: str) -> None: + def delete_log(self, collection_id: UUID) -> None: + topic_name = create_topic_name( + self._tenant, self._topic_namespace, collection_id + ) t = Table("embeddings_queue") q = ( self.querybuilder() @@ -115,17 +119,17 @@ def delete_topic(self, topic_name: str) -> None: @trace_method("SqlEmbeddingsQueue.submit_embedding", OpenTelemetryGranularity.ALL) @override def submit_embedding( - self, topic_name: str, embedding: SubmitEmbeddingRecord + self, collection_id: UUID, embedding: OperationRecord ) -> SeqId: if not self._running: raise RuntimeError("Component not running") - return self.submit_embeddings(topic_name, [embedding])[0] + return self.submit_embeddings(collection_id, [embedding])[0] @trace_method("SqlEmbeddingsQueue.submit_embeddings", OpenTelemetryGranularity.ALL) @override def submit_embeddings( - self, topic_name: str, embeddings: Sequence[SubmitEmbeddingRecord] + self, collection_id: UUID, embeddings: Sequence[OperationRecord] ) -> Sequence[SeqId]: if not self._running: raise RuntimeError("Component not running") @@ -142,6 +146,10 @@ def submit_embeddings( """ ) + topic_name = create_topic_name( + self._tenant, self._topic_namespace, collection_id + ) + t = Table("embeddings_queue") insert = ( self.querybuilder() @@ -187,6 +195,7 @@ def submit_embeddings( encoding=submit_embedding_record["encoding"], metadata=submit_embedding_record["metadata"], operation=submit_embedding_record["operation"], + collection_id=collection_id, ) embedding_records.append(embedding_record) self._notify_all(topic_name, embedding_records) @@ -196,7 +205,7 @@ def submit_embeddings( @override def subscribe( self, - topic_name: str, + collection_id: UUID, consume_fn: ConsumerCallbackFn, start: Optional[SeqId] = None, end: Optional[SeqId] = None, @@ -205,6 +214,10 @@ def subscribe( if not self._running: raise RuntimeError("Component not running") + topic_name = create_topic_name( + self._tenant, self._topic_namespace, collection_id + ) + subscription_id = id or uuid.uuid4() start, end = self._validate_range(start, end) @@ -265,7 +278,7 @@ def max_batch_size(self) -> int: OpenTelemetryGranularity.ALL, ) def _prepare_vector_encoding_metadata( - self, embedding: SubmitEmbeddingRecord + self, embedding: OperationRecord ) -> Tuple[Optional[bytes], Optional[str], Optional[str]]: if embedding["embedding"]: encoding_type = cast(ScalarEncoding, embedding["encoding"]) diff --git a/chromadb/db/mixins/sysdb.py b/chromadb/db/mixins/sysdb.py index 7373aabd0f33..dbab7310e914 100644 --- a/chromadb/db/mixins/sysdb.py +++ b/chromadb/db/mixins/sysdb.py @@ -20,7 +20,7 @@ OpenTelemetryGranularity, trace_method, ) -from chromadb.ingest import CollectionAssignmentPolicy, Producer +from chromadb.ingest import Producer from chromadb.types import ( Database, OptionalArgument, @@ -35,13 +35,11 @@ class SqlSysDB(SqlDB, SysDB): - _assignment_policy: CollectionAssignmentPolicy - # Used only to delete topics on collection deletion. + # Used only to delete log streams on collection deletion. # TODO: refactor to remove this dependency into a separate interface _producer: Producer def __init__(self, system: System): - self._assignment_policy = system.instance(CollectionAssignmentPolicy) super().__init__(system) self._opentelemetry_client = system.require(OpenTelemetryClient) @@ -143,7 +141,6 @@ def create_segment(self, segment: Segment) -> None: "segment_id": str(segment["id"]), "segment_type": segment["type"], "segment_scope": segment["scope"].value, - "segment_topic": str(segment["topic"]), "collection": str(segment["collection"]), } ) @@ -156,14 +153,12 @@ def create_segment(self, segment: Segment) -> None: segments.id, segments.type, segments.scope, - segments.topic, segments.collection, ) .insert( ParameterValue(self.uuid_to_db(segment["id"])), ParameterValue(segment["type"]), ParameterValue(segment["scope"].value), - ParameterValue(segment["topic"]), ParameterValue(self.uuid_to_db(segment["collection"])), ) ) @@ -224,10 +219,8 @@ def create_collection( else: raise UniqueConstraintError(f"Collection {name} already exists") - topic = self._assignment_policy.assign_collection(id) collection = Collection( id=id, - topic=topic, name=name, metadata=metadata, dimension=dimension, @@ -244,14 +237,12 @@ def create_collection( .into(collections) .columns( collections.id, - collections.topic, collections.name, collections.dimension, collections.database_id, ) .insert( ParameterValue(self.uuid_to_db(collection["id"])), - ParameterValue(collection["topic"]), ParameterValue(collection["name"]), ParameterValue(collection["dimension"]), # Get the database id for the database with the given name and tenant @@ -287,7 +278,6 @@ def get_segments( id: Optional[UUID] = None, type: Optional[str] = None, scope: Optional[SegmentScope] = None, - topic: Optional[str] = None, collection: Optional[UUID] = None, ) -> Sequence[Segment]: add_attributes_to_current_span( @@ -295,7 +285,6 @@ def get_segments( "segment_id": str(id), "segment_type": type if type else "", "segment_scope": scope.value if scope else "", - "segment_topic": topic if topic else "", "collection": str(collection), } ) @@ -308,7 +297,6 @@ def get_segments( segments_t.id, segments_t.type, segments_t.scope, - segments_t.topic, segments_t.collection, metadata_t.key, metadata_t.str_value, @@ -325,8 +313,6 @@ def get_segments( q = q.where(segments_t.type == ParameterValue(type)) if scope: q = q.where(segments_t.scope == ParameterValue(scope.value)) - if topic: - q = q.where(segments_t.topic == ParameterValue(topic)) if collection: q = q.where( segments_t.collection == ParameterValue(self.uuid_to_db(collection)) @@ -342,15 +328,13 @@ def get_segments( rows = list(segment_rows) type = str(rows[0][1]) scope = SegmentScope(str(rows[0][2])) - topic = str(rows[0][3]) if rows[0][3] else None - collection = self.uuid_from_db(rows[0][4]) if rows[0][4] else None + collection = self.uuid_from_db(rows[0][3]) if rows[0][3] else None metadata = self._metadata_from_rows(rows) segments.append( Segment( id=cast(UUID, id), type=type, scope=scope, - topic=topic, collection=collection, metadata=metadata, ) @@ -363,7 +347,6 @@ def get_segments( def get_collections( self, id: Optional[UUID] = None, - topic: Optional[str] = None, name: Optional[str] = None, tenant: str = DEFAULT_TENANT, database: str = DEFAULT_DATABASE, @@ -380,7 +363,6 @@ def get_collections( add_attributes_to_current_span( { "collection_id": str(id), - "collection_topic": topic if topic else "", "collection_name": name if name else "", } ) @@ -394,7 +376,6 @@ def get_collections( .select( collections_t.id, collections_t.name, - collections_t.topic, collections_t.dimension, databases_t.name, databases_t.tenant_id, @@ -411,8 +392,6 @@ def get_collections( ) if id: q = q.where(collections_t.id == ParameterValue(self.uuid_to_db(id))) - if topic: - q = q.where(collections_t.topic == ParameterValue(topic)) if name: q = q.where(collections_t.name == ParameterValue(name)) @@ -439,24 +418,24 @@ def get_collections( id = self.uuid_from_db(str(collection_id)) rows = list(collection_rows) name = str(rows[0][1]) - topic = str(rows[0][2]) - dimension = int(rows[0][3]) if rows[0][3] else None + dimension = int(rows[0][2]) if rows[0][2] else None metadata = self._metadata_from_rows(rows) collections.append( Collection( id=cast(UUID, id), - topic=topic, name=name, metadata=metadata, dimension=dimension, - tenant=str(rows[0][5]), - database=str(rows[0][4]), + tenant=str(rows[0][4]), + database=str(rows[0][3]), ) ) # apply limit and offset if limit is not None: - collections = collections[offset:offset+limit] + if offset is None: + offset = 0 + collections = collections[offset: offset + limit] else: collections = collections[offset:] @@ -494,7 +473,8 @@ def delete_collection( tenant: str = DEFAULT_TENANT, database: str = DEFAULT_DATABASE, ) -> None: - """Delete a topic and all associated segments from the SysDB""" + """Delete a collection and all associated segments from the SysDB. Deletes + the log stream for this collection as well.""" add_attributes_to_current_span( { "collection_id": str(id), @@ -519,18 +499,17 @@ def delete_collection( with self.tx() as cur: # no need for explicit del from metadata table because of ON DELETE CASCADE sql, params = get_sql(q, self.parameter_format()) - sql = sql + " RETURNING id, topic" + sql = sql + " RETURNING id" result = cur.execute(sql, params).fetchone() if not result: raise NotFoundError(f"Collection {id} not found") - self._producer.delete_topic(result[1]) + self._producer.delete_log(result[0]) @trace_method("SqlSysDB.update_segment", OpenTelemetryGranularity.ALL) @override def update_segment( self, id: UUID, - topic: OptionalArgument[Optional[str]] = Unspecified(), collection: OptionalArgument[Optional[UUID]] = Unspecified(), metadata: OptionalArgument[Optional[UpdateMetadata]] = Unspecified(), ) -> None: @@ -549,9 +528,6 @@ def update_segment( .where(segments_t.id == ParameterValue(self.uuid_to_db(id))) ) - if not topic == Unspecified(): - q = q.set(segments_t.topic, ParameterValue(topic)) - if not collection == Unspecified(): collection = cast(Optional[UUID], collection) q = q.set( @@ -589,7 +565,6 @@ def update_segment( def update_collection( self, id: UUID, - topic: OptionalArgument[Optional[str]] = Unspecified(), name: OptionalArgument[str] = Unspecified(), dimension: OptionalArgument[Optional[int]] = Unspecified(), metadata: OptionalArgument[Optional[UpdateMetadata]] = Unspecified(), @@ -608,9 +583,6 @@ def update_collection( .where(collections_t.id == ParameterValue(self.uuid_to_db(id))) ) - if not topic == Unspecified(): - q = q.set(collections_t.topic, ParameterValue(topic)) - if not name == Unspecified(): q = q.set(collections_t.name, ParameterValue(name)) diff --git a/chromadb/db/system.py b/chromadb/db/system.py index 15cbf5691c18..f971806ebe12 100644 --- a/chromadb/db/system.py +++ b/chromadb/db/system.py @@ -60,17 +60,15 @@ def get_segments( id: Optional[UUID] = None, type: Optional[str] = None, scope: Optional[SegmentScope] = None, - topic: Optional[str] = None, collection: Optional[UUID] = None, ) -> Sequence[Segment]: - """Find segments by id, type, scope, topic or collection.""" + """Find segments by id, type, scope or collection.""" pass @abstractmethod def update_segment( self, id: UUID, - topic: OptionalArgument[Optional[str]] = Unspecified(), collection: OptionalArgument[Optional[UUID]] = Unspecified(), metadata: OptionalArgument[Optional[UpdateMetadata]] = Unspecified(), ) -> None: @@ -91,8 +89,8 @@ def create_collection( database: str = DEFAULT_DATABASE, ) -> Tuple[Collection, bool]: """Create a new collection any associated resources - (Such as the necessary topics) in the SysDB. If get_or_create is True, the - collectionwill be created if one with the same name does not exist. + in the SysDB. If get_or_create is True, the + collection will be created if one with the same name does not exist. The metadata will be updated using the same protocol as update_collection. If get_or_create is False and the collection already exists, a error will be raised. @@ -105,7 +103,7 @@ def create_collection( def delete_collection( self, id: UUID, tenant: str = DEFAULT_TENANT, database: str = DEFAULT_DATABASE ) -> None: - """Delete a collection, topic, all associated segments and any associate resources + """Delete a collection, all associated segments and any associate resources (log stream) from the SysDB and the system at large.""" pass @@ -113,21 +111,19 @@ def delete_collection( def get_collections( self, id: Optional[UUID] = None, - topic: Optional[str] = None, name: Optional[str] = None, tenant: str = DEFAULT_TENANT, database: str = DEFAULT_DATABASE, limit: Optional[int] = None, offset: Optional[int] = None, ) -> Sequence[Collection]: - """Find collections by id, topic or name. If name is provided, tenant and database must also be provided.""" + """Find collections by id or name. If name is provided, tenant and database must also be provided.""" pass @abstractmethod def update_collection( self, id: UUID, - topic: OptionalArgument[str] = Unspecified(), name: OptionalArgument[str] = Unspecified(), dimension: OptionalArgument[Optional[int]] = Unspecified(), metadata: OptionalArgument[Optional[UpdateMetadata]] = Unspecified(), diff --git a/chromadb/ingest/__init__.py b/chromadb/ingest/__init__.py index 73f9cb065f2b..39cda71525da 100644 --- a/chromadb/ingest/__init__.py +++ b/chromadb/ingest/__init__.py @@ -1,7 +1,7 @@ from abc import abstractmethod from typing import Callable, Optional, Sequence from chromadb.types import ( - SubmitEmbeddingRecord, + OperationRecord, EmbeddingRecord, SeqId, Vector, @@ -38,25 +38,21 @@ class Producer(Component): """Interface for writing embeddings to an ingest stream""" @abstractmethod - def create_topic(self, topic_name: str) -> None: - pass - - @abstractmethod - def delete_topic(self, topic_name: str) -> None: + def delete_log(self, collection_id: UUID) -> None: pass @abstractmethod def submit_embedding( - self, topic_name: str, embedding: SubmitEmbeddingRecord + self, collection_id: UUID, embedding: OperationRecord ) -> SeqId: - """Add an embedding record to the given topic. Returns the SeqID of the record.""" + """Add an embedding record to the given collections log. Returns the SeqID of the record.""" pass @abstractmethod def submit_embeddings( - self, topic_name: str, embeddings: Sequence[SubmitEmbeddingRecord] + self, collection_id: UUID, embeddings: Sequence[OperationRecord] ) -> Sequence[SeqId]: - """Add a batch of embedding records to the given topic. Returns the SeqIDs of + """Add a batch of embedding records to the given collections log. Returns the SeqIDs of the records. The returned SeqIDs will be in the same order as the given SubmitEmbeddingRecords. However, it is not guaranteed that the SeqIDs will be processed in the same order as the given SubmitEmbeddingRecords. If the number @@ -80,14 +76,14 @@ class Consumer(Component): @abstractmethod def subscribe( self, - topic_name: str, + collection_id: UUID, consume_fn: ConsumerCallbackFn, start: Optional[SeqId] = None, end: Optional[SeqId] = None, id: Optional[UUID] = None, ) -> UUID: """Register a function that will be called to recieve embeddings for a given - topic. The given function may be called any number of times, with any number of + collections log stream. The given function may be called any number of times, with any number of records, and may be called concurrently. Only records between start (exclusive) and end (inclusive) SeqIDs will be @@ -118,17 +114,3 @@ def min_seqid(self) -> SeqId: def max_seqid(self) -> SeqId: """Return the maximum possible SeqID in this implementation.""" pass - - -class CollectionAssignmentPolicy(Component): - """Interface for assigning collections to topics""" - - @abstractmethod - def assign_collection(self, collection_id: UUID) -> str: - """Return the topic that should be used for the given collection""" - pass - - @abstractmethod - def get_topics(self) -> Sequence[str]: - """Return the list of topics that this policy is currently using""" - pass diff --git a/chromadb/ingest/impl/simple_policy.py b/chromadb/ingest/impl/simple_policy.py deleted file mode 100644 index 9267ba06a40b..000000000000 --- a/chromadb/ingest/impl/simple_policy.py +++ /dev/null @@ -1,61 +0,0 @@ -from typing import Sequence -from uuid import UUID -from overrides import overrides -from chromadb.config import System -from chromadb.ingest import CollectionAssignmentPolicy -from chromadb.ingest.impl.utils import create_topic_name - - -class SimpleAssignmentPolicy(CollectionAssignmentPolicy): - """Simple assignment policy that assigns a 1 collection to 1 topic based on the - id of the collection.""" - - _tenant_id: str - _topic_ns: str - - def __init__(self, system: System): - self._tenant_id = system.settings.tenant_id - self._topic_ns = system.settings.topic_namespace - super().__init__(system) - - def _topic(self, collection_id: UUID) -> str: - return create_topic_name(self._tenant_id, self._topic_ns, str(collection_id)) - - @overrides - def assign_collection(self, collection_id: UUID) -> str: - return self._topic(collection_id) - - @overrides - def get_topics(self) -> Sequence[str]: - raise NotImplementedError( - "SimpleAssignmentPolicy does not support get_topics, each collection has its own topic" - ) - - -class RendezvousHashingAssignmentPolicy(CollectionAssignmentPolicy): - """The rendezvous hashing assignment policy assigns a collection to a topic based on the - rendezvous hashing algorithm. This is not actually used in the python sysdb. It is only used in the - go sysdb. However, it is useful here in order to provide a way to get the topic list used for the whole system. - """ - - _tenant_id: str - _topic_ns: str - - def __init__(self, system: System): - self._tenant_id = system.settings.tenant_id - self._topic_ns = system.settings.topic_namespace - super().__init__(system) - - @overrides - def assign_collection(self, collection_id: UUID) -> str: - raise NotImplementedError( - "RendezvousHashingAssignmentPolicy is not implemented" - ) - - @overrides - def get_topics(self) -> Sequence[str]: - # Mirrors go/internal/coordinator/assignment_policy.go - return [ - f"persistent://{self._tenant_id}/{self._topic_ns}/chroma_log_{i}" - for i in range(16) - ] diff --git a/chromadb/ingest/impl/utils.py b/chromadb/ingest/impl/utils.py index 34b46d3899a7..b28a7d8a1128 100644 --- a/chromadb/ingest/impl/utils.py +++ b/chromadb/ingest/impl/utils.py @@ -1,5 +1,6 @@ import re from typing import Tuple +from uuid import UUID topic_regex = r"persistent:\/\/(?P.+)\/(?P.+)\/(?P.+)" @@ -12,5 +13,5 @@ def parse_topic_name(topic_name: str) -> Tuple[str, str, str]: return match.group("tenant"), match.group("namespace"), match.group("topic") -def create_topic_name(tenant: str, namespace: str, topic: str) -> str: - return f"persistent://{tenant}/{namespace}/{topic}" +def create_topic_name(tenant: str, namespace: str, collection_id: UUID) -> str: + return f"persistent://{tenant}/{namespace}/{str(collection_id)}" diff --git a/chromadb/logservice/logservice.py b/chromadb/logservice/logservice.py index 83161d519db0..61adb9a881c0 100644 --- a/chromadb/logservice/logservice.py +++ b/chromadb/logservice/logservice.py @@ -12,7 +12,7 @@ from chromadb.proto.logservice_pb2_grpc import LogServiceStub from chromadb.telemetry.opentelemetry.grpc import OtelInterceptor from chromadb.types import ( - SubmitEmbeddingRecord, + OperationRecord, SeqId, ) from chromadb.config import System @@ -22,7 +22,7 @@ trace_method, ) from overrides import override -from typing import Sequence, Optional, Dict, cast +from typing import Sequence, Optional, cast from uuid import UUID import logging @@ -67,31 +67,29 @@ def stop(self) -> None: def reset_state(self) -> None: super().reset_state() + @trace_method("LogService.delete_log", OpenTelemetryGranularity.ALL) @override - def create_topic(self, topic_name: str) -> None: - raise NotImplementedError("Not implemented") - - @trace_method("LogService.delete_topic", OpenTelemetryGranularity.ALL) - @override - def delete_topic(self, topic_name: str) -> None: + def delete_log(self, collection_id: UUID) -> None: raise NotImplementedError("Not implemented") @trace_method("LogService.submit_embedding", OpenTelemetryGranularity.ALL) @override def submit_embedding( - self, topic_name: str, embedding: SubmitEmbeddingRecord + self, collection_id: UUID, embedding: OperationRecord ) -> SeqId: if not self._running: raise RuntimeError("Component not running") - return self.submit_embeddings(topic_name, [embedding])[0] # type: ignore + return self.submit_embeddings(collection_id, [embedding])[0] # type: ignore @trace_method("LogService.submit_embeddings", OpenTelemetryGranularity.ALL) @override def submit_embeddings( - self, topic_name: str, embeddings: Sequence[SubmitEmbeddingRecord] + self, collection_id: UUID, embeddings: Sequence[OperationRecord] ) -> Sequence[SeqId]: - logger.info(f"Submitting {len(embeddings)} embeddings to {topic_name}") + logger.info( + f"Submitting {len(embeddings)} embeddings to log for collection {collection_id}" + ) if not self._running: raise RuntimeError("Component not running") @@ -100,38 +98,30 @@ def submit_embeddings( return [] # push records to the log service - collection_id_to_embeddings: Dict[UUID, list[SubmitEmbeddingRecord]] = {} - for embedding in embeddings: - collection_id = cast(UUID, embedding.get("collection_id")) - if collection_id is None: - raise ValueError("collection_id is required") - if collection_id not in collection_id_to_embeddings: - collection_id_to_embeddings[collection_id] = [] - collection_id_to_embeddings[collection_id].append(embedding) - counts = [] - for collection_id, records in collection_id_to_embeddings.items(): - protos_to_submit = [to_proto_submit(record) for record in records] - counts.append( - self.push_logs( - collection_id, - cast(Sequence[SubmitEmbeddingRecord], protos_to_submit), - ) + protos_to_submit = [to_proto_submit(record) for record in embeddings] + counts.append( + self.push_logs( + collection_id, + cast(Sequence[OperationRecord], protos_to_submit), ) + ) + # This returns counts, which is completely incorrect + # TODO: Fix this return counts @trace_method("LogService.subscribe", OpenTelemetryGranularity.ALL) @override def subscribe( self, - topic_name: str, + collection_id: UUID, consume_fn: ConsumerCallbackFn, start: Optional[SeqId] = None, end: Optional[SeqId] = None, id: Optional[UUID] = None, ) -> UUID: - logger.info(f"Subscribing to {topic_name}, noop for logservice") + logger.info(f"Subscribing to log for {collection_id}, noop for logservice") return UUID(int=0) @trace_method("LogService.unsubscribe", OpenTelemetryGranularity.ALL) @@ -152,9 +142,7 @@ def max_seqid(self) -> SeqId: def max_batch_size(self) -> int: return sys.maxsize - def push_logs( - self, collection_id: UUID, records: Sequence[SubmitEmbeddingRecord] - ) -> int: + def push_logs(self, collection_id: UUID, records: Sequence[OperationRecord]) -> int: request = PushLogsRequest(collection_id=str(collection_id), records=records) response = self._log_service_stub.PushLogs(request) return response.record_count # type: ignore diff --git a/chromadb/migrations/sysdb/00005-remove-topic.sqlite.sql b/chromadb/migrations/sysdb/00005-remove-topic.sqlite.sql new file mode 100644 index 000000000000..3ed0e0284234 --- /dev/null +++ b/chromadb/migrations/sysdb/00005-remove-topic.sqlite.sql @@ -0,0 +1,4 @@ +-- Remove the topic column from the Collections and Segments tables + +ALTER TABLE collections DROP COLUMN topic; +ALTER TABLE segments DROP COLUMN topic; diff --git a/chromadb/proto/chroma_pb2.py b/chromadb/proto/chroma_pb2.py index d54e7c6e22d2..df92b355affb 100644 --- a/chromadb/proto/chroma_pb2.py +++ b/chromadb/proto/chroma_pb2.py @@ -13,7 +13,7 @@ -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x1b\x63hromadb/proto/chroma.proto\x12\x06\x63hroma\"&\n\x06Status\x12\x0e\n\x06reason\x18\x01 \x01(\t\x12\x0c\n\x04\x63ode\x18\x02 \x01(\x05\"U\n\x06Vector\x12\x11\n\tdimension\x18\x01 \x01(\x05\x12\x0e\n\x06vector\x18\x02 \x01(\x0c\x12(\n\x08\x65ncoding\x18\x03 \x01(\x0e\x32\x16.chroma.ScalarEncoding\"\x1a\n\tFilePaths\x12\r\n\x05paths\x18\x01 \x03(\t\"\xc3\x02\n\x07Segment\x12\n\n\x02id\x18\x01 \x01(\t\x12\x0c\n\x04type\x18\x02 \x01(\t\x12#\n\x05scope\x18\x03 \x01(\x0e\x32\x14.chroma.SegmentScope\x12\x12\n\x05topic\x18\x04 \x01(\tH\x00\x88\x01\x01\x12\x17\n\ncollection\x18\x05 \x01(\tH\x01\x88\x01\x01\x12-\n\x08metadata\x18\x06 \x01(\x0b\x32\x16.chroma.UpdateMetadataH\x02\x88\x01\x01\x12\x32\n\nfile_paths\x18\x07 \x03(\x0b\x32\x1e.chroma.Segment.FilePathsEntry\x1a\x43\n\x0e\x46ilePathsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12 \n\x05value\x18\x02 \x01(\x0b\x32\x11.chroma.FilePaths:\x02\x38\x01\x42\x08\n\x06_topicB\r\n\x0b_collectionB\x0b\n\t_metadata\"\xdf\x01\n\nCollection\x12\n\n\x02id\x18\x01 \x01(\t\x12\x0c\n\x04name\x18\x02 \x01(\t\x12\r\n\x05topic\x18\x03 \x01(\t\x12-\n\x08metadata\x18\x04 \x01(\x0b\x32\x16.chroma.UpdateMetadataH\x00\x88\x01\x01\x12\x16\n\tdimension\x18\x05 \x01(\x05H\x01\x88\x01\x01\x12\x0e\n\x06tenant\x18\x06 \x01(\t\x12\x10\n\x08\x64\x61tabase\x18\x07 \x01(\t\x12\x13\n\x0blogPosition\x18\x08 \x01(\x03\x12\x0f\n\x07version\x18\t \x01(\x05\x42\x0b\n\t_metadataB\x0c\n\n_dimension\"4\n\x08\x44\x61tabase\x12\n\n\x02id\x18\x01 \x01(\t\x12\x0c\n\x04name\x18\x02 \x01(\t\x12\x0e\n\x06tenant\x18\x03 \x01(\t\"\x16\n\x06Tenant\x12\x0c\n\x04name\x18\x01 \x01(\t\"b\n\x13UpdateMetadataValue\x12\x16\n\x0cstring_value\x18\x01 \x01(\tH\x00\x12\x13\n\tint_value\x18\x02 \x01(\x03H\x00\x12\x15\n\x0b\x66loat_value\x18\x03 \x01(\x01H\x00\x42\x07\n\x05value\"\x96\x01\n\x0eUpdateMetadata\x12\x36\n\x08metadata\x18\x01 \x03(\x0b\x32$.chroma.UpdateMetadata.MetadataEntry\x1aL\n\rMetadataEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12*\n\x05value\x18\x02 \x01(\x0b\x32\x1b.chroma.UpdateMetadataValue:\x02\x38\x01\"\xcc\x01\n\x15SubmitEmbeddingRecord\x12\n\n\x02id\x18\x01 \x01(\t\x12#\n\x06vector\x18\x02 \x01(\x0b\x32\x0e.chroma.VectorH\x00\x88\x01\x01\x12-\n\x08metadata\x18\x03 \x01(\x0b\x32\x16.chroma.UpdateMetadataH\x01\x88\x01\x01\x12$\n\toperation\x18\x04 \x01(\x0e\x32\x11.chroma.Operation\x12\x15\n\rcollection_id\x18\x05 \x01(\tB\t\n\x07_vectorB\x0b\n\t_metadata\"S\n\x15VectorEmbeddingRecord\x12\n\n\x02id\x18\x01 \x01(\t\x12\x0e\n\x06seq_id\x18\x02 \x01(\x0c\x12\x1e\n\x06vector\x18\x03 \x01(\x0b\x32\x0e.chroma.Vector\"q\n\x11VectorQueryResult\x12\n\n\x02id\x18\x01 \x01(\t\x12\x0e\n\x06seq_id\x18\x02 \x01(\x0c\x12\x10\n\x08\x64istance\x18\x03 \x01(\x02\x12#\n\x06vector\x18\x04 \x01(\x0b\x32\x0e.chroma.VectorH\x00\x88\x01\x01\x42\t\n\x07_vector\"@\n\x12VectorQueryResults\x12*\n\x07results\x18\x01 \x03(\x0b\x32\x19.chroma.VectorQueryResult\"4\n\x11GetVectorsRequest\x12\x0b\n\x03ids\x18\x01 \x03(\t\x12\x12\n\nsegment_id\x18\x02 \x01(\t\"D\n\x12GetVectorsResponse\x12.\n\x07records\x18\x01 \x03(\x0b\x32\x1d.chroma.VectorEmbeddingRecord\"\x86\x01\n\x13QueryVectorsRequest\x12\x1f\n\x07vectors\x18\x01 \x03(\x0b\x32\x0e.chroma.Vector\x12\t\n\x01k\x18\x02 \x01(\x05\x12\x13\n\x0b\x61llowed_ids\x18\x03 \x03(\t\x12\x1a\n\x12include_embeddings\x18\x04 \x01(\x08\x12\x12\n\nsegment_id\x18\x05 \x01(\t\"C\n\x14QueryVectorsResponse\x12+\n\x07results\x18\x01 \x03(\x0b\x32\x1a.chroma.VectorQueryResults*8\n\tOperation\x12\x07\n\x03\x41\x44\x44\x10\x00\x12\n\n\x06UPDATE\x10\x01\x12\n\n\x06UPSERT\x10\x02\x12\n\n\x06\x44\x45LETE\x10\x03*(\n\x0eScalarEncoding\x12\x0b\n\x07\x46LOAT32\x10\x00\x12\t\n\x05INT32\x10\x01*(\n\x0cSegmentScope\x12\n\n\x06VECTOR\x10\x00\x12\x0c\n\x08METADATA\x10\x01\x32\xa2\x01\n\x0cVectorReader\x12\x45\n\nGetVectors\x12\x19.chroma.GetVectorsRequest\x1a\x1a.chroma.GetVectorsResponse\"\x00\x12K\n\x0cQueryVectors\x12\x1b.chroma.QueryVectorsRequest\x1a\x1c.chroma.QueryVectorsResponse\"\x00\x42:Z8github.com/chroma-core/chroma/go/pkg/proto/coordinatorpbb\x06proto3') +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x1b\x63hromadb/proto/chroma.proto\x12\x06\x63hroma\"&\n\x06Status\x12\x0e\n\x06reason\x18\x01 \x01(\t\x12\x0c\n\x04\x63ode\x18\x02 \x01(\x05\"U\n\x06Vector\x12\x11\n\tdimension\x18\x01 \x01(\x05\x12\x0e\n\x06vector\x18\x02 \x01(\x0c\x12(\n\x08\x65ncoding\x18\x03 \x01(\x0e\x32\x16.chroma.ScalarEncoding\"\x1a\n\tFilePaths\x12\r\n\x05paths\x18\x01 \x03(\t\"\xa5\x02\n\x07Segment\x12\n\n\x02id\x18\x01 \x01(\t\x12\x0c\n\x04type\x18\x02 \x01(\t\x12#\n\x05scope\x18\x03 \x01(\x0e\x32\x14.chroma.SegmentScope\x12\x17\n\ncollection\x18\x05 \x01(\tH\x00\x88\x01\x01\x12-\n\x08metadata\x18\x06 \x01(\x0b\x32\x16.chroma.UpdateMetadataH\x01\x88\x01\x01\x12\x32\n\nfile_paths\x18\x07 \x03(\x0b\x32\x1e.chroma.Segment.FilePathsEntry\x1a\x43\n\x0e\x46ilePathsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12 \n\x05value\x18\x02 \x01(\x0b\x32\x11.chroma.FilePaths:\x02\x38\x01\x42\r\n\x0b_collectionB\x0b\n\t_metadata\"\xd0\x01\n\nCollection\x12\n\n\x02id\x18\x01 \x01(\t\x12\x0c\n\x04name\x18\x02 \x01(\t\x12-\n\x08metadata\x18\x04 \x01(\x0b\x32\x16.chroma.UpdateMetadataH\x00\x88\x01\x01\x12\x16\n\tdimension\x18\x05 \x01(\x05H\x01\x88\x01\x01\x12\x0e\n\x06tenant\x18\x06 \x01(\t\x12\x10\n\x08\x64\x61tabase\x18\x07 \x01(\t\x12\x13\n\x0blogPosition\x18\x08 \x01(\x03\x12\x0f\n\x07version\x18\t \x01(\x05\x42\x0b\n\t_metadataB\x0c\n\n_dimension\"4\n\x08\x44\x61tabase\x12\n\n\x02id\x18\x01 \x01(\t\x12\x0c\n\x04name\x18\x02 \x01(\t\x12\x0e\n\x06tenant\x18\x03 \x01(\t\"\x16\n\x06Tenant\x12\x0c\n\x04name\x18\x01 \x01(\t\"b\n\x13UpdateMetadataValue\x12\x16\n\x0cstring_value\x18\x01 \x01(\tH\x00\x12\x13\n\tint_value\x18\x02 \x01(\x03H\x00\x12\x15\n\x0b\x66loat_value\x18\x03 \x01(\x01H\x00\x42\x07\n\x05value\"\x96\x01\n\x0eUpdateMetadata\x12\x36\n\x08metadata\x18\x01 \x03(\x0b\x32$.chroma.UpdateMetadata.MetadataEntry\x1aL\n\rMetadataEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12*\n\x05value\x18\x02 \x01(\x0b\x32\x1b.chroma.UpdateMetadataValue:\x02\x38\x01\"\xaf\x01\n\x0fOperationRecord\x12\n\n\x02id\x18\x01 \x01(\t\x12#\n\x06vector\x18\x02 \x01(\x0b\x32\x0e.chroma.VectorH\x00\x88\x01\x01\x12-\n\x08metadata\x18\x03 \x01(\x0b\x32\x16.chroma.UpdateMetadataH\x01\x88\x01\x01\x12$\n\toperation\x18\x04 \x01(\x0e\x32\x11.chroma.OperationB\t\n\x07_vectorB\x0b\n\t_metadata\"S\n\x15VectorEmbeddingRecord\x12\n\n\x02id\x18\x01 \x01(\t\x12\x0e\n\x06seq_id\x18\x02 \x01(\x0c\x12\x1e\n\x06vector\x18\x03 \x01(\x0b\x32\x0e.chroma.Vector\"q\n\x11VectorQueryResult\x12\n\n\x02id\x18\x01 \x01(\t\x12\x0e\n\x06seq_id\x18\x02 \x01(\x0c\x12\x10\n\x08\x64istance\x18\x03 \x01(\x02\x12#\n\x06vector\x18\x04 \x01(\x0b\x32\x0e.chroma.VectorH\x00\x88\x01\x01\x42\t\n\x07_vector\"@\n\x12VectorQueryResults\x12*\n\x07results\x18\x01 \x03(\x0b\x32\x19.chroma.VectorQueryResult\"4\n\x11GetVectorsRequest\x12\x0b\n\x03ids\x18\x01 \x03(\t\x12\x12\n\nsegment_id\x18\x02 \x01(\t\"D\n\x12GetVectorsResponse\x12.\n\x07records\x18\x01 \x03(\x0b\x32\x1d.chroma.VectorEmbeddingRecord\"\x86\x01\n\x13QueryVectorsRequest\x12\x1f\n\x07vectors\x18\x01 \x03(\x0b\x32\x0e.chroma.Vector\x12\t\n\x01k\x18\x02 \x01(\x05\x12\x13\n\x0b\x61llowed_ids\x18\x03 \x03(\t\x12\x1a\n\x12include_embeddings\x18\x04 \x01(\x08\x12\x12\n\nsegment_id\x18\x05 \x01(\t\"C\n\x14QueryVectorsResponse\x12+\n\x07results\x18\x01 \x03(\x0b\x32\x1a.chroma.VectorQueryResults*8\n\tOperation\x12\x07\n\x03\x41\x44\x44\x10\x00\x12\n\n\x06UPDATE\x10\x01\x12\n\n\x06UPSERT\x10\x02\x12\n\n\x06\x44\x45LETE\x10\x03*(\n\x0eScalarEncoding\x12\x0b\n\x07\x46LOAT32\x10\x00\x12\t\n\x05INT32\x10\x01*(\n\x0cSegmentScope\x12\n\n\x06VECTOR\x10\x00\x12\x0c\n\x08METADATA\x10\x01\x32\xa2\x01\n\x0cVectorReader\x12\x45\n\nGetVectors\x12\x19.chroma.GetVectorsRequest\x1a\x1a.chroma.GetVectorsResponse\"\x00\x12K\n\x0cQueryVectors\x12\x1b.chroma.QueryVectorsRequest\x1a\x1c.chroma.QueryVectorsResponse\"\x00\x42:Z8github.com/chroma-core/chroma/go/pkg/proto/coordinatorpbb\x06proto3') _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) @@ -25,12 +25,12 @@ _SEGMENT_FILEPATHSENTRY._serialized_options = b'8\001' _UPDATEMETADATA_METADATAENTRY._options = None _UPDATEMETADATA_METADATAENTRY._serialized_options = b'8\001' - _globals['_OPERATION']._serialized_start=1880 - _globals['_OPERATION']._serialized_end=1936 - _globals['_SCALARENCODING']._serialized_start=1938 - _globals['_SCALARENCODING']._serialized_end=1978 - _globals['_SEGMENTSCOPE']._serialized_start=1980 - _globals['_SEGMENTSCOPE']._serialized_end=2020 + _globals['_OPERATION']._serialized_start=1806 + _globals['_OPERATION']._serialized_end=1862 + _globals['_SCALARENCODING']._serialized_start=1864 + _globals['_SCALARENCODING']._serialized_end=1904 + _globals['_SEGMENTSCOPE']._serialized_start=1906 + _globals['_SEGMENTSCOPE']._serialized_end=1946 _globals['_STATUS']._serialized_start=39 _globals['_STATUS']._serialized_end=77 _globals['_VECTOR']._serialized_start=79 @@ -38,37 +38,37 @@ _globals['_FILEPATHS']._serialized_start=166 _globals['_FILEPATHS']._serialized_end=192 _globals['_SEGMENT']._serialized_start=195 - _globals['_SEGMENT']._serialized_end=518 - _globals['_SEGMENT_FILEPATHSENTRY']._serialized_start=413 - _globals['_SEGMENT_FILEPATHSENTRY']._serialized_end=480 - _globals['_COLLECTION']._serialized_start=521 - _globals['_COLLECTION']._serialized_end=744 - _globals['_DATABASE']._serialized_start=746 - _globals['_DATABASE']._serialized_end=798 - _globals['_TENANT']._serialized_start=800 - _globals['_TENANT']._serialized_end=822 - _globals['_UPDATEMETADATAVALUE']._serialized_start=824 - _globals['_UPDATEMETADATAVALUE']._serialized_end=922 - _globals['_UPDATEMETADATA']._serialized_start=925 - _globals['_UPDATEMETADATA']._serialized_end=1075 - _globals['_UPDATEMETADATA_METADATAENTRY']._serialized_start=999 - _globals['_UPDATEMETADATA_METADATAENTRY']._serialized_end=1075 - _globals['_SUBMITEMBEDDINGRECORD']._serialized_start=1078 - _globals['_SUBMITEMBEDDINGRECORD']._serialized_end=1282 - _globals['_VECTOREMBEDDINGRECORD']._serialized_start=1284 - _globals['_VECTOREMBEDDINGRECORD']._serialized_end=1367 - _globals['_VECTORQUERYRESULT']._serialized_start=1369 - _globals['_VECTORQUERYRESULT']._serialized_end=1482 - _globals['_VECTORQUERYRESULTS']._serialized_start=1484 - _globals['_VECTORQUERYRESULTS']._serialized_end=1548 - _globals['_GETVECTORSREQUEST']._serialized_start=1550 - _globals['_GETVECTORSREQUEST']._serialized_end=1602 - _globals['_GETVECTORSRESPONSE']._serialized_start=1604 - _globals['_GETVECTORSRESPONSE']._serialized_end=1672 - _globals['_QUERYVECTORSREQUEST']._serialized_start=1675 - _globals['_QUERYVECTORSREQUEST']._serialized_end=1809 - _globals['_QUERYVECTORSRESPONSE']._serialized_start=1811 - _globals['_QUERYVECTORSRESPONSE']._serialized_end=1878 - _globals['_VECTORREADER']._serialized_start=2023 - _globals['_VECTORREADER']._serialized_end=2185 + _globals['_SEGMENT']._serialized_end=488 + _globals['_SEGMENT_FILEPATHSENTRY']._serialized_start=393 + _globals['_SEGMENT_FILEPATHSENTRY']._serialized_end=460 + _globals['_COLLECTION']._serialized_start=491 + _globals['_COLLECTION']._serialized_end=699 + _globals['_DATABASE']._serialized_start=701 + _globals['_DATABASE']._serialized_end=753 + _globals['_TENANT']._serialized_start=755 + _globals['_TENANT']._serialized_end=777 + _globals['_UPDATEMETADATAVALUE']._serialized_start=779 + _globals['_UPDATEMETADATAVALUE']._serialized_end=877 + _globals['_UPDATEMETADATA']._serialized_start=880 + _globals['_UPDATEMETADATA']._serialized_end=1030 + _globals['_UPDATEMETADATA_METADATAENTRY']._serialized_start=954 + _globals['_UPDATEMETADATA_METADATAENTRY']._serialized_end=1030 + _globals['_OPERATIONRECORD']._serialized_start=1033 + _globals['_OPERATIONRECORD']._serialized_end=1208 + _globals['_VECTOREMBEDDINGRECORD']._serialized_start=1210 + _globals['_VECTOREMBEDDINGRECORD']._serialized_end=1293 + _globals['_VECTORQUERYRESULT']._serialized_start=1295 + _globals['_VECTORQUERYRESULT']._serialized_end=1408 + _globals['_VECTORQUERYRESULTS']._serialized_start=1410 + _globals['_VECTORQUERYRESULTS']._serialized_end=1474 + _globals['_GETVECTORSREQUEST']._serialized_start=1476 + _globals['_GETVECTORSREQUEST']._serialized_end=1528 + _globals['_GETVECTORSRESPONSE']._serialized_start=1530 + _globals['_GETVECTORSRESPONSE']._serialized_end=1598 + _globals['_QUERYVECTORSREQUEST']._serialized_start=1601 + _globals['_QUERYVECTORSREQUEST']._serialized_end=1735 + _globals['_QUERYVECTORSRESPONSE']._serialized_start=1737 + _globals['_QUERYVECTORSRESPONSE']._serialized_end=1804 + _globals['_VECTORREADER']._serialized_start=1949 + _globals['_VECTORREADER']._serialized_end=2111 # @@protoc_insertion_point(module_scope) diff --git a/chromadb/proto/chroma_pb2.pyi b/chromadb/proto/chroma_pb2.pyi index 6e8b267a5843..1e4ac4979a26 100644 --- a/chromadb/proto/chroma_pb2.pyi +++ b/chromadb/proto/chroma_pb2.pyi @@ -56,7 +56,7 @@ class FilePaths(_message.Message): def __init__(self, paths: _Optional[_Iterable[str]] = ...) -> None: ... class Segment(_message.Message): - __slots__ = ["id", "type", "scope", "topic", "collection", "metadata", "file_paths"] + __slots__ = ["id", "type", "scope", "collection", "metadata", "file_paths"] class FilePathsEntry(_message.Message): __slots__ = ["key", "value"] KEY_FIELD_NUMBER: _ClassVar[int] @@ -67,24 +67,21 @@ class Segment(_message.Message): ID_FIELD_NUMBER: _ClassVar[int] TYPE_FIELD_NUMBER: _ClassVar[int] SCOPE_FIELD_NUMBER: _ClassVar[int] - TOPIC_FIELD_NUMBER: _ClassVar[int] COLLECTION_FIELD_NUMBER: _ClassVar[int] METADATA_FIELD_NUMBER: _ClassVar[int] FILE_PATHS_FIELD_NUMBER: _ClassVar[int] id: str type: str scope: SegmentScope - topic: str collection: str metadata: UpdateMetadata file_paths: _containers.MessageMap[str, FilePaths] - def __init__(self, id: _Optional[str] = ..., type: _Optional[str] = ..., scope: _Optional[_Union[SegmentScope, str]] = ..., topic: _Optional[str] = ..., collection: _Optional[str] = ..., metadata: _Optional[_Union[UpdateMetadata, _Mapping]] = ..., file_paths: _Optional[_Mapping[str, FilePaths]] = ...) -> None: ... + def __init__(self, id: _Optional[str] = ..., type: _Optional[str] = ..., scope: _Optional[_Union[SegmentScope, str]] = ..., collection: _Optional[str] = ..., metadata: _Optional[_Union[UpdateMetadata, _Mapping]] = ..., file_paths: _Optional[_Mapping[str, FilePaths]] = ...) -> None: ... class Collection(_message.Message): - __slots__ = ["id", "name", "topic", "metadata", "dimension", "tenant", "database", "logPosition", "version"] + __slots__ = ["id", "name", "metadata", "dimension", "tenant", "database", "logPosition", "version"] ID_FIELD_NUMBER: _ClassVar[int] NAME_FIELD_NUMBER: _ClassVar[int] - TOPIC_FIELD_NUMBER: _ClassVar[int] METADATA_FIELD_NUMBER: _ClassVar[int] DIMENSION_FIELD_NUMBER: _ClassVar[int] TENANT_FIELD_NUMBER: _ClassVar[int] @@ -93,14 +90,13 @@ class Collection(_message.Message): VERSION_FIELD_NUMBER: _ClassVar[int] id: str name: str - topic: str metadata: UpdateMetadata dimension: int tenant: str database: str logPosition: int version: int - def __init__(self, id: _Optional[str] = ..., name: _Optional[str] = ..., topic: _Optional[str] = ..., metadata: _Optional[_Union[UpdateMetadata, _Mapping]] = ..., dimension: _Optional[int] = ..., tenant: _Optional[str] = ..., database: _Optional[str] = ..., logPosition: _Optional[int] = ..., version: _Optional[int] = ...) -> None: ... + def __init__(self, id: _Optional[str] = ..., name: _Optional[str] = ..., metadata: _Optional[_Union[UpdateMetadata, _Mapping]] = ..., dimension: _Optional[int] = ..., tenant: _Optional[str] = ..., database: _Optional[str] = ..., logPosition: _Optional[int] = ..., version: _Optional[int] = ...) -> None: ... class Database(_message.Message): __slots__ = ["id", "name", "tenant"] @@ -141,19 +137,17 @@ class UpdateMetadata(_message.Message): metadata: _containers.MessageMap[str, UpdateMetadataValue] def __init__(self, metadata: _Optional[_Mapping[str, UpdateMetadataValue]] = ...) -> None: ... -class SubmitEmbeddingRecord(_message.Message): - __slots__ = ["id", "vector", "metadata", "operation", "collection_id"] +class OperationRecord(_message.Message): + __slots__ = ["id", "vector", "metadata", "operation"] ID_FIELD_NUMBER: _ClassVar[int] VECTOR_FIELD_NUMBER: _ClassVar[int] METADATA_FIELD_NUMBER: _ClassVar[int] OPERATION_FIELD_NUMBER: _ClassVar[int] - COLLECTION_ID_FIELD_NUMBER: _ClassVar[int] id: str vector: Vector metadata: UpdateMetadata operation: Operation - collection_id: str - def __init__(self, id: _Optional[str] = ..., vector: _Optional[_Union[Vector, _Mapping]] = ..., metadata: _Optional[_Union[UpdateMetadata, _Mapping]] = ..., operation: _Optional[_Union[Operation, str]] = ..., collection_id: _Optional[str] = ...) -> None: ... + def __init__(self, id: _Optional[str] = ..., vector: _Optional[_Union[Vector, _Mapping]] = ..., metadata: _Optional[_Union[UpdateMetadata, _Mapping]] = ..., operation: _Optional[_Union[Operation, str]] = ...) -> None: ... class VectorEmbeddingRecord(_message.Message): __slots__ = ["id", "seq_id", "vector"] diff --git a/chromadb/proto/convert.py b/chromadb/proto/convert.py index 78eaeb89101f..6a5f93d31dd3 100644 --- a/chromadb/proto/convert.py +++ b/chromadb/proto/convert.py @@ -13,7 +13,7 @@ Segment, SegmentScope, SeqId, - SubmitEmbeddingRecord, + OperationRecord, UpdateMetadata, Vector, VectorEmbeddingRecord, @@ -112,7 +112,7 @@ def to_proto_update_metadata(metadata: UpdateMetadata) -> proto.UpdateMetadata: def from_proto_submit( - submit_embedding_record: proto.SubmitEmbeddingRecord, seq_id: SeqId + submit_embedding_record: proto.OperationRecord, seq_id: SeqId ) -> EmbeddingRecord: embedding, encoding = from_proto_vector(submit_embedding_record.vector) record = EmbeddingRecord( @@ -132,7 +132,6 @@ def from_proto_segment(segment: proto.Segment) -> Segment: id=UUID(hex=segment.id), type=segment.type, scope=from_proto_segment_scope(segment.scope), - topic=segment.topic if segment.HasField("topic") else None, collection=None if not segment.HasField("collection") else UUID(hex=segment.collection), @@ -147,7 +146,6 @@ def to_proto_segment(segment: Segment) -> proto.Segment: id=segment["id"].hex, type=segment["type"], scope=to_proto_segment_scope(segment["scope"]), - topic=segment["topic"], collection=None if segment["collection"] is None else segment["collection"].hex, metadata=None if segment["metadata"] is None @@ -195,7 +193,6 @@ def from_proto_collection(collection: proto.Collection) -> Collection: return Collection( id=UUID(hex=collection.id), name=collection.name, - topic=collection.topic, metadata=from_proto_metadata(collection.metadata) if collection.HasField("metadata") else None, @@ -211,7 +208,6 @@ def to_proto_collection(collection: Collection) -> proto.Collection: return proto.Collection( id=collection["id"].hex, name=collection["name"], - topic=collection["topic"], metadata=None if collection["metadata"] is None else to_proto_update_metadata(collection["metadata"]), @@ -238,8 +234,8 @@ def to_proto_operation(operation: Operation) -> proto.Operation: def to_proto_submit( - submit_record: SubmitEmbeddingRecord, -) -> proto.SubmitEmbeddingRecord: + submit_record: OperationRecord, +) -> proto.OperationRecord: vector = None if submit_record["embedding"] is not None and submit_record["encoding"] is not None: vector = to_proto_vector(submit_record["embedding"], submit_record["encoding"]) @@ -248,12 +244,11 @@ def to_proto_submit( if submit_record["metadata"] is not None: metadata = to_proto_update_metadata(submit_record["metadata"]) - return proto.SubmitEmbeddingRecord( + return proto.OperationRecord( id=submit_record["id"], vector=vector, metadata=metadata, operation=to_proto_operation(submit_record["operation"]), - collection_id=submit_record["collection_id"].hex, ) diff --git a/chromadb/proto/logservice_pb2.py b/chromadb/proto/logservice_pb2.py index 39f62f13aa7f..0c7ca972ebe9 100644 --- a/chromadb/proto/logservice_pb2.py +++ b/chromadb/proto/logservice_pb2.py @@ -16,7 +16,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x1f\x63hromadb/proto/logservice.proto\x12\x06\x63hroma\x1a\x1b\x63hromadb/proto/chroma.proto"X\n\x0fPushLogsRequest\x12\x15\n\rcollection_id\x18\x01 \x01(\t\x12.\n\x07records\x18\x02 \x03(\x0b\x32\x1d.chroma.SubmitEmbeddingRecord"(\n\x10PushLogsResponse\x12\x14\n\x0crecord_count\x18\x01 \x01(\x05"j\n\x0fPullLogsRequest\x12\x15\n\rcollection_id\x18\x01 \x01(\t\x12\x15\n\rstart_from_id\x18\x02 \x01(\x03\x12\x12\n\nbatch_size\x18\x03 \x01(\x05\x12\x15\n\rend_timestamp\x18\x04 \x01(\x03"J\n\tRecordLog\x12\x0e\n\x06log_id\x18\x01 \x01(\x03\x12-\n\x06record\x18\x02 \x01(\x0b\x32\x1d.chroma.SubmitEmbeddingRecord"6\n\x10PullLogsResponse\x12"\n\x07records\x18\x01 \x03(\x0b\x32\x11.chroma.RecordLog"V\n\x0e\x43ollectionInfo\x12\x15\n\rcollection_id\x18\x01 \x01(\t\x12\x14\n\x0c\x66irst_log_id\x18\x02 \x01(\x03\x12\x17\n\x0f\x66irst_log_id_ts\x18\x03 \x01(\x03"&\n$GetAllCollectionInfoToCompactRequest"\\\n%GetAllCollectionInfoToCompactResponse\x12\x33\n\x13\x61ll_collection_info\x18\x01 \x03(\x0b\x32\x16.chroma.CollectionInfo2\x8e\x02\n\nLogService\x12?\n\x08PushLogs\x12\x17.chroma.PushLogsRequest\x1a\x18.chroma.PushLogsResponse"\x00\x12?\n\x08PullLogs\x12\x17.chroma.PullLogsRequest\x1a\x18.chroma.PullLogsResponse"\x00\x12~\n\x1dGetAllCollectionInfoToCompact\x12,.chroma.GetAllCollectionInfoToCompactRequest\x1a-.chroma.GetAllCollectionInfoToCompactResponse"\x00\x42\x39Z7github.com/chroma-core/chroma/go/pkg/proto/logservicepbb\x06proto3' + b'\n\x1f\x63hromadb/proto/logservice.proto\x12\x06\x63hroma\x1a\x1b\x63hromadb/proto/chroma.proto"R\n\x0fPushLogsRequest\x12\x15\n\rcollection_id\x18\x01 \x01(\t\x12(\n\x07records\x18\x02 \x03(\x0b\x32\x17.chroma.OperationRecord"(\n\x10PushLogsResponse\x12\x14\n\x0crecord_count\x18\x01 \x01(\x05"j\n\x0fPullLogsRequest\x12\x15\n\rcollection_id\x18\x01 \x01(\t\x12\x15\n\rstart_from_id\x18\x02 \x01(\x03\x12\x12\n\nbatch_size\x18\x03 \x01(\x05\x12\x15\n\rend_timestamp\x18\x04 \x01(\x03"D\n\tRecordLog\x12\x0e\n\x06log_id\x18\x01 \x01(\x03\x12\'\n\x06record\x18\x02 \x01(\x0b\x32\x17.chroma.OperationRecord"6\n\x10PullLogsResponse\x12"\n\x07records\x18\x01 \x03(\x0b\x32\x11.chroma.RecordLog"V\n\x0e\x43ollectionInfo\x12\x15\n\rcollection_id\x18\x01 \x01(\t\x12\x14\n\x0c\x66irst_log_id\x18\x02 \x01(\x03\x12\x17\n\x0f\x66irst_log_id_ts\x18\x03 \x01(\x03"&\n$GetAllCollectionInfoToCompactRequest"\\\n%GetAllCollectionInfoToCompactResponse\x12\x33\n\x13\x61ll_collection_info\x18\x01 \x03(\x0b\x32\x16.chroma.CollectionInfo2\x8e\x02\n\nLogService\x12?\n\x08PushLogs\x12\x17.chroma.PushLogsRequest\x1a\x18.chroma.PushLogsResponse"\x00\x12?\n\x08PullLogs\x12\x17.chroma.PullLogsRequest\x1a\x18.chroma.PullLogsResponse"\x00\x12~\n\x1dGetAllCollectionInfoToCompact\x12,.chroma.GetAllCollectionInfoToCompactRequest\x1a-.chroma.GetAllCollectionInfoToCompactResponse"\x00\x42\x39Z7github.com/chroma-core/chroma/go/pkg/proto/logservicepbb\x06proto3' ) _globals = globals() @@ -30,21 +30,21 @@ b"Z7github.com/chroma-core/chroma/go/pkg/proto/logservicepb" ) _globals["_PUSHLOGSREQUEST"]._serialized_start = 72 - _globals["_PUSHLOGSREQUEST"]._serialized_end = 160 - _globals["_PUSHLOGSRESPONSE"]._serialized_start = 162 - _globals["_PUSHLOGSRESPONSE"]._serialized_end = 202 - _globals["_PULLLOGSREQUEST"]._serialized_start = 204 - _globals["_PULLLOGSREQUEST"]._serialized_end = 310 - _globals["_RECORDLOG"]._serialized_start = 312 - _globals["_RECORDLOG"]._serialized_end = 386 - _globals["_PULLLOGSRESPONSE"]._serialized_start = 388 - _globals["_PULLLOGSRESPONSE"]._serialized_end = 442 - _globals["_COLLECTIONINFO"]._serialized_start = 444 - _globals["_COLLECTIONINFO"]._serialized_end = 530 - _globals["_GETALLCOLLECTIONINFOTOCOMPACTREQUEST"]._serialized_start = 532 - _globals["_GETALLCOLLECTIONINFOTOCOMPACTREQUEST"]._serialized_end = 570 - _globals["_GETALLCOLLECTIONINFOTOCOMPACTRESPONSE"]._serialized_start = 572 - _globals["_GETALLCOLLECTIONINFOTOCOMPACTRESPONSE"]._serialized_end = 664 - _globals["_LOGSERVICE"]._serialized_start = 667 - _globals["_LOGSERVICE"]._serialized_end = 937 + _globals["_PUSHLOGSREQUEST"]._serialized_end = 154 + _globals["_PUSHLOGSRESPONSE"]._serialized_start = 156 + _globals["_PUSHLOGSRESPONSE"]._serialized_end = 196 + _globals["_PULLLOGSREQUEST"]._serialized_start = 198 + _globals["_PULLLOGSREQUEST"]._serialized_end = 304 + _globals["_RECORDLOG"]._serialized_start = 306 + _globals["_RECORDLOG"]._serialized_end = 374 + _globals["_PULLLOGSRESPONSE"]._serialized_start = 376 + _globals["_PULLLOGSRESPONSE"]._serialized_end = 430 + _globals["_COLLECTIONINFO"]._serialized_start = 432 + _globals["_COLLECTIONINFO"]._serialized_end = 518 + _globals["_GETALLCOLLECTIONINFOTOCOMPACTREQUEST"]._serialized_start = 520 + _globals["_GETALLCOLLECTIONINFOTOCOMPACTREQUEST"]._serialized_end = 558 + _globals["_GETALLCOLLECTIONINFOTOCOMPACTRESPONSE"]._serialized_start = 560 + _globals["_GETALLCOLLECTIONINFOTOCOMPACTRESPONSE"]._serialized_end = 652 + _globals["_LOGSERVICE"]._serialized_start = 655 + _globals["_LOGSERVICE"]._serialized_end = 925 # @@protoc_insertion_point(module_scope) diff --git a/chromadb/proto/logservice_pb2.pyi b/chromadb/proto/logservice_pb2.pyi index f4be90b553a5..78680253a6d2 100644 --- a/chromadb/proto/logservice_pb2.pyi +++ b/chromadb/proto/logservice_pb2.pyi @@ -17,14 +17,12 @@ class PushLogsRequest(_message.Message): COLLECTION_ID_FIELD_NUMBER: _ClassVar[int] RECORDS_FIELD_NUMBER: _ClassVar[int] collection_id: str - records: _containers.RepeatedCompositeFieldContainer[ - _chroma_pb2.SubmitEmbeddingRecord - ] + records: _containers.RepeatedCompositeFieldContainer[_chroma_pb2.OperationRecord] def __init__( self, collection_id: _Optional[str] = ..., records: _Optional[ - _Iterable[_Union[_chroma_pb2.SubmitEmbeddingRecord, _Mapping]] + _Iterable[_Union[_chroma_pb2.OperationRecord, _Mapping]] ] = ..., ) -> None: ... @@ -57,11 +55,11 @@ class RecordLog(_message.Message): LOG_ID_FIELD_NUMBER: _ClassVar[int] RECORD_FIELD_NUMBER: _ClassVar[int] log_id: int - record: _chroma_pb2.SubmitEmbeddingRecord + record: _chroma_pb2.OperationRecord def __init__( self, log_id: _Optional[int] = ..., - record: _Optional[_Union[_chroma_pb2.SubmitEmbeddingRecord, _Mapping]] = ..., + record: _Optional[_Union[_chroma_pb2.OperationRecord, _Mapping]] = ..., ) -> None: ... class PullLogsResponse(_message.Message): diff --git a/chromadb/segment/impl/distributed/server.py b/chromadb/segment/impl/distributed/server.py deleted file mode 100644 index 7b08e1e5d667..000000000000 --- a/chromadb/segment/impl/distributed/server.py +++ /dev/null @@ -1,171 +0,0 @@ -from typing import Any, Dict, List, Sequence, Set -from uuid import UUID -from chromadb.config import Settings, System -from chromadb.ingest import CollectionAssignmentPolicy, Consumer -from chromadb.proto.chroma_pb2_grpc import ( - # SegmentServerServicer, - # add_SegmentServerServicer_to_server, - VectorReaderServicer, - add_VectorReaderServicer_to_server, -) -import chromadb.proto.chroma_pb2 as proto -import grpc -from concurrent import futures -from chromadb.segment import SegmentImplementation, SegmentType -from chromadb.telemetry.opentelemetry import OpenTelemetryClient -from chromadb.types import EmbeddingRecord -from chromadb.segment.distributed import MemberlistProvider, Memberlist -from chromadb.utils.rendezvous_hash import assign, murmur3hasher -import logging -import os - -# This file is a prototype. It will be replaced with a real distributed segment server -# written in a different language. This is just a proof of concept to get the distributed -# segment type working end to end. - -# Run this with python -m chromadb.segment.impl.distributed.server - -SEGMENT_TYPE_IMPLS = { - SegmentType.HNSW_DISTRIBUTED: "chromadb.segment.impl.vector.local_persistent_hnsw.PersistentLocalHnswSegment", -} - - -class SegmentServer(VectorReaderServicer): - _segment_cache: Dict[UUID, SegmentImplementation] = {} - _system: System - _opentelemetry_client: OpenTelemetryClient - _memberlist_provider: MemberlistProvider - _curr_memberlist: Memberlist - _assigned_topics: Set[str] - _topic_to_subscription: Dict[str, UUID] - _consumer: Consumer - - def __init__(self, system: System) -> None: - super().__init__() - self._system = system - - # Init dependency services - self._opentelemetry_client = system.require(OpenTelemetryClient) - # TODO: add term and epoch to segment server - self._memberlist_provider = system.require(MemberlistProvider) - self._memberlist_provider.set_memberlist_name("query-service-memberlist") - self._assignment_policy = system.require(CollectionAssignmentPolicy) - self._consumer = system.require(Consumer) - - # Init data - self._topic_to_subscription = {} - self._assigned_topics = set() - self._curr_memberlist = self._memberlist_provider.get_memberlist() - self._compute_assigned_topics() - - self._memberlist_provider.register_updated_memberlist_callback( - self._on_memberlist_update - ) - - def _compute_assigned_topics(self) -> None: - """Uses rendezvous hashing to compute the topics that this node is responsible for""" - if not self._curr_memberlist: - self._assigned_topics = set() - return - topics = self._assignment_policy.get_topics() - my_ip = os.environ["MY_POD_IP"] - new_assignments: List[str] = [] - for topic in topics: - assigned = assign(topic, self._curr_memberlist, murmur3hasher) - if assigned == my_ip: - new_assignments.append(topic) - new_assignments_set = set(new_assignments) - # TODO: We need to lock around this assignment - net_new_assignments = new_assignments_set - self._assigned_topics - removed_assignments = self._assigned_topics - new_assignments_set - - for topic in removed_assignments: - subscription = self._topic_to_subscription[topic] - self._consumer.unsubscribe(subscription) - del self._topic_to_subscription[topic] - - for topic in net_new_assignments: - subscription = self._consumer.subscribe(topic, self._on_message) - self._topic_to_subscription[topic] = subscription - - self._assigned_topics = new_assignments_set - print( - f"Topic assigment updated and now assigned to {len(self._assigned_topics)} topics" - ) - - def _on_memberlist_update(self, memberlist: Memberlist) -> None: - """Called when the memberlist is updated""" - self._curr_memberlist = memberlist - if len(self._curr_memberlist) > 0: - self._compute_assigned_topics() - else: - # In this case we'd want to warn that there are no members but - # this is not an error, as it could be that the cluster is just starting up - print("Memberlist is empty") - - def _on_message(self, embedding_records: Sequence[EmbeddingRecord]) -> None: - """Called when a message is received from the consumer""" - print(f"Received {len(embedding_records)} records") - print( - f"First record: {embedding_records[0]} is for collection {embedding_records[0]['collection_id']}" - ) - return None - - def QueryVectors( - self, request: proto.QueryVectorsRequest, context: Any - ) -> proto.QueryVectorsResponse: - context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Query segment not implemented yet") - return proto.QueryVectorsResponse() - - # @trace_method( - # "SegmentServer.GetVectors", OpenTelemetryGranularity.OPERATION_AND_SEGMENT - # ) - # def GetVectors( - # self, request: proto.GetVectorsRequest, context: Any - # ) -> proto.GetVectorsResponse: - # segment_id = UUID(hex=request.segment_id) - # if segment_id not in self._segment_cache: - # context.set_code(grpc.StatusCode.NOT_FOUND) - # context.set_details("Segment not found") - # return proto.GetVectorsResponse() - # else: - # segment = self._segment_cache[segment_id] - # segment = cast(VectorReader, segment) - # segment_results = segment.get_vectors(request.ids) - # return_records = [] - # for record in segment_results: - # # TODO: encoding should be based on stored encoding for segment - # # For now we just assume float32 - # return_record = to_proto_vector_embedding_record( - # record, ScalarEncoding.FLOAT32 - # ) - # return_records.append(return_record) - # return proto.GetVectorsResponse(records=return_records) - - # def _cls(self, segment: Segment) -> Type[SegmentImplementation]: - # classname = SEGMENT_TYPE_IMPLS[SegmentType(segment["type"])] - # cls = get_class(classname, SegmentImplementation) - # return cls - - # def _create_instance(self, segment: Segment) -> None: - # if segment["id"] not in self._segment_cache: - # cls = self._cls(segment) - # instance = cls(self._system, segment) - # instance.start() - # self._segment_cache[segment["id"]] = instance - - -if __name__ == "__main__": - logging.basicConfig(level=logging.INFO) - system = System(Settings()) - server = grpc.server(futures.ThreadPoolExecutor(max_workers=10)) - segment_server = SegmentServer(system) - # add_SegmentServerServicer_to_server(segment_server, server) # type: ignore - add_VectorReaderServicer_to_server(segment_server, server) # type: ignore - server.add_insecure_port( - f"[::]:{system.settings.require('chroma_server_grpc_port')}" - ) - system.start() - server.start() - server.wait_for_termination() diff --git a/chromadb/segment/impl/manager/distributed.py b/chromadb/segment/impl/manager/distributed.py index c114b8a3c967..fb0352340c82 100644 --- a/chromadb/segment/impl/manager/distributed.py +++ b/chromadb/segment/impl/manager/distributed.py @@ -125,7 +125,7 @@ def hint_use_collection(self, collection_id: UUID, hint_type: Operation) -> None collection=collection_id, scope=SegmentScope.VECTOR ) known_types = set([k.value for k in SEGMENT_TYPE_IMPLS.keys()]) - segment = next(filter(lambda s: s["type"] in known_types, segments)) + segment = next(filter(lambda s: s["type"] in known_types, segments)) # noqa # grpc_url = self._segment_directory.get_segment_endpoint(segment) # if grpc_url not in self._segment_server_stubs: @@ -172,7 +172,6 @@ def _segment(type: SegmentType, scope: SegmentScope, collection: Collection) -> id=uuid4(), type=type.value, scope=scope, - topic=collection["topic"], collection=collection["id"], metadata=metadata, ) diff --git a/chromadb/segment/impl/manager/local.py b/chromadb/segment/impl/manager/local.py index c5afef2d0123..19710b921d82 100644 --- a/chromadb/segment/impl/manager/local.py +++ b/chromadb/segment/impl/manager/local.py @@ -8,7 +8,11 @@ S, ) import logging -from chromadb.segment.impl.manager.cache.cache import SegmentLRUCache, BasicCache,SegmentCache +from chromadb.segment.impl.manager.cache.cache import ( + SegmentLRUCache, + BasicCache, + SegmentCache, +) import os from chromadb.config import System, get_class @@ -42,6 +46,7 @@ SegmentType.HNSW_LOCAL_PERSISTED: "chromadb.segment.impl.vector.local_persistent_hnsw.PersistentLocalHnswSegment", } + class LocalSegmentManager(SegmentManager): _sysdb: SysDB _system: System @@ -61,15 +66,21 @@ def __init__(self, system: System): self._opentelemetry_client = system.require(OpenTelemetryClient) self.logger = logging.getLogger(__name__) self._instances = {} - self.segment_cache: Dict[SegmentScope, SegmentCache] = {SegmentScope.METADATA: BasicCache()} - if system.settings.chroma_segment_cache_policy == "LRU" and system.settings.chroma_memory_limit_bytes > 0: - self.segment_cache[SegmentScope.VECTOR] = SegmentLRUCache(capacity=system.settings.chroma_memory_limit_bytes,callback=lambda k, v: self.callback_cache_evict(v), size_func=lambda k: self._get_segment_disk_size(k)) + self.segment_cache: Dict[SegmentScope, SegmentCache] = { + SegmentScope.METADATA: BasicCache() + } + if ( + system.settings.chroma_segment_cache_policy == "LRU" + and system.settings.chroma_memory_limit_bytes > 0 + ): + self.segment_cache[SegmentScope.VECTOR] = SegmentLRUCache( + capacity=system.settings.chroma_memory_limit_bytes, + callback=lambda k, v: self.callback_cache_evict(v), + size_func=lambda k: self._get_segment_disk_size(k), + ) else: self.segment_cache[SegmentScope.VECTOR] = BasicCache() - - - self._lock = Lock() # TODO: prototyping with distributed segment for now, but this should be a configurable option @@ -81,8 +92,8 @@ def __init__(self, system: System): else: self._max_file_handles = ctypes.windll.msvcrt._getmaxstdio() # type: ignore segment_limit = ( - self._max_file_handles - // PersistentLocalHnswSegment.get_file_handle_count() + self._max_file_handles + // PersistentLocalHnswSegment.get_file_handle_count() ) self._vector_instances_file_handle_cache = LRUCache( segment_limit, callback=lambda _, v: v.close_persistent_index() @@ -95,7 +106,6 @@ def callback_cache_evict(self, segment: Segment): instance.stop() del self._instances[segment["id"]] - @override def start(self) -> None: for instance in self._instances.values(): @@ -158,20 +168,27 @@ def delete_segments(self, collection_id: UUID) -> Sequence[UUID]: OpenTelemetryGranularity.OPERATION_AND_SEGMENT, ) def _get_segment_disk_size(self, collection_id: UUID) -> int: - segments = self._sysdb.get_segments(collection=collection_id, scope=SegmentScope.VECTOR) + segments = self._sysdb.get_segments( + collection=collection_id, scope=SegmentScope.VECTOR + ) if len(segments) == 0: return 0 # With local segment manager (single server chroma), a collection always have one segment. size = get_directory_size( - os.path.join(self._system.settings.require("persist_directory"), str(segments[0]["id"]))) + os.path.join( + self._system.settings.require("persist_directory"), + str(segments[0]["id"]), + ) + ) return size - def _get_segment_sysdb(self, collection_id:UUID, scope: SegmentScope): + def _get_segment_sysdb(self, collection_id: UUID, scope: SegmentScope): segments = self._sysdb.get_segments(collection=collection_id, scope=scope) known_types = set([k.value for k in SEGMENT_TYPE_IMPLS.keys()]) # Get the first segment of a known type segment = next(filter(lambda s: s["type"] in known_types, segments)) return segment + @override def get_segment(self, collection_id: UUID, type: Type[S]) -> S: if type == MetadataReader: @@ -236,7 +253,6 @@ def _segment(type: SegmentType, scope: SegmentScope, collection: Collection) -> id=uuid4(), type=type.value, scope=scope, - topic=collection["topic"], collection=collection["id"], - metadata=metadata + metadata=metadata, ) diff --git a/chromadb/segment/impl/metadata/sqlite.py b/chromadb/segment/impl/metadata/sqlite.py index 927c1f1fbba4..996c97e1c784 100644 --- a/chromadb/segment/impl/metadata/sqlite.py +++ b/chromadb/segment/impl/metadata/sqlite.py @@ -45,7 +45,7 @@ class SqliteMetadataSegment(MetadataReader): _db: SqliteDB _id: UUID _opentelemetry_client: OpenTelemetryClient - _topic: Optional[str] + _collection_id: Optional[UUID] _subscription: Optional[UUID] def __init__(self, system: System, segment: Segment): @@ -53,15 +53,17 @@ def __init__(self, system: System, segment: Segment): self._consumer = system.instance(Consumer) self._id = segment["id"] self._opentelemetry_client = system.require(OpenTelemetryClient) - self._topic = segment["topic"] + self._collection_id = segment["collection"] @trace_method("SqliteMetadataSegment.start", OpenTelemetryGranularity.ALL) @override def start(self) -> None: - if self._topic: + if self._collection_id: seq_id = self.max_seqid() self._subscription = self._consumer.subscribe( - self._topic, self._write_metadata, start=seq_id + collection_id=self._collection_id, + consume_fn=self._write_metadata, + start=seq_id, ) @trace_method("SqliteMetadataSegment.stop", OpenTelemetryGranularity.ALL) @@ -279,7 +281,8 @@ def _insert_record( except sqlite3.IntegrityError: # Can't use INSERT OR REPLACE here because it changes the primary key. if upsert: - return self._update_record(cur, record) + # Cast here because the OpenTel decorators obfuscate the type + return cast(None, self._update_record(cur, record)) else: logger.warning(f"Insert of existing embedding ID: {record['id']}") # We are trying to add for a record that already exists. Fail the call. diff --git a/chromadb/segment/impl/vector/local_hnsw.py b/chromadb/segment/impl/vector/local_hnsw.py index e4437881b2a9..ec163279ab27 100644 --- a/chromadb/segment/impl/vector/local_hnsw.py +++ b/chromadb/segment/impl/vector/local_hnsw.py @@ -35,7 +35,7 @@ class LocalHnswSegment(VectorReader): _id: UUID _consumer: Consumer - _topic: Optional[str] + _collection: Optional[UUID] _subscription: UUID _settings: Settings _params: HnswParams @@ -56,7 +56,7 @@ class LocalHnswSegment(VectorReader): def __init__(self, system: System, segment: Segment): self._consumer = system.instance(Consumer) self._id = segment["id"] - self._topic = segment["topic"] + self._collection = segment["collection"] self._settings = system.settings self._params = HnswParams(segment["metadata"] or {}) @@ -84,10 +84,10 @@ def propagate_collection_metadata(metadata: Metadata) -> Optional[Metadata]: @override def start(self) -> None: super().start() - if self._topic: + if self._collection: seq_id = self.max_seqid() self._subscription = self._consumer.subscribe( - self._topic, self._write_records, start=seq_id + self._collection, self._write_records, start=seq_id ) @trace_method("LocalHnswSegment.stop", OpenTelemetryGranularity.ALL) diff --git a/chromadb/test/conftest.py b/chromadb/test/conftest.py index 8a8cd9790722..f196c6627474 100644 --- a/chromadb/test/conftest.py +++ b/chromadb/test/conftest.py @@ -17,6 +17,7 @@ Tuple, Callable, ) +from uuid import UUID import hypothesis import pytest @@ -29,7 +30,7 @@ from chromadb.config import Settings, System from chromadb.db.mixins import embeddings_queue from chromadb.ingest import Producer -from chromadb.types import SeqId, SubmitEmbeddingRecord +from chromadb.types import SeqId, OperationRecord from chromadb.api.client import Client as ClientCreator root_logger = logging.getLogger() @@ -527,24 +528,24 @@ class ProducerFn(Protocol): def __call__( self, producer: Producer, - topic: str, - embeddings: Iterator[SubmitEmbeddingRecord], + collection_id: UUID, + embeddings: Iterator[OperationRecord], n: int, - ) -> Tuple[Sequence[SubmitEmbeddingRecord], Sequence[SeqId]]: + ) -> Tuple[Sequence[OperationRecord], Sequence[SeqId]]: ... def produce_n_single( producer: Producer, - topic: str, - embeddings: Iterator[SubmitEmbeddingRecord], + collection_id: UUID, + embeddings: Iterator[OperationRecord], n: int, -) -> Tuple[Sequence[SubmitEmbeddingRecord], Sequence[SeqId]]: +) -> Tuple[Sequence[OperationRecord], Sequence[SeqId]]: submitted_embeddings = [] seq_ids = [] for _ in range(n): e = next(embeddings) - seq_id = producer.submit_embedding(topic, e) + seq_id = producer.submit_embedding(collection_id, e) submitted_embeddings.append(e) seq_ids.append(seq_id) return submitted_embeddings, seq_ids @@ -552,16 +553,16 @@ def produce_n_single( def produce_n_batch( producer: Producer, - topic: str, - embeddings: Iterator[SubmitEmbeddingRecord], + collection_id: UUID, + embeddings: Iterator[OperationRecord], n: int, -) -> Tuple[Sequence[SubmitEmbeddingRecord], Sequence[SeqId]]: +) -> Tuple[Sequence[OperationRecord], Sequence[SeqId]]: submitted_embeddings = [] seq_ids: Sequence[SeqId] = [] for _ in range(n): e = next(embeddings) submitted_embeddings.append(e) - seq_ids = producer.submit_embeddings(topic, submitted_embeddings) + seq_ids = producer.submit_embeddings(collection_id, submitted_embeddings) return submitted_embeddings, seq_ids diff --git a/chromadb/test/db/test_system.py b/chromadb/test/db/test_system.py index e3a8a966bb0c..673fc9a46c1a 100644 --- a/chromadb/test/db/test_system.py +++ b/chromadb/test/db/test_system.py @@ -11,7 +11,6 @@ from chromadb.config import ( DEFAULT_DATABASE, DEFAULT_TENANT, - Component, System, Settings, ) @@ -25,17 +24,10 @@ # These are the sample collections that are used in the tests below. Tests can override # the fields as needed. - -# HACK: In order to get the real grpc tests passing, we need the topic to use rendezvous -# hashing. This is because the grpc tests use the real grpc sysdb server and the -# rendezvous hashing is done in the segment server. We don't have a easy way to parameterize -# the assignment policy in the grpc tests, so we just use rendezvous hashing for all tests. -# by harcoding the topic to what we expect rendezvous hashing to return with 16 topics. sample_collections = [ Collection( id=uuid.UUID(int=1), name="test_collection_1", - topic=f"persistent://{TENANT}/{NAMESPACE}/chroma_log_1", metadata={"test_str": "str1", "test_int": 1, "test_float": 1.3}, dimension=128, database=DEFAULT_DATABASE, @@ -44,7 +36,6 @@ Collection( id=uuid.UUID(int=2), name="test_collection_2", - topic=f"persistent://{TENANT}/{NAMESPACE}/chroma_log_14", metadata={"test_str": "str2", "test_int": 2, "test_float": 2.3}, dimension=None, database=DEFAULT_DATABASE, @@ -53,7 +44,6 @@ Collection( id=uuid.UUID(int=3), name="test_collection_3", - topic=f"persistent://{TENANT}/{NAMESPACE}/chroma_log_14", metadata={"test_str": "str3", "test_int": 3, "test_float": 3.3}, dimension=None, database=DEFAULT_DATABASE, @@ -62,21 +52,12 @@ ] -class MockAssignmentPolicy(Component): - def assign_collection(self, collection_id: uuid.UUID) -> str: - for collection in sample_collections: - if collection["id"] == collection_id: - return collection["topic"] - raise ValueError(f"Unknown collection ID: {collection_id}") - - def sqlite() -> Generator[SysDB, None, None]: """Fixture generator for sqlite DB""" db = SqliteDB( System( Settings( allow_reset=True, - chroma_collection_assignment_policy_impl="chromadb.test.db.test_system.MockAssignmentPolicy", ) ) ) @@ -94,7 +75,6 @@ def sqlite_persistent() -> Generator[SysDB, None, None]: allow_reset=True, is_persistent=True, persist_directory=save_path, - chroma_collection_assignment_policy_impl="chromadb.test.db.test_system.MockAssignmentPolicy", ) ) ) @@ -111,7 +91,6 @@ def grpc_with_mock_server() -> Generator[SysDB, None, None]: system = System( Settings( allow_reset=True, - chroma_collection_assignment_policy_impl="chromadb.test.db.test_system.MockAssignmentPolicy", chroma_server_grpc_port=50051, ) ) @@ -127,7 +106,6 @@ def grpc_with_real_server() -> Generator[SysDB, None, None]: system = System( Settings( allow_reset=True, - chroma_collection_assignment_policy_impl="chromadb.test.db.test_system.MockAssignmentPolicy", chroma_server_grpc_port=50051, ) ) @@ -179,26 +157,11 @@ def test_create_get_delete_collections(sysdb: SysDB) -> None: result = sysdb.get_collections(name=collection["name"]) assert result == [collection] - # Find by topic - for collection in sample_collections: - result = sysdb.get_collections(topic=collection["topic"]) - assert collection in result - # Find by id for collection in sample_collections: result = sysdb.get_collections(id=collection["id"]) assert result == [collection] - # Find by id and topic (positive case) - for collection in sample_collections: - result = sysdb.get_collections(id=collection["id"], topic=collection["topic"]) - assert result == [collection] - - # find by id and topic (negative case) - for collection in sample_collections: - result = sysdb.get_collections(id=collection["id"], topic="other_topic") - assert result == [] - # Delete c1 = sample_collections[0] sysdb.delete_collection(c1["id"]) @@ -220,7 +183,6 @@ def test_update_collections(sysdb: SysDB) -> None: coll = Collection( name=sample_collections[0]["name"], id=sample_collections[0]["id"], - topic=sample_collections[0]["topic"], metadata=sample_collections[0]["metadata"], dimension=sample_collections[0]["dimension"], database=DEFAULT_DATABASE, @@ -242,12 +204,6 @@ def test_update_collections(sysdb: SysDB) -> None: result = sysdb.get_collections(name=coll["name"]) assert result == [coll] - # Update topic - coll["topic"] = "new_topic" - sysdb.update_collection(coll["id"], topic=coll["topic"]) - result = sysdb.get_collections(topic=coll["topic"]) - assert result == [coll] - # Update dimension coll["dimension"] = 128 sysdb.update_collection(coll["id"], dimension=coll["dimension"]) @@ -623,14 +579,12 @@ def test_get_database_with_tenants(sysdb: SysDB) -> None: id=uuid.UUID("00000000-d7d7-413b-92e1-731098a6e492"), type="test_type_a", scope=SegmentScope.VECTOR, - topic=None, collection=sample_collections[0]["id"], metadata={"test_str": "str1", "test_int": 1, "test_float": 1.3}, ), Segment( id=uuid.UUID("11111111-d7d7-413b-92e1-731098a6e492"), type="test_type_b", - topic="test_topic_2", scope=SegmentScope.VECTOR, collection=sample_collections[1]["id"], metadata={"test_str": "str2", "test_int": 2, "test_float": 2.3}, @@ -638,7 +592,6 @@ def test_get_database_with_tenants(sysdb: SysDB) -> None: Segment( id=uuid.UUID("22222222-d7d7-413b-92e1-731098a6e492"), type="test_type_b", - topic="test_topic_3", scope=SegmentScope.METADATA, collection=None, metadata={"test_str": "str3", "test_int": 3, "test_float": 3.3}, @@ -721,7 +674,6 @@ def test_update_segment(sysdb: SysDB) -> None: id=uuid.uuid4(), type="test_type_a", scope=SegmentScope.VECTOR, - topic="test_topic_a", collection=sample_collections[0]["id"], metadata=metadata, ) @@ -735,16 +687,11 @@ def test_update_segment(sysdb: SysDB) -> None: sysdb.create_segment(segment) # TODO: revisit update segment - push collection id - # Update topic to new value - segment["topic"] = "new_topic" - sysdb.update_segment(segment["id"], topic=segment["topic"]) + result = sysdb.get_segments(id=segment["id"]) result[0]["collection"] = segment["collection"] assert result == [segment] - # Update topic to None - segment["topic"] = None - sysdb.update_segment(segment["id"], topic=segment["topic"]) result = sysdb.get_segments(id=segment["id"]) result[0]["collection"] = segment["collection"] assert result == [segment] diff --git a/chromadb/test/ingest/test_producer_consumer.py b/chromadb/test/ingest/test_producer_consumer.py index 31450cb7dfea..7b2a5c23c8cd 100644 --- a/chromadb/test/ingest/test_producer_consumer.py +++ b/chromadb/test/ingest/test_producer_consumer.py @@ -2,6 +2,7 @@ import os import shutil import tempfile +from uuid import UUID import pytest from itertools import count from typing import ( @@ -17,10 +18,9 @@ ) from chromadb.ingest import Producer, Consumer from chromadb.db.impl.sqlite import SqliteDB -from chromadb.ingest.impl.utils import create_topic_name from chromadb.test.conftest import ProducerFn from chromadb.types import ( - SubmitEmbeddingRecord, + OperationRecord, Operation, EmbeddingRecord, ScalarEncoding, @@ -28,7 +28,6 @@ from chromadb.config import System, Settings from pytest import FixtureRequest, approx from asyncio import Event, wait_for, TimeoutError -import uuid def sqlite() -> Generator[Tuple[Producer, Consumer], None, None]: @@ -71,8 +70,8 @@ def producer_consumer( @pytest.fixture(scope="module") -def sample_embeddings() -> Iterator[SubmitEmbeddingRecord]: - def create_record(i: int) -> SubmitEmbeddingRecord: +def sample_embeddings() -> Iterator[OperationRecord]: + def create_record(i: int) -> OperationRecord: vector = [i + i * 0.1, i + 1 + i * 0.1] metadata: Optional[Dict[str, Union[str, int, float]]] if i % 2 == 0: @@ -80,13 +79,12 @@ def create_record(i: int) -> SubmitEmbeddingRecord: else: metadata = {"str_key": f"value_{i}", "int_key": i, "float_key": i + i * 0.1} - record = SubmitEmbeddingRecord( + record = OperationRecord( id=f"embedding_{i}", embedding=vector, encoding=ScalarEncoding.FLOAT32, metadata=metadata, operation=Operation.ADD, - collection_id=uuid.uuid4(), ) return record @@ -131,7 +129,7 @@ def assert_approx_equal(a: Sequence[float], b: Sequence[float]) -> None: def assert_records_match( - inserted_records: Sequence[SubmitEmbeddingRecord], + inserted_records: Sequence[OperationRecord], consumed_records: Sequence[EmbeddingRecord], ) -> None: """Given a list of inserted and consumed records, make sure they match""" @@ -147,25 +145,20 @@ def assert_records_match( assert_approx_equal(inserted["embedding"], consumed["embedding"]) -def full_topic_name(topic_name: str) -> str: - return create_topic_name("default", "default", topic_name) - - @pytest.mark.asyncio async def test_backfill( producer_consumer: Tuple[Producer, Consumer], - sample_embeddings: Iterator[SubmitEmbeddingRecord], + sample_embeddings: Iterator[OperationRecord], produce_fns: ProducerFn, ) -> None: producer, consumer = producer_consumer producer.reset_state() consumer.reset_state() - topic_name = full_topic_name("test_topic") - producer.create_topic(topic_name) - embeddings = produce_fns(producer, topic_name, sample_embeddings, 3)[0] + collection_id = UUID("00000000-0000-0000-0000-000000000000") + embeddings = produce_fns(producer, collection_id, sample_embeddings, 3)[0] consume_fn = CapturingConsumeFn() - consumer.subscribe(topic_name, consume_fn, start=consumer.min_seqid()) + consumer.subscribe(collection_id, consume_fn, start=consumer.min_seqid()) recieved = await consume_fn.get(3) assert_records_match(embeddings, recieved) @@ -174,61 +167,57 @@ async def test_backfill( @pytest.mark.asyncio async def test_notifications( producer_consumer: Tuple[Producer, Consumer], - sample_embeddings: Iterator[SubmitEmbeddingRecord], + sample_embeddings: Iterator[OperationRecord], ) -> None: producer, consumer = producer_consumer producer.reset_state() consumer.reset_state() - topic_name = full_topic_name("test_topic") - - producer.create_topic(topic_name) + collection_id = UUID("00000000-0000-0000-0000-000000000000") - embeddings: List[SubmitEmbeddingRecord] = [] + embeddings: List[OperationRecord] = [] consume_fn = CapturingConsumeFn() - consumer.subscribe(topic_name, consume_fn, start=consumer.min_seqid()) + consumer.subscribe(collection_id, consume_fn, start=consumer.min_seqid()) for i in range(10): e = next(sample_embeddings) embeddings.append(e) - producer.submit_embedding(topic_name, e) + producer.submit_embedding(collection_id, e) received = await consume_fn.get(i + 1) assert_records_match(embeddings, received) @pytest.mark.asyncio -async def test_multiple_topics( +async def test_multiple_collections( producer_consumer: Tuple[Producer, Consumer], - sample_embeddings: Iterator[SubmitEmbeddingRecord], + sample_embeddings: Iterator[OperationRecord], ) -> None: producer, consumer = producer_consumer producer.reset_state() consumer.reset_state() - topic_name_1 = full_topic_name("test_topic_1") - topic_name_2 = full_topic_name("test_topic_2") - producer.create_topic(topic_name_1) - producer.create_topic(topic_name_2) + collection_1 = UUID("00000000-0000-0000-0000-000000000001") + collection_2 = UUID("00000000-0000-0000-0000-000000000002") - embeddings_1: List[SubmitEmbeddingRecord] = [] - embeddings_2: List[SubmitEmbeddingRecord] = [] + embeddings_1: List[OperationRecord] = [] + embeddings_2: List[OperationRecord] = [] consume_fn_1 = CapturingConsumeFn() consume_fn_2 = CapturingConsumeFn() - consumer.subscribe(topic_name_1, consume_fn_1, start=consumer.min_seqid()) - consumer.subscribe(topic_name_2, consume_fn_2, start=consumer.min_seqid()) + consumer.subscribe(collection_1, consume_fn_1, start=consumer.min_seqid()) + consumer.subscribe(collection_2, consume_fn_2, start=consumer.min_seqid()) for i in range(10): e_1 = next(sample_embeddings) embeddings_1.append(e_1) - producer.submit_embedding(topic_name_1, e_1) + producer.submit_embedding(collection_1, e_1) results_2 = await consume_fn_1.get(i + 1) assert_records_match(embeddings_1, results_2) e_2 = next(sample_embeddings) embeddings_2.append(e_2) - producer.submit_embedding(topic_name_2, e_2) + producer.submit_embedding(collection_2, e_2) results_2 = await consume_fn_2.get(i + 1) assert_records_match(embeddings_2, results_2) @@ -236,28 +225,27 @@ async def test_multiple_topics( @pytest.mark.asyncio async def test_start_seq_id( producer_consumer: Tuple[Producer, Consumer], - sample_embeddings: Iterator[SubmitEmbeddingRecord], + sample_embeddings: Iterator[OperationRecord], produce_fns: ProducerFn, ) -> None: producer, consumer = producer_consumer producer.reset_state() consumer.reset_state() - topic_name = full_topic_name("test_topic") - producer.create_topic(topic_name) + collection = UUID("00000000-0000-0000-0000-000000000000") consume_fn_1 = CapturingConsumeFn() consume_fn_2 = CapturingConsumeFn() - consumer.subscribe(topic_name, consume_fn_1, start=consumer.min_seqid()) + consumer.subscribe(collection, consume_fn_1, start=consumer.min_seqid()) - embeddings = produce_fns(producer, topic_name, sample_embeddings, 5)[0] + embeddings = produce_fns(producer, collection, sample_embeddings, 5)[0] results_1 = await consume_fn_1.get(5) assert_records_match(embeddings, results_1) start = consume_fn_1.embeddings[-1]["seq_id"] - consumer.subscribe(topic_name, consume_fn_2, start=start) - second_embeddings = produce_fns(producer, topic_name, sample_embeddings, 5)[0] + consumer.subscribe(collection, consume_fn_2, start=start) + second_embeddings = produce_fns(producer, collection, sample_embeddings, 5)[0] assert isinstance(embeddings, list) embeddings.extend(second_embeddings) results_2 = await consume_fn_2.get(5) @@ -267,27 +255,26 @@ async def test_start_seq_id( @pytest.mark.asyncio async def test_end_seq_id( producer_consumer: Tuple[Producer, Consumer], - sample_embeddings: Iterator[SubmitEmbeddingRecord], + sample_embeddings: Iterator[OperationRecord], produce_fns: ProducerFn, ) -> None: producer, consumer = producer_consumer producer.reset_state() consumer.reset_state() - topic_name = full_topic_name("test_topic") - producer.create_topic(topic_name) + collection = UUID("00000000-0000-0000-0000-000000000000") consume_fn_1 = CapturingConsumeFn() consume_fn_2 = CapturingConsumeFn() - consumer.subscribe(topic_name, consume_fn_1, start=consumer.min_seqid()) + consumer.subscribe(collection, consume_fn_1, start=consumer.min_seqid()) - embeddings = produce_fns(producer, topic_name, sample_embeddings, 10)[0] + embeddings = produce_fns(producer, collection, sample_embeddings, 10)[0] results_1 = await consume_fn_1.get(10) assert_records_match(embeddings, results_1) end = consume_fn_1.embeddings[-5]["seq_id"] - consumer.subscribe(topic_name, consume_fn_2, start=consumer.min_seqid(), end=end) + consumer.subscribe(collection, consume_fn_2, start=consumer.min_seqid(), end=end) results_2 = await consume_fn_2.get(6) assert_records_match(embeddings[:6], results_2) @@ -300,29 +287,28 @@ async def test_end_seq_id( @pytest.mark.asyncio async def test_submit_batch( producer_consumer: Tuple[Producer, Consumer], - sample_embeddings: Iterator[SubmitEmbeddingRecord], + sample_embeddings: Iterator[OperationRecord], ) -> None: producer, consumer = producer_consumer producer.reset_state() consumer.reset_state() - topic_name = full_topic_name("test_topic") + collection = UUID("00000000-0000-0000-0000-000000000000") embeddings = [next(sample_embeddings) for _ in range(100)] - producer.create_topic(topic_name) - producer.submit_embeddings(topic_name, embeddings=embeddings) + producer.submit_embeddings(collection, embeddings=embeddings) consume_fn = CapturingConsumeFn() - consumer.subscribe(topic_name, consume_fn, start=consumer.min_seqid()) + consumer.subscribe(collection, consume_fn, start=consumer.min_seqid()) recieved = await consume_fn.get(100) assert_records_match(embeddings, recieved) @pytest.mark.asyncio -async def test_multiple_topics_batch( +async def test_multiple_collections_batch( producer_consumer: Tuple[Producer, Consumer], - sample_embeddings: Iterator[SubmitEmbeddingRecord], + sample_embeddings: Iterator[OperationRecord], produce_fns: ProducerFn, ) -> None: producer, consumer = producer_consumer @@ -332,14 +318,13 @@ async def test_multiple_topics_batch( N_TOPICS = 2 consume_fns = [CapturingConsumeFn() for _ in range(N_TOPICS)] for i in range(N_TOPICS): - producer.create_topic(full_topic_name(f"test_topic_{i}")) consumer.subscribe( - full_topic_name(f"test_topic_{i}"), + UUID(f"00000000-0000-0000-0000-00000000000{i}"), consume_fns[i], start=consumer.min_seqid(), ) - embeddings_n: List[List[SubmitEmbeddingRecord]] = [[] for _ in range(N_TOPICS)] + embeddings_n: List[List[OperationRecord]] = [[] for _ in range(N_TOPICS)] PRODUCE_BATCH_SIZE = 10 N_TO_PRODUCE = 100 @@ -349,7 +334,7 @@ async def test_multiple_topics_batch( embeddings_n[n].extend( produce_fns( producer, - full_topic_name(f"test_topic_{n}"), + UUID(f"00000000-0000-0000-0000-00000000000{n}"), sample_embeddings, PRODUCE_BATCH_SIZE, )[0] @@ -362,25 +347,25 @@ async def test_multiple_topics_batch( @pytest.mark.asyncio async def test_max_batch_size( producer_consumer: Tuple[Producer, Consumer], - sample_embeddings: Iterator[SubmitEmbeddingRecord], + sample_embeddings: Iterator[OperationRecord], ) -> None: producer, consumer = producer_consumer producer.reset_state() consumer.reset_state() - topic_name = full_topic_name("test_topic") + collection = UUID("00000000-0000-0000-0000-000000000000") max_batch_size = producer.max_batch_size assert max_batch_size > 0 # Make sure that we can produce a batch of size max_batch_size embeddings = [next(sample_embeddings) for _ in range(max_batch_size)] consume_fn = CapturingConsumeFn() - consumer.subscribe(topic_name, consume_fn, start=consumer.min_seqid()) - producer.submit_embeddings(topic_name, embeddings=embeddings) + consumer.subscribe(collection, consume_fn, start=consumer.min_seqid()) + producer.submit_embeddings(collection, embeddings=embeddings) received = await consume_fn.get(max_batch_size, timeout_secs=120) assert_records_match(embeddings, received) embeddings = [next(sample_embeddings) for _ in range(max_batch_size + 1)] # Make sure that we can't produce a batch of size > max_batch_size with pytest.raises(ValueError) as e: - producer.submit_embeddings(topic_name, embeddings=embeddings) + producer.submit_embeddings(collection, embeddings=embeddings) assert "Cannot submit more than" in str(e.value) diff --git a/chromadb/test/property/strategies.py b/chromadb/test/property/strategies.py index 9196e6b22ece..79f7a1fd3260 100644 --- a/chromadb/test/property/strategies.py +++ b/chromadb/test/property/strategies.py @@ -239,11 +239,12 @@ def embedding_function_strategy( class ExternalCollection: """ An external view of a collection. - + This strategy only contains information about a collection that a client of Chroma sees -- that is, it contains none of Chroma's internal bookkeeping. It should be used to test the API and client code. """ + name: str metadata: Optional[types.Metadata] embedding_function: Optional[types.EmbeddingFunction[Embeddable]] @@ -258,10 +259,10 @@ class Collection(ExternalCollection): collection. It is a superset of ExternalCollection and should be used to test internal Chroma logic. """ + id: uuid.UUID dimension: int dtype: npt.DTypeLike - topic: str known_metadata_keys: types.Metadata known_document_keywords: List[str] has_documents: bool = False @@ -332,7 +333,6 @@ def collections( return Collection( id=uuid.uuid4(), name=name, - topic="topic", metadata=metadata, dimension=dimension, dtype=dtype, diff --git a/chromadb/test/segment/test_metadata.py b/chromadb/test/segment/test_metadata.py index aaa8ec13bab1..8d414f093d4c 100644 --- a/chromadb/test/segment/test_metadata.py +++ b/chromadb/test/segment/test_metadata.py @@ -2,7 +2,17 @@ import shutil import tempfile import pytest -from typing import Generator, List, Callable, Iterator, Dict, Optional, Union, Sequence +from typing import ( + Generator, + List, + Callable, + Iterator, + Dict, + Optional, + Union, + Sequence, + cast, +) from chromadb.api.types import validate_metadata from chromadb.config import System, Settings @@ -10,7 +20,7 @@ from chromadb.db.impl.sqlite import SqliteDB from chromadb.test.conftest import ProducerFn from chromadb.types import ( - SubmitEmbeddingRecord, + OperationRecord, MetadataEmbeddingRecord, Operation, ScalarEncoding, @@ -63,8 +73,8 @@ def system(request: FixtureRequest) -> Generator[System, None, None]: @pytest.fixture(scope="function") -def sample_embeddings() -> Iterator[SubmitEmbeddingRecord]: - def create_record(i: int) -> SubmitEmbeddingRecord: +def sample_embeddings() -> Iterator[OperationRecord]: + def create_record(i: int) -> OperationRecord: vector = [i + i * 0.1, i + 1 + i * 0.1] metadata: Optional[Dict[str, Union[str, int, float, bool]]] if i == 0: @@ -82,13 +92,12 @@ def create_record(i: int) -> SubmitEmbeddingRecord: metadata["bool_key"] = False metadata["chroma:document"] = _build_document(i) - record = SubmitEmbeddingRecord( + record = OperationRecord( id=f"embedding_{i}", embedding=vector, encoding=ScalarEncoding.FLOAT32, metadata=metadata, operation=Operation.ADD, - collection_id=uuid.UUID(int=0), ) return record @@ -118,8 +127,7 @@ def _build_document(i: int) -> str: id=uuid.uuid4(), type="test_type", scope=SegmentScope.METADATA, - topic="persistent://test/test/test_topic_1", - collection=None, + collection=uuid.UUID(int=0), metadata=None, ) @@ -127,8 +135,7 @@ def _build_document(i: int) -> str: id=uuid.uuid4(), type="test_type", scope=SegmentScope.METADATA, - topic="persistent://test/test/test_topic_2", - collection=None, + collection=uuid.UUID(int=1), metadata=None, ) @@ -145,15 +152,17 @@ def sync(segment: MetadataReader, seq_id: SeqId) -> None: def test_insert_and_count( system: System, - sample_embeddings: Iterator[SubmitEmbeddingRecord], + sample_embeddings: Iterator[OperationRecord], produce_fns: ProducerFn, ) -> None: producer = system.instance(Producer) system.reset_state() - topic = str(segment_definition["topic"]) + collection_id = segment_definition["collection"] + # We know that the collection_id exists so we can cast + collection_id = cast(uuid.UUID, collection_id) - max_id = produce_fns(producer, topic, sample_embeddings, 3)[1][-1] + max_id = produce_fns(producer, collection_id, sample_embeddings, 3)[1][-1] segment = SqliteMetadataSegment(system, segment_definition) segment.start() @@ -163,7 +172,7 @@ def test_insert_and_count( assert segment.count() == 3 for i in range(3): - max_id = producer.submit_embedding(topic, next(sample_embeddings)) + max_id = producer.submit_embedding(collection_id, next(sample_embeddings)) sync(segment, max_id) @@ -171,7 +180,7 @@ def test_insert_and_count( def assert_equiv_records( - expected: Sequence[SubmitEmbeddingRecord], actual: Sequence[MetadataEmbeddingRecord] + expected: Sequence[OperationRecord], actual: Sequence[MetadataEmbeddingRecord] ) -> None: assert len(expected) == len(actual) sorted_expected = sorted(expected, key=lambda r: r["id"]) @@ -183,14 +192,16 @@ def assert_equiv_records( def test_get( system: System, - sample_embeddings: Iterator[SubmitEmbeddingRecord], + sample_embeddings: Iterator[OperationRecord], produce_fns: ProducerFn, ) -> None: producer = system.instance(Producer) system.reset_state() - topic = str(segment_definition["topic"]) + collection_id = segment_definition["collection"] + # We know that the collection_id exists so we can cast + collection_id = cast(uuid.UUID, collection_id) - embeddings, seq_ids = produce_fns(producer, topic, sample_embeddings, 10) + embeddings, seq_ids = produce_fns(producer, collection_id, sample_embeddings, 10) segment = SqliteMetadataSegment(system, segment_definition) segment.start() @@ -285,17 +296,19 @@ def test_get( def test_fulltext( system: System, - sample_embeddings: Iterator[SubmitEmbeddingRecord], + sample_embeddings: Iterator[OperationRecord], produce_fns: ProducerFn, ) -> None: producer = system.instance(Producer) system.reset_state() - topic = str(segment_definition["topic"]) + collection_id = segment_definition["collection"] + # We know that the collection_id exists so we can cast + collection_id = cast(uuid.UUID, collection_id) segment = SqliteMetadataSegment(system, segment_definition) segment.start() - max_id = produce_fns(producer, topic, sample_embeddings, 100)[1][-1] + max_id = produce_fns(producer, collection_id, sample_embeddings, 100)[1][-1] sync(segment, max_id) @@ -380,17 +393,19 @@ def test_fulltext( def test_delete( system: System, - sample_embeddings: Iterator[SubmitEmbeddingRecord], + sample_embeddings: Iterator[OperationRecord], produce_fns: ProducerFn, ) -> None: producer = system.instance(Producer) system.reset_state() - topic = str(segment_definition["topic"]) + collection_id = segment_definition["collection"] + # We know that the collection_id exists so we can cast + collection_id = cast(uuid.UUID, collection_id) segment = SqliteMetadataSegment(system, segment_definition) segment.start() - embeddings, seq_ids = produce_fns(producer, topic, sample_embeddings, 10) + embeddings, seq_ids = produce_fns(producer, collection_id, sample_embeddings, 10) max_id = seq_ids[-1] sync(segment, max_id) @@ -400,17 +415,16 @@ def test_delete( assert_equiv_records(embeddings[:1], results) # Delete by ID - delete_embedding = SubmitEmbeddingRecord( + delete_embedding = OperationRecord( id="embedding_0", embedding=None, encoding=None, metadata=None, operation=Operation.DELETE, - collection_id=uuid.UUID(int=0), ) - max_id = produce_fns(producer, topic, (delete_embedding for _ in range(1)), 1)[1][ - -1 - ] + max_id = produce_fns( + producer, collection_id, (delete_embedding for _ in range(1)), 1 + )[1][-1] sync(segment, max_id) @@ -418,43 +432,42 @@ def test_delete( assert segment.get_metadata(ids=["embedding_0"]) == [] # Delete is idempotent - max_id = produce_fns(producer, topic, (delete_embedding for _ in range(1)), 1)[1][ - -1 - ] + max_id = produce_fns( + producer, collection_id, (delete_embedding for _ in range(1)), 1 + )[1][-1] sync(segment, max_id) assert segment.count() == 9 assert segment.get_metadata(ids=["embedding_0"]) == [] # re-add - max_id = producer.submit_embedding(topic, embeddings[0]) + max_id = producer.submit_embedding(collection_id, embeddings[0]) sync(segment, max_id) assert segment.count() == 10 results = segment.get_metadata(ids=["embedding_0"]) -def test_update( - system: System, sample_embeddings: Iterator[SubmitEmbeddingRecord] -) -> None: +def test_update(system: System, sample_embeddings: Iterator[OperationRecord]) -> None: producer = system.instance(Producer) system.reset_state() - topic = str(segment_definition["topic"]) + collection_id = segment_definition["collection"] + # We know that the collection_id exists so we can cast + collection_id = cast(uuid.UUID, collection_id) segment = SqliteMetadataSegment(system, segment_definition) segment.start() - _test_update(sample_embeddings, producer, segment, topic, Operation.UPDATE) + _test_update(sample_embeddings, producer, segment, collection_id, Operation.UPDATE) # Update nonexisting ID - update_record = SubmitEmbeddingRecord( + update_record = OperationRecord( id="no_such_id", metadata={"foo": "bar"}, embedding=None, encoding=None, operation=Operation.UPDATE, - collection_id=uuid.UUID(int=0), ) - max_id = producer.submit_embedding(topic, update_record) + max_id = producer.submit_embedding(collection_id, update_record) sync(segment, max_id) results = segment.get_metadata(ids=["no_such_id"]) assert len(results) == 0 @@ -463,30 +476,31 @@ def test_update( def test_upsert( system: System, - sample_embeddings: Iterator[SubmitEmbeddingRecord], + sample_embeddings: Iterator[OperationRecord], produce_fns: ProducerFn, ) -> None: producer = system.instance(Producer) system.reset_state() - topic = str(segment_definition["topic"]) + collection_id = segment_definition["collection"] + # We know that the collection_id exists so we can cast + collection_id = cast(uuid.UUID, collection_id) segment = SqliteMetadataSegment(system, segment_definition) segment.start() - _test_update(sample_embeddings, producer, segment, topic, Operation.UPSERT) + _test_update(sample_embeddings, producer, segment, collection_id, Operation.UPSERT) # upsert previously nonexisting ID - update_record = SubmitEmbeddingRecord( + update_record = OperationRecord( id="no_such_id", metadata={"foo": "bar"}, embedding=None, encoding=None, operation=Operation.UPSERT, - collection_id=uuid.UUID(int=0), ) max_id = produce_fns( producer=producer, - topic=topic, + collection_id=collection_id, embeddings=(update_record for _ in range(1)), n=1, )[1][-1] @@ -496,10 +510,10 @@ def test_upsert( def _test_update( - sample_embeddings: Iterator[SubmitEmbeddingRecord], + sample_embeddings: Iterator[OperationRecord], producer: Producer, segment: MetadataReader, - topic: str, + collection_id: uuid.UUID, op: Operation, ) -> None: """test code common between update and upsert paths""" @@ -508,7 +522,7 @@ def _test_update( max_id = 0 for e in embeddings: - max_id = producer.submit_embedding(topic, e) + max_id = producer.submit_embedding(collection_id, e) sync(segment, max_id) @@ -516,15 +530,14 @@ def _test_update( assert_equiv_records(embeddings[:1], results) # Update embedding with no metadata - update_record = SubmitEmbeddingRecord( + update_record = OperationRecord( id="embedding_0", metadata={"chroma:document": "foo bar"}, embedding=None, encoding=None, operation=op, - collection_id=uuid.UUID(int=0), ) - max_id = producer.submit_embedding(topic, update_record) + max_id = producer.submit_embedding(collection_id, update_record) sync(segment, max_id) results = segment.get_metadata(ids=["embedding_0"]) assert results[0]["metadata"] == {"chroma:document": "foo bar"} @@ -532,15 +545,14 @@ def _test_update( assert results[0]["metadata"] == {"chroma:document": "foo bar"} # Update and overrwrite key - update_record = SubmitEmbeddingRecord( + update_record = OperationRecord( id="embedding_0", metadata={"chroma:document": "biz buz"}, embedding=None, encoding=None, operation=op, - collection_id=uuid.UUID(int=0), ) - max_id = producer.submit_embedding(topic, update_record) + max_id = producer.submit_embedding(collection_id, update_record) sync(segment, max_id) results = segment.get_metadata(ids=["embedding_0"]) assert results[0]["metadata"] == {"chroma:document": "biz buz"} @@ -550,29 +562,27 @@ def _test_update( assert len(results) == 0 # Update and add key - update_record = SubmitEmbeddingRecord( + update_record = OperationRecord( id="embedding_0", metadata={"baz": 42}, embedding=None, encoding=None, operation=op, - collection_id=uuid.UUID(int=0), ) - max_id = producer.submit_embedding(topic, update_record) + max_id = producer.submit_embedding(collection_id, update_record) sync(segment, max_id) results = segment.get_metadata(ids=["embedding_0"]) assert results[0]["metadata"] == {"chroma:document": "biz buz", "baz": 42} # Update and delete key - update_record = SubmitEmbeddingRecord( + update_record = OperationRecord( id="embedding_0", metadata={"chroma:document": None}, embedding=None, encoding=None, operation=op, - collection_id=uuid.UUID(int=0), ) - max_id = producer.submit_embedding(topic, update_record) + max_id = producer.submit_embedding(collection_id, update_record) sync(segment, max_id) results = segment.get_metadata(ids=["embedding_0"]) assert results[0]["metadata"] == {"baz": 42} @@ -582,17 +592,17 @@ def _test_update( def test_limit( system: System, - sample_embeddings: Iterator[SubmitEmbeddingRecord], + sample_embeddings: Iterator[OperationRecord], produce_fns: ProducerFn, ) -> None: producer = system.instance(Producer) system.reset_state() - topic = str(segment_definition["topic"]) - max_id = produce_fns(producer, topic, sample_embeddings, 3)[1][-1] + collection_id = cast(uuid.UUID, segment_definition["collection"]) + max_id = produce_fns(producer, collection_id, sample_embeddings, 3)[1][-1] - topic2 = str(segment_definition2["topic"]) - max_id2 = produce_fns(producer, topic2, sample_embeddings, 3)[1][-1] + collection_id_2 = cast(uuid.UUID, segment_definition2["collection"]) + max_id2 = produce_fns(producer, collection_id_2, sample_embeddings, 3)[1][-1] segment = SqliteMetadataSegment(system, segment_definition) segment.start() @@ -606,7 +616,7 @@ def test_limit( assert segment.count() == 3 for i in range(3): - max_id = producer.submit_embedding(topic, next(sample_embeddings)) + max_id = producer.submit_embedding(collection_id, next(sample_embeddings)) sync(segment, max_id) @@ -626,17 +636,19 @@ def test_limit( def test_delete_segment( system: System, - sample_embeddings: Iterator[SubmitEmbeddingRecord], + sample_embeddings: Iterator[OperationRecord], produce_fns: ProducerFn, ) -> None: producer = system.instance(Producer) system.reset_state() - topic = str(segment_definition["topic"]) + collection_id = segment_definition["collection"] + # We know that the collection_id exists so we can cast + collection_id = cast(uuid.UUID, collection_id) segment = SqliteMetadataSegment(system, segment_definition) segment.start() - embeddings, seq_ids = produce_fns(producer, topic, sample_embeddings, 10) + embeddings, seq_ids = produce_fns(producer, collection_id, sample_embeddings, 10) max_id = seq_ids[-1] sync(segment, max_id) @@ -682,18 +694,20 @@ def test_delete_segment( def test_delete_single_fts_record( - system: System, - sample_embeddings: Iterator[SubmitEmbeddingRecord], - produce_fns: ProducerFn, + system: System, + sample_embeddings: Iterator[OperationRecord], + produce_fns: ProducerFn, ) -> None: producer = system.instance(Producer) system.reset_state() - topic = str(segment_definition["topic"]) + collection_id = segment_definition["collection"] + # We know that the collection_id exists so we can cast + collection_id = cast(uuid.UUID, collection_id) segment = SqliteMetadataSegment(system, segment_definition) segment.start() - embeddings, seq_ids = produce_fns(producer, topic, sample_embeddings, 10) + embeddings, seq_ids = produce_fns(producer, collection_id, sample_embeddings, 10) max_id = seq_ids[-1] sync(segment, max_id) @@ -704,17 +718,16 @@ def test_delete_single_fts_record( _id = segment._id _db = system.instance(SqliteDB) # Delete by ID - delete_embedding = SubmitEmbeddingRecord( + delete_embedding = OperationRecord( id="embedding_0", embedding=None, encoding=None, metadata=None, operation=Operation.DELETE, - collection_id=uuid.UUID(int=0), ) - max_id = produce_fns(producer, topic, (delete_embedding for _ in range(1)), 1)[1][ - -1 - ] + max_id = produce_fns( + producer, collection_id, (delete_embedding for _ in range(1)), 1 + )[1][-1] t = Table("embeddings") sync(segment, max_id) diff --git a/chromadb/test/segment/test_vector.py b/chromadb/test/segment/test_vector.py index 1ba9802c66fd..4ccc57865477 100644 --- a/chromadb/test/segment/test_vector.py +++ b/chromadb/test/segment/test_vector.py @@ -3,7 +3,7 @@ from chromadb.config import System, Settings from chromadb.test.conftest import ProducerFn from chromadb.types import ( - SubmitEmbeddingRecord, + OperationRecord, VectorQuery, Operation, ScalarEncoding, @@ -78,20 +78,19 @@ def system(request: FixtureRequest) -> Generator[System, None, None]: @pytest.fixture(scope="function") -def sample_embeddings() -> Iterator[SubmitEmbeddingRecord]: +def sample_embeddings() -> Iterator[OperationRecord]: """Generate a sequence of embeddings with the property that for each embedding (other than the first and last), it's nearest neighbor is the previous in the sequence, and it's second nearest neighbor is the subsequent""" - def create_record(i: int) -> SubmitEmbeddingRecord: + def create_record(i: int) -> OperationRecord: vector = [i**1.1, i**1.1] - record = SubmitEmbeddingRecord( + record = OperationRecord( id=f"embedding_{i}", embedding=vector, encoding=ScalarEncoding.FLOAT32, metadata=None, operation=Operation.ADD, - collection_id=uuid.UUID(int=0), ) return record @@ -112,8 +111,7 @@ def create_random_segment_definition() -> Segment: id=uuid.uuid4(), type="test_type", scope=SegmentScope.VECTOR, - topic="persistent://test/test/test_topic_1", - collection=None, + collection=uuid.UUID(int=0), metadata=test_hnsw_config, ) @@ -130,7 +128,7 @@ def sync(segment: VectorReader, seq_id: SeqId) -> None: def test_insert_and_count( system: System, - sample_embeddings: Iterator[SubmitEmbeddingRecord], + sample_embeddings: Iterator[OperationRecord], vector_reader: Type[VectorReader], produce_fns: ProducerFn, ) -> None: @@ -138,10 +136,15 @@ def test_insert_and_count( system.reset_state() segment_definition = create_random_segment_definition() - topic = str(segment_definition["topic"]) + collection_id = segment_definition["collection"] + # We know that the segment definition has a collection_id + collection_id = cast(uuid.UUID, collection_id) max_id = produce_fns( - producer=producer, topic=topic, n=3, embeddings=sample_embeddings + producer=producer, + collection_id=collection_id, + n=3, + embeddings=sample_embeddings, )[1][-1] segment = vector_reader(system, segment_definition) @@ -152,7 +155,10 @@ def test_insert_and_count( assert segment.count() == 3 max_id = produce_fns( - producer=producer, topic=topic, n=3, embeddings=sample_embeddings + producer=producer, + collection_id=collection_id, + n=3, + embeddings=sample_embeddings, )[1][-1] sync(segment, max_id) @@ -169,20 +175,25 @@ def approx_equal_vector(a: Vector, b: Vector, epsilon: float = 0.0001) -> bool: def test_get_vectors( system: System, - sample_embeddings: Iterator[SubmitEmbeddingRecord], + sample_embeddings: Iterator[OperationRecord], vector_reader: Type[VectorReader], produce_fns: ProducerFn, ) -> None: producer = system.instance(Producer) system.reset_state() segment_definition = create_random_segment_definition() - topic = str(segment_definition["topic"]) + collection_id = segment_definition["collection"] + # We know that the segment definition has a collection_id + collection_id = cast(uuid.UUID, collection_id) segment = vector_reader(system, segment_definition) segment.start() embeddings, seq_ids = produce_fns( - producer=producer, topic=topic, embeddings=sample_embeddings, n=10 + producer=producer, + collection_id=collection_id, + embeddings=sample_embeddings, + n=10, ) sync(segment, seq_ids[-1]) @@ -213,20 +224,25 @@ def test_get_vectors( def test_ann_query( system: System, - sample_embeddings: Iterator[SubmitEmbeddingRecord], + sample_embeddings: Iterator[OperationRecord], vector_reader: Type[VectorReader], produce_fns: ProducerFn, ) -> None: producer = system.instance(Producer) system.reset_state() segment_definition = create_random_segment_definition() - topic = str(segment_definition["topic"]) + collection_id = segment_definition["collection"] + # We know that the segment definition has a collection_id + collection_id = cast(uuid.UUID, collection_id) segment = vector_reader(system, segment_definition) segment.start() embeddings, seq_ids = produce_fns( - producer=producer, topic=topic, embeddings=sample_embeddings, n=100 + producer=producer, + collection_id=collection_id, + embeddings=sample_embeddings, + n=100, ) sync(segment, seq_ids[-1]) @@ -277,38 +293,42 @@ def test_ann_query( def test_delete( system: System, - sample_embeddings: Iterator[SubmitEmbeddingRecord], + sample_embeddings: Iterator[OperationRecord], vector_reader: Type[VectorReader], produce_fns: ProducerFn, ) -> None: producer = system.instance(Producer) system.reset_state() segment_definition = create_random_segment_definition() - topic = str(segment_definition["topic"]) + collection_id = segment_definition["collection"] + # We know that the segment definition has a collection_id + collection_id = cast(uuid.UUID, collection_id) segment = vector_reader(system, segment_definition) segment.start() embeddings, seq_ids = produce_fns( - producer=producer, topic=topic, embeddings=sample_embeddings, n=5 + producer=producer, + collection_id=collection_id, + embeddings=sample_embeddings, + n=5, ) sync(segment, seq_ids[-1]) assert segment.count() == 5 - delete_record = SubmitEmbeddingRecord( + delete_record = OperationRecord( id=embeddings[0]["id"], embedding=None, encoding=None, metadata=None, operation=Operation.DELETE, - collection_id=uuid.UUID(int=0), ) assert isinstance(seq_ids, List) seq_ids.append( produce_fns( producer=producer, - topic=topic, + collection_id=collection_id, n=1, embeddings=(delete_record for _ in range(1)), )[1][0] @@ -344,7 +364,7 @@ def test_delete( seq_ids.append( produce_fns( producer=producer, - topic=topic, + collection_id=collection_id, n=1, embeddings=(delete_record for _ in range(1)), )[1][0] @@ -357,9 +377,9 @@ def test_delete( def _test_update( producer: Producer, - topic: str, + collection_id: uuid.UUID, segment: VectorReader, - sample_embeddings: Iterator[SubmitEmbeddingRecord], + sample_embeddings: Iterator[OperationRecord], operation: Operation, ) -> None: """Tests the common code paths between update & upsert""" @@ -368,21 +388,20 @@ def _test_update( seq_ids: List[SeqId] = [] for e in embeddings: - seq_ids.append(producer.submit_embedding(topic, e)) + seq_ids.append(producer.submit_embedding(collection_id, e)) sync(segment, seq_ids[-1]) assert segment.count() == 3 seq_ids.append( producer.submit_embedding( - topic, - SubmitEmbeddingRecord( + collection_id, + OperationRecord( id=embeddings[0]["id"], embedding=[10.0, 10.0], encoding=ScalarEncoding.FLOAT32, metadata=None, operation=operation, - collection_id=uuid.UUID(int=0), ), ) ) @@ -419,32 +438,33 @@ def _test_update( def test_update( system: System, - sample_embeddings: Iterator[SubmitEmbeddingRecord], + sample_embeddings: Iterator[OperationRecord], vector_reader: Type[VectorReader], produce_fns: ProducerFn, ) -> None: producer = system.instance(Producer) system.reset_state() segment_definition = create_random_segment_definition() - topic = str(segment_definition["topic"]) + collection_id = segment_definition["collection"] + # We know that the segment definition has a collection_id + collection_id = cast(uuid.UUID, collection_id) segment = vector_reader(system, segment_definition) segment.start() - _test_update(producer, topic, segment, sample_embeddings, Operation.UPDATE) + _test_update(producer, collection_id, segment, sample_embeddings, Operation.UPDATE) # test updating a nonexistent record - update_record = SubmitEmbeddingRecord( + update_record = OperationRecord( id="no_such_record", embedding=[10.0, 10.0], encoding=ScalarEncoding.FLOAT32, metadata=None, operation=Operation.UPDATE, - collection_id=uuid.UUID(int=0), ) seq_id = produce_fns( producer=producer, - topic=topic, + collection_id=collection_id, n=1, embeddings=(update_record for _ in range(1)), )[1][0] @@ -457,32 +477,33 @@ def test_update( def test_upsert( system: System, - sample_embeddings: Iterator[SubmitEmbeddingRecord], + sample_embeddings: Iterator[OperationRecord], vector_reader: Type[VectorReader], produce_fns: ProducerFn, ) -> None: producer = system.instance(Producer) system.reset_state() segment_definition = create_random_segment_definition() - topic = str(segment_definition["topic"]) + collection_id = segment_definition["collection"] + # We know that the segment definition has a collection_id + collection_id = cast(uuid.UUID, collection_id) segment = vector_reader(system, segment_definition) segment.start() - _test_update(producer, topic, segment, sample_embeddings, Operation.UPSERT) + _test_update(producer, collection_id, segment, sample_embeddings, Operation.UPSERT) # test updating a nonexistent record - upsert_record = SubmitEmbeddingRecord( + upsert_record = OperationRecord( id="no_such_record", embedding=[42, 42], encoding=ScalarEncoding.FLOAT32, metadata=None, operation=Operation.UPSERT, - collection_id=uuid.UUID(int=0), ) seq_id = produce_fns( producer=producer, - topic=topic, + collection_id=collection_id, n=1, embeddings=(upsert_record for _ in range(1)), )[1][0] @@ -502,62 +523,67 @@ def test_delete_without_add( producer = system.instance(Producer) system.reset_state() segment_definition = create_random_segment_definition() - topic = str(segment_definition["topic"]) + collection_id = segment_definition["collection"] + # We know that the segment definition has a collection_id + collection_id = cast(uuid.UUID, collection_id) segment = vector_reader(system, segment_definition) segment.start() assert segment.count() == 0 - delete_record = SubmitEmbeddingRecord( + delete_record = OperationRecord( id="not_in_db", embedding=None, encoding=None, metadata=None, operation=Operation.DELETE, - collection_id=uuid.UUID(int=0), ) try: - producer.submit_embedding(topic, delete_record) + producer.submit_embedding(collection_id, delete_record) except BaseException: pytest.fail("Unexpected error. Deleting on an empty segment should not raise.") def test_delete_with_local_segment_storage( system: System, - sample_embeddings: Iterator[SubmitEmbeddingRecord], + sample_embeddings: Iterator[OperationRecord], vector_reader: Type[VectorReader], produce_fns: ProducerFn, ) -> None: producer = system.instance(Producer) system.reset_state() segment_definition = create_random_segment_definition() - topic = str(segment_definition["topic"]) + collection_id = segment_definition["collection"] + # We know that the segment definition has a collection_id + collection_id = cast(uuid.UUID, collection_id) segment = vector_reader(system, segment_definition) segment.start() embeddings, seq_ids = produce_fns( - producer=producer, topic=topic, embeddings=sample_embeddings, n=5 + producer=producer, + collection_id=collection_id, + embeddings=sample_embeddings, + n=5, ) sync(segment, seq_ids[-1]) assert segment.count() == 5 - delete_record = SubmitEmbeddingRecord( + delete_record = OperationRecord( id=embeddings[0]["id"], embedding=None, encoding=None, metadata=None, operation=Operation.DELETE, - collection_id=uuid.UUID(int=0), ) assert isinstance(seq_ids, List) seq_ids.append( produce_fns( producer=producer, - topic=topic, + collection_id=collection_id, n=1, embeddings=(delete_record for _ in range(1)), )[1][0] @@ -602,38 +628,42 @@ def test_delete_with_local_segment_storage( def test_reset_state_ignored_for_allow_reset_false( system: System, - sample_embeddings: Iterator[SubmitEmbeddingRecord], + sample_embeddings: Iterator[OperationRecord], vector_reader: Type[VectorReader], produce_fns: ProducerFn, ) -> None: producer = system.instance(Producer) system.reset_state() segment_definition = create_random_segment_definition() - topic = str(segment_definition["topic"]) + collection_id = segment_definition["collection"] + # We know that the segment definition has a collection_id + collection_id = cast(uuid.UUID, collection_id) segment = vector_reader(system, segment_definition) segment.start() embeddings, seq_ids = produce_fns( - producer=producer, topic=topic, embeddings=sample_embeddings, n=5 + producer=producer, + collection_id=collection_id, + embeddings=sample_embeddings, + n=5, ) sync(segment, seq_ids[-1]) assert segment.count() == 5 - delete_record = SubmitEmbeddingRecord( + delete_record = OperationRecord( id=embeddings[0]["id"], embedding=None, encoding=None, metadata=None, operation=Operation.DELETE, - collection_id=uuid.UUID(int=0), ) assert isinstance(seq_ids, List) seq_ids.append( produce_fns( producer=producer, - topic=topic, + collection_id=collection_id, n=1, embeddings=(delete_record for _ in range(1)), )[1][0] diff --git a/chromadb/types.py b/chromadb/types.py index 96597e18033d..1940214b4c9d 100644 --- a/chromadb/types.py +++ b/chromadb/types.py @@ -26,7 +26,6 @@ class SegmentScope(Enum): class Collection(TypedDict): id: UUID name: str - topic: str metadata: Optional[Metadata] dimension: Optional[int] tenant: str @@ -47,9 +46,6 @@ class Segment(TypedDict): id: UUID type: NamespacedName scope: SegmentScope - # If a segment has a topic, it implies that this segment is a consumer of the topic - # and indexes the contents of the topic. - topic: Optional[str] # If a segment has a collection, it implies that this segment implements the full # collection and can be used to service queries (for it's given scope.) collection: Optional[UUID] @@ -108,13 +104,12 @@ class EmbeddingRecord(TypedDict): collection_id: Optional[UUID] -class SubmitEmbeddingRecord(TypedDict): +class OperationRecord(TypedDict): id: str embedding: Optional[Vector] encoding: Optional[ScalarEncoding] metadata: Optional[UpdateMetadata] operation: Operation - collection_id: UUID # The collection the operation is being performed on class VectorQuery(TypedDict): diff --git a/go/cmd/coordinator/cmd.go b/go/cmd/coordinator/cmd.go index 896618a73cc2..c105eafc57f3 100644 --- a/go/cmd/coordinator/cmd.go +++ b/go/cmd/coordinator/cmd.go @@ -56,7 +56,6 @@ func init() { Cmd.Flags().StringVar(&conf.KubernetesNamespace, "kubernetes-namespace", "chroma", "Kubernetes namespace") Cmd.Flags().StringVar(&conf.WorkerMemberlistName, "worker-memberlist-name", "query-service-memberlist", "Worker memberlist name") Cmd.Flags().StringVar(&conf.WorkerPodLabel, "worker-pod-label", "query-service", "Worker pod label") - Cmd.Flags().StringVar(&conf.AssignmentPolicy, "assignment-policy", "rendezvous", "Assignment policy") Cmd.Flags().DurationVar(&conf.WatchInterval, "watch-interval", 60*time.Second, "Watch interval") } diff --git a/go/go.mod b/go/go.mod index e96303639d53..cd45083f25e5 100644 --- a/go/go.mod +++ b/go/go.mod @@ -5,7 +5,7 @@ go 1.21 require ( ariga.io/atlas-provider-gorm v0.3.1 github.com/apache/pulsar-client-go v0.9.1-0.20231030094548-620ecf4addfb - github.com/google/uuid v1.4.0 + github.com/google/uuid v1.6.0 github.com/pingcap/log v1.1.0 github.com/rs/zerolog v1.31.0 github.com/spf13/cobra v1.7.0 @@ -17,8 +17,8 @@ require ( go.opentelemetry.io/otel/trace v1.24.0 go.uber.org/automaxprocs v1.5.3 go.uber.org/zap v1.26.0 - google.golang.org/grpc v1.61.1 - google.golang.org/protobuf v1.32.0 + google.golang.org/grpc v1.62.1 + google.golang.org/protobuf v1.33.0 gorm.io/driver/sqlite v1.5.4 gorm.io/gorm v1.25.7 k8s.io/apimachinery v0.28.3 @@ -63,7 +63,7 @@ require ( go.uber.org/atomic v1.9.0 // indirect golang.org/x/mod v0.15.0 // indirect golang.org/x/tools v0.17.0 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20240102182953-50ed04b92917 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20240123012728-ef4313101c80 // indirect gorm.io/driver/mysql v1.5.2 // indirect gorm.io/driver/sqlserver v1.5.2 // indirect ) @@ -79,7 +79,7 @@ require ( github.com/go-openapi/swag v0.22.3 // indirect github.com/go-sql-driver/mysql v1.7.1 // indirect github.com/gogo/protobuf v1.3.2 // indirect - github.com/golang/protobuf v1.5.3 // indirect + github.com/golang/protobuf v1.5.4 // indirect github.com/google/gnostic-models v0.6.8 // indirect github.com/google/go-cmp v0.6.0 // indirect github.com/google/gofuzz v1.2.0 // indirect @@ -105,15 +105,15 @@ require ( github.com/spf13/pflag v1.0.5 // indirect github.com/stretchr/objx v0.5.1 // indirect go.uber.org/multierr v1.11.0 // indirect - golang.org/x/crypto v0.18.0 // indirect - golang.org/x/net v0.20.0 // indirect - golang.org/x/oauth2 v0.15.0 // indirect - golang.org/x/sys v0.17.0 // indirect - golang.org/x/term v0.16.0 // indirect + golang.org/x/crypto v0.21.0 // indirect + golang.org/x/net v0.22.0 // indirect + golang.org/x/oauth2 v0.16.0 // indirect + golang.org/x/sys v0.18.0 // indirect + golang.org/x/term v0.18.0 // indirect golang.org/x/text v0.14.0 // indirect golang.org/x/time v0.3.0 // indirect google.golang.org/appengine v1.6.8 // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20240102182953-50ed04b92917 + google.golang.org/genproto/googleapis/rpc v0.0.0-20240325203815-454cdb8f5daa gopkg.in/inf.v0 v0.9.1 // indirect gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect gopkg.in/yaml.v2 v2.4.0 // indirect diff --git a/go/go.sum b/go/go.sum index 2e0c93785672..9bc3edc2f273 100644 --- a/go/go.sum +++ b/go/go.sum @@ -131,6 +131,8 @@ github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaS github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= github.com/golang/protobuf v1.5.3 h1:KhyjKVUg7Usr/dYsdSqoFveMYd5ko72D+zANwlG1mmg= github.com/golang/protobuf v1.5.3/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= +github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek= +github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps= github.com/golang/snappy v0.0.1 h1:Qgr9rKW7uDUkrbSmQeiDsGa8SjGyCOGtuasMWwvp2P4= github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/google/gnostic-models v0.6.8 h1:yo/ABAfM5IMRsS1VnXjTBvUb61tFIHozhlYvRgGre9I= @@ -151,6 +153,8 @@ github.com/google/pprof v0.0.0-20210720184732-4bb14d4b1be1/go.mod h1:kpwsk12EmLe github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.4.0 h1:MtMxsa51/r9yyhkyLsVeVt0B+BGQZzpQiTQ4eHZ8bc4= github.com/google/uuid v1.4.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= +github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/gorilla/mux v1.7.4/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= github.com/gorilla/securecookie v1.1.1/go.mod h1:ra0sb63/xPlUeL+yeDciTfxMRAA+MP+HVt/4epWDjd4= github.com/gorilla/sessions v1.2.1/go.mod h1:dk2InVEVJ0sfLlnXv9EAgkf6ecYs/i80K/zI+bUmuGM= @@ -364,6 +368,8 @@ golang.org/x/crypto v0.9.0/go.mod h1:yrmDGqONDYtNj3tH8X9dzUun2m2lzPa9ngI6/RUPGR0 golang.org/x/crypto v0.12.0/go.mod h1:NF0Gs7EO5K4qLn+Ylc+fih8BSTeIjAP05siRnAh98yw= golang.org/x/crypto v0.18.0 h1:PGVlW0xEltQnzFZ55hkuX5+KLyrMYhHld1YHO4AKcdc= golang.org/x/crypto v0.18.0/go.mod h1:R0j02AL6hcrfOiy9T4ZYp/rcWeMxM3L6QYxlOuEG1mg= +golang.org/x/crypto v0.21.0 h1:X31++rzVUdKhX5sWmSOFZxx8UW/ldWx55cbf08iNAMA= +golang.org/x/crypto v0.21.0/go.mod h1:0BP7YvVV9gBbVKyeTG0Gyn+gZm94bibOW5BjDEYAOMs= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= @@ -394,9 +400,13 @@ golang.org/x/net v0.10.0/go.mod h1:0qNGK6F8kojg2nk9dLZ2mShWaEBan6FAoqfSigmmuDg= golang.org/x/net v0.14.0/go.mod h1:PpSgVXXLK0OxS0F31C1/tv6XNguvCrnXIDrFMspZIUI= golang.org/x/net v0.20.0 h1:aCL9BSgETF1k+blQaYUBx9hJ9LOGP3gAVemcZlf1Kpo= golang.org/x/net v0.20.0/go.mod h1:z8BVo6PvndSri0LbOE3hAn0apkU+1YvI6E70E9jsnvY= +golang.org/x/net v0.22.0 h1:9sGLhx7iRIHEiX0oAJ3MRZMUCElJgy7Br1nO+AMN3Tc= +golang.org/x/net v0.22.0/go.mod h1:JKghWKKOSdJwpW2GEx0Ja7fmaKnMsbu+MWVZTokSYmg= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.15.0 h1:s8pnnxNVzjWyrvYdFUQq5llS1PX2zhPXmccZv99h7uQ= golang.org/x/oauth2 v0.15.0/go.mod h1:q48ptWNTY5XWf+JNten23lcvHpLJ0ZSxF5ttTHKVCAM= +golang.org/x/oauth2 v0.16.0 h1:aDkGMBSYxElaoP81NpoUoz2oo2R2wHdZpGToUxfyQrQ= +golang.org/x/oauth2 v0.16.0/go.mod h1:hqZ+0LWXsiVoZpeld6jVt06P3adbS2Uu911W1SsJv2o= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -433,6 +443,8 @@ golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.12.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.17.0 h1:25cE3gD+tdBA7lp7QfhuV+rJiE9YXTcS3VG1SqssI/Y= golang.org/x/sys v0.17.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.18.0 h1:DBdB3niSjOA/O0blCZBqDefyWNYveAYMNF1Wum0DYQ4= +golang.org/x/sys v0.18.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= @@ -441,6 +453,8 @@ golang.org/x/term v0.8.0/go.mod h1:xPskH00ivmX89bAKVGSKKtLOWNx2+17Eiy94tnKShWo= golang.org/x/term v0.11.0/go.mod h1:zC9APTIj3jG3FdV/Ons+XE1riIZXG4aZ4GTHiPZJPIU= golang.org/x/term v0.16.0 h1:m+B6fahuftsE9qjo0VWp2FW0mB3MTJvR0BaMQrq0pmE= golang.org/x/term v0.16.0/go.mod h1:yn7UURbUtPyrVJPGPq404EukNFxcm/foM+bV/bfcDsY= +golang.org/x/term v0.18.0 h1:FcHjZXDMxI8mM3nwhX9HlKop4C0YQvCVCdwYl2wOtE8= +golang.org/x/term v0.18.0/go.mod h1:ILwASektA3OnRv7amZ1xhE/KTR+u50pbXfZ03+6Nx58= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= @@ -476,12 +490,19 @@ google.golang.org/appengine v1.6.8 h1:IhEN5q69dyKagZPYMSdIjS2HqprW324FRQZJcGqPAs google.golang.org/appengine v1.6.8/go.mod h1:1jJ3jBArFh5pcgW8gCtRJnepW8FzD1V44FJffLiz/Ds= google.golang.org/genproto v0.0.0-20231212172506-995d672761c0 h1:YJ5pD9rF8o9Qtta0Cmy9rdBwkSjrTCT6XTiUQVOtIos= google.golang.org/genproto v0.0.0-20231212172506-995d672761c0/go.mod h1:l/k7rMz0vFTBPy+tFSGvXEd3z+BcoG1k7EHbqm+YBsY= +google.golang.org/genproto v0.0.0-20240123012728-ef4313101c80 h1:KAeGQVN3M9nD0/bQXnr/ClcEMJ968gUXJQ9pwfSynuQ= google.golang.org/genproto/googleapis/api v0.0.0-20240102182953-50ed04b92917 h1:rcS6EyEaoCO52hQDupoSfrxI3R6C2Tq741is7X8OvnM= google.golang.org/genproto/googleapis/api v0.0.0-20240102182953-50ed04b92917/go.mod h1:CmlNWB9lSezaYELKS5Ym1r44VrrbPUa7JTvw+6MbpJ0= +google.golang.org/genproto/googleapis/api v0.0.0-20240123012728-ef4313101c80 h1:Lj5rbfG876hIAYFjqiJnPHfhXbv+nzTWfm04Fg/XSVU= +google.golang.org/genproto/googleapis/api v0.0.0-20240123012728-ef4313101c80/go.mod h1:4jWUdICTdgc3Ibxmr8nAJiiLHwQBY0UI0XZcEMaFKaA= google.golang.org/genproto/googleapis/rpc v0.0.0-20240102182953-50ed04b92917 h1:6G8oQ016D88m1xAKljMlBOOGWDZkes4kMhgGFlf8WcQ= google.golang.org/genproto/googleapis/rpc v0.0.0-20240102182953-50ed04b92917/go.mod h1:xtjpI3tXFPP051KaWnhvxkiubL/6dJ18vLVf7q2pTOU= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240325203815-454cdb8f5daa h1:RBgMaUMP+6soRkik4VoN8ojR2nex2TqZwjSSogic+eo= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240325203815-454cdb8f5daa/go.mod h1:WtryC6hu0hhx87FDGxWCDptyssuo68sk10vYjF+T9fY= google.golang.org/grpc v1.61.1 h1:kLAiWrZs7YeDM6MumDe7m3y4aM6wacLzM1Y/wiLP9XY= google.golang.org/grpc v1.61.1/go.mod h1:VUbo7IFqmF1QtCAstipjG0GIoq49KvMe9+h1jFLBNJs= +google.golang.org/grpc v1.62.1 h1:B4n+nfKzOICUXMgyrNd19h/I9oH0L1pizfk1d4zSgTk= +google.golang.org/grpc v1.62.1/go.mod h1:IWTG0VlJLCh1SkC58F7np9ka9mx/WNkjl4PGJaiq+QE= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= @@ -492,6 +513,8 @@ google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp0 google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= google.golang.org/protobuf v1.32.0 h1:pPC6BG5ex8PDFnkbrGU3EixyhKcQ2aDuBS36lqK/C7I= google.golang.org/protobuf v1.32.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= +google.golang.org/protobuf v1.33.0 h1:uNO2rsAINq/JlFpSdYEKIZ0uKD/R9cpdv0T+yoGwGmI= +google.golang.org/protobuf v1.33.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= diff --git a/go/migrations/20240327075032.sql b/go/migrations/20240327075032.sql new file mode 100644 index 000000000000..e98caf50adb0 --- /dev/null +++ b/go/migrations/20240327075032.sql @@ -0,0 +1,4 @@ +-- Modify "collections" table +ALTER TABLE "collections" DROP COLUMN "topic"; +-- Modify "segments" table +ALTER TABLE "segments" DROP COLUMN "topic"; diff --git a/go/migrations/atlas.sum b/go/migrations/atlas.sum index 6e102db3e0b1..44d15d1d915d 100644 --- a/go/migrations/atlas.sum +++ b/go/migrations/atlas.sum @@ -1,3 +1,4 @@ -h1:/GQrxINPAbkke6SbB5PIM8idH6RIYDFUvT8K9eBkbHU= +h1:Uk5EXzkUN9oinZXA4sMmuQMTXRTtTpkxBMD6Gv2dxw4= 20240313233558.sql h1:shyeY6BuLGJ1Ia/G/hH+NZS6HZqHxhBJ2Pfdoeerz7I= 20240321194713.sql h1:K5CAwiFb9kx+O8E/3Dq2C7jzMa7P+ZvqGL5HtLKe2YU= +20240327075032.sql h1:zE+/KCknuhtExHiKoZSfhFzahpbs2B7O/JgYbfxkjp0= diff --git a/go/pkg/common/errors.go b/go/pkg/common/errors.go index 209ea7a21af2..53da456208f0 100644 --- a/go/pkg/common/errors.go +++ b/go/pkg/common/errors.go @@ -17,7 +17,6 @@ var ( ErrCollectionNotFound = errors.New("collection not found") ErrCollectionIDFormat = errors.New("collection id format error") ErrCollectionNameEmpty = errors.New("collection name is empty") - ErrCollectionTopicEmpty = errors.New("collection topic is empty") ErrCollectionUniqueConstraintViolation = errors.New("collection unique constraint violation") ErrCollectionDeleteNonExistingCollection = errors.New("delete non existing collection") ErrCollectionLogPositionStale = errors.New("collection log position Stale") @@ -30,7 +29,6 @@ var ( // Segment errors ErrSegmentIDFormat = errors.New("segment id format error") - ErrInvalidTopicUpdate = errors.New("invalid topic update, reset topic true and topic value not empty") ErrInvalidCollectionUpdate = errors.New("invalid collection update, reset collection true and collection value not empty") ErrSegmentUniqueConstraintViolation = errors.New("unique constraint violation") ErrSegmentDeleteNonExistingSegment = errors.New("delete non existing segment") diff --git a/go/pkg/coordinator/apis.go b/go/pkg/coordinator/apis.go index 13f75943c78b..a45dab0102c5 100644 --- a/go/pkg/coordinator/apis.go +++ b/go/pkg/coordinator/apis.go @@ -2,6 +2,7 @@ package coordinator import ( "context" + "github.com/chroma-core/chroma/go/pkg/common" "github.com/chroma-core/chroma/go/pkg/metastore/db/dbmodel" "github.com/chroma-core/chroma/go/pkg/model" @@ -17,11 +18,11 @@ type ICoordinator interface { common.Component ResetState(ctx context.Context) error CreateCollection(ctx context.Context, createCollection *model.CreateCollection) (*model.Collection, error) - GetCollections(ctx context.Context, collectionID types.UniqueID, collectionName *string, collectionTopic *string, tenantID string, dataName string) ([]*model.Collection, error) + GetCollections(ctx context.Context, collectionID types.UniqueID, collectionName *string, tenantID string, dataName string) ([]*model.Collection, error) DeleteCollection(ctx context.Context, deleteCollection *model.DeleteCollection) error UpdateCollection(ctx context.Context, updateCollection *model.UpdateCollection) (*model.Collection, error) CreateSegment(ctx context.Context, createSegment *model.CreateSegment) error - GetSegments(ctx context.Context, segmentID types.UniqueID, segmentType *string, scope *string, topic *string, collectionID types.UniqueID) ([]*model.Segment, error) + GetSegments(ctx context.Context, segmentID types.UniqueID, segmentType *string, scope *string, collectionID types.UniqueID) ([]*model.Segment, error) DeleteSegment(ctx context.Context, segmentID types.UniqueID) error UpdateSegment(ctx context.Context, updateSegment *model.UpdateSegment) (*model.Segment, error) CreateDatabase(ctx context.Context, createDatabase *model.CreateDatabase) (*model.Database, error) @@ -71,11 +72,6 @@ func (s *Coordinator) GetTenant(ctx context.Context, getTenant *model.GetTenant) func (s *Coordinator) CreateCollection(ctx context.Context, createCollection *model.CreateCollection) (*model.Collection, error) { log.Info("create collection", zap.Any("createCollection", createCollection)) - collectionTopic, err := s.assignCollection(createCollection.ID) - if err != nil { - return nil, err - } - createCollection.Topic = collectionTopic collection, err := s.catalog.CreateCollection(ctx, createCollection, createCollection.Ts) if err != nil { return nil, err @@ -83,8 +79,8 @@ func (s *Coordinator) CreateCollection(ctx context.Context, createCollection *mo return collection, nil } -func (s *Coordinator) GetCollections(ctx context.Context, collectionID types.UniqueID, collectionName *string, collectionTopic *string, tenantID string, databaseName string) ([]*model.Collection, error) { - return s.catalog.GetCollections(ctx, collectionID, collectionName, collectionTopic, tenantID, databaseName) +func (s *Coordinator) GetCollections(ctx context.Context, collectionID types.UniqueID, collectionName *string, tenantID string, databaseName string) ([]*model.Collection, error) { + return s.catalog.GetCollections(ctx, collectionID, collectionName, tenantID, databaseName) } func (s *Coordinator) DeleteCollection(ctx context.Context, deleteCollection *model.DeleteCollection) error { @@ -106,8 +102,8 @@ func (s *Coordinator) CreateSegment(ctx context.Context, segment *model.CreateSe return nil } -func (s *Coordinator) GetSegments(ctx context.Context, segmentID types.UniqueID, segmentType *string, scope *string, topic *string, collectionID types.UniqueID) ([]*model.Segment, error) { - return s.catalog.GetSegments(ctx, segmentID, segmentType, scope, topic, collectionID) +func (s *Coordinator) GetSegments(ctx context.Context, segmentID types.UniqueID, segmentType *string, scope *string, collectionID types.UniqueID) ([]*model.Segment, error) { + return s.catalog.GetSegments(ctx, segmentID, segmentType, scope, collectionID) } func (s *Coordinator) DeleteSegment(ctx context.Context, segmentID types.UniqueID) error { diff --git a/go/pkg/coordinator/apis_test.go b/go/pkg/coordinator/apis_test.go index 47a8b9b32186..3b05931f6e5d 100644 --- a/go/pkg/coordinator/apis_test.go +++ b/go/pkg/coordinator/apis_test.go @@ -2,13 +2,14 @@ package coordinator import ( "context" + "sort" + "strconv" + "testing" + "github.com/chroma-core/chroma/go/pkg/metastore/db/dao" "github.com/pingcap/log" "github.com/stretchr/testify/suite" "gorm.io/gorm" - "sort" - "strconv" - "testing" "github.com/chroma-core/chroma/go/pkg/common" "github.com/chroma-core/chroma/go/pkg/metastore/db/dbcore" @@ -48,9 +49,8 @@ func (suite *APIsTestSuite) SetupTest() { collection.ID = types.NewUniqueID() collection.Name = "collection_" + suite.T().Name() + strconv.Itoa(index) } - assignmentPolicy := NewMockAssignmentPolicy(suite.sampleCollections) ctx := context.Background() - c, err := NewCoordinator(ctx, assignmentPolicy, suite.db, nil, nil) + c, err := NewCoordinator(ctx, suite.db, nil, nil) if err != nil { suite.T().Fatalf("error creating coordinator: %v", err) } @@ -59,7 +59,6 @@ func (suite *APIsTestSuite) SetupTest() { _, errCollectionCreation := c.CreateCollection(ctx, &model.CreateCollection{ ID: collection.ID, Name: collection.Name, - Topic: collection.Topic, Metadata: collection.Metadata, Dimension: collection.Dimension, TenantID: collection.TenantID, @@ -82,8 +81,7 @@ func (suite *APIsTestSuite) TearDownTest() { func testCollection(t *rapid.T) { db := dbcore.ConfigDatabaseForTesting() ctx := context.Background() - assignmentPolicy := NewSimpleAssignmentPolicy("test-tenant", "test-topic") - c, err := NewCoordinator(ctx, assignmentPolicy, db, nil, nil) + c, err := NewCoordinator(ctx, db, nil, nil) if err != nil { t.Fatalf("error creating coordinator: %v", err) } @@ -110,15 +108,13 @@ func testCollection(t *rapid.T) { if err != nil { if err == common.ErrCollectionNameEmpty && collection.Name == "" { t.Logf("expected error for empty collection name") - } else if err == common.ErrCollectionTopicEmpty { - t.Logf("expected error for empty collection topic") } else { t.Fatalf("error creating collection: %v", err) } } if err == nil { // verify the correctness - collectionList, err := c.GetCollections(ctx, collection.ID, nil, nil, common.DefaultTenant, common.DefaultDatabase) + collectionList, err := c.GetCollections(ctx, collection.ID, nil, common.DefaultTenant, common.DefaultDatabase) if err != nil { t.Fatalf("error getting collections: %v", err) } @@ -138,8 +134,7 @@ func testCollection(t *rapid.T) { func testSegment(t *rapid.T) { db := dbcore.ConfigDatabaseForTesting() ctx := context.Background() - assignmentPolicy := NewSimpleAssignmentPolicy("test-tenant", "test-topic") - c, err := NewCoordinator(ctx, assignmentPolicy, db, nil, nil) + c, err := NewCoordinator(ctx, db, nil, nil) if err != nil { t.Fatalf("error creating coordinator: %v", err) } @@ -153,7 +148,6 @@ func testSegment(t *rapid.T) { metadata.Set("int_value", intValue) metadata.Set("float_value", floatValue) - testTopic := "test-segment-topic" t.Repeat(map[string]func(*rapid.T){ "create_segment": func(t *rapid.T) { segment := rapid.Custom[*model.CreateSegment](func(t *rapid.T) *model.CreateSegment { @@ -161,7 +155,6 @@ func testSegment(t *rapid.T) { ID: types.MustParse(rapid.StringMatching(`[0-9a-f]{8}-[0-9a-f]{4}-4[0-9a-f]{3}-[89ab][0-9a-f]{3}-[0-9a-f]{12}`).Draw(t, "segment_id")), Type: "test-segment-type", Scope: "test-segment-scope", - Topic: &testTopic, Metadata: nil, CollectionID: types.MustParse(rapid.StringMatching(`[0-9a-f]{8}-[0-9a-f]{4}-4[0-9a-f]{3}-[89ab][0-9a-f]{3}-[0-9a-f]{12}`).Draw(t, "collection_id")), } @@ -236,7 +229,6 @@ func SampleCollections(tenantID string, databaseName string) []*model.Collection { ID: types.MustParse("93ffe3ec-0107-48d4-8695-51f978c509dc"), Name: "test_collection_1", - Topic: "test_topic_1", Metadata: metadata1, Dimension: &dimension, TenantID: tenantID, @@ -245,7 +237,6 @@ func SampleCollections(tenantID string, databaseName string) []*model.Collection { ID: types.MustParse("f444f1d7-d06c-4357-ac22-5a4a1f92d761"), Name: "test_collection_2", - Topic: "test_topic_2", Metadata: metadata2, Dimension: nil, TenantID: tenantID, @@ -254,7 +245,6 @@ func SampleCollections(tenantID string, databaseName string) []*model.Collection { ID: types.MustParse("43babc1a-e403-4a50-91a9-16621ba29ab0"), Name: "test_collection_3", - Topic: "test_topic_3", Metadata: metadata3, Dimension: nil, TenantID: tenantID, @@ -264,28 +254,9 @@ func SampleCollections(tenantID string, databaseName string) []*model.Collection return sampleCollections } -type MockAssignmentPolicy struct { - collections []*model.Collection -} - -func NewMockAssignmentPolicy(collecions []*model.Collection) *MockAssignmentPolicy { - return &MockAssignmentPolicy{ - collections: collecions, - } -} - -func (m *MockAssignmentPolicy) AssignCollection(collectionID types.UniqueID) (string, error) { - for _, collection := range m.collections { - if collection.ID == collectionID { - return collection.Topic, nil - } - } - return "", common.ErrCollectionNotFound -} - func (suite *APIsTestSuite) TestCreateGetDeleteCollections() { ctx := context.Background() - results, err := suite.coordinator.GetCollections(ctx, types.NilUniqueID(), nil, nil, suite.tenantName, suite.databaseName) + results, err := suite.coordinator.GetCollections(ctx, types.NilUniqueID(), nil, suite.tenantName, suite.databaseName) suite.NoError(err) sort.Slice(results, func(i, j int) bool { @@ -304,40 +275,18 @@ func (suite *APIsTestSuite) TestCreateGetDeleteCollections() { // Find by name for _, collection := range suite.sampleCollections { - result, err := suite.coordinator.GetCollections(ctx, types.NilUniqueID(), &collection.Name, nil, suite.tenantName, suite.databaseName) - suite.NoError(err) - suite.Equal([]*model.Collection{collection}, result) - } - - // Find by topic - for _, collection := range suite.sampleCollections { - result, err := suite.coordinator.GetCollections(ctx, types.NilUniqueID(), nil, &collection.Topic, suite.tenantName, suite.databaseName) + result, err := suite.coordinator.GetCollections(ctx, types.NilUniqueID(), &collection.Name, suite.tenantName, suite.databaseName) suite.NoError(err) suite.Equal([]*model.Collection{collection}, result) } // Find by id for _, collection := range suite.sampleCollections { - result, err := suite.coordinator.GetCollections(ctx, collection.ID, nil, nil, suite.tenantName, suite.databaseName) + result, err := suite.coordinator.GetCollections(ctx, collection.ID, nil, suite.tenantName, suite.databaseName) suite.NoError(err) suite.Equal([]*model.Collection{collection}, result) } - // Find by id and topic (positive case) - for _, collection := range suite.sampleCollections { - result, err := suite.coordinator.GetCollections(ctx, collection.ID, nil, &collection.Topic, suite.tenantName, suite.databaseName) - suite.NoError(err) - suite.Equal([]*model.Collection{collection}, result) - } - - // find by id and topic (negative case) - for _, collection := range suite.sampleCollections { - otherTopic := "other topic" - result, err := suite.coordinator.GetCollections(ctx, collection.ID, nil, &otherTopic, suite.tenantName, suite.databaseName) - suite.NoError(err) - suite.Empty(result) - } - // Delete c1 := suite.sampleCollections[0] deleteCollection := &model.DeleteCollection{ @@ -348,13 +297,13 @@ func (suite *APIsTestSuite) TestCreateGetDeleteCollections() { err = suite.coordinator.DeleteCollection(ctx, deleteCollection) suite.NoError(err) - results, err = suite.coordinator.GetCollections(ctx, types.NilUniqueID(), nil, nil, suite.tenantName, suite.databaseName) + results, err = suite.coordinator.GetCollections(ctx, types.NilUniqueID(), nil, suite.tenantName, suite.databaseName) suite.NoError(err) suite.NotContains(results, c1) suite.Len(results, len(suite.sampleCollections)-1) suite.ElementsMatch(results, suite.sampleCollections[1:]) - byIDResult, err := suite.coordinator.GetCollections(ctx, c1.ID, nil, nil, suite.tenantName, suite.databaseName) + byIDResult, err := suite.coordinator.GetCollections(ctx, c1.ID, nil, suite.tenantName, suite.databaseName) suite.NoError(err) suite.Empty(byIDResult) @@ -368,7 +317,6 @@ func (suite *APIsTestSuite) TestUpdateCollections() { coll := &model.Collection{ Name: suite.sampleCollections[0].Name, ID: suite.sampleCollections[0].ID, - Topic: suite.sampleCollections[0].Topic, Metadata: suite.sampleCollections[0].Metadata, Dimension: suite.sampleCollections[0].Dimension, TenantID: suite.sampleCollections[0].TenantID, @@ -380,16 +328,7 @@ func (suite *APIsTestSuite) TestUpdateCollections() { result, err := suite.coordinator.UpdateCollection(ctx, &model.UpdateCollection{ID: coll.ID, Name: &coll.Name}) suite.NoError(err) suite.Equal(coll, result) - resultList, err := suite.coordinator.GetCollections(ctx, types.NilUniqueID(), &coll.Name, nil, suite.tenantName, suite.databaseName) - suite.NoError(err) - suite.Equal([]*model.Collection{coll}, resultList) - - // Update topic - coll.Topic = "new_topic" - result, err = suite.coordinator.UpdateCollection(ctx, &model.UpdateCollection{ID: coll.ID, Topic: &coll.Topic}) - suite.NoError(err) - suite.Equal(coll, result) - resultList, err = suite.coordinator.GetCollections(ctx, types.NilUniqueID(), nil, &coll.Topic, suite.tenantName, suite.databaseName) + resultList, err := suite.coordinator.GetCollections(ctx, types.NilUniqueID(), &coll.Name, suite.tenantName, suite.databaseName) suite.NoError(err) suite.Equal([]*model.Collection{coll}, resultList) @@ -399,7 +338,7 @@ func (suite *APIsTestSuite) TestUpdateCollections() { result, err = suite.coordinator.UpdateCollection(ctx, &model.UpdateCollection{ID: coll.ID, Dimension: coll.Dimension}) suite.NoError(err) suite.Equal(coll, result) - resultList, err = suite.coordinator.GetCollections(ctx, coll.ID, nil, nil, suite.tenantName, suite.databaseName) + resultList, err = suite.coordinator.GetCollections(ctx, coll.ID, nil, suite.tenantName, suite.databaseName) suite.NoError(err) suite.Equal([]*model.Collection{coll}, resultList) @@ -410,7 +349,7 @@ func (suite *APIsTestSuite) TestUpdateCollections() { result, err = suite.coordinator.UpdateCollection(ctx, &model.UpdateCollection{ID: coll.ID, Metadata: coll.Metadata}) suite.NoError(err) suite.Equal(coll, result) - resultList, err = suite.coordinator.GetCollections(ctx, coll.ID, nil, nil, suite.tenantName, suite.databaseName) + resultList, err = suite.coordinator.GetCollections(ctx, coll.ID, nil, suite.tenantName, suite.databaseName) suite.NoError(err) suite.Equal([]*model.Collection{coll}, resultList) @@ -419,7 +358,7 @@ func (suite *APIsTestSuite) TestUpdateCollections() { result, err = suite.coordinator.UpdateCollection(ctx, &model.UpdateCollection{ID: coll.ID, Metadata: coll.Metadata, ResetMetadata: true}) suite.NoError(err) suite.Equal(coll, result) - resultList, err = suite.coordinator.GetCollections(ctx, coll.ID, nil, nil, suite.tenantName, suite.databaseName) + resultList, err = suite.coordinator.GetCollections(ctx, coll.ID, nil, suite.tenantName, suite.databaseName) suite.NoError(err) suite.Equal([]*model.Collection{coll}, resultList) } @@ -440,7 +379,6 @@ func (suite *APIsTestSuite) TestCreateUpdateWithDatabase() { _, err = suite.coordinator.CreateCollection(ctx, &model.CreateCollection{ ID: suite.sampleCollections[0].ID, Name: suite.sampleCollections[0].Name, - Topic: suite.sampleCollections[0].Topic, Metadata: suite.sampleCollections[0].Metadata, Dimension: suite.sampleCollections[0].Dimension, TenantID: suite.sampleCollections[0].TenantID, @@ -453,7 +391,7 @@ func (suite *APIsTestSuite) TestCreateUpdateWithDatabase() { Name: &newName1, }) suite.NoError(err) - result, err := suite.coordinator.GetCollections(ctx, suite.sampleCollections[1].ID, nil, nil, suite.tenantName, suite.databaseName) + result, err := suite.coordinator.GetCollections(ctx, suite.sampleCollections[1].ID, nil, suite.tenantName, suite.databaseName) suite.NoError(err) suite.Len(result, 1) suite.Equal(newName1, result[0].Name) @@ -465,7 +403,7 @@ func (suite *APIsTestSuite) TestCreateUpdateWithDatabase() { }) suite.NoError(err) //suite.Equal(newName0, collection.Name) - result, err = suite.coordinator.GetCollections(ctx, suite.sampleCollections[0].ID, nil, nil, suite.tenantName, newDatabaseName) + result, err = suite.coordinator.GetCollections(ctx, suite.sampleCollections[0].ID, nil, suite.tenantName, newDatabaseName) suite.NoError(err) suite.Len(result, 1) suite.Equal(newName0, result[0].Name) @@ -495,7 +433,6 @@ func (suite *APIsTestSuite) TestGetMultipleWithDatabase() { _, err := suite.coordinator.CreateCollection(ctx, &model.CreateCollection{ ID: collection.ID, Name: collection.Name, - Topic: collection.Topic, Metadata: collection.Metadata, Dimension: collection.Dimension, TenantID: collection.TenantID, @@ -504,7 +441,7 @@ func (suite *APIsTestSuite) TestGetMultipleWithDatabase() { suite.NoError(err) suite.sampleCollections[index] = collection } - result, err := suite.coordinator.GetCollections(ctx, types.NilUniqueID(), nil, nil, suite.tenantName, newDatabaseName) + result, err := suite.coordinator.GetCollections(ctx, types.NilUniqueID(), nil, suite.tenantName, newDatabaseName) suite.NoError(err) suite.Equal(len(suite.sampleCollections), len(result)) sort.Slice(result, func(i, j int) bool { @@ -512,7 +449,7 @@ func (suite *APIsTestSuite) TestGetMultipleWithDatabase() { }) suite.Equal(suite.sampleCollections, result) - result, err = suite.coordinator.GetCollections(ctx, types.NilUniqueID(), nil, nil, suite.tenantName, suite.databaseName) + result, err = suite.coordinator.GetCollections(ctx, types.NilUniqueID(), nil, suite.tenantName, suite.databaseName) suite.NoError(err) suite.Equal(len(suite.sampleCollections), len(result)) @@ -565,7 +502,6 @@ func (suite *APIsTestSuite) TestCreateDatabaseWithTenants() { _, err = suite.coordinator.CreateCollection(ctx, &model.CreateCollection{ ID: suite.sampleCollections[0].ID, Name: suite.sampleCollections[0].Name, - Topic: suite.sampleCollections[0].Topic, Metadata: suite.sampleCollections[0].Metadata, Dimension: suite.sampleCollections[0].Dimension, TenantID: newTenantName, @@ -579,7 +515,6 @@ func (suite *APIsTestSuite) TestCreateDatabaseWithTenants() { _, err = suite.coordinator.CreateCollection(ctx, &model.CreateCollection{ ID: suite.sampleCollections[1].ID, Name: suite.sampleCollections[1].Name, - Topic: suite.sampleCollections[1].Topic, Metadata: suite.sampleCollections[1].Metadata, Dimension: suite.sampleCollections[1].Dimension, TenantID: suite.tenantName, @@ -591,7 +526,7 @@ func (suite *APIsTestSuite) TestCreateDatabaseWithTenants() { expected := []*model.Collection{suite.sampleCollections[0]} expected[0].TenantID = newTenantName expected[0].DatabaseName = newDatabaseName - result, err := suite.coordinator.GetCollections(ctx, types.NilUniqueID(), nil, nil, newTenantName, newDatabaseName) + result, err := suite.coordinator.GetCollections(ctx, types.NilUniqueID(), nil, newTenantName, newDatabaseName) suite.NoError(err) suite.Len(result, 1) suite.Equal(expected[0], result[0]) @@ -599,14 +534,14 @@ func (suite *APIsTestSuite) TestCreateDatabaseWithTenants() { expected = []*model.Collection{suite.sampleCollections[1]} expected[0].TenantID = suite.tenantName expected[0].DatabaseName = newDatabaseName - result, err = suite.coordinator.GetCollections(ctx, types.NilUniqueID(), nil, nil, suite.tenantName, newDatabaseName) + result, err = suite.coordinator.GetCollections(ctx, types.NilUniqueID(), nil, suite.tenantName, newDatabaseName) suite.NoError(err) suite.Len(result, 1) suite.Equal(expected[0], result[0]) // A new tenant DOES NOT have a default database. This does not error, instead 0 // results are returned - result, err = suite.coordinator.GetCollections(ctx, types.NilUniqueID(), nil, nil, newTenantName, suite.databaseName) + result, err = suite.coordinator.GetCollections(ctx, types.NilUniqueID(), nil, newTenantName, suite.databaseName) suite.NoError(err) suite.Nil(result) @@ -704,13 +639,10 @@ func SampleSegments(sampleCollections []*model.Collection) []*model.Segment { metadata3.Set("test_int", &model.SegmentMetadataValueInt64Type{Value: 3}) metadata3.Set("test_float", &model.SegmentMetadataValueFloat64Type{Value: 3.3}) - testTopic2 := "test_topic_2" - testTopic3 := "test_topic_3" sampleSegments := []*model.Segment{ { ID: types.MustParse("00000000-d7d7-413b-92e1-731098a6e492"), Type: "test_type_a", - Topic: nil, Scope: "VECTOR", CollectionID: sampleCollections[0].ID, Metadata: metadata1, @@ -719,7 +651,6 @@ func SampleSegments(sampleCollections []*model.Collection) []*model.Segment { { ID: types.MustParse("11111111-d7d7-413b-92e1-731098a6e492"), Type: "test_type_b", - Topic: &testTopic2, Scope: "VECTOR", CollectionID: sampleCollections[1].ID, Metadata: metadata2, @@ -728,7 +659,6 @@ func SampleSegments(sampleCollections []*model.Collection) []*model.Segment { { ID: types.MustParse("22222222-d7d7-413b-92e1-731098a6e492"), Type: "test_type_b", - Topic: &testTopic3, Scope: "METADATA", CollectionID: types.NilUniqueID(), Metadata: metadata3, // This segment is not assigned to any collection @@ -747,7 +677,6 @@ func (suite *APIsTestSuite) TestCreateGetDeleteSegments() { errSegmentCreation := c.CreateSegment(ctx, &model.CreateSegment{ ID: segment.ID, Type: segment.Type, - Topic: segment.Topic, Scope: segment.Scope, CollectionID: segment.CollectionID, Metadata: segment.Metadata, @@ -757,7 +686,7 @@ func (suite *APIsTestSuite) TestCreateGetDeleteSegments() { var results []*model.Segment for _, segment := range sampleSegments { - result, err := c.GetSegments(ctx, segment.ID, nil, nil, nil, types.NilUniqueID()) + result, err := c.GetSegments(ctx, segment.ID, nil, nil, types.NilUniqueID()) suite.NoError(err) suite.Equal([]*model.Segment{segment}, result) results = append(results, result...) @@ -771,7 +700,6 @@ func (suite *APIsTestSuite) TestCreateGetDeleteSegments() { err := c.CreateSegment(ctx, &model.CreateSegment{ ID: sampleSegments[0].ID, Type: sampleSegments[0].Type, - Topic: sampleSegments[0].Topic, Scope: sampleSegments[0].Scope, CollectionID: sampleSegments[0].CollectionID, Metadata: sampleSegments[0].Metadata, @@ -780,34 +708,34 @@ func (suite *APIsTestSuite) TestCreateGetDeleteSegments() { // Find by id for _, segment := range sampleSegments { - result, err := c.GetSegments(ctx, segment.ID, nil, nil, nil, types.NilUniqueID()) + result, err := c.GetSegments(ctx, segment.ID, nil, nil, types.NilUniqueID()) suite.NoError(err) suite.Equal([]*model.Segment{segment}, result) } // Find by type testTypeA := "test_type_a" - result, err := c.GetSegments(ctx, types.NilUniqueID(), &testTypeA, nil, nil, types.NilUniqueID()) + result, err := c.GetSegments(ctx, types.NilUniqueID(), &testTypeA, nil, types.NilUniqueID()) suite.NoError(err) suite.Equal(sampleSegments[:1], result) testTypeB := "test_type_b" - result, err = c.GetSegments(ctx, types.NilUniqueID(), &testTypeB, nil, nil, types.NilUniqueID()) + result, err = c.GetSegments(ctx, types.NilUniqueID(), &testTypeB, nil, types.NilUniqueID()) suite.NoError(err) suite.ElementsMatch(sampleSegments[1:], result) // Find by collection ID - result, err = c.GetSegments(ctx, types.NilUniqueID(), nil, nil, nil, suite.sampleCollections[0].ID) + result, err = c.GetSegments(ctx, types.NilUniqueID(), nil, nil, suite.sampleCollections[0].ID) suite.NoError(err) suite.Equal(sampleSegments[:1], result) // Find by type and collection ID (positive case) - result, err = c.GetSegments(ctx, types.NilUniqueID(), &testTypeA, nil, nil, suite.sampleCollections[0].ID) + result, err = c.GetSegments(ctx, types.NilUniqueID(), &testTypeA, nil, suite.sampleCollections[0].ID) suite.NoError(err) suite.Equal(sampleSegments[:1], result) // Find by type and collection ID (negative case) - result, err = c.GetSegments(ctx, types.NilUniqueID(), &testTypeB, nil, nil, suite.sampleCollections[0].ID) + result, err = c.GetSegments(ctx, types.NilUniqueID(), &testTypeB, nil, suite.sampleCollections[0].ID) suite.NoError(err) suite.Empty(result) @@ -816,7 +744,7 @@ func (suite *APIsTestSuite) TestCreateGetDeleteSegments() { err = c.DeleteSegment(ctx, s1.ID) suite.NoError(err) - results, err = c.GetSegments(ctx, types.NilUniqueID(), nil, nil, nil, types.NilUniqueID()) + results, err = c.GetSegments(ctx, types.NilUniqueID(), nil, nil, types.NilUniqueID()) suite.NoError(err) suite.NotContains(results, s1) suite.Len(results, len(sampleSegments)-1) @@ -833,8 +761,6 @@ func (suite *APIsTestSuite) TestCreateGetDeleteSegments() { } func (suite *APIsTestSuite) TestUpdateSegment() { - testTopic := "test_topic_a" - metadata := model.NewSegmentMetadata[model.SegmentMetadataValueType]() metadata.Set("test_str", &model.SegmentMetadataValueStringType{Value: "str1"}) metadata.Set("test_int", &model.SegmentMetadataValueInt64Type{Value: 1}) @@ -844,7 +770,6 @@ func (suite *APIsTestSuite) TestUpdateSegment() { ID: types.UniqueID(uuid.New()), Type: "test_type_a", Scope: "VECTOR", - Topic: &testTopic, CollectionID: suite.sampleCollections[0].ID, Metadata: metadata, FilePaths: map[string][]string{}, @@ -854,39 +779,13 @@ func (suite *APIsTestSuite) TestUpdateSegment() { errSegmentCreation := suite.coordinator.CreateSegment(ctx, &model.CreateSegment{ ID: segment.ID, Type: segment.Type, - Topic: segment.Topic, Scope: segment.Scope, CollectionID: segment.CollectionID, Metadata: segment.Metadata, }) suite.NoError(errSegmentCreation) - // Update topic to new value collectionID := segment.CollectionID.String() - newTopic := "new_topic" - segment.Topic = &newTopic - _, err := suite.coordinator.UpdateSegment(ctx, &model.UpdateSegment{ - Collection: &collectionID, - ID: segment.ID, - Topic: segment.Topic, - }) - suite.NoError(err) - result, err := suite.coordinator.GetSegments(ctx, segment.ID, nil, nil, nil, types.NilUniqueID()) - suite.NoError(err) - suite.Equal([]*model.Segment{segment}, result) - - // Update topic to None - segment.Topic = nil - _, err = suite.coordinator.UpdateSegment(ctx, &model.UpdateSegment{ - Collection: &collectionID, - ID: segment.ID, - Topic: segment.Topic, - ResetTopic: true, - }) - suite.NoError(err) - result, err = suite.coordinator.GetSegments(ctx, segment.ID, nil, nil, nil, types.NilUniqueID()) - suite.NoError(err) - suite.Equal([]*model.Segment{segment}, result) // TODO: revisit why we need this // Update collection to new value @@ -913,12 +812,12 @@ func (suite *APIsTestSuite) TestUpdateSegment() { // Add a new metadata key segment.Metadata.Set("test_str2", &model.SegmentMetadataValueStringType{Value: "str2"}) - _, err = suite.coordinator.UpdateSegment(ctx, &model.UpdateSegment{ + _, err := suite.coordinator.UpdateSegment(ctx, &model.UpdateSegment{ Collection: &collectionID, ID: segment.ID, Metadata: segment.Metadata}) suite.NoError(err) - result, err = suite.coordinator.GetSegments(ctx, segment.ID, nil, nil, nil, types.NilUniqueID()) + result, err := suite.coordinator.GetSegments(ctx, segment.ID, nil, nil, types.NilUniqueID()) suite.NoError(err) suite.Equal([]*model.Segment{segment}, result) @@ -929,7 +828,7 @@ func (suite *APIsTestSuite) TestUpdateSegment() { ID: segment.ID, Metadata: segment.Metadata}) suite.NoError(err) - result, err = suite.coordinator.GetSegments(ctx, segment.ID, nil, nil, nil, types.NilUniqueID()) + result, err = suite.coordinator.GetSegments(ctx, segment.ID, nil, nil, types.NilUniqueID()) suite.NoError(err) suite.Equal([]*model.Segment{segment}, result) @@ -942,7 +841,7 @@ func (suite *APIsTestSuite) TestUpdateSegment() { ID: segment.ID, Metadata: newMetadata}) suite.NoError(err) - result, err = suite.coordinator.GetSegments(ctx, segment.ID, nil, nil, nil, types.NilUniqueID()) + result, err = suite.coordinator.GetSegments(ctx, segment.ID, nil, nil, types.NilUniqueID()) suite.NoError(err) suite.Equal([]*model.Segment{segment}, result) @@ -955,7 +854,7 @@ func (suite *APIsTestSuite) TestUpdateSegment() { ResetMetadata: true}, ) suite.NoError(err) - result, err = suite.coordinator.GetSegments(ctx, segment.ID, nil, nil, nil, types.NilUniqueID()) + result, err = suite.coordinator.GetSegments(ctx, segment.ID, nil, nil, types.NilUniqueID()) suite.NoError(err) suite.Equal([]*model.Segment{segment}, result) } diff --git a/go/pkg/coordinator/assignment_policy.go b/go/pkg/coordinator/assignment_policy.go deleted file mode 100644 index dbaa59be8489..000000000000 --- a/go/pkg/coordinator/assignment_policy.go +++ /dev/null @@ -1,77 +0,0 @@ -package coordinator - -import ( - "fmt" - - "github.com/chroma-core/chroma/go/pkg/types" - "github.com/chroma-core/chroma/go/pkg/utils" -) - -type CollectionAssignmentPolicy interface { - AssignCollection(collectionID types.UniqueID) (string, error) -} - -// SimpleAssignmentPolicy is a simple assignment policy that assigns a 1 collection to 1 -// topic based on the id of the collection. -type SimpleAssignmentPolicy struct { - tenantID string - topicNS string -} - -func NewSimpleAssignmentPolicy(tenantID string, topicNS string) *SimpleAssignmentPolicy { - return &SimpleAssignmentPolicy{ - tenantID: tenantID, - topicNS: topicNS, - } -} - -func (s *SimpleAssignmentPolicy) AssignCollection(collectionID types.UniqueID) (string, error) { - return createTopicName(s.tenantID, s.topicNS, collectionID.String()), nil -} - -func createTopicName(tenantID string, topicNS string, log_name string) string { - return fmt.Sprintf("persistent://%s/%s/%s", tenantID, topicNS, log_name) -} - -// RendezvousAssignmentPolicy is an assignment policy that assigns a collection to a topic -// For now it assumes there are 16 topics and uses the rendezvous hashing algorithm to -// assign a collection to a topic. - -var Topics = [16]string{ - "chroma_log_0", - "chroma_log_1", - "chroma_log_2", - "chroma_log_3", - "chroma_log_4", - "chroma_log_5", - "chroma_log_6", - "chroma_log_7", - "chroma_log_8", - "chroma_log_9", - "chroma_log_10", - "chroma_log_11", - "chroma_log_12", - "chroma_log_13", - "chroma_log_14", - "chroma_log_15", -} - -type RendezvousAssignmentPolicy struct { - tenantID string - topicNS string -} - -func NewRendezvousAssignmentPolicy(tenantID string, topicNS string) *RendezvousAssignmentPolicy { - return &RendezvousAssignmentPolicy{ - tenantID: tenantID, - topicNS: topicNS, - } -} - -func (r *RendezvousAssignmentPolicy) AssignCollection(collectionID types.UniqueID) (string, error) { - assignment, error := utils.Assign(collectionID.String(), Topics[:], utils.Murmur3Hasher) - if error != nil { - return "", error - } - return createTopicName(r.tenantID, r.topicNS, assignment), nil -} diff --git a/go/pkg/coordinator/coordinator.go b/go/pkg/coordinator/coordinator.go index d52aeaf8954e..abef2c29ca73 100644 --- a/go/pkg/coordinator/coordinator.go +++ b/go/pkg/coordinator/coordinator.go @@ -9,7 +9,6 @@ import ( "github.com/chroma-core/chroma/go/pkg/metastore/db/dao" "github.com/chroma-core/chroma/go/pkg/metastore/db/dbcore" "github.com/chroma-core/chroma/go/pkg/notification" - "github.com/chroma-core/chroma/go/pkg/types" "gorm.io/gorm" ) @@ -19,16 +18,14 @@ var _ ICoordinator = (*Coordinator)(nil) // Currently, it only has the system catalog related APIs and will be extended to // support other functionalities such as membership managed and propagation. type Coordinator struct { - ctx context.Context - collectionAssignmentPolicy CollectionAssignmentPolicy - notificationProcessor notification.NotificationProcessor - catalog metastore.Catalog + ctx context.Context + notificationProcessor notification.NotificationProcessor + catalog metastore.Catalog } -func NewCoordinator(ctx context.Context, assignmentPolicy CollectionAssignmentPolicy, db *gorm.DB, notificationStore notification.NotificationStore, notifier notification.Notifier) (*Coordinator, error) { +func NewCoordinator(ctx context.Context, db *gorm.DB, notificationStore notification.NotificationStore, notifier notification.Notifier) (*Coordinator, error) { s := &Coordinator{ - ctx: ctx, - collectionAssignmentPolicy: assignmentPolicy, + ctx: ctx, } notificationProcessor := notification.NewSimpleNotificationProcessor(ctx, notificationStore, notifier) @@ -57,7 +54,3 @@ func (s *Coordinator) Stop() error { } return nil } - -func (c *Coordinator) assignCollection(collectionID types.UniqueID) (string, error) { - return c.collectionAssignmentPolicy.AssignCollection(collectionID) -} diff --git a/go/pkg/coordinator/grpc/collection_service.go b/go/pkg/coordinator/grpc/collection_service.go index a6b9816ec620..ad724a1ca410 100644 --- a/go/pkg/coordinator/grpc/collection_service.go +++ b/go/pkg/coordinator/grpc/collection_service.go @@ -4,6 +4,7 @@ import ( "context" "encoding/json" "errors" + "github.com/chroma-core/chroma/go/pkg/grpcutils" "github.com/chroma-core/chroma/go/pkg/common" @@ -97,7 +98,6 @@ func (s *Server) CreateCollection(ctx context.Context, req *coordinatorpb.Create func (s *Server) GetCollections(ctx context.Context, req *coordinatorpb.GetCollectionsRequest) (*coordinatorpb.GetCollectionsResponse, error) { collectionID := req.Id collectionName := req.Name - collectionTopic := req.Topic tenantID := req.Tenant databaseName := req.Database @@ -110,7 +110,7 @@ func (s *Server) GetCollections(ctx context.Context, req *coordinatorpb.GetColle return res, nil } - collections, err := s.coordinator.GetCollections(ctx, parsedCollectionID, collectionName, collectionTopic, tenantID, databaseName) + collections, err := s.coordinator.GetCollections(ctx, parsedCollectionID, collectionName, tenantID, databaseName) if err != nil { log.Error("error getting collections", zap.Error(err)) res.Status = failResponseWithError(err, errorCode) @@ -169,7 +169,6 @@ func (s *Server) UpdateCollection(ctx context.Context, req *coordinatorpb.Update updateCollection := &model.UpdateCollection{ ID: parsedCollectionID, Name: req.Name, - Topic: req.Topic, Dimension: req.Dimension, } diff --git a/go/pkg/coordinator/grpc/collection_service_test.go b/go/pkg/coordinator/grpc/collection_service_test.go index 9e86c8ff4f13..dd4ddd757470 100644 --- a/go/pkg/coordinator/grpc/collection_service_test.go +++ b/go/pkg/coordinator/grpc/collection_service_test.go @@ -2,6 +2,11 @@ package grpc import ( "context" + "reflect" + "strconv" + "testing" + "time" + "github.com/chroma-core/chroma/go/pkg/common" "github.com/chroma-core/chroma/go/pkg/grpcutils" "github.com/chroma-core/chroma/go/pkg/metastore/coordinator" @@ -16,10 +21,6 @@ import ( "gorm.io/gorm" "k8s.io/apimachinery/pkg/util/rand" "pgregory.net/rapid" - "reflect" - "strconv" - "testing" - "time" ) type CollectionServiceTestSuite struct { @@ -36,7 +37,6 @@ func (suite *CollectionServiceTestSuite) SetupSuite() { log.Info("setup suite") suite.db = dbcore.ConfigDatabaseForTesting() s, err := NewWithGrpcProvider(Config{ - AssignmentPolicy: "simple", SystemCatalogProvider: "database", NotificationStoreProvider: "memory", NotifierProvider: "memory", @@ -68,12 +68,10 @@ func (suite *CollectionServiceTestSuite) TearDownSuite() { // Collection created should have the right metadata, the metadata should be a flat map, with keys as strings and values as strings, ints, or floats // Collection created should have the right name // Collection created should have the right ID -// Collection created should have the right topic // Collection created should have the right timestamp func testCollection(t *rapid.T) { db := dbcore.ConfigDatabaseForTesting() s, err := NewWithGrpcProvider(Config{ - AssignmentPolicy: "simple", SystemCatalogProvider: "memory", NotificationStoreProvider: "memory", NotifierProvider: "memory", @@ -112,10 +110,6 @@ func testCollection(t *rapid.T) { if err == common.ErrCollectionNameEmpty && createCollectionRequest.Name == "" { t.Logf("expected error for empty collection name") collectionsWithErrors = append(collectionsWithErrors, res.Collection) - } else if err == common.ErrCollectionTopicEmpty { - t.Logf("expected error for empty collection topic") - collectionsWithErrors = append(collectionsWithErrors, res.Collection) - // TODO: check the topic name not empty } else { t.Fatalf("error creating collection: %v", err) collectionsWithErrors = append(collectionsWithErrors, res.Collection) @@ -185,7 +179,6 @@ func validateDatabase(suite *CollectionServiceTestSuite, collectionId string, co suite.Len(collectionsInDB.Collections, 1) suite.Equal(collection.Id, collection.Id) suite.Equal(collection.Name, collection.Name) - suite.Equal(collection.Topic, collection.Topic) suite.Equal(collection.LogPosition, collection.LogPosition) suite.Equal(collection.Version, collection.Version) @@ -203,8 +196,7 @@ func (suite *CollectionServiceTestSuite) TestServer_FlushCollectionCompaction() log.Info("TestServer_FlushCollectionCompaction") // create test collection collectionName := "collection_service_test_flush_collection_compaction" - collectionTopic := "collection_service_test_flush_collection_compaction_topic" - collectionID, err := dao.CreateTestCollection(suite.db, collectionName, collectionTopic, 128, suite.databaseId) + collectionID, err := dao.CreateTestCollection(suite.db, collectionName, 128, suite.databaseId) suite.NoError(err) // flush collection compaction diff --git a/go/pkg/coordinator/grpc/proto_model_convert.go b/go/pkg/coordinator/grpc/proto_model_convert.go index 61359b2fdc0e..cc7fbb12fcb6 100644 --- a/go/pkg/coordinator/grpc/proto_model_convert.go +++ b/go/pkg/coordinator/grpc/proto_model_convert.go @@ -40,7 +40,6 @@ func convertCollectionToProto(collection *model.Collection) *coordinatorpb.Colle collectionpb := &coordinatorpb.Collection{ Id: collection.ID.String(), Name: collection.Name, - Topic: collection.Topic, Dimension: collection.Dimension, Tenant: collection.TenantID, Database: collection.DatabaseName, @@ -157,7 +156,6 @@ func convertSegmentToProto(segment *model.Segment) *coordinatorpb.Segment { Id: segment.ID.String(), Type: segment.Type, Scope: segmentSceope, - Topic: segment.Topic, Collection: nil, Metadata: nil, FilePaths: filePaths, @@ -229,7 +227,6 @@ func convertSegmentToModel(segmentpb *coordinatorpb.Segment) (*model.CreateSegme ID: segmentID, Type: segmentpb.Type, Scope: segmentpb.Scope.String(), - Topic: segmentpb.Topic, CollectionID: collectionID, Metadata: metadata, }, nil diff --git a/go/pkg/coordinator/grpc/proto_model_convert_test.go b/go/pkg/coordinator/grpc/proto_model_convert_test.go index 6033fff5a372..659addbc8d56 100644 --- a/go/pkg/coordinator/grpc/proto_model_convert_test.go +++ b/go/pkg/coordinator/grpc/proto_model_convert_test.go @@ -53,7 +53,6 @@ func TestConvertCollectionToProto(t *testing.T) { collection := &model.Collection{ ID: types.NewUniqueID(), Name: "test_collection", - Topic: "test_topic", Dimension: &dimention, Metadata: &model.CollectionMetadata[model.CollectionMetadataValueType]{ Metadata: map[string]model.CollectionMetadataValueType{ @@ -67,7 +66,6 @@ func TestConvertCollectionToProto(t *testing.T) { assert.NotNil(t, collectionpb) assert.Equal(t, collection.ID.String(), collectionpb.Id) assert.Equal(t, collection.Name, collectionpb.Name) - assert.Equal(t, collection.Topic, collectionpb.Topic) assert.Equal(t, collection.Dimension, collectionpb.Dimension) assert.NotNil(t, collectionpb.Metadata) assert.Equal(t, "value1", collectionpb.Metadata.Metadata["key1"].GetStringValue()) @@ -182,12 +180,10 @@ func TestConvertSegmentToProto(t *testing.T) { assert.Nil(t, segmentpb) // Test case 2: segment is not nil - testTopic := "test_topic" segment := &model.Segment{ ID: types.NewUniqueID(), Type: "test_type", Scope: "METADATA", - Topic: &testTopic, Metadata: nil, FilePaths: map[string][]string{}, } @@ -196,7 +192,6 @@ func TestConvertSegmentToProto(t *testing.T) { assert.Equal(t, segment.ID.String(), segmentpb.Id) assert.Equal(t, segment.Type, segmentpb.Type) assert.Equal(t, coordinatorpb.SegmentScope_METADATA, segmentpb.Scope) - assert.Equal(t, segment.Topic, segmentpb.Topic) assert.Nil(t, segmentpb.Collection) assert.Nil(t, segmentpb.Metadata) } diff --git a/go/pkg/coordinator/grpc/segment_service.go b/go/pkg/coordinator/grpc/segment_service.go index df4e61397cc3..ae3da6595ebc 100644 --- a/go/pkg/coordinator/grpc/segment_service.go +++ b/go/pkg/coordinator/grpc/segment_service.go @@ -43,7 +43,6 @@ func (s *Server) GetSegments(ctx context.Context, req *coordinatorpb.GetSegments segmentID := req.Id segmentType := req.Type scope := req.Scope - topic := req.Topic collectionID := req.Collection res := &coordinatorpb.GetSegmentsResponse{} @@ -67,7 +66,7 @@ func (s *Server) GetSegments(ctx context.Context, req *coordinatorpb.GetSegments scopeString := scope.String() scopeValue = &scopeString } - segments, err := s.coordinator.GetSegments(ctx, parsedSegmentID, segmentType, scopeValue, topic, parsedCollectionID) + segments, err := s.coordinator.GetSegments(ctx, parsedSegmentID, segmentType, scopeValue, parsedCollectionID) if err != nil { log.Error("get segments error", zap.Error(err)) res.Status = failResponseWithError(err, errorCode) @@ -112,16 +111,10 @@ func (s *Server) UpdateSegment(ctx context.Context, req *coordinatorpb.UpdateSeg res := &coordinatorpb.UpdateSegmentResponse{} updateSegment := &model.UpdateSegment{ ID: types.MustParse(req.Id), - ResetTopic: req.GetResetTopic(), ResetCollection: req.GetResetCollection(), ResetMetadata: req.GetResetMetadata(), } - topic := req.GetTopic() - if topic == "" { - updateSegment.Topic = nil - } else { - updateSegment.Topic = &topic - } + collection := req.GetCollection() if collection == "" { updateSegment.Collection = nil diff --git a/go/pkg/coordinator/grpc/server.go b/go/pkg/coordinator/grpc/server.go index b531cbc3a447..24738aeb2078 100644 --- a/go/pkg/coordinator/grpc/server.go +++ b/go/pkg/coordinator/grpc/server.go @@ -48,9 +48,6 @@ type Config struct { WorkerMemberlistName string WorkerPodLabel string - // Assignment policy config can be "simple" or "rendezvous" - AssignmentPolicy string - // Watcher config WatchInterval time.Duration @@ -90,22 +87,6 @@ func NewWithGrpcProvider(config Config, provider grpcutils.GrpcProvider, db *gor healthServer: health.NewServer(), } - var assignmentPolicy coordinator.CollectionAssignmentPolicy - if config.AssignmentPolicy == "simple" { - log.Info("Using simple assignment policy") - assignmentPolicy = coordinator.NewSimpleAssignmentPolicy(config.PulsarTenant, config.PulsarNamespace) - } else if config.AssignmentPolicy == "rendezvous" { - log.Info("Using rendezvous assignment policy") - //err := utils.CreateTopics(config.PulsarAdminURL, config.PulsarTenant, config.PulsarNamespace, coordinator.Topics[:]) - //if err != nil { - // log.Error("Failed to create topics", zap.Error(err)) - // return nil, err - //} - assignmentPolicy = coordinator.NewRendezvousAssignmentPolicy(config.PulsarTenant, config.PulsarNamespace) - } else { - return nil, errors.New("invalid assignment policy, only simple and rendezvous are supported") - } - var notificationStore notification.NotificationStore if config.NotificationStoreProvider == "memory" { log.Info("Using memory notification store") @@ -145,7 +126,7 @@ func NewWithGrpcProvider(config Config, provider grpcutils.GrpcProvider, db *gor defer producer.Close() } - coordinator, err := coordinator.NewCoordinator(ctx, assignmentPolicy, db, notificationStore, notifier) + coordinator, err := coordinator.NewCoordinator(ctx, db, notificationStore, notifier) if err != nil { return nil, err } diff --git a/go/pkg/coordinator/grpc/tenant_database_service_test.go b/go/pkg/coordinator/grpc/tenant_database_service_test.go index 4f37b060734c..825765e26104 100644 --- a/go/pkg/coordinator/grpc/tenant_database_service_test.go +++ b/go/pkg/coordinator/grpc/tenant_database_service_test.go @@ -2,6 +2,9 @@ package grpc import ( "context" + "testing" + "time" + "github.com/chroma-core/chroma/go/pkg/common" "github.com/chroma-core/chroma/go/pkg/grpcutils" "github.com/chroma-core/chroma/go/pkg/metastore/coordinator" @@ -15,8 +18,6 @@ import ( codes "google.golang.org/grpc/codes" "google.golang.org/grpc/status" "gorm.io/gorm" - "testing" - "time" ) type TenantDatabaseServiceTestSuite struct { @@ -30,7 +31,6 @@ func (suite *TenantDatabaseServiceTestSuite) SetupSuite() { log.Info("setup suite") suite.db = dbcore.ConfigDatabaseForTesting() s, err := NewWithGrpcProvider(Config{ - AssignmentPolicy: "simple", SystemCatalogProvider: "database", NotificationStoreProvider: "memory", NotifierProvider: "memory", diff --git a/go/pkg/logservice/grpc/record_log_service.go b/go/pkg/logservice/grpc/record_log_service.go index 40d38a3e57bd..fe3db4b87d4e 100644 --- a/go/pkg/logservice/grpc/record_log_service.go +++ b/go/pkg/logservice/grpc/record_log_service.go @@ -28,9 +28,6 @@ func (s *Server) PushLogs(ctx context.Context, req *logservicepb.PushLogsRequest } var recordsContent [][]byte for _, record := range req.Records { - // We remove the collection id for space reasons, as its double stored in the wrapping database RecordLog object. - // PullLogs will rehydrate the collection id from the database. - record.CollectionId = "" data, err := proto.Marshal(record) if err != nil { log.Error("marshaling error", zap.Error(err)) @@ -66,7 +63,7 @@ func (s *Server) PullLogs(ctx context.Context, req *logservicepb.PullLogsRequest return nil, grpcutils.BuildInternalGrpcError(err.Error()) } for index := range recordLogs { - record := &coordinatorpb.SubmitEmbeddingRecord{} + record := &coordinatorpb.OperationRecord{} if err := proto.Unmarshal(*recordLogs[index].Record, record); err != nil { log.Error("Unmarshal error", zap.Error(err)) grpcError, err := grpcutils.BuildInvalidArgumentGrpcError("records", "marshaling error") @@ -75,8 +72,6 @@ func (s *Server) PullLogs(ctx context.Context, req *logservicepb.PullLogsRequest } return nil, grpcError } - // Here we rehydrate the collection id from the database since in PushLogs we removed it for space reasons. - record.CollectionId = *recordLogs[index].CollectionID recordLog := &logservicepb.RecordLog{ LogId: recordLogs[index].ID, Record: record, diff --git a/go/pkg/logservice/grpc/record_log_service_test.go b/go/pkg/logservice/grpc/record_log_service_test.go index 3e453e351ed8..1aad091ee00a 100644 --- a/go/pkg/logservice/grpc/record_log_service_test.go +++ b/go/pkg/logservice/grpc/record_log_service_test.go @@ -72,28 +72,25 @@ func encodeVector(dimension int32, vector []float32, encoding coordinatorpb.Scal } } -func GetTestEmbeddingRecords(collectionId string) (recordsToSubmit []*coordinatorpb.SubmitEmbeddingRecord) { +func GetTestEmbeddingRecords(collectionId string) (recordsToSubmit []*coordinatorpb.OperationRecord) { testVector1 := []float32{1.0, 2.0, 3.0} testVector2 := []float32{1.2, 2.24, 3.2} testVector3 := []float32{7.0, 8.0, 9.0} - recordsToSubmit = make([]*coordinatorpb.SubmitEmbeddingRecord, 0) - recordsToSubmit = append(recordsToSubmit, &coordinatorpb.SubmitEmbeddingRecord{ - Id: types.NewUniqueID().String(), - Vector: encodeVector(10, testVector1, coordinatorpb.ScalarEncoding_FLOAT32), - Operation: coordinatorpb.Operation_ADD, - CollectionId: collectionId, + recordsToSubmit = make([]*coordinatorpb.OperationRecord, 0) + recordsToSubmit = append(recordsToSubmit, &coordinatorpb.OperationRecord{ + Id: types.NewUniqueID().String(), + Vector: encodeVector(10, testVector1, coordinatorpb.ScalarEncoding_FLOAT32), + Operation: coordinatorpb.Operation_ADD, }) - recordsToSubmit = append(recordsToSubmit, &coordinatorpb.SubmitEmbeddingRecord{ - Id: types.NewUniqueID().String(), - Vector: encodeVector(6, testVector2, coordinatorpb.ScalarEncoding_FLOAT32), - Operation: coordinatorpb.Operation_UPDATE, - CollectionId: collectionId, + recordsToSubmit = append(recordsToSubmit, &coordinatorpb.OperationRecord{ + Id: types.NewUniqueID().String(), + Vector: encodeVector(6, testVector2, coordinatorpb.ScalarEncoding_FLOAT32), + Operation: coordinatorpb.Operation_UPDATE, }) - recordsToSubmit = append(recordsToSubmit, &coordinatorpb.SubmitEmbeddingRecord{ - Id: types.NewUniqueID().String(), - Vector: encodeVector(10, testVector3, coordinatorpb.ScalarEncoding_FLOAT32), - Operation: coordinatorpb.Operation_ADD, - CollectionId: collectionId, + recordsToSubmit = append(recordsToSubmit, &coordinatorpb.OperationRecord{ + Id: types.NewUniqueID().String(), + Vector: encodeVector(10, testVector3, coordinatorpb.ScalarEncoding_FLOAT32), + Operation: coordinatorpb.Operation_ADD, }) return recordsToSubmit } @@ -116,13 +113,12 @@ func (suite *RecordLogServiceTestSuite) TestServer_PushLogs() { for index := range recordLogs { suite.Equal(int64(index+1), recordLogs[index].ID) suite.Equal(suite.collectionId.String(), *recordLogs[index].CollectionID) - record := &coordinatorpb.SubmitEmbeddingRecord{} + record := &coordinatorpb.OperationRecord{} if unmarshalErr := proto.Unmarshal(*recordLogs[index].Record, record); err != nil { suite.NoError(unmarshalErr) } suite.Equal(recordsToSubmit[index].Id, record.Id) suite.Equal(recordsToSubmit[index].Operation, record.Operation) - suite.Equal("", record.CollectionId) suite.Equal(recordsToSubmit[index].Metadata, record.Metadata) suite.Equal(recordsToSubmit[index].Vector.Dimension, record.Vector.Dimension) suite.Equal(recordsToSubmit[index].Vector.Encoding, record.Vector.Encoding) @@ -134,9 +130,9 @@ func (suite *RecordLogServiceTestSuite) TestServer_PullLogs() { // push some records recordsToSubmit := GetTestEmbeddingRecords(suite.collectionId.String()) // deep clone the records since PushLogs will mutate the records and we need a source of truth - recordsToSubmit_sot := make([]*coordinatorpb.SubmitEmbeddingRecord, len(recordsToSubmit)) + recordsToSubmit_sot := make([]*coordinatorpb.OperationRecord, len(recordsToSubmit)) for i := range recordsToSubmit { - recordsToSubmit_sot[i] = proto.Clone(recordsToSubmit[i]).(*coordinatorpb.SubmitEmbeddingRecord) + recordsToSubmit_sot[i] = proto.Clone(recordsToSubmit[i]).(*coordinatorpb.OperationRecord) } pushRequest := logservicepb.PushLogsRequest{ CollectionId: suite.collectionId.String(), @@ -158,7 +154,6 @@ func (suite *RecordLogServiceTestSuite) TestServer_PullLogs() { suite.Equal(int64(index+1), pullResponse.Records[index].LogId) suite.Equal(recordsToSubmit_sot[index].Id, pullResponse.Records[index].Record.Id) suite.Equal(recordsToSubmit_sot[index].Operation, pullResponse.Records[index].Record.Operation) - suite.Equal(recordsToSubmit_sot[index].CollectionId, pullResponse.Records[index].Record.CollectionId) suite.Equal(recordsToSubmit_sot[index].Metadata, pullResponse.Records[index].Record.Metadata) suite.Equal(recordsToSubmit_sot[index].Vector.Dimension, pullResponse.Records[index].Record.Vector.Dimension) suite.Equal(recordsToSubmit_sot[index].Vector.Encoding, pullResponse.Records[index].Record.Vector.Encoding) @@ -171,7 +166,7 @@ func (suite *RecordLogServiceTestSuite) TestServer_Bad_CollectionId() { // push some records pushRequest := logservicepb.PushLogsRequest{ CollectionId: "badId", - Records: []*coordinatorpb.SubmitEmbeddingRecord{}, + Records: []*coordinatorpb.OperationRecord{}, } _, err := suite.s.PushLogs(context.Background(), &pushRequest) suite.Error(err) diff --git a/go/pkg/logservice/testutils/record_log_test_util.go b/go/pkg/logservice/testutils/record_log_test_util.go index a6f7c3d9aa0b..9f5435098898 100644 --- a/go/pkg/logservice/testutils/record_log_test_util.go +++ b/go/pkg/logservice/testutils/record_log_test_util.go @@ -1,22 +1,21 @@ package testutils import ( + "strconv" + "github.com/chroma-core/chroma/go/pkg/metastore/db/dbmodel" "github.com/chroma-core/chroma/go/pkg/types" "gorm.io/gorm" - "strconv" ) func CreateCollections(db *gorm.DB, collectionIds ...types.UniqueID) error { // create test collections for index, collectionId := range collectionIds { collectionName := "collection" + strconv.Itoa(index+1) - collectionTopic := "topic" + strconv.Itoa(index+1) var collectionDimension int32 = 6 collection := &dbmodel.Collection{ ID: collectionId.String(), Name: &collectionName, - Topic: &collectionTopic, Dimension: &collectionDimension, DatabaseID: types.NewUniqueID().String(), } diff --git a/go/pkg/metastore/catalog.go b/go/pkg/metastore/catalog.go index 15f73bc0d1f3..52a5d91037a8 100644 --- a/go/pkg/metastore/catalog.go +++ b/go/pkg/metastore/catalog.go @@ -2,6 +2,7 @@ package metastore import ( "context" + "github.com/chroma-core/chroma/go/pkg/metastore/db/dbmodel" "github.com/chroma-core/chroma/go/pkg/model" @@ -14,11 +15,11 @@ import ( type Catalog interface { ResetState(ctx context.Context) error CreateCollection(ctx context.Context, createCollection *model.CreateCollection, ts types.Timestamp) (*model.Collection, error) - GetCollections(ctx context.Context, collectionID types.UniqueID, collectionName *string, collectionTopic *string, tenantID string, databaseName string) ([]*model.Collection, error) + GetCollections(ctx context.Context, collectionID types.UniqueID, collectionName *string, tenantID string, databaseName string) ([]*model.Collection, error) DeleteCollection(ctx context.Context, deleteCollection *model.DeleteCollection) error UpdateCollection(ctx context.Context, updateCollection *model.UpdateCollection, ts types.Timestamp) (*model.Collection, error) CreateSegment(ctx context.Context, createSegment *model.CreateSegment, ts types.Timestamp) (*model.Segment, error) - GetSegments(ctx context.Context, segmentID types.UniqueID, segmentType *string, scope *string, topic *string, collectionID types.UniqueID) ([]*model.Segment, error) + GetSegments(ctx context.Context, segmentID types.UniqueID, segmentType *string, scope *string, collectionID types.UniqueID) ([]*model.Segment, error) DeleteSegment(ctx context.Context, segmentID types.UniqueID) error UpdateSegment(ctx context.Context, segmentInfo *model.UpdateSegment, ts types.Timestamp) (*model.Segment, error) CreateDatabase(ctx context.Context, createDatabase *model.CreateDatabase, ts types.Timestamp) (*model.Database, error) diff --git a/go/pkg/metastore/coordinator/model_db_convert.go b/go/pkg/metastore/coordinator/model_db_convert.go index 717b713cf192..7c57feb896d6 100644 --- a/go/pkg/metastore/coordinator/model_db_convert.go +++ b/go/pkg/metastore/coordinator/model_db_convert.go @@ -17,7 +17,6 @@ func convertCollectionToModel(collectionAndMetadataList []*dbmodel.CollectionAnd collection := &model.Collection{ ID: types.MustParse(collectionAndMetadata.Collection.ID), Name: *collectionAndMetadata.Collection.Name, - Topic: *collectionAndMetadata.Collection.Topic, Dimension: collectionAndMetadata.Collection.Dimension, TenantID: collectionAndMetadata.TenantID, DatabaseName: collectionAndMetadata.DatabaseName, @@ -98,7 +97,6 @@ func convertSegmentToModel(segmentAndMetadataList []*dbmodel.SegmentAndMetadata) ID: types.MustParse(segmentAndMetadata.Segment.ID), Type: segmentAndMetadata.Segment.Type, Scope: segmentAndMetadata.Segment.Scope, - Topic: segmentAndMetadata.Segment.Topic, Ts: segmentAndMetadata.Segment.Ts, } if segmentAndMetadata.Segment.CollectionID != nil { diff --git a/go/pkg/metastore/coordinator/model_db_convert_test.go b/go/pkg/metastore/coordinator/model_db_convert_test.go index 949c020f4131..27d4c1dd4da5 100644 --- a/go/pkg/metastore/coordinator/model_db_convert_test.go +++ b/go/pkg/metastore/coordinator/model_db_convert_test.go @@ -76,14 +76,12 @@ func TestConvertSegmentToModel(t *testing.T) { // Test case 3: segmentAndMetadataList contains one segment with all fields set segmentID := types.MustParse("515fc331-e117-4b86-bd84-85341128c337") - segmentTopic := "segment_topic" collectionID := "d9a75e2e-2929-45c4-af06-75b15630edd0" segmentAndMetadata := &dbmodel.SegmentAndMetadata{ Segment: &dbmodel.Segment{ ID: segmentID.String(), Type: "segment_type", Scope: "segment_scope", - Topic: &segmentTopic, CollectionID: &collectionID, }, SegmentMetadata: []*dbmodel.SegmentMetadata{}, @@ -94,7 +92,6 @@ func TestConvertSegmentToModel(t *testing.T) { assert.Equal(t, segmentID, modelSegments[0].ID) assert.Equal(t, "segment_type", modelSegments[0].Type) assert.Equal(t, "segment_scope", modelSegments[0].Scope) - assert.Equal(t, "segment_topic", *modelSegments[0].Topic) assert.Equal(t, types.MustParse(collectionID), modelSegments[0].CollectionID) assert.Nil(t, modelSegments[0].Metadata) } @@ -136,13 +133,11 @@ func TestConvertCollectionToModel(t *testing.T) { // Test case 3: collectionAndMetadataList contains one collection with all fields set collectionID := types.MustParse("d9a75e2e-2929-45c4-af06-75b15630edd0") collectionName := "collection_name" - collectionTopic := "collection_topic" collectionDimension := int32(3) collectionAndMetadata := &dbmodel.CollectionAndMetadata{ Collection: &dbmodel.Collection{ ID: collectionID.String(), Name: &collectionName, - Topic: &collectionTopic, Dimension: &collectionDimension, }, CollectionMetadata: []*dbmodel.CollectionMetadata{}, @@ -152,7 +147,6 @@ func TestConvertCollectionToModel(t *testing.T) { assert.Len(t, modelCollections, 1) assert.Equal(t, collectionID, modelCollections[0].ID) assert.Equal(t, collectionName, modelCollections[0].Name) - assert.Equal(t, collectionTopic, modelCollections[0].Topic) assert.Equal(t, collectionDimension, *modelCollections[0].Dimension) assert.Nil(t, modelCollections[0].Metadata) } diff --git a/go/pkg/metastore/coordinator/table_catalog.go b/go/pkg/metastore/coordinator/table_catalog.go index e1ae1e53d5c9..fcbe8e767747 100644 --- a/go/pkg/metastore/coordinator/table_catalog.go +++ b/go/pkg/metastore/coordinator/table_catalog.go @@ -2,6 +2,8 @@ package coordinator import ( "context" + "time" + "github.com/chroma-core/chroma/go/pkg/common" "github.com/chroma-core/chroma/go/pkg/metastore" "github.com/chroma-core/chroma/go/pkg/metastore/db/dbmodel" @@ -10,7 +12,6 @@ import ( "github.com/chroma-core/chroma/go/pkg/types" "github.com/pingcap/log" "go.uber.org/zap" - "time" ) // The catalog backed by databases using GORM. @@ -227,7 +228,7 @@ func (tc *Catalog) CreateCollection(ctx context.Context, createCollection *model } collectionName := createCollection.Name - existing, err := tc.metaDomain.CollectionDb(txCtx).GetCollections(nil, &collectionName, nil, tenantID, databaseName) + existing, err := tc.metaDomain.CollectionDb(txCtx).GetCollections(nil, &collectionName, tenantID, databaseName) if err != nil { log.Error("error getting collection", zap.Error(err)) return err @@ -258,7 +259,6 @@ func (tc *Catalog) CreateCollection(ctx context.Context, createCollection *model dbCollection := &dbmodel.Collection{ ID: createCollection.ID.String(), Name: &createCollection.Name, - Topic: &createCollection.Topic, Dimension: createCollection.Dimension, DatabaseID: databases[0].ID, Ts: ts, @@ -280,7 +280,7 @@ func (tc *Catalog) CreateCollection(ctx context.Context, createCollection *model } } // get collection - collectionList, err := tc.metaDomain.CollectionDb(txCtx).GetCollections(types.FromUniqueID(createCollection.ID), nil, nil, tenantID, databaseName) + collectionList, err := tc.metaDomain.CollectionDb(txCtx).GetCollections(types.FromUniqueID(createCollection.ID), nil, tenantID, databaseName) if err != nil { log.Error("error getting collection", zap.Error(err)) return err @@ -306,8 +306,8 @@ func (tc *Catalog) CreateCollection(ctx context.Context, createCollection *model return result, nil } -func (tc *Catalog) GetCollections(ctx context.Context, collectionID types.UniqueID, collectionName *string, collectionTopic *string, tenandID string, databaseName string) ([]*model.Collection, error) { - collectionAndMetadataList, err := tc.metaDomain.CollectionDb(ctx).GetCollections(types.FromUniqueID(collectionID), collectionName, collectionTopic, tenandID, databaseName) +func (tc *Catalog) GetCollections(ctx context.Context, collectionID types.UniqueID, collectionName *string, tenantID string, databaseName string) ([]*model.Collection, error) { + collectionAndMetadataList, err := tc.metaDomain.CollectionDb(ctx).GetCollections(types.FromUniqueID(collectionID), collectionName, tenantID, databaseName) if err != nil { return nil, err } @@ -319,7 +319,7 @@ func (tc *Catalog) DeleteCollection(ctx context.Context, deleteCollection *model log.Info("deleting collection", zap.Any("deleteCollection", deleteCollection)) return tc.txImpl.Transaction(ctx, func(txCtx context.Context) error { collectionID := deleteCollection.ID - collectionAndMetadata, err := tc.metaDomain.CollectionDb(txCtx).GetCollections(types.FromUniqueID(collectionID), nil, nil, deleteCollection.TenantID, deleteCollection.DatabaseName) + collectionAndMetadata, err := tc.metaDomain.CollectionDb(txCtx).GetCollections(types.FromUniqueID(collectionID), nil, deleteCollection.TenantID, deleteCollection.DatabaseName) if err != nil { return err } @@ -359,7 +359,6 @@ func (tc *Catalog) UpdateCollection(ctx context.Context, updateCollection *model dbCollection := &dbmodel.Collection{ ID: updateCollection.ID.String(), Name: updateCollection.Name, - Topic: updateCollection.Topic, Dimension: updateCollection.Dimension, Ts: ts, } @@ -400,7 +399,7 @@ func (tc *Catalog) UpdateCollection(ctx context.Context, updateCollection *model } databaseName := updateCollection.DatabaseName tenantID := updateCollection.TenantID - collectionList, err := tc.metaDomain.CollectionDb(txCtx).GetCollections(types.FromUniqueID(updateCollection.ID), nil, nil, tenantID, databaseName) + collectionList, err := tc.metaDomain.CollectionDb(txCtx).GetCollections(types.FromUniqueID(updateCollection.ID), nil, tenantID, databaseName) if err != nil { return err } @@ -430,9 +429,6 @@ func (tc *Catalog) CreateSegment(ctx context.Context, createSegment *model.Creat Scope: createSegment.Scope, Ts: ts, } - if createSegment.Topic != nil { - dbSegment.Topic = createSegment.Topic - } err := tc.metaDomain.SegmentDb(txCtx).Insert(dbSegment) if err != nil { log.Error("error inserting segment", zap.Error(err)) @@ -451,7 +447,7 @@ func (tc *Catalog) CreateSegment(ctx context.Context, createSegment *model.Creat } } // get segment - segmentList, err := tc.metaDomain.SegmentDb(txCtx).GetSegments(createSegment.ID, nil, nil, nil, types.NilUniqueID()) + segmentList, err := tc.metaDomain.SegmentDb(txCtx).GetSegments(createSegment.ID, nil, nil, types.NilUniqueID()) if err != nil { log.Error("error getting segment", zap.Error(err)) return err @@ -467,8 +463,8 @@ func (tc *Catalog) CreateSegment(ctx context.Context, createSegment *model.Creat return result, nil } -func (tc *Catalog) GetSegments(ctx context.Context, segmentID types.UniqueID, segmentType *string, scope *string, topic *string, collectionID types.UniqueID) ([]*model.Segment, error) { - segmentAndMetadataList, err := tc.metaDomain.SegmentDb(ctx).GetSegments(segmentID, segmentType, scope, topic, collectionID) +func (tc *Catalog) GetSegments(ctx context.Context, segmentID types.UniqueID, segmentType *string, scope *string, collectionID types.UniqueID) ([]*model.Segment, error) { + segmentAndMetadataList, err := tc.metaDomain.SegmentDb(ctx).GetSegments(segmentID, segmentType, scope, collectionID) if err != nil { return nil, err } @@ -478,7 +474,6 @@ func (tc *Catalog) GetSegments(ctx context.Context, segmentID types.UniqueID, se ID: types.MustParse(segmentAndMetadata.Segment.ID), Type: segmentAndMetadata.Segment.Type, Scope: segmentAndMetadata.Segment.Scope, - Topic: segmentAndMetadata.Segment.Topic, Ts: segmentAndMetadata.Segment.Ts, FilePaths: segmentAndMetadata.Segment.FilePaths, } @@ -496,7 +491,7 @@ func (tc *Catalog) GetSegments(ctx context.Context, segmentID types.UniqueID, se func (tc *Catalog) DeleteSegment(ctx context.Context, segmentID types.UniqueID) error { return tc.txImpl.Transaction(ctx, func(txCtx context.Context) error { - segment, err := tc.metaDomain.SegmentDb(txCtx).GetSegments(segmentID, nil, nil, nil, types.NilUniqueID()) + segment, err := tc.metaDomain.SegmentDb(txCtx).GetSegments(segmentID, nil, nil, types.NilUniqueID()) if err != nil { return err } @@ -524,7 +519,7 @@ func (tc *Catalog) UpdateSegment(ctx context.Context, updateSegment *model.Updat err := tc.txImpl.Transaction(ctx, func(txCtx context.Context) error { // TODO: we should push in collection_id here, add a GET to fix test for now if updateSegment.Collection == nil { - results, err := tc.metaDomain.SegmentDb(txCtx).GetSegments(updateSegment.ID, nil, nil, nil, types.NilUniqueID()) + results, err := tc.metaDomain.SegmentDb(txCtx).GetSegments(updateSegment.ID, nil, nil, types.NilUniqueID()) if err != nil { return err } @@ -541,8 +536,6 @@ func (tc *Catalog) UpdateSegment(ctx context.Context, updateSegment *model.Updat // update segment dbSegment := &dbmodel.UpdateSegment{ ID: updateSegment.ID.String(), - Topic: updateSegment.Topic, - ResetTopic: updateSegment.ResetTopic, Collection: updateSegment.Collection, ResetCollection: updateSegment.ResetCollection, } @@ -593,7 +586,7 @@ func (tc *Catalog) UpdateSegment(ctx context.Context, updateSegment *model.Updat } // get segment - segmentList, err := tc.metaDomain.SegmentDb(txCtx).GetSegments(updateSegment.ID, nil, nil, nil, types.NilUniqueID()) + segmentList, err := tc.metaDomain.SegmentDb(txCtx).GetSegments(updateSegment.ID, nil, nil, types.NilUniqueID()) if err != nil { log.Error("error getting segment", zap.Error(err)) return err diff --git a/go/pkg/metastore/coordinator/table_catalog_test.go b/go/pkg/metastore/coordinator/table_catalog_test.go index 44f4dba0ef08..d913925550e0 100644 --- a/go/pkg/metastore/coordinator/table_catalog_test.go +++ b/go/pkg/metastore/coordinator/table_catalog_test.go @@ -47,8 +47,7 @@ func TestCatalog_CreateCollection(t *testing.T) { mockMetaDomain.CollectionDb(context.Background()).(*mocks.ICollectionDb).On("Insert", &dbmodel.Collection{ ID: "00000000-0000-0000-0000-000000000001", Name: &name, - // Topic: "test_topic", - Ts: ts, + Ts: ts, }).Return(nil) // mock the insert collection metadata method @@ -87,9 +86,6 @@ func TestCatalog_GetCollections(t *testing.T) { // create a mock collection name collectionName := "test_collection" - // create a mock collection topic - collectionTopic := "test_topic" - // create a mock collection and metadata list name := "test_collection" testKey := "test_key" @@ -97,10 +93,9 @@ func TestCatalog_GetCollections(t *testing.T) { collectionAndMetadataList := []*dbmodel.CollectionAndMetadata{ { Collection: &dbmodel.Collection{ - ID: "00000000-0000-0000-0000-000000000001", - Name: &name, - Topic: &collectionTopic, - Ts: types.Timestamp(1234567890), + ID: "00000000-0000-0000-0000-000000000001", + Name: &name, + Ts: types.Timestamp(1234567890), }, CollectionMetadata: []*dbmodel.CollectionMetadata{ { @@ -115,10 +110,10 @@ func TestCatalog_GetCollections(t *testing.T) { // mock the get collections method mockMetaDomain.On("CollectionDb", context.Background()).Return(&mocks.ICollectionDb{}) - mockMetaDomain.CollectionDb(context.Background()).(*mocks.ICollectionDb).On("GetCollections", types.FromUniqueID(collectionID), &collectionName, &collectionTopic, common.DefaultTenant, common.DefaultDatabase).Return(collectionAndMetadataList, nil) + mockMetaDomain.CollectionDb(context.Background()).(*mocks.ICollectionDb).On("GetCollections", types.FromUniqueID(collectionID), &collectionName, common.DefaultTenant, common.DefaultDatabase).Return(collectionAndMetadataList, nil) // call the GetCollections method - collections, err := catalog.GetCollections(context.Background(), collectionID, &collectionName, &collectionTopic, defaultTenant, defaultDatabase) + collections, err := catalog.GetCollections(context.Background(), collectionID, &collectionName, defaultTenant, defaultDatabase) // assert that the method returned no error assert.NoError(t, err) @@ -130,7 +125,6 @@ func TestCatalog_GetCollections(t *testing.T) { { ID: types.MustParse("00000000-0000-0000-0000-000000000001"), Name: "test_collection", - Topic: collectionTopic, Ts: types.Timestamp(1234567890), Metadata: metadata, }, diff --git a/go/pkg/metastore/db/dao/collection.go b/go/pkg/metastore/db/dao/collection.go index f2f381b6b0dd..3a41b833022b 100644 --- a/go/pkg/metastore/db/dao/collection.go +++ b/go/pkg/metastore/db/dao/collection.go @@ -3,10 +3,11 @@ package dao import ( "database/sql" "errors" + "strings" + "github.com/chroma-core/chroma/go/pkg/common" "github.com/jackc/pgx/v5/pgconn" "gorm.io/gorm/clause" - "strings" "go.uber.org/zap" "gorm.io/gorm" @@ -25,14 +26,14 @@ func (s *collectionDb) DeleteAll() error { return s.db.Where("1 = 1").Delete(&dbmodel.Collection{}).Error } -func (s *collectionDb) GetCollections(id *string, name *string, topic *string, tenantID string, databaseName string) ([]*dbmodel.CollectionAndMetadata, error) { +func (s *collectionDb) GetCollections(id *string, name *string, tenantID string, databaseName string) ([]*dbmodel.CollectionAndMetadata, error) { var getCollectionInput strings.Builder getCollectionInput.WriteString("GetCollections input: ") var collections []*dbmodel.CollectionAndMetadata query := s.db.Table("collections"). - Select("collections.id, collections.log_position, collections.version, collections.name, collections.topic, collections.dimension, collections.database_id, databases.name, databases.tenant_id, collection_metadata.key, collection_metadata.str_value, collection_metadata.int_value, collection_metadata.float_value"). + Select("collections.id, collections.log_position, collections.version, collections.name, collections.dimension, collections.database_id, databases.name, databases.tenant_id, collection_metadata.key, collection_metadata.str_value, collection_metadata.int_value, collection_metadata.float_value"). Joins("LEFT JOIN collection_metadata ON collections.id = collection_metadata.collection_id"). Joins("INNER JOIN databases ON collections.database_id = databases.id"). Order("collections.id") @@ -51,10 +52,6 @@ func (s *collectionDb) GetCollections(id *string, name *string, topic *string, t query = query.Where("collections.id = ?", *id) getCollectionInput.WriteString("collections.id: " + *id + ", ") } - if topic != nil { - query = query.Where("collections.topic = ?", *topic) - getCollectionInput.WriteString("collections.topic: " + *topic + ", ") - } if name != nil { query = query.Where("collections.name = ?", *name) getCollectionInput.WriteString("collections.name: " + *name + ", ") @@ -77,7 +74,6 @@ func (s *collectionDb) GetCollections(id *string, name *string, topic *string, t logPosition int64 version int32 collectionName string - collectionTopic string collectionDimension sql.NullInt32 collectionDatabaseID string databaseName string @@ -88,7 +84,7 @@ func (s *collectionDb) GetCollections(id *string, name *string, topic *string, t floatValue sql.NullFloat64 ) - err := rows.Scan(&collectionID, &logPosition, &version, &collectionName, &collectionTopic, &collectionDimension, &collectionDatabaseID, &databaseName, &databaseTenantID, &key, &strValue, &intValue, &floatValue) + err := rows.Scan(&collectionID, &logPosition, &version, &collectionName, &collectionDimension, &collectionDatabaseID, &databaseName, &databaseTenantID, &key, &strValue, &intValue, &floatValue) if err != nil { log.Error("scan collection failed", zap.Error(err)) return nil, err @@ -101,7 +97,6 @@ func (s *collectionDb) GetCollections(id *string, name *string, topic *string, t Collection: &dbmodel.Collection{ ID: collectionID, Name: &collectionName, - Topic: &collectionTopic, DatabaseID: collectionDatabaseID, LogPosition: logPosition, Version: version, @@ -186,9 +181,6 @@ func generateCollectionUpdatesWithoutID(in *dbmodel.Collection) map[string]inter if in.Name != nil { ret["name"] = *in.Name } - if in.Topic != nil { - ret["topic"] = *in.Topic - } if in.Dimension != nil { ret["dimension"] = *in.Dimension } diff --git a/go/pkg/metastore/db/dao/collection_test.go b/go/pkg/metastore/db/dao/collection_test.go index 8e86a6203b51..7be7828e2253 100644 --- a/go/pkg/metastore/db/dao/collection_test.go +++ b/go/pkg/metastore/db/dao/collection_test.go @@ -1,10 +1,11 @@ package dao import ( + "testing" + "github.com/chroma-core/chroma/go/pkg/metastore/db/dbcore" "github.com/pingcap/log" "github.com/stretchr/testify/suite" - "testing" "github.com/chroma-core/chroma/go/pkg/metastore/db/dbmodel" "gorm.io/gorm" @@ -42,8 +43,7 @@ func (suite *CollectionDbTestSuite) TearDownSuite() { func (suite *CollectionDbTestSuite) TestCollectionDb_GetCollections() { collectionName := "test_collection_get_collections" - collectionTopic := "test_collection_topic" - collectionID, err := CreateTestCollection(suite.db, collectionName, collectionTopic, 128, suite.databaseId) + collectionID, err := CreateTestCollection(suite.db, collectionName, 128, suite.databaseId) suite.NoError(err) testKey := "test" @@ -65,30 +65,23 @@ func (suite *CollectionDbTestSuite) TestCollectionDb_GetCollections() { suite.NoError(err) suite.Equal(collectionID, scanedCollectionID) } - collections, err := suite.collectionDb.GetCollections(nil, nil, nil, suite.tenantName, suite.databaseName) + collections, err := suite.collectionDb.GetCollections(nil, nil, suite.tenantName, suite.databaseName) suite.NoError(err) suite.Len(collections, 1) suite.Equal(collectionID, collections[0].Collection.ID) suite.Equal(collectionName, *collections[0].Collection.Name) - suite.Equal(collectionTopic, *collections[0].Collection.Topic) suite.Len(collections[0].CollectionMetadata, 1) suite.Equal(metadata.Key, collections[0].CollectionMetadata[0].Key) suite.Equal(metadata.StrValue, collections[0].CollectionMetadata[0].StrValue) // Test when filtering by ID - collections, err = suite.collectionDb.GetCollections(nil, nil, nil, suite.tenantName, suite.databaseName) + collections, err = suite.collectionDb.GetCollections(nil, nil, suite.tenantName, suite.databaseName) suite.NoError(err) suite.Len(collections, 1) suite.Equal(collectionID, collections[0].Collection.ID) // Test when filtering by name - collections, err = suite.collectionDb.GetCollections(nil, &collectionName, nil, suite.tenantName, suite.databaseName) - suite.NoError(err) - suite.Len(collections, 1) - suite.Equal(collectionID, collections[0].Collection.ID) - - // Test when filtering by topic - collections, err = suite.collectionDb.GetCollections(nil, nil, &collectionTopic, suite.tenantName, suite.databaseName) + collections, err = suite.collectionDb.GetCollections(nil, &collectionName, suite.tenantName, suite.databaseName) suite.NoError(err) suite.Len(collections, 1) suite.Equal(collectionID, collections[0].Collection.ID) @@ -100,10 +93,9 @@ func (suite *CollectionDbTestSuite) TestCollectionDb_GetCollections() { func (suite *CollectionDbTestSuite) TestCollectionDb_UpdateLogPositionAndVersion() { collectionName := "test_collection_get_collections" - collectionTopic := "test_topic" - collectionID, err := CreateTestCollection(suite.db, collectionName, collectionTopic, 128, suite.databaseId) + collectionID, err := CreateTestCollection(suite.db, collectionName, 128, suite.databaseId) // verify default values - collections, err := suite.collectionDb.GetCollections(&collectionID, nil, nil, "", "") + collections, err := suite.collectionDb.GetCollections(&collectionID, nil, "", "") suite.NoError(err) suite.Len(collections, 1) suite.Equal(int64(0), collections[0].Collection.LogPosition) @@ -113,7 +105,7 @@ func (suite *CollectionDbTestSuite) TestCollectionDb_UpdateLogPositionAndVersion version, err := suite.collectionDb.UpdateLogPositionAndVersion(collectionID, int64(10), 0) suite.NoError(err) suite.Equal(int32(1), version) - collections, err = suite.collectionDb.GetCollections(&collectionID, nil, nil, "", "") + collections, err = suite.collectionDb.GetCollections(&collectionID, nil, "", "") suite.Len(collections, 1) suite.Equal(int64(10), collections[0].Collection.LogPosition) suite.Equal(int32(1), collections[0].Collection.Version) diff --git a/go/pkg/metastore/db/dao/segment.go b/go/pkg/metastore/db/dao/segment.go index a69cd13ce6a0..670cddb82673 100644 --- a/go/pkg/metastore/db/dao/segment.go +++ b/go/pkg/metastore/db/dao/segment.go @@ -4,6 +4,7 @@ import ( "database/sql" "encoding/json" "errors" + "github.com/chroma-core/chroma/go/pkg/common" "github.com/chroma-core/chroma/go/pkg/model" "github.com/jackc/pgx/v5/pgconn" @@ -51,11 +52,11 @@ func (s *segmentDb) Insert(in *dbmodel.Segment) error { return nil } -func (s *segmentDb) GetSegments(id types.UniqueID, segmentType *string, scope *string, topic *string, collectionID types.UniqueID) ([]*dbmodel.SegmentAndMetadata, error) { +func (s *segmentDb) GetSegments(id types.UniqueID, segmentType *string, scope *string, collectionID types.UniqueID) ([]*dbmodel.SegmentAndMetadata, error) { var segments []*dbmodel.SegmentAndMetadata query := s.db.Table("segments"). - Select("segments.id, segments.collection_id, segments.type, segments.scope, segments.topic, segments.file_paths, segment_metadata.key, segment_metadata.str_value, segment_metadata.int_value, segment_metadata.float_value"). + Select("segments.id, segments.collection_id, segments.type, segments.scope, segments.file_paths, segment_metadata.key, segment_metadata.str_value, segment_metadata.int_value, segment_metadata.float_value"). Joins("LEFT JOIN segment_metadata ON segments.id = segment_metadata.segment_id"). Order("segments.id") @@ -68,16 +69,13 @@ func (s *segmentDb) GetSegments(id types.UniqueID, segmentType *string, scope *s if scope != nil { query = query.Where("scope = ?", scope) } - if topic != nil { - query = query.Where("topic = ?", topic) - } if collectionID != types.NilUniqueID() { query = query.Where("collection_id = ?", collectionID.String()) } rows, err := query.Rows() if err != nil { - log.Error("get segments failed", zap.String("segmentID", id.String()), zap.String("segmentType", *segmentType), zap.String("scope", *scope), zap.String("collectionTopic", *topic), zap.Error(err)) + log.Error("get segments failed", zap.String("segmentID", id.String()), zap.String("segmentType", *segmentType), zap.String("scope", *scope), zap.Error(err)) return nil, err } defer rows.Close() @@ -92,7 +90,6 @@ func (s *segmentDb) GetSegments(id types.UniqueID, segmentType *string, scope *s collectionID sql.NullString segmentType string scope string - topic sql.NullString filePathsJson string key sql.NullString strValue sql.NullString @@ -100,7 +97,7 @@ func (s *segmentDb) GetSegments(id types.UniqueID, segmentType *string, scope *s floatValue sql.NullFloat64 ) - err := rows.Scan(&segmentID, &collectionID, &segmentType, &scope, &topic, &filePathsJson, &key, &strValue, &intValue, &floatValue) + err := rows.Scan(&segmentID, &collectionID, &segmentType, &scope, &filePathsJson, &key, &strValue, &intValue, &floatValue) if err != nil { log.Error("scan segment failed", zap.Error(err)) } @@ -128,12 +125,6 @@ func (s *segmentDb) GetSegments(id types.UniqueID, segmentType *string, scope *s currentSegment.Segment.CollectionID = nil } - if topic.Valid { - currentSegment.Segment.Topic = &topic.String - } else { - currentSegment.Segment.Topic = nil - } - if currentSegmentID != "" { segments = append(segments, currentSegment) } @@ -174,21 +165,8 @@ func (s *segmentDb) GetSegments(id types.UniqueID, segmentType *string, scope *s } func generateSegmentUpdatesWithoutID(in *dbmodel.UpdateSegment) map[string]interface{} { - // Case 1: if ResetTopic is true and topic is nil, then set the topic to nil - // Case 2: if ResetTopic is true and topic is not nil -> THIS SHOULD NEVER HAPPEN - // Case 3: if ResetTopic is false and topic is not nil - set the topic to the value in topic - // Case 4: if ResetTopic is false and topic is nil, then leave the topic as is log.Info("generate segment updates without id", zap.Any("in", in)) ret := map[string]interface{}{} - if in.ResetTopic { - if in.Topic == nil { - ret["topic"] = nil - } - } else { - if in.Topic != nil { - ret["topic"] = *in.Topic - } - } // TODO: check this //if in.ResetCollection { diff --git a/go/pkg/metastore/db/dao/segment_test.go b/go/pkg/metastore/db/dao/segment_test.go index 7712ccf0bedd..f2a5cc0409ca 100644 --- a/go/pkg/metastore/db/dao/segment_test.go +++ b/go/pkg/metastore/db/dao/segment_test.go @@ -1,13 +1,14 @@ package dao import ( + "strconv" + "testing" + "github.com/chroma-core/chroma/go/pkg/metastore/db/dbcore" "github.com/chroma-core/chroma/go/pkg/model" "github.com/pingcap/log" "github.com/stretchr/testify/suite" "k8s.io/apimachinery/pkg/util/rand" - "strconv" - "testing" "github.com/chroma-core/chroma/go/pkg/metastore/db/dbmodel" "github.com/chroma-core/chroma/go/pkg/types" @@ -31,13 +32,11 @@ func (suite *SegmentDbTestSuite) SetupSuite() { func (suite *SegmentDbTestSuite) TestSegmentDb_GetSegments() { uniqueID := types.NewUniqueID() collectionID := uniqueID.String() - testTopic := "test_segment_topic" segment := &dbmodel.Segment{ ID: uniqueID.String(), CollectionID: &collectionID, Type: "test_type", Scope: "test_scope", - Topic: &testTopic, } err := suite.db.Create(segment).Error suite.NoError(err) @@ -53,44 +52,37 @@ func (suite *SegmentDbTestSuite) TestSegmentDb_GetSegments() { suite.NoError(err) // Test when all parameters are nil - segments, err := suite.segmentDb.GetSegments(types.NilUniqueID(), nil, nil, nil, types.NilUniqueID()) + segments, err := suite.segmentDb.GetSegments(types.NilUniqueID(), nil, nil, types.NilUniqueID()) suite.NoError(err) suite.Len(segments, 1) suite.Equal(segment.ID, segments[0].Segment.ID) suite.Equal(segment.CollectionID, segments[0].Segment.CollectionID) suite.Equal(segment.Type, segments[0].Segment.Type) suite.Equal(segment.Scope, segments[0].Segment.Scope) - suite.Equal(segment.Topic, segments[0].Segment.Topic) suite.Len(segments[0].SegmentMetadata, 1) suite.Equal(metadata.Key, segments[0].SegmentMetadata[0].Key) suite.Equal(metadata.StrValue, segments[0].SegmentMetadata[0].StrValue) // Test when filtering by ID - segments, err = suite.segmentDb.GetSegments(types.MustParse(segment.ID), nil, nil, nil, types.NilUniqueID()) + segments, err = suite.segmentDb.GetSegments(types.MustParse(segment.ID), nil, nil, types.NilUniqueID()) suite.NoError(err) suite.Len(segments, 1) suite.Equal(segment.ID, segments[0].Segment.ID) // Test when filtering by type - segments, err = suite.segmentDb.GetSegments(types.NilUniqueID(), &segment.Type, nil, nil, types.NilUniqueID()) + segments, err = suite.segmentDb.GetSegments(types.NilUniqueID(), &segment.Type, nil, types.NilUniqueID()) suite.NoError(err) suite.Len(segments, 1) suite.Equal(segment.ID, segments[0].Segment.ID) // Test when filtering by scope - segments, err = suite.segmentDb.GetSegments(types.NilUniqueID(), nil, &segment.Scope, nil, types.NilUniqueID()) - suite.NoError(err) - suite.Len(segments, 1) - suite.Equal(segment.ID, segments[0].Segment.ID) - - // Test when filtering by topic - segments, err = suite.segmentDb.GetSegments(types.NilUniqueID(), nil, nil, segment.Topic, types.NilUniqueID()) + segments, err = suite.segmentDb.GetSegments(types.NilUniqueID(), nil, &segment.Scope, types.NilUniqueID()) suite.NoError(err) suite.Len(segments, 1) suite.Equal(segment.ID, segments[0].Segment.ID) // Test when filtering by collection ID - segments, err = suite.segmentDb.GetSegments(types.NilUniqueID(), nil, nil, nil, types.MustParse(*segment.CollectionID)) + segments, err = suite.segmentDb.GetSegments(types.NilUniqueID(), nil, nil, types.MustParse(*segment.CollectionID)) suite.NoError(err) suite.Len(segments, 1) suite.Equal(segment.ID, segments[0].Segment.ID) @@ -106,10 +98,10 @@ func (suite *SegmentDbTestSuite) TestSegmentDb_RegisterFilePath() { // create a collection for testing databaseId := types.NewUniqueID().String() collectionName := "test_segment_register_file_paths" - collectionID, err := CreateTestCollection(suite.db, collectionName, "test_topic", 128, databaseId) + collectionID, err := CreateTestCollection(suite.db, collectionName, 128, databaseId) suite.NoError(err) - segments, err := suite.segmentDb.GetSegments(types.NilUniqueID(), nil, nil, nil, types.MustParse(collectionID)) + segments, err := suite.segmentDb.GetSegments(types.NilUniqueID(), nil, nil, types.MustParse(collectionID)) suite.NoError(err) // create entries to flush @@ -140,7 +132,7 @@ func (suite *SegmentDbTestSuite) TestSegmentDb_RegisterFilePath() { suite.NoError(err) // verify file paths registered - segments, err = suite.segmentDb.GetSegments(types.NilUniqueID(), nil, nil, nil, types.MustParse(collectionID)) + segments, err = suite.segmentDb.GetSegments(types.NilUniqueID(), nil, nil, types.MustParse(collectionID)) suite.NoError(err) for _, segment := range segments { suite.Contains(segmentsFilePaths, segment.Segment.ID) diff --git a/go/pkg/metastore/db/dao/test_utils.go b/go/pkg/metastore/db/dao/test_utils.go index 6ae3293d1c1f..874dcabc112d 100644 --- a/go/pkg/metastore/db/dao/test_utils.go +++ b/go/pkg/metastore/db/dao/test_utils.go @@ -1,12 +1,13 @@ package dao import ( + "time" + "github.com/chroma-core/chroma/go/pkg/metastore/db/dbmodel" "github.com/chroma-core/chroma/go/pkg/types" "github.com/pingcap/log" "go.uber.org/zap" "gorm.io/gorm" - "time" ) const SegmentType = "urn:chroma:segment/vector/hnsw-distributed" @@ -51,7 +52,7 @@ func CleanUpTestDatabase(db *gorm.DB, tenantName string, databaseName string) er collectionDb := &collectionDb{ db: db, } - collections, err := collectionDb.GetCollections(nil, nil, nil, tenantName, databaseName) + collections, err := collectionDb.GetCollections(nil, nil, tenantName, databaseName) log.Info("clean up test database", zap.Int("collections", len(collections))) if err != nil { return err @@ -105,8 +106,8 @@ func CleanUpTestTenant(db *gorm.DB, tenantName string) error { return nil } -func CreateTestCollection(db *gorm.DB, collectionName string, topic string, dimension int32, databaseID string) (string, error) { - log.Info("create test collection", zap.String("collectionName", collectionName), zap.String("topic", topic), zap.Int32("dimension", dimension), zap.String("databaseID", databaseID)) +func CreateTestCollection(db *gorm.DB, collectionName string, dimension int32, databaseID string) (string, error) { + log.Info("create test collection", zap.String("collectionName", collectionName), zap.Int32("dimension", dimension), zap.String("databaseID", databaseID)) collectionDb := &collectionDb{ db: db, } @@ -118,7 +119,6 @@ func CreateTestCollection(db *gorm.DB, collectionName string, topic string, dime err := collectionDb.Insert(&dbmodel.Collection{ ID: collectionId, Name: &collectionName, - Topic: &topic, Dimension: &dimension, DatabaseID: databaseID, }) @@ -165,7 +165,7 @@ func CleanUpTestCollection(db *gorm.DB, collectionId string) error { if err != nil { return err } - segments, err := segmentDb.GetSegments(types.NilUniqueID(), nil, nil, nil, types.MustParse(collectionId)) + segments, err := segmentDb.GetSegments(types.NilUniqueID(), nil, nil, types.MustParse(collectionId)) if err != nil { return err } diff --git a/go/pkg/metastore/db/dbmodel/collection.go b/go/pkg/metastore/db/dbmodel/collection.go index 30a9ab945ac1..c6c769b4fa2c 100644 --- a/go/pkg/metastore/db/dbmodel/collection.go +++ b/go/pkg/metastore/db/dbmodel/collection.go @@ -9,7 +9,6 @@ import ( type Collection struct { ID string `gorm:"id;primaryKey"` Name *string `gorm:"name;unique"` - Topic *string `gorm:"topic"` Dimension *int32 `gorm:"dimension"` DatabaseID string `gorm:"database_id"` Ts types.Timestamp `gorm:"ts;type:bigint;default:0"` @@ -33,7 +32,7 @@ type CollectionAndMetadata struct { //go:generate mockery --name=ICollectionDb type ICollectionDb interface { - GetCollections(collectionID *string, collectionName *string, collectionTopic *string, tenantID string, databaseName string) ([]*CollectionAndMetadata, error) + GetCollections(collectionID *string, collectionName *string, tenantID string, databaseName string) ([]*CollectionAndMetadata, error) DeleteCollectionByID(collectionID string) (int, error) Insert(in *Collection) error Update(in *Collection) error diff --git a/go/pkg/metastore/db/dbmodel/mocks/ICollectionDb.go b/go/pkg/metastore/db/dbmodel/mocks/ICollectionDb.go index b819b0b1889c..186247562683 100644 --- a/go/pkg/metastore/db/dbmodel/mocks/ICollectionDb.go +++ b/go/pkg/metastore/db/dbmodel/mocks/ICollectionDb.go @@ -58,9 +58,9 @@ func (_m *ICollectionDb) DeleteCollectionByID(collectionID string) (int, error) return r0, r1 } -// GetCollections provides a mock function with given fields: collectionID, collectionName, collectionTopic, tenantID, databaseName -func (_m *ICollectionDb) GetCollections(collectionID *string, collectionName *string, collectionTopic *string, tenantID string, databaseName string) ([]*dbmodel.CollectionAndMetadata, error) { - ret := _m.Called(collectionID, collectionName, collectionTopic, tenantID, databaseName) +// GetCollections provides a mock function with given fields: collectionID, collectionName, tenantID, databaseName +func (_m *ICollectionDb) GetCollections(collectionID *string, collectionName *string, tenantID string, databaseName string) ([]*dbmodel.CollectionAndMetadata, error) { + ret := _m.Called(collectionID, collectionName, tenantID, databaseName) if len(ret) == 0 { panic("no return value specified for GetCollections") @@ -68,19 +68,19 @@ func (_m *ICollectionDb) GetCollections(collectionID *string, collectionName *st var r0 []*dbmodel.CollectionAndMetadata var r1 error - if rf, ok := ret.Get(0).(func(*string, *string, *string, string, string) ([]*dbmodel.CollectionAndMetadata, error)); ok { - return rf(collectionID, collectionName, collectionTopic, tenantID, databaseName) + if rf, ok := ret.Get(0).(func(*string, *string, string, string) ([]*dbmodel.CollectionAndMetadata, error)); ok { + return rf(collectionID, collectionName, tenantID, databaseName) } - if rf, ok := ret.Get(0).(func(*string, *string, *string, string, string) []*dbmodel.CollectionAndMetadata); ok { - r0 = rf(collectionID, collectionName, collectionTopic, tenantID, databaseName) + if rf, ok := ret.Get(0).(func(*string, *string, string, string) []*dbmodel.CollectionAndMetadata); ok { + r0 = rf(collectionID, collectionName, tenantID, databaseName) } else { if ret.Get(0) != nil { r0 = ret.Get(0).([]*dbmodel.CollectionAndMetadata) } } - if rf, ok := ret.Get(1).(func(*string, *string, *string, string, string) error); ok { - r1 = rf(collectionID, collectionName, collectionTopic, tenantID, databaseName) + if rf, ok := ret.Get(1).(func(*string, *string, string, string) error); ok { + r1 = rf(collectionID, collectionName, tenantID, databaseName) } else { r1 = ret.Error(1) } diff --git a/go/pkg/metastore/db/dbmodel/mocks/ISegmentDb.go b/go/pkg/metastore/db/dbmodel/mocks/ISegmentDb.go index 9a1290307c63..5fa22500ce34 100644 --- a/go/pkg/metastore/db/dbmodel/mocks/ISegmentDb.go +++ b/go/pkg/metastore/db/dbmodel/mocks/ISegmentDb.go @@ -42,25 +42,25 @@ func (_m *ISegmentDb) DeleteSegmentByID(id string) error { return r0 } -// GetSegments provides a mock function with given fields: id, segmentType, scope, topic, collectionID -func (_m *ISegmentDb) GetSegments(id types.UniqueID, segmentType *string, scope *string, topic *string, collectionID types.UniqueID) ([]*dbmodel.SegmentAndMetadata, error) { - ret := _m.Called(id, segmentType, scope, topic, collectionID) +// GetSegments provides a mock function with given fields: id, segmentType, scope, collectionID +func (_m *ISegmentDb) GetSegments(id types.UniqueID, segmentType *string, scope *string, collectionID types.UniqueID) ([]*dbmodel.SegmentAndMetadata, error) { + ret := _m.Called(id, segmentType, scope, collectionID) var r0 []*dbmodel.SegmentAndMetadata var r1 error - if rf, ok := ret.Get(0).(func(types.UniqueID, *string, *string, *string, types.UniqueID) ([]*dbmodel.SegmentAndMetadata, error)); ok { - return rf(id, segmentType, scope, topic, collectionID) + if rf, ok := ret.Get(0).(func(types.UniqueID, *string, *string, types.UniqueID) ([]*dbmodel.SegmentAndMetadata, error)); ok { + return rf(id, segmentType, scope, collectionID) } - if rf, ok := ret.Get(0).(func(types.UniqueID, *string, *string, *string, types.UniqueID) []*dbmodel.SegmentAndMetadata); ok { - r0 = rf(id, segmentType, scope, topic, collectionID) + if rf, ok := ret.Get(0).(func(types.UniqueID, *string, *string, types.UniqueID) []*dbmodel.SegmentAndMetadata); ok { + r0 = rf(id, segmentType, scope, collectionID) } else { if ret.Get(0) != nil { r0 = ret.Get(0).([]*dbmodel.SegmentAndMetadata) } } - if rf, ok := ret.Get(1).(func(types.UniqueID, *string, *string, *string, types.UniqueID) error); ok { - r1 = rf(id, segmentType, scope, topic, collectionID) + if rf, ok := ret.Get(1).(func(types.UniqueID, *string, *string, types.UniqueID) error); ok { + r1 = rf(id, segmentType, scope, collectionID) } else { r1 = ret.Error(1) } diff --git a/go/pkg/metastore/db/dbmodel/segment.go b/go/pkg/metastore/db/dbmodel/segment.go index 14eaf19ca4c7..e5a3af0a8b34 100644 --- a/go/pkg/metastore/db/dbmodel/segment.go +++ b/go/pkg/metastore/db/dbmodel/segment.go @@ -1,9 +1,10 @@ package dbmodel import ( - "github.com/chroma-core/chroma/go/pkg/model" "time" + "github.com/chroma-core/chroma/go/pkg/model" + "github.com/chroma-core/chroma/go/pkg/types" ) @@ -16,7 +17,6 @@ type Segment struct { ID string `gorm:"id;primaryKey"` Type string `gorm:"type;type:string;not null"` Scope string `gorm:"scope"` - Topic *string `gorm:"topic"` Ts types.Timestamp `gorm:"ts;type:bigint;default:0"` IsDeleted bool `gorm:"is_deleted;type:bool;default:false"` CreatedAt time.Time `gorm:"created_at;type:timestamp;not null;default:current_timestamp"` @@ -35,15 +35,13 @@ type SegmentAndMetadata struct { type UpdateSegment struct { ID string - Topic *string - ResetTopic bool Collection *string ResetCollection bool } //go:generate mockery --name=ISegmentDb type ISegmentDb interface { - GetSegments(id types.UniqueID, segmentType *string, scope *string, topic *string, collectionID types.UniqueID) ([]*SegmentAndMetadata, error) + GetSegments(id types.UniqueID, segmentType *string, scope *string, collectionID types.UniqueID) ([]*SegmentAndMetadata, error) DeleteSegmentByID(id string) error Insert(*Segment) error Update(*UpdateSegment) error diff --git a/go/pkg/metastore/mocks/Catalog.go b/go/pkg/metastore/mocks/Catalog.go index e2df8575aa58..596d2b6a33fa 100644 --- a/go/pkg/metastore/mocks/Catalog.go +++ b/go/pkg/metastore/mocks/Catalog.go @@ -97,25 +97,25 @@ func (_m *Catalog) DeleteSegment(ctx context.Context, segmentID types.UniqueID) return r0 } -// GetCollections provides a mock function with given fields: ctx, collectionID, collectionName, collectionTopic -func (_m *Catalog) GetCollections(ctx context.Context, collectionID types.UniqueID, collectionName *string, collectionTopic *string) ([]*model.Collection, error) { - ret := _m.Called(ctx, collectionID, collectionName, collectionTopic) +// GetCollections provides a mock function with given fields: ctx, collectionID, collectionName +func (_m *Catalog) GetCollections(ctx context.Context, collectionID types.UniqueID, collectionName *string) ([]*model.Collection, error) { + ret := _m.Called(ctx, collectionID, collectionName) var r0 []*model.Collection var r1 error - if rf, ok := ret.Get(0).(func(context.Context, types.UniqueID, *string, *string) ([]*model.Collection, error)); ok { - return rf(ctx, collectionID, collectionName, collectionTopic) + if rf, ok := ret.Get(0).(func(context.Context, types.UniqueID, *string) ([]*model.Collection, error)); ok { + return rf(ctx, collectionID, collectionName) } - if rf, ok := ret.Get(0).(func(context.Context, types.UniqueID, *string, *string) []*model.Collection); ok { - r0 = rf(ctx, collectionID, collectionName, collectionTopic) + if rf, ok := ret.Get(0).(func(context.Context, types.UniqueID, *string) []*model.Collection); ok { + r0 = rf(ctx, collectionID, collectionName) } else { if ret.Get(0) != nil { r0 = ret.Get(0).([]*model.Collection) } } - if rf, ok := ret.Get(1).(func(context.Context, types.UniqueID, *string, *string) error); ok { - r1 = rf(ctx, collectionID, collectionName, collectionTopic) + if rf, ok := ret.Get(1).(func(context.Context, types.UniqueID, *string) error); ok { + r1 = rf(ctx, collectionID, collectionName) } else { r1 = ret.Error(1) } @@ -123,25 +123,25 @@ func (_m *Catalog) GetCollections(ctx context.Context, collectionID types.Unique return r0, r1 } -// GetSegments provides a mock function with given fields: ctx, segmentID, segmentType, scope, topic, collectionID, ts -func (_m *Catalog) GetSegments(ctx context.Context, segmentID types.UniqueID, segmentType *string, scope *string, topic *string, collectionID types.UniqueID, ts int64) ([]*model.Segment, error) { - ret := _m.Called(ctx, segmentID, segmentType, scope, topic, collectionID, ts) +// GetSegments provides a mock function with given fields: ctx, segmentID, segmentType, scope, collectionID, ts +func (_m *Catalog) GetSegments(ctx context.Context, segmentID types.UniqueID, segmentType *string, scope *string, collectionID types.UniqueID, ts int64) ([]*model.Segment, error) { + ret := _m.Called(ctx, segmentID, segmentType, scope, collectionID, ts) var r0 []*model.Segment var r1 error - if rf, ok := ret.Get(0).(func(context.Context, types.UniqueID, *string, *string, *string, types.UniqueID, int64) ([]*model.Segment, error)); ok { - return rf(ctx, segmentID, segmentType, scope, topic, collectionID, ts) + if rf, ok := ret.Get(0).(func(context.Context, types.UniqueID, *string, *string, types.UniqueID, int64) ([]*model.Segment, error)); ok { + return rf(ctx, segmentID, segmentType, scope, collectionID, ts) } - if rf, ok := ret.Get(0).(func(context.Context, types.UniqueID, *string, *string, *string, types.UniqueID, int64) []*model.Segment); ok { - r0 = rf(ctx, segmentID, segmentType, scope, topic, collectionID, ts) + if rf, ok := ret.Get(0).(func(context.Context, types.UniqueID, *string, *string, types.UniqueID, int64) []*model.Segment); ok { + r0 = rf(ctx, segmentID, segmentType, scope, collectionID, ts) } else { if ret.Get(0) != nil { r0 = ret.Get(0).([]*model.Segment) } } - if rf, ok := ret.Get(1).(func(context.Context, types.UniqueID, *string, *string, *string, types.UniqueID, int64) error); ok { - r1 = rf(ctx, segmentID, segmentType, scope, topic, collectionID, ts) + if rf, ok := ret.Get(1).(func(context.Context, types.UniqueID, *string, *string, types.UniqueID, int64) error); ok { + r1 = rf(ctx, segmentID, segmentType, scope, collectionID, ts) } else { r1 = ret.Error(1) } diff --git a/go/pkg/model/collection.go b/go/pkg/model/collection.go index 1340c44df5b0..ec35569daab7 100644 --- a/go/pkg/model/collection.go +++ b/go/pkg/model/collection.go @@ -7,7 +7,6 @@ import ( type Collection struct { ID types.UniqueID Name string - Topic string Dimension *int32 Metadata *CollectionMetadata[CollectionMetadataValueType] TenantID string @@ -20,7 +19,6 @@ type Collection struct { type CreateCollection struct { ID types.UniqueID Name string - Topic string Dimension *int32 Metadata *CollectionMetadata[CollectionMetadataValueType] GetOrCreate bool @@ -39,7 +37,6 @@ type DeleteCollection struct { type UpdateCollection struct { ID types.UniqueID Name *string - Topic *string Dimension *int32 Metadata *CollectionMetadata[CollectionMetadataValueType] ResetMetadata bool @@ -62,15 +59,12 @@ type FlushCollectionInfo struct { TenantLastCompactionTime int64 } -func FilterCollection(collection *Collection, collectionID types.UniqueID, collectionName *string, collectionTopic *string) bool { +func FilterCollection(collection *Collection, collectionID types.UniqueID, collectionName *string) bool { if collectionID != types.NilUniqueID() && collectionID != collection.ID { return false } if collectionName != nil && *collectionName != collection.Name { return false } - if collectionTopic != nil && *collectionTopic != collection.Topic { - return false - } return true } diff --git a/go/pkg/model/segment.go b/go/pkg/model/segment.go index 07030e77c915..5e30c96df1c8 100644 --- a/go/pkg/model/segment.go +++ b/go/pkg/model/segment.go @@ -8,7 +8,6 @@ type Segment struct { ID types.UniqueID Type string Scope string - Topic *string CollectionID types.UniqueID Metadata *SegmentMetadata[SegmentMetadataValueType] Ts types.Timestamp @@ -19,7 +18,6 @@ type CreateSegment struct { ID types.UniqueID Type string Scope string - Topic *string CollectionID types.UniqueID Metadata *SegmentMetadata[SegmentMetadataValueType] Ts types.Timestamp @@ -27,7 +25,6 @@ type CreateSegment struct { type UpdateSegment struct { ID types.UniqueID - Topic *string ResetTopic bool Collection *string ResetCollection bool @@ -40,7 +37,6 @@ type GetSegments struct { ID types.UniqueID Type *string Scope *string - Topic *string CollectionID types.UniqueID } @@ -61,10 +57,6 @@ func FilterSegments(segment *Segment, segmentID types.UniqueID, segmentType *str return false } - if topic != nil && *segment.Topic != *topic { - return false - } - if collectionID != types.NilUniqueID() && segment.CollectionID != collectionID { return false } diff --git a/go/pkg/proto/coordinatorpb/chroma.pb.go b/go/pkg/proto/coordinatorpb/chroma.pb.go index 62566a9a0839..201f5b43bd1e 100644 --- a/go/pkg/proto/coordinatorpb/chroma.pb.go +++ b/go/pkg/proto/coordinatorpb/chroma.pb.go @@ -1,6 +1,6 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.31.0 +// protoc-gen-go v1.33.0 // protoc v4.23.4 // source: chromadb/proto/chroma.proto @@ -20,6 +20,7 @@ const ( _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) ) +// Types here should mirror chromadb/types.py type Operation int32 const ( @@ -334,12 +335,9 @@ type Segment struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` - Type string `protobuf:"bytes,2,opt,name=type,proto3" json:"type,omitempty"` - Scope SegmentScope `protobuf:"varint,3,opt,name=scope,proto3,enum=chroma.SegmentScope" json:"scope,omitempty"` - Topic *string `protobuf:"bytes,4,opt,name=topic,proto3,oneof" json:"topic,omitempty"` // TODO should channel <> segment binding exist here? - // If a segment has a collection, it implies that this segment implements the full - // collection and can be used to service queries (for it's given scope.) + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Type string `protobuf:"bytes,2,opt,name=type,proto3" json:"type,omitempty"` + Scope SegmentScope `protobuf:"varint,3,opt,name=scope,proto3,enum=chroma.SegmentScope" json:"scope,omitempty"` Collection *string `protobuf:"bytes,5,opt,name=collection,proto3,oneof" json:"collection,omitempty"` Metadata *UpdateMetadata `protobuf:"bytes,6,opt,name=metadata,proto3,oneof" json:"metadata,omitempty"` FilePaths map[string]*FilePaths `protobuf:"bytes,7,rep,name=file_paths,json=filePaths,proto3" json:"file_paths,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` @@ -398,13 +396,6 @@ func (x *Segment) GetScope() SegmentScope { return SegmentScope_VECTOR } -func (x *Segment) GetTopic() string { - if x != nil && x.Topic != nil { - return *x.Topic - } - return "" -} - func (x *Segment) GetCollection() string { if x != nil && x.Collection != nil { return *x.Collection @@ -433,7 +424,6 @@ type Collection struct { Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` - Topic string `protobuf:"bytes,3,opt,name=topic,proto3" json:"topic,omitempty"` Metadata *UpdateMetadata `protobuf:"bytes,4,opt,name=metadata,proto3,oneof" json:"metadata,omitempty"` Dimension *int32 `protobuf:"varint,5,opt,name=dimension,proto3,oneof" json:"dimension,omitempty"` Tenant string `protobuf:"bytes,6,opt,name=tenant,proto3" json:"tenant,omitempty"` @@ -488,13 +478,6 @@ func (x *Collection) GetName() string { return "" } -func (x *Collection) GetTopic() string { - if x != nil { - return x.Topic - } - return "" -} - func (x *Collection) GetMetadata() *UpdateMetadata { if x != nil { return x.Metadata @@ -789,20 +772,20 @@ func (x *UpdateMetadata) GetMetadata() map[string]*UpdateMetadataValue { return nil } -type SubmitEmbeddingRecord struct { +// Represents an operation on the log +type OperationRecord struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` - Vector *Vector `protobuf:"bytes,2,opt,name=vector,proto3,oneof" json:"vector,omitempty"` - Metadata *UpdateMetadata `protobuf:"bytes,3,opt,name=metadata,proto3,oneof" json:"metadata,omitempty"` - Operation Operation `protobuf:"varint,4,opt,name=operation,proto3,enum=chroma.Operation" json:"operation,omitempty"` - CollectionId string `protobuf:"bytes,5,opt,name=collection_id,json=collectionId,proto3" json:"collection_id,omitempty"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Vector *Vector `protobuf:"bytes,2,opt,name=vector,proto3,oneof" json:"vector,omitempty"` + Metadata *UpdateMetadata `protobuf:"bytes,3,opt,name=metadata,proto3,oneof" json:"metadata,omitempty"` + Operation Operation `protobuf:"varint,4,opt,name=operation,proto3,enum=chroma.Operation" json:"operation,omitempty"` } -func (x *SubmitEmbeddingRecord) Reset() { - *x = SubmitEmbeddingRecord{} +func (x *OperationRecord) Reset() { + *x = OperationRecord{} if protoimpl.UnsafeEnabled { mi := &file_chromadb_proto_chroma_proto_msgTypes[9] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -810,13 +793,13 @@ func (x *SubmitEmbeddingRecord) Reset() { } } -func (x *SubmitEmbeddingRecord) String() string { +func (x *OperationRecord) String() string { return protoimpl.X.MessageStringOf(x) } -func (*SubmitEmbeddingRecord) ProtoMessage() {} +func (*OperationRecord) ProtoMessage() {} -func (x *SubmitEmbeddingRecord) ProtoReflect() protoreflect.Message { +func (x *OperationRecord) ProtoReflect() protoreflect.Message { mi := &file_chromadb_proto_chroma_proto_msgTypes[9] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -828,46 +811,39 @@ func (x *SubmitEmbeddingRecord) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use SubmitEmbeddingRecord.ProtoReflect.Descriptor instead. -func (*SubmitEmbeddingRecord) Descriptor() ([]byte, []int) { +// Deprecated: Use OperationRecord.ProtoReflect.Descriptor instead. +func (*OperationRecord) Descriptor() ([]byte, []int) { return file_chromadb_proto_chroma_proto_rawDescGZIP(), []int{9} } -func (x *SubmitEmbeddingRecord) GetId() string { +func (x *OperationRecord) GetId() string { if x != nil { return x.Id } return "" } -func (x *SubmitEmbeddingRecord) GetVector() *Vector { +func (x *OperationRecord) GetVector() *Vector { if x != nil { return x.Vector } return nil } -func (x *SubmitEmbeddingRecord) GetMetadata() *UpdateMetadata { +func (x *OperationRecord) GetMetadata() *UpdateMetadata { if x != nil { return x.Metadata } return nil } -func (x *SubmitEmbeddingRecord) GetOperation() Operation { +func (x *OperationRecord) GetOperation() Operation { if x != nil { return x.Operation } return Operation_ADD } -func (x *SubmitEmbeddingRecord) GetCollectionId() string { - if x != nil { - return x.CollectionId - } - return "" -} - type VectorEmbeddingRecord struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -1295,162 +1271,155 @@ var file_chromadb_proto_chroma_proto_rawDesc = []byte{ 0x6f, 0x64, 0x69, 0x6e, 0x67, 0x52, 0x08, 0x65, 0x6e, 0x63, 0x6f, 0x64, 0x69, 0x6e, 0x67, 0x22, 0x21, 0x0a, 0x09, 0x46, 0x69, 0x6c, 0x65, 0x50, 0x61, 0x74, 0x68, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x70, 0x61, 0x74, 0x68, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x70, 0x61, 0x74, - 0x68, 0x73, 0x22, 0x88, 0x03, 0x0a, 0x07, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x0e, + 0x68, 0x73, 0x22, 0xe3, 0x02, 0x0a, 0x07, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x2a, 0x0a, 0x05, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x14, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, - 0x6e, 0x74, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x52, 0x05, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x12, 0x19, - 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, - 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x88, 0x01, 0x01, 0x12, 0x23, 0x0a, 0x0a, 0x63, 0x6f, 0x6c, - 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, - 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x88, 0x01, 0x01, 0x12, 0x37, - 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x16, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, - 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x48, 0x02, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, - 0x64, 0x61, 0x74, 0x61, 0x88, 0x01, 0x01, 0x12, 0x3d, 0x0a, 0x0a, 0x66, 0x69, 0x6c, 0x65, 0x5f, - 0x70, 0x61, 0x74, 0x68, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x63, 0x68, - 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x46, 0x69, 0x6c, - 0x65, 0x50, 0x61, 0x74, 0x68, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x09, 0x66, 0x69, 0x6c, - 0x65, 0x50, 0x61, 0x74, 0x68, 0x73, 0x1a, 0x4f, 0x0a, 0x0e, 0x46, 0x69, 0x6c, 0x65, 0x50, 0x61, - 0x74, 0x68, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x27, 0x0a, 0x05, 0x76, 0x61, - 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x63, 0x68, 0x72, 0x6f, - 0x6d, 0x61, 0x2e, 0x46, 0x69, 0x6c, 0x65, 0x50, 0x61, 0x74, 0x68, 0x73, 0x52, 0x05, 0x76, 0x61, - 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x42, 0x08, 0x0a, 0x06, 0x5f, 0x74, 0x6f, 0x70, 0x69, - 0x63, 0x42, 0x0d, 0x0a, 0x0b, 0x5f, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x42, 0x0b, 0x0a, 0x09, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x22, 0xad, 0x02, - 0x0a, 0x0a, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x0e, 0x0a, 0x02, - 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x12, 0x0a, 0x04, - 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, - 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x37, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, - 0x74, 0x61, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, - 0x61, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, - 0x48, 0x00, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x88, 0x01, 0x01, 0x12, - 0x21, 0x0a, 0x09, 0x64, 0x69, 0x6d, 0x65, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, - 0x28, 0x05, 0x48, 0x01, 0x52, 0x09, 0x64, 0x69, 0x6d, 0x65, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x88, - 0x01, 0x01, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x18, 0x06, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x06, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x64, 0x61, - 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x64, 0x61, - 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x12, 0x20, 0x0a, 0x0b, 0x6c, 0x6f, 0x67, 0x50, 0x6f, 0x73, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x6c, 0x6f, 0x67, - 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, - 0x69, 0x6f, 0x6e, 0x18, 0x09, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, - 0x6f, 0x6e, 0x42, 0x0b, 0x0a, 0x09, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x42, - 0x0c, 0x0a, 0x0a, 0x5f, 0x64, 0x69, 0x6d, 0x65, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x46, 0x0a, - 0x08, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, - 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x16, 0x0a, - 0x06, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, - 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x22, 0x1c, 0x0a, 0x06, 0x54, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x12, - 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, - 0x61, 0x6d, 0x65, 0x22, 0x85, 0x01, 0x0a, 0x13, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x4d, 0x65, - 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x23, 0x0a, 0x0c, 0x73, - 0x74, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x48, 0x00, 0x52, 0x0b, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x75, 0x65, - 0x12, 0x1d, 0x0a, 0x09, 0x69, 0x6e, 0x74, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x03, 0x48, 0x00, 0x52, 0x08, 0x69, 0x6e, 0x74, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, - 0x21, 0x0a, 0x0b, 0x66, 0x6c, 0x6f, 0x61, 0x74, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x01, 0x48, 0x00, 0x52, 0x0a, 0x66, 0x6c, 0x6f, 0x61, 0x74, 0x56, 0x61, 0x6c, - 0x75, 0x65, 0x42, 0x07, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x22, 0xac, 0x01, 0x0a, 0x0e, - 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x40, - 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x24, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, - 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, - 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, - 0x1a, 0x58, 0x0a, 0x0d, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, - 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, - 0x6b, 0x65, 0x79, 0x12, 0x31, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x55, 0x70, 0x64, 0x61, - 0x74, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, - 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xfb, 0x01, 0x0a, 0x15, 0x53, - 0x75, 0x62, 0x6d, 0x69, 0x74, 0x45, 0x6d, 0x62, 0x65, 0x64, 0x64, 0x69, 0x6e, 0x67, 0x52, 0x65, - 0x63, 0x6f, 0x72, 0x64, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x02, 0x69, 0x64, 0x12, 0x2b, 0x0a, 0x06, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x56, 0x65, - 0x63, 0x74, 0x6f, 0x72, 0x48, 0x00, 0x52, 0x06, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x88, 0x01, - 0x01, 0x12, 0x37, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x55, 0x70, 0x64, - 0x61, 0x74, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x48, 0x01, 0x52, 0x08, 0x6d, - 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x88, 0x01, 0x01, 0x12, 0x2f, 0x0a, 0x09, 0x6f, 0x70, - 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x11, 0x2e, - 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x52, 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x0a, 0x0d, 0x63, - 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, - 0x42, 0x09, 0x0a, 0x07, 0x5f, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x42, 0x0b, 0x0a, 0x09, 0x5f, - 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x22, 0x66, 0x0a, 0x15, 0x56, 0x65, 0x63, 0x74, - 0x6f, 0x72, 0x45, 0x6d, 0x62, 0x65, 0x64, 0x64, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x63, 0x6f, 0x72, - 0x64, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, - 0x64, 0x12, 0x15, 0x0a, 0x06, 0x73, 0x65, 0x71, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0c, 0x52, 0x05, 0x73, 0x65, 0x71, 0x49, 0x64, 0x12, 0x26, 0x0a, 0x06, 0x76, 0x65, 0x63, 0x74, - 0x6f, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, - 0x61, 0x2e, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x06, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72, - 0x22, 0x8e, 0x01, 0x0a, 0x11, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x51, 0x75, 0x65, 0x72, 0x79, - 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x15, 0x0a, 0x06, 0x73, 0x65, 0x71, 0x5f, 0x69, 0x64, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x05, 0x73, 0x65, 0x71, 0x49, 0x64, 0x12, 0x1a, 0x0a, - 0x08, 0x64, 0x69, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x02, 0x52, - 0x08, 0x64, 0x69, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x12, 0x2b, 0x0a, 0x06, 0x76, 0x65, 0x63, - 0x74, 0x6f, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x63, 0x68, 0x72, 0x6f, - 0x6d, 0x61, 0x2e, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x48, 0x00, 0x52, 0x06, 0x76, 0x65, 0x63, - 0x74, 0x6f, 0x72, 0x88, 0x01, 0x01, 0x42, 0x09, 0x0a, 0x07, 0x5f, 0x76, 0x65, 0x63, 0x74, 0x6f, - 0x72, 0x22, 0x49, 0x0a, 0x12, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x51, 0x75, 0x65, 0x72, 0x79, - 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x33, 0x0a, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, - 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, - 0x61, 0x2e, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x73, - 0x75, 0x6c, 0x74, 0x52, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0x44, 0x0a, 0x11, - 0x47, 0x65, 0x74, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x12, 0x10, 0x0a, 0x03, 0x69, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x03, - 0x69, 0x64, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, - 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, - 0x49, 0x64, 0x22, 0x4d, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x37, 0x0a, 0x07, 0x72, 0x65, 0x63, 0x6f, - 0x72, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x63, 0x68, 0x72, 0x6f, - 0x6d, 0x61, 0x2e, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x45, 0x6d, 0x62, 0x65, 0x64, 0x64, 0x69, - 0x6e, 0x67, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x52, 0x07, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, - 0x73, 0x22, 0xbc, 0x01, 0x0a, 0x13, 0x51, 0x75, 0x65, 0x72, 0x79, 0x56, 0x65, 0x63, 0x74, 0x6f, - 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x28, 0x0a, 0x07, 0x76, 0x65, 0x63, - 0x74, 0x6f, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x63, 0x68, 0x72, - 0x6f, 0x6d, 0x61, 0x2e, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x07, 0x76, 0x65, 0x63, 0x74, - 0x6f, 0x72, 0x73, 0x12, 0x0c, 0x0a, 0x01, 0x6b, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x01, - 0x6b, 0x12, 0x1f, 0x0a, 0x0b, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x64, 0x5f, 0x69, 0x64, 0x73, - 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0a, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x64, 0x49, - 0x64, 0x73, 0x12, 0x2d, 0x0a, 0x12, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x65, 0x6d, - 0x62, 0x65, 0x64, 0x64, 0x69, 0x6e, 0x67, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x11, - 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x45, 0x6d, 0x62, 0x65, 0x64, 0x64, 0x69, 0x6e, 0x67, - 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, - 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, - 0x22, 0x4c, 0x0a, 0x14, 0x51, 0x75, 0x65, 0x72, 0x79, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x34, 0x0a, 0x07, 0x72, 0x65, 0x73, 0x75, - 0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x63, 0x68, 0x72, 0x6f, - 0x6d, 0x61, 0x2e, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, - 0x73, 0x75, 0x6c, 0x74, 0x73, 0x52, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x2a, 0x38, - 0x0a, 0x09, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x07, 0x0a, 0x03, 0x41, - 0x44, 0x44, 0x10, 0x00, 0x12, 0x0a, 0x0a, 0x06, 0x55, 0x50, 0x44, 0x41, 0x54, 0x45, 0x10, 0x01, - 0x12, 0x0a, 0x0a, 0x06, 0x55, 0x50, 0x53, 0x45, 0x52, 0x54, 0x10, 0x02, 0x12, 0x0a, 0x0a, 0x06, - 0x44, 0x45, 0x4c, 0x45, 0x54, 0x45, 0x10, 0x03, 0x2a, 0x28, 0x0a, 0x0e, 0x53, 0x63, 0x61, 0x6c, - 0x61, 0x72, 0x45, 0x6e, 0x63, 0x6f, 0x64, 0x69, 0x6e, 0x67, 0x12, 0x0b, 0x0a, 0x07, 0x46, 0x4c, - 0x4f, 0x41, 0x54, 0x33, 0x32, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x49, 0x4e, 0x54, 0x33, 0x32, - 0x10, 0x01, 0x2a, 0x28, 0x0a, 0x0c, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x63, 0x6f, - 0x70, 0x65, 0x12, 0x0a, 0x0a, 0x06, 0x56, 0x45, 0x43, 0x54, 0x4f, 0x52, 0x10, 0x00, 0x12, 0x0c, - 0x0a, 0x08, 0x4d, 0x45, 0x54, 0x41, 0x44, 0x41, 0x54, 0x41, 0x10, 0x01, 0x32, 0xa2, 0x01, 0x0a, - 0x0c, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x45, 0x0a, - 0x0a, 0x47, 0x65, 0x74, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x12, 0x19, 0x2e, 0x63, 0x68, - 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1a, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, + 0x6e, 0x74, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x52, 0x05, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x12, 0x23, + 0x0a, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x09, 0x48, 0x00, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x88, 0x01, 0x01, 0x12, 0x37, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, + 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x55, + 0x70, 0x64, 0x61, 0x74, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x48, 0x01, 0x52, + 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x88, 0x01, 0x01, 0x12, 0x3d, 0x0a, 0x0a, + 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x1e, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x2e, 0x46, 0x69, 0x6c, 0x65, 0x50, 0x61, 0x74, 0x68, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x52, 0x09, 0x66, 0x69, 0x6c, 0x65, 0x50, 0x61, 0x74, 0x68, 0x73, 0x1a, 0x4f, 0x0a, 0x0e, 0x46, + 0x69, 0x6c, 0x65, 0x50, 0x61, 0x74, 0x68, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, + 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, + 0x27, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, + 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x46, 0x69, 0x6c, 0x65, 0x50, 0x61, 0x74, 0x68, + 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x42, 0x0d, 0x0a, 0x0b, + 0x5f, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x0b, 0x0a, 0x09, 0x5f, + 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x22, 0x97, 0x02, 0x0a, 0x0a, 0x43, 0x6f, 0x6c, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x37, 0x0a, 0x08, 0x6d, + 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, + 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x4d, 0x65, 0x74, + 0x61, 0x64, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, + 0x61, 0x88, 0x01, 0x01, 0x12, 0x21, 0x0a, 0x09, 0x64, 0x69, 0x6d, 0x65, 0x6e, 0x73, 0x69, 0x6f, + 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x05, 0x48, 0x01, 0x52, 0x09, 0x64, 0x69, 0x6d, 0x65, 0x6e, + 0x73, 0x69, 0x6f, 0x6e, 0x88, 0x01, 0x01, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x65, 0x6e, 0x61, 0x6e, + 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x12, + 0x1a, 0x0a, 0x08, 0x64, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x08, 0x64, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x12, 0x20, 0x0a, 0x0b, 0x6c, + 0x6f, 0x67, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x0b, 0x6c, 0x6f, 0x67, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x18, 0x0a, + 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x09, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, + 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x0b, 0x0a, 0x09, 0x5f, 0x6d, 0x65, 0x74, 0x61, + 0x64, 0x61, 0x74, 0x61, 0x42, 0x0c, 0x0a, 0x0a, 0x5f, 0x64, 0x69, 0x6d, 0x65, 0x6e, 0x73, 0x69, + 0x6f, 0x6e, 0x22, 0x46, 0x0a, 0x08, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x12, 0x0e, + 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x12, + 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, + 0x6d, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x06, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x22, 0x1c, 0x0a, 0x06, 0x54, 0x65, + 0x6e, 0x61, 0x6e, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x85, 0x01, 0x0a, 0x13, 0x55, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x56, 0x61, 0x6c, 0x75, 0x65, + 0x12, 0x23, 0x0a, 0x0c, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x0b, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, + 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x1d, 0x0a, 0x09, 0x69, 0x6e, 0x74, 0x5f, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x48, 0x00, 0x52, 0x08, 0x69, 0x6e, 0x74, 0x56, + 0x61, 0x6c, 0x75, 0x65, 0x12, 0x21, 0x0a, 0x0b, 0x66, 0x6c, 0x6f, 0x61, 0x74, 0x5f, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x01, 0x48, 0x00, 0x52, 0x0a, 0x66, 0x6c, 0x6f, + 0x61, 0x74, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x42, 0x07, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x22, 0xac, 0x01, 0x0a, 0x0e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, + 0x61, 0x74, 0x61, 0x12, 0x40, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, + 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x55, + 0x70, 0x64, 0x61, 0x74, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4d, 0x65, + 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x08, 0x6d, 0x65, 0x74, + 0x61, 0x64, 0x61, 0x74, 0x61, 0x1a, 0x58, 0x0a, 0x0d, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, + 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x31, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, + 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x56, + 0x61, 0x6c, 0x75, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, + 0xd0, 0x01, 0x0a, 0x0f, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x63, + 0x6f, 0x72, 0x64, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x02, 0x69, 0x64, 0x12, 0x2b, 0x0a, 0x06, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x56, 0x65, 0x63, + 0x74, 0x6f, 0x72, 0x48, 0x00, 0x52, 0x06, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x88, 0x01, 0x01, + 0x12, 0x37, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x55, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x48, 0x01, 0x52, 0x08, 0x6d, 0x65, + 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x88, 0x01, 0x01, 0x12, 0x2f, 0x0a, 0x09, 0x6f, 0x70, 0x65, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x11, 0x2e, 0x63, + 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x09, 0x0a, 0x07, 0x5f, 0x76, + 0x65, 0x63, 0x74, 0x6f, 0x72, 0x42, 0x0b, 0x0a, 0x09, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, + 0x74, 0x61, 0x22, 0x66, 0x0a, 0x15, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x45, 0x6d, 0x62, 0x65, + 0x64, 0x64, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x12, 0x0e, 0x0a, 0x02, 0x69, + 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x15, 0x0a, 0x06, 0x73, + 0x65, 0x71, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x05, 0x73, 0x65, 0x71, + 0x49, 0x64, 0x12, 0x26, 0x0a, 0x06, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x56, 0x65, 0x63, 0x74, + 0x6f, 0x72, 0x52, 0x06, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x22, 0x8e, 0x01, 0x0a, 0x11, 0x56, + 0x65, 0x63, 0x74, 0x6f, 0x72, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, + 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, + 0x12, 0x15, 0x0a, 0x06, 0x73, 0x65, 0x71, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, + 0x52, 0x05, 0x73, 0x65, 0x71, 0x49, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x64, 0x69, 0x73, 0x74, 0x61, + 0x6e, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x02, 0x52, 0x08, 0x64, 0x69, 0x73, 0x74, 0x61, + 0x6e, 0x63, 0x65, 0x12, 0x2b, 0x0a, 0x06, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x56, 0x65, 0x63, + 0x74, 0x6f, 0x72, 0x48, 0x00, 0x52, 0x06, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x88, 0x01, 0x01, + 0x42, 0x09, 0x0a, 0x07, 0x5f, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x22, 0x49, 0x0a, 0x12, 0x56, + 0x65, 0x63, 0x74, 0x6f, 0x72, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, + 0x73, 0x12, 0x33, 0x0a, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x56, 0x65, 0x63, 0x74, + 0x6f, 0x72, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, 0x07, 0x72, + 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0x44, 0x0a, 0x11, 0x47, 0x65, 0x74, 0x56, 0x65, 0x63, + 0x74, 0x6f, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x69, + 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x03, 0x69, 0x64, 0x73, 0x12, 0x1d, 0x0a, + 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x22, 0x4d, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x22, 0x00, 0x12, 0x4b, 0x0a, 0x0c, 0x51, 0x75, 0x65, 0x72, 0x79, 0x56, 0x65, 0x63, - 0x74, 0x6f, 0x72, 0x73, 0x12, 0x1b, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x51, 0x75, - 0x65, 0x72, 0x79, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x1c, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, - 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, - 0x00, 0x42, 0x3a, 0x5a, 0x38, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, - 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2d, 0x63, 0x6f, 0x72, 0x65, 0x2f, 0x63, 0x68, 0x72, 0x6f, - 0x6d, 0x61, 0x2f, 0x67, 0x6f, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, - 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x70, 0x62, 0x62, 0x06, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x73, 0x65, 0x12, 0x37, 0x0a, 0x07, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x73, 0x18, 0x01, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x56, 0x65, 0x63, + 0x74, 0x6f, 0x72, 0x45, 0x6d, 0x62, 0x65, 0x64, 0x64, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x63, 0x6f, + 0x72, 0x64, 0x52, 0x07, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x73, 0x22, 0xbc, 0x01, 0x0a, 0x13, + 0x51, 0x75, 0x65, 0x72, 0x79, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x28, 0x0a, 0x07, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x18, 0x01, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x56, 0x65, + 0x63, 0x74, 0x6f, 0x72, 0x52, 0x07, 0x76, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x12, 0x0c, 0x0a, + 0x01, 0x6b, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x01, 0x6b, 0x12, 0x1f, 0x0a, 0x0b, 0x61, + 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x64, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, + 0x52, 0x0a, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x64, 0x49, 0x64, 0x73, 0x12, 0x2d, 0x0a, 0x12, + 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x65, 0x6d, 0x62, 0x65, 0x64, 0x64, 0x69, 0x6e, + 0x67, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x11, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, + 0x65, 0x45, 0x6d, 0x62, 0x65, 0x64, 0x64, 0x69, 0x6e, 0x67, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x73, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x22, 0x4c, 0x0a, 0x14, 0x51, 0x75, + 0x65, 0x72, 0x79, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x34, 0x0a, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x01, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x56, 0x65, 0x63, + 0x74, 0x6f, 0x72, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x52, + 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x2a, 0x38, 0x0a, 0x09, 0x4f, 0x70, 0x65, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x07, 0x0a, 0x03, 0x41, 0x44, 0x44, 0x10, 0x00, 0x12, 0x0a, + 0x0a, 0x06, 0x55, 0x50, 0x44, 0x41, 0x54, 0x45, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x55, 0x50, + 0x53, 0x45, 0x52, 0x54, 0x10, 0x02, 0x12, 0x0a, 0x0a, 0x06, 0x44, 0x45, 0x4c, 0x45, 0x54, 0x45, + 0x10, 0x03, 0x2a, 0x28, 0x0a, 0x0e, 0x53, 0x63, 0x61, 0x6c, 0x61, 0x72, 0x45, 0x6e, 0x63, 0x6f, + 0x64, 0x69, 0x6e, 0x67, 0x12, 0x0b, 0x0a, 0x07, 0x46, 0x4c, 0x4f, 0x41, 0x54, 0x33, 0x32, 0x10, + 0x00, 0x12, 0x09, 0x0a, 0x05, 0x49, 0x4e, 0x54, 0x33, 0x32, 0x10, 0x01, 0x2a, 0x28, 0x0a, 0x0c, + 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x12, 0x0a, 0x0a, 0x06, + 0x56, 0x45, 0x43, 0x54, 0x4f, 0x52, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x4d, 0x45, 0x54, 0x41, + 0x44, 0x41, 0x54, 0x41, 0x10, 0x01, 0x32, 0xa2, 0x01, 0x0a, 0x0c, 0x56, 0x65, 0x63, 0x74, 0x6f, + 0x72, 0x52, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x45, 0x0a, 0x0a, 0x47, 0x65, 0x74, 0x56, 0x65, + 0x63, 0x74, 0x6f, 0x72, 0x73, 0x12, 0x19, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x47, + 0x65, 0x74, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x1a, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x56, 0x65, 0x63, + 0x74, 0x6f, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4b, + 0x0a, 0x0c, 0x51, 0x75, 0x65, 0x72, 0x79, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x12, 0x1b, + 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x56, 0x65, 0x63, + 0x74, 0x6f, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x63, 0x68, + 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, + 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x3a, 0x5a, 0x38, 0x67, + 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, + 0x2d, 0x63, 0x6f, 0x72, 0x65, 0x2f, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2f, 0x67, 0x6f, 0x2f, + 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x69, + 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -1480,7 +1449,7 @@ var file_chromadb_proto_chroma_proto_goTypes = []interface{}{ (*Tenant)(nil), // 9: chroma.Tenant (*UpdateMetadataValue)(nil), // 10: chroma.UpdateMetadataValue (*UpdateMetadata)(nil), // 11: chroma.UpdateMetadata - (*SubmitEmbeddingRecord)(nil), // 12: chroma.SubmitEmbeddingRecord + (*OperationRecord)(nil), // 12: chroma.OperationRecord (*VectorEmbeddingRecord)(nil), // 13: chroma.VectorEmbeddingRecord (*VectorQueryResult)(nil), // 14: chroma.VectorQueryResult (*VectorQueryResults)(nil), // 15: chroma.VectorQueryResults @@ -1498,9 +1467,9 @@ var file_chromadb_proto_chroma_proto_depIdxs = []int32{ 20, // 3: chroma.Segment.file_paths:type_name -> chroma.Segment.FilePathsEntry 11, // 4: chroma.Collection.metadata:type_name -> chroma.UpdateMetadata 21, // 5: chroma.UpdateMetadata.metadata:type_name -> chroma.UpdateMetadata.MetadataEntry - 4, // 6: chroma.SubmitEmbeddingRecord.vector:type_name -> chroma.Vector - 11, // 7: chroma.SubmitEmbeddingRecord.metadata:type_name -> chroma.UpdateMetadata - 0, // 8: chroma.SubmitEmbeddingRecord.operation:type_name -> chroma.Operation + 4, // 6: chroma.OperationRecord.vector:type_name -> chroma.Vector + 11, // 7: chroma.OperationRecord.metadata:type_name -> chroma.UpdateMetadata + 0, // 8: chroma.OperationRecord.operation:type_name -> chroma.Operation 4, // 9: chroma.VectorEmbeddingRecord.vector:type_name -> chroma.Vector 4, // 10: chroma.VectorQueryResult.vector:type_name -> chroma.Vector 14, // 11: chroma.VectorQueryResults.results:type_name -> chroma.VectorQueryResult @@ -1635,7 +1604,7 @@ func file_chromadb_proto_chroma_proto_init() { } } file_chromadb_proto_chroma_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SubmitEmbeddingRecord); i { + switch v := v.(*OperationRecord); i { case 0: return &v.state case 1: diff --git a/go/pkg/proto/coordinatorpb/coordinator.pb.go b/go/pkg/proto/coordinatorpb/coordinator.pb.go index 6995099b9d19..2b75f87cfa8d 100644 --- a/go/pkg/proto/coordinatorpb/coordinator.pb.go +++ b/go/pkg/proto/coordinatorpb/coordinator.pb.go @@ -1,6 +1,6 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.31.0 +// protoc-gen-go v1.33.0 // protoc v4.23.4 // source: chromadb/proto/coordinator.proto @@ -633,7 +633,6 @@ type GetSegmentsRequest struct { Id *string `protobuf:"bytes,1,opt,name=id,proto3,oneof" json:"id,omitempty"` Type *string `protobuf:"bytes,2,opt,name=type,proto3,oneof" json:"type,omitempty"` Scope *SegmentScope `protobuf:"varint,3,opt,name=scope,proto3,enum=chroma.SegmentScope,oneof" json:"scope,omitempty"` - Topic *string `protobuf:"bytes,4,opt,name=topic,proto3,oneof" json:"topic,omitempty"` Collection *string `protobuf:"bytes,5,opt,name=collection,proto3,oneof" json:"collection,omitempty"` // Collection ID } @@ -690,13 +689,6 @@ func (x *GetSegmentsRequest) GetScope() SegmentScope { return SegmentScope_VECTOR } -func (x *GetSegmentsRequest) GetTopic() string { - if x != nil && x.Topic != nil { - return *x.Topic - } - return "" -} - func (x *GetSegmentsRequest) GetCollection() string { if x != nil && x.Collection != nil { return *x.Collection @@ -765,11 +757,6 @@ type UpdateSegmentRequest struct { unknownFields protoimpl.UnknownFields Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` - // Types that are assignable to TopicUpdate: - // - // *UpdateSegmentRequest_Topic - // *UpdateSegmentRequest_ResetTopic - TopicUpdate isUpdateSegmentRequest_TopicUpdate `protobuf_oneof:"topic_update"` // Types that are assignable to CollectionUpdate: // // *UpdateSegmentRequest_Collection @@ -821,27 +808,6 @@ func (x *UpdateSegmentRequest) GetId() string { return "" } -func (m *UpdateSegmentRequest) GetTopicUpdate() isUpdateSegmentRequest_TopicUpdate { - if m != nil { - return m.TopicUpdate - } - return nil -} - -func (x *UpdateSegmentRequest) GetTopic() string { - if x, ok := x.GetTopicUpdate().(*UpdateSegmentRequest_Topic); ok { - return x.Topic - } - return "" -} - -func (x *UpdateSegmentRequest) GetResetTopic() bool { - if x, ok := x.GetTopicUpdate().(*UpdateSegmentRequest_ResetTopic); ok { - return x.ResetTopic - } - return false -} - func (m *UpdateSegmentRequest) GetCollectionUpdate() isUpdateSegmentRequest_CollectionUpdate { if m != nil { return m.CollectionUpdate @@ -884,22 +850,6 @@ func (x *UpdateSegmentRequest) GetResetMetadata() bool { return false } -type isUpdateSegmentRequest_TopicUpdate interface { - isUpdateSegmentRequest_TopicUpdate() -} - -type UpdateSegmentRequest_Topic struct { - Topic string `protobuf:"bytes,2,opt,name=topic,proto3,oneof"` -} - -type UpdateSegmentRequest_ResetTopic struct { - ResetTopic bool `protobuf:"varint,3,opt,name=reset_topic,json=resetTopic,proto3,oneof"` -} - -func (*UpdateSegmentRequest_Topic) isUpdateSegmentRequest_TopicUpdate() {} - -func (*UpdateSegmentRequest_ResetTopic) isUpdateSegmentRequest_TopicUpdate() {} - type isUpdateSegmentRequest_CollectionUpdate interface { isUpdateSegmentRequest_CollectionUpdate() } @@ -1254,7 +1204,6 @@ type GetCollectionsRequest struct { Id *string `protobuf:"bytes,1,opt,name=id,proto3,oneof" json:"id,omitempty"` Name *string `protobuf:"bytes,2,opt,name=name,proto3,oneof" json:"name,omitempty"` - Topic *string `protobuf:"bytes,3,opt,name=topic,proto3,oneof" json:"topic,omitempty"` Tenant string `protobuf:"bytes,4,opt,name=tenant,proto3" json:"tenant,omitempty"` Database string `protobuf:"bytes,5,opt,name=database,proto3" json:"database,omitempty"` } @@ -1305,13 +1254,6 @@ func (x *GetCollectionsRequest) GetName() string { return "" } -func (x *GetCollectionsRequest) GetTopic() string { - if x != nil && x.Topic != nil { - return *x.Topic - } - return "" -} - func (x *GetCollectionsRequest) GetTenant() string { if x != nil { return x.Tenant @@ -1387,7 +1329,6 @@ type UpdateCollectionRequest struct { unknownFields protoimpl.UnknownFields Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` - Topic *string `protobuf:"bytes,2,opt,name=topic,proto3,oneof" json:"topic,omitempty"` Name *string `protobuf:"bytes,3,opt,name=name,proto3,oneof" json:"name,omitempty"` Dimension *int32 `protobuf:"varint,4,opt,name=dimension,proto3,oneof" json:"dimension,omitempty"` // Types that are assignable to MetadataUpdate: @@ -1436,13 +1377,6 @@ func (x *UpdateCollectionRequest) GetId() string { return "" } -func (x *UpdateCollectionRequest) GetTopic() string { - if x != nil && x.Topic != nil { - return *x.Topic - } - return "" -} - func (x *UpdateCollectionRequest) GetName() string { if x != nil && x.Name != nil { return *x.Name @@ -2112,46 +2046,39 @@ var file_chromadb_proto_coordinator_proto_rawDesc = []byte{ 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x26, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, - 0xe6, 0x01, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x52, + 0xc1, 0x01, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x13, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x02, 0x69, 0x64, 0x88, 0x01, 0x01, 0x12, 0x17, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x88, 0x01, 0x01, 0x12, 0x2f, 0x0a, 0x05, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x14, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x48, 0x02, 0x52, 0x05, 0x73, 0x63, 0x6f, - 0x70, 0x65, 0x88, 0x01, 0x01, 0x12, 0x19, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x04, - 0x20, 0x01, 0x28, 0x09, 0x48, 0x03, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x88, 0x01, 0x01, - 0x12, 0x23, 0x0a, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, - 0x20, 0x01, 0x28, 0x09, 0x48, 0x04, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, - 0x6f, 0x6e, 0x88, 0x01, 0x01, 0x42, 0x05, 0x0a, 0x03, 0x5f, 0x69, 0x64, 0x42, 0x07, 0x0a, 0x05, - 0x5f, 0x74, 0x79, 0x70, 0x65, 0x42, 0x08, 0x0a, 0x06, 0x5f, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x42, - 0x08, 0x0a, 0x06, 0x5f, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x42, 0x0d, 0x0a, 0x0b, 0x5f, 0x63, 0x6f, - 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x6a, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x53, - 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, - 0x2b, 0x0a, 0x08, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, - 0x0b, 0x32, 0x0f, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, - 0x6e, 0x74, 0x52, 0x08, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x26, 0x0a, 0x06, - 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x63, - 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, - 0x61, 0x74, 0x75, 0x73, 0x22, 0xc7, 0x02, 0x0a, 0x14, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x53, - 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, - 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x16, 0x0a, - 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x05, - 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x21, 0x0a, 0x0b, 0x72, 0x65, 0x73, 0x65, 0x74, 0x5f, 0x74, - 0x6f, 0x70, 0x69, 0x63, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x48, 0x00, 0x52, 0x0a, 0x72, 0x65, - 0x73, 0x65, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x20, 0x0a, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, - 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x0a, + 0x70, 0x65, 0x88, 0x01, 0x01, 0x12, 0x23, 0x0a, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x48, 0x03, 0x52, 0x0a, 0x63, 0x6f, 0x6c, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x88, 0x01, 0x01, 0x42, 0x05, 0x0a, 0x03, 0x5f, 0x69, + 0x64, 0x42, 0x07, 0x0a, 0x05, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x42, 0x08, 0x0a, 0x06, 0x5f, 0x73, + 0x63, 0x6f, 0x70, 0x65, 0x42, 0x0d, 0x0a, 0x0b, 0x5f, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x22, 0x6a, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2b, 0x0a, 0x08, 0x73, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x63, + 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x08, 0x73, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x26, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, + 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, + 0xfc, 0x01, 0x0a, 0x14, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x20, 0x0a, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2b, 0x0a, 0x10, 0x72, 0x65, 0x73, 0x65, 0x74, 0x5f, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, - 0x20, 0x01, 0x28, 0x08, 0x48, 0x01, 0x52, 0x0f, 0x72, 0x65, 0x73, 0x65, 0x74, 0x43, 0x6f, 0x6c, + 0x20, 0x01, 0x28, 0x08, 0x48, 0x00, 0x52, 0x0f, 0x72, 0x65, 0x73, 0x65, 0x74, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x34, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, - 0x61, 0x48, 0x02, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x27, 0x0a, + 0x61, 0x48, 0x01, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x27, 0x0a, 0x0e, 0x72, 0x65, 0x73, 0x65, 0x74, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, - 0x07, 0x20, 0x01, 0x28, 0x08, 0x48, 0x02, 0x52, 0x0d, 0x72, 0x65, 0x73, 0x65, 0x74, 0x4d, 0x65, - 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x42, 0x0e, 0x0a, 0x0c, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x5f, - 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x13, 0x0a, 0x11, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x07, 0x20, 0x01, 0x28, 0x08, 0x48, 0x01, 0x52, 0x0d, 0x72, 0x65, 0x73, 0x65, 0x74, 0x4d, 0x65, + 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x42, 0x13, 0x0a, 0x11, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x11, 0x0a, 0x0f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x22, 0x3f, 0x0a, 0x15, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, @@ -2195,33 +2122,29 @@ var file_chromadb_proto_coordinator_proto_rawDesc = []byte{ 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x26, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x53, 0x74, 0x61, - 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0xae, 0x01, 0x0a, 0x15, + 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x89, 0x01, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x13, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x02, 0x69, 0x64, 0x88, 0x01, 0x01, 0x12, 0x17, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, - 0x88, 0x01, 0x01, 0x12, 0x19, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x09, 0x48, 0x02, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x88, 0x01, 0x01, 0x12, 0x16, - 0x0a, 0x06, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, - 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x64, 0x61, 0x74, 0x61, 0x62, 0x61, - 0x73, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x64, 0x61, 0x74, 0x61, 0x62, 0x61, - 0x73, 0x65, 0x42, 0x05, 0x0a, 0x03, 0x5f, 0x69, 0x64, 0x42, 0x07, 0x0a, 0x05, 0x5f, 0x6e, 0x61, - 0x6d, 0x65, 0x42, 0x08, 0x0a, 0x06, 0x5f, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x22, 0x76, 0x0a, 0x16, - 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x34, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x63, 0x68, - 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, - 0x0b, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x26, 0x0a, 0x06, - 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x63, - 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, - 0x61, 0x74, 0x75, 0x73, 0x22, 0x93, 0x02, 0x0a, 0x17, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, - 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, - 0x12, 0x19, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x48, - 0x01, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x88, 0x01, 0x01, 0x12, 0x17, 0x0a, 0x04, 0x6e, - 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x48, 0x02, 0x52, 0x04, 0x6e, 0x61, 0x6d, + 0x88, 0x01, 0x01, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x64, + 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x64, + 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x42, 0x05, 0x0a, 0x03, 0x5f, 0x69, 0x64, 0x42, 0x07, + 0x0a, 0x05, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x76, 0x0a, 0x16, 0x47, 0x65, 0x74, 0x43, 0x6f, + 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x12, 0x34, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, + 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0b, 0x63, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x26, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, + 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, + 0xee, 0x01, 0x0a, 0x17, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, + 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x17, 0x0a, 0x04, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x88, 0x01, 0x01, 0x12, 0x21, 0x0a, 0x09, 0x64, 0x69, 0x6d, 0x65, 0x6e, 0x73, 0x69, 0x6f, - 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x48, 0x03, 0x52, 0x09, 0x64, 0x69, 0x6d, 0x65, 0x6e, + 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x48, 0x02, 0x52, 0x09, 0x64, 0x69, 0x6d, 0x65, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x88, 0x01, 0x01, 0x12, 0x34, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, @@ -2229,187 +2152,186 @@ var file_chromadb_proto_coordinator_proto_rawDesc = []byte{ 0x0e, 0x72, 0x65, 0x73, 0x65, 0x74, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x48, 0x00, 0x52, 0x0d, 0x72, 0x65, 0x73, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x42, 0x11, 0x0a, 0x0f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, - 0x74, 0x61, 0x5f, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x08, 0x0a, 0x06, 0x5f, 0x74, 0x6f, - 0x70, 0x69, 0x63, 0x42, 0x07, 0x0a, 0x05, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x42, 0x0c, 0x0a, 0x0a, - 0x5f, 0x64, 0x69, 0x6d, 0x65, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x42, 0x0a, 0x18, 0x55, 0x70, - 0x64, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x26, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, - 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x6f, - 0x0a, 0x0c, 0x4e, 0x6f, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x0e, - 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x02, 0x69, 0x64, 0x12, 0x23, + 0x74, 0x61, 0x5f, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x07, 0x0a, 0x05, 0x5f, 0x6e, 0x61, + 0x6d, 0x65, 0x42, 0x0c, 0x0a, 0x0a, 0x5f, 0x64, 0x69, 0x6d, 0x65, 0x6e, 0x73, 0x69, 0x6f, 0x6e, + 0x22, 0x42, 0x0a, 0x18, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x26, 0x0a, 0x06, + 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x63, + 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x22, 0x6f, 0x0a, 0x0c, 0x4e, 0x6f, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x02, 0x69, 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6c, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x79, 0x70, + 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x16, 0x0a, + 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x73, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x3c, 0x0a, 0x12, 0x52, 0x65, 0x73, 0x65, 0x74, 0x53, 0x74, + 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x26, 0x0a, 0x06, 0x73, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x63, 0x68, + 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x22, 0x44, 0x0a, 0x25, 0x47, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x43, 0x6f, + 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x46, 0x6f, 0x72, 0x54, + 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1b, 0x0a, 0x09, + 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, + 0x08, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x49, 0x64, 0x22, 0x69, 0x0a, 0x18, 0x54, 0x65, 0x6e, + 0x61, 0x6e, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x5f, + 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, + 0x49, 0x64, 0x12, 0x30, 0x0a, 0x14, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x61, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x12, 0x6c, 0x61, 0x73, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x54, 0x69, 0x6d, 0x65, 0x22, 0x89, 0x01, 0x0a, 0x26, 0x47, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, + 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x46, 0x6f, + 0x72, 0x54, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, + 0x5f, 0x0a, 0x1b, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x5f, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x63, + 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x01, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x54, 0x65, + 0x6e, 0x61, 0x6e, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x18, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x4c, 0x61, + 0x73, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, + 0x22, 0x88, 0x01, 0x0a, 0x25, 0x53, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x43, 0x6f, 0x6d, 0x70, + 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x46, 0x6f, 0x72, 0x54, 0x65, 0x6e, + 0x61, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x5f, 0x0a, 0x1b, 0x74, 0x65, + 0x6e, 0x61, 0x6e, 0x74, 0x5f, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x20, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x54, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x4c, + 0x61, 0x73, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, + 0x65, 0x52, 0x18, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x43, 0x6f, 0x6d, + 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x22, 0xde, 0x01, 0x0a, 0x1a, + 0x46, 0x6c, 0x75, 0x73, 0x68, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6d, 0x70, + 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, + 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x50, 0x0a, 0x0a, 0x66, 0x69, 0x6c, + 0x65, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x31, 0x2e, + 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x53, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, + 0x6f, 0x2e, 0x46, 0x69, 0x6c, 0x65, 0x50, 0x61, 0x74, 0x68, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x52, 0x09, 0x66, 0x69, 0x6c, 0x65, 0x50, 0x61, 0x74, 0x68, 0x73, 0x1a, 0x4f, 0x0a, 0x0e, 0x46, + 0x69, 0x6c, 0x65, 0x50, 0x61, 0x74, 0x68, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, + 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, + 0x27, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, + 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x46, 0x69, 0x6c, 0x65, 0x50, 0x61, 0x74, 0x68, + 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x92, 0x02, 0x0a, + 0x20, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x49, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, - 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, - 0x3c, 0x0a, 0x12, 0x52, 0x65, 0x73, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x26, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x53, - 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x44, 0x0a, - 0x25, 0x47, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, - 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x46, 0x6f, 0x72, 0x54, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, - 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x08, 0x74, 0x65, 0x6e, 0x61, 0x6e, - 0x74, 0x49, 0x64, 0x22, 0x69, 0x0a, 0x18, 0x54, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x4c, 0x61, 0x73, - 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x12, - 0x1b, 0x0a, 0x09, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x08, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x30, 0x0a, 0x14, - 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, - 0x74, 0x69, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x12, 0x6c, 0x61, 0x73, 0x74, - 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x22, 0x89, - 0x01, 0x0a, 0x26, 0x47, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x46, 0x6f, 0x72, 0x54, 0x65, 0x6e, 0x61, 0x6e, - 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5f, 0x0a, 0x1b, 0x74, 0x65, 0x6e, - 0x61, 0x6e, 0x74, 0x5f, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x20, - 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x54, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x4c, 0x61, - 0x73, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, - 0x52, 0x18, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x43, 0x6f, 0x6d, 0x70, - 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x22, 0x88, 0x01, 0x0a, 0x25, 0x53, - 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x54, 0x69, 0x6d, 0x65, 0x46, 0x6f, 0x72, 0x54, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x12, 0x5f, 0x0a, 0x1b, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x5f, 0x6c, - 0x61, 0x73, 0x74, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, - 0x69, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x63, 0x68, 0x72, 0x6f, - 0x6d, 0x61, 0x2e, 0x54, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x43, 0x6f, 0x6d, - 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x18, 0x74, 0x65, 0x6e, - 0x61, 0x6e, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x22, 0xde, 0x01, 0x0a, 0x1a, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x53, - 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, - 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, - 0x74, 0x49, 0x64, 0x12, 0x50, 0x0a, 0x0a, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x70, 0x61, 0x74, 0x68, - 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, - 0x2e, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6d, - 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x46, 0x69, 0x6c, 0x65, - 0x50, 0x61, 0x74, 0x68, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x09, 0x66, 0x69, 0x6c, 0x65, - 0x50, 0x61, 0x74, 0x68, 0x73, 0x1a, 0x4f, 0x0a, 0x0e, 0x46, 0x69, 0x6c, 0x65, 0x50, 0x61, 0x74, - 0x68, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x27, 0x0a, 0x05, 0x76, 0x61, 0x6c, - 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, - 0x61, 0x2e, 0x46, 0x69, 0x6c, 0x65, 0x50, 0x61, 0x74, 0x68, 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, - 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x92, 0x02, 0x0a, 0x20, 0x46, 0x6c, 0x75, 0x73, 0x68, - 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x74, - 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, - 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, - 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x21, 0x0a, - 0x0c, 0x6c, 0x6f, 0x67, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x03, 0x52, 0x0b, 0x6c, 0x6f, 0x67, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x12, 0x2d, 0x0a, 0x12, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x76, - 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x11, 0x63, 0x6f, - 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, - 0x5a, 0x0a, 0x17, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x61, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x22, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x53, - 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x15, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6d, - 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x22, 0xa9, 0x01, 0x0a, 0x21, - 0x46, 0x6c, 0x75, 0x73, 0x68, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, - 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, - 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x2d, 0x0a, 0x12, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, + 0x6e, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x6c, 0x6f, 0x67, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x6c, 0x6f, 0x67, 0x50, 0x6f, + 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2d, 0x0a, 0x12, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x11, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x56, 0x65, - 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x30, 0x0a, 0x14, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x63, 0x6f, - 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x03, 0x52, 0x12, 0x6c, 0x61, 0x73, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x32, 0xf4, 0x0a, 0x0a, 0x05, 0x53, 0x79, 0x73, 0x44, - 0x42, 0x12, 0x51, 0x0a, 0x0e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x44, 0x61, 0x74, 0x61, 0x62, - 0x61, 0x73, 0x65, 0x12, 0x1d, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x43, 0x72, 0x65, - 0x61, 0x74, 0x65, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x43, 0x72, 0x65, 0x61, - 0x74, 0x65, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x22, 0x00, 0x12, 0x48, 0x0a, 0x0b, 0x47, 0x65, 0x74, 0x44, 0x61, 0x74, 0x61, 0x62, - 0x61, 0x73, 0x65, 0x12, 0x1a, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x47, 0x65, 0x74, - 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x1b, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x44, 0x61, 0x74, 0x61, - 0x62, 0x61, 0x73, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4b, - 0x0a, 0x0c, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x12, 0x1b, - 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x65, - 0x6e, 0x61, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x63, 0x68, - 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x65, 0x6e, 0x61, 0x6e, - 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x42, 0x0a, 0x09, 0x47, - 0x65, 0x74, 0x54, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x12, 0x18, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, - 0x61, 0x2e, 0x47, 0x65, 0x74, 0x54, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x19, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x54, - 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, - 0x4e, 0x0a, 0x0d, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, - 0x12, 0x1c, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, - 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, - 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, - 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, - 0x4e, 0x0a, 0x0d, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, - 0x12, 0x1c, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, + 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x5a, 0x0a, 0x17, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x6e, 0x66, 0x6f, + 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, + 0x46, 0x6c, 0x75, 0x73, 0x68, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6d, 0x70, + 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x15, 0x73, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, + 0x6f, 0x22, 0xa9, 0x01, 0x0a, 0x21, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x43, 0x6f, 0x6c, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, + 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x2d, 0x0a, 0x12, + 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, + 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x11, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x30, 0x0a, 0x14, 0x6c, + 0x61, 0x73, 0x74, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, + 0x69, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x12, 0x6c, 0x61, 0x73, 0x74, 0x43, + 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x32, 0xf4, 0x0a, + 0x0a, 0x05, 0x53, 0x79, 0x73, 0x44, 0x42, 0x12, 0x51, 0x0a, 0x0e, 0x43, 0x72, 0x65, 0x61, 0x74, + 0x65, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x12, 0x1d, 0x2e, 0x63, 0x68, 0x72, 0x6f, + 0x6d, 0x61, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, + 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, + 0x61, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x48, 0x0a, 0x0b, 0x47, 0x65, + 0x74, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x12, 0x1a, 0x2e, 0x63, 0x68, 0x72, 0x6f, + 0x6d, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x47, + 0x65, 0x74, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x00, 0x12, 0x4b, 0x0a, 0x0c, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x65, + 0x6e, 0x61, 0x6e, 0x74, 0x12, 0x1b, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x43, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x54, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x1c, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, + 0x65, 0x54, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x00, 0x12, 0x42, 0x0a, 0x09, 0x47, 0x65, 0x74, 0x54, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x12, 0x18, + 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x54, 0x65, 0x6e, 0x61, 0x6e, + 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x19, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, + 0x61, 0x2e, 0x47, 0x65, 0x74, 0x54, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4e, 0x0a, 0x0d, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x1c, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, + 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x43, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4e, 0x0a, 0x0d, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x53, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x1c, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, + 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x44, 0x65, + 0x6c, 0x65, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x48, 0x0a, 0x0b, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x73, 0x12, 0x1a, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x47, 0x65, + 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x1b, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, + 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, + 0x4e, 0x0a, 0x0d, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x12, 0x1c, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, - 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x53, 0x65, + 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, - 0x48, 0x0a, 0x0b, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x1a, - 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, - 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x63, 0x68, 0x72, - 0x6f, 0x6d, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4e, 0x0a, 0x0d, 0x55, 0x70, 0x64, - 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x1c, 0x2e, 0x63, 0x68, 0x72, - 0x6f, 0x6d, 0x61, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, - 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, - 0x61, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x57, 0x0a, 0x10, 0x43, 0x72, 0x65, - 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1f, 0x2e, - 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6c, - 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, - 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, - 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x00, 0x12, 0x57, 0x0a, 0x10, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, - 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1f, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, - 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, - 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x51, 0x0a, 0x0e, 0x47, - 0x65, 0x74, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x1d, 0x2e, - 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x63, - 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x57, - 0x0a, 0x10, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, - 0x6f, 0x6e, 0x12, 0x1f, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x55, 0x70, 0x64, 0x61, - 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x55, 0x70, 0x64, - 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x42, 0x0a, 0x0a, 0x52, 0x65, 0x73, 0x65, 0x74, - 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x1a, 0x1a, 0x2e, - 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x52, 0x65, 0x73, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, - 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x81, 0x01, 0x0a, 0x1e, - 0x47, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x46, 0x6f, 0x72, 0x54, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x12, 0x2d, - 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x43, - 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x46, 0x6f, 0x72, - 0x54, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2e, 0x2e, - 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x43, 0x6f, - 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x46, 0x6f, 0x72, 0x54, - 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, - 0x69, 0x0a, 0x1e, 0x53, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x46, 0x6f, 0x72, 0x54, 0x65, 0x6e, 0x61, 0x6e, - 0x74, 0x12, 0x2d, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x53, 0x65, 0x74, 0x4c, 0x61, - 0x73, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, - 0x46, 0x6f, 0x72, 0x54, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, - 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x00, 0x12, 0x72, 0x0a, 0x19, 0x46, 0x6c, - 0x75, 0x73, 0x68, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, - 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x28, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, + 0x57, 0x0a, 0x10, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x12, 0x1f, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x43, 0x72, 0x65, + 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x43, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x57, 0x0a, 0x10, 0x44, 0x65, 0x6c, 0x65, + 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1f, 0x2e, 0x63, + 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, + 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6c, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x00, 0x12, 0x51, 0x0a, 0x0e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x73, 0x12, 0x1d, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x47, 0x65, 0x74, + 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x43, + 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x00, 0x12, 0x57, 0x0a, 0x10, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x6f, + 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1f, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, + 0x61, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x63, 0x68, 0x72, 0x6f, + 0x6d, 0x61, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x42, 0x0a, + 0x0a, 0x52, 0x65, 0x73, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x16, 0x2e, 0x67, 0x6f, + 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, + 0x70, 0x74, 0x79, 0x1a, 0x1a, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x52, 0x65, 0x73, + 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x00, 0x12, 0x81, 0x01, 0x0a, 0x1e, 0x47, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x43, 0x6f, 0x6d, + 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x46, 0x6f, 0x72, 0x54, 0x65, + 0x6e, 0x61, 0x6e, 0x74, 0x12, 0x2d, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x47, 0x65, + 0x74, 0x4c, 0x61, 0x73, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, + 0x69, 0x6d, 0x65, 0x46, 0x6f, 0x72, 0x54, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x2e, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x47, 0x65, 0x74, + 0x4c, 0x61, 0x73, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, + 0x6d, 0x65, 0x46, 0x6f, 0x72, 0x54, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x69, 0x0a, 0x1e, 0x53, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, + 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x46, 0x6f, + 0x72, 0x54, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x12, 0x2d, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, + 0x2e, 0x53, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x46, 0x6f, 0x72, 0x54, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x00, + 0x12, 0x72, 0x0a, 0x19, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x28, 0x2e, + 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x43, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x29, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x46, 0x6c, 0x75, 0x73, 0x68, - 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x3a, - 0x5a, 0x38, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x63, 0x68, 0x72, - 0x6f, 0x6d, 0x61, 0x2d, 0x63, 0x6f, 0x72, 0x65, 0x2f, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2f, - 0x67, 0x6f, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x63, 0x6f, 0x6f, - 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x33, + 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x00, 0x42, 0x3a, 0x5a, 0x38, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, + 0x6f, 0x6d, 0x2f, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2d, 0x63, 0x6f, 0x72, 0x65, 0x2f, 0x63, + 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2f, 0x67, 0x6f, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2f, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x70, 0x62, + 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -2944,8 +2866,6 @@ func file_chromadb_proto_coordinator_proto_init() { } file_chromadb_proto_coordinator_proto_msgTypes[12].OneofWrappers = []interface{}{} file_chromadb_proto_coordinator_proto_msgTypes[14].OneofWrappers = []interface{}{ - (*UpdateSegmentRequest_Topic)(nil), - (*UpdateSegmentRequest_ResetTopic)(nil), (*UpdateSegmentRequest_Collection)(nil), (*UpdateSegmentRequest_ResetCollection)(nil), (*UpdateSegmentRequest_Metadata)(nil), diff --git a/go/pkg/proto/logservicepb/logservice.pb.go b/go/pkg/proto/logservicepb/logservice.pb.go index 2122524d933c..a69a399975ca 100644 --- a/go/pkg/proto/logservicepb/logservice.pb.go +++ b/go/pkg/proto/logservicepb/logservice.pb.go @@ -1,6 +1,6 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.31.0 +// protoc-gen-go v1.33.0 // protoc v4.23.4 // source: chromadb/proto/logservice.proto @@ -26,8 +26,8 @@ type PushLogsRequest struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - CollectionId string `protobuf:"bytes,1,opt,name=collection_id,json=collectionId,proto3" json:"collection_id,omitempty"` - Records []*coordinatorpb.SubmitEmbeddingRecord `protobuf:"bytes,2,rep,name=records,proto3" json:"records,omitempty"` + CollectionId string `protobuf:"bytes,1,opt,name=collection_id,json=collectionId,proto3" json:"collection_id,omitempty"` + Records []*coordinatorpb.OperationRecord `protobuf:"bytes,2,rep,name=records,proto3" json:"records,omitempty"` } func (x *PushLogsRequest) Reset() { @@ -69,7 +69,7 @@ func (x *PushLogsRequest) GetCollectionId() string { return "" } -func (x *PushLogsRequest) GetRecords() []*coordinatorpb.SubmitEmbeddingRecord { +func (x *PushLogsRequest) GetRecords() []*coordinatorpb.OperationRecord { if x != nil { return x.Records } @@ -199,8 +199,8 @@ type RecordLog struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - LogId int64 `protobuf:"varint,1,opt,name=log_id,json=logId,proto3" json:"log_id,omitempty"` - Record *coordinatorpb.SubmitEmbeddingRecord `protobuf:"bytes,2,opt,name=record,proto3" json:"record,omitempty"` + LogId int64 `protobuf:"varint,1,opt,name=log_id,json=logId,proto3" json:"log_id,omitempty"` + Record *coordinatorpb.OperationRecord `protobuf:"bytes,2,opt,name=record,proto3" json:"record,omitempty"` } func (x *RecordLog) Reset() { @@ -242,7 +242,7 @@ func (x *RecordLog) GetLogId() int64 { return 0 } -func (x *RecordLog) GetRecord() *coordinatorpb.SubmitEmbeddingRecord { +func (x *RecordLog) GetRecord() *coordinatorpb.OperationRecord { if x != nil { return x.Record } @@ -452,76 +452,75 @@ var file_chromadb_proto_logservice_proto_rawDesc = []byte{ 0x2f, 0x6c, 0x6f, 0x67, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x06, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x1a, 0x1b, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x64, 0x62, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x6f, 0x0a, 0x0f, 0x50, 0x75, 0x73, 0x68, 0x4c, 0x6f, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x69, 0x0a, 0x0f, 0x50, 0x75, 0x73, 0x68, 0x4c, 0x6f, 0x67, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x37, + 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x31, 0x0a, 0x07, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, - 0x1d, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x53, 0x75, 0x62, 0x6d, 0x69, 0x74, 0x45, - 0x6d, 0x62, 0x65, 0x64, 0x64, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x52, 0x07, - 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x73, 0x22, 0x35, 0x0a, 0x10, 0x50, 0x75, 0x73, 0x68, 0x4c, - 0x6f, 0x67, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x72, - 0x65, 0x63, 0x6f, 0x72, 0x64, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x05, 0x52, 0x0b, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0x9e, - 0x01, 0x0a, 0x0f, 0x50, 0x75, 0x6c, 0x6c, 0x4c, 0x6f, 0x67, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x22, 0x0a, 0x0d, 0x73, 0x74, 0x61, 0x72, 0x74, - 0x5f, 0x66, 0x72, 0x6f, 0x6d, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, - 0x73, 0x74, 0x61, 0x72, 0x74, 0x46, 0x72, 0x6f, 0x6d, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x62, - 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, - 0x09, 0x62, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x65, 0x6e, - 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x04, 0x20, 0x01, 0x28, - 0x03, 0x52, 0x0c, 0x65, 0x6e, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x22, - 0x59, 0x0a, 0x09, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x4c, 0x6f, 0x67, 0x12, 0x15, 0x0a, 0x06, - 0x6c, 0x6f, 0x67, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x6c, 0x6f, - 0x67, 0x49, 0x64, 0x12, 0x35, 0x0a, 0x06, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x53, 0x75, 0x62, - 0x6d, 0x69, 0x74, 0x45, 0x6d, 0x62, 0x65, 0x64, 0x64, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x63, 0x6f, - 0x72, 0x64, 0x52, 0x06, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x22, 0x3f, 0x0a, 0x10, 0x50, 0x75, - 0x6c, 0x6c, 0x4c, 0x6f, 0x67, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2b, - 0x0a, 0x07, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, - 0x11, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x4c, - 0x6f, 0x67, 0x52, 0x07, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x73, 0x22, 0x7e, 0x0a, 0x0e, 0x43, - 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x23, 0x0a, - 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x49, 0x64, 0x12, 0x20, 0x0a, 0x0c, 0x66, 0x69, 0x72, 0x73, 0x74, 0x5f, 0x6c, 0x6f, 0x67, 0x5f, - 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x66, 0x69, 0x72, 0x73, 0x74, 0x4c, - 0x6f, 0x67, 0x49, 0x64, 0x12, 0x25, 0x0a, 0x0f, 0x66, 0x69, 0x72, 0x73, 0x74, 0x5f, 0x6c, 0x6f, - 0x67, 0x5f, 0x69, 0x64, 0x5f, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x66, - 0x69, 0x72, 0x73, 0x74, 0x4c, 0x6f, 0x67, 0x49, 0x64, 0x54, 0x73, 0x22, 0x26, 0x0a, 0x24, 0x47, - 0x65, 0x74, 0x41, 0x6c, 0x6c, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, - 0x6e, 0x66, 0x6f, 0x54, 0x6f, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x22, 0x6f, 0x0a, 0x25, 0x47, 0x65, 0x74, 0x41, 0x6c, 0x6c, 0x43, 0x6f, 0x6c, - 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x54, 0x6f, 0x43, 0x6f, 0x6d, - 0x70, 0x61, 0x63, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x46, 0x0a, 0x13, - 0x61, 0x6c, 0x6c, 0x5f, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, - 0x6e, 0x66, 0x6f, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x63, 0x68, 0x72, 0x6f, - 0x6d, 0x61, 0x2e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, - 0x6f, 0x52, 0x11, 0x61, 0x6c, 0x6c, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x49, 0x6e, 0x66, 0x6f, 0x32, 0x8e, 0x02, 0x0a, 0x0a, 0x4c, 0x6f, 0x67, 0x53, 0x65, 0x72, 0x76, - 0x69, 0x63, 0x65, 0x12, 0x3f, 0x0a, 0x08, 0x50, 0x75, 0x73, 0x68, 0x4c, 0x6f, 0x67, 0x73, 0x12, - 0x17, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x50, 0x75, 0x73, 0x68, 0x4c, 0x6f, 0x67, - 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x18, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, - 0x61, 0x2e, 0x50, 0x75, 0x73, 0x68, 0x4c, 0x6f, 0x67, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x22, 0x00, 0x12, 0x3f, 0x0a, 0x08, 0x50, 0x75, 0x6c, 0x6c, 0x4c, 0x6f, 0x67, 0x73, - 0x12, 0x17, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x50, 0x75, 0x6c, 0x6c, 0x4c, 0x6f, - 0x67, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x18, 0x2e, 0x63, 0x68, 0x72, 0x6f, - 0x6d, 0x61, 0x2e, 0x50, 0x75, 0x6c, 0x6c, 0x4c, 0x6f, 0x67, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7e, 0x0a, 0x1d, 0x47, 0x65, 0x74, 0x41, 0x6c, 0x6c, 0x43, - 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x54, 0x6f, 0x43, - 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x12, 0x2c, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, - 0x47, 0x65, 0x74, 0x41, 0x6c, 0x6c, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x49, 0x6e, 0x66, 0x6f, 0x54, 0x6f, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2d, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x47, 0x65, - 0x74, 0x41, 0x6c, 0x6c, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, - 0x66, 0x6f, 0x54, 0x6f, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x39, 0x5a, 0x37, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, - 0x63, 0x6f, 0x6d, 0x2f, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2d, 0x63, 0x6f, 0x72, 0x65, 0x2f, - 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2f, 0x67, 0x6f, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2f, 0x6c, 0x6f, 0x67, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x70, 0x62, - 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x17, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x52, 0x07, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, + 0x73, 0x22, 0x35, 0x0a, 0x10, 0x50, 0x75, 0x73, 0x68, 0x4c, 0x6f, 0x67, 0x73, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x5f, + 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0b, 0x72, 0x65, 0x63, + 0x6f, 0x72, 0x64, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0x9e, 0x01, 0x0a, 0x0f, 0x50, 0x75, 0x6c, + 0x6c, 0x4c, 0x6f, 0x67, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x23, 0x0a, 0x0d, + 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, + 0x64, 0x12, 0x22, 0x0a, 0x0d, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x66, 0x72, 0x6f, 0x6d, 0x5f, + 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x73, 0x74, 0x61, 0x72, 0x74, 0x46, + 0x72, 0x6f, 0x6d, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, + 0x69, 0x7a, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, 0x62, 0x61, 0x74, 0x63, 0x68, + 0x53, 0x69, 0x7a, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x65, 0x6e, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, + 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x65, 0x6e, 0x64, + 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x22, 0x53, 0x0a, 0x09, 0x52, 0x65, 0x63, + 0x6f, 0x72, 0x64, 0x4c, 0x6f, 0x67, 0x12, 0x15, 0x0a, 0x06, 0x6c, 0x6f, 0x67, 0x5f, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x6c, 0x6f, 0x67, 0x49, 0x64, 0x12, 0x2f, 0x0a, + 0x06, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, + 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x52, 0x06, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x22, 0x3f, + 0x0a, 0x10, 0x50, 0x75, 0x6c, 0x6c, 0x4c, 0x6f, 0x67, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x2b, 0x0a, 0x07, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x73, 0x18, 0x01, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x52, 0x65, 0x63, + 0x6f, 0x72, 0x64, 0x4c, 0x6f, 0x67, 0x52, 0x07, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x73, 0x22, + 0x7e, 0x0a, 0x0e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, + 0x6f, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x20, 0x0a, 0x0c, 0x66, 0x69, 0x72, 0x73, 0x74, 0x5f, + 0x6c, 0x6f, 0x67, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x66, 0x69, + 0x72, 0x73, 0x74, 0x4c, 0x6f, 0x67, 0x49, 0x64, 0x12, 0x25, 0x0a, 0x0f, 0x66, 0x69, 0x72, 0x73, + 0x74, 0x5f, 0x6c, 0x6f, 0x67, 0x5f, 0x69, 0x64, 0x5f, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x0c, 0x66, 0x69, 0x72, 0x73, 0x74, 0x4c, 0x6f, 0x67, 0x49, 0x64, 0x54, 0x73, 0x22, + 0x26, 0x0a, 0x24, 0x47, 0x65, 0x74, 0x41, 0x6c, 0x6c, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x54, 0x6f, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x6f, 0x0a, 0x25, 0x47, 0x65, 0x74, 0x41, 0x6c, + 0x6c, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x54, + 0x6f, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x46, 0x0a, 0x13, 0x61, 0x6c, 0x6c, 0x5f, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x16, 0x2e, + 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x11, 0x61, 0x6c, 0x6c, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x32, 0x8e, 0x02, 0x0a, 0x0a, 0x4c, 0x6f, 0x67, + 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x3f, 0x0a, 0x08, 0x50, 0x75, 0x73, 0x68, 0x4c, + 0x6f, 0x67, 0x73, 0x12, 0x17, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x50, 0x75, 0x73, + 0x68, 0x4c, 0x6f, 0x67, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x18, 0x2e, 0x63, + 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x50, 0x75, 0x73, 0x68, 0x4c, 0x6f, 0x67, 0x73, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x3f, 0x0a, 0x08, 0x50, 0x75, 0x6c, 0x6c, + 0x4c, 0x6f, 0x67, 0x73, 0x12, 0x17, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x50, 0x75, + 0x6c, 0x6c, 0x4c, 0x6f, 0x67, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x18, 0x2e, + 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2e, 0x50, 0x75, 0x6c, 0x6c, 0x4c, 0x6f, 0x67, 0x73, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7e, 0x0a, 0x1d, 0x47, 0x65, 0x74, + 0x41, 0x6c, 0x6c, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, + 0x6f, 0x54, 0x6f, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x12, 0x2c, 0x2e, 0x63, 0x68, 0x72, + 0x6f, 0x6d, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x41, 0x6c, 0x6c, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x54, 0x6f, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, + 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2d, 0x2e, 0x63, 0x68, 0x72, 0x6f, 0x6d, + 0x61, 0x2e, 0x47, 0x65, 0x74, 0x41, 0x6c, 0x6c, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x54, 0x6f, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x39, 0x5a, 0x37, 0x67, 0x69, 0x74, + 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2d, 0x63, + 0x6f, 0x72, 0x65, 0x2f, 0x63, 0x68, 0x72, 0x6f, 0x6d, 0x61, 0x2f, 0x67, 0x6f, 0x2f, 0x70, 0x6b, + 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x6c, 0x6f, 0x67, 0x73, 0x65, 0x72, 0x76, 0x69, + 0x63, 0x65, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -546,11 +545,11 @@ var file_chromadb_proto_logservice_proto_goTypes = []interface{}{ (*CollectionInfo)(nil), // 5: chroma.CollectionInfo (*GetAllCollectionInfoToCompactRequest)(nil), // 6: chroma.GetAllCollectionInfoToCompactRequest (*GetAllCollectionInfoToCompactResponse)(nil), // 7: chroma.GetAllCollectionInfoToCompactResponse - (*coordinatorpb.SubmitEmbeddingRecord)(nil), // 8: chroma.SubmitEmbeddingRecord + (*coordinatorpb.OperationRecord)(nil), // 8: chroma.OperationRecord } var file_chromadb_proto_logservice_proto_depIdxs = []int32{ - 8, // 0: chroma.PushLogsRequest.records:type_name -> chroma.SubmitEmbeddingRecord - 8, // 1: chroma.RecordLog.record:type_name -> chroma.SubmitEmbeddingRecord + 8, // 0: chroma.PushLogsRequest.records:type_name -> chroma.OperationRecord + 8, // 1: chroma.RecordLog.record:type_name -> chroma.OperationRecord 3, // 2: chroma.PullLogsResponse.records:type_name -> chroma.RecordLog 5, // 3: chroma.GetAllCollectionInfoToCompactResponse.all_collection_info:type_name -> chroma.CollectionInfo 0, // 4: chroma.LogService.PushLogs:input_type -> chroma.PushLogsRequest diff --git a/idl/chromadb/proto/chroma.proto b/idl/chromadb/proto/chroma.proto index 44d899e4530f..70a684bb7f81 100644 --- a/idl/chromadb/proto/chroma.proto +++ b/idl/chromadb/proto/chroma.proto @@ -10,7 +10,6 @@ message Status { } // Types here should mirror chromadb/types.py - enum Operation { ADD = 0; UPDATE = 1; @@ -42,9 +41,6 @@ message Segment { string id = 1; string type = 2; SegmentScope scope = 3; - optional string topic = 4; // TODO should channel <> segment binding exist here? - // If a segment has a collection, it implies that this segment implements the full - // collection and can be used to service queries (for it's given scope.) optional string collection = 5; optional UpdateMetadata metadata = 6; map file_paths = 7; @@ -53,7 +49,6 @@ message Segment { message Collection { string id = 1; string name = 2; - string topic = 3; optional UpdateMetadata metadata = 4; optional int32 dimension = 5; string tenant = 6; @@ -84,12 +79,12 @@ message UpdateMetadata { map metadata = 1; } -message SubmitEmbeddingRecord { +// Represents an operation on the log +message OperationRecord { string id = 1; optional Vector vector = 2; optional UpdateMetadata metadata = 3; Operation operation = 4; - string collection_id = 5; } message VectorEmbeddingRecord { diff --git a/idl/chromadb/proto/coordinator.proto b/idl/chromadb/proto/coordinator.proto index 3695999ded87..43668bdfad2d 100644 --- a/idl/chromadb/proto/coordinator.proto +++ b/idl/chromadb/proto/coordinator.proto @@ -64,7 +64,6 @@ message GetSegmentsRequest { optional string id = 1; optional string type = 2; optional SegmentScope scope = 3; - optional string topic = 4; optional string collection = 5; // Collection ID } @@ -76,10 +75,6 @@ message GetSegmentsResponse { message UpdateSegmentRequest { string id = 1; - oneof topic_update { - string topic = 2; - bool reset_topic = 3; - } oneof collection_update { string collection = 4; bool reset_collection = 5; @@ -123,7 +118,6 @@ message DeleteCollectionResponse { message GetCollectionsRequest { optional string id = 1; optional string name = 2; - optional string topic = 3; string tenant = 4; string database = 5; } @@ -135,7 +129,6 @@ message GetCollectionsResponse { message UpdateCollectionRequest { string id = 1; - optional string topic = 2; optional string name = 3; optional int32 dimension = 4; oneof metadata_update { diff --git a/idl/chromadb/proto/logservice.proto b/idl/chromadb/proto/logservice.proto index e55a8ef331aa..f067af336d00 100644 --- a/idl/chromadb/proto/logservice.proto +++ b/idl/chromadb/proto/logservice.proto @@ -7,7 +7,7 @@ import "chromadb/proto/chroma.proto"; message PushLogsRequest { string collection_id = 1; - repeated SubmitEmbeddingRecord records = 2; + repeated OperationRecord records = 2; } message PushLogsResponse { @@ -23,7 +23,7 @@ message PullLogsRequest { message RecordLog { int64 log_id = 1; - SubmitEmbeddingRecord record = 2; + OperationRecord record = 2; } message PullLogsResponse { diff --git a/rust/worker/src/compactor/scheduler.rs b/rust/worker/src/compactor/scheduler.rs index 1ec961a077f9..244146b31719 100644 --- a/rust/worker/src/compactor/scheduler.rs +++ b/rust/worker/src/compactor/scheduler.rs @@ -72,7 +72,7 @@ impl Scheduler { // TODO: add a cache to avoid fetching the same collection multiple times let result = self .sysdb - .get_collections(collection_id, None, None, None, None) + .get_collections(collection_id, None, None, None) .await; match result { @@ -206,7 +206,6 @@ mod tests { fn filter_collections( collection: &Collection, collection_id: Option, - topic: Option, name: Option, tenant: Option, database: Option, @@ -214,9 +213,6 @@ mod tests { if collection_id.is_some() && collection_id.unwrap() != collection.id { return false; } - if topic.is_some() && topic.unwrap() != collection.topic { - return false; - } if name.is_some() && name.unwrap() != collection.name { return false; } @@ -235,7 +231,6 @@ mod tests { async fn get_collections( &mut self, collection_id: Option, - topic: Option, name: Option, tenant: Option, database: Option, @@ -245,7 +240,6 @@ mod tests { if !TestSysDb::filter_collections( &collection, collection_id, - topic.clone(), name.clone(), tenant.clone(), database.clone(), @@ -262,7 +256,6 @@ mod tests { id: Option, r#type: Option, scope: Option, - topic: Option, collection: Option, ) -> Result, GetSegmentsError> { Ok(Vec::new()) @@ -318,7 +311,6 @@ mod tests { let collection_1 = Collection { id: collection_uuid_1, name: "collection_1".to_string(), - topic: "collection_1".to_string(), metadata: None, dimension: Some(1), tenant: "tenant_1".to_string(), @@ -330,7 +322,6 @@ mod tests { let collection_2 = Collection { id: collection_uuid_2, name: "collection_2".to_string(), - topic: "collection_2".to_string(), metadata: None, dimension: Some(1), tenant: "tenant_2".to_string(), diff --git a/rust/worker/src/execution/operators/pull_log.rs b/rust/worker/src/execution/operators/pull_log.rs index d0a9cc7ae61d..8379667f2f24 100644 --- a/rust/worker/src/execution/operators/pull_log.rs +++ b/rust/worker/src/execution/operators/pull_log.rs @@ -99,12 +99,7 @@ impl Operator for PullLogsOperator { let mut result = Vec::new(); loop { let logs = client_clone - .read( - input.collection_id.to_string(), - offset, - batch_size, - input.end_timestamp, - ) + .read(input.collection_id, offset, batch_size, input.end_timestamp) .await; let mut logs = match logs { diff --git a/rust/worker/src/execution/orchestration/compact.rs b/rust/worker/src/execution/orchestration/compact.rs index bce69aa59e5a..060b120dd322 100644 --- a/rust/worker/src/execution/orchestration/compact.rs +++ b/rust/worker/src/execution/orchestration/compact.rs @@ -83,7 +83,7 @@ impl CompactOrchestrator { async fn get_collection_id_for_segment_id(&mut self, segment_id: Uuid) -> Option { let segments = self .sysdb - .get_segments(Some(segment_id), None, None, None, None) + .get_segments(Some(segment_id), None, None, None) .await; match segments { Ok(segments) => match segments.get(0) { diff --git a/rust/worker/src/execution/orchestration/hnsw.rs b/rust/worker/src/execution/orchestration/hnsw.rs index 579497cc03b7..1e76d9c7b913 100644 --- a/rust/worker/src/execution/orchestration/hnsw.rs +++ b/rust/worker/src/execution/orchestration/hnsw.rs @@ -94,7 +94,7 @@ impl HnswQueryOrchestrator { async fn get_collection_id_for_segment_id(&mut self, segment_id: Uuid) -> Option { let segments = self .sysdb - .get_segments(Some(segment_id), None, None, None, None) + .get_segments(Some(segment_id), None, None, None) .await; match segments { Ok(segments) => match segments.get(0) { diff --git a/rust/worker/src/ingest/ingest.rs b/rust/worker/src/ingest/ingest.rs deleted file mode 100644 index 770b0681de89..000000000000 --- a/rust/worker/src/ingest/ingest.rs +++ /dev/null @@ -1,418 +0,0 @@ -use async_trait::async_trait; -use bytes::Bytes; -use futures::{StreamExt, TryStreamExt}; -use prost::Message; -use std::{ - collections::{HashMap, HashSet}, - fmt::Debug, - sync::{Arc, RwLock}, -}; - -use crate::{ - assignment::{ - self, - assignment_policy::{self, AssignmentPolicy}, - }, - chroma_proto, - config::{Configurable, WorkerConfig}, - errors::{ChromaError, ErrorCodes}, - memberlist::{CustomResourceMemberlistProvider, Memberlist}, - sysdb::sysdb::{GrpcSysDb, SysDb}, - system::{Component, ComponentContext, ComponentHandle, Handler, Receiver, StreamHandler}, - types::{EmbeddingRecord, EmbeddingRecordConversionError, SeqId}, -}; - -use pulsar::{Consumer, DeserializeMessage, Payload, Pulsar, SubType, TokioExecutor}; -use thiserror::Error; - -use super::message_id::PulsarMessageIdWrapper; - -/// An ingest component is responsible for ingesting data into the system from the log -/// stream. -/// # Notes -/// The only current implementation of the ingest is the Pulsar ingest. -pub(crate) struct Ingest { - assignment_policy: RwLock>, - assigned_topics: RwLock>, - topic_to_handle: RwLock>>, - queue_size: usize, - my_ip: String, - pulsar_tenant: String, - pulsar_namespace: String, - pulsar: Pulsar, - sysdb: Box, - scheduler: Option)>>>, -} - -impl Component for Ingest { - fn queue_size(&self) -> usize { - return self.queue_size; - } -} - -impl Debug for Ingest { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - f.debug_struct("Ingest") - .field("queue_size", &self.queue_size) - .finish() - } -} - -#[derive(Error, Debug)] -pub(crate) enum IngestConfigurationError { - #[error(transparent)] - PulsarError(#[from] pulsar::Error), -} - -impl ChromaError for IngestConfigurationError { - fn code(&self) -> ErrorCodes { - match self { - IngestConfigurationError::PulsarError(_e) => ErrorCodes::Internal, - } - } -} - -// TODO: Nest the ingest assignment policy inside the ingest component config so its -// specific to the ingest component and can be used here -#[async_trait] -impl Configurable for Ingest { - async fn try_from_config(worker_config: &WorkerConfig) -> Result> { - let assignment_policy = assignment_policy::RendezvousHashingAssignmentPolicy::new( - worker_config.pulsar_tenant.clone(), - worker_config.pulsar_namespace.clone(), - ); - - println!("Pulsar connection url: {}", worker_config.pulsar_url); - let pulsar = match Pulsar::builder(worker_config.pulsar_url.clone(), TokioExecutor) - .build() - .await - { - Ok(pulsar) => pulsar, - Err(e) => { - return Err(Box::new(IngestConfigurationError::PulsarError(e))); - } - }; - - // TODO: Sysdb should have a dynamic resolution in sysdb - let sysdb = GrpcSysDb::try_from_config(worker_config).await; - let sysdb = match sysdb { - Ok(sysdb) => sysdb, - Err(err) => { - return Err(err); - } - }; - - let ingest = Ingest { - assignment_policy: RwLock::new(Box::new(assignment_policy)), - assigned_topics: RwLock::new(vec![]), - topic_to_handle: RwLock::new(HashMap::new()), - queue_size: worker_config.ingest.queue_size, - my_ip: worker_config.my_ip.clone(), - pulsar: pulsar, - pulsar_tenant: worker_config.pulsar_tenant.clone(), - pulsar_namespace: worker_config.pulsar_namespace.clone(), - sysdb: Box::new(sysdb), - scheduler: None, - }; - Ok(ingest) - } -} - -impl Ingest { - fn get_topics(&self) -> Vec { - // This mirrors the current python and go code, which assumes a fixed set of topics - let mut topics = Vec::with_capacity(16); - for i in 0..16 { - let topic = format!( - "persistent://{}/{}/chroma_log_{}", - self.pulsar_tenant, self.pulsar_namespace, i - ); - topics.push(topic); - } - return topics; - } - - pub(crate) fn subscribe( - &mut self, - scheduler: Box)>>, - ) { - self.scheduler = Some(scheduler); - } -} - -#[async_trait] -impl Handler for Ingest { - async fn handle(&mut self, msg: Memberlist, ctx: &ComponentContext) { - let mut new_assignments = HashSet::new(); - let candidate_topics: Vec = self.get_topics(); - println!( - "Performing assignment for topics: {:?}. My ip: {}", - candidate_topics, self.my_ip - ); - // Scope for assigner write lock to be released so we don't hold it over await - { - let mut assigner = match self.assignment_policy.write() { - Ok(assigner) => assigner, - Err(err) => { - println!("Failed to read assignment policy: {:?}", err); - return; - } - }; - - // Use the assignment policy to assign topics to this worker - assigner.set_members(msg); - for topic in candidate_topics.iter() { - let assignment = assigner.assign(topic); - let assignment = match assignment { - Ok(assignment) => assignment, - Err(err) => { - // TODO: Log error - continue; - } - }; - if assignment == self.my_ip { - new_assignments.insert(topic); - } - } - } - - // Compute the topics we need to add/remove - let mut to_remove = Vec::new(); - let mut to_add = Vec::new(); - - // Scope for assigned topics read lock to be released so we don't hold it over await - { - let assigned_topics_handle = self.assigned_topics.read(); - match assigned_topics_handle { - Ok(assigned_topics) => { - // Compute the diff between the current assignments and the new assignments - for topic in assigned_topics.iter() { - if !new_assignments.contains(topic) { - to_remove.push(topic.to_string()); - } - } - for topic in new_assignments.iter() { - if !assigned_topics.contains(*topic) { - to_add.push(topic.to_string()); - } - } - } - Err(err) => { - // TODO: Log error and handle lock poisoning - } - } - } - - // Unsubscribe from topics we no longer need to listen to - for topic in to_remove.iter() { - match self.topic_to_handle.write() { - Ok(mut topic_to_handle) => { - let handle = topic_to_handle.remove(topic); - match handle { - Some(mut handle) => { - handle.stop(); - } - None => { - // TODO: This should log an error - println!("No handle found for topic: {}", topic); - } - } - } - Err(err) => { - // TODO: Log an error and handle lock poisoning - } - } - } - - // Subscribe to new topics - for topic in to_add.iter() { - // Do the subscription and register the stream to this ingest component - let consumer: Consumer = self - .pulsar - .consumer() - .with_topic(topic.to_string()) - .with_subscription_type(SubType::Exclusive) - .build() - .await - .unwrap(); - println!("Created consumer for topic: {}", topic); - - let scheduler = match &self.scheduler { - Some(scheduler) => scheduler.clone(), - None => { - // TODO: log error - return; - } - }; - - let ingest_topic_component = - PulsarIngestTopic::new(consumer, self.sysdb.clone(), scheduler); - - let handle = ctx.system.clone().start_component(ingest_topic_component); - - // Bookkeep the handle so we can shut the stream down later - match self.topic_to_handle.write() { - Ok(mut topic_to_handle) => { - topic_to_handle.insert(topic.to_string(), handle); - } - Err(err) => { - // TODO: log error and handle lock poisoning - println!("Failed to write topic to handle: {:?}", err); - } - } - } - } -} - -impl DeserializeMessage for chroma_proto::SubmitEmbeddingRecord { - type Output = Self; - - fn deserialize_message(payload: &Payload) -> chroma_proto::SubmitEmbeddingRecord { - // Its a bit strange to unwrap here, but the pulsar api doesn't give us a way to - // return an error, so we have to panic if we can't decode the message - // also we are forced to clone since the api doesn't give us a way to borrow the bytes - // TODO: can we not clone? - // TODO: I think just typing this to Result<> would allow errors to propagate - let record = - chroma_proto::SubmitEmbeddingRecord::decode(Bytes::from(payload.data.clone())).unwrap(); - return record; - } -} - -struct PulsarIngestTopic { - consumer: RwLock>>, - sysdb: Box, - scheduler: Box)>>, -} - -impl Debug for PulsarIngestTopic { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - f.debug_struct("PulsarIngestTopic").finish() - } -} - -impl PulsarIngestTopic { - fn new( - consumer: Consumer, - sysdb: Box, - scheduler: Box)>>, - ) -> Self { - PulsarIngestTopic { - consumer: RwLock::new(Some(consumer)), - sysdb: sysdb, - scheduler: scheduler, - } - } -} - -#[async_trait] -impl Component for PulsarIngestTopic { - fn queue_size(&self) -> usize { - 1000 - } - - async fn on_start(&mut self, ctx: &ComponentContext) -> () { - println!("Starting PulsarIngestTopic for topic"); - let stream = match self.consumer.write() { - Ok(mut consumer_handle) => consumer_handle.take(), - Err(err) => { - println!("Failed to take consumer handle: {:?}", err); - None - } - }; - let stream = match stream { - Some(stream) => stream, - None => { - return; - } - }; - let stream = stream.then(|result| async { - match result { - Ok(msg) => { - println!( - "PulsarIngestTopic received message with id: {:?}", - msg.message_id - ); - // Convert the Pulsar Message to an EmbeddingRecord - let proto_embedding_record = msg.deserialize(); - let id = msg.message_id; - let seq_id: SeqId = PulsarMessageIdWrapper(id).into(); - let embedding_record: Result = - (proto_embedding_record, seq_id).try_into(); - match embedding_record { - Ok(embedding_record) => { - return Some(Box::new(embedding_record)); - } - Err(err) => { - // TODO: Handle and log - println!("PulsarIngestTopic received error while performing conversion: {:?}", err); - } - } - None - } - Err(err) => { - // TODO: Log an error - println!("PulsarIngestTopic received error: {:?}", err); - // Put this on a dead letter queue, this concept does not exist in our - // system yet - None - } - } - }); - self.register_stream(stream, ctx); - } -} - -#[async_trait] -impl Handler>> for PulsarIngestTopic { - async fn handle( - &mut self, - message: Option>, - _ctx: &ComponentContext, - ) -> () { - // Use the sysdb to tenant id for the embedding record - let embedding_record = match message { - Some(embedding_record) => embedding_record, - None => { - return; - } - }; - - // TODO: Cache this - let coll = self - .sysdb - .get_collections(Some(embedding_record.collection_id), None, None, None, None) - .await; - - let coll = match coll { - Ok(coll) => coll, - Err(err) => { - println!( - "PulsarIngestTopic received error while fetching collection: {:?}", - err - ); - return; - } - }; - - let coll = match coll.first() { - Some(coll) => coll, - None => { - println!("PulsarIngestTopic received empty collection"); - return; - } - }; - - let tenant_id = &coll.tenant; - - let _ = self - .scheduler - .send((tenant_id.clone(), embedding_record)) - .await; - - // TODO: Handle res - } -} - -#[async_trait] -impl StreamHandler>> for PulsarIngestTopic {} diff --git a/rust/worker/src/ingest/mod.rs b/rust/worker/src/ingest/mod.rs index ae7aaf8d7b52..73d2868ef1de 100644 --- a/rust/worker/src/ingest/mod.rs +++ b/rust/worker/src/ingest/mod.rs @@ -1,8 +1,2 @@ pub(crate) mod config; -mod ingest; mod message_id; -mod scheduler; - -// Re-export the ingest provider for use in the worker -pub(crate) use ingest::*; -pub(crate) use scheduler::*; diff --git a/rust/worker/src/ingest/scheduler.rs b/rust/worker/src/ingest/scheduler.rs deleted file mode 100644 index 7a8e7d84b92a..000000000000 --- a/rust/worker/src/ingest/scheduler.rs +++ /dev/null @@ -1,213 +0,0 @@ -// A scheduler recieves embedding records for a given batch of documents -// and schedules them to be ingested to the segment manager - -use crate::{ - system::{Component, ComponentContext, Handler, Receiver}, - types::EmbeddingRecord, -}; -use async_trait::async_trait; -use rand::prelude::SliceRandom; -use rand::Rng; -use std::{ - collections::{btree_map::Range, HashMap}, - fmt::{Debug, Formatter, Result}, - sync::Arc, -}; - -pub(crate) struct RoundRobinScheduler { - // The segment manager to schedule to, a segment manager is a component - // segment_manager: SegmentManager - curr_wake_up: Option>, - tenant_to_queue: HashMap>>, - new_tenant_channel: Option>, - subscribers: Option>>>>, -} - -impl Debug for RoundRobinScheduler { - fn fmt(&self, f: &mut Formatter<'_>) -> Result { - f.debug_struct("Scheduler").finish() - } -} - -impl RoundRobinScheduler { - pub(crate) fn new() -> Self { - RoundRobinScheduler { - curr_wake_up: None, - tenant_to_queue: HashMap::new(), - new_tenant_channel: None, - subscribers: Some(Vec::new()), - } - } - - pub(crate) fn subscribe(&mut self, subscriber: Box>>) { - match self.subscribers { - Some(ref mut subscribers) => { - subscribers.push(subscriber); - } - None => {} - } - } -} - -#[async_trait] -impl Component for RoundRobinScheduler { - fn queue_size(&self) -> usize { - 1000 - } - - async fn on_start(&mut self, ctx: &ComponentContext) { - let sleep_sender = ctx.sender.clone(); - let (new_tenant_tx, mut new_tenant_rx) = tokio::sync::mpsc::channel(1000); - self.new_tenant_channel = Some(new_tenant_tx); - let cancellation_token = ctx.cancellation_token.clone(); - let subscribers = self.subscribers.take(); - let mut subscribers = match subscribers { - Some(subscribers) => subscribers, - None => { - // TODO: log + error - return; - } - }; - tokio::spawn(async move { - let mut tenant_queues: HashMap< - String, - tokio::sync::mpsc::Receiver>, - > = HashMap::new(); - loop { - // TODO: handle cancellation - let mut did_work = false; - for tenant_queue in tenant_queues.values_mut() { - match tenant_queue.try_recv() { - Ok(message) => { - // Randomly pick a subscriber to send the message to - // This serves as a crude load balancing between available threads - // Future improvements here could be - // - Use a work stealing scheduler - // - Use rayon - // - We need to enforce partial order over writes to a given key - // so we need a mechanism to ensure that all writes to a given key - // occur in order - let mut subscriber = None; - { - let mut rng = rand::thread_rng(); - subscriber = subscribers.choose_mut(&mut rng); - } - match subscriber { - Some(subscriber) => { - let res = subscriber.send(message).await; - } - None => {} - } - did_work = true; - } - Err(tokio::sync::mpsc::error::TryRecvError::Empty) => { - continue; - } - Err(_) => { - // TODO: Handle a erroneous channel - // log an error - continue; - } - }; - } - - match new_tenant_rx.try_recv() { - Ok(new_tenant_message) => { - tenant_queues.insert(new_tenant_message.tenant, new_tenant_message.channel); - } - Err(tokio::sync::mpsc::error::TryRecvError::Empty) => { - // no - op - } - Err(_) => { - // TODO: handle erroneous channel - // log an error - continue; - } - }; - - if !did_work { - // Send a sleep message to the sender - let (wake_tx, wake_rx) = tokio::sync::oneshot::channel(); - let sleep_res = sleep_sender.send(SleepMessage { sender: wake_tx }).await; - let wake_res = wake_rx.await; - } - } - }); - } -} - -#[async_trait] -impl Handler<(String, Box)> for RoundRobinScheduler { - async fn handle( - &mut self, - message: (String, Box), - _ctx: &ComponentContext, - ) { - let (tenant, embedding_record) = message; - // Check if the tenant is already in the tenant set, if not we need to inform the scheduler loop - // of a new tenant - if self.tenant_to_queue.get(&tenant).is_none() { - // Create a new channel for the tenant - let (sender, reciever) = tokio::sync::mpsc::channel(1000); - // Add the tenant to the tenant set - self.tenant_to_queue.insert(tenant.clone(), sender); - // Send the new tenant message to the scheduler loop - let new_tenant_channel = match self.new_tenant_channel { - Some(ref mut channel) => channel, - None => { - // TODO: this is an error - // It should always be populated by on_start - return; - } - }; - let res = new_tenant_channel - .send(NewTenantMessage { - tenant: tenant.clone(), - channel: reciever, - }) - .await; - // TODO: handle this res - } - - // Send the embedding record to the tenant's channel - let res = self - .tenant_to_queue - .get(&tenant) - .unwrap() - .send(embedding_record) - .await; - // TODO: handle this res - - // Check if the scheduler is sleeping, if so wake it up - // TODO: we need to init with a wakeup otherwise we are off by one - if self.curr_wake_up.is_some() { - // Send a wake up message to the scheduler loop - let res = self.curr_wake_up.take().unwrap().send(WakeMessage {}); - // TOOD: handle this res - } - } -} - -#[async_trait] -impl Handler for RoundRobinScheduler { - async fn handle(&mut self, message: SleepMessage, _ctx: &ComponentContext) { - // Set the current wake up channel - self.curr_wake_up = Some(message.sender); - } -} - -/// Used by round robin scheduler to wake its scheduler loop -#[derive(Debug)] -struct WakeMessage {} - -/// The round robin scheduler will sleep when there is no work to be done and send a sleep message -/// this allows the manager to wake it up when there is work to be scheduled -#[derive(Debug)] -struct SleepMessage { - sender: tokio::sync::oneshot::Sender, -} - -struct NewTenantMessage { - tenant: String, - channel: tokio::sync::mpsc::Receiver>, -} diff --git a/rust/worker/src/lib.rs b/rust/worker/src/lib.rs index 1ebc7c2fe7bb..d2fce8d0abb9 100644 --- a/rust/worker/src/lib.rs +++ b/rust/worker/src/lib.rs @@ -70,14 +70,6 @@ pub async fn worker_entrypoint() { // The two root components are ingest, and the gRPC server let mut system: system::System = system::System::new(); - let mut ingest = match ingest::Ingest::try_from_config(&config.worker).await { - Ok(ingest) => ingest, - Err(err) => { - println!("Failed to create ingest component: {:?}", err); - return; - } - }; - let mut memberlist = match memberlist::CustomResourceMemberlistProvider::try_from_config(&config.worker).await { Ok(memberlist) => memberlist, @@ -87,8 +79,6 @@ pub async fn worker_entrypoint() { } }; - let mut scheduler = ingest::RoundRobinScheduler::new(); - let segment_manager = match segment::SegmentManager::try_from_config(&config.worker).await { Ok(segment_manager) => segment_manager, Err(err) => { @@ -116,19 +106,10 @@ pub async fn worker_entrypoint() { worker_server.set_segment_manager(segment_manager.clone()); // Boot the system - // memberlist -> ingest -> scheduler -> NUM_THREADS x segment_ingestor -> segment_manager + // memberlist -> (This is broken for now until we have compaction manager) NUM_THREADS x segment_ingestor -> segment_manager // server <- segment_manager - for recv in segment_ingestor_receivers { - scheduler.subscribe(recv); - } - - let mut scheduler_handler = system.start_component(scheduler); - ingest.subscribe(scheduler_handler.receiver()); - - let mut ingest_handle = system.start_component(ingest); - let recv = ingest_handle.receiver(); - memberlist.subscribe(recv); + // memberlist.subscribe(recv); let mut memberlist_handle = system.start_component(memberlist); let server_join_handle = tokio::spawn(async move { @@ -136,10 +117,5 @@ pub async fn worker_entrypoint() { }); // Join on all handles - let _ = tokio::join!( - ingest_handle.join(), - memberlist_handle.join(), - scheduler_handler.join(), - server_join_handle, - ); + let _ = tokio::join!(memberlist_handle.join(), server_join_handle,); } diff --git a/rust/worker/src/log/log.rs b/rust/worker/src/log/log.rs index 0f1c5c6c16c7..466c02cff86f 100644 --- a/rust/worker/src/log/log.rs +++ b/rust/worker/src/log/log.rs @@ -11,6 +11,7 @@ use async_trait::async_trait; use std::collections::HashMap; use std::fmt::Debug; use thiserror::Error; +use uuid::Uuid; // CollectionInfo is a struct that contains information about a collection for the // compacting process. It contains information about the collection id, the first log id, @@ -34,7 +35,7 @@ pub(crate) struct CollectionRecord { pub(crate) trait Log: Send + Sync + LogClone + Debug { async fn read( &mut self, - collection_id: String, + collection_id: Uuid, offset: i64, batch_size: i32, end_timestamp: Option, @@ -117,7 +118,7 @@ impl Configurable for GrpcLog { impl Log for GrpcLog { async fn read( &mut self, - collection_id: String, + collection_id: Uuid, offset: i64, batch_size: i32, end_timestamp: Option, @@ -127,7 +128,7 @@ impl Log for GrpcLog { None => -1, }; let request = self.client.pull_logs(chroma_proto::PullLogsRequest { - collection_id, + collection_id: collection_id.to_string(), start_from_id: offset, batch_size, end_timestamp, @@ -138,7 +139,7 @@ impl Log for GrpcLog { let logs = response.into_inner().records; let mut result = Vec::new(); for log in logs { - let embedding_record = log.try_into(); + let embedding_record = (log, collection_id).try_into(); match embedding_record { Ok(embedding_record) => { result.push(embedding_record); @@ -264,7 +265,7 @@ impl InMemoryLog { impl Log for InMemoryLog { async fn read( &mut self, - collection_id: String, + collection_id: Uuid, offset: i64, batch_size: i32, end_timestamp: Option, @@ -274,7 +275,7 @@ impl Log for InMemoryLog { None => i64::MAX, }; - let logs = match self.logs.get(&collection_id) { + let logs = match self.logs.get(&collection_id.to_string()) { Some(logs) => logs, None => return Ok(Vec::new()), }; diff --git a/rust/worker/src/segment/segment_manager.rs b/rust/worker/src/segment/segment_manager.rs index 314a5b99cdfd..a1cdcf02736f 100644 --- a/rust/worker/src/segment/segment_manager.rs +++ b/rust/worker/src/segment/segment_manager.rs @@ -206,7 +206,6 @@ impl SegmentManager { None, None, Some(SegmentScope::VECTOR), - None, Some(collection_uuid.clone()), ) .await; diff --git a/rust/worker/src/sysdb/sysdb.rs b/rust/worker/src/sysdb/sysdb.rs index 990268e66ec8..fa4d6c387df2 100644 --- a/rust/worker/src/sysdb/sysdb.rs +++ b/rust/worker/src/sysdb/sysdb.rs @@ -20,7 +20,6 @@ pub(crate) trait SysDb: Send + Sync + SysDbClone + Debug { async fn get_collections( &mut self, collection_id: Option, - topic: Option, name: Option, tenant: Option, database: Option, @@ -31,7 +30,6 @@ pub(crate) trait SysDb: Send + Sync + SysDbClone + Debug { id: Option, r#type: Option, scope: Option, - topic: Option, collection: Option, ) -> Result, GetSegmentsError>; } @@ -107,7 +105,6 @@ impl SysDb for GrpcSysDb { async fn get_collections( &mut self, collection_id: Option, - topic: Option, name: Option, tenant: Option, database: Option, @@ -127,7 +124,6 @@ impl SysDb for GrpcSysDb { .client .get_collections(chroma_proto::GetCollectionsRequest { id: collection_id_str, - topic: topic, name: name, tenant: if tenant.is_some() { tenant.unwrap() @@ -171,7 +167,6 @@ impl SysDb for GrpcSysDb { id: Option, r#type: Option, scope: Option, - topic: Option, collection: Option, ) -> Result, GetSegmentsError> { let res = self @@ -189,7 +184,6 @@ impl SysDb for GrpcSysDb { } else { None }, - topic: topic, collection: if collection.is_some() { Some(collection.unwrap().to_string()) } else { diff --git a/rust/worker/src/types/collection.rs b/rust/worker/src/types/collection.rs index ecfdeef13460..1f8766a1ee1f 100644 --- a/rust/worker/src/types/collection.rs +++ b/rust/worker/src/types/collection.rs @@ -10,7 +10,6 @@ use uuid::Uuid; pub(crate) struct Collection { pub(crate) id: Uuid, pub(crate) name: String, - pub(crate) topic: String, pub(crate) metadata: Option, pub(crate) dimension: Option, pub(crate) tenant: String, @@ -54,7 +53,6 @@ impl TryFrom for Collection { Ok(Collection { id: collection_uuid, name: proto_collection.name, - topic: proto_collection.topic, metadata: collection_metadata, dimension: proto_collection.dimension, tenant: proto_collection.tenant, @@ -74,7 +72,6 @@ mod test { let proto_collection = chroma_proto::Collection { id: "00000000-0000-0000-0000-000000000000".to_string(), name: "foo".to_string(), - topic: "bar".to_string(), metadata: None, dimension: None, tenant: "baz".to_string(), @@ -85,7 +82,6 @@ mod test { let converted_collection: Collection = proto_collection.try_into().unwrap(); assert_eq!(converted_collection.id, Uuid::nil()); assert_eq!(converted_collection.name, "foo".to_string()); - assert_eq!(converted_collection.topic, "bar".to_string()); assert_eq!(converted_collection.metadata, None); assert_eq!(converted_collection.dimension, None); assert_eq!(converted_collection.tenant, "baz".to_string()); diff --git a/rust/worker/src/types/embedding_record.rs b/rust/worker/src/types/embedding_record.rs index 6ba9bdf255ac..396be6c22c5f 100644 --- a/rust/worker/src/types/embedding_record.rs +++ b/rust/worker/src/types/embedding_record.rs @@ -7,8 +7,8 @@ use crate::{ errors::{ChromaError, ErrorCodes}, }; +use chroma_proto::OperationRecord; use chroma_proto::RecordLog; -use chroma_proto::SubmitEmbeddingRecord; use num_bigint::BigInt; use thiserror::Error; use uuid::Uuid; @@ -24,7 +24,8 @@ pub(crate) struct EmbeddingRecord { pub(crate) collection_id: Uuid, } -pub(crate) type SubmitEmbeddingRecordWithSeqId = (chroma_proto::SubmitEmbeddingRecord, SeqId); +pub(crate) type OperationRecordWithSeqIdAndCollectionId = + (chroma_proto::OperationRecord, SeqId, Uuid); #[derive(Error, Debug)] pub(crate) enum EmbeddingRecordConversionError { @@ -50,24 +51,20 @@ impl_base_convert_error!(EmbeddingRecordConversionError, { EmbeddingRecordConversionError::VectorConversionError(inner) => inner.code(), }); -impl TryFrom for EmbeddingRecord { +impl TryFrom for EmbeddingRecord { type Error = EmbeddingRecordConversionError; fn try_from( - proto_submit_with_seq_id: SubmitEmbeddingRecordWithSeqId, + proto_submit_with_seq_id: OperationRecordWithSeqIdAndCollectionId, ) -> Result { let proto_submit = proto_submit_with_seq_id.0; let seq_id = proto_submit_with_seq_id.1; + let collection_id = proto_submit_with_seq_id.2; let op = match proto_submit.operation.try_into() { Ok(op) => op, Err(e) => return Err(EmbeddingRecordConversionError::OperationConversionError(e)), }; - let collection_uuid = match Uuid::try_parse(&proto_submit.collection_id) { - Ok(uuid) => uuid, - Err(_) => return Err(EmbeddingRecordConversionError::InvalidUuid), - }; - let (embedding, encoding) = match proto_submit.vector { Some(proto_vector) => match proto_vector.try_into() { Ok((embedding, encoding)) => (Some(embedding), Some(encoding)), @@ -96,15 +93,19 @@ impl TryFrom for EmbeddingRecord { encoding: encoding, metadata: metadata, operation: op, - collection_id: collection_uuid, + collection_id: collection_id, }) } } -impl TryFrom for EmbeddingRecord { +type RecordLogWithCollectionId = (RecordLog, Uuid); + +impl TryFrom for EmbeddingRecord { type Error = EmbeddingRecordConversionError; - fn try_from(record_log: RecordLog) -> Result { + fn try_from(record_log_collection_id: RecordLogWithCollectionId) -> Result { + let record_log = record_log_collection_id.0; + let collection_uuid = record_log_collection_id.1; let proto_submit = record_log .record .ok_or(EmbeddingRecordConversionError::DecodeError( @@ -117,11 +118,6 @@ impl TryFrom for EmbeddingRecord { Err(e) => return Err(EmbeddingRecordConversionError::OperationConversionError(e)), }; - let collection_uuid = match Uuid::try_parse(&proto_submit.collection_id) { - Ok(uuid) => uuid, - Err(_) => return Err(EmbeddingRecordConversionError::InvalidUuid), - }; - let (embedding, encoding) = match proto_submit.vector { Some(proto_vector) => match proto_vector.try_into() { Ok((embedding, encoding)) => (Some(embedding), Some(encoding)), @@ -281,6 +277,7 @@ mod tests { use std::collections::HashMap; use num_bigint::BigInt; + use uuid::uuid; use super::*; use crate::{chroma_proto, types::UpdateMetadataValue}; @@ -311,15 +308,18 @@ mod tests { encoding: chroma_proto::ScalarEncoding::Float32 as i32, dimension: 3, }; - let proto_submit = chroma_proto::SubmitEmbeddingRecord { + let proto_submit = chroma_proto::OperationRecord { id: "00000000-0000-0000-0000-000000000000".to_string(), vector: Some(proto_vector), metadata: Some(metadata), operation: chroma_proto::Operation::Add as i32, - collection_id: "00000000-0000-0000-0000-000000000000".to_string(), }; - let converted_embedding_record: EmbeddingRecord = - EmbeddingRecord::try_from((proto_submit, BigInt::from(42))).unwrap(); + let converted_embedding_record: EmbeddingRecord = EmbeddingRecord::try_from(( + proto_submit, + BigInt::from(42), + uuid!("00000000-0000-0000-0000-000000000000"), + )) + .unwrap(); assert_eq!(converted_embedding_record.id, Uuid::nil().to_string()); assert_eq!(converted_embedding_record.seq_id, BigInt::from(42)); assert_eq!( @@ -353,18 +353,19 @@ mod tests { encoding: chroma_proto::ScalarEncoding::Float32 as i32, dimension: 3, }; - let proto_submit = chroma_proto::SubmitEmbeddingRecord { + let proto_submit = chroma_proto::OperationRecord { id: "00000000-0000-0000-0000-000000000000".to_string(), vector: Some(proto_vector), metadata: Some(metadata), operation: chroma_proto::Operation::Add as i32, - collection_id: "00000000-0000-0000-0000-000000000000".to_string(), }; let record_log = chroma_proto::RecordLog { log_id: 42, record: Some(proto_submit), }; - let converted_embedding_record = EmbeddingRecord::try_from(record_log).unwrap(); + let converted_embedding_record = + EmbeddingRecord::try_from((record_log, uuid!("00000000-0000-0000-0000-000000000000"))) + .unwrap(); assert_eq!(converted_embedding_record.id, Uuid::nil().to_string()); assert_eq!(converted_embedding_record.seq_id, BigInt::from(42)); assert_eq!( diff --git a/rust/worker/src/types/segment.rs b/rust/worker/src/types/segment.rs index d85d1293eeaa..caf7df347eb4 100644 --- a/rust/worker/src/types/segment.rs +++ b/rust/worker/src/types/segment.rs @@ -18,7 +18,6 @@ pub(crate) struct Segment { pub(crate) id: Uuid, pub(crate) r#type: SegmentType, pub(crate) scope: SegmentScope, - pub(crate) topic: Option, pub(crate) collection: Option, pub(crate) metadata: Option, pub(crate) file_path: HashMap>, @@ -94,7 +93,6 @@ impl TryFrom for Segment { id: segment_uuid, r#type: segment_type, scope: scope, - topic: proto_segment.topic, collection: collection_uuid, metadata: segment_metadata, file_path: file_paths, @@ -124,7 +122,6 @@ mod tests { id: "00000000-0000-0000-0000-000000000000".to_string(), r#type: "urn:chroma:segment/vector/hnsw-distributed".to_string(), scope: chroma_proto::SegmentScope::Vector as i32, - topic: Some("test".to_string()), collection: Some("00000000-0000-0000-0000-000000000000".to_string()), metadata: Some(metadata), file_paths: HashMap::new(), @@ -133,7 +130,6 @@ mod tests { assert_eq!(converted_segment.id, Uuid::nil()); assert_eq!(converted_segment.r#type, SegmentType::HnswDistributed); assert_eq!(converted_segment.scope, SegmentScope::VECTOR); - assert_eq!(converted_segment.topic, Some("test".to_string())); assert_eq!(converted_segment.collection, Some(Uuid::nil())); let metadata = converted_segment.metadata.unwrap(); assert_eq!(metadata.len(), 1);