/** * Set up the root fragment (which will run locally), and submit it for execution. * * @param rootFragment * @param rootOperator * @throws ExecutionSetupException */ private void setupRootFragment(final PlanFragment rootFragment, final FragmentRoot rootOperator) throws ExecutionSetupException { @SuppressWarnings("resource") final FragmentContext rootContext = new FragmentContext( drillbitContext, rootFragment, queryContext, initiatingClient, drillbitContext.getFunctionImplementationRegistry()); @SuppressWarnings("resource") final IncomingBuffers buffers = new IncomingBuffers(rootFragment, rootContext); rootContext.setBuffers(buffers); queryManager.addFragmentStatusTracker(rootFragment, true); rootRunner = new FragmentExecutor( rootContext, rootFragment, queryManager.newRootStatusHandler(rootContext, drillbitContext), rootOperator); final RootFragmentManager fragmentManager = new RootFragmentManager(rootFragment.getHandle(), buffers, rootRunner); if (buffers.isDone()) { // if we don't have to wait for any incoming data, start the fragment runner. bee.addFragmentRunner(fragmentManager.getRunnable()); } else { // if we do, record the fragment manager in the workBus. // TODO aren't we managing our own work? What does this do? It looks like this will never get // run drillbitContext.getWorkBus().addFragmentManager(fragmentManager); } }
private void runPhysicalPlan(final PhysicalPlan plan) throws ExecutionSetupException { validatePlan(plan); setupSortMemoryAllocations(plan); acquireQuerySemaphore(plan); final QueryWorkUnit work = getQueryWorkUnit(plan); final List<PlanFragment> planFragments = work.getFragments(); final PlanFragment rootPlanFragment = work.getRootFragment(); assert queryId == rootPlanFragment.getHandle().getQueryId(); drillbitContext .getWorkBus() .addFragmentStatusListener(queryId, queryManager.getFragmentStatusListener()); drillbitContext .getClusterCoordinator() .addDrillbitStatusListener(queryManager.getDrillbitStatusListener()); logger.debug("Submitting fragments to run."); // set up the root fragment first so we'll have incoming buffers available. setupRootFragment(rootPlanFragment, work.getRootOperator()); setupNonRootFragments(planFragments); drillbitContext.getAllocator().resetFragmentLimits(); // TODO a global effect for this query?!? moveToState(QueryState.RUNNING, null); logger.debug("Fragments running."); }
/** * Set up the root fragment (which will run locally), and submit it for execution. * * @param rootFragment * @param rootOperator * @throws ExecutionSetupException */ private void setupRootFragment(final PlanFragment rootFragment, final FragmentRoot rootOperator) throws ExecutionSetupException { @SuppressWarnings("resource") final FragmentContext rootContext = new FragmentContext( drillbitContext, rootFragment, queryContext, initiatingClient, drillbitContext.getFunctionImplementationRegistry()); @SuppressWarnings("resource") final IncomingBuffers buffers = new IncomingBuffers(rootFragment, rootContext); rootContext.setBuffers(buffers); queryManager.addFragmentStatusTracker(rootFragment, true); final ControlTunnel tunnel = drillbitContext.getController().getTunnel(queryContext.getCurrentEndpoint()); final FragmentExecutor rootRunner = new FragmentExecutor( rootContext, rootFragment, new FragmentStatusReporter(rootContext, tunnel), rootOperator); final RootFragmentManager fragmentManager = new RootFragmentManager(rootFragment.getHandle(), buffers, rootRunner); if (buffers.isDone()) { // if we don't have to wait for any incoming data, start the fragment runner. bee.addFragmentRunner(fragmentManager.getRunnable()); } else { // if we do, record the fragment manager in the workBus. drillbitContext.getWorkBus().addFragmentManager(fragmentManager); } }
/** * Constructor. Sets up the Foreman, but does not initiate any execution. * * @param bee used to submit additional work * @param drillbitContext * @param connection * @param queryId the id for the query * @param queryRequest the query to execute */ public Foreman( final WorkerBee bee, final DrillbitContext drillbitContext, final UserClientConnection connection, final QueryId queryId, final RunQuery queryRequest) { this.bee = bee; this.queryId = queryId; this.queryRequest = queryRequest; this.drillbitContext = drillbitContext; initiatingClient = connection; this.closeFuture = initiatingClient.getChannel().closeFuture(); closeFuture.addListener(closeListener); queryContext = new QueryContext(connection.getSession(), drillbitContext); queryManager = new QueryManager( queryId, queryRequest, drillbitContext.getPersistentStoreProvider(), stateListener, this); // TODO reference escapes before ctor is complete via stateListener, this recordNewState(QueryState.PENDING); }
private void parseAndRunLogicalPlan(final String json) throws ExecutionSetupException { LogicalPlan logicalPlan; try { logicalPlan = drillbitContext.getPlanReader().readLogicalPlan(json); } catch (final IOException e) { throw new ForemanException("Failure parsing logical plan.", e); } if (logicalPlan.getProperties().resultMode == ResultMode.LOGICAL) { throw new ForemanException( "Failure running plan. You requested a result mode of LOGICAL and submitted a logical plan. In this case you're output mode must be PHYSICAL or EXEC."); } log(logicalPlan); final PhysicalPlan physicalPlan = convert(logicalPlan); if (logicalPlan.getProperties().resultMode == ResultMode.PHYSICAL) { returnPhysical(physicalPlan); return; } log(physicalPlan); runPhysicalPlan(physicalPlan); }
@Test public void test(final DrillbitContext bitContext) throws Exception { final DrillConfig c = DrillConfig.create(); new NonStrictExpectations() { { bitContext.getMetrics(); result = new MetricRegistry(); bitContext.getAllocator(); result = new TopLevelAllocator(); bitContext.getConfig(); result = c; bitContext.getCache(); result = new LocalCache(); } }; bitContext.getCache().run(); StoragePluginRegistry r = new StoragePluginRegistry(bitContext); SchemaPlus plus = Frameworks.createRootSchema(); r.init(); r.getSchemaFactory().registerSchemas(null, plus); printSchema(plus, 0); }
private void parseAndRunPhysicalPlan(final String json) throws ExecutionSetupException { try { final PhysicalPlan plan = drillbitContext.getPlanReader().readPhysicalPlan(json); runPhysicalPlan(plan); } catch (final IOException e) { throw new ForemanSetupException("Failure while parsing physical plan.", e); } }
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; }
public QueryContext( final UserSession session, final DrillbitContext drillbitContext, QueryId queryId) { this.drillbitContext = drillbitContext; this.session = session; queryOptions = new QueryOptionManager(session.getOptions()); executionControls = new ExecutionControls(queryOptions, drillbitContext.getEndpoint()); plannerSettings = new PlannerSettings(queryOptions, getFunctionRegistry()); plannerSettings.setNumEndPoints(drillbitContext.getBits().size()); table = new DrillOperatorTable(getFunctionRegistry(), drillbitContext.getOptionManager()); queryContextInfo = Utilities.createQueryContextInfo(session.getDefaultSchemaName()); contextInformation = new ContextInformation(session.getCredentials(), queryContextInfo); allocator = drillbitContext .getAllocator() .newChildAllocator( "query:" + QueryIdHelper.getQueryId(queryId), PlannerSettings.getInitialPlanningMemorySize(), plannerSettings.getPlanningMemoryLimit()); bufferManager = new BufferManagerImpl(this.allocator); viewExpansionContext = new ViewExpansionContext(this); schemaTreeProvider = new SchemaTreeProvider(drillbitContext); }
@Override public AbstractGroupScan getPhysicalScan(JSONOptions selection) throws IOException { FormatSelection formatSelection = selection.getWith(context.getConfig(), FormatSelection.class); FormatPlugin plugin; if (formatSelection.getFormat() instanceof NamedFormatPluginConfig) { plugin = formatsByName.get(((NamedFormatPluginConfig) formatSelection.getFormat()).name); } else { plugin = formatPluginsByConfig.get(formatSelection.getFormat()); } if (plugin == null) throw new IOException( String.format( "Failure getting requested format plugin named '%s'. It was not one of the format plugins registered.", formatSelection.getFormat())); return plugin.getGroupScan(formatSelection.getSelection()); }
/** * Assign each unit of work to a minor fragment, given that a list of DrillbitEndpoints, whose * index in the list correspond to the minor fragment id for each fragment. A given * DrillbitEndpoint can appear multiple times in this list. This method will try to assign work * based on the affinity of each work unit, but will also evenly distribute the work units among * all of the minor fragments * * @param incomingEndpoints The list of incomingEndpoints, indexed by minor fragment id * @param units the list of work units to be assigned * @return A multimap that maps each minor fragment id to a list of work units */ public static <T extends CompleteWork> ListMultimap<Integer, T> getMappings( List<DrillbitEndpoint> incomingEndpoints, List<T> units, DrillbitContext context) { boolean useOldAssignmentCode = context == null ? false : context .getOptionManager() .getOption(ExecConstants.USE_OLD_ASSIGNMENT_CREATOR) .bool_val; if (useOldAssignmentCode) { return OldAssignmentCreator.getMappings(incomingEndpoints, units); } else { AssignmentCreator<T> creator = new AssignmentCreator(incomingEndpoints, units); return creator.getMappings(); } }
/** * Send all the remote fragments belonging to a single target drillbit in one request. * * @param assignment the drillbit assigned to these fragments * @param fragments the set of fragments * @param latch the countdown latch used to track the requests to all endpoints * @param fragmentSubmitFailures the submission failure counter used to track the requests to all * endpoints */ private void sendRemoteFragments( final DrillbitEndpoint assignment, final Collection<PlanFragment> fragments, final CountDownLatch latch, final FragmentSubmitFailures fragmentSubmitFailures) { @SuppressWarnings("resource") final Controller controller = drillbitContext.getController(); final InitializeFragments.Builder fb = InitializeFragments.newBuilder(); for (final PlanFragment planFragment : fragments) { fb.addFragment(planFragment); } final InitializeFragments initFrags = fb.build(); logger.debug("Sending remote fragments to \nNode:\n{} \n\nData:\n{}", assignment, initFrags); final FragmentSubmitListener listener = new FragmentSubmitListener(assignment, initFrags, latch, fragmentSubmitFailures); controller.getTunnel(assignment).sendFragments(listener, initFrags); }
/** * This limits the number of "small" and "large" queries that a Drill cluster will run * simultaneously, if queueing is enabled. If the query is unable to run, this will block until it * can. Beware that this is called under run(), and so will consume a Thread while it waits for * the required distributed semaphore. * * @param plan the query plan * @throws ForemanSetupException */ private void acquireQuerySemaphore(final PhysicalPlan plan) throws ForemanSetupException { final OptionManager optionManager = queryContext.getOptions(); final boolean queuingEnabled = optionManager.getOption(ExecConstants.ENABLE_QUEUE); if (queuingEnabled) { final long queueThreshold = optionManager.getOption(ExecConstants.QUEUE_THRESHOLD_SIZE); double totalCost = 0; for (final PhysicalOperator ops : plan.getSortedOperators()) { totalCost += ops.getCost(); } final long queueTimeout = optionManager.getOption(ExecConstants.QUEUE_TIMEOUT); final String queueName; try { @SuppressWarnings("resource") final ClusterCoordinator clusterCoordinator = drillbitContext.getClusterCoordinator(); final DistributedSemaphore distributedSemaphore; // get the appropriate semaphore if (totalCost > queueThreshold) { final int largeQueue = (int) optionManager.getOption(ExecConstants.LARGE_QUEUE_SIZE); distributedSemaphore = clusterCoordinator.getSemaphore("query.large", largeQueue); queueName = "large"; } else { final int smallQueue = (int) optionManager.getOption(ExecConstants.SMALL_QUEUE_SIZE); distributedSemaphore = clusterCoordinator.getSemaphore("query.small", smallQueue); queueName = "small"; } lease = distributedSemaphore.acquire(queueTimeout, TimeUnit.MILLISECONDS); } catch (final Exception e) { throw new ForemanSetupException("Unable to acquire slot for query.", e); } if (lease == null) { throw UserException.resourceError() .message( "Unable to acquire queue resources for query within timeout. Timeout for %s queue was set at %d seconds.", queueName, queueTimeout / 1000) .build(logger); } } }
public Foreman( WorkerBee bee, DrillbitContext dContext, UserClientConnection connection, QueryId queryId, RunQuery queryRequest) { this.queryId = queryId; this.queryRequest = queryRequest; this.context = new QueryContext(connection.getSession(), queryId, dContext); this.initiatingClient = connection; this.fragmentManager = new QueryManager(new ForemanManagerListener(), dContext.getController()); this.bee = bee; this.state = new AtomicState<QueryState>(QueryState.PENDING) { protected QueryState getStateFromNumber(int i) { return QueryState.valueOf(i); } }; }
public FileSystemPlugin(FileSystemConfig config, DrillbitContext context, String name) throws ExecutionSetupException { try { this.config = config; this.context = context; Configuration fsConf = new Configuration(); fsConf.set(FileSystem.FS_DEFAULT_NAME_KEY, config.connection); fsConf.set("fs.classpath.impl", ClassPathFileSystem.class.getName()); fsConf.set("fs.drill-local.impl", LocalSyncableFileSystem.class.getName()); this.fs = FileSystemCreator.getFileSystem(context.getConfig(), fsConf); this.formatsByName = FormatCreator.getFormatPlugins(context, fs, config); List<FormatMatcher> matchers = Lists.newArrayList(); formatPluginsByConfig = Maps.newHashMap(); for (FormatPlugin p : formatsByName.values()) { matchers.add(p.getMatcher()); formatPluginsByConfig.put(p.getConfig(), p); } List<WorkspaceSchemaFactory> factories = null; if (config.workspaces == null || config.workspaces.isEmpty()) { factories = Collections.singletonList( new WorkspaceSchemaFactory(this, "default", name, fs, "/", matchers)); } else { factories = Lists.newArrayList(); for (Map.Entry<String, String> space : config.workspaces.entrySet()) { factories.add( new WorkspaceSchemaFactory( this, space.getKey(), name, fs, space.getValue(), matchers)); } } this.schemaFactory = new FileSystemSchemaFactory(name, factories); } catch (IOException e) { throw new ExecutionSetupException("Failure setting up file system plugin.", e); } }
@Override public void run() { // if a cancel thread has already entered this executor, we have not reason to continue. if (!hasCloseoutThread.compareAndSet(false, true)) { return; } final Thread myThread = Thread.currentThread(); myThreadRef.set(myThread); final String originalThreadName = myThread.getName(); final FragmentHandle fragmentHandle = fragmentContext.getHandle(); final DrillbitContext drillbitContext = fragmentContext.getDrillbitContext(); final ClusterCoordinator clusterCoordinator = drillbitContext.getClusterCoordinator(); final DrillbitStatusListener drillbitStatusListener = new FragmentDrillbitStatusListener(); final String newThreadName = QueryIdHelper.getExecutorThreadName(fragmentHandle); try { myThread.setName(newThreadName); // if we didn't get the root operator when the executor was created, create it now. final FragmentRoot rootOperator = this.rootOperator != null ? this.rootOperator : drillbitContext.getPlanReader().readFragmentOperator(fragment.getFragmentJson()); root = ImplCreator.getExec(fragmentContext, rootOperator); if (root == null) { return; } clusterCoordinator.addDrillbitStatusListener(drillbitStatusListener); updateState(FragmentState.RUNNING); acceptExternalEvents.countDown(); injector.injectPause(fragmentContext.getExecutionControls(), "fragment-running", logger); final DrillbitEndpoint endpoint = drillbitContext.getEndpoint(); logger.debug( "Starting fragment {}:{} on {}:{}", fragmentHandle.getMajorFragmentId(), fragmentHandle.getMinorFragmentId(), endpoint.getAddress(), endpoint.getUserPort()); final UserGroupInformation queryUserUgi = fragmentContext.isImpersonationEnabled() ? ImpersonationUtil.createProxyUgi(fragmentContext.getQueryUserName()) : ImpersonationUtil.getProcessUserUGI(); queryUserUgi.doAs( new PrivilegedExceptionAction<Void>() { public Void run() throws Exception { injector.injectChecked( fragmentContext.getExecutionControls(), "fragment-execution", IOException.class); /* * Run the query until root.next returns false OR we no longer need to continue. */ while (shouldContinue() && root.next()) { // loop } return null; } }); } catch (OutOfMemoryError | OutOfMemoryException e) { if (!(e instanceof OutOfMemoryError) || "Direct buffer memory".equals(e.getMessage())) { fail(UserException.memoryError(e).build(logger)); } else { // we have a heap out of memory error. The JVM in unstable, exit. CatastrophicFailure.exit( e, "Unable to handle out of memory condition in FragmentExecutor.", -2); } } catch (AssertionError | Exception e) { fail(e); } finally { // no longer allow this thread to be interrupted. We synchronize here to make sure that cancel // can't set an // interruption after we have moved beyond this block. synchronized (myThreadRef) { myThreadRef.set(null); Thread.interrupted(); } // We need to sure we countDown at least once. We'll do it here to guarantee that. acceptExternalEvents.countDown(); // here we could be in FAILED, RUNNING, or CANCELLATION_REQUESTED cleanup(FragmentState.FINISHED); clusterCoordinator.removeDrillbitStatusListener(drillbitStatusListener); myThread.setName(originalThreadName); } }
@Override public InfoSchemaGroupScan getPhysicalScan(JSONOptions selection) throws IOException { SelectedTable table = selection.getWith(context.getConfig(), SelectedTable.class); return new InfoSchemaGroupScan(table); }
@Test public void testAllocators() throws Exception { // Setup a drillbit (initializes a root allocator) final DrillConfig config = DrillConfig.create(TEST_CONFIGURATIONS); final RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet(); final Drillbit bit = new Drillbit(config, serviceSet); bit.run(); final DrillbitContext bitContext = bit.getContext(); FunctionImplementationRegistry functionRegistry = bitContext.getFunctionImplementationRegistry(); StoragePluginRegistry storageRegistry = new StoragePluginRegistry(bitContext); // Create a few Fragment Contexts BitControl.PlanFragment.Builder pfBuilder1 = BitControl.PlanFragment.newBuilder(); pfBuilder1.setMemInitial(1500000); BitControl.PlanFragment pf1 = pfBuilder1.build(); BitControl.PlanFragment.Builder pfBuilder2 = BitControl.PlanFragment.newBuilder(); pfBuilder2.setMemInitial(500000); BitControl.PlanFragment pf2 = pfBuilder1.build(); FragmentContext fragmentContext1 = new FragmentContext(bitContext, pf1, null, functionRegistry); FragmentContext fragmentContext2 = new FragmentContext(bitContext, pf2, null, functionRegistry); // Get a few physical operators. Easiest way is to read a physical plan. PhysicalPlanReader planReader = new PhysicalPlanReader( config, config.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance(), storageRegistry); PhysicalPlan plan = planReader.readPhysicalPlan( Files.toString(FileUtils.getResourceAsFile(planFile), Charsets.UTF_8)); List<PhysicalOperator> physicalOperators = plan.getSortedOperators(); Iterator<PhysicalOperator> physicalOperatorIterator = physicalOperators.iterator(); PhysicalOperator physicalOperator1 = physicalOperatorIterator.next(); PhysicalOperator physicalOperator2 = physicalOperatorIterator.next(); PhysicalOperator physicalOperator3 = physicalOperatorIterator.next(); PhysicalOperator physicalOperator4 = physicalOperatorIterator.next(); PhysicalOperator physicalOperator5 = physicalOperatorIterator.next(); PhysicalOperator physicalOperator6 = physicalOperatorIterator.next(); // Create some bogus Operator profile defs and stats to create operator contexts OpProfileDef def; OperatorStats stats; // Use some bogus operator type to create a new operator context. def = new OpProfileDef( physicalOperator1.getOperatorId(), UserBitShared.CoreOperatorType.MOCK_SUB_SCAN_VALUE, OperatorContext.getChildCount(physicalOperator1)); stats = fragmentContext1.getStats().getOperatorStats(def, fragmentContext1.getAllocator()); // Add a couple of Operator Contexts // Initial allocation = 1000000 bytes for all operators OperatorContext oContext11 = fragmentContext1.newOperatorContext(physicalOperator1, true); DrillBuf b11 = oContext11.getAllocator().buffer(1000000); OperatorContext oContext12 = fragmentContext1.newOperatorContext(physicalOperator2, stats, true); DrillBuf b12 = oContext12.getAllocator().buffer(500000); OperatorContext oContext21 = fragmentContext1.newOperatorContext(physicalOperator3, true); def = new OpProfileDef( physicalOperator4.getOperatorId(), UserBitShared.CoreOperatorType.TEXT_WRITER_VALUE, OperatorContext.getChildCount(physicalOperator4)); stats = fragmentContext2.getStats().getOperatorStats(def, fragmentContext2.getAllocator()); OperatorContext oContext22 = fragmentContext2.newOperatorContext(physicalOperator4, stats, true); DrillBuf b22 = oContext22.getAllocator().buffer(2000000); // New Fragment begins BitControl.PlanFragment.Builder pfBuilder3 = BitControl.PlanFragment.newBuilder(); pfBuilder3.setMemInitial(1000000); BitControl.PlanFragment pf3 = pfBuilder3.build(); FragmentContext fragmentContext3 = new FragmentContext(bitContext, pf3, null, functionRegistry); // New fragment starts an operator that allocates an amount within the limit def = new OpProfileDef( physicalOperator5.getOperatorId(), UserBitShared.CoreOperatorType.UNION_VALUE, OperatorContext.getChildCount(physicalOperator5)); stats = fragmentContext3.getStats().getOperatorStats(def, fragmentContext3.getAllocator()); OperatorContext oContext31 = fragmentContext3.newOperatorContext(physicalOperator5, stats, true); DrillBuf b31a = oContext31.getAllocator().buffer(200000); // Previously running operator completes b22.release(); ((AutoCloseable) oContext22).close(); // Fragment 3 asks for more and fails boolean outOfMem = false; try { DrillBuf b31b = oContext31.getAllocator().buffer(4400000); if (b31b != null) { b31b.release(); } else { outOfMem = true; } } catch (Exception e) { outOfMem = true; } assertEquals(true, (boolean) outOfMem); // Operator is Exempt from Fragment limits. Fragment 3 asks for more and succeeds outOfMem = false; OperatorContext oContext32 = fragmentContext3.newOperatorContext(physicalOperator6, false); DrillBuf b32 = null; try { b32 = oContext32.getAllocator().buffer(4400000); } catch (Exception e) { outOfMem = true; } finally { if (b32 != null) { b32.release(); } else { outOfMem = true; } closeOp(oContext32); } assertEquals(false, (boolean) outOfMem); b11.release(); closeOp(oContext11); b12.release(); closeOp(oContext12); closeOp(oContext21); b31a.release(); closeOp(oContext31); fragmentContext1.close(); fragmentContext2.close(); fragmentContext3.close(); bit.close(); serviceSet.close(); }
public DrillbitEndpoint getCurrentEndpoint() { return drillbitContext.getEndpoint(); }
public StoragePluginRegistry getStorage() { return drillbitContext.getStorage(); }
public LogicalPlanPersistence getLpPersistence() { return drillbitContext.getLpPersistence(); }
public Collection<DrillbitEndpoint> getActiveEndpoints() { return drillbitContext.getBits(); }
public DrillConfig getConfig() { return drillbitContext.getConfig(); }
@Override public FunctionImplementationRegistry getFunctionRegistry() { return drillbitContext.getFunctionImplementationRegistry(); }
public RemoteFunctionRegistry getRemoteFunctionRegistry() { return drillbitContext.getRemoteFunctionRegistry(); }