@Override public AbstractLogicalExpression cloneExpression() { VariableReferenceExpression varRef = new VariableReferenceExpression(variable); varRef.setSourceLocation(sourceLoc); return varRef; }
/** * Finds a variable assigned to a given expression and returns a new {@link VariableReferenceExpression} * referring to this variable. * @param assignVarList list of variables * @param assignExprList list of expressions assigned to those variables * @param searchExpr expression to search for * @return said value, {@code null} if a variable is not found */ public static VariableReferenceExpression findAssignedVariable(List<LogicalVariable> assignVarList, List<Mutable<ILogicalExpression>> assignExprList, ILogicalExpression searchExpr) { for (int i = 0, n = assignExprList.size(); i < n; i++) { ILogicalExpression expr = assignExprList.get(i).getValue(); if (expr.equals(searchExpr)) { VariableReferenceExpression result = new VariableReferenceExpression(assignVarList.get(i)); result.setSourceLocation(expr.getSourceLocation()); return result; } } return null; } }
private static AbstractFunctionCallExpression createPointExpression(List<LogicalVariable> pointVars, SourceLocation sourceLoc) { List<Mutable<ILogicalExpression>> expressions = new ArrayList<>(); AbstractFunctionCallExpression createPointExpr1 = new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.CREATE_POINT)); createPointExpr1.setSourceLocation(sourceLoc); VariableReferenceExpression pointVarRef0 = new VariableReferenceExpression(pointVars.get(0)); pointVarRef0.setSourceLocation(sourceLoc); expressions.add(new MutableObject<ILogicalExpression>(pointVarRef0)); VariableReferenceExpression pointVarRef1 = new VariableReferenceExpression(pointVars.get(1)); pointVarRef1.setSourceLocation(sourceLoc); expressions.add(new MutableObject<ILogicalExpression>(pointVarRef1)); createPointExpr1.getArguments().addAll(expressions); return createPointExpr1; }
public static void prepareVarAndExpression(List<String> field, LogicalVariable resVar, List<LogicalVariable> vars, List<Mutable<ILogicalExpression>> assignExpressions, List<Mutable<ILogicalExpression>> varRefs, IVariableContext context, SourceLocation sourceLoc) { VariableReferenceExpression dummyVarRef = new VariableReferenceExpression(DUMMY_VAR); dummyVarRef.setSourceLocation(sourceLoc); ScalarFunctionCallExpression f = createFieldAccessExpression(dummyVarRef, field, sourceLoc); f.substituteVar(DUMMY_VAR, resVar); assignExpressions.add(new MutableObject<ILogicalExpression>(f)); LogicalVariable v = context.newVar(); vars.add(v); if (varRefs != null) { VariableReferenceExpression vRef = new VariableReferenceExpression(v); vRef.setSourceLocation(sourceLoc); varRefs.add(new MutableObject<ILogicalExpression>(vRef)); } }
private void setPrimaryFilterVar(UnnestMapOperator primaryOp, LogicalVariable minFilterVar, LogicalVariable maxFilterVar, IOptimizationContext context) throws AlgebricksException { primaryOp.setMinFilterVars(Collections.singletonList(minFilterVar)); primaryOp.setMaxFilterVars(Collections.singletonList(maxFilterVar)); VariableReferenceExpression minFilterVarRef = new VariableReferenceExpression(minFilterVar); minFilterVarRef.setSourceLocation(primaryOp.getSourceLocation()); VariableReferenceExpression maxFilterVarRef = new VariableReferenceExpression(maxFilterVar); maxFilterVarRef.setSourceLocation(primaryOp.getSourceLocation()); List<Mutable<ILogicalExpression>> indexFilterExpression = Arrays.asList(new MutableObject<>(minFilterVarRef), new MutableObject<>(maxFilterVarRef)); primaryOp.setAdditionalFilteringExpressions(indexFilterExpression); context.computeAndSetTypeEnvironmentForOperator(primaryOp); }
private ILogicalExpression createEqExpr(LogicalVariable lhsVar, IAObject rhsValue, SourceLocation sourceLoc) throws CompilationException { VariableReferenceExpression lhsExpr = new VariableReferenceExpression(lhsVar); lhsExpr.setSourceLocation(sourceLoc); ILogicalExpression rhsExpr = translateConstantValue(rhsValue, sourceLoc); AbstractFunctionCallExpression opExpr = createFunctionCallExpressionForBuiltinOperator(OperatorType.EQ, sourceLoc); opExpr.getArguments().add(new MutableObject<>(lhsExpr)); opExpr.getArguments().add(new MutableObject<>(rhsExpr)); return opExpr; }
public static void resetLOJMissingPlaceholderVarInGroupByOp(AccessMethodAnalysisContext analysisCtx, LogicalVariable newMissingPlaceholderVaraible, IOptimizationContext context) throws AlgebricksException { //reset the missing placeholder variable in groupby operator ScalarFunctionCallExpression isMissingFuncExpr = analysisCtx.getLOJIsMissingFuncInGroupBy(); isMissingFuncExpr.getArguments().clear(); VariableReferenceExpression newMissingVarRef = new VariableReferenceExpression(newMissingPlaceholderVaraible); newMissingVarRef.setSourceLocation(isMissingFuncExpr.getSourceLocation()); isMissingFuncExpr.getArguments().add(new MutableObject<ILogicalExpression>(newMissingVarRef)); //recompute type environment. OperatorPropertiesUtil.typeOpRec(analysisCtx.getLOJGroupbyOpRef(), context); }
private boolean testAndModifyRedundantOp(AssignOperator access, AbstractLogicalOperator op2) { if (op2.getOperatorTag() != LogicalOperatorTag.ASSIGN) { return false; } AssignOperator a2 = (AssignOperator) op2; ILogicalExpression accessExpr0 = getFirstExpr(access); if (accessExpr0.equals(getFirstExpr(a2))) { VariableReferenceExpression varRef = new VariableReferenceExpression(a2.getVariables().get(0)); varRef.setSourceLocation(accessExpr0.getSourceLocation()); access.getExpressions().get(0).setValue(varRef); return true; } else { return false; } }
private void removeGroupByAndOuterUnnest(Mutable<ILogicalOperator> opRef, IOptimizationContext context, LeftOuterUnnestOperator outerUnnest, GroupByOperator gbyOperator, LeftOuterJoinOperator lojOperator, LogicalVariable listifyVar) throws AlgebricksException { List<LogicalVariable> lhs = new ArrayList<>(); List<Mutable<ILogicalExpression>> rhs = new ArrayList<>(); lhs.add(outerUnnest.getVariable()); VariableReferenceExpression listifyVarRef = new VariableReferenceExpression(listifyVar); listifyVarRef.setSourceLocation(gbyOperator.getSourceLocation()); rhs.add(new MutableObject<ILogicalExpression>(listifyVarRef)); List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> gbyList = gbyOperator.getGroupByList(); for (Pair<LogicalVariable, Mutable<ILogicalExpression>> gbyPair : gbyList) { lhs.add(gbyPair.first); rhs.add(gbyPair.second); } AssignOperator assignOp = new AssignOperator(lhs, rhs); assignOp.setSourceLocation(outerUnnest.getSourceLocation()); assignOp.getInputs().add(new MutableObject<ILogicalOperator>(lojOperator)); context.computeAndSetTypeEnvironmentForOperator(assignOp); opRef.setValue(assignOp); }
private boolean extractFirstArg(AbstractFunctionCallExpression fce, ILogicalOperator op, IOptimizationContext context) throws AlgebricksException { ILogicalExpression firstArg = fce.getArguments().get(0).getValue(); if (firstArg.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) { return false; } SourceLocation sourceLoc = firstArg.getSourceLocation(); LogicalVariable var1 = context.newVar(); AssignOperator assignOp = new AssignOperator(new ArrayList<>(Collections.singletonList(var1)), new ArrayList<>(Collections.singletonList(new MutableObject<>(firstArg)))); assignOp.setSourceLocation(sourceLoc); VariableReferenceExpression var1Ref = new VariableReferenceExpression(var1); var1Ref.setSourceLocation(sourceLoc); fce.getArguments().get(0).setValue(var1Ref); assignOp.getInputs().add(new MutableObject<>(op.getInputs().get(0).getValue())); op.getInputs().get(0).setValue(assignOp); context.computeAndSetTypeEnvironmentForOperator(assignOp); return true; }
@Override public boolean transform(Mutable<ILogicalExpression> exprRef) { ILogicalExpression expr = exprRef.getValue(); if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) { return false; } AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr; if (!funcExpr.getFunctionIdentifier().equals(BuiltinFunctions.META_KEY)) { return false; } // function is meta key access for (int i = 0; i < metaKeyAccessExpressions.size(); i++) { if (metaKeyAccessExpressions.get(i).equals(funcExpr)) { VariableReferenceExpression varRef = new VariableReferenceExpression(keyVars.get(i)); varRef.setSourceLocation(expr.getSourceLocation()); exprRef.setValue(varRef); return true; } } return false; } }
protected static <T> boolean extractComplexExpressions(ILogicalOperator op, List<T> exprList, Function<T, Mutable<ILogicalExpression>> exprGetter, Predicate<ILogicalExpression> retainPredicate, IOptimizationContext context) throws AlgebricksException { if (!hasComplexExpressions(exprList, exprGetter)) { return false; } boolean rewritten = false; Mutable<ILogicalOperator> inputOpRef = op.getInputs().get(0); for (T item : exprList) { Mutable<ILogicalExpression> exprMutable = exprGetter.apply(item); ILogicalExpression expr = exprMutable.getValue(); if (expr.getExpressionTag() != LogicalExpressionTag.VARIABLE && !retainPredicate.test(expr)) { LogicalVariable v = extractExprIntoAssignOpRef(expr, inputOpRef, context); VariableReferenceExpression vRef = new VariableReferenceExpression(v); vRef.setSourceLocation(expr.getSourceLocation()); exprMutable.setValue(vRef); rewritten = true; } } context.computeAndSetTypeEnvironmentForOperator(op); return rewritten; }
public static void prepareMetaKeyAccessExpression(List<String> field, LogicalVariable resVar, List<Mutable<ILogicalExpression>> assignExpressions, List<LogicalVariable> vars, List<Mutable<ILogicalExpression>> varRefs, IVariableContext context, SourceLocation sourceLoc) { IAObject value = (field.size() > 1) ? new AOrderedList(field) : new AString(field.get(0)); ScalarFunctionCallExpression metaKeyFunction = new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.META_KEY)); metaKeyFunction.setSourceLocation(sourceLoc); VariableReferenceExpression resVarRef = new VariableReferenceExpression(resVar); resVarRef.setSourceLocation(sourceLoc); metaKeyFunction.getArguments().add(new MutableObject<ILogicalExpression>(resVarRef)); metaKeyFunction.getArguments() .add(new MutableObject<>(new ConstantExpression(new AsterixConstantValue(value)))); assignExpressions.add(new MutableObject<ILogicalExpression>(metaKeyFunction)); LogicalVariable v = context.newVar(); vars.add(v); if (varRefs != null) { VariableReferenceExpression vRef = new VariableReferenceExpression(v); vRef.setSourceLocation(sourceLoc); varRefs.add(new MutableObject<ILogicalExpression>(vRef)); } }
@Override public ILogicalOperator visitOrderOperator(OrderOperator op, Void arg) throws AlgebricksException { visitSingleInputOperator(op); if (correlatedKeyVars.isEmpty()) { return op; } orderingExprs.clear(); orderingExprs.addAll(OperatorManipulationUtil.cloneOrderExpressions(op.getOrderExpressions())); List<Pair<IOrder, Mutable<ILogicalExpression>>> orderExprList = new ArrayList<>(); // Adds keyVars to the prefix of sorting columns. for (LogicalVariable keyVar : correlatedKeyVars) { VariableReferenceExpression keyVarRef = new VariableReferenceExpression(keyVar); keyVarRef.setSourceLocation(op.getSourceLocation()); orderExprList.add(new Pair<>(OrderOperator.ASC_ORDER, new MutableObject<>(keyVarRef))); } orderExprList.addAll(op.getOrderExpressions()); // Creates an order operator with the new expression list. OrderOperator orderOp = new OrderOperator(orderExprList); orderOp.setSourceLocation(op.getSourceLocation()); orderOp.getInputs().addAll(op.getInputs()); context.computeAndSetTypeEnvironmentForOperator(orderOp); return orderOp; }
private Pair<ILogicalOperator, LogicalVariable> createUnnestForAggregatedList(LogicalVariable aggVar, SourceLocation sourceLoc) { LogicalVariable unnestVar = context.newVar(); // Creates an unnest function expression. VariableReferenceExpression aggVarRef = new VariableReferenceExpression(aggVar); aggVarRef.setSourceLocation(sourceLoc); Mutable<ILogicalExpression> unnestArg = new MutableObject<>(aggVarRef); List<Mutable<ILogicalExpression>> unnestArgList = new ArrayList<>(); unnestArgList.add(unnestArg); UnnestingFunctionCallExpression unnestExpr = new UnnestingFunctionCallExpression( FunctionUtil.getFunctionInfo(BuiltinFunctions.SCAN_COLLECTION), unnestArgList); unnestExpr.setSourceLocation(sourceLoc); UnnestOperator unnestOp = new UnnestOperator(unnestVar, new MutableObject<>(unnestExpr)); unnestOp.setSourceLocation(sourceLoc); return new Pair<>(unnestOp, unnestVar); }
@Override public ILogicalOperator visitOrderOperator(OrderOperator op, Void arg) throws AlgebricksException { boolean underJoin = hasJoinAncestor; visitSingleInputOperator(op); if (!rewritten || !underJoin) { return op; } // Adjust the ordering if its input operator pipeline has been rewritten. List<Pair<IOrder, Mutable<ILogicalExpression>>> orderExprList = new ArrayList<>(); // Adds keyVars to the prefix of sorting columns. for (LogicalVariable liveVar : liveVarsFromSubplanInput) { VariableReferenceExpression liveVarRef = new VariableReferenceExpression(liveVar); liveVarRef.setSourceLocation(op.getSourceLocation()); orderExprList.add(new Pair<IOrder, Mutable<ILogicalExpression>>(OrderOperator.ASC_ORDER, new MutableObject<ILogicalExpression>(liveVarRef))); } orderExprList.addAll(op.getOrderExpressions()); // Creates an order operator with the new expression list. OrderOperator orderOp = new OrderOperator(orderExprList); orderOp.setSourceLocation(op.getSourceLocation()); orderOp.getInputs().addAll(op.getInputs()); context.computeAndSetTypeEnvironmentForOperator(orderOp); return orderOp; }
protected ILogicalExpression translateVariableRef(VariableExpr varExpr) throws CompilationException { LogicalVariable var = context.getVar(varExpr.getVar().getId()); if (var == null) { throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, varExpr.getSourceLocation(), varExpr.toString()); } VariableReferenceExpression varRef = new VariableReferenceExpression(var); varRef.setSourceLocation(varExpr.getSourceLocation()); return varRef; }
private Pair<List<Mutable<ILogicalExpression>>, ILogicalOperator> translateWindowBoundaryExpr( Expression boundaryExpr, List<Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>>> valueExprs, Mutable<ILogicalOperator> tupSource, OperatorType boundaryOperator) throws CompilationException { SourceLocation sourceLoc = boundaryExpr.getSourceLocation(); if (valueExprs.size() != 1) { throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, valueExprs.size()); } ILogicalExpression valueExpr = valueExprs.get(0).second.getValue(); AbstractFunctionCallExpression resultExpr = createFunctionCallExpressionForBuiltinOperator(boundaryOperator, sourceLoc); resultExpr.getArguments().add(new MutableObject<>(valueExpr.cloneExpression())); Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = langExprToAlgExpression(boundaryExpr, tupSource); resultExpr.getArguments().add(new MutableObject<>(eo.first)); LogicalVariable resultVar = context.newVar(); AssignOperator assignOp = new AssignOperator(resultVar, new MutableObject<>(resultExpr)); assignOp.setSourceLocation(sourceLoc); assignOp.getInputs().add(eo.second); VariableReferenceExpression resultVarRefExpr = new VariableReferenceExpression(resultVar); resultVarRefExpr.setSourceLocation(sourceLoc); return new Pair<>(mkSingletonArrayList(new MutableObject<>(resultVarRefExpr)), assignOp); }
protected void buildVarExprList(Collection<LogicalVariable> vars, IOptimizationContext context, GroupByOperator g, List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> outVeList) throws AlgebricksException { SourceLocation sourceLoc = g.getSourceLocation(); for (LogicalVariable ov : vars) { LogicalVariable newVar = context.newVar(); VariableReferenceExpression varExpr = new VariableReferenceExpression(newVar); varExpr.setSourceLocation(sourceLoc); outVeList.add(new Pair<LogicalVariable, Mutable<ILogicalExpression>>(ov, new MutableObject<ILogicalExpression>(varExpr))); for (ILogicalPlan p : g.getNestedPlans()) { for (Mutable<ILogicalOperator> r : p.getRoots()) { OperatorManipulationUtil.substituteVarRec((AbstractLogicalOperator) r.getValue(), ov, newVar, true, context); } } } }
private Pair<ILogicalOperator, LogicalVariable> createRecordConstructorAssignOp(Set<LogicalVariable> inputLiveVars, SourceLocation sourceLoc) { // Creates a nested record. List<Mutable<ILogicalExpression>> recordConstructorArgs = new ArrayList<>(); for (LogicalVariable inputLiveVar : inputLiveVars) { if (!correlatedKeyVars.contains(inputLiveVar)) { recordConstructorArgs.add(new MutableObject<>(new ConstantExpression( new AsterixConstantValue(new AString(Integer.toString(inputLiveVar.getId())))))); VariableReferenceExpression inputLiveVarRef = new VariableReferenceExpression(inputLiveVar); inputLiveVarRef.setSourceLocation(sourceLoc); recordConstructorArgs.add(new MutableObject<>(inputLiveVarRef)); } } LogicalVariable recordVar = context.newVar(); ScalarFunctionCallExpression openRecConstr = new ScalarFunctionCallExpression( FunctionUtil.getFunctionInfo(BuiltinFunctions.OPEN_RECORD_CONSTRUCTOR), recordConstructorArgs); openRecConstr.setSourceLocation(sourceLoc); Mutable<ILogicalExpression> recordExprRef = new MutableObject<ILogicalExpression>(openRecConstr); AssignOperator assignOp = new AssignOperator(recordVar, recordExprRef); assignOp.setSourceLocation(sourceLoc); return new Pair<>(assignOp, recordVar); }