├── .gitignore ├── CMakeLists.txt ├── LICENSE.txt ├── README.md ├── go.mod ├── go.sum ├── licenses ├── APL2.txt └── BSL-Couchbase.txt └── secondary ├── README.md ├── TODO.rst ├── adminport ├── admin_test.go ├── client │ └── admin_httpc.go ├── common │ └── admin.go └── server │ └── admin_httpd.go ├── audit ├── audit.go └── audit_descriptor.json ├── bin ├── 2ifab.py ├── README.md ├── cbq_drop.sh ├── cbq_sanity.sh ├── periodicstats.py └── projcalls.py ├── build.sh ├── cmd ├── cbindex │ └── main.go ├── cbindexperf │ ├── README.markdown │ ├── cfg │ │ ├── ConfigUsage.md │ │ ├── idx_vector_limit.json │ │ └── idx_vector_projection.json │ ├── config.go │ ├── executor.go │ ├── main.go │ └── test_executor.go ├── cbindexplan │ ├── main.go │ └── sample │ │ ├── index.json │ │ └── plan.json ├── indexer │ └── main.go └── projector │ ├── main.go │ ├── regulator_ce.go │ └── regulator_ee.go ├── collatejson ├── .gitignore ├── LICENSE ├── README.rst ├── TODO.rst ├── array.go ├── codec.go ├── codec_test.go ├── collate.go ├── collate_test.go ├── desc.go ├── desc_test.go ├── docs │ └── data2bin.pdf ├── n1qlcollate_test.go ├── string.go ├── testdata │ ├── arrays │ ├── arrays.ref │ ├── basics │ ├── basics.ref │ ├── empty │ ├── empty.ref │ ├── numbers │ ├── numbers.ref │ ├── objects │ ├── objects.ref │ ├── sortorder │ ├── sortorder.ref │ ├── strings │ └── strings.ref ├── tools │ ├── checkfiles │ │ └── checkfiles.go │ └── validate │ │ ├── json.prod │ │ ├── validate.go │ │ └── web2 ├── unicode.go ├── unicode_test.go └── util │ └── sort.go ├── common ├── aggr.go ├── alternate_shard_id.go ├── buffer_pool.go ├── build_mode.go ├── cbauthutil │ └── config_refresh.go ├── cluster_info.go ├── cluster_info_lite.go ├── cluster_info_provider.go ├── collections │ ├── collection_defs.go │ ├── leb128.go │ └── leb128_test.go ├── config.go ├── const.go ├── counter.go ├── dcp_seqno.go ├── defn_holders.go ├── deployment_model.go ├── evaluator.go ├── event_notifier.go ├── index.go ├── internal_version.go ├── json │ ├── LICENSE │ ├── README.md │ ├── bench_test.go │ ├── decode.go │ ├── decode_test.go │ ├── encode.go │ ├── encode_test.go │ ├── example_test.go │ ├── fold.go │ ├── fold_test.go │ ├── indent.go │ ├── number_test.go │ ├── scanner.go │ ├── scanner_test.go │ ├── stream.go │ ├── stream_test.go │ ├── tagkey_test.go │ ├── tags.go │ ├── tags_test.go │ └── testdata │ │ └── code.json.gz ├── key_partition_container.go ├── limits_config.go ├── memstat.go ├── metakv_gsi.go ├── mutation.go ├── mutation_test.go ├── op_monitor.go ├── op_monitor_test.go ├── partition_container.go ├── pause_resume_defs.go ├── permission_cache.go ├── protoutil │ └── proto_helpers.go ├── queryutil │ └── queryutil.go ├── random_sample.go ├── random_sample_test.go ├── rebalance_defs.go ├── regulator_ce.go ├── regulator_ee.go ├── retry_helper.go ├── router.go ├── sample.go ├── scan_result.go ├── services_notifier.go ├── settings.go ├── shard_stats.go ├── sort.go ├── stats.go ├── stats_test.go ├── storage_mode.go ├── timestamp.go ├── timestamp_test.go ├── util.go ├── util_test.go ├── uuid.go ├── vbuckets.go ├── vbuckets_test.go ├── vector_encoding.go ├── vector_encoding_test.go ├── vector_quantizer.go └── vector_quantizer_test.go ├── conversions └── convert.go ├── dataport ├── app.go ├── client.go ├── client_test.go ├── endpoint.go ├── endpoint_buffer.go ├── protobuf.go ├── protobuf_test.go ├── server.go ├── server_test.go └── transport_test.go ├── dcp ├── client.go ├── client_collections.go ├── client_test.go ├── conn_pool.go ├── conn_pool_test.go ├── examples │ ├── basic │ │ └── basic.go │ ├── bucketlist │ │ └── bucketlist.go │ ├── failoverlog │ │ └── failoverlog.go │ ├── hello │ │ └── hello.go │ ├── hello_observe │ │ └── hello_observe.go │ ├── hello_tap │ │ └── hello_tap.go │ ├── loadfile │ │ └── loadfile.go │ ├── upr_bench │ │ └── bench.go │ ├── upr_feed │ │ └── feed.go │ └── upr_restart │ │ └── restart.go ├── pools.go ├── pools_test.go ├── random_scanner.go ├── tap.go ├── transport │ ├── client │ │ ├── atomic_mutation_queue.go │ │ ├── dcp_feed.go │ │ ├── example │ │ │ └── tap_example.go │ │ ├── mc.go │ │ ├── mc_test.go │ │ ├── tap_feed.go │ │ ├── tap_feed_test.go │ │ └── transport.go │ ├── debug │ │ └── mcdebug.go │ ├── gocache │ │ ├── gocache.go │ │ └── mc_storage.go │ ├── mc_constants.go │ ├── mc_constants_test.go │ ├── mc_req.go │ ├── mc_req_test.go │ ├── mc_res.go │ ├── mc_res_test.go │ ├── seq_order.go │ ├── server │ │ ├── mc_conn_handler.go │ │ └── server_test.go │ ├── tap.go │ └── tap_test.go ├── upr.go ├── util.go ├── util_test.go ├── vbmap.go └── vbmap_test.go ├── docs ├── DeploymentPlan.md ├── JanCodeDrop.md ├── README.md ├── comparison.rst ├── configuration.rst ├── dcp_requirements.txt ├── design │ ├── README.md │ ├── images │ │ ├── Bootstrap.svg │ │ ├── DeleteWorkflow.svg │ │ ├── Deployment.svg │ │ ├── InitialBuild_Load.svg │ │ ├── InitialBuild_Prepare.svg │ │ ├── InsertWorkflow.svg │ │ ├── ScanWorkflow.svg │ │ ├── SystemDiagram.svg │ │ ├── SystemDiagramDDL.svg │ │ ├── SystemDiagramMultipleBuckets.svg │ │ └── SystemDiagramScan.svg │ ├── markdown │ │ ├── bootstrap.md │ │ ├── deployment.md │ │ ├── index_manager.md │ │ ├── index_manager_design.md │ │ ├── indexer.md │ │ ├── initialbuild.md │ │ ├── invariant.md │ │ ├── mutation.md │ │ ├── projector.md │ │ ├── projector_design.md │ │ ├── query.md │ │ ├── router.md │ │ ├── system.md │ │ ├── system_query.md │ │ └── terminology.md │ ├── other │ │ ├── CB_Cluster_2iQuery_Administration.pdf │ │ └── presentation.pdf │ ├── overview.md │ └── yed_files │ │ ├── Bootstrap.graphml │ │ ├── DeleteWorkflow.graphml │ │ ├── Deployment.graphml │ │ ├── InitialBuild_Load.graphml │ │ ├── InitialBuild_Prepare.graphml │ │ ├── InsertWorkflow.graphml │ │ ├── ScanWorkflow.graphml │ │ ├── SystemDiagram.graphml │ │ ├── SystemDiagramDDL.graphml │ │ ├── SystemDiagramMultipleBuckets.graphml │ │ └── SystemDiagramScan.graphml ├── metrics_metadata.json ├── n1ql_integration.rst ├── n1ql_scan.rst ├── projector-sizing.txt ├── querysdk.rst ├── restful-2i.md ├── settings_stats.md └── stress-projector.md ├── fdb ├── .gitignore ├── LICENSE ├── README.md ├── advlock.go ├── commit.go ├── commit_test.go ├── config.go ├── doc.go ├── docs.go ├── error.go ├── error.sh ├── examples │ └── custom_comparator │ │ ├── comparator.c │ │ ├── comparator.go │ │ ├── comparator.h │ │ ├── comparator_test.go │ │ └── other.go ├── file.go ├── forestdb.go ├── forestdb_test.go ├── info.go ├── iterator.go ├── iterator_test.go ├── kv.go ├── kv_test.go ├── log.go ├── snapshot_marker.go └── snapshot_marker_test.go ├── indexer ├── api.go ├── array.go ├── autofailover_service_manager.go ├── bhive_slice.go ├── bhive_slice_community.go ├── buf_pool.go ├── cluster_manager_agent.go ├── common.go ├── compaction_manager.go ├── constant.go ├── cpu.go ├── cpu_throttle.go ├── ddl_service_manager.go ├── error.go ├── flusher.go ├── forestdb_iterator.go ├── forestdb_iterator_test.go ├── forestdb_slice_writer.go ├── forestdb_snapshot.go ├── forestdb_snapshot_reader.go ├── generic_service_manager.go ├── index_entry.go ├── index_entry_test.go ├── index_reader.go ├── index_snapshot.go ├── index_writer.go ├── indexer.go ├── kv_sender.go ├── map_holder.go ├── master_service_manager.go ├── memdb_slice_impl.go ├── memdb_slice_test.go ├── message.go ├── metering_community.go ├── metering_enterprise.go ├── mock_slice.go ├── mutation.go ├── mutation_manager.go ├── mutation_queue_atomic.go ├── mutation_queue_atomic_test.go ├── partition_instance.go ├── pause_copier_community.go ├── pause_copier_enterprise.go ├── pause_objutil.go ├── pause_pauser.go ├── pause_resumer.go ├── pause_service_manager.go ├── plasma_community.go ├── plasma_enterprise.go ├── plasma_slice.go ├── queue.go ├── rebalance_defs.go ├── rebalance_provider.go ├── rebalance_service_manager.go ├── rebalancer.go ├── request_handler.go ├── request_handler_cache.go ├── restapi.go ├── row_heap.go ├── row_heap_test.go ├── scan_coordinator.go ├── scan_coordinator_mocker.go ├── scan_coordinator_test.go ├── scan_pipeline.go ├── scan_pipeline_vector.go ├── scan_pipeline_vector_test.go ├── scan_protocol.go ├── scan_request.go ├── scan_request_test.go ├── scan_scatter.go ├── sched_index_creator.go ├── serverless_service_manager.go ├── settings.go ├── shard_rebalancer.go ├── shard_transfer_manager.go ├── shard_transfer_manager_community.go ├── slab_manager.go ├── slice.go ├── slice_container.go ├── snapshot.go ├── snapshot_container.go ├── snapshot_map.go ├── stats_manager.go ├── storage_manager.go ├── storage_stats_manager_community.go ├── storage_stats_manager_enterprise.go ├── stream_reader.go ├── stream_state.go ├── timekeeper.go ├── timestamp.go ├── util.go ├── vector.go └── waiters_map.go ├── iowrap └── io_wrappers.go ├── logging ├── logging.go ├── logging_test.go └── systemevent │ └── system_event.go ├── manager ├── client │ ├── defn.go │ └── metadata_provider.go ├── common │ ├── token.go │ └── topology.go ├── constant.go ├── coordinator.go ├── env.go ├── error.go ├── event_mgr.go ├── lifecycle.go ├── manager.go ├── meta_repo.go ├── metadata.go ├── restore.go ├── test │ ├── README.md │ ├── config.json │ ├── coordinator_test.go │ ├── event_test.go │ ├── manager_test.go │ ├── meta_repo_test.go │ ├── metadata_provider_test.go │ ├── request_handler_test.go │ ├── restore │ │ └── testclient.go │ └── util │ │ └── util.go ├── topology.go └── watcher.go ├── memdb ├── README.md ├── file.go ├── item.go ├── iterator.go ├── memdb.go ├── memdb_test.go ├── nodelist.go ├── nodelist_test.go ├── nodetable │ ├── table.go │ └── table_test.go ├── skiplist │ ├── access_barrier.go │ ├── builder.go │ ├── item.go │ ├── iterator.go │ ├── merger.go │ ├── merger_test.go │ ├── node.go │ ├── node_alloc_amd64.go │ ├── node_amd64.go │ ├── skiplist.go │ ├── skiplist_test.go │ └── stats.go ├── system.go └── system_windows.go ├── mock └── mockcbauth │ └── mock_cbauth.go ├── natsort └── sort.go ├── pipeline ├── block_buffer.go ├── data_pipeline.go ├── pipeline.go └── pipeline_test.go ├── planner ├── executor.go ├── index_usage.go ├── indexer_constraint.go ├── indexer_node.go ├── planner.go ├── proxy.go ├── random_placement.go ├── sample │ ├── index.json │ ├── mixedWorkload.json │ ├── uniformPlan.json │ └── uniformWorkload.json ├── shard_dealer.go ├── shard_dealer_test.go ├── sim_test.go ├── simulator.go ├── solution.go ├── usage_based_cost_method.go └── util.go ├── platform ├── platform.go └── platform_windows.go ├── projector ├── adminport.go ├── client │ ├── client.go │ └── client_test.go ├── dcp.go ├── dcp_seqno_local.go ├── engine.go ├── fake_upr.go ├── feed.go ├── kvdata.go ├── memThrottler │ └── throttler.go ├── memmanager │ ├── collect_stats.go │ └── manager.go ├── projector.go ├── stats_manager.go ├── subscriber.go ├── util.go ├── vbucket.go ├── watcher.go └── worker.go ├── protobuf ├── data │ ├── mutation.go │ └── mutation.proto ├── projector │ ├── common.go │ ├── common.proto │ ├── eg_default4i.go │ ├── index.go │ ├── index.proto │ ├── n1ql_evaluate.go │ ├── n1ql_evaluate_test.go │ ├── partn_key.go │ ├── partn_key.proto │ ├── partn_single.go │ ├── partn_single.proto │ ├── partn_tp.go │ ├── partn_tp.proto │ ├── projector.go │ └── projector.proto └── query │ ├── codec.go │ ├── encoder.go │ ├── query.go │ └── query.proto ├── querycmd └── docmd.go ├── queryport ├── app.go ├── client │ ├── cbq_client.go │ ├── client.go │ ├── conn_pool.go │ ├── conn_pool_test.go │ ├── defs.go │ ├── error.go │ ├── meta_client.go │ ├── queue.go │ ├── scan_client.go │ ├── scatter.go │ └── settings.go ├── n1ql │ ├── monitor.go │ ├── secondary_index.go │ └── secondary_index_test.go ├── server.go └── server_test.go ├── security ├── tls.go ├── tls_setting.go └── tls_tools.go ├── stats ├── avgvar.go ├── histogram.go ├── timing.go └── values.go ├── stubs ├── build_community.go ├── build_enterprise.go ├── nitro │ ├── bhive │ │ ├── bhive_community.go │ │ └── bhive_enterprise.go │ ├── mm │ │ ├── mm_community.go │ │ └── mm_enterprise.go │ └── plasma │ │ ├── plasma_community.go │ │ └── plasma_enterprise.go └── regulator │ └── regulator_ce.go ├── system └── systemStats.go ├── testcode ├── const.go ├── options.go ├── test_action.go └── test_action_ci.go ├── tests ├── README.markdown ├── ci │ ├── config │ │ ├── TestPerfScanLatency_Lookup_StaleFalse.json │ │ ├── TestPerfScanLatency_Lookup_StaleOk.json │ │ ├── TestPerfScanLatency_Range_StaleFalse.json │ │ └── TestPerfScanLatency_Range_StaleOk.json │ ├── scripts │ │ ├── .gitignore │ │ ├── build │ │ ├── builder │ │ ├── ci_setup.md │ │ ├── del-failed │ │ ├── dobuild │ │ ├── domain │ │ ├── dotest │ │ ├── dowatch │ │ ├── kick │ │ ├── perfstat.go │ │ ├── redo │ │ ├── setup │ │ │ ├── apache2.conf │ │ │ ├── apache_server.dockerfile │ │ │ ├── ci_machine.dockerfile │ │ │ ├── container-runner.sh │ │ │ ├── htaccess │ │ │ ├── run_ci_dc.yaml │ │ │ ├── run_standalone_dc.yaml │ │ │ └── standalone-runner.sh │ │ └── setupvm │ └── skip.txt ├── config │ ├── blrcluster_conf.json │ ├── build_validation.json │ ├── buildtest_fdb.json │ ├── buildtest_memdb.json │ ├── buildtest_plasma.json │ ├── clusterrun_conf.json │ ├── remote1node_conf.json │ ├── remoteSc1node_conf.json │ └── remoteWindows1node_conf.json ├── framework │ ├── backuprestore │ │ └── util.go │ ├── clusterutility │ │ └── cluster_setup.go │ ├── common │ │ ├── constants.go │ │ ├── memdb_slice_mock.go │ │ ├── types.go │ │ └── util.go │ ├── datautility │ │ ├── jsondocscanner.go │ │ └── jsonloader.go │ ├── kvutility │ │ ├── kvcollectionops.go │ │ └── kvdataoperations.go │ ├── secondaryindex │ │ ├── forestdb_file.go │ │ ├── n1qlclient.go │ │ ├── secondaryindexarray.go │ │ ├── secondaryindexmanagement.go │ │ ├── secondaryindexscan.go │ │ └── secondaryindexstats.go │ └── validation │ │ └── validation.go ├── functionaltests │ ├── cluster_setup.go │ ├── common_test.go │ ├── runtest_clusterrun.sh │ ├── set00_indexoperations_test.go │ ├── set01_indextypes_test.go │ ├── set02_mutations_test.go │ ├── set03_planner_test.go │ ├── set04_restful_test.go │ ├── set05_misc_test.go │ ├── set06_datashapes_test.go │ ├── set07_system_test.go │ ├── set08_compaction_stats_settings_test.go │ ├── set09_arrayindex_test.go │ ├── set10_n1qlclient_test.go │ ├── set11_spock_test.go │ ├── set12_spock_desc_test.go │ ├── set13_groups_aggrs_test.go │ ├── set14_rebalance_test.go │ ├── set15_alter_test.go │ ├── set16_partition_test.go │ ├── set17_collection_test.go │ ├── set18_convert_test.go │ ├── set19_ephemeral_bucket_test.go │ ├── set20_scheduled_index_test.go │ ├── set21_flatten_arrayindex_test.go │ ├── set22_oso_test.go │ ├── set23_missing_leading_key_test.go │ ├── set24_shard_affinity_test.go │ ├── set25_vector_index_test.go │ ├── set26_vector_index_rebalance_codebook_test.go │ ├── set27_bhive_index_test.go │ ├── set27_vector_xattr_test.go │ ├── set28_vector_backup_restore_test.go │ ├── set29_bhive_index_rebalance_codebook_test.go │ ├── set99_storage_stats_test.go │ └── setup.sh ├── largedatatests │ ├── common_test.go │ ├── concurrency_test.go │ └── perf_test.go ├── perftests │ ├── README.markdown │ ├── TestPerfScanLatency_Lookup_StaleFalse.json │ ├── TestPerfScanLatency_Lookup_StaleOk.json │ ├── TestPerfScanLatency_Range_StaleFalse.json │ ├── TestPerfScanLatency_Range_StaleOk.json │ ├── common_test.go │ └── perfsanity_test.go ├── plasmatests │ ├── README-smat.md │ ├── crash_test.go │ ├── plasma_diag.go │ ├── plasma_parallel_test.go │ ├── plasmatests.go │ └── smat_generate_test.go ├── serverlesstests │ ├── cluster_setup.go │ ├── common_test.go │ ├── ftr.http │ ├── pause.http │ ├── set00_indexoperations_test.go │ ├── set01_rebalance_test.go │ ├── set02_rebalance_panic_test.go │ ├── set03_rebalance_cancel_test.go │ ├── set05_pause_resume_test.go │ ├── set06_backup_restore_test.go │ └── set99_rebalance_cleanup_test.go └── testdata │ ├── Users100.txt.gz │ ├── Users_Template │ ├── cities │ ├── invalidjson │ ├── json.prod │ ├── numbertests │ ├── planner │ ├── greedy │ │ ├── new_equiv_index.json │ │ ├── new_equiv_index_1_replica.json │ │ ├── new_equiv_index_2_replica.json │ │ ├── new_index_1.json │ │ ├── new_index_with_1_replica.json │ │ ├── new_index_with_2_replicas.json │ │ └── topologies │ │ │ ├── 1_empty_1_10pct_filled_node_1_sg.json │ │ │ ├── 1_empty_1_30pct_filled_node_1_sg.json │ │ │ ├── 1_empty_1_60pct_filled_node_1_sg.json │ │ │ ├── 1_empty_2_non_empty_nodes_1_sg.json │ │ │ ├── 1_empty_2_non_empty_nodes_2_sg.json │ │ │ ├── 2_empty_1_non_empty_nodes_1_sg.json │ │ │ ├── 2_empty_1_non_empty_nodes_2_sg.json │ │ │ ├── 3_empty_nodes_1_sg.json │ │ │ ├── 3_nodes_equiv_index.json │ │ │ ├── 3_nodes_equiv_index_1.json │ │ │ ├── 3_non_empty_nodes_1_sg.json │ │ │ ├── 3_non_empty_nodes_2_sg.json │ │ │ ├── 5_nodes_3_sg_equiv_map_0001.json │ │ │ ├── 5_nodes_3_sg_equiv_map_10101.json │ │ │ ├── 6_nodes_3_sg_equiv_map_101010.json │ │ │ └── 8_nodes_4_sg_equiv_map_11000000.json │ ├── index │ │ ├── array-small-5-2.json │ │ ├── primary-small-5-2.json │ │ ├── replica-3-constraint.json │ │ ├── replica-3.json │ │ ├── small-2M-5-1.json │ │ └── small-2M-6-1.json │ ├── plan │ │ ├── empty-1-zone.json │ │ ├── empty-2-zone.json │ │ ├── empty-3-zone.json │ │ ├── identical-8-0.json │ │ ├── identical-8-1.json │ │ ├── min-memory-empty-plan.json │ │ ├── min-memory-plan.json │ │ ├── min-memory-replica-plan.json │ │ ├── mixed-small-medium-30-3.json │ │ ├── replica-3-zone.json │ │ ├── replica-repair-1-zone.json │ │ ├── replica-repair-2-zone.json │ │ ├── replica-repair-unbalanced-sg-1.json │ │ ├── replica-repair-unbalanced-sg.json │ │ ├── scale-identical-8-0.json │ │ ├── scale-replica-3-zone.json │ │ ├── travel-sample-plan.json │ │ ├── uniform-small-10-3.json │ │ └── uniform-small-30-3-90.json │ ├── scaleup │ │ ├── heterogenous-6-2_3-nodes-1-scaled_1-sg.json │ │ ├── heterogenous-6-2_3-nodes-1-scaled_2-sg.json │ │ ├── heterogenous-6-2_3-nodes-1-scaled_2-sg_scaled-alone.json │ │ ├── heterogenous-6-2_3-nodes-2-scaled_1-sg.json │ │ ├── heterogenous_3-nodes-1-scaled_replica.json │ │ ├── heterogenous_3-nodes-2-scaled_replica.json │ │ ├── heterogenous_3_nodes_equiv_index.json │ │ └── heterogenous_3_nodes_equiv_index_1.json │ ├── servergroup │ │ └── sg-3_8_nodes-3_empty_nodes.json │ ├── shard_assignment │ │ ├── 2_nodes_1_new_index.json │ │ ├── 2_nodes_1_new_index_2_replicas.json │ │ ├── 2_nodes_1_new_index_node_full.json │ │ ├── 2_nodes_2_new_index.json │ │ ├── 2_nodes_2_new_index_2_replicas.json │ │ ├── 2_nodes_2_replicas_2_partns.json │ │ ├── 2_nodes_3_new_index.json │ │ ├── 2_nodes_3_new_index_2_replicas.json │ │ ├── 3_nodes_prune_unwanted_slots.json │ │ ├── 5_nodes_1_new_index_3_replicas_2_full_nodes.json │ │ ├── 5_nodes_1_new_index_3_replicas_different_replica_order.json │ │ ├── 5_nodes_1_new_index_3_replicas_prune_replica_shard.json │ │ ├── 5_nodes_1_new_index_more_shard_replicas.json │ │ └── 5_nodes_1_new_partn_index_slot_grouping.json │ ├── shard_assignment_rebalance │ │ ├── mixed_mode_add_failover_heavy_node_1_replica.json │ │ ├── mixed_mode_add_failover_light_node_1_replica.json │ │ ├── mixed_mode_eject_72_node_cyclic_dependency.json │ │ ├── mixed_mode_eject_72_node_diff_replica.json │ │ ├── mixed_mode_eject_72_node_distributed_replicas.json │ │ ├── mixed_mode_eject_72_node_same_replica.json │ │ └── mixed_mode_swap_node_1_replica.json │ ├── tenantaware │ │ ├── new_index_1.json │ │ ├── new_index_2.json │ │ └── topology │ │ │ ├── 1_empty_node_1_sg.json │ │ │ ├── 1_non_empty_node_1_sg.json │ │ │ ├── 2_empty_nodes_1_sg.json │ │ │ ├── 2_non_empty_node_1_sg.json │ │ │ ├── 4_empty_nodes_2_sg.json │ │ │ ├── 4_non_empty_nodes_2_sg_a.json │ │ │ ├── 4_non_empty_nodes_2_sg_a_rebal.json │ │ │ ├── 4_non_empty_nodes_2_sg_b.json │ │ │ ├── 4_non_empty_nodes_2_sg_b_rebal.json │ │ │ ├── 4_non_empty_nodes_2_sg_c.json │ │ │ ├── 4_non_empty_nodes_2_sg_c_rebal.json │ │ │ ├── 4_non_empty_nodes_2_sg_d.json │ │ │ ├── 4_non_empty_nodes_2_sg_d_rebal.json │ │ │ ├── 4_non_empty_nodes_2_sg_e.json │ │ │ ├── 4_non_empty_nodes_2_sg_e_rebal.json │ │ │ ├── 4_non_empty_nodes_2_sg_f_rebal.json │ │ │ ├── 4_non_empty_nodes_2_sg_g_rebal.json │ │ │ ├── 4_non_empty_nodes_2_sg_h_rebal.json │ │ │ ├── 6_non_empty_nodes_3_sg_a.json │ │ │ ├── 6_non_empty_nodes_3_sg_b.json │ │ │ ├── 6_non_empty_nodes_3_sg_c.json │ │ │ ├── 6_non_empty_nodes_3_sg_d.json │ │ │ ├── 6_non_empty_nodes_3_sg_e.json │ │ │ ├── 6_non_empty_nodes_3_sg_f.json │ │ │ ├── defrag │ │ │ ├── 4_non_empty_nodes_3_sg_a.json │ │ │ ├── 4_non_empty_nodes_3_sg_a_out.json │ │ │ ├── 4_non_empty_nodes_3_sg_b.json │ │ │ ├── 4_non_empty_nodes_3_sg_b_out.json │ │ │ ├── 4_non_empty_nodes_3_sg_d.json │ │ │ ├── 4_non_empty_nodes_3_sg_d_out.json │ │ │ ├── 4_non_empty_nodes_3_sg_e.json │ │ │ ├── 4_non_empty_nodes_3_sg_e_out.json │ │ │ ├── 4_non_empty_nodes_3_sg_f.json │ │ │ ├── 4_non_empty_nodes_3_sg_f_out.json │ │ │ ├── 4_non_empty_nodes_3_sg_g.json │ │ │ ├── 4_non_empty_nodes_3_sg_g_out.json │ │ │ ├── 4_non_empty_nodes_3_sg_h.json │ │ │ ├── 4_non_empty_nodes_3_sg_h_out.json │ │ │ ├── 4_non_empty_nodes_3_sg_i.json │ │ │ ├── 4_non_empty_nodes_3_sg_i_out.json │ │ │ ├── 4_non_empty_nodes_3_sg_l.json │ │ │ ├── 4_non_empty_nodes_3_sg_l_out.json │ │ │ ├── 6_non_empty_nodes_3_sg_c.json │ │ │ ├── 6_non_empty_nodes_3_sg_c_out.json │ │ │ ├── 6_non_empty_nodes_3_sg_j.json │ │ │ ├── 6_non_empty_nodes_3_sg_j_out.json │ │ │ ├── 6_non_empty_nodes_3_sg_k.json │ │ │ └── 6_non_empty_nodes_3_sg_k_out.json │ │ │ ├── rebalance │ │ │ ├── 6_non_empty_nodes_3_sg_a.json │ │ │ ├── 6_non_empty_nodes_3_sg_a_out.json │ │ │ ├── 6_non_empty_nodes_3_sg_b.json │ │ │ ├── 6_non_empty_nodes_3_sg_b_out.json │ │ │ ├── 6_non_empty_nodes_3_sg_c.json │ │ │ ├── 6_non_empty_nodes_3_sg_c_out.json │ │ │ ├── 6_non_empty_nodes_3_sg_d.json │ │ │ ├── 6_non_empty_nodes_3_sg_d_out.json │ │ │ ├── 6_non_empty_nodes_3_sg_e.json │ │ │ ├── 6_non_empty_nodes_3_sg_e_out.json │ │ │ ├── 6_non_empty_nodes_3_sg_h.json │ │ │ ├── 6_non_empty_nodes_3_sg_h_out.json │ │ │ ├── 6_non_empty_nodes_3_sg_i.json │ │ │ ├── 6_non_empty_nodes_3_sg_i_out.json │ │ │ ├── 8_non_empty_nodes_4_sg_f.json │ │ │ ├── 8_non_empty_nodes_4_sg_f_out.json │ │ │ ├── 8_non_empty_nodes_4_sg_g.json │ │ │ ├── 8_non_empty_nodes_4_sg_g_out.json │ │ │ ├── 8_non_empty_nodes_4_sg_h.json │ │ │ └── 8_non_empty_nodes_4_sg_h_out.json │ │ │ ├── replica_repair │ │ │ ├── 4_non_empty_nodes_2_sg_e.json │ │ │ ├── 4_non_empty_nodes_2_sg_e_out.json │ │ │ ├── 8_non_empty_nodes_4_sg_a.json │ │ │ ├── 8_non_empty_nodes_4_sg_a_out.json │ │ │ ├── 8_non_empty_nodes_4_sg_b.json │ │ │ ├── 8_non_empty_nodes_4_sg_b_out.json │ │ │ ├── 8_non_empty_nodes_4_sg_c.json │ │ │ ├── 8_non_empty_nodes_4_sg_c_out.json │ │ │ ├── 8_non_empty_nodes_4_sg_d.json │ │ │ └── 8_non_empty_nodes_4_sg_d_out.json │ │ │ ├── resume │ │ │ ├── resume_cluster_a.json │ │ │ ├── resume_cluster_a_out.json │ │ │ ├── resume_cluster_b.json │ │ │ ├── resume_cluster_b_out.json │ │ │ ├── resume_cluster_c.json │ │ │ ├── resume_cluster_d.json │ │ │ ├── resume_tenant_a.json │ │ │ ├── resume_tenant_b.json │ │ │ ├── resume_tenant_c.json │ │ │ └── resume_tenant_d.json │ │ │ ├── scaledown │ │ │ ├── 4_non_empty_nodes_2_sg_k.json │ │ │ ├── 4_non_empty_nodes_2_sg_k_out.json │ │ │ ├── 8_non_empty_nodes_4_sg_a.json │ │ │ ├── 8_non_empty_nodes_4_sg_a_out.json │ │ │ ├── 8_non_empty_nodes_4_sg_b.json │ │ │ ├── 8_non_empty_nodes_4_sg_c.json │ │ │ ├── 8_non_empty_nodes_4_sg_c_out.json │ │ │ ├── 8_non_empty_nodes_4_sg_d.json │ │ │ ├── 8_non_empty_nodes_4_sg_e.json │ │ │ ├── 8_non_empty_nodes_4_sg_e_out.json │ │ │ ├── 8_non_empty_nodes_4_sg_f.json │ │ │ ├── 8_non_empty_nodes_4_sg_g.json │ │ │ ├── 8_non_empty_nodes_4_sg_h.json │ │ │ ├── 8_non_empty_nodes_4_sg_i.json │ │ │ └── 8_non_empty_nodes_4_sg_j.json │ │ │ └── swap │ │ │ ├── 2_non_empty_nodes_1_sg_f.json │ │ │ ├── 2_non_empty_nodes_1_sg_f_out.json │ │ │ ├── 2_non_empty_nodes_1_sg_g.json │ │ │ ├── 8_non_empty_nodes_4_sg_a.json │ │ │ ├── 8_non_empty_nodes_4_sg_a_out.json │ │ │ ├── 8_non_empty_nodes_4_sg_b.json │ │ │ ├── 8_non_empty_nodes_4_sg_b_out.json │ │ │ ├── 8_non_empty_nodes_4_sg_c.json │ │ │ ├── 8_non_empty_nodes_4_sg_c_out.json │ │ │ ├── 8_non_empty_nodes_4_sg_d.json │ │ │ ├── 8_non_empty_nodes_4_sg_d_out.json │ │ │ ├── 8_non_empty_nodes_4_sg_e.json │ │ │ └── 8_non_empty_nodes_4_sg_e_out.json │ └── workload │ │ ├── mixed-all-30-3.json │ │ ├── mixed-small-medium-30-3.json │ │ ├── uniform-medium-10-3.json │ │ ├── uniform-medium-30-3.json │ │ ├── uniform-small-10-1.json │ │ ├── uniform-small-10-3.json │ │ └── uniform-small-30-3.json │ ├── proj-gctrace │ ├── gctrace-backfill │ └── gctrace-idle │ ├── projects.json.bz2 │ ├── projects.prod │ ├── propernames │ ├── statistics.json │ ├── users.json.bz2 │ ├── users.prod │ └── web2.bz2 ├── tools ├── cluster_ops │ ├── Makefile │ ├── Readme.md │ ├── init_cluster.go │ └── main.go ├── codec │ └── encode.go ├── datapath │ └── datapath.go ├── dumpconfig │ └── dumpconfig.go ├── fdb │ ├── fdb_slice_thrpt.go │ ├── fdb_throughput.c │ └── fdb_throughput.go ├── loadgen │ ├── companies │ ├── loadgen.go │ ├── projects.prod │ ├── propernames │ ├── users.prod │ └── util.go ├── logd │ └── main.go ├── multibuckets │ └── multibuckets.go ├── n1qlexpr │ ├── doc.json │ └── n1ql_expr.go ├── n1qlperf │ ├── README.markdown │ ├── config.go │ ├── executor.go │ ├── lookup.json │ ├── main.go │ └── n1qlperf.sh ├── planner │ └── loganalyse │ │ └── main.go ├── querycmd │ ├── bench.go │ ├── benchtimeit.go │ ├── bufferedscan.go │ ├── bufferedscan.sh │ ├── consistency.go │ ├── main.go │ ├── mb13339.go │ ├── mb14786.go │ ├── mb16115.go │ ├── memdb.go │ ├── primarykeys.go │ ├── sanity.go │ └── scan_retry.go ├── randdocs │ ├── Makefile │ ├── cmd │ │ ├── config.json │ │ └── main.go │ ├── generate_json.go │ ├── randdocs.go │ ├── siftsmall.go │ └── siftsmall │ │ ├── siftsmall_base.fvecs │ │ ├── siftsmall_groundtruth.ivecs │ │ ├── siftsmall_learn.fvecs │ │ └── siftsmall_query.fvecs ├── recovery │ └── recovery.go ├── restful │ └── restful.go ├── streamwait │ └── streamwait.go ├── support │ └── parseStorageStats.go ├── ts │ └── ts.go └── upr │ └── upr.go ├── transport ├── transport.go ├── transport_flags.go └── util.go └── vector ├── codebook ├── codebook.go └── mock_codebook.go ├── codebook_impl.go ├── codebook_ivfpq.go ├── codebook_ivfpq_test.go ├── codebook_ivfsq.go ├── codebook_ivfsq_test.go ├── faiss ├── autotune.go ├── faiss.go ├── index.go ├── index_io.go └── index_ivf.go ├── indexivf.go ├── indexivfpq_test.go ├── util ├── index_evaluator.go └── index_evaluator_test.go └── utils.go /.gitignore: -------------------------------------------------------------------------------- 1 | *.log 2 | *.swp 3 | *.swo 4 | *.pyc 5 | .gdb 6 | .vimsession 7 | *.idat.* 8 | .DS_Store 9 | secondary/tests/testdata/Users10k.txt.gz 10 | secondary/triage 11 | secondary/bin/indexer 12 | secondary/bin/projector 13 | secondary/cmd/indexer/indexer 14 | secondary/cmd/projector/projector 15 | secondary/protobuf/data/*.pb.go 16 | secondary/protobuf/projector/*.pb.go 17 | secondary/protobuf/query/*.pb.go 18 | -------------------------------------------------------------------------------- /LICENSE.txt: -------------------------------------------------------------------------------- 1 | Source code in this repository is licensed under various licenses. The 2 | Business Source License 1.1 (BSL) is one such license. Each file indicates in 3 | a section at the beginning of the file the name of the license that applies to 4 | it. All licenses used in this repository can be found in the top-level 5 | licenses directory. 6 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | indexing 2 | ======== 3 | 4 | Couchbase Secondary Indexes 5 | -------------------------------------------------------------------------------- /secondary/TODO.rst: -------------------------------------------------------------------------------- 1 | - https://issues.couchbase.com/browse/MB-13375 2 | (Range statistics, should be moved to backlog?) 3 | - GetFailoverLog() issue from goxdcr. 4 | - projector memory profiling, dynamic settings for `memprofile` 5 | - Upstream (DCP) record replay. 6 | - Integrate new transport with queryport. 7 | 8 | CBIDXT-282: queryport - multiplexing requests on the same connection. 9 | CBIDXT-289: queryport, the stream response error value is not 10 | comparable with defined error objects. 11 | -------------------------------------------------------------------------------- /secondary/audit/audit_descriptor.json: -------------------------------------------------------------------------------- 1 | { 2 | "version": 2, 3 | "module": "index", 4 | "events": [ 5 | { 6 | "id": 49152, 7 | "name": "HTTP 401: Unauthorized", 8 | "description": "Authentication is required to access the requested resource", 9 | "sync": false, 10 | "enabled": true, 11 | "mandatory_fields": { 12 | "timestamp": "", 13 | "real_userid": {"domain": "", "user": ""}, 14 | "remote": {"ip": "", "port": 1}, 15 | "local": {"ip": "", "port": 1}, 16 | 17 | "service": "", 18 | "method": "", 19 | "url": "" 20 | }, 21 | "optional_fields": { 22 | "message": "" 23 | } 24 | }, 25 | { 26 | "id": 49153, 27 | "name": "HTTP 403: Forbidden", 28 | "description": "The user does not have permission to access the requested resource", 29 | "sync": false, 30 | "enabled": true, 31 | "mandatory_fields": { 32 | "timestamp": "", 33 | "real_userid": {"domain": "", "user": ""}, 34 | "remote": {"ip": "", "port": 1}, 35 | "local": {"ip": "", "port": 1}, 36 | 37 | "service": "", 38 | "method": "", 39 | "url": "" 40 | }, 41 | "optional_fields": { 42 | "message": "" 43 | } 44 | } 45 | ] 46 | } 47 | -------------------------------------------------------------------------------- /secondary/bin/README.md: -------------------------------------------------------------------------------- 1 | 2 | build.sh will place binaries in this directory 3 | -------------------------------------------------------------------------------- /secondary/bin/cbq_sanity.sh: -------------------------------------------------------------------------------- 1 | #! /usr/bin/env bash 2 | 3 | URL=http://localhost:9499/query 4 | HEADER='-H Content-Type:text/plain' 5 | 6 | run_query(){ 7 | echo $1 8 | curl -s -X POST $HEADER --data "$1" $URL | $2 9 | echo 10 | } 11 | 12 | # cleanup 13 | run_query 'DROP PRIMARY INDEX ON default:default' 'grep "error\|cause\|msg\|success" -i' 14 | run_query 'DROP INDEX default:default.idx1' 'grep "error\|cause\|msg\|success" -i' 15 | run_query 'DROP PRIMARY INDEX ON default:`beer-sample`' 'grep "error\|cause\|msg\|success" -i' 16 | run_query 'DROP INDEX default:`beer-sample`.idx2' 'grep "error\|cause\|msg\|success" -i' 17 | 18 | # create-index 19 | run_query 'CREATE PRIMARY INDEX ON default:default USING GSI' 'grep "error\|cause\|msg\|success" -i' 20 | run_query 'CREATE INDEX idx1 ON default:default(age) USING GSI' 'grep "error\|cause\|msg\|success" -i' 21 | run_query 'CREATE PRIMARY INDEX ON default:`beer-sample` USING GSI' 'grep "error\|cause\|msg\|success" -i' 22 | run_query 'CREATE INDEX idx2 ON default:`beer-sample`(abv) USING GSI' 'grep "error\|cause\|msg\|success" -i' 23 | 24 | SLEEP=5 25 | echo "Sleeping for $SLEEP seconds ..." 26 | #sleep $SLEEP 27 | 28 | # query 29 | run_query 'SELECT \* FROM system:`indexes`' 'cat' 30 | run_query 'SELECT \* FROM default:default LIMIT 2' 'cat' 31 | run_query 'SELECT count(*) FROM default:default WHERE age > 10' 'cat' 32 | run_query 'SELECT count(*) FROM default:default WHERE gender = "female"' 'cat' 33 | run_query 'SELECT \* FROM default:`beer-sample` LIMIT 2' 'cat' 34 | run_query 'SELECT count(*) FROM default:`beer-sample` WHERE abv > 10' 'cat' 35 | run_query 'SELECT count(*) FROM default:`beer-sample` WHERE type = "beer"' 'cat' 36 | -------------------------------------------------------------------------------- /secondary/bin/projcalls.py: -------------------------------------------------------------------------------- 1 | import sys 2 | import re 3 | 4 | def checkcalls(logfile) : 5 | enter_re = re.compile(r'.*\[Info\].*PROJ.*(##[a-f0-9]*).*do.*".*') 6 | exit_re = re.compile(r'.*\[Info\].*PROJ.*(##[a-f0-9]*).*do.*return') 7 | tokens = {} 8 | for line in open(logfile).readlines() : 9 | m = enter_re.match(line) 10 | if m : 11 | token = m.groups()[0] 12 | tokens[token] = (True, line) 13 | m = exit_re.match(line) 14 | if m : 15 | token = m.groups()[0] 16 | del tokens[token] 17 | 18 | if len(tokens) > 0 : 19 | print("pending projector calls ...") 20 | keys = sorted([ int(token[2:], 16) for token in tokens.keys() ]) 21 | for key in keys : 22 | key = "##%s"%hex(key)[2:] 23 | print(key, tokens[key]) 24 | else : 25 | print("no pending calls") 26 | 27 | checkcalls(sys.argv[1]) 28 | -------------------------------------------------------------------------------- /secondary/cmd/cbindexperf/cfg/ConfigUsage.md: -------------------------------------------------------------------------------- 1 | Usage for Config Files 2 | ====================== 3 | 4 | ### idx_vector_{details}.json 5 | * load data using randdocs tool 6 | > randdocs -config ./cmd/config.json -genVectors 7 | * create index idx_vector something like below 8 | > CREATE INDEX idx_vector 9 | ON default(name, description VECTOR, city) 10 | WITH { "dimension":128, "description": "IVF256,PQ32x8", "similarity":"L2"}; 11 | * run cbindexperf command 12 | > cbindexperf -cluster 127.0.0.1:9001 -auth="Administrator:asdasd" -configfile ~/cbindexPerfConfig/scan_idx_vector_projection.json -resultfile result.json -logLevel info -------------------------------------------------------------------------------- /secondary/cmd/cbindexplan/sample/index.json: -------------------------------------------------------------------------------- 1 | [{"name" : "index1", 2 | "bucket" : "bucket2", 3 | "isPrimary" : false, 4 | "secExprs" : ["field1"], 5 | "isArrayIndex" : false, 6 | "replica" : 2, 7 | "numDoc" : 5000, 8 | "DocKeySize" : 200, 9 | "SecKeySize" : 200, 10 | "ArrKeySize" : 0, 11 | "ArrSize" : 0, 12 | "MutationRate" : 0, 13 | "ScanRate" : 0}, 14 | {"name" : "index2", 15 | "bucket" : "bucket2", 16 | "isPrimary" : false, 17 | "secExprs" : ["field2"], 18 | "isArrayIndex" : false, 19 | "replica" : 2, 20 | "numDoc" : 5000, 21 | "DocKeySize" : 200, 22 | "SecKeySize" : 200, 23 | "ArrKeySize" : 0, 24 | "ArrSize" : 0, 25 | "MutationRate" : 0, 26 | "ScanRate" : 0}, 27 | {"name" : "index3", 28 | "bucket" : "bucket3", 29 | "isPrimary" : false, 30 | "secExprs" : ["field3"], 31 | "isArrayIndex" : false, 32 | "replica" : 2, 33 | "numDoc" : 5000, 34 | "DocKeySize" : 200, 35 | "SecKeySize" : 200, 36 | "ArrKeySize" : 0, 37 | "ArrSize" : 0, 38 | "MutationRate" : 0, 39 | "ScanRate" : 0}] 40 | -------------------------------------------------------------------------------- /secondary/cmd/projector/regulator_ce.go: -------------------------------------------------------------------------------- 1 | // +build community 2 | 3 | package main 4 | 5 | func startRegulator() { 6 | // CE does not use the regulator 7 | } -------------------------------------------------------------------------------- /secondary/cmd/projector/regulator_ee.go: -------------------------------------------------------------------------------- 1 | // +build !community 2 | 3 | package main 4 | 5 | import ( 6 | "github.com/couchbase/cbauth/service" 7 | "github.com/couchbase/indexing/secondary/projector" 8 | regulator "github.com/couchbase/regulator/factory" 9 | ) 10 | 11 | func startRegulator() { 12 | // MB-52130: start the KV regulator (piggybacked in the projector process) 13 | // TODO: only start when in serverless 14 | regulator.InitKvRegulator(service.NodeID(projector.GetNodeUUID()), options.caFile, options.kvaddrs) 15 | } -------------------------------------------------------------------------------- /secondary/collatejson/.gitignore: -------------------------------------------------------------------------------- 1 | *.swp 2 | .vimsession 3 | -------------------------------------------------------------------------------- /secondary/collatejson/README.rst: -------------------------------------------------------------------------------- 1 | Collatejson library, written in golang, provides encoding and decoding function 2 | to transform JSON text into binary representation without loosing information. 3 | That is, 4 | 5 | * binary representation should preserve the sort order such that, sorting 6 | binary encoded json documents much match sorting by functions that parse 7 | and compare JSON documents. 8 | * it must be possible to get back the original document, in semantically 9 | correct form, from its binary representation. 10 | 11 | Notes: 12 | 13 | * items in a property object are sorted by its property name before they 14 | are compared with other property object. 15 | 16 | for api documentation and bench marking try, 17 | 18 | .. code-block:: bash 19 | 20 | godoc github.com/couchbaselabs/go-collatejson | less 21 | cd go-collatejson 22 | go test -test.bench=. 23 | 24 | to measure relative difference in sorting 100K elements using encoding/json 25 | library and this library try, 26 | 27 | .. code-block:: bash 28 | 29 | go test -test.bench=Sort 30 | 31 | examples/* contains reference sort ordering for different json elements. 32 | 33 | For known issues refer to `TODO.rst` 34 | -------------------------------------------------------------------------------- /secondary/collatejson/TODO.rst: -------------------------------------------------------------------------------- 1 | * codec.Encode() converts the input JSON to golang native before 2 | applying the encoding algorithm, 3 | if err := json.Unmarshal(text, &m); err != nil { 4 | return nil, err 5 | } 6 | explore possibilities to avoid a call to json.Unmarshal() 7 | 8 | * codec.Decode() returns JSON output, for couchbase 2i project 9 | the JSON string will always the following JSON format. 10 | [expr1, docid] - for simple key 11 | [expr1, expr2, ..., docid] - for composite key 12 | it would be a good optimization to implement a variant of Decode() 13 | that will return as, 14 | [expr1], docid - for simple key 15 | [expr1, expr2 ...], docid - for composite key 16 | 17 | * Jens' comments, 18 | * Also BTW, there’s a lot of appending of byte slices going on in 19 | collate.go. I suspect this is inefficient, allocating lots of small slices 20 | and then copying them together. It’s probably cheaper (and simpler) to use 21 | an io.Writer instead. 22 | * The CouchDB collation spec uses Unicode collation, and strangely enough 23 | the collation order for ASCII characters is not the same as ASCII order. I 24 | solved this by creating a mapping table that converts the bytes 0-127 into 25 | their priority in the Unicode collation. 26 | 27 | * create a new directory examples_len/ that contains the sorted list of json 28 | items without using `lenprefix` 29 | 30 | * Are we going to differentiate between float and integer ? 31 | Looks like dparval is parsing input json's number type as all float values. 32 | 33 | * JSON supports integers of arbitrary size ? If so how to do collation on 34 | big-integers ? 35 | Even big-integers are parsed are returned as float by dparval. 36 | 37 | * Encoding and decoding of utf8 strings. 38 | -------------------------------------------------------------------------------- /secondary/collatejson/docs/data2bin.pdf: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/couchbase/indexing/ba752cd53c7865086c4d99f312eafbe3d23982a9/secondary/collatejson/docs/data2bin.pdf -------------------------------------------------------------------------------- /secondary/collatejson/testdata/arrays: -------------------------------------------------------------------------------- 1 | [ "hello", "world" ] 2 | [ 10.2 ] 3 | [ false ] 4 | [ "hello", "world", "fine" ] 5 | [ "hello", "python" ] 6 | [ 20, "hello", "perl" ] 7 | [ "hello", 10.2, "php" ] 8 | [ [20], "hello", null, "go", "next-java" ] 9 | [ { "key1" : 10, "key2" : [ 20, 30 ] } ] 10 | [ 200, 300 ] 11 | [ 20, "hello", null, "go", "next-java" ] 12 | [ null ] 13 | [ true ] 14 | [ 10 ] 15 | [ "hello", "python", "perl" ] 16 | -------------------------------------------------------------------------------- /secondary/collatejson/testdata/arrays.ref: -------------------------------------------------------------------------------- 1 | [null] 2 | [false] 3 | [true] 4 | [10] 5 | [10.2] 6 | [20,"hello",null,"go","next-java"] 7 | [20,"hello","perl"] 8 | [200,300] 9 | ["hello",10.2,"php"] 10 | ["hello","python"] 11 | ["hello","python","perl"] 12 | ["hello","world"] 13 | ["hello","world","fine"] 14 | [[20],"hello",null,"go","next-java"] 15 | [{"key1":10,"key2":[20,30]}] 16 | -------------------------------------------------------------------------------- /secondary/collatejson/testdata/basics: -------------------------------------------------------------------------------- 1 | 0.000000000000000000001 2 | null 3 | 1 4 | false 5 | 10 6 | -2 7 | 5 8 | true 9 | -4 10 | 4.1 11 | null 12 | 0.2 13 | -0.2 14 | true 15 | -4.1 16 | -100.0000001 17 | null 18 | 100.0000001 19 | true 20 | false 21 | null 22 | "These" 23 | "are" 24 | "universal" 25 | "data" 26 | "structures." 27 | "Virtually" 28 | true 29 | false 30 | null 31 | "all" 32 | "modern" 33 | "programming" 34 | true 35 | false 36 | null 37 | "languages" 38 | "support" 39 | "them" 40 | "in" 41 | "one" 42 | "form" 43 | "or" 44 | "another." 45 | "It" 46 | "makes" 47 | "sense" 48 | "that" 49 | "a data" 50 | "format" 51 | "that" 52 | "is" 53 | "interchangeable" 54 | "with" 55 | "programming" 56 | "languages" 57 | "also" 58 | "be" 59 | "based" 60 | "on" 61 | "these" 62 | true 63 | false 64 | null 65 | "structures." 66 | -------------------------------------------------------------------------------- /secondary/collatejson/testdata/basics.ref: -------------------------------------------------------------------------------- 1 | null 2 | null 3 | null 4 | null 5 | null 6 | null 7 | null 8 | false 9 | false 10 | false 11 | false 12 | false 13 | true 14 | true 15 | true 16 | true 17 | true 18 | true 19 | -100.0000001 20 | -4.1 21 | -4 22 | -2 23 | -0.2 24 | 0.000000000000000000001 25 | 0.2 26 | 1 27 | 4.1 28 | 5 29 | 10 30 | 100.0000001 31 | "It" 32 | "These" 33 | "Virtually" 34 | "a data" 35 | "all" 36 | "also" 37 | "another." 38 | "are" 39 | "based" 40 | "be" 41 | "data" 42 | "form" 43 | "format" 44 | "in" 45 | "interchangeable" 46 | "is" 47 | "languages" 48 | "languages" 49 | "makes" 50 | "modern" 51 | "on" 52 | "one" 53 | "or" 54 | "programming" 55 | "programming" 56 | "sense" 57 | "structures." 58 | "structures." 59 | "support" 60 | "that" 61 | "that" 62 | "them" 63 | "these" 64 | "universal" 65 | "with" 66 | -------------------------------------------------------------------------------- /secondary/collatejson/testdata/empty: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/couchbase/indexing/ba752cd53c7865086c4d99f312eafbe3d23982a9/secondary/collatejson/testdata/empty -------------------------------------------------------------------------------- /secondary/collatejson/testdata/empty.ref: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/couchbase/indexing/ba752cd53c7865086c4d99f312eafbe3d23982a9/secondary/collatejson/testdata/empty.ref -------------------------------------------------------------------------------- /secondary/collatejson/testdata/numbers: -------------------------------------------------------------------------------- 1 | 0.000000000000000000001 2 | 1 3 | 10 4 | -2 5 | 5 6 | -4 7 | 4.1 8 | 0.2 9 | -0.2 10 | -4.1 11 | -100.0000001 12 | 100.0000001 13 | -------------------------------------------------------------------------------- /secondary/collatejson/testdata/numbers.ref: -------------------------------------------------------------------------------- 1 | -100.0000001 2 | -4.1 3 | -4 4 | -2 5 | -0.2 6 | 0.000000000000000000001 7 | 0.2 8 | 1 9 | 4.1 10 | 5 11 | 10 12 | 100.0000001 13 | -------------------------------------------------------------------------------- /secondary/collatejson/testdata/objects: -------------------------------------------------------------------------------- 1 | { "python" : "good", "perl" : "ugly", "php" : "bad" } 2 | { "python" : 10, "perl" : 20.2, "haskell" : [1,2] } 3 | { "python" : 10, "perl" : 20.2, "php" : [1,2] } 4 | { "perl" : "cryptic" } 5 | -------------------------------------------------------------------------------- /secondary/collatejson/testdata/objects.ref: -------------------------------------------------------------------------------- 1 | {"perl":"cryptic"} 2 | {"haskell":[1,2],"perl":20.2,"python":10} 3 | {"perl":20.2,"php":[1,2],"python":10} 4 | {"perl":"ugly","php":"bad","python":"good"} 5 | -------------------------------------------------------------------------------- /secondary/collatejson/testdata/sortorder: -------------------------------------------------------------------------------- 1 | "hello world" 2 | -10 3 | 10.3 4 | 10 5 | 10.2 6 | 10.1 7 | 10.6 8 | 30 9 | [ 200, 300 ] 10 | false 11 | null 12 | true 13 | { "key1" : 10, "key2" : [ 20, 30 ] } 14 | -------------------------------------------------------------------------------- /secondary/collatejson/testdata/sortorder.ref: -------------------------------------------------------------------------------- 1 | null 2 | false 3 | true 4 | -10 5 | 10 6 | 10.1 7 | 10.2 8 | 10.3 9 | 10.6 10 | 30 11 | "hello world" 12 | [200,300] 13 | {"key1":10,"key2":[20,30]} 14 | -------------------------------------------------------------------------------- /secondary/collatejson/testdata/strings: -------------------------------------------------------------------------------- 1 | "These" 2 | "are" 3 | "universal" 4 | "data" 5 | "structures." 6 | "Virtually" 7 | "all" 8 | "modern" 9 | "programming" 10 | "languages" 11 | "support" 12 | "them" 13 | "in" 14 | "one" 15 | "form" 16 | "or" 17 | "another." 18 | "It" 19 | "makes" 20 | "sense" 21 | "that" 22 | "a data" 23 | "format" 24 | "that" 25 | "is" 26 | "interchangeable" 27 | "with" 28 | "languages" 29 | "progrbmming" 30 | "also" 31 | "be" 32 | "based" 33 | "on" 34 | "these" 35 | "structures." 36 | "prográmming" 37 | -------------------------------------------------------------------------------- /secondary/collatejson/testdata/strings.ref: -------------------------------------------------------------------------------- 1 | "It" 2 | "These" 3 | "Virtually" 4 | "a data" 5 | "all" 6 | "also" 7 | "another." 8 | "are" 9 | "based" 10 | "be" 11 | "data" 12 | "form" 13 | "format" 14 | "in" 15 | "interchangeable" 16 | "is" 17 | "languages" 18 | "languages" 19 | "makes" 20 | "modern" 21 | "on" 22 | "one" 23 | "or" 24 | "programming" 25 | "progrbmming" 26 | "prográmming" 27 | "sense" 28 | "structures." 29 | "structures." 30 | "support" 31 | "that" 32 | "that" 33 | "them" 34 | "these" 35 | "universal" 36 | "with" 37 | -------------------------------------------------------------------------------- /secondary/collatejson/tools/validate/json.prod: -------------------------------------------------------------------------------- 1 | s : value. 2 | 3 | object : "{" properties "}". 4 | 5 | properties : property 6 | | properties "," property. 7 | 8 | property : DQ (bag "./web2") DQ ":" value. 9 | 10 | array : "[" values "]". 11 | 12 | value : (weigh 0.1) basic 13 | | (weigh 0.4) array 14 | | (weigh 0.4) object. 15 | 16 | values : value 17 | | values "," value. 18 | 19 | basic : (weigh 0.1) "true" 20 | | (weigh 0.2) "false" 21 | | (weigh 0.3) "null" 22 | | (weigh 0.4) number 23 | | (weigh 0.5) string. 24 | 25 | string : DQ (bag "./web2") DQ. 26 | 27 | number : (range 0 100000) 28 | | (rangef 0.0 100.0). 29 | 30 | -------------------------------------------------------------------------------- /secondary/collatejson/unicode_test.go: -------------------------------------------------------------------------------- 1 | // Copyright 2013-Present Couchbase, Inc. 2 | // 3 | // Use of this software is governed by the Business Source License included 4 | // in the file licenses/BSL-Couchbase.txt. As of the Change Date specified 5 | // in that file, in accordance with the Business Source License, use of this 6 | // software will be governed by the Apache License, Version 2.0, included in 7 | // the file licenses/APL2.txt. 8 | 9 | // +build ignore 10 | 11 | package collatejson 12 | 13 | func BenchmarkUtf8(b *testing.B) { 14 | s := "prográmming" 15 | codec := NewCodec() 16 | codec.SortbyUTF8(true) 17 | for i := 0; i < b.N; i++ { 18 | codec.EncodeUnicodeString(s) 19 | } 20 | } 21 | 22 | func BenchmarkNFKD(b *testing.B) { 23 | s := "prográmming" 24 | codec := NewCodec() 25 | codec.SortbyNFKD(true) 26 | for i := 0; i < b.N; i++ { 27 | codec.EncodeUnicodeString(s) 28 | } 29 | } 30 | 31 | func BenchmarkStringCollate(b *testing.B) { 32 | s := "prográmming" 33 | codec := NewCodec() 34 | for i := 0; i < b.N; i++ { 35 | codec.EncodeUnicodeString(s) 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /secondary/collatejson/util/sort.go: -------------------------------------------------------------------------------- 1 | // Copyright 2013-Present Couchbase, Inc. 2 | // 3 | // Use of this software is governed by the Business Source License included 4 | // in the file licenses/BSL-Couchbase.txt. As of the Change Date specified 5 | // in that file, in accordance with the Business Source License, use of this 6 | // software will be governed by the Apache License, Version 2.0, included in 7 | // the file licenses/APL2.txt. 8 | 9 | // sorting interface for byte-slice. 10 | 11 | package util 12 | 13 | import "bytes" 14 | 15 | // ByteSlices to implement Sort interface. 16 | type ByteSlices [][]byte 17 | 18 | func (b ByteSlices) Len() int { 19 | return len(b) 20 | } 21 | 22 | func (b ByteSlices) Less(i, j int) bool { 23 | if bytes.Compare(b[i], b[j]) > 0 { 24 | return false 25 | } 26 | return true 27 | } 28 | 29 | func (b ByteSlices) Swap(i, j int) { 30 | b[i], b[j] = b[j], b[i] 31 | } 32 | -------------------------------------------------------------------------------- /secondary/common/buffer_pool.go: -------------------------------------------------------------------------------- 1 | package common 2 | 3 | import "sync" 4 | 5 | // Thread safe byte buffer pool 6 | // A buffer pointer received by Get() method should be 7 | // put back using Put() method. This ensures that we not 8 | // need to create a new buf slice with len == 0 9 | type BytesBufPool struct { 10 | pool *sync.Pool 11 | } 12 | 13 | func NewByteBufferPool(size int) *BytesBufPool { 14 | newBufFn := func() interface{} { 15 | b := make([]byte, size, size) 16 | return &b 17 | } 18 | 19 | return &BytesBufPool{ 20 | pool: &sync.Pool{ 21 | New: newBufFn, 22 | }, 23 | } 24 | } 25 | 26 | func (p *BytesBufPool) Get() *[]byte { 27 | 28 | return p.pool.Get().(*[]byte) 29 | } 30 | 31 | func (p *BytesBufPool) Put(buf *[]byte) { 32 | p.pool.Put(buf) 33 | } 34 | -------------------------------------------------------------------------------- /secondary/common/build_mode.go: -------------------------------------------------------------------------------- 1 | // Copyright 2014-Present Couchbase, Inc. 2 | // 3 | // Use of this software is governed by the Business Source License included 4 | // in the file licenses/BSL-Couchbase.txt. As of the Change Date specified 5 | // in that file, in accordance with the Business Source License, use of this 6 | // software will be governed by the Apache License, Version 2.0, included in 7 | // the file licenses/APL2.txt. 8 | 9 | package common 10 | 11 | import ( 12 | "sync" 13 | ) 14 | 15 | type BuildMode byte 16 | 17 | const ( 18 | COMMUNITY = iota 19 | ENTERPRISE 20 | ) 21 | 22 | func (b BuildMode) String() string { 23 | switch b { 24 | case COMMUNITY: 25 | return "Community" 26 | case ENTERPRISE: 27 | return "Enterprise" 28 | default: 29 | return "Invalid" 30 | } 31 | } 32 | 33 | //Global Build Mode 34 | var gBuildMode BuildMode 35 | var bLock sync.RWMutex 36 | 37 | func GetBuildMode() BuildMode { 38 | 39 | bLock.RLock() 40 | defer bLock.RUnlock() 41 | return gBuildMode 42 | 43 | } 44 | 45 | func SetBuildMode(mode BuildMode) { 46 | 47 | bLock.Lock() 48 | defer bLock.Unlock() 49 | gBuildMode = mode 50 | 51 | } 52 | -------------------------------------------------------------------------------- /secondary/common/deployment_model.go: -------------------------------------------------------------------------------- 1 | // Copyright 2022-Present Couchbase, Inc. 2 | // 3 | // Use of this software is governed by the Business Source License included 4 | // in the file licenses/BSL-Couchbase.txt. As of the Change Date specified 5 | // in that file, in accordance with the Business Source License, use of this 6 | // software will be governed by the Apache License, Version 2.0, included in 7 | // the file licenses/APL2.txt. 8 | 9 | package common 10 | 11 | import "sync" 12 | 13 | type DeploymentModel byte 14 | 15 | const ( 16 | DEFAULT_DEPLOYMENT DeploymentModel = iota 17 | SERVERLESS_DEPLOYMENT 18 | PROVISIONED_DEPLOYMENT 19 | ) 20 | 21 | func (b DeploymentModel) String() string { 22 | switch b { 23 | case DEFAULT_DEPLOYMENT: 24 | return "default" 25 | case SERVERLESS_DEPLOYMENT: 26 | return "serverless" 27 | case PROVISIONED_DEPLOYMENT: 28 | return "provisioned" 29 | default: 30 | return "invalid" 31 | } 32 | } 33 | 34 | func MakeDeploymentModel(model string) DeploymentModel { 35 | if model == "serverless" { 36 | return SERVERLESS_DEPLOYMENT 37 | } else if model == "provisioned" { 38 | return PROVISIONED_DEPLOYMENT 39 | } 40 | return DEFAULT_DEPLOYMENT 41 | } 42 | 43 | // Global Deployment Model 44 | var gDeploymentModel DeploymentModel 45 | var gDeploymentModelOnce sync.Once 46 | 47 | func GetDeploymentModel() DeploymentModel { 48 | return gDeploymentModel 49 | } 50 | 51 | func SetDeploymentModel(dm DeploymentModel) { 52 | gDeploymentModelOnce.Do(func() { 53 | gDeploymentModel = dm 54 | }) 55 | } 56 | 57 | func IsServerlessDeployment() bool { 58 | 59 | if gDeploymentModel == SERVERLESS_DEPLOYMENT { 60 | return true 61 | } 62 | return false 63 | } 64 | 65 | func IsProvisionedDeployment() bool { 66 | return gDeploymentModel == PROVISIONED_DEPLOYMENT 67 | } 68 | 69 | func IsDefaultDeployment() bool { 70 | return gDeploymentModel == DEFAULT_DEPLOYMENT 71 | } 72 | -------------------------------------------------------------------------------- /secondary/common/json/LICENSE: -------------------------------------------------------------------------------- 1 | Copyright (c) 2009 The Go Authors. All rights reserved. 2 | 3 | Redistribution and use in source and binary forms, with or without 4 | modification, are permitted provided that the following conditions are 5 | met: 6 | 7 | * Redistributions of source code must retain the above copyright 8 | notice, this list of conditions and the following disclaimer. 9 | * Redistributions in binary form must reproduce the above 10 | copyright notice, this list of conditions and the following disclaimer 11 | in the documentation and/or other materials provided with the 12 | distribution. 13 | * Neither the name of Google Inc. nor the names of its 14 | contributors may be used to endorse or promote products derived from 15 | this software without specific prior written permission. 16 | 17 | THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS 18 | "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT 19 | LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR 20 | A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 21 | OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 22 | SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT 23 | LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, 24 | DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY 25 | THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT 26 | (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE 27 | OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. -------------------------------------------------------------------------------- /secondary/common/json/README.md: -------------------------------------------------------------------------------- 1 | # json 2 | A fork of the Golang encoding/json package, with custom fixes to support Unmarshaling JSON numbers that are integers into int64 -------------------------------------------------------------------------------- /secondary/common/json/tags.go: -------------------------------------------------------------------------------- 1 | // Copyright 2011 The Go Authors. All rights reserved. 2 | // Use of this source code is governed by a BSD-style 3 | // license that can be found in the LICENSE file. 4 | 5 | package json 6 | 7 | import ( 8 | "strings" 9 | ) 10 | 11 | // tagOptions is the string following a comma in a struct field's "json" 12 | // tag, or the empty string. It does not include the leading comma. 13 | type tagOptions string 14 | 15 | // parseTag splits a struct field's json tag into its name and 16 | // comma-separated options. 17 | func parseTag(tag string) (string, tagOptions) { 18 | if idx := strings.Index(tag, ","); idx != -1 { 19 | return tag[:idx], tagOptions(tag[idx+1:]) 20 | } 21 | return tag, tagOptions("") 22 | } 23 | 24 | // Contains reports whether a comma-separated list of options 25 | // contains a particular substr flag. substr must be surrounded by a 26 | // string boundary or commas. 27 | func (o tagOptions) Contains(optionName string) bool { 28 | if len(o) == 0 { 29 | return false 30 | } 31 | s := string(o) 32 | for s != "" { 33 | var next string 34 | i := strings.Index(s, ",") 35 | if i >= 0 { 36 | s, next = s[:i], s[i+1:] 37 | } 38 | if s == optionName { 39 | return true 40 | } 41 | s = next 42 | } 43 | return false 44 | } 45 | -------------------------------------------------------------------------------- /secondary/common/json/tags_test.go: -------------------------------------------------------------------------------- 1 | // Copyright 2011 The Go Authors. All rights reserved. 2 | // Use of this source code is governed by a BSD-style 3 | // license that can be found in the LICENSE file. 4 | 5 | package json 6 | 7 | import ( 8 | "testing" 9 | ) 10 | 11 | func TestTagParsing(t *testing.T) { 12 | name, opts := parseTag("field,foobar,foo") 13 | if name != "field" { 14 | t.Fatalf("name = %q, want field", name) 15 | } 16 | for _, tt := range []struct { 17 | opt string 18 | want bool 19 | }{ 20 | {"foobar", true}, 21 | {"foo", true}, 22 | {"bar", false}, 23 | } { 24 | if opts.Contains(tt.opt) != tt.want { 25 | t.Errorf("Contains(%q) = %v", tt.opt, !tt.want) 26 | } 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /secondary/common/json/testdata/code.json.gz: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/couchbase/indexing/ba752cd53c7865086c4d99f312eafbe3d23982a9/secondary/common/json/testdata/code.json.gz -------------------------------------------------------------------------------- /secondary/common/limits_config.go: -------------------------------------------------------------------------------- 1 | // Copyright 2021-Present Couchbase, Inc. 2 | // 3 | // Use of this software is governed by the Business Source License included 4 | // in the file licenses/BSL-Couchbase.txt. As of the Change Date specified 5 | // in that file, in accordance with the Business Source License, use of this 6 | // software will be governed by the Apache License, Version 2.0, included in 7 | // the file licenses/APL2.txt. 8 | 9 | package common 10 | 11 | type LimitsCache struct{} 12 | 13 | func NewLimitsCache() (*LimitsCache, error) { 14 | lcCache := &LimitsCache{} 15 | return lcCache, nil 16 | } 17 | 18 | // 19 | // TODO: 20 | // It is recommended to use ConfigRefreshCallback before calling 21 | // GetLimitsConfig(). But each process can register only one 22 | // ConfigRefreshCallback. So, directly calling GetLimitsConfig(), 23 | // without a callback also does the trick. 24 | // As per current implementation, GetLimitsConfig() is a local 25 | // call and returns last cached value received from ns_server. 26 | // 27 | 28 | // This should return true only when limits are enforced 29 | // and cluster version >= 7.1 30 | // 31 | // Update: Remove free tier limits as per MB-56328 32 | func (lcCache *LimitsCache) EnforceLimits() (bool, error) { 33 | return false, nil 34 | } 35 | -------------------------------------------------------------------------------- /secondary/common/partition_container.go: -------------------------------------------------------------------------------- 1 | // Copyright 2014-Present Couchbase, Inc. 2 | // 3 | // Use of this software is governed by the Business Source License included 4 | // in the file licenses/BSL-Couchbase.txt. As of the Change Date specified 5 | // in that file, in accordance with the Business Source License, use of this 6 | // software will be governed by the Apache License, Version 2.0, included in 7 | // the file licenses/APL2.txt. 8 | 9 | package common 10 | 11 | type PartitionKey []byte 12 | type PartitionId int 13 | 14 | //Endpoint provides an Indexer address(host:port) 15 | //which is hosting a partition 16 | type Endpoint string //host:port 17 | 18 | //PartitionContainer contains all the partitions for an index instance 19 | //and provides methods to lookup topology information(i.e. endpoints) 20 | //based on the PartitionKey 21 | type PartitionContainer interface { 22 | AddPartition(PartitionId, PartitionDefn) 23 | UpdatePartition(PartitionId, PartitionDefn) 24 | RemovePartition(PartitionId) 25 | 26 | GetEndpointsByPartitionKey(PartitionKey) []Endpoint 27 | GetPartitionIdByPartitionKey(PartitionKey) PartitionId 28 | GetEndpointsByPartitionId(PartitionId) []Endpoint 29 | 30 | GetAllPartitions() []PartitionDefn 31 | GetAllPartitionIds() ([]PartitionId, []int) 32 | GetPartitionById(PartitionId) PartitionDefn 33 | GetNumPartitions() int 34 | 35 | Clone() PartitionContainer 36 | CheckPartitionExists(PartitionId) bool 37 | } 38 | 39 | //PartitionDefn is a generic interface which defines 40 | //a partition 41 | type PartitionDefn interface { 42 | GetPartitionId() PartitionId 43 | GetVersion() int 44 | Endpoints() []Endpoint 45 | GetShardIds() []ShardId 46 | AddShardIds([]ShardId) 47 | } 48 | -------------------------------------------------------------------------------- /secondary/common/random_sample.go: -------------------------------------------------------------------------------- 1 | package common 2 | 3 | import ( 4 | "fmt" 5 | 6 | memcached "github.com/couchbase/indexing/secondary/dcp/transport/client" 7 | "github.com/couchbase/indexing/secondary/logging" 8 | ) 9 | 10 | const S_RESULT_CHANNEL_SIZE = 100 11 | 12 | //FetchRandomKVSample fetches a given sampleSize number of documents 13 | //from a collection. It returns "datach" on which sampled documents are 14 | //returned in DcpEvent format. Any error during sampling is sent on 15 | //the err channel. datach is closed by the sender when all the sample 16 | //docs have been sent. 17 | // 18 | //Caller must close the donech to indicate it no longer wants to read from 19 | //datach. If any scan is in progress and donech is closed, the scan will be aborted. 20 | 21 | func FetchRandomKVSample(cluster, pooln, bucketn, 22 | scope, collection, cid string, sampleSize int64, 23 | donech chan bool) (datach chan *memcached.DcpEvent, errch chan error, ret error) { 24 | 25 | //panic safe 26 | defer func() { 27 | if r := recover(); r != nil { 28 | ret = fmt.Errorf("%v", r) 29 | logging.Warnf("FetchRandomKVSample failed : %v", ret) 30 | } 31 | }() 32 | 33 | logging.Infof("FetchRandomKVSample %v:%v:%v SampleSize %v", bucketn, scope, collection, sampleSize) 34 | 35 | bucket, err := ConnectBucket(cluster, pooln, bucketn) 36 | if err != nil { 37 | logging.Warnf("FetchRandomKVSample failed : %v", err) 38 | return nil, nil, err 39 | } 40 | 41 | defer func() { 42 | if err != nil { 43 | bucket.Close() 44 | } 45 | }() 46 | 47 | rs, err := bucket.NewRandomScanner(cid, sampleSize) 48 | if err != nil { 49 | logging.Warnf("FetchRandomKVSample failed : %v", err) 50 | return nil, nil, err 51 | } 52 | 53 | go func() { 54 | 55 | defer bucket.Close() 56 | select { 57 | 58 | case <-donech: 59 | rs.StopScan() 60 | 61 | } 62 | }() 63 | 64 | return rs.StartRandomScan() 65 | 66 | } 67 | -------------------------------------------------------------------------------- /secondary/common/random_sample_test.go: -------------------------------------------------------------------------------- 1 | package common 2 | 3 | import ( 4 | "testing" 5 | 6 | memcached "github.com/couchbase/indexing/secondary/dcp/transport/client" 7 | "github.com/couchbase/indexing/secondary/logging" 8 | ) 9 | 10 | func SkipTestFetchRandomKVSample(t *testing.T) { 11 | 12 | cluster := "127.0.0.1:9000" 13 | pool := "default" 14 | bucket := "default" 15 | scope := "_default" 16 | collection := "_default" 17 | cid := "0" 18 | 19 | donech := make(chan bool) 20 | datach, errch, err := FetchRandomKVSample(cluster, pool, bucket, scope, collection, cid, 800, donech) 21 | if err != nil { 22 | t.Errorf("FetchRandomKVSample error %v", err) 23 | } 24 | 25 | processResponse(datach, errch) 26 | close(donech) 27 | 28 | } 29 | 30 | func processResponse(datach chan *memcached.DcpEvent, 31 | errch chan error) { 32 | 33 | numDocs := 0 34 | numErrs := 0 35 | sample: 36 | for { 37 | select { 38 | case _, ok := <-datach: 39 | if ok { 40 | numDocs++ 41 | } else { 42 | break sample 43 | } 44 | 45 | case err := <-errch: 46 | //forward to the caller 47 | if err != nil { 48 | numErrs++ 49 | } 50 | 51 | } 52 | } 53 | 54 | logging.Infof("Total Docs Fetched %v, Err Count %v", numDocs, numErrs) 55 | } 56 | -------------------------------------------------------------------------------- /secondary/common/regulator_ce.go: -------------------------------------------------------------------------------- 1 | //go:build community 2 | 3 | // Copyright 2022-Present Couchbase, Inc. 4 | // 5 | // Use of this software is governed by the Business Source License included 6 | // in the file licenses/BSL-Couchbase.txt. As of the Change Date specified 7 | // in that file, in accordance with the Business Source License, use of this 8 | // software will be governed by the Apache License, Version 2.0, included in 9 | // the file licenses/APL2.txt. 10 | 11 | package common 12 | 13 | func CheckIngressLockdown(bucket string) error { 14 | panic("CheckIngressLockdown Not implemented for Community Edition") 15 | return nil 16 | } 17 | -------------------------------------------------------------------------------- /secondary/common/regulator_ee.go: -------------------------------------------------------------------------------- 1 | //go:build !community 2 | 3 | // Copyright 2022-Present Couchbase, Inc. 4 | // 5 | // Use of this software is governed by the Business Source License included 6 | // in the file licenses/BSL-Couchbase.txt. As of the Change Date specified 7 | // in that file, in accordance with the Business Source License, use of this 8 | // software will be governed by the Apache License, Version 2.0, included in 9 | // the file licenses/APL2.txt. 10 | 11 | package common 12 | 13 | import "fmt" 14 | 15 | import "github.com/couchbase/indexing/secondary/logging" 16 | import "github.com/couchbase/regulator" 17 | 18 | func CheckIngressLockdown(bucket string) error { 19 | if regulator.Instance == nil { 20 | // This is not expected. 21 | // TODO: Should we panic here? 22 | logging.Errorf("Regulator is not initialised. Allowing ingress for bucket %v.", bucket) 23 | return nil 24 | } 25 | 26 | bctx := regulator.NewBucketCtx(bucket) 27 | acc, _, err := regulator.CheckAccess(bctx, true) 28 | if err != nil { 29 | return err 30 | } 31 | 32 | // TODO: Return per service disk usage in the error. 33 | 34 | if acc == regulator.AccessNoIngress { 35 | return ErrNoIngress 36 | } 37 | 38 | if acc == regulator.AccessError { 39 | return fmt.Errorf("AccessError") 40 | } 41 | 42 | return nil 43 | } 44 | -------------------------------------------------------------------------------- /secondary/common/retry_helper.go: -------------------------------------------------------------------------------- 1 | package common 2 | 3 | import ( 4 | "time" 5 | ) 6 | 7 | // Helper object to execute a function with retries and exponential backoff 8 | type RetryHelper struct { 9 | interval time.Duration // backoff interval 10 | retries, maxRetries int 11 | call retryFunc 12 | factor int // backoff factor 13 | } 14 | 15 | // retryAttempt param can be used used to print log message during every attempt 16 | type retryFunc func(retryAttempt int, lastErr error) error 17 | 18 | func NewRetryHelper( 19 | maxRetries int, 20 | interval time.Duration, 21 | factor int, 22 | call retryFunc) *RetryHelper { 23 | 24 | return &RetryHelper{ 25 | interval: interval, 26 | maxRetries: maxRetries, 27 | factor: factor, 28 | call: call, 29 | } 30 | } 31 | 32 | func (r *RetryHelper) Run() error { 33 | var err error 34 | 35 | for ; r.retries < r.maxRetries+1; r.retries++ { 36 | err = r.call(r.retries, err) 37 | if err == nil { 38 | break 39 | } else { 40 | time.Sleep(r.interval) 41 | r.interval = r.interval * time.Duration(r.factor) 42 | } 43 | } 44 | 45 | return err 46 | } 47 | 48 | // Retry until no error or cond(err) returns true 49 | func (r *RetryHelper) RunWithConditionalError(cond func(error) bool) error { 50 | var err error 51 | 52 | for ; r.retries < r.maxRetries+1; r.retries++ { 53 | err = r.call(r.retries, err) 54 | if err == nil || cond(err) { 55 | break 56 | } else { 57 | time.Sleep(r.interval) 58 | r.interval = r.interval * time.Duration(r.factor) 59 | } 60 | } 61 | 62 | return err 63 | } 64 | -------------------------------------------------------------------------------- /secondary/common/shard_stats.go: -------------------------------------------------------------------------------- 1 | package common 2 | 3 | type ShardStats struct { 4 | // UUID of the shard 5 | ShardId ShardId 6 | BackstoreShardId ShardId 7 | 8 | // AlternateShardId of the shard 9 | AlternateShardId string 10 | 11 | // In-memory consumption by the shard 12 | MemSz int64 13 | MemSzIndex int64 14 | BufMemUsed int64 15 | LSSBufMemUsed int64 16 | 17 | // Total data size of the shard 18 | LSSDataSize int64 19 | 20 | // Total number of items in the shard 21 | ItemsCount int64 22 | 23 | // For computing resident percent of the shard 24 | CachedRecords int64 25 | TotalRecords int64 26 | Instances map[string]bool 27 | 28 | // DiskStats of the shards 29 | LSSDiskSize int64 30 | RecoveryDiskSize int64 31 | } 32 | 33 | func NewShardStats(alternateShardId string) *ShardStats { 34 | return &ShardStats{ 35 | AlternateShardId: alternateShardId, 36 | Instances: make(map[string]bool), 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /secondary/common/sort.go: -------------------------------------------------------------------------------- 1 | // Copyright 2013-Present Couchbase, Inc. 2 | // 3 | // Use of this software is governed by the Business Source License included 4 | // in the file licenses/BSL-Couchbase.txt. As of the Change Date specified 5 | // in that file, in accordance with the Business Source License, use of this 6 | // software will be governed by the Apache License, Version 2.0, included in 7 | // the file licenses/APL2.txt. 8 | 9 | // sorting interface for byte-slice. 10 | 11 | package common 12 | 13 | import "bytes" 14 | 15 | // ByteSlices to implement Sort interface. 16 | type ByteSlices [][]byte 17 | 18 | func (b ByteSlices) Len() int { 19 | return len(b) 20 | } 21 | 22 | func (b ByteSlices) Less(i, j int) bool { 23 | if bytes.Compare(b[i], b[j]) > 0 { 24 | return false 25 | } 26 | return true 27 | } 28 | 29 | func (b ByteSlices) Swap(i, j int) { 30 | b[i], b[j] = b[j], b[i] 31 | } 32 | -------------------------------------------------------------------------------- /secondary/common/uuid.go: -------------------------------------------------------------------------------- 1 | // Copyright 2014-Present Couchbase, Inc. 2 | // 3 | // Use of this software is governed by the Business Source License included 4 | // in the file licenses/BSL-Couchbase.txt. As of the Change Date specified 5 | // in that file, in accordance with the Business Source License, use of this 6 | // software will be governed by the Apache License, Version 2.0, included in 7 | // the file licenses/APL2.txt. 8 | 9 | package common 10 | 11 | import ( 12 | "bytes" 13 | crypt "crypto/rand" 14 | "encoding/binary" 15 | "io" 16 | rnd "math/rand" 17 | "strconv" 18 | ) 19 | 20 | type UUID []byte 21 | 22 | func NewUUID() (UUID, error) { 23 | uuid := make([]byte, 8) 24 | n, err := io.ReadFull(crypt.Reader, uuid) 25 | if n != len(uuid) || err != nil { 26 | return UUID(nil), err 27 | } 28 | return UUID(uuid), nil 29 | } 30 | 31 | func (u UUID) Uint64() uint64 { 32 | return binary.LittleEndian.Uint64(([]byte)(u)) 33 | } 34 | 35 | func (u UUID) Str() string { 36 | var buf bytes.Buffer 37 | for i := 0; i < len(u); i++ { 38 | if i > 0 { 39 | buf.WriteString(":") 40 | } 41 | buf.WriteString(strconv.FormatUint(uint64(u[i]), 16)) 42 | } 43 | return buf.String() 44 | } 45 | 46 | func SeedProcess() { 47 | uuid, err := NewUUID() 48 | if err != nil { 49 | seed := uuid.Uint64() 50 | rnd.Seed(int64(seed)) 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /secondary/common/vector_encoding_test.go: -------------------------------------------------------------------------------- 1 | package common 2 | 3 | import ( 4 | "testing" 5 | ) 6 | 7 | func TestVectorEncoding(t *testing.T) { 8 | //dummy := GenDummyCentroidId() 9 | encodeBuf := make([]byte, 0, 100) 10 | val, err := EncodedCentroidId(1234, encodeBuf[:0]) 11 | if err != nil { 12 | t.Fatalf("Error observed during encoding, err: %v", err) 13 | } 14 | 15 | decVal, _, err := DecodeCentroidId(val, encodeBuf[:]) 16 | if err != nil { 17 | t.Fatalf("Error observed during decoding, err: %v", err) 18 | } 19 | if decVal != 1234 { 20 | t.Fatalf("Encoded and decoded values are not same. EncodedVal: %v, decodedVal: %v", 1234, decVal) 21 | } 22 | 23 | val, err = EncodedCentroidIds([]int64{1234, -1, 4567}, encodeBuf[:0]) 24 | if err != nil { 25 | t.Fatalf("Error observed during encoding, err: %v", err) 26 | } 27 | 28 | DecodeCentroidId(val, encodeBuf[:]) // Does not work now - until we add support for array of vectors 29 | } 30 | -------------------------------------------------------------------------------- /secondary/dcp/client_test.go: -------------------------------------------------------------------------------- 1 | package couchbase 2 | 3 | import "testing" 4 | 5 | func TestWriteOptionsString(t *testing.T) { 6 | tests := []struct { 7 | opts WriteOptions 8 | exp string 9 | }{ 10 | {Raw, "raw"}, 11 | {AddOnly, "addonly"}, 12 | {Persist, "persist"}, 13 | {Indexable, "indexable"}, 14 | {Append, "append"}, 15 | {AddOnly | Raw, "raw|addonly"}, 16 | {0, "0x0"}, 17 | {Raw | AddOnly | Persist | Indexable | Append, 18 | "raw|addonly|persist|indexable|append"}, 19 | {Raw | 8192, "raw|0x2000"}, 20 | } 21 | 22 | for _, test := range tests { 23 | got := test.opts.String() 24 | if got != test.exp { 25 | t.Errorf("Expected %v, got %v", test.exp, got) 26 | } 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /secondary/dcp/examples/basic/basic.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import ( 4 | "flag" 5 | "fmt" 6 | "log" 7 | "net/url" 8 | "os" 9 | 10 | "github.com/couchbase/indexing/secondary/dcp" 11 | ) 12 | 13 | func mf(err error, msg string) { 14 | if err != nil { 15 | log.Fatalf("%v: %v", msg, err) 16 | } 17 | } 18 | 19 | func main() { 20 | bname := flag.String("bucket", "", 21 | "bucket to connect to (defaults to username)") 22 | 23 | flag.Usage = func() { 24 | fmt.Fprintf(os.Stderr, 25 | "%v [flags] http://user:pass@host:8091/\n\nFlags:\n", 26 | os.Args[0]) 27 | flag.PrintDefaults() 28 | os.Exit(64) 29 | } 30 | 31 | flag.Parse() 32 | 33 | if flag.NArg() < 1 { 34 | flag.Usage() 35 | } 36 | 37 | u, err := url.Parse(flag.Arg(0)) 38 | mf(err, "parse") 39 | 40 | if *bname == "" && u.User != nil { 41 | *bname = u.User.Username() 42 | } 43 | 44 | c, err := couchbase.Connect(u.String()) 45 | mf(err, "connect - "+u.String()) 46 | 47 | p, err := c.GetPool("default") 48 | mf(err, "pool") 49 | 50 | b, err := p.GetBucket(*bname) 51 | mf(err, "bucket") 52 | 53 | err = b.Set(",k", 90, map[string]interface{}{"x": 1}) 54 | mf(err, "set") 55 | 56 | ob := map[string]interface{}{} 57 | err = b.Get(",k", &ob) 58 | mf(err, "get") 59 | } 60 | -------------------------------------------------------------------------------- /secondary/dcp/examples/bucketlist/bucketlist.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import ( 4 | "flag" 5 | "fmt" 6 | "log" 7 | "net/url" 8 | "os" 9 | 10 | "github.com/couchbase/indexing/secondary/dcp" 11 | ) 12 | 13 | func mf(err error, msg string) { 14 | if err != nil { 15 | log.Fatalf("%v: %v", msg, err) 16 | } 17 | } 18 | 19 | func main() { 20 | 21 | flag.Usage = func() { 22 | fmt.Fprintf(os.Stderr, 23 | "%v [flags] http://user:pass@host:8091/\n\nFlags:\n", 24 | os.Args[0]) 25 | flag.PrintDefaults() 26 | os.Exit(64) 27 | } 28 | 29 | flag.Parse() 30 | 31 | if flag.NArg() < 1 { 32 | flag.Usage() 33 | } 34 | 35 | u, err := url.Parse(flag.Arg(0)) 36 | mf(err, "parse") 37 | 38 | bucketInfo, err := couchbase.GetBucketList(u.String()) 39 | fmt.Printf("List of buckets and password %v", bucketInfo) 40 | 41 | } 42 | -------------------------------------------------------------------------------- /secondary/dcp/examples/hello_observe/hello_observe.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import ( 4 | "flag" 5 | "fmt" 6 | "log" 7 | "time" 8 | 9 | "github.com/couchbase/indexing/secondary/dcp" 10 | ) 11 | 12 | var poolName = flag.String("pool", "default", "Pool name") 13 | var writeFlag = flag.Bool("write", false, "If true, will write a value to the key") 14 | 15 | func main() { 16 | flag.Parse() 17 | 18 | if len(flag.Args()) < 3 { 19 | log.Fatalf("Usage: hello_observe [-pool poolname] [-write] server bucket key") 20 | } 21 | 22 | c, err := couchbase.Connect(flag.Arg(0)) 23 | if err != nil { 24 | log.Fatalf("Error connecting: %v", err) 25 | } 26 | fmt.Printf("Connected to ver=%s\n", c.Info.ImplementationVersion) 27 | 28 | pool, err := c.GetPool(*poolName) 29 | if err != nil { 30 | log.Fatalf("Can't get pool %q: %v", *poolName, err) 31 | } 32 | 33 | bucket, err := pool.GetBucket(flag.Arg(1)) 34 | if err != nil { 35 | log.Fatalf("Can't get bucket %q: %v", flag.Arg(1), err) 36 | } 37 | 38 | key := flag.Arg(2) 39 | 40 | result, err := bucket.Observe(key) 41 | if err != nil { 42 | log.Fatalf("Observe returned error %v", err) 43 | } 44 | log.Printf("Observe result: %+v", result) 45 | 46 | if *writeFlag { 47 | log.Printf("Now writing to key %q with persistence...", key) 48 | start := time.Now() 49 | err = bucket.Write(key, 0, 0, "observe test", couchbase.Persist) 50 | if err != nil { 51 | log.Fatalf("Write returned error %v", err) 52 | } 53 | log.Printf("Write with persistence took %s", time.Since(start)) 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /secondary/dcp/examples/loadfile/loadfile.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import ( 4 | "flag" 5 | "fmt" 6 | "io/ioutil" 7 | "os" 8 | "path" 9 | 10 | "github.com/couchbase/indexing/secondary/dcp" 11 | ) 12 | 13 | func maybeFatal(e error, f string, args ...interface{}) { 14 | if e != nil { 15 | fmt.Fprintf(os.Stderr, f, args...) 16 | os.Exit(64) 17 | } 18 | } 19 | 20 | func main() { 21 | cbServ := flag.String("couchbase", "http://localhost:8091/", 22 | "URL to couchbase") 23 | cbBucket := flag.String("bucket", "default", "couchbase bucket") 24 | includeExt := flag.Bool("includeExt", false, "include file extension in document ID") 25 | verbose := flag.Bool("v", false, "verbose output") 26 | 27 | flag.Parse() 28 | 29 | b, err := couchbase.GetBucket(*cbServ, "default", *cbBucket) 30 | maybeFatal(err, "Error connecting to couchbase: %v\n", err) 31 | 32 | for _, filename := range flag.Args() { 33 | key := pathToID(filename, *includeExt) 34 | bytes, err := ioutil.ReadFile(filename) 35 | maybeFatal(err, "Error reading file contents: %v\n", err) 36 | b.SetRaw(key, 0, bytes) 37 | if *verbose { 38 | fmt.Printf("Loaded %s to key %s\n", filename, key) 39 | } 40 | } 41 | if *verbose { 42 | fmt.Printf("Loaded %d documents into bucket %s\n", len(flag.Args()), *cbBucket) 43 | } 44 | } 45 | 46 | func pathToID(p string, includeExt bool) string { 47 | _, file := path.Split(p) 48 | if includeExt { 49 | return file 50 | } 51 | ext := path.Ext(file) 52 | return file[0 : len(file)-len(ext)] 53 | } 54 | -------------------------------------------------------------------------------- /secondary/dcp/transport/client/tap_feed_test.go: -------------------------------------------------------------------------------- 1 | package memcached 2 | 3 | import ( 4 | "testing" 5 | 6 | "github.com/couchbase/indexing/secondary/dcp/transport" 7 | ) 8 | 9 | func TestMakeTapEvent(t *testing.T) { 10 | e := makeTapEvent(transport.MCRequest{ 11 | Opcode: transport.TAP_MUTATION, 12 | Key: []byte("hi"), 13 | Body: []byte("world"), 14 | Cas: 0x4321000012340000, 15 | }) 16 | if e.Cas != 0x4321000012340000 { 17 | t.Fatalf("Expected Cas to match") 18 | } 19 | e = makeTapEvent(transport.MCRequest{ 20 | Opcode: transport.TAP_DELETE, 21 | Key: []byte("hi"), 22 | Body: []byte("world"), 23 | Cas: 0x9321000012340000, 24 | }) 25 | if e.Cas != 0x9321000012340000 { 26 | t.Fatalf("Expected Cas to match") 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /secondary/dcp/transport/client/transport.go: -------------------------------------------------------------------------------- 1 | package memcached 2 | 3 | import ( 4 | "errors" 5 | "io" 6 | 7 | "github.com/couchbase/indexing/secondary/dcp/transport" 8 | ) 9 | 10 | var errNoConn = errors.New("no connection") 11 | 12 | // UnwrapMemcachedError converts memcached errors to normal responses. 13 | // 14 | // If the error is a memcached response, declare the error to be nil 15 | // so a client can handle the status without worrying about whether it 16 | // indicates success or failure. 17 | func UnwrapMemcachedError(rv *transport.MCResponse, 18 | err error) (*transport.MCResponse, error) { 19 | 20 | if rv == err { 21 | return rv, nil 22 | } 23 | return rv, err 24 | } 25 | 26 | // ReceiveHook is called after every packet is received (or attempted to be) 27 | var ReceiveHook func(*transport.MCResponse, int, error) 28 | 29 | func getResponse(s io.Reader, hdrBytes []byte) (rv *transport.MCResponse, n int, err error) { 30 | if s == nil { 31 | return nil, 0, errNoConn 32 | } 33 | 34 | rv = &transport.MCResponse{} 35 | n, err = rv.Receive(s, hdrBytes) 36 | 37 | if ReceiveHook != nil { 38 | ReceiveHook(rv, n, err) 39 | } 40 | 41 | if err == nil && rv.Status != transport.SUCCESS { 42 | err = rv 43 | } 44 | return rv, n, err 45 | } 46 | 47 | // TransmitHook is called after each packet is transmitted. 48 | var TransmitHook func(*transport.MCRequest, int, error) 49 | 50 | func transmitRequest(o io.Writer, req *transport.MCRequest) (int, error) { 51 | if o == nil { 52 | return 0, errNoConn 53 | } 54 | n, err := req.Transmit(o) 55 | if TransmitHook != nil { 56 | TransmitHook(req, n, err) 57 | } 58 | return n, err 59 | } 60 | 61 | func transmitResponse(o io.Writer, res *transport.MCResponse) (int, error) { 62 | if o == nil { 63 | return 0, errNoConn 64 | } 65 | n, err := res.Transmit(o) 66 | return n, err 67 | } 68 | -------------------------------------------------------------------------------- /secondary/dcp/transport/gocache/gocache.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import ( 4 | "flag" 5 | "fmt" 6 | "io" 7 | "net" 8 | 9 | "github.com/couchbase/indexing/secondary/dcp/transport" 10 | "github.com/couchbase/indexing/secondary/dcp/transport/server" 11 | "github.com/couchbase/indexing/secondary/logging" 12 | ) 13 | 14 | var port = flag.Int("port", 11212, "Port on which to listen") 15 | 16 | type chanReq struct { 17 | req *transport.MCRequest 18 | res chan *transport.MCResponse 19 | } 20 | 21 | type reqHandler struct { 22 | ch chan chanReq 23 | } 24 | 25 | func (rh *reqHandler) HandleMessage(w io.Writer, req *transport.MCRequest) *transport.MCResponse { 26 | cr := chanReq{ 27 | req, 28 | make(chan *transport.MCResponse), 29 | } 30 | 31 | rh.ch <- cr 32 | return <-cr.res 33 | } 34 | 35 | func connectionHandler(s net.Conn, h memcached.RequestHandler) { 36 | // Explicitly ignoring errors since they all result in the 37 | // client getting hung up on and many are common. 38 | _ = memcached.HandleIO(s, h) 39 | } 40 | 41 | func waitForConnections(ls net.Listener) { 42 | reqChannel := make(chan chanReq) 43 | 44 | go RunServer(reqChannel) 45 | handler := &reqHandler{reqChannel} 46 | 47 | logging.Warnf("Listening on port %d", *port) 48 | for { 49 | s, e := ls.Accept() 50 | if e == nil { 51 | logging.Warnf("Got a connection from %v", s.RemoteAddr()) 52 | go connectionHandler(s, handler) 53 | } else { 54 | logging.Warnf("Error accepting from %s", ls) 55 | } 56 | } 57 | } 58 | 59 | func main() { 60 | flag.Parse() 61 | ls, e := net.Listen("tcp", fmt.Sprintf(":%d", *port)) 62 | if e != nil { 63 | logging.Fatalf("Got an error: %s", e) 64 | } 65 | 66 | waitForConnections(ls) 67 | } 68 | -------------------------------------------------------------------------------- /secondary/dcp/transport/mc_constants_test.go: -------------------------------------------------------------------------------- 1 | package transport 2 | 3 | import ( 4 | "strings" 5 | "testing" 6 | ) 7 | 8 | func TestCommandCodeStringin(t *testing.T) { 9 | if GET.String() != "GET" { 10 | t.Fatalf("Expected \"GET\" for GET, got \"%v\"", GET.String()) 11 | } 12 | 13 | cc := CommandCode(0x80) 14 | if cc.String() != "0x80" { 15 | t.Fatalf("Expected \"0x80\" for 0x80, got \"%v\"", cc.String()) 16 | } 17 | } 18 | 19 | func TestStatusNameString(t *testing.T) { 20 | if SUCCESS.String() != "SUCCESS" { 21 | t.Fatalf("Expected \"SUCCESS\" for SUCCESS, got \"%v\"", 22 | SUCCESS.String()) 23 | } 24 | 25 | s := Status(0x80) 26 | if s.String() != "0x80" { 27 | t.Fatalf("Expected \"0x80\" for 0x80, got \"%v\"", s.String()) 28 | } 29 | } 30 | 31 | func TestIsQuiet(t *testing.T) { 32 | for v, k := range CommandNames { 33 | isq := strings.HasSuffix(k, "Q") && (k != CommandNames[DCP_STREAMREQ]) 34 | if v.IsQuiet() != isq { 35 | t.Errorf("Expected quiet=%v for %v, got %v", isq, v, v.IsQuiet()) 36 | } 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /secondary/dcp/util.go: -------------------------------------------------------------------------------- 1 | package couchbase 2 | 3 | import ( 4 | "fmt" 5 | "net/url" 6 | "strings" 7 | ) 8 | 9 | // CleanupHost returns the hostname with the given suffix removed. 10 | func CleanupHost(h, commonSuffix string) string { 11 | if strings.HasSuffix(h, commonSuffix) { 12 | return h[:len(h)-len(commonSuffix)] 13 | } 14 | return h 15 | } 16 | 17 | // FindCommonSuffix returns the longest common suffix from the given 18 | // strings. 19 | func FindCommonSuffix(input []string) string { 20 | rv := "" 21 | if len(input) < 2 { 22 | return "" 23 | } 24 | from := input 25 | for i := len(input[0]); i > 0; i-- { 26 | common := true 27 | suffix := input[0][i:] 28 | for _, s := range from { 29 | if !strings.HasSuffix(s, suffix) { 30 | common = false 31 | break 32 | } 33 | } 34 | if common { 35 | rv = suffix 36 | } 37 | } 38 | return rv 39 | } 40 | 41 | // ParseURL is a wrapper around url.Parse with some sanity-checking 42 | func ParseURL(urlStr string) (result *url.URL, err error) { 43 | result, err = url.Parse(urlStr) 44 | if result != nil && result.Scheme == "" { 45 | result = nil 46 | err = fmt.Errorf("invalid URL <%s>", urlStr) 47 | } 48 | return 49 | } 50 | -------------------------------------------------------------------------------- /secondary/dcp/util_test.go: -------------------------------------------------------------------------------- 1 | package couchbase 2 | 3 | import ( 4 | "testing" 5 | ) 6 | 7 | func TestCleanupHost(t *testing.T) { 8 | tests := []struct { 9 | name, full, suffix, exp string 10 | }{ 11 | {"empty", "", "", ""}, 12 | {"empty suffix", "aprefix", "", "aprefix"}, 13 | {"empty host", "", "asuffix", ""}, 14 | {"matched suffix", "server1.example.com:11210", ".example.com:11210", "server1"}, 15 | } 16 | 17 | for _, test := range tests { 18 | got := CleanupHost(test.full, test.suffix) 19 | if got != test.exp { 20 | t.Errorf("Error on %v: got %q, expected %q", 21 | test.name, got, test.exp) 22 | } 23 | } 24 | } 25 | 26 | func TestFindCommonSuffix(t *testing.T) { 27 | tests := []struct { 28 | name, exp string 29 | strings []string 30 | }{ 31 | {"empty", "", nil}, 32 | {"one", "", []string{"blah"}}, 33 | {"two", ".com", []string{"blah.com", "foo.com"}}, 34 | } 35 | 36 | for _, test := range tests { 37 | got := FindCommonSuffix(test.strings) 38 | if got != test.exp { 39 | t.Errorf("Error on %v: got %q, expected %q", 40 | test.name, got, test.exp) 41 | } 42 | } 43 | } 44 | 45 | func TestParseURL(t *testing.T) { 46 | tests := []struct { 47 | in string 48 | works bool 49 | }{ 50 | {"", false}, 51 | {"http://whatever/", true}, 52 | {"http://%/", false}, 53 | } 54 | 55 | for _, test := range tests { 56 | got, err := ParseURL(test.in) 57 | switch { 58 | case err == nil && test.works, 59 | !(err == nil || test.works): 60 | case err == nil && !test.works: 61 | t.Errorf("Expected failure on %v, got %v", test.in, got) 62 | case test.works && err != nil: 63 | t.Errorf("Expected success on %v, got %v", test.in, err) 64 | } 65 | } 66 | } 67 | -------------------------------------------------------------------------------- /secondary/dcp/vbmap_test.go: -------------------------------------------------------------------------------- 1 | package couchbase 2 | 3 | import ( 4 | "testing" 5 | "unsafe" 6 | ) 7 | 8 | func testBucket() Bucket { 9 | b := Bucket{vBucketServerMap: unsafe.Pointer(&VBucketServerMap{ 10 | VBucketMap: make([][]int, 256), 11 | })} 12 | return b 13 | } 14 | 15 | /* 16 | key: k0 master: 10.1.7.1:11210 vBucketId: 9 couchApiBase: http://10.1.7.1:8092/default replicas: 10.1.7.2:11210 17 | key: k1 master: 10.1.7.1:11210 vBucketId: 14 couchApiBase: http://10.1.7.1:8092/default replicas: 10.1.7.3:11210 18 | key: k2 master: 10.1.7.1:11210 vBucketId: 7 couchApiBase: http://10.1.7.1:8092/default replicas: 10.1.7.2:11210 19 | key: k3 master: 10.1.7.1:11210 vBucketId: 0 couchApiBase: http://10.1.7.1:8092/default replicas: 10.1.7.2:11210 20 | key: k4 master: 10.1.7.2:11210 vBucketId: 100 couchApiBase: http://10.1.7.2:8092/default replicas: 10.1.7.5:11210 21 | key: k5 master: 10.1.7.2:11210 vBucketId: 99 couchApiBase: http://10.1.7.2:8092/default replicas: 10.1.7.5:11210 22 | */ 23 | 24 | func TestVBHash(t *testing.T) { 25 | b := testBucket() 26 | m := map[string]uint32{ 27 | "k0": 9, 28 | "k1": 14, 29 | "k2": 7, 30 | "k3": 0, 31 | "k4": 100, 32 | "k5": 99, 33 | } 34 | 35 | for k, v := range m { 36 | assert(t, k, b.VBHash(k), v) 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /secondary/docs/JanCodeDrop.md: -------------------------------------------------------------------------------- 1 | ##2i Jan Code Drop Features 2 | 3 | ###Indexer 4 | 5 | ####QE Ready 6 | 7 | - Create/Drop/List/Scan On Single Indexer Node 8 | - Multi Indexer Nodes 9 | - Rebalance KV Node In/Out 10 | - Failover KV Node(Graceful) 11 | - Bucket Delete 12 | - Projector Process Failure Recovery 13 | - KV Failure(Memcached Crash/Node Restart) Recovery 14 | - Indexer Process Failure Recovery 15 | - Deferred Index Build 16 | - Auto-discovery of Cluster Services 17 | - CbAuth Integration 18 | - Configurable Indexer Settings 19 | - Indexer Statistics 20 | 21 | ####Implemented 22 | 23 | - Failover KV Node(Hard) [MB-13239](https://issues.couchbase.com/browse/MB-13239) 24 | - Bucket Flush [MB-13239](https://issues.couchbase.com/browse/MB-13239) 25 | - Index Compaction [MB-13111](https://issues.couchbase.com/browse/MB-13239) 26 | - In-memory Snapshots 27 | - Multi Buckets 28 | 29 | ### N1QL 30 | 31 | - CREATE PRIMARY INDEX 32 | * any bucket, any number of times. 33 | * with deployment plan. 34 | * with or without deferred build. 35 | - CREATE INDEX 36 | * any bucket, any number of times, any expression. 37 | * WHERE clause 38 | * with deployment plan. 39 | * with or without deferred build. 40 | - BUILD INDEX. 41 | - DROP INDEX. 42 | - LIST INDEX. 43 | - All kinds SELECT. 44 | 45 | -------------------------------------------------------------------------------- /secondary/docs/README.md: -------------------------------------------------------------------------------- 1 | Design Documents 2 | ================ 3 | 4 | Design documents for secondary index production project. 5 | 6 | -------------------------------------------------------------------------------- /secondary/docs/dcp_requirements.txt: -------------------------------------------------------------------------------- 1 | interfaces to UPR. 2 | 3 | - all interfaces are abstracted in projector/upr.go 4 | - upr.go references Timestamp structure defined in common/timestamp.go. 5 | 6 | Other than that, following are expectations from dcp client, 7 | 8 | - when a new vbucket stream start, a MutationEvent with command as StreamBegin 9 | is expected from upr. 10 | - similarly when a stream end, a MutationEvent with command as StreamEnd is 11 | expected from upr. 12 | - in the latest iteration of projector code, I have incorporated OpSync 13 | generation into projector, hence dcp client don't have to worry about that. 14 | - StreamBegin and StreamEnd events are guaranteed by couchbase-client. 15 | - if channel with dcp client is closed abruptly, then projector will take care 16 | of generating StreamEnd for all active vbucket streams. 17 | - as per interface, failoverTimestamp and kvTimestamp are expected back from 18 | upr-client, but I can help here once the protocol sequences are sorted out. 19 | - there seem to be some discussion around snapshot. 20 | - I am assuming that ADD_STREAM, CLOSE_STREAM, SET_VBUCKET_STATE are not 21 | relevant for secondary index. 22 | - Right now the Opcommand for MutationEvent does not account for 23 | SNAPSHOT_MARKER, if that is required you can add that and send it downstream 24 | to projector, I will handle it. 25 | - EXPIRATION and FLUSH commands from DCP are not yet handled by secondary index. 26 | 27 | There seem to be some discussions around snapshot_start and snapshot_end for 28 | STREAM_REQUEST. This particular change will affect several parts of seconday 29 | index, hence John, Deep and Sarath will have to be involved in those 30 | discussions. 31 | -------------------------------------------------------------------------------- /secondary/docs/design/README.md: -------------------------------------------------------------------------------- 1 | ##Couchbase Secondary Indexes 2 | 3 | Secondary Index Project aims to provide an alternative to the existing map-reduce based 4 | *Hash Partitioned Secondary Indexes*(known as Views) available in Couchbase. 5 | This project is being built from ground up to improve the existing solution and 6 | support new use-cases e.g. N1QL. 7 | 8 | ###Goals and Motivation 9 | 10 | - **Performance** – Support index clustering to avoid scatter/gather and index working set management 11 | - **Independent Scaling** – Enable secondary index to scale independent of Key-value workload 12 | - **Partitioning Scheme Flexibility** – Can support for different partitioning schemes for index data (hash, range, key) 13 | - **Indexing Structure Flexibility** – Can support for different indexing structure (b-tree/trie, hash, R+ tree) 14 | - **Scan Consistency** – Using timestamp to ensure query stability and consistency 15 | 16 | 17 | ###Version1 Features 18 | 19 | #####Indexing 20 | - Key Based Partitioning Support 21 | - Replicated Index Data to support Index Node Failure 22 | - Horizontal Scalability(Rebalance) 23 | - Cluster Management using ns_server(master election) 24 | - Distributed Index Metadata Management 25 | - ForestDB Integration as backend for Persistence/Query 26 | - Support for Crash Recovery and Compaction 27 | - Error Management (Recovery for all Indexing component failures) 28 | - Administration UI (Management And Statistics) 29 | 30 | #####KV Related 31 | - Independent Scaling from KV Cluster 32 | - Mutation Stream via DCP 33 | - Support KV Failover(data loss), Rebalance 34 | 35 | #####Query Related 36 | - Consistency/Stability Options 37 | - Active Replica for Query 38 | 39 | 40 | Please see [Design Documentation](overview.md) for more details. 41 | 42 | -------------------------------------------------------------------------------- /secondary/docs/design/markdown/deployment.md: -------------------------------------------------------------------------------- 1 | ##Deployment Options 2 | 3 | Below diagram depicts the production deployment for Secondary Indexes.
4 | KV, Index and Query should be running in their own clusters. 5 |
6 | 7 | ![](https://rawgithub.com/couchbase/indexing/master/secondary/docs/design/images/Deployment.svg) 8 | 9 |
10 | In development environment, KV, Indexer and Query can be colocated on the same node. 11 | -------------------------------------------------------------------------------- /secondary/docs/design/markdown/indexer.md: -------------------------------------------------------------------------------- 1 | ##Indexer 2 | -------------------------------------------------------------------------------- /secondary/docs/design/markdown/system_query.md: -------------------------------------------------------------------------------- 1 | ## System Diagram Index-Query 2 | 3 | ###Index-Query System Diagram For Scan Request 4 | 5 | ![](https://rawgithub.com/couchbase/indexing/master/secondary/docs/design/images/SystemDiagramScan.svg) 6 | 7 | __Annotations__ 8 | 9 | 1. Index Scan Request from Query Server to Index Client. 10 | 2. Index Client sends the Scan request to __ANY__ Indexer(which gets promoted to scan co-ordinator). 11 | 3. Scan Co-ordinator requests required Indexers(local and remote) to run the Scan. 12 | 4. Scan Co-ordinator gathers the results. 13 | 5. Local Indexer runs a Scan on persisted index snapshots to formulate the results. 14 | 6. Scan Co-ordinator returns the results to requesting Index Client. 15 | 7. Scan Results are returned to Query Server. 16 | 17 | *For detailed execution flow and time-ordering of events, see [Query Execution Flow](query.md).* 18 | 19 | ###Index-Query System Diagram For DDL Request 20 | 21 | 22 | ![](https://rawgithub.com/couchbase/indexing/master/secondary/docs/design/images/SystemDiagramDDL.svg) 23 | 24 | __Annotations__ 25 | 26 | 1. Index DDL Request from Query Server to Index Client. 27 | 2. Index Client sends the DDL request to Index Coordinator Master. 28 | 3. Index Coordinator Master decides the topology and informs participating Local Indexers to process DDL. 29 | 4. Local Indexers allocates/deallocates storage for new DDL request. 30 | 5. Index Coordinator Master replicates the updated metadata. 31 | 6. Metadata is persisted at both master and replica Index Coordinator. 32 | 7. Index Coordinator Master notifies __ALL__ KV projectors for the new DDL request. 33 | 8. Index Coordinator returns status of DDL request to Index Client. 34 | 9. DDL Request Status is returned to Query Server. 35 | 36 | *For detailed execution flow and time-ordering of events, see [Initial Index Build](initialbuild.md).* 37 | -------------------------------------------------------------------------------- /secondary/docs/design/other/CB_Cluster_2iQuery_Administration.pdf: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/couchbase/indexing/ba752cd53c7865086c4d99f312eafbe3d23982a9/secondary/docs/design/other/CB_Cluster_2iQuery_Administration.pdf -------------------------------------------------------------------------------- /secondary/docs/design/other/presentation.pdf: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/couchbase/indexing/ba752cd53c7865086c4d99f312eafbe3d23982a9/secondary/docs/design/other/presentation.pdf -------------------------------------------------------------------------------- /secondary/fdb/.gitignore: -------------------------------------------------------------------------------- 1 | # Compiled Object files, Static and Dynamic libs (Shared Objects) 2 | *.o 3 | *.a 4 | *.so 5 | 6 | # Folders 7 | _obj 8 | _test 9 | 10 | # Architecture specific extensions/prefixes 11 | *.[568vq] 12 | [568vq].out 13 | 14 | *.cgo1.go 15 | *.cgo2.c 16 | _cgo_defun.c 17 | _cgo_gotypes.go 18 | _cgo_export.* 19 | 20 | _testmain.go 21 | 22 | *.exe 23 | 24 | # sublime 25 | *.sublime-* -------------------------------------------------------------------------------- /secondary/fdb/README.md: -------------------------------------------------------------------------------- 1 | # goforestdb 2 | 3 | Go bindings for ForestDB 4 | 5 | ## Building 6 | 7 | 1. Obtain and build forestdb: https://github.com/couchbaselabs/forestdb (run `make install` to install the library) 8 | 1. Install header files to system location 9 | 1. On Ubuntu 14.04: `cd && mkdir /usr/local/include/libforestdb && cp include/libforestdb/* /usr/local/include/libforestdb` 10 | 1. `go get -u -v -t github.com/couchbase/goforestdb` 11 | 12 | ## Documentation 13 | 14 | See [godocs](http://godoc.org/github.com/couchbase/goforestdb) 15 | 16 | ## Sample usage (without proper error handling): 17 | 18 | // Open a database 19 | db, _ := Open("test", nil) 20 | 21 | // Close it properly when we're done 22 | defer db.Close() 23 | 24 | // Store the document 25 | doc, _ := NewDoc([]byte("key"), nil, []byte("value")) 26 | defer doc.Close() 27 | db.Set(doc) 28 | 29 | // Lookup the document 30 | doc2, _ := NewDoc([]byte("key"), nil, nil) 31 | defer doc2.Close() 32 | db.Get(doc2) 33 | 34 | // Delete the document 35 | doc3, _ := NewDoc([]byte("key"), nil, nil) 36 | defer doc3.Close() 37 | db.Delete(doc3) 38 | -------------------------------------------------------------------------------- /secondary/fdb/advlock.go: -------------------------------------------------------------------------------- 1 | package forestdb 2 | 3 | import "runtime/debug" 4 | 5 | type advLock struct { 6 | initialized bool 7 | ch chan int 8 | } 9 | 10 | func newAdvLock() advLock { 11 | l := advLock{} 12 | l.Init() 13 | return l 14 | } 15 | 16 | func (l *advLock) Init() { 17 | if !l.initialized { 18 | l.ch = make(chan int, 1) 19 | l.ch <- 1 20 | l.initialized = true 21 | } 22 | } 23 | 24 | func (l *advLock) Destroy() { 25 | l.initialized = false 26 | close(l.ch) 27 | } 28 | 29 | func (l *advLock) Lock() { 30 | printStack := true 31 | loop: 32 | for { 33 | select { 34 | case <-l.ch: 35 | break loop 36 | default: 37 | if printStack { 38 | Log.Debugf("Unable to acquire lock\n%s", string(debug.Stack())) 39 | printStack = false 40 | } 41 | } 42 | } 43 | } 44 | 45 | func (l *advLock) Unlock() { 46 | l.ch <- 1 47 | } 48 | -------------------------------------------------------------------------------- /secondary/fdb/docs.go: -------------------------------------------------------------------------------- 1 | /* 2 | ForestDB bindings for go 3 | 4 | Sample usage (without proper error handling): 5 | 6 | // Open a database 7 | db, _ := Open("test", nil) 8 | 9 | // Close it properly when we're done 10 | defer db.Close() 11 | 12 | // Store the document 13 | doc, _ := NewDoc([]byte("key"), nil, []byte("value")) 14 | defer doc.Close() 15 | db.Set(doc) 16 | 17 | // Lookup the document 18 | doc2, _ := NewDoc([]byte("key"), nil, nil) 19 | defer doc2.Close() 20 | db.Get(doc2) 21 | 22 | // Delete the document 23 | doc3, _ := NewDoc([]byte("key"), nil, nil) 24 | defer doc3.Close() 25 | db.Delete(doc3) 26 | */ 27 | package forestdb 28 | -------------------------------------------------------------------------------- /secondary/fdb/error.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash -e 2 | 3 | cat >error.go < 9 | import "C" 10 | import "fmt" 11 | type Error int 12 | func (e Error) Error() string { 13 | msg, ok := codeMap[int(e)] 14 | if !ok { 15 | msg = fmt.Sprintf("Unknown forestdb error %d", int(e)) 16 | } 17 | return msg 18 | } 19 | const RESULT_SUCCESS C.fdb_status = C.FDB_RESULT_SUCCESS 20 | const RESULT_ITERATOR_FAIL = FDB_RESULT_ITERATOR_FAIL 21 | EOF 22 | 23 | header="../../../../../../../forestdb/include/libforestdb/fdb_errors.h" 24 | symbols="`grep -oh 'FDB_RESULT_[A-Z0-9_]*' $header | sort | uniq`" 25 | 26 | echo "$symbols" | \ 27 | awk 'BEGIN {print "const ("}; \ 28 | /FDB_RESULT_LAST/ {next;} \ 29 | /FDB_RESULT_/ {print $1 " Error = C." $1} \ 30 | END {print ")"}' \ 31 | >> error.go 32 | 33 | echo "$symbols" | \ 34 | awk 'BEGIN {print "var codeMap = map[int]string{"}; \ 35 | /FDB_RESULT_LAST/ {next;} \ 36 | /FDB_RESULT_/ {print "C." $1 " : `" $1 "`,"} \ 37 | END {print "}"}' \ 38 | >> error.go 39 | 40 | gofmt -w error.go 41 | -------------------------------------------------------------------------------- /secondary/fdb/examples/custom_comparator/comparator.c: -------------------------------------------------------------------------------- 1 | #include 2 | #include "comparator.h" 3 | 4 | extern int CompareBytesReversed(void *key1, size_t keylen1, void *key2, size_t keylen2); 5 | 6 | int cmp_variable(void *key1, size_t keylen1, void *key2, size_t keylen2) { 7 | return CompareBytesReversed(key1, keylen1, key2, keylen2); 8 | } 9 | -------------------------------------------------------------------------------- /secondary/fdb/examples/custom_comparator/comparator.go: -------------------------------------------------------------------------------- 1 | package custom_comparator 2 | 3 | //#include 4 | import "C" 5 | 6 | import ( 7 | "bytes" 8 | "unsafe" 9 | ) 10 | 11 | //export CompareBytesReversed 12 | func CompareBytesReversed(k1 unsafe.Pointer, k1len C.size_t, k2 unsafe.Pointer, k2len C.size_t) int { 13 | key1 := C.GoBytes(k1, C.int(k1len)) 14 | key2 := C.GoBytes(k2, C.int(k2len)) 15 | return -bytes.Compare(key1, key2) 16 | } 17 | -------------------------------------------------------------------------------- /secondary/fdb/examples/custom_comparator/comparator.h: -------------------------------------------------------------------------------- 1 | #include 2 | 3 | int cmp_variable(void *key1, size_t keylen1, void *key2, size_t keylen2); -------------------------------------------------------------------------------- /secondary/fdb/examples/custom_comparator/other.go: -------------------------------------------------------------------------------- 1 | package custom_comparator 2 | 3 | //#include "comparator.h" 4 | import "C" 5 | 6 | import ( 7 | "unsafe" 8 | ) 9 | 10 | var CompareBytesReversedPointer unsafe.Pointer = unsafe.Pointer(C.cmp_variable) 11 | -------------------------------------------------------------------------------- /secondary/fdb/log.go: -------------------------------------------------------------------------------- 1 | package forestdb 2 | 3 | // Logger interface 4 | type Logger interface { 5 | // Warnings, logged by default. 6 | Warnf(format string, v ...interface{}) 7 | // Errors, logged by default. 8 | Errorf(format string, v ...interface{}) 9 | // Fatal errors. Will not terminate execution. 10 | Fatalf(format string, v ...interface{}) 11 | // Informational messages. 12 | Infof(format string, v ...interface{}) 13 | // Timing utility 14 | Debugf(format string, v ...interface{}) 15 | // Program execution tracing. Not logged by default 16 | Tracef(format string, v ...interface{}) 17 | } 18 | 19 | type Dummy struct { 20 | } 21 | 22 | func (*Dummy) Fatalf(_ string, _ ...interface{}) { 23 | } 24 | 25 | func (*Dummy) Errorf(_ string, _ ...interface{}) { 26 | } 27 | 28 | func (*Dummy) Warnf(_ string, _ ...interface{}) { 29 | } 30 | 31 | func (*Dummy) Infof(_ string, _ ...interface{}) { 32 | } 33 | 34 | func (*Dummy) Debugf(_ string, _ ...interface{}) { 35 | } 36 | 37 | func (*Dummy) Tracef(_ string, _ ...interface{}) { 38 | } 39 | 40 | // Logger to use 41 | var Log Logger = &Dummy{} 42 | -------------------------------------------------------------------------------- /secondary/indexer/pause_copier_community.go: -------------------------------------------------------------------------------- 1 | //go:build community 2 | // +build community 3 | 4 | package indexer 5 | 6 | import c "github.com/couchbase/indexing/secondary/common" 7 | 8 | func setCopierTestConfigIfEnabled(config c.Config) (string, bool) { 9 | return "", false 10 | } 11 | 12 | func unsetCopierTestConfigIfEnabled(config c.Config) (string, bool) { 13 | return "", false 14 | } 15 | 16 | func MakeFileCopier(location, region, endpoint, staging string, ratelimit int, cfg c.Config) (Copier, error) { 17 | return nil, nil 18 | } 19 | 20 | func MakeFileCopierForPauseResume(task *taskObj, cfg c.Config) (Copier, error) { 21 | return nil, nil 22 | } 23 | -------------------------------------------------------------------------------- /secondary/indexer/rebalance_provider.go: -------------------------------------------------------------------------------- 1 | package indexer 2 | 3 | import "github.com/couchbase/indexing/secondary/manager" 4 | 5 | type RebalanceProvider interface { 6 | Cancel() 7 | InitGlobalTopology(*manager.ClusterIndexMetadata) 8 | } 9 | -------------------------------------------------------------------------------- /secondary/indexer/shard_transfer_manager_community.go: -------------------------------------------------------------------------------- 1 | //go:build community 2 | // +build community 3 | 4 | package indexer 5 | 6 | import "github.com/couchbase/indexing/secondary/common" 7 | 8 | type ShardTransferManager struct { 9 | config common.Config 10 | cmdCh chan Message 11 | } 12 | 13 | func NewShardTransferManager(config common.Config, supvWrkrCh chan Message) *ShardTransferManager { 14 | return nil 15 | } 16 | 17 | func (stm *ShardTransferManager) ProcessCommand(cmd Message) { 18 | // no-op 19 | } 20 | -------------------------------------------------------------------------------- /secondary/indexer/snapshot.go: -------------------------------------------------------------------------------- 1 | // Copyright 2014-Present Couchbase, Inc. 2 | // 3 | // Use of this software is governed by the Business Source License included 4 | // in the file licenses/BSL-Couchbase.txt. As of the Change Date specified 5 | // in that file, in accordance with the Business Source License, use of this 6 | // software will be governed by the Apache License, Version 2.0, included in 7 | // the file licenses/APL2.txt. 8 | 9 | package indexer 10 | 11 | import ( 12 | "github.com/couchbase/indexing/secondary/common" 13 | ) 14 | 15 | //Snapshot interface 16 | type Snapshot interface { 17 | IndexReader 18 | 19 | Open() error 20 | Close() error 21 | IsOpen() bool 22 | 23 | Id() SliceId 24 | IndexInstId() common.IndexInstId 25 | IndexDefnId() common.IndexDefnId 26 | 27 | Timestamp() *common.TsVbuuid 28 | 29 | Info() SnapshotInfo 30 | } 31 | 32 | type SnapshotInfo interface { 33 | Timestamp() *common.TsVbuuid 34 | IsCommitted() bool 35 | IsOSOSnap() bool 36 | Stats() map[string]interface{} 37 | } 38 | -------------------------------------------------------------------------------- /secondary/indexer/snapshot_map.go: -------------------------------------------------------------------------------- 1 | package indexer 2 | 3 | import ( 4 | "sync" 5 | "sync/atomic" 6 | "unsafe" 7 | 8 | "github.com/couchbase/indexing/secondary/common" 9 | ) 10 | 11 | type IndexSnapMap map[common.IndexInstId]*IndexSnapshotContainer 12 | 13 | type IndexSnapshotContainer struct { 14 | sync.Mutex 15 | snap IndexSnapshot 16 | deleted bool 17 | } 18 | 19 | type IndexSnapMapHolder struct { 20 | ptr *unsafe.Pointer 21 | } 22 | 23 | func (ism *IndexSnapMapHolder) Init() { 24 | ism.ptr = new(unsafe.Pointer) 25 | } 26 | 27 | func (ism *IndexSnapMapHolder) Set(indexSnapMap IndexSnapMap) { 28 | atomic.StorePointer(ism.ptr, unsafe.Pointer(&indexSnapMap)) 29 | } 30 | 31 | func (ism *IndexSnapMapHolder) Get() IndexSnapMap { 32 | if ptr := atomic.LoadPointer(ism.ptr); ptr != nil { 33 | return *(*IndexSnapMap)(ptr) 34 | } else { 35 | return make(IndexSnapMap) 36 | } 37 | } 38 | 39 | func (ipm *IndexSnapMapHolder) Clone() IndexSnapMap { 40 | if ptr := atomic.LoadPointer(ipm.ptr); ptr != nil { 41 | currMap := *(*IndexSnapMap)(ptr) 42 | return copyIndexSnapMap(currMap) 43 | } else { 44 | return make(IndexSnapMap) 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /secondary/indexer/storage_stats_manager_community.go: -------------------------------------------------------------------------------- 1 | // Copyright 2023-Present Couchbase, Inc. 2 | // 3 | // Use of this software is governed by the Business Source License included 4 | // in the file licenses/BSL-Couchbase.txt. As of the Change Date specified 5 | // in that file, in accordance with the Business Source License, use of this 6 | // software will be governed by the Apache License, Version 2.0, included in 7 | // the file licenses/APL2.txt. 8 | 9 | //go:build community 10 | // +build community 11 | 12 | package indexer 13 | 14 | func populateAggregatedStorageMetrics(st []byte) []byte { 15 | return st 16 | } 17 | 18 | func populateStorageTenantMetrics(st []byte) []byte { 19 | return st 20 | } 21 | -------------------------------------------------------------------------------- /secondary/indexer/waiters_map.go: -------------------------------------------------------------------------------- 1 | package indexer 2 | 3 | import ( 4 | "sync" 5 | "sync/atomic" 6 | "unsafe" 7 | 8 | "github.com/couchbase/indexing/secondary/common" 9 | ) 10 | 11 | type SnapshotWaitersContainer struct { 12 | sync.Mutex 13 | waiters []*snapshotWaiter 14 | } 15 | 16 | type SnapshotWaitersMap map[common.IndexInstId]*SnapshotWaitersContainer 17 | 18 | type SnapshotWaitersMapHolder struct { 19 | ptr *unsafe.Pointer 20 | } 21 | 22 | func (swmh *SnapshotWaitersMapHolder) Init() { 23 | swmh.ptr = new(unsafe.Pointer) 24 | } 25 | 26 | func (swmh *SnapshotWaitersMapHolder) Set(waiterMap SnapshotWaitersMap) { 27 | atomic.StorePointer(swmh.ptr, unsafe.Pointer(&waiterMap)) 28 | } 29 | 30 | func (swmh *SnapshotWaitersMapHolder) Get() SnapshotWaitersMap { 31 | if ptr := atomic.LoadPointer(swmh.ptr); ptr != nil { 32 | return *(*SnapshotWaitersMap)(ptr) 33 | } else { 34 | return make(SnapshotWaitersMap) 35 | } 36 | } 37 | 38 | func (swmh *SnapshotWaitersMapHolder) Clone() SnapshotWaitersMap { 39 | if ptr := atomic.LoadPointer(swmh.ptr); ptr != nil { 40 | currMap := *(*SnapshotWaitersMap)(ptr) 41 | return copySnapshotWaiterMap(currMap) 42 | } else { 43 | return make(SnapshotWaitersMap) 44 | } 45 | } 46 | 47 | func copySnapshotWaiterMap(waiterMap SnapshotWaitersMap) SnapshotWaitersMap { 48 | cloneMap := make(SnapshotWaitersMap) 49 | for instId, waiterContainer := range waiterMap { 50 | cloneMap[instId] = waiterContainer 51 | } 52 | return cloneMap 53 | } 54 | -------------------------------------------------------------------------------- /secondary/manager/test/README.md: -------------------------------------------------------------------------------- 1 | 2 | To run test, you will need to run an instance of gmeta in the same host. To run gometa, you need a config file as follows: 3 | 4 | { 5 | "Host" : { 6 | "ElectionAddr" : "localhost:9883", 7 | "MessageAddr" : "localhost:9884", 8 | "RequestAddr" : "localhost:9885" 9 | } 10 | } 11 | 12 | To run gometa 13 | 14 | -config="" 15 | 16 | -------------------------------------------------------------------------------- /secondary/manager/test/config.json: -------------------------------------------------------------------------------- 1 | { 2 | "Host" : { 3 | "ElectionAddr" : "localhost:5000", 4 | "MessageAddr" : "localhost:5001", 5 | "RequestAddr" : "localhost:5002" 6 | } 7 | } 8 | 9 | -------------------------------------------------------------------------------- /secondary/memdb/README.md: -------------------------------------------------------------------------------- 1 | # memdb 2 | 3 | A high performance DRAM storage engine 4 | 5 | ### License 6 | 7 | Apache 2.0 8 | -------------------------------------------------------------------------------- /secondary/memdb/nodelist.go: -------------------------------------------------------------------------------- 1 | package memdb 2 | 3 | import ( 4 | "bytes" 5 | "github.com/couchbase/indexing/secondary/memdb/skiplist" 6 | ) 7 | 8 | type NodeList struct { 9 | head *skiplist.Node 10 | exposeItemCopy bool 11 | } 12 | 13 | func NewNodeList(head *skiplist.Node, exposeItemCopy bool) *NodeList { 14 | return &NodeList{ 15 | head: head, 16 | exposeItemCopy: exposeItemCopy, 17 | } 18 | } 19 | 20 | func (l *NodeList) Keys() (keys [][]byte) { 21 | var key []byte 22 | var itm *Item 23 | 24 | node := l.head 25 | for node != nil { 26 | itm = (*Item)(node.Item()) 27 | if l.exposeItemCopy { 28 | // Exposed to GSI slice mutation path, return copy 29 | key = itm.BytesCopy() 30 | } else { 31 | key = itm.Bytes() 32 | } 33 | 34 | keys = append(keys, key) 35 | node = node.GetLink() 36 | } 37 | 38 | return 39 | } 40 | 41 | func (l *NodeList) Remove(key []byte) *skiplist.Node { 42 | var prev *skiplist.Node 43 | node := l.head 44 | for node != nil { 45 | nodeKey := (*Item)(node.Item()).Bytes() 46 | if bytes.Equal(nodeKey, key) { 47 | if prev == nil { 48 | l.head = node.GetLink() 49 | return node 50 | } else { 51 | prev.SetLink(node.GetLink()) 52 | return node 53 | } 54 | } 55 | prev = node 56 | node = node.GetLink() 57 | } 58 | 59 | return nil 60 | } 61 | 62 | func (l *NodeList) Add(node *skiplist.Node) { 63 | node.SetLink(l.head) 64 | l.head = node 65 | } 66 | 67 | func (l *NodeList) Head() *skiplist.Node { 68 | return l.head 69 | } 70 | -------------------------------------------------------------------------------- /secondary/memdb/nodelist_test.go: -------------------------------------------------------------------------------- 1 | package memdb 2 | 3 | import ( 4 | "fmt" 5 | "testing" 6 | ) 7 | 8 | func TestNodeList(t *testing.T) { 9 | db := New() 10 | defer db.Close() 11 | 12 | n := 10 13 | var list *NodeList 14 | w := db.NewWriter() 15 | for i := 0; i < n; i++ { 16 | ptr := w.Put2([]byte(fmt.Sprintf("%010d", i))) 17 | if list == nil { 18 | list = NewNodeList(ptr, false) 19 | } else { 20 | list.Add(ptr) 21 | } 22 | } 23 | 24 | count := 0 25 | for i, k := range list.Keys() { 26 | expected := fmt.Sprintf("%010d", n-i-1) 27 | if expected != string(k) { 28 | t.Errorf("Expected %s, got %s", expected, string(k)) 29 | } 30 | count++ 31 | } 32 | 33 | if count != n { 34 | t.Errorf("Expected %d, got %d", n, count) 35 | } 36 | 37 | list.Remove([]byte(fmt.Sprintf("%010d", 2))) 38 | list.Remove([]byte(fmt.Sprintf("%010d", 5))) 39 | list.Remove([]byte(fmt.Sprintf("%010d", 8))) 40 | 41 | count = len(list.Keys()) 42 | if count != n-3 { 43 | t.Errorf("Expected %d, got %d", n-3, count) 44 | } 45 | 46 | for i := 10; i < 13; i++ { 47 | ptr := w.Put2([]byte(fmt.Sprintf("%010d", i))) 48 | list.Add(ptr) 49 | } 50 | 51 | count = len(list.Keys()) 52 | if count != n { 53 | t.Errorf("Expected %d, got %d", n, count) 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /secondary/memdb/skiplist/item.go: -------------------------------------------------------------------------------- 1 | package skiplist 2 | 3 | import ( 4 | "bytes" 5 | "fmt" 6 | "unsafe" 7 | ) 8 | 9 | func compare(cmp CompareFn, this, that unsafe.Pointer) int { 10 | if this == nil { 11 | return 1 12 | } else if that == nil { 13 | return -1 14 | } 15 | 16 | return cmp(this, that) 17 | } 18 | 19 | type byteKeyItem []byte 20 | 21 | func (itm *byteKeyItem) String() string { 22 | return string(*itm) 23 | } 24 | 25 | func (itm byteKeyItem) Size() int { 26 | return len(itm) 27 | } 28 | 29 | func NewByteKeyItem(k []byte) unsafe.Pointer { 30 | itm := byteKeyItem(k) 31 | return unsafe.Pointer(&itm) 32 | } 33 | 34 | func CompareBytes(this, that unsafe.Pointer) int { 35 | thisItem := (*byteKeyItem)(this) 36 | thatItem := (*byteKeyItem)(that) 37 | return bytes.Compare([]byte(*thisItem), []byte(*thatItem)) 38 | } 39 | 40 | type intKeyItem int 41 | 42 | func (itm *intKeyItem) String() string { 43 | return fmt.Sprint(*itm) 44 | } 45 | 46 | func (itm intKeyItem) Size() int { 47 | return int(unsafe.Sizeof(itm)) 48 | } 49 | 50 | func CompareInt(this, that unsafe.Pointer) int { 51 | thisItem := (*intKeyItem)(this) 52 | thatItem := (*intKeyItem)(that) 53 | return int(*thisItem - *thatItem) 54 | } 55 | -------------------------------------------------------------------------------- /secondary/memdb/skiplist/merger_test.go: -------------------------------------------------------------------------------- 1 | package skiplist 2 | 3 | import "testing" 4 | import "fmt" 5 | 6 | func TestMerger(t *testing.T) { 7 | var lists []*Skiplist 8 | var iters []*Iterator 9 | 10 | s := New() 11 | cmp := CompareBytes 12 | buf := s.MakeBuf() 13 | defer s.FreeBuf(buf) 14 | 15 | n := 5 16 | 17 | for i := 0; i < n; i++ { 18 | lists = append(lists, New()) 19 | } 20 | 21 | for i := 0; i < 10000; i++ { 22 | if i >= 1000 && i <= 8000 && i%n == 0 { 23 | continue 24 | } 25 | s := lists[i%n] 26 | s.Insert(NewByteKeyItem([]byte(fmt.Sprintf("%010d", i))), cmp, buf, &s.Stats) 27 | } 28 | 29 | for i := 0; i < n; i++ { 30 | buf := s.MakeBuf() 31 | iters = append(iters, lists[i].NewIterator(cmp, buf)) 32 | } 33 | 34 | mit := NewMergeIterator(iters) 35 | 36 | i := 0 37 | for mit.SeekFirst(); mit.Valid(); i++ { 38 | if i >= 1000 && i <= 8000 && i%n == 0 { 39 | continue 40 | } 41 | expected := fmt.Sprintf("%010d", i) 42 | got := string(*((*byteKeyItem)(mit.Get()))) 43 | if got != expected { 44 | t.Errorf("Expected %s, got %v", expected, got) 45 | } 46 | mit.Next() 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /secondary/memdb/system_windows.go: -------------------------------------------------------------------------------- 1 | // Copyright 2013-Present Couchbase, Inc. 2 | // 3 | // Use of this software is governed by the Business Source License included 4 | // in the file licenses/BSL-Couchbase.txt. As of the Change Date specified 5 | // in that file, in accordance with the Business Source License, use of this 6 | // software will be governed by the Apache License, Version 2.0, included in 7 | // the file licenses/APL2.txt. 8 | 9 | // +build windows 10 | 11 | package memdb 12 | 13 | func GetDefaultNumFD() int { 14 | // window - default 512 file descriptor per process 15 | return 512 16 | } 17 | 18 | func GetIOConcurrency(concurrency float64) int { 19 | 20 | var limit int 21 | 22 | if concurrency > 1 { 23 | limit = int(concurrency) 24 | } else if concurrency <= 0 { 25 | limit = int(float64(GetDefaultNumFD()) * defaultIOConcurrency) 26 | } else { 27 | limit = int(float64(GetDefaultNumFD()) * concurrency) 28 | } 29 | 30 | // Throttle IO concurrency based on golang thread limit. Unless we provide a way to increase the number 31 | // of threads per golang process, this will set a hard limit of IO concurrency at ~7K. There is a way 32 | // to increase threads per golang process to allow higher IO concurrency, but we should not do that unless 33 | // we have test system stability under such a high thread count. 34 | numThreads := int(float64(limit) * threadFDMultiplier) 35 | if numThreads > maxThreadLimit { 36 | limit = defaultThreadLimit 37 | } 38 | 39 | return limit 40 | } 41 | -------------------------------------------------------------------------------- /secondary/planner/sample/index.json: -------------------------------------------------------------------------------- 1 | [{"name" : "index1", 2 | "bucket" : "bucket2", 3 | "isPrimary" : false, 4 | "secExprs" : ["name"], 5 | "isArrayIndex" : false, 6 | "replica" : 3, 7 | "numDoc" : 5000, 8 | "DocKeySize" : 200, 9 | "SecKeySize" : 200, 10 | "ArrKeySize" : 0, 11 | "ArrSize" : 0, 12 | "MutationRate" : 0, 13 | "ScanRate" : 0}, 14 | {"name" : "index2", 15 | "bucket" : "bucket2", 16 | "isPrimary" : false, 17 | "secExprs" : ["name"], 18 | "isArrayIndex" : false, 19 | "replica" : 3, 20 | "numDoc" : 5000, 21 | "DocKeySize" : 200, 22 | "SecKeySize" : 200, 23 | "ArrKeySize" : 0, 24 | "ArrSize" : 0, 25 | "MutationRate" : 0, 26 | "ScanRate" : 0}] 27 | -------------------------------------------------------------------------------- /secondary/planner/sample/mixedWorkload.json: -------------------------------------------------------------------------------- 1 | { 2 | "name" : "Mixed Workload", 3 | "comment" : "go test -iteration=1 -memQuotaFactor=2 -cpuQuotaFactor=2 -placement=mixedWorkload.json", 4 | "workload" : [ 5 | { 6 | "name" : "bucket1", 7 | "replica" : 3, 8 | "workload" : [ 9 | { 10 | "name" : "small", 11 | "minNumDoc" : 500000, 12 | "maxNumDoc" : 20000000, 13 | "minDocKeySize" : 20, 14 | "maxDocKeySize" : 200, 15 | "minSecKeySize" : 20, 16 | "maxSecKeySize" : 200, 17 | "minArrKeySize" : 0, 18 | "maxArrKeySize" : 0, 19 | "minArrSize" : 0, 20 | "maxArrSize" : 0, 21 | "minMutationRate" : 10000, 22 | "maxMutationRate" : 30000, 23 | "minScanRate" : 1000, 24 | "maxScanRate" : 10000 25 | }, 26 | { 27 | "name" : "medium", 28 | "minNumDoc" : 30000000, 29 | "maxNumDoc" : 300000000, 30 | "minDocKeySize" : 20, 31 | "maxDocKeySize" : 200, 32 | "minSecKeySize" : 20, 33 | "maxSecKeySize" : 200, 34 | "minArrKeySize" : 0, 35 | "maxArrKeySize" : 0, 36 | "minArrSize" : 0, 37 | "maxArrSize" : 0, 38 | "minMutationRate" : 30000, 39 | "maxMutationRate" : 100000, 40 | "minScanRate" : 1000, 41 | "maxScanRate" : 10000 42 | }, 43 | { 44 | "name" : "large", 45 | "minNumDoc" : 500000000, 46 | "maxNumDoc" : 1000000000, 47 | "minDocKeySize" : 20, 48 | "maxDocKeySize" : 20, 49 | "minSecKeySize" : 20, 50 | "maxSecKeySize" : 20, 51 | "minArrKeySize" : 0, 52 | "maxArrKeySize" : 0, 53 | "minArrSize" : 0, 54 | "maxArrSize" : 0, 55 | "minMutationRate" : 200000, 56 | "maxMutationRate" : 500000, 57 | "minScanRate" : 1000, 58 | "maxScanRate" : 10000 59 | }], 60 | "distribution" : [50,40,10] 61 | }], 62 | "distribution" : [100], 63 | "minNumIndex" : 30, 64 | "maxNumIndex" : 30 65 | } 66 | 67 | -------------------------------------------------------------------------------- /secondary/planner/sample/uniformWorkload.json: -------------------------------------------------------------------------------- 1 | { 2 | "name" : "Planner Simulation 1", 3 | "comment" : "go test -iteration=1 -memQuotaFactor=3 -cpuQuotaFactor=4 -placement=uniformWorkload.json", 4 | "workload" : [ 5 | { 6 | "name" : "bucket1", 7 | "replica" : 3, 8 | "workload" : [ 9 | { 10 | "name" : "small", 11 | "minNumDoc" : 500000, 12 | "maxNumDoc" : 20000000, 13 | "minDocKeySize" : 20, 14 | "maxDocKeySize" : 200, 15 | "minSecKeySize" : 20, 16 | "maxSecKeySize" : 200, 17 | "minArrKeySize" : 0, 18 | "maxArrKeySize" : 0, 19 | "minArrSize" : 0, 20 | "maxArrSize" : 0, 21 | "minMutationRate" : 10000, 22 | "maxMutationRate" : 100000, 23 | "minScanRate" : 1000, 24 | "maxScanRate" : 10000 25 | }], 26 | "distribution" : [100] 27 | }], 28 | "distribution" : [100], 29 | "minNumIndex" : 30, 30 | "maxNumIndex" : 30 31 | } 32 | 33 | -------------------------------------------------------------------------------- /secondary/platform/platform.go: -------------------------------------------------------------------------------- 1 | // Copyright 2013-Present Couchbase, Inc. 2 | // 3 | // Use of this software is governed by the Business Source License included 4 | // in the file licenses/BSL-Couchbase.txt. As of the Change Date specified 5 | // in that file, in accordance with the Business Source License, use of this 6 | // software will be governed by the Apache License, Version 2.0, included in 7 | // the file licenses/APL2.txt. 8 | 9 | // +build !windows 10 | 11 | package platform 12 | 13 | func HideConsole(_ bool) { 14 | } 15 | -------------------------------------------------------------------------------- /secondary/platform/platform_windows.go: -------------------------------------------------------------------------------- 1 | // Copyright 2013-Present Couchbase, Inc. 2 | // 3 | // Use of this software is governed by the Business Source License included 4 | // in the file licenses/BSL-Couchbase.txt. As of the Change Date specified 5 | // in that file, in accordance with the Business Source License, use of this 6 | // software will be governed by the Apache License, Version 2.0, included in 7 | // the file licenses/APL2.txt. 8 | 9 | // +build windows 10 | 11 | package platform 12 | 13 | import "syscall" 14 | 15 | // Hide console on windows without removing it unlike -H windowsgui. 16 | func HideConsole(hide bool) { 17 | var k32 = syscall.NewLazyDLL("kernel32.dll") 18 | var cw = k32.NewProc("GetConsoleWindow") 19 | var u32 = syscall.NewLazyDLL("user32.dll") 20 | var sw = u32.NewProc("ShowWindow") 21 | hwnd, _, _ := cw.Call() 22 | if hwnd == 0 { 23 | return 24 | } 25 | if hide { 26 | var SW_HIDE uintptr = 0 27 | sw.Call(hwnd, SW_HIDE) 28 | } else { 29 | var SW_RESTORE uintptr = 9 30 | sw.Call(hwnd, SW_RESTORE) 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /secondary/projector/client/client_test.go: -------------------------------------------------------------------------------- 1 | package client 2 | 3 | import "testing" 4 | 5 | import c "github.com/couchbase/indexing/secondary/common" 6 | 7 | func TestRetry100_5(t *testing.T) { 8 | adminport := "localhost:9999" 9 | maxvbs := c.SystemConfig["maxVbuckets"].Int() 10 | config := c.SystemConfig.SectionConfig("indexer.projectorclient.", true) 11 | config.SetValue("retryInterval", 100) 12 | config.SetValue("maxRetries", 5) 13 | client := NewClient(adminport, maxvbs, config) 14 | client.GetVbmap("default", "default", []string{"localhost:9000"}) 15 | } 16 | 17 | func TestRetry0_5(t *testing.T) { 18 | adminport := "localhost:9999" 19 | maxvbs := c.SystemConfig["maxVbuckets"].Int() 20 | config := c.SystemConfig.SectionConfig("indexer.projectorclient.", true) 21 | config.SetValue("retryInterval", 0) 22 | config.SetValue("maxRetries", 5) 23 | client := NewClient(adminport, maxvbs, config) 24 | client.GetVbmap("default", "default", []string{"localhost:9000"}) 25 | } 26 | 27 | //func TestRetry100_0(t *testing.T) { 28 | // adminport := "localhost:9999" 29 | // config := c.SystemConfig.SectionConfig("indexer.projectorclient", true) 30 | // config.SetValue("retryInterval", 100) 31 | // config.SetValue("maxRetries", 0) 32 | // client := NewClient(adminport, config) 33 | // client.GetVbmap("default", "default", []string{"localhost:9000"}) 34 | //} 35 | -------------------------------------------------------------------------------- /secondary/projector/subscriber.go: -------------------------------------------------------------------------------- 1 | package projector 2 | 3 | import c "github.com/couchbase/indexing/secondary/common" 4 | 5 | // Subscriber interface abstracts engines (aka instances) 6 | // that can supply `evaluators`, to transform mutations into 7 | // custom-messages, and `routers`, to supply distribution topology 8 | // for custom-messages. 9 | type Subscriber interface { 10 | // GetEvaluators will return a map of uuid to Evaluator interface. 11 | // - return ErrorInconsistentFeed for malformed tables. 12 | GetEvaluators() (map[uint64]c.Evaluator, error) 13 | 14 | // GetRouters will return a map of uuid to Router interface. 15 | // - return ErrorInconsistentFeed for malformed tables. 16 | GetRouters() (map[uint64]c.Router, error) 17 | } 18 | -------------------------------------------------------------------------------- /secondary/projector/watcher.go: -------------------------------------------------------------------------------- 1 | package projector 2 | 3 | import "time" 4 | 5 | import c "github.com/couchbase/indexing/secondary/common" 6 | import protobuf "github.com/couchbase/indexing/secondary/protobuf/projector" 7 | import "github.com/golang/protobuf/proto" 8 | 9 | // watch for, 10 | // 1. stale feeds and shut them down. 11 | // 2. crashed routines and cleanup feeds. 12 | func (p *Projector) watcherDameon(watchInterval, staleTimeout int) { 13 | watchTick := time.NewTicker(time.Duration(watchInterval) * time.Millisecond) 14 | defer func() { 15 | watchTick.Stop() 16 | }() 17 | 18 | for { 19 | <-watchTick.C 20 | topics := p.listTopics() 21 | for _, topic := range topics { 22 | feed, err := p.GetFeed(topic) 23 | if err != nil { 24 | continue 25 | } 26 | status, err := feed.StaleCheck(staleTimeout) 27 | if status == "exit" && err != c.ErrorClosed { 28 | req := &protobuf.ShutdownTopicRequest{ 29 | Topic: proto.String(topic), 30 | } 31 | p.doShutdownTopic(req, 0xFFFE) 32 | } 33 | } 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /secondary/protobuf/data/mutation.go: -------------------------------------------------------------------------------- 1 | package protoData 2 | 3 | import "encoding/binary" 4 | import c "github.com/couchbase/indexing/secondary/common" 5 | 6 | func (pl *Payload) Value() interface{} { 7 | if pl.Vbmap != nil { 8 | return pl.Vbmap 9 | } else if pl.Vbkeys != nil { 10 | return pl.Vbkeys 11 | } else if pl.AuthRequest != nil { 12 | return pl.AuthRequest 13 | } 14 | return nil 15 | } 16 | 17 | func (kv *KeyVersions) Snapshot() (typ uint32, start, end uint64) { 18 | uuids := kv.GetUuids() 19 | keys := kv.GetKeys() 20 | oldkeys := kv.GetOldkeys() 21 | for i, cmd := range kv.GetCommands() { 22 | if byte(cmd) == c.Snapshot { 23 | typ = uint32(uuids[i]) 24 | start = binary.BigEndian.Uint64(keys[i]) 25 | end = binary.BigEndian.Uint64(oldkeys[i]) 26 | } 27 | } 28 | return 29 | } 30 | -------------------------------------------------------------------------------- /secondary/protobuf/projector/partn_key.proto: -------------------------------------------------------------------------------- 1 | syntax = "proto2"; 2 | 3 | package protoProjector; 4 | 5 | message KeyPartition { 6 | required uint64 numPartition = 1; 7 | repeated uint64 partitions = 2; 8 | repeated string endpoints = 3; 9 | } 10 | -------------------------------------------------------------------------------- /secondary/protobuf/projector/partn_single.proto: -------------------------------------------------------------------------------- 1 | syntax = "proto2"; 2 | 3 | package protoProjector; 4 | 5 | // SinglePartition is an oxymoron - the purpose of partition is to 6 | // scale-out, but using this partition-scheme for an index means the full 7 | // data set is kept on the same node. 8 | message SinglePartition { 9 | repeated string endpoints = 1; 10 | optional string coordEndpoint = 2; 11 | } 12 | -------------------------------------------------------------------------------- /secondary/protobuf/projector/partn_tp.proto: -------------------------------------------------------------------------------- 1 | syntax = "proto2"; 2 | 3 | package protoProjector; 4 | 5 | // Example TestPartition, can be used during development. 6 | message TestPartition { 7 | repeated string endpoints = 1; // endpoint address 8 | optional string coordEndpoint = 2; 9 | } 10 | -------------------------------------------------------------------------------- /secondary/protobuf/query/encoder.go: -------------------------------------------------------------------------------- 1 | package protoQuery 2 | 3 | import "github.com/couchbase/indexing/secondary/transport" 4 | import "net" 5 | 6 | const ( 7 | LenOffset int = 0 8 | LenSize int = 4 9 | FlagOffset int = LenOffset + LenSize 10 | FlagSize int = 2 11 | DataOffset int = FlagOffset + FlagSize 12 | ) 13 | 14 | func EncodeAndWrite(conn net.Conn, buf []byte, r interface{}) (err error) { 15 | var data []byte 16 | data, err = ProtobufEncodeInBuf(r, buf[transport.MaxSendBufSize:][:0]) 17 | if err != nil { 18 | return 19 | } 20 | flags := transport.TransportFlag(0).SetProtobuf() 21 | err = transport.Send(conn, buf, flags, data, false) 22 | return 23 | } 24 | -------------------------------------------------------------------------------- /secondary/queryport/app.go: -------------------------------------------------------------------------------- 1 | package queryport 2 | 3 | import ( 4 | c "github.com/couchbase/indexing/secondary/common" 5 | "github.com/couchbase/indexing/secondary/logging" 6 | 7 | "net" 8 | 9 | protobuf "github.com/couchbase/indexing/secondary/protobuf/query" 10 | ) 11 | 12 | // Application is example application logic that uses query-port server 13 | func Application(config c.Config) { 14 | killch := make(chan bool) 15 | s, err := NewServer( 16 | "", 17 | "localhost:9990", 18 | func(req interface{}, ctx interface{}, 19 | conn net.Conn, quitch <-chan bool, clientVersion uint32) { 20 | requestHandler(req, conn, quitch, killch) 21 | }, 22 | nil, 23 | config) 24 | 25 | if err != nil { 26 | logging.Fatalf("Listen failed - %v", err) 27 | } 28 | <-killch 29 | s.Close() 30 | } 31 | 32 | // will be spawned as a go-routine by server's connection handler. 33 | func requestHandler( 34 | req interface{}, 35 | conn net.Conn, // Write handle to the tcp socket 36 | quitch <-chan bool, // client / connection might have quit (done) 37 | killch chan bool, // application is shutting down the server. 38 | ) { 39 | 40 | var responses []*protobuf.ResponseStream 41 | 42 | switch req.(type) { 43 | case *protobuf.StatisticsRequest: 44 | // responses = getStatistics() 45 | case *protobuf.ScanRequest: 46 | // responses = scanIndex() 47 | case *protobuf.ScanAllRequest: 48 | // responses = fullTableScan() 49 | } 50 | 51 | buf := make([]byte, 1024, 1024) 52 | loop: 53 | for _, resp := range responses { 54 | // query storage backend for request 55 | protobuf.EncodeAndWrite(conn, buf, resp) 56 | select { 57 | case <-quitch: 58 | close(killch) 59 | break loop 60 | } 61 | } 62 | protobuf.EncodeAndWrite(conn, buf, &protobuf.StreamEndResponse{}) 63 | // Free resources. 64 | } 65 | -------------------------------------------------------------------------------- /secondary/queryport/client/defs.go: -------------------------------------------------------------------------------- 1 | package client 2 | 3 | //TODO move all these defs to common 4 | 5 | import common "github.com/couchbase/indexing/secondary/common" 6 | 7 | type RequestType string 8 | 9 | const ( 10 | CREATE RequestType = "create" 11 | DROP RequestType = "drop" 12 | BUILD RequestType = "build" 13 | ) 14 | 15 | type IndexRequest struct { 16 | Version uint64 `json:"version,omitempty"` 17 | Type RequestType `json:"type,omitempty"` 18 | Index common.IndexDefn `json:"index,omitempty"` 19 | IndexIds IndexIdList `json:indexIds,omitempty"` 20 | Plan map[string]interface{} `json:plan,omitempty"` 21 | } 22 | 23 | type IndexResponse struct { 24 | Version uint64 `json:"version,omitempty"` 25 | Code string `json:"code,omitempty"` 26 | Error string `json:"error,omitempty"` 27 | } 28 | 29 | type IndexIdList struct { 30 | DefnIds []uint64 `json:"defnIds,omitempty"` 31 | } 32 | 33 | // 34 | // Response 35 | // 36 | 37 | const ( 38 | RESP_SUCCESS string = "success" 39 | RESP_ERROR string = "error" 40 | ) 41 | -------------------------------------------------------------------------------- /secondary/stats/timing.go: -------------------------------------------------------------------------------- 1 | package stats 2 | 3 | import "time" 4 | import "fmt" 5 | 6 | type TimingStat struct { 7 | Count Int64Val 8 | Sum Int64Val 9 | SumOfSq Int64Val 10 | bitmap uint64 11 | } 12 | 13 | func (t *TimingStat) Init() { 14 | t.Count.Init() 15 | t.Sum.Init() 16 | t.SumOfSq.Init() 17 | 18 | // Set the default value of filter bitmap to AllStatsFilter 19 | t.bitmap = AllStatsFilter 20 | } 21 | 22 | func (t *TimingStat) Put(dur time.Duration) { 23 | t.Count.Add(1) 24 | t.Sum.Add(int64(dur)) 25 | t.SumOfSq.Add(int64(dur * dur)) 26 | } 27 | 28 | func (t TimingStat) Value() string { 29 | return fmt.Sprintf("%d %d %d", t.Count.Value(), t.Sum.Value(), t.SumOfSq.Value()) 30 | } 31 | 32 | func (t *TimingStat) Map(bitmap uint64) bool { 33 | return (t.bitmap & bitmap) != 0 34 | } 35 | 36 | func (t *TimingStat) AddFilter(filter uint64) { 37 | t.bitmap |= filter 38 | } 39 | 40 | func (t *TimingStat) GetValue() interface{} { 41 | return fmt.Sprintf("%d %d %d", t.Count.Value(), t.Sum.Value(), t.SumOfSq.Value()) 42 | } 43 | -------------------------------------------------------------------------------- /secondary/stubs/build_community.go: -------------------------------------------------------------------------------- 1 | // +build community 2 | 3 | package stubs 4 | 5 | func UsePlasma() bool { 6 | return false 7 | } 8 | -------------------------------------------------------------------------------- /secondary/stubs/build_enterprise.go: -------------------------------------------------------------------------------- 1 | // +build !community 2 | 3 | package stubs 4 | 5 | func UsePlasma() bool { 6 | return true 7 | } 8 | -------------------------------------------------------------------------------- /secondary/stubs/nitro/bhive/bhive_community.go: -------------------------------------------------------------------------------- 1 | //go:build community 2 | // +build community 3 | 4 | package bhive 5 | 6 | import "net/http" 7 | import "fmt" 8 | 9 | type StubType int 10 | 11 | var Diag StubType 12 | 13 | func SetMemoryQuota(_ int64) { 14 | } 15 | 16 | func GetMandatoryQuota() (int64, int64) { 17 | return 0, 0 18 | } 19 | 20 | func GetWorkingSetSize() int64 { 21 | return 0 22 | } 23 | 24 | func (d *StubType) HandleHttp(w http.ResponseWriter, r *http.Request) { 25 | fmt.Fprintf(w, "not implemented") 26 | } 27 | -------------------------------------------------------------------------------- /secondary/stubs/nitro/bhive/bhive_enterprise.go: -------------------------------------------------------------------------------- 1 | //go:build !community 2 | // +build !community 3 | 4 | package bhive 5 | 6 | import ( 7 | ee "github.com/couchbase/bhive" 8 | ) 9 | 10 | var Diag = &ee.Diag 11 | 12 | func SetMemoryQuota(sz int64) { 13 | ee.SetMemoryQuota(uint64(sz)) 14 | } 15 | 16 | func GetMandatoryQuota() (int64, int64) { 17 | return ee.GetMandatoryQuota() 18 | } 19 | 20 | func GetWorkingSetSize() int64 { 21 | return ee.GetWorkingSetSize() 22 | } 23 | -------------------------------------------------------------------------------- /secondary/stubs/nitro/mm/mm_community.go: -------------------------------------------------------------------------------- 1 | // +build community 2 | 3 | package mm 4 | 5 | import ( 6 | "errors" 7 | 8 | "github.com/couchbase/indexing/secondary/memdb/skiplist" 9 | ) 10 | 11 | var ErrJemallocProfilingNotSupported = errors.New("jemalloc profiling not supported") 12 | 13 | var Malloc skiplist.MallocFn 14 | var Free skiplist.FreeFn 15 | 16 | var Debug *bool = &[]bool{false}[0] 17 | 18 | func FreeOSMemory() { 19 | } 20 | 21 | func Size() uint64 { 22 | return 0 23 | } 24 | 25 | func Stats() string { 26 | return "" 27 | } 28 | 29 | func StatsJson() string { 30 | return "{}" 31 | } 32 | 33 | func ProfActivate() error { 34 | return ErrJemallocProfilingNotSupported 35 | } 36 | 37 | func ProfDeactivate() error { 38 | return ErrJemallocProfilingNotSupported 39 | } 40 | 41 | func ProfDump(_ string) error { 42 | return ErrJemallocProfilingNotSupported 43 | } 44 | -------------------------------------------------------------------------------- /secondary/stubs/nitro/mm/mm_enterprise.go: -------------------------------------------------------------------------------- 1 | // +build !community 2 | 3 | package mm 4 | 5 | import ( 6 | ee "github.com/couchbase/nitro/mm" 7 | ) 8 | 9 | var Malloc = ee.Malloc 10 | var Free = ee.Free 11 | 12 | var Debug = &ee.Debug 13 | 14 | func FreeOSMemory() { 15 | ee.FreeOSMemory() 16 | } 17 | 18 | func Size() uint64 { 19 | return ee.Size() 20 | } 21 | 22 | func AllocSize() uint64 { 23 | return ee.AllocSize() 24 | } 25 | 26 | func Stats() string { 27 | return ee.Stats() 28 | } 29 | 30 | func StatsJson() string { 31 | return ee.StatsJson() 32 | } 33 | 34 | func GetAllocStats() (uint64, uint64) { 35 | return ee.GetAllocStats() 36 | } 37 | 38 | func ProfActivate() error { 39 | return ee.ProfActivate() 40 | } 41 | 42 | func ProfDeactivate() error { 43 | return ee.ProfDeactivate() 44 | } 45 | 46 | func ProfDump(filePath string) error { 47 | return ee.ProfDump(filePath) 48 | } 49 | -------------------------------------------------------------------------------- /secondary/stubs/nitro/plasma/plasma_community.go: -------------------------------------------------------------------------------- 1 | //go:build community 2 | // +build community 3 | 4 | package plasma 5 | 6 | import ( 7 | "fmt" 8 | "net/http" 9 | ) 10 | 11 | type StubType int 12 | 13 | var Diag StubType 14 | 15 | func SetMemoryQuota(_ int64, _ bool) { 16 | } 17 | 18 | func GetMandatoryQuota() (int64, int64) { 19 | return 0, 0 20 | } 21 | 22 | func GetWorkingSetSize() int64 { 23 | return 0 24 | } 25 | 26 | func SetLogReclaimBlockSize(_ int64) { 27 | } 28 | 29 | func MemoryInUse() int64 { 30 | return 0 31 | } 32 | 33 | func TenantQuotaNeeded() int64 { 34 | return 0 35 | } 36 | 37 | func (d *StubType) HandleHttp(w http.ResponseWriter, r *http.Request) { 38 | fmt.Fprintf(w, "not implemented") 39 | } 40 | -------------------------------------------------------------------------------- /secondary/stubs/nitro/plasma/plasma_enterprise.go: -------------------------------------------------------------------------------- 1 | //go:build !community 2 | // +build !community 3 | 4 | package plasma 5 | 6 | import ( 7 | ee "github.com/couchbase/plasma" 8 | ) 9 | 10 | var Diag = &ee.Diag 11 | 12 | func SetMemoryQuota(sz int64, force bool) { 13 | ee.SetMemoryQuota2(sz, force) 14 | } 15 | 16 | func GetMandatoryQuota() (int64, int64) { 17 | return ee.GetMandatoryQuota() 18 | } 19 | 20 | func GetWorkingSetSize() int64 { 21 | return ee.GetWorkingSetSize() 22 | } 23 | 24 | func SetLogReclaimBlockSize(sz int64) { 25 | ee.SetLogReclaimBlockSize(sz) 26 | } 27 | 28 | func MemoryInUse() int64 { 29 | return ee.MemoryInUse() 30 | } 31 | 32 | func TenantQuotaNeeded() int64 { 33 | return ee.TenantQuotaMandatory() 34 | } 35 | -------------------------------------------------------------------------------- /secondary/stubs/regulator/regulator_ce.go: -------------------------------------------------------------------------------- 1 | //go:build community 2 | 3 | package regulator 4 | 5 | type Ctx interface{} 6 | 7 | type Units uint64 8 | 9 | type UnitType uint 10 | 11 | type AggregateRecorder struct { 12 | } 13 | 14 | func (ar *AggregateRecorder) AddBytes(bytes uint64) error { 15 | return nil 16 | } 17 | 18 | func (ar *AggregateRecorder) AddBytesWithType(bytes uint64, variant UnitType) error { 19 | return nil 20 | } 21 | 22 | func (ar *AggregateRecorder) State() (metered, pending Units, bytesPending uint64) { 23 | return 24 | } 25 | 26 | func (ar *AggregateRecorder) Commit() (committed Units, err error) { 27 | return 28 | } 29 | 30 | func (ar *AggregateRecorder) Abort() error { 31 | return nil 32 | } 33 | 34 | func (ar *AggregateRecorder) Flush() error { 35 | return nil 36 | } 37 | -------------------------------------------------------------------------------- /secondary/testcode/const.go: -------------------------------------------------------------------------------- 1 | package testcode 2 | 3 | type TestActionTag int 4 | 5 | const ( 6 | MASTER_SHARDTOKEN_SCHEDULEACK TestActionTag = iota + 1 7 | SOURCE_SHARDTOKEN_AFTER_TRANSFER 8 | DEST_SHARDTOKEN_AFTER_RESTORE 9 | DEST_SHARDTOKEN_DURING_DEFERRED_INDEX_RECOVERY 10 | DEST_SHARDTOKEN_DURING_NON_DEFERRED_INDEX_RECOVERY 11 | DEST_SHARDTOKEN_DURING_INDEX_BUILD 12 | MASTER_SHARDTOKEN_BEFORE_DROP_ON_SOURCE 13 | MASTER_SHARDTOKEN_AFTER_DROP_ON_SOURCE 14 | MASTER_SHARDTOKEN_ALL_TOKENS_PROCESSED 15 | DEST_INDEXER_BEFORE_INDEX_RECOVERY 16 | DEST_INDEXER_AFTER_INDEX_RECOVERY 17 | LIFECYCLE_MANAGER_CREATE_INDEX 18 | BUILD_INDEX_TRAINING 19 | ) 20 | 21 | type TestAction int 22 | 23 | const ( 24 | NONE TestAction = iota // No action to be taken 25 | INDEXER_PANIC // Panic indexer at the tag 26 | REBALANCE_CANCEL // Cancel rebalance at the tag 27 | EXEC_N1QL_STATEMENT // Execute N1QL statement at the tag 28 | SLEEP // Sleep at the tag 29 | INJECT_ERROR // raise custom error at tag 30 | ) 31 | 32 | func isMasterTag(tag TestActionTag) bool { 33 | switch tag { 34 | case MASTER_SHARDTOKEN_SCHEDULEACK, 35 | MASTER_SHARDTOKEN_BEFORE_DROP_ON_SOURCE, 36 | MASTER_SHARDTOKEN_AFTER_DROP_ON_SOURCE, 37 | MASTER_SHARDTOKEN_ALL_TOKENS_PROCESSED: 38 | return true 39 | } 40 | return false 41 | } 42 | -------------------------------------------------------------------------------- /secondary/testcode/test_action.go: -------------------------------------------------------------------------------- 1 | //go:build !2ici_test 2 | // +build !2ici_test 3 | 4 | package testcode 5 | 6 | import ( 7 | "github.com/couchbase/indexing/secondary/common" 8 | ) 9 | 10 | func TestActionAtTag(cfg common.Config, tag TestActionTag) error { 11 | // Note: This function is a no-op for non-CI builds. See test_action_ci.go 12 | // for the implementation for CI builds 13 | return nil 14 | } 15 | 16 | func IgnoreAlternateShardIds(cfg common.Config, defn *common.IndexDefn) { 17 | // No-op 18 | } 19 | 20 | func CorruptIndex(cfg common.Config, index *common.IndexInst) { 21 | // No-op 22 | } 23 | -------------------------------------------------------------------------------- /secondary/tests/README.markdown: -------------------------------------------------------------------------------- 1 | ## Tests for 2i 2 | 3 | # Status 4 | Currently, it has framework utilities required for basic functional testcases. The utitilies are - KV data utilities (for SET, GET, DEL of KV), JSON data utilties (to load json from file, to scan json document), 2i API wrappers (Range, Lookup, Create 2i, Drop 2i, List indexes) 5 | 6 | # Todo 7 | Need to modify the logic for computing expected scan results from JSON document 8 | 9 | # Usage 10 | Tests can be run using "go test" command from /indexing/secondary/tests/functionaltests/ location 11 | 12 | # 2i APIs and helper methods used in tests 13 | Create 2i 14 | Drop 2i 15 | List indexes 16 | Range 17 | ResponseReader.GetEntries 18 | client.NewClient 19 | client.NewClusterClient 20 | client.Remoteaddr 21 | client.Inclusion 22 | common.SystemConfig.SectionConfig 23 | common.SecondaryKey 24 | go-couchbase - Get, Set, Delete 25 | n1ql - n1ql.ParseExpression -------------------------------------------------------------------------------- /secondary/tests/ci/config/TestPerfScanLatency_Lookup_StaleFalse.json: -------------------------------------------------------------------------------- 1 | { 2 | "Concurrency" : 10, 3 | "Clients": 1, 4 | "ScanSpecs" : [ 5 | { 6 | "Type" : "Lookup", 7 | "Bucket" : "default", 8 | "Id" : 3, 9 | "Index" : "index_company", 10 | "Lookups" : [ ["WARETEL"] ], 11 | "Limit" : 1, 12 | "Repeat" : 1000, 13 | "Consistency": true, 14 | "NInterval": 1 15 | } 16 | ] 17 | } 18 | -------------------------------------------------------------------------------- /secondary/tests/ci/config/TestPerfScanLatency_Lookup_StaleOk.json: -------------------------------------------------------------------------------- 1 | { 2 | "Concurrency" : 10, 3 | "Clients": 1, 4 | "ScanSpecs" : [ 5 | { 6 | "Type" : "Lookup", 7 | "Bucket" : "default", 8 | "Id" : 1, 9 | "Index" : "index_company", 10 | "Lookups" : [ ["WARETEL"] ], 11 | "Limit" : 1, 12 | "Repeat" : 1000, 13 | "Consistency": false, 14 | "NInterval": 1 15 | } 16 | ] 17 | } 18 | -------------------------------------------------------------------------------- /secondary/tests/ci/config/TestPerfScanLatency_Range_StaleFalse.json: -------------------------------------------------------------------------------- 1 | { 2 | "Concurrency" : 10, 3 | "Clients": 1, 4 | "ScanSpecs" : [ 5 | { 6 | "Type" : "Range", 7 | "Bucket" : "default", 8 | "Id" : 4, 9 | "Index" : "index_company", 10 | "Low" : [ 11 | "E" 12 | ], 13 | "High" : [ 14 | "P" 15 | ], 16 | "Limit" : 1000000000, 17 | "Repeat" : 1000, 18 | "Consistency": true, 19 | "NInterval": 1 20 | } 21 | ] 22 | } 23 | -------------------------------------------------------------------------------- /secondary/tests/ci/config/TestPerfScanLatency_Range_StaleOk.json: -------------------------------------------------------------------------------- 1 | { 2 | "Concurrency" : 10, 3 | "Clients": 1, 4 | "ScanSpecs" : [ 5 | { 6 | "Type" : "Range", 7 | "Bucket" : "default", 8 | "Id" : 2, 9 | "Index" : "index_company", 10 | "Low" : [ 11 | "E" 12 | ], 13 | "High" : [ 14 | "P" 15 | ], 16 | "Limit" : 1000000000, 17 | "Repeat" : 1000, 18 | "Consistency": false, 19 | "NInterval": 1 20 | } 21 | ] 22 | } 23 | -------------------------------------------------------------------------------- /secondary/tests/ci/scripts/.gitignore: -------------------------------------------------------------------------------- 1 | perfstat 2 | -------------------------------------------------------------------------------- /secondary/tests/ci/scripts/build: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | export PATH=$PATH:~/bin 4 | 5 | rm -rf ~/.ciscripts 6 | git clone -q ssh://github.com/couchbase/indexing ~/.ciscripts 7 | cd ~/.ciscripts 8 | (source ~/.cienv && git checkout $BRANCH) 9 | 10 | while true; do 11 | git pull -q 12 | find secondary/tests/ci/scripts -not -name 'build' -and -not -name '.*' -type f -exec cp -v {} ~/bin/ \; 13 | domain 14 | sleep 600 15 | done 16 | 17 | -------------------------------------------------------------------------------- /secondary/tests/ci/scripts/del-failed: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | cd /var/www 3 | if [ "$1" == "all" ]; then 4 | rm `find . -name '*fail*' | awk -F- '{print "*"$2"-"$3"*"}' | sed 's/.fail.html//'` 5 | else 6 | rm `find . -name '*fail*' -mtime +2 | awk -F- '{print "*"$2"-"$3"*"}' | sed 's/.fail.html//'` 7 | fi 8 | -------------------------------------------------------------------------------- /secondary/tests/ci/scripts/dowatch: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | source ~/.cienv 4 | 5 | if [ "$WORKSPACE" = "" ]; then 6 | echo 'WORKSPACE not set' 7 | exit 2 8 | fi 9 | 10 | if [ "$CIBOT" = "true" ]; then 11 | sleep 60 12 | exit 0 13 | fi 14 | 15 | if [ "$STORAGE" = "memdb" ]; then 16 | sleep 60 17 | exit 0 18 | fi 19 | 20 | for i in {1..240}; do 21 | cd $WORKSPACE 22 | sleep 60 23 | current="`git ls-remote ssh://github.com/couchbase/indexing $BRANCH | awk '{print $1}'`" 24 | good="`cat ~/indexing.good`" 25 | if [ "$current" != "$good" ]; then exit 0; fi 26 | current="`git ls-remote ssh://github.com/couchbase/plasma $BRANCH | awk '{print $1}'`" 27 | good="`cat ~/plasma.good`" 28 | if [ "$current" != "$good" ]; then exit 0; fi 29 | done 30 | 31 | -------------------------------------------------------------------------------- /secondary/tests/ci/scripts/kick: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | killall build 3 | killall domain 4 | killall dobuild 5 | killall dotest 6 | killall make 7 | pkill -e -f cluster_ 8 | pkill -e -f /opt/ 9 | killall beam.smp epmd 10 | killall python 11 | killall -9 memcached.json 12 | killall -9 memcached 13 | killall -9 indexer 14 | killall -9 projector 15 | -------------------------------------------------------------------------------- /secondary/tests/ci/scripts/redo: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | cd $HOME/bin 3 | kick 4 | sleep 3 5 | build & 6 | 7 | -------------------------------------------------------------------------------- /secondary/tests/ci/scripts/setup/apache_server.dockerfile: -------------------------------------------------------------------------------- 1 | FROM httpd:2.4.59-alpine 2 | 3 | COPY apache2.conf /usr/local/apache2/conf/httpd.conf 4 | -------------------------------------------------------------------------------- /secondary/tests/ci/scripts/setup/container-runner.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | perl -pi -e "s/export STORAGE=.*/export STORAGE=plasma/g" ~/.cienv 3 | source $HOME/.cienv 4 | 5 | echo "container runner started" 6 | 7 | echo '#include 8 | int main(){for(;;)pause();} 9 | ' >pause.c 10 | gcc -o pause pause.c 11 | 12 | [ -z "$CONTAINER_INIT_SCRIPT" ] && export CONTAINER_INIT_SCRIPT="$ciscripts_dir/secondary/tests/ci/scripts/build" 13 | 14 | echo "running $CONTAINER_INIT_SCRIPT" 15 | 16 | bash $CONTAINER_INIT_SCRIPT & 17 | ./pause 18 | -------------------------------------------------------------------------------- /secondary/tests/ci/scripts/setup/run_ci_dc.yaml: -------------------------------------------------------------------------------- 1 | name: ci_runner 2 | 3 | services: 4 | ci: 5 | build: 6 | dockerfile: ci_machine.dockerfile 7 | args: 8 | PEGGED: $PEGGED 9 | # RELEASE: 8.0.0 10 | # MANIFEST: couchbase-server/morpheus/8.0.0.xml 11 | CIBOT: true 12 | MODE: "unit,functional" 13 | USERNAME: cbci # gerrit username 14 | tags: 15 | - ci_machine:latest 16 | - ci_machine:$PEGGED 17 | secrets: 18 | - github_key 19 | - gerrit_key 20 | ulimits: 21 | nofile: 22 | soft: 200000 23 | hard: 400000 24 | secrets: 25 | - github_key 26 | - gerrit_key 27 | ports: 28 | - "8091-8095:8091-8095" 29 | - "11200-11220:11200-11220" 30 | - "9000-9120:9000-9120" 31 | - 2222:22 32 | volumes: 33 | - build:/home/bot/build:rw 34 | - var-www:/var/www:rw 35 | - depscache:/home/bot/.cbdepscache:rw 36 | - depscache:/home/bot/.cbdepcache:rw 37 | - $HOME/testdata:/home/bot/testdata 38 | init: true 39 | # comment both of the below to run standalone tests 40 | restart: unless-stopped 41 | entrypoint: /home/bot/.ciscripts/secondary/tests/ci/scripts/setup/container-runner.sh 42 | # cpus: 6.0 43 | cpuset: "0-5" 44 | mem_limit: 12G 45 | memswap_limit: "0" 46 | apache: 47 | build: 48 | dockerfile: apache_server.dockerfile 49 | restart: unless-stopped 50 | volumes: 51 | - var-www:/var/www:ro 52 | ports: 53 | - 80:80 54 | cpus: 0.5 55 | mem_limit: 1G 56 | secrets: 57 | github_key: 58 | file: $HOME/.ssh/id_rsa_server_buildnode_to_github 59 | gerrit_key: 60 | file: $HOME/.ssh/cbci 61 | volumes: 62 | var-www: 63 | build: 64 | driver: local 65 | driver_opts: 66 | type: none 67 | device: /opt/ci 68 | o: bind 69 | depscache: 70 | -------------------------------------------------------------------------------- /secondary/tests/ci/scripts/setup/standalone-runner.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | # this file is supposed to be a trimmed down version of dobuild 4 | # dobuild clones gerrit patches. here the expectation is that 5 | # all the clone has happened already in the $WORKSPACE 6 | 7 | source $HOME/.cienv 8 | 9 | if [ ! -d "/mnt/project/goproj/src/github.com/couchbase/indexing" ]; 10 | then 11 | domain 12 | exit $? 13 | fi; 14 | 15 | echo "cloning custom repo into $WORKSPACE" 16 | cp -r /mnt/project/* $WORKSPACE 17 | rm -rf $WORKSPACE/analytics/CMakeLists.txt 18 | cd $WORKSPACE/goproj/src/github.com/couchbase/indexing 19 | cp -r secondary/tests/ci/scripts/* /home/bot/bin/ 20 | 21 | export PATH=/home/bot/bin:$PATH 22 | 23 | echo "running standalone runner - $WORKSPACE" 24 | 25 | export TS="$(date +%d.%m.%Y-%H.%M)" 26 | echo '
'
27 | echo 'Building using ' $(which builder) '...'
28 | 
29 | cd $WORKSPACE
30 | make clean
31 | rm -rf build
32 | rm -rf analytics/CMakeLists.txt
33 | 
34 | # add 2ici_test flag
35 | sed -i 's/SET (TAGS "jemalloc")/SET (TAGS "jemalloc 2ici_test")/' $WORKSPACE/goproj/src/github.com/couchbase/indexing/CMakeLists.txt
36 | 
37 | builder
38 | rc=$?
39 | echo "builder exit code $rc"
40 | test $rc -eq 0 || (cat $WORKSPACE/make.log && exit 2)
41 | 
42 | echo 'Build done'
43 | echo 'Testing using ' $(which dotest) '...'
44 | 
45 | dotest
46 | rc=$?
47 | echo 'Test done'
48 | echo '
' 49 | 50 | if [ $rc -eq 0 ]; then status=pass; else status=fail; fi 51 | echo '
'
52 | gzip ${WORKSPACE}/logs.tar 2>&1 1>/dev/null
53 | echo "Version: versions-$TS.cfg"
54 | echo "Build Log: make-$TS.log"
55 | echo "Server Log: logs-$TS.tar.gz"
56 | echo "

Finished

" 57 | 58 | cp $WORKSPACE/logs.tar.gz /mnt/project/ 59 | cp $WORKSPACE/make.log /mnt/project/ 60 | cp $WORKSPACE/test.log /mnt/project/ 61 | -------------------------------------------------------------------------------- /secondary/tests/ci/scripts/setupvm: -------------------------------------------------------------------------------- 1 | #!/bin/bash -xv 2 | set -e 3 | sudo rm -rf /var/www/lost+found 4 | sudo rm -rf /opt/build/lost+found 5 | sudo cp -r ~admin/xvdb/.htaccess /var/www/ 6 | sudo cp -r ~admin/xvdb/var/www/* /var/www/ 7 | sudo chown -r buildbot.buildbot /var/www 8 | sudo chown -r buildbot.buildbot /opt/build 9 | if grep -qs CCACHE_DIR ~/.profile; then 10 | echo ccache already setup 11 | else 12 | echo 'export PATH="/usr/lib/ccache:$PATH"' >> ~/.profile 13 | echo 'export CCACHE_DIR=/opt/.ccache' >> ~/.profile 14 | echo 'max_size = 12G' >> /opt/.ccache/ccache.conf 15 | fi 16 | -------------------------------------------------------------------------------- /secondary/tests/ci/skip.txt: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/couchbase/indexing/ba752cd53c7865086c4d99f312eafbe3d23982a9/secondary/tests/ci/skip.txt -------------------------------------------------------------------------------- /secondary/tests/config/blrcluster_conf.json: -------------------------------------------------------------------------------- 1 | { 2 | "KVAddress": "122.248.204.207:8091", 3 | "Username": "Administrator", 4 | "Password": "asdasd" 5 | } 6 | -------------------------------------------------------------------------------- /secondary/tests/config/build_validation.json: -------------------------------------------------------------------------------- 1 | { 2 | "KVAddress": "127.0.0.1:9000", 3 | "Username": "Administrator", 4 | "Password": "asdasd", 5 | "IndexUsing": "memory_optimized" 6 | } 7 | -------------------------------------------------------------------------------- /secondary/tests/config/buildtest_fdb.json: -------------------------------------------------------------------------------- 1 | { 2 | "KVAddress": "127.0.0.1:9000", 3 | "Username": "Administrator", 4 | "Password": "asdasd", 5 | "IndexUsing": "forestdb", 6 | "Nodes": [ 7 | "127.0.0.1:9000", 8 | "127.0.0.1:9001", 9 | "127.0.0.1:9002", 10 | "127.0.0.1:9003" 11 | ], 12 | "MultipleIndexerTests":true 13 | } 14 | -------------------------------------------------------------------------------- /secondary/tests/config/buildtest_memdb.json: -------------------------------------------------------------------------------- 1 | { 2 | "KVAddress": "127.0.0.1:9000", 3 | "Username": "Administrator", 4 | "Password": "asdasd", 5 | "IndexUsing": "memory_optimized", 6 | "Nodes": [ 7 | "127.0.0.1:9000", 8 | "127.0.0.1:9001", 9 | "127.0.0.1:9002", 10 | "127.0.0.1:9003" 11 | ], 12 | "MultipleIndexerTests":true 13 | } 14 | -------------------------------------------------------------------------------- /secondary/tests/config/buildtest_plasma.json: -------------------------------------------------------------------------------- 1 | { 2 | "KVAddress": "127.0.0.1:9000", 3 | "Username": "Administrator", 4 | "Password": "asdasd", 5 | "IndexUsing": "plasma", 6 | "Nodes": [ 7 | "127.0.0.1:9000", 8 | "127.0.0.1:9001", 9 | "127.0.0.1:9002", 10 | "127.0.0.1:9003", 11 | "127.0.0.1:9004", 12 | "127.0.0.1:9005", 13 | "127.0.0.1:9006" 14 | ], 15 | "MultipleIndexerTests":true 16 | } 17 | -------------------------------------------------------------------------------- /secondary/tests/config/clusterrun_conf.json: -------------------------------------------------------------------------------- 1 | { 2 | "KVAddress": "127.0.0.1:9000", 3 | "Username": "Administrator", 4 | "Password": "asdasd", 5 | "IndexUsing": "memory_optimized", 6 | "Nodes": [ 7 | "127.0.0.1:9000", 8 | "127.0.0.1:9001", 9 | "127.0.0.1:9002", 10 | "127.0.0.1:9003" 11 | ], 12 | "MultipleIndexerTests":true 13 | } 14 | -------------------------------------------------------------------------------- /secondary/tests/config/remote1node_conf.json: -------------------------------------------------------------------------------- 1 | { 2 | "KVAddress": "54.151.180.34:8091", 3 | "Username": "Administrator", 4 | "Password": "password" 5 | } -------------------------------------------------------------------------------- /secondary/tests/config/remoteSc1node_conf.json: -------------------------------------------------------------------------------- 1 | { 2 | "KVAddress": "172.23.107.133:8091", 3 | "Username": "Administrator", 4 | "Password": "password" 5 | } -------------------------------------------------------------------------------- /secondary/tests/config/remoteWindows1node_conf.json: -------------------------------------------------------------------------------- 1 | { 2 | "KVAddress": "172.23.106.123:8091", 3 | "Username": "Administrator", 4 | "Password": "password" 5 | } 6 | -------------------------------------------------------------------------------- /secondary/tests/framework/common/constants.go: -------------------------------------------------------------------------------- 1 | package common 2 | 3 | const IndexTypesStaticJSONDataDrive = "https://drive.google.com/uc?export=download&id=0B-2n-TmVRNyDelV6amphY2pZa2M" 4 | const IndexTypesStaticJSONDataS3 = "https://s3-ap-southeast-1.amazonaws.com/indexing-test/testdata/Users10k.txt.gz" 5 | const IndexTypesMutationJSONDataS3 = "https://s3-ap-southeast-1.amazonaws.com/indexing-test/testdata/Users_mut.txt.gz" 6 | const IndexTypesTwitterFeed1JSONDataS3 = "https://s3-ap-southeast-1.amazonaws.com/indexing-test/testdata/TwitterFeed1.txt.gz" 7 | const MissingLiteral = "~[]{}falsenilNA~" 8 | 9 | const CODEBOOK_COPY_PREFIX = "codebook_v1" 10 | const REBALANCE_STAGING_DIR = "staging2" 11 | const CODEBOOK_DIR = "codebook" 12 | -------------------------------------------------------------------------------- /secondary/tests/framework/datautility/jsonloader.go: -------------------------------------------------------------------------------- 1 | package datautility 2 | 3 | import ( 4 | "fmt" 5 | json "github.com/couchbase/indexing/secondary/common/json" 6 | tc "github.com/couchbase/indexing/secondary/tests/framework/common" 7 | ) 8 | 9 | func LoadJSONFromCompressedFile(path, docidfield string) tc.KeyValues { 10 | file, err := tc.ReadCompressedFile(path) 11 | tc.HandleError(err, "Error while decompressing data file "+path) 12 | 13 | var data interface{} 14 | json.Unmarshal(file, &data) 15 | 16 | m := data.([]interface{}) 17 | keyValues := make(tc.KeyValues) 18 | 19 | var i = 0 20 | var k string 21 | if len(docidfield) > 0 { 22 | for _, v := range m { 23 | k = fmt.Sprintf("%v", v.(map[string]interface{})[docidfield]) 24 | keyValues[k] = v 25 | i++ 26 | } 27 | } else { 28 | for _, v := range m { 29 | k = fmt.Sprintf("%v", i) 30 | keyValues[k] = v 31 | i++ 32 | } 33 | } 34 | 35 | return keyValues 36 | } 37 | -------------------------------------------------------------------------------- /secondary/tests/functionaltests/runtest_clusterrun.sh: -------------------------------------------------------------------------------- 1 | export NS_SERVER_CBAUTH_URL="http://127.0.0.1:9000/_cbauth" 2 | export NS_SERVER_CBAUTH_USER="Administrator" 3 | export NS_SERVER_CBAUTH_PWD="asdasd" 4 | export NS_SERVER_CBAUTH_RPC_URL="http://127.0.0.1:9000/cbauth-demo" 5 | 6 | export CBAUTH_REVRPC_URL="http://Administrator:asdasd@127.0.0.1:9000/query" 7 | 8 | go "$@" 9 | -------------------------------------------------------------------------------- /secondary/tests/functionaltests/setup.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | export WORKSPACE="$(PWD)/../../../../../../../../" 4 | export GOPATH="$WORKSPACE/goproj":"$WORKSPACE/godeps" 5 | export GO111MODULE=auto 6 | export CGO_CFLAGS="-I$WORKSPACE/sigar/include -I$WORKSPACE/build/tlm/deps/zstd-cpp.exploded/include -I$WORKSPACE/build/tlm/deps/jemalloc.exploded/include -I$WORKSPACE/forestdb/include/ -DJEMALLOC=1" 7 | export CGO_LDFLAGS="-L $WORKSPACE/install/lib -Wl,-rpath $WORKSPACE/install/lib" 8 | export CBAUTH_REVRPC_URL="http://Administrator:asdasd@127.0.0.1:9000/query" 9 | -------------------------------------------------------------------------------- /secondary/tests/perftests/TestPerfScanLatency_Lookup_StaleFalse.json: -------------------------------------------------------------------------------- 1 | { 2 | "Concurrency" : 200, 3 | "Clients": 20, 4 | "ScanSpecs" : [ 5 | { 6 | "Type" : "Lookup", 7 | "Bucket" : "default", 8 | "Id" : 3, 9 | "Index" : "index_company", 10 | "Lookups" : [ ["WARETEL"] ], 11 | "Limit" : 1, 12 | "Repeat" : 100000, 13 | "Consistency": true, 14 | "NInterval": 1 15 | } 16 | ] 17 | } -------------------------------------------------------------------------------- /secondary/tests/perftests/TestPerfScanLatency_Lookup_StaleOk.json: -------------------------------------------------------------------------------- 1 | { 2 | "Concurrency" : 200, 3 | "Clients": 20, 4 | "ScanSpecs" : [ 5 | { 6 | "Type" : "Lookup", 7 | "Bucket" : "default", 8 | "Id" : 1, 9 | "Index" : "index_company", 10 | "Lookups" : [ ["WARETEL"] ], 11 | "Limit" : 1, 12 | "Repeat" : 100000, 13 | "Consistency": false, 14 | "NInterval": 1 15 | } 16 | ] 17 | } -------------------------------------------------------------------------------- /secondary/tests/perftests/TestPerfScanLatency_Range_StaleFalse.json: -------------------------------------------------------------------------------- 1 | { 2 | "Concurrency" : 200, 3 | "Clients": 20, 4 | "ScanSpecs" : [ 5 | { 6 | "Type" : "Range", 7 | "Bucket" : "default", 8 | "Id" : 4, 9 | "Index" : "index_company", 10 | "Low" : [ 11 | "E" 12 | ], 13 | "High" : [ 14 | "P" 15 | ], 16 | "Limit" : 1000000000, 17 | "Repeat" : 100000, 18 | "Consistency": true, 19 | "NInterval": 1 20 | } 21 | ] 22 | } -------------------------------------------------------------------------------- /secondary/tests/perftests/TestPerfScanLatency_Range_StaleOk.json: -------------------------------------------------------------------------------- 1 | { 2 | "Concurrency" : 200, 3 | "Clients": 20, 4 | "ScanSpecs" : [ 5 | { 6 | "Type" : "Range", 7 | "Bucket" : "default", 8 | "Id" : 2, 9 | "Index" : "index_company", 10 | "Low" : [ 11 | "E" 12 | ], 13 | "High" : [ 14 | "P" 15 | ], 16 | "Limit" : 1000000000, 17 | "Repeat" : 100000, 18 | "Consistency": false, 19 | "NInterval": 1 20 | } 21 | ] 22 | } -------------------------------------------------------------------------------- /secondary/tests/plasmatests/README-smat.md: -------------------------------------------------------------------------------- 1 | # Instructions for smat testing for plasma 2 | 3 | [smat](https://github.com/mschoch/smat) is a framework that provides 4 | state machine assisted fuzz testing. 5 | 6 | To run the smat tests for plasma... 7 | 8 | ## Prerequisites 9 | 10 | $ go get github.com/dvyukov/go-fuzz/go-fuzz 11 | $ go get github.com/dvyukov/go-fuzz/go-fuzz-build 12 | 13 | ## Steps 14 | 15 | 1. Generate initial smat corpus: 16 | ``` 17 | go test -tags=gofuzz -run=TestGenerateSmatCorpus 18 | ``` 19 | 20 | 2. Build go-fuzz test program with instrumentation: 21 | ``` 22 | go-fuzz-build github.com/couchbase/indexing/secondary/indexer/plasmatests 23 | ``` 24 | 25 | 3. Run go-fuzz: 26 | ``` 27 | go-fuzz -bin=./indexer-fuzz.zip -workdir=workdir/ -timeout=2000 28 | ``` 29 | -------------------------------------------------------------------------------- /secondary/tests/plasmatests/crash_test.go: -------------------------------------------------------------------------------- 1 | package indexer 2 | 3 | import ( 4 | "log" 5 | "os" 6 | "testing" 7 | 8 | "github.com/mschoch/smat" 9 | ) 10 | 11 | func TestCrasher(t *testing.T) { 12 | // paste in your crash here: 13 | crasher := []byte("\x00\x1f>]|\x9b\xba\xd9") 14 | // turn on logger 15 | smat.Logger = log.New(os.Stderr, "smat ", log.LstdFlags) 16 | // fuzz the crasher input 17 | smat.Fuzz(&smatContext{}, smat.ActionID('S'), smat.ActionID('T'), 18 | actionMap, crasher) 19 | } 20 | -------------------------------------------------------------------------------- /secondary/tests/plasmatests/plasma_diag.go: -------------------------------------------------------------------------------- 1 | package indexer 2 | 3 | import ( 4 | "net/http" 5 | 6 | "github.com/couchbase/plasma" 7 | ) 8 | 9 | func init() { 10 | http.HandleFunc("/plasmaDiag", plasma.Diag.HandleHttp) 11 | go func() { 12 | http.ListenAndServe(":8080", nil) 13 | }() 14 | } 15 | -------------------------------------------------------------------------------- /secondary/tests/serverlesstests/set99_rebalance_cleanup_test.go: -------------------------------------------------------------------------------- 1 | package serverlesstests 2 | 3 | import ( 4 | "testing" 5 | ) 6 | 7 | // This test removes the SHARD_REBALANCE_DIR from file system 8 | func TestRebalanceStorageDirCleanup(t *testing.T) { 9 | cleanupStorageDir(t) 10 | } 11 | -------------------------------------------------------------------------------- /secondary/tests/testdata/Users100.txt.gz: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/couchbase/indexing/ba752cd53c7865086c4d99f312eafbe3d23982a9/secondary/tests/testdata/Users100.txt.gz -------------------------------------------------------------------------------- /secondary/tests/testdata/Users_Template: -------------------------------------------------------------------------------- 1 | [ 2 | '{{repeat(100, 100)}}', 3 | { 4 | _id: '{{objectId()}}', 5 | docid: 'User{{guid()}}', 6 | guid: '{{guid()}}', 7 | isActive: '{{bool()}}', 8 | balance: '{{floating(1000, 4000, 2, "$0,0.00")}}', 9 | picture: 'http://placehold.it/32x32', 10 | age: '{{integer(20, 40)}}', 11 | eyeColor: '{{random("blue", "brown", "green")}}', 12 | name: '{{firstName()}} {{surname()}}', 13 | gender: '{{gender()}}', 14 | company: '{{company().toUpperCase()}}', 15 | email: '{{email()}}', 16 | phone: '+1 {{phone()}}', 17 | address: 18 | { 19 | number: '{{integer(100, 999)}}', 20 | street: '{{street()}}', 21 | city: '{{city()}}', 22 | state: '{{state()}}', 23 | pin: '{{integer(100, 10000)}}' 24 | }, 25 | about: '{{lorem(1, "paragraphs")}}', 26 | registered: '{{date(new Date(2014, 0, 1), new Date(), "YYYY-MM-ddThh:mm:ss Z")}}', 27 | latitude: '{{floating(-90.000001, 90)}}', 28 | longitude: '{{floating(-180.000001, 180)}}', 29 | tags: [ 30 | '{{repeat(7)}}', 31 | '{{lorem(1, "words")}}' 32 | ], 33 | friends: [ 34 | '{{repeat(3)}}', 35 | { 36 | id: '{{index()}}', 37 | name: '{{firstName()}} {{surname()}}' 38 | } 39 | ], 40 | greeting: function (tags) { 41 | return 'Hello, ' + this.name + '! You have ' + tags.integer(1, 10) + ' unread messages.'; 42 | }, 43 | favoriteFruit: function (tags) { 44 | var fruits = ['apple', 'banana', 'strawberry']; 45 | return fruits[tags.integer(0, fruits.length - 1)]; 46 | } 47 | } 48 | ] -------------------------------------------------------------------------------- /secondary/tests/testdata/invalidjson: -------------------------------------------------------------------------------- 1 | # Inavlid numbers in json 2 | 00 3 | -00 4 | 000475 5 | 0360 6 | -000475 7 | -0360 8 | 36.e3 9 | .67e4 10 | 1.67e9.5 11 | -------------------------------------------------------------------------------- /secondary/tests/testdata/json.prod: -------------------------------------------------------------------------------- 1 | json : "{" age properties "}". 2 | 3 | properties : properties "," property {100} 4 | | property. 5 | 6 | age : DQ "age" DQ ": " range(15, 80) ", " NL. 7 | property : DQ bag("./web2") DQ ":" value. 8 | 9 | value : TRUE 10 | | FALSE 11 | | NULL 12 | | number 13 | | string 14 | | array 15 | | json. 16 | 17 | string : DQ bag("./web2") DQ. 18 | 19 | number : range(0, 100000) 20 | | rangef(0.0, 100.0) {900}. 21 | 22 | array : "[" values "]". 23 | 24 | values : values "," value 25 | | value. 26 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/greedy/new_equiv_index.json: -------------------------------------------------------------------------------- 1 | [ 2 | { 3 | "name" : "newEquivIdx", 4 | "bucket" : "default", 5 | "isPrimary" : false, 6 | "secExprs" : ["equivField"], 7 | "isArrayIndex" : false, 8 | "replica" : 1, 9 | "defnId" : 987654, 10 | "NoUsageInfo" : true, 11 | "NeedsEstimate" : true 12 | } 13 | ] 14 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/greedy/new_equiv_index_1_replica.json: -------------------------------------------------------------------------------- 1 | [ 2 | { 3 | "name" : "newEquivIdx", 4 | "bucket" : "default", 5 | "isPrimary" : false, 6 | "secExprs" : ["equivField"], 7 | "isArrayIndex" : false, 8 | "replica" : 2, 9 | "defnId" : 987654, 10 | "NoUsageInfo" : true, 11 | "NeedsEstimate" : true 12 | } 13 | ] 14 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/greedy/new_equiv_index_2_replica.json: -------------------------------------------------------------------------------- 1 | [ 2 | { 3 | "name" : "newEquivIdx", 4 | "bucket" : "default", 5 | "isPrimary" : false, 6 | "secExprs" : ["equivField"], 7 | "isArrayIndex" : false, 8 | "replica" : 3, 9 | "defnId" : 987654, 10 | "NoUsageInfo" : true, 11 | "NeedsEstimate" : true 12 | } 13 | ] 14 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/greedy/new_index_1.json: -------------------------------------------------------------------------------- 1 | [ 2 | { 3 | "name" : "idx1", 4 | "bucket" : "bucket2", 5 | "isPrimary" : false, 6 | "secExprs" : ["name1"], 7 | "isArrayIndex" : false, 8 | "replica" : 1, 9 | "defnId" : 987654, 10 | "NoUsageInfo" : true, 11 | "NeedsEstimate" : true 12 | } 13 | ] 14 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/greedy/new_index_with_1_replica.json: -------------------------------------------------------------------------------- 1 | [ 2 | { 3 | "name" : "idx1", 4 | "bucket" : "bucket2", 5 | "isPrimary" : false, 6 | "secExprs" : ["name1"], 7 | "isArrayIndex" : false, 8 | "replica" : 2, 9 | "defnId" : 987654, 10 | "NoUsageInfo" : true, 11 | "NeedsEstimate" : true 12 | } 13 | ] 14 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/greedy/new_index_with_2_replicas.json: -------------------------------------------------------------------------------- 1 | [ 2 | { 3 | "name" : "idx1", 4 | "bucket" : "bucket2", 5 | "isPrimary" : false, 6 | "secExprs" : ["name1"], 7 | "isArrayIndex" : false, 8 | "replica" : 3, 9 | "defnId" : 987654, 10 | "NoUsageInfo" : true, 11 | "NeedsEstimate" : true 12 | } 13 | ] 14 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/greedy/topologies/1_empty_1_10pct_filled_node_1_sg.json: -------------------------------------------------------------------------------- 1 | { 2 | "placement":[ 3 | { 4 | "nodeId":"127.0.0.1:9001", 5 | "nodeUUID":"fdb15192216a984a53fc19de249e0ba4", 6 | "indexerId":"", 7 | "restUrl":"", 8 | "serverGroup":"index 1", 9 | "memUsage":20000000000, 10 | "dataSize":20000000000, 11 | "cpuUsage":1, 12 | "indexes": [ 13 | { 14 | "defnId": 1111, 15 | "instId": 0, 16 | "name": "index1", 17 | "bucket": "bucket2", 18 | "host": null, 19 | "avgArrSize": 0, 20 | "avgArrKeySize": 0, 21 | "mutationRate": 0, 22 | "scanRate": 0, 23 | "memUsage": 20000000000, 24 | "dataSize": 20000000000, 25 | "cpuUsage": 1 26 | } 27 | ] 28 | }, 29 | { 30 | "nodeId":"127.0.0.1:9002", 31 | "nodeUUID":"abc15192216a984a53fc19de249e0ba4", 32 | "indexerId":"", 33 | "restUrl":"", 34 | "serverGroup":"index 1" 35 | } 36 | ], 37 | "memQuota":200000000000, 38 | "cpuQuota":30, 39 | "isLive":true 40 | } 41 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/greedy/topologies/1_empty_1_30pct_filled_node_1_sg.json: -------------------------------------------------------------------------------- 1 | { 2 | "placement":[ 3 | { 4 | "nodeId":"127.0.0.1:9001", 5 | "nodeUUID":"fdb15192216a984a53fc19de249e0ba4", 6 | "indexerId":"", 7 | "restUrl":"", 8 | "serverGroup":"index 1", 9 | "memUsage":60000000000, 10 | "dataSize":60000000000, 11 | "cpuUsage":1, 12 | "indexes": [ 13 | { 14 | "defnId": 1111, 15 | "instId": 0, 16 | "name": "index1", 17 | "bucket": "bucket2", 18 | "host": null, 19 | "avgArrSize": 0, 20 | "avgArrKeySize": 0, 21 | "mutationRate": 0, 22 | "scanRate": 0, 23 | "memUsage": 60000000000, 24 | "dataSize": 60000000000, 25 | "cpuUsage": 1 26 | } 27 | ] 28 | }, 29 | { 30 | "nodeId":"127.0.0.1:9002", 31 | "nodeUUID":"abc15192216a984a53fc19de249e0ba4", 32 | "indexerId":"", 33 | "restUrl":"", 34 | "serverGroup":"index 1" 35 | } 36 | ], 37 | "memQuota":200000000000, 38 | "cpuQuota":30, 39 | "isLive":true 40 | } 41 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/greedy/topologies/1_empty_1_60pct_filled_node_1_sg.json: -------------------------------------------------------------------------------- 1 | { 2 | "placement":[ 3 | { 4 | "nodeId":"127.0.0.1:9001", 5 | "nodeUUID":"fdb15192216a984a53fc19de249e0ba4", 6 | "indexerId":"", 7 | "restUrl":"", 8 | "serverGroup":"index 1", 9 | "memUsage":120000000000, 10 | "dataSize":120000000000, 11 | "cpuUsage":1, 12 | "indexes": [ 13 | { 14 | "defnId": 1111, 15 | "instId": 0, 16 | "name": "index1", 17 | "bucket": "bucket2", 18 | "host": null, 19 | "avgArrSize": 0, 20 | "avgArrKeySize": 0, 21 | "mutationRate": 0, 22 | "scanRate": 0, 23 | "memUsage": 120000000000, 24 | "dataSize": 120000000000, 25 | "cpuUsage": 1 26 | } 27 | ] 28 | }, 29 | { 30 | "nodeId":"127.0.0.1:9002", 31 | "nodeUUID":"abc15192216a984a53fc19de249e0ba4", 32 | "indexerId":"", 33 | "restUrl":"", 34 | "serverGroup":"index 1" 35 | } 36 | ], 37 | "memQuota":200000000000, 38 | "cpuQuota":30, 39 | "isLive":true 40 | } 41 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/greedy/topologies/1_empty_2_non_empty_nodes_1_sg.json: -------------------------------------------------------------------------------- 1 | { 2 | "placement":[ 3 | { 4 | "nodeId":"127.0.0.1:9001", 5 | "nodeUUID":"fdb15192216a984a53fc19de249e0ba4", 6 | "indexerId":"", 7 | "restUrl":"", 8 | "serverGroup":"index 1", 9 | "memUsage":2600000, 10 | "cpuUsage":1, 11 | "memOverhead": 165423720, 12 | "dataSize":0, 13 | "indexes": [ 14 | { 15 | "defnId": 1111, 16 | "instId": 0, 17 | "name": "index1", 18 | "bucket": "bucket2", 19 | "host": null, 20 | "avgArrSize": 0, 21 | "avgArrKeySize": 0, 22 | "mutationRate": 0, 23 | "scanRate": 0, 24 | "memUsage": 2600000, 25 | "cpuUsage": 1, 26 | "memOverhead": 165423720 27 | } 28 | ] 29 | }, 30 | { 31 | "nodeId":"127.0.0.1:9002", 32 | "nodeUUID":"abc15192216a984a53fc19de249e0ba4", 33 | "indexerId":"", 34 | "restUrl":"", 35 | "serverGroup":"index 1", 36 | "memUsage":0, 37 | "cpuUsage":0, 38 | "memOverhead":0 39 | }, 40 | { 41 | "nodeId":"127.0.0.1:9003", 42 | "nodeUUID":"def15192216a984a53fc19de249e0ba4", 43 | "indexerId":"", 44 | "restUrl":"", 45 | "serverGroup":"index 1", 46 | "memUsage":1600000, 47 | "cpuUsage":1, 48 | "memOverhead": 165423720, 49 | "dataSize":0, 50 | "indexes": [ 51 | { 52 | "defnId": 3333, 53 | "instId": 0, 54 | "name": "index3", 55 | "bucket": "bucket2", 56 | "host": null, 57 | "avgArrSize": 0, 58 | "avgArrKeySize": 0, 59 | "mutationRate": 0, 60 | "scanRate": 0, 61 | "memUsage": 1600000, 62 | "cpuUsage": 1, 63 | "memOverhead": 165423720 64 | } 65 | ] 66 | } 67 | ], 68 | "memQuota":100000000000, 69 | "cpuQuota":30, 70 | "isLive":true 71 | } 72 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/greedy/topologies/1_empty_2_non_empty_nodes_2_sg.json: -------------------------------------------------------------------------------- 1 | { 2 | "placement":[ 3 | { 4 | "nodeId":"127.0.0.1:9001", 5 | "nodeUUID":"fdb15192216a984a53fc19de249e0ba4", 6 | "indexerId":"", 7 | "restUrl":"", 8 | "serverGroup":"index 2", 9 | "memUsage":2600000, 10 | "cpuUsage":1, 11 | "memOverhead": 165423720, 12 | "dataSize":0, 13 | "indexes": [ 14 | { 15 | "defnId": 1111, 16 | "instId": 0, 17 | "name": "index1", 18 | "bucket": "bucket2", 19 | "host": null, 20 | "avgArrSize": 0, 21 | "avgArrKeySize": 0, 22 | "mutationRate": 0, 23 | "scanRate": 0, 24 | "memUsage": 2600000, 25 | "cpuUsage": 1, 26 | "memOverhead": 165423720 27 | } 28 | ] 29 | }, 30 | { 31 | "nodeId":"127.0.0.1:9002", 32 | "nodeUUID":"abc15192216a984a53fc19de249e0ba4", 33 | "indexerId":"", 34 | "restUrl":"", 35 | "serverGroup":"index 1", 36 | "memUsage":0, 37 | "cpuUsage":0, 38 | "memOverhead":0 39 | }, 40 | { 41 | "nodeId":"127.0.0.1:9003", 42 | "nodeUUID":"def15192216a984a53fc19de249e0ba4", 43 | "indexerId":"", 44 | "restUrl":"", 45 | "serverGroup":"index 1", 46 | "memUsage":1600000, 47 | "cpuUsage":1, 48 | "memOverhead": 165423720, 49 | "dataSize":0, 50 | "indexes": [ 51 | { 52 | "defnId": 3333, 53 | "instId": 0, 54 | "name": "index3", 55 | "bucket": "bucket2", 56 | "host": null, 57 | "avgArrSize": 0, 58 | "avgArrKeySize": 0, 59 | "mutationRate": 0, 60 | "scanRate": 0, 61 | "memUsage": 1600000, 62 | "cpuUsage": 1, 63 | "memOverhead": 165423720 64 | } 65 | ] 66 | } 67 | ], 68 | "memQuota":100000000000, 69 | "cpuQuota":30, 70 | "isLive":true 71 | } 72 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/greedy/topologies/2_empty_1_non_empty_nodes_1_sg.json: -------------------------------------------------------------------------------- 1 | { 2 | "placement":[ 3 | { 4 | "nodeId":"127.0.0.1:9001", 5 | "nodeUUID":"fdb15192216a984a53fc19de249e0ba4", 6 | "indexerId":"", 7 | "restUrl":"", 8 | "serverGroup":"index 1", 9 | "memUsage":0, 10 | "cpuUsage":0, 11 | "memOverhead":0 12 | }, 13 | { 14 | "nodeId":"127.0.0.1:9002", 15 | "nodeUUID":"abc15192216a984a53fc19de249e0ba4", 16 | "indexerId":"", 17 | "restUrl":"", 18 | "serverGroup":"index 1", 19 | "memUsage":0, 20 | "cpuUsage":0, 21 | "memOverhead":0 22 | }, 23 | { 24 | "nodeId":"127.0.0.1:9003", 25 | "nodeUUID":"def15192216a984a53fc19de249e0ba4", 26 | "indexerId":"", 27 | "restUrl":"", 28 | "serverGroup":"index 1", 29 | "memUsage":1600000, 30 | "cpuUsage":1, 31 | "memOverhead": 165423720, 32 | "indexes": [ 33 | { 34 | "defnId": 3333, 35 | "instId": 0, 36 | "name": "index3", 37 | "bucket": "bucket2", 38 | "host": null, 39 | "avgArrSize": 0, 40 | "avgArrKeySize": 0, 41 | "mutationRate": 0, 42 | "scanRate": 0, 43 | "memUsage": 1600000, 44 | "cpuUsage": 1, 45 | "memOverhead": 165423720 46 | } 47 | ] 48 | } 49 | ], 50 | "memQuota":100000000000, 51 | "cpuQuota":30, 52 | "isLive":true 53 | } 54 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/greedy/topologies/2_empty_1_non_empty_nodes_2_sg.json: -------------------------------------------------------------------------------- 1 | { 2 | "placement":[ 3 | { 4 | "nodeId":"127.0.0.1:9001", 5 | "nodeUUID":"fdb15192216a984a53fc19de249e0ba4", 6 | "indexerId":"", 7 | "restUrl":"", 8 | "serverGroup":"index 1", 9 | "memUsage":0, 10 | "cpuUsage":0, 11 | "memOverhead":0 12 | }, 13 | { 14 | "nodeId":"127.0.0.1:9002", 15 | "nodeUUID":"abc15192216a984a53fc19de249e0ba4", 16 | "indexerId":"", 17 | "restUrl":"", 18 | "serverGroup":"index 2", 19 | "memUsage":0, 20 | "cpuUsage":0, 21 | "memOverhead":0 22 | }, 23 | { 24 | "nodeId":"127.0.0.1:9003", 25 | "nodeUUID":"def15192216a984a53fc19de249e0ba4", 26 | "indexerId":"", 27 | "restUrl":"", 28 | "serverGroup":"index 1", 29 | "memUsage":1600000, 30 | "cpuUsage":1, 31 | "memOverhead": 165423720, 32 | "indexes": [ 33 | { 34 | "defnId": 3333, 35 | "instId": 0, 36 | "name": "index3", 37 | "bucket": "bucket2", 38 | "host": null, 39 | "avgArrSize": 0, 40 | "avgArrKeySize": 0, 41 | "mutationRate": 0, 42 | "scanRate": 0, 43 | "memUsage": 1600000, 44 | "cpuUsage": 1, 45 | "memOverhead": 165423720 46 | } 47 | ] 48 | } 49 | ], 50 | "memQuota":100000000000, 51 | "cpuQuota":30, 52 | "isLive":true 53 | } 54 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/greedy/topologies/3_empty_nodes_1_sg.json: -------------------------------------------------------------------------------- 1 | { 2 | "placement":[ 3 | { 4 | "nodeId":"127.0.0.1:9001", 5 | "nodeUUID":"fdb15192216a984a53fc19de249e0ba4", 6 | "indexerId":"", 7 | "restUrl":"", 8 | "serverGroup":"index 1", 9 | "memUsage":0, 10 | "cpuUsage":0, 11 | "memOverhead":0, 12 | "dataSize":0 13 | }, 14 | { 15 | "nodeId":"127.0.0.1:9002", 16 | "nodeUUID":"abc15192216a984a53fc19de249e0ba4", 17 | "indexerId":"", 18 | "restUrl":"", 19 | "serverGroup":"index 1", 20 | "memUsage":0, 21 | "cpuUsage":0, 22 | "memOverhead":0, 23 | "dataSize":0 24 | }, 25 | { 26 | "nodeId":"127.0.0.1:9003", 27 | "nodeUUID":"def15192216a984a53fc19de249e0ba4", 28 | "indexerId":"", 29 | "restUrl":"", 30 | "serverGroup":"index 1", 31 | "memUsage":0, 32 | "cpuUsage":0, 33 | "memOverhead":0, 34 | "dataSize":0 35 | } 36 | ], 37 | "memQuota":100000000000, 38 | "cpuQuota":30, 39 | "isLive":true 40 | } 41 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/index/replica-3-constraint.json: -------------------------------------------------------------------------------- 1 | [{"name" : "index1", 2 | "bucket" : "bucket2", 3 | "isPrimary" : false, 4 | "secExprs" : ["name"], 5 | "isArrayIndex" : false, 6 | "replica" : 3, 7 | "numDoc" : 5000, 8 | "DocKeySize" : 200, 9 | "SecKeySize" : 200, 10 | "ArrKeySize" : 0, 11 | "ArrSize" : 0, 12 | "MutationRate" : 0, 13 | "ScanRate" : 0}, 14 | {"name" : "index2", 15 | "bucket" : "bucket2", 16 | "isPrimary" : false, 17 | "secExprs" : ["name"], 18 | "isArrayIndex" : false, 19 | "replica" : 3, 20 | "numDoc" : 5000, 21 | "DocKeySize" : 200, 22 | "SecKeySize" : 200, 23 | "ArrKeySize" : 0, 24 | "ArrSize" : 0, 25 | "MutationRate" : 0, 26 | "ScanRate" : 0}] 27 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/index/replica-3.json: -------------------------------------------------------------------------------- 1 | [{"name" : "index1", 2 | "bucket" : "bucket2", 3 | "isPrimary" : false, 4 | "secExprs" : ["name"], 5 | "isArrayIndex" : false, 6 | "replica" : 3, 7 | "numDoc" : 5000, 8 | "DocKeySize" : 200, 9 | "SecKeySize" : 200, 10 | "ArrKeySize" : 0, 11 | "ArrSize" : 0, 12 | "MutationRate" : 0, 13 | "ScanRate" : 0}] 14 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/plan/empty-1-zone.json: -------------------------------------------------------------------------------- 1 | { 2 | "placement": [ 3 | { 4 | "nodeId": "127.0.0.1:9001", 5 | "nodeUUID": "dad15192216a984a53fc19de249e0ba4", 6 | "serverGroup": "index 1", 7 | "memUsage": 0, 8 | "cpuUsage": 0, 9 | "memOverhead": 104857600, 10 | "actualMemUsage": 0, 11 | "actualMemOverhead": 0, 12 | "indexes": [] 13 | }, 14 | { 15 | "nodeId": "127.0.0.1:9002", 16 | "nodeUUID": "fd3bdfd9cef59ef7def316a81d704329", 17 | "serverGroup": "index 1", 18 | "memUsage": 0, 19 | "cpuUsage": 0, 20 | "memOverhead": 104857600, 21 | "actualMemUsage": 0, 22 | "actualMemOverhead": 0, 23 | "indexes": [] 24 | }, 25 | { 26 | "nodeId": "127.0.0.1:9003", 27 | "nodeUUID": "b0a4f1bd3f3339003a1e4683e37e1763", 28 | "serverGroup": "index 1", 29 | "memUsage": 0, 30 | "cpuUsage": 0, 31 | "memOverhead": 104857600, 32 | "actualMemUsage": 0, 33 | "actualMemOverhead": 0, 34 | "indexes": [] 35 | } 36 | ], 37 | "memQuota": 530294000000, 38 | "cpuQuota": 16, 39 | "isLive": true 40 | } 41 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/plan/empty-2-zone.json: -------------------------------------------------------------------------------- 1 | { 2 | "placement": [ 3 | { 4 | "nodeId": "127.0.0.1:9001", 5 | "nodeUUID": "dad15192216a984a53fc19de249e0ba4", 6 | "serverGroup": "index 1", 7 | "memUsage": 0, 8 | "cpuUsage": 0, 9 | "memOverhead": 104857600, 10 | "actualMemUsage": 0, 11 | "actualMemOverhead": 0, 12 | "indexes": [] 13 | }, 14 | { 15 | "nodeId": "127.0.0.1:9002", 16 | "nodeUUID": "fd3bdfd9cef59ef7def316a81d704329", 17 | "serverGroup": "index 1", 18 | "memUsage": 0, 19 | "cpuUsage": 0, 20 | "memOverhead": 104857600, 21 | "actualMemUsage": 0, 22 | "actualMemOverhead": 0, 23 | "indexes": [] 24 | }, 25 | { 26 | "nodeId": "127.0.0.1:9003", 27 | "nodeUUID": "b0a4f1bd3f3339003a1e4683e37e1763", 28 | "serverGroup": "index 2", 29 | "memUsage": 0, 30 | "cpuUsage": 0, 31 | "memOverhead": 104857600, 32 | "actualMemUsage": 0, 33 | "actualMemOverhead": 0, 34 | "indexes": [] 35 | } 36 | ], 37 | "memQuota": 530294000, 38 | "cpuQuota": 16, 39 | "isLive": true 40 | } 41 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/plan/empty-3-zone.json: -------------------------------------------------------------------------------- 1 | { 2 | "placement": [ 3 | { 4 | "nodeId": "127.0.0.1:9001", 5 | "nodeUUID": "dad15192216a984a53fc19de249e0ba4", 6 | "serverGroup": "index 1", 7 | "memUsage": 0, 8 | "cpuUsage": 0, 9 | "memOverhead": 104857600, 10 | "actualMemUsage": 0, 11 | "actualMemOverhead": 0, 12 | "indexes": [] 13 | }, 14 | { 15 | "nodeId": "127.0.0.1:9003", 16 | "nodeUUID": "b0a4f1bd3f3339003a1e4683e37e1763", 17 | "serverGroup": "index 2", 18 | "memUsage": 0, 19 | "cpuUsage": 0, 20 | "memOverhead": 104857600, 21 | "actualMemUsage": 0, 22 | "actualMemOverhead": 0, 23 | "indexes": [] 24 | }, 25 | { 26 | "nodeId": "127.0.0.1:9005", 27 | "nodeUUID": "6f7b7ca95ca4bac2cab1e74b36793589", 28 | "serverGroup": "index 3", 29 | "memUsage": 0, 30 | "cpuUsage": 0, 31 | "memOverhead": 104857600, 32 | "actualMemUsage": 0, 33 | "actualMemOverhead": 0, 34 | "indexes": [] 35 | } 36 | ], 37 | "memQuota": 530294000, 38 | "cpuQuota": 16, 39 | "isLive": true 40 | } 41 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/plan/min-memory-empty-plan.json: -------------------------------------------------------------------------------- 1 | { 2 | "placement": [ 3 | { 4 | "nodeId": "192.168.1.8:9000", 5 | "nodeUUID": "c6fe7e12b609acc9767c7b671c908773", 6 | "storageMode":"plasma", 7 | "memUsage": 0, 8 | "cpuUsage": 0, 9 | "memOverhead": 0, 10 | "actualDataSize": 0, 11 | "actualMemUsage": 0, 12 | "actualMemOverhead": 0, 13 | "actualMemMin": 0, 14 | "indexes": [] 15 | }, 16 | { 17 | "nodeId": "127.0.0.1:9001", 18 | "nodeUUID": "f18c1f3d3c1fabf2a63b51d06f536bf9", 19 | "storageMode":"plasma", 20 | "memUsage": 0, 21 | "cpuUsage": 0, 22 | "memOverhead": 0, 23 | "actualMemUsage": 0, 24 | "actualDataSize": 0, 25 | "actualMemOverhead": 0, 26 | "actualMemMin": 0, 27 | "indexes": [] 28 | }, 29 | { 30 | "nodeId": "127.0.0.1:9002", 31 | "nodeUUID": "f28c1f3d3c1fabf2a63b51d06f536bf9", 32 | "storageMode":"plasma", 33 | "memUsage": 0, 34 | "cpuUsage": 0, 35 | "memOverhead": 0, 36 | "actualMemUsage": 0, 37 | "actualDataSize": 0, 38 | "actualMemOverhead": 0, 39 | "actualMemMin": 0, 40 | "indexes": [] 41 | }, 42 | { 43 | "nodeId": "127.0.0.1:9003", 44 | "nodeUUID": "f38c1f3d3c1fabf2a63b51d06f536bf9", 45 | "storageMode":"plasma", 46 | "memUsage": 0, 47 | "cpuUsage": 0, 48 | "memOverhead": 0, 49 | "actualMemUsage": 0, 50 | "actualDataSize": 0, 51 | "actualMemOverhead": 0, 52 | "actualMemMin": 0, 53 | "indexes": [] 54 | } 55 | ], 56 | "memQuota": 512000000, 57 | "cpuQuota": 8, 58 | "isLive": true 59 | } 60 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/tenantaware/new_index_1.json: -------------------------------------------------------------------------------- 1 | [ 2 | { 3 | "name" : "idx1", 4 | "bucket" : "bucket1", 5 | "isPrimary" : false, 6 | "secExprs" : ["name1"], 7 | "isArrayIndex" : false, 8 | "replica" : 2, 9 | "defnId" : 987654 10 | } 11 | ] 12 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/tenantaware/new_index_2.json: -------------------------------------------------------------------------------- 1 | [ 2 | { 3 | "name" : "idx1", 4 | "bucket" : "bucket7", 5 | "isPrimary" : false, 6 | "secExprs" : ["name1"], 7 | "isArrayIndex" : false, 8 | "replica" : 2, 9 | "defnId" : 987654 10 | } 11 | ] 12 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/tenantaware/topology/1_empty_node_1_sg.json: -------------------------------------------------------------------------------- 1 | { 2 | "placement":[ 3 | { 4 | "nodeId":"127.0.0.1:9001", 5 | "nodeUUID":"fdb15192216a984a53fc19de249e0ba4", 6 | "serverGroup":"sg1", 7 | "mandatoryQuota":0 8 | } 9 | ], 10 | "isLive":true, 11 | "usageThreshold": { "memHighThreshold" : 80, 12 | "memLowThreshold" : 60, 13 | "unitsHighThreshold" : 60, 14 | "unitsLowThreshold" : 40, 15 | "memQuota": 1000000000, 16 | "unitsQuota" : 10000 17 | } 18 | } 19 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/tenantaware/topology/1_non_empty_node_1_sg.json: -------------------------------------------------------------------------------- 1 | { 2 | "placement":[ 3 | { 4 | "nodeId":"127.0.0.1:9001", 5 | "nodeUUID":"abc15192216a984a53fc19de249e0ba4", 6 | "serverGroup":"sg1", 7 | "mandatoryQuota":300000000, 8 | "actualUnits":1000, 9 | "indexes": [ 10 | { 11 | "defnId": 1111, 12 | "instId": 1212, 13 | "name": "idx2", 14 | "bucket": "bucket1", 15 | "instance": { 16 | "replicaId":0 17 | } 18 | } 19 | ] 20 | } 21 | ], 22 | "isLive":true, 23 | "usageThreshold": { "memHighThreshold" : 80, 24 | "memLowThreshold" : 60, 25 | "unitsHighThreshold" : 60, 26 | "unitsLowThreshold" : 40, 27 | "memQuota": 1000000000, 28 | "unitsQuota" : 10000 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/tenantaware/topology/2_empty_nodes_1_sg.json: -------------------------------------------------------------------------------- 1 | { 2 | "placement":[ 3 | { 4 | "nodeId":"127.0.0.1:9001", 5 | "nodeUUID":"fdb15192216a984a53fc19de249e0ba4", 6 | "serverGroup":"sg1", 7 | "mandatoryQuota":0 8 | }, 9 | { 10 | "nodeId":"127.0.0.1:9002", 11 | "nodeUUID":"abc15192216a984a53fc19de249e0ba4", 12 | "serverGroup":"sg1", 13 | "mandatoryQuota":0 14 | } 15 | ], 16 | "isLive":true, 17 | "usageThreshold": { "memHighThreshold" : 80, 18 | "memLowThreshold" : 60, 19 | "unitsHighThreshold" : 60, 20 | "unitsLowThreshold" : 40, 21 | "memQuota": 1000000000, 22 | "unitsQuota" : 10000 23 | } 24 | } 25 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/tenantaware/topology/2_non_empty_node_1_sg.json: -------------------------------------------------------------------------------- 1 | { 2 | "placement":[ 3 | { 4 | "nodeId":"127.0.0.1:9001", 5 | "nodeUUID":"abc15192216a984a53fc19de249e0ba4", 6 | "serverGroup":"sg1", 7 | "mandatoryQuota":300000000, 8 | "actualUnits":1000, 9 | "indexes": [ 10 | { 11 | "defnId": 1111, 12 | "instId": 1212, 13 | "name": "idx2", 14 | "bucket": "bucket1", 15 | "instance": { 16 | "replicaId":0 17 | } 18 | } 19 | ] 20 | }, 21 | { 22 | "nodeId":"127.0.0.1:9002", 23 | "nodeUUID":"xxx15192216a53fc19de249e0ba4", 24 | "serverGroup":"sg1", 25 | "mandatoryQuota":300000000, 26 | "actualUnits":1000, 27 | "indexes": [ 28 | { 29 | "defnId": 4444, 30 | "instId": 4242, 31 | "name": "idx2", 32 | "bucket": "bucket2", 33 | "instance": { 34 | "replicaId":0 35 | } 36 | } 37 | ] 38 | } 39 | ], 40 | "isLive":true, 41 | "usageThreshold": { "memHighThreshold" : 80, 42 | "memLowThreshold" : 60, 43 | "unitsHighThreshold" : 60, 44 | "unitsLowThreshold" : 40, 45 | "memQuota": 1000000000, 46 | "unitsQuota" : 10000 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/tenantaware/topology/4_empty_nodes_2_sg.json: -------------------------------------------------------------------------------- 1 | { 2 | "placement":[ 3 | { 4 | "nodeId":"127.0.0.1:9001", 5 | "nodeUUID":"fdb15192216a984a53fc19de249e0ba4", 6 | "serverGroup":"sg1", 7 | "mandatoryQuota":0 8 | }, 9 | { 10 | "nodeId":"127.0.0.1:9002", 11 | "nodeUUID":"abc15192216a984a53fc19de249e0ba4", 12 | "serverGroup":"sg2", 13 | "mandatoryQuota":0 14 | }, 15 | { 16 | "nodeId":"127.0.0.1:9003", 17 | "nodeUUID":"def15192216a984a53fc19de249e0ba4", 18 | "serverGroup":"sg1", 19 | "mandatoryQuota":0 20 | }, 21 | { 22 | "nodeId":"127.0.0.1:9004", 23 | "nodeUUID":"zzz15192216a984a53fc19de249e0ba4", 24 | "serverGroup":"sg2", 25 | "mandatoryQuota":0 26 | } 27 | ], 28 | "isLive":true, 29 | "usageThreshold": { "memHighThreshold" : 80, 30 | "memLowThreshold" : 60, 31 | "unitsHighThreshold" : 60, 32 | "unitsLowThreshold" : 40, 33 | "memQuota": 1000000000, 34 | "unitsQuota" : 10000 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/tenantaware/topology/4_non_empty_nodes_2_sg_a.json: -------------------------------------------------------------------------------- 1 | { 2 | "placement":[ 3 | { 4 | "nodeId":"127.0.0.1:9001", 5 | "nodeUUID":"fdb15192216a984a53fc19de249e0ba4", 6 | "serverGroup":"sg1", 7 | "mandatoryQuota":300000000, 8 | "actualUnits":1000, 9 | "indexes": [ 10 | { 11 | "defnId": 1111, 12 | "instId": 2222, 13 | "name": "idx2", 14 | "bucket": "bucket1", 15 | "instance": { 16 | "replicaId":1 17 | } 18 | } 19 | ] 20 | }, 21 | { 22 | "nodeId":"127.0.0.1:9002", 23 | "nodeUUID":"abc15192216a984a53fc19de249e0ba4", 24 | "serverGroup":"sg2", 25 | "mandatoryQuota":300000000, 26 | "actualUnits":1000, 27 | "indexes": [ 28 | { 29 | "defnId": 1111, 30 | "instId": 1212, 31 | "name": "idx2", 32 | "bucket": "bucket1", 33 | "instance": { 34 | "replicaId":0 35 | } 36 | } 37 | ] 38 | }, 39 | { 40 | "nodeId":"127.0.0.1:9003", 41 | "nodeUUID":"def15192216a984a53fc19de249e0ba4", 42 | "serverGroup":"sg1", 43 | "mandatoryQuota":300000000, 44 | "actualUnits":1000 45 | }, 46 | { 47 | "nodeId":"127.0.0.1:9004", 48 | "nodeUUID":"zzz15192216a984a53fc19de249e0ba4", 49 | "serverGroup":"sg3", 50 | "mandatoryQuota":300000000, 51 | "actualUnits":1000 52 | } 53 | ], 54 | "isLive":true, 55 | "usageThreshold": { "memHighThreshold" : 80, 56 | "memLowThreshold" : 60, 57 | "unitsHighThreshold" : 60, 58 | "unitsLowThreshold" : 40, 59 | "memQuota": 1000000000, 60 | "unitsQuota" : 10000 61 | } 62 | } 63 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/tenantaware/topology/4_non_empty_nodes_2_sg_b.json: -------------------------------------------------------------------------------- 1 | { 2 | "placement":[ 3 | { 4 | "nodeId":"127.0.0.1:9001", 5 | "nodeUUID":"fdb15192216a984a53fc19de249e0ba4", 6 | "serverGroup":"sg1", 7 | "mandatoryQuota":300000000, 8 | "actualUnits":1000, 9 | "indexes": [ 10 | { 11 | "defnId": 1111, 12 | "instId": 2222, 13 | "name": "idx2", 14 | "bucket": "bucket1", 15 | "instance": { 16 | "replicaId":1 17 | } 18 | } 19 | ] 20 | }, 21 | { 22 | "nodeId":"127.0.0.1:9002", 23 | "nodeUUID":"abc15192216a984a53fc19de249e0ba4", 24 | "serverGroup":"sg2", 25 | "mandatoryQuota":300000000, 26 | "actualUnits":1000 27 | }, 28 | { 29 | "nodeId":"127.0.0.1:9003", 30 | "nodeUUID":"def15192216a984a53fc19de249e0ba4", 31 | "serverGroup":"sg1", 32 | "mandatoryQuota":300000000, 33 | "actualUnits":1000 34 | }, 35 | { 36 | "nodeId":"127.0.0.1:9004", 37 | "nodeUUID":"zzz15192216a984a53fc19de249e0ba4", 38 | "serverGroup":"sg3", 39 | "mandatoryQuota":300000000, 40 | "actualUnits":1000, 41 | "indexes": [ 42 | { 43 | "defnId": 1111, 44 | "instId": 1212, 45 | "name": "idx2", 46 | "bucket": "bucket1", 47 | "instance": { 48 | "replicaId":0 49 | } 50 | } 51 | ] 52 | } 53 | ], 54 | "isLive":true, 55 | "usageThreshold": { "memHighThreshold" : 80, 56 | "memLowThreshold" : 60, 57 | "unitsHighThreshold" : 60, 58 | "unitsLowThreshold" : 40, 59 | "memQuota": 1000000000, 60 | "unitsQuota" : 10000 61 | } 62 | } 63 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/tenantaware/topology/defrag/4_non_empty_nodes_3_sg_a_out.json: -------------------------------------------------------------------------------- 1 | { 2 | 3 | 4 | "127.0.0.1:9001" : { 5 | "memory_used_actual": 600000000, 6 | "units_used_actual": 6000, 7 | "num_tenants" :2, 8 | "num_index_repaired": 0 9 | }, 10 | "127.0.0.1:9003" : { 11 | "memory_used_actual": 300000000, 12 | "units_used_actual": 3000, 13 | "num_tenants" : 1, 14 | "num_index_repaired": 0 15 | }, 16 | "127.0.0.1:9004" : { 17 | "memory_used_actual": 600000000, 18 | "units_used_actual": 6000, 19 | "num_tenants" :2, 20 | "num_index_repaired": 0 21 | }, 22 | "127.0.0.1:9006" : { 23 | "memory_used_actual": 300000000, 24 | "units_used_actual": 3000, 25 | "num_tenants" : 1, 26 | "num_index_repaired": 0 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/tenantaware/topology/defrag/4_non_empty_nodes_3_sg_b_out.json: -------------------------------------------------------------------------------- 1 | { 2 | 3 | 4 | "127.0.0.1:9001" : { 5 | "memory_used_actual": 600000000, 6 | "units_used_actual": 6000, 7 | "num_tenants" :2, 8 | "num_index_repaired": 0 9 | }, 10 | "127.0.0.1:9002" : { 11 | "memory_used_actual": 600000000, 12 | "units_used_actual": 4000, 13 | "num_tenants" : 3, 14 | "num_index_repaired": 0 15 | }, 16 | "127.0.0.1:9004" : { 17 | "memory_used_actual": 600000000, 18 | "units_used_actual": 6000, 19 | "num_tenants" :2, 20 | "num_index_repaired": 0 21 | }, 22 | "127.0.0.1:9005" : { 23 | "memory_used_actual": 600000000, 24 | "units_used_actual": 4000, 25 | "num_tenants" : 3, 26 | "num_index_repaired": 0 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/tenantaware/topology/defrag/4_non_empty_nodes_3_sg_d_out.json: -------------------------------------------------------------------------------- 1 | { 2 | 3 | 4 | "127.0.0.1:9001" : { 5 | "memory_used_actual": 700000000, 6 | "units_used_actual": 5000, 7 | "num_tenants" : 3, 8 | "num_index_repaired": 0 9 | }, 10 | "127.0.0.1:9003" : { 11 | "memory_used_actual": 0, 12 | "units_used_actual": 0, 13 | "num_tenants" : 0, 14 | "num_index_repaired": 0 15 | }, 16 | "127.0.0.1:9004" : { 17 | "memory_used_actual": 700000000, 18 | "units_used_actual": 5000, 19 | "num_tenants" : 3, 20 | "num_index_repaired": 0 21 | }, 22 | "127.0.0.1:9006" : { 23 | "memory_used_actual": 0, 24 | "units_used_actual": 0, 25 | "num_tenants" : 0, 26 | "num_index_repaired": 0 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/tenantaware/topology/defrag/4_non_empty_nodes_3_sg_e_out.json: -------------------------------------------------------------------------------- 1 | { 2 | 3 | 4 | "127.0.0.1:9001" : { 5 | "memory_used_actual": 700000000, 6 | "units_used_actual": 5000, 7 | "num_tenants" : 3, 8 | "num_index_repaired": 0 9 | }, 10 | "127.0.0.1:9003" : { 11 | "memory_used_actual": 650000000, 12 | "units_used_actual": 4500, 13 | "num_tenants" : 3, 14 | "num_index_repaired": 0 15 | }, 16 | "127.0.0.1:9004" : { 17 | "memory_used_actual": 700000000, 18 | "units_used_actual": 5000, 19 | "num_tenants" : 3, 20 | "num_index_repaired": 0 21 | }, 22 | "127.0.0.1:9006" : { 23 | "memory_used_actual": 650000000, 24 | "units_used_actual": 4500, 25 | "num_tenants" : 3, 26 | "num_index_repaired": 0 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/tenantaware/topology/defrag/4_non_empty_nodes_3_sg_f_out.json: -------------------------------------------------------------------------------- 1 | { 2 | 3 | 4 | "127.0.0.1:9001" : { 5 | "memory_used_actual": 600000000, 6 | "units_used_actual": 6000, 7 | "num_tenants" :2, 8 | "num_index_repaired": 0 9 | }, 10 | "127.0.0.1:9003" : { 11 | "memory_used_actual": 300000000, 12 | "units_used_actual": 3000, 13 | "num_tenants" : 1, 14 | "num_index_repaired": 1 15 | }, 16 | "127.0.0.1:9004" : { 17 | "memory_used_actual": 600000000, 18 | "units_used_actual": 6000, 19 | "num_tenants" :2, 20 | "num_index_repaired": 1 21 | }, 22 | "127.0.0.1:9006" : { 23 | "memory_used_actual": 300000000, 24 | "units_used_actual": 3000, 25 | "num_tenants" : 1, 26 | "num_index_repaired": 0 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/tenantaware/topology/defrag/4_non_empty_nodes_3_sg_g_out.json: -------------------------------------------------------------------------------- 1 | { 2 | 3 | 4 | "127.0.0.1:9001" : { 5 | "memory_used_actual": 600000000, 6 | "units_used_actual": 6000, 7 | "num_tenants" :2, 8 | "num_index_repaired": 4 9 | }, 10 | "127.0.0.1:9003" : { 11 | "memory_used_actual": 300000000, 12 | "units_used_actual": 3000, 13 | "num_tenants" : 1, 14 | "num_index_repaired": 2 15 | }, 16 | "127.0.0.1:9004" : { 17 | "memory_used_actual": 600000000, 18 | "units_used_actual": 6000, 19 | "num_tenants" :2, 20 | "num_index_repaired": 0 21 | }, 22 | "127.0.0.1:9006" : { 23 | "memory_used_actual": 300000000, 24 | "units_used_actual": 3000, 25 | "num_tenants" : 1, 26 | "num_index_repaired": 0 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/tenantaware/topology/defrag/4_non_empty_nodes_3_sg_h_out.json: -------------------------------------------------------------------------------- 1 | { 2 | 3 | "127.0.0.1:9002" : { 4 | "memory_used_actual": 300000000, 5 | "units_used_actual": 3000, 6 | "num_tenants" : 2, 7 | "num_index_repaired": 0 8 | }, 9 | "127.0.0.1:9005" : { 10 | "memory_used_actual": 300000000, 11 | "units_used_actual": 3000, 12 | "num_tenants" : 2, 13 | "num_index_repaired": 0 14 | }, 15 | "127.0.0.1:9003" : { 16 | "memory_used_actual": 0, 17 | "units_used_actual": 0, 18 | "num_tenants" : 0, 19 | "num_index_repaired": 0 20 | }, 21 | "127.0.0.1:9006" : { 22 | "memory_used_actual": 0, 23 | "units_used_actual": 0, 24 | "num_tenants" : 0, 25 | "num_index_repaired": 0 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/tenantaware/topology/defrag/4_non_empty_nodes_3_sg_i_out.json: -------------------------------------------------------------------------------- 1 | { 2 | 3 | "127.0.0.1:9002" : { 4 | "memory_used_actual": 200000000, 5 | "units_used_actual": 1000, 6 | "num_tenants" : 1, 7 | "num_index_repaired": 0 8 | }, 9 | "127.0.0.1:9005" : { 10 | "memory_used_actual": 200000000, 11 | "units_used_actual": 1000, 12 | "num_tenants" : 1, 13 | "num_index_repaired": 0 14 | }, 15 | "127.0.0.1:9003" : { 16 | "memory_used_actual": 0, 17 | "units_used_actual": 0, 18 | "num_tenants" : 0, 19 | "num_index_repaired": 0 20 | }, 21 | "127.0.0.1:9006" : { 22 | "memory_used_actual": 0, 23 | "units_used_actual": 0, 24 | "num_tenants" : 0, 25 | "num_index_repaired": 0 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/tenantaware/topology/defrag/4_non_empty_nodes_3_sg_l_out.json: -------------------------------------------------------------------------------- 1 | { 2 | 3 | 4 | "127.0.0.1:9001" : { 5 | "memory_used_actual": 500000000, 6 | "units_used_actual": 800, 7 | "num_tenants" :3, 8 | "num_index_repaired": 0 9 | }, 10 | "127.0.0.1:9004" : { 11 | "memory_used_actual": 500000000, 12 | "units_used_actual": 800, 13 | "num_tenants" :3, 14 | "num_index_repaired": 2 15 | } 16 | } 17 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/tenantaware/topology/defrag/6_non_empty_nodes_3_sg_c_out.json: -------------------------------------------------------------------------------- 1 | { 2 | 3 | 4 | "127.0.0.1:9001" : { 5 | "memory_used_actual": 400000000, 6 | "units_used_actual": 4000, 7 | "num_tenants" : 1, 8 | "num_index_repaired": 0 9 | }, 10 | "127.0.0.1:9002" : { 11 | "memory_used_actual": 500000000, 12 | "units_used_actual": 3000, 13 | "num_tenants" : 3, 14 | "num_index_repaired": 0 15 | }, 16 | "127.0.0.1:9003" : { 17 | "memory_used_actual": 300000000, 18 | "units_used_actual": 3000, 19 | "num_tenants" : 1, 20 | "num_index_repaired": 0 21 | }, 22 | "127.0.0.1:9004" : { 23 | "memory_used_actual": 400000000, 24 | "units_used_actual": 4000, 25 | "num_tenants" : 1, 26 | "num_index_repaired": 0 27 | }, 28 | "127.0.0.1:9005" : { 29 | "memory_used_actual": 500000000, 30 | "units_used_actual": 3000, 31 | "num_tenants" : 3, 32 | "num_index_repaired": 0 33 | }, 34 | "127.0.0.1:9006" : { 35 | "memory_used_actual": 300000000, 36 | "units_used_actual": 3000, 37 | "num_tenants" : 1, 38 | "num_index_repaired": 0 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/tenantaware/topology/defrag/6_non_empty_nodes_3_sg_j_out.json: -------------------------------------------------------------------------------- 1 | 2 | { 3 | 4 | 5 | "127.0.0.1:9001" : { 6 | "memory_used_actual": 400000000, 7 | "units_used_actual": 4000, 8 | "num_tenants" : 1, 9 | "num_index_repaired": 0 10 | }, 11 | "127.0.0.1:9002" : { 12 | "memory_used_actual": 100000000, 13 | "units_used_actual": 1000, 14 | "num_tenants" : 1, 15 | "num_index_repaired": 0 16 | }, 17 | "127.0.0.1:9003" : { 18 | "memory_used_actual": 400000000, 19 | "units_used_actual": 4000, 20 | "num_tenants" : 2, 21 | "num_index_repaired": 0 22 | }, 23 | "127.0.0.1:9004" : { 24 | "memory_used_actual": 400000000, 25 | "units_used_actual": 4000, 26 | "num_tenants" : 1, 27 | "num_index_repaired": 0 28 | }, 29 | "127.0.0.1:9005" : { 30 | "memory_used_actual": 100000000, 31 | "units_used_actual": 1000, 32 | "num_tenants" : 1, 33 | "num_index_repaired": 0 34 | }, 35 | "127.0.0.1:9006" : { 36 | "memory_used_actual": 400000000, 37 | "units_used_actual": 4000, 38 | "num_tenants" : 2, 39 | "num_index_repaired": 0 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/tenantaware/topology/defrag/6_non_empty_nodes_3_sg_k_out.json: -------------------------------------------------------------------------------- 1 | { 2 | 3 | 4 | "127.0.0.1:9001" : { 5 | "memory_used_actual": 500000000, 6 | "units_used_actual": 5000, 7 | "num_tenants" : 2, 8 | "num_index_repaired": 0 9 | }, 10 | "127.0.0.1:9002" : { 11 | "memory_used_actual": 0, 12 | "units_used_actual": 0, 13 | "num_tenants" : 0, 14 | "num_index_repaired": 0 15 | }, 16 | "127.0.0.1:9003" : { 17 | "memory_used_actual": 400000000, 18 | "units_used_actual": 4000, 19 | "num_tenants" : 2, 20 | "num_index_repaired": 0 21 | }, 22 | "127.0.0.1:9004" : { 23 | "memory_used_actual": 500000000, 24 | "units_used_actual": 5000, 25 | "num_tenants" : 2, 26 | "num_index_repaired": 0 27 | }, 28 | "127.0.0.1:9005" : { 29 | "memory_used_actual": 0, 30 | "units_used_actual": 0, 31 | "num_tenants" : 0, 32 | "num_index_repaired": 0 33 | }, 34 | "127.0.0.1:9006" : { 35 | "memory_used_actual": 400000000, 36 | "units_used_actual": 4000, 37 | "num_tenants" : 2, 38 | "num_index_repaired": 0 39 | } 40 | } 41 | 42 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/workload/mixed-small-medium-30-3.json: -------------------------------------------------------------------------------- 1 | { 2 | "name" : "Mixed Workload", 3 | "comment" : "go test -iteration=1 -memQuotaFactor=2 -cpuQuotaFactor=2 -placement=mixedWorkload.json", 4 | "workload" : [ 5 | { 6 | "name" : "bucket1", 7 | "replica" : 3, 8 | "workload" : [ 9 | { 10 | "name" : "small", 11 | "minNumDoc" : 20000000, 12 | "maxNumDoc" : 50000000, 13 | "minDocKeySize" : 20, 14 | "maxDocKeySize" : 200, 15 | "minSecKeySize" : 20, 16 | "maxSecKeySize" : 200, 17 | "minArrKeySize" : 0, 18 | "maxArrKeySize" : 0, 19 | "minArrSize" : 0, 20 | "maxArrSize" : 0, 21 | "minMutationRate" : 10000, 22 | "maxMutationRate" : 30000, 23 | "minScanRate" : 1000, 24 | "maxScanRate" : 10000 25 | }, 26 | { 27 | "name" : "medium", 28 | "minNumDoc" : 200000000, 29 | "maxNumDoc" : 500000000, 30 | "minDocKeySize" : 20, 31 | "maxDocKeySize" : 200, 32 | "minSecKeySize" : 20, 33 | "maxSecKeySize" : 200, 34 | "minArrKeySize" : 0, 35 | "maxArrKeySize" : 0, 36 | "minArrSize" : 0, 37 | "maxArrSize" : 0, 38 | "minMutationRate" : 10000, 39 | "maxMutationRate" : 30000, 40 | "minScanRate" : 1000, 41 | "maxScanRate" : 10000 42 | }], 43 | "distribution" : [60,40] 44 | }], 45 | "distribution" : [100], 46 | "minNumIndex" : 30, 47 | "maxNumIndex" : 30 48 | } 49 | 50 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/workload/uniform-medium-10-3.json: -------------------------------------------------------------------------------- 1 | { 2 | "name" : "Medium Workload (200-500M)", 3 | "comment" : "", 4 | "workload" : [ 5 | { 6 | "name" : "bucket1", 7 | "replica" : 3, 8 | "workload" : [ 9 | { 10 | "name" : "small", 11 | "minNumDoc" : 200000000, 12 | "maxNumDoc" : 500000000, 13 | "minDocKeySize" : 20, 14 | "maxDocKeySize" : 200, 15 | "minSecKeySize" : 20, 16 | "maxSecKeySize" : 200, 17 | "minArrKeySize" : 0, 18 | "maxArrKeySize" : 0, 19 | "minArrSize" : 0, 20 | "maxArrSize" : 0, 21 | "minMutationRate" : 10000, 22 | "maxMutationRate" : 100000, 23 | "minScanRate" : 1000, 24 | "maxScanRate" : 10000 25 | }], 26 | "distribution" : [100] 27 | }], 28 | "distribution" : [100], 29 | "minNumIndex" : 10, 30 | "maxNumIndex" : 10 31 | } 32 | 33 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/workload/uniform-medium-30-3.json: -------------------------------------------------------------------------------- 1 | { 2 | "name" : "Medium Workload (200-500M)", 3 | "comment" : "", 4 | "workload" : [ 5 | { 6 | "name" : "bucket1", 7 | "replica" : 3, 8 | "workload" : [ 9 | { 10 | "name" : "small", 11 | "minNumDoc" : 200000000, 12 | "maxNumDoc" : 500000000, 13 | "minDocKeySize" : 20, 14 | "maxDocKeySize" : 200, 15 | "minSecKeySize" : 20, 16 | "maxSecKeySize" : 200, 17 | "minArrKeySize" : 0, 18 | "maxArrKeySize" : 0, 19 | "minArrSize" : 0, 20 | "maxArrSize" : 0, 21 | "minMutationRate" : 10000, 22 | "maxMutationRate" : 100000, 23 | "minScanRate" : 1000, 24 | "maxScanRate" : 10000 25 | }], 26 | "distribution" : [100] 27 | }], 28 | "distribution" : [100], 29 | "minNumIndex" : 30, 30 | "maxNumIndex" : 30 31 | } 32 | 33 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/workload/uniform-small-10-1.json: -------------------------------------------------------------------------------- 1 | { 2 | "name" : "Small Workload (20-50M)", 3 | "comment" : "", 4 | "workload" : [ 5 | { 6 | "name" : "bucket1", 7 | "replica" : 1, 8 | "workload" : [ 9 | { 10 | "name" : "small", 11 | "minNumDoc" : 20000000, 12 | "maxNumDoc" : 50000000, 13 | "minDocKeySize" : 20, 14 | "maxDocKeySize" : 200, 15 | "minSecKeySize" : 20, 16 | "maxSecKeySize" : 200, 17 | "minArrKeySize" : 0, 18 | "maxArrKeySize" : 0, 19 | "minArrSize" : 0, 20 | "maxArrSize" : 0, 21 | "minMutationRate" : 10000, 22 | "maxMutationRate" : 100000, 23 | "minScanRate" : 1000, 24 | "maxScanRate" : 10000 25 | }], 26 | "distribution" : [100] 27 | }], 28 | "distribution" : [100], 29 | "minNumIndex" : 30, 30 | "maxNumIndex" : 30 31 | } 32 | 33 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/workload/uniform-small-10-3.json: -------------------------------------------------------------------------------- 1 | { 2 | "name" : "Small Workload (20-50M)", 3 | "comment" : "", 4 | "workload" : [ 5 | { 6 | "name" : "bucket1", 7 | "replica" : 3, 8 | "workload" : [ 9 | { 10 | "name" : "small", 11 | "minNumDoc" : 20000000, 12 | "maxNumDoc" : 50000000, 13 | "minDocKeySize" : 20, 14 | "maxDocKeySize" : 200, 15 | "minSecKeySize" : 20, 16 | "maxSecKeySize" : 200, 17 | "minArrKeySize" : 0, 18 | "maxArrKeySize" : 0, 19 | "minArrSize" : 0, 20 | "maxArrSize" : 0, 21 | "minMutationRate" : 10000, 22 | "maxMutationRate" : 100000, 23 | "minScanRate" : 1000, 24 | "maxScanRate" : 10000 25 | }], 26 | "distribution" : [100] 27 | }], 28 | "distribution" : [100], 29 | "minNumIndex" : 10, 30 | "maxNumIndex" : 10 31 | } 32 | 33 | -------------------------------------------------------------------------------- /secondary/tests/testdata/planner/workload/uniform-small-30-3.json: -------------------------------------------------------------------------------- 1 | { 2 | "name" : "Small Workload (20-50M)", 3 | "comment" : "", 4 | "workload" : [ 5 | { 6 | "name" : "bucket1", 7 | "replica" : 3, 8 | "workload" : [ 9 | { 10 | "name" : "small", 11 | "minNumDoc" : 20000000, 12 | "maxNumDoc" : 50000000, 13 | "minDocKeySize" : 20, 14 | "maxDocKeySize" : 200, 15 | "minSecKeySize" : 20, 16 | "maxSecKeySize" : 200, 17 | "minArrKeySize" : 0, 18 | "maxArrKeySize" : 0, 19 | "minArrSize" : 0, 20 | "maxArrSize" : 0, 21 | "minMutationRate" : 10000, 22 | "maxMutationRate" : 100000, 23 | "minScanRate" : 1000, 24 | "maxScanRate" : 10000 25 | }], 26 | "distribution" : [100] 27 | }], 28 | "distribution" : [100], 29 | "minNumIndex" : 30, 30 | "maxNumIndex" : 30 31 | } 32 | 33 | -------------------------------------------------------------------------------- /secondary/tests/testdata/projects.json.bz2: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/couchbase/indexing/ba752cd53c7865086c4d99f312eafbe3d23982a9/secondary/tests/testdata/projects.json.bz2 -------------------------------------------------------------------------------- /secondary/tests/testdata/projects.prod: -------------------------------------------------------------------------------- 1 | json : "{ " properties "}". 2 | 3 | properties : DQ "type" DQ ": " DQ "project" DQ ", " NL 4 | DQ "name" DQ ": " DQ projectname DQ ", " NL 5 | DQ "members" DQ ": " "[" members "]" ", " NL 6 | DQ "language" DQ ": " DQ language DQ NL. 7 | 8 | projectname : bag("propernames"). 9 | members : members "," DQ bag("propernames") DQ {100} 10 | | DQ bag("propernames") DQ. 11 | language : "python" 12 | | "haskell" 13 | | "lua" 14 | | "golang". 15 | -------------------------------------------------------------------------------- /secondary/tests/testdata/users.json.bz2: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/couchbase/indexing/ba752cd53c7865086c4d99f312eafbe3d23982a9/secondary/tests/testdata/users.json.bz2 -------------------------------------------------------------------------------- /secondary/tests/testdata/users.prod: -------------------------------------------------------------------------------- 1 | json : "{ " properties "}". 2 | 3 | properties : DQ "type" DQ ": " DQ "user" DQ ", " NL 4 | DQ "first-name" DQ ": " DQ fname DQ ", " NL 5 | lastname 6 | age 7 | emailid 8 | city 9 | gender. 10 | 11 | fname : bag("./propernames"). 12 | lastname : DQ "last-name" DQ ": " DQ bag("./propernames") DQ ", " NL. 13 | age : DQ "age" DQ ": " range(15, 80) ", " NL. 14 | emailid : DQ "emailid" DQ ": " DQ $fname "@gmail.com" DQ ", " NL. 15 | city : DQ "city" DQ ": " DQ bag("./cities") DQ ", " NL. 16 | gender : DQ "gender" DQ ": " DQ "male" DQ NL 17 | | DQ "gender" DQ ": " DQ "female" DQ NL. 18 | -------------------------------------------------------------------------------- /secondary/tests/testdata/web2.bz2: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/couchbase/indexing/ba752cd53c7865086c4d99f312eafbe3d23982a9/secondary/tests/testdata/web2.bz2 -------------------------------------------------------------------------------- /secondary/tools/cluster_ops/Makefile: -------------------------------------------------------------------------------- 1 | all: clean comp run 2 | 3 | export GOPATH=$(PWD)/../../../../../../../../goproj:$(PWD)/../../../../../../../../godeps 4 | export GO111MODULE=auto 5 | export CGO_CFLAGS=-I$(PWD)/../../../../../../../../sigar/include -I$(PWD)/../../../../../../../../build/tlm/deps/zstd-cpp.exploded/include -I$(PWD)/../../../../../../../../build/tlm/deps/jemalloc.exploded/include -I$(PWD)/../../../../../../../../forestdb/include/ -DJEMALLOC=1 6 | export CGO_LDFLAGS=-L $(PWD)/../../../../../../../../install/lib -Wl,-rpath $(PWD)/../../../../../../../../install/lib 7 | 8 | clean: 9 | rm -rf ./cluster_ops 10 | 11 | comp: 12 | go build -o cluster_ops main.go init_cluster.go 13 | 14 | run: 15 | # Performs a rebalance on the cluster without adding or removing 16 | # nodes from the cluster. See Readme.md for other options to compile 17 | # and run 18 | ./cluster_ops 19 | -------------------------------------------------------------------------------- /secondary/tools/cluster_ops/Readme.md: -------------------------------------------------------------------------------- 1 | Usage: 2 | 3 | Do "make comp" to compile the tool 4 | 5 | ./cluster_ops -> Performs only rebalance on the existing cluster. If cluster is not initialised, initialises the cluster and adds nodes 127.0.0.1:9000 (kv+n1ql), 127.0.0.1:9001 (index), 127.0.0.1:9002 (index) services to cluster 6 | ./cluster_ops --Nodes 127.0.0.1:9003 --Nodes 127.0.0.1:9004 -> Initialises cluster with nodes 127.0.0.1:9003 & 127.0.0.1:9004 to the cluster 7 | ./cluster_ops --addNodes 127.0.0.1:9003 --addNodes 127.0.0.1:9004 -> Adds nodes 127.0.0.1:9003 & 127.0.0.1:9004 to the cluster 8 | and performs rebalance (Unless -rebalance=false is set explicitly) 9 | ./cluster_ops --ejectNodes 127.0.0.1:9003 --ejectNodes 127.0.0.1:9004 -> Ejects nodes 127.0.0.1:9003 & 127.0.0.1:9004 to the cluster 10 | and performs rebalance (Unless -rebalance=false is set explicitly) 11 | -------------------------------------------------------------------------------- /secondary/tools/fdb/fdb_slice_thrpt.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import ( 4 | "fmt" 5 | "log" 6 | "net/http" 7 | _ "net/http/pprof" 8 | "os" 9 | "runtime/pprof" 10 | "time" 11 | 12 | "github.com/couchbase/indexing/secondary/common" 13 | "github.com/couchbase/indexing/secondary/indexer" 14 | ) 15 | 16 | func main() { 17 | 18 | go func() { 19 | http.ListenAndServe("localhost:6060", nil) 20 | }() 21 | 22 | f, _ := os.Create("cpu.prof") 23 | pprof.StartCPUProfile(f) 24 | do_test1() 25 | pprof.StopCPUProfile() 26 | 27 | f, _ = os.Create("mem.prof") 28 | pprof.WriteHeapProfile(f) 29 | f.Close() 30 | 31 | } 32 | 33 | func do_test1() { 34 | 35 | log.Println("***** TEST1 TWO KV STORE 5M INSERT EACH WITH GET *****") 36 | 37 | config := common.SystemConfig.SectionConfig( 38 | "indexer.", true /*trim*/) 39 | 40 | stats := &indexer.IndexStats{} 41 | stats.Init() 42 | 43 | slice, _ := indexer.NewForestDBSlice(".", 0, 1, 1, 1, false, 1, config, stats) 44 | 45 | log.Println("***** GENERATING *****") 46 | numItems := 5000000 47 | keys := make([][]byte, numItems) 48 | //lenkeys := make([]int, numItems) 49 | vals := make([][]byte, numItems) 50 | //lenvals := make([]int, numItems) 51 | for i := 0; i < numItems; i++ { 52 | keys[i] = []byte(fmt.Sprintf("perf%v", i)) 53 | //lenkeys[i] = len(keys[i]) 54 | vals[i] = []byte(fmt.Sprintf("body%v", i)) 55 | // lenvals[i] = len(vals[i]) 56 | } 57 | 58 | time.Sleep(3 * time.Second) 59 | log.Println("***** LOADING*****") 60 | start := time.Now() 61 | for i := 0; i < numItems; i++ { 62 | slice.Insert(keys[i], vals[i], nil, nil) 63 | } 64 | // dbfile.Commit(forestdb.COMMIT_MANUAL_WAL_FLUSH) 65 | slice.NewSnapshot(nil, false) 66 | 67 | elapsed := time.Since(start) 68 | log.Printf("***** RESULT : Docs Inserted %v Time Taken %v", numItems, elapsed) 69 | } 70 | -------------------------------------------------------------------------------- /secondary/tools/loadgen/companies: -------------------------------------------------------------------------------- 1 | montalvo 2 | freescale 3 | wipro 4 | -------------------------------------------------------------------------------- /secondary/tools/loadgen/projects.prod: -------------------------------------------------------------------------------- 1 | s : "{ " properties "}". 2 | 3 | properties : DQ "type" DQ ": " DQ "project" DQ ", " NL 4 | DQ "name" DQ ": " DQ projectname DQ ", " NL 5 | DQ "members" DQ ": " "[" members "]" ", " NL 6 | DQ "language" DQ ": " DQ language DQ NL. 7 | 8 | projectname : (bag "./propernames"). 9 | members : DQ (bag "./propernames") DQ 10 | | members "," DQ (bag "./propernames") DQ. 11 | language : "python" 12 | | "haskell" 13 | | "lua" 14 | | "golang". 15 | -------------------------------------------------------------------------------- /secondary/tools/loadgen/users.prod: -------------------------------------------------------------------------------- 1 | (let guid (uuid)) 2 | 3 | s : "{ " properties "}". 4 | 5 | properties : type ",\n" 6 | docid ",\n" 7 | age ",\n" 8 | active ",\n" 9 | company ",\n" 10 | "\"first-name\": \"" fname DQ ",\n" 11 | lastname ",\n" 12 | emailid ",\n" 13 | gender. 14 | 15 | type : "\"type\": \"user\"". 16 | docid : "\"docid\": \"" (sprintf "User-%v" $guid) DQ. 17 | age : "\"age\": " (range 15 80). 18 | active : "\"isActive\": " (choice "true" "false"). 19 | company : "\"company\": \"" (bag "./companies") DQ. 20 | fname : (bag "./propernames"). 21 | lastname : "\"last-name\": \"" (bag "./propernames") DQ. 22 | emailid : "\"emailid\": \"" $fname "@gmail.com" DQ. 23 | gender : "\"gender\": \"" (choice "male" "female") DQ. 24 | -------------------------------------------------------------------------------- /secondary/tools/loadgen/util.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import "fmt" 4 | import "time" 5 | import "math/rand" 6 | 7 | func ranget(starts, ends string) string { 8 | start, err := time.Parse(time.RFC3339, starts) 9 | if err != nil { 10 | panic(fmt.Errorf("parsing first argument %v: %v\n", starts, err)) 11 | } 12 | end, err := time.Parse(time.RFC3339, ends) 13 | if err != nil { 14 | panic(fmt.Errorf("parsing second argument %v: %v\n", ends, err)) 15 | } 16 | t := start.Add(time.Duration(rand.Int63n(int64(end.Sub(start))))) 17 | return t.Format(time.RFC3339) 18 | } 19 | -------------------------------------------------------------------------------- /secondary/tools/n1qlexpr/doc.json: -------------------------------------------------------------------------------- 1 | { "age": 78, "friends": ["tom", "jerry"]} 2 | -------------------------------------------------------------------------------- /secondary/tools/n1qlexpr/n1ql_expr.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import "log" 4 | import "flag" 5 | import "io/ioutil" 6 | import "fmt" 7 | import "encoding/json" 8 | 9 | import "github.com/couchbase/query/parser/n1ql" 10 | import qexpr "github.com/couchbase/query/expression" 11 | import qvalue "github.com/couchbase/query/value" 12 | 13 | var options struct { 14 | expr string 15 | data string 16 | json bool 17 | } 18 | 19 | func argParse() { 20 | flag.StringVar(&options.expr, "expr", "", 21 | "input expression") 22 | flag.StringVar(&options.data, "data", "", 23 | "data similar to curl's --data") 24 | flag.BoolVar(&options.json, "json", false, 25 | "marshal parsed expression back to JSON") 26 | flag.Parse() 27 | } 28 | 29 | func main() { 30 | argParse() 31 | expr, err := n1ql.ParseExpression(options.expr) 32 | if err != nil { 33 | log.Fatal(err) 34 | } 35 | 36 | if options.json { 37 | exprb, err := json.Marshal(expr) 38 | if err != nil { 39 | log.Fatal(err) 40 | } 41 | exprstr := string(exprb) 42 | fmt.Printf("input: %v\n", options.expr) 43 | fmt.Printf("expr: %T %v\n", expr, expr) 44 | fmt.Printf("json: %T %v\n", exprstr, exprstr) 45 | } else { 46 | exprstr := qexpr.NewStringer().Visit(expr) 47 | fmt.Printf("input: %v\n", options.expr) 48 | fmt.Printf("expr: %T %v\n", expr, expr) 49 | fmt.Printf("output: %T %v\n", exprstr, exprstr) 50 | } 51 | 52 | if expr != nil && options.data != "" { 53 | if options.data[0] == '@' { 54 | docbytes, err := ioutil.ReadFile(options.data[1:]) 55 | if err != nil { 56 | log.Fatal(err) 57 | } 58 | context := qexpr.NewIndexContext() 59 | doc := qvalue.NewAnnotatedValue(docbytes) 60 | v, err := expr.Evaluate(doc, context) 61 | fmsg := "Evaluate() scalar:%v err:%v" 62 | fmt.Printf(fmsg, v, err) 63 | } 64 | } 65 | } 66 | -------------------------------------------------------------------------------- /secondary/tools/n1qlperf/lookup.json: -------------------------------------------------------------------------------- 1 | { 2 | "Concurrency" : 1, 3 | "Clients": 1, 4 | "ScanSpecs" : [ 5 | { 6 | "Type" : "Lookup", 7 | "Bucket" : "default", 8 | "Id" : 7962660343139546135, 9 | "Index" : "index_eyeColor", 10 | "Lookups" : [ ["blue"] ], 11 | "Limit" : 1, 12 | "Repeat" : 100000, 13 | "Consistency": false 14 | } 15 | ] 16 | } 17 | -------------------------------------------------------------------------------- /secondary/tools/n1qlperf/n1qlperf.sh: -------------------------------------------------------------------------------- 1 | export NS_SERVER_CBAUTH_URL="http://127.0.0.1:9000/_cbauth" 2 | export NS_SERVER_CBAUTH_USER="Administrator" 3 | export NS_SERVER_CBAUTH_PWD="asdasd" 4 | export NS_SERVER_CBAUTH_RPC_URL="http://127.0.0.1:9000/cbauth-demo" 5 | 6 | export CBAUTH_REVRPC_URL="http://Administrator:asdasd@127.0.0.1:9000/query" 7 | 8 | ./n1qlperf "$@" 9 | 10 | -------------------------------------------------------------------------------- /secondary/tools/querycmd/benchtimeit.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import "time" 4 | import "fmt" 5 | 6 | import qclient "github.com/couchbase/indexing/secondary/queryport/client" 7 | 8 | func doBenchtimeit(cluster string, client *qclient.GsiClient) (err error) { 9 | start := time.Now() 10 | for i := 0; i < 1000000; i++ { 11 | client.Bridge().Timeit(0x1111, 1) 12 | } 13 | fmt.Printf("time take by Timeit(): %v\n", time.Since(start)/1000000) 14 | return 15 | } 16 | -------------------------------------------------------------------------------- /secondary/tools/querycmd/bufferedscan.sh: -------------------------------------------------------------------------------- 1 | export NS_SERVER_CBAUTH_URL="http://127.0.0.1:9000/_cbauth" 2 | export NS_SERVER_CBAUTH_USER="Administrator" 3 | export NS_SERVER_CBAUTH_PWD="asdasd" 4 | export NS_SERVER_CBAUTH_RPC_URL="http://127.0.0.1:9000/cbauth-demo" 5 | 6 | export CBAUTH_REVRPC_URL="http://Administrator:asdasd@127.0.0.1:9000/query" 7 | 8 | ./querycmd "$@" 9 | -------------------------------------------------------------------------------- /secondary/tools/querycmd/mb13339.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import "fmt" 4 | import "log" 5 | import "os" 6 | 7 | import qclient "github.com/couchbase/indexing/secondary/queryport/client" 8 | import "github.com/couchbase/indexing/secondary/querycmd" 9 | 10 | // test case to simulate 11 | // https://issues.couchbase.com/browse/MB-13339 12 | 13 | func doMB13339( 14 | cluster string, client *qclient.GsiClient) (err error) { 15 | 16 | for _, args := range mb13339Commands { 17 | cmd, _, _, err := querycmd.ParseArgs(args) 18 | if err != nil { 19 | log.Fatal(err) 20 | } 21 | err = querycmd.HandleCommand(client, cmd, true, os.Stdout) 22 | if err != nil { 23 | fmt.Printf("%#v\n", cmd) 24 | fmt.Printf(" %v\n", err) 25 | } 26 | fmt.Println() 27 | } 28 | return 29 | } 30 | 31 | var mb13339Commands = [][]string{ 32 | []string{ 33 | "-type", "create", "-bucket", "beer-sample", "-index", "index-city", 34 | "-fields", "city", 35 | }, 36 | []string{ 37 | "-type", "create", "-bucket", "beer-sample", "-index", "index-type", 38 | "-fields", "type", 39 | }, 40 | []string{ 41 | "-type", "count", "-bucket", "beer-sample", "-index", "index-city", 42 | "-low", "[\"A\"]", "-high", "[\"s\"]", 43 | }, 44 | []string{"-type", "list", "-bucket", "beer-sample"}, 45 | []string{ // FIXME: drop and then retry ? 46 | "-type", "drop", "-indexes", "beer-sample:index-type", 47 | }, 48 | []string{ // retry 49 | "-type", "create", "-bucket", "beer-sample", "-index", "index-type", 50 | "-fields", "type", 51 | }, 52 | []string{ 53 | "-type", "count", "-bucket", "beer-sample", "-index", "index-type", 54 | "-equal", "[\"brewery\"]", 55 | }, 56 | []string{"-type", "list", "-bucket", "beer-sample"}, 57 | []string{ 58 | "-type", "drop", "-indexes", "beer-sample:index-city", 59 | }, 60 | []string{ 61 | "-type", "drop", "-indexes", "beer-sample:index-type", 62 | }, 63 | } 64 | -------------------------------------------------------------------------------- /secondary/tools/querycmd/mb14786.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import "fmt" 4 | import "log" 5 | import "os" 6 | 7 | import qclient "github.com/couchbase/indexing/secondary/queryport/client" 8 | import "github.com/couchbase/indexing/secondary/querycmd" 9 | 10 | // test case to simulate 11 | // https://issues.couchbase.com/browse/MB-13339 12 | 13 | func doMB14786( 14 | cluster string, client *qclient.GsiClient) (err error) { 15 | 16 | for _, args := range mb14786Commands { 17 | cmd, _, _, err := querycmd.ParseArgs(args) 18 | if err != nil { 19 | log.Fatal(err) 20 | } 21 | err = querycmd.HandleCommand(client, cmd, true, os.Stdout) 22 | if err != nil { 23 | fmt.Printf("%#v\n", cmd) 24 | fmt.Printf(" %v\n", err) 25 | } 26 | fmt.Println() 27 | } 28 | return 29 | } 30 | 31 | var mb14786Commands = [][]string{ 32 | []string{"-type", "drop", "-bucket", "beer-sample", "-index", "index_abv1"}, 33 | []string{"-type", "drop", "-bucket", "beer-sample", "-index", "index_abv2"}, 34 | 35 | []string{ 36 | "-type", "create", "-bucket", "beer-sample", "-index", "index_abv1", 37 | "-fields", "abv", 38 | }, 39 | []string{ 40 | "-type", "create", "-bucket", "beer-sample", "-index", "index_abv2", 41 | "-fields", "abv", 42 | }, 43 | 44 | []string{ 45 | "-type", "count", "-bucket", "beer-sample", "-index", "index_abv1", 46 | "-low", "[1.0]", "-high", "[100.0]", 47 | }, 48 | []string{ 49 | "-type", "count", "-bucket", "beer-sample", "-index", "index_abv2", 50 | "-low", "[1.0]", "-high", "[100.0]", 51 | }, 52 | []string{ 53 | "-type", "count", "-bucket", "beer-sample", "-index", "index_abv1", 54 | "-low", "[1.0]", "-high", "[100.0]", 55 | }, 56 | } 57 | -------------------------------------------------------------------------------- /secondary/tools/querycmd/scan_retry.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import ( 4 | "fmt" 5 | "log" 6 | "os" 7 | "time" 8 | 9 | c "github.com/couchbase/indexing/secondary/common" 10 | 11 | "github.com/couchbase/indexing/secondary/querycmd" 12 | qclient "github.com/couchbase/indexing/secondary/queryport/client" 13 | ) 14 | 15 | func doScanRetry(cluster string, client *qclient.GsiClient) (err error) { 16 | 17 | crArgs := retryCommands[0] 18 | cmd, _, _, err := querycmd.ParseArgs(crArgs) 19 | if err != nil { 20 | log.Fatal(err) 21 | } 22 | err = querycmd.HandleCommand(client, cmd, true, os.Stdout) 23 | if err != nil { 24 | fmt.Printf("%#v\n", cmd) 25 | fmt.Printf(" %v\n", err) 26 | } 27 | fmt.Println() 28 | 29 | for { 30 | index, _ := querycmd.GetIndex(client, cmd.Bucket, cmd.IndexName) 31 | if index == nil { 32 | time.Sleep(5 * time.Second) 33 | continue 34 | } 35 | defnID := uint64(index.Definition.DefnId) 36 | 37 | crArgs := retryCommands[1] 38 | cmd, _, _, err := querycmd.ParseArgs(crArgs) 39 | if err != nil { 40 | log.Fatal(err) 41 | } 42 | low, high, incl := cmd.Low, cmd.High, cmd.Inclusion 43 | cons := c.SessionConsistency 44 | 45 | fmt.Printf("CountRange:\n") 46 | count, err := client.CountRange( 47 | uint64(defnID), "requestId", low, high, incl, cons, nil) 48 | if err == nil { 49 | fmt.Printf( 50 | "Index %q/%q has %v entries\n", cmd.Bucket, cmd.IndexName, count) 51 | } else { 52 | fmt.Printf("fail: %v %v\n", defnID, err) 53 | } 54 | fmt.Println() 55 | time.Sleep(1 * time.Second) 56 | } 57 | return 58 | } 59 | 60 | var retryCommands = [][]string{ 61 | []string{ 62 | "-type", "create", "-bucket", "default", "-index", "iname", 63 | "-fields", "name", 64 | }, 65 | []string{ 66 | "-type", "count", "-bucket", "default", "-index", "iname", 67 | "-low", `["A"]`, "-high", `["Z"]`, "-limit", "10000", 68 | }, 69 | } 70 | -------------------------------------------------------------------------------- /secondary/tools/randdocs/Makefile: -------------------------------------------------------------------------------- 1 | all: clean comp run 2 | 3 | export GOPATH=$(PWD)/../../../../../../../../goproj:$(PWD)/../../../../../../../../godeps 4 | export GO111MODULE=auto 5 | export CGO_CFLAGS=-I$(PWD)/../../../../../../../../sigar/include -I$(PWD)/../../../../../../../../build/tlm/deps/zstd-cpp.exploded/include -I$(PWD)/../../../../../../../../build/tlm/deps/jemalloc.exploded/include -I$(PWD)/../../../../../../../../forestdb/include/ -DJEMALLOC=1 6 | export CGO_LDFLAGS=-L $(PWD)/../../../../../../../../install/lib -Wl,-rpath $(PWD)/../../../../../../../../install/lib 7 | 8 | clean: 9 | rm -rf ./randdocs 10 | 11 | comp: 12 | go build -o randdocs cmd/main.go 13 | 14 | run: 15 | ./randdocs -config ./cmd/config.json 16 | 17 | nomodcomp: 18 | env GO111MODULE=off go build -o randdocs cmd/main.go 19 | 20 | loadrandvecs: 21 | ./randdocs -config ./cmd/config.json -genVectors 22 | 23 | loadsiftsmall: 24 | ./randdocs -config ./cmd/config.json -useSIFTSmall -NumDocs 40000 -skipNormalData 25 | -------------------------------------------------------------------------------- /secondary/tools/randdocs/cmd/config.json: -------------------------------------------------------------------------------- 1 | { 2 | "ClusterAddr": "127.0.0.1:9000", 3 | "Bucket": "default", 4 | "Scope": "", 5 | "Collection": "", 6 | "NumDocs": 1000, 7 | "DocIdLen": 25, 8 | "FieldSize": 256, 9 | "ArrayLen": 0, 10 | "JunkFieldSize": 0, 11 | "Iterations": 1, 12 | "Threads": 8, 13 | "DocsNumOffset": 0, 14 | "OpsPerSec": 100000, 15 | "VecDimension": 128, 16 | "UseSIFTSmall": false 17 | } 18 | -------------------------------------------------------------------------------- /secondary/tools/randdocs/siftsmall/siftsmall_base.fvecs: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/couchbase/indexing/ba752cd53c7865086c4d99f312eafbe3d23982a9/secondary/tools/randdocs/siftsmall/siftsmall_base.fvecs -------------------------------------------------------------------------------- /secondary/tools/randdocs/siftsmall/siftsmall_groundtruth.ivecs: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/couchbase/indexing/ba752cd53c7865086c4d99f312eafbe3d23982a9/secondary/tools/randdocs/siftsmall/siftsmall_groundtruth.ivecs -------------------------------------------------------------------------------- /secondary/tools/randdocs/siftsmall/siftsmall_learn.fvecs: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/couchbase/indexing/ba752cd53c7865086c4d99f312eafbe3d23982a9/secondary/tools/randdocs/siftsmall/siftsmall_learn.fvecs -------------------------------------------------------------------------------- /secondary/tools/randdocs/siftsmall/siftsmall_query.fvecs: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/couchbase/indexing/ba752cd53c7865086c4d99f312eafbe3d23982a9/secondary/tools/randdocs/siftsmall/siftsmall_query.fvecs -------------------------------------------------------------------------------- /secondary/vector/faiss/autotune.go: -------------------------------------------------------------------------------- 1 | package faiss 2 | 3 | /* 4 | #include 5 | #include 6 | */ 7 | import "C" 8 | import ( 9 | "unsafe" 10 | ) 11 | 12 | type ParameterSpace struct { 13 | ps *C.FaissParameterSpace 14 | } 15 | 16 | // NewParameterSpace creates a new ParameterSpace. 17 | func NewParameterSpace() (*ParameterSpace, error) { 18 | var ps *C.FaissParameterSpace 19 | if c := C.faiss_ParameterSpace_new(&ps); c != 0 { 20 | return nil, getLastError() 21 | } 22 | return &ParameterSpace{ps}, nil 23 | } 24 | 25 | // SetIndexParameter sets one of the parameters. 26 | func (p *ParameterSpace) SetIndexParameter(idx Index, name string, val float64) error { 27 | cname := C.CString(name) 28 | 29 | defer func() { 30 | C.free(unsafe.Pointer(cname)) 31 | }() 32 | 33 | c := C.faiss_ParameterSpace_set_index_parameter( 34 | p.ps, idx.cPtr(), cname, C.double(val)) 35 | if c != 0 { 36 | return getLastError() 37 | } 38 | return nil 39 | } 40 | 41 | // Delete frees the memory associated with p. 42 | func (p *ParameterSpace) Delete() { 43 | if p.ps != nil { 44 | C.faiss_ParameterSpace_free(p.ps) 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /secondary/vector/faiss/faiss.go: -------------------------------------------------------------------------------- 1 | // Package faiss provides bindings to Faiss, a library for vector similarity 2 | // search. 3 | // More detailed documentation can be found at the Faiss wiki: 4 | // https://github.com/facebookresearch/faiss/wiki. 5 | package faiss 6 | 7 | /* 8 | #cgo LDFLAGS: -lfaiss_c 9 | 10 | #include 11 | #include 12 | #include 13 | */ 14 | import "C" 15 | import "errors" 16 | 17 | func getLastError() error { 18 | return errors.New(C.GoString(C.faiss_get_last_error())) 19 | } 20 | 21 | // Metric type 22 | const ( 23 | MetricInnerProduct = C.METRIC_INNER_PRODUCT 24 | MetricL2 = C.METRIC_L2 25 | ) 26 | 27 | //Quantizer type 28 | const ( 29 | QT_8bit = C.QT_8bit 30 | QT_4bit = C.QT_4bit 31 | QT_8bit_uniform = C.QT_8bit_uniform 32 | QT_4bit_uniform = C.QT_4bit_uniform 33 | QT_fp16 = C.QT_fp16 34 | QT_8bit_direct = C.QT_8bit_direct 35 | QT_6bit = C.QT_6bit 36 | ) 37 | 38 | const DEFAULT_EF_SEARCH = 16 39 | -------------------------------------------------------------------------------- /secondary/vector/indexivf.go: -------------------------------------------------------------------------------- 1 | package vector 2 | 3 | import ( 4 | "fmt" 5 | 6 | "github.com/couchbase/indexing/secondary/common" 7 | faiss "github.com/couchbase/indexing/secondary/vector/faiss" 8 | ) 9 | 10 | func NewIndexFlat(dim, metric int) (*faiss.IndexImpl, error) { 11 | 12 | description := "Flat" 13 | return faiss.IndexFactory(dim, description, metric) 14 | } 15 | 16 | func NewIndexIVFPQ(dim, nlist, nsub, nbits, metric int, useFastScan bool) (*faiss.IndexImpl, error) { 17 | fs := "" 18 | if useFastScan { 19 | fs = "fs" 20 | } 21 | 22 | description := fmt.Sprintf("IVF%v,PQ%vx%v%v", nlist, nsub, nbits, fs) 23 | return faiss.IndexFactory(dim, description, metric) 24 | } 25 | 26 | func NewIndexIVFPQ_HNSW(dim, nlist, nsub, nbits, metric int, useFastScan bool) (*faiss.IndexImpl, error) { 27 | fs := "" 28 | if useFastScan { 29 | fs = "fs" 30 | } 31 | 32 | description := fmt.Sprintf("IVF%v_HNSW,PQ%vx%v%v", nlist, nsub, nbits, fs) 33 | return faiss.IndexFactory(dim, description, metric) 34 | 35 | } 36 | 37 | func NewIndexIVFSQ(dim, nlist, metric int, sqRange common.ScalarQuantizerRange) (*faiss.IndexImpl, error) { 38 | 39 | description := fmt.Sprintf("IVF%v,%v", nlist, sqRange) 40 | return faiss.IndexFactory(dim, description, metric) 41 | } 42 | 43 | func NewIndexIVFSQ_HNSW(dim, nlist, metric int, sqRange common.ScalarQuantizerRange) (*faiss.IndexImpl, error) { 44 | if sqRange == common.SQ_FP16 { 45 | sqRange = "SQfp16" 46 | } 47 | 48 | description := fmt.Sprintf("IVF%v_HNSW,%v", nlist, sqRange) 49 | return faiss.IndexFactory(dim, description, metric) 50 | 51 | } 52 | -------------------------------------------------------------------------------- /secondary/vector/util/index_evaluator_test.go: -------------------------------------------------------------------------------- 1 | package vectorutil 2 | 3 | import ( 4 | "testing" 5 | 6 | "github.com/couchbase/indexing/secondary/common" 7 | c "github.com/couchbase/indexing/secondary/common" 8 | ) 9 | 10 | func SkipTestFetchSampleVectorsForIndexes(t *testing.T) { 11 | 12 | cluster := "127.0.0.1:8091" 13 | pool := "default" 14 | bucket := "default" 15 | scope := "_default" 16 | collection := "_default" 17 | cid := "0" 18 | 19 | dim := 128 20 | 21 | meta := &c.VectorMetadata{ 22 | IsCompositeIndex: true, 23 | Dimension: dim, 24 | } 25 | 26 | idxDefn := c.IndexDefn{ 27 | DefnId: c.IndexDefnId(200), 28 | Name: "index_evaluator", 29 | Using: common.PlasmaDB, 30 | Bucket: bucket, 31 | IsPrimary: false, 32 | SecExprs: []string{"description"}, 33 | ExprType: c.N1QL, 34 | IsVectorIndex: true, 35 | HasVectorAttr: []bool{true}, 36 | Scope: scope, 37 | Collection: collection, 38 | VectorMeta: meta, 39 | } 40 | 41 | indexInst := &common.IndexInst{ 42 | InstId: c.IndexInstId(300), 43 | Defn: idxDefn, 44 | } 45 | 46 | indexInsts := []*c.IndexInst{indexInst} 47 | 48 | vectors, err := FetchSampleVectorsForIndexes(cluster, pool, bucket, scope, collection, cid, indexInsts, 800, 1) 49 | 50 | t.Logf("Vectors %v, Err %v", len(vectors[0])/dim, err) 51 | 52 | t.Logf("Sample Vector %v", vectors[0][:dim]) 53 | } 54 | -------------------------------------------------------------------------------- /secondary/vector/utils.go: -------------------------------------------------------------------------------- 1 | package vector 2 | 3 | import ( 4 | "math/rand" 5 | "time" 6 | ) 7 | 8 | func genRandomVecs(dims int, n int, seed int64) [][]float32 { 9 | if seed != 0 { 10 | rand.Seed(seed) 11 | } else { 12 | rand.Seed(time.Now().UnixNano()) 13 | } 14 | 15 | vecs := make([][]float32, n) 16 | for i := 0; i < n; i++ { 17 | vecs[i] = make([]float32, dims) 18 | for j := 0; j < dims; j++ { 19 | vecs[i][j] = rand.Float32() 20 | } 21 | } 22 | 23 | return vecs 24 | } 25 | 26 | func convertTo1D(vecs [][]float32) []float32 { 27 | 28 | var vecs1D []float32 29 | for _, vec := range vecs { 30 | vecs1D = append(vecs1D, vec...) 31 | } 32 | return vecs1D 33 | } 34 | 35 | func compareVecs(vec1, vec2 []float32) bool { 36 | // Check if vectors have the same dimension 37 | if len(vec1) != len(vec2) { 38 | return false 39 | } 40 | 41 | // Iterate over each element and compare 42 | for i := 0; i < len(vec1); i++ { 43 | if vec1[i] != vec2[i] { 44 | return false 45 | } 46 | } 47 | 48 | return true 49 | } 50 | --------------------------------------------------------------------------------