private QueryWorkUnit getQueryWorkUnit(final PhysicalPlan plan) throws ExecutionSetupException { final PhysicalOperator rootOperator = plan.getSortedOperators(false).iterator().next(); final Fragment rootFragment = rootOperator.accept(MakeFragmentsVisitor.INSTANCE, null); final SimpleParallelizer parallelizer = new SimpleParallelizer(queryContext); final QueryWorkUnit queryWorkUnit = parallelizer.getFragments( queryContext.getOptions().getOptionList(), queryContext.getCurrentEndpoint(), queryId, queryContext.getActiveEndpoints(), drillbitContext.getPlanReader(), rootFragment, initiatingClient.getSession(), queryContext.getQueryContextInfo()); if (logger.isTraceEnabled()) { final StringBuilder sb = new StringBuilder(); sb.append("PlanFragments for query "); sb.append(queryId); sb.append('\n'); final List<PlanFragment> planFragments = queryWorkUnit.getFragments(); final int fragmentCount = planFragments.size(); int fragmentIndex = 0; for (final PlanFragment planFragment : planFragments) { final FragmentHandle fragmentHandle = planFragment.getHandle(); sb.append("PlanFragment("); sb.append(++fragmentIndex); sb.append('/'); sb.append(fragmentCount); sb.append(") major_fragment_id "); sb.append(fragmentHandle.getMajorFragmentId()); sb.append(" minor_fragment_id "); sb.append(fragmentHandle.getMinorFragmentId()); sb.append('\n'); final DrillbitEndpoint endpointAssignment = planFragment.getAssignment(); sb.append(" DrillbitEndpoint address "); sb.append(endpointAssignment.getAddress()); sb.append('\n'); String jsonString = "<<malformed JSON>>"; sb.append(" fragment_json: "); final ObjectMapper objectMapper = new ObjectMapper(); try { final Object json = objectMapper.readValue(planFragment.getFragmentJson(), Object.class); jsonString = objectMapper.defaultPrettyPrintingWriter().writeValueAsString(json); } catch (final Exception e) { // we've already set jsonString to a fallback value } sb.append(jsonString); logger.trace(sb.toString()); } } return queryWorkUnit; }
private void runPhysicalPlan(PhysicalPlan plan) { if (plan.getProperties().resultMode != ResultMode.EXEC) { fail( String.format( "Failure running plan. You requested a result mode of %s and a physical plan can only be output as EXEC", plan.getProperties().resultMode), new Exception()); } PhysicalOperator rootOperator = plan.getSortedOperators(false).iterator().next(); MakeFragmentsVisitor makeFragmentsVisitor = new MakeFragmentsVisitor(); Fragment rootFragment; try { rootFragment = rootOperator.accept(makeFragmentsVisitor, null); } catch (FragmentSetupException e) { fail("Failure while fragmenting query.", e); return; } PlanningSet planningSet = StatsCollector.collectStats(rootFragment); SimpleParallelizer parallelizer = new SimpleParallelizer(); try { QueryWorkUnit work = parallelizer.getFragments( context.getCurrentEndpoint(), queryId, context.getActiveEndpoints(), context.getPlanReader(), rootFragment, planningSet, context.getConfig().getInt(ExecConstants.GLOBAL_MAX_WIDTH), context.getConfig().getInt(ExecConstants.MAX_WIDTH_PER_ENDPOINT)); this.context .getWorkBus() .setFragmentStatusListener( work.getRootFragment().getHandle().getQueryId(), fragmentManager); List<PlanFragment> leafFragments = Lists.newArrayList(); List<PlanFragment> intermediateFragments = Lists.newArrayList(); // store fragments in distributed grid. logger.debug("Storing fragments"); for (PlanFragment f : work.getFragments()) { // store all fragments in grid since they are part of handshake. context.getCache().storeFragment(f); if (f.getLeafFragment()) { leafFragments.add(f); } else { intermediateFragments.add(f); } } logger.debug("Fragments stored."); logger.debug("Submitting fragments to run."); fragmentManager.runFragments( bee, work.getRootFragment(), work.getRootOperator(), initiatingClient, leafFragments, intermediateFragments); logger.debug("Fragments running."); } catch (ExecutionSetupException | RpcException e) { fail("Failure while setting up query.", e); } }
protected Prel convertToPrel(RelNode drel) throws RelConversionException, SqlUnsupportedException { Preconditions.checkArgument(drel.getConvention() == DrillRel.DRILL_LOGICAL); RelTraitSet traits = drel.getTraitSet().plus(Prel.DRILL_PHYSICAL).plus(DrillDistributionTrait.SINGLETON); Prel phyRelNode; try { final RelNode relNode = planner.transform(DrillSqlWorker.PHYSICAL_MEM_RULES, traits, drel); phyRelNode = (Prel) relNode.accept(new PrelFinalizer()); } catch (RelOptPlanner.CannotPlanException ex) { logger.error(ex.getMessage()); if (JoinUtils.checkCartesianJoin(drel, new ArrayList<Integer>(), new ArrayList<Integer>())) { throw new UnsupportedRelOperatorException( "This query cannot be planned possibly due to either a cartesian join or an inequality join"); } else { throw ex; } } OptionManager queryOptions = context.getOptions(); if (context.getPlannerSettings().isMemoryEstimationEnabled() && !MemoryEstimationVisitor.enoughMemory( phyRelNode, queryOptions, context.getActiveEndpoints().size())) { log("Not enough memory for this plan", phyRelNode, logger); logger.debug("Re-planning without hash operations."); queryOptions.setOption( OptionValue.createBoolean( OptionValue.OptionType.QUERY, PlannerSettings.HASHJOIN.getOptionName(), false)); queryOptions.setOption( OptionValue.createBoolean( OptionValue.OptionType.QUERY, PlannerSettings.HASHAGG.getOptionName(), false)); try { final RelNode relNode = planner.transform(DrillSqlWorker.PHYSICAL_MEM_RULES, traits, drel); phyRelNode = (Prel) relNode.accept(new PrelFinalizer()); } catch (RelOptPlanner.CannotPlanException ex) { logger.error(ex.getMessage()); if (JoinUtils.checkCartesianJoin( drel, new ArrayList<Integer>(), new ArrayList<Integer>())) { throw new UnsupportedRelOperatorException( "This query cannot be planned possibly due to either a cartesian join or an inequality join"); } else { throw ex; } } } /* The order of the following transformation is important */ /* * 0.) For select * from join query, we need insert project on top of scan and a top project just * under screen operator. The project on top of scan will rename from * to T1*, while the top project * will rename T1* to *, before it output the final result. Only the top project will allow * duplicate columns, since user could "explicitly" ask for duplicate columns ( select *, col, *). * The rest of projects will remove the duplicate column when we generate POP in json format. */ phyRelNode = StarColumnConverter.insertRenameProject(phyRelNode); /* * 1.) * Join might cause naming conflicts from its left and right child. * In such case, we have to insert Project to rename the conflicting names. */ phyRelNode = JoinPrelRenameVisitor.insertRenameProject(phyRelNode); /* * 1.1) Swap left / right for INNER hash join, if left's row count is < (1 + margin) right's row count. * We want to have smaller dataset on the right side, since hash table builds on right side. */ if (context.getPlannerSettings().isHashJoinSwapEnabled()) { phyRelNode = SwapHashJoinVisitor.swapHashJoin( phyRelNode, new Double(context.getPlannerSettings().getHashJoinSwapMarginFactor())); } /* * 1.2) Break up all expressions with complex outputs into their own project operations */ phyRelNode = ((Prel) phyRelNode) .accept( new SplitUpComplexExpressions( planner.getTypeFactory(), context.getDrillOperatorTable(), context.getPlannerSettings().functionImplementationRegistry), null); /* * 1.3) Projections that contain reference to flatten are rewritten as Flatten operators followed by Project */ phyRelNode = ((Prel) phyRelNode) .accept( new RewriteProjectToFlatten( planner.getTypeFactory(), context.getDrillOperatorTable()), null); /* * 2.) * Since our operators work via names rather than indices, we have to make to reorder any * output before we return data to the user as we may have accidentally shuffled things. * This adds a trivial project to reorder columns prior to output. */ phyRelNode = FinalColumnReorderer.addFinalColumnOrdering(phyRelNode); /* * 3.) * If two fragments are both estimated to be parallelization one, remove the exchange * separating them */ phyRelNode = ExcessiveExchangeIdentifier.removeExcessiveEchanges(phyRelNode, targetSliceSize); /* 4.) * Add ProducerConsumer after each scan if the option is set * Use the configured queueSize */ /* DRILL-1617 Disabling ProducerConsumer as it produces incorrect results if (context.getOptions().getOption(PlannerSettings.PRODUCER_CONSUMER.getOptionName()).bool_val) { long queueSize = context.getOptions().getOption(PlannerSettings.PRODUCER_CONSUMER_QUEUE_SIZE.getOptionName()).num_val; phyRelNode = ProducerConsumerPrelVisitor.addProducerConsumerToScans(phyRelNode, (int) queueSize); } */ /* 5.) * if the client does not support complex types (Map, Repeated) * insert a project which which would convert */ if (!context.getSession().isSupportComplexTypes()) { logger.debug("Client does not support complex types, add ComplexToJson operator."); phyRelNode = ComplexToJsonPrelVisitor.addComplexToJsonPrel(phyRelNode); } /* 6.) * Insert LocalExchange (mux and/or demux) nodes */ phyRelNode = InsertLocalExchangeVisitor.insertLocalExchanges(phyRelNode, queryOptions); /* 7.) * Next, we add any required selection vector removers given the supported encodings of each * operator. This will ultimately move to a new trait but we're managing here for now to avoid * introducing new issues in planning before the next release */ phyRelNode = SelectionVectorPrelVisitor.addSelectionRemoversWhereNecessary(phyRelNode); /* 8.) * Finally, Make sure that the no rels are repeats. * This could happen in the case of querying the same table twice as Optiq may canonicalize these. */ phyRelNode = RelUniqifier.uniqifyGraph(phyRelNode); return phyRelNode; }