Provide hashing support for arrays.

The core of this patch is hash_array() and associated typcache
infrastructure, which works just about exactly like the existing support
for array comparison.

In addition I did some work to ensure that the planner won't think that an
array type is hashable unless its element type is hashable, and similarly
for sorting.  This includes adding a datatype parameter to op_hashjoinable
and op_mergejoinable, and adding an explicit "hashable" flag to
SortGroupClause.  The lack of a cross-check on the element type was a
pre-existing bug in mergejoin support --- but it didn't matter so much
before, because if you couldn't sort the element type there wasn't any good
alternative to failing anyhow.  Now that we have the alternative of hashing
the array type, there are cases where we can avoid a failure by being picky
at the planner stage, so it's time to be picky.

The issue of exactly how to combine the per-element hash values to produce
an array hash is still open for discussion, but the rest of this is pretty
solid, so I'll commit it as-is.
This commit is contained in:
Tom Lane 2010-10-30 21:55:20 -04:00
parent bd1ff97133
commit 186cbbda8f
30 changed files with 376 additions and 89 deletions

View File

@ -1794,7 +1794,8 @@ std_typanalyze(VacAttrStats *stats)
/* Look for default "<" and "=" operators for column's type */
get_sort_group_operators(stats->attrtypid,
false, false, false,
&ltopr, &eqopr, NULL);
&ltopr, &eqopr, NULL,
NULL);
/* If column has no "=" operator, we can't do much of anything */
if (!OidIsValid(eqopr))

View File

@ -1882,6 +1882,7 @@ _copySortGroupClause(SortGroupClause *from)
COPY_SCALAR_FIELD(eqop);
COPY_SCALAR_FIELD(sortop);
COPY_SCALAR_FIELD(nulls_first);
COPY_SCALAR_FIELD(hashable);
return newnode;
}

View File

@ -2227,6 +2227,7 @@ _equalSortGroupClause(SortGroupClause *a, SortGroupClause *b)
COMPARE_SCALAR_FIELD(eqop);
COMPARE_SCALAR_FIELD(sortop);
COMPARE_SCALAR_FIELD(nulls_first);
COMPARE_SCALAR_FIELD(hashable);
return true;
}

View File

@ -2073,6 +2073,7 @@ _outSortGroupClause(StringInfo str, SortGroupClause *node)
WRITE_OID_FIELD(eqop);
WRITE_OID_FIELD(sortop);
WRITE_BOOL_FIELD(nulls_first);
WRITE_BOOL_FIELD(hashable);
}
static void

View File

@ -264,6 +264,7 @@ _readSortGroupClause(void)
READ_OID_FIELD(eqop);
READ_OID_FIELD(sortop);
READ_BOOL_FIELD(nulls_first);
READ_BOOL_FIELD(hashable);
READ_DONE();
}

View File

@ -925,7 +925,9 @@ generate_join_implied_equalities_normal(PlannerInfo *root,
(IsA(inner_em->em_expr, RelabelType) &&
IsA(((RelabelType *) inner_em->em_expr)->arg, Var)))
score++;
if (!enable_hashjoin || op_hashjoinable(eq_op))
if (!enable_hashjoin ||
op_hashjoinable(eq_op,
exprType((Node *) outer_em->em_expr)))
score++;
if (score > best_score)
{

View File

@ -958,6 +958,7 @@ create_unique_plan(PlannerInfo *root, UniquePath *best_path)
sortcl->eqop = eqop;
sortcl->sortop = sortop;
sortcl->nulls_first = false;
sortcl->hashable = false; /* no need to make this accurate */
sortList = lappend(sortList, sortcl);
groupColPos++;
}

View File

@ -16,6 +16,7 @@
#include "catalog/pg_operator.h"
#include "catalog/pg_type.h"
#include "nodes/nodeFuncs.h"
#include "optimizer/clauses.h"
#include "optimizer/cost.h"
#include "optimizer/joininfo.h"
@ -1486,6 +1487,7 @@ check_mergejoinable(RestrictInfo *restrictinfo)
{
Expr *clause = restrictinfo->clause;
Oid opno;
Node *leftarg;
if (restrictinfo->pseudoconstant)
return;
@ -1495,8 +1497,9 @@ check_mergejoinable(RestrictInfo *restrictinfo)
return;
opno = ((OpExpr *) clause)->opno;
leftarg = linitial(((OpExpr *) clause)->args);
if (op_mergejoinable(opno) &&
if (op_mergejoinable(opno, exprType(leftarg)) &&
!contain_volatile_functions((Node *) clause))
restrictinfo->mergeopfamilies = get_mergejoin_opfamilies(opno);
@ -1521,6 +1524,7 @@ check_hashjoinable(RestrictInfo *restrictinfo)
{
Expr *clause = restrictinfo->clause;
Oid opno;
Node *leftarg;
if (restrictinfo->pseudoconstant)
return;
@ -1530,8 +1534,9 @@ check_hashjoinable(RestrictInfo *restrictinfo)
return;
opno = ((OpExpr *) clause)->opno;
leftarg = linitial(((OpExpr *) clause)->args);
if (op_hashjoinable(opno) &&
if (op_hashjoinable(opno, exprType(leftarg)) &&
!contain_volatile_functions((Node *) clause))
restrictinfo->hashjoinoperator = opno;
}

View File

@ -518,6 +518,7 @@ make_agg_subplan(PlannerInfo *root, MinMaxAggInfo *info)
info->aggsortop);
sortcl->sortop = info->aggsortop;
sortcl->nulls_first = info->nulls_first;
sortcl->hashable = false; /* no need to make this accurate */
subparse->sortClause = list_make1(sortcl);
/* set up LIMIT 1 */

View File

@ -861,28 +861,45 @@ testexpr_is_hashable(Node *testexpr)
return false;
}
/*
* Check expression is hashable + strict
*
* We could use op_hashjoinable() and op_strict(), but do it like this to
* avoid a redundant cache lookup.
*/
static bool
hash_ok_operator(OpExpr *expr)
{
Oid opid = expr->opno;
HeapTuple tup;
Form_pg_operator optup;
/* quick out if not a binary operator */
if (list_length(expr->args) != 2)
return false;
/* else must look up the operator properties */
tup = SearchSysCache1(OPEROID, ObjectIdGetDatum(opid));
if (!HeapTupleIsValid(tup))
elog(ERROR, "cache lookup failed for operator %u", opid);
optup = (Form_pg_operator) GETSTRUCT(tup);
if (!optup->oprcanhash || !func_strict(optup->oprcode))
if (opid == ARRAY_EQ_OP)
{
ReleaseSysCache(tup);
return false;
/* array_eq is strict, but must check input type to ensure hashable */
Node *leftarg = linitial(expr->args);
return op_hashjoinable(opid, exprType(leftarg));
}
else
{
/* else must look up the operator properties */
HeapTuple tup;
Form_pg_operator optup;
tup = SearchSysCache1(OPEROID, ObjectIdGetDatum(opid));
if (!HeapTupleIsValid(tup))
elog(ERROR, "cache lookup failed for operator %u", opid);
optup = (Form_pg_operator) GETSTRUCT(tup);
if (!optup->oprcanhash || !func_strict(optup->oprcode))
{
ReleaseSysCache(tup);
return false;
}
ReleaseSysCache(tup);
return true;
}
ReleaseSysCache(tup);
return true;
}

View File

@ -18,6 +18,7 @@
#include "catalog/pg_operator.h"
#include "miscadmin.h"
#include "nodes/nodeFuncs.h"
#include "optimizer/clauses.h"
#include "optimizer/cost.h"
#include "optimizer/pathnode.h"
@ -878,6 +879,7 @@ create_unique_path(PlannerInfo *root, RelOptInfo *rel, Path *subpath,
Relids left_varnos;
Relids right_varnos;
Relids all_varnos;
Oid opinputtype;
/* Is it a binary opclause? */
if (!IsA(op, OpExpr) ||
@ -908,6 +910,7 @@ create_unique_path(PlannerInfo *root, RelOptInfo *rel, Path *subpath,
left_varnos = pull_varnos(left_expr);
right_varnos = pull_varnos(right_expr);
all_varnos = bms_union(left_varnos, right_varnos);
opinputtype = exprType(left_expr);
/* Does it reference both sides? */
if (!bms_overlap(all_varnos, sjinfo->syn_righthand) ||
@ -946,14 +949,14 @@ create_unique_path(PlannerInfo *root, RelOptInfo *rel, Path *subpath,
if (all_btree)
{
/* oprcanmerge is considered a hint... */
if (!op_mergejoinable(opno) ||
if (!op_mergejoinable(opno, opinputtype) ||
get_mergejoin_opfamilies(opno) == NIL)
all_btree = false;
}
if (all_hash)
{
/* ... but oprcanhash had better be correct */
if (!op_hashjoinable(opno))
if (!op_hashjoinable(opno, opinputtype))
all_hash = false;
}
if (!(all_btree || all_hash))

View File

@ -18,7 +18,6 @@
#include "nodes/nodeFuncs.h"
#include "optimizer/tlist.h"
#include "optimizer/var.h"
#include "utils/lsyscache.h"
/*****************************************************************************
@ -348,10 +347,7 @@ grouping_is_sortable(List *groupClause)
/*
* grouping_is_hashable - is it possible to implement grouping list by hashing?
*
* We assume hashing is OK if the equality operators are marked oprcanhash.
* (If there isn't actually a supporting hash function, the executor will
* complain at runtime; but this is a misdeclaration of the operator, not
* a system bug.)
* We rely on the parser to have set the hashable flag correctly.
*/
bool
grouping_is_hashable(List *groupClause)
@ -362,7 +358,7 @@ grouping_is_hashable(List *groupClause)
{
SortGroupClause *groupcl = (SortGroupClause *) lfirst(glitem);
if (!op_hashjoinable(groupcl->eqop))
if (!groupcl->hashable)
return false;
}
return true;

View File

@ -1659,6 +1659,7 @@ transformSetOperationTree(ParseState *pstate, SelectStmt *stmt,
SortGroupClause *grpcl = makeNode(SortGroupClause);
Oid sortop;
Oid eqop;
bool hashable;
ParseCallbackState pcbstate;
setup_parser_errposition_callback(&pcbstate, pstate,
@ -1667,7 +1668,8 @@ transformSetOperationTree(ParseState *pstate, SelectStmt *stmt,
/* determine the eqop and optional sortop */
get_sort_group_operators(rescoltype,
false, true, false,
&sortop, &eqop, NULL);
&sortop, &eqop, NULL,
&hashable);
cancel_parser_errposition_callback(&pcbstate);
@ -1676,6 +1678,7 @@ transformSetOperationTree(ParseState *pstate, SelectStmt *stmt,
grpcl->eqop = eqop;
grpcl->sortop = sortop;
grpcl->nulls_first = false; /* OK with or without sortop */
grpcl->hashable = hashable;
op->groupClauses = lappend(op->groupClauses, grpcl);
}

View File

@ -1937,6 +1937,7 @@ addTargetToSortList(ParseState *pstate, TargetEntry *tle,
Oid restype = exprType((Node *) tle->expr);
Oid sortop;
Oid eqop;
bool hashable;
bool reverse;
int location;
ParseCallbackState pcbstate;
@ -1972,13 +1973,15 @@ addTargetToSortList(ParseState *pstate, TargetEntry *tle,
case SORTBY_ASC:
get_sort_group_operators(restype,
true, true, false,
&sortop, &eqop, NULL);
&sortop, &eqop, NULL,
&hashable);
reverse = false;
break;
case SORTBY_DESC:
get_sort_group_operators(restype,
false, true, true,
NULL, &eqop, &sortop);
NULL, &eqop, &sortop,
&hashable);
reverse = true;
break;
case SORTBY_USING:
@ -2000,11 +2003,17 @@ addTargetToSortList(ParseState *pstate, TargetEntry *tle,
errmsg("operator %s is not a valid ordering operator",
strVal(llast(sortby->useOp))),
errhint("Ordering operators must be \"<\" or \">\" members of btree operator families.")));
/*
* Also see if the equality operator is hashable.
*/
hashable = op_hashjoinable(eqop, restype);
break;
default:
elog(ERROR, "unrecognized sortby_dir: %d", sortby->sortby_dir);
sortop = InvalidOid; /* keep compiler quiet */
eqop = InvalidOid;
hashable = false;
reverse = false;
break;
}
@ -2020,6 +2029,7 @@ addTargetToSortList(ParseState *pstate, TargetEntry *tle,
sortcl->eqop = eqop;
sortcl->sortop = sortop;
sortcl->hashable = hashable;
switch (sortby->sortby_nulls)
{
@ -2074,8 +2084,6 @@ addTargetToGroupList(ParseState *pstate, TargetEntry *tle,
bool resolveUnknown)
{
Oid restype = exprType((Node *) tle->expr);
Oid sortop;
Oid eqop;
/* if tlist item is an UNKNOWN literal, change it to TEXT */
if (restype == UNKNOWNOID && resolveUnknown)
@ -2092,6 +2100,9 @@ addTargetToGroupList(ParseState *pstate, TargetEntry *tle,
if (!targetIsInSortList(tle, InvalidOid, grouplist))
{
SortGroupClause *grpcl = makeNode(SortGroupClause);
Oid sortop;
Oid eqop;
bool hashable;
ParseCallbackState pcbstate;
setup_parser_errposition_callback(&pcbstate, pstate, location);
@ -2099,7 +2110,8 @@ addTargetToGroupList(ParseState *pstate, TargetEntry *tle,
/* determine the eqop and optional sortop */
get_sort_group_operators(restype,
false, true, false,
&sortop, &eqop, NULL);
&sortop, &eqop, NULL,
&hashable);
cancel_parser_errposition_callback(&pcbstate);
@ -2107,6 +2119,7 @@ addTargetToGroupList(ParseState *pstate, TargetEntry *tle,
grpcl->eqop = eqop;
grpcl->sortop = sortop;
grpcl->nulls_first = false; /* OK with or without sortop */
grpcl->hashable = hashable;
grouplist = lappend(grouplist, grpcl);
}

View File

@ -171,6 +171,9 @@ LookupOperNameTypeNames(ParseState *pstate, List *opername,
* If an operator is missing and the corresponding needXX flag is true,
* throw a standard error message, else return InvalidOid.
*
* In addition to the operator OIDs themselves, this function can identify
* whether the "=" operator is hashable.
*
* Callers can pass NULL pointers for any results they don't care to get.
*
* Note: the results are guaranteed to be exact or binary-compatible matches,
@ -180,30 +183,40 @@ LookupOperNameTypeNames(ParseState *pstate, List *opername,
void
get_sort_group_operators(Oid argtype,
bool needLT, bool needEQ, bool needGT,
Oid *ltOpr, Oid *eqOpr, Oid *gtOpr)
Oid *ltOpr, Oid *eqOpr, Oid *gtOpr,
bool *isHashable)
{
TypeCacheEntry *typentry;
int cache_flags;
Oid lt_opr;
Oid eq_opr;
Oid gt_opr;
bool hashable;
/*
* Look up the operators using the type cache.
*
* Note: the search algorithm used by typcache.c ensures that the results
* are consistent, ie all from the same opclass.
* are consistent, ie all from matching opclasses.
*/
typentry = lookup_type_cache(argtype,
TYPECACHE_LT_OPR | TYPECACHE_EQ_OPR | TYPECACHE_GT_OPR);
if (isHashable != NULL)
cache_flags = TYPECACHE_LT_OPR | TYPECACHE_EQ_OPR | TYPECACHE_GT_OPR |
TYPECACHE_HASH_PROC;
else
cache_flags = TYPECACHE_LT_OPR | TYPECACHE_EQ_OPR | TYPECACHE_GT_OPR;
typentry = lookup_type_cache(argtype, cache_flags);
lt_opr = typentry->lt_opr;
eq_opr = typentry->eq_opr;
gt_opr = typentry->gt_opr;
hashable = OidIsValid(typentry->hash_proc);
/*
* If the datatype is an array, then we can use array_lt and friends ...
* but only if there are suitable operators for the element type. (This
* check is not in the raw typcache.c code ... should it be?) Testing all
* three operator IDs here should be redundant, but let's do it anyway.
* but only if there are suitable operators for the element type.
* Likewise, array types are only hashable if the element type is.
* Testing all three operator IDs here should be redundant, but let's do
* it anyway.
*/
if (lt_opr == ARRAY_LT_OP ||
eq_opr == ARRAY_EQ_OP ||
@ -213,10 +226,7 @@ get_sort_group_operators(Oid argtype,
if (OidIsValid(elem_type))
{
typentry = lookup_type_cache(elem_type,
TYPECACHE_LT_OPR | TYPECACHE_EQ_OPR | TYPECACHE_GT_OPR);
#ifdef NOT_USED
/* We should do this ... */
typentry = lookup_type_cache(elem_type, cache_flags);
if (!OidIsValid(typentry->eq_opr))
{
/* element type is neither sortable nor hashable */
@ -228,22 +238,13 @@ get_sort_group_operators(Oid argtype,
/* element type is hashable but not sortable */
lt_opr = gt_opr = InvalidOid;
}
#else
/*
* ... but for the moment we have to do this. This is because
* anyarray has sorting but not hashing support. So, if the
* element type is only hashable, there is nothing we can do with
* the array type.
*/
if (!OidIsValid(typentry->lt_opr) ||
!OidIsValid(typentry->eq_opr) ||
!OidIsValid(typentry->gt_opr))
lt_opr = eq_opr = gt_opr = InvalidOid; /* not sortable */
#endif
hashable = OidIsValid(typentry->hash_proc);
}
else
{
lt_opr = eq_opr = gt_opr = InvalidOid; /* bogus array type? */
hashable = false;
}
}
/* Report errors if needed */
@ -267,6 +268,8 @@ get_sort_group_operators(Oid argtype,
*eqOpr = eq_opr;
if (gtOpr)
*gtOpr = gt_opr;
if (isHashable)
*isHashable = hashable;
}

View File

@ -3448,6 +3448,117 @@ array_cmp(FunctionCallInfo fcinfo)
}
/*-----------------------------------------------------------------------------
* array hashing
* Hash the elements and combine the results.
*----------------------------------------------------------------------------
*/
Datum
hash_array(PG_FUNCTION_ARGS)
{
ArrayType *array = PG_GETARG_ARRAYTYPE_P(0);
int ndims = ARR_NDIM(array);
int *dims = ARR_DIMS(array);
Oid element_type = ARR_ELEMTYPE(array);
uint32 result = 0;
int nitems;
TypeCacheEntry *typentry;
int typlen;
bool typbyval;
char typalign;
char *ptr;
bits8 *bitmap;
int bitmask;
int i;
FunctionCallInfoData locfcinfo;
/*
* We arrange to look up the hash function only once per series of calls,
* assuming the element type doesn't change underneath us. The typcache
* is used so that we have no memory leakage when being used as an index
* support function.
*/
typentry = (TypeCacheEntry *) fcinfo->flinfo->fn_extra;
if (typentry == NULL ||
typentry->type_id != element_type)
{
typentry = lookup_type_cache(element_type,
TYPECACHE_HASH_PROC_FINFO);
if (!OidIsValid(typentry->hash_proc_finfo.fn_oid))
ereport(ERROR,
(errcode(ERRCODE_UNDEFINED_FUNCTION),
errmsg("could not identify a hash function for type %s",
format_type_be(element_type))));
fcinfo->flinfo->fn_extra = (void *) typentry;
}
typlen = typentry->typlen;
typbyval = typentry->typbyval;
typalign = typentry->typalign;
/*
* apply the hash function to each array element.
*/
InitFunctionCallInfoData(locfcinfo, &typentry->hash_proc_finfo, 1,
NULL, NULL);
/* Loop over source data */
nitems = ArrayGetNItems(ndims, dims);
ptr = ARR_DATA_PTR(array);
bitmap = ARR_NULLBITMAP(array);
bitmask = 1;
for (i = 0; i < nitems; i++)
{
uint32 elthash;
/* Get element, checking for NULL */
if (bitmap && (*bitmap & bitmask) == 0)
{
/* Treat nulls as having hashvalue 0 */
elthash = 0;
}
else
{
Datum elt;
elt = fetch_att(ptr, typbyval, typlen);
ptr = att_addlength_pointer(ptr, typlen, ptr);
ptr = (char *) att_align_nominal(ptr, typalign);
/* Apply the hash function */
locfcinfo.arg[0] = elt;
locfcinfo.argnull[0] = false;
locfcinfo.isnull = false;
elthash = DatumGetUInt32(FunctionCallInvoke(&locfcinfo));
}
/* advance bitmap pointer if any */
if (bitmap)
{
bitmask <<= 1;
if (bitmask == 0x100)
{
bitmap++;
bitmask = 1;
}
}
/*
* Combine hash values of successive elements by rotating the previous
* value left 1 bit, then XOR'ing in the new element's hash value.
*/
result = (result << 1) | (result >> 31);
result ^= elthash;
}
/* Avoid leaking memory when handed toasted input. */
PG_FREE_IF_COPY(array, 0);
PG_RETURN_UINT32(result);
}
/*-----------------------------------------------------------------------------
* array overlap/containment comparisons
* These use the same methods of comparing array elements as array_eq.

View File

@ -34,6 +34,7 @@
#include "utils/datum.h"
#include "utils/lsyscache.h"
#include "utils/syscache.h"
#include "utils/typcache.h"
/* Hook for plugins to get control in get_attavgwidth() */
get_attavgwidth_hook_type get_attavgwidth_hook = NULL;
@ -1054,20 +1055,47 @@ op_input_types(Oid opno, Oid *lefttype, Oid *righttype)
* will fail to find any mergejoin plans unless there are suitable btree
* opfamily entries for this operator and associated sortops. The pg_operator
* flag is just a hint to tell the planner whether to bother looking.)
*
* In some cases (currently only array_eq), mergejoinability depends on the
* specific input data type the operator is invoked for, so that must be
* passed as well. We currently assume that only one input's type is needed
* to check this --- by convention, pass the left input's data type.
*/
bool
op_mergejoinable(Oid opno)
op_mergejoinable(Oid opno, Oid inputtype)
{
HeapTuple tp;
bool result = false;
tp = SearchSysCache1(OPEROID, ObjectIdGetDatum(opno));
if (HeapTupleIsValid(tp))
if (opno == ARRAY_EQ_OP)
{
Form_pg_operator optup = (Form_pg_operator) GETSTRUCT(tp);
/*
* For array_eq, can sort if element type has a default btree opclass.
* We could use GetDefaultOpClass, but that's fairly expensive and not
* cached, so let's use the typcache instead.
*/
Oid elem_type = get_base_element_type(inputtype);
result = optup->oprcanmerge;
ReleaseSysCache(tp);
if (OidIsValid(elem_type))
{
TypeCacheEntry *typentry;
typentry = lookup_type_cache(elem_type, TYPECACHE_BTREE_OPFAMILY);
if (OidIsValid(typentry->btree_opf))
result = true;
}
}
else
{
/* For all other operators, rely on pg_operator.oprcanmerge */
tp = SearchSysCache1(OPEROID, ObjectIdGetDatum(opno));
if (HeapTupleIsValid(tp))
{
Form_pg_operator optup = (Form_pg_operator) GETSTRUCT(tp);
result = optup->oprcanmerge;
ReleaseSysCache(tp);
}
}
return result;
}
@ -1077,20 +1105,43 @@ op_mergejoinable(Oid opno)
*
* Returns true if the operator is hashjoinable. (There must be a suitable
* hash opfamily entry for this operator if it is so marked.)
*
* In some cases (currently only array_eq), hashjoinability depends on the
* specific input data type the operator is invoked for, so that must be
* passed as well. We currently assume that only one input's type is needed
* to check this --- by convention, pass the left input's data type.
*/
bool
op_hashjoinable(Oid opno)
op_hashjoinable(Oid opno, Oid inputtype)
{
HeapTuple tp;
bool result = false;
tp = SearchSysCache1(OPEROID, ObjectIdGetDatum(opno));
if (HeapTupleIsValid(tp))
if (opno == ARRAY_EQ_OP)
{
Form_pg_operator optup = (Form_pg_operator) GETSTRUCT(tp);
/* For array_eq, can hash if element type has a default hash opclass */
Oid elem_type = get_base_element_type(inputtype);
result = optup->oprcanhash;
ReleaseSysCache(tp);
if (OidIsValid(elem_type))
{
TypeCacheEntry *typentry;
typentry = lookup_type_cache(elem_type, TYPECACHE_HASH_OPFAMILY);
if (OidIsValid(typentry->hash_opf))
result = true;
}
}
else
{
/* For all other operators, rely on pg_operator.oprcanhash */
tp = SearchSysCache1(OPEROID, ObjectIdGetDatum(opno));
if (HeapTupleIsValid(tp))
{
Form_pg_operator optup = (Form_pg_operator) GETSTRUCT(tp);
result = optup->oprcanhash;
ReleaseSysCache(tp);
}
}
return result;
}

View File

@ -10,10 +10,10 @@
* be used for grouping and sorting the type (GROUP BY, ORDER BY ASC/DESC).
*
* Several seemingly-odd choices have been made to support use of the type
* cache by the generic array comparison routines array_eq() and array_cmp().
* Because those routines are used as index support operations, they cannot
* leak memory. To allow them to execute efficiently, all information that
* either of them would like to re-use across calls is made available in the
* cache by the generic array handling routines array_eq(), array_cmp(),
* and hash_array(). Because those routines are used as index support
* operations, they cannot leak memory. To allow them to execute efficiently,
* all information that they would like to re-use across calls is kept in the
* type cache.
*
* Once created, a type cache entry lives as long as the backend does, so
@ -193,7 +193,9 @@ lookup_type_cache(Oid type_id, int flags)
ReleaseSysCache(tp);
}
/* If we haven't already found the opclass, try to do so */
/*
* If we haven't already found the opclasses, try to do so
*/
if ((flags & (TYPECACHE_EQ_OPR | TYPECACHE_LT_OPR | TYPECACHE_GT_OPR |
TYPECACHE_CMP_PROC |
TYPECACHE_EQ_OPR_FINFO | TYPECACHE_CMP_PROC_FINFO |
@ -208,7 +210,7 @@ lookup_type_cache(Oid type_id, int flags)
typentry->btree_opf = get_opclass_family(opclass);
typentry->btree_opintype = get_opclass_input_type(opclass);
}
/* Only care about hash opclass if no btree opclass... */
/* If no btree opclass, we force lookup of the hash opclass */
if (typentry->btree_opf == InvalidOid)
{
if (typentry->hash_opf == InvalidOid)
@ -224,12 +226,30 @@ lookup_type_cache(Oid type_id, int flags)
else
{
/*
* If we find a btree opclass where previously we only found a
* hash opclass, forget the hash equality operator so we can use
* the btree operator instead.
* In case we find a btree opclass where previously we only found
* a hash opclass, reset eq_opr and derived information so that
* we can fetch the btree equality operator instead of the hash
* equality operator. (They're probably the same operator, but
* we don't assume that here.)
*/
typentry->eq_opr = InvalidOid;
typentry->eq_opr_finfo.fn_oid = InvalidOid;
typentry->hash_proc = InvalidOid;
typentry->hash_proc_finfo.fn_oid = InvalidOid;
}
}
if ((flags & (TYPECACHE_HASH_PROC | TYPECACHE_HASH_PROC_FINFO |
TYPECACHE_HASH_OPFAMILY)) &&
typentry->hash_opf == InvalidOid)
{
Oid opclass;
opclass = GetDefaultOpClass(type_id, HASH_AM_OID);
if (OidIsValid(opclass))
{
typentry->hash_opf = get_opclass_family(opclass);
typentry->hash_opintype = get_opclass_input_type(opclass);
}
}
@ -248,6 +268,14 @@ lookup_type_cache(Oid type_id, int flags)
typentry->hash_opintype,
typentry->hash_opintype,
HTEqualStrategyNumber);
/*
* Reset info about hash function whenever we pick up new info about
* equality operator. This is so we can ensure that the hash function
* matches the operator.
*/
typentry->hash_proc = InvalidOid;
typentry->hash_proc_finfo.fn_oid = InvalidOid;
}
if ((flags & TYPECACHE_LT_OPR) && typentry->lt_opr == InvalidOid)
{
@ -274,6 +302,24 @@ lookup_type_cache(Oid type_id, int flags)
typentry->btree_opintype,
BTORDER_PROC);
}
if ((flags & (TYPECACHE_HASH_PROC | TYPECACHE_HASH_PROC_FINFO)) &&
typentry->hash_proc == InvalidOid)
{
/*
* We insist that the eq_opr, if one has been determined, match the
* hash opclass; else report there is no hash function.
*/
if (typentry->hash_opf != InvalidOid &&
(!OidIsValid(typentry->eq_opr) ||
typentry->eq_opr == get_opfamily_member(typentry->hash_opf,
typentry->hash_opintype,
typentry->hash_opintype,
HTEqualStrategyNumber)))
typentry->hash_proc = get_opfamily_proc(typentry->hash_opf,
typentry->hash_opintype,
typentry->hash_opintype,
HASHPROC);
}
/*
* Set up fmgr lookup info as requested
@ -300,6 +346,13 @@ lookup_type_cache(Oid type_id, int flags)
fmgr_info_cxt(typentry->cmp_proc, &typentry->cmp_proc_finfo,
CacheMemoryContext);
}
if ((flags & TYPECACHE_HASH_PROC_FINFO) &&
typentry->hash_proc_finfo.fn_oid == InvalidOid &&
typentry->hash_proc != InvalidOid)
{
fmgr_info_cxt(typentry->hash_proc, &typentry->hash_proc_finfo,
CacheMemoryContext);
}
/*
* If it's a composite type (row type), get tupdesc if requested

View File

@ -53,6 +53,6 @@
*/
/* yyyymmddN */
#define CATALOG_VERSION_NO 201010241
#define CATALOG_VERSION_NO 201010301
#endif

View File

@ -561,6 +561,8 @@ DATA(insert ( 2235 1033 1033 1 974 405 ));
DATA(insert ( 2969 2950 2950 1 2972 405 ));
/* numeric_ops */
DATA(insert ( 1998 1700 1700 1 1752 405 ));
/* array_ops */
DATA(insert ( 627 2277 2277 1 1070 405 ));
/*

View File

@ -130,6 +130,7 @@ DATA(insert ( 3522 3500 3500 1 3514 ));
DATA(insert ( 427 1042 1042 1 1080 ));
DATA(insert ( 431 18 18 1 454 ));
DATA(insert ( 435 1082 1082 1 450 ));
DATA(insert ( 627 2277 2277 1 626 ));
DATA(insert ( 1971 700 700 1 451 ));
DATA(insert ( 1971 701 701 1 452 ));
DATA(insert ( 1975 869 869 1 422 ));

View File

@ -92,6 +92,7 @@ typedef FormData_pg_opclass *Form_pg_opclass;
DATA(insert ( 403 abstime_ops PGNSP PGUID 421 702 t 0 ));
DATA(insert ( 403 array_ops PGNSP PGUID 397 2277 t 0 ));
DATA(insert ( 405 array_ops PGNSP PGUID 627 2277 t 0 ));
DATA(insert ( 403 bit_ops PGNSP PGUID 423 1560 t 0 ));
DATA(insert ( 403 bool_ops PGNSP PGUID 424 16 t 0 ));
DATA(insert ( 403 bpchar_ops PGNSP PGUID 426 1042 t 0 ));

View File

@ -439,7 +439,7 @@ DATA(insert OID = 1060 ( ">" PGNSP PGUID b f f 1042 1042 16 1058 1059 bpchar
DATA(insert OID = 1061 ( ">=" PGNSP PGUID b f f 1042 1042 16 1059 1058 bpcharge scalargtsel scalargtjoinsel ));
/* generic array comparison operators */
DATA(insert OID = 1070 ( "=" PGNSP PGUID b t f 2277 2277 16 1070 1071 array_eq eqsel eqjoinsel ));
DATA(insert OID = 1070 ( "=" PGNSP PGUID b t t 2277 2277 16 1070 1071 array_eq eqsel eqjoinsel ));
#define ARRAY_EQ_OP 1070
DATA(insert OID = 1071 ( "<>" PGNSP PGUID b f f 2277 2277 16 1071 1070 array_ne neqsel neqjoinsel ));
DATA(insert OID = 1072 ( "<" PGNSP PGUID b f f 2277 2277 16 1073 1075 array_lt scalarltsel scalarltjoinsel ));

View File

@ -60,6 +60,7 @@ typedef FormData_pg_opfamily *Form_pg_opfamily;
DATA(insert OID = 421 ( 403 abstime_ops PGNSP PGUID ));
DATA(insert OID = 397 ( 403 array_ops PGNSP PGUID ));
DATA(insert OID = 627 ( 405 array_ops PGNSP PGUID ));
DATA(insert OID = 423 ( 403 bit_ops PGNSP PGUID ));
DATA(insert OID = 424 ( 403 bool_ops PGNSP PGUID ));
#define BOOL_BTREE_FAM_OID 424

View File

@ -907,6 +907,9 @@ DESCR("convert float8 to int8");
/* OIDS 600 - 699 */
DATA(insert OID = 626 ( hash_array PGNSP PGUID 12 1 0 0 f f f t f i 1 0 23 "2277" _null_ _null_ _null_ _null_ hash_array _null_ _null_ _null_ ));
DESCR("hash");
DATA(insert OID = 652 ( float4 PGNSP PGUID 12 1 0 0 f f f t f i 1 0 700 "20" _null_ _null_ _null_ _null_ i8tof _null_ _null_ _null_ ));
DESCR("convert int8 to float4");
DATA(insert OID = 653 ( int8 PGNSP PGUID 12 1 0 0 f f f t f i 1 0 20 "700" _null_ _null_ _null_ _null_ ftoi8 _null_ _null_ _null_ ));

View File

@ -757,6 +757,8 @@ typedef struct RangeTblEntry
* or InvalidOid if not available.
* nulls_first means about what you'd expect. If sortop is InvalidOid
* then nulls_first is meaningless and should be set to false.
* hashable is TRUE if eqop is hashable (note this condition also depends
* on the datatype of the input expression).
*
* In an ORDER BY item, all fields must be valid. (The eqop isn't essential
* here, but it's cheap to get it along with the sortop, and requiring it
@ -773,6 +775,11 @@ typedef struct RangeTblEntry
* and nulls_first to false. A grouping item of this kind can only be
* implemented by hashing, and of course it'll never match an ORDER BY item.
*
* The hashable flag is provided since we generally have the requisite
* information readily available when the SortGroupClause is constructed,
* and it's relatively expensive to get it again later. Note there is no
* need for a "sortable" flag since OidIsValid(sortop) serves the purpose.
*
* A query might have both ORDER BY and DISTINCT (or DISTINCT ON) clauses.
* In SELECT DISTINCT, the distinctClause list is as long or longer than the
* sortClause list, while in SELECT DISTINCT ON it's typically shorter.
@ -789,6 +796,7 @@ typedef struct SortGroupClause
Oid eqop; /* the equality operator ('=' op) */
Oid sortop; /* the ordering operator ('<' op), or 0 */
bool nulls_first; /* do NULLs come before normal values? */
bool hashable; /* can eqop be implemented by hashing? */
} SortGroupClause;
/*

View File

@ -48,7 +48,8 @@ extern Operator compatible_oper(ParseState *pstate, List *op,
/* Routines for identifying "<", "=", ">" operators for a type */
extern void get_sort_group_operators(Oid argtype,
bool needLT, bool needEQ, bool needGT,
Oid *ltOpr, Oid *eqOpr, Oid *gtOpr);
Oid *ltOpr, Oid *eqOpr, Oid *gtOpr,
bool *isHashable);
/* Convenience routines for common calls on the above */
extern Oid compatible_oper_opid(List *op, Oid arg1, Oid arg2, bool noError);

View File

@ -192,6 +192,7 @@ extern Datum array_gt(PG_FUNCTION_ARGS);
extern Datum array_le(PG_FUNCTION_ARGS);
extern Datum array_ge(PG_FUNCTION_ARGS);
extern Datum btarraycmp(PG_FUNCTION_ARGS);
extern Datum hash_array(PG_FUNCTION_ARGS);
extern Datum arrayoverlap(PG_FUNCTION_ARGS);
extern Datum arraycontains(PG_FUNCTION_ARGS);
extern Datum arraycontained(PG_FUNCTION_ARGS);

View File

@ -67,8 +67,8 @@ extern Oid get_opclass_input_type(Oid opclass);
extern RegProcedure get_opcode(Oid opno);
extern char *get_opname(Oid opno);
extern void op_input_types(Oid opno, Oid *lefttype, Oid *righttype);
extern bool op_mergejoinable(Oid opno);
extern bool op_hashjoinable(Oid opno);
extern bool op_mergejoinable(Oid opno, Oid inputtype);
extern bool op_hashjoinable(Oid opno, Oid inputtype);
extern bool op_strict(Oid opno);
extern char op_volatile(Oid opno);
extern Oid get_commutator(Oid opno);

View File

@ -49,16 +49,18 @@ typedef struct TypeCacheEntry
Oid lt_opr; /* the less-than operator */
Oid gt_opr; /* the greater-than operator */
Oid cmp_proc; /* the btree comparison function */
Oid hash_proc; /* the hash calculation function */
/*
* Pre-set-up fmgr call info for the equality operator and the btree
* comparison function. These are kept in the type cache to avoid
* problems with memory leaks in repeated calls to array_eq and array_cmp.
* There is not currently a need to maintain call info for the lt_opr or
* gt_opr.
* Pre-set-up fmgr call info for the equality operator, the btree
* comparison function, and the hash calculation function. These are kept
* in the type cache to avoid problems with memory leaks in repeated calls
* to array_eq, array_cmp, hash_array. There is not currently a need to
* maintain call info for the lt_opr or gt_opr.
*/
FmgrInfo eq_opr_finfo;
FmgrInfo cmp_proc_finfo;
FmgrInfo hash_proc_finfo;
/*
* Tuple descriptor if it's a composite type (row type). NULL if not
@ -79,10 +81,13 @@ typedef struct TypeCacheEntry
#define TYPECACHE_LT_OPR 0x0002
#define TYPECACHE_GT_OPR 0x0004
#define TYPECACHE_CMP_PROC 0x0008
#define TYPECACHE_EQ_OPR_FINFO 0x0010
#define TYPECACHE_CMP_PROC_FINFO 0x0020
#define TYPECACHE_TUPDESC 0x0040
#define TYPECACHE_BTREE_OPFAMILY 0x0080
#define TYPECACHE_HASH_PROC 0x0010
#define TYPECACHE_EQ_OPR_FINFO 0x0020
#define TYPECACHE_CMP_PROC_FINFO 0x0040
#define TYPECACHE_HASH_PROC_FINFO 0x0080
#define TYPECACHE_TUPDESC 0x0100
#define TYPECACHE_BTREE_OPFAMILY 0x0200
#define TYPECACHE_HASH_OPFAMILY 0x0400
extern TypeCacheEntry *lookup_type_cache(Oid type_id, int flags);