├── .github
└── ISSUE_TEMPLATE
│ ├── bug_report.md
│ └── feature_request.md
├── .gitignore
├── .gitmodules
├── .travis.yml
├── .travis
├── cleanup.sh
├── install_clang_tools.sh
├── install_cuda.sh
├── install_golang_tools.sh
├── pre_install.sh
└── run_unittest.sh
├── CMakeLists.txt
├── CODEOWNERS
├── CODE_OF_CONDUCT.md
├── CONTRIBUTING.md
├── LICENSE
├── NOTICE
├── README.md
├── api
├── api_test.go
├── ares_suite_test.go
├── common
│ ├── ares_suite_test.go
│ ├── error.go
│ ├── query_request.go
│ ├── query_response.go
│ ├── request.go
│ ├── request_test.go
│ └── response.go
├── data_handler.go
├── data_handler_test.go
├── data_request.go
├── debug_handler.go
├── debug_handler_test.go
├── debug_request.go
├── debug_response.go
├── doc.go
├── enum_handler.go
├── enum_handler_test.go
├── error.go
├── health_check_handler.go
├── health_handler_test.go
├── panic_handler.go
├── panic_handler_test.go
├── query_handler.go
├── query_handler_test.go
├── schema_handler.go
├── schema_handler_test.go
├── schema_request.go
├── schema_response.go
├── sql_query_handler.go
├── sql_query_handler_test.go
└── ui
│ ├── debug
│ ├── archive.html
│ ├── backfill.html
│ ├── batch.html
│ ├── bootstrap.html
│ ├── css
│ │ ├── batch.css
│ │ └── index.css
│ ├── index.html
│ ├── js
│ │ ├── archive.js
│ │ ├── backfill.js
│ │ ├── batch.js
│ │ ├── bootstrap.js
│ │ ├── index.js
│ │ ├── memory.js
│ │ ├── purge.js
│ │ ├── redologs.js
│ │ ├── snapshot.js
│ │ ├── summary.js
│ │ └── utils.js
│ ├── memory.html
│ ├── purge.html
│ ├── redologs.html
│ ├── snapshot.html
│ └── summary.html
│ ├── npm.updated
│ ├── package.json
│ └── swagger
│ ├── index.html
│ └── swagger.json
├── broker
├── broker_schema_mutator.go
├── broker_schema_mutator_test.go
├── broker_suite_test.go
├── common
│ ├── constant.go
│ ├── mocks
│ │ ├── BlockingPlanNode.go
│ │ ├── MergeNode.go
│ │ ├── QueryExecutor.go
│ │ └── StreamingPlanNode.go
│ └── types.go
├── config
│ └── config.go
├── executor.go
├── handler.go
├── handler_test.go
├── query_compiler.go
├── query_compiler_test.go
├── query_plan_agg.go
├── query_plan_agg_test.go
├── query_plan_non_agg.go
├── query_plan_non_agg_test.go
├── result_merge.go
├── result_merge_test.go
└── util
│ ├── assignment.go
│ ├── assignment_test.go
│ └── broker_suite_test.go
├── cgoutils
├── ares_suite_test.go
├── memory.go
├── memory.h
├── memory
│ ├── cuda_malloc.cu
│ ├── malloc.c
│ └── rmm_alloc.cu
├── memory_test.go
├── utils.go
└── utils.h
├── client
├── ares_suite_test.go
├── connector.go
├── connector_test.go
├── mocks
│ └── Connector.go
└── schema.go
├── cluster
├── ares_suite_test.go
├── kvstore
│ ├── etcd.go
│ └── transaction.go
├── shard
│ ├── ares_suite_test.go
│ ├── mocks
│ │ └── ShardSet.go
│ ├── shardset.go
│ ├── shardset_test.go
│ └── types.go
├── topology
│ ├── ares_suite_test.go
│ ├── dynamic.go
│ ├── dynamic_test.go
│ ├── healthtracking_dynamic.go
│ ├── healthtracking_dynamic_test.go
│ ├── host.go
│ ├── host_test.go
│ ├── map.go
│ ├── map_test.go
│ ├── mocks
│ │ ├── DynamicOptions.go
│ │ ├── DynamicTopology.go
│ │ ├── HealthTrackingDynamicTopoloy.go
│ │ ├── Host.go
│ │ ├── HostShardSet.go
│ │ ├── Initializer.go
│ │ ├── Map.go
│ │ ├── MapProvider.go
│ │ ├── MapWatch.go
│ │ ├── StaticOptions.go
│ │ └── Topology.go
│ ├── options.go
│ ├── options_test.go
│ ├── shard_owner.go
│ ├── static.go
│ ├── static_test.go
│ ├── testutil
│ │ └── topology.go
│ └── types.go
└── types.go
├── cmd
├── arescli
│ └── main.go
├── aresd
│ ├── cmd
│ │ ├── cmd.go
│ │ └── config_parser.go
│ └── main.go
├── broker
│ ├── cmd
│ │ └── cmd.go
│ └── main.go
├── controller
│ ├── cmd
│ │ └── cmd.go
│ └── main.go
└── subscriber
│ └── main.go
├── common
├── config.go
├── log.go
└── metrics.go
├── config
├── ares-broker.yaml
├── ares-controller.yaml
└── ares.yaml
├── controller
├── client
│ ├── controller.go
│ ├── controller_client_suite_test.go
│ ├── controller_test.go
│ └── mocks
│ │ └── ControllerClient.go
├── generated
│ └── proto
│ │ ├── entity.pb.go
│ │ ├── entity.proto
│ │ ├── enum.pb.go
│ │ └── enum.proto
├── handlers
│ ├── assignment.go
│ ├── assignment_test.go
│ ├── composite.go
│ ├── config.go
│ ├── config_test.go
│ ├── doc.go
│ ├── error.go
│ ├── health.go
│ ├── membership.go
│ ├── membership_test.go
│ ├── models.go
│ ├── module.go
│ ├── namespace.go
│ ├── namespace_test.go
│ ├── placement.go
│ ├── placement_test.go
│ ├── schema.go
│ ├── schema_test.go
│ └── ui.go
├── models
│ ├── Instance.go
│ ├── assignment.go
│ ├── defaults.go
│ ├── job.go
│ └── subscriber.go
├── mutators
│ ├── common
│ │ ├── errors.go
│ │ ├── job_config_validator.go
│ │ ├── job_config_validator_test.go
│ │ └── types.go
│ ├── etcd
│ │ ├── assignment_mutator.go
│ │ ├── assignment_mutator_test.go
│ │ ├── common.go
│ │ ├── enum_mutator.go
│ │ ├── enum_mutator_test.go
│ │ ├── job_mutator.go
│ │ ├── job_mutator_test.go
│ │ ├── membership_mutator.go
│ │ ├── membership_mutator_test.go
│ │ ├── namespace_mutator.go
│ │ ├── namespace_mutator_test.go
│ │ ├── placement_mutator.go
│ │ ├── placement_mutator_test.go
│ │ ├── schema_mutator.go
│ │ ├── schema_mutator_test.go
│ │ ├── subscriber_mutator.go
│ │ └── subscriber_mutator_test.go
│ ├── mocks
│ │ ├── EnumMutator.go
│ │ ├── EnumReader.go
│ │ ├── IngestionAssignmentMutator.go
│ │ ├── JobMutator.go
│ │ ├── MembershipMutator.go
│ │ ├── NamespaceMutator.go
│ │ ├── SubscriberMutator.go
│ │ └── TableSchemaMutator.go
│ └── module.go
├── tasks
│ ├── Module.go
│ ├── common
│ │ ├── errors.go
│ │ └── types.go
│ └── etcd
│ │ ├── ingestion_assignment.go
│ │ ├── ingestion_assignment_test.go
│ │ ├── leader_elector.go
│ │ └── leader_elector_test.go
└── ui
│ ├── .gitignore
│ ├── README.md
│ ├── package-lock.json
│ ├── package.json
│ ├── public
│ ├── favicon.ico
│ ├── index.html
│ └── manifest.json
│ ├── src
│ ├── App.css
│ ├── App.js
│ ├── App.test.js
│ ├── index.css
│ ├── index.js
│ ├── logo.svg
│ └── serviceWorker.js
│ └── swagger
│ ├── index.html
│ └── swagger.json
├── datanode
├── ares_suite_test.go
├── bootstrap
│ ├── ares_suite_test.go
│ ├── bootstrap_details.go
│ ├── bootstrap_details_test.go
│ ├── bootstrap_server.go
│ ├── bootstrap_server_bm_test.go
│ ├── bootstrap_server_test.go
│ ├── mocks
│ │ └── Bootstrapable.go
│ ├── options.go
│ └── types.go
├── bootstrap_manager.go
├── bootstrap_manager_test.go
├── client
│ ├── datanode_client_suite_test.go
│ ├── mocks
│ │ ├── DataNodeQueryClient.go
│ │ ├── Peer.go
│ │ └── PeerSource.go
│ ├── query_client.go
│ ├── query_client_test.go
│ └── types.go
├── datanode.go
├── datanode_test.go
├── generated
│ └── proto
│ │ ├── generate.go
│ │ └── rpc
│ │ ├── mocks
│ │ ├── PeerDataNodeClient.go
│ │ ├── PeerDataNode_BenchmarkFileTransferClient.go
│ │ ├── PeerDataNode_FetchVectorPartyRawDataClient.go
│ │ └── PeerDataNode_KeepAliveClient.go
│ │ ├── peer_streaming.pb.go
│ │ └── peer_streaming.proto
├── options.go
├── peer_source.go
├── peer_source_test.go
└── types.go
├── diskstore
├── ares_suite_test.go
├── diskstore.go
├── diskstore_util.go
├── diskstore_util_test.go
├── local_diskstore.go
├── local_diskstore_test.go
└── mocks
│ └── DiskStore.go
├── docker
├── Dockerfile
└── README.md
├── docs
└── logo.png
├── examples
├── 1k_trips
│ ├── data
│ │ ├── cities.csv
│ │ └── trips.csv
│ ├── queries
│ │ ├── total_fare.aql
│ │ ├── total_trips.aql
│ │ └── total_trips.sql
│ └── schema
│ │ ├── cities.json
│ │ └── trips.json
├── README.md
├── examples
├── examples.go
└── utils
│ └── example_utils.go
├── go.mod
├── go.sum
├── img
└── aresdb-logo.png
├── integration
├── ares_suite_test.go
├── config
│ └── ares.yaml
├── integration_test.go
└── test-data
│ ├── data
│ └── arraytest.csv
│ ├── queries
│ ├── array_query_contains.aql
│ ├── array_query_elementat.aql
│ └── array_query_length.aql
│ └── schema
│ └── arraytest.json
├── memstore
├── archive_store.go
├── archive_store_test.go
├── archive_vector_party.go
├── archiving.go
├── archiving_test.go
├── ares_suite_test.go
├── backfill.go
├── backfill_manager.go
├── backfill_manager_test.go
├── backfill_test.go
├── batchstats.go
├── batchstats_test.go
├── bootstrap.go
├── bootstrap_test.go
├── common
│ ├── ares_suite_test.go
│ ├── batch.go
│ ├── common.go
│ ├── data_type.go
│ ├── data_type_test.go
│ ├── data_value.go
│ ├── data_value_test.go
│ ├── host_memory_manager.go
│ ├── job.go
│ ├── mocks
│ │ ├── BootStrapToken.go
│ │ ├── EnumUpdater.go
│ │ ├── HostMemoryManager.go
│ │ ├── ListVectorParty.go
│ │ ├── LiveVectorParty.go
│ │ ├── PrimaryKey.go
│ │ ├── TableSchemaReader.go
│ │ └── VectorParty.go
│ ├── pinnable.go
│ ├── primary_key.go
│ ├── primary_key_test.go
│ ├── schema.go
│ ├── types.go
│ ├── upsert_batch.go
│ ├── upsert_batch_builder.go
│ ├── upsert_batch_builder_test.go
│ ├── upsert_batch_header.go
│ ├── upsert_batch_header_test.go
│ ├── upsert_batch_test.go
│ ├── vector_party.go
│ └── vector_party_serializer.go
├── cuckoo_index.go
├── cuckoo_index_test.go
├── host_memory_manager.go
├── host_memory_manager_test.go
├── ingestion.go
├── ingestion_test.go
├── job_manager.go
├── job_manager_test.go
├── job_status.go
├── json_marshaller_test.go
├── list
│ ├── archive_vector_party.go
│ ├── archive_vector_party_test.go
│ ├── ares_suite_test.go
│ ├── live_vector_party.go
│ ├── live_vector_party_test.go
│ ├── memory_pool.go
│ ├── memory_pool_test.go
│ ├── slab.go
│ ├── slab_test.go
│ ├── test_factory.go
│ ├── test_factory_test.go
│ └── vector_party.go
├── live_store.go
├── live_store_test.go
├── live_vector_party.go
├── live_vector_party_test.go
├── memstore.go
├── memstore_test.go
├── merge.go
├── merge_arrayvp_test.go
├── merge_test.go
├── mocks
│ ├── BatchReader.go
│ ├── Job.go
│ ├── MemStore.go
│ ├── RedoLogBrowser.go
│ ├── Scheduler.go
│ └── sortedColumnIterator.go
├── options.go
├── purge.go
├── purge_manager.go
├── purge_test.go
├── recovery.go
├── recovery_test.go
├── redo_log_browser.go
├── redo_log_browser_test.go
├── scheduler.go
├── scheduler_test.go
├── schema.go
├── schema_test.go
├── snapshot.go
├── snapshot_manager.go
├── snapshot_manager_test.go
├── snapshot_test.go
├── table_shard.go
├── table_shard_test.go
├── test_factory.go
├── test_factory_test.go
├── tests
│ └── test_factory_base.go
├── vector_party.go
├── vector_party_serializer_test.go
├── vector_party_test.go
└── vectors
│ ├── vector.go
│ └── vector_test.go
├── metastore
├── ares_suite_test.go
├── common
│ ├── ares_suite_test.go
│ ├── data_type.go
│ ├── errors.go
│ ├── model.go
│ ├── model_test.go
│ └── types.go
├── disk_metastore.go
├── disk_metastore_test.go
├── mocks
│ ├── TableSchemaMutator.go
│ ├── TableSchemaReader.go
│ ├── TableSchemaValidator.go
│ ├── TableSchemaWatchable.go
│ └── metastore.go
├── schema_fetch.go
├── schema_fetch_test.go
├── validator.go
└── validator_test.go
├── query
├── algorithm.cu
├── algorithm.hpp
├── algorithm_unittest.cu
├── aql_batchexecutor.go
├── aql_compiler.go
├── aql_compiler_test.go
├── aql_context.go
├── aql_context_test.go
├── aql_nonaggr_batchexecutor.go
├── aql_postprocessor.go
├── aql_postprocessor_test.go
├── aql_processor.go
├── aql_processor_array_test.go
├── aql_processor_test.go
├── ares_suite_test.go
├── binder.hpp
├── common
│ ├── aql.go
│ ├── aql_query_result.go
│ ├── aql_query_result_test.go
│ ├── ares_suite_test.go
│ ├── constant.go
│ ├── dim_util.go
│ ├── dim_util_test.go
│ ├── dimval.go
│ ├── dimval_test.go
│ ├── hll.go
│ ├── hll_test.go
│ ├── time_bucketizer.go
│ ├── time_filter.go
│ └── time_filter_test.go
├── concurrent_unordered_map.hpp
├── context
│ ├── query_context_helper.go
│ └── query_context_options.go
├── device_allocator.go
├── device_allocator_test.go
├── device_manager.go
├── device_manager_test.go
├── dimension_transform.cu
├── expr
│ ├── ast.go
│ ├── ast_test.go
│ ├── parser.go
│ ├── parser_test.go
│ ├── scanner.go
│ ├── scanner_test.go
│ ├── token.go
│ └── token_test.go
├── expression_vm.h
├── filter.cu
├── functor.cu
├── functor.hpp
├── functor_unittest.cu
├── geo_intersects.cu
├── hash_lookup.cu
├── hash_reduction.cu
├── hll.cu
├── hll.go
├── hll_test.go
├── iterator.cu
├── iterator.hpp
├── iterator_unittest.cu
├── measure_transform.cu
├── memory.cu
├── memory.hpp
├── scratch_space_transform.cu
├── sort_reduce.cu
├── sql
│ ├── .DS_Store
│ ├── SqlBase.g4
│ ├── antlrgen
│ │ ├── SqlBase.interp
│ │ ├── SqlBase.tokens
│ │ ├── SqlBaseLexer.interp
│ │ ├── SqlBaseLexer.tokens
│ │ ├── sqlbase_base_visitor.go
│ │ ├── sqlbase_lexer.go
│ │ ├── sqlbase_parser.go
│ │ └── sqlbase_visitor.go
│ ├── ares_suite_test.go
│ ├── readme
│ ├── sql_parser.go
│ ├── sql_parser_test.go
│ ├── tree
│ │ ├── AliasedRelation.go
│ │ ├── AllColumns.go
│ │ ├── AstVisitor.go
│ │ ├── Expression.go
│ │ ├── GroupBy.go
│ │ ├── GroupingElement.go
│ │ ├── Identifier.go
│ │ ├── Join.go
│ │ ├── JoinCriteria.go
│ │ ├── JoinOn.go
│ │ ├── JoinUsing.go
│ │ ├── LogicalBinaryExpression.go
│ │ ├── NaturalJoin.go
│ │ ├── Node.go
│ │ ├── OrderBy.go
│ │ ├── QualifiedName.go
│ │ ├── Query.go
│ │ ├── QueryBody.go
│ │ ├── QuerySpecification.go
│ │ ├── Relation.go
│ │ ├── Select.go
│ │ ├── SelectItem.go
│ │ ├── SimpleGroupBy.go
│ │ ├── SingleColumn.go
│ │ ├── SortItem.go
│ │ ├── Statement.go
│ │ ├── Table.go
│ │ ├── TableSubquery.go
│ │ ├── With.go
│ │ └── WithQuery.go
│ └── util
│ │ ├── caseInsensitiveStream.go
│ │ ├── udfRegister.go
│ │ └── util.go
├── stats.go
├── stats_test.go
├── time_bucketizer.go
├── time_bucketizer_test.go
├── time_series_aggregate.go
├── time_series_aggregate.h
├── transform.cu
├── transform.hpp
├── unittest_utils.hpp
├── utils.cu
└── utils.hpp
├── redolog
├── composite_redolog_manager.go
├── composite_redolog_manager_test.go
├── file_redolog_manager.go
├── file_redolog_manager_test.go
├── kafka_redolog_manager.go
├── kafka_redolog_manager_test.go
├── mocks
│ └── RedologManager.go
├── redolog_manager.go
├── redolog_manager_master.go
├── redolog_manager_master_test.go
└── redolog_suite_test.go
├── scripts
├── clang-lint.sh
└── golang-lint.sh
├── subscriber
├── common
│ ├── consumer
│ │ ├── consumer.go
│ │ ├── consumer_suite_test.go
│ │ └── kafka
│ │ │ ├── kafka.go
│ │ │ ├── kafka_suite_test.go
│ │ │ └── kafka_test.go
│ ├── job
│ │ ├── controller.go
│ │ ├── controller_test.go
│ │ ├── driver.go
│ │ ├── driver_test.go
│ │ ├── failure_handler.go
│ │ ├── job_suite_test.go
│ │ ├── processor.go
│ │ ├── processor_test.go
│ │ ├── retry_failure_handler.go
│ │ ├── streaming_processor.go
│ │ └── streaming_processor_test.go
│ ├── message
│ │ ├── json_decoder.go
│ │ ├── json_decoder_test.go
│ │ ├── message_decoder.go
│ │ ├── message_decoder_test.go
│ │ ├── message_parser.go
│ │ ├── message_parser_test.go
│ │ ├── message_suite_test.go
│ │ └── test_config
│ │ │ └── hp_config
│ │ │ ├── schemas
│ │ │ └── hp-api-test_topic
│ │ │ │ └── test_topic.1.avsc
│ │ │ └── topic_registry.csv
│ ├── rules
│ │ ├── job_config.go
│ │ ├── job_config_test.go
│ │ ├── rules_suite_test.go
│ │ ├── transformation.go
│ │ └── transformation_test.go
│ ├── sink
│ │ ├── ares_database.go
│ │ ├── ares_database_test.go
│ │ ├── kafka.go
│ │ ├── kafka_test.go
│ │ ├── sink.go
│ │ ├── sink_suite_test.go
│ │ └── sink_test.go
│ └── tools
│ │ ├── batcher.go
│ │ ├── batcher_test.go
│ │ ├── tools_suite_test.go
│ │ ├── util.go
│ │ └── util_test.go
└── config
│ ├── config_suite_test.go
│ ├── service_config.go
│ ├── service_config_test.go
│ ├── test-ares-cluster-empty.yaml
│ ├── test-ares-ns-empty.yaml
│ ├── test-controller-disable.yaml
│ ├── test-job-empty.yaml
│ ├── test-kafka.yaml
│ ├── test.yaml
│ └── test
│ └── jobs
│ └── job1-local.json
├── testing
├── data
│ ├── batches
│ │ ├── archiveBatch
│ │ ├── archiving
│ │ │ ├── archiveBatch0
│ │ │ ├── archiveBatch1
│ │ │ ├── batch-101
│ │ │ ├── batch-110
│ │ │ ├── batch-120
│ │ │ └── batch-99
│ │ ├── backfill
│ │ │ ├── backfillBase
│ │ │ ├── backfillNew
│ │ │ ├── backfillTempLiveStore
│ │ │ ├── buildIndex
│ │ │ └── cloneVPForWrite
│ │ ├── live
│ │ │ ├── batch-120
│ │ │ └── batch-130
│ │ ├── merge-nil-base
│ │ ├── merge-with-deleted-columns
│ │ │ └── mergedBatch
│ │ ├── merge-with-deleted-rows
│ │ │ ├── baseBatch
│ │ │ ├── mergedBatch
│ │ │ ├── mergedBatch2
│ │ │ └── patchBatch
│ │ ├── mergedBatch
│ │ ├── no-sort-columns
│ │ │ ├── baseBatch
│ │ │ ├── mergedBatch
│ │ │ └── patchBatch
│ │ ├── patchBatch0
│ │ └── patchBatch1
│ ├── bootstrap
│ │ ├── data
│ │ │ ├── dimtable1_0
│ │ │ │ └── snapshots
│ │ │ │ │ └── 1560032167_605
│ │ │ │ │ └── -2147483648
│ │ │ │ │ ├── 0.data
│ │ │ │ │ ├── 1.data
│ │ │ │ │ ├── 2.data
│ │ │ │ │ ├── 3.data
│ │ │ │ │ └── 4.data
│ │ │ └── facttable1_0
│ │ │ │ └── archiving_batches
│ │ │ │ ├── 2019-06-01_1559436638
│ │ │ │ ├── 0.data
│ │ │ │ ├── 1.data
│ │ │ │ ├── 2.data
│ │ │ │ ├── 3.data
│ │ │ │ ├── 4.data
│ │ │ │ └── 5.data
│ │ │ │ ├── 2019-06-02_1559523900
│ │ │ │ ├── 0.data
│ │ │ │ ├── 1.data
│ │ │ │ ├── 2.data
│ │ │ │ ├── 3.data
│ │ │ │ ├── 4.data
│ │ │ │ └── 5.data
│ │ │ │ ├── 2019-06-03_1559611162
│ │ │ │ ├── 0.data
│ │ │ │ ├── 1.data
│ │ │ │ ├── 2.data
│ │ │ │ ├── 3.data
│ │ │ │ ├── 4.data
│ │ │ │ └── 5.data
│ │ │ │ ├── 2019-06-04_1559698827
│ │ │ │ ├── 0.data
│ │ │ │ ├── 1.data
│ │ │ │ ├── 2.data
│ │ │ │ ├── 3.data
│ │ │ │ ├── 4.data
│ │ │ │ └── 5.data
│ │ │ │ ├── 2019-06-05_1559779414
│ │ │ │ ├── 0.data
│ │ │ │ ├── 1.data
│ │ │ │ ├── 2.data
│ │ │ │ ├── 3.data
│ │ │ │ ├── 4.data
│ │ │ │ └── 5.data
│ │ │ │ ├── 2019-06-06_1559866672
│ │ │ │ ├── 0.data
│ │ │ │ ├── 1.data
│ │ │ │ ├── 2.data
│ │ │ │ ├── 3.data
│ │ │ │ ├── 4.data
│ │ │ │ └── 5.data
│ │ │ │ ├── 2019-06-07_1559954536
│ │ │ │ ├── 0.data
│ │ │ │ ├── 1.data
│ │ │ │ ├── 2.data
│ │ │ │ ├── 3.data
│ │ │ │ ├── 4.data
│ │ │ │ └── 5.data
│ │ │ │ ├── 2019-06-08_1560042370
│ │ │ │ ├── 0.data
│ │ │ │ ├── 1.data
│ │ │ │ ├── 2.data
│ │ │ │ ├── 3.data
│ │ │ │ ├── 4.data
│ │ │ │ └── 5.data
│ │ │ │ └── 2019-06-09_1560049865
│ │ │ │ ├── 0.data
│ │ │ │ ├── 1.data
│ │ │ │ ├── 2.data
│ │ │ │ ├── 3.data
│ │ │ │ ├── 4.data
│ │ │ │ └── 5.data
│ │ └── metastore
│ │ │ ├── dimtable1
│ │ │ ├── schema
│ │ │ └── shards
│ │ │ │ └── 0
│ │ │ │ ├── checkpoint-offset
│ │ │ │ ├── commit-offset
│ │ │ │ └── snapshot
│ │ │ └── facttable1
│ │ │ ├── schema
│ │ │ └── shards
│ │ │ └── 0
│ │ │ ├── batches
│ │ │ ├── 18048
│ │ │ ├── 18049
│ │ │ ├── 18050
│ │ │ ├── 18051
│ │ │ ├── 18052
│ │ │ ├── 18053
│ │ │ ├── 18054
│ │ │ ├── 18055
│ │ │ └── 18056
│ │ │ ├── checkpoint-offset
│ │ │ ├── commit-offset
│ │ │ ├── redolog-offset
│ │ │ └── version
│ ├── data-sender
│ │ └── sample.csv
│ ├── integration
│ │ └── sample-ares-root
│ │ │ ├── data
│ │ │ └── abc_0
│ │ │ │ └── redologs
│ │ │ │ └── 1501869573.redolog
│ │ │ └── metastore
│ │ │ └── abc
│ │ │ ├── enums
│ │ │ └── c2
│ │ │ └── schema
│ ├── query
│ │ ├── hll
│ │ ├── hll_empty_results
│ │ └── hll_query_results
│ ├── upsert-batches
│ │ ├── backfill
│ │ │ ├── upsertBatch0
│ │ │ ├── upsertBatch1
│ │ │ ├── upsertBatch2
│ │ │ └── upsertBatch3
│ │ └── testReadUpsertBatch
│ ├── vectors
│ │ ├── invalid_bool_value
│ │ └── v0
│ └── vps
│ │ ├── archiving
│ │ ├── sortedVP0
│ │ ├── sortedVP00
│ │ ├── sortedVP1
│ │ ├── sortedVP2
│ │ ├── sortedVP3_array
│ │ ├── sortedVP4_array
│ │ ├── vp00
│ │ ├── vp01
│ │ ├── vp02
│ │ ├── vp03_array
│ │ ├── vp04_array
│ │ ├── vp10
│ │ ├── vp11
│ │ ├── vp12
│ │ ├── vp13
│ │ ├── vp13_array
│ │ ├── vp14
│ │ └── vp14_array
│ │ ├── backfill
│ │ ├── backfillBase0
│ │ ├── backfillBase1
│ │ ├── backfillBase2
│ │ ├── backfillBase3
│ │ ├── backfillBase4
│ │ ├── backfillBase5
│ │ ├── backfillBase6
│ │ ├── backfillNew0
│ │ ├── backfillNew1
│ │ ├── backfillNew2
│ │ ├── backfillNew3
│ │ ├── backfillNew4
│ │ ├── backfillNew5
│ │ ├── backfillNew6
│ │ ├── backfillTempLiveStore0
│ │ ├── backfillTempLiveStore1
│ │ ├── backfillTempLiveStore2
│ │ ├── backfillTempLiveStore4
│ │ ├── backfillTempLiveStore5
│ │ ├── backfillTempLiveStore6
│ │ ├── buildIndex0
│ │ ├── buildIndex1
│ │ ├── buildIndex2
│ │ ├── buildIndex3
│ │ ├── cloneVPForWrite0
│ │ ├── cloneVPForWrite1
│ │ ├── cloneVPForWrite2
│ │ └── cloneVPForWrite3
│ │ ├── host-memory-manager
│ │ └── c1
│ │ ├── invalid_value_length
│ │ ├── list
│ │ ├── length_not_match
│ │ ├── live_vp_bool
│ │ ├── live_vp_uint32
│ │ ├── live_vp_uuid
│ │ └── unknown_data_type
│ │ ├── merge-nil-base
│ │ ├── mergedVP0
│ │ ├── mergedVP1
│ │ ├── mergedVP2
│ │ ├── mergedVP3
│ │ ├── mergedVP4
│ │ └── mergedVP5
│ │ ├── merge-with-deleted-columns
│ │ └── mergedVP5
│ │ ├── merge-with-deleted-rows
│ │ ├── mergedVP0
│ │ ├── mergedVP1
│ │ ├── mergedVP10
│ │ ├── mergedVP11
│ │ ├── mergedVP2
│ │ ├── mergedVP3
│ │ ├── mergedVP4
│ │ └── mergedVP5
│ │ ├── mergedVP0
│ │ ├── mergedVP1
│ │ ├── mergedVP2
│ │ ├── mergedVP3
│ │ ├── mergedVP4
│ │ ├── mergedVP5
│ │ ├── no-sort-columns
│ │ ├── baseVP0
│ │ ├── mergedVP0
│ │ ├── mergedVP1
│ │ ├── patchVP0
│ │ └── patchVP1
│ │ ├── patchVP00
│ │ ├── patchVP01
│ │ ├── patchVP02
│ │ ├── patchVP03
│ │ ├── patchVP04
│ │ ├── patchVP05
│ │ ├── patchVP10
│ │ ├── patchVP11
│ │ ├── patchVP12
│ │ ├── patchVP13
│ │ ├── patchVP14
│ │ ├── patchVP15
│ │ ├── serializer
│ │ ├── mode0_int8
│ │ ├── mode1_bool
│ │ ├── mode2_int8
│ │ └── mode3_int8
│ │ ├── sortedVP0
│ │ ├── sortedVP1
│ │ ├── sortedVP2
│ │ ├── sortedVP3
│ │ ├── sortedVP4
│ │ ├── sortedVP5
│ │ ├── sortedVP6
│ │ ├── sortedVP7
│ │ └── sortedVP8
└── utils.go
└── utils
├── ares_suite_test.go
├── config.go
├── consistenthasing
├── consistenthashing.go
└── consistenthasing_test.go
├── constants.go
├── di.go
├── env.go
├── env_test.go
├── error.go
├── error_test.go
├── file_system.go
├── hash.go
├── hash_test.go
├── hll.go
├── hll_test.go
├── http.go
├── http_test.go
├── io.go
├── kafka.go
├── key.go
├── memory.go
├── memory_test.go
├── metrics.go
├── metrics_test.go
├── mocks
├── ReadCloser.go
├── WriteSyncCloser.go
├── file_info.go
└── file_system.go
├── options.go
├── reflection.go
├── reflection_test.go
├── resource_mapper.go
├── serialization.go
├── serialization_test.go
├── slices.go
├── slices_test.go
├── sort_utils.go
├── stream_serialization.go
├── stream_serialization_test.go
├── sync.go
├── table_writer.go
├── table_writer_test.go
├── test_utils.go
├── time.go
├── time_test.go
├── uuid.go
└── uuid_test.go
/.github/ISSUE_TEMPLATE/bug_report.md:
--------------------------------------------------------------------------------
1 | ---
2 | name: Bug report
3 | about: Create a report to help us improve
4 | title: ''
5 | labels: ''
6 | assignees: ''
7 |
8 | ---
9 |
10 | **Describe the bug**
11 | A clear and concise description of what the bug is.
12 |
13 | **To Reproduce**
14 | Steps to reproduce the behavior:
15 | 1. Go to '...'
16 | 2. Click on '....'
17 | 3. Scroll down to '....'
18 | 4. See error
19 |
20 | **Expected behavior**
21 | A clear and concise description of what you expected to happen.
22 |
23 | **Screenshots**
24 | If applicable, add screenshots to help explain your problem.
25 |
26 | **Desktop (please complete the following information):**
27 | - OS: [e.g. iOS]
28 | - Browser [e.g. chrome, safari]
29 | - Version [e.g. 22]
30 |
31 | **Smartphone (please complete the following information):**
32 | - Device: [e.g. iPhone6]
33 | - OS: [e.g. iOS8.1]
34 | - Browser [e.g. stock browser, safari]
35 | - Version [e.g. 22]
36 |
37 | **Additional context**
38 | Add any other context about the problem here.
39 |
--------------------------------------------------------------------------------
/.github/ISSUE_TEMPLATE/feature_request.md:
--------------------------------------------------------------------------------
1 | ---
2 | name: Feature request
3 | about: Suggest an idea for this project
4 | title: ''
5 | labels: ''
6 | assignees: ''
7 |
8 | ---
9 |
10 | **Is your feature request related to a problem? Please describe.**
11 | A clear and concise description of what the problem is. Ex. I'm always frustrated when [...]
12 |
13 | **Describe the solution you'd like**
14 | A clear and concise description of what you want to happen.
15 |
16 | **Describe alternatives you've considered**
17 | A clear and concise description of any alternative solutions or features you've considered.
18 |
19 | **Additional context**
20 | Add any other context or screenshots about the feature request here.
21 |
--------------------------------------------------------------------------------
/.gitignore:
--------------------------------------------------------------------------------
1 | vendor
2 | coverage.*
3 | junit.xml
4 | .idea
5 | .gopath
6 | .*.swp
7 | *.coverprofile
8 | *.a
9 | *.o
10 | *.so
11 | gtest
12 | build
13 | goenv
14 | ares-root
15 | default.yaml
16 | bower_components
17 | node_modules
18 | CMakeCache.txt
19 | CMakeFiles/
20 | Makefile
21 | cmake-build-debug/
22 | bin/
23 | .bin/
24 | lib/
25 | .DS_Store
26 | thirdparty/
27 | CTestTestfile.cmake
28 | cmake_install.cmake
29 | googletest-prefix/
30 | include/
31 | rdkafka-prefix/
32 | rmm-prefix/
33 | testing/data/
34 |
--------------------------------------------------------------------------------
/.gitmodules:
--------------------------------------------------------------------------------
1 | [submodule "thirdparty/librdkafka"]
2 | path = thirdparty/librdkafka
3 | url = https://github.com/edenhill/librdkafka.git
4 |
5 | [submodule "thirdparty/rmm"]
6 | path = thirdparty/rmm
7 | url = https://github.com/rapidsai/rmm
8 |
9 | [submodule "thirdparty/googletest"]
10 | path = thirdparty/googletest
11 | url = https://github.com/google/googletest.git
12 | branch = release-1.8.1
13 |
14 | [submodule "thirdparty/cudf"]
15 | path = thirdparty/cudf
16 | url = https://github.com/rapidsai/cudf
17 |
--------------------------------------------------------------------------------
/.travis/cleanup.sh:
--------------------------------------------------------------------------------
1 | #!/usr/bin/env bash
2 | rm -rf ./build
--------------------------------------------------------------------------------
/.travis/install_clang_tools.sh:
--------------------------------------------------------------------------------
1 | #!/usr/bin/env bash
2 | set -ex
3 | # install google test
4 | output_dir=./build
5 | cmake_version=3.12.0
6 | # install cppcheck
7 | sudo apt-get install cppcheck
8 |
9 | # install cpplint
10 | sudo apt-get install python-pip
11 | sudo pip install cpplint
12 |
13 | # install cmake
14 | pushd .
15 | cd /tmp
16 | wget https://github.com/Kitware/CMake/releases/download/v${cmake_version}/cmake-${cmake_version}-Linux-x86_64.tar.gz
17 | tar xzf cmake-${cmake_version}-Linux-x86_64.tar.gz
18 | popd
19 | mv /tmp/cmake-${cmake_version}-Linux-x86_64 ./build/cmake
20 |
--------------------------------------------------------------------------------
/.travis/install_cuda.sh:
--------------------------------------------------------------------------------
1 | #!/usr/bin/env bash
2 | set -ex
3 | if [ "$(ls -A $CUDA_LIB_LOCAL)" ]; then
4 | echo "Fetched cuda deps from cache"
5 | else
6 | wget http://developer.download.nvidia.com/compute/cuda/repos/ubuntu1404/x86_64/${CUDA_PKG}
7 | sudo apt-key adv --fetch-keys http://developer.download.nvidia.com/compute/cuda/repos/ubuntu1404/x86_64/7fa2af80.pub
8 | sudo dpkg -i ${CUDA_PKG}
9 | rm ${CUDA_PKG}
10 | sudo apt-get -y update
11 | sudo apt-get install -y --no-install-recommends cuda-core-${CUDA_VERSION} cuda-cudart-dev-${CUDA_VERSION} cuda-cublas-dev-${CUDA_VERSION} cuda-curand-dev-${CUDA_VERSION}
12 | mkdir -p ${CUDA_LIB_LOCAL}
13 | sudo cp -rL /usr/local/cuda-${CUDA_VERSION}/bin /usr/local/cuda-${CUDA_VERSION}/lib64 /usr/local/cuda-${CUDA_VERSION}/include /usr/local/cuda-${CUDA_VERSION}/nvvm ${CUDA_LIB_LOCAL}
14 | fi
15 |
--------------------------------------------------------------------------------
/.travis/install_golang_tools.sh:
--------------------------------------------------------------------------------
1 | #!/usr/bin/env bash
2 | set -ex
3 | go get github.com/axw/gocov/gocov@v1.0.0
4 | go get github.com/AlekSi/gocov-xml
5 | go get -u golang.org/x/lint/golint
6 | go get -u github.com/onsi/ginkgo/ginkgo@v1.6.0
7 | go get github.com/modocache/gover
8 | go get github.com/mattn/goveralls@v0.0.5
9 |
--------------------------------------------------------------------------------
/.travis/pre_install.sh:
--------------------------------------------------------------------------------
1 | #!/usr/bin/env bash
2 | apt-get clean
3 | apt-get update -q
4 | apt-get install -y --no-install-recommends apt-utils wget gnupg software-properties-common
5 | apt-get install -y apt-transport-https ca-certificates
6 | wget -qO - https://packages.confluent.io/deb/5.1/archive.key | apt-key add -
7 | add-apt-repository "deb [arch=amd64] https://packages.confluent.io/deb/5.1 stable main"
8 | apt-get update
9 | apt-get install -y librdkafka-dev
10 |
11 | # a hack to overwrite /usr/include/librdkafka/rdkafka.h
12 | # since "RD_KAFKA_RESP_ERR__FATAL" is not present in librdkafka-dev v0.11.6
13 | # even the confluent kafka claims it requires version >= v0.11.5
14 | sudo cp -rf thirdparty/librdkafka/src/rdkafka.h /usr/include/librdkafka/rdkafka.h
15 |
16 |
--------------------------------------------------------------------------------
/CODEOWNERS:
--------------------------------------------------------------------------------
1 | * @lucafuji @jshencode @voyager-dw @shz117 @lxning
2 |
--------------------------------------------------------------------------------
/CONTRIBUTING.md:
--------------------------------------------------------------------------------
1 | Contributing
2 | ------------
3 |
4 | We'd love your help in making AresDB great. If you find a bug or need a new feature, open an issue and we will respond as fast as we can. If you want to implement new feature(s) and/or fix bug(s) yourself, open a pull request with the appropriate unit tests and we will merge it after review.
5 | Before we can merge your changes, you must agree to the [Uber Contributor License Agreement](https://cla-assistant.io/uber/aresdb).
6 |
--------------------------------------------------------------------------------
/api/ares_suite_test.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package api
16 |
17 | import (
18 | . "github.com/onsi/ginkgo"
19 | . "github.com/onsi/gomega"
20 |
21 | "testing"
22 |
23 | "github.com/onsi/ginkgo/reporters"
24 | )
25 |
26 | func TestAPI(t *testing.T) {
27 | RegisterFailHandler(Fail)
28 | junitReporter := reporters.NewJUnitReporter("junit.xml")
29 | RunSpecsWithDefaultAndCustomReporters(t, "Ares API Suite", []Reporter{junitReporter})
30 | }
31 |
--------------------------------------------------------------------------------
/api/common/ares_suite_test.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package common
16 |
17 | import (
18 | . "github.com/onsi/ginkgo"
19 | . "github.com/onsi/gomega"
20 |
21 | "testing"
22 |
23 | "github.com/onsi/ginkgo/reporters"
24 | )
25 |
26 | func TestAPI(t *testing.T) {
27 | RegisterFailHandler(Fail)
28 | junitReporter := reporters.NewJUnitReporter("junit.xml")
29 | RunSpecsWithDefaultAndCustomReporters(t, "Ares API Common Suite", []Reporter{junitReporter})
30 | }
31 |
--------------------------------------------------------------------------------
/api/common/query_response.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package common
16 |
17 | import (
18 | queryCom "github.com/uber/aresdb/query/common"
19 | )
20 |
21 | // AQLResponse represents queryAQL response.
22 | // swagger:response aqlResponse
23 | type AQLResponse struct {
24 | //in: body
25 | Body queryCom.AQLResponse
26 | }
27 |
--------------------------------------------------------------------------------
/api/common/response.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package common
16 |
17 | // NoContentResponse represents Response with no content.
18 | // swagger:response noContentResponse
19 | type NoContentResponse struct{}
20 |
21 | // StringArrayResponse represents string array response.
22 | // swagger:response stringArrayResponse
23 | type StringArrayResponse struct {
24 | //in: body
25 | Body []string
26 | }
27 |
28 | // NewStringArrayResponse creates a StringArrayResponse.
29 | func NewStringArrayResponse() StringArrayResponse {
30 | return StringArrayResponse{
31 | Body: []string{},
32 | }
33 | }
34 |
--------------------------------------------------------------------------------
/api/data_request.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package api
16 |
17 | // PostDataRequest represents post data request.
18 | // swagger:parameters postData
19 | type PostDataRequest struct {
20 | // in: path
21 | TableName string `path:"table" json:"table"`
22 | // in: path
23 | Shard int `path:"shard" json:"shard"`
24 | // in: body
25 | Body []byte `body:""`
26 | }
27 |
--------------------------------------------------------------------------------
/api/doc.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | // Package api provide the RESTful APIs to interact with AresDB.
16 | //
17 | // Schemes: http
18 | // BasePath: /
19 | // Version: 0.0.1
20 | //
21 | // Consumes:
22 | // - application/json
23 | //
24 | // Produces:
25 | // - application/json
26 | //
27 | // swagger:meta
28 | package api
29 |
--------------------------------------------------------------------------------
/api/ui/debug/css/index.css:
--------------------------------------------------------------------------------
1 | /* Copyright (c) 2017-2018 Uber Technologies, Inc. */
2 | /* */
3 | /* Licensed under the Apache License, Version 2.0 (the "License"); */
4 | /* you may not use this file except in compliance with the License. */
5 | /* You may obtain a copy of the License at */
6 | /* */
7 | /* http://www.apache.org/licenses/LICENSE-2.0 */
8 | /* */
9 | /* Unless required by applicable law or agreed to in writing, software */
10 | /* distributed under the License is distributed on an "AS IS" BASIS, */
11 | /* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. */
12 | /* See the License for the specific language governing permissions and */
13 | /* limitations under the License. */
14 |
15 | body, html {width: 100%; height: 100%; margin: 0; padding: 0}
16 | .fit-full-container {display: block; width: 100%; height: 100%; border: none;}
17 |
--------------------------------------------------------------------------------
/api/ui/npm.updated:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/api/ui/npm.updated
--------------------------------------------------------------------------------
/api/ui/package.json:
--------------------------------------------------------------------------------
1 | {
2 | "name": "debug",
3 | "version": "1.0.0",
4 | "description": "Ares Debug UI",
5 | "main": "index.js",
6 | "license": "MIT",
7 | "dependencies": {
8 | "bootstrap": "^4.1.3",
9 | "chart.js": "^2.7.1",
10 | "datatables.net": "^1.10.19",
11 | "datatables.net-dt": "^1.10.19",
12 | "google-palette": "^1.0.0",
13 | "jquery": "^3.2.1",
14 | "jquery-datetimepicker": "^2.5.14",
15 | "jquery-ui-dist": "^1.12.1",
16 | "select2": "^4.0.5",
17 | "jquery.json-viewer": "^1.1.0",
18 | "bootstrap-toggle": "^2.2.2",
19 | "swagger-ui-dist": "^3.19.0"
20 | }
21 | }
22 |
--------------------------------------------------------------------------------
/api/ui/swagger/index.html:
--------------------------------------------------------------------------------
1 |
2 |
3 |
4 |
5 | AresDB Swagger UI
6 |
7 |
8 |
9 |
10 |
11 |
12 |
13 |
14 |
15 |
16 |
31 |
32 |
33 |
--------------------------------------------------------------------------------
/broker/broker_suite_test.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package broker
16 |
17 | import (
18 | . "github.com/onsi/ginkgo"
19 | . "github.com/onsi/gomega"
20 |
21 | "testing"
22 |
23 | "github.com/onsi/ginkgo/reporters"
24 | )
25 |
26 | func TestDataNode(t *testing.T) {
27 | RegisterFailHandler(Fail)
28 | junitReporter := reporters.NewJUnitReporter("junit.xml")
29 | RunSpecsWithDefaultAndCustomReporters(t, "Ares Broker Suite", []Reporter{junitReporter})
30 | }
31 |
--------------------------------------------------------------------------------
/broker/common/constant.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package common
16 |
17 | import "github.com/uber/aresdb/query/expr"
18 |
19 | type AggType int
20 |
21 | const (
22 | Count AggType = iota
23 | Sum
24 | Max
25 | Min
26 | Avg
27 | Hll
28 | )
29 |
30 | var CallNameToAggType = map[string]AggType{
31 | expr.CountCallName: Count,
32 | expr.SumCallName: Sum,
33 | expr.AvgCallName: Avg,
34 | expr.MaxCallName: Max,
35 | expr.MinCallName: Min,
36 | expr.HllCallName: Hll,
37 | }
38 |
--------------------------------------------------------------------------------
/broker/common/mocks/QueryExecutor.go:
--------------------------------------------------------------------------------
1 | // Code generated by mockery v1.0.0
2 | package mocks
3 |
4 | import context "context"
5 | import http "net/http"
6 | import mock "github.com/stretchr/testify/mock"
7 |
8 | // QueryExecutor is an autogenerated mock type for the QueryExecutor type
9 | type QueryExecutor struct {
10 | mock.Mock
11 | }
12 |
13 | // Execute provides a mock function with given fields: ctx, namespace, sqlQuery, w
14 | func (_m *QueryExecutor) Execute(ctx context.Context, namespace string, sqlQuery string, w http.ResponseWriter) error {
15 | ret := _m.Called(ctx, namespace, sqlQuery, w)
16 |
17 | var r0 error
18 | if rf, ok := ret.Get(0).(func(context.Context, string, string, http.ResponseWriter) error); ok {
19 | r0 = rf(ctx, namespace, sqlQuery, w)
20 | } else {
21 | r0 = ret.Error(0)
22 | }
23 |
24 | return r0
25 | }
26 |
--------------------------------------------------------------------------------
/broker/common/mocks/StreamingPlanNode.go:
--------------------------------------------------------------------------------
1 | // Code generated by mockery v1.0.0
2 | package mocks
3 |
4 | import io "io"
5 | import mock "github.com/stretchr/testify/mock"
6 |
7 | // StreamingPlanNode is an autogenerated mock type for the StreamingPlanNode type
8 | type StreamingPlanNode struct {
9 | mock.Mock
10 | }
11 |
12 | // Execute provides a mock function with given fields: writer
13 | func (_m *StreamingPlanNode) Execute(writer io.Writer) error {
14 | ret := _m.Called(writer)
15 |
16 | var r0 error
17 | if rf, ok := ret.Get(0).(func(io.Writer) error); ok {
18 | r0 = rf(writer)
19 | } else {
20 | r0 = ret.Error(0)
21 | }
22 |
23 | return r0
24 | }
25 |
--------------------------------------------------------------------------------
/broker/config/config.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package config
16 |
17 | import (
18 | "github.com/uber/aresdb/common"
19 | )
20 |
21 | type BrokerConfig struct {
22 | // HTTP port for serving.
23 | Port int `yaml:"port"`
24 |
25 | HTTP common.HTTPConfig `yaml:"http"`
26 | Cluster common.ClusterConfig `yaml:"cluster"`
27 | }
28 |
--------------------------------------------------------------------------------
/broker/handler_test.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package broker
16 |
17 | import (
18 | "fmt"
19 | "github.com/onsi/ginkgo"
20 | . "github.com/onsi/gomega"
21 | )
22 |
23 | var _ = ginkgo.Describe("broker handler", func() {
24 | ginkgo.It("getRequestID should work", func() {
25 | h := NewQueryHandler(nil, "inst1")
26 | for i := 0; i < 10; i++ {
27 | Ω(h.getReqestID()).Should(Equal(fmt.Sprintf("inst1_%d", i+1)))
28 | }
29 | })
30 | })
31 |
--------------------------------------------------------------------------------
/broker/util/broker_suite_test.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package util
16 |
17 | import (
18 | . "github.com/onsi/ginkgo"
19 | . "github.com/onsi/gomega"
20 |
21 | "testing"
22 |
23 | "github.com/onsi/ginkgo/reporters"
24 | )
25 |
26 | func TestDataNode(t *testing.T) {
27 | RegisterFailHandler(Fail)
28 | junitReporter := reporters.NewJUnitReporter("junit.xml")
29 | RunSpecsWithDefaultAndCustomReporters(t, "Ares Broker Util Suite", []Reporter{junitReporter})
30 | }
31 |
--------------------------------------------------------------------------------
/cgoutils/ares_suite_test.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package cgoutils
16 |
17 | import (
18 | "github.com/onsi/ginkgo"
19 | "github.com/onsi/gomega"
20 |
21 | "testing"
22 |
23 | "github.com/onsi/ginkgo/reporters"
24 | )
25 |
26 | func TestCGoUtils(t *testing.T) {
27 | gomega.RegisterFailHandler(ginkgo.Fail)
28 | junitReporter := reporters.NewJUnitReporter("junit.xml")
29 | ginkgo.RunSpecsWithDefaultAndCustomReporters(t, "Ares CGo Utils Suite", []ginkgo.Reporter{junitReporter})
30 | }
31 |
--------------------------------------------------------------------------------
/cgoutils/utils.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package cgoutils
16 |
17 | // #include
18 | import "C"
19 | import (
20 | "github.com/uber/aresdb/utils"
21 | "unsafe"
22 | )
23 |
24 | // DoCGoCall is the function wrapper to call a cgo function, check whether there is any exception thrown by the function
25 | // and converted it to a golang error if any.
26 | func DoCGoCall(f func() (uintptr, unsafe.Pointer)) uintptr {
27 | res, pStrErr := f()
28 | if pStrErr != nil {
29 | errMsg := C.GoString((*C.char)(pStrErr))
30 | C.free(pStrErr)
31 | panic(utils.StackError(nil, errMsg))
32 | }
33 | return res
34 | }
35 |
--------------------------------------------------------------------------------
/cgoutils/utils.h:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | #ifndef CGOUTILS_UTILS_H_
16 | #define CGOUTILS_UTILS_H_
17 |
18 | // CGoCallResHandle wraps the return result of a CGo function call along with
19 | // the error string if any.
20 | typedef struct {
21 | void *res;
22 | const char *pStrErr;
23 | } CGoCallResHandle;
24 |
25 | #endif // CGOUTILS_UTILS_H_
26 |
--------------------------------------------------------------------------------
/client/ares_suite_test.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package client
16 |
17 | import (
18 | . "github.com/onsi/ginkgo"
19 | . "github.com/onsi/gomega"
20 |
21 | "testing"
22 |
23 | "github.com/onsi/ginkgo/reporters"
24 | )
25 |
26 | func TestClient(t *testing.T) {
27 | RegisterFailHandler(Fail)
28 | junitReporter := reporters.NewJUnitReporter("junit.xml")
29 | RunSpecsWithDefaultAndCustomReporters(t, "Ares Client Suite", []Reporter{junitReporter})
30 | }
31 |
--------------------------------------------------------------------------------
/cluster/ares_suite_test.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package cluster
16 |
17 | import (
18 | . "github.com/onsi/ginkgo"
19 | "github.com/onsi/ginkgo/reporters"
20 | . "github.com/onsi/gomega"
21 | "testing"
22 | )
23 |
24 | func TestDataNode(t *testing.T) {
25 | RegisterFailHandler(Fail)
26 | junitReporter := reporters.NewJUnitReporter("junit.xml")
27 | RunSpecsWithDefaultAndCustomReporters(t, "Ares Cluster Suite", []Reporter{junitReporter})
28 | }
29 |
--------------------------------------------------------------------------------
/cluster/shard/ares_suite_test.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package shard
16 |
17 | import (
18 | . "github.com/onsi/ginkgo"
19 | . "github.com/onsi/gomega"
20 |
21 | "testing"
22 |
23 | "github.com/onsi/ginkgo/reporters"
24 | )
25 |
26 | func TestDataNode(t *testing.T) {
27 | RegisterFailHandler(Fail)
28 | junitReporter := reporters.NewJUnitReporter("junit.xml")
29 | RunSpecsWithDefaultAndCustomReporters(t, "Ares Datanode Shard Suite", []Reporter{junitReporter})
30 | }
31 |
--------------------------------------------------------------------------------
/cluster/shard/mocks/ShardSet.go:
--------------------------------------------------------------------------------
1 | // Code generated by mockery v1.0.0
2 | package mocks
3 |
4 | import mock "github.com/stretchr/testify/mock"
5 | import shard "github.com/m3db/m3/src/cluster/shard"
6 |
7 | // ShardSet is an autogenerated mock type for the ShardSet type
8 | type ShardSet struct {
9 | mock.Mock
10 | }
11 |
12 | // All provides a mock function with given fields:
13 | func (_m *ShardSet) All() []shard.Shard {
14 | ret := _m.Called()
15 |
16 | var r0 []shard.Shard
17 | if rf, ok := ret.Get(0).(func() []shard.Shard); ok {
18 | r0 = rf()
19 | } else {
20 | if ret.Get(0) != nil {
21 | r0 = ret.Get(0).([]shard.Shard)
22 | }
23 | }
24 |
25 | return r0
26 | }
27 |
28 | // AllIDs provides a mock function with given fields:
29 | func (_m *ShardSet) AllIDs() []uint32 {
30 | ret := _m.Called()
31 |
32 | var r0 []uint32
33 | if rf, ok := ret.Get(0).(func() []uint32); ok {
34 | r0 = rf()
35 | } else {
36 | if ret.Get(0) != nil {
37 | r0 = ret.Get(0).([]uint32)
38 | }
39 | }
40 |
41 | return r0
42 | }
43 |
--------------------------------------------------------------------------------
/cluster/topology/ares_suite_test.go:
--------------------------------------------------------------------------------
1 | package topology
2 |
3 | import (
4 | . "github.com/onsi/ginkgo"
5 | . "github.com/onsi/gomega"
6 |
7 | "testing"
8 |
9 | "github.com/onsi/ginkgo/reporters"
10 | )
11 |
12 | func TestDataNode(t *testing.T) {
13 | RegisterFailHandler(Fail)
14 | junitReporter := reporters.NewJUnitReporter("junit.xml")
15 | RunSpecsWithDefaultAndCustomReporters(t, "Ares Datanode Topology Suite", []Reporter{junitReporter})
16 | }
17 |
--------------------------------------------------------------------------------
/cluster/topology/mocks/Host.go:
--------------------------------------------------------------------------------
1 | // Code generated by mockery v1.0.0
2 | package mocks
3 |
4 | import mock "github.com/stretchr/testify/mock"
5 |
6 | // Host is an autogenerated mock type for the Host type
7 | type Host struct {
8 | mock.Mock
9 | }
10 |
11 | // Address provides a mock function with given fields:
12 | func (_m *Host) Address() string {
13 | ret := _m.Called()
14 |
15 | var r0 string
16 | if rf, ok := ret.Get(0).(func() string); ok {
17 | r0 = rf()
18 | } else {
19 | r0 = ret.Get(0).(string)
20 | }
21 |
22 | return r0
23 | }
24 |
25 | // ID provides a mock function with given fields:
26 | func (_m *Host) ID() string {
27 | ret := _m.Called()
28 |
29 | var r0 string
30 | if rf, ok := ret.Get(0).(func() string); ok {
31 | r0 = rf()
32 | } else {
33 | r0 = ret.Get(0).(string)
34 | }
35 |
36 | return r0
37 | }
38 |
39 | // String provides a mock function with given fields:
40 | func (_m *Host) String() string {
41 | ret := _m.Called()
42 |
43 | var r0 string
44 | if rf, ok := ret.Get(0).(func() string); ok {
45 | r0 = rf()
46 | } else {
47 | r0 = ret.Get(0).(string)
48 | }
49 |
50 | return r0
51 | }
52 |
--------------------------------------------------------------------------------
/cluster/topology/mocks/HostShardSet.go:
--------------------------------------------------------------------------------
1 | // Code generated by mockery v1.0.0
2 | package mocks
3 |
4 | import mock "github.com/stretchr/testify/mock"
5 | import shard "github.com/uber/aresdb/cluster/shard"
6 | import topology "github.com/uber/aresdb/cluster/topology"
7 |
8 | // HostShardSet is an autogenerated mock type for the HostShardSet type
9 | type HostShardSet struct {
10 | mock.Mock
11 | }
12 |
13 | // Host provides a mock function with given fields:
14 | func (_m *HostShardSet) Host() topology.Host {
15 | ret := _m.Called()
16 |
17 | var r0 topology.Host
18 | if rf, ok := ret.Get(0).(func() topology.Host); ok {
19 | r0 = rf()
20 | } else {
21 | if ret.Get(0) != nil {
22 | r0 = ret.Get(0).(topology.Host)
23 | }
24 | }
25 |
26 | return r0
27 | }
28 |
29 | // ShardSet provides a mock function with given fields:
30 | func (_m *HostShardSet) ShardSet() shard.ShardSet {
31 | ret := _m.Called()
32 |
33 | var r0 shard.ShardSet
34 | if rf, ok := ret.Get(0).(func() shard.ShardSet); ok {
35 | r0 = rf()
36 | } else {
37 | if ret.Get(0) != nil {
38 | r0 = ret.Get(0).(shard.ShardSet)
39 | }
40 | }
41 |
42 | return r0
43 | }
44 |
--------------------------------------------------------------------------------
/cluster/topology/mocks/MapProvider.go:
--------------------------------------------------------------------------------
1 | // Code generated by mockery v1.0.0
2 | package mocks
3 |
4 | import mock "github.com/stretchr/testify/mock"
5 | import topology "github.com/uber/aresdb/cluster/topology"
6 |
7 | // MapProvider is an autogenerated mock type for the MapProvider type
8 | type MapProvider struct {
9 | mock.Mock
10 | }
11 |
12 | // TopologyMap provides a mock function with given fields:
13 | func (_m *MapProvider) TopologyMap() (topology.Map, error) {
14 | ret := _m.Called()
15 |
16 | var r0 topology.Map
17 | if rf, ok := ret.Get(0).(func() topology.Map); ok {
18 | r0 = rf()
19 | } else {
20 | if ret.Get(0) != nil {
21 | r0 = ret.Get(0).(topology.Map)
22 | }
23 | }
24 |
25 | var r1 error
26 | if rf, ok := ret.Get(1).(func() error); ok {
27 | r1 = rf()
28 | } else {
29 | r1 = ret.Error(1)
30 | }
31 |
32 | return r0, r1
33 | }
34 |
--------------------------------------------------------------------------------
/cluster/topology/mocks/MapWatch.go:
--------------------------------------------------------------------------------
1 | // Code generated by mockery v1.0.0
2 | package mocks
3 |
4 | import mock "github.com/stretchr/testify/mock"
5 | import topology "github.com/uber/aresdb/cluster/topology"
6 |
7 | // MapWatch is an autogenerated mock type for the MapWatch type
8 | type MapWatch struct {
9 | mock.Mock
10 | }
11 |
12 | // C provides a mock function with given fields:
13 | func (_m *MapWatch) C() <-chan struct{} {
14 | ret := _m.Called()
15 |
16 | var r0 <-chan struct{}
17 | if rf, ok := ret.Get(0).(func() <-chan struct{}); ok {
18 | r0 = rf()
19 | } else {
20 | if ret.Get(0) != nil {
21 | r0 = ret.Get(0).(<-chan struct{})
22 | }
23 | }
24 |
25 | return r0
26 | }
27 |
28 | // Close provides a mock function with given fields:
29 | func (_m *MapWatch) Close() {
30 | _m.Called()
31 | }
32 |
33 | // Get provides a mock function with given fields:
34 | func (_m *MapWatch) Get() topology.Map {
35 | ret := _m.Called()
36 |
37 | var r0 topology.Map
38 | if rf, ok := ret.Get(0).(func() topology.Map); ok {
39 | r0 = rf()
40 | } else {
41 | if ret.Get(0) != nil {
42 | r0 = ret.Get(0).(topology.Map)
43 | }
44 | }
45 |
46 | return r0
47 | }
48 |
--------------------------------------------------------------------------------
/cmd/aresd/main.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package main
16 |
17 | import "github.com/uber/aresdb/cmd/aresd/cmd"
18 |
19 | func main() {
20 | cmd.Execute()
21 | }
22 |
--------------------------------------------------------------------------------
/cmd/broker/main.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package main
16 |
17 | import "github.com/uber/aresdb/cmd/broker/cmd"
18 |
19 | func main() {
20 | cmd.Execute()
21 | }
22 |
--------------------------------------------------------------------------------
/cmd/controller/main.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package main
16 |
17 | import "github.com/uber/aresdb/cmd/controller/cmd"
18 |
19 | func main() {
20 | cmd.Execute()
21 | }
22 |
--------------------------------------------------------------------------------
/config/ares-broker.yaml:
--------------------------------------------------------------------------------
1 | port: 9474
2 |
3 | http:
4 | max_connections: 300
5 | read_time_out_in_seconds: 20
6 | write_time_out_in_seconds: 300 # 5 minutes to write the result
7 |
8 | cluster:
9 | namespace: "dist"
10 | instance_id: ""
11 | # example controller client configs
12 | controller:
13 | address: localhost:6708
14 | heartbeat:
15 | timeout: 10
16 | interval: 1
17 | etcd:
18 | zone: local
19 | env: dev
20 | service: ares-datanode
21 | etcdClusters:
22 | - zone: local
23 | endpoints:
24 | - 127.0.0.1:2379
25 |
--------------------------------------------------------------------------------
/config/ares-controller.yaml:
--------------------------------------------------------------------------------
1 | service:
2 | name: ares-controller
3 |
4 | logging:
5 | level: info
6 | development: false
7 | sampling:
8 | initial: 100
9 | thereafter: 100
10 | encoding: json
11 |
12 | etcd:
13 | enabled: true
14 | zone: local
15 | env: dev
16 | service: ares-controller
17 | etcdClusters:
18 | - zone: local
19 | endpoints:
20 | - 127.0.0.1:2379
21 |
22 | ingestionAssignmentTask:
23 | intervalInSeconds: 60
24 |
25 | ui:
26 | path: "controller/ui"
27 |
--------------------------------------------------------------------------------
/config/ares.yaml:
--------------------------------------------------------------------------------
1 | port: 9374
2 | debug_port: 43202
3 | root_path: ares-root
4 | total_memory_size: 161061273600 # 150gb
5 | query:
6 | device_memory_utilization: 0.95
7 | device_choosing_timeout: 10
8 | # enable timezone column for queries with "timezone": "timezone(city_id)"
9 | timezone_table:
10 | table_name: api_cities
11 | enable_hash_reduction: false
12 |
13 | disk_store:
14 | write_sync: true
15 | meta_store:
16 | write_sync: true
17 | http:
18 | max_connections: 650
19 | max_ingestion_connections: 300
20 | max_query_connections: 300
21 | read_time_out_in_seconds: 20
22 | write_time_out_in_seconds: 300 # 5 minutes to write the result
23 |
24 | cluster:
25 | enable: false
26 | distributed: false
27 | namespace: ""
28 | instance_id: ""
29 | # example controller client configs
30 | controller:
31 | address: localhost:6708
32 | heartbeat:
33 | timeout: 10
34 | interval: 1
35 | etcd:
36 | zone: local
37 | env: dev
38 | service: ares-datanode
39 | etcdClusters:
40 | - zone: local
41 | endpoints:
42 | - 127.0.0.1:2379
43 |
44 | redolog:
45 | disk:
46 | disabled: false
47 | kafka:
48 | enabled: false
49 |
50 |
--------------------------------------------------------------------------------
/controller/client/controller_client_suite_test.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 | package client
15 |
16 | import (
17 | "testing"
18 |
19 | . "github.com/onsi/ginkgo"
20 | . "github.com/onsi/gomega"
21 | )
22 |
23 | func TestControllerClient(t *testing.T) {
24 | RegisterFailHandler(Fail)
25 | RunSpecs(t, "Controller Client Suite")
26 | }
27 |
--------------------------------------------------------------------------------
/controller/generated/proto/entity.proto:
--------------------------------------------------------------------------------
1 | syntax = "proto3";
2 | package proto;
3 |
4 | message EntityConfig {
5 | string name = 1;
6 | bool tomstoned = 2;
7 | bytes config = 3;
8 | }
9 |
10 | message EntityName {
11 | string name = 1;
12 | bool tomstoned = 2;
13 | int32 incarnation = 3;
14 | int64 last_updated_at = 4;
15 | }
16 |
17 | message EntityList {
18 | int64 last_updated_at = 1;
19 | repeated EntityName entities = 2;
20 | }
--------------------------------------------------------------------------------
/controller/generated/proto/enum.proto:
--------------------------------------------------------------------------------
1 | syntax = "proto3";
2 | package proto;
3 |
4 | message EnumCases {
5 | repeated string cases = 1;
6 | }
7 |
8 | message EnumNodeList {
9 | int32 numEnumNodes = 1;
10 | }
--------------------------------------------------------------------------------
/controller/handlers/doc.go:
--------------------------------------------------------------------------------
1 | // Package handler provide the RESTful APIs to interact with Ares-Controller.
2 | //
3 | // Schemes: http
4 | // BasePath: /
5 | // Version: 0.0.1
6 | //
7 | // Consumes:
8 | // - application/json
9 | //
10 | // Produces:
11 | // - application/json
12 | //
13 | // swagger:meta
14 | package handlers
15 |
--------------------------------------------------------------------------------
/controller/handlers/module.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package handlers
16 |
17 | import (
18 | "go.uber.org/fx"
19 | )
20 |
21 | // Module is handler fx module
22 | var Module = fx.Provide(
23 | NewHealthHandler,
24 | NewConfigHandler,
25 | NewSchemaHandler,
26 | NewMembershipHandler,
27 | NewNamespaceHandler,
28 | NewAssignmentHandler,
29 | NewPlacementHandler,
30 | NewUIHandler,
31 | )
32 |
--------------------------------------------------------------------------------
/controller/models/Instance.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 | package models
15 |
16 | // Instance is the external view of instances
17 | type Instance struct {
18 | Address string `json:"address"`
19 | Host string `json:"host"`
20 | Port uint32 `json:"port"`
21 | Name string `json:"name"`
22 | }
23 |
--------------------------------------------------------------------------------
/controller/models/assignment.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 | package models
15 |
16 | // IngestionAssignment models job assignments for a subscriber
17 | type IngestionAssignment struct {
18 | Subscriber string `json:"subscriber"`
19 | Jobs []JobConfig `json:"jobs"`
20 | Instances map[string]Instance `json:"instances"`
21 | }
22 |
--------------------------------------------------------------------------------
/controller/models/subscriber.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 | package models
15 |
16 | // Subscriber models a subscriber instance
17 | type Subscriber struct {
18 | // Name is subscriber instance id
19 | Name string `json:"name"`
20 | // Host is host name of subscriber process
21 | Host string `json:"host"`
22 | }
23 |
--------------------------------------------------------------------------------
/controller/mutators/mocks/EnumReader.go:
--------------------------------------------------------------------------------
1 | // Code generated by mockery v1.0.0. DO NOT EDIT.
2 |
3 | package mocks
4 |
5 | import mock "github.com/stretchr/testify/mock"
6 |
7 | // EnumReader is an autogenerated mock type for the EnumReader type
8 | type EnumReader struct {
9 | mock.Mock
10 | }
11 |
12 | // GetEnumCases provides a mock function with given fields: namespace, table, column
13 | func (_m *EnumReader) GetEnumCases(namespace string, table string, column string) ([]string, error) {
14 | ret := _m.Called(namespace, table, column)
15 |
16 | var r0 []string
17 | if rf, ok := ret.Get(0).(func(string, string, string) []string); ok {
18 | r0 = rf(namespace, table, column)
19 | } else {
20 | if ret.Get(0) != nil {
21 | r0 = ret.Get(0).([]string)
22 | }
23 | }
24 |
25 | var r1 error
26 | if rf, ok := ret.Get(1).(func(string, string, string) error); ok {
27 | r1 = rf(namespace, table, column)
28 | } else {
29 | r1 = ret.Error(1)
30 | }
31 |
32 | return r0, r1
33 | }
34 |
--------------------------------------------------------------------------------
/controller/tasks/common/errors.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 | package common
15 |
16 | import "errors"
17 |
18 | var (
19 | // ErrNotEnoughSubscribers indicates there are too few subscribers in a namespace
20 | ErrNotEnoughSubscribers = errors.New("not enough subscribers")
21 | )
22 |
--------------------------------------------------------------------------------
/controller/tasks/etcd/leader_elector_test.go:
--------------------------------------------------------------------------------
1 | package etcd
2 |
3 | import (
4 | "github.com/golang/mock/gomock"
5 | "github.com/m3db/m3/src/cluster/services"
6 | "github.com/m3db/m3/src/cluster/services/leader/campaign"
7 | "github.com/stretchr/testify/assert"
8 | "testing"
9 | )
10 |
11 | func TestLeaderElect(t *testing.T) {
12 |
13 | t.Run("leader elector should work", func(t *testing.T) {
14 | // test setup
15 |
16 | ctrl := gomock.NewController(t)
17 | defer ctrl.Finish()
18 | leaderService := services.NewMockLeaderService(ctrl)
19 |
20 | leaderCh := make(chan campaign.Status, 1)
21 | leaderCh <- campaign.NewStatus(campaign.Leader)
22 | leaderService.EXPECT().Campaign("", gomock.Any()).Return(leaderCh, nil).AnyTimes()
23 | leaderService.EXPECT().Resign("").DoAndReturn(func(electionID string) error {
24 | leaderCh <- campaign.NewStatus(campaign.Follower)
25 | return nil
26 | })
27 |
28 | elector := NewLeaderElector(leaderService)
29 | err := elector.Start()
30 | assert.NoError(t, err)
31 | <-elector.C()
32 | assert.Equal(t, elector.Status(), Leader)
33 |
34 | err = elector.Resign()
35 | assert.NoError(t, err)
36 | <-elector.C()
37 | assert.Equal(t, elector.Status(), Follower)
38 | })
39 | }
40 |
--------------------------------------------------------------------------------
/controller/ui/.gitignore:
--------------------------------------------------------------------------------
1 | # See https://help.github.com/articles/ignoring-files/ for more about ignoring files.
2 |
3 | # dependencies
4 | /node_modules
5 | /.pnp
6 | .pnp.js
7 |
8 | # testing
9 | /coverage
10 |
11 | # production
12 | /build
13 |
14 | # misc
15 | .DS_Store
16 | .env.local
17 | .env.development.local
18 | .env.test.local
19 | .env.production.local
20 |
21 | npm-debug.log*
22 | yarn-debug.log*
23 | yarn-error.log*
24 |
--------------------------------------------------------------------------------
/controller/ui/package.json:
--------------------------------------------------------------------------------
1 | {
2 | "name": "ui",
3 | "version": "0.1.0",
4 | "private": true,
5 | "dependencies": {
6 | "immutability-helper": "^2.9.0",
7 | "react": "^16.6.3",
8 | "react-bootstrap": "^0.32.4",
9 | "react-dom": "^16.6.3",
10 | "react-json-editor-ajrm": "^2.5.9",
11 | "react-scripts": "2.1.1",
12 | "react-table": "^6.10.0",
13 | "swagger-ui-dist": "^3.20.5"
14 | },
15 | "scripts": {
16 | "start": "react-scripts start",
17 | "build": "react-scripts build",
18 | "test": "react-scripts test",
19 | "eject": "react-scripts eject"
20 | },
21 | "eslintConfig": {
22 | "extends": "react-app"
23 | },
24 | "engines": {
25 | "node": ">=8.0.0",
26 | "npm": ">=6.0.0"
27 | },
28 | "browserslist": [
29 | ">0.2%",
30 | "not dead",
31 | "not ie <= 11",
32 | "not op_mini all"
33 | ]
34 | }
35 |
--------------------------------------------------------------------------------
/controller/ui/public/favicon.ico:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/controller/ui/public/favicon.ico
--------------------------------------------------------------------------------
/controller/ui/public/manifest.json:
--------------------------------------------------------------------------------
1 | {
2 | "short_name": "React App",
3 | "name": "Create React App Sample",
4 | "icons": [
5 | {
6 | "src": "favicon.ico",
7 | "sizes": "64x64 32x32 24x24 16x16",
8 | "type": "image/x-icon"
9 | }
10 | ],
11 | "start_url": ".",
12 | "display": "standalone",
13 | "theme_color": "#000000",
14 | "background_color": "#ffffff"
15 | }
16 |
--------------------------------------------------------------------------------
/controller/ui/src/App.css:
--------------------------------------------------------------------------------
1 | .App {
2 | /*text-align: center;*/
3 | }
4 |
5 | .App-logo {
6 | animation: App-logo-spin infinite 20s linear;
7 | height: 40vmin;
8 | }
9 |
10 | .App-header {
11 | flex-direction: column;
12 | align-items: center;
13 | justify-content: center;
14 | font-size: calc(10px + 1vmin);
15 | }
16 |
17 | .App-link {
18 | color: #61dafb;
19 | }
20 |
21 | @keyframes App-logo-spin {
22 | from {
23 | transform: rotate(0deg);
24 | }
25 | to {
26 | transform: rotate(360deg);
27 | }
28 | }
29 |
--------------------------------------------------------------------------------
/controller/ui/src/App.test.js:
--------------------------------------------------------------------------------
1 | import React from 'react';
2 | import ReactDOM from 'react-dom';
3 | import App from './App';
4 |
5 | it('renders without crashing', () => {
6 | const div = document.createElement('div');
7 | ReactDOM.render(, div);
8 | ReactDOM.unmountComponentAtNode(div);
9 | });
10 |
--------------------------------------------------------------------------------
/controller/ui/src/index.css:
--------------------------------------------------------------------------------
1 | body {
2 | margin: 0;
3 | padding: 0;
4 | font-family: -apple-system, BlinkMacSystemFont, "Segoe UI", "Roboto", "Oxygen",
5 | "Ubuntu", "Cantarell", "Fira Sans", "Droid Sans", "Helvetica Neue",
6 | sans-serif;
7 | -webkit-font-smoothing: antialiased;
8 | -moz-osx-font-smoothing: grayscale;
9 | }
10 |
11 | code {
12 | font-family: source-code-pro, Menlo, Monaco, Consolas, "Courier New",
13 | monospace;
14 | }
15 |
--------------------------------------------------------------------------------
/controller/ui/src/index.js:
--------------------------------------------------------------------------------
1 | import React from 'react';
2 | import ReactDOM from 'react-dom';
3 | import './index.css';
4 | import App from './App';
5 | import * as serviceWorker from './serviceWorker';
6 |
7 | ReactDOM.render(, document.getElementById('root'));
8 |
9 | // If you want your app to work offline and load faster, you can change
10 | // unregister() to register() below. Note this comes with some pitfalls.
11 | // Learn more about service workers: http://bit.ly/CRA-PWA
12 | serviceWorker.unregister();
13 |
--------------------------------------------------------------------------------
/controller/ui/swagger/index.html:
--------------------------------------------------------------------------------
1 |
2 |
3 |
4 |
5 | AresDB Swagger UI
6 |
7 |
8 |
9 |
10 |
11 |
12 |
13 |
14 |
15 |
16 |
31 |
32 |
33 |
--------------------------------------------------------------------------------
/datanode/ares_suite_test.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package datanode
16 |
17 | import (
18 | . "github.com/onsi/ginkgo"
19 | . "github.com/onsi/gomega"
20 |
21 | "testing"
22 |
23 | "github.com/onsi/ginkgo/reporters"
24 | )
25 |
26 | func TestDataNode(t *testing.T) {
27 | RegisterFailHandler(Fail)
28 | junitReporter := reporters.NewJUnitReporter("junit.xml")
29 | RunSpecsWithDefaultAndCustomReporters(t, "Ares Datanode Suite", []Reporter{junitReporter})
30 | }
31 |
--------------------------------------------------------------------------------
/datanode/bootstrap/ares_suite_test.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package bootstrap
16 |
17 | import (
18 | . "github.com/onsi/ginkgo"
19 | "github.com/onsi/ginkgo/reporters"
20 | . "github.com/onsi/gomega"
21 | "testing"
22 | )
23 |
24 | func TestBootstrap(t *testing.T) {
25 | RegisterFailHandler(Fail)
26 | junitReporter := reporters.NewJUnitReporter("junit.xml")
27 | RunSpecsWithDefaultAndCustomReporters(t, "Ares Bootstrap Suite", []Reporter{junitReporter})
28 | }
29 |
--------------------------------------------------------------------------------
/datanode/bootstrap/mocks/Bootstrapable.go:
--------------------------------------------------------------------------------
1 | // Code generated by mockery v1.0.0. DO NOT EDIT.
2 |
3 | package mocks
4 |
5 | import bootstrap "github.com/uber/aresdb/datanode/bootstrap"
6 | import client "github.com/uber/aresdb/datanode/client"
7 | import mock "github.com/stretchr/testify/mock"
8 | import topology "github.com/uber/aresdb/cluster/topology"
9 |
10 | // Bootstrapable is an autogenerated mock type for the Bootstrapable type
11 | type Bootstrapable struct {
12 | mock.Mock
13 | }
14 |
15 | // Bootstrap provides a mock function with given fields: peerSource, origin, topo, topoState, options
16 | func (_m *Bootstrapable) Bootstrap(peerSource client.PeerSource, origin string, topo topology.Topology, topoState *topology.StateSnapshot, options bootstrap.Options) error {
17 | ret := _m.Called(peerSource, origin, topo, topoState, options)
18 |
19 | var r0 error
20 | if rf, ok := ret.Get(0).(func(client.PeerSource, string, topology.Topology, *topology.StateSnapshot, bootstrap.Options) error); ok {
21 | r0 = rf(peerSource, origin, topo, topoState, options)
22 | } else {
23 | r0 = ret.Error(0)
24 | }
25 |
26 | return r0
27 | }
28 |
--------------------------------------------------------------------------------
/datanode/client/datanode_client_suite_test.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 | package client
15 |
16 | import (
17 | "testing"
18 |
19 | . "github.com/onsi/ginkgo"
20 | . "github.com/onsi/gomega"
21 | )
22 |
23 | func TestControllerClient(t *testing.T) {
24 | RegisterFailHandler(Fail)
25 | RunSpecs(t, "Datanode Client Suite")
26 | }
27 |
--------------------------------------------------------------------------------
/datanode/client/mocks/Peer.go:
--------------------------------------------------------------------------------
1 | // Code generated by mockery v1.0.0
2 | package mocks
3 |
4 | import client "github.com/uber/aresdb/datanode/client"
5 | import mock "github.com/stretchr/testify/mock"
6 | import topology "github.com/uber/aresdb/cluster/topology"
7 |
8 | // Peer is an autogenerated mock type for the Peer type
9 | type Peer struct {
10 | mock.Mock
11 | }
12 |
13 | // BorrowConnection provides a mock function with given fields: fn
14 | func (_m *Peer) BorrowConnection(fn client.WithConnectionFn) error {
15 | ret := _m.Called(fn)
16 |
17 | var r0 error
18 | if rf, ok := ret.Get(0).(func(client.WithConnectionFn) error); ok {
19 | r0 = rf(fn)
20 | } else {
21 | r0 = ret.Error(0)
22 | }
23 |
24 | return r0
25 | }
26 |
27 | // Close provides a mock function with given fields:
28 | func (_m *Peer) Close() {
29 | _m.Called()
30 | }
31 |
32 | // Host provides a mock function with given fields:
33 | func (_m *Peer) Host() topology.Host {
34 | ret := _m.Called()
35 |
36 | var r0 topology.Host
37 | if rf, ok := ret.Get(0).(func() topology.Host); ok {
38 | r0 = rf()
39 | } else {
40 | if ret.Get(0) != nil {
41 | r0 = ret.Get(0).(topology.Host)
42 | }
43 | }
44 |
45 | return r0
46 | }
47 |
--------------------------------------------------------------------------------
/datanode/client/mocks/PeerSource.go:
--------------------------------------------------------------------------------
1 | // Code generated by mockery v1.0.0. DO NOT EDIT.
2 |
3 | package mocks
4 |
5 | import client "github.com/uber/aresdb/datanode/client"
6 | import mock "github.com/stretchr/testify/mock"
7 |
8 | // PeerSource is an autogenerated mock type for the PeerSource type
9 | type PeerSource struct {
10 | mock.Mock
11 | }
12 |
13 | // BorrowConnection provides a mock function with given fields: hostIDs, fn
14 | func (_m *PeerSource) BorrowConnection(hostIDs []string, fn client.WithConnectionFn) error {
15 | ret := _m.Called(hostIDs, fn)
16 |
17 | var r0 error
18 | if rf, ok := ret.Get(0).(func([]string, client.WithConnectionFn) error); ok {
19 | r0 = rf(hostIDs, fn)
20 | } else {
21 | r0 = ret.Error(0)
22 | }
23 |
24 | return r0
25 | }
26 |
27 | // Close provides a mock function with given fields:
28 | func (_m *PeerSource) Close() {
29 | _m.Called()
30 | }
31 |
--------------------------------------------------------------------------------
/datanode/generated/proto/generate.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | //go:generate protoc -I rpc/ rpc/peer_streaming.proto --go_out=plugins=grpc:rpc
16 | //go:generate mockery -dir rpc -output rpc/mocks -name PeerDataNode.*Client
17 |
18 | package proto
19 |
--------------------------------------------------------------------------------
/diskstore/ares_suite_test.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package diskstore
16 |
17 | import (
18 | . "github.com/onsi/ginkgo"
19 | . "github.com/onsi/gomega"
20 |
21 | "testing"
22 |
23 | "github.com/onsi/ginkgo/reporters"
24 | )
25 |
26 | func TestMemStore(t *testing.T) {
27 | RegisterFailHandler(Fail)
28 | junitReporter := reporters.NewJUnitReporter("junit.xml")
29 | RunSpecsWithDefaultAndCustomReporters(t, "Ares DiskStore Suite", []Reporter{junitReporter})
30 | }
31 |
--------------------------------------------------------------------------------
/docker/README.md:
--------------------------------------------------------------------------------
1 | ## AresDB Docker
2 | To facilitate the installation process, we have published the [Dockerfile](./Dockerfile) so
3 | you can get AresDB running in a docker container.
4 |
5 | ### Prerequisites
6 |
7 | * [nvidia-driver](https://github.com/NVIDIA/nvidia-docker/wiki/Frequently-Asked-Questions#how-do-i-install-the-nvidia-driver)
8 | * A [supported version](https://github.com/NVIDIA/nvidia-docker/wiki/Frequently-Asked-Questions#which-docker-packages-are-supported) of [Docker]
9 | * [nvida-docker](https://github.com/NVIDIA/nvidia-docker)
10 |
11 |
12 | ### Build
13 |
14 | ```bash
15 | $ mkdir aresdb-docker
16 | $ wget -O aresdb-docker/Dockerfile https://raw.githubusercontent.com/uber/aresdb/master/docker/Dockerfile
17 | $ docker build -t aresdb:latest aresdb-docker
18 | ```
19 |
20 | ### Run AresDB
21 | ```
22 | nvidia-docker run -p 9374:9374/tcp -p 43202:43202/tcp -it aresdb:latest
23 | root@9e4c5150659c:~/go/src/github.com/aresdb# make run_server >> ./log/aresdb.log 2>&1 &
24 | ```
25 |
26 | This command will compile AresDB, run the server in background and map the ports in the docker container on the docker host.
27 | Refer to [Swagger](https://github.com/uber/aresdb/wiki/Swagger) to start interactiving with AresDB.
--------------------------------------------------------------------------------
/docs/logo.png:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/docs/logo.png
--------------------------------------------------------------------------------
/examples/1k_trips/data/cities.csv:
--------------------------------------------------------------------------------
1 | id,name
2 | 1,"San Francisco"
3 | 2,"New York"
4 | 3,"Washington, D.C."
5 | 4,"Los Angeles"
6 | 5,"Austin"
7 | 6,"Paris"
8 | 7,"London"
9 | 8,"Las Vegas"
10 | 9,"Seattle"
11 | 10,"Denver"
--------------------------------------------------------------------------------
/examples/1k_trips/queries/total_fare.aql:
--------------------------------------------------------------------------------
1 | {
2 | "queries": [
3 | {
4 | "table": "trips",
5 | "measures": [
6 | {
7 | "alias": "value",
8 | "sqlExpression": "sum(fare)",
9 | "rowFilters": [
10 | "status='completed'"
11 | ]
12 | }
13 | ],
14 | "timeFilter": {
15 | "column": "request_at",
16 | "from": "24 hours ago",
17 | "to": "this quarter-hour"
18 | },
19 | "dimensions": [
20 | {
21 | "alias": "ts",
22 | "sqlExpression": "request_at",
23 | "timeBucketizer": "hour"
24 | }
25 | ],
26 | "joins": []
27 | }
28 | ]
29 | }
--------------------------------------------------------------------------------
/examples/1k_trips/queries/total_trips.aql:
--------------------------------------------------------------------------------
1 | {
2 | "queries": [
3 | {
4 | "table": "trips",
5 | "measures": [
6 | {
7 | "alias": "value",
8 | "sqlExpression": "count(*)",
9 | "rowFilters": [
10 | "status='completed'"
11 | ]
12 | }
13 | ],
14 | "timeFilter": {
15 | "column": "request_at",
16 | "from": "24 hours ago",
17 | "to": "this quarter-hour"
18 | },
19 | "dimensions": [
20 | {
21 | "alias": "ts",
22 | "sqlExpression": "request_at",
23 | "timeBucketizer": "hour"
24 | }
25 | ],
26 | "joins": []
27 | }
28 | ]
29 | }
30 |
--------------------------------------------------------------------------------
/examples/1k_trips/queries/total_trips.sql:
--------------------------------------------------------------------------------
1 | {
2 | "queries": ["SELECT count(*) AS value FROM trips WHERE status='completed' AND aql_time_filter(request_at, \"24 hours ago\", \"this quarter-hour\", America/New_York) GROUP BY aql_time_bucket_hour(request_at, \"\", America/New_York)"]
3 | }
--------------------------------------------------------------------------------
/examples/1k_trips/schema/cities.json:
--------------------------------------------------------------------------------
1 | {
2 | "name": "cities",
3 | "columns": [
4 | {
5 | "name": "id",
6 | "type": "Uint16"
7 | },
8 | {
9 | "name": "name",
10 | "type": "BigEnum"
11 | }
12 | ],
13 | "primaryKeyColumns": [0],
14 | "isFactTable": false,
15 | "config": {
16 | "batchSize": 2097152,
17 | "initPrimaryKeyNumBuckets": 8192,
18 | "maxRedoLogFileSize": 1073741824,
19 | "redoLogRotationInterval": 10800,
20 | "snapshotIntervalMinutes": 360,
21 | "snapshotThreshold": 6291456
22 | }
23 | }
--------------------------------------------------------------------------------
/examples/1k_trips/schema/trips.json:
--------------------------------------------------------------------------------
1 | {
2 | "name": "trips",
3 | "columns": [
4 | {
5 | "name": "request_at",
6 | "type": "Uint32"
7 | },
8 | {
9 | "name": "uuid",
10 | "type": "UUID"
11 | },
12 | {
13 | "name": "city_id",
14 | "type": "Uint16"
15 | },
16 | {
17 | "name": "status",
18 | "type": "SmallEnum"
19 | },
20 | {
21 | "name": "fare",
22 | "type": "Float32"
23 | },
24 | {
25 | "name": "driver_uuid",
26 | "type": "UUID"
27 | },
28 | {
29 | "name": "driver_uuid_hll",
30 | "type": "Uint32"
31 | },
32 | {
33 | "name": "request_point",
34 | "type": "GeoPoint"
35 | }
36 | ],
37 | "primaryKeyColumns": [1],
38 | "archivingSortColumns": [2,3],
39 | "config": {
40 | "archivingDelayMinutes": 1440,
41 | "archivingIntervalMinutes": 180,
42 | "backfillIntervalMinutes": 60,
43 | "backfillMaxBufferSize": 1048576,
44 | "backfillStoreBatchSize": 20000,
45 | "backfillThresholdInBytes": 20971520,
46 | "batchSize": 2097152,
47 | "maxRedoLogFileSize": 1073741824,
48 | "recordRetentionInDays": 3,
49 | "redoLogRotationInterval": 10800
50 | },
51 | "isFactTable": true
52 | }
--------------------------------------------------------------------------------
/examples/README.md:
--------------------------------------------------------------------------------
1 | ## AresDB Examples
2 |
3 | AresDB Examples provide examples code and dataset including table schema, data and queries for user to get familiar with how to use the system
4 |
5 | ### Prerequisites
6 |
7 | Before running the examples code, a running aresdb server is needed.
8 | Please refer to how to use [aresdb docker](../docker/README.md) to quickly start a aresdb server instance
9 |
10 | ### How To Use
11 |
12 | Under this examples directory, each directory is a sample data set containing schema, data and queries, e.g. 1k_trips
13 |
14 | To run the examples code:
15 | ```
16 | go build examples.go
17 | ./examples tables --dataset 1k_trips
18 | ./examples data --dataset 1k_trips
19 | ./examples query --dataset 1k_trips
20 | ```
21 |
--------------------------------------------------------------------------------
/examples/examples:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/examples/examples
--------------------------------------------------------------------------------
/img/aresdb-logo.png:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/img/aresdb-logo.png
--------------------------------------------------------------------------------
/integration/config/ares.yaml:
--------------------------------------------------------------------------------
1 | port: 9374
2 | root_path: ares-root
3 | total_memory_size: 161061273600 # 150gb
4 | query:
5 | device_memory_utilization: 0.95
6 | device_choosing_timeout: 10
7 | # enable timezone column for queries with "timezone": "timezone(city_id)"
8 | timezone_table:
9 | table_name: api_cities
10 | enable_hash_reduction: false
11 |
12 | disk_store:
13 | write_sync: true
14 | meta_store:
15 | write_sync: true
16 | http:
17 | max_connections: 650
18 | max_ingestion_connections: 300
19 | max_query_connections: 300
20 | read_time_out_in_seconds: 20
21 | write_time_out_in_seconds: 300 # 5 minutes to write the result
22 |
23 | cluster:
24 | enable: false
25 | distributed: false
26 | namespace: ""
27 | instance_id: ""
28 | # example controller client configs
29 | controller:
30 | address: localhost:6708
31 | heartbeat:
32 | timeout: 10
33 | interval: 1
34 | etcd:
35 | zone: local
36 | env: dev
37 | service: ares-datanode
38 | etcdClusters:
39 | - zone: local
40 | endpoints:
41 | - 127.0.0.1:2379
42 |
43 | redolog:
44 | disk:
45 | disabled: false
46 | kafka:
47 | enabled: false
48 |
49 |
--------------------------------------------------------------------------------
/integration/test-data/data/arraytest.csv:
--------------------------------------------------------------------------------
1 | request_at,uuid,city_id,status,fare,array_bool,array_int8,array_uint8,array_int16,array_uint16,array_int32,array_uint32,array_smallenum,array_bigenum,array_uuid,array_geopoint
2 | {time-1d},{uuid},{uint16},{smallenum},{float32},{array_bool},{array_int8},{array_uint8},{array_int16},{array_uint16},{array_int32},{array_uint32},{array_smallenum},{array_bigenum},{array_uuid},{array_geopoint}
3 | {time-2d},{uuid},{uint16},{smallenum},{float32},{array_bool},{array_int8},{array_uint8},{array_int16},{array_uint16},{array_int32},{array_uint32},{array_smallenum},{array_bigenum},{array_uuid},{array_geopoint}
4 |
--------------------------------------------------------------------------------
/integration/test-data/queries/array_query_contains.aql:
--------------------------------------------------------------------------------
1 | {
2 | "queries": [
3 | {
4 | "table": "arraytest",
5 | "measures": [
6 | {
7 | "alias": "value",
8 | "sqlExpression": "count(*)",
9 | "rowFilters": [
10 | "contains(array_int16, 20)"
11 | ]
12 | }
13 | ],
14 | "timeFilter": {
15 | "column": "request_at",
16 | "from": "24 hours ago",
17 | "to": "this quarter-hour"
18 | },
19 | "dimensions": [
20 | {
21 | "alias": "ts",
22 | "sqlExpression": "request_at",
23 | "timeBucketizer": "day"
24 | }
25 | ],
26 | "joins": []
27 | }
28 | ]
29 | }
30 |
--------------------------------------------------------------------------------
/integration/test-data/queries/array_query_elementat.aql:
--------------------------------------------------------------------------------
1 | {
2 | "queries": [
3 | {
4 | "table": "arraytest",
5 | "measures": [
6 | {
7 | "alias": "value",
8 | "sqlExpression": "count(*)",
9 | "rowFilters": [
10 | "element_at(array_int16, 0)=10"
11 | ]
12 | }
13 | ],
14 | "timeFilter": {
15 | "column": "request_at",
16 | "from": "24 hours ago",
17 | "to": "this quarter-hour"
18 | },
19 | "dimensions": [
20 | {
21 | "alias": "ts",
22 | "sqlExpression": "request_at",
23 | "timeBucketizer": "day"
24 | }
25 | ],
26 | "joins": []
27 | }
28 | ]
29 | }
30 |
--------------------------------------------------------------------------------
/integration/test-data/queries/array_query_length.aql:
--------------------------------------------------------------------------------
1 | {
2 | "queries": [
3 | {
4 | "table": "arraytest",
5 | "measures": [
6 | {
7 | "alias": "value",
8 | "sqlExpression": "count(*)"
9 | }
10 | ],
11 | "timeFilter": {
12 | "column": "request_at",
13 | "from": "24 hours ago",
14 | "to": "this quarter-hour"
15 | },
16 | "dimensions": [
17 | {
18 | "alias": "ts",
19 | "sqlExpression": "request_at",
20 | "timeBucketizer": "day"
21 | },
22 | {
23 | "alias": "element",
24 | "sqlExpression": "length(array_int32)"
25 | }
26 | ],
27 | "joins": []
28 | }
29 | ]
30 | }
31 |
--------------------------------------------------------------------------------
/memstore/ares_suite_test.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package memstore
16 |
17 | import (
18 | . "github.com/onsi/ginkgo"
19 | "github.com/onsi/ginkgo/reporters"
20 | . "github.com/onsi/gomega"
21 | "testing"
22 | )
23 |
24 | func TestMemStore(t *testing.T) {
25 | RegisterFailHandler(Fail)
26 | junitReporter := reporters.NewJUnitReporter("junit.xml")
27 | RunSpecsWithDefaultAndCustomReporters(t, "Ares MemStore Suite", []Reporter{junitReporter})
28 | }
29 |
--------------------------------------------------------------------------------
/memstore/common/ares_suite_test.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package common
16 |
17 | import (
18 | . "github.com/onsi/ginkgo"
19 | . "github.com/onsi/gomega"
20 |
21 | "testing"
22 |
23 | "github.com/onsi/ginkgo/reporters"
24 | )
25 |
26 | func TestMemStore(t *testing.T) {
27 | RegisterFailHandler(Fail)
28 | junitReporter := reporters.NewJUnitReporter("junit.xml")
29 | RunSpecsWithDefaultAndCustomReporters(t, "Ares MemStore Suite", []Reporter{junitReporter})
30 | }
31 |
--------------------------------------------------------------------------------
/memstore/common/job.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package common
16 |
17 | // JobType now we only have archiving job type.
18 | type JobType string
19 |
20 | const (
21 | // ArchivingJobType is the archiving job type.
22 | ArchivingJobType JobType = "archiving"
23 | // BackfillJobType is the backfill job type.
24 | BackfillJobType JobType = "backfill"
25 | // SnapshotJobType is the snapshot job type.
26 | SnapshotJobType JobType = "snapshot"
27 | // PurgeJobType is the purge job type.
28 | PurgeJobType JobType = "purge"
29 | )
30 |
--------------------------------------------------------------------------------
/memstore/common/mocks/BootStrapToken.go:
--------------------------------------------------------------------------------
1 | // Code generated by mockery v1.0.0. DO NOT EDIT.
2 | package mocks
3 |
4 | import mock "github.com/stretchr/testify/mock"
5 |
6 | // BootStrapToken is an autogenerated mock type for the BootStrapToken type
7 | type BootStrapToken struct {
8 | mock.Mock
9 | }
10 |
11 | // AcquireToken provides a mock function with given fields: table, shard
12 | func (_m *BootStrapToken) AcquireToken(table string, shard uint32) bool {
13 | ret := _m.Called(table, shard)
14 |
15 | var r0 bool
16 | if rf, ok := ret.Get(0).(func(string, uint32) bool); ok {
17 | r0 = rf(table, shard)
18 | } else {
19 | r0 = ret.Get(0).(bool)
20 | }
21 |
22 | return r0
23 | }
24 |
25 | // ReleaseToken provides a mock function with given fields: table, shard
26 | func (_m *BootStrapToken) ReleaseToken(table string, shard uint32) {
27 | _m.Called(table, shard)
28 | }
29 |
--------------------------------------------------------------------------------
/memstore/common/mocks/EnumUpdater.go:
--------------------------------------------------------------------------------
1 | // Code generated by mockery v1.0.0
2 | package mocks
3 |
4 | import mock "github.com/stretchr/testify/mock"
5 |
6 | // EnumUpdater is an autogenerated mock type for the EnumUpdater type
7 | type EnumUpdater struct {
8 | mock.Mock
9 | }
10 |
11 | // UpdateEnum provides a mock function with given fields: table, column, enumList
12 | func (_m *EnumUpdater) UpdateEnum(table string, column string, enumList []string) error {
13 | ret := _m.Called(table, column, enumList)
14 |
15 | var r0 error
16 | if rf, ok := ret.Get(0).(func(string, string, []string) error); ok {
17 | r0 = rf(table, column, enumList)
18 | } else {
19 | r0 = ret.Error(0)
20 | }
21 |
22 | return r0
23 | }
24 |
--------------------------------------------------------------------------------
/memstore/common/mocks/ListVectorParty.go:
--------------------------------------------------------------------------------
1 | // Code generated by mockery v1.0.0. DO NOT EDIT.
2 |
3 | package mocks
4 |
5 | import (
6 | mock "github.com/stretchr/testify/mock"
7 | "unsafe"
8 | )
9 |
10 | // ListVectorParty is an autogenerated mock type for the ListVectorParty type
11 | type ListVectorParty struct {
12 | mock.Mock
13 | }
14 |
15 | // GetListValue provides a mock function with given fields: row
16 | func (_m *ListVectorParty) GetListValue(row int) (unsafe.Pointer, bool) {
17 | ret := _m.Called(row)
18 |
19 | var r0 unsafe.Pointer
20 | if rf, ok := ret.Get(0).(func(int) unsafe.Pointer); ok {
21 | r0 = rf(row)
22 | } else {
23 | r0 = ret.Get(0).(unsafe.Pointer)
24 | }
25 |
26 | var r1 bool
27 | if rf, ok := ret.Get(1).(func(int) bool); ok {
28 | r1 = rf(row)
29 | } else {
30 | r1 = ret.Get(1).(bool)
31 | }
32 |
33 | return r0, r1
34 | }
35 |
36 | // SetListValue provides a mock function with given fields: row, val, valid
37 | func (_m *ListVectorParty) SetListValue(row int, val unsafe.Pointer, valid bool) {
38 | _m.Called(row, val, valid)
39 | }
40 |
--------------------------------------------------------------------------------
/memstore/common/types.go:
--------------------------------------------------------------------------------
1 | package common
2 |
3 | import (
4 | "github.com/uber/aresdb/utils"
5 | )
6 |
7 | type TableSchemaReader interface {
8 | // GetSchema returns schema for a table.
9 | GetSchema(table string) (*TableSchema, error)
10 | // GetSchemas returns all table schemas.
11 | GetSchemas() map[string]*TableSchema
12 |
13 | // Provide exclusive access to read/write data protected by MemStore.
14 | utils.RWLocker
15 | }
16 |
17 | type EnumUpdater interface {
18 | // UpdateEnum can update enum for one column
19 | UpdateEnum(table, column string, enumList []string) error
20 | }
21 |
22 | // BootStrapToken used to Acqure/Release token during data purge operations
23 | type BootStrapToken interface {
24 | // Call AcquireToken to reserve usage token before any data purge operation
25 | // when return result is true, then you can proceed to the purge operation and later call ReleaseToken to release the token
26 | // when return result is false, then some bootstrap work is going on, no purge operation is permitted
27 | AcquireToken(table string, shard uint32) bool
28 | // Call ReleaseToken wheneven you call AcquireToken with true return value to release token
29 | ReleaseToken(table string, shard uint32)
30 | }
31 |
--------------------------------------------------------------------------------
/memstore/list/ares_suite_test.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package list
16 |
17 | import (
18 | . "github.com/onsi/ginkgo"
19 | . "github.com/onsi/gomega"
20 |
21 | "testing"
22 |
23 | "github.com/onsi/ginkgo/reporters"
24 | )
25 |
26 | func TestListDataType(t *testing.T) {
27 | RegisterFailHandler(Fail)
28 | junitReporter := reporters.NewJUnitReporter("junit.xml")
29 | RunSpecsWithDefaultAndCustomReporters(t, "Ares List Data Type Suite", []Reporter{junitReporter})
30 | }
31 |
--------------------------------------------------------------------------------
/metastore/ares_suite_test.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package metastore
16 |
17 | import (
18 | . "github.com/onsi/ginkgo"
19 | . "github.com/onsi/gomega"
20 |
21 | "testing"
22 |
23 | "github.com/onsi/ginkgo/reporters"
24 | )
25 |
26 | func TestMetaStore(t *testing.T) {
27 | RegisterFailHandler(Fail)
28 | junitReporter := reporters.NewJUnitReporter("junit.xml")
29 | RunSpecsWithDefaultAndCustomReporters(t, "Ares MetaStore Suite", []Reporter{junitReporter})
30 | }
31 |
--------------------------------------------------------------------------------
/metastore/common/ares_suite_test.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package common
16 |
17 | import (
18 | . "github.com/onsi/ginkgo"
19 | . "github.com/onsi/gomega"
20 |
21 | "testing"
22 |
23 | "github.com/onsi/ginkgo/reporters"
24 | )
25 |
26 | func TestMetaStore(t *testing.T) {
27 | RegisterFailHandler(Fail)
28 | junitReporter := reporters.NewJUnitReporter("junit.xml")
29 | RunSpecsWithDefaultAndCustomReporters(t, "Ares MetaStore common Suite", []Reporter{junitReporter})
30 | }
31 |
--------------------------------------------------------------------------------
/metastore/mocks/TableSchemaValidator.go:
--------------------------------------------------------------------------------
1 | // Code generated by mockery v1.0.0
2 | package mocks
3 |
4 | import common "github.com/uber/aresdb/metastore/common"
5 |
6 | import mock "github.com/stretchr/testify/mock"
7 |
8 | // TableSchemaValidator is an autogenerated mock type for the TableSchemaValidator type
9 | type TableSchemaValidator struct {
10 | mock.Mock
11 | }
12 |
13 | // SetNewTable provides a mock function with given fields: table
14 | func (_m *TableSchemaValidator) SetNewTable(table common.Table) {
15 | _m.Called(table)
16 | }
17 |
18 | // SetOldTable provides a mock function with given fields: table
19 | func (_m *TableSchemaValidator) SetOldTable(table common.Table) {
20 | _m.Called(table)
21 | }
22 |
23 | // Validate provides a mock function with given fields:
24 | func (_m *TableSchemaValidator) Validate() error {
25 | ret := _m.Called()
26 |
27 | var r0 error
28 | if rf, ok := ret.Get(0).(func() error); ok {
29 | r0 = rf()
30 | } else {
31 | r0 = ret.Error(0)
32 | }
33 |
34 | return r0
35 | }
36 |
--------------------------------------------------------------------------------
/query/aql_context_test.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package query
16 |
17 | import (
18 | "encoding/json"
19 | "github.com/onsi/ginkgo"
20 | . "github.com/onsi/gomega"
21 | "strconv"
22 | )
23 |
24 | var _ = ginkgo.Describe("AQL context", func() {
25 | ginkgo.It("column usage marshal json should work", func() {
26 | usage := columnUsage(15)
27 | b, err := json.Marshal(usage)
28 | Ω(err).Should(BeNil())
29 | Ω(strconv.Unquote(string(b))).Should(Equal("allBatches+liveBatches+firstArchiveBatch+lastArchiveBatch"))
30 | })
31 | })
32 |
--------------------------------------------------------------------------------
/query/ares_suite_test.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package query
16 |
17 | import (
18 | "github.com/onsi/ginkgo"
19 | . "github.com/onsi/gomega"
20 |
21 | "github.com/onsi/ginkgo/reporters"
22 | "testing"
23 | )
24 |
25 | func TestQuery(t *testing.T) {
26 | RegisterFailHandler(ginkgo.Fail)
27 | junitReporter := reporters.NewJUnitReporter("junit.xml")
28 | ginkgo.RunSpecsWithDefaultAndCustomReporters(
29 | t, "Ares Query Suite", []ginkgo.Reporter{junitReporter})
30 | }
31 |
--------------------------------------------------------------------------------
/query/common/ares_suite_test.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package common
16 |
17 | import (
18 | "github.com/onsi/ginkgo"
19 | . "github.com/onsi/gomega"
20 |
21 | "testing"
22 |
23 | "github.com/onsi/ginkgo/reporters"
24 | )
25 |
26 | func TestQueryCommon(t *testing.T) {
27 | RegisterFailHandler(ginkgo.Fail)
28 | junitReporter := reporters.NewJUnitReporter("junit.xml")
29 | ginkgo.RunSpecsWithDefaultAndCustomReporters(t, "Ares Query Common Suite", []ginkgo.Reporter{junitReporter})
30 | }
31 |
--------------------------------------------------------------------------------
/query/common/constant.go:
--------------------------------------------------------------------------------
1 | package common
2 |
3 | import (
4 | memCom "github.com/uber/aresdb/memstore/common"
5 | "github.com/uber/aresdb/query/expr"
6 | )
7 |
8 | // DataTypeToExprType maps data type from the column schema format to
9 | // expression AST format.
10 | var DataTypeToExprType = map[memCom.DataType]expr.Type{
11 | memCom.Bool: expr.Boolean,
12 | memCom.Int8: expr.Signed,
13 | memCom.Int16: expr.Signed,
14 | memCom.Int32: expr.Signed,
15 | memCom.Int64: expr.Signed,
16 | memCom.Uint8: expr.Unsigned,
17 | memCom.Uint16: expr.Unsigned,
18 | memCom.Uint32: expr.Unsigned,
19 | memCom.Float32: expr.Float,
20 | memCom.SmallEnum: expr.Unsigned,
21 | memCom.BigEnum: expr.Unsigned,
22 | memCom.GeoPoint: expr.GeoPoint,
23 | memCom.GeoShape: expr.GeoShape,
24 | memCom.UUID: expr.UUID,
25 | }
26 |
--------------------------------------------------------------------------------
/query/sql/.DS_Store:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/query/sql/.DS_Store
--------------------------------------------------------------------------------
/query/sql/ares_suite_test.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package sql
16 |
17 | import (
18 | "github.com/onsi/ginkgo"
19 | . "github.com/onsi/gomega"
20 |
21 | "github.com/onsi/ginkgo/reporters"
22 | "testing"
23 | )
24 |
25 | func TestQuery(t *testing.T) {
26 | RegisterFailHandler(ginkgo.Fail)
27 | junitReporter := reporters.NewJUnitReporter("junit.xml")
28 | ginkgo.RunSpecsWithDefaultAndCustomReporters(
29 | t, "Ares SQL Suite", []ginkgo.Reporter{junitReporter})
30 | }
31 |
--------------------------------------------------------------------------------
/query/sql/readme:
--------------------------------------------------------------------------------
1 | TODO:
2 | - support namespace operations
3 | - support table schema operations: SHOW TABLES, DESCRIBE, CREATE TABLE, DROP TABLE
4 | - support write operations: INSERT, DELETE, ALTER, DROP/ADD COLUM
5 | - support EXPLAIN
6 | - support sub queries: WITH, VALUES + subquery
7 | - SET operation: INTERSECT, UNIN, EXCEPT
8 | - window function?
9 | - support string operations: CONCAT, LIKE
10 |
--------------------------------------------------------------------------------
/query/sql/tree/AllColumns.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package tree
16 |
17 | // AllColumns are AllColumns
18 | type AllColumns struct {
19 | // SelectItem is ISelectItem
20 | ISelectItem
21 | // Prefix is QualifiedName
22 | Prefix *QualifiedName
23 | }
24 |
25 | // NewAllColumns creates AllColumns
26 | func NewAllColumns(location *NodeLocation, prefix *QualifiedName) *AllColumns {
27 | return &AllColumns{
28 | NewSelectItem(location),
29 | prefix,
30 | }
31 | }
32 |
33 | // Accept accepts visitor
34 | func (n *AllColumns) Accept(visitor AstVisitor, ctx interface{}) interface{} {
35 | return visitor.VisitAllColumns(n, ctx)
36 | }
37 |
--------------------------------------------------------------------------------
/query/sql/tree/Expression.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package tree
16 |
17 | // IExpression is interface
18 | type IExpression interface {
19 | // INode is interface
20 | INode
21 | }
22 |
23 | // Expression is an IExpression
24 | type Expression struct {
25 | // Node is an INode
26 | *Node
27 | }
28 |
29 | // NewExpression creates Expression
30 | func NewExpression(location *NodeLocation) *Expression {
31 | return &Expression{
32 | NewNode(location),
33 | }
34 | }
35 |
36 | // Accept accepts visitor
37 | func (e *Expression) Accept(visitor AstVisitor, ctx interface{}) interface{} {
38 | return visitor.VisitExpression(e, ctx)
39 | }
40 |
--------------------------------------------------------------------------------
/query/sql/tree/GroupBy.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package tree
16 |
17 | // GroupBy is GroupBy
18 | type GroupBy struct {
19 | // Node is INode
20 | INode
21 | IsDistinct bool
22 | GroupingElements []IGroupingElement
23 | }
24 |
25 | // NewGroupBy creates GroupBy
26 | func NewGroupBy(location *NodeLocation, distinct bool, elements []IGroupingElement) *GroupBy {
27 | return &GroupBy{
28 | NewNode(location),
29 | distinct,
30 | elements,
31 | }
32 | }
33 |
34 | // Accept accepts visitor
35 | func (e *GroupBy) Accept(visitor AstVisitor, ctx interface{}) interface{} {
36 | return visitor.VisitGroupBy(e, ctx)
37 | }
38 |
--------------------------------------------------------------------------------
/query/sql/tree/JoinCriteria.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package tree
16 |
17 | // IJoinCriteria is interface
18 | type IJoinCriteria interface {
19 | // GetNodes returns the list of nodes
20 | GetNodes() []INode
21 | }
22 |
--------------------------------------------------------------------------------
/query/sql/tree/JoinOn.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package tree
16 |
17 | import (
18 | "github.com/uber/aresdb/query/sql/util"
19 | )
20 |
21 | // JoinOn is natural join
22 | type JoinOn struct {
23 | // IJoinCriteria is interface
24 | IJoinCriteria
25 | Expr IExpression
26 | }
27 |
28 | // NewJoinOn creates JoinOn
29 | func NewJoinOn(expr IExpression) *JoinOn {
30 | util.RequireNonNull(expr, "expression is null")
31 | return &JoinOn{
32 | Expr: expr,
33 | }
34 | }
35 |
36 | // GetNodes returns empty
37 | func (j *JoinOn) GetNodes() []INode {
38 | return []INode{
39 | j.Expr,
40 | }
41 | }
42 |
--------------------------------------------------------------------------------
/query/sql/tree/JoinUsing.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package tree
16 |
17 | import (
18 | "github.com/uber/aresdb/query/sql/util"
19 | )
20 |
21 | // JoinUsing is natural join
22 | type JoinUsing struct {
23 | // IJoinCriteria is interface
24 | IJoinCriteria
25 | Columns []*Identifier
26 | }
27 |
28 | // NewJoinUsing creates JoinUsing
29 | func NewJoinUsing(columns []*Identifier) *JoinUsing {
30 | util.RequireNonNull(columns, "columns is null")
31 | return &JoinUsing{
32 | Columns: columns,
33 | }
34 | }
35 |
36 | // GetNodes returns empty
37 | func (j *JoinUsing) GetNodes() []INode {
38 | return []INode{}
39 | }
40 |
--------------------------------------------------------------------------------
/query/sql/tree/NaturalJoin.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package tree
16 |
17 | // NaturalJoin is natural join
18 | type NaturalJoin struct {
19 | // IJoinCriteria is interface
20 | IJoinCriteria
21 | }
22 |
23 | // NewNaturalJoin creates NaturalJoin
24 | func NewNaturalJoin() *NaturalJoin {
25 | return &NaturalJoin{}
26 | }
27 |
28 | // GetNodes returns empty
29 | func (n *NaturalJoin) GetNodes() []INode {
30 | return nil
31 | }
32 |
--------------------------------------------------------------------------------
/query/sql/tree/OrderBy.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package tree
16 |
17 | // OrderBy is OrderBy
18 | type OrderBy struct {
19 | // Node is INode
20 | INode
21 | // SortItems is list of SortItems
22 | SortItems []*SortItem
23 | }
24 |
25 | // NewOrderBy creates OrderBy
26 | func NewOrderBy(location *NodeLocation, sortItems []*SortItem) *OrderBy {
27 | return &OrderBy{
28 | NewNode(location),
29 | sortItems,
30 | }
31 | }
32 |
33 | // Accept accepts visitor
34 | func (n *OrderBy) Accept(visitor AstVisitor, ctx interface{}) interface{} {
35 | return visitor.VisitOrderBy(n, ctx)
36 | }
37 |
--------------------------------------------------------------------------------
/query/sql/tree/Relation.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package tree
16 |
17 | // IRelation is interface
18 | type IRelation interface {
19 | // INode is interface
20 | INode
21 | }
22 |
23 | // Relation is IRelation
24 | type Relation struct {
25 | INode
26 | }
27 |
28 | // NewRelation create Relation
29 | func NewRelation(location *NodeLocation) *Relation {
30 | return &Relation{
31 | NewNode(location),
32 | }
33 | }
34 |
35 | // Accept accepts visitor
36 | func (r *Relation) Accept(visitor AstVisitor, ctx interface{}) interface{} {
37 | return visitor.VisitRelation(r, ctx)
38 | }
39 |
--------------------------------------------------------------------------------
/query/sql/tree/SelectItem.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package tree
16 |
17 | // ISelectItem is interface
18 | type ISelectItem interface {
19 | // INode is interface
20 | INode
21 | }
22 |
23 | // SelectItem is SelectItem
24 | type SelectItem struct {
25 | // Node is INode
26 | INode
27 | }
28 |
29 | // NewSelectItem creates SelectItem
30 | func NewSelectItem(location *NodeLocation) *SelectItem {
31 | return &SelectItem{
32 | NewNode(location),
33 | }
34 | }
35 |
36 | // Accept accepts visitor
37 | func (r *SelectItem) Accept(visitor AstVisitor, ctx interface{}) interface{} {
38 | return visitor.VisitSelectItem(r, ctx)
39 | }
40 |
--------------------------------------------------------------------------------
/query/sql/tree/Statement.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package tree
16 |
17 | // IStatement is interface
18 | type IStatement interface {
19 | // INode is interface
20 | INode
21 | }
22 |
23 | // Statement is Statement
24 | type Statement struct {
25 | // Node is INode
26 | INode
27 | }
28 |
29 | // NewStatement creates Statement
30 | func NewStatement(location *NodeLocation) *Statement {
31 | return &Statement{
32 | NewNode(location),
33 | }
34 | }
35 |
36 | // Accept accepts visitor
37 | func (q *Statement) Accept(visitor AstVisitor, ctx interface{}) interface{} {
38 | return visitor.VisitStatement(q, ctx)
39 | }
40 |
--------------------------------------------------------------------------------
/query/sql/tree/Table.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package tree
16 |
17 | // Table is table name
18 | type Table struct {
19 | IQueryBody
20 | // Name is table name
21 | Name *QualifiedName
22 | }
23 |
24 | // NewTable creates table name
25 | func NewTable(location *NodeLocation, name *QualifiedName) *Table {
26 | return &Table{
27 | NewQueryBody(location),
28 | name,
29 | }
30 | }
31 |
32 | // Accept accepts visitor
33 | func (q *Table) Accept(visitor AstVisitor, ctx interface{}) interface{} {
34 | return visitor.VisitTable(q, ctx)
35 | }
36 |
--------------------------------------------------------------------------------
/query/sql/tree/TableSubquery.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package tree
16 |
17 | // TableSubquery is subquery
18 | type TableSubquery struct {
19 | IQueryBody
20 | // Query is subquery
21 | Query *Query
22 | }
23 |
24 | // NewTableSubquery creates TableSubquery
25 | func NewTableSubquery(location *NodeLocation, query *Query) *TableSubquery {
26 | return &TableSubquery{
27 | NewQueryBody(location),
28 | query,
29 | }
30 | }
31 |
32 | // Accept accepts visitor
33 | func (q *TableSubquery) Accept(visitor AstVisitor, ctx interface{}) interface{} {
34 | return visitor.VisitTableSubquery(q, ctx)
35 | }
36 |
--------------------------------------------------------------------------------
/query/sql/tree/With.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package tree
16 |
17 | // With is With
18 | type With struct {
19 | // Node is INode
20 | INode
21 | // Recursive is with Recursive
22 | Recursive bool
23 | // Queries is WithQuery
24 | Queries []*WithQuery
25 | }
26 |
27 | // NewWith creates With
28 | func NewWith(location *NodeLocation, recursive bool, queries []*WithQuery) *With {
29 | return &With{
30 | NewNode(location),
31 | recursive,
32 | queries,
33 | }
34 | }
35 |
36 | // Accept accepts visitor
37 | func (n *With) Accept(visitor AstVisitor, ctx interface{}) interface{} {
38 | return visitor.VisitWith(n, ctx)
39 | }
40 |
--------------------------------------------------------------------------------
/redolog/redolog_suite_test.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package redolog
16 |
17 | import (
18 | . "github.com/onsi/ginkgo"
19 | . "github.com/onsi/gomega"
20 |
21 | "testing"
22 |
23 | "github.com/onsi/ginkgo/reporters"
24 | )
25 |
26 | func TestMemStore(t *testing.T) {
27 | RegisterFailHandler(Fail)
28 | junitReporter := reporters.NewJUnitReporter("junit.xml")
29 | RunSpecsWithDefaultAndCustomReporters(t, "Ares imports package Suite", []Reporter{junitReporter})
30 | }
31 |
--------------------------------------------------------------------------------
/scripts/clang-lint.sh:
--------------------------------------------------------------------------------
1 | #!/usr/bin/env bash
2 | C_SRC=$1
3 | if [[ ! -z ${C_SRC} ]]
4 | then
5 | cppcheck --std=c++11 --language=c++ --inline-suppr --suppress=selfInitialization ${C_SRC}
6 | cpplint --extensions=cu,hpp ${C_SRC}
7 | fi
--------------------------------------------------------------------------------
/scripts/golang-lint.sh:
--------------------------------------------------------------------------------
1 | #!/usr/bin/env bash
2 | GO_SRC=$1
3 | if [[ ! -z ${GO_SRC} ]]
4 | then
5 | gofmt -w ${GO_SRC}
6 | golint -set_exit_status ${GO_SRC}
7 | go vet -unsafeptr=false ./...
8 | fi
--------------------------------------------------------------------------------
/subscriber/common/consumer/consumer_suite_test.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package consumer_test
16 |
17 | import (
18 | "testing"
19 |
20 | . "github.com/onsi/ginkgo"
21 | . "github.com/onsi/gomega"
22 | )
23 |
24 | func TestConsumer(t *testing.T) {
25 | RegisterFailHandler(Fail)
26 | RunSpecs(t, "Consumer Suite")
27 | }
28 |
--------------------------------------------------------------------------------
/subscriber/common/consumer/kafka/kafka_suite_test.go:
--------------------------------------------------------------------------------
1 | package kafka_test
2 |
3 | import (
4 | "testing"
5 |
6 | . "github.com/onsi/ginkgo"
7 | . "github.com/onsi/gomega"
8 | )
9 |
10 | func TestKafka(t *testing.T) {
11 | RegisterFailHandler(Fail)
12 | RunSpecs(t, "Kafka Suite")
13 | }
14 |
--------------------------------------------------------------------------------
/subscriber/common/job/failure_handler.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package job
16 |
17 | import (
18 | "github.com/uber/aresdb/client"
19 | "github.com/uber/aresdb/subscriber/common/sink"
20 | )
21 |
22 | const retryHandler string = "retry"
23 |
24 | // FailureHandler interface will be implemented by failure handler
25 | // that are used when saving to storage layer fails
26 | type FailureHandler interface {
27 |
28 | // HandleFailure will provide a contingent plan to
29 | // keep track of failed save
30 | HandleFailure(destination sink.Destination, rows []client.Row) error
31 | }
32 |
--------------------------------------------------------------------------------
/subscriber/common/job/job_suite_test.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package job_test
16 |
17 | import (
18 | "testing"
19 |
20 | . "github.com/onsi/ginkgo"
21 | . "github.com/onsi/gomega"
22 | )
23 |
24 | func TestJob(t *testing.T) {
25 | RegisterFailHandler(Fail)
26 | RunSpecs(t, "Job Suite")
27 | }
28 |
--------------------------------------------------------------------------------
/subscriber/common/job/processor_test.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package job
16 |
17 | import (
18 | "errors"
19 |
20 | . "github.com/onsi/ginkgo"
21 | . "github.com/onsi/gomega"
22 | )
23 |
24 | var _ = Describe("driver", func() {
25 | p := ProcessorError{
26 | 1,
27 | int64(13434),
28 | errors.New("test"),
29 | }
30 |
31 | It("ErrorToJSON", func() {
32 | j := p.ErrorToJSON()
33 | Ω(j).ShouldNot(BeEmpty())
34 | })
35 | })
36 |
--------------------------------------------------------------------------------
/subscriber/common/message/message_suite_test.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package message_test
16 |
17 | import (
18 | "testing"
19 |
20 | . "github.com/onsi/ginkgo"
21 | . "github.com/onsi/gomega"
22 | )
23 |
24 | func TestMessage(t *testing.T) {
25 | RegisterFailHandler(Fail)
26 | RunSpecs(t, "Message Suite")
27 | }
28 |
--------------------------------------------------------------------------------
/subscriber/common/message/test_config/hp_config/schemas/hp-api-test_topic/test_topic.1.avsc:
--------------------------------------------------------------------------------
1 | {
2 | "fields": [
3 | {
4 | "name": "project",
5 | "type": "string"
6 | }
7 | ],
8 | "name": "test_topic",
9 | "namespace": "api",
10 | "owner": "pbathala@uber.com",
11 | "schema_id": 1,
12 | "type": "record"
13 | }
--------------------------------------------------------------------------------
/subscriber/common/message/test_config/hp_config/topic_registry.csv:
--------------------------------------------------------------------------------
1 | topic_name,schema_name
2 | test_topic,foo
--------------------------------------------------------------------------------
/subscriber/common/rules/rules_suite_test.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package rules_test
16 |
17 | import (
18 | "testing"
19 |
20 | . "github.com/onsi/ginkgo"
21 | . "github.com/onsi/gomega"
22 | )
23 |
24 | func TestRules(t *testing.T) {
25 | RegisterFailHandler(Fail)
26 | RunSpecs(t, "Rules Suite")
27 | }
28 |
--------------------------------------------------------------------------------
/subscriber/common/sink/sink_suite_test.go:
--------------------------------------------------------------------------------
1 | package sink_test
2 |
3 | import (
4 | "testing"
5 |
6 | . "github.com/onsi/ginkgo"
7 | . "github.com/onsi/gomega"
8 | )
9 |
10 | func TestSink(t *testing.T) {
11 | RegisterFailHandler(Fail)
12 | RunSpecs(t, "Sink Suite")
13 | }
14 |
--------------------------------------------------------------------------------
/subscriber/common/tools/tools_suite_test.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package tools_test
16 |
17 | import (
18 | "testing"
19 |
20 | . "github.com/onsi/ginkgo"
21 | . "github.com/onsi/gomega"
22 | )
23 |
24 | func TestTools(t *testing.T) {
25 | RegisterFailHandler(Fail)
26 | RunSpecs(t, "Tools Suite")
27 | }
28 |
--------------------------------------------------------------------------------
/subscriber/config/config_suite_test.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package config_test
16 |
17 | import (
18 | "testing"
19 |
20 | . "github.com/onsi/ginkgo"
21 | . "github.com/onsi/gomega"
22 | )
23 |
24 | func TestConfig(t *testing.T) {
25 | RegisterFailHandler(Fail)
26 | RunSpecs(t, "Config Suite")
27 | }
28 |
--------------------------------------------------------------------------------
/subscriber/config/test-ares-cluster-empty.yaml:
--------------------------------------------------------------------------------
1 | graphite:
2 | address: "localhost:4756"
3 |
4 | ares:
5 | # A list of namespaces. Each namespace includes a list of ares clusters
6 | namespaces:
7 | dev01:
8 | - dev-ares01
9 |
10 | # A list of jobs which are defined in config/jobs
11 | jobs:
12 | # A list of namespaces. Each namespace includes a list of jobs.
13 | namespaces:
14 | job-test:
15 | - test1
16 | trips:
17 | - job1
18 |
19 | controller:
20 | # enable sync up with aresDB controller
21 | enable: false
22 | address: "localhost:5436"
23 | refreshInterval: 10
24 | serviceName: "ares-controller-staging"
25 |
26 | zookeeper:
27 | server: "localhost:2181"
28 |
--------------------------------------------------------------------------------
/subscriber/config/test-ares-ns-empty.yaml:
--------------------------------------------------------------------------------
1 | graphite:
2 | address: "localhost:4756"
3 |
4 | ares:
5 |
6 | # A list of ares clusters. Each cluster includes master node address and timeout in seconds.
7 | clusters:
8 | clusters:
9 | dev-ares01:
10 | aresDB:
11 | address: "ares01:9000"
12 | timeout: 20
13 | dev-ares02:
14 | aresDB:
15 | address: "ares02:9000"
16 | timeout: 20
17 |
18 |
19 | # A list of jobs which are defined in config/jobs
20 | jobs:
21 | # A list of namespaces. Each namespace includes a list of jobs.
22 | namespaces:
23 | job-test:
24 | - test1
25 | trips:
26 | - job1
27 |
28 | controller:
29 | # enable sync up with aresDB controller
30 | enable: false
31 | address: "localhost:5436"
32 | refreshInterval: 10
33 | serviceName: "ares-controller-staging"
34 |
35 | zookeeper:
36 | server: "localhost:2181"
37 |
--------------------------------------------------------------------------------
/subscriber/config/test-controller-disable.yaml:
--------------------------------------------------------------------------------
1 | graphite:
2 | address: "localhost:4756"
3 |
4 | ares:
5 | # A list of namespaces. Each namespace includes a list of ares clusters
6 | namespaces:
7 | dev01:
8 | - dev-ares01
9 |
10 | # A list of ares clusters. Each cluster includes master node address and timeout in seconds.
11 | clusters:
12 | clusters:
13 | dev-ares01:
14 | aresDB:
15 | address: "ares01:9000"
16 | timeout: 20
17 | dev-ares02:
18 | aresDB:
19 | address: "ares02:9000"
20 | timeout: 20
21 |
22 | # A list of jobs which are defined in config/jobs
23 | jobs:
24 | # A list of namespaces. Each namespace includes a list of jobs.
25 | namespaces:
26 | job-test:
27 | - test1
28 | trips:
29 | - job1
30 |
31 | controller:
32 | # enable sync up with aresDB controller
33 | enable: true
34 | address: "localhost:5436"
35 | refreshInterval: 10
36 | serviceName: "ares-controller-staging"
37 |
38 | zookeeper:
39 | server: "localhost:2181"
40 |
--------------------------------------------------------------------------------
/subscriber/config/test-job-empty.yaml:
--------------------------------------------------------------------------------
1 | graphite:
2 | address: "localhost:4756"
3 |
4 | ares:
5 | # A list of namespaces. Each namespace includes a list of ares clusters
6 | namespaces:
7 | dev01:
8 | - dev-ares01
9 |
10 | # A list of ares clusters. Each cluster includes master node address and timeout in seconds.
11 | clusters:
12 | clusters:
13 | dev-ares01:
14 | aresDB:
15 | address: "ares01:9000"
16 | timeout: 20
17 | dev-ares02:
18 | aresDB:
19 | address: "ares02:9000"
20 | timeout: 20
21 |
22 | # A list of jobs which are defined in config/jobs
23 | jobs:
24 | # A list of namespaces. Each namespace includes a list of jobs.
25 |
26 | controller:
27 | # enable sync up with aresDB controller
28 | enable: false
29 | address: "localhost:5436"
30 | refreshInterval: 10
31 | serviceName: "ares-controller-staging"
32 |
33 | zookeeper:
34 | server: "localhost:2181"
35 |
--------------------------------------------------------------------------------
/testing/data/batches/archiveBatch:
--------------------------------------------------------------------------------
1 | columns:
2 | - sortedVP0
3 | - sortedVP1
4 | - sortedVP2
5 | - sortedVP3
6 | - sortedVP4
7 | - sortedVP5
8 |
--------------------------------------------------------------------------------
/testing/data/batches/archiving/archiveBatch0:
--------------------------------------------------------------------------------
1 | columns:
2 | - archiving/sortedVP0
3 | - archiving/sortedVP1
4 | - archiving/sortedVP2
5 | - archiving/sortedVP3_array
6 | - archiving/sortedVP4_array
7 |
8 |
--------------------------------------------------------------------------------
/testing/data/batches/archiving/archiveBatch1:
--------------------------------------------------------------------------------
1 | columns:
2 | - archiving/sortedVP00
3 | - archiving/sortedVP1
4 | - archiving/sortedVP2
5 | - archiving/sortedVP3_array
6 |
--------------------------------------------------------------------------------
/testing/data/batches/archiving/batch-101:
--------------------------------------------------------------------------------
1 | columns:
2 | - archiving/vp00
3 | - archiving/vp01
4 | - archiving/vp02
5 | - archiving/vp03_array
6 | - archiving/vp04_array
7 |
--------------------------------------------------------------------------------
/testing/data/batches/archiving/batch-110:
--------------------------------------------------------------------------------
1 | columns:
2 | - archiving/vp10
3 | - archiving/vp11
4 | - archiving/vp12
5 | - archiving/vp13_array
6 | - archiving/vp14_array
7 |
--------------------------------------------------------------------------------
/testing/data/batches/archiving/batch-120:
--------------------------------------------------------------------------------
1 | columns:
2 | - archiving/vp14
3 | - archiving/vp11
4 | - archiving/vp12
5 | - archiving/vp13_array
6 |
--------------------------------------------------------------------------------
/testing/data/batches/archiving/batch-99:
--------------------------------------------------------------------------------
1 | columns:
2 |
3 |
--------------------------------------------------------------------------------
/testing/data/batches/backfill/backfillBase:
--------------------------------------------------------------------------------
1 | columns:
2 | - backfill/backfillBase0
3 | - backfill/backfillBase1
4 | - backfill/backfillBase2
5 | - backfill/backfillBase3
6 | - backfill/backfillBase4
7 | - backfill/backfillBase5
8 | - backfill/backfillBase6
9 |
--------------------------------------------------------------------------------
/testing/data/batches/backfill/backfillNew:
--------------------------------------------------------------------------------
1 | columns:
2 | - backfill/backfillNew0
3 | - backfill/backfillNew1
4 | - backfill/backfillNew2
5 | - backfill/backfillNew3
6 | - backfill/backfillNew4
7 | - backfill/backfillNew5
8 | - backfill/backfillNew6
9 |
--------------------------------------------------------------------------------
/testing/data/batches/backfill/backfillTempLiveStore:
--------------------------------------------------------------------------------
1 | columns:
2 | - backfill/backfillTempLiveStore0
3 | - backfill/backfillTempLiveStore1
4 | - backfill/backfillTempLiveStore2
5 | - null
6 | - backfill/backfillTempLiveStore4
7 | - backfill/backfillTempLiveStore5
8 | - backfill/backfillTempLiveStore6
9 |
--------------------------------------------------------------------------------
/testing/data/batches/backfill/buildIndex:
--------------------------------------------------------------------------------
1 | columns:
2 | - backfill/buildIndex0
3 | - backfill/buildIndex1
4 | - backfill/buildIndex2
5 | - backfill/buildIndex3
--------------------------------------------------------------------------------
/testing/data/batches/backfill/cloneVPForWrite:
--------------------------------------------------------------------------------
1 | columns:
2 | - backfill/cloneVPForWrite0
3 | - backfill/cloneVPForWrite1
4 | - backfill/cloneVPForWrite2
5 | - backfill/cloneVPForWrite3
6 |
--------------------------------------------------------------------------------
/testing/data/batches/live/batch-120:
--------------------------------------------------------------------------------
1 | # batch for timezone table
2 | columns:
3 | - archiving/vp00
4 | - archiving/vp13
5 |
--------------------------------------------------------------------------------
/testing/data/batches/live/batch-130:
--------------------------------------------------------------------------------
1 | columns:
2 | - archiving/vp10
3 | - archiving/vp00
4 |
--------------------------------------------------------------------------------
/testing/data/batches/merge-nil-base:
--------------------------------------------------------------------------------
1 | columns:
2 | - merge-nil-base/mergedVP0
3 | - merge-nil-base/mergedVP1
4 | - merge-nil-base/mergedVP2
5 | - merge-nil-base/mergedVP3
6 | - merge-nil-base/mergedVP4
7 | - merge-nil-base/mergedVP5
--------------------------------------------------------------------------------
/testing/data/batches/merge-with-deleted-columns/mergedBatch:
--------------------------------------------------------------------------------
1 | columns:
2 | - mergedVP0
3 | - mergedVP1
4 | - mergedVP2
5 | - mergedVP3
6 | - mergedVP4
7 | - merge-with-deleted-columns/mergedVP5
8 |
9 |
--------------------------------------------------------------------------------
/testing/data/batches/merge-with-deleted-rows/baseBatch:
--------------------------------------------------------------------------------
1 | columns:
2 | - sortedVP0
3 | - sortedVP1
4 |
--------------------------------------------------------------------------------
/testing/data/batches/merge-with-deleted-rows/mergedBatch:
--------------------------------------------------------------------------------
1 | columns:
2 | - merge-with-deleted-rows/mergedVP0
3 | - merge-with-deleted-rows/mergedVP1
4 | - merge-with-deleted-rows/mergedVP2
5 | - merge-with-deleted-rows/mergedVP3
6 | - merge-with-deleted-rows/mergedVP4
7 | - merge-with-deleted-rows/mergedVP5
8 |
9 |
--------------------------------------------------------------------------------
/testing/data/batches/merge-with-deleted-rows/mergedBatch2:
--------------------------------------------------------------------------------
1 | columns:
2 | - merge-with-deleted-rows/mergedVP10
3 | - merge-with-deleted-rows/mergedVP11
4 |
5 |
--------------------------------------------------------------------------------
/testing/data/batches/merge-with-deleted-rows/patchBatch:
--------------------------------------------------------------------------------
1 | columns:
2 | - patchVP10
3 | - patchVP11
4 |
--------------------------------------------------------------------------------
/testing/data/batches/mergedBatch:
--------------------------------------------------------------------------------
1 | columns:
2 | - mergedVP0
3 | - mergedVP1
4 | - mergedVP2
5 | - mergedVP3
6 | - mergedVP4
7 | - mergedVP5
8 |
--------------------------------------------------------------------------------
/testing/data/batches/no-sort-columns/baseBatch:
--------------------------------------------------------------------------------
1 | columns:
2 | - no-sort-columns/baseVP0
3 |
4 |
--------------------------------------------------------------------------------
/testing/data/batches/no-sort-columns/mergedBatch:
--------------------------------------------------------------------------------
1 | columns:
2 | - no-sort-columns/mergedVP0
3 | - no-sort-columns/mergedVP1
4 |
--------------------------------------------------------------------------------
/testing/data/batches/no-sort-columns/patchBatch:
--------------------------------------------------------------------------------
1 | columns:
2 | - no-sort-columns/patchVP0
3 | - no-sort-columns/patchVP1
4 |
--------------------------------------------------------------------------------
/testing/data/batches/patchBatch0:
--------------------------------------------------------------------------------
1 | columns:
2 | - patchVP00
3 | - patchVP01
4 | - patchVP02
5 | - patchVP03
6 | - patchVP04
7 | - patchVP05
8 |
--------------------------------------------------------------------------------
/testing/data/batches/patchBatch1:
--------------------------------------------------------------------------------
1 | columns:
2 | - patchVP10
3 | - patchVP11
4 | - patchVP12
5 | - patchVP13
6 | - patchVP14
7 | - patchVP15
8 |
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/dimtable1_0/snapshots/1560032167_605/-2147483648/0.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/dimtable1_0/snapshots/1560032167_605/-2147483648/0.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/dimtable1_0/snapshots/1560032167_605/-2147483648/1.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/dimtable1_0/snapshots/1560032167_605/-2147483648/1.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/dimtable1_0/snapshots/1560032167_605/-2147483648/2.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/dimtable1_0/snapshots/1560032167_605/-2147483648/2.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/dimtable1_0/snapshots/1560032167_605/-2147483648/3.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/dimtable1_0/snapshots/1560032167_605/-2147483648/3.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/dimtable1_0/snapshots/1560032167_605/-2147483648/4.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/dimtable1_0/snapshots/1560032167_605/-2147483648/4.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-01_1559436638/0.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-01_1559436638/0.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-01_1559436638/1.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-01_1559436638/1.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-01_1559436638/2.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-01_1559436638/2.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-01_1559436638/3.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-01_1559436638/3.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-01_1559436638/4.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-01_1559436638/4.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-01_1559436638/5.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-01_1559436638/5.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-02_1559523900/0.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-02_1559523900/0.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-02_1559523900/1.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-02_1559523900/1.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-02_1559523900/2.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-02_1559523900/2.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-02_1559523900/3.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-02_1559523900/3.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-02_1559523900/4.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-02_1559523900/4.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-02_1559523900/5.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-02_1559523900/5.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-03_1559611162/0.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-03_1559611162/0.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-03_1559611162/1.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-03_1559611162/1.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-03_1559611162/2.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-03_1559611162/2.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-03_1559611162/3.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-03_1559611162/3.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-03_1559611162/4.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-03_1559611162/4.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-03_1559611162/5.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-03_1559611162/5.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-04_1559698827/0.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-04_1559698827/0.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-04_1559698827/1.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-04_1559698827/1.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-04_1559698827/2.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-04_1559698827/2.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-04_1559698827/3.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-04_1559698827/3.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-04_1559698827/4.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-04_1559698827/4.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-04_1559698827/5.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-04_1559698827/5.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-05_1559779414/0.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-05_1559779414/0.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-05_1559779414/1.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-05_1559779414/1.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-05_1559779414/2.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-05_1559779414/2.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-05_1559779414/3.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-05_1559779414/3.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-05_1559779414/4.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-05_1559779414/4.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-05_1559779414/5.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-05_1559779414/5.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-06_1559866672/0.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-06_1559866672/0.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-06_1559866672/1.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-06_1559866672/1.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-06_1559866672/2.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-06_1559866672/2.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-06_1559866672/3.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-06_1559866672/3.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-06_1559866672/4.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-06_1559866672/4.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-06_1559866672/5.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-06_1559866672/5.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-07_1559954536/0.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-07_1559954536/0.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-07_1559954536/1.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-07_1559954536/1.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-07_1559954536/2.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-07_1559954536/2.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-07_1559954536/3.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-07_1559954536/3.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-07_1559954536/4.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-07_1559954536/4.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-07_1559954536/5.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-07_1559954536/5.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-08_1560042370/0.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-08_1560042370/0.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-08_1560042370/1.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-08_1560042370/1.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-08_1560042370/2.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-08_1560042370/2.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-08_1560042370/3.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-08_1560042370/3.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-08_1560042370/4.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-08_1560042370/4.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-08_1560042370/5.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-08_1560042370/5.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-09_1560049865/0.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-09_1560049865/0.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-09_1560049865/1.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-09_1560049865/1.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-09_1560049865/2.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-09_1560049865/2.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-09_1560049865/3.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-09_1560049865/3.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-09_1560049865/4.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-09_1560049865/4.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-09_1560049865/5.data:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/bootstrap/data/facttable1_0/archiving_batches/2019-06-09_1560049865/5.data
--------------------------------------------------------------------------------
/testing/data/bootstrap/metastore/dimtable1/schema:
--------------------------------------------------------------------------------
1 | {
2 | "name": "dimtable1",
3 | "columns": [
4 | {
5 | "name": "uuid",
6 | "type": "UUID",
7 | "config": {}
8 | },
9 | {
10 | "name": "cid",
11 | "type": "Uint16",
12 | "config": {}
13 | },
14 | {
15 | "name": "ctime",
16 | "type": "Uint32",
17 | "config": {}
18 | },
19 | {
20 | "name": "location",
21 | "type": "GeoPoint",
22 | "config": {}
23 | },
24 | {
25 | "name": "puuid",
26 | "type": "UUID",
27 | "config": {}
28 | }
29 | ],
30 | "primaryKeyColumns": [
31 | 0
32 | ],
33 | "isFactTable": false,
34 | "version": 0
35 | }
36 |
--------------------------------------------------------------------------------
/testing/data/bootstrap/metastore/dimtable1/shards/0/checkpoint-offset:
--------------------------------------------------------------------------------
1 | 1000
2 |
--------------------------------------------------------------------------------
/testing/data/bootstrap/metastore/dimtable1/shards/0/commit-offset:
--------------------------------------------------------------------------------
1 | 10000
2 |
--------------------------------------------------------------------------------
/testing/data/bootstrap/metastore/dimtable1/shards/0/snapshot:
--------------------------------------------------------------------------------
1 | 1560032167,605,-2147483648,603670
--------------------------------------------------------------------------------
/testing/data/bootstrap/metastore/facttable1/schema:
--------------------------------------------------------------------------------
1 | {
2 | "name": "facttable1",
3 | "columns": [
4 | {
5 | "name": "end",
6 | "type": "Uint32",
7 | "config": {
8 | "preloadingDays": 7
9 | },
10 | "hllConfig": {}
11 | },
12 | {
13 | "name": "uuid1",
14 | "type": "UUID",
15 | "config": {},
16 | "hllConfig": {}
17 | },
18 | {
19 | "name": "uuid2",
20 | "type": "UUID",
21 | "config": {
22 | "preloadingDays": 7
23 | },
24 | "hllConfig": {}
25 | },
26 | {
27 | "name": "rid",
28 | "type": "Int16",
29 | "config": {
30 | "preloadingDays": 7
31 | },
32 | "hllConfig": {}
33 | },
34 | {
35 | "name": "vid",
36 | "type": "Int32",
37 | "config": {
38 | "preloadingDays": 7
39 | },
40 | "hllConfig": {}
41 | },
42 | {
43 | "name": "cid",
44 | "type": "Int32",
45 | "config": {},
46 | "hllConfig": {}
47 | }
48 | ],
49 | "primaryKeyColumns": [
50 | 0,
51 | 1
52 | ],
53 | "isFactTable": true,
54 | "archivingSortColumns": [
55 | 3,
56 | 4
57 | ],
58 | "version": 2
59 | }
60 |
--------------------------------------------------------------------------------
/testing/data/bootstrap/metastore/facttable1/shards/0/batches/18048:
--------------------------------------------------------------------------------
1 | 1559349772,2905321
2 | 1559356996,10798997
3 | 1559364255,16903286
4 | 1559371512,21127307
5 | 1559378738,27544540
6 | 1559385969,33620438
7 | 1559393222,38096853
8 | 1559400474,43362959
9 | 1559407691,51819222
10 | 1559414912,59808210
11 | 1559422130,67555106
12 | 1559429345,74016086
13 | 1559436638,78347676
14 |
--------------------------------------------------------------------------------
/testing/data/bootstrap/metastore/facttable1/shards/0/batches/18049:
--------------------------------------------------------------------------------
1 | 1559436638,3339544
2 | 1559443864,10761277
3 | 1559451113,16714988
4 | 1559458426,21357923
5 | 1559465656,28336271
6 | 1559472892,34097271
7 | 1559480193,38747952
8 | 1559487429,44203114
9 | 1559494682,52551298
10 | 1559502051,60755257
11 | 1559509310,67666847
12 | 1559516564,74714274
13 | 1559523900,78332865
14 |
--------------------------------------------------------------------------------
/testing/data/bootstrap/metastore/facttable1/shards/0/batches/18050:
--------------------------------------------------------------------------------
1 | 1559523900,4166976
2 | 1559531160,10375953
3 | 1559538377,14202107
4 | 1559545647,16996117
5 | 1559552897,21336480
6 | 1559560099,25207157
7 | 1559567510,28593916
8 | 1559574750,32838963
9 | 1559582006,39649716
10 | 1559589318,45634715
11 | 1559596559,50387357
12 | 1559603789,54781371
13 | 1559611162,56592541
14 |
--------------------------------------------------------------------------------
/testing/data/bootstrap/metastore/facttable1/shards/0/batches/18051:
--------------------------------------------------------------------------------
1 | 1559611162,3445550
2 | 1559618407,7989557
3 | 1559625622,11237234
4 | 1559633447,14421102
5 | 1559640701,18890821
6 | 1559647902,22855348
7 | 1559655191,26392094
8 | 1559662432,31606454
9 | 1559669646,39195802
10 | 1559677015,45425852
11 | 1559684263,50015928
12 | 1559691501,54972487
13 | 1559698827,55947153
14 |
--------------------------------------------------------------------------------
/testing/data/bootstrap/metastore/facttable1/shards/0/batches/18052:
--------------------------------------------------------------------------------
1 | 1559698827,4645321
2 | 1559706076,9228424
3 | 1559713294,12413264
4 | 1559721324,16848650
5 | 1559728534,22473861
6 | 1559735744,26994208
7 | 1559743161,30611566
8 | 1559750414,36690107
9 | 1559757632,43934424
10 | 1559764957,50189567
11 | 1559772209,54861950
12 | 1559779414,60331185
13 |
--------------------------------------------------------------------------------
/testing/data/bootstrap/metastore/facttable1/shards/0/batches/18053:
--------------------------------------------------------------------------------
1 | 1559779414,206239
2 | 1559786749,6604630
3 | 1559793982,11267000
4 | 1559801195,14096705
5 | 1559808520,17985384
6 | 1559815778,23020742
7 | 1559822994,27136908
8 | 1559830418,30950908
9 | 1559837678,37814383
10 | 1559844910,45561890
11 | 1559852265,51795253
12 | 1559859469,56446467
13 | 1559866672,61299984
14 |
--------------------------------------------------------------------------------
/testing/data/bootstrap/metastore/facttable1/shards/0/batches/18054:
--------------------------------------------------------------------------------
1 | 1559866672,940305
2 | 1559874687,7700803
3 | 1559881942,12227996
4 | 1559889159,15227558
5 | 1559896487,20513319
6 | 1559903695,26710802
7 | 1559910919,31500566
8 | 1559918294,36310000
9 | 1559925508,44452312
10 | 1559932710,52915194
11 | 1559940110,60000839
12 | 1559947320,65756754
13 | 1559954536,70530940
14 |
--------------------------------------------------------------------------------
/testing/data/bootstrap/metastore/facttable1/shards/0/batches/18055:
--------------------------------------------------------------------------------
1 | 1559954536,2874758
2 | 1559962564,12409316
3 | 1559969787,18033176
4 | 1559977027,22195724
5 | 1559984381,28983261
6 | 1559991614,34946535
7 | 1559998853,39577926
8 | 1560006181,45119716
9 | 1560013404,53191884
10 | 1560020657,60829581
11 | 1560027944,67452779
12 | 1560035153,73697937
13 | 1560042370,76868634
14 |
--------------------------------------------------------------------------------
/testing/data/bootstrap/metastore/facttable1/shards/0/batches/18056:
--------------------------------------------------------------------------------
1 | 1560042370,4091891
2 | 1560049865,11315565
3 |
--------------------------------------------------------------------------------
/testing/data/bootstrap/metastore/facttable1/shards/0/checkpoint-offset:
--------------------------------------------------------------------------------
1 | 1000
2 |
--------------------------------------------------------------------------------
/testing/data/bootstrap/metastore/facttable1/shards/0/commit-offset:
--------------------------------------------------------------------------------
1 | 10000
2 |
--------------------------------------------------------------------------------
/testing/data/bootstrap/metastore/facttable1/shards/0/redolog-offset:
--------------------------------------------------------------------------------
1 | 1560052917,6894
--------------------------------------------------------------------------------
/testing/data/bootstrap/metastore/facttable1/shards/0/version:
--------------------------------------------------------------------------------
1 | 1560049865
--------------------------------------------------------------------------------
/testing/data/data-sender/sample.csv:
--------------------------------------------------------------------------------
1 | c1,c2
2 | 100,case1
3 | 101,case2
4 | 102,case1
5 | 101,case1
6 | 103,
7 | 104,case3
8 | 105,
9 |
--------------------------------------------------------------------------------
/testing/data/integration/sample-ares-root/data/abc_0/redologs/1501869573.redolog:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/integration/sample-ares-root/data/abc_0/redologs/1501869573.redolog
--------------------------------------------------------------------------------
/testing/data/integration/sample-ares-root/metastore/abc/enums/c2:
--------------------------------------------------------------------------------
1 | case1
2 | case2
3 |
4 | case3
5 |
--------------------------------------------------------------------------------
/testing/data/integration/sample-ares-root/metastore/abc/schema:
--------------------------------------------------------------------------------
1 | {
2 | "name": "abc",
3 | "columns": [
4 | {
5 | "name": "c1",
6 | "type": "Uint8"
7 | },
8 | {
9 | "name": "c2",
10 | "type": "SmallEnum",
11 | "highPriorityDays": 10,
12 | "mediumPriorityDays": 10
13 | }
14 | ],
15 | "primaryKeyColumns": [
16 | 0
17 | ],
18 | "isFactTable": false,
19 | "config": {
20 | "initPrimaryKeyNumBuckets": 8,
21 | "batchSize": 10,
22 | "archivingDelayMinutes": 10,
23 | "archivingIntervalMinutes": 10,
24 | "backfillMaxBufferSize": 4294967296,
25 | "backfillIntervalMinutes": 60,
26 | "backfillThresholdInBytes": 2097152
27 | },
28 | "archivingSortColumns": [
29 | 0,
30 | 1
31 | ]
32 | }
--------------------------------------------------------------------------------
/testing/data/query/hll:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/query/hll
--------------------------------------------------------------------------------
/testing/data/query/hll_empty_results:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/query/hll_empty_results
--------------------------------------------------------------------------------
/testing/data/query/hll_query_results:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/uber/aresdb/a8d2aedc6850b10a6cc9381ba780800290b2756d/testing/data/query/hll_query_results
--------------------------------------------------------------------------------
/testing/data/upsert-batches/backfill/upsertBatch0:
--------------------------------------------------------------------------------
1 | columns:
2 | - column_id: 0
3 | data_type: Uint32
4 | - column_id: 1
5 | data_type: Uint32
6 | - column_id: 2
7 | data_type: Uint32
8 | # changes on sort column.
9 | - column_id: 5
10 | data_type: Uint32
11 | rows:
12 | - 0;0;0;0 # value does not change
13 | - 1;0;1;11 # value changed
14 | - 11;11;0;11 # new entry
15 |
--------------------------------------------------------------------------------
/testing/data/upsert-batches/backfill/upsertBatch1:
--------------------------------------------------------------------------------
1 | columns:
2 | - column_id: 0
3 | data_type: Uint32
4 | - column_id: 1
5 | data_type: Uint32
6 | - column_id: 2
7 | data_type: Uint32
8 | # deleted column
9 | - column_id: 3
10 | data_type: Uint32
11 | # changes on unsort column.
12 | - column_id: 4
13 | data_type: Uint32
14 | rows:
15 | - 1;0;1;0;21 # value changed on unsort column, but it's in temp live store.
16 | - 2;1;2;10;12 # fork the column.
17 | - 3;1;3;11;13 # apply changes on the forked column
18 |
--------------------------------------------------------------------------------
/testing/data/upsert-batches/backfill/upsertBatch2:
--------------------------------------------------------------------------------
1 | columns:
2 | - column_id: 0
3 | data_type: Uint32
4 | - column_id: 1
5 | data_type: Uint32
6 | - column_id: 2
7 | data_type: Uint32
8 | # changes on sort column.
9 | - column_id: 5
10 | data_type: Uint32
11 | # changes on unsort column.
12 | - column_id: 4
13 | data_type: Uint32
14 |
15 | rows:
16 | - 4;2;4;12;14 # value changed on unsort column, but sort column is already changed.
17 |
--------------------------------------------------------------------------------
/testing/data/upsert-batches/backfill/upsertBatch3:
--------------------------------------------------------------------------------
1 | columns:
2 | - column_id: 0
3 | data_type: Uint32
4 | - column_id: 1
5 | data_type: Uint32
6 | - column_id: 2
7 | data_type: Uint32
8 | - column_id: 4
9 | data_type: Uint32
10 | - column_id: 5
11 | data_type: Uint32
12 | - column_id: 6
13 | data_type: Int16[]
14 | rows:
15 | - 5;3;5;5;3;53,54 # array column change, while keep the same size, so should be in-place update
16 | - 6;3;6;6;3;61,62,63 # array column change, size increase, should be same as sort column change
17 | - 7;3;7;7;3;71 # array column change, size decrease, should be same as sort column change
18 | - 8;4;8;88;4;83,84 # array column change together with unsorted column change, should be in-place update
19 | - 9;4;9;9;94;94,95,96 # array column change, size change together with sort column change
20 | - 12;12;0;12;12;120,121 # new record
21 |
--------------------------------------------------------------------------------
/testing/data/upsert-batches/testReadUpsertBatch:
--------------------------------------------------------------------------------
1 | columns:
2 | - column_id: 2
3 | data_type: Uint16
4 | - column_id: 1
5 | data_type: Bool
6 | rows:
7 | - 16;1
8 | - null;0
9 | - 0;1
10 |
--------------------------------------------------------------------------------
/testing/data/vectors/invalid_bool_value:
--------------------------------------------------------------------------------
1 | data_type: Bool
2 | length: 6
3 | values:
4 | - invalid
5 | - false
6 | - null
7 | - false
8 | - true
9 | - null
--------------------------------------------------------------------------------
/testing/data/vectors/v0:
--------------------------------------------------------------------------------
1 | data_type: Bool
2 | length: 6
3 | values:
4 | - true
5 | - false
6 | - null
7 | - false
8 | - true
9 | - null
10 |
--------------------------------------------------------------------------------
/testing/data/vps/archiving/sortedVP0:
--------------------------------------------------------------------------------
1 | data_type: Uint32
2 | length: 5
3 | values:
4 | - 0
5 | - 10
6 | - 20
7 | - 30
8 | - 40
--------------------------------------------------------------------------------
/testing/data/vps/archiving/sortedVP00:
--------------------------------------------------------------------------------
1 | data_type: Uint32
2 | length: 5
3 | has_counts: true
4 | values:
5 | - 0,1
6 | - 10,2
7 | - 20,3
8 | - 30,4
9 | - 40,5
10 |
--------------------------------------------------------------------------------
/testing/data/vps/archiving/sortedVP1:
--------------------------------------------------------------------------------
1 | data_type: Bool
2 | length: 3
3 | has_counts: true
4 | values:
5 | - null,3
6 | - false,4
7 | - true,5
--------------------------------------------------------------------------------
/testing/data/vps/archiving/sortedVP2:
--------------------------------------------------------------------------------
1 | data_type: Float32
2 | length: 5
3 | has_counts: true
4 | values:
5 | - null,1
6 | - 1.1,2
7 | - 1.2,3
8 | - 1.3,4
9 | - null,5
--------------------------------------------------------------------------------
/testing/data/vps/archiving/sortedVP3_array:
--------------------------------------------------------------------------------
1 | data_type: Int16[]
2 | length: 5
3 | values:
4 | - 311,312
5 | - 421,null
6 | - 531
7 | - null,541
8 | - null
9 |
--------------------------------------------------------------------------------
/testing/data/vps/archiving/sortedVP4_array:
--------------------------------------------------------------------------------
1 | data_type: UUID[]
2 | length: 5
3 | values:
4 | - 03000000-0000-0000-0100-000000000000,03000000-0000-0000-0200-000000000000
5 | - 04000000-0000-0000-0100-000000000000,null
6 | - 05000000-0000-0000-0100-000000000000
7 | - null,06000000-0000-0000-0100-000000000000
8 | - null
9 |
--------------------------------------------------------------------------------
/testing/data/vps/archiving/vp00:
--------------------------------------------------------------------------------
1 | data_type: Uint32
2 | length: 5
3 | values:
4 | - 100
5 | - 110
6 | - 120
7 | - 130
8 | - 140
--------------------------------------------------------------------------------
/testing/data/vps/archiving/vp01:
--------------------------------------------------------------------------------
1 | data_type: Bool
2 | length: 5
3 | values:
4 | - false
5 | - true
6 | - false
7 | - null
8 | - null
--------------------------------------------------------------------------------
/testing/data/vps/archiving/vp02:
--------------------------------------------------------------------------------
1 | data_type: Float32
2 | length: 5
3 | values:
4 | - null
5 | - 1.1
6 | - 1.2
7 | - 1.3
8 | - null
--------------------------------------------------------------------------------
/testing/data/vps/archiving/vp03_array:
--------------------------------------------------------------------------------
1 | data_type: Int16[]
2 | length: 5
3 | values:
4 | - 11,12,13
5 | - 21,null
6 | - null
7 | - 31,null,33
8 | - 41
9 |
--------------------------------------------------------------------------------
/testing/data/vps/archiving/vp04_array:
--------------------------------------------------------------------------------
1 | data_type: UUID[]
2 | length: 5
3 | values:
4 | - 01000000-0000-0000-0100-000000000000,01000000-0000-0000-0200-000000000000,01000000-0000-0000-0300-000000000000
5 | - 02000000-0000-0000-0100-000000000000,null
6 | - null
7 | - 03000000-0000-0000-0100-000000000000,null,03000000-0000-0000-0200-000000000000
8 | - 04000000-0000-0000-0100-000000000000
9 |
--------------------------------------------------------------------------------
/testing/data/vps/archiving/vp10:
--------------------------------------------------------------------------------
1 | data_type: Uint32
2 | length: 6
3 | values:
4 | - 90
5 | - 100
6 | - 110
7 | - 120
8 | - 130
9 | - 140
--------------------------------------------------------------------------------
/testing/data/vps/archiving/vp11:
--------------------------------------------------------------------------------
1 | data_type: Bool
2 | length: 6
3 | values:
4 | - null
5 | - false
6 | - true
7 | - null
8 | - false
9 | - null
--------------------------------------------------------------------------------
/testing/data/vps/archiving/vp12:
--------------------------------------------------------------------------------
1 | data_type: Float32
2 | length: 6
3 | values:
4 | - null
5 | - 1.0
6 | - null
7 | - 1.2
8 | - 1.3
9 | - null
--------------------------------------------------------------------------------
/testing/data/vps/archiving/vp13:
--------------------------------------------------------------------------------
1 | data_type: Uint32
2 | length: 5
3 | values:
4 | - 0
5 | - 1
6 | - 2
7 | - 3
8 | - 1
--------------------------------------------------------------------------------
/testing/data/vps/archiving/vp13_array:
--------------------------------------------------------------------------------
1 | data_type: Int16[]
2 | length: 6
3 | values:
4 | - null,null
5 | - null,121
6 | - null
7 | - 131,132,143
8 | - 141,142,143
9 | - 151
10 |
--------------------------------------------------------------------------------
/testing/data/vps/archiving/vp14:
--------------------------------------------------------------------------------
1 | data_type: Uint32
2 | length: 6
3 | values:
4 | - 90
5 | - 100
6 | - 110
7 | - null
8 | - null
9 | - 140
--------------------------------------------------------------------------------
/testing/data/vps/archiving/vp14_array:
--------------------------------------------------------------------------------
1 | data_type: UUID[]
2 | length: 6
3 | values:
4 | - null,null
5 | - null,12000000-0000-0000-0100-000000000000
6 | - null
7 | - 13000000-0000-0000-0100-000000000000,13000000-0000-0000-0200-000000000000,14000000-0000-0000-0300-000000000000
8 | - 14000000-0000-0000-0100-000000000000,14000000-0000-0000-0200-000000000000,14000000-0000-0000-0300-000000000000
9 | - 15000000-0000-0000-0100-000000000000
10 |
--------------------------------------------------------------------------------
/testing/data/vps/backfill/backfillBase0:
--------------------------------------------------------------------------------
1 | data_type: Uint32
2 | length: 10
3 | has_counts: false
4 | values:
5 | - 0
6 | - 1
7 | - 2
8 | - 3
9 | - 4
10 | - 5
11 | - 6
12 | - 7
13 | - 8
14 | - 9
15 |
16 |
--------------------------------------------------------------------------------
/testing/data/vps/backfill/backfillBase1:
--------------------------------------------------------------------------------
1 | # sort columns, pk
2 | data_type: Uint32
3 | length: 5
4 | has_counts: true
5 | values:
6 | - 0,2
7 | - 1,4
8 | - 2,5
9 | - 3,8
10 | - 4,10
11 |
12 |
--------------------------------------------------------------------------------
/testing/data/vps/backfill/backfillBase2:
--------------------------------------------------------------------------------
1 | # primary key
2 | data_type: Uint32
3 | length: 10
4 | has_counts: false
5 | values:
6 | - 0
7 | - 1
8 | - 2
9 | - 3
10 | - 4
11 | - 5
12 | - 6
13 | - 7
14 | - 8
15 | - 9
16 |
17 |
--------------------------------------------------------------------------------
/testing/data/vps/backfill/backfillBase3:
--------------------------------------------------------------------------------
1 | data_type: Uint32
2 | length: 0
3 | has_counts: false
4 | values:
5 |
--------------------------------------------------------------------------------
/testing/data/vps/backfill/backfillBase4:
--------------------------------------------------------------------------------
1 | data_type: Uint32
2 | length: 10
3 | has_counts: false
4 | values:
5 | - 0
6 | - 1
7 | - 2
8 | - 3
9 | - 4
10 | - 5
11 | - 6
12 | - 7
13 | - 8
14 | - 9
15 |
16 |
--------------------------------------------------------------------------------
/testing/data/vps/backfill/backfillBase5:
--------------------------------------------------------------------------------
1 | # sort columns
2 | data_type: Uint32
3 | length: 5
4 | has_counts: true
5 | values:
6 | - 0,2
7 | - 1,4
8 | - 2,5
9 | - 3,8
10 | - 4,10
11 |
--------------------------------------------------------------------------------
/testing/data/vps/backfill/backfillBase6:
--------------------------------------------------------------------------------
1 | # array column
2 | data_type: Int16[]
3 | length: 10
4 | has_counts: true
5 | values:
6 | - 1,2,3
7 | - 11,12
8 | - 21
9 | - 31,32
10 | - 41,42,43
11 | - 51,52
12 | - 61,62
13 | - 71,72
14 | - 81,82
15 | - 91,92
16 |
--------------------------------------------------------------------------------
/testing/data/vps/backfill/backfillNew0:
--------------------------------------------------------------------------------
1 | data_type: Uint32
2 | length: 12
3 | has_counts: false
4 | values:
5 | - 0
6 | - 1
7 | - 2
8 | - 3
9 | - 4
10 | - 5
11 | - 6
12 | - 7
13 | - 8
14 | - 9
15 | - 11
16 | - 12
17 |
18 |
--------------------------------------------------------------------------------
/testing/data/vps/backfill/backfillNew1:
--------------------------------------------------------------------------------
1 | # sort columns, pk
2 | data_type: Uint32
3 | length: 7
4 | has_counts: true
5 | values:
6 | - 0,2
7 | - 1,4
8 | - 2,5
9 | - 3,8
10 | - 4,10
11 | - 11,11
12 | - 12,12
13 |
14 |
--------------------------------------------------------------------------------
/testing/data/vps/backfill/backfillNew2:
--------------------------------------------------------------------------------
1 | # primary key
2 | data_type: Uint32
3 | length: 12
4 | has_counts: false
5 | values:
6 | - 0
7 | - 1
8 | - 2
9 | - 3
10 | - 4
11 | - 5
12 | - 6
13 | - 7
14 | - 8
15 | - 9
16 | - 0
17 | - 0
18 |
--------------------------------------------------------------------------------
/testing/data/vps/backfill/backfillNew3:
--------------------------------------------------------------------------------
1 | data_type: Uint32
2 | length: 12
3 | has_counts: false
4 | values:
5 |
--------------------------------------------------------------------------------
/testing/data/vps/backfill/backfillNew4:
--------------------------------------------------------------------------------
1 | # primary key
2 | data_type: Uint32
3 | length: 12
4 | has_counts: false
5 | values:
6 | - 0
7 | - 21
8 | - 12
9 | - 13
10 | - 14
11 | - 5
12 | - 6
13 | - 7
14 | - 88
15 | - 9
16 | - null
17 | - 12
18 |
19 |
--------------------------------------------------------------------------------
/testing/data/vps/backfill/backfillNew5:
--------------------------------------------------------------------------------
1 | # sort columns, pk
2 | data_type: Uint32
3 | length: 9
4 | has_counts: true
5 | values:
6 | - 0,1
7 | - 11,2
8 | - 1,4
9 | - 12,5
10 | - 3,8
11 | - 4,9
12 | - 94,10
13 | - 11,11
14 | - 12,12
15 |
--------------------------------------------------------------------------------
/testing/data/vps/backfill/backfillNew6:
--------------------------------------------------------------------------------
1 | # array column
2 | data_type: Int16[]
3 | length: 12
4 | has_counts: true
5 | values:
6 | - 1,2,3
7 | - 11,12
8 | - 21
9 | - 31,32
10 | - 41,42,43
11 | - 53,54
12 | - 61,62,63
13 | - 71
14 | - 83,84
15 | - 94,95,96
16 | - null
17 | - 120,121
18 |
--------------------------------------------------------------------------------
/testing/data/vps/backfill/backfillTempLiveStore0:
--------------------------------------------------------------------------------
1 | data_type: Uint32
2 | length: 7
3 | has_counts: false
4 | values:
5 | - 1
6 | - 11
7 | - 4
8 | - 6
9 | - 7
10 | - 9
11 | - 12
12 |
13 |
--------------------------------------------------------------------------------
/testing/data/vps/backfill/backfillTempLiveStore1:
--------------------------------------------------------------------------------
1 | data_type: Uint32
2 | length: 7
3 | has_counts: false
4 | values:
5 | - 0
6 | - 11
7 | - 2
8 | - 3
9 | - 3
10 | - 4
11 | - 12
12 |
13 |
--------------------------------------------------------------------------------
/testing/data/vps/backfill/backfillTempLiveStore2:
--------------------------------------------------------------------------------
1 | data_type: Uint32
2 | length: 7
3 | has_counts: false
4 | values:
5 | - 1
6 | - 0
7 | - 4
8 | - 6
9 | - 7
10 | - 9
11 | - 0
12 |
13 |
--------------------------------------------------------------------------------
/testing/data/vps/backfill/backfillTempLiveStore4:
--------------------------------------------------------------------------------
1 | data_type: Uint32
2 | length: 7
3 | has_counts: false
4 | values:
5 | - 21
6 | - null
7 | - 14
8 | - 6
9 | - 7
10 | - 9
11 | - 12
12 |
--------------------------------------------------------------------------------
/testing/data/vps/backfill/backfillTempLiveStore5:
--------------------------------------------------------------------------------
1 | data_type: Uint32
2 | length: 7
3 | has_counts: false
4 | values:
5 | - 11
6 | - 11
7 | - 12
8 | - 3
9 | - 3
10 | - 94
11 | - 12
12 |
--------------------------------------------------------------------------------
/testing/data/vps/backfill/backfillTempLiveStore6:
--------------------------------------------------------------------------------
1 | data_type: Int16[]
2 | length: 7
3 | has_counts: false
4 | values:
5 | - 11,12
6 | - null
7 | - 41,42,43
8 | - 61,62,63
9 | - 71
10 | - 94,95,96
11 | - 120,121
12 |
--------------------------------------------------------------------------------
/testing/data/vps/backfill/buildIndex0:
--------------------------------------------------------------------------------
1 | data_type: Uint32
2 | length: 3
3 | has_counts: false
4 | values:
5 | - 0
6 | - 1
7 | - 2
8 |
9 |
--------------------------------------------------------------------------------
/testing/data/vps/backfill/buildIndex1:
--------------------------------------------------------------------------------
1 | data_type: Uint8
2 | length: 2
3 | has_counts: true
4 | values:
5 | - 0,2
6 | - 1,3
7 |
8 |
--------------------------------------------------------------------------------
/testing/data/vps/backfill/buildIndex2:
--------------------------------------------------------------------------------
1 | data_type: Uint8
2 | length: 3
3 | has_counts: true
4 | values:
5 | - 0,1
6 | - 1,2
7 | - 2,3
8 |
9 |
--------------------------------------------------------------------------------
/testing/data/vps/backfill/buildIndex3:
--------------------------------------------------------------------------------
1 | data_type: Uint8
2 | length: 3
3 | has_counts: false
4 | values:
5 | - 0
6 | - 1
7 | - 2
8 |
9 |
--------------------------------------------------------------------------------
/testing/data/vps/backfill/cloneVPForWrite0:
--------------------------------------------------------------------------------
1 | # time column, all values valid.
2 | data_type: Uint32
3 | length: 3
4 | has_counts: false
5 | values:
6 | - 0
7 | - 1
8 | - 2
9 |
10 |
--------------------------------------------------------------------------------
/testing/data/vps/backfill/cloneVPForWrite1:
--------------------------------------------------------------------------------
1 | data_type: Uint32
2 | length: 3
3 | has_counts: true
4 | values:
5 | - 0,1
6 | - 1,2
7 | - 2,3
8 |
9 |
--------------------------------------------------------------------------------
/testing/data/vps/backfill/cloneVPForWrite2:
--------------------------------------------------------------------------------
1 | # all values null
2 | data_type: Uint32
3 | length: 0
4 | has_counts: false
5 | values:
6 |
7 |
--------------------------------------------------------------------------------
/testing/data/vps/backfill/cloneVPForWrite3:
--------------------------------------------------------------------------------
1 | # has null vector
2 | data_type: Uint32
3 | length: 3
4 | has_counts: false
5 | values:
6 | - 0
7 | - null
8 | - 2
9 |
--------------------------------------------------------------------------------
/testing/data/vps/host-memory-manager/c1:
--------------------------------------------------------------------------------
1 | data_type: Int8
2 | length: 4
3 | has_counts: false
4 | values:
5 | - 1
6 | - null
7 | - 2
8 | - 3
9 |
--------------------------------------------------------------------------------
/testing/data/vps/invalid_value_length:
--------------------------------------------------------------------------------
1 | data_type: Bool
2 | length: 3
3 | has_counts: true
4 | values:
5 | - null,0
6 | - false,3
7 | - true,4
8 | - null,5
--------------------------------------------------------------------------------
/testing/data/vps/list/length_not_match:
--------------------------------------------------------------------------------
1 | data_type: Uint32
2 | length: 5
3 | values:
4 | - 1
5 | - 1,null,3
6 | -
7 | - null
--------------------------------------------------------------------------------
/testing/data/vps/list/live_vp_bool:
--------------------------------------------------------------------------------
1 | data_type: Bool[]
2 | length: 4
3 | values:
4 | - true
5 | - true,null,false
6 | -
7 | - "null"
8 |
--------------------------------------------------------------------------------
/testing/data/vps/list/live_vp_uint32:
--------------------------------------------------------------------------------
1 | data_type: Uint32[]
2 | length: 4
3 | values:
4 | - 1
5 | - 1,null,3
6 | -
7 | - "null"
8 |
--------------------------------------------------------------------------------
/testing/data/vps/list/live_vp_uuid:
--------------------------------------------------------------------------------
1 | data_type: UUID[]
2 | length: 5
3 | values:
4 | - 03000000-0000-0100-0000-000000000000,03000000-0000-0200-0000-000000000000
5 | - 04000000-0000-0100-0000-000000000000,null
6 | - 05000000-0000-0100-0000-000000000000
7 | - null,06000000-0000-0100-0000-000000000000
8 | - null
9 |
--------------------------------------------------------------------------------
/testing/data/vps/list/unknown_data_type:
--------------------------------------------------------------------------------
1 | data_type: Unknown
2 | length: 5
3 | values:
4 | - 1
5 | - 1,null,3
6 | -
7 | - null
--------------------------------------------------------------------------------
/testing/data/vps/merge-nil-base/mergedVP0:
--------------------------------------------------------------------------------
1 | data_type: Uint32
2 | length: 8
3 | has_counts: false
4 | values:
5 | - 100
6 | - 110
7 | - 120
8 | - 130
9 | - 140
10 | - 150
11 | - 160
12 | - 170
13 |
14 |
--------------------------------------------------------------------------------
/testing/data/vps/merge-nil-base/mergedVP1:
--------------------------------------------------------------------------------
1 | data_type: Bool
2 | length: 3
3 | has_counts: true
4 | values:
5 | - null,4
6 | - false,6
7 | - true,8
8 |
9 |
--------------------------------------------------------------------------------
/testing/data/vps/merge-nil-base/mergedVP2:
--------------------------------------------------------------------------------
1 | data_type: Float32
2 | length: 5
3 | has_counts: true
4 | values:
5 | - -0.1,2
6 | - 0.0,3
7 | - 0.1,4
8 | - null,6
9 | - 0.1,8
10 |
11 |
12 |
--------------------------------------------------------------------------------
/testing/data/vps/merge-nil-base/mergedVP3:
--------------------------------------------------------------------------------
1 | data_type: Bool
2 | length: 5
3 | has_counts: false
4 | values:
5 | - null
6 | - null
7 | - null
8 | - null
9 | - null
10 |
--------------------------------------------------------------------------------
/testing/data/vps/merge-nil-base/mergedVP4:
--------------------------------------------------------------------------------
1 | data_type: Uint32
2 | length: 7
3 | has_counts: true
4 | values:
5 | - 1,1
6 | - 2,2
7 | - 2,3
8 | - 2,4
9 | - 1,6
10 | - null,7
11 | - 2,8
--------------------------------------------------------------------------------
/testing/data/vps/merge-nil-base/mergedVP5:
--------------------------------------------------------------------------------
1 | data_type: Bool
2 | length: 8
3 | has_counts: false
4 | values:
5 | - null
6 | - null
7 | - null
8 | - null
9 | - null
10 | - null
11 | - null
12 | - null
13 |
--------------------------------------------------------------------------------
/testing/data/vps/merge-with-deleted-columns/mergedVP5:
--------------------------------------------------------------------------------
1 | data_type: Bool
2 | length: 13
3 | has_counts: false
4 | values:
5 | - null
6 | - null
7 | - null
8 | - null
9 | - null
10 | - null
11 | - null
12 | - null
13 | - null
14 | - null
15 | - null
16 | - null
17 | - null
18 |
--------------------------------------------------------------------------------
/testing/data/vps/merge-with-deleted-rows/mergedVP0:
--------------------------------------------------------------------------------
1 | data_type: Uint32
2 | length: 10
3 | has_counts: false
4 | values:
5 | - 100
6 | - 110
7 | - 120
8 | - 10
9 | - 130
10 | - 140
11 | - 150
12 | - 30
13 | - 160
14 | - 170
15 |
16 |
--------------------------------------------------------------------------------
/testing/data/vps/merge-with-deleted-rows/mergedVP1:
--------------------------------------------------------------------------------
1 | data_type: Bool
2 | length: 3
3 | has_counts: true
4 | values:
5 | - null,5
6 | - false,8
7 | - true,10
8 |
9 |
--------------------------------------------------------------------------------
/testing/data/vps/merge-with-deleted-rows/mergedVP10:
--------------------------------------------------------------------------------
1 | data_type: Uint32
2 | length: 6
3 | has_counts: false
4 | values:
5 | - 10
6 | - 30
7 | - 140
8 | - 150
9 | - 160
10 | - 170
11 |
12 |
--------------------------------------------------------------------------------
/testing/data/vps/merge-with-deleted-rows/mergedVP11:
--------------------------------------------------------------------------------
1 | data_type: Bool
2 | length: 3
3 | has_counts: true
4 | values:
5 | - null,1
6 | - false,4
7 | - true,6
--------------------------------------------------------------------------------
/testing/data/vps/merge-with-deleted-rows/mergedVP2:
--------------------------------------------------------------------------------
1 | data_type: Float32
2 | length: 7
3 | has_counts: true
4 | values:
5 | - -0.1,2
6 | - 0.0,3
7 | - 0.05,4
8 | - 0.1,5
9 | - null,7
10 | - 0.1,8
11 | - 0.1,10
12 |
13 |
14 |
--------------------------------------------------------------------------------
/testing/data/vps/merge-with-deleted-rows/mergedVP3:
--------------------------------------------------------------------------------
1 | data_type: Bool
2 | length: 7
3 | has_counts: false
4 | values:
5 | - null
6 | - null
7 | - null
8 | - null
9 | - null
10 | - null
11 | - null
12 |
--------------------------------------------------------------------------------
/testing/data/vps/merge-with-deleted-rows/mergedVP4:
--------------------------------------------------------------------------------
1 | data_type: Uint32
2 | length: 9
3 | has_counts: true
4 | values:
5 | - 1,1
6 | - 2,2
7 | - 2,3
8 | - 2,4
9 | - 2,5
10 | - 1,7
11 | - 1,8
12 | - null,9
13 | - 2,10
14 |
--------------------------------------------------------------------------------
/testing/data/vps/merge-with-deleted-rows/mergedVP5:
--------------------------------------------------------------------------------
1 | data_type: Bool
2 | length: 10
3 | has_counts: false
4 | values:
5 | - null
6 | - null
7 | - null
8 | - null
9 | - null
10 | - null
11 | - null
12 | - null
13 | - null
14 | - null
15 |
--------------------------------------------------------------------------------
/testing/data/vps/mergedVP0:
--------------------------------------------------------------------------------
1 | data_type: Uint32
2 | length: 13
3 | has_counts: false
4 | values:
5 | - 100
6 | - 110
7 | - 0
8 | - 120
9 | - 10
10 | - 20
11 | - 130
12 | - 140
13 | - 150
14 | - 30
15 | - 40
16 | - 160
17 | - 170
18 |
19 |
--------------------------------------------------------------------------------
/testing/data/vps/mergedVP1:
--------------------------------------------------------------------------------
1 | data_type: Bool
2 | length: 3
3 | has_counts: true
4 | values:
5 | - null,7
6 | - false,10
7 | - true,13
8 |
9 |
--------------------------------------------------------------------------------
/testing/data/vps/mergedVP2:
--------------------------------------------------------------------------------
1 | data_type: Float32
2 | length: 8
3 | has_counts: true
4 | values:
5 | - -0.1,2
6 | - 0.0,4
7 | - 0.05,5
8 | - 0.1,7
9 | - null,9
10 | - 0.1,10
11 | - null,11
12 | - 0.1,13
13 |
14 |
15 |
--------------------------------------------------------------------------------
/testing/data/vps/mergedVP3:
--------------------------------------------------------------------------------
1 | data_type: Bool
2 | length: 8
3 | has_counts: false
4 | values:
5 | - null
6 | - null
7 | - null
8 | - null
9 | - null
10 | - null
11 | - null
12 | - null
13 |
--------------------------------------------------------------------------------
/testing/data/vps/mergedVP4:
--------------------------------------------------------------------------------
1 | data_type: Uint32
2 | length: 11
3 | has_counts: true
4 | values:
5 | - 1,1
6 | - 2,2
7 | - 1,3
8 | - 2,4
9 | - 2,5
10 | - 2,7
11 | - 1,9
12 | - 1,10
13 | - 1,11
14 | - null,12
15 | - 2,13
--------------------------------------------------------------------------------
/testing/data/vps/mergedVP5:
--------------------------------------------------------------------------------
1 | data_type: Bool
2 | length: 13
3 | has_counts: false
4 | values:
5 | - null
6 | - null
7 | - null
8 | - null
9 | - null
10 | - null
11 | - null
12 | - null
13 | - null
14 | - null
15 | - null
16 | - null
17 | - null
18 |
--------------------------------------------------------------------------------
/testing/data/vps/no-sort-columns/baseVP0:
--------------------------------------------------------------------------------
1 | data_type: Uint32
2 | length: 4
3 | has_counts: false
4 | values:
5 | - 0
6 | - 10
7 | - 20
8 | - 30
9 |
--------------------------------------------------------------------------------
/testing/data/vps/no-sort-columns/mergedVP0:
--------------------------------------------------------------------------------
1 | data_type: Uint32
2 | length: 8
3 | has_counts: false
4 | values:
5 | - 0
6 | - 10
7 | - 20
8 | - 30
9 | - 100
10 | - 110
11 | - 120
12 | - 130
13 |
--------------------------------------------------------------------------------
/testing/data/vps/no-sort-columns/mergedVP1:
--------------------------------------------------------------------------------
1 | data_type: Bool
2 | length: 8
3 | has_counts: false
4 | values:
5 | - null
6 | - null
7 | - null
8 | - null
9 | - null
10 | - null
11 | - false
12 | - true
13 |
--------------------------------------------------------------------------------
/testing/data/vps/no-sort-columns/patchVP0:
--------------------------------------------------------------------------------
1 | data_type: Uint32
2 | length: 4
3 | has_counts: false
4 | values:
5 | - 100
6 | - 110
7 | - 120
8 | - 130
9 |
--------------------------------------------------------------------------------
/testing/data/vps/no-sort-columns/patchVP1:
--------------------------------------------------------------------------------
1 | data_type: Bool
2 | length: 4
3 | has_counts: false
4 | values:
5 | - null
6 | - null
7 | - false
8 | - true
9 |
--------------------------------------------------------------------------------
/testing/data/vps/patchVP00:
--------------------------------------------------------------------------------
1 | data_type: Uint32
2 | length: 4
3 | has_counts: false
4 | values:
5 | - 100
6 | - 110
7 | - 120
8 | - 130
9 |
--------------------------------------------------------------------------------
/testing/data/vps/patchVP01:
--------------------------------------------------------------------------------
1 | data_type: Bool
2 | length: 4
3 | has_counts: false
4 | values:
5 | - null
6 | - null
7 | - null
8 | - null
9 |
--------------------------------------------------------------------------------
/testing/data/vps/patchVP02:
--------------------------------------------------------------------------------
1 | data_type: Float32
2 | length: 4
3 | has_counts: false
4 | values:
5 | - -0.1
6 | - -0.1
7 | - 0.0
8 | - 0.1
9 |
--------------------------------------------------------------------------------
/testing/data/vps/patchVP03:
--------------------------------------------------------------------------------
1 | data_type: Bool
2 | length: 4
3 | has_counts: false
4 | values:
5 | - null
6 | - null
7 | - null
8 | - null
9 |
--------------------------------------------------------------------------------
/testing/data/vps/patchVP04:
--------------------------------------------------------------------------------
1 | data_type: Int32
2 | length: 4
3 | has_counts: false
4 | values:
5 | - 1
6 | - 2
7 | - 2
8 | - 2
9 |
--------------------------------------------------------------------------------
/testing/data/vps/patchVP05:
--------------------------------------------------------------------------------
1 | data_type: Bool
2 | length: 4
3 | has_counts: false
4 | values:
5 | - null
6 | - null
7 | - null
8 | - null
9 |
--------------------------------------------------------------------------------
/testing/data/vps/patchVP10:
--------------------------------------------------------------------------------
1 | data_type: Uint32
2 | length: 4
3 | has_counts: false
4 | values:
5 | - 140
6 | - 150
7 | - 160
8 | - 170
9 |
--------------------------------------------------------------------------------
/testing/data/vps/patchVP11:
--------------------------------------------------------------------------------
1 | data_type: Bool
2 | length: 4
3 | has_counts: false
4 | values:
5 | - false
6 | - false
7 | - true
8 | - true
9 |
--------------------------------------------------------------------------------
/testing/data/vps/patchVP12:
--------------------------------------------------------------------------------
1 | data_type: Float32
2 | length: 4
3 | has_counts: false
4 | values:
5 | - null
6 | - null
7 | - 0.1
8 | - 0.1
9 |
--------------------------------------------------------------------------------
/testing/data/vps/patchVP13:
--------------------------------------------------------------------------------
1 | data_type: Bool
2 | length: 4
3 | has_counts: false
4 | values:
5 | - null
6 | - null
7 | - null
8 | - null
9 |
--------------------------------------------------------------------------------
/testing/data/vps/patchVP14:
--------------------------------------------------------------------------------
1 | data_type: Int32
2 | length: 4
3 | has_counts: false
4 | values:
5 | - 1
6 | - 1
7 | - null
8 | - 2
9 |
--------------------------------------------------------------------------------
/testing/data/vps/patchVP15:
--------------------------------------------------------------------------------
1 | data_type: Bool
2 | length: 4
3 | has_counts: false
4 | values:
5 | - null
6 | - null
7 | - null
8 | - null
9 |
--------------------------------------------------------------------------------
/testing/data/vps/serializer/mode0_int8:
--------------------------------------------------------------------------------
1 | data_type: Int8
2 | length: 4
3 | has_counts: false
4 | values:
5 | - null
6 | - null
7 | - null
8 | - null
9 |
--------------------------------------------------------------------------------
/testing/data/vps/serializer/mode1_bool:
--------------------------------------------------------------------------------
1 | data_type: Bool
2 | length: 4
3 | has_counts: false
4 | values:
5 | - false
6 | - false
7 | - true
8 | - true
9 |
--------------------------------------------------------------------------------
/testing/data/vps/serializer/mode2_int8:
--------------------------------------------------------------------------------
1 | data_type: Int8
2 | length: 4
3 | has_counts: false
4 | values:
5 | - 1
6 | - null
7 | - 2
8 | - 3
9 |
--------------------------------------------------------------------------------
/testing/data/vps/serializer/mode3_int8:
--------------------------------------------------------------------------------
1 | data_type: Int8
2 | length: 4
3 | has_counts: true
4 | values:
5 | - 1,1
6 | - 2,2
7 | - 3,3
8 | - 4,4
9 |
--------------------------------------------------------------------------------
/testing/data/vps/sortedVP0:
--------------------------------------------------------------------------------
1 | data_type: Uint32
2 | length: 5
3 | has_counts: false
4 | values:
5 | - 0
6 | - 10
7 | - 20
8 | - 30
9 | - 40
10 |
--------------------------------------------------------------------------------
/testing/data/vps/sortedVP1:
--------------------------------------------------------------------------------
1 | data_type: Bool
2 | length: 3
3 | has_counts: true
4 | values:
5 | - null,3
6 | - false,4
7 | - true,5
8 |
--------------------------------------------------------------------------------
/testing/data/vps/sortedVP2:
--------------------------------------------------------------------------------
1 | data_type: Float32
2 | length: 5
3 | has_counts: true
4 | values:
5 | - 0.0,1
6 | - 0.05,2
7 | - 0.1,3
8 | - 0.1,4
9 | - null,5
10 |
--------------------------------------------------------------------------------
/testing/data/vps/sortedVP3:
--------------------------------------------------------------------------------
1 | data_type: Bool
2 | length: 5
3 | has_counts: false
4 | values:
5 | - null
6 | - null
7 | - null
8 | - null
9 | - null
10 |
--------------------------------------------------------------------------------
/testing/data/vps/sortedVP4:
--------------------------------------------------------------------------------
1 | data_type: Uint32
2 | length: 5
3 | has_counts: true
4 | values:
5 | - 1,1
6 | - 2,2
7 | - 2,3
8 | - 1,4
9 | - 1,5
10 |
--------------------------------------------------------------------------------
/testing/data/vps/sortedVP5:
--------------------------------------------------------------------------------
1 | data_type: Bool
2 | length: 5
3 | has_counts: false
4 | values:
5 | - null
6 | - null
7 | - null
8 | - null
9 | - null
10 |
--------------------------------------------------------------------------------
/testing/data/vps/sortedVP6:
--------------------------------------------------------------------------------
1 | data_type: Uint16
2 | length: 4
3 | has_counts: true
4 | values:
5 | - 1,2
6 | - 2,5
7 | - 1,10
8 | - 2,20
9 |
--------------------------------------------------------------------------------
/testing/data/vps/sortedVP7:
--------------------------------------------------------------------------------
1 | data_type: Uint32
2 | length: 2
3 | has_counts: true
4 | values:
5 | - 1,5
6 | - 2,20
7 |
--------------------------------------------------------------------------------
/testing/data/vps/sortedVP8:
--------------------------------------------------------------------------------
1 | data_type: Uint8
2 | length: 6
3 | has_counts: true
4 | values:
5 | - 1,2
6 | - 1,3
7 | - 2,4
8 | - 3,5
9 | - 2,10
10 | - 2,20
11 |
--------------------------------------------------------------------------------
/utils/ares_suite_test.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package utils
16 |
17 | import (
18 | "github.com/onsi/ginkgo"
19 | "github.com/onsi/gomega"
20 |
21 | "testing"
22 |
23 | "github.com/onsi/ginkgo/reporters"
24 | )
25 |
26 | func TestUtils(t *testing.T) {
27 | gomega.RegisterFailHandler(ginkgo.Fail)
28 | junitReporter := reporters.NewJUnitReporter("junit.xml")
29 | ginkgo.RunSpecsWithDefaultAndCustomReporters(t, "Ares Utils Suite", []ginkgo.Reporter{junitReporter})
30 | }
31 |
--------------------------------------------------------------------------------
/utils/config.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package utils
16 |
17 | import (
18 | "github.com/spf13/viper"
19 | )
20 |
21 | // BindEnvironments binds environment variables to viper
22 | func BindEnvironments(v *viper.Viper) {
23 | v.SetEnvPrefix("ares")
24 | v.BindEnv("env")
25 | }
26 |
--------------------------------------------------------------------------------
/utils/constants.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package utils
16 |
17 | const (
18 | // GolangMemoryFootprintFactor is a rough estimation of the factor
19 | // of memory occupied by golang against application live data set
20 | // (memory occupied by application objects).
21 | GolangMemoryFootprintFactor = 2
22 | )
23 |
--------------------------------------------------------------------------------
/utils/hll.go:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2017-2018 Uber Technologies, Inc.
2 | //
3 | // Licensed under the Apache License, Version 2.0 (the "License");
4 | // you may not use this file except in compliance with the License.
5 | // You may obtain a copy of the License at
6 | //
7 | // http://www.apache.org/licenses/LICENSE-2.0
8 | //
9 | // Unless required by applicable law or agreed to in writing, software
10 | // distributed under the License is distributed on an "AS IS" BASIS,
11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | // See the License for the specific language governing permissions and
13 | // limitations under the License.
14 |
15 | package utils
16 |
17 | const (
18 | // use lower 14 bits in hash as group
19 | groupBits = 14
20 | // max 16 bits for group
21 | maxGroupBits = 16
22 | )
23 |
24 | // ComputeHLLValue compute hll value based on hash value
25 | func ComputeHLLValue(hash uint64) uint32 {
26 | group := uint32(hash & ((1 << groupBits) - 1))
27 | var rho uint32
28 | for {
29 | h := hash & (1 << (rho + groupBits))
30 | if rho+groupBits < 64 && h == 0 {
31 | rho++
32 | } else {
33 | break
34 | }
35 | }
36 | return rho<