From 61303304e4bc1fa982e4c3918555f1ed84fe1c40 Mon Sep 17 00:00:00 2001 From: Usamoi Date: Sun, 8 May 2022 17:18:45 +0800 Subject: [PATCH] Re-reorganization the common crates. --- Cargo.lock | 53 ++++-------------- Cargo.toml | 4 -- common/ast/tests/it/udfs/udf_parser.rs | 2 +- common/ast/tests/it/udfs/udf_transformer.rs | 2 +- common/base/Cargo.toml | 15 ++++++ common/base/src/{ => base}/format.rs | 0 .../src/{ => base}/http_shutdown_handlers.rs | 3 +- common/base/src/base/mod.rs | 54 +++++++++++++++++++ common/base/src/{ => base}/net.rs | 0 common/base/src/{ => base}/profiling.rs | 0 common/base/src/{ => base}/progress.rs | 0 common/base/src/{ => base}/runtime.rs | 2 +- common/base/src/{ => base}/runtime_tracker.rs | 0 common/base/src/{ => base}/shutdown_signal.rs | 0 common/base/src/{ => base}/stop_handle.rs | 2 +- common/base/src/{ => base}/stoppable.rs | 0 common/base/src/{ => base}/string_func.rs | 0 common/base/src/{ => base}/thread.rs | 2 +- common/base/src/{ => base}/uniq_id.rs | 0 .../src/lib.rs => base/src/containers/mod.rs} | 0 .../src => base/src/containers}/pool.rs | 5 +- .../src => base/src/infallible}/condvar.rs | 0 .../src => base/src/infallible}/exit_guard.rs | 0 .../src/lib.rs => base/src/infallible/mod.rs} | 2 +- .../src => base/src/infallible}/mutex.rs | 0 .../src => base/src/infallible}/rwlock.rs | 2 +- .../src/infallible}/rwlock_upgrade_read.rs | 0 common/base/src/lib.rs | 45 ++-------------- .../src/mem_allocator}/allocators.rs | 10 ++-- .../src/mem_allocator}/malloc_size.rs | 16 +++--- .../lib.rs => base/src/mem_allocator/mod.rs} | 0 .../src/mem_allocator}/profiling.rs | 0 .../src => base/src/mem_allocator}/sizeof.rs | 8 +-- .../src/lib.rs => base/src/rangemap/mod.rs} | 4 +- .../src => base/src/rangemap}/range_map.rs | 14 ++--- .../src/rangemap/range_map_key.rs} | 20 +++---- .../tests/it/derive.rs | 8 +-- common/base/tests/it/format.rs | 2 +- common/base/tests/it/main.rs | 7 +++ .../tests/it/malloc_size.rs | 10 ++-- common/{infallible => base}/tests/it/mutex.rs | 2 +- .../tests => base/tests/it}/pool.rs | 8 +-- common/base/tests/it/progress.rs | 2 +- .../tests/it/range_key_test.rs | 4 +- .../tests/it/range_map_test.rs | 20 +++---- common/base/tests/it/runtime.rs | 2 +- .../{infallible => base}/tests/it/rwlock.rs | 2 +- common/base/tests/it/stoppable.rs | 2 +- common/base/tests/it/string_func.rs | 2 +- common/containers/Cargo.toml | 20 ------- common/containers/tests/main.rs | 15 ------ common/contexts/src/dal/dal_runtime.rs | 2 +- common/datablocks/Cargo.toml | 2 +- common/datablocks/src/memory.rs | 2 +- common/datavalues/Cargo.toml | 2 +- common/grpc/src/dns_resolver.rs | 4 +- common/grpc/tests/it/dns_resolver.rs | 2 +- common/infallible/Cargo.toml | 19 ------- common/infallible/tests/it/main.rs | 16 ------ common/macros/src/async_entrypoint.rs | 4 +- common/macros/src/malloc_sizeof.rs | 10 ++-- common/management/src/cluster/cluster_mgr.rs | 4 +- common/management/src/stage/stage_mgr.rs | 2 +- common/management/src/udf/udf_mgr.rs | 2 +- common/management/src/user/user_mgr.rs | 2 +- common/management/tests/it/cluster.rs | 2 +- common/management/tests/it/setting.rs | 2 +- common/management/tests/it/stage.rs | 2 +- common/management/tests/it/udf.rs | 2 +- common/management/tests/it/user.rs | 4 +- common/mem-allocator/Cargo.toml | 31 ----------- common/mem-allocator/tests/it/main.rs | 16 ------ common/meta/api/src/kv_api_test_suite.rs | 2 +- common/meta/embedded/src/meta_embedded.rs | 2 +- common/meta/embedded/tests/it/kv_api_impl.rs | 2 +- .../meta/embedded/tests/it/schema_api_impl.rs | 2 +- common/meta/grpc/Cargo.toml | 1 - common/meta/grpc/src/grpc_client.rs | 6 +-- common/meta/grpc/tests/it/grpc_client.rs | 2 +- common/meta/grpc/tests/it/grpc_server.rs | 2 +- common/meta/raft-store/tests/it/log.rs | 2 +- common/meta/raft-store/tests/it/state.rs | 2 +- .../raft-store/tests/it/state_machine/mod.rs | 2 +- .../tests/it/state_machine/schema_api_impl.rs | 2 +- common/meta/raft-store/tests/it/testing.rs | 2 +- common/meta/sled-store/tests/it/sled_tree.rs | 2 +- .../meta/sled-store/tests/it/sled_txn_tree.rs | 2 +- .../meta/sled-store/tests/it/testing/mod.rs | 2 +- common/metrics/Cargo.toml | 2 +- common/metrics/src/recorder.rs | 2 +- common/planners/Cargo.toml | 1 - common/range-map/Cargo.toml | 11 ---- common/range-map/tests/it/main.rs | 16 ------ common/streams/Cargo.toml | 1 - common/streams/src/stream_error.rs | 2 +- common/streams/src/stream_progress.rs | 4 +- common/streams/tests/it/sources/source_csv.rs | 2 +- .../streams/tests/it/sources/source_ndjson.rs | 2 +- .../tests/it/sources/source_parquet.rs | 2 +- common/streams/tests/it/stream_cast.rs | 2 +- common/streams/tests/it/stream_datablock.rs | 2 +- common/streams/tests/it/stream_limit_by.rs | 2 +- common/streams/tests/it/stream_progress.rs | 4 +- common/streams/tests/it/stream_skip.rs | 2 +- metasrv/Cargo.toml | 2 - metasrv/src/api/grpc/grpc_service.rs | 2 +- metasrv/src/api/grpc_server.rs | 12 ++--- metasrv/src/api/http/debug/pprof.rs | 4 +- metasrv/src/api/http_service.rs | 6 +-- metasrv/src/bin/metasrv.rs | 6 +-- metasrv/src/configs/config.rs | 2 +- metasrv/src/meta_service/raftmeta.rs | 10 ++-- metasrv/src/metrics/metric_service.rs | 6 +-- metasrv/src/network.rs | 4 +- metasrv/src/store/meta_raft_store.rs | 4 +- metasrv/src/watcher/watcher_manager.rs | 16 +++--- metasrv/src/watcher/watcher_stream.rs | 2 +- .../tests/it/api/http/cluster_state_test.rs | 4 +- metasrv/tests/it/api/http/config.rs | 2 +- metasrv/tests/it/api/http/health.rs | 2 +- metasrv/tests/it/api/http_service.rs | 4 +- metasrv/tests/it/grpc/metasrv_grpc_api.rs | 4 +- metasrv/tests/it/grpc/metasrv_grpc_export.rs | 2 +- metasrv/tests/it/grpc/metasrv_grpc_kv_api.rs | 2 +- .../metasrv_grpc_kv_api_restart_cluster.rs | 4 +- .../tests/it/grpc/metasrv_grpc_schema_api.rs | 2 +- ...tasrv_grpc_schema_api_follower_follower.rs | 2 +- ...metasrv_grpc_schema_api_leader_follower.rs | 2 +- metasrv/tests/it/grpc/metasrv_grpc_tls.rs | 2 +- metasrv/tests/it/grpc/metasrv_grpc_watch.rs | 2 +- metasrv/tests/it/meta_node/meta_node_all.rs | 4 +- .../tests/it/meta_node/meta_node_kv_api.rs | 2 +- metasrv/tests/it/store.rs | 2 +- metasrv/tests/it/tests/service.rs | 6 +-- query/Cargo.toml | 4 +- query/benches/suites/mod.rs | 2 +- query/src/api/http/debug/jeprof.rs | 2 +- query/src/api/http/debug/pprof.rs | 4 +- query/src/api/rpc/flight_client.rs | 2 +- query/src/api/rpc/flight_dispatcher.rs | 8 +-- query/src/api/rpc/flight_service_stream.rs | 6 +-- query/src/api/rpc_service.rs | 8 +-- query/src/bin/databend-query.rs | 2 +- query/src/catalogs/catalog_context.rs | 2 +- query/src/catalogs/table_memory_meta.rs | 2 +- query/src/clusters/cluster.rs | 18 +++---- .../common/service/http_shutdown_handles.rs | 6 +-- query/src/configs/config_meta.rs | 2 +- query/src/configs/config_storage.rs | 2 +- query/src/databases/database_context.rs | 2 +- query/src/databases/database_factory.rs | 2 +- .../interpreter_factory_interceptor.rs | 2 +- query/src/interpreters/interpreter_insert.rs | 2 +- .../plan_schedulers/plan_scheduler_stream.rs | 4 +- .../pipelines/new/executor/executor_notify.rs | 4 +- .../pipelines/new/executor/executor_tasks.rs | 2 +- .../new/executor/executor_worker_context.rs | 2 +- .../executor/pipeline_complete_executor.rs | 2 +- .../new/executor/pipeline_executor.rs | 4 +- .../new/executor/pipeline_pulling_executor.rs | 2 +- .../new/executor/pipeline_pushing_executor.rs | 2 +- .../new/processors/sinks/sync_sink_sender.rs | 2 +- .../new/processors/sources/async_source.rs | 4 +- .../new/processors/sources/blocks_source.rs | 2 +- .../sources/sync_ck_source_receiver.rs | 2 +- .../new/processors/sources/sync_source.rs | 4 +- .../sources/sync_source_receiver.rs | 2 +- .../transforms/transform_create_sets.rs | 2 +- .../pipelines/processors/processor_merge.rs | 4 +- .../pipelines/processors/processor_mixed.rs | 6 +-- .../transforms/transform_create_sets.rs | 6 +-- .../transforms/transform_group_by_final.rs | 2 +- .../servers/clickhouse/clickhouse_handler.rs | 10 ++-- .../servers/clickhouse/clickhouse_session.rs | 8 +-- .../clickhouse/interactive_worker_base.rs | 10 ++-- .../servers/clickhouse/reject_connection.rs | 2 +- .../clickhouse/writers/query_writer.rs | 2 +- .../servers/http/v1/http_query_handlers.rs | 2 +- query/src/servers/http/v1/load.rs | 2 +- .../servers/http/v1/query/execute_state.rs | 10 ++-- .../src/servers/http/v1/query/expiring_map.rs | 6 +-- query/src/servers/http/v1/query/http_query.rs | 8 +-- .../http/v1/query/http_query_manager.rs | 8 +-- .../http/v1/query/result_data_manager.rs | 6 +-- query/src/servers/mysql/mysql_handler.rs | 10 ++-- .../servers/mysql/mysql_interactive_worker.rs | 2 +- query/src/servers/mysql/mysql_session.rs | 8 +-- query/src/servers/mysql/reject_connection.rs | 6 +-- query/src/servers/server.rs | 8 +-- query/src/sessions/query_ctx.rs | 14 ++--- query/src/sessions/query_ctx_shared.rs | 8 +-- query/src/sessions/session.rs | 6 +-- query/src/sessions/session_ctx.rs | 2 +- query/src/sessions/session_info.rs | 2 +- query/src/sessions/session_mgr.rs | 8 +-- query/src/sessions/session_settings.rs | 2 +- query/src/storages/fuse/cache/memory_cache.rs | 2 +- .../src/storages/fuse/io/read/block_reader.rs | 2 +- query/src/storages/fuse/operations/commit.rs | 4 +- query/src/storages/fuse/operations/read.rs | 4 +- query/src/storages/memory/memory_table.rs | 4 +- query/src/storages/s3/s3_stage_source.rs | 2 +- query/src/storages/s3/s3_stage_table.rs | 2 +- query/src/storages/storage_context.rs | 2 +- query/src/storages/storage_factory.rs | 2 +- query/src/storages/system/configs_table.rs | 2 +- query/src/storages/system/processes_table.rs | 2 +- query/src/storages/system/query_log_table.rs | 2 +- .../table_functions/table_function_factory.rs | 2 +- query/src/users/role_cache_mgr.rs | 6 +-- query/tests/it/api/http/cluster.rs | 2 +- query/tests/it/api/http/config.rs | 2 +- query/tests/it/api/http/health.rs | 2 +- query/tests/it/api/http/logs.rs | 2 +- query/tests/it/api/http/status.rs | 2 +- query/tests/it/api/http_service.rs | 4 +- query/tests/it/api/rpc/flight_actions.rs | 2 +- query/tests/it/api/rpc/flight_dispatcher.rs | 2 +- query/tests/it/api/rpc/flight_service.rs | 2 +- query/tests/it/api/rpc/flight_tickets.rs | 2 +- query/tests/it/api/rpc_service.rs | 6 +-- query/tests/it/catalogs/database_catalog.rs | 2 +- query/tests/it/catalogs/immutable_catalogs.rs | 2 +- query/tests/it/clusters.rs | 2 +- query/tests/it/common/expression_evaluator.rs | 2 +- query/tests/it/functions/context_function.rs | 2 +- .../access/management_mode_access.rs | 2 +- .../tests/it/interpreters/interpreter_call.rs | 2 +- .../interpreter_database_create.rs | 2 +- .../interpreters/interpreter_database_drop.rs | 2 +- .../interpreter_database_show_create.rs | 2 +- .../it/interpreters/interpreter_empty.rs | 2 +- .../it/interpreters/interpreter_explain.rs | 2 +- .../interpreter_factory_interceptor.rs | 2 +- .../it/interpreters/interpreter_insert.rs | 2 +- .../interpreter_privilege_grant.rs | 2 +- .../interpreter_privilege_revoke.rs | 2 +- .../it/interpreters/interpreter_role_grant.rs | 2 +- .../interpreters/interpreter_role_revoke.rs | 2 +- .../it/interpreters/interpreter_select.rs | 2 +- .../it/interpreters/interpreter_setting.rs | 2 +- .../interpreter_show_databases.rs | 2 +- .../interpreters/interpreter_show_engines.rs | 2 +- .../interpreter_show_functions.rs | 2 +- .../it/interpreters/interpreter_show_grant.rs | 2 +- .../interpreters/interpreter_show_metrics.rs | 2 +- .../interpreter_show_processlist.rs | 2 +- .../it/interpreters/interpreter_show_roles.rs | 2 +- .../interpreters/interpreter_show_settings.rs | 2 +- .../interpreters/interpreter_show_tab_stat.rs | 2 +- .../interpreters/interpreter_show_tables.rs | 2 +- .../it/interpreters/interpreter_show_users.rs | 2 +- .../interpreters/interpreter_table_create.rs | 2 +- .../interpreter_table_describe.rs | 2 +- .../it/interpreters/interpreter_table_drop.rs | 2 +- .../interpreters/interpreter_table_rename.rs | 2 +- .../interpreter_table_show_create.rs | 2 +- .../interpreter_table_truncate.rs | 2 +- .../interpreters/interpreter_use_database.rs | 2 +- .../it/interpreters/interpreter_user_alter.rs | 2 +- .../interpreters/interpreter_user_create.rs | 2 +- .../it/interpreters/interpreter_user_drop.rs | 2 +- .../it/interpreters/interpreter_user_stage.rs | 2 +- .../interpreter_user_udf_alter.rs | 2 +- .../interpreter_user_udf_create.rs | 2 +- .../interpreters/interpreter_user_udf_drop.rs | 2 +- .../plan_schedulers/plan_scheduler.rs | 2 +- query/tests/it/metrics.rs | 2 +- query/tests/it/optimizers/optimizer.rs | 2 +- .../optimizers/optimizer_constant_folding.rs | 2 +- .../optimizer_expression_transform.rs | 2 +- .../tests/it/optimizers/optimizer_scatters.rs | 2 +- .../optimizers/optimizer_statistics_exact.rs | 2 +- .../optimizers/optimizer_top_n_push_down.rs | 2 +- .../pipelines/new/executor/executor_graph.rs | 8 +-- .../it/pipelines/new/processors/port_test.rs | 2 +- query/tests/it/pipelines/processors/pipe.rs | 2 +- .../pipelines/processors/pipeline_builder.rs | 2 +- .../pipelines/processors/pipeline_display.rs | 2 +- .../pipelines/processors/pipeline_walker.rs | 2 +- .../pipelines/processors/processor_empty.rs | 2 +- .../pipelines/processors/processor_merge.rs | 2 +- .../pipelines/processors/processor_mixed.rs | 2 +- .../transforms/transform_aggregator_final.rs | 2 +- .../transform_aggregator_partial.rs | 2 +- .../transforms/transform_expression.rs | 2 +- .../pipelines/transforms/transform_filter.rs | 2 +- .../transforms/transform_group_by_final.rs | 2 +- .../transforms/transform_group_by_partial.rs | 2 +- .../pipelines/transforms/transform_limit.rs | 2 +- .../transforms/transform_limit_by.rs | 2 +- .../transforms/transform_projection.rs | 2 +- .../it/pipelines/transforms/transform_sort.rs | 2 +- .../pipelines/transforms/transform_source.rs | 2 +- query/tests/it/servers/clickhouse.rs | 2 +- .../it/servers/http/clickhouse_handler.rs | 2 +- .../it/servers/http/http_query_handlers.rs | 4 +- query/tests/it/servers/mysql/mysql_handler.rs | 2 +- query/tests/it/sessions/query_ctx.rs | 2 +- query/tests/it/sessions/session.rs | 4 +- query/tests/it/sessions/session_context.rs | 2 +- query/tests/it/sessions/session_setting.rs | 2 +- query/tests/it/sql/plan_parser.rs | 2 +- .../sql/statements/query/query_normalizer.rs | 2 +- .../query/query_qualified_rewriter.rs | 2 +- .../query/query_schema_joined_analyzer.rs | 2 +- .../tests/it/sql/statements/statement_copy.rs | 2 +- .../sql/statements/statement_create_table.rs | 2 +- .../it/sql/statements/statement_select.rs | 2 +- query/tests/it/sql/statements/value_source.rs | 2 +- query/tests/it/storages/fuse/io.rs | 4 +- .../it/storages/fuse/operations/commit.rs | 2 +- .../it/storages/fuse/operations/optimize.rs | 2 +- .../it/storages/fuse/operations/purge_drop.rs | 2 +- .../fuse/operations/purge_truncate.rs | 2 +- .../it/storages/fuse/operations/read_plan.rs | 2 +- query/tests/it/storages/fuse/pruning.rs | 2 +- query/tests/it/storages/fuse/table.rs | 2 +- .../table_functions/fuse_snapshot_table.rs | 2 +- query/tests/it/storages/index/bloom_filter.rs | 2 +- query/tests/it/storages/index/range_filter.rs | 2 +- query/tests/it/storages/memory.rs | 2 +- query/tests/it/storages/null.rs | 2 +- .../it/storages/system/clusters_table.rs | 2 +- .../tests/it/storages/system/columns_table.rs | 2 +- .../tests/it/storages/system/configs_table.rs | 2 +- .../it/storages/system/contributors_table.rs | 2 +- .../tests/it/storages/system/credits_table.rs | 2 +- .../it/storages/system/databases_table.rs | 2 +- .../tests/it/storages/system/engines_table.rs | 2 +- .../it/storages/system/functions_table.rs | 2 +- .../tests/it/storages/system/metrics_table.rs | 2 +- .../it/storages/system/query_log_table.rs | 2 +- query/tests/it/storages/system/roles_table.rs | 2 +- .../it/storages/system/settings_table.rs | 2 +- .../tests/it/storages/system/tables_table.rs | 2 +- .../tests/it/storages/system/tracing_table.rs | 2 +- query/tests/it/storages/system/users_table.rs | 2 +- .../tests/it/table_functions/numbers_table.rs | 2 +- query/tests/it/tests/sessions.rs | 4 +- query/tests/it/users/auth/auth_mgr.rs | 2 +- query/tests/it/users/role_cache_mgr.rs | 2 +- query/tests/it/users/role_mgr.rs | 2 +- query/tests/it/users/user_mgr.rs | 2 +- query/tests/it/users/user_udf.rs | 2 +- tools/metactl/src/main.rs | 2 +- 346 files changed, 593 insertions(+), 742 deletions(-) rename common/base/src/{ => base}/format.rs (100%) rename common/base/src/{ => base}/http_shutdown_handlers.rs (99%) create mode 100644 common/base/src/base/mod.rs rename common/base/src/{ => base}/net.rs (100%) rename common/base/src/{ => base}/profiling.rs (100%) rename common/base/src/{ => base}/progress.rs (100%) rename common/base/src/{ => base}/runtime.rs (99%) rename common/base/src/{ => base}/runtime_tracker.rs (100%) rename common/base/src/{ => base}/shutdown_signal.rs (100%) rename common/base/src/{ => base}/stop_handle.rs (99%) rename common/base/src/{ => base}/stoppable.rs (100%) rename common/base/src/{ => base}/string_func.rs (100%) rename common/base/src/{ => base}/thread.rs (97%) rename common/base/src/{ => base}/uniq_id.rs (100%) rename common/{containers/src/lib.rs => base/src/containers/mod.rs} (100%) rename common/{containers/src => base/src/containers}/pool.rs (98%) rename common/{infallible/src => base/src/infallible}/condvar.rs (100%) rename common/{infallible/src => base/src/infallible}/exit_guard.rs (100%) rename common/{infallible/src/lib.rs => base/src/infallible/mod.rs} (95%) rename common/{infallible/src => base/src/infallible}/mutex.rs (100%) rename common/{infallible/src => base/src/infallible}/rwlock.rs (97%) rename common/{infallible/src => base/src/infallible}/rwlock_upgrade_read.rs (100%) rename common/{mem-allocator/src => base/src/mem_allocator}/allocators.rs (95%) rename common/{mem-allocator/src => base/src/mem_allocator}/malloc_size.rs (97%) rename common/{mem-allocator/src/lib.rs => base/src/mem_allocator/mod.rs} (100%) rename common/{mem-allocator/src => base/src/mem_allocator}/profiling.rs (100%) rename common/{mem-allocator/src => base/src/mem_allocator}/sizeof.rs (90%) rename common/{range-map/src/lib.rs => base/src/rangemap/mod.rs} (92%) rename common/{range-map/src => base/src/rangemap}/range_map.rs (79%) rename common/{range-map/src/range_key.rs => base/src/rangemap/range_map_key.rs} (76%) rename common/{mem-allocator => base}/tests/it/derive.rs (93%) rename common/{mem-allocator => base}/tests/it/malloc_size.rs (89%) rename common/{infallible => base}/tests/it/mutex.rs (96%) rename common/{containers/tests => base/tests/it}/pool.rs (94%) rename common/{range-map => base}/tests/it/range_key_test.rs (96%) rename common/{range-map => base}/tests/it/range_map_test.rs (76%) rename common/{infallible => base}/tests/it/rwlock.rs (96%) delete mode 100644 common/containers/Cargo.toml delete mode 100644 common/containers/tests/main.rs delete mode 100644 common/infallible/Cargo.toml delete mode 100644 common/infallible/tests/it/main.rs delete mode 100644 common/mem-allocator/Cargo.toml delete mode 100644 common/mem-allocator/tests/it/main.rs delete mode 100644 common/range-map/Cargo.toml delete mode 100644 common/range-map/tests/it/main.rs diff --git a/Cargo.lock b/Cargo.lock index 9bf9ec9f0f5b..0a1523d8d3b8 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -857,17 +857,23 @@ dependencies = [ name = "common-base" version = "0.1.0" dependencies = [ + "anyhow", "async-trait", "common-exception", + "common-macros", "common-tracing", "ctrlc", "futures", "hyper", + "libc", + "parking_lot 0.12.0", "poem", "pprof", "serde", "serde_json", "serde_yaml", + "tikv-jemalloc-ctl", + "tikv-jemalloc-sys", "tokio", "toml", "uuid", @@ -897,16 +903,6 @@ dependencies = [ "walkdir", ] -[[package]] -name = "common-containers" -version = "0.1.0" -dependencies = [ - "anyhow", - "async-trait", - "common-base", - "common-tracing", -] - [[package]] name = "common-contexts" version = "0.1.0" @@ -924,9 +920,9 @@ dependencies = [ "ahash", "comfy-table", "common-arrow", + "common-base", "common-datavalues", "common-exception", - "common-infallible", "common-io", "pretty_assertions", "regex", @@ -939,10 +935,10 @@ dependencies = [ "chrono", "chrono-tz", "common-arrow", + "common-base", "common-exception", "common-io", "common-macros", - "common-mem-allocator", "criterion", "dyn-clone", "enum_dispatch", @@ -1038,13 +1034,6 @@ dependencies = [ "trust-dns-resolver", ] -[[package]] -name = "common-infallible" -version = "0.1.0" -dependencies = [ - "parking_lot 0.12.0", -] - [[package]] name = "common-io" version = "0.1.0" @@ -1089,19 +1078,6 @@ dependencies = [ "serde_json", ] -[[package]] -name = "common-mem-allocator" -version = "0.1.0" -dependencies = [ - "common-base", - "common-infallible", - "common-macros", - "libc", - "parking_lot 0.12.0", - "tikv-jemalloc-ctl", - "tikv-jemalloc-sys", -] - [[package]] name = "common-meta-api" version = "0.1.0" @@ -1146,7 +1122,6 @@ version = "0.1.0" dependencies = [ "common-arrow", "common-base", - "common-containers", "common-exception", "common-grpc", "common-meta-api", @@ -1243,8 +1218,8 @@ dependencies = [ name = "common-metrics" version = "0.1.0" dependencies = [ + "common-base", "common-exception", - "common-infallible", "common-tracing", "metrics", "metrics-exporter-prometheus", @@ -1263,7 +1238,6 @@ dependencies = [ "common-datavalues", "common-exception", "common-functions", - "common-infallible", "common-meta-types", "once_cell", "pretty_assertions", @@ -1297,10 +1271,6 @@ dependencies = [ "tonic-build 0.7.0", ] -[[package]] -name = "common-range-map" -version = "0.1.0" - [[package]] name = "common-streams" version = "0.1.0" @@ -1314,7 +1284,6 @@ dependencies = [ "common-datavalues", "common-exception", "common-functions", - "common-infallible", "common-io", "common-tracing", "csv-async", @@ -1721,7 +1690,6 @@ dependencies = [ "clap 3.1.8", "common-arrow", "common-base", - "common-containers", "common-exception", "common-grpc", "common-macros", @@ -1731,7 +1699,6 @@ dependencies = [ "common-meta-sled-store", "common-meta-types", "common-metrics", - "common-range-map", "common-tracing", "futures", "maplit", @@ -1802,11 +1769,9 @@ dependencies = [ "common-exception", "common-functions", "common-grpc", - "common-infallible", "common-io", "common-macros", "common-management", - "common-mem-allocator", "common-meta-api", "common-meta-embedded", "common-meta-grpc", diff --git a/Cargo.toml b/Cargo.toml index 67fceffc062e..8bedfc7e9a76 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -5,18 +5,15 @@ members = [ "common/base", "common/building", "common/cache", - "common/containers", "common/contexts", "common/datablocks", "common/datavalues", "common/exception", "common/grpc", "common/functions", - "common/infallible", "common/io", "common/macros", "common/management", - "common/mem-allocator", "common/planners", "common/proto-conv", "common/protos", @@ -25,7 +22,6 @@ members = [ "common/meta/raft-store", "common/meta/sled-store", "common/meta/types", - "common/range-map", "common/streams", "common/codegen", "common/tracing", diff --git a/common/ast/tests/it/udfs/udf_parser.rs b/common/ast/tests/it/udfs/udf_parser.rs index 7eb0531722d7..d1af9dfe6be7 100644 --- a/common/ast/tests/it/udfs/udf_parser.rs +++ b/common/ast/tests/it/udfs/udf_parser.rs @@ -13,7 +13,7 @@ // limitations under the License. use common_ast::udfs::*; -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use pretty_assertions::assert_eq; use sqlparser::ast::Expr; diff --git a/common/ast/tests/it/udfs/udf_transformer.rs b/common/ast/tests/it/udfs/udf_transformer.rs index 3c3658f14db9..6b9249fabe25 100644 --- a/common/ast/tests/it/udfs/udf_transformer.rs +++ b/common/ast/tests/it/udfs/udf_transformer.rs @@ -14,7 +14,7 @@ use async_trait::async_trait; use common_ast::udfs::*; -use common_base::tokio; +use common_base::base::tokio; use common_exception::ErrorCode; use common_exception::Result; use pretty_assertions::assert_eq; diff --git a/common/base/Cargo.toml b/common/base/Cargo.toml index e5f469962828..073fb6ffd10b 100644 --- a/common/base/Cargo.toml +++ b/common/base/Cargo.toml @@ -12,6 +12,13 @@ test = false [features] tracing = ["tokio/tracing"] +memory-profiling = [ + "tikv-jemalloc-ctl", + "libc", + "tikv-jemalloc-sys/stats", + "tikv-jemalloc-sys/profiling", + "tikv-jemalloc-sys/unprefixed_malloc_on_supported_platforms", +] [dependencies] # In alphabetical order # Workspace dependencies @@ -25,6 +32,8 @@ async-trait = "0.1.53" ctrlc = { version = "3.2.1", features = ["termination"] } futures = "0.3.21" hyper = "0.14.18" +libc = { version = "0.2", optional = true } +parking_lot = "0.12.0" poem = { version = "=1.3.16", features = ["rustls"] } serde = { version = "1.0.136", features = ["derive"] } serde_json = { version = "1.0.79", default-features = false, features = ["raw_value"] } @@ -35,6 +44,12 @@ pprof = { git = "https://github.com/tikv/pprof-rs", rev = "666d9e2", features = "protobuf-codec", "protobuf", ] } +tikv-jemalloc-ctl = { version = "0.4", optional = true } +tikv-jemalloc-sys = { version = "0.4.3" } tokio = { version = "1.17.0", features = ["full"] } toml = { version = "0.5.8", default-features = false } uuid = { version = "0.8.2", features = ["serde", "v4"] } + +[dev-dependencies] +anyhow = "1.0.56" +common-macros = { path = "../macros" } diff --git a/common/base/src/format.rs b/common/base/src/base/format.rs similarity index 100% rename from common/base/src/format.rs rename to common/base/src/base/format.rs diff --git a/common/base/src/http_shutdown_handlers.rs b/common/base/src/base/http_shutdown_handlers.rs similarity index 99% rename from common/base/src/http_shutdown_handlers.rs rename to common/base/src/base/http_shutdown_handlers.rs index 4d3ec207b8c5..c58c95e2c29f 100644 --- a/common/base/src/http_shutdown_handlers.rs +++ b/common/base/src/base/http_shutdown_handlers.rs @@ -26,11 +26,10 @@ use poem::listener::Listener; use poem::listener::RustlsConfig; use poem::listener::TcpListener; use poem::Endpoint; +use tokio::sync::broadcast; use tokio::sync::oneshot; use tokio::task::JoinHandle; -use crate::tokio::sync::broadcast; - pub struct HttpShutdownHandler { service_name: String, join_handle: Option>>, diff --git a/common/base/src/base/mod.rs b/common/base/src/base/mod.rs new file mode 100644 index 000000000000..c47e032f193f --- /dev/null +++ b/common/base/src/base/mod.rs @@ -0,0 +1,54 @@ +// Copyright 2021 Datafuse Labs. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +mod format; +mod http_shutdown_handlers; +mod net; +mod profiling; +mod progress; +mod runtime; +mod runtime_tracker; +mod shutdown_signal; +mod stop_handle; +mod stoppable; +mod string_func; +mod thread; +mod uniq_id; + +pub use format::Format; +pub use http_shutdown_handlers::HttpShutdownHandler; +pub use net::get_free_tcp_port; +pub use net::get_free_udp_port; +pub use profiling::Profiling; +pub use progress::Progress; +pub use progress::ProgressValues; +pub use runtime::Dropper; +pub use runtime::Runtime; +pub use runtime::TrySpawn; +pub use runtime_tracker::RuntimeTracker; +pub use runtime_tracker::ThreadTracker; +pub use shutdown_signal::signal_stream; +pub use shutdown_signal::DummySignalStream; +pub use shutdown_signal::SignalStream; +pub use shutdown_signal::SignalType; +pub use stop_handle::StopHandle; +pub use stoppable::Stoppable; +pub use string_func::escape_for_key; +pub use string_func::mask_string; +pub use string_func::unescape_for_key; +pub use thread::Thread; +pub use tokio; +pub use uniq_id::GlobalSequence; +pub use uniq_id::GlobalUniqName; +pub use uuid; diff --git a/common/base/src/net.rs b/common/base/src/base/net.rs similarity index 100% rename from common/base/src/net.rs rename to common/base/src/base/net.rs diff --git a/common/base/src/profiling.rs b/common/base/src/base/profiling.rs similarity index 100% rename from common/base/src/profiling.rs rename to common/base/src/base/profiling.rs diff --git a/common/base/src/progress.rs b/common/base/src/base/progress.rs similarity index 100% rename from common/base/src/progress.rs rename to common/base/src/base/progress.rs diff --git a/common/base/src/runtime.rs b/common/base/src/base/runtime.rs similarity index 99% rename from common/base/src/runtime.rs rename to common/base/src/base/runtime.rs index e2ae5c38a7b0..5da21f6b330d 100644 --- a/common/base/src/runtime.rs +++ b/common/base/src/base/runtime.rs @@ -22,7 +22,7 @@ use tokio::runtime::Handle; use tokio::sync::oneshot; use tokio::task::JoinHandle; -use crate::runtime_tracker::RuntimeTracker; +use super::runtime_tracker::RuntimeTracker; /// Methods to spawn tasks. pub trait TrySpawn { diff --git a/common/base/src/runtime_tracker.rs b/common/base/src/base/runtime_tracker.rs similarity index 100% rename from common/base/src/runtime_tracker.rs rename to common/base/src/base/runtime_tracker.rs diff --git a/common/base/src/shutdown_signal.rs b/common/base/src/base/shutdown_signal.rs similarity index 100% rename from common/base/src/shutdown_signal.rs rename to common/base/src/base/shutdown_signal.rs diff --git a/common/base/src/stop_handle.rs b/common/base/src/base/stop_handle.rs similarity index 99% rename from common/base/src/stop_handle.rs rename to common/base/src/base/stop_handle.rs index 7d9c91a787a7..d4c56bb34077 100644 --- a/common/base/src/stop_handle.rs +++ b/common/base/src/base/stop_handle.rs @@ -21,7 +21,7 @@ use common_tracing::tracing; use futures::Future; use tokio::sync::broadcast; -use crate::Stoppable; +use super::Stoppable; /// Handle a group of `Stoppable` tasks. /// When a user press ctrl-c, it calls the `stop()` method on every task to close them. diff --git a/common/base/src/stoppable.rs b/common/base/src/base/stoppable.rs similarity index 100% rename from common/base/src/stoppable.rs rename to common/base/src/base/stoppable.rs diff --git a/common/base/src/string_func.rs b/common/base/src/base/string_func.rs similarity index 100% rename from common/base/src/string_func.rs rename to common/base/src/base/string_func.rs diff --git a/common/base/src/thread.rs b/common/base/src/base/thread.rs similarity index 97% rename from common/base/src/thread.rs rename to common/base/src/base/thread.rs index 96ec0c11bc7b..458ee2f29cf1 100644 --- a/common/base/src/thread.rs +++ b/common/base/src/base/thread.rs @@ -15,7 +15,7 @@ use std::thread::Builder; use std::thread::JoinHandle; -use crate::runtime_tracker::ThreadTracker; +use super::runtime_tracker::ThreadTracker; pub struct Thread; diff --git a/common/base/src/uniq_id.rs b/common/base/src/base/uniq_id.rs similarity index 100% rename from common/base/src/uniq_id.rs rename to common/base/src/base/uniq_id.rs diff --git a/common/containers/src/lib.rs b/common/base/src/containers/mod.rs similarity index 100% rename from common/containers/src/lib.rs rename to common/base/src/containers/mod.rs diff --git a/common/containers/src/pool.rs b/common/base/src/containers/pool.rs similarity index 98% rename from common/containers/src/pool.rs rename to common/base/src/containers/pool.rs index cd0880abbc2c..95fe8c650a98 100644 --- a/common/containers/src/pool.rs +++ b/common/base/src/containers/pool.rs @@ -21,10 +21,11 @@ use std::sync::Mutex; use std::time::Duration; use async_trait::async_trait; -use common_base::tokio; -use common_base::tokio::time::sleep; use common_tracing::tracing; +use crate::base::tokio; +use crate::base::tokio::time::sleep; + pub type PoolItem = Arc>>; /// To build or check an item. diff --git a/common/infallible/src/condvar.rs b/common/base/src/infallible/condvar.rs similarity index 100% rename from common/infallible/src/condvar.rs rename to common/base/src/infallible/condvar.rs diff --git a/common/infallible/src/exit_guard.rs b/common/base/src/infallible/exit_guard.rs similarity index 100% rename from common/infallible/src/exit_guard.rs rename to common/base/src/infallible/exit_guard.rs diff --git a/common/infallible/src/lib.rs b/common/base/src/infallible/mod.rs similarity index 95% rename from common/infallible/src/lib.rs rename to common/base/src/infallible/mod.rs index b2affa58af44..10d664939fae 100644 --- a/common/infallible/src/lib.rs +++ b/common/base/src/infallible/mod.rs @@ -27,7 +27,7 @@ pub use rwlock_upgrade_read::RwLockUpgradableReadGuard; #[macro_export] macro_rules! exit_scope { ($x:block) => { - use common_infallible::ExitGuard; + use common_base::infallible::ExitGuard; let _exit_guard = ExitGuard::create(move || $x); }; } diff --git a/common/infallible/src/mutex.rs b/common/base/src/infallible/mutex.rs similarity index 100% rename from common/infallible/src/mutex.rs rename to common/base/src/infallible/mutex.rs diff --git a/common/infallible/src/rwlock.rs b/common/base/src/infallible/rwlock.rs similarity index 97% rename from common/infallible/src/rwlock.rs rename to common/base/src/infallible/rwlock.rs index 012624753716..bd9040adecdd 100644 --- a/common/infallible/src/rwlock.rs +++ b/common/base/src/infallible/rwlock.rs @@ -16,7 +16,7 @@ use parking_lot::RwLock as ParkingRwLock; use parking_lot::RwLockReadGuard; use parking_lot::RwLockWriteGuard; -use crate::RwLockUpgradableReadGuard; +use super::RwLockUpgradableReadGuard; /// A simple wrapper around the lock() function of a std::sync::RwLock /// The only difference is that you don't need to call unwrap() on it. diff --git a/common/infallible/src/rwlock_upgrade_read.rs b/common/base/src/infallible/rwlock_upgrade_read.rs similarity index 100% rename from common/infallible/src/rwlock_upgrade_read.rs rename to common/base/src/infallible/rwlock_upgrade_read.rs diff --git a/common/base/src/lib.rs b/common/base/src/lib.rs index dac8b5dda56a..54fc5af01687 100644 --- a/common/base/src/lib.rs +++ b/common/base/src/lib.rs @@ -14,43 +14,8 @@ #![feature(thread_local)] -mod format; -mod http_shutdown_handlers; -mod net; -mod profiling; -mod progress; -mod runtime; -mod runtime_tracker; -mod shutdown_signal; -mod stop_handle; -mod stoppable; -mod string_func; -mod thread; -mod uniq_id; - -pub use format::Format; -pub use http_shutdown_handlers::HttpShutdownHandler; -pub use net::get_free_tcp_port; -pub use net::get_free_udp_port; -pub use profiling::Profiling; -pub use progress::Progress; -pub use progress::ProgressValues; -pub use runtime::Dropper; -pub use runtime::Runtime; -pub use runtime::TrySpawn; -pub use runtime_tracker::RuntimeTracker; -pub use runtime_tracker::ThreadTracker; -pub use shutdown_signal::signal_stream; -pub use shutdown_signal::DummySignalStream; -pub use shutdown_signal::SignalStream; -pub use shutdown_signal::SignalType; -pub use stop_handle::StopHandle; -pub use stoppable::Stoppable; -pub use string_func::escape_for_key; -pub use string_func::mask_string; -pub use string_func::unescape_for_key; -pub use thread::Thread; -pub use tokio; -pub use uniq_id::GlobalSequence; -pub use uniq_id::GlobalUniqName; -pub use uuid; +pub mod base; +pub mod containers; +pub mod infallible; +pub mod mem_allocator; +pub mod rangemap; diff --git a/common/mem-allocator/src/allocators.rs b/common/base/src/mem_allocator/allocators.rs similarity index 95% rename from common/mem-allocator/src/allocators.rs rename to common/base/src/mem_allocator/allocators.rs index f265b36503ff..d08cd65255ba 100644 --- a/common/mem-allocator/src/allocators.rs +++ b/common/base/src/mem_allocator/allocators.rs @@ -12,9 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. -use crate::malloc_size::MallocSizeOf; -use crate::malloc_size::MallocSizeOfOps; -use crate::malloc_size::MallocUnconditionalSizeOf; +use super::malloc_size::MallocSizeOf; +use super::malloc_size::MallocSizeOfOps; +use super::malloc_size::MallocUnconditionalSizeOf; /* This Source Code Form is subject to the terms of the Mozilla Public * License, v. 2.0. If a copy of the MPL was not distributed with this @@ -32,10 +32,10 @@ mod platform { use std::os::raw::c_int; use std::os::raw::c_void; - use common_base::ThreadTracker; use tikv_jemalloc_sys as ffi; - use crate::malloc_size::VoidPtrToSizeFn; + use crate::base::ThreadTracker; + use crate::mem_allocator::malloc_size::VoidPtrToSizeFn; /// Get the size of a heap block. pub unsafe extern "C" fn usable_size(ptr: *const c_void) -> usize { diff --git a/common/mem-allocator/src/malloc_size.rs b/common/base/src/mem_allocator/malloc_size.rs similarity index 97% rename from common/mem-allocator/src/malloc_size.rs rename to common/base/src/mem_allocator/malloc_size.rs index 9b8b1b7aba5e..ed3aef9eb7c4 100644 --- a/common/mem-allocator/src/malloc_size.rs +++ b/common/base/src/mem_allocator/malloc_size.rs @@ -524,7 +524,7 @@ impl MallocSizeOf for parking_lot::Mutex { } } -impl MallocSizeOf for common_infallible::Mutex { +impl MallocSizeOf for crate::infallible::Mutex { fn size_of(&self, ops: &mut MallocSizeOfOps) -> usize { self.lock().size_of(ops) } @@ -542,7 +542,7 @@ impl MallocSizeOf for parking_lot::RwLock { } } -impl MallocSizeOf for common_infallible::RwLock { +impl MallocSizeOf for crate::infallible::RwLock { fn size_of(&self, ops: &mut MallocSizeOfOps) -> usize { self.read().size_of(ops) } @@ -581,9 +581,9 @@ impl MallocUnconditionalShallowSizeOf for Arc { macro_rules! malloc_size_of_is_0( ($($ty:ty),+) => ( $( - impl $crate::MallocSizeOf for $ty { + impl $crate::mem_allocator::MallocSizeOf for $ty { #[inline(always)] - fn size_of(&self, _: &mut $crate::MallocSizeOfOps) -> usize { + fn size_of(&self, _: &mut $crate::mem_allocator::MallocSizeOfOps) -> usize { 0 } #[inline(always)] @@ -593,9 +593,9 @@ macro_rules! malloc_size_of_is_0( ); (any: $($ty:ident<$($gen:ident),+>),+) => ( $( - impl<$($gen),+> $crate::MallocSizeOf for $ty<$($gen),+> { + impl<$($gen),+> $crate::mem_allocator::MallocSizeOf for $ty<$($gen),+> { #[inline(always)] - fn size_of(&self, _: &mut $crate::MallocSizeOfOps) -> usize { + fn size_of(&self, _: &mut $crate::mem_allocator::MallocSizeOfOps) -> usize { 0 } #[inline(always)] @@ -605,9 +605,9 @@ macro_rules! malloc_size_of_is_0( ); ($($ty:ident<$($gen:ident),+>),+) => ( $( - impl<$($gen: $crate::MallocSizeOf),+> $crate::MallocSizeOf for $ty<$($gen),+> { + impl<$($gen: $crate::mem_allocator::MallocSizeOf),+> $crate::mem_allocator::MallocSizeOf for $ty<$($gen),+> { #[inline(always)] - fn size_of(&self, _: &mut $crate::MallocSizeOfOps) -> usize { + fn size_of(&self, _: &mut $crate::mem_allocator::MallocSizeOfOps) -> usize { 0 } #[inline(always)] diff --git a/common/mem-allocator/src/lib.rs b/common/base/src/mem_allocator/mod.rs similarity index 100% rename from common/mem-allocator/src/lib.rs rename to common/base/src/mem_allocator/mod.rs diff --git a/common/mem-allocator/src/profiling.rs b/common/base/src/mem_allocator/profiling.rs similarity index 100% rename from common/mem-allocator/src/profiling.rs rename to common/base/src/mem_allocator/profiling.rs diff --git a/common/mem-allocator/src/sizeof.rs b/common/base/src/mem_allocator/sizeof.rs similarity index 90% rename from common/mem-allocator/src/sizeof.rs rename to common/base/src/mem_allocator/sizeof.rs index ced9fe1812a9..a7b9b978cb45 100644 --- a/common/mem-allocator/src/sizeof.rs +++ b/common/base/src/mem_allocator/sizeof.rs @@ -23,10 +23,10 @@ use std::mem::size_of; use std::mem::size_of_val; use std::sync::Arc; -use crate::malloc_size::MallocShallowSizeOf; -use crate::malloc_size::MallocSizeOf; -use crate::malloc_size::MallocSizeOfOps; -use crate::malloc_size::MallocUnconditionalShallowSizeOf; +use super::malloc_size::MallocShallowSizeOf; +use super::malloc_size::MallocSizeOf; +use super::malloc_size::MallocSizeOfOps; +use super::malloc_size::MallocUnconditionalShallowSizeOf; impl MallocShallowSizeOf for Box { fn shallow_size_of(&self, _ops: &mut MallocSizeOfOps) -> usize { diff --git a/common/range-map/src/lib.rs b/common/base/src/rangemap/mod.rs similarity index 92% rename from common/range-map/src/lib.rs rename to common/base/src/rangemap/mod.rs index 651b1c84e305..580c3e14f5d9 100644 --- a/common/range-map/src/lib.rs +++ b/common/base/src/rangemap/mod.rs @@ -12,8 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -mod range_key; mod range_map; +mod range_map_key; -pub use range_key::RangeKey; pub use range_map::RangeMap; +pub use range_map_key::RangeMapKey; diff --git a/common/range-map/src/range_map.rs b/common/base/src/rangemap/range_map.rs similarity index 79% rename from common/range-map/src/range_map.rs rename to common/base/src/rangemap/range_map.rs index 269363f3bf41..f9a122afcbda 100644 --- a/common/range-map/src/range_map.rs +++ b/common/base/src/rangemap/range_map.rs @@ -17,11 +17,11 @@ use core::ops::Bound; use core::ops::Range; use std::collections::BTreeMap; -use super::range_key::RangeKey; +use super::range_map_key::RangeMapKey; #[derive(Clone, Debug, Default)] pub struct RangeMap { - pub(crate) map: BTreeMap, V>, + pub(crate) map: BTreeMap, V>, } impl RangeMap @@ -38,7 +38,7 @@ where pub fn insert(&mut self, range: Range, key: K, val: V) { assert!(range.start <= range.end); - let range_key: RangeKey = RangeKey::new(range, key); + let range_key: RangeMapKey = RangeMapKey::new(range, key); self.map.insert(range_key, val); } @@ -49,9 +49,9 @@ where // 2. `get_by_point(2)` return [1,5],[2,4],[2,6] // 3. `get_by_point(5)` return [2,4],[2,6] // Use the default key when construct `RangeKey::key` for search. - pub fn get_by_point(&self, point: &T) -> Vec<(&RangeKey, &V)> { + pub fn get_by_point(&self, point: &T) -> Vec<(&RangeMapKey, &V)> { let key = point.clone(); - let range_key = RangeKey::new(key.clone()..key.clone(), K::default()); + let range_key = RangeMapKey::new(key.clone()..key.clone(), K::default()); self.map .range((Bound::Included(range_key), Bound::Unbounded)) @@ -60,10 +60,10 @@ where } pub fn remove(&mut self, range: Range, k: K) { - self.map.remove(&RangeKey::new(range, k)); + self.map.remove(&RangeMapKey::new(range, k)); } - pub fn remove_by_key(&mut self, key: &RangeKey) { + pub fn remove_by_key(&mut self, key: &RangeMapKey) { self.map.remove(key); } } diff --git a/common/range-map/src/range_key.rs b/common/base/src/rangemap/range_map_key.rs similarity index 76% rename from common/range-map/src/range_key.rs rename to common/base/src/rangemap/range_map_key.rs index 2683e273bf8d..892443518647 100644 --- a/common/range-map/src/range_key.rs +++ b/common/base/src/rangemap/range_map_key.rs @@ -16,26 +16,26 @@ use core::cmp::Ordering; use core::ops::Range; use std::fmt::Debug; -/// `RangeKey` is a wrapper of `range` and `user defined key` +/// `RangeMapKey` is a wrapper of `range` and `user defined key` #[derive(Eq, Debug, Clone, PartialEq)] -pub struct RangeKey { +pub struct RangeMapKey { // range pub range: Range, // user defined key pub key: K, } -impl RangeKey +impl RangeMapKey where T: Eq + Ord, K: Eq + Ord + Default, { - pub fn new(range: Range, key: K) -> RangeKey { - RangeKey { range, key } + pub fn new(range: Range, key: K) -> RangeMapKey { + RangeMapKey { range, key } } } -impl ToString for RangeKey +impl ToString for RangeMapKey where T: Debug, K: Debug, @@ -45,14 +45,14 @@ where } } -impl Ord for RangeKey +impl Ord for RangeMapKey where T: Ord + Debug + Clone, K: Ord + Debug + Clone, { /// the compare weight is: range.end > range.start > key /// example: ((2,3),5) < ((5,1),3) since 2 < 5 - fn cmp(&self, other: &RangeKey) -> Ordering { + fn cmp(&self, other: &RangeMapKey) -> Ordering { let ret = self.range.end.cmp(&other.range.end); if !ret.is_eq() { return ret; @@ -65,12 +65,12 @@ where } } -impl PartialOrd for RangeKey +impl PartialOrd for RangeMapKey where T: Ord + Debug + Clone, K: Ord + Debug + Clone, { - fn partial_cmp(&self, other: &RangeKey) -> Option { + fn partial_cmp(&self, other: &RangeMapKey) -> Option { Some(self.cmp(other)) } } diff --git a/common/mem-allocator/tests/it/derive.rs b/common/base/tests/it/derive.rs similarity index 93% rename from common/mem-allocator/tests/it/derive.rs rename to common/base/tests/it/derive.rs index 34bfad08efe6..ab3a22b75a8c 100644 --- a/common/mem-allocator/tests/it/derive.rs +++ b/common/base/tests/it/derive.rs @@ -15,11 +15,11 @@ use std::collections::HashMap; use std::sync::Arc; -use common_infallible::Mutex; +use common_base::infallible::Mutex; +use common_base::mem_allocator::malloc_size; +use common_base::mem_allocator::MallocSizeOf; +use common_base::mem_allocator::MallocSizeOfExt; use common_macros::MallocSizeOf; -use common_mem_allocator::malloc_size; -use common_mem_allocator::MallocSizeOf; -use common_mem_allocator::MallocSizeOfExt; #[test] fn derive_vec() { diff --git a/common/base/tests/it/format.rs b/common/base/tests/it/format.rs index d9b259b56698..6aeb9bacdc7a 100644 --- a/common/base/tests/it/format.rs +++ b/common/base/tests/it/format.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::*; +use common_base::base::*; use common_exception::Result; use serde::Deserialize; use serde::Serialize; diff --git a/common/base/tests/it/main.rs b/common/base/tests/it/main.rs index ce9c1479b93c..69aeef2bac80 100644 --- a/common/base/tests/it/main.rs +++ b/common/base/tests/it/main.rs @@ -12,8 +12,15 @@ // See the License for the specific language governing permissions and // limitations under the License. +mod derive; mod format; +mod malloc_size; +mod mutex; +mod pool; mod progress; +mod range_key_test; +mod range_map_test; mod runtime; +mod rwlock; mod stoppable; mod string_func; diff --git a/common/mem-allocator/tests/it/malloc_size.rs b/common/base/tests/it/malloc_size.rs similarity index 89% rename from common/mem-allocator/tests/it/malloc_size.rs rename to common/base/tests/it/malloc_size.rs index e3f800ce26c8..51027cf3d37a 100644 --- a/common/mem-allocator/tests/it/malloc_size.rs +++ b/common/base/tests/it/malloc_size.rs @@ -13,11 +13,11 @@ // limitations under the License. #[allow(unused)] -use common_mem_allocator::*; +use common_base::mem_allocator::*; #[test] fn test_large_vec() { - use common_mem_allocator::new_malloc_size_ops; + use common_base::mem_allocator::new_malloc_size_ops; const N: usize = 128 * 1024 * 1024; let val = vec![1u8; N]; let mut ops = new_malloc_size_ops(); @@ -32,7 +32,7 @@ fn btree_set() { set.insert(vec![t]); } // ~36 per value - assert!(common_mem_allocator::malloc_size(&set) > 3000); + assert!(common_base::mem_allocator::malloc_size(&set) > 3000); } #[test] @@ -41,11 +41,11 @@ fn special_malloc_size_of_0() { phantom: std::marker::PhantomData

, } - malloc_size_of_is_0!(any: Data

); + common_base::malloc_size_of_is_0!(any: Data

); // MallocSizeOf is not implemented for [u8; 333] assert_eq!( - common_mem_allocator::malloc_size(&Data::<[u8; 333]> { + common_base::mem_allocator::malloc_size(&Data::<[u8; 333]> { phantom: std::marker::PhantomData }), 0 diff --git a/common/infallible/tests/it/mutex.rs b/common/base/tests/it/mutex.rs similarity index 96% rename from common/infallible/tests/it/mutex.rs rename to common/base/tests/it/mutex.rs index e4078f25ea1c..cc7d1de2c233 100644 --- a/common/infallible/tests/it/mutex.rs +++ b/common/base/tests/it/mutex.rs @@ -17,7 +17,7 @@ fn test_mutex() { use std::sync::Arc; use std::thread; - use common_infallible::Mutex; + use common_base::infallible::Mutex; let a = 7u8; let mutex = Arc::new(Mutex::new(a)); let mutex2 = mutex.clone(); diff --git a/common/containers/tests/pool.rs b/common/base/tests/it/pool.rs similarity index 94% rename from common/containers/tests/pool.rs rename to common/base/tests/it/pool.rs index 3776f63dfbb1..e65b3cbba527 100644 --- a/common/containers/tests/pool.rs +++ b/common/base/tests/it/pool.rs @@ -15,10 +15,10 @@ use std::time::Duration; use async_trait::async_trait; -use common_base::tokio; -use common_base::GlobalSequence; -use common_containers::ItemManager; -use common_containers::Pool; +use common_base::base::tokio; +use common_base::base::GlobalSequence; +use common_base::containers::ItemManager; +use common_base::containers::Pool; use common_tracing::tracing; #[tokio::test(flavor = "multi_thread", worker_threads = 1)] diff --git a/common/base/tests/it/progress.rs b/common/base/tests/it/progress.rs index a40d0e76623e..683961786098 100644 --- a/common/base/tests/it/progress.rs +++ b/common/base/tests/it/progress.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::*; +use common_base::base::*; use common_exception::Result; #[test] diff --git a/common/range-map/tests/it/range_key_test.rs b/common/base/tests/it/range_key_test.rs similarity index 96% rename from common/range-map/tests/it/range_key_test.rs rename to common/base/tests/it/range_key_test.rs index cc48c51eecdf..67faf35ab0f0 100644 --- a/common/range-map/tests/it/range_key_test.rs +++ b/common/base/tests/it/range_key_test.rs @@ -16,7 +16,7 @@ use core::cmp::Ordering; use std::collections::BTreeMap; use std::collections::BTreeSet; -use common_range_map::RangeKey; +use common_base::rangemap::RangeMapKey; fn upsert_cmp_map(map: &mut BTreeMap>, k: String, v: String) { if map.get(&k).is_none() { @@ -32,7 +32,7 @@ fn upsert_cmp_map(map: &mut BTreeMap>, k: String, v: St fn test_range_wrapper_reflexive_property() { let mut tests = vec![]; for i in 0..10 { - tests.push(RangeKey::new(0..i, ())); + tests.push(RangeMapKey::new(0..i, ())); } let mut less_map: BTreeMap> = BTreeMap::new(); diff --git a/common/range-map/tests/it/range_map_test.rs b/common/base/tests/it/range_map_test.rs similarity index 76% rename from common/range-map/tests/it/range_map_test.rs rename to common/base/tests/it/range_map_test.rs index 9a039ab2efdd..4c6632724ad9 100644 --- a/common/range-map/tests/it/range_map_test.rs +++ b/common/base/tests/it/range_map_test.rs @@ -12,8 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_range_map::RangeKey; -use common_range_map::RangeMap; +use common_base::rangemap::RangeMap; +use common_base::rangemap::RangeMapKey; #[test] fn test_range_set() { @@ -21,10 +21,10 @@ fn test_range_set() { { let mut a = RangeMap::new(); - let r11 = (&RangeKey::new(1..1, 11), &11); - let r15 = (&RangeKey::new(1..5, 15), &15); - let r24 = (&RangeKey::new(2..4, 24), &24); - let r26 = (&RangeKey::new(2..6, 26), &26); + let r11 = (&RangeMapKey::new(1..1, 11), &11); + let r15 = (&RangeMapKey::new(1..5, 15), &15); + let r24 = (&RangeMapKey::new(2..4, 24), &24); + let r26 = (&RangeMapKey::new(2..6, 26), &26); a.insert(1..1, 11, 11); a.insert(1..5, 15, 15); @@ -49,10 +49,10 @@ fn test_range_set() { let a5 = "5".to_string(); let a6 = "6".to_string(); - let r11 = (&RangeKey::new(a1.clone()..a1.clone(), 11), &11); - let r15 = (&RangeKey::new(a1.clone()..a5.clone(), 15), &15); - let r24 = (&RangeKey::new(a2.clone()..a4.clone(), 24), &24); - let r26 = (&RangeKey::new(a2.clone()..a6.clone(), 26), &26); + let r11 = (&RangeMapKey::new(a1.clone()..a1.clone(), 11), &11); + let r15 = (&RangeMapKey::new(a1.clone()..a5.clone(), 15), &15); + let r24 = (&RangeMapKey::new(a2.clone()..a4.clone(), 24), &24); + let r26 = (&RangeMapKey::new(a2.clone()..a6.clone(), 26), &26); a.insert(a1.clone()..a1.clone(), 11, 11); a.insert(a1.clone()..a5.clone(), 15, 15); diff --git a/common/base/tests/it/runtime.rs b/common/base/tests/it/runtime.rs index 93bbd8e9800f..e7e908341b0e 100644 --- a/common/base/tests/it/runtime.rs +++ b/common/base/tests/it/runtime.rs @@ -15,7 +15,7 @@ use std::sync::Arc; use std::sync::Mutex; -use common_base::*; +use common_base::base::*; use common_exception::Result; #[tokio::test(flavor = "multi_thread", worker_threads = 8)] diff --git a/common/infallible/tests/it/rwlock.rs b/common/base/tests/it/rwlock.rs similarity index 96% rename from common/infallible/tests/it/rwlock.rs rename to common/base/tests/it/rwlock.rs index dedf92308da1..81600d57ec9e 100644 --- a/common/infallible/tests/it/rwlock.rs +++ b/common/base/tests/it/rwlock.rs @@ -17,7 +17,7 @@ fn test_rwlock() { use std::sync::Arc; use std::thread; - use common_infallible::RwLock; + use common_base::infallible::RwLock; let a = 7u8; let rwlock = Arc::new(RwLock::new(a)); diff --git a/common/base/tests/it/stoppable.rs b/common/base/tests/it/stoppable.rs index ebaed86e5824..60d7177a480b 100644 --- a/common/base/tests/it/stoppable.rs +++ b/common/base/tests/it/stoppable.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::*; +use common_base::base::*; use common_exception::Result; use common_tracing::tracing; use tokio::sync::broadcast; diff --git a/common/base/tests/it/string_func.rs b/common/base/tests/it/string_func.rs index 82f6014a6212..e6183ffd6ef4 100644 --- a/common/base/tests/it/string_func.rs +++ b/common/base/tests/it/string_func.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::*; +use common_base::base::*; use common_exception::Result; #[test] diff --git a/common/containers/Cargo.toml b/common/containers/Cargo.toml deleted file mode 100644 index 1baa157deea3..000000000000 --- a/common/containers/Cargo.toml +++ /dev/null @@ -1,20 +0,0 @@ -[package] -name = "common-containers" -version = "0.1.0" -authors = ["Databend Authors "] -license = "Apache-2.0" -publish = false -edition = "2021" - -[lib] -doctest = false -test = false - -[dependencies] -common-base = { path = "../base" } -common-tracing = { path = "../tracing" } - -async-trait = "0.1.53" - -[dev-dependencies] -anyhow = "1.0.56" diff --git a/common/containers/tests/main.rs b/common/containers/tests/main.rs deleted file mode 100644 index 570ec2356189..000000000000 --- a/common/containers/tests/main.rs +++ /dev/null @@ -1,15 +0,0 @@ -// Copyright 2021 Datafuse Labs. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -mod pool; diff --git a/common/contexts/src/dal/dal_runtime.rs b/common/contexts/src/dal/dal_runtime.rs index aeb9d3a2455e..90716b88bbfa 100644 --- a/common/contexts/src/dal/dal_runtime.rs +++ b/common/contexts/src/dal/dal_runtime.rs @@ -17,7 +17,7 @@ use std::io::Result; use std::sync::Arc; use async_trait::async_trait; -use common_base::tokio::runtime::Handle; +use common_base::base::tokio::runtime::Handle; use opendal::ops::OpCreate; use opendal::ops::OpDelete; use opendal::ops::OpList; diff --git a/common/datablocks/Cargo.toml b/common/datablocks/Cargo.toml index d7d33a2d24c9..b81556795dc1 100644 --- a/common/datablocks/Cargo.toml +++ b/common/datablocks/Cargo.toml @@ -13,9 +13,9 @@ test = false [dependencies] # In alphabetical order # Workspace dependencies common-arrow = { path = "../arrow" } +common-base = { path = "../base" } common-datavalues = { path = "../datavalues" } common-exception = { path = "../exception" } -common-infallible = { path = "../infallible" } common-io = { path = "../io" } # Github dependencies diff --git a/common/datablocks/src/memory.rs b/common/datablocks/src/memory.rs index 459f0eb02b32..79eeb93f205f 100644 --- a/common/datablocks/src/memory.rs +++ b/common/datablocks/src/memory.rs @@ -15,7 +15,7 @@ use std::collections::HashMap; use std::sync::Arc; -use common_infallible::RwLock; +use common_base::infallible::RwLock; use crate::DataBlock; diff --git a/common/datavalues/Cargo.toml b/common/datavalues/Cargo.toml index 34e6287e0358..b6505562652d 100644 --- a/common/datavalues/Cargo.toml +++ b/common/datavalues/Cargo.toml @@ -13,10 +13,10 @@ test = false [dependencies] # In alphabetical order # Workspace dependencies common-arrow = { path = "../arrow" } +common-base = { path = "../base" } common-exception = { path = "../exception" } common-io = { path = "../io" } common-macros = { path = "../macros" } -common-mem-allocator = { path = "../mem-allocator" } # Github dependencies opensrv-clickhouse = { git = "https://github.com/datafuselabs/opensrv", rev = "9690be9", package = "opensrv-clickhouse" } diff --git a/common/grpc/src/dns_resolver.rs b/common/grpc/src/dns_resolver.rs index 6fcef9e995b6..4810b97029ff 100644 --- a/common/grpc/src/dns_resolver.rs +++ b/common/grpc/src/dns_resolver.rs @@ -22,8 +22,8 @@ use std::task::Poll; use std::time::Duration; use anyerror::AnyError; -use common_base::tokio; -use common_base::tokio::task::JoinHandle; +use common_base::base::tokio; +use common_base::base::tokio::task::JoinHandle; use common_exception::ErrorCode; use common_exception::Result; use common_tracing::tracing; diff --git a/common/grpc/tests/it/dns_resolver.rs b/common/grpc/tests/it/dns_resolver.rs index 689af85b9a13..a11ae71d6311 100644 --- a/common/grpc/tests/it/dns_resolver.rs +++ b/common/grpc/tests/it/dns_resolver.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use common_grpc::DNSResolver; diff --git a/common/infallible/Cargo.toml b/common/infallible/Cargo.toml deleted file mode 100644 index fb68c8211a27..000000000000 --- a/common/infallible/Cargo.toml +++ /dev/null @@ -1,19 +0,0 @@ -[package] -name = "common-infallible" -version = "0.1.0" -authors = ["Databend Authors "] -license = "Apache-2.0" -publish = false -edition = "2021" - -[lib] -doctest = false -test = false - -[dependencies] # In alphabetical order -# Workspace dependencies - -# Github dependencies - -# Crates.io dependencies -parking_lot = "0.12.0" diff --git a/common/infallible/tests/it/main.rs b/common/infallible/tests/it/main.rs deleted file mode 100644 index cdc1c57550ee..000000000000 --- a/common/infallible/tests/it/main.rs +++ /dev/null @@ -1,16 +0,0 @@ -// Copyright 2021 Datafuse Labs. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -mod mutex; -mod rwlock; diff --git a/common/macros/src/async_entrypoint.rs b/common/macros/src/async_entrypoint.rs index 65c4c0fbcabd..ebfec5521373 100644 --- a/common/macros/src/async_entrypoint.rs +++ b/common/macros/src/async_entrypoint.rs @@ -70,7 +70,7 @@ fn parse_knobs(mut input: syn::ItemFn, is_test: bool, has_tracker: bool) -> Toke }; let rt = quote::quote_spanned! {last_stmt_start_span => - common_base::Runtime::with_default_worker_threads().unwrap() + common_base::base::Runtime::with_default_worker_threads().unwrap() }; let wait_in_future = match has_tracker { @@ -85,7 +85,7 @@ fn parse_knobs(mut input: syn::ItemFn, is_test: bool, has_tracker: bool) -> Toke let header = if is_test { quote::quote! { - use common_base::Runtime; + use common_base::base::Runtime; #[::core::prelude::v1::test] } } else { diff --git a/common/macros/src/malloc_sizeof.rs b/common/macros/src/malloc_sizeof.rs index 69311dea7541..06b82d1d1350 100644 --- a/common/macros/src/malloc_sizeof.rs +++ b/common/macros/src/malloc_sizeof.rs @@ -64,9 +64,9 @@ pub fn malloc_size_of_derive(s: synstructure::Structure) -> proc_macro2::TokenSt } let path = if conditional { - quote! { common_mem_allocator::MallocConditionalSizeOf::conditional_size_of } + quote! { common_base::mem_allocator::MallocConditionalSizeOf::conditional_size_of } } else { - quote! { common_mem_allocator::MallocSizeOf::size_of } + quote! { common_base::mem_allocator::MallocSizeOf::size_of } }; if let syn::Type::Array(..) = binding.ast().ty { @@ -90,14 +90,14 @@ pub fn malloc_size_of_derive(s: synstructure::Structure) -> proc_macro2::TokenSt let ident = ¶m.ident; where_clause .predicates - .push(parse_quote!(#ident: common_mem_allocator::MallocSizeOf)); + .push(parse_quote!(#ident: common_base::mem_allocator::MallocSizeOf)); } let tokens = quote! { - impl #impl_generics common_mem_allocator::MallocSizeOf for #name #ty_generics #where_clause { + impl #impl_generics common_base::mem_allocator::MallocSizeOf for #name #ty_generics #where_clause { #[inline] #[allow(unused_variables, unused_mut, unreachable_code)] - fn size_of(&self, ops: &mut common_mem_allocator::MallocSizeOfOps) -> usize { + fn size_of(&self, ops: &mut common_base::mem_allocator::MallocSizeOfOps) -> usize { let mut sum = 0; match *self { #match_body diff --git a/common/management/src/cluster/cluster_mgr.rs b/common/management/src/cluster/cluster_mgr.rs index 77d0a3092085..08e2a29bb421 100644 --- a/common/management/src/cluster/cluster_mgr.rs +++ b/common/management/src/cluster/cluster_mgr.rs @@ -17,8 +17,8 @@ use std::sync::Arc; use std::time::Duration; use std::time::UNIX_EPOCH; -use common_base::escape_for_key; -use common_base::unescape_for_key; +use common_base::base::escape_for_key; +use common_base::base::unescape_for_key; use common_exception::ErrorCode; use common_exception::Result; use common_meta_api::KVApi; diff --git a/common/management/src/stage/stage_mgr.rs b/common/management/src/stage/stage_mgr.rs index 6c123c19d853..b63eca2b8996 100644 --- a/common/management/src/stage/stage_mgr.rs +++ b/common/management/src/stage/stage_mgr.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use common_base::escape_for_key; +use common_base::base::escape_for_key; use common_exception::ErrorCode; use common_exception::Result; use common_meta_api::KVApi; diff --git a/common/management/src/udf/udf_mgr.rs b/common/management/src/udf/udf_mgr.rs index dc34d28e78ef..5057ba2e7dcd 100644 --- a/common/management/src/udf/udf_mgr.rs +++ b/common/management/src/udf/udf_mgr.rs @@ -15,7 +15,7 @@ use std::sync::Arc; use common_ast::udfs::UDFParser; -use common_base::escape_for_key; +use common_base::base::escape_for_key; use common_exception::ErrorCode; use common_exception::Result; use common_functions::is_builtin_function; diff --git a/common/management/src/user/user_mgr.rs b/common/management/src/user/user_mgr.rs index 964ea8d46814..2177a55db553 100644 --- a/common/management/src/user/user_mgr.rs +++ b/common/management/src/user/user_mgr.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use common_base::escape_for_key; +use common_base::base::escape_for_key; use common_exception::ErrorCode; use common_exception::Result; use common_exception::ToErrorCode; diff --git a/common/management/tests/it/cluster.rs b/common/management/tests/it/cluster.rs index d2592b0bd614..75efffda5a53 100644 --- a/common/management/tests/it/cluster.rs +++ b/common/management/tests/it/cluster.rs @@ -16,7 +16,7 @@ use std::sync::Arc; use std::time::Duration; use std::time::UNIX_EPOCH; -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use common_management::*; use common_meta_api::KVApi; diff --git a/common/management/tests/it/setting.rs b/common/management/tests/it/setting.rs index a4179711b625..102f57b90f46 100644 --- a/common/management/tests/it/setting.rs +++ b/common/management/tests/it/setting.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use common_base::tokio; +use common_base::base::tokio; use common_datavalues::DataValue; use common_exception::Result; use common_management::*; diff --git a/common/management/tests/it/stage.rs b/common/management/tests/it/stage.rs index 4cd7764e0bc9..398568b60b15 100644 --- a/common/management/tests/it/stage.rs +++ b/common/management/tests/it/stage.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use common_management::*; use common_meta_api::KVApi; diff --git a/common/management/tests/it/udf.rs b/common/management/tests/it/udf.rs index e73d2d77794f..0c5765b61f9e 100644 --- a/common/management/tests/it/udf.rs +++ b/common/management/tests/it/udf.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use common_management::*; use common_meta_api::KVApi; diff --git a/common/management/tests/it/user.rs b/common/management/tests/it/user.rs index 18df87976761..b4e2ecf89395 100644 --- a/common/management/tests/it/user.rs +++ b/common/management/tests/it/user.rs @@ -15,8 +15,8 @@ use std::sync::Arc; use async_trait::async_trait; -use common_base::escape_for_key; -use common_base::tokio; +use common_base::base::escape_for_key; +use common_base::base::tokio; use common_exception::ErrorCode; use common_management::*; use common_meta_api::KVApi; diff --git a/common/mem-allocator/Cargo.toml b/common/mem-allocator/Cargo.toml deleted file mode 100644 index d38f1a7c5b82..000000000000 --- a/common/mem-allocator/Cargo.toml +++ /dev/null @@ -1,31 +0,0 @@ -[package] -name = "common-mem-allocator" -version = "0.1.0" -edition = "2021" - -# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html - -[lib] -doctest = false -test = false - -[dependencies] -common-base = { path = "../base" } -common-infallible = { path = "../infallible" } -libc = { version = "0.2", optional = true } - -parking_lot = "0.12.0" -tikv-jemalloc-ctl = { version = "0.4", optional = true } -tikv-jemalloc-sys = { version = "0.4.3" } - -[dev-dependencies] -common-macros = { path = "../macros" } - -[features] -memory-profiling = [ - "tikv-jemalloc-ctl", - "libc", - "tikv-jemalloc-sys/stats", - "tikv-jemalloc-sys/profiling", - "tikv-jemalloc-sys/unprefixed_malloc_on_supported_platforms", -] diff --git a/common/mem-allocator/tests/it/main.rs b/common/mem-allocator/tests/it/main.rs deleted file mode 100644 index bd5eb2cfcf14..000000000000 --- a/common/mem-allocator/tests/it/main.rs +++ /dev/null @@ -1,16 +0,0 @@ -// Copyright 2021 Datafuse Labs. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -mod derive; -mod malloc_size; diff --git a/common/meta/api/src/kv_api_test_suite.rs b/common/meta/api/src/kv_api_test_suite.rs index 8526e9750cb8..87fcdd774c8c 100644 --- a/common/meta/api/src/kv_api_test_suite.rs +++ b/common/meta/api/src/kv_api_test_suite.rs @@ -15,7 +15,7 @@ use std::time::SystemTime; use std::time::UNIX_EPOCH; -use common_base::tokio; +use common_base::base::tokio; use common_meta_types::txn_condition; use common_meta_types::txn_op; use common_meta_types::txn_op_response; diff --git a/common/meta/embedded/src/meta_embedded.rs b/common/meta/embedded/src/meta_embedded.rs index 2929e95fddf2..a4532ff15224 100644 --- a/common/meta/embedded/src/meta_embedded.rs +++ b/common/meta/embedded/src/meta_embedded.rs @@ -16,7 +16,7 @@ use std::sync::atomic::AtomicUsize; use std::sync::atomic::Ordering; use std::sync::Arc; -use common_base::tokio::sync::Mutex; +use common_base::base::tokio::sync::Mutex; use common_meta_raft_store::config::RaftConfig; use common_meta_raft_store::state_machine::StateMachine; pub use common_meta_sled_store::init_temp_sled_db; diff --git a/common/meta/embedded/tests/it/kv_api_impl.rs b/common/meta/embedded/tests/it/kv_api_impl.rs index 0df7ae2f4d0c..f350fb27f51f 100644 --- a/common/meta/embedded/tests/it/kv_api_impl.rs +++ b/common/meta/embedded/tests/it/kv_api_impl.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_meta_api::KVApiTestSuite; use common_meta_embedded::MetaEmbedded; diff --git a/common/meta/embedded/tests/it/schema_api_impl.rs b/common/meta/embedded/tests/it/schema_api_impl.rs index 44e21b451804..2e807683397d 100644 --- a/common/meta/embedded/tests/it/schema_api_impl.rs +++ b/common/meta/embedded/tests/it/schema_api_impl.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_meta_api::SchemaApiTestSuite; use common_meta_embedded::MetaEmbedded; diff --git a/common/meta/grpc/Cargo.toml b/common/meta/grpc/Cargo.toml index b065897d3828..9c8dafb7dca3 100644 --- a/common/meta/grpc/Cargo.toml +++ b/common/meta/grpc/Cargo.toml @@ -16,7 +16,6 @@ test = false [dependencies] common-arrow = { path = "../../arrow" } common-base = { path = "../../base" } -common-containers = { path = "../../containers" } common-exception = { path = "../../exception" } common-grpc = { path = "../../grpc" } common-meta-api = { path = "../api" } diff --git a/common/meta/grpc/src/grpc_client.rs b/common/meta/grpc/src/grpc_client.rs index 095c287e91cd..48210e385e2d 100644 --- a/common/meta/grpc/src/grpc_client.rs +++ b/common/meta/grpc/src/grpc_client.rs @@ -18,9 +18,9 @@ use std::sync::Arc; use std::time::Duration; use common_arrow::arrow_format::flight::data::BasicAuth; -use common_base::tokio::sync::RwLock; -use common_containers::ItemManager; -use common_containers::Pool; +use common_base::base::tokio::sync::RwLock; +use common_base::containers::ItemManager; +use common_base::containers::Pool; use common_exception::Result; use common_grpc::ConnectionFactory; use common_grpc::GrpcConnectionError; diff --git a/common/meta/grpc/tests/it/grpc_client.rs b/common/meta/grpc/tests/it/grpc_client.rs index 456eb3174fd5..30b76e0745db 100644 --- a/common/meta/grpc/tests/it/grpc_client.rs +++ b/common/meta/grpc/tests/it/grpc_client.rs @@ -14,7 +14,7 @@ use std::time::Duration; -use common_base::tokio; +use common_base::base::tokio; use common_exception::ErrorCode; use common_meta_api::SchemaApi; use common_meta_grpc::MetaGrpcClient; diff --git a/common/meta/grpc/tests/it/grpc_server.rs b/common/meta/grpc/tests/it/grpc_server.rs index 9c6300724cf0..e10db1c01f6c 100644 --- a/common/meta/grpc/tests/it/grpc_server.rs +++ b/common/meta/grpc/tests/it/grpc_server.rs @@ -16,7 +16,7 @@ use std::pin::Pin; use std::thread::sleep; use std::time::Duration; -use common_base::tokio; +use common_base::base::tokio; use common_meta_types::protobuf::meta_service_server::MetaService; use common_meta_types::protobuf::meta_service_server::MetaServiceServer; use common_meta_types::protobuf::ExportedChunk; diff --git a/common/meta/raft-store/tests/it/log.rs b/common/meta/raft-store/tests/it/log.rs index a12c7f392cf8..9aa9a7322270 100644 --- a/common/meta/raft-store/tests/it/log.rs +++ b/common/meta/raft-store/tests/it/log.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_meta_raft_store::log::RaftLog; use common_meta_sled_store::openraft; use common_meta_types::Cmd; diff --git a/common/meta/raft-store/tests/it/state.rs b/common/meta/raft-store/tests/it/state.rs index 6035f32f79dd..f981e218cbb6 100644 --- a/common/meta/raft-store/tests/it/state.rs +++ b/common/meta/raft-store/tests/it/state.rs @@ -11,7 +11,7 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_meta_raft_store::state::RaftState; use common_meta_sled_store::openraft; use openraft::storage::HardState; diff --git a/common/meta/raft-store/tests/it/state_machine/mod.rs b/common/meta/raft-store/tests/it/state_machine/mod.rs index aa41176749ed..8aed36f9adfb 100644 --- a/common/meta/raft-store/tests/it/state_machine/mod.rs +++ b/common/meta/raft-store/tests/it/state_machine/mod.rs @@ -15,7 +15,7 @@ use std::time::SystemTime; use std::time::UNIX_EPOCH; -use common_base::tokio; +use common_base::base::tokio; use common_meta_api::KVApi; use common_meta_raft_store::state_machine::testing::pretty_snapshot; use common_meta_raft_store::state_machine::testing::snapshot_logs; diff --git a/common/meta/raft-store/tests/it/state_machine/schema_api_impl.rs b/common/meta/raft-store/tests/it/state_machine/schema_api_impl.rs index 3167ad61fabd..8d910a0e3ef0 100644 --- a/common/meta/raft-store/tests/it/state_machine/schema_api_impl.rs +++ b/common/meta/raft-store/tests/it/state_machine/schema_api_impl.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_meta_api::SchemaApiTestSuite; use common_meta_raft_store::state_machine::StateMachine; diff --git a/common/meta/raft-store/tests/it/testing.rs b/common/meta/raft-store/tests/it/testing.rs index 4fca375252d4..319b894f5701 100644 --- a/common/meta/raft-store/tests/it/testing.rs +++ b/common/meta/raft-store/tests/it/testing.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::GlobalSequence; +use common_base::base::GlobalSequence; use common_meta_raft_store::config::RaftConfig; use common_meta_sled_store::get_sled_db; use common_meta_sled_store::sled; diff --git a/common/meta/sled-store/tests/it/sled_tree.rs b/common/meta/sled-store/tests/it/sled_tree.rs index b570c1dba617..77dfa800462f 100644 --- a/common/meta/sled-store/tests/it/sled_tree.rs +++ b/common/meta/sled-store/tests/it/sled_tree.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_meta_sled_store::SledTree; use common_meta_types::Cmd; use common_meta_types::LogEntry; diff --git a/common/meta/sled-store/tests/it/sled_txn_tree.rs b/common/meta/sled-store/tests/it/sled_txn_tree.rs index d1a46d4f78ba..d62b8b4106d1 100644 --- a/common/meta/sled-store/tests/it/sled_txn_tree.rs +++ b/common/meta/sled-store/tests/it/sled_txn_tree.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_meta_sled_store::SledTree; use common_meta_sled_store::Store; use common_meta_types::Endpoint; diff --git a/common/meta/sled-store/tests/it/testing/mod.rs b/common/meta/sled-store/tests/it/testing/mod.rs index 3c8670b59060..efe98c7f0f88 100644 --- a/common/meta/sled-store/tests/it/testing/mod.rs +++ b/common/meta/sled-store/tests/it/testing/mod.rs @@ -15,7 +15,7 @@ pub mod fake_key_spaces; pub mod fake_state_machine_meta; -use common_base::GlobalSequence; +use common_base::base::GlobalSequence; use common_meta_sled_store::get_sled_db; /// 1. Open a temp sled::Db for all tests. diff --git a/common/metrics/Cargo.toml b/common/metrics/Cargo.toml index ac45ef739b0f..2424c79efcab 100644 --- a/common/metrics/Cargo.toml +++ b/common/metrics/Cargo.toml @@ -12,8 +12,8 @@ test = false [dependencies] # Workspace dependencies +common-base = { path = "../base" } common-exception = { path = "../exception" } -common-infallible = { path = "../infallible" } common-tracing = { path = "../tracing" } # Crates.io dependencies diff --git a/common/metrics/src/recorder.rs b/common/metrics/src/recorder.rs index b63e7fa89b71..116e7942fb20 100644 --- a/common/metrics/src/recorder.rs +++ b/common/metrics/src/recorder.rs @@ -15,7 +15,7 @@ use std::sync::Arc; use std::sync::Once; -use common_infallible::RwLock; +use common_base::infallible::RwLock; use common_tracing::tracing; use metrics::counter; use metrics_exporter_prometheus::PrometheusBuilder; diff --git a/common/planners/Cargo.toml b/common/planners/Cargo.toml index e42bc5a82d48..12ce0f0c8513 100644 --- a/common/planners/Cargo.toml +++ b/common/planners/Cargo.toml @@ -16,7 +16,6 @@ common-datablocks = { path = "../datablocks" } common-datavalues = { path = "../datavalues" } common-exception = { path = "../exception" } common-functions = { path = "../functions" } -common-infallible = { path = "../infallible" } common-meta-types = { path = "../meta/types" } # Github dependencies diff --git a/common/range-map/Cargo.toml b/common/range-map/Cargo.toml deleted file mode 100644 index 6d72a8e0682f..000000000000 --- a/common/range-map/Cargo.toml +++ /dev/null @@ -1,11 +0,0 @@ -[package] -name = "common-range-map" -version = "0.1.0" -authors = ["Databend Authors "] -license = "Apache-2.0" -publish = false -edition = "2021" - -[lib] -doctest = false -test = false diff --git a/common/range-map/tests/it/main.rs b/common/range-map/tests/it/main.rs deleted file mode 100644 index 4ddcc21e238a..000000000000 --- a/common/range-map/tests/it/main.rs +++ /dev/null @@ -1,16 +0,0 @@ -// Copyright 2021 Datafuse Labs. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -mod range_key_test; -mod range_map_test; diff --git a/common/streams/Cargo.toml b/common/streams/Cargo.toml index 8da4fa7988c8..8f0b7aa02da9 100644 --- a/common/streams/Cargo.toml +++ b/common/streams/Cargo.toml @@ -18,7 +18,6 @@ common-datablocks = { path = "../datablocks" } common-datavalues = { path = "../datavalues" } common-exception = { path = "../exception" } common-functions = { path = "../functions" } -common-infallible = { path = "../infallible" } common-io = { path = "../io" } common-tracing = { path = "../tracing" } diff --git a/common/streams/src/stream_error.rs b/common/streams/src/stream_error.rs index b939a84d04ba..c9d202c9e4c4 100644 --- a/common/streams/src/stream_error.rs +++ b/common/streams/src/stream_error.rs @@ -16,10 +16,10 @@ use std::sync::Arc; use std::task::Context; use std::task::Poll; +use common_base::infallible::Mutex; use common_datablocks::DataBlock; use common_exception::ErrorCode; use common_exception::Result; -use common_infallible::Mutex; use futures::Stream; use pin_project_lite::pin_project; diff --git a/common/streams/src/stream_progress.rs b/common/streams/src/stream_progress.rs index 9f71e7ebd364..11bd63955a78 100644 --- a/common/streams/src/stream_progress.rs +++ b/common/streams/src/stream_progress.rs @@ -16,8 +16,8 @@ use std::sync::Arc; use std::task::Context; use std::task::Poll; -use common_base::Progress; -use common_base::ProgressValues; +use common_base::base::Progress; +use common_base::base::ProgressValues; use common_datablocks::DataBlock; use common_exception::Result; use futures::Stream; diff --git a/common/streams/tests/it/sources/source_csv.rs b/common/streams/tests/it/sources/source_csv.rs index 703eb2cc1802..86209b7e1a33 100644 --- a/common/streams/tests/it/sources/source_csv.rs +++ b/common/streams/tests/it/sources/source_csv.rs @@ -15,7 +15,7 @@ use std::fs::File; use std::io::Write; -use common_base::tokio; +use common_base::base::tokio; use common_datablocks::assert_blocks_eq; use common_datavalues::prelude::*; use common_exception::Result; diff --git a/common/streams/tests/it/sources/source_ndjson.rs b/common/streams/tests/it/sources/source_ndjson.rs index 3ef7ef379984..657c2bea2b1b 100644 --- a/common/streams/tests/it/sources/source_ndjson.rs +++ b/common/streams/tests/it/sources/source_ndjson.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_datablocks::assert_blocks_eq; use common_exception::Result; use common_io::prelude::FormatSettings; diff --git a/common/streams/tests/it/sources/source_parquet.rs b/common/streams/tests/it/sources/source_parquet.rs index 5d9d52dfe531..98b0e068713a 100644 --- a/common/streams/tests/it/sources/source_parquet.rs +++ b/common/streams/tests/it/sources/source_parquet.rs @@ -18,7 +18,7 @@ use common_arrow::arrow::io::parquet::write::RowGroupIterator; use common_arrow::arrow::io::parquet::write::Version; use common_arrow::arrow::io::parquet::write::WriteOptions; use common_arrow::parquet::compression::CompressionOptions; -use common_base::tokio; +use common_base::base::tokio; use common_datablocks::assert_blocks_eq; use common_datablocks::DataBlock; use common_exception::ErrorCode; diff --git a/common/streams/tests/it/stream_cast.rs b/common/streams/tests/it/stream_cast.rs index 5a910b2f9fd5..fd40af9284f7 100644 --- a/common/streams/tests/it/stream_cast.rs +++ b/common/streams/tests/it/stream_cast.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_datablocks::*; use common_datavalues::prelude::*; use common_functions::scalars::CastFunction; diff --git a/common/streams/tests/it/stream_datablock.rs b/common/streams/tests/it/stream_datablock.rs index 81690fc45abb..3ceb75f1b779 100644 --- a/common/streams/tests/it/stream_datablock.rs +++ b/common/streams/tests/it/stream_datablock.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_datablocks::*; use common_datavalues::prelude::*; use common_streams::*; diff --git a/common/streams/tests/it/stream_limit_by.rs b/common/streams/tests/it/stream_limit_by.rs index bb7839beb8cc..4105b0eef25e 100644 --- a/common/streams/tests/it/stream_limit_by.rs +++ b/common/streams/tests/it/stream_limit_by.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_datablocks::*; use common_datavalues::prelude::*; use common_exception::Result; diff --git a/common/streams/tests/it/stream_progress.rs b/common/streams/tests/it/stream_progress.rs index 869b39224e46..858e115356f4 100644 --- a/common/streams/tests/it/stream_progress.rs +++ b/common/streams/tests/it/stream_progress.rs @@ -14,8 +14,8 @@ use std::sync::Arc; -use common_base::tokio; -use common_base::*; +use common_base::base::tokio; +use common_base::base::*; use common_datablocks::*; use common_datavalues::prelude::*; use common_exception::Result; diff --git a/common/streams/tests/it/stream_skip.rs b/common/streams/tests/it/stream_skip.rs index d7224b65d4e6..4c341b89c892 100644 --- a/common/streams/tests/it/stream_skip.rs +++ b/common/streams/tests/it/stream_skip.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_datablocks::*; use common_datavalues::prelude::*; use common_streams::*; diff --git a/metasrv/Cargo.toml b/metasrv/Cargo.toml index b1484ea96121..a6fcf3b68c1f 100644 --- a/metasrv/Cargo.toml +++ b/metasrv/Cargo.toml @@ -26,7 +26,6 @@ tokio-console = ["common-tracing/console", "common-base/tracing"] # Workspace dependencies common-arrow = { path = "../common/arrow" } common-base = { path = "../common/base" } -common-containers = { path = "../common/containers" } common-exception = { path = "../common/exception" } common-grpc = { path = "../common/grpc" } common-macros = { path = "../common/macros" } @@ -36,7 +35,6 @@ common-meta-raft-store = { path = "../common/meta/raft-store" } common-meta-sled-store = { path = "../common/meta/sled-store" } common-meta-types = { path = "../common/meta/types" } common-metrics = { path = "../common/metrics" } -common-range-map = { path = "../common/range-map" } common-tracing = { path = "../common/tracing" } # Github dependencies diff --git a/metasrv/src/api/grpc/grpc_service.rs b/metasrv/src/api/grpc/grpc_service.rs index 2a53a1690870..2f4f26fb334e 100644 --- a/metasrv/src/api/grpc/grpc_service.rs +++ b/metasrv/src/api/grpc/grpc_service.rs @@ -18,7 +18,7 @@ use std::task::Context; use std::task::Poll; use common_arrow::arrow_format::flight::data::BasicAuth; -use common_base::tokio::sync::mpsc; +use common_base::base::tokio::sync::mpsc; use common_grpc::GrpcClaim; use common_grpc::GrpcToken; use common_meta_grpc::MetaGrpcReadReq; diff --git a/metasrv/src/api/grpc_server.rs b/metasrv/src/api/grpc_server.rs index 6bd743cef34b..9a53483d40a5 100644 --- a/metasrv/src/api/grpc_server.rs +++ b/metasrv/src/api/grpc_server.rs @@ -15,12 +15,12 @@ use std::sync::Arc; use anyerror::AnyError; -use common_base::tokio; -use common_base::tokio::sync::oneshot; -use common_base::tokio::sync::oneshot::Receiver; -use common_base::tokio::sync::oneshot::Sender; -use common_base::tokio::task::JoinHandle; -use common_base::Stoppable; +use common_base::base::tokio; +use common_base::base::tokio::sync::oneshot; +use common_base::base::tokio::sync::oneshot::Receiver; +use common_base::base::tokio::sync::oneshot::Sender; +use common_base::base::tokio::task::JoinHandle; +use common_base::base::Stoppable; use common_meta_types::protobuf::meta_service_server::MetaServiceServer; use common_meta_types::protobuf::FILE_DESCRIPTOR_SET; use common_meta_types::MetaError; diff --git a/metasrv/src/api/http/debug/pprof.rs b/metasrv/src/api/http/debug/pprof.rs index 8c41b4bc8c1c..a1b893ea05b4 100644 --- a/metasrv/src/api/http/debug/pprof.rs +++ b/metasrv/src/api/http/debug/pprof.rs @@ -12,8 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio::time::Duration; -use common_base::Profiling; +use common_base::base::tokio::time::Duration; +use common_base::base::Profiling; use common_tracing::tracing; use poem::error::InternalServerError; use poem::web::Query; diff --git a/metasrv/src/api/http_service.rs b/metasrv/src/api/http_service.rs index d85cebc0ea0a..19c4e6295b1a 100644 --- a/metasrv/src/api/http_service.rs +++ b/metasrv/src/api/http_service.rs @@ -14,9 +14,9 @@ use std::sync::Arc; -use common_base::tokio::sync::broadcast; -use common_base::HttpShutdownHandler; -use common_base::Stoppable; +use common_base::base::tokio::sync::broadcast; +use common_base::base::HttpShutdownHandler; +use common_base::base::Stoppable; use common_exception::Result; use common_tracing::tracing; use poem::get; diff --git a/metasrv/src/bin/metasrv.rs b/metasrv/src/bin/metasrv.rs index d975c82c736f..532459352122 100644 --- a/metasrv/src/bin/metasrv.rs +++ b/metasrv/src/bin/metasrv.rs @@ -14,9 +14,9 @@ use std::sync::Arc; -use common_base::RuntimeTracker; -use common_base::StopHandle; -use common_base::Stoppable; +use common_base::base::RuntimeTracker; +use common_base::base::StopHandle; +use common_base::base::Stoppable; use common_macros::databend_main; use common_meta_sled_store::init_sled_db; use common_metrics::init_default_metrics_recorder; diff --git a/metasrv/src/configs/config.rs b/metasrv/src/configs/config.rs index 928f4c6b856b..e5f6788468fb 100644 --- a/metasrv/src/configs/config.rs +++ b/metasrv/src/configs/config.rs @@ -13,7 +13,7 @@ // limitations under the License. use clap::Parser; -use common_base::Format; +use common_base::base::Format; use common_meta_raft_store::config as raft_config; use common_meta_raft_store::config::RaftConfig; use common_meta_types::MetaError; diff --git a/metasrv/src/meta_service/raftmeta.rs b/metasrv/src/meta_service/raftmeta.rs index 248be81cfb08..324cd59d23df 100644 --- a/metasrv/src/meta_service/raftmeta.rs +++ b/metasrv/src/meta_service/raftmeta.rs @@ -17,11 +17,11 @@ use std::fmt::Debug; use std::net::Ipv4Addr; use std::sync::Arc; -use common_base::tokio; -use common_base::tokio::sync::watch; -use common_base::tokio::sync::Mutex; -use common_base::tokio::sync::RwLockReadGuard; -use common_base::tokio::task::JoinHandle; +use common_base::base::tokio; +use common_base::base::tokio::sync::watch; +use common_base::base::tokio::sync::Mutex; +use common_base::base::tokio::sync::RwLockReadGuard; +use common_base::base::tokio::task::JoinHandle; use common_grpc::DNSResolver; use common_meta_raft_store::config::RaftConfig; use common_meta_raft_store::state_machine::StateMachine; diff --git a/metasrv/src/metrics/metric_service.rs b/metasrv/src/metrics/metric_service.rs index 98c26d7fb796..547ec2c99b61 100644 --- a/metasrv/src/metrics/metric_service.rs +++ b/metasrv/src/metrics/metric_service.rs @@ -14,9 +14,9 @@ use std::net::SocketAddr; -use common_base::tokio::sync::broadcast; -use common_base::HttpShutdownHandler; -use common_base::Stoppable; +use common_base::base::tokio::sync::broadcast; +use common_base::base::HttpShutdownHandler; +use common_base::base::Stoppable; use common_exception::ErrorCode; use common_exception::Result; use common_metrics::PrometheusHandle; diff --git a/metasrv/src/network.rs b/metasrv/src/network.rs index 7a266e934c2e..40f471d93222 100644 --- a/metasrv/src/network.rs +++ b/metasrv/src/network.rs @@ -15,8 +15,8 @@ use std::sync::Arc; use std::time::Duration; -use common_containers::ItemManager; -use common_containers::Pool; +use common_base::containers::ItemManager; +use common_base::containers::Pool; use common_meta_sled_store::openraft; use common_meta_sled_store::openraft::MessageSummary; use common_meta_types::protobuf::raft_service_client::RaftServiceClient; diff --git a/metasrv/src/store/meta_raft_store.rs b/metasrv/src/store/meta_raft_store.rs index ab1ffb2d91c0..92b4f88894bc 100644 --- a/metasrv/src/store/meta_raft_store.rs +++ b/metasrv/src/store/meta_raft_store.rs @@ -19,8 +19,8 @@ use std::io::ErrorKind; use std::ops::RangeBounds; use anyerror::AnyError; -use common_base::tokio::sync::RwLock; -use common_base::tokio::sync::RwLockWriteGuard; +use common_base::base::tokio::sync::RwLock; +use common_base::base::tokio::sync::RwLockWriteGuard; use common_meta_raft_store::config::RaftConfig; use common_meta_raft_store::log::RaftLog; use common_meta_raft_store::state::RaftState; diff --git a/metasrv/src/watcher/watcher_manager.rs b/metasrv/src/watcher/watcher_manager.rs index cb3111b33092..3a076d4e4c81 100644 --- a/metasrv/src/watcher/watcher_manager.rs +++ b/metasrv/src/watcher/watcher_manager.rs @@ -14,9 +14,11 @@ // use core::ops::Range; -use common_base::tokio; -use common_base::tokio::sync::mpsc; -use common_base::tokio::sync::mpsc::Sender; +use common_base::base::tokio; +use common_base::base::tokio::sync::mpsc; +use common_base::base::tokio::sync::mpsc::Sender; +use common_base::rangemap::RangeMap; +use common_base::rangemap::RangeMapKey; use common_meta_raft_store::state_machine::StateMachineSubscriber; use common_meta_types::protobuf::watch_request::FilterType; use common_meta_types::protobuf::Event; @@ -24,8 +26,6 @@ use common_meta_types::protobuf::WatchRequest; use common_meta_types::protobuf::WatchResponse; use common_meta_types::PbSeqV; use common_meta_types::SeqV; -use common_range_map::RangeKey; -use common_range_map::RangeMap; use common_tracing::tracing; use tonic::Status; @@ -115,7 +115,7 @@ impl WatcherManagerCore { } #[tracing::instrument(level = "debug", skip(self))] - fn close_stream(&mut self, key: RangeKey) { + fn close_stream(&mut self, key: RangeMapKey) { self.watcher_range_map.remove_by_key(&key); } @@ -129,7 +129,7 @@ impl WatcherManagerCore { let prev = kv.prev; let is_delete_event = current.is_none(); - let mut remove_range_keys: Vec> = vec![]; + let mut remove_range_keys: Vec> = vec![]; for range_key_stream in set.iter() { let filter = range_key_stream.1.filter_type; @@ -158,7 +158,7 @@ impl WatcherManagerCore { watcher_id, err ); - remove_range_keys.push(RangeKey::new( + remove_range_keys.push(RangeMapKey::new( stream.key.clone()..stream.key_end.clone(), watcher_id, )); diff --git a/metasrv/src/watcher/watcher_stream.rs b/metasrv/src/watcher/watcher_stream.rs index 484a7be14a8c..1329f96cdcee 100644 --- a/metasrv/src/watcher/watcher_stream.rs +++ b/metasrv/src/watcher/watcher_stream.rs @@ -13,7 +13,7 @@ // limitations under the License. // -use common_base::tokio::sync::mpsc::error::SendError; +use common_base::base::tokio::sync::mpsc::error::SendError; use common_meta_types::protobuf::watch_request::FilterType; use common_meta_types::protobuf::WatchResponse; use tonic::Status; diff --git a/metasrv/tests/it/api/http/cluster_state_test.rs b/metasrv/tests/it/api/http/cluster_state_test.rs index 696c60d2a603..e678fe72d5dd 100644 --- a/metasrv/tests/it/api/http/cluster_state_test.rs +++ b/metasrv/tests/it/api/http/cluster_state_test.rs @@ -18,8 +18,8 @@ use std::fs::File; use std::io::Read; use std::string::String; -use common_base::tokio; -use common_base::Stoppable; +use common_base::base::tokio; +use common_base::base::Stoppable; use common_meta_types::Node; use databend_meta::api::http::v1::cluster_state::nodes_handler; use databend_meta::api::http::v1::cluster_state::state_handler; diff --git a/metasrv/tests/it/api/http/config.rs b/metasrv/tests/it/api/http/config.rs index 954f1fe7cb1f..75e556557d99 100644 --- a/metasrv/tests/it/api/http/config.rs +++ b/metasrv/tests/it/api/http/config.rs @@ -14,7 +14,7 @@ * limitations under the License. * */ -use common_base::tokio; +use common_base::base::tokio; use databend_meta::api::http::v1::config::config_handler; use databend_meta::configs::Config; use poem::get; diff --git a/metasrv/tests/it/api/http/health.rs b/metasrv/tests/it/api/http/health.rs index 956c24888ca9..123a6588c3a0 100644 --- a/metasrv/tests/it/api/http/health.rs +++ b/metasrv/tests/it/api/http/health.rs @@ -14,7 +14,7 @@ * limitations under the License. * */ -use common_base::tokio; +use common_base::base::tokio; use databend_meta::api::http::v1::health::health_handler; use poem::get; use poem::http::Method; diff --git a/metasrv/tests/it/api/http_service.rs b/metasrv/tests/it/api/http_service.rs index 56be8d0e22aa..2c651a49808d 100644 --- a/metasrv/tests/it/api/http_service.rs +++ b/metasrv/tests/it/api/http_service.rs @@ -15,8 +15,8 @@ use std::fs::File; use std::io::Read; -use common_base::tokio; -use common_base::Stoppable; +use common_base::base::tokio; +use common_base::base::Stoppable; use common_exception::Result; use databend_meta::api::HttpService; use databend_meta::configs::Config; diff --git a/metasrv/tests/it/grpc/metasrv_grpc_api.rs b/metasrv/tests/it/grpc/metasrv_grpc_api.rs index 6d45d0016d2e..75d8923981e9 100644 --- a/metasrv/tests/it/grpc/metasrv_grpc_api.rs +++ b/metasrv/tests/it/grpc/metasrv_grpc_api.rs @@ -14,8 +14,8 @@ //! Test arrow-grpc API of metasrv -use common_base::tokio; -use common_base::Stoppable; +use common_base::base::tokio; +use common_base::base::Stoppable; use common_meta_api::KVApi; use common_meta_grpc::MetaGrpcClient; use common_meta_types::MatchSeq; diff --git a/metasrv/tests/it/grpc/metasrv_grpc_export.rs b/metasrv/tests/it/grpc/metasrv_grpc_export.rs index a2bc683818d9..8a9ccf380d8e 100644 --- a/metasrv/tests/it/grpc/metasrv_grpc_export.rs +++ b/metasrv/tests/it/grpc/metasrv_grpc_export.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_meta_api::KVApi; use common_meta_grpc::MetaGrpcClient; use common_meta_types::protobuf::Empty; diff --git a/metasrv/tests/it/grpc/metasrv_grpc_kv_api.rs b/metasrv/tests/it/grpc/metasrv_grpc_kv_api.rs index 1896943758e6..970d493f7471 100644 --- a/metasrv/tests/it/grpc/metasrv_grpc_kv_api.rs +++ b/metasrv/tests/it/grpc/metasrv_grpc_kv_api.rs @@ -16,7 +16,7 @@ use std::sync::Arc; use std::sync::Mutex; use async_trait::async_trait; -use common_base::tokio; +use common_base::base::tokio; use common_meta_api::KVApiBuilder; use common_meta_api::KVApiTestSuite; use common_meta_grpc::MetaGrpcClient; diff --git a/metasrv/tests/it/grpc/metasrv_grpc_kv_api_restart_cluster.rs b/metasrv/tests/it/grpc/metasrv_grpc_kv_api_restart_cluster.rs index 6dda90dad515..95062c24adf9 100644 --- a/metasrv/tests/it/grpc/metasrv_grpc_kv_api_restart_cluster.rs +++ b/metasrv/tests/it/grpc/metasrv_grpc_kv_api_restart_cluster.rs @@ -17,8 +17,8 @@ use std::time::Duration; -use common_base::tokio; -use common_base::Stoppable; +use common_base::base::tokio; +use common_base::base::Stoppable; use common_meta_api::KVApi; use common_meta_grpc::MetaGrpcClient; use common_meta_types::MatchSeq; diff --git a/metasrv/tests/it/grpc/metasrv_grpc_schema_api.rs b/metasrv/tests/it/grpc/metasrv_grpc_schema_api.rs index 8e92eb657e24..418b9f90e979 100644 --- a/metasrv/tests/it/grpc/metasrv_grpc_schema_api.rs +++ b/metasrv/tests/it/grpc/metasrv_grpc_schema_api.rs @@ -14,7 +14,7 @@ //! Test metasrv SchemaApi on a single node. -use common_base::tokio; +use common_base::base::tokio; use common_meta_api::SchemaApiTestSuite; use common_meta_grpc::MetaGrpcClient; diff --git a/metasrv/tests/it/grpc/metasrv_grpc_schema_api_follower_follower.rs b/metasrv/tests/it/grpc/metasrv_grpc_schema_api_follower_follower.rs index 9350ebe1ccda..4b73b46218d6 100644 --- a/metasrv/tests/it/grpc/metasrv_grpc_schema_api_follower_follower.rs +++ b/metasrv/tests/it/grpc/metasrv_grpc_schema_api_follower_follower.rs @@ -14,7 +14,7 @@ //! Test metasrv SchemaApi by writing to one follower and then reading from another follower. -use common_base::tokio; +use common_base::base::tokio; use common_meta_api::SchemaApiTestSuite; use crate::init_meta_ut; diff --git a/metasrv/tests/it/grpc/metasrv_grpc_schema_api_leader_follower.rs b/metasrv/tests/it/grpc/metasrv_grpc_schema_api_leader_follower.rs index c3defc73bd74..1b507418f116 100644 --- a/metasrv/tests/it/grpc/metasrv_grpc_schema_api_leader_follower.rs +++ b/metasrv/tests/it/grpc/metasrv_grpc_schema_api_leader_follower.rs @@ -13,7 +13,7 @@ // limitations under the License. //! Test metasrv SchemaApi by writing to leader and then reading from a follower. -use common_base::tokio; +use common_base::base::tokio; use common_meta_api::SchemaApiTestSuite; use crate::init_meta_ut; diff --git a/metasrv/tests/it/grpc/metasrv_grpc_tls.rs b/metasrv/tests/it/grpc/metasrv_grpc_tls.rs index c114a6a63bc2..ecf535f9f3fa 100644 --- a/metasrv/tests/it/grpc/metasrv_grpc_tls.rs +++ b/metasrv/tests/it/grpc/metasrv_grpc_tls.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::ErrorCode; use common_grpc::RpcClientTlsConfig; use common_meta_api::SchemaApi; diff --git a/metasrv/tests/it/grpc/metasrv_grpc_watch.rs b/metasrv/tests/it/grpc/metasrv_grpc_watch.rs index 693d3a42afef..e6abcd5a59f7 100644 --- a/metasrv/tests/it/grpc/metasrv_grpc_watch.rs +++ b/metasrv/tests/it/grpc/metasrv_grpc_watch.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_meta_api::KVApi; use common_meta_grpc::MetaGrpcClient; use common_meta_types::protobuf::watch_request::FilterType; diff --git a/metasrv/tests/it/meta_node/meta_node_all.rs b/metasrv/tests/it/meta_node/meta_node_all.rs index 3cf954d9e15a..b8351e264f74 100644 --- a/metasrv/tests/it/meta_node/meta_node_all.rs +++ b/metasrv/tests/it/meta_node/meta_node_all.rs @@ -15,8 +15,8 @@ use std::collections::BTreeSet; use std::sync::Arc; -use common_base::tokio; -use common_base::tokio::time::Duration; +use common_base::base::tokio; +use common_base::base::tokio::time::Duration; use common_meta_api::KVApi; use common_meta_sled_store::openraft; use common_meta_sled_store::openraft::LogIdOptionExt; diff --git a/metasrv/tests/it/meta_node/meta_node_kv_api.rs b/metasrv/tests/it/meta_node/meta_node_kv_api.rs index edb389af9fec..9777cd4b92c1 100644 --- a/metasrv/tests/it/meta_node/meta_node_kv_api.rs +++ b/metasrv/tests/it/meta_node/meta_node_kv_api.rs @@ -16,7 +16,7 @@ use std::sync::Arc; use std::sync::Mutex; use async_trait::async_trait; -use common_base::tokio; +use common_base::base::tokio; use common_meta_api::KVApiBuilder; use common_meta_api::KVApiTestSuite; use common_tracing::tracing::Instrument; diff --git a/metasrv/tests/it/store.rs b/metasrv/tests/it/store.rs index 551d14b71420..42dc470b255b 100644 --- a/metasrv/tests/it/store.rs +++ b/metasrv/tests/it/store.rs @@ -15,7 +15,7 @@ use std::sync::Arc; use std::sync::Mutex; -use common_base::tokio; +use common_base::base::tokio; use common_meta_raft_store::state_machine::testing::pretty_snapshot; use common_meta_raft_store::state_machine::testing::snapshot_logs; use common_meta_raft_store::state_machine::SerializableSnapshot; diff --git a/metasrv/tests/it/tests/service.rs b/metasrv/tests/it/tests/service.rs index 33cbe00cbc75..820f73f3e23e 100644 --- a/metasrv/tests/it/tests/service.rs +++ b/metasrv/tests/it/tests/service.rs @@ -15,9 +15,9 @@ use std::sync::Arc; use anyhow::Result; -use common_base::tokio; -use common_base::GlobalSequence; -use common_base::Stoppable; +use common_base::base::tokio; +use common_base::base::GlobalSequence; +use common_base::base::Stoppable; use common_meta_grpc::MetaGrpcClient; use common_meta_sled_store::openraft::NodeId; use common_meta_types::protobuf::raft_service_client::RaftServiceClient; diff --git a/query/Cargo.toml b/query/Cargo.toml index ada05f734bab..a7d7460efa04 100644 --- a/query/Cargo.toml +++ b/query/Cargo.toml @@ -21,7 +21,7 @@ test = false default = ["simd"] simd = ["common-arrow/simd"] tokio-console = ["common-tracing/console", "common-base/tracing"] -memory-profiling = ["common-mem-allocator/memory-profiling", "tempfile"] +memory-profiling = ["common-base/memory-profiling", "tempfile"] [dependencies] # Workspace dependencies @@ -35,11 +35,9 @@ common-datavalues = { path = "../common/datavalues" } common-exception = { path = "../common/exception" } common-functions = { path = "../common/functions" } common-grpc = { path = "../common/grpc" } -common-infallible = { path = "../common/infallible" } common-io = { path = "../common/io" } common-macros = { path = "../common/macros" } common-management = { path = "../common/management" } -common-mem-allocator = { path = "../common/mem-allocator" } common-meta-api = { path = "../common/meta/api" } common-meta-embedded = { path = "../common/meta/embedded" } common-meta-grpc = { path = "../common/meta/grpc" } diff --git a/query/benches/suites/mod.rs b/query/benches/suites/mod.rs index 414c6d3e6618..20d99bff1743 100644 --- a/query/benches/suites/mod.rs +++ b/query/benches/suites/mod.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use common_planners::PlanNode; use criterion::Criterion; diff --git a/query/src/api/http/debug/jeprof.rs b/query/src/api/http/debug/jeprof.rs index 572a42b56502..4ecd81f557fb 100644 --- a/query/src/api/http/debug/jeprof.rs +++ b/query/src/api/http/debug/jeprof.rs @@ -12,8 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +use common_base::mem_allocator::dump_profile; use common_exception::ErrorCode; -use common_mem_allocator::dump_profile; use poem::error::InternalServerError; use poem::http::StatusCode; use poem::web::IntoResponse; diff --git a/query/src/api/http/debug/pprof.rs b/query/src/api/http/debug/pprof.rs index 55548107109a..66e0978cf6ad 100644 --- a/query/src/api/http/debug/pprof.rs +++ b/query/src/api/http/debug/pprof.rs @@ -12,8 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio::time::Duration; -use common_base::Profiling; +use common_base::base::tokio::time::Duration; +use common_base::base::Profiling; use common_tracing::tracing; use poem::error::InternalServerError; use poem::web::IntoResponse; diff --git a/query/src/api/rpc/flight_client.rs b/query/src/api/rpc/flight_client.rs index 7d2e7753130a..1f578079fcbd 100644 --- a/query/src/api/rpc/flight_client.rs +++ b/query/src/api/rpc/flight_client.rs @@ -18,7 +18,7 @@ use common_arrow::arrow_format::flight::data::Action; use common_arrow::arrow_format::flight::data::FlightData; use common_arrow::arrow_format::flight::data::Ticket; use common_arrow::arrow_format::flight::service::flight_service_client::FlightServiceClient; -use common_base::tokio::time::Duration; +use common_base::base::tokio::time::Duration; use common_datavalues::DataSchemaRef; use common_exception::ErrorCode; use common_exception::Result; diff --git a/query/src/api/rpc/flight_dispatcher.rs b/query/src/api/rpc/flight_dispatcher.rs index 8c99c3ea59f2..b4b711d7d8d2 100644 --- a/query/src/api/rpc/flight_dispatcher.rs +++ b/query/src/api/rpc/flight_dispatcher.rs @@ -17,15 +17,15 @@ use std::sync::atomic::AtomicBool; use std::sync::atomic::Ordering; use std::sync::Arc; -use common_base::tokio::sync::mpsc::Sender; -use common_base::tokio::sync::*; -use common_base::TrySpawn; +use common_base::base::tokio::sync::mpsc::Sender; +use common_base::base::tokio::sync::*; +use common_base::base::TrySpawn; +use common_base::infallible::RwLock; use common_datablocks::DataBlock; use common_datavalues::DataSchemaRef; use common_exception::ErrorCode; use common_exception::Result; use common_exception::ToErrorCode; -use common_infallible::RwLock; use common_tracing::tracing; use common_tracing::tracing::Instrument; use common_tracing::tracing::Span; diff --git a/query/src/api/rpc/flight_service_stream.rs b/query/src/api/rpc/flight_service_stream.rs index 433b74e206a3..a96605041eed 100644 --- a/query/src/api/rpc/flight_service_stream.rs +++ b/query/src/api/rpc/flight_service_stream.rs @@ -18,9 +18,9 @@ use common_arrow::arrow::io::flight::serialize_batch; use common_arrow::arrow::io::ipc::write::WriteOptions; use common_arrow::arrow::io::ipc::IpcField; use common_arrow::arrow_format::flight::data::FlightData; -use common_base::tokio::macros::support::Pin; -use common_base::tokio::macros::support::Poll; -use common_base::tokio::sync::mpsc::Receiver; +use common_base::base::tokio::macros::support::Pin; +use common_base::base::tokio::macros::support::Poll; +use common_base::base::tokio::sync::mpsc::Receiver; use common_datablocks::DataBlock; use futures::task::Context; use tokio_stream::Stream; diff --git a/query/src/api/rpc_service.rs b/query/src/api/rpc_service.rs index 659753ef3c40..6c05d4d8075a 100644 --- a/query/src/api/rpc_service.rs +++ b/query/src/api/rpc_service.rs @@ -17,9 +17,9 @@ use std::net::SocketAddr; use std::sync::Arc; use common_arrow::arrow_format::flight::service::flight_service_server::FlightServiceServer; -use common_base::tokio; -use common_base::tokio::net::TcpListener; -use common_base::tokio::sync::Notify; +use common_base::base::tokio; +use common_base::base::tokio::net::TcpListener; +use common_base::base::tokio::sync::Notify; use common_exception::ErrorCode; use common_exception::Result; use common_tracing::tracing; @@ -97,7 +97,7 @@ impl RpcService { .add_service(FlightServiceServer::new(flight_api_service)) .serve_with_incoming_shutdown(listener_stream, self.shutdown_notify()); - common_base::tokio::spawn(server); + common_base::base::tokio::spawn(server); Ok(()) } } diff --git a/query/src/bin/databend-query.rs b/query/src/bin/databend-query.rs index 1cee0d0a7c11..9b47c0cb86c7 100644 --- a/query/src/bin/databend-query.rs +++ b/query/src/bin/databend-query.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use common_base::RuntimeTracker; +use common_base::base::RuntimeTracker; use common_macros::databend_main; use common_meta_embedded::MetaEmbedded; use common_metrics::init_default_metrics_recorder; diff --git a/query/src/catalogs/catalog_context.rs b/query/src/catalogs/catalog_context.rs index 89ce2add7bbc..1c493654c7e7 100644 --- a/query/src/catalogs/catalog_context.rs +++ b/query/src/catalogs/catalog_context.rs @@ -14,8 +14,8 @@ use std::sync::Arc; +use common_base::infallible::RwLock; use common_datablocks::InMemoryData; -use common_infallible::RwLock; use common_meta_api::SchemaApi; use crate::databases::DatabaseFactory; diff --git a/query/src/catalogs/table_memory_meta.rs b/query/src/catalogs/table_memory_meta.rs index 76bcad376064..eb8119d437c8 100644 --- a/query/src/catalogs/table_memory_meta.rs +++ b/query/src/catalogs/table_memory_meta.rs @@ -18,9 +18,9 @@ use std::sync::atomic::AtomicU64; use std::sync::atomic::Ordering; use std::sync::Arc; +use common_base::infallible::RwLock; use common_exception::ErrorCode; use common_exception::Result; -use common_infallible::RwLock; use common_meta_types::MetaId; use crate::storages::Table; diff --git a/query/src/clusters/cluster.rs b/query/src/clusters/cluster.rs index 194c6ddf8041..106f7d0219ad 100644 --- a/query/src/clusters/cluster.rs +++ b/query/src/clusters/cluster.rs @@ -19,15 +19,15 @@ use std::sync::Arc; use std::time::Duration; use common_arrow::arrow_format::flight::service::flight_service_client::FlightServiceClient; -use common_base::tokio; -use common_base::tokio::sync::Mutex; -use common_base::tokio::sync::Notify; -use common_base::tokio::task::JoinHandle; -use common_base::tokio::time::sleep as tokio_async_sleep; -use common_base::DummySignalStream; -use common_base::GlobalUniqName; -use common_base::SignalStream; -use common_base::SignalType; +use common_base::base::tokio; +use common_base::base::tokio::sync::Mutex; +use common_base::base::tokio::sync::Notify; +use common_base::base::tokio::task::JoinHandle; +use common_base::base::tokio::time::sleep as tokio_async_sleep; +use common_base::base::DummySignalStream; +use common_base::base::GlobalUniqName; +use common_base::base::SignalStream; +use common_base::base::SignalType; use common_exception::ErrorCode; use common_exception::Result; use common_grpc::ConnectionFactory; diff --git a/query/src/common/service/http_shutdown_handles.rs b/query/src/common/service/http_shutdown_handles.rs index 20d1093c5852..df3fae972f90 100644 --- a/query/src/common/service/http_shutdown_handles.rs +++ b/query/src/common/service/http_shutdown_handles.rs @@ -14,8 +14,8 @@ use std::net::SocketAddr; -use common_base::tokio::sync::oneshot; -use common_base::tokio::task::JoinHandle; +use common_base::base::tokio::sync::oneshot; +use common_base::base::tokio::task::JoinHandle; use common_exception::ErrorCode; use common_exception::Result; use common_tracing::tracing; @@ -76,7 +76,7 @@ impl HttpShutdownHandler { } let (tx, rx) = oneshot::channel(); - let join_handle = common_base::tokio::spawn( + let join_handle = common_base::base::tokio::spawn( poem::Server::new_with_acceptor(acceptor).run_with_graceful_shutdown( ep, rx.map(|_| ()), diff --git a/query/src/configs/config_meta.rs b/query/src/configs/config_meta.rs index b767a13ded59..d5db47f8fd07 100644 --- a/query/src/configs/config_meta.rs +++ b/query/src/configs/config_meta.rs @@ -15,7 +15,7 @@ use std::fmt; use clap::Args; -use common_base::mask_string; +use common_base::base::mask_string; use common_grpc::RpcClientConf; use common_grpc::RpcClientTlsConfig; use common_meta_grpc::MetaGrpcClientConf; diff --git a/query/src/configs/config_storage.rs b/query/src/configs/config_storage.rs index 1703a0efd9c5..8161e194ce5b 100644 --- a/query/src/configs/config_storage.rs +++ b/query/src/configs/config_storage.rs @@ -16,7 +16,7 @@ use std::fmt; use std::str::FromStr; use clap::Args; -use common_base::mask_string; +use common_base::base::mask_string; use serde::Deserialize; use serde::Serialize; diff --git a/query/src/databases/database_context.rs b/query/src/databases/database_context.rs index e673774ec9ed..ee73821d94cc 100644 --- a/query/src/databases/database_context.rs +++ b/query/src/databases/database_context.rs @@ -14,8 +14,8 @@ use std::sync::Arc; +use common_base::infallible::RwLock; use common_datablocks::InMemoryData; -use common_infallible::RwLock; use common_meta_api::SchemaApi; /// Database Context. diff --git a/query/src/databases/database_factory.rs b/query/src/databases/database_factory.rs index 83525d399c99..efc974d3a419 100644 --- a/query/src/databases/database_factory.rs +++ b/query/src/databases/database_factory.rs @@ -16,9 +16,9 @@ use std::collections::HashMap; use std::sync::Arc; +use common_base::infallible::RwLock; use common_exception::ErrorCode; use common_exception::Result; -use common_infallible::RwLock; use common_meta_types::DatabaseInfo; use crate::configs::Config; diff --git a/query/src/interpreters/interpreter_factory_interceptor.rs b/query/src/interpreters/interpreter_factory_interceptor.rs index 34d7d529163f..4c7eea17beef 100644 --- a/query/src/interpreters/interpreter_factory_interceptor.rs +++ b/query/src/interpreters/interpreter_factory_interceptor.rs @@ -15,8 +15,8 @@ use std::sync::Arc; use std::time::SystemTime; +use common_base::infallible::Mutex; use common_exception::Result; -use common_infallible::Mutex; use common_planners::PlanNode; use common_streams::ErrorStream; use common_streams::ProgressStream; diff --git a/query/src/interpreters/interpreter_insert.rs b/query/src/interpreters/interpreter_insert.rs index fed8bdad285c..87f572e68f4e 100644 --- a/query/src/interpreters/interpreter_insert.rs +++ b/query/src/interpreters/interpreter_insert.rs @@ -16,12 +16,12 @@ use std::collections::VecDeque; use std::sync::Arc; use chrono_tz::Tz; +use common_base::infallible::Mutex; use common_datavalues::DataType; use common_exception::ErrorCode; use common_exception::Result; use common_functions::scalars::CastFunction; use common_functions::scalars::FunctionContext; -use common_infallible::Mutex; use common_meta_types::GrantObject; use common_meta_types::UserPrivilegeType; use common_planners::InsertInputSource; diff --git a/query/src/interpreters/plan_schedulers/plan_scheduler_stream.rs b/query/src/interpreters/plan_schedulers/plan_scheduler_stream.rs index 157ef19093ec..0b7bf9bc343a 100644 --- a/query/src/interpreters/plan_schedulers/plan_scheduler_stream.rs +++ b/query/src/interpreters/plan_schedulers/plan_scheduler_stream.rs @@ -19,8 +19,8 @@ use std::sync::atomic::Ordering; use std::sync::Arc; use std::task::Context; -use common_base::tokio::macros::support::Pin; -use common_base::tokio::macros::support::Poll; +use common_base::base::tokio::macros::support::Pin; +use common_base::base::tokio::macros::support::Poll; use common_datablocks::DataBlock; use common_exception::Result; use common_meta_types::NodeInfo; diff --git a/query/src/pipelines/new/executor/executor_notify.rs b/query/src/pipelines/new/executor/executor_notify.rs index 608b548e434d..a585d3143f89 100644 --- a/query/src/pipelines/new/executor/executor_notify.rs +++ b/query/src/pipelines/new/executor/executor_notify.rs @@ -16,8 +16,8 @@ use std::sync::atomic::AtomicUsize; use std::sync::atomic::Ordering; use std::sync::Arc; -use common_infallible::Condvar; -use common_infallible::Mutex; +use common_base::infallible::Condvar; +use common_base::infallible::Mutex; struct WorkerNotify { waiting: Mutex, diff --git a/query/src/pipelines/new/executor/executor_tasks.rs b/query/src/pipelines/new/executor/executor_tasks.rs index 929d2cbadc0e..9c09a728d7f7 100644 --- a/query/src/pipelines/new/executor/executor_tasks.rs +++ b/query/src/pipelines/new/executor/executor_tasks.rs @@ -17,8 +17,8 @@ use std::sync::atomic::AtomicBool; use std::sync::atomic::Ordering; use std::sync::Arc; +use common_base::infallible::Mutex; use common_exception::Result; -use common_infallible::Mutex; use petgraph::prelude::NodeIndex; use crate::pipelines::new::executor::executor_worker_context::ExecutorTask; diff --git a/query/src/pipelines/new/executor/executor_worker_context.rs b/query/src/pipelines/new/executor/executor_worker_context.rs index baec6d547ad2..c3cd368c2cfe 100644 --- a/query/src/pipelines/new/executor/executor_worker_context.rs +++ b/query/src/pipelines/new/executor/executor_worker_context.rs @@ -16,7 +16,7 @@ use std::fmt::Debug; use std::fmt::Formatter; use std::sync::Arc; -use common_base::TrySpawn; +use common_base::base::TrySpawn; use common_exception::ErrorCode; use common_exception::Result; use petgraph::prelude::NodeIndex; diff --git a/query/src/pipelines/new/executor/pipeline_complete_executor.rs b/query/src/pipelines/new/executor/pipeline_complete_executor.rs index da1da34468be..cb615c226d6d 100644 --- a/query/src/pipelines/new/executor/pipeline_complete_executor.rs +++ b/query/src/pipelines/new/executor/pipeline_complete_executor.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use common_base::Runtime; +use common_base::base::Runtime; use common_exception::ErrorCode; use common_exception::Result; diff --git a/query/src/pipelines/new/executor/pipeline_executor.rs b/query/src/pipelines/new/executor/pipeline_executor.rs index 97f4bcac0e5d..f7f80376a4fa 100644 --- a/query/src/pipelines/new/executor/pipeline_executor.rs +++ b/query/src/pipelines/new/executor/pipeline_executor.rs @@ -16,8 +16,8 @@ use std::collections::VecDeque; use std::sync::Arc; use std::thread::JoinHandle; -use common_base::Runtime; -use common_base::Thread; +use common_base::base::Runtime; +use common_base::base::Thread; use common_exception::ErrorCode; use common_exception::Result; use common_tracing::tracing; diff --git a/query/src/pipelines/new/executor/pipeline_pulling_executor.rs b/query/src/pipelines/new/executor/pipeline_pulling_executor.rs index 4ab5f8003b7e..585a6594c595 100644 --- a/query/src/pipelines/new/executor/pipeline_pulling_executor.rs +++ b/query/src/pipelines/new/executor/pipeline_pulling_executor.rs @@ -16,7 +16,7 @@ use std::sync::mpsc::Receiver; use std::sync::mpsc::SyncSender; use std::sync::Arc; -use common_base::Runtime; +use common_base::base::Runtime; use common_datablocks::DataBlock; use common_exception::ErrorCode; use common_exception::Result; diff --git a/query/src/pipelines/new/executor/pipeline_pushing_executor.rs b/query/src/pipelines/new/executor/pipeline_pushing_executor.rs index f0c14ebb6c81..fe377d9f9379 100644 --- a/query/src/pipelines/new/executor/pipeline_pushing_executor.rs +++ b/query/src/pipelines/new/executor/pipeline_pushing_executor.rs @@ -18,10 +18,10 @@ use std::sync::mpsc::Receiver; use std::sync::mpsc::SyncSender; use std::sync::Arc; +use common_base::infallible::Mutex; use common_datablocks::DataBlock; use common_exception::ErrorCode; use common_exception::Result; -use common_infallible::Mutex; use crate::pipelines::new::executor::PipelineExecutor; use crate::pipelines::new::processors::port::OutputPort; diff --git a/query/src/pipelines/new/processors/sinks/sync_sink_sender.rs b/query/src/pipelines/new/processors/sinks/sync_sink_sender.rs index 1331f7a04c7d..edaef623dac6 100644 --- a/query/src/pipelines/new/processors/sinks/sync_sink_sender.rs +++ b/query/src/pipelines/new/processors/sinks/sync_sink_sender.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use common_base::tokio::sync::mpsc::Sender; +use common_base::base::tokio::sync::mpsc::Sender; use common_datablocks::DataBlock; use common_exception::Result; diff --git a/query/src/pipelines/new/processors/sources/async_source.rs b/query/src/pipelines/new/processors/sources/async_source.rs index 251a7d27f77c..3ea75d4856b1 100644 --- a/query/src/pipelines/new/processors/sources/async_source.rs +++ b/query/src/pipelines/new/processors/sources/async_source.rs @@ -14,8 +14,8 @@ use std::sync::Arc; -use common_base::Progress; -use common_base::ProgressValues; +use common_base::base::Progress; +use common_base::base::ProgressValues; use common_datablocks::DataBlock; use common_exception::Result; use futures::Future; diff --git a/query/src/pipelines/new/processors/sources/blocks_source.rs b/query/src/pipelines/new/processors/sources/blocks_source.rs index 4a3fec95a33a..ef7a17739c1c 100644 --- a/query/src/pipelines/new/processors/sources/blocks_source.rs +++ b/query/src/pipelines/new/processors/sources/blocks_source.rs @@ -15,9 +15,9 @@ use std::collections::VecDeque; use std::sync::Arc; +use common_base::infallible::Mutex; use common_datablocks::DataBlock; use common_exception::Result; -use common_infallible::Mutex; use crate::pipelines::new::processors::port::OutputPort; use crate::pipelines::new::processors::processor::ProcessorPtr; diff --git a/query/src/pipelines/new/processors/sources/sync_ck_source_receiver.rs b/query/src/pipelines/new/processors/sources/sync_ck_source_receiver.rs index a348811fa89f..92ec4fb9d577 100644 --- a/query/src/pipelines/new/processors/sources/sync_ck_source_receiver.rs +++ b/query/src/pipelines/new/processors/sources/sync_ck_source_receiver.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use common_base::tokio::sync::mpsc::Receiver; +use common_base::base::tokio::sync::mpsc::Receiver; use common_datablocks::DataBlock; use common_datavalues::DataSchemaRef; use common_exception::Result; diff --git a/query/src/pipelines/new/processors/sources/sync_source.rs b/query/src/pipelines/new/processors/sources/sync_source.rs index f42365097469..39b8faed4002 100644 --- a/query/src/pipelines/new/processors/sources/sync_source.rs +++ b/query/src/pipelines/new/processors/sources/sync_source.rs @@ -14,8 +14,8 @@ use std::sync::Arc; -use common_base::Progress; -use common_base::ProgressValues; +use common_base::base::Progress; +use common_base::base::ProgressValues; use common_datablocks::DataBlock; use common_exception::Result; diff --git a/query/src/pipelines/new/processors/sources/sync_source_receiver.rs b/query/src/pipelines/new/processors/sources/sync_source_receiver.rs index 8c641f685dac..bd2407949028 100644 --- a/query/src/pipelines/new/processors/sources/sync_source_receiver.rs +++ b/query/src/pipelines/new/processors/sources/sync_source_receiver.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use common_base::tokio::sync::mpsc::Receiver; +use common_base::base::tokio::sync::mpsc::Receiver; use common_datablocks::DataBlock; use common_exception::Result; diff --git a/query/src/pipelines/new/processors/transforms/transform_create_sets.rs b/query/src/pipelines/new/processors/transforms/transform_create_sets.rs index 44b0ca3c8f46..95b586329106 100644 --- a/query/src/pipelines/new/processors/transforms/transform_create_sets.rs +++ b/query/src/pipelines/new/processors/transforms/transform_create_sets.rs @@ -14,13 +14,13 @@ use std::sync::Arc; +use common_base::infallible::Mutex; use common_datablocks::DataBlock; use common_datavalues::DataSchemaRef; use common_datavalues::DataType; use common_datavalues::DataValue; use common_exception::ErrorCode; use common_exception::Result; -use common_infallible::Mutex; use common_planners::Expression; use common_planners::PlanNode; use common_planners::SelectPlan; diff --git a/query/src/pipelines/processors/processor_merge.rs b/query/src/pipelines/processors/processor_merge.rs index 4c66ee096efe..e6c6484b156c 100644 --- a/query/src/pipelines/processors/processor_merge.rs +++ b/query/src/pipelines/processors/processor_merge.rs @@ -15,8 +15,8 @@ use std::any::Any; use std::sync::Arc; -use common_base::tokio::sync::mpsc; -use common_base::TrySpawn; +use common_base::base::tokio::sync::mpsc; +use common_base::base::TrySpawn; use common_datablocks::DataBlock; use common_exception::ErrorCode; use common_exception::Result; diff --git a/query/src/pipelines/processors/processor_mixed.rs b/query/src/pipelines/processors/processor_mixed.rs index 377a522c4091..23488ac2ece9 100644 --- a/query/src/pipelines/processors/processor_mixed.rs +++ b/query/src/pipelines/processors/processor_mixed.rs @@ -18,12 +18,12 @@ use std::sync::atomic::AtomicUsize; use std::sync::atomic::Ordering; use std::sync::Arc; -use common_base::tokio::sync::mpsc; -use common_base::TrySpawn; +use common_base::base::tokio::sync::mpsc; +use common_base::base::TrySpawn; +use common_base::infallible::RwLock; use common_datablocks::DataBlock; use common_exception::ErrorCode; use common_exception::Result; -use common_infallible::RwLock; use common_streams::SendableDataBlockStream; use common_tracing::tracing; use tokio_stream::wrappers::ReceiverStream; diff --git a/query/src/pipelines/transforms/transform_create_sets.rs b/query/src/pipelines/transforms/transform_create_sets.rs index 2b9fc6ccd21c..ac1cd1d693f7 100644 --- a/query/src/pipelines/transforms/transform_create_sets.rs +++ b/query/src/pipelines/transforms/transform_create_sets.rs @@ -15,13 +15,13 @@ use std::any::Any; use std::sync::Arc; -use common_base::tokio::task::JoinHandle; -use common_base::TrySpawn; +use common_base::base::tokio::task::JoinHandle; +use common_base::base::TrySpawn; +use common_base::infallible::Mutex; use common_datavalues::DataSchemaRef; use common_datavalues::DataValue; use common_exception::ErrorCode; use common_exception::Result; -use common_infallible::Mutex; use common_planners::Expression; use common_streams::SendableDataBlockStream; use common_streams::SubQueriesStream; diff --git a/query/src/pipelines/transforms/transform_group_by_final.rs b/query/src/pipelines/transforms/transform_group_by_final.rs index fae2801058ab..ed6f3543758e 100644 --- a/query/src/pipelines/transforms/transform_group_by_final.rs +++ b/query/src/pipelines/transforms/transform_group_by_final.rs @@ -19,6 +19,7 @@ use std::sync::Arc; use std::time::Instant; use bumpalo::Bump; +use common_base::infallible::RwLock; use common_datablocks::DataBlock; use common_datablocks::HashMethodKind; use common_datavalues::prelude::MutableColumn; @@ -26,7 +27,6 @@ use common_datavalues::prelude::*; use common_exception::Result; use common_functions::aggregates::get_layout_offsets; use common_functions::aggregates::StateAddr; -use common_infallible::RwLock; use common_planners::Expression; use common_streams::DataBlockStream; use common_streams::SendableDataBlockStream; diff --git a/query/src/servers/clickhouse/clickhouse_handler.rs b/query/src/servers/clickhouse/clickhouse_handler.rs index 165df4eef459..3c50165b655d 100644 --- a/query/src/servers/clickhouse/clickhouse_handler.rs +++ b/query/src/servers/clickhouse/clickhouse_handler.rs @@ -15,11 +15,11 @@ use std::net::SocketAddr; use std::sync::Arc; -use common_base::tokio; -use common_base::tokio::net::TcpStream; -use common_base::tokio::task::JoinHandle; -use common_base::Runtime; -use common_base::TrySpawn; +use common_base::base::tokio; +use common_base::base::tokio::net::TcpStream; +use common_base::base::tokio::task::JoinHandle; +use common_base::base::Runtime; +use common_base::base::TrySpawn; use common_exception::ErrorCode; use common_exception::Result; use common_tracing::tracing; diff --git a/query/src/servers/clickhouse/clickhouse_session.rs b/query/src/servers/clickhouse/clickhouse_session.rs index e10f07fa8893..eb3177a245cc 100644 --- a/query/src/servers/clickhouse/clickhouse_session.rs +++ b/query/src/servers/clickhouse/clickhouse_session.rs @@ -14,10 +14,10 @@ use std::net::Shutdown; -use common_base::tokio::net::TcpStream; -use common_base::Runtime; -use common_base::Thread; -use common_base::TrySpawn; +use common_base::base::tokio::net::TcpStream; +use common_base::base::Runtime; +use common_base::base::Thread; +use common_base::base::TrySpawn; use common_exception::ErrorCode; use common_exception::Result; use common_exception::ToErrorCode; diff --git a/query/src/servers/clickhouse/interactive_worker_base.rs b/query/src/servers/clickhouse/interactive_worker_base.rs index 2363f379e1b5..154446f41d3c 100644 --- a/query/src/servers/clickhouse/interactive_worker_base.rs +++ b/query/src/servers/clickhouse/interactive_worker_base.rs @@ -18,11 +18,11 @@ use std::sync::Arc; use std::time::Duration; use std::time::Instant; -use common_base::tokio; -use common_base::tokio::sync::mpsc::channel; -use common_base::tokio::time::interval; -use common_base::ProgressValues; -use common_base::TrySpawn; +use common_base::base::tokio; +use common_base::base::tokio::sync::mpsc::channel; +use common_base::base::tokio::time::interval; +use common_base::base::ProgressValues; +use common_base::base::TrySpawn; use common_datablocks::DataBlock; use common_datavalues::DataSchemaRef; use common_exception::ErrorCode; diff --git a/query/src/servers/clickhouse/reject_connection.rs b/query/src/servers/clickhouse/reject_connection.rs index aa167ad5981e..574177d41652 100644 --- a/query/src/servers/clickhouse/reject_connection.rs +++ b/query/src/servers/clickhouse/reject_connection.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use common_base::tokio::net::TcpStream; +use common_base::base::tokio::net::TcpStream; use common_exception::ErrorCode; use common_exception::Result; use opensrv_clickhouse::connection::Connection; diff --git a/query/src/servers/clickhouse/writers/query_writer.rs b/query/src/servers/clickhouse/writers/query_writer.rs index 51c3ae86a0fa..acb00eb3574c 100644 --- a/query/src/servers/clickhouse/writers/query_writer.rs +++ b/query/src/servers/clickhouse/writers/query_writer.rs @@ -14,7 +14,7 @@ use std::borrow::Cow; -use common_base::ProgressValues; +use common_base::base::ProgressValues; use common_datablocks::DataBlock; use common_datavalues::prelude::*; use common_exception::ErrorCode; diff --git a/query/src/servers/http/v1/http_query_handlers.rs b/query/src/servers/http/v1/http_query_handlers.rs index 59c208f1774d..bcbf01b8a8a4 100644 --- a/query/src/servers/http/v1/http_query_handlers.rs +++ b/query/src/servers/http/v1/http_query_handlers.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::ProgressValues; +use common_base::base::ProgressValues; use common_datavalues::DataSchemaRef; use common_exception::ErrorCode; use common_io::prelude::FormatSettings; diff --git a/query/src/servers/http/v1/load.rs b/query/src/servers/http/v1/load.rs index bee51b267555..d74d452c2870 100644 --- a/query/src/servers/http/v1/load.rs +++ b/query/src/servers/http/v1/load.rs @@ -16,7 +16,7 @@ use std::sync::Arc; use async_compat::CompatExt; use async_stream::stream; -use common_base::ProgressValues; +use common_base::base::ProgressValues; use common_exception::ErrorCode; use common_exception::ToErrorCode; use common_io::prelude::parse_escape_string; diff --git a/query/src/servers/http/v1/query/execute_state.rs b/query/src/servers/http/v1/query/execute_state.rs index fbae18c2e321..7075b43b5833 100644 --- a/query/src/servers/http/v1/query/execute_state.rs +++ b/query/src/servers/http/v1/query/execute_state.rs @@ -16,11 +16,11 @@ use std::sync::Arc; use std::time::Duration; use std::time::Instant; -use common_base::tokio; -use common_base::tokio::sync::mpsc; -use common_base::tokio::sync::RwLock; -use common_base::ProgressValues; -use common_base::TrySpawn; +use common_base::base::tokio; +use common_base::base::tokio::sync::mpsc; +use common_base::base::tokio::sync::RwLock; +use common_base::base::ProgressValues; +use common_base::base::TrySpawn; use common_datablocks::DataBlock; use common_exception::ErrorCode; use common_exception::Result; diff --git a/query/src/servers/http/v1/query/expiring_map.rs b/query/src/servers/http/v1/query/expiring_map.rs index 33c63addf6cd..bd2c2d8bd6c0 100644 --- a/query/src/servers/http/v1/query/expiring_map.rs +++ b/query/src/servers/http/v1/query/expiring_map.rs @@ -18,9 +18,9 @@ use std::hash::Hash; use std::sync::Arc; use std::time::Duration; -use common_base::tokio::task; -use common_base::tokio::time::sleep; -use common_infallible::RwLock; +use common_base::base::tokio::task; +use common_base::base::tokio::time::sleep; +use common_base::infallible::RwLock; use crate::servers::http::v1::query::expirable::Expirable; use crate::servers::http::v1::query::expirable::ExpiringState; diff --git a/query/src/servers/http/v1/query/http_query.rs b/query/src/servers/http/v1/query/http_query.rs index 04fbed64097c..f75034c99679 100644 --- a/query/src/servers/http/v1/query/http_query.rs +++ b/query/src/servers/http/v1/query/http_query.rs @@ -17,10 +17,10 @@ use std::sync::Arc; use std::time::Duration; use std::time::Instant; -use common_base::tokio::sync::mpsc; -use common_base::tokio::sync::Mutex as TokioMutex; -use common_base::tokio::sync::RwLock; -use common_base::ProgressValues; +use common_base::base::tokio::sync::mpsc; +use common_base::base::tokio::sync::Mutex as TokioMutex; +use common_base::base::tokio::sync::RwLock; +use common_base::base::ProgressValues; use common_exception::ErrorCode; use common_exception::Result; use common_io::prelude::FormatSettings; diff --git a/query/src/servers/http/v1/query/http_query_manager.rs b/query/src/servers/http/v1/query/http_query_manager.rs index 5e95c6face96..f4694cc573ff 100644 --- a/query/src/servers/http/v1/query/http_query_manager.rs +++ b/query/src/servers/http/v1/query/http_query_manager.rs @@ -16,11 +16,11 @@ use std::collections::HashMap; use std::sync::Arc; use std::time::Duration; -use common_base::tokio; -use common_base::tokio::sync::RwLock; -use common_base::tokio::time::sleep; +use common_base::base::tokio; +use common_base::base::tokio::sync::RwLock; +use common_base::base::tokio::time::sleep; +use common_base::infallible::Mutex; use common_exception::Result; -use common_infallible::Mutex; use common_tracing::tracing; use super::expiring_map::ExpiringMap; diff --git a/query/src/servers/http/v1/query/result_data_manager.rs b/query/src/servers/http/v1/query/result_data_manager.rs index 010a53203ff2..c18ba9ef8640 100644 --- a/query/src/servers/http/v1/query/result_data_manager.rs +++ b/query/src/servers/http/v1/query/result_data_manager.rs @@ -14,9 +14,9 @@ use std::time::Instant; -use common_base::tokio; -use common_base::tokio::sync::mpsc; -use common_base::tokio::sync::mpsc::error::TryRecvError; +use common_base::base::tokio; +use common_base::base::tokio::sync::mpsc; +use common_base::base::tokio::sync::mpsc::error::TryRecvError; use common_datablocks::DataBlock; use common_exception::ErrorCode; use common_exception::Result; diff --git a/query/src/servers/mysql/mysql_handler.rs b/query/src/servers/mysql/mysql_handler.rs index c4a8419eeaf3..1619d256141d 100644 --- a/query/src/servers/mysql/mysql_handler.rs +++ b/query/src/servers/mysql/mysql_handler.rs @@ -16,11 +16,11 @@ use std::future::Future; use std::net::SocketAddr; use std::sync::Arc; -use common_base::tokio; -use common_base::tokio::net::TcpStream; -use common_base::tokio::task::JoinHandle; -use common_base::Runtime; -use common_base::TrySpawn; +use common_base::base::tokio; +use common_base::base::tokio::net::TcpStream; +use common_base::base::tokio::task::JoinHandle; +use common_base::base::Runtime; +use common_base::base::TrySpawn; use common_exception::ErrorCode; use common_exception::Result; use common_tracing::tracing; diff --git a/query/src/servers/mysql/mysql_interactive_worker.rs b/query/src/servers/mysql/mysql_interactive_worker.rs index 53079372d0af..b94c1885fc16 100644 --- a/query/src/servers/mysql/mysql_interactive_worker.rs +++ b/query/src/servers/mysql/mysql_interactive_worker.rs @@ -16,7 +16,7 @@ use std::marker::PhantomData; use std::sync::Arc; use std::time::Instant; -use common_base::TrySpawn; +use common_base::base::TrySpawn; use common_datablocks::DataBlock; use common_exception::ErrorCode; use common_exception::Result; diff --git a/query/src/servers/mysql/mysql_session.rs b/query/src/servers/mysql/mysql_session.rs index c2cbb5c87326..f823122ea20d 100644 --- a/query/src/servers/mysql/mysql_session.rs +++ b/query/src/servers/mysql/mysql_session.rs @@ -14,10 +14,10 @@ use std::net::Shutdown; -use common_base::tokio::net::TcpStream; -use common_base::Runtime; -use common_base::Thread; -use common_base::TrySpawn; +use common_base::base::tokio::net::TcpStream; +use common_base::base::Runtime; +use common_base::base::Thread; +use common_base::base::TrySpawn; use common_exception::ErrorCode; use common_exception::Result; use common_exception::ToErrorCode; diff --git a/query/src/servers/mysql/reject_connection.rs b/query/src/servers/mysql/reject_connection.rs index 06e9e949f145..0ef9423d3eab 100644 --- a/query/src/servers/mysql/reject_connection.rs +++ b/query/src/servers/mysql/reject_connection.rs @@ -12,9 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio::io::AsyncReadExt; -use common_base::tokio::io::AsyncWriteExt; -use common_base::tokio::net::TcpStream; +use common_base::base::tokio::io::AsyncReadExt; +use common_base::base::tokio::io::AsyncWriteExt; +use common_base::base::tokio::net::TcpStream; use common_exception::Result; use opensrv_mysql::ErrorKind; diff --git a/query/src/servers/server.rs b/query/src/servers/server.rs index 01fa32116790..3e5981ccaabd 100644 --- a/query/src/servers/server.rs +++ b/query/src/servers/server.rs @@ -17,10 +17,10 @@ use std::sync::atomic::AtomicBool; use std::sync::atomic::Ordering; use std::sync::Arc; -use common_base::signal_stream; -use common_base::DummySignalStream; -use common_base::SignalStream; -use common_base::SignalType; +use common_base::base::signal_stream; +use common_base::base::DummySignalStream; +use common_base::base::SignalStream; +use common_base::base::SignalType; use common_exception::Result; use common_tracing::tracing; use futures::stream::Abortable; diff --git a/query/src/sessions/query_ctx.rs b/query/src/sessions/query_ctx.rs index 9184806442b6..07cef1a49393 100644 --- a/query/src/sessions/query_ctx.rs +++ b/query/src/sessions/query_ctx.rs @@ -20,19 +20,19 @@ use std::sync::atomic::Ordering::Acquire; use std::sync::Arc; use chrono_tz::Tz; -use common_base::tokio::task::JoinHandle; -use common_base::Progress; -use common_base::ProgressValues; -use common_base::Runtime; -use common_base::TrySpawn; +use common_base::base::tokio::task::JoinHandle; +use common_base::base::Progress; +use common_base::base::ProgressValues; +use common_base::base::Runtime; +use common_base::base::TrySpawn; +use common_base::infallible::Mutex; +use common_base::infallible::RwLock; use common_contexts::DalContext; use common_contexts::DalMetrics; use common_datablocks::DataBlock; use common_exception::ErrorCode; use common_exception::Result; use common_functions::scalars::FunctionContext; -use common_infallible::Mutex; -use common_infallible::RwLock; use common_io::prelude::FormatSettings; use common_meta_types::TableInfo; use common_meta_types::UserInfo; diff --git a/query/src/sessions/query_ctx_shared.rs b/query/src/sessions/query_ctx_shared.rs index b04cc76fd290..bfd1c3fd4d35 100644 --- a/query/src/sessions/query_ctx_shared.rs +++ b/query/src/sessions/query_ctx_shared.rs @@ -18,13 +18,13 @@ use std::sync::atomic::AtomicUsize; use std::sync::Arc; use chrono_tz::Tz; -use common_base::Progress; -use common_base::Runtime; +use common_base::base::Progress; +use common_base::base::Runtime; +use common_base::infallible::Mutex; +use common_base::infallible::RwLock; use common_contexts::DalContext; use common_exception::ErrorCode; use common_exception::Result; -use common_infallible::Mutex; -use common_infallible::RwLock; use common_io::prelude::FormatSettings; use common_meta_types::UserInfo; use common_planners::PlanNode; diff --git a/query/src/sessions/session.rs b/query/src/sessions/session.rs index 725577fd9c31..84d90f81d29b 100644 --- a/query/src/sessions/session.rs +++ b/query/src/sessions/session.rs @@ -16,11 +16,11 @@ use std::net::SocketAddr; use std::sync::atomic::AtomicUsize; use std::sync::Arc; +use common_base::infallible::RwLock; +use common_base::mem_allocator::malloc_size; use common_exception::ErrorCode; use common_exception::Result; -use common_infallible::RwLock; use common_macros::MallocSizeOf; -use common_mem_allocator::malloc_size; use common_meta_types::GrantObject; use common_meta_types::UserInfo; use common_meta_types::UserPrivilegeType; @@ -146,7 +146,7 @@ impl Session { self.session_ctx.set_client_host(host); self.session_ctx.set_io_shutdown_tx(Some(tx)); - common_base::tokio::spawn(async move { + common_base::base::tokio::spawn(async move { if let Ok(tx) = rx.await { (io_shutdown)(); tx.send(()).ok(); diff --git a/query/src/sessions/session_ctx.rs b/query/src/sessions/session_ctx.rs index 4e5c3843d782..22e4f879931f 100644 --- a/query/src/sessions/session_ctx.rs +++ b/query/src/sessions/session_ctx.rs @@ -17,8 +17,8 @@ use std::sync::atomic::AtomicBool; use std::sync::atomic::Ordering; use std::sync::Arc; +use common_base::infallible::RwLock; use common_exception::Result; -use common_infallible::RwLock; use common_macros::MallocSizeOf; use common_meta_types::UserInfo; use futures::channel::oneshot::Sender; diff --git a/query/src/sessions/session_info.rs b/query/src/sessions/session_info.rs index 25c9a291b793..c426f991d08b 100644 --- a/query/src/sessions/session_info.rs +++ b/query/src/sessions/session_info.rs @@ -15,7 +15,7 @@ use std::net::SocketAddr; use std::sync::Arc; -use common_base::ProgressValues; +use common_base::base::ProgressValues; use common_contexts::DalMetrics; use common_meta_types::UserInfo; diff --git a/query/src/sessions/session_mgr.rs b/query/src/sessions/session_mgr.rs index 2100d1f43e4e..a959e3b96fee 100644 --- a/query/src/sessions/session_mgr.rs +++ b/query/src/sessions/session_mgr.rs @@ -19,13 +19,13 @@ use std::str::FromStr; use std::sync::Arc; use std::time::Duration; -use common_base::tokio; -use common_base::Runtime; -use common_base::SignalStream; +use common_base::base::tokio; +use common_base::base::Runtime; +use common_base::base::SignalStream; +use common_base::infallible::RwLock; use common_contexts::DalRuntime; use common_exception::ErrorCode; use common_exception::Result; -use common_infallible::RwLock; use common_metrics::label_counter; use common_tracing::init_query_logger; use common_tracing::tracing; diff --git a/query/src/sessions/session_settings.rs b/query/src/sessions/session_settings.rs index 28782a283410..099d85e9f0b7 100644 --- a/query/src/sessions/session_settings.rs +++ b/query/src/sessions/session_settings.rs @@ -18,10 +18,10 @@ use std::fmt::Debug; use std::fmt::Formatter; use std::sync::Arc; +use common_base::infallible::RwLock; use common_datavalues::prelude::*; use common_exception::ErrorCode; use common_exception::Result; -use common_infallible::RwLock; use common_meta_types::UserSetting; use itertools::Itertools; diff --git a/query/src/storages/fuse/cache/memory_cache.rs b/query/src/storages/fuse/cache/memory_cache.rs index 24eb26e6b842..c2b8dbe96208 100644 --- a/query/src/storages/fuse/cache/memory_cache.rs +++ b/query/src/storages/fuse/cache/memory_cache.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use common_base::tokio::sync::RwLock; +use common_base::base::tokio::sync::RwLock; use common_cache::Count; use common_cache::DefaultHashBuilder; use common_cache::LruCache; diff --git a/query/src/storages/fuse/io/read/block_reader.rs b/query/src/storages/fuse/io/read/block_reader.rs index e5e655631393..f5de9a7e28f9 100644 --- a/query/src/storages/fuse/io/read/block_reader.rs +++ b/query/src/storages/fuse/io/read/block_reader.rs @@ -200,7 +200,7 @@ impl BlockReader { } async fn read_column(o: Object, offset: u64, length: u64) -> Result> { - let handler = common_base::tokio::spawn(async move { + let handler = common_base::base::tokio::spawn(async move { let mut chunk = vec![0; length as usize]; let mut r = o.range_reader(offset..offset + length).await?; r.read_exact(&mut chunk).await?; diff --git a/query/src/storages/fuse/operations/commit.rs b/query/src/storages/fuse/operations/commit.rs index 96c20f8eb292..45d68d5a3626 100644 --- a/query/src/storages/fuse/operations/commit.rs +++ b/query/src/storages/fuse/operations/commit.rs @@ -20,7 +20,7 @@ use std::time::Instant; use backoff::backoff::Backoff; use backoff::ExponentialBackoffBuilder; -use common_base::ProgressValues; +use common_base::base::ProgressValues; use common_cache::Cache; use common_datavalues::DataSchema; use common_exception::ErrorCode; @@ -103,7 +103,7 @@ impl FuseTable { name.as_str(), tbl.table_info.ident ); - common_base::tokio::time::sleep(d).await; + common_base::base::tokio::time::sleep(d).await; let catalog = ctx.get_catalog(); let (ident, meta) = catalog.get_table_meta_by_id(tid).await?; diff --git a/query/src/storages/fuse/operations/read.rs b/query/src/storages/fuse/operations/read.rs index 2ff861b48cfb..0d8e2d12fb07 100644 --- a/query/src/storages/fuse/operations/read.rs +++ b/query/src/storages/fuse/operations/read.rs @@ -15,8 +15,8 @@ use std::sync::Arc; -use common_base::Progress; -use common_base::ProgressValues; +use common_base::base::Progress; +use common_base::base::ProgressValues; use common_datablocks::DataBlock; use common_exception::ErrorCode; use common_exception::Result; diff --git a/query/src/storages/memory/memory_table.rs b/query/src/storages/memory/memory_table.rs index e672f500f520..8e67b35c99e2 100644 --- a/query/src/storages/memory/memory_table.rs +++ b/query/src/storages/memory/memory_table.rs @@ -18,11 +18,11 @@ use std::collections::HashSet; use std::collections::VecDeque; use std::sync::Arc; +use common_base::infallible::Mutex; +use common_base::infallible::RwLock; use common_datablocks::DataBlock; use common_datavalues::ColumnRef; use common_exception::Result; -use common_infallible::Mutex; -use common_infallible::RwLock; use common_meta_types::TableInfo; use common_planners::Extras; use common_planners::Partitions; diff --git a/query/src/storages/s3/s3_stage_source.rs b/query/src/storages/s3/s3_stage_source.rs index 56251992328a..e4077e32d355 100644 --- a/query/src/storages/s3/s3_stage_source.rs +++ b/query/src/storages/s3/s3_stage_source.rs @@ -16,11 +16,11 @@ use std::collections::VecDeque; use std::future::Future; use std::sync::Arc; +use common_base::infallible::Mutex; use common_datablocks::DataBlock; use common_datavalues::DataSchemaRef; use common_exception::ErrorCode; use common_exception::Result; -use common_infallible::Mutex; use common_io::prelude::S3File; use common_meta_types::StageFileFormatType; use common_meta_types::StageStorage; diff --git a/query/src/storages/s3/s3_stage_table.rs b/query/src/storages/s3/s3_stage_table.rs index 787744779af3..96002a8f224d 100644 --- a/query/src/storages/s3/s3_stage_table.rs +++ b/query/src/storages/s3/s3_stage_table.rs @@ -16,9 +16,9 @@ use std::any::Any; use std::collections::VecDeque; use std::sync::Arc; +use common_base::infallible::Mutex; use common_exception::ErrorCode; use common_exception::Result; -use common_infallible::Mutex; use common_meta_types::TableInfo; use common_planners::Extras; use common_planners::Partitions; diff --git a/query/src/storages/storage_context.rs b/query/src/storages/storage_context.rs index 5671c9e448db..c87fdcbb2c18 100644 --- a/query/src/storages/storage_context.rs +++ b/query/src/storages/storage_context.rs @@ -14,8 +14,8 @@ use std::sync::Arc; +use common_base::infallible::RwLock; use common_datablocks::InMemoryData; -use common_infallible::RwLock; use common_meta_api::SchemaApi; /// Storage Context. diff --git a/query/src/storages/storage_factory.rs b/query/src/storages/storage_factory.rs index 9cdb0b8a2bc7..4b27475ab6be 100644 --- a/query/src/storages/storage_factory.rs +++ b/query/src/storages/storage_factory.rs @@ -15,9 +15,9 @@ use std::collections::HashMap; use std::sync::Arc; +use common_base::infallible::RwLock; use common_exception::ErrorCode; use common_exception::Result; -use common_infallible::RwLock; use common_meta_types::TableInfo; use crate::configs::Config; diff --git a/query/src/storages/system/configs_table.rs b/query/src/storages/system/configs_table.rs index 9027a9bf4b7f..c00e1eafd250 100644 --- a/query/src/storages/system/configs_table.rs +++ b/query/src/storages/system/configs_table.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use common_base::mask_string; +use common_base::base::mask_string; use common_datablocks::DataBlock; use common_datavalues::prelude::*; use common_exception::Result; diff --git a/query/src/storages/system/processes_table.rs b/query/src/storages/system/processes_table.rs index fd471e54bd4e..f85bfba2d185 100644 --- a/query/src/storages/system/processes_table.rs +++ b/query/src/storages/system/processes_table.rs @@ -15,7 +15,7 @@ use std::net::SocketAddr; use std::sync::Arc; -use common_base::ProgressValues; +use common_base::base::ProgressValues; use common_contexts::DalMetrics; use common_datablocks::DataBlock; use common_datavalues::prelude::*; diff --git a/query/src/storages/system/query_log_table.rs b/query/src/storages/system/query_log_table.rs index cd9681288ceb..85c325b8c2b0 100644 --- a/query/src/storages/system/query_log_table.rs +++ b/query/src/storages/system/query_log_table.rs @@ -16,10 +16,10 @@ use std::any::Any; use std::collections::VecDeque; use std::sync::Arc; +use common_base::infallible::RwLock; use common_datablocks::DataBlock; use common_datavalues::prelude::*; use common_exception::Result; -use common_infallible::RwLock; use common_meta_types::TableIdent; use common_meta_types::TableInfo; use common_meta_types::TableMeta; diff --git a/query/src/table_functions/table_function_factory.rs b/query/src/table_functions/table_function_factory.rs index b1793c6dfe77..5c55486b4f48 100644 --- a/query/src/table_functions/table_function_factory.rs +++ b/query/src/table_functions/table_function_factory.rs @@ -16,9 +16,9 @@ use std::collections::HashMap; use std::sync::Arc; +use common_base::infallible::RwLock; use common_exception::ErrorCode; use common_exception::Result; -use common_infallible::RwLock; use common_meta_types::MetaId; use common_planners::Expression; diff --git a/query/src/users/role_cache_mgr.rs b/query/src/users/role_cache_mgr.rs index 4d7243535ad1..ecdddfe54a12 100644 --- a/query/src/users/role_cache_mgr.rs +++ b/query/src/users/role_cache_mgr.rs @@ -19,10 +19,10 @@ use std::sync::Arc; use std::time::Duration; use std::time::Instant; -use common_base::tokio; -use common_base::tokio::task::JoinHandle; +use common_base::base::tokio; +use common_base::base::tokio::task::JoinHandle; +use common_base::infallible::RwLock; use common_exception::Result; -use common_infallible::RwLock; use common_meta_types::RoleInfo; use common_tracing::tracing; diff --git a/query/tests/it/api/http/cluster.rs b/query/tests/it/api/http/cluster.rs index 0d94b5dfc876..e88b2a14face 100644 --- a/query/tests/it/api/http/cluster.rs +++ b/query/tests/it/api/http/cluster.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use common_meta_types::NodeInfo; use databend_query::api::http::v1::cluster::*; diff --git a/query/tests/it/api/http/config.rs b/query/tests/it/api/http/config.rs index a884d396ca5c..9775204ee37b 100644 --- a/query/tests/it/api/http/config.rs +++ b/query/tests/it/api/http/config.rs @@ -14,7 +14,7 @@ * limitations under the License. * */ -use common_base::tokio; +use common_base::base::tokio; use databend_query::api::http::v1::config::config_handler; use poem::get; use poem::http::Method; diff --git a/query/tests/it/api/http/health.rs b/query/tests/it/api/http/health.rs index 0efd756f50f7..9e684fd3df2a 100644 --- a/query/tests/it/api/http/health.rs +++ b/query/tests/it/api/http/health.rs @@ -14,7 +14,7 @@ * limitations under the License. * */ -use common_base::tokio; +use common_base::base::tokio; use databend_query::api::http::v1::health::health_handler; use poem::get; use poem::http::Method; diff --git a/query/tests/it/api/http/logs.rs b/query/tests/it/api/http/logs.rs index 54b5662f1ade..2abf40f68ae4 100644 --- a/query/tests/it/api/http/logs.rs +++ b/query/tests/it/api/http/logs.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::api::http::v1::logs::logs_handler; use poem::get; diff --git a/query/tests/it/api/http/status.rs b/query/tests/it/api/http/status.rs index 7529f939522d..70558f9cb3e9 100644 --- a/query/tests/it/api/http/status.rs +++ b/query/tests/it/api/http/status.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use common_meta_types::UserIdentity; use databend_query::api::http::v1::status::status_handler; diff --git a/query/tests/it/api/http_service.rs b/query/tests/it/api/http_service.rs index a3f952ddb4fe..2ca2f5c9c1f0 100644 --- a/query/tests/it/api/http_service.rs +++ b/query/tests/it/api/http_service.rs @@ -15,8 +15,8 @@ use std::fs::File; use std::io::Read; -use common_base::get_free_tcp_port; -use common_base::tokio; +use common_base::base::get_free_tcp_port; +use common_base::base::tokio; use common_exception::Result; use databend_query::api::HttpService; use databend_query::servers::Server; diff --git a/query/tests/it/api/rpc/flight_actions.rs b/query/tests/it/api/rpc/flight_actions.rs index 00f4af127f45..ab14b1b8edfa 100644 --- a/query/tests/it/api/rpc/flight_actions.rs +++ b/query/tests/it/api/rpc/flight_actions.rs @@ -15,7 +15,7 @@ use std::convert::TryInto; use common_arrow::arrow_format::flight::data::Action; -use common_base::tokio; +use common_base::base::tokio; use common_datavalues::DataValue; use common_exception::Result; use common_planners::Expression; diff --git a/query/tests/it/api/rpc/flight_dispatcher.rs b/query/tests/it/api/rpc/flight_dispatcher.rs index 4c6e421d251f..cdc5118631e8 100644 --- a/query/tests/it/api/rpc/flight_dispatcher.rs +++ b/query/tests/it/api/rpc/flight_dispatcher.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_datablocks::assert_blocks_eq; use common_datavalues::DataValue; use common_exception::Result; diff --git a/query/tests/it/api/rpc/flight_service.rs b/query/tests/it/api/rpc/flight_service.rs index f9e38d136927..47097780291b 100644 --- a/query/tests/it/api/rpc/flight_service.rs +++ b/query/tests/it/api/rpc/flight_service.rs @@ -18,7 +18,7 @@ use std::sync::Arc; use common_arrow::arrow_format::flight::data::Action; use common_arrow::arrow_format::flight::data::Ticket; use common_arrow::arrow_format::flight::service::flight_service_server::FlightService; -use common_base::tokio; +use common_base::base::tokio; use common_datavalues::DataValue; use common_exception::ErrorCode; use common_exception::Result; diff --git a/query/tests/it/api/rpc/flight_tickets.rs b/query/tests/it/api/rpc/flight_tickets.rs index 7d9b0294ac97..36df623c1a6c 100644 --- a/query/tests/it/api/rpc/flight_tickets.rs +++ b/query/tests/it/api/rpc/flight_tickets.rs @@ -15,7 +15,7 @@ use std::convert::TryInto; use common_arrow::arrow_format::flight::data::Ticket; -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::api::FlightTicket; use databend_query::api::StreamTicket; diff --git a/query/tests/it/api/rpc_service.rs b/query/tests/it/api/rpc_service.rs index 74b304c98e5a..a11abd21293c 100644 --- a/query/tests/it/api/rpc_service.rs +++ b/query/tests/it/api/rpc_service.rs @@ -18,9 +18,9 @@ use std::sync::Arc; use common_arrow::arrow_format::flight::data::Empty; use common_arrow::arrow_format::flight::service::flight_service_client::FlightServiceClient; -use common_base::tokio; -use common_base::tokio::net::TcpListener; -use common_base::tokio::sync::Notify; +use common_base::base::tokio; +use common_base::base::tokio::net::TcpListener; +use common_base::base::tokio::sync::Notify; use common_exception::ErrorCode; use common_exception::Result; use common_grpc::ConnectionFactory; diff --git a/query/tests/it/catalogs/database_catalog.rs b/query/tests/it/catalogs/database_catalog.rs index 2cbdb592ea5a..f3dd08274e2e 100644 --- a/query/tests/it/catalogs/database_catalog.rs +++ b/query/tests/it/catalogs/database_catalog.rs @@ -15,7 +15,7 @@ use std::sync::Arc; use chrono::Utc; -use common_base::tokio; +use common_base::base::tokio; use common_datavalues::prelude::*; use common_exception::Result; use common_meta_types::CreateDatabaseReq; diff --git a/query/tests/it/catalogs/immutable_catalogs.rs b/query/tests/it/catalogs/immutable_catalogs.rs index 76326a374cab..b9f3c33274fe 100644 --- a/query/tests/it/catalogs/immutable_catalogs.rs +++ b/query/tests/it/catalogs/immutable_catalogs.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use common_meta_types::CreateDatabaseReq; use common_meta_types::DatabaseNameIdent; diff --git a/query/tests/it/clusters.rs b/query/tests/it/clusters.rs index e1800c75b2cb..24dabfb85152 100644 --- a/query/tests/it/clusters.rs +++ b/query/tests/it/clusters.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::clusters::ClusterDiscovery; use pretty_assertions::assert_eq; diff --git a/query/tests/it/common/expression_evaluator.rs b/query/tests/it/common/expression_evaluator.rs index 797a9cba2fa4..8e446f11aafb 100644 --- a/query/tests/it/common/expression_evaluator.rs +++ b/query/tests/it/common/expression_evaluator.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_datablocks::DataBlock; use common_datavalues::Column; use common_datavalues::DataField; diff --git a/query/tests/it/functions/context_function.rs b/query/tests/it/functions/context_function.rs index 1323cf924872..1b20b9f32c34 100644 --- a/query/tests/it/functions/context_function.rs +++ b/query/tests/it/functions/context_function.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::procedures::ContextFunction; diff --git a/query/tests/it/interpreters/access/management_mode_access.rs b/query/tests/it/interpreters/access/management_mode_access.rs index a53f6a6312ae..5c4f63185b27 100644 --- a/query/tests/it/interpreters/access/management_mode_access.rs +++ b/query/tests/it/interpreters/access/management_mode_access.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::interpreters::InterpreterFactory; use databend_query::sql::PlanParser; diff --git a/query/tests/it/interpreters/interpreter_call.rs b/query/tests/it/interpreters/interpreter_call.rs index f9abea3e16a9..4b0737a5ede2 100644 --- a/query/tests/it/interpreters/interpreter_call.rs +++ b/query/tests/it/interpreters/interpreter_call.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::ErrorCode; use common_exception::Result; use common_meta_types::GrantObject; diff --git a/query/tests/it/interpreters/interpreter_database_create.rs b/query/tests/it/interpreters/interpreter_database_create.rs index af13ad7d24d8..f1d7220b6aad 100644 --- a/query/tests/it/interpreters/interpreter_database_create.rs +++ b/query/tests/it/interpreters/interpreter_database_create.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::interpreters::*; use databend_query::sql::PlanParser; diff --git a/query/tests/it/interpreters/interpreter_database_drop.rs b/query/tests/it/interpreters/interpreter_database_drop.rs index 21f727cce86e..136151a3ae8e 100644 --- a/query/tests/it/interpreters/interpreter_database_drop.rs +++ b/query/tests/it/interpreters/interpreter_database_drop.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::interpreters::*; use databend_query::sql::PlanParser; diff --git a/query/tests/it/interpreters/interpreter_database_show_create.rs b/query/tests/it/interpreters/interpreter_database_show_create.rs index e84e0a5fcc7e..5a28e07e9910 100644 --- a/query/tests/it/interpreters/interpreter_database_show_create.rs +++ b/query/tests/it/interpreters/interpreter_database_show_create.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::interpreters::*; use databend_query::sql::PlanParser; diff --git a/query/tests/it/interpreters/interpreter_empty.rs b/query/tests/it/interpreters/interpreter_empty.rs index dd6dd5b1d07e..bd96a2877876 100644 --- a/query/tests/it/interpreters/interpreter_empty.rs +++ b/query/tests/it/interpreters/interpreter_empty.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::interpreters::*; use databend_query::sql::*; diff --git a/query/tests/it/interpreters/interpreter_explain.rs b/query/tests/it/interpreters/interpreter_explain.rs index aa29f030c344..4735b47aea59 100644 --- a/query/tests/it/interpreters/interpreter_explain.rs +++ b/query/tests/it/interpreters/interpreter_explain.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::interpreters::*; use databend_query::sql::PlanParser; diff --git a/query/tests/it/interpreters/interpreter_factory_interceptor.rs b/query/tests/it/interpreters/interpreter_factory_interceptor.rs index 9776cd2ac480..650af9180ac9 100644 --- a/query/tests/it/interpreters/interpreter_factory_interceptor.rs +++ b/query/tests/it/interpreters/interpreter_factory_interceptor.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::interpreters::*; use databend_query::sql::*; diff --git a/query/tests/it/interpreters/interpreter_insert.rs b/query/tests/it/interpreters/interpreter_insert.rs index 1aa4de8b6d9b..5932b349a4cc 100644 --- a/query/tests/it/interpreters/interpreter_insert.rs +++ b/query/tests/it/interpreters/interpreter_insert.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::interpreters::*; use databend_query::sql::*; diff --git a/query/tests/it/interpreters/interpreter_privilege_grant.rs b/query/tests/it/interpreters/interpreter_privilege_grant.rs index 47e4efff1ee8..c94bda89a2aa 100644 --- a/query/tests/it/interpreters/interpreter_privilege_grant.rs +++ b/query/tests/it/interpreters/interpreter_privilege_grant.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use common_meta_types::AuthInfo; use common_meta_types::GrantObject; diff --git a/query/tests/it/interpreters/interpreter_privilege_revoke.rs b/query/tests/it/interpreters/interpreter_privilege_revoke.rs index b691e856b6ac..f5be2f7a4a0a 100644 --- a/query/tests/it/interpreters/interpreter_privilege_revoke.rs +++ b/query/tests/it/interpreters/interpreter_privilege_revoke.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use common_meta_types::AuthInfo; use common_meta_types::GrantObject; diff --git a/query/tests/it/interpreters/interpreter_role_grant.rs b/query/tests/it/interpreters/interpreter_role_grant.rs index 0e78f1673bbd..278196bc7088 100644 --- a/query/tests/it/interpreters/interpreter_role_grant.rs +++ b/query/tests/it/interpreters/interpreter_role_grant.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::ErrorCode; use common_exception::Result; use common_meta_types::RoleInfo; diff --git a/query/tests/it/interpreters/interpreter_role_revoke.rs b/query/tests/it/interpreters/interpreter_role_revoke.rs index 7ed5f5fcf9dc..030c626bde27 100644 --- a/query/tests/it/interpreters/interpreter_role_revoke.rs +++ b/query/tests/it/interpreters/interpreter_role_revoke.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::ErrorCode; use common_exception::Result; use common_meta_types::RoleInfo; diff --git a/query/tests/it/interpreters/interpreter_select.rs b/query/tests/it/interpreters/interpreter_select.rs index 0ae9a1b94407..1481c9396489 100644 --- a/query/tests/it/interpreters/interpreter_select.rs +++ b/query/tests/it/interpreters/interpreter_select.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::interpreters::*; use databend_query::sql::*; diff --git a/query/tests/it/interpreters/interpreter_setting.rs b/query/tests/it/interpreters/interpreter_setting.rs index c16f34e89a21..92c6866df6ce 100644 --- a/query/tests/it/interpreters/interpreter_setting.rs +++ b/query/tests/it/interpreters/interpreter_setting.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::interpreters::*; use databend_query::sql::PlanParser; diff --git a/query/tests/it/interpreters/interpreter_show_databases.rs b/query/tests/it/interpreters/interpreter_show_databases.rs index 7e2898600821..4bd9132722eb 100644 --- a/query/tests/it/interpreters/interpreter_show_databases.rs +++ b/query/tests/it/interpreters/interpreter_show_databases.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::interpreters::*; use databend_query::sql::PlanParser; diff --git a/query/tests/it/interpreters/interpreter_show_engines.rs b/query/tests/it/interpreters/interpreter_show_engines.rs index 77126f024857..827f29cc1128 100644 --- a/query/tests/it/interpreters/interpreter_show_engines.rs +++ b/query/tests/it/interpreters/interpreter_show_engines.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::interpreters::*; use databend_query::sql::PlanParser; diff --git a/query/tests/it/interpreters/interpreter_show_functions.rs b/query/tests/it/interpreters/interpreter_show_functions.rs index d8b9db3eb79a..30e6f778422d 100644 --- a/query/tests/it/interpreters/interpreter_show_functions.rs +++ b/query/tests/it/interpreters/interpreter_show_functions.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::interpreters::*; use databend_query::sql::PlanParser; diff --git a/query/tests/it/interpreters/interpreter_show_grant.rs b/query/tests/it/interpreters/interpreter_show_grant.rs index 837234cb5a5e..e7da5d38335f 100644 --- a/query/tests/it/interpreters/interpreter_show_grant.rs +++ b/query/tests/it/interpreters/interpreter_show_grant.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use common_meta_types::GrantObject; use common_meta_types::RoleInfo; diff --git a/query/tests/it/interpreters/interpreter_show_metrics.rs b/query/tests/it/interpreters/interpreter_show_metrics.rs index 469f82fc7bb5..da524b14dadd 100644 --- a/query/tests/it/interpreters/interpreter_show_metrics.rs +++ b/query/tests/it/interpreters/interpreter_show_metrics.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use common_metrics::init_default_metrics_recorder; use databend_query::interpreters::*; diff --git a/query/tests/it/interpreters/interpreter_show_processlist.rs b/query/tests/it/interpreters/interpreter_show_processlist.rs index 3d8b301572b6..ce273054ca72 100644 --- a/query/tests/it/interpreters/interpreter_show_processlist.rs +++ b/query/tests/it/interpreters/interpreter_show_processlist.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::interpreters::*; use databend_query::sql::PlanParser; diff --git a/query/tests/it/interpreters/interpreter_show_roles.rs b/query/tests/it/interpreters/interpreter_show_roles.rs index 19e3999a2bd2..789a89a7ce03 100644 --- a/query/tests/it/interpreters/interpreter_show_roles.rs +++ b/query/tests/it/interpreters/interpreter_show_roles.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::interpreters::*; use databend_query::sql::PlanParser; diff --git a/query/tests/it/interpreters/interpreter_show_settings.rs b/query/tests/it/interpreters/interpreter_show_settings.rs index 266ea2e109eb..a9c6188c7b3f 100644 --- a/query/tests/it/interpreters/interpreter_show_settings.rs +++ b/query/tests/it/interpreters/interpreter_show_settings.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::interpreters::*; use databend_query::sql::PlanParser; diff --git a/query/tests/it/interpreters/interpreter_show_tab_stat.rs b/query/tests/it/interpreters/interpreter_show_tab_stat.rs index d007197f137a..cbff47ffff63 100644 --- a/query/tests/it/interpreters/interpreter_show_tab_stat.rs +++ b/query/tests/it/interpreters/interpreter_show_tab_stat.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::interpreters::*; use databend_query::sql::PlanParser; diff --git a/query/tests/it/interpreters/interpreter_show_tables.rs b/query/tests/it/interpreters/interpreter_show_tables.rs index a7ed39b9a4cb..e7489a1bd32b 100644 --- a/query/tests/it/interpreters/interpreter_show_tables.rs +++ b/query/tests/it/interpreters/interpreter_show_tables.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::interpreters::*; use databend_query::sql::PlanParser; diff --git a/query/tests/it/interpreters/interpreter_show_users.rs b/query/tests/it/interpreters/interpreter_show_users.rs index dc7107a03e2f..c362589bd4c3 100644 --- a/query/tests/it/interpreters/interpreter_show_users.rs +++ b/query/tests/it/interpreters/interpreter_show_users.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::interpreters::*; use databend_query::sql::PlanParser; diff --git a/query/tests/it/interpreters/interpreter_table_create.rs b/query/tests/it/interpreters/interpreter_table_create.rs index 8430d3fdfeee..fe4860944852 100644 --- a/query/tests/it/interpreters/interpreter_table_create.rs +++ b/query/tests/it/interpreters/interpreter_table_create.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::interpreters::*; use databend_query::sql::PlanParser; diff --git a/query/tests/it/interpreters/interpreter_table_describe.rs b/query/tests/it/interpreters/interpreter_table_describe.rs index e4be0d46a04d..3e452fc9f118 100644 --- a/query/tests/it/interpreters/interpreter_table_describe.rs +++ b/query/tests/it/interpreters/interpreter_table_describe.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::interpreters::*; use databend_query::sql::PlanParser; diff --git a/query/tests/it/interpreters/interpreter_table_drop.rs b/query/tests/it/interpreters/interpreter_table_drop.rs index e44ebeecacf2..f89c684aab1c 100644 --- a/query/tests/it/interpreters/interpreter_table_drop.rs +++ b/query/tests/it/interpreters/interpreter_table_drop.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::interpreters::*; use databend_query::sql::PlanParser; diff --git a/query/tests/it/interpreters/interpreter_table_rename.rs b/query/tests/it/interpreters/interpreter_table_rename.rs index 60d93c272ec6..e4705974cab5 100644 --- a/query/tests/it/interpreters/interpreter_table_rename.rs +++ b/query/tests/it/interpreters/interpreter_table_rename.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::interpreters::*; use databend_query::sql::PlanParser; diff --git a/query/tests/it/interpreters/interpreter_table_show_create.rs b/query/tests/it/interpreters/interpreter_table_show_create.rs index fd553116986f..ea1535823543 100644 --- a/query/tests/it/interpreters/interpreter_table_show_create.rs +++ b/query/tests/it/interpreters/interpreter_table_show_create.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::interpreters::*; use databend_query::sql::PlanParser; diff --git a/query/tests/it/interpreters/interpreter_table_truncate.rs b/query/tests/it/interpreters/interpreter_table_truncate.rs index a8b94056aa43..1e15945ecbc9 100644 --- a/query/tests/it/interpreters/interpreter_table_truncate.rs +++ b/query/tests/it/interpreters/interpreter_table_truncate.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::interpreters::*; use databend_query::sql::PlanParser; diff --git a/query/tests/it/interpreters/interpreter_use_database.rs b/query/tests/it/interpreters/interpreter_use_database.rs index 76f97e0a1945..745af396b372 100644 --- a/query/tests/it/interpreters/interpreter_use_database.rs +++ b/query/tests/it/interpreters/interpreter_use_database.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::interpreters::*; use databend_query::sql::PlanParser; diff --git a/query/tests/it/interpreters/interpreter_user_alter.rs b/query/tests/it/interpreters/interpreter_user_alter.rs index 160f462a3fb4..0f1966be9585 100644 --- a/query/tests/it/interpreters/interpreter_user_alter.rs +++ b/query/tests/it/interpreters/interpreter_user_alter.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use common_meta_types::AuthInfo; use common_meta_types::PasswordHashMethod; diff --git a/query/tests/it/interpreters/interpreter_user_create.rs b/query/tests/it/interpreters/interpreter_user_create.rs index af3530c85db3..2ca63d02d6a7 100644 --- a/query/tests/it/interpreters/interpreter_user_create.rs +++ b/query/tests/it/interpreters/interpreter_user_create.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::interpreters::*; use databend_query::sql::*; diff --git a/query/tests/it/interpreters/interpreter_user_drop.rs b/query/tests/it/interpreters/interpreter_user_drop.rs index 8eaf840ff15e..19f75ec93a4c 100644 --- a/query/tests/it/interpreters/interpreter_user_drop.rs +++ b/query/tests/it/interpreters/interpreter_user_drop.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use common_meta_types::AuthInfo; use common_meta_types::PasswordHashMethod; diff --git a/query/tests/it/interpreters/interpreter_user_stage.rs b/query/tests/it/interpreters/interpreter_user_stage.rs index b30dfc989b2f..5636a740db05 100644 --- a/query/tests/it/interpreters/interpreter_user_stage.rs +++ b/query/tests/it/interpreters/interpreter_user_stage.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::interpreters::InterpreterFactory; use databend_query::sql::*; diff --git a/query/tests/it/interpreters/interpreter_user_udf_alter.rs b/query/tests/it/interpreters/interpreter_user_udf_alter.rs index df90c0aee11e..707d41916339 100644 --- a/query/tests/it/interpreters/interpreter_user_udf_alter.rs +++ b/query/tests/it/interpreters/interpreter_user_udf_alter.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::interpreters::*; use databend_query::sql::*; diff --git a/query/tests/it/interpreters/interpreter_user_udf_create.rs b/query/tests/it/interpreters/interpreter_user_udf_create.rs index f2ea46485ff3..40f65be8dd77 100644 --- a/query/tests/it/interpreters/interpreter_user_udf_create.rs +++ b/query/tests/it/interpreters/interpreter_user_udf_create.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::ErrorCode; use common_exception::Result; use databend_query::interpreters::*; diff --git a/query/tests/it/interpreters/interpreter_user_udf_drop.rs b/query/tests/it/interpreters/interpreter_user_udf_drop.rs index 0c389a78d301..56776362901c 100644 --- a/query/tests/it/interpreters/interpreter_user_udf_drop.rs +++ b/query/tests/it/interpreters/interpreter_user_udf_drop.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::interpreters::*; use databend_query::sql::*; diff --git a/query/tests/it/interpreters/plan_schedulers/plan_scheduler.rs b/query/tests/it/interpreters/plan_schedulers/plan_scheduler.rs index efa30cae183c..249fe7c2ec23 100644 --- a/query/tests/it/interpreters/plan_schedulers/plan_scheduler.rs +++ b/query/tests/it/interpreters/plan_schedulers/plan_scheduler.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use common_base::tokio; +use common_base::base::tokio; use common_datavalues::DataValue; use common_exception::Result; use common_planners::*; diff --git a/query/tests/it/metrics.rs b/query/tests/it/metrics.rs index 582151c9af90..57a33b86117e 100644 --- a/query/tests/it/metrics.rs +++ b/query/tests/it/metrics.rs @@ -14,7 +14,7 @@ use std::net::SocketAddr; -use common_base::tokio; +use common_base::base::tokio; use common_metrics::init_default_metrics_recorder; use databend_query::metrics::MetricService; use databend_query::servers::Server; diff --git a/query/tests/it/optimizers/optimizer.rs b/query/tests/it/optimizers/optimizer.rs index 238f05223d54..ab388b7e7f6c 100644 --- a/query/tests/it/optimizers/optimizer.rs +++ b/query/tests/it/optimizers/optimizer.rs @@ -60,7 +60,7 @@ pub fn generate_partitions(workers: u64, total: u64) -> Partitions { partitions } -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::optimizers::Optimizers; use databend_query::sql::PlanParser; diff --git a/query/tests/it/optimizers/optimizer_constant_folding.rs b/query/tests/it/optimizers/optimizer_constant_folding.rs index 96310de12db9..0f9b0b439c33 100644 --- a/query/tests/it/optimizers/optimizer_constant_folding.rs +++ b/query/tests/it/optimizers/optimizer_constant_folding.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::optimizers::*; use databend_query::sql::PlanParser; diff --git a/query/tests/it/optimizers/optimizer_expression_transform.rs b/query/tests/it/optimizers/optimizer_expression_transform.rs index 501f336c0042..8177a704a30e 100644 --- a/query/tests/it/optimizers/optimizer_expression_transform.rs +++ b/query/tests/it/optimizers/optimizer_expression_transform.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::optimizers::*; use databend_query::sql::PlanParser; diff --git a/query/tests/it/optimizers/optimizer_scatters.rs b/query/tests/it/optimizers/optimizer_scatters.rs index 91892c85182b..53dd296170cb 100644 --- a/query/tests/it/optimizers/optimizer_scatters.rs +++ b/query/tests/it/optimizers/optimizer_scatters.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::optimizers::Optimizer; use databend_query::optimizers::ScattersOptimizer; diff --git a/query/tests/it/optimizers/optimizer_statistics_exact.rs b/query/tests/it/optimizers/optimizer_statistics_exact.rs index 96133b38428e..0b33eb1da3a5 100644 --- a/query/tests/it/optimizers/optimizer_statistics_exact.rs +++ b/query/tests/it/optimizers/optimizer_statistics_exact.rs @@ -14,7 +14,7 @@ use std::mem::size_of; -use common_base::tokio; +use common_base::base::tokio; use common_datavalues::*; use common_exception::Result; use common_meta_types::TableInfo; diff --git a/query/tests/it/optimizers/optimizer_top_n_push_down.rs b/query/tests/it/optimizers/optimizer_top_n_push_down.rs index 073db84f433e..413f911901ef 100644 --- a/query/tests/it/optimizers/optimizer_top_n_push_down.rs +++ b/query/tests/it/optimizers/optimizer_top_n_push_down.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::optimizers::*; use databend_query::sql::PlanParser; diff --git a/query/tests/it/pipelines/new/executor/executor_graph.rs b/query/tests/it/pipelines/new/executor/executor_graph.rs index 524cca1a147e..ca301c30b948 100644 --- a/query/tests/it/pipelines/new/executor/executor_graph.rs +++ b/query/tests/it/pipelines/new/executor/executor_graph.rs @@ -14,10 +14,10 @@ use std::sync::Arc; -use common_base::tokio; -use common_base::tokio::sync::mpsc::channel; -use common_base::tokio::sync::mpsc::Receiver; -use common_base::tokio::sync::mpsc::Sender; +use common_base::base::tokio; +use common_base::base::tokio::sync::mpsc::channel; +use common_base::base::tokio::sync::mpsc::Receiver; +use common_base::base::tokio::sync::mpsc::Sender; use common_datablocks::DataBlock; use common_exception::Result; use databend_query::pipelines::new::executor::RunningGraph; diff --git a/query/tests/it/pipelines/new/processors/port_test.rs b/query/tests/it/pipelines/new/processors/port_test.rs index 896dec9544ce..53bac83c9944 100644 --- a/query/tests/it/pipelines/new/processors/port_test.rs +++ b/query/tests/it/pipelines/new/processors/port_test.rs @@ -15,7 +15,7 @@ use std::sync::Arc; use std::sync::Barrier; -use common_base::tokio; +use common_base::base::tokio; use common_exception::ErrorCode; use common_exception::Result; use databend_query::pipelines::new::processors::connect; diff --git a/query/tests/it/pipelines/processors/pipe.rs b/query/tests/it/pipelines/processors/pipe.rs index 18e53a05bae5..f39f615a7a16 100644 --- a/query/tests/it/pipelines/processors/pipe.rs +++ b/query/tests/it/pipelines/processors/pipe.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::pipelines::processors::*; use pretty_assertions::assert_eq; diff --git a/query/tests/it/pipelines/processors/pipeline_builder.rs b/query/tests/it/pipelines/processors/pipeline_builder.rs index 076baab8b4ad..f5f235c7221c 100644 --- a/query/tests/it/pipelines/processors/pipeline_builder.rs +++ b/query/tests/it/pipelines/processors/pipeline_builder.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::pipelines::processors::*; use databend_query::sql::PlanParser; diff --git a/query/tests/it/pipelines/processors/pipeline_display.rs b/query/tests/it/pipelines/processors/pipeline_display.rs index c7dc0e8af0c0..ea5803523fc4 100644 --- a/query/tests/it/pipelines/processors/pipeline_display.rs +++ b/query/tests/it/pipelines/processors/pipeline_display.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::pipelines::processors::*; use databend_query::sql::PlanParser; diff --git a/query/tests/it/pipelines/processors/pipeline_walker.rs b/query/tests/it/pipelines/processors/pipeline_walker.rs index 4d64eea902a3..19d3ccb83720 100644 --- a/query/tests/it/pipelines/processors/pipeline_walker.rs +++ b/query/tests/it/pipelines/processors/pipeline_walker.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::pipelines::processors::*; use databend_query::sql::PlanParser; diff --git a/query/tests/it/pipelines/processors/processor_empty.rs b/query/tests/it/pipelines/processors/processor_empty.rs index 649e7cae9c6b..b331238da598 100644 --- a/query/tests/it/pipelines/processors/processor_empty.rs +++ b/query/tests/it/pipelines/processors/processor_empty.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::pipelines::processors::*; use pretty_assertions::assert_eq; diff --git a/query/tests/it/pipelines/processors/processor_merge.rs b/query/tests/it/pipelines/processors/processor_merge.rs index aec1c4dc2079..7169b55c91ef 100644 --- a/query/tests/it/pipelines/processors/processor_merge.rs +++ b/query/tests/it/pipelines/processors/processor_merge.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::pipelines::processors::*; use futures::TryStreamExt; diff --git a/query/tests/it/pipelines/processors/processor_mixed.rs b/query/tests/it/pipelines/processors/processor_mixed.rs index 2166d632d197..da641dbdcff2 100644 --- a/query/tests/it/pipelines/processors/processor_mixed.rs +++ b/query/tests/it/pipelines/processors/processor_mixed.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::pipelines::processors::*; use databend_query::pipelines::transforms::SourceTransform; diff --git a/query/tests/it/pipelines/transforms/transform_aggregator_final.rs b/query/tests/it/pipelines/transforms/transform_aggregator_final.rs index 5418417f1e9f..43402acb35e3 100644 --- a/query/tests/it/pipelines/transforms/transform_aggregator_final.rs +++ b/query/tests/it/pipelines/transforms/transform_aggregator_final.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use common_planners::*; use common_planners::{self}; diff --git a/query/tests/it/pipelines/transforms/transform_aggregator_partial.rs b/query/tests/it/pipelines/transforms/transform_aggregator_partial.rs index 2c7f49ffd41a..5f0576f2d515 100644 --- a/query/tests/it/pipelines/transforms/transform_aggregator_partial.rs +++ b/query/tests/it/pipelines/transforms/transform_aggregator_partial.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use common_planners::*; use common_planners::{self}; diff --git a/query/tests/it/pipelines/transforms/transform_expression.rs b/query/tests/it/pipelines/transforms/transform_expression.rs index 71b750d65f6a..debb8acce686 100644 --- a/query/tests/it/pipelines/transforms/transform_expression.rs +++ b/query/tests/it/pipelines/transforms/transform_expression.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use common_planners::*; use databend_query::pipelines::processors::*; diff --git a/query/tests/it/pipelines/transforms/transform_filter.rs b/query/tests/it/pipelines/transforms/transform_filter.rs index bc42cfb088ef..0b340d00181e 100644 --- a/query/tests/it/pipelines/transforms/transform_filter.rs +++ b/query/tests/it/pipelines/transforms/transform_filter.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use common_planners::*; use databend_query::pipelines::processors::*; diff --git a/query/tests/it/pipelines/transforms/transform_group_by_final.rs b/query/tests/it/pipelines/transforms/transform_group_by_final.rs index 4c57dccbdea1..a03e28d3332f 100644 --- a/query/tests/it/pipelines/transforms/transform_group_by_final.rs +++ b/query/tests/it/pipelines/transforms/transform_group_by_final.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use common_planners::*; use common_planners::{self}; diff --git a/query/tests/it/pipelines/transforms/transform_group_by_partial.rs b/query/tests/it/pipelines/transforms/transform_group_by_partial.rs index 7c742c07be18..4bab6c1b26f7 100644 --- a/query/tests/it/pipelines/transforms/transform_group_by_partial.rs +++ b/query/tests/it/pipelines/transforms/transform_group_by_partial.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use common_planners::*; use common_planners::{self}; diff --git a/query/tests/it/pipelines/transforms/transform_limit.rs b/query/tests/it/pipelines/transforms/transform_limit.rs index 350fb315d252..c4bb27746003 100644 --- a/query/tests/it/pipelines/transforms/transform_limit.rs +++ b/query/tests/it/pipelines/transforms/transform_limit.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use common_planners::*; use databend_query::pipelines::processors::*; diff --git a/query/tests/it/pipelines/transforms/transform_limit_by.rs b/query/tests/it/pipelines/transforms/transform_limit_by.rs index e27e11baf1e6..88d4b7f850d5 100644 --- a/query/tests/it/pipelines/transforms/transform_limit_by.rs +++ b/query/tests/it/pipelines/transforms/transform_limit_by.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use common_base::tokio; +use common_base::base::tokio; use common_datavalues::DataSchemaRefExt; use common_exception::Result; use common_planners::*; diff --git a/query/tests/it/pipelines/transforms/transform_projection.rs b/query/tests/it/pipelines/transforms/transform_projection.rs index 0de0c95e057b..c257e58cb168 100644 --- a/query/tests/it/pipelines/transforms/transform_projection.rs +++ b/query/tests/it/pipelines/transforms/transform_projection.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use common_planners::*; use databend_query::pipelines::processors::*; diff --git a/query/tests/it/pipelines/transforms/transform_sort.rs b/query/tests/it/pipelines/transforms/transform_sort.rs index 60cc1b6cb499..ab2699a7a833 100644 --- a/query/tests/it/pipelines/transforms/transform_sort.rs +++ b/query/tests/it/pipelines/transforms/transform_sort.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use common_planners::*; use common_planners::{self}; diff --git a/query/tests/it/pipelines/transforms/transform_source.rs b/query/tests/it/pipelines/transforms/transform_source.rs index efd0d4a28566..3095646e2397 100644 --- a/query/tests/it/pipelines/transforms/transform_source.rs +++ b/query/tests/it/pipelines/transforms/transform_source.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::pipelines::processors::*; use databend_query::pipelines::transforms::SourceTransform; diff --git a/query/tests/it/servers/clickhouse.rs b/query/tests/it/servers/clickhouse.rs index 64bfb4f777d6..382dfa2f7dbf 100644 --- a/query/tests/it/servers/clickhouse.rs +++ b/query/tests/it/servers/clickhouse.rs @@ -16,7 +16,7 @@ use std::net::SocketAddr; use std::time::Duration; use clickhouse_driver::prelude::*; -use common_base::tokio; +use common_base::base::tokio; use common_exception::ErrorCode; use common_exception::Result; use databend_query::servers::ClickHouseHandler; diff --git a/query/tests/it/servers/http/clickhouse_handler.rs b/query/tests/it/servers/http/clickhouse_handler.rs index 2a15986b646a..670aa6d66e31 100644 --- a/query/tests/it/servers/http/clickhouse_handler.rs +++ b/query/tests/it/servers/http/clickhouse_handler.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use databend_query::servers::http::middleware::HTTPSessionEndpoint; use databend_query::servers::http::middleware::HTTPSessionMiddleware; use databend_query::servers::http::v1::clickhouse_router; diff --git a/query/tests/it/servers/http/http_query_handlers.rs b/query/tests/it/servers/http/http_query_handlers.rs index 18d7a8620cc5..62a7232839b1 100644 --- a/query/tests/it/servers/http/http_query_handlers.rs +++ b/query/tests/it/servers/http/http_query_handlers.rs @@ -18,8 +18,8 @@ use std::time::Duration; use base64::encode_config; use base64::URL_SAFE_NO_PAD; -use common_base::get_free_tcp_port; -use common_base::tokio; +use common_base::base::get_free_tcp_port; +use common_base::base::tokio; use common_exception::ErrorCode; use common_exception::Result; use common_meta_types::AuthInfo; diff --git a/query/tests/it/servers/mysql/mysql_handler.rs b/query/tests/it/servers/mysql/mysql_handler.rs index 92fc8cf24533..e82e2f45d713 100644 --- a/query/tests/it/servers/mysql/mysql_handler.rs +++ b/query/tests/it/servers/mysql/mysql_handler.rs @@ -16,7 +16,7 @@ use std::net::SocketAddr; use std::sync::Arc; use std::time::Duration; -use common_base::tokio; +use common_base::base::tokio; use common_exception::ErrorCode; use common_exception::Result; use common_exception::ToErrorCode; diff --git a/query/tests/it/sessions/query_ctx.rs b/query/tests/it/sessions/query_ctx.rs index 173c94e61948..4d53032c3676 100644 --- a/query/tests/it/sessions/query_ctx.rs +++ b/query/tests/it/sessions/query_ctx.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::configs::FsStorageConfig; use databend_query::configs::S3StorageConfig; diff --git a/query/tests/it/sessions/session.rs b/query/tests/it/sessions/session.rs index 533e0f772dc9..d2e6af013d76 100644 --- a/query/tests/it/sessions/session.rs +++ b/query/tests/it/sessions/session.rs @@ -12,9 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; +use common_base::mem_allocator::malloc_size; use common_exception::Result; -use common_mem_allocator::malloc_size; use databend_query::sessions::Session; use databend_query::sessions::SessionManager; use databend_query::sessions::SessionType; diff --git a/query/tests/it/sessions/session_context.rs b/query/tests/it/sessions/session_context.rs index 7183c7240844..53d6c832a735 100644 --- a/query/tests/it/sessions/session_context.rs +++ b/query/tests/it/sessions/session_context.rs @@ -16,7 +16,7 @@ use std::net::SocketAddr; use std::sync::Arc; -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use common_meta_types::UserInfo; use databend_query::clusters::Cluster; diff --git a/query/tests/it/sessions/session_setting.rs b/query/tests/it/sessions/session_setting.rs index acd8daf5858c..2a7525c738c0 100644 --- a/query/tests/it/sessions/session_setting.rs +++ b/query/tests/it/sessions/session_setting.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::sessions::Session; use databend_query::sessions::SessionManager; diff --git a/query/tests/it/sql/plan_parser.rs b/query/tests/it/sql/plan_parser.rs index ccb74038d5a5..2338108e044c 100644 --- a/query/tests/it/sql/plan_parser.rs +++ b/query/tests/it/sql/plan_parser.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::sql::PlanParser; use pretty_assertions::assert_eq; diff --git a/query/tests/it/sql/statements/query/query_normalizer.rs b/query/tests/it/sql/statements/query/query_normalizer.rs index 50f5173064c1..aa5f75fff7e8 100644 --- a/query/tests/it/sql/statements/query/query_normalizer.rs +++ b/query/tests/it/sql/statements/query/query_normalizer.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::ErrorCode; use common_exception::Result; use databend_query::sql::statements::query::QueryNormalizer; diff --git a/query/tests/it/sql/statements/query/query_qualified_rewriter.rs b/query/tests/it/sql/statements/query/query_qualified_rewriter.rs index 363adc1a54e6..ddac1379b1fa 100644 --- a/query/tests/it/sql/statements/query/query_qualified_rewriter.rs +++ b/query/tests/it/sql/statements/query/query_qualified_rewriter.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::ErrorCode; use common_exception::Result; use databend_query::sql::statements::query::JoinedSchemaAnalyzer; diff --git a/query/tests/it/sql/statements/query/query_schema_joined_analyzer.rs b/query/tests/it/sql/statements/query/query_schema_joined_analyzer.rs index aa2e3f100844..051fbf0fe64a 100644 --- a/query/tests/it/sql/statements/query/query_schema_joined_analyzer.rs +++ b/query/tests/it/sql/statements/query/query_schema_joined_analyzer.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::ErrorCode; use common_exception::Result; use databend_query::sql::statements::query::JoinedSchemaAnalyzer; diff --git a/query/tests/it/sql/statements/statement_copy.rs b/query/tests/it/sql/statements/statement_copy.rs index 3901d8ff693c..46607a8c14be 100644 --- a/query/tests/it/sql/statements/statement_copy.rs +++ b/query/tests/it/sql/statements/statement_copy.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::ErrorCode; use common_exception::Result; use databend_query::sql::statements::AnalyzableStatement; diff --git a/query/tests/it/sql/statements/statement_create_table.rs b/query/tests/it/sql/statements/statement_create_table.rs index 9899aaf6ea8b..91a4ec9425c4 100644 --- a/query/tests/it/sql/statements/statement_create_table.rs +++ b/query/tests/it/sql/statements/statement_create_table.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::ErrorCode; use common_exception::Result; use databend_query::sql::statements::AnalyzableStatement; diff --git a/query/tests/it/sql/statements/statement_select.rs b/query/tests/it/sql/statements/statement_select.rs index d23ddc5f85d0..385cc120a9b2 100644 --- a/query/tests/it/sql/statements/statement_select.rs +++ b/query/tests/it/sql/statements/statement_select.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::ErrorCode; use common_exception::Result; use databend_query::sql::statements::AnalyzableStatement; diff --git a/query/tests/it/sql/statements/value_source.rs b/query/tests/it/sql/statements/value_source.rs index b08adf5cb4ae..2983b91983fa 100644 --- a/query/tests/it/sql/statements/value_source.rs +++ b/query/tests/it/sql/statements/value_source.rs @@ -14,7 +14,7 @@ use std::io::Cursor; -use common_base::tokio; +use common_base::base::tokio; use common_datavalues::prelude::*; use common_exception::Result; use common_io::prelude::*; diff --git a/query/tests/it/storages/fuse/io.rs b/query/tests/it/storages/fuse/io.rs index 61dd79170af3..2390345c6441 100644 --- a/query/tests/it/storages/fuse/io.rs +++ b/query/tests/it/storages/fuse/io.rs @@ -15,7 +15,7 @@ use std::sync::Arc; -use common_base::tokio; +use common_base::base::tokio; use common_datablocks::DataBlock; use common_datavalues::prelude::*; use common_exception::ErrorCode; @@ -336,7 +336,7 @@ fn test_meta_locations() -> Result<()> { Ok(()) } -use common_infallible::Mutex; +use common_base::infallible::Mutex; #[derive(Debug)] struct MockDataAccessor { diff --git a/query/tests/it/storages/fuse/operations/commit.rs b/query/tests/it/storages/fuse/operations/commit.rs index 394eccb5102f..67a15c5ff5ff 100644 --- a/query/tests/it/storages/fuse/operations/commit.rs +++ b/query/tests/it/storages/fuse/operations/commit.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. // -use common_base::tokio; +use common_base::base::tokio; use common_datablocks::DataBlock; use common_exception::Result; use futures::TryStreamExt; diff --git a/query/tests/it/storages/fuse/operations/optimize.rs b/query/tests/it/storages/fuse/operations/optimize.rs index 1f25bc664b30..3c89cfe755c2 100644 --- a/query/tests/it/storages/fuse/operations/optimize.rs +++ b/query/tests/it/storages/fuse/operations/optimize.rs @@ -13,7 +13,7 @@ // limitations under the License. // -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use futures::TryStreamExt; diff --git a/query/tests/it/storages/fuse/operations/purge_drop.rs b/query/tests/it/storages/fuse/operations/purge_drop.rs index d096cb169e2e..3db52086c5c6 100644 --- a/query/tests/it/storages/fuse/operations/purge_drop.rs +++ b/query/tests/it/storages/fuse/operations/purge_drop.rs @@ -13,7 +13,7 @@ // limitations under the License. // -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use crate::storages::fuse::table_test_fixture::append_sample_data; diff --git a/query/tests/it/storages/fuse/operations/purge_truncate.rs b/query/tests/it/storages/fuse/operations/purge_truncate.rs index af1bf5ad572f..f6aa7f55b677 100644 --- a/query/tests/it/storages/fuse/operations/purge_truncate.rs +++ b/query/tests/it/storages/fuse/operations/purge_truncate.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. // -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use crate::storages::fuse::table_test_fixture::append_sample_data; diff --git a/query/tests/it/storages/fuse/operations/read_plan.rs b/query/tests/it/storages/fuse/operations/read_plan.rs index 0e69e1f0669b..bbc154e021cd 100644 --- a/query/tests/it/storages/fuse/operations/read_plan.rs +++ b/query/tests/it/storages/fuse/operations/read_plan.rs @@ -16,7 +16,7 @@ use std::collections::HashMap; use std::iter::Iterator; -use common_base::tokio; +use common_base::base::tokio; use common_datavalues::DataValue; use common_exception::Result; use common_planners::Extras; diff --git a/query/tests/it/storages/fuse/pruning.rs b/query/tests/it/storages/fuse/pruning.rs index 57e0b91153d5..567daae75b06 100644 --- a/query/tests/it/storages/fuse/pruning.rs +++ b/query/tests/it/storages/fuse/pruning.rs @@ -15,7 +15,7 @@ use std::sync::Arc; -use common_base::tokio; +use common_base::base::tokio; use common_datablocks::DataBlock; use common_datavalues::prelude::*; use common_exception::Result; diff --git a/query/tests/it/storages/fuse/table.rs b/query/tests/it/storages/fuse/table.rs index 38d97f2635bb..dde8437e182a 100644 --- a/query/tests/it/storages/fuse/table.rs +++ b/query/tests/it/storages/fuse/table.rs @@ -15,7 +15,7 @@ use std::default::Default; -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use common_meta_types::TableInfo; use common_planners::ReadDataSourcePlan; diff --git a/query/tests/it/storages/fuse/table_functions/fuse_snapshot_table.rs b/query/tests/it/storages/fuse/table_functions/fuse_snapshot_table.rs index 1d86ba945356..83f9eeca9f2e 100644 --- a/query/tests/it/storages/fuse/table_functions/fuse_snapshot_table.rs +++ b/query/tests/it/storages/fuse/table_functions/fuse_snapshot_table.rs @@ -13,7 +13,7 @@ // limitations under the License. // -use common_base::tokio; +use common_base::base::tokio; use common_datablocks::DataBlock; use common_datavalues::prelude::*; use common_exception::ErrorCode; diff --git a/query/tests/it/storages/index/bloom_filter.rs b/query/tests/it/storages/index/bloom_filter.rs index 149511d54215..313a7ee4f050 100644 --- a/query/tests/it/storages/index/bloom_filter.rs +++ b/query/tests/it/storages/index/bloom_filter.rs @@ -11,7 +11,7 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_datablocks::DataBlock; use common_datavalues::prelude::*; use common_exception::Result; diff --git a/query/tests/it/storages/index/range_filter.rs b/query/tests/it/storages/index/range_filter.rs index 2c63fc20eb95..a45b9dc0ae1d 100644 --- a/query/tests/it/storages/index/range_filter.rs +++ b/query/tests/it/storages/index/range_filter.rs @@ -14,7 +14,7 @@ use std::collections::HashMap; -use common_base::tokio; +use common_base::base::tokio; use common_datavalues::prelude::*; use common_exception::Result; use common_planners::*; diff --git a/query/tests/it/storages/memory.rs b/query/tests/it/storages/memory.rs index 15c4afbde918..cf8ad6188487 100644 --- a/query/tests/it/storages/memory.rs +++ b/query/tests/it/storages/memory.rs @@ -13,7 +13,7 @@ // limitations under the License. // -use common_base::tokio; +use common_base::base::tokio; use common_datablocks::assert_blocks_sorted_eq; use common_datablocks::DataBlock; use common_datavalues::prelude::*; diff --git a/query/tests/it/storages/null.rs b/query/tests/it/storages/null.rs index 1879595c6ca3..2d15cc18640c 100644 --- a/query/tests/it/storages/null.rs +++ b/query/tests/it/storages/null.rs @@ -13,7 +13,7 @@ // limitations under the License. // -use common_base::tokio; +use common_base::base::tokio; use common_datablocks::DataBlock; use common_datavalues::prelude::*; use common_exception::Result; diff --git a/query/tests/it/storages/system/clusters_table.rs b/query/tests/it/storages/system/clusters_table.rs index 180efd4561c6..f2b076a0ed11 100644 --- a/query/tests/it/storages/system/clusters_table.rs +++ b/query/tests/it/storages/system/clusters_table.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::storages::system::ClustersTable; use databend_query::storages::ToReadDataSourcePlan; diff --git a/query/tests/it/storages/system/columns_table.rs b/query/tests/it/storages/system/columns_table.rs index 54603224d6fa..a642721edbf6 100644 --- a/query/tests/it/storages/system/columns_table.rs +++ b/query/tests/it/storages/system/columns_table.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::storages::system::ColumnsTable; use databend_query::storages::ToReadDataSourcePlan; diff --git a/query/tests/it/storages/system/configs_table.rs b/query/tests/it/storages/system/configs_table.rs index 56b3ddaaa513..73cc3604ff08 100644 --- a/query/tests/it/storages/system/configs_table.rs +++ b/query/tests/it/storages/system/configs_table.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::storages::system::ConfigsTable; use databend_query::storages::ToReadDataSourcePlan; diff --git a/query/tests/it/storages/system/contributors_table.rs b/query/tests/it/storages/system/contributors_table.rs index 9058b15ed0b3..dc6136f071ee 100644 --- a/query/tests/it/storages/system/contributors_table.rs +++ b/query/tests/it/storages/system/contributors_table.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::storages::system::ContributorsTable; use databend_query::storages::ToReadDataSourcePlan; diff --git a/query/tests/it/storages/system/credits_table.rs b/query/tests/it/storages/system/credits_table.rs index 27b50a804155..31850ff02727 100644 --- a/query/tests/it/storages/system/credits_table.rs +++ b/query/tests/it/storages/system/credits_table.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::storages::system::CreditsTable; use databend_query::storages::ToReadDataSourcePlan; diff --git a/query/tests/it/storages/system/databases_table.rs b/query/tests/it/storages/system/databases_table.rs index ff28e59f9428..5f85787cdd87 100644 --- a/query/tests/it/storages/system/databases_table.rs +++ b/query/tests/it/storages/system/databases_table.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::storages::system::DatabasesTable; use databend_query::storages::ToReadDataSourcePlan; diff --git a/query/tests/it/storages/system/engines_table.rs b/query/tests/it/storages/system/engines_table.rs index 3a26bc55dfac..7b216961ef1b 100644 --- a/query/tests/it/storages/system/engines_table.rs +++ b/query/tests/it/storages/system/engines_table.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::storages::system::EnginesTable; use databend_query::storages::ToReadDataSourcePlan; diff --git a/query/tests/it/storages/system/functions_table.rs b/query/tests/it/storages/system/functions_table.rs index 9138deaccc24..47878cbe6dc0 100644 --- a/query/tests/it/storages/system/functions_table.rs +++ b/query/tests/it/storages/system/functions_table.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::storages::system::FunctionsTable; use databend_query::storages::ToReadDataSourcePlan; diff --git a/query/tests/it/storages/system/metrics_table.rs b/query/tests/it/storages/system/metrics_table.rs index ee2b74bd77a1..8ede5f870def 100644 --- a/query/tests/it/storages/system/metrics_table.rs +++ b/query/tests/it/storages/system/metrics_table.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_datablocks::pretty_format_blocks; use common_exception::Result; use common_metrics::init_default_metrics_recorder; diff --git a/query/tests/it/storages/system/query_log_table.rs b/query/tests/it/storages/system/query_log_table.rs index a551bd4b9c82..ef7d411e0660 100644 --- a/query/tests/it/storages/system/query_log_table.rs +++ b/query/tests/it/storages/system/query_log_table.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use common_base::tokio; +use common_base::base::tokio; use common_datablocks::assert_blocks_sorted_eq; use common_datablocks::DataBlock; use common_datavalues::prelude::Series; diff --git a/query/tests/it/storages/system/roles_table.rs b/query/tests/it/storages/system/roles_table.rs index 005c3e7b2e1c..db31d664bc00 100644 --- a/query/tests/it/storages/system/roles_table.rs +++ b/query/tests/it/storages/system/roles_table.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use common_meta_types::RoleInfo; use databend_query::storages::system::RolesTable; diff --git a/query/tests/it/storages/system/settings_table.rs b/query/tests/it/storages/system/settings_table.rs index 49ab0ddbcd36..f34242dd7b6b 100644 --- a/query/tests/it/storages/system/settings_table.rs +++ b/query/tests/it/storages/system/settings_table.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::storages::system::SettingsTable; use databend_query::storages::ToReadDataSourcePlan; diff --git a/query/tests/it/storages/system/tables_table.rs b/query/tests/it/storages/system/tables_table.rs index 0462686e7829..3bb14e76ea74 100644 --- a/query/tests/it/storages/system/tables_table.rs +++ b/query/tests/it/storages/system/tables_table.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::storages::system::TablesTable; use databend_query::storages::ToReadDataSourcePlan; diff --git a/query/tests/it/storages/system/tracing_table.rs b/query/tests/it/storages/system/tracing_table.rs index 096dcb8574fa..0031fa1953e1 100644 --- a/query/tests/it/storages/system/tracing_table.rs +++ b/query/tests/it/storages/system/tracing_table.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use databend_query::storages::system::TracingTable; use databend_query::storages::Table; diff --git a/query/tests/it/storages/system/users_table.rs b/query/tests/it/storages/system/users_table.rs index 64c298082add..d8a5d30fc220 100644 --- a/query/tests/it/storages/system/users_table.rs +++ b/query/tests/it/storages/system/users_table.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use common_meta_types::AuthInfo; use common_meta_types::AuthType; diff --git a/query/tests/it/table_functions/numbers_table.rs b/query/tests/it/table_functions/numbers_table.rs index 187671b0f20b..2e9be1a39d3b 100644 --- a/query/tests/it/table_functions/numbers_table.rs +++ b/query/tests/it/table_functions/numbers_table.rs @@ -13,7 +13,7 @@ // limitations under the License. // -use common_base::tokio; +use common_base::base::tokio; use common_datavalues::prelude::*; use common_exception::Result; use common_planners::*; diff --git a/query/tests/it/tests/sessions.rs b/query/tests/it/tests/sessions.rs index 43c88ab01200..4421593e0d48 100644 --- a/query/tests/it/tests/sessions.rs +++ b/query/tests/it/tests/sessions.rs @@ -15,8 +15,8 @@ use std::env; use std::sync::Arc; -use common_base::tokio::runtime::Runtime; -use common_base::Thread; +use common_base::base::tokio::runtime::Runtime; +use common_base::base::Thread; use common_exception::Result; use databend_query::configs::Config; use databend_query::sessions::SessionManager; diff --git a/query/tests/it/users/auth/auth_mgr.rs b/query/tests/it/users/auth/auth_mgr.rs index 6885fdbbb7d3..ada523b6f39d 100644 --- a/query/tests/it/users/auth/auth_mgr.rs +++ b/query/tests/it/users/auth/auth_mgr.rs @@ -14,7 +14,7 @@ use base64::encode_config; use base64::URL_SAFE_NO_PAD; -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use common_meta_types::UserIdentity; use databend_query::users::auth::jwt::CustomClaims; diff --git a/query/tests/it/users/role_cache_mgr.rs b/query/tests/it/users/role_cache_mgr.rs index d61260c1e757..ce45007d41b8 100644 --- a/query/tests/it/users/role_cache_mgr.rs +++ b/query/tests/it/users/role_cache_mgr.rs @@ -15,7 +15,7 @@ use std::collections::HashMap; use std::collections::HashSet; -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use common_meta_types::GrantObject; use common_meta_types::RoleInfo; diff --git a/query/tests/it/users/role_mgr.rs b/query/tests/it/users/role_mgr.rs index 4e179ed7610d..c133b6676a5a 100644 --- a/query/tests/it/users/role_mgr.rs +++ b/query/tests/it/users/role_mgr.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::ErrorCode; use common_exception::Result; use common_meta_types::GrantObject; diff --git a/query/tests/it/users/user_mgr.rs b/query/tests/it/users/user_mgr.rs index 462fab8a1d50..eff823f409dc 100644 --- a/query/tests/it/users/user_mgr.rs +++ b/query/tests/it/users/user_mgr.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::ErrorCode; use common_exception::Result; use common_meta_types::AuthInfo; diff --git a/query/tests/it/users/user_udf.rs b/query/tests/it/users/user_udf.rs index 8ebbe2b60fff..b7d5be3559ee 100644 --- a/query/tests/it/users/user_udf.rs +++ b/query/tests/it/users/user_udf.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::tokio; +use common_base::base::tokio; use common_exception::Result; use common_meta_types::UserDefinedFunction; use databend_query::users::UserApiProvider; diff --git a/tools/metactl/src/main.rs b/tools/metactl/src/main.rs index 174330beb9a0..ed81c735d699 100644 --- a/tools/metactl/src/main.rs +++ b/tools/metactl/src/main.rs @@ -22,7 +22,7 @@ use std::net::SocketAddr; use anyhow::anyhow; use clap::Parser; -use common_base::tokio; +use common_base::base::tokio; use common_meta_raft_store::config::RaftConfig; use common_meta_raft_store::sled_key_spaces::KeySpaceKV; use common_meta_sled_store::get_sled_db;