Clone Tools
  • last updated a few minutes ago
Constraints
Constraints: committers
 
Constraints: files
Constraints: dates
DRILL-6320: Fixed license headers.

closes #1207

  1. … 2065 more files in changeset.
DRILL-6053: Avoid excessive locking in LocalPersistentStore

closes #1163

  1. … 18 more files in changeset.
DRILL-4134: Allocator Improvements

- make Allocator mostly lockless

- change BaseAllocator maps to direct references

- add documentation around memory management model

- move transfer and ownership methods to DrillBuf

- Improve debug messaging.

- Fix/revert sort changes

- Remove unused fragment limit flag

- Add time to HistoricalLog events

- Remove reservation amount from RootAllocator constructor (since not allowed)

- Fix concurrency issue where allocator is closing at same moment as incoming batch transfer, causing leaked memory and/or query failure.

- Add new AutoCloseables.close(Iterable<AutoCloseable>)

- Remove extraneous DataResponseHandler and Impl (and update TestBitRpc to use smarter mock of FragmentManager)

- Remove the concept of poison pill record batches, using instead FragmentContext.isOverMemoryLimit()

- Update incoming data batches so that they are transferred under protection of a close lock

- Improve field names in IncomingBuffers and move synchronization to collectors as opposed to IncomingBuffers (also change decrementing to decrementToZero rather than two part check).

This closes #238.

    • -0
    • +43
    ./AutoCloseableLock.java
  1. … 119 more files in changeset.
DRILL-2755: Use and handle InterruptedException during query processing.

- Interrupt FragmentExecutor thread as part of FragmentExecutor.cancel()

- Handle InterruptedException in ExternalSortBatch.newSV2(). If the fragment status says

should not continue, then throw the InterruptedException to caller which returns IterOutcome.STOP

- Add comments reg not handling of InterruptedException in SendingAccountor.waitForSendComplete()

- Handle InterruptedException in OrderedPartitionRecordBatch.getPartitionVectors()

If interrupted in Thread.sleep calls and fragment status says should not run, then

return IterOutcome.STOP downstream.

- Interrupt partitioner threads if PartitionerRecordBatch is interrupted while waiting for

partitioner threads to complete.

- Preserve interrupt status if not handled

- Handle null RecordBatches returned by RawBatchBuffer.getNext() in MergingRecordBatch.buildSchema()

- Change timeout in Foreman to be proportional to the number of intermediate fragments sent instead

of hard coded limit of 90s.

- Change TimedRunnable to enforce a timeout of 15s per runnable.

Total timeout is (5s * numOfRunnableTasks) / parallelism.

- Add unit tests

* Testing cancelling a query interrupts the query fragments which are currently blocked

* Testing interrupting the partitioner sender which in turn interrupts its helper threads

* Testing TimedRunanble enforeces timeout for the whole task list.

  1. … 28 more files in changeset.
DRILL-2762: Update Fragment state reporting and error collection

DeferredException

- Add new throwAndClear operation on to allow checking for exceptions preClose in FragmentContext

- Add new getAndClear operation

BufferManager

- Ensure close() can be called multiple times by clearing managed buffer list on close().

FragmentContext/FragmentExecutor

- Update FragmentContext to have a preClose so that we can check closure state before doing final close.

- Update so that there is only a single state maintained between FragmentContext and FragmentExecutor

- Clean up FragmentExecutor run() method to better manage error states and have only single terminal point (avoiding multiple messages to Foreman).

- Add new CANCELLATION_REQUESTED state for FragmentState.

- Move all users of isCancelled or isFailed in main code to use shouldContinue()

- Update receivingFragmentFinished message to not cancel fragment (only inform root operator of cancellation)

WorkManager Updates

- Add new afterExecute command to the WorkManager ExecutorService so that we get log entries if a thread leaks an exception. (Otherwise logs don't show these exceptions and they only go to standard out.)

Profile Page

- Update profile page to show last update and last progress.

- Change durations to non-time presentation

Foreman/QueryManager

- Extract listenable interfaces into anonymous inner classes from body of Foreman

QueryManager

- Update QueryManager to track completed nodes rather than completed fragments using NodeTracker

- Update DrillbitStatusListener to decrement expected completion messages on Nodes that have died to avoid query hang when a node dies

FragmentData/MinorFragmentProfile

- Add ability to track last status update as well as last time fragment made progress

AbstractRecordBatch

- Update awareness of current cancellation state to avoid cancellation delays

Misc. Other changes

- Move ByteCode optimization code to only record assembly and code as trace messages

- Update SimpleRootExec to create fake ExecutorState to make existing tests work.

- Update sort to exit prematurely in the case that the fragment was asked to cancel.

- Add finals to all edited files.

- Modify control handler and FragmentManager to directly support receivingFragmentFinished

- Update receiver propagation message to avoid premature removal of fragment manager

- Update UserException.Builder to log a message if we're creating a new UserException (ERROR for System, INFO otherwise).

- Update Profile pages to use min and max instead of sorts.

    • -0
    • +89
    ./ExtendedLatch.java
  1. … 44 more files in changeset.