You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hawq.apache.org by zhangjackey <gi...@git.apache.org> on 2018/04/04 08:17:07 UTC

[GitHub] incubator-hawq pull request #1351: HAWQ-1603.add new hook API

GitHub user zhangjackey opened a pull request:

    https://github.com/apache/incubator-hawq/pull/1351

    HAWQ-1603.add new hook API

    add two hook API.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/zhangjackey/incubator-hawq v_expr

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/incubator-hawq/pull/1351.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1351
    
----
commit 6a44fd37be021c083004619e74834078bf967c0c
Author: Shujie Zhang <sh...@...>
Date:   2018-04-04T08:10:07Z

    add new hook API

----


---

[GitHub] incubator-hawq pull request #1351: HAWQ-1603.add new hook API

Posted by zhangjackey <gi...@git.apache.org>.
Github user zhangjackey commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/1351#discussion_r181933851
  
    --- Diff: contrib/vexecutor/execVQual.c ---
    @@ -123,3 +123,431 @@ VirtualNodeProc(ScanState* state,TupleTableSlot *slot){
         ExecStoreVirtualTuple(slot);
         return true;
     }
    +
    +/*
    + * get values from vectorized tuple slot
    + * copy from src/backend/executor/execQual.c
    + */
    +static Datum
    +VExecEvalScalarVar(ExprState *exprstate, ExprContext *econtext,
    +			bool *isNull, ExprDoneCond *isDone)
    +{
    +	Var		   *variable = (Var *) exprstate->expr;
    +	TupleTableSlot *slot;
    +	AttrNumber	attnum;
    +	TupleBatch tb;
    +
    +	if (isDone)
    +		*isDone = ExprSingleResult;
    +
    +	Assert(econtext->ecxt_scantuple != NULL || econtext->ecxt_innertuple != NULL || econtext->ecxt_outertuple != NULL);
    +	/*
    +	 * Get the input slot and attribute number we want
    +	 *
    +	 * The asserts check that references to system attributes only appear at
    +	 * the level of a relation scan; at higher levels, system attributes must
    +	 * be treated as ordinary variables (since we no longer have access to the
    +	 * original tuple).
    +	 */
    +	attnum = variable->varattno;
    +
    +	switch (variable->varno)
    +	{
    +		case INNER:				/* get the tuple from the inner node */
    +			slot = econtext->ecxt_innertuple;
    +			Assert(attnum > 0);
    +			break;
    +
    +		case OUTER:				/* get the tuple from the outer node */
    +			slot = econtext->ecxt_outertuple;
    +			Assert(attnum > 0);
    +			break;
    +
    +		default:				/* get the tuple from the relation being
    +								 * scanned */
    +			slot = econtext->ecxt_scantuple;
    +			break;
    +	}
    +
    +	/* isNull is a single value, it can not be used when data is vectorized */
    +	*isNull = false;
    +
    --- End diff --
    
    add Assert in next commit.


---

[GitHub] incubator-hawq pull request #1351: HAWQ-1603.add new hook API

Posted by wengyanqing <gi...@git.apache.org>.
Github user wengyanqing commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/1351#discussion_r180625706
  
    --- Diff: contrib/vexecutor/execVQual.c ---
    @@ -123,3 +123,431 @@ VirtualNodeProc(ScanState* state,TupleTableSlot *slot){
         ExecStoreVirtualTuple(slot);
         return true;
     }
    +
    +/*
    + * get values from vectorized tuple slot
    + * copy from src/backend/executor/execQual.c
    + */
    +static Datum
    +VExecEvalScalarVar(ExprState *exprstate, ExprContext *econtext,
    +			bool *isNull, ExprDoneCond *isDone)
    +{
    +	Var		   *variable = (Var *) exprstate->expr;
    +	TupleTableSlot *slot;
    +	AttrNumber	attnum;
    +	TupleBatch tb;
    +
    +	if (isDone)
    +		*isDone = ExprSingleResult;
    +
    +	Assert(econtext->ecxt_scantuple != NULL || econtext->ecxt_innertuple != NULL || econtext->ecxt_outertuple != NULL);
    +	/*
    +	 * Get the input slot and attribute number we want
    +	 *
    +	 * The asserts check that references to system attributes only appear at
    +	 * the level of a relation scan; at higher levels, system attributes must
    +	 * be treated as ordinary variables (since we no longer have access to the
    +	 * original tuple).
    +	 */
    +	attnum = variable->varattno;
    +
    +	switch (variable->varno)
    +	{
    +		case INNER:				/* get the tuple from the inner node */
    +			slot = econtext->ecxt_innertuple;
    +			Assert(attnum > 0);
    +			break;
    +
    +		case OUTER:				/* get the tuple from the outer node */
    +			slot = econtext->ecxt_outertuple;
    +			Assert(attnum > 0);
    +			break;
    +
    +		default:				/* get the tuple from the relation being
    +								 * scanned */
    +			slot = econtext->ecxt_scantuple;
    +			break;
    +	}
    +
    +	/* isNull is a single value, it can not be used when data is vectorized */
    +	*isNull = false;
    +
    +	/* Fetch the value from the slot */
    +	tb = (TupleBatch )slot->PRIVATE_tb;
    +
    +	Assert(NULL != tb);
    +
    +	return PointerGetDatum(tb->datagroup[attnum]);
    --- End diff --
    
    attnum's value starts with 1, it should be attnum -1 as the index of tb->datagroup ?


---

[GitHub] incubator-hawq pull request #1351: HAWQ-1603.add new hook API

Posted by wengyanqing <gi...@git.apache.org>.
Github user wengyanqing commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/1351#discussion_r180626789
  
    --- Diff: contrib/vexecutor/vexecutor.c ---
    @@ -163,24 +165,35 @@ static PlanState* VExecInitNode(PlanState *node,EState *eState,int eflags,Memory
     			case T_AppendOnlyScan:
     			case T_ParquetScan:
     			case T_TableScanState:
    -				START_MEMORY_ACCOUNT(curMemoryAccount);
    +				//START_MEMORY_ACCOUNT(curMemoryAccount);
    --- End diff --
    
    Any reason that comment this code line ?


---

[GitHub] incubator-hawq pull request #1351: HAWQ-1603.add new hook API

Posted by wengyanqing <gi...@git.apache.org>.
Github user wengyanqing commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/1351#discussion_r180627154
  
    --- Diff: contrib/vexecutor/execVQual.c ---
    @@ -123,3 +123,431 @@ VirtualNodeProc(ScanState* state,TupleTableSlot *slot){
         ExecStoreVirtualTuple(slot);
         return true;
     }
    +
    +/*
    + * get values from vectorized tuple slot
    + * copy from src/backend/executor/execQual.c
    + */
    +static Datum
    +VExecEvalScalarVar(ExprState *exprstate, ExprContext *econtext,
    +			bool *isNull, ExprDoneCond *isDone)
    +{
    +	Var		   *variable = (Var *) exprstate->expr;
    +	TupleTableSlot *slot;
    +	AttrNumber	attnum;
    +	TupleBatch tb;
    +
    +	if (isDone)
    +		*isDone = ExprSingleResult;
    +
    +	Assert(econtext->ecxt_scantuple != NULL || econtext->ecxt_innertuple != NULL || econtext->ecxt_outertuple != NULL);
    +	/*
    +	 * Get the input slot and attribute number we want
    +	 *
    +	 * The asserts check that references to system attributes only appear at
    +	 * the level of a relation scan; at higher levels, system attributes must
    +	 * be treated as ordinary variables (since we no longer have access to the
    +	 * original tuple).
    +	 */
    +	attnum = variable->varattno;
    +
    +	switch (variable->varno)
    +	{
    +		case INNER:				/* get the tuple from the inner node */
    +			slot = econtext->ecxt_innertuple;
    +			Assert(attnum > 0);
    +			break;
    +
    +		case OUTER:				/* get the tuple from the outer node */
    +			slot = econtext->ecxt_outertuple;
    +			Assert(attnum > 0);
    +			break;
    +
    +		default:				/* get the tuple from the relation being
    +								 * scanned */
    +			slot = econtext->ecxt_scantuple;
    +			break;
    +	}
    +
    +	/* isNull is a single value, it can not be used when data is vectorized */
    +	*isNull = false;
    +
    --- End diff --
    
    It's better to check or assert to make sure slot is not null here.


---

[GitHub] incubator-hawq pull request #1351: HAWQ-1603.add new hook API

Posted by zhangjackey <gi...@git.apache.org>.
Github user zhangjackey commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/1351#discussion_r181942632
  
    --- Diff: contrib/vexecutor/vexecutor.c ---
    @@ -163,24 +165,35 @@ static PlanState* VExecInitNode(PlanState *node,EState *eState,int eflags,Memory
     			case T_AppendOnlyScan:
     			case T_ParquetScan:
     			case T_TableScanState:
    -				START_MEMORY_ACCOUNT(curMemoryAccount);
    +				//START_MEMORY_ACCOUNT(curMemoryAccount);
    --- End diff --
    
    if the plan->memoryAccount is not NULL, we use it directly, otherwise, we do not enable it.


---

[GitHub] incubator-hawq pull request #1351: HAWQ-1603.add new hook API

Posted by zhangjackey <gi...@git.apache.org>.
Github user zhangjackey closed the pull request at:

    https://github.com/apache/incubator-hawq/pull/1351


---

[GitHub] incubator-hawq pull request #1351: HAWQ-1603.add new hook API

Posted by zhangjackey <gi...@git.apache.org>.
Github user zhangjackey commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/1351#discussion_r181942311
  
    --- Diff: contrib/vexecutor/execVQual.c ---
    @@ -123,3 +123,431 @@ VirtualNodeProc(ScanState* state,TupleTableSlot *slot){
         ExecStoreVirtualTuple(slot);
         return true;
     }
    +
    +/*
    + * get values from vectorized tuple slot
    + * copy from src/backend/executor/execQual.c
    + */
    +static Datum
    +VExecEvalScalarVar(ExprState *exprstate, ExprContext *econtext,
    +			bool *isNull, ExprDoneCond *isDone)
    +{
    +	Var		   *variable = (Var *) exprstate->expr;
    +	TupleTableSlot *slot;
    +	AttrNumber	attnum;
    +	TupleBatch tb;
    +
    +	if (isDone)
    +		*isDone = ExprSingleResult;
    +
    +	Assert(econtext->ecxt_scantuple != NULL || econtext->ecxt_innertuple != NULL || econtext->ecxt_outertuple != NULL);
    +	/*
    +	 * Get the input slot and attribute number we want
    +	 *
    +	 * The asserts check that references to system attributes only appear at
    +	 * the level of a relation scan; at higher levels, system attributes must
    +	 * be treated as ordinary variables (since we no longer have access to the
    +	 * original tuple).
    +	 */
    +	attnum = variable->varattno;
    +
    +	switch (variable->varno)
    +	{
    +		case INNER:				/* get the tuple from the inner node */
    +			slot = econtext->ecxt_innertuple;
    +			Assert(attnum > 0);
    +			break;
    +
    +		case OUTER:				/* get the tuple from the outer node */
    +			slot = econtext->ecxt_outertuple;
    +			Assert(attnum > 0);
    +			break;
    +
    +		default:				/* get the tuple from the relation being
    +								 * scanned */
    +			slot = econtext->ecxt_scantuple;
    +			break;
    +	}
    +
    +	/* isNull is a single value, it can not be used when data is vectorized */
    +	*isNull = false;
    +
    +	/* Fetch the value from the slot */
    +	tb = (TupleBatch )slot->PRIVATE_tb;
    +
    +	Assert(NULL != tb);
    +
    +	return PointerGetDatum(tb->datagroup[attnum]);
    --- End diff --
    
    yes, change it to attnum - 1.


---

[GitHub] incubator-hawq pull request #1351: HAWQ-1603.add new hook API

Posted by wengyanqing <gi...@git.apache.org>.
Github user wengyanqing commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/1351#discussion_r180626694
  
    --- Diff: contrib/vexecutor/execVQual.c ---
    @@ -123,3 +123,431 @@ VirtualNodeProc(ScanState* state,TupleTableSlot *slot){
         ExecStoreVirtualTuple(slot);
         return true;
     }
    +
    +/*
    + * get values from vectorized tuple slot
    + * copy from src/backend/executor/execQual.c
    + */
    +static Datum
    +VExecEvalScalarVar(ExprState *exprstate, ExprContext *econtext,
    +			bool *isNull, ExprDoneCond *isDone)
    +{
    +	Var		   *variable = (Var *) exprstate->expr;
    +	TupleTableSlot *slot;
    +	AttrNumber	attnum;
    +	TupleBatch tb;
    +
    +	if (isDone)
    +		*isDone = ExprSingleResult;
    +
    +	Assert(econtext->ecxt_scantuple != NULL || econtext->ecxt_innertuple != NULL || econtext->ecxt_outertuple != NULL);
    +	/*
    +	 * Get the input slot and attribute number we want
    +	 *
    +	 * The asserts check that references to system attributes only appear at
    +	 * the level of a relation scan; at higher levels, system attributes must
    +	 * be treated as ordinary variables (since we no longer have access to the
    +	 * original tuple).
    +	 */
    +	attnum = variable->varattno;
    +
    +	switch (variable->varno)
    +	{
    +		case INNER:				/* get the tuple from the inner node */
    +			slot = econtext->ecxt_innertuple;
    +			Assert(attnum > 0);
    +			break;
    +
    +		case OUTER:				/* get the tuple from the outer node */
    +			slot = econtext->ecxt_outertuple;
    +			Assert(attnum > 0);
    +			break;
    +
    +		default:				/* get the tuple from the relation being
    +								 * scanned */
    +			slot = econtext->ecxt_scantuple;
    +			break;
    +	}
    +
    +	/* isNull is a single value, it can not be used when data is vectorized */
    +	*isNull = false;
    +
    +	/* Fetch the value from the slot */
    +	tb = (TupleBatch )slot->PRIVATE_tb;
    +
    +	Assert(NULL != tb);
    +
    +	return PointerGetDatum(tb->datagroup[attnum]);
    +}
    +
    +
    +/*
    + * get values from vectorized tuple slot
    + * copy from src/backend/executor/execQual.c
    + */
    +static Datum
    +VExecEvalVar(ExprState *exprstate, ExprContext *econtext,
    +			bool *isNull, ExprDoneCond *isDone)
    +{
    +	Var		   *variable = (Var *) exprstate->expr;
    +	TupleTableSlot *slot;
    +	AttrNumber	attnum;
    +
    +	if (isDone)
    +		*isDone = ExprSingleResult;
    +
    +	Assert(econtext->ecxt_scantuple != NULL || econtext->ecxt_innertuple != NULL || econtext->ecxt_outertuple != NULL);
    +	/*
    +	 * Get the input slot and attribute number we want
    +	 *
    +	 * The asserts check that references to system attributes only appear at
    +	 * the level of a relation scan; at higher levels, system attributes must
    +	 * be treated as ordinary variables (since we no longer have access to the
    +	 * original tuple).
    +	 */
    +	attnum = variable->varattno;
    +
    +	switch (variable->varno)
    +	{
    +		case INNER:				/* get the tuple from the inner node */
    +			slot = econtext->ecxt_innertuple;
    +			Assert(attnum > 0);
    +			break;
    +
    +		case OUTER:				/* get the tuple from the outer node */
    +			slot = econtext->ecxt_outertuple;
    +			Assert(attnum > 0);
    +			break;
    +
    +		default:				/* get the tuple from the relation being
    +								 * scanned */
    +			slot = econtext->ecxt_scantuple;
    +			break;
    +	}
    +
    +	if (attnum != InvalidAttrNumber)
    +	{
    +		TupleBatch tb;
    +		/*
    +		 * Scalar variable case.
    +		 *
    +		 * If it's a user attribute, check validity (bogus system attnums will
    +		 * be caught inside slot_getattr).  What we have to check for here
    +		 * is the possibility of an attribute having been changed in type
    +		 * since the plan tree was created.  Ideally the plan would get
    +		 * invalidated and not re-used, but until that day arrives, we need
    +		 * defenses.  Fortunately it's sufficient to check once on the first
    +		 * time through.
    +		 *
    +		 * Note: we allow a reference to a dropped attribute.  slot_getattr
    +		 * will force a NULL result in such cases.
    +		 *
    +		 * Note: ideally we'd check typmod as well as typid, but that seems
    +		 * impractical at the moment: in many cases the tupdesc will have
    +		 * been generated by ExecTypeFromTL(), and that can't guarantee to
    +		 * generate an accurate typmod in all cases, because some expression
    +		 * node types don't carry typmod.
    +		 */
    +		if (attnum > 0)
    +		{
    +			TupleDesc	slot_tupdesc = slot->tts_tupleDescriptor;
    +			Form_pg_attribute attr;
    +
    +			if (attnum > slot_tupdesc->natts)	/* should never happen */
    +				elog(ERROR, "attribute number %d exceeds number of columns %d",
    +					 attnum, slot_tupdesc->natts);
    +
    +			attr = slot_tupdesc->attrs[attnum - 1];
    +
    +			/* can't check type if dropped, since atttypid is probably 0 */
    +			if (!attr->attisdropped)
    +			{
    +				if (variable->vartype != attr->atttypid)
    +					ereport(ERROR,
    +							(errmsg("attribute %d has wrong type", attnum),
    +							 errdetail("Table has type %s, but query expects %s.",
    +									   format_type_be(attr->atttypid),
    +									   format_type_be(variable->vartype))));
    +			}
    +		}
    +
    +		/* Skip the checking on future executions of node */
    +		exprstate->evalfunc = VExecEvalScalarVar;
    +
    +		/* isNull is a single value, it can not be used when data is vectorized */
    +		*isNull = false;
    +
    +		/* Fetch the value from the slot */
    +		tb = (TupleBatch )slot->PRIVATE_tb;
    +
    +		Assert(NULL != tb);
    +		return PointerGetDatum(tb->datagroup[attnum]);
    +	}
    +	else
    +	{
    +		/* NOT support so far */
    +		Assert(false);
    +	}
    +
    +	return PointerGetDatum(NULL);
    +}
    +
    +
    +/* ----------------------------------------------------------------
    + *		VExecEvalNot
    + *		VExecEvalOr
    + *		VExecEvalAnd
    + *
    + * copy from src/backend/executor/execQual.c
    + *
    + *		Evaluate boolean expressions, with appropriate short-circuiting.
    + *
    + *		The query planner reformulates clause expressions in the
    + *		qualification to conjunctive normal form.  If we ever get
    + *		an AND to evaluate, we can be sure that it's not a top-level
    + *		clause in the qualification, but appears lower (as a function
    + *		argument, for example), or in the target list.	Not that you
    + *		need to know this, mind you...
    + * ----------------------------------------------------------------
    + */
    +static Datum
    +VExecEvalNot(BoolExprState *notclause, ExprContext *econtext,
    +			bool *isNull, ExprDoneCond *isDone)
    +{
    +	ExprState  *clause = linitial(notclause->args);
    +	Datum		expr_value;
    +	vbool		*ret;
    +	int			i;
    +
    +	if (isDone)
    +		*isDone = ExprSingleResult;
    +
    +	expr_value = ExecEvalExpr(clause, econtext, isNull, NULL);
    +
    +	ret = (vbool*)DatumGetPointer(expr_value);
    +	for(i = 0; i < ret->header.dim; i++)
    +	{
    +		if(!ret->header.isnull[i])
    +			ret->values[i] = !ret->values[i];
    +	}
    +
    +	/*
    +	 * evaluation of 'not' is simple.. expr is false, then return 'true' and
    +	 * vice versa.
    +	 */
    +	return PointerGetDatum(ret);
    +}
    +
    +/* ----------------------------------------------------------------
    + *		ExecEvalOr
    + * ----------------------------------------------------------------
    + */
    +static Datum
    +VExecEvalOr(BoolExprState *orExpr, ExprContext *econtext,
    +		   bool *isNull, ExprDoneCond *isDone)
    +{
    +	List	   *clauses = orExpr->args;
    +	ListCell   *clause;
    +	bool		AnyNull;
    +	vbool	*res = NULL;
    +	vbool	*next = 	NULL;
    +	bool		skip = true;
    +	int 		i = 0;
    +
    +	if (isDone)
    +		*isDone = ExprSingleResult;
    +
    +	AnyNull = false;
    +
    +	/*
    +	 * If any of the clauses is TRUE, the OR result is TRUE regardless of the
    +	 * states of the rest of the clauses, so we can stop evaluating and return
    +	 * TRUE immediately.  If none are TRUE and one or more is NULL, we return
    +	 * NULL; otherwise we return FALSE.  This makes sense when you interpret
    +	 * NULL as "don't know": if we have a TRUE then the OR is TRUE even if we
    +	 * aren't sure about some of the other inputs. If all the known inputs are
    +	 * FALSE, but we have one or more "don't knows", then we have to report
    +	 * that we "don't know" what the OR's result should be --- perhaps one of
    +	 * the "don't knows" would have been TRUE if we'd known its value.  Only
    +	 * when all the inputs are known to be FALSE can we state confidently that
    +	 * the OR's result is FALSE.
    +	 */
    +	foreach(clause, clauses)
    +	{
    +		ExprState  *clausestate = (ExprState *) lfirst(clause);
    +		Datum		clause_value;
    +
    +		/*
    +		 * to check if all the values is true, then skip to evaluate some
    +		 * expressions
    +		 */
    +		skip = true;
    +
    +		clause_value = ExecEvalExpr(clausestate, econtext, isNull, NULL);
    +
    +		if(NULL == res)
    +		{
    +			res = DatumGetPointer(clause_value);
    +			for(i = 0; i < res->header.dim; i++)
    +			{
    +				if(res->header.isnull[i] || !res->values[i])
    +				{
    +					skip = false;
    +					break;
    +				}
    +			}
    +		}
    +		else
    +		{
    +			next = DatumGetPointer(clause_value);
    +			for(i = 0; i < res->header.dim; i++)
    +			{
    +				res->header.isnull[i] = (res->header.isnull[i] || next->header.isnull[i]);
    +				res->values[i] = (res->values[i] || next->values[i]);
    +				if(skip && (res->header.isnull[i] || !res->values[i]))
    +					skip = false;
    +			}
    +		}
    +
    +		if(skip)
    +		{
    +			*isNull = false;
    +			return PointerGetDatum(res);
    +		}
    +	}
    +
    +	*isNull = false;
    +	return PointerGetDatum(res);
    +}
    +
    +/* ----------------------------------------------------------------
    + *		ExecEvalAnd
    + * ----------------------------------------------------------------
    + */
    +static Datum
    +VExecEvalAnd(BoolExprState *andExpr, ExprContext *econtext,
    +			bool *isNull, ExprDoneCond *isDone)
    +{
    +	List	   *clauses = andExpr->args;
    +	ListCell   *clause;
    +	bool		AnyNull;
    +	vbool	*res = NULL;
    +	vbool	*next = 	NULL;
    +	bool		skip = true;
    +	int 		i = 0;
    +
    +	if (isDone)
    +		*isDone = ExprSingleResult;
    +
    +	AnyNull = false;
    +
    +	/*
    +	 * If any of the clauses is FALSE, the AND result is FALSE regardless of
    +	 * the states of the rest of the clauses, so we can stop evaluating and
    +	 * return FALSE immediately.  If none are FALSE and one or more is NULL,
    +	 * we return NULL; otherwise we return TRUE.  This makes sense when you
    +	 * interpret NULL as "don't know", using the same sort of reasoning as for
    +	 * OR, above.
    +	 */
    +
    +	foreach(clause, clauses)
    +	{
    +		ExprState  *clausestate = (ExprState *) lfirst(clause);
    +		Datum		clause_value;
    +
    +		/*
    +		 * to check if all the values is false, then skip to evaluate some
    +		 * expressions
    +		 */
    +		skip = true;
    +
    +		clause_value = ExecEvalExpr(clausestate, econtext, isNull, NULL);
    +
    +		if(NULL == res)
    +		{
    +			res = DatumGetPointer(clause_value);
    +			for(i = 0; i < res->header.dim; i++)
    +			{
    +				if(res->header.isnull[i] || res->values[i])
    +				{
    +					skip = false;
    +					break;
    +				}
    +			}
    +		}
    +		else
    +		{
    +			next = DatumGetPointer(clause_value);
    +			for(i = 0; i < res->header.dim; i++)
    +			{
    +				res->header.isnull[i] = (res->header.isnull[i] ||next->header.isnull[i]);
    +				res->values[i] = (res->values[i] || next->values[i]);
    +				if(skip && (res->header.isnull[i] || res->values[i]))
    +					skip = false;
    +			}
    +		}
    +
    +		if(skip)
    +		{
    +			*isNull = false;
    +			return PointerGetDatum(res);
    +		}
    +	}
    +
    +	*isNull = false;
    +	return PointerGetDatum(res);
    +}
    +
    +/*
    + * Init the vectorized expressions
    + */
    +ExprState *
    +VExecInitExpr(Expr *node, PlanState *parent)
    +{
    +	ExprState *state = NULL;
    +	if(NULL == parent->vectorized)
    +		return NULL;
    +
    +	/*
    +	 * Because Var is the leaf node of the expression tree, it have to be
    +	 * refactored first, otherwise the all call stack should be refactored.
    +	 */
    +	switch (nodeTag(node))
    +	{
    +		case T_Var:
    +			state = (ExprState *) makeNode(ExprState);
    +			state->evalfunc = VExecEvalVar;
    +			break;
    +		case T_BoolExpr:
    +			{
    +				BoolExpr   *boolexpr = (BoolExpr *) node;
    +				BoolExprState *bstate = makeNode(BoolExprState);
    +
    +				switch (boolexpr->boolop)
    +				{
    +					case AND_EXPR:
    +						bstate->xprstate.evalfunc = (ExprStateEvalFunc) VExecEvalAnd;
    +						break;
    +					case OR_EXPR:
    +						bstate->xprstate.evalfunc = (ExprStateEvalFunc) VExecEvalOr;
    +						break;
    +					case NOT_EXPR:
    +						bstate->xprstate.evalfunc = (ExprStateEvalFunc) VExecEvalNot;
    +						break;
    +					default:
    +						elog(ERROR, "unrecognized boolop: %d",
    +							 (int) boolexpr->boolop);
    +						break;
    +				}
    +				bstate->args = (List *)
    +					ExecInitExpr((Expr *) boolexpr->args, parent);
    +				state = (ExprState *) bstate;
    +			}
    +			break;
    +		/*TODO: More and more expressions should be vectorized */
    +		default:
    +			break;
    --- End diff --
    
    It's better to think about how to fallback to original executor when we find any unsupported behavior to make sure the query could have right result. There are so many places that just error out or assert(false) which is not friendly.


---

[GitHub] incubator-hawq pull request #1351: HAWQ-1603.add new hook API

Posted by wengyanqing <gi...@git.apache.org>.
Github user wengyanqing commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/1351#discussion_r180626391
  
    --- Diff: contrib/vexecutor/execVQual.c ---
    @@ -123,3 +123,431 @@ VirtualNodeProc(ScanState* state,TupleTableSlot *slot){
         ExecStoreVirtualTuple(slot);
         return true;
     }
    +
    +/*
    + * get values from vectorized tuple slot
    + * copy from src/backend/executor/execQual.c
    + */
    +static Datum
    +VExecEvalScalarVar(ExprState *exprstate, ExprContext *econtext,
    +			bool *isNull, ExprDoneCond *isDone)
    +{
    +	Var		   *variable = (Var *) exprstate->expr;
    +	TupleTableSlot *slot;
    +	AttrNumber	attnum;
    +	TupleBatch tb;
    +
    +	if (isDone)
    +		*isDone = ExprSingleResult;
    +
    +	Assert(econtext->ecxt_scantuple != NULL || econtext->ecxt_innertuple != NULL || econtext->ecxt_outertuple != NULL);
    +	/*
    +	 * Get the input slot and attribute number we want
    +	 *
    +	 * The asserts check that references to system attributes only appear at
    +	 * the level of a relation scan; at higher levels, system attributes must
    +	 * be treated as ordinary variables (since we no longer have access to the
    +	 * original tuple).
    +	 */
    +	attnum = variable->varattno;
    +
    +	switch (variable->varno)
    +	{
    +		case INNER:				/* get the tuple from the inner node */
    +			slot = econtext->ecxt_innertuple;
    +			Assert(attnum > 0);
    +			break;
    +
    +		case OUTER:				/* get the tuple from the outer node */
    +			slot = econtext->ecxt_outertuple;
    +			Assert(attnum > 0);
    +			break;
    +
    +		default:				/* get the tuple from the relation being
    +								 * scanned */
    +			slot = econtext->ecxt_scantuple;
    +			break;
    +	}
    +
    +	/* isNull is a single value, it can not be used when data is vectorized */
    +	*isNull = false;
    +
    +	/* Fetch the value from the slot */
    +	tb = (TupleBatch )slot->PRIVATE_tb;
    +
    +	Assert(NULL != tb);
    +
    +	return PointerGetDatum(tb->datagroup[attnum]);
    +}
    +
    +
    +/*
    + * get values from vectorized tuple slot
    + * copy from src/backend/executor/execQual.c
    + */
    +static Datum
    +VExecEvalVar(ExprState *exprstate, ExprContext *econtext,
    +			bool *isNull, ExprDoneCond *isDone)
    +{
    +	Var		   *variable = (Var *) exprstate->expr;
    +	TupleTableSlot *slot;
    +	AttrNumber	attnum;
    +
    +	if (isDone)
    +		*isDone = ExprSingleResult;
    +
    +	Assert(econtext->ecxt_scantuple != NULL || econtext->ecxt_innertuple != NULL || econtext->ecxt_outertuple != NULL);
    +	/*
    +	 * Get the input slot and attribute number we want
    +	 *
    +	 * The asserts check that references to system attributes only appear at
    +	 * the level of a relation scan; at higher levels, system attributes must
    +	 * be treated as ordinary variables (since we no longer have access to the
    +	 * original tuple).
    +	 */
    +	attnum = variable->varattno;
    +
    +	switch (variable->varno)
    +	{
    +		case INNER:				/* get the tuple from the inner node */
    +			slot = econtext->ecxt_innertuple;
    +			Assert(attnum > 0);
    +			break;
    +
    +		case OUTER:				/* get the tuple from the outer node */
    +			slot = econtext->ecxt_outertuple;
    +			Assert(attnum > 0);
    +			break;
    +
    +		default:				/* get the tuple from the relation being
    +								 * scanned */
    +			slot = econtext->ecxt_scantuple;
    +			break;
    +	}
    +
    +	if (attnum != InvalidAttrNumber)
    +	{
    +		TupleBatch tb;
    +		/*
    +		 * Scalar variable case.
    +		 *
    +		 * If it's a user attribute, check validity (bogus system attnums will
    +		 * be caught inside slot_getattr).  What we have to check for here
    +		 * is the possibility of an attribute having been changed in type
    +		 * since the plan tree was created.  Ideally the plan would get
    +		 * invalidated and not re-used, but until that day arrives, we need
    +		 * defenses.  Fortunately it's sufficient to check once on the first
    +		 * time through.
    +		 *
    +		 * Note: we allow a reference to a dropped attribute.  slot_getattr
    +		 * will force a NULL result in such cases.
    +		 *
    +		 * Note: ideally we'd check typmod as well as typid, but that seems
    +		 * impractical at the moment: in many cases the tupdesc will have
    +		 * been generated by ExecTypeFromTL(), and that can't guarantee to
    +		 * generate an accurate typmod in all cases, because some expression
    +		 * node types don't carry typmod.
    +		 */
    +		if (attnum > 0)
    +		{
    +			TupleDesc	slot_tupdesc = slot->tts_tupleDescriptor;
    +			Form_pg_attribute attr;
    +
    +			if (attnum > slot_tupdesc->natts)	/* should never happen */
    +				elog(ERROR, "attribute number %d exceeds number of columns %d",
    +					 attnum, slot_tupdesc->natts);
    +
    +			attr = slot_tupdesc->attrs[attnum - 1];
    +
    +			/* can't check type if dropped, since atttypid is probably 0 */
    +			if (!attr->attisdropped)
    +			{
    +				if (variable->vartype != attr->atttypid)
    +					ereport(ERROR,
    +							(errmsg("attribute %d has wrong type", attnum),
    +							 errdetail("Table has type %s, but query expects %s.",
    +									   format_type_be(attr->atttypid),
    +									   format_type_be(variable->vartype))));
    +			}
    +		}
    +
    +		/* Skip the checking on future executions of node */
    +		exprstate->evalfunc = VExecEvalScalarVar;
    +
    +		/* isNull is a single value, it can not be used when data is vectorized */
    +		*isNull = false;
    +
    +		/* Fetch the value from the slot */
    +		tb = (TupleBatch )slot->PRIVATE_tb;
    +
    +		Assert(NULL != tb);
    +		return PointerGetDatum(tb->datagroup[attnum]);
    +	}
    +	else
    +	{
    +		/* NOT support so far */
    +		Assert(false);
    +	}
    +
    +	return PointerGetDatum(NULL);
    +}
    +
    +
    +/* ----------------------------------------------------------------
    + *		VExecEvalNot
    + *		VExecEvalOr
    + *		VExecEvalAnd
    + *
    + * copy from src/backend/executor/execQual.c
    + *
    + *		Evaluate boolean expressions, with appropriate short-circuiting.
    + *
    + *		The query planner reformulates clause expressions in the
    + *		qualification to conjunctive normal form.  If we ever get
    + *		an AND to evaluate, we can be sure that it's not a top-level
    + *		clause in the qualification, but appears lower (as a function
    + *		argument, for example), or in the target list.	Not that you
    + *		need to know this, mind you...
    + * ----------------------------------------------------------------
    + */
    +static Datum
    +VExecEvalNot(BoolExprState *notclause, ExprContext *econtext,
    +			bool *isNull, ExprDoneCond *isDone)
    +{
    +	ExprState  *clause = linitial(notclause->args);
    +	Datum		expr_value;
    +	vbool		*ret;
    +	int			i;
    +
    +	if (isDone)
    +		*isDone = ExprSingleResult;
    +
    +	expr_value = ExecEvalExpr(clause, econtext, isNull, NULL);
    +
    +	ret = (vbool*)DatumGetPointer(expr_value);
    +	for(i = 0; i < ret->header.dim; i++)
    +	{
    +		if(!ret->header.isnull[i])
    +			ret->values[i] = !ret->values[i];
    +	}
    +
    +	/*
    +	 * evaluation of 'not' is simple.. expr is false, then return 'true' and
    +	 * vice versa.
    +	 */
    +	return PointerGetDatum(ret);
    +}
    +
    +/* ----------------------------------------------------------------
    + *		ExecEvalOr
    + * ----------------------------------------------------------------
    + */
    +static Datum
    +VExecEvalOr(BoolExprState *orExpr, ExprContext *econtext,
    +		   bool *isNull, ExprDoneCond *isDone)
    +{
    +	List	   *clauses = orExpr->args;
    +	ListCell   *clause;
    +	bool		AnyNull;
    +	vbool	*res = NULL;
    +	vbool	*next = 	NULL;
    +	bool		skip = true;
    +	int 		i = 0;
    +
    +	if (isDone)
    +		*isDone = ExprSingleResult;
    +
    +	AnyNull = false;
    +
    +	/*
    +	 * If any of the clauses is TRUE, the OR result is TRUE regardless of the
    +	 * states of the rest of the clauses, so we can stop evaluating and return
    +	 * TRUE immediately.  If none are TRUE and one or more is NULL, we return
    +	 * NULL; otherwise we return FALSE.  This makes sense when you interpret
    +	 * NULL as "don't know": if we have a TRUE then the OR is TRUE even if we
    +	 * aren't sure about some of the other inputs. If all the known inputs are
    +	 * FALSE, but we have one or more "don't knows", then we have to report
    +	 * that we "don't know" what the OR's result should be --- perhaps one of
    +	 * the "don't knows" would have been TRUE if we'd known its value.  Only
    +	 * when all the inputs are known to be FALSE can we state confidently that
    +	 * the OR's result is FALSE.
    +	 */
    +	foreach(clause, clauses)
    +	{
    +		ExprState  *clausestate = (ExprState *) lfirst(clause);
    +		Datum		clause_value;
    +
    +		/*
    +		 * to check if all the values is true, then skip to evaluate some
    +		 * expressions
    +		 */
    +		skip = true;
    +
    +		clause_value = ExecEvalExpr(clausestate, econtext, isNull, NULL);
    +
    +		if(NULL == res)
    +		{
    +			res = DatumGetPointer(clause_value);
    +			for(i = 0; i < res->header.dim; i++)
    +			{
    +				if(res->header.isnull[i] || !res->values[i])
    +				{
    +					skip = false;
    +					break;
    +				}
    +			}
    +		}
    +		else
    +		{
    +			next = DatumGetPointer(clause_value);
    +			for(i = 0; i < res->header.dim; i++)
    +			{
    +				res->header.isnull[i] = (res->header.isnull[i] || next->header.isnull[i]);
    +				res->values[i] = (res->values[i] || next->values[i]);
    +				if(skip && (res->header.isnull[i] || !res->values[i]))
    +					skip = false;
    +			}
    +		}
    +
    +		if(skip)
    +		{
    +			*isNull = false;
    +			return PointerGetDatum(res);
    +		}
    +	}
    +
    +	*isNull = false;
    +	return PointerGetDatum(res);
    +}
    +
    +/* ----------------------------------------------------------------
    + *		ExecEvalAnd
    + * ----------------------------------------------------------------
    + */
    +static Datum
    +VExecEvalAnd(BoolExprState *andExpr, ExprContext *econtext,
    +			bool *isNull, ExprDoneCond *isDone)
    +{
    +	List	   *clauses = andExpr->args;
    +	ListCell   *clause;
    +	bool		AnyNull;
    +	vbool	*res = NULL;
    +	vbool	*next = 	NULL;
    +	bool		skip = true;
    +	int 		i = 0;
    +
    +	if (isDone)
    +		*isDone = ExprSingleResult;
    +
    +	AnyNull = false;
    +
    +	/*
    +	 * If any of the clauses is FALSE, the AND result is FALSE regardless of
    +	 * the states of the rest of the clauses, so we can stop evaluating and
    +	 * return FALSE immediately.  If none are FALSE and one or more is NULL,
    +	 * we return NULL; otherwise we return TRUE.  This makes sense when you
    +	 * interpret NULL as "don't know", using the same sort of reasoning as for
    +	 * OR, above.
    +	 */
    +
    +	foreach(clause, clauses)
    +	{
    +		ExprState  *clausestate = (ExprState *) lfirst(clause);
    +		Datum		clause_value;
    +
    +		/*
    +		 * to check if all the values is false, then skip to evaluate some
    +		 * expressions
    +		 */
    +		skip = true;
    +
    +		clause_value = ExecEvalExpr(clausestate, econtext, isNull, NULL);
    +
    +		if(NULL == res)
    +		{
    +			res = DatumGetPointer(clause_value);
    +			for(i = 0; i < res->header.dim; i++)
    +			{
    +				if(res->header.isnull[i] || res->values[i])
    +				{
    +					skip = false;
    +					break;
    +				}
    +			}
    +		}
    +		else
    +		{
    +			next = DatumGetPointer(clause_value);
    +			for(i = 0; i < res->header.dim; i++)
    +			{
    +				res->header.isnull[i] = (res->header.isnull[i] ||next->header.isnull[i]);
    +				res->values[i] = (res->values[i] || next->values[i]);
    +				if(skip && (res->header.isnull[i] || res->values[i]))
    +					skip = false;
    +			}
    +		}
    +
    +		if(skip)
    +		{
    +			*isNull = false;
    +			return PointerGetDatum(res);
    +		}
    +	}
    +
    +	*isNull = false;
    +	return PointerGetDatum(res);
    +}
    +
    +/*
    + * Init the vectorized expressions
    + */
    +ExprState *
    +VExecInitExpr(Expr *node, PlanState *parent)
    +{
    +	ExprState *state = NULL;
    +	if(NULL == parent->vectorized)
    +		return NULL;
    +
    +	/*
    +	 * Because Var is the leaf node of the expression tree, it have to be
    +	 * refactored first, otherwise the all call stack should be refactored.
    +	 */
    +	switch (nodeTag(node))
    +	{
    +		case T_Var:
    +			state = (ExprState *) makeNode(ExprState);
    +			state->evalfunc = VExecEvalVar;
    +			break;
    +		case T_BoolExpr:
    +			{
    +				BoolExpr   *boolexpr = (BoolExpr *) node;
    +				BoolExprState *bstate = makeNode(BoolExprState);
    +
    +				switch (boolexpr->boolop)
    +				{
    +					case AND_EXPR:
    +						bstate->xprstate.evalfunc = (ExprStateEvalFunc) VExecEvalAnd;
    +						break;
    +					case OR_EXPR:
    +						bstate->xprstate.evalfunc = (ExprStateEvalFunc) VExecEvalOr;
    +						break;
    +					case NOT_EXPR:
    +						bstate->xprstate.evalfunc = (ExprStateEvalFunc) VExecEvalNot;
    +						break;
    +					default:
    +						elog(ERROR, "unrecognized boolop: %d",
    +							 (int) boolexpr->boolop);
    +						break;
    +				}
    +				bstate->args = (List *)
    +					ExecInitExpr((Expr *) boolexpr->args, parent);
    +				state = (ExprState *) bstate;
    +			}
    +			break;
    +		/*TODO: More and more expressions should be vectorized */
    +		default:
    +			break;
    +	}
    +
    --- End diff --
    
    Does it need "state->expr = node;" here?


---

[GitHub] incubator-hawq pull request #1351: HAWQ-1603.add new hook API

Posted by zhangjackey <gi...@git.apache.org>.
Github user zhangjackey commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/1351#discussion_r181933267
  
    --- Diff: contrib/vexecutor/execVQual.c ---
    @@ -123,3 +123,431 @@ VirtualNodeProc(ScanState* state,TupleTableSlot *slot){
         ExecStoreVirtualTuple(slot);
         return true;
     }
    +
    +/*
    + * get values from vectorized tuple slot
    + * copy from src/backend/executor/execQual.c
    + */
    +static Datum
    +VExecEvalScalarVar(ExprState *exprstate, ExprContext *econtext,
    +			bool *isNull, ExprDoneCond *isDone)
    +{
    +	Var		   *variable = (Var *) exprstate->expr;
    +	TupleTableSlot *slot;
    +	AttrNumber	attnum;
    +	TupleBatch tb;
    +
    +	if (isDone)
    +		*isDone = ExprSingleResult;
    +
    +	Assert(econtext->ecxt_scantuple != NULL || econtext->ecxt_innertuple != NULL || econtext->ecxt_outertuple != NULL);
    +	/*
    +	 * Get the input slot and attribute number we want
    +	 *
    +	 * The asserts check that references to system attributes only appear at
    +	 * the level of a relation scan; at higher levels, system attributes must
    +	 * be treated as ordinary variables (since we no longer have access to the
    +	 * original tuple).
    +	 */
    +	attnum = variable->varattno;
    +
    +	switch (variable->varno)
    +	{
    +		case INNER:				/* get the tuple from the inner node */
    +			slot = econtext->ecxt_innertuple;
    +			Assert(attnum > 0);
    +			break;
    +
    +		case OUTER:				/* get the tuple from the outer node */
    +			slot = econtext->ecxt_outertuple;
    +			Assert(attnum > 0);
    +			break;
    +
    +		default:				/* get the tuple from the relation being
    +								 * scanned */
    +			slot = econtext->ecxt_scantuple;
    +			break;
    +	}
    +
    +	/* isNull is a single value, it can not be used when data is vectorized */
    +	*isNull = false;
    +
    +	/* Fetch the value from the slot */
    +	tb = (TupleBatch )slot->PRIVATE_tb;
    +
    +	Assert(NULL != tb);
    +
    +	return PointerGetDatum(tb->datagroup[attnum]);
    +}
    +
    +
    +/*
    + * get values from vectorized tuple slot
    + * copy from src/backend/executor/execQual.c
    + */
    +static Datum
    +VExecEvalVar(ExprState *exprstate, ExprContext *econtext,
    +			bool *isNull, ExprDoneCond *isDone)
    +{
    +	Var		   *variable = (Var *) exprstate->expr;
    +	TupleTableSlot *slot;
    +	AttrNumber	attnum;
    +
    +	if (isDone)
    +		*isDone = ExprSingleResult;
    +
    +	Assert(econtext->ecxt_scantuple != NULL || econtext->ecxt_innertuple != NULL || econtext->ecxt_outertuple != NULL);
    +	/*
    +	 * Get the input slot and attribute number we want
    +	 *
    +	 * The asserts check that references to system attributes only appear at
    +	 * the level of a relation scan; at higher levels, system attributes must
    +	 * be treated as ordinary variables (since we no longer have access to the
    +	 * original tuple).
    +	 */
    +	attnum = variable->varattno;
    +
    +	switch (variable->varno)
    +	{
    +		case INNER:				/* get the tuple from the inner node */
    +			slot = econtext->ecxt_innertuple;
    +			Assert(attnum > 0);
    +			break;
    +
    +		case OUTER:				/* get the tuple from the outer node */
    +			slot = econtext->ecxt_outertuple;
    +			Assert(attnum > 0);
    +			break;
    +
    +		default:				/* get the tuple from the relation being
    +								 * scanned */
    +			slot = econtext->ecxt_scantuple;
    +			break;
    +	}
    +
    +	if (attnum != InvalidAttrNumber)
    +	{
    +		TupleBatch tb;
    +		/*
    +		 * Scalar variable case.
    +		 *
    +		 * If it's a user attribute, check validity (bogus system attnums will
    +		 * be caught inside slot_getattr).  What we have to check for here
    +		 * is the possibility of an attribute having been changed in type
    +		 * since the plan tree was created.  Ideally the plan would get
    +		 * invalidated and not re-used, but until that day arrives, we need
    +		 * defenses.  Fortunately it's sufficient to check once on the first
    +		 * time through.
    +		 *
    +		 * Note: we allow a reference to a dropped attribute.  slot_getattr
    +		 * will force a NULL result in such cases.
    +		 *
    +		 * Note: ideally we'd check typmod as well as typid, but that seems
    +		 * impractical at the moment: in many cases the tupdesc will have
    +		 * been generated by ExecTypeFromTL(), and that can't guarantee to
    +		 * generate an accurate typmod in all cases, because some expression
    +		 * node types don't carry typmod.
    +		 */
    +		if (attnum > 0)
    +		{
    +			TupleDesc	slot_tupdesc = slot->tts_tupleDescriptor;
    +			Form_pg_attribute attr;
    +
    +			if (attnum > slot_tupdesc->natts)	/* should never happen */
    +				elog(ERROR, "attribute number %d exceeds number of columns %d",
    +					 attnum, slot_tupdesc->natts);
    +
    +			attr = slot_tupdesc->attrs[attnum - 1];
    +
    +			/* can't check type if dropped, since atttypid is probably 0 */
    +			if (!attr->attisdropped)
    +			{
    +				if (variable->vartype != attr->atttypid)
    +					ereport(ERROR,
    +							(errmsg("attribute %d has wrong type", attnum),
    +							 errdetail("Table has type %s, but query expects %s.",
    +									   format_type_be(attr->atttypid),
    +									   format_type_be(variable->vartype))));
    +			}
    +		}
    +
    +		/* Skip the checking on future executions of node */
    +		exprstate->evalfunc = VExecEvalScalarVar;
    +
    +		/* isNull is a single value, it can not be used when data is vectorized */
    +		*isNull = false;
    +
    +		/* Fetch the value from the slot */
    +		tb = (TupleBatch )slot->PRIVATE_tb;
    +
    +		Assert(NULL != tb);
    +		return PointerGetDatum(tb->datagroup[attnum]);
    +	}
    +	else
    +	{
    +		/* NOT support so far */
    +		Assert(false);
    +	}
    +
    +	return PointerGetDatum(NULL);
    +}
    +
    +
    +/* ----------------------------------------------------------------
    + *		VExecEvalNot
    + *		VExecEvalOr
    + *		VExecEvalAnd
    + *
    + * copy from src/backend/executor/execQual.c
    + *
    + *		Evaluate boolean expressions, with appropriate short-circuiting.
    + *
    + *		The query planner reformulates clause expressions in the
    + *		qualification to conjunctive normal form.  If we ever get
    + *		an AND to evaluate, we can be sure that it's not a top-level
    + *		clause in the qualification, but appears lower (as a function
    + *		argument, for example), or in the target list.	Not that you
    + *		need to know this, mind you...
    + * ----------------------------------------------------------------
    + */
    +static Datum
    +VExecEvalNot(BoolExprState *notclause, ExprContext *econtext,
    +			bool *isNull, ExprDoneCond *isDone)
    +{
    +	ExprState  *clause = linitial(notclause->args);
    +	Datum		expr_value;
    +	vbool		*ret;
    +	int			i;
    +
    +	if (isDone)
    +		*isDone = ExprSingleResult;
    +
    +	expr_value = ExecEvalExpr(clause, econtext, isNull, NULL);
    +
    +	ret = (vbool*)DatumGetPointer(expr_value);
    +	for(i = 0; i < ret->header.dim; i++)
    +	{
    +		if(!ret->header.isnull[i])
    +			ret->values[i] = !ret->values[i];
    +	}
    +
    +	/*
    +	 * evaluation of 'not' is simple.. expr is false, then return 'true' and
    +	 * vice versa.
    +	 */
    +	return PointerGetDatum(ret);
    +}
    +
    +/* ----------------------------------------------------------------
    + *		ExecEvalOr
    + * ----------------------------------------------------------------
    + */
    +static Datum
    +VExecEvalOr(BoolExprState *orExpr, ExprContext *econtext,
    +		   bool *isNull, ExprDoneCond *isDone)
    +{
    +	List	   *clauses = orExpr->args;
    +	ListCell   *clause;
    +	bool		AnyNull;
    +	vbool	*res = NULL;
    +	vbool	*next = 	NULL;
    +	bool		skip = true;
    +	int 		i = 0;
    +
    +	if (isDone)
    +		*isDone = ExprSingleResult;
    +
    +	AnyNull = false;
    +
    +	/*
    +	 * If any of the clauses is TRUE, the OR result is TRUE regardless of the
    +	 * states of the rest of the clauses, so we can stop evaluating and return
    +	 * TRUE immediately.  If none are TRUE and one or more is NULL, we return
    +	 * NULL; otherwise we return FALSE.  This makes sense when you interpret
    +	 * NULL as "don't know": if we have a TRUE then the OR is TRUE even if we
    +	 * aren't sure about some of the other inputs. If all the known inputs are
    +	 * FALSE, but we have one or more "don't knows", then we have to report
    +	 * that we "don't know" what the OR's result should be --- perhaps one of
    +	 * the "don't knows" would have been TRUE if we'd known its value.  Only
    +	 * when all the inputs are known to be FALSE can we state confidently that
    +	 * the OR's result is FALSE.
    +	 */
    +	foreach(clause, clauses)
    +	{
    +		ExprState  *clausestate = (ExprState *) lfirst(clause);
    +		Datum		clause_value;
    +
    +		/*
    +		 * to check if all the values is true, then skip to evaluate some
    +		 * expressions
    +		 */
    +		skip = true;
    +
    +		clause_value = ExecEvalExpr(clausestate, econtext, isNull, NULL);
    +
    +		if(NULL == res)
    +		{
    +			res = DatumGetPointer(clause_value);
    +			for(i = 0; i < res->header.dim; i++)
    +			{
    +				if(res->header.isnull[i] || !res->values[i])
    +				{
    +					skip = false;
    +					break;
    +				}
    +			}
    +		}
    +		else
    +		{
    +			next = DatumGetPointer(clause_value);
    +			for(i = 0; i < res->header.dim; i++)
    +			{
    +				res->header.isnull[i] = (res->header.isnull[i] || next->header.isnull[i]);
    +				res->values[i] = (res->values[i] || next->values[i]);
    +				if(skip && (res->header.isnull[i] || !res->values[i]))
    +					skip = false;
    +			}
    +		}
    +
    +		if(skip)
    +		{
    +			*isNull = false;
    +			return PointerGetDatum(res);
    +		}
    +	}
    +
    +	*isNull = false;
    +	return PointerGetDatum(res);
    +}
    +
    +/* ----------------------------------------------------------------
    + *		ExecEvalAnd
    + * ----------------------------------------------------------------
    + */
    +static Datum
    +VExecEvalAnd(BoolExprState *andExpr, ExprContext *econtext,
    +			bool *isNull, ExprDoneCond *isDone)
    +{
    +	List	   *clauses = andExpr->args;
    +	ListCell   *clause;
    +	bool		AnyNull;
    +	vbool	*res = NULL;
    +	vbool	*next = 	NULL;
    +	bool		skip = true;
    +	int 		i = 0;
    +
    +	if (isDone)
    +		*isDone = ExprSingleResult;
    +
    +	AnyNull = false;
    +
    +	/*
    +	 * If any of the clauses is FALSE, the AND result is FALSE regardless of
    +	 * the states of the rest of the clauses, so we can stop evaluating and
    +	 * return FALSE immediately.  If none are FALSE and one or more is NULL,
    +	 * we return NULL; otherwise we return TRUE.  This makes sense when you
    +	 * interpret NULL as "don't know", using the same sort of reasoning as for
    +	 * OR, above.
    +	 */
    +
    +	foreach(clause, clauses)
    +	{
    +		ExprState  *clausestate = (ExprState *) lfirst(clause);
    +		Datum		clause_value;
    +
    +		/*
    +		 * to check if all the values is false, then skip to evaluate some
    +		 * expressions
    +		 */
    +		skip = true;
    +
    +		clause_value = ExecEvalExpr(clausestate, econtext, isNull, NULL);
    +
    +		if(NULL == res)
    +		{
    +			res = DatumGetPointer(clause_value);
    +			for(i = 0; i < res->header.dim; i++)
    +			{
    +				if(res->header.isnull[i] || res->values[i])
    +				{
    +					skip = false;
    +					break;
    +				}
    +			}
    +		}
    +		else
    +		{
    +			next = DatumGetPointer(clause_value);
    +			for(i = 0; i < res->header.dim; i++)
    +			{
    +				res->header.isnull[i] = (res->header.isnull[i] ||next->header.isnull[i]);
    +				res->values[i] = (res->values[i] || next->values[i]);
    +				if(skip && (res->header.isnull[i] || res->values[i]))
    +					skip = false;
    +			}
    +		}
    +
    +		if(skip)
    +		{
    +			*isNull = false;
    +			return PointerGetDatum(res);
    +		}
    +	}
    +
    +	*isNull = false;
    +	return PointerGetDatum(res);
    +}
    +
    +/*
    + * Init the vectorized expressions
    + */
    +ExprState *
    +VExecInitExpr(Expr *node, PlanState *parent)
    +{
    +	ExprState *state = NULL;
    +	if(NULL == parent->vectorized)
    +		return NULL;
    +
    +	/*
    +	 * Because Var is the leaf node of the expression tree, it have to be
    +	 * refactored first, otherwise the all call stack should be refactored.
    +	 */
    +	switch (nodeTag(node))
    +	{
    +		case T_Var:
    +			state = (ExprState *) makeNode(ExprState);
    +			state->evalfunc = VExecEvalVar;
    +			break;
    +		case T_BoolExpr:
    +			{
    +				BoolExpr   *boolexpr = (BoolExpr *) node;
    +				BoolExprState *bstate = makeNode(BoolExprState);
    +
    +				switch (boolexpr->boolop)
    +				{
    +					case AND_EXPR:
    +						bstate->xprstate.evalfunc = (ExprStateEvalFunc) VExecEvalAnd;
    +						break;
    +					case OR_EXPR:
    +						bstate->xprstate.evalfunc = (ExprStateEvalFunc) VExecEvalOr;
    +						break;
    +					case NOT_EXPR:
    +						bstate->xprstate.evalfunc = (ExprStateEvalFunc) VExecEvalNot;
    +						break;
    +					default:
    +						elog(ERROR, "unrecognized boolop: %d",
    +							 (int) boolexpr->boolop);
    +						break;
    +				}
    +				bstate->args = (List *)
    +					ExecInitExpr((Expr *) boolexpr->args, parent);
    +				state = (ExprState *) bstate;
    +			}
    +			break;
    +		/*TODO: More and more expressions should be vectorized */
    +		default:
    +			break;
    +	}
    +
    --- End diff --
    
    yes, add in next commit.


---