Skip to content

Commit

Permalink
[Chore](vectorized) remove all isVectorized (apache#21076)
Browse files Browse the repository at this point in the history
isVectorized is always true now
  • Loading branch information
BiteTheDDDDt authored Jun 25, 2023
1 parent 58b3e5e commit 0122aa7
Show file tree
Hide file tree
Showing 27 changed files with 93 additions and 359 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@
import org.apache.doris.catalog.FunctionSet;
import org.apache.doris.catalog.Type;
import org.apache.doris.common.AnalysisException;
import org.apache.doris.common.util.VectorizedUtil;
import org.apache.doris.planner.DataPartition;
import org.apache.doris.qe.ConnectContext;
import org.apache.doris.thrift.TPartitionType;
Expand Down Expand Up @@ -257,7 +256,6 @@ public static boolean isSetUsingSetForDistinct(List<FunctionCallExpr> distinctAg
// for vectorized execution, we force it to using hash set to execution
if (distinctAggExprs.size() == 1
&& distinctAggExprs.get(0).getFnParams().isDistinct()
&& VectorizedUtil.isVectorized()
&& ConnectContext.get().getSessionVariable().enableSingleDistinctColumnOpt()) {
isSetUsingSetForDistinct = true;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,6 @@
import org.apache.doris.catalog.Type;
import org.apache.doris.common.AnalysisException;
import org.apache.doris.common.TreeNode;
import org.apache.doris.common.util.VectorizedUtil;
import org.apache.doris.thrift.TExprNode;

import com.google.common.base.Joiner;
Expand Down Expand Up @@ -263,18 +262,6 @@ private static boolean isNTileFn(Function fn) {
* that we want to equal.
*/
public static Expr rewrite(AnalyticExpr analyticExpr) {
Function fn = analyticExpr.getFnCall().getFn();
// TODO(zc)
// if (AnalyticExpr.isPercentRankFn(fn)) {
// return createPercentRank(analyticExpr);
// } else if (AnalyticExpr.isCumeDistFn(fn)) {
// return createCumeDist(analyticExpr);
// } else if (AnalyticExpr.isNtileFn(fn)) {
// return createNtile(analyticExpr);
// }
if (isNTileFn(fn) && !VectorizedUtil.isVectorized()) {
return createNTile(analyticExpr);
}
return null;
}

Expand Down Expand Up @@ -573,19 +560,6 @@ public void analyzeImpl(Analyzer analyzer) throws AnalysisException {

standardize(analyzer);

// But in Vectorized mode, after calculate a window, will be call reset() to reset state,
// And then restarted calculate next new window;
if (!VectorizedUtil.isVectorized()) {
// min/max is not currently supported on sliding windows (i.e. start bound is not
// unbounded).
if (window != null && isMinMax(fn)
&& window.getLeftBoundary().getType() != BoundaryType.UNBOUNDED_PRECEDING) {
throw new AnalysisException(
"'" + getFnCall().toSql() + "' is only supported with an "
+ "UNBOUNDED PRECEDING start bound.");
}
}

setChildren();
}

Expand Down Expand Up @@ -752,28 +726,6 @@ private void standardize(Analyzer analyzer) throws AnalysisException {
window.getLeftBoundary());
fnCall = new FunctionCallExpr(new FunctionName(LASTVALUE),
getFnCall().getParams());
} else {
//TODO: Now we don't want to first_value to rewrite in vectorized mode;
//if have to rewrite in future, could exec this rule;
if (!VectorizedUtil.isVectorized()) {
List<Expr> paramExprs = Expr.cloneList(getFnCall().getParams().exprs());

if (window.getRightBoundary().getType() == BoundaryType.PRECEDING) {
// The number of rows preceding for the end bound determines the number of
// rows at the beginning of each partition that should have a NULL value.
paramExprs.add(window.getRightBoundary().getExpr());
} else {
// -1 indicates that no NULL values are inserted even though we set the end
// bound to the start bound (which is PRECEDING) below; this is different from
// the default behavior of windows with an end bound PRECEDING.
paramExprs.add(new IntLiteral(-1, Type.BIGINT));
}

window = new AnalyticWindow(window.getType(),
new Boundary(BoundaryType.UNBOUNDED_PRECEDING, null),
window.getLeftBoundary());
fnCall = new FunctionCallExpr("FIRST_VALUE_REWRITE", new FunctionParams(paramExprs));
}
}

fnCall.setIsAnalyticFnCall(true);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,6 @@
import org.apache.doris.common.IdGenerator;
import org.apache.doris.common.Pair;
import org.apache.doris.common.util.TimeUtils;
import org.apache.doris.common.util.VectorizedUtil;
import org.apache.doris.planner.PlanNode;
import org.apache.doris.planner.RuntimeFilter;
import org.apache.doris.qe.ConnectContext;
Expand Down Expand Up @@ -955,11 +954,7 @@ public SlotDescriptor registerColumnRef(TableName tblName, String colName) throw
result = globalState.descTbl.addSlotDescriptor(d);
result.setColumn(col);
boolean isNullable;
if (VectorizedUtil.isVectorized()) {
isNullable = col.isAllowNull();
} else {
isNullable = col.isAllowNull() || isOuterJoined(d.getId());
}
isNullable = col.isAllowNull();
result.setIsNullable(isNullable);

slotRefMap.put(key, result);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@


import org.apache.doris.common.AnalysisException;
import org.apache.doris.common.util.VectorizedUtil;
import org.apache.doris.qe.ConnectContext;
import org.apache.doris.thrift.TExprNode;
import org.apache.doris.thrift.TRuntimeFilterType;
Expand Down Expand Up @@ -81,10 +80,6 @@ public void analyzeImpl(Analyzer analyzer) throws AnalysisException {
throw new AnalysisException(
"Please enable the session variable 'enable_projection' through `set enable_projection = true;`");
}

if (!VectorizedUtil.isVectorized()) {
throw new AnalysisException("In bitmap syntax is currently only supported in the vectorization engine.");
}
}

@Override
Expand Down
4 changes: 1 addition & 3 deletions fe/fe-core/src/main/java/org/apache/doris/analysis/Expr.java
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,6 @@
import org.apache.doris.common.Config;
import org.apache.doris.common.TreeNode;
import org.apache.doris.common.io.Writable;
import org.apache.doris.common.util.VectorizedUtil;
import org.apache.doris.qe.ConnectContext;
import org.apache.doris.rewrite.mvrewrite.MVExprEquivalent;
import org.apache.doris.statistics.ExprStats;
Expand Down Expand Up @@ -1914,8 +1913,7 @@ protected Function getBuiltinFunction(String name, Type[] argTypes, Function.Com

protected Function getTableFunction(String name, Type[] argTypes, Function.CompareMode mode) {
FunctionName fnName = new FunctionName(name);
Function searchDesc = new Function(fnName, Arrays.asList(argTypes), Type.INVALID, false,
VectorizedUtil.isVectorized());
Function searchDesc = new Function(fnName, Arrays.asList(argTypes), Type.INVALID, false);
Function f = Env.getCurrentEnv().getTableFunction(searchDesc, mode);
return f;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,6 @@
import org.apache.doris.common.AnalysisException;
import org.apache.doris.common.ErrorCode;
import org.apache.doris.common.ErrorReport;
import org.apache.doris.common.util.VectorizedUtil;
import org.apache.doris.mysql.privilege.PrivPredicate;
import org.apache.doris.qe.ConnectContext;
import org.apache.doris.thrift.TExprNode;
Expand Down Expand Up @@ -336,11 +335,8 @@ public FunctionCallExpr(
this(fnName, params, false);
this.orderByElements = orderByElements;
if (!orderByElements.isEmpty()) {
if (!VectorizedUtil.isVectorized()) {
throw new AnalysisException(
"ORDER BY for arguments only support in vec exec engine");
} else if (!AggregateFunction.SUPPORT_ORDER_BY_AGGREGATE_FUNCTION_NAME_SET.contains(
fnName.getFunction().toLowerCase())) {
if (!AggregateFunction.SUPPORT_ORDER_BY_AGGREGATE_FUNCTION_NAME_SET
.contains(fnName.getFunction().toLowerCase())) {
throw new AnalysisException(
"ORDER BY not support for the function:" + fnName.getFunction().toLowerCase());
}
Expand Down Expand Up @@ -1274,9 +1270,6 @@ public void analyzeImpl(Analyzer analyzer) throws AnalysisException {
}
// Prevent the cast type in vector exec engine
Type type = getChild(0).type;
if (!VectorizedUtil.isVectorized()) {
type = getChild(0).type.getMaxResolutionType();
}
fn = getBuiltinFunction(fnName.getFunction(), new Type[] { type },
Function.CompareMode.IS_NONSTRICT_SUPERTYPE_OF);
} else if (fnName.getFunction().equalsIgnoreCase("count_distinct")) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,6 @@
import org.apache.doris.common.ErrorCode;
import org.apache.doris.common.ErrorReport;
import org.apache.doris.common.UserException;
import org.apache.doris.common.util.VectorizedUtil;
import org.apache.doris.rewrite.ExprRewriter;
import org.apache.doris.thrift.TQueryOptions;

Expand Down Expand Up @@ -194,10 +193,6 @@ public void analyze(Analyzer analyzer) throws AnalysisException, UserException {
}

private void analyzeLimit(Analyzer analyzer) throws AnalysisException {
if (!VectorizedUtil.isVectorized() && limitElement.getOffset() > 0 && !hasOrderByClause()) {
throw new AnalysisException("OFFSET requires an ORDER BY clause: "
+ limitElement.toSql().trim());
}
limitElement.analyze(analyzer);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,6 @@
import org.apache.doris.common.AnalysisException;
import org.apache.doris.common.TableAliasGenerator;
import org.apache.doris.common.UserException;
import org.apache.doris.common.util.VectorizedUtil;
import org.apache.doris.policy.RowPolicy;
import org.apache.doris.qe.ConnectContext;

Expand Down Expand Up @@ -864,8 +863,7 @@ private static boolean mergeExpr(SelectStmt stmt, Expr expr,
// For the case of a NOT IN with an eq join conjunct, replace the join
// conjunct with a conjunct that uses the null-matching eq operator.
if (expr instanceof InPredicate && markTuple == null) {
joinOp = VectorizedUtil.isVectorized()
? JoinOperator.NULL_AWARE_LEFT_ANTI_JOIN : JoinOperator.LEFT_ANTI_JOIN;
joinOp = JoinOperator.NULL_AWARE_LEFT_ANTI_JOIN;
List<TupleId> tIds = Lists.newArrayList();
joinConjunct.getIds(tIds, null);
if (tIds.size() <= 1 || !tIds.contains(inlineView.getDesc().getId())) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -136,7 +136,6 @@ private static AliasFunction createBuiltin(String name, ArrayList<Type> argTypes
aliasFunction.setUserVisible(userVisible);
aliasFunction.originFunction = originFunction;
aliasFunction.parameters = parameters;
aliasFunction.vectorized = isVectorized;
return aliasFunction;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -757,10 +757,6 @@ public List<Comparable> getInfo(boolean isVerbose) {
return row;
}

boolean isVectorized() {
return vectorized;
}

public void setNullableMode(NullableMode nullableMode) {
this.nullableMode = nullableMode;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -372,7 +372,7 @@ public Function specializeTemplateFunction(Function templateFunction, Function r
if (templateFunction instanceof ScalarFunction) {
ScalarFunction f = (ScalarFunction) templateFunction;
specializedFunction = new ScalarFunction(f.getFunctionName(), newArgTypes, newRetType.get(0), f.hasVarArgs(),
f.getSymbolName(), f.getBinaryType(), f.isUserVisible(), f.isVectorized(), f.getNullableMode());
f.getSymbolName(), f.getBinaryType(), f.isUserVisible(), true, f.getNullableMode());
} else {
throw new TypeException(templateFunction
+ " is not support for template since it's not a ScalarFunction");
Expand Down Expand Up @@ -417,7 +417,7 @@ public Function resolveInferenceFunction(Function inferenceFunction, Function re
if (newRetType != null && inferenceFunction instanceof ScalarFunction) {
ScalarFunction f = (ScalarFunction) inferenceFunction;
return new ScalarFunction(f.getFunctionName(), Lists.newArrayList(newTypes), newRetType, f.hasVarArgs(),
f.getSymbolName(), f.getBinaryType(), f.isUserVisible(), f.isVectorized(), f.getNullableMode());
f.getSymbolName(), f.getBinaryType(), f.isUserVisible(), true, f.getNullableMode());
}
return null;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,20 +20,6 @@
import org.apache.doris.qe.ConnectContext;

public class VectorizedUtil {
/**
* 1. Return false if there is no current connection (Rule1 to be changed)
* 2. Returns the vectorized switch value of the query 'globalState.enableQueryVec'
* 3. If it is not currently a query, return the vectorized switch value of the session 'enableVectorizedEngine'
* @return true: vec. false: non-vec
*/
public static boolean isVectorized() {
ConnectContext connectContext = ConnectContext.get();
if (connectContext == null) {
return false;
}
return true;
}

public static boolean isPipeline() {
ConnectContext connectContext = ConnectContext.get();
if (connectContext == null) {
Expand Down
Loading

0 comments on commit 0122aa7

Please sign in to comment.