mirror of https://github.com/citusdata/citus.git
Merge pull request #5389 from citusdata/marcocitus/remove-master_get_table_metadata
commit
e4760e348a
|
@ -95,74 +95,13 @@ PG_FUNCTION_INFO_V1(master_stage_shard_placement_row);
|
|||
|
||||
|
||||
/*
|
||||
* master_get_table_metadata takes in a relation name, and returns partition
|
||||
* related metadata for the relation. These metadata are grouped and returned in
|
||||
* a tuple, and are used by the caller when creating new shards. The function
|
||||
* errors if given relation does not exist, or is not partitioned.
|
||||
* master_get_table_metadata is a deprecated UDF.
|
||||
*/
|
||||
Datum
|
||||
master_get_table_metadata(PG_FUNCTION_ARGS)
|
||||
{
|
||||
CheckCitusVersion(ERROR);
|
||||
|
||||
text *relationName = PG_GETARG_TEXT_P(0);
|
||||
Oid relationId = ResolveRelationId(relationName, false);
|
||||
|
||||
Datum partitionKeyExpr = 0;
|
||||
Datum partitionKey = 0;
|
||||
TupleDesc metadataDescriptor = NULL;
|
||||
Datum values[TABLE_METADATA_FIELDS];
|
||||
bool isNulls[TABLE_METADATA_FIELDS];
|
||||
|
||||
/* find partition tuple for partitioned relation */
|
||||
CitusTableCacheEntry *partitionEntry = GetCitusTableCacheEntry(relationId);
|
||||
|
||||
/* create tuple descriptor for return value */
|
||||
TypeFuncClass resultTypeClass = get_call_result_type(fcinfo, NULL,
|
||||
&metadataDescriptor);
|
||||
if (resultTypeClass != TYPEFUNC_COMPOSITE)
|
||||
{
|
||||
ereport(ERROR, (errmsg("return type must be a row type")));
|
||||
}
|
||||
|
||||
/* form heap tuple for table metadata */
|
||||
memset(values, 0, sizeof(values));
|
||||
memset(isNulls, false, sizeof(isNulls));
|
||||
|
||||
char *partitionKeyString = partitionEntry->partitionKeyString;
|
||||
|
||||
/* reference tables do not have partition key */
|
||||
if (partitionKeyString == NULL)
|
||||
{
|
||||
partitionKey = PointerGetDatum(NULL);
|
||||
isNulls[3] = true;
|
||||
}
|
||||
else
|
||||
{
|
||||
/* get decompiled expression tree for partition key */
|
||||
partitionKeyExpr =
|
||||
PointerGetDatum(cstring_to_text(partitionEntry->partitionKeyString));
|
||||
partitionKey = DirectFunctionCall2(pg_get_expr, partitionKeyExpr,
|
||||
ObjectIdGetDatum(relationId));
|
||||
}
|
||||
|
||||
uint64 shardMaxSizeInBytes = (int64) ShardMaxSize * 1024L;
|
||||
|
||||
/* get storage type */
|
||||
char shardStorageType = ShardStorageType(relationId);
|
||||
|
||||
values[0] = ObjectIdGetDatum(relationId);
|
||||
values[1] = shardStorageType;
|
||||
values[2] = partitionEntry->partitionMethod;
|
||||
values[3] = partitionKey;
|
||||
values[4] = Int32GetDatum(ShardReplicationFactor);
|
||||
values[5] = Int64GetDatum(shardMaxSizeInBytes);
|
||||
values[6] = Int32GetDatum(ShardPlacementPolicy);
|
||||
|
||||
HeapTuple metadataTuple = heap_form_tuple(metadataDescriptor, values, isNulls);
|
||||
Datum metadataDatum = HeapTupleGetDatum(metadataTuple);
|
||||
|
||||
PG_RETURN_DATUM(metadataDatum);
|
||||
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
|
||||
errmsg("master_get_table_metadata is deprecated")));
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -7,3 +7,4 @@
|
|||
#include "udfs/worker_fix_partition_shard_index_names/11.0-1.sql"
|
||||
|
||||
DROP FUNCTION IF EXISTS pg_catalog.master_apply_delete_command(text);
|
||||
DROP FUNCTION pg_catalog.master_get_table_metadata(text);
|
||||
|
|
|
@ -11,4 +11,16 @@ CREATE FUNCTION pg_catalog.master_apply_delete_command(text)
|
|||
COMMENT ON FUNCTION pg_catalog.master_apply_delete_command(text)
|
||||
IS 'drop shards matching delete criteria and update metadata';
|
||||
|
||||
|
||||
CREATE FUNCTION pg_catalog.master_get_table_metadata(
|
||||
relation_name text,
|
||||
OUT logical_relid oid,
|
||||
OUT part_storage_type "char",
|
||||
OUT part_method "char", OUT part_key text,
|
||||
OUT part_replica_count integer,
|
||||
OUT part_max_size bigint,
|
||||
OUT part_placement_policy integer)
|
||||
RETURNS record
|
||||
LANGUAGE C STABLE STRICT
|
||||
AS 'MODULE_PATHNAME', $$master_get_table_metadata$$;
|
||||
COMMENT ON FUNCTION master_get_table_metadata(relation_name text)
|
||||
IS 'fetch metadata values for the table';
|
||||
|
|
|
@ -610,11 +610,10 @@ BEGIN;
|
|||
(1 row)
|
||||
|
||||
COMMIT;
|
||||
-- hide first column (relationId) as it might change
|
||||
SELECT part_storage_type, part_method, part_key, part_replica_count, part_max_size, part_placement_policy FROM master_get_table_metadata('citus_local_table_4');
|
||||
part_storage_type | part_method | part_key | part_replica_count | part_max_size | part_placement_policy
|
||||
SELECT partmethod, repmodel FROM pg_dist_partition WHERE logicalrelid = 'citus_local_table_4'::regclass;
|
||||
partmethod | repmodel
|
||||
---------------------------------------------------------------------
|
||||
t | n | | 1 | 1536000 | 2
|
||||
n | s
|
||||
(1 row)
|
||||
|
||||
SELECT master_get_table_ddl_events('citus_local_table_4');
|
||||
|
|
|
@ -895,10 +895,11 @@ SELECT * FROM multi_extension.print_extension_changes();
|
|||
previous_object | current_object
|
||||
---------------------------------------------------------------------
|
||||
function master_apply_delete_command(text) integer |
|
||||
function master_get_table_metadata(text) record |
|
||||
| function fix_all_partition_shard_index_names() SETOF regclass
|
||||
| function fix_partition_shard_index_names(regclass) void
|
||||
| function worker_fix_partition_shard_index_names(regclass,text,text) void
|
||||
(4 rows)
|
||||
(5 rows)
|
||||
|
||||
DROP TABLE multi_extension.prev_objects, multi_extension.extension_diff;
|
||||
-- show running version
|
||||
|
|
|
@ -3,13 +3,6 @@
|
|||
--
|
||||
-- Tests that check the metadata returned by the master node.
|
||||
SET citus.next_shard_id TO 740000;
|
||||
SELECT part_storage_type, part_key, part_replica_count, part_max_size,
|
||||
part_placement_policy FROM master_get_table_metadata('lineitem');
|
||||
part_storage_type | part_key | part_replica_count | part_max_size | part_placement_policy
|
||||
---------------------------------------------------------------------
|
||||
t | l_orderkey | 2 | 1536000 | 2
|
||||
(1 row)
|
||||
|
||||
SELECT * FROM master_get_table_ddl_events('lineitem') order by 1;
|
||||
master_get_table_ddl_events
|
||||
---------------------------------------------------------------------
|
||||
|
|
|
@ -1406,14 +1406,6 @@ SELECT master_create_empty_shard('reference_schema.reference_table_ddl');
|
|||
ERROR: relation "reference_schema.reference_table_ddl" is a reference table
|
||||
DETAIL: We currently don't support creating shards on reference tables
|
||||
-- get/update the statistics
|
||||
SELECT part_storage_type, part_key, part_replica_count, part_max_size,
|
||||
part_placement_policy
|
||||
FROM master_get_table_metadata('reference_schema.reference_table_ddl');
|
||||
part_storage_type | part_key | part_replica_count | part_max_size | part_placement_policy
|
||||
---------------------------------------------------------------------
|
||||
t | | 2 | 1536000 | 2
|
||||
(1 row)
|
||||
|
||||
SELECT shardid AS a_shard_id FROM pg_dist_shard WHERE logicalrelid = 'reference_schema.reference_table_ddl'::regclass \gset
|
||||
SELECT master_update_shard_statistics(:a_shard_id);
|
||||
master_update_shard_statistics
|
||||
|
|
|
@ -155,7 +155,6 @@ ORDER BY 1;
|
|||
function master_get_new_placementid()
|
||||
function master_get_new_shardid()
|
||||
function master_get_table_ddl_events(text)
|
||||
function master_get_table_metadata(text)
|
||||
function master_move_shard_placement(bigint,text,integer,text,integer,citus.shard_transfer_mode)
|
||||
function master_remove_distributed_table_metadata_from_workers(regclass,text,text)
|
||||
function master_remove_node(text,integer)
|
||||
|
@ -260,5 +259,5 @@ ORDER BY 1;
|
|||
view citus_worker_stat_activity
|
||||
view pg_dist_shard_placement
|
||||
view time_partitions
|
||||
(244 rows)
|
||||
(243 rows)
|
||||
|
||||
|
|
|
@ -1,247 +0,0 @@
|
|||
-- print version above 11 (eg. 12 and above)
|
||||
SHOW server_version \gset
|
||||
SELECT substring(:'server_version', '\d+')::int > 11 AS version_above_eleven;
|
||||
version_above_eleven
|
||||
---------------------------------------------------------------------
|
||||
f
|
||||
(1 row)
|
||||
|
||||
-- list all postgres objects belonging to the citus extension
|
||||
SELECT pg_catalog.pg_describe_object(classid, objid, 0) AS description
|
||||
FROM pg_catalog.pg_depend, pg_catalog.pg_extension e
|
||||
WHERE refclassid = 'pg_catalog.pg_extension'::pg_catalog.regclass
|
||||
AND refobjid = e.oid
|
||||
AND deptype = 'e'
|
||||
AND e.extname='citus'
|
||||
ORDER BY 1;
|
||||
description
|
||||
---------------------------------------------------------------------
|
||||
event trigger citus_cascade_to_partition
|
||||
function alter_distributed_table(regclass,text,integer,text,boolean)
|
||||
function alter_old_partitions_set_access_method(regclass,timestamp with time zone,name)
|
||||
function alter_role_if_exists(text,text)
|
||||
function alter_table_set_access_method(regclass,text)
|
||||
function any_value(anyelement)
|
||||
function any_value_agg(anyelement,anyelement)
|
||||
function array_cat_agg(anyarray)
|
||||
function assign_distributed_transaction_id(integer,bigint,timestamp with time zone)
|
||||
function authinfo_valid(text)
|
||||
function broadcast_intermediate_result(text,text)
|
||||
function check_distributed_deadlocks()
|
||||
function citus_activate_node(text,integer)
|
||||
function citus_add_inactive_node(text,integer,integer,noderole,name)
|
||||
function citus_add_local_table_to_metadata(regclass,boolean)
|
||||
function citus_add_node(text,integer,integer,noderole,name)
|
||||
function citus_add_rebalance_strategy(name,regproc,regproc,regproc,real,real,real)
|
||||
function citus_add_secondary_node(text,integer,text,integer,name)
|
||||
function citus_blocking_pids(integer)
|
||||
function citus_conninfo_cache_invalidate()
|
||||
function citus_copy_shard_placement(bigint,text,integer,text,integer,boolean,citus.shard_transfer_mode)
|
||||
function citus_create_restore_point(text)
|
||||
function citus_disable_node(text,integer)
|
||||
function citus_dist_local_group_cache_invalidate()
|
||||
function citus_dist_node_cache_invalidate()
|
||||
function citus_dist_object_cache_invalidate()
|
||||
function citus_dist_partition_cache_invalidate()
|
||||
function citus_dist_placement_cache_invalidate()
|
||||
function citus_dist_shard_cache_invalidate()
|
||||
function citus_dist_stat_activity()
|
||||
function citus_drain_node(text,integer,citus.shard_transfer_mode,name)
|
||||
function citus_drop_all_shards(regclass,text,text)
|
||||
function citus_drop_trigger()
|
||||
function citus_executor_name(integer)
|
||||
function citus_extradata_container(internal)
|
||||
function citus_finish_pg_upgrade()
|
||||
function citus_get_active_worker_nodes()
|
||||
function citus_internal.downgrade_columnar_storage(regclass)
|
||||
function citus_internal.find_groupid_for_node(text,integer)
|
||||
function citus_internal.pg_dist_node_trigger_func()
|
||||
function citus_internal.pg_dist_rebalance_strategy_enterprise_check()
|
||||
function citus_internal.pg_dist_rebalance_strategy_trigger_func()
|
||||
function citus_internal.pg_dist_shard_placement_trigger_func()
|
||||
function citus_internal.refresh_isolation_tester_prepared_statement()
|
||||
function citus_internal.replace_isolation_tester_func()
|
||||
function citus_internal.restore_isolation_tester_func()
|
||||
function citus_internal.upgrade_columnar_storage(regclass)
|
||||
function citus_isolation_test_session_is_blocked(integer,integer[])
|
||||
function citus_json_concatenate(json,json)
|
||||
function citus_json_concatenate_final(json)
|
||||
function citus_jsonb_concatenate(jsonb,jsonb)
|
||||
function citus_jsonb_concatenate_final(jsonb)
|
||||
function citus_move_shard_placement(bigint,text,integer,text,integer,citus.shard_transfer_mode)
|
||||
function citus_node_capacity_1(integer)
|
||||
function citus_prepare_pg_upgrade()
|
||||
function citus_query_stats()
|
||||
function citus_relation_size(regclass)
|
||||
function citus_remote_connection_stats()
|
||||
function citus_remove_node(text,integer)
|
||||
function citus_server_id()
|
||||
function citus_set_coordinator_host(text,integer,noderole,name)
|
||||
function citus_set_default_rebalance_strategy(text)
|
||||
function citus_set_node_property(text,integer,text,boolean)
|
||||
function citus_shard_allowed_on_node_true(bigint,integer)
|
||||
function citus_shard_cost_1(bigint)
|
||||
function citus_shard_cost_by_disk_size(bigint)
|
||||
function citus_shard_sizes()
|
||||
function citus_stat_statements()
|
||||
function citus_stat_statements_reset()
|
||||
function citus_table_is_visible(oid)
|
||||
function citus_table_size(regclass)
|
||||
function citus_text_send_as_jsonb(text)
|
||||
function citus_total_relation_size(regclass,boolean)
|
||||
function citus_truncate_trigger()
|
||||
function citus_unmark_object_distributed(oid,oid,integer)
|
||||
function citus_update_node(integer,text,integer,boolean,integer)
|
||||
function citus_update_shard_statistics(bigint)
|
||||
function citus_update_table_statistics(regclass)
|
||||
function citus_validate_rebalance_strategy_functions(regproc,regproc,regproc)
|
||||
function citus_version()
|
||||
function citus_worker_stat_activity()
|
||||
function column_name_to_column(regclass,text)
|
||||
function column_to_column_name(regclass,text)
|
||||
function coord_combine_agg(oid,cstring,anyelement)
|
||||
function coord_combine_agg_ffunc(internal,oid,cstring,anyelement)
|
||||
function coord_combine_agg_sfunc(internal,oid,cstring,anyelement)
|
||||
function create_distributed_function(regprocedure,text,text)
|
||||
function create_distributed_table(regclass,text,citus.distribution_type,text,integer)
|
||||
function create_intermediate_result(text,text)
|
||||
function create_reference_table(regclass)
|
||||
function create_time_partitions(regclass,interval,timestamp with time zone,timestamp with time zone)
|
||||
function distributed_tables_colocated(regclass,regclass)
|
||||
function drop_old_time_partitions(regclass,timestamp with time zone)
|
||||
function dump_global_wait_edges()
|
||||
function dump_local_wait_edges()
|
||||
function fetch_intermediate_results(text[],text,integer)
|
||||
function fix_pre_citus10_partitioned_table_constraint_names()
|
||||
function fix_pre_citus10_partitioned_table_constraint_names(regclass)
|
||||
function get_all_active_transactions()
|
||||
function get_colocated_shard_array(bigint)
|
||||
function get_colocated_table_array(regclass)
|
||||
function get_current_transaction_id()
|
||||
function get_global_active_transactions()
|
||||
function get_missing_time_partition_ranges(regclass,interval,timestamp with time zone,timestamp with time zone)
|
||||
function get_rebalance_progress()
|
||||
function get_rebalance_table_shards_plan(regclass,real,integer,bigint[],boolean,name,real)
|
||||
function get_shard_id_for_distribution_column(regclass,"any")
|
||||
function isolate_tenant_to_new_shard(regclass,"any",text)
|
||||
function json_cat_agg(json)
|
||||
function jsonb_cat_agg(jsonb)
|
||||
function lock_relation_if_exists(text,text)
|
||||
function lock_shard_metadata(integer,bigint[])
|
||||
function lock_shard_resources(integer,bigint[])
|
||||
function master_activate_node(text,integer)
|
||||
function master_add_inactive_node(text,integer,integer,noderole,name)
|
||||
function master_add_node(text,integer,integer,noderole,name)
|
||||
function master_add_secondary_node(text,integer,text,integer,name)
|
||||
function master_append_table_to_shard(bigint,text,text,integer)
|
||||
function master_apply_delete_command(text)
|
||||
function master_copy_shard_placement(bigint,text,integer,text,integer,boolean,citus.shard_transfer_mode)
|
||||
function master_create_empty_shard(text)
|
||||
function master_disable_node(text,integer)
|
||||
function master_drain_node(text,integer,citus.shard_transfer_mode,name)
|
||||
function master_get_active_worker_nodes()
|
||||
function master_get_new_placementid()
|
||||
function master_get_new_shardid()
|
||||
function master_get_table_ddl_events(text)
|
||||
function master_get_table_metadata(text)
|
||||
function master_move_shard_placement(bigint,text,integer,text,integer,citus.shard_transfer_mode)
|
||||
function master_remove_distributed_table_metadata_from_workers(regclass,text,text)
|
||||
function master_remove_node(text,integer)
|
||||
function master_remove_partition_metadata(regclass,text,text)
|
||||
function master_run_on_worker(text[],integer[],text[],boolean)
|
||||
function master_set_node_property(text,integer,text,boolean)
|
||||
function master_unmark_object_distributed(oid,oid,integer)
|
||||
function master_update_node(integer,text,integer,boolean,integer)
|
||||
function master_update_shard_statistics(bigint)
|
||||
function master_update_table_statistics(regclass)
|
||||
function notify_constraint_dropped()
|
||||
function poolinfo_valid(text)
|
||||
function read_intermediate_result(text,citus_copy_format)
|
||||
function read_intermediate_results(text[],citus_copy_format)
|
||||
function rebalance_table_shards(regclass,real,integer,bigint[],citus.shard_transfer_mode,boolean,name)
|
||||
function recover_prepared_transactions()
|
||||
function relation_is_a_known_shard(regclass)
|
||||
function remove_local_tables_from_metadata()
|
||||
function replicate_reference_tables()
|
||||
function replicate_table_shards(regclass,integer,integer,bigint[],citus.shard_transfer_mode)
|
||||
function role_exists(name)
|
||||
function run_command_on_colocated_placements(regclass,regclass,text,boolean)
|
||||
function run_command_on_placements(regclass,text,boolean)
|
||||
function run_command_on_shards(regclass,text,boolean)
|
||||
function run_command_on_workers(text,boolean)
|
||||
function shard_name(regclass,bigint)
|
||||
function start_metadata_sync_to_node(text,integer)
|
||||
function stop_metadata_sync_to_node(text,integer)
|
||||
function time_partition_range(regclass)
|
||||
function truncate_local_data_after_distributing_table(regclass)
|
||||
function undistribute_table(regclass,boolean)
|
||||
function update_distributed_table_colocation(regclass,text)
|
||||
function worker_append_table_to_shard(text,text,text,integer)
|
||||
function worker_apply_inter_shard_ddl_command(bigint,text,bigint,text,text)
|
||||
function worker_apply_sequence_command(text)
|
||||
function worker_apply_sequence_command(text,regtype)
|
||||
function worker_apply_shard_ddl_command(bigint,text)
|
||||
function worker_apply_shard_ddl_command(bigint,text,text)
|
||||
function worker_change_sequence_dependency(regclass,regclass,regclass)
|
||||
function worker_cleanup_job_schema_cache()
|
||||
function worker_create_or_alter_role(text,text,text)
|
||||
function worker_create_or_replace_object(text)
|
||||
function worker_create_schema(bigint,text)
|
||||
function worker_create_truncate_trigger(regclass)
|
||||
function worker_drop_distributed_table(text)
|
||||
function worker_fetch_foreign_file(text,text,bigint,text[],integer[])
|
||||
function worker_fetch_partition_file(bigint,integer,integer,integer,text,integer)
|
||||
function worker_fix_pre_citus10_partitioned_table_constraint_names(regclass,bigint,text)
|
||||
function worker_hash("any")
|
||||
function worker_hash_partition_table(bigint,integer,text,text,oid,anyarray)
|
||||
function worker_last_saved_explain_analyze()
|
||||
function worker_merge_files_into_table(bigint,integer,text[],text[])
|
||||
function worker_partial_agg(oid,anyelement)
|
||||
function worker_partial_agg_ffunc(internal)
|
||||
function worker_partial_agg_sfunc(internal,oid,anyelement)
|
||||
function worker_partition_query_result(text,text,integer,citus.distribution_type,text[],text[],boolean)
|
||||
function worker_range_partition_table(bigint,integer,text,text,oid,anyarray)
|
||||
function worker_record_sequence_dependency(regclass,regclass,name)
|
||||
function worker_repartition_cleanup(bigint)
|
||||
function worker_save_query_explain_analyze(text,jsonb)
|
||||
schema citus
|
||||
schema citus_internal
|
||||
schema columnar
|
||||
sequence columnar.storageid_seq
|
||||
sequence pg_dist_colocationid_seq
|
||||
sequence pg_dist_groupid_seq
|
||||
sequence pg_dist_node_nodeid_seq
|
||||
sequence pg_dist_placement_placementid_seq
|
||||
sequence pg_dist_shardid_seq
|
||||
table citus.pg_dist_object
|
||||
table columnar.chunk
|
||||
table columnar.chunk_group
|
||||
table columnar.options
|
||||
table columnar.stripe
|
||||
table pg_dist_authinfo
|
||||
table pg_dist_colocation
|
||||
table pg_dist_local_group
|
||||
table pg_dist_node
|
||||
table pg_dist_node_metadata
|
||||
table pg_dist_partition
|
||||
table pg_dist_placement
|
||||
table pg_dist_poolinfo
|
||||
table pg_dist_rebalance_strategy
|
||||
table pg_dist_shard
|
||||
table pg_dist_transaction
|
||||
type citus.distribution_type
|
||||
type citus.shard_transfer_mode
|
||||
type citus_copy_format
|
||||
type noderole
|
||||
view citus_dist_stat_activity
|
||||
view citus_lock_waits
|
||||
view citus_shard_indexes_on_worker
|
||||
view citus_shards
|
||||
view citus_shards_on_worker
|
||||
view citus_stat_statements
|
||||
view citus_tables
|
||||
view citus_worker_stat_activity
|
||||
view pg_dist_shard_placement
|
||||
view time_partitions
|
||||
(227 rows)
|
||||
|
|
@ -419,8 +419,7 @@ BEGIN;
|
|||
SELECT count(*) FROM pg_locks where relation='citus_local_table_4'::regclass;
|
||||
COMMIT;
|
||||
|
||||
-- hide first column (relationId) as it might change
|
||||
SELECT part_storage_type, part_method, part_key, part_replica_count, part_max_size, part_placement_policy FROM master_get_table_metadata('citus_local_table_4');
|
||||
SELECT partmethod, repmodel FROM pg_dist_partition WHERE logicalrelid = 'citus_local_table_4'::regclass;
|
||||
SELECT master_get_table_ddl_events('citus_local_table_4');
|
||||
|
||||
SELECT column_to_column_name(logicalrelid, partkey)
|
||||
|
|
|
@ -7,9 +7,6 @@
|
|||
SET citus.next_shard_id TO 740000;
|
||||
|
||||
|
||||
SELECT part_storage_type, part_key, part_replica_count, part_max_size,
|
||||
part_placement_policy FROM master_get_table_metadata('lineitem');
|
||||
|
||||
SELECT * FROM master_get_table_ddl_events('lineitem') order by 1;
|
||||
|
||||
SELECT * FROM master_get_new_shardid();
|
||||
|
|
|
@ -899,10 +899,6 @@ ALTER TABLE reference_schema.reference_table_ddl_test RENAME TO reference_table_
|
|||
SELECT master_create_empty_shard('reference_schema.reference_table_ddl');
|
||||
|
||||
-- get/update the statistics
|
||||
SELECT part_storage_type, part_key, part_replica_count, part_max_size,
|
||||
part_placement_policy
|
||||
FROM master_get_table_metadata('reference_schema.reference_table_ddl');
|
||||
|
||||
SELECT shardid AS a_shard_id FROM pg_dist_shard WHERE logicalrelid = 'reference_schema.reference_table_ddl'::regclass \gset
|
||||
SELECT master_update_shard_statistics(:a_shard_id);
|
||||
|
||||
|
|
Loading…
Reference in New Issue