Commit Graph

1642 Commits

Author SHA1 Message Date
Vishesh Garg 7e35e50052
Fix issues with MSQ Compaction (#17250)
The patch makes the following changes:
1. Fixes a bug causing compaction to fail on array, complex, and other non-primitive-type columns
2. Updates compaction status check to be conscious of partition dimensions when comparing dimension ordering.
3. Ensures only string columns are specified as partition dimensions
4. Ensures `rollup` is true if and only if metricsSpec is non-empty
5. Ensures disjoint intervals aren't submitted for compaction
6. Adds `compactionReason` to compaction task context.
2024-10-06 21:48:26 +05:30
Clint Wylie 0bd13bcd51
Projections prototype (#17214) 2024-10-05 04:38:57 -07:00
Clint Wylie 04fe56835d
add druid.expressions.allowVectorizeFallback and default to false (#17248)
changes:

adds ExpressionProcessing.allowVectorizeFallback() and ExpressionProcessingConfig.allowVectorizeFallback(), defaulting to false until few remaining bugs can be fixed (mostly complex types and some odd interactions with mixed types)
add cannotVectorizeUnlessFallback functions to make it easy to toggle the default of this config, and easy to know what to delete when we remove it in the future
2024-10-05 12:42:42 +05:30
Gian Merlino d1709a329f
Dart: Skip final getCounters, postFinish to idle historicals. (#17255)
In a Dart query, all Historicals are given worker IDs, but not all of them
are going to actually be started or receive work orders.

Attempting to send a getCounters or postFinish command to a worker that
never received a work order is not only wasteful, but it causes errors due
to the workers not knowing about that query ID.
2024-10-04 23:05:21 -07:00
Shivam Garg 93b5a8326b
Upgrade commons-io to 2.17.0 (#17227) 2024-10-04 09:56:56 +02:00
Gian Merlino fc00664760
KafkaInputFormat: Fix handling of CSV/TSV keyFormat. (#17226)
* KafkaInputFormat: Fix handling of CSV/TSV keyFormat.

Follow-up to #16630, which fixed a similar issue for the valueFormat.

* Simplify.
2024-10-03 13:05:09 -07:00
Gian Merlino db7cc4634c
Dart: Smoother handling of stage early-exit. (#17228)
Stages can be instructed to exit before they finish, especially when a
downstream stage includes a "LIMIT". This patch has improvements related
to early-exiting stages.

Bug fix:

- WorkerStageKernel: Don't allow fail() to set an exception if the stage is
  already in a terminal state (FINISHED or FAILED). If fail() is called while
  in a terminal state, log the exception, then throw it away. If it's a
  cancellation exception, don't even log it. This fixes a bug where a stage
  that exited early could transition to FINISHED and then to FAILED, causing
  the overall query to fail.

Performance:

- DartWorkerManager previously sent stopWorker commands to workers
  even when "interrupt" was false. Now it only sends those commands when
  "interrupt" is true. The method javadoc already claimed this is what the
  method did, but the implementation did not match the javadoc. This reduces
  the number of RPCs by 1 per worker per query.

Quieter logging:

- In ReadableByteChunksFrameChannel, skip logging exception from setError if
  the channel has been closed. Channels are closed when readers are done with
  them, so at that point, we wouldn't be interested in the errors.

- In RunWorkOrder, skip calling notifyListener on failure of the main work,
  in the case when stop() has already been called. The stop() method will
  set its own error using CanceledFault. This enables callers to detect
  when a stage was canceled vs. failed for some other reason.

- In WorkerStageKernel, skip logging cancellation errors in fail(). This is
  made possible by the previous change in RunWorkOrder.
2024-10-03 20:09:02 +05:30
Gian Merlino 316f8c81d2
RunWorkOrder: Account for two simultaneous statistics collectors. (#17216)
* RunWorkOrder: Account for two simultaneous statistics collectors.

As a follow up to #17057, divide the amount of partitionStatsMemory
by two, to account for the fact that there are at some times going to
be two copies of the full collector. First there will be one for processors
and one for the accumulated collector. Then, after the processor ones are
GCed, a snapshot of the accumulated collector will be created.

Also includes an optimization to "addAll" for the two KeyCollectors,
for the case where we're adding into an empty collector. This is always
going to happen once per stage due to the "withAccumulation" call.

* Fix missing variable.

* Don't divide by numProcessingThreads twice.

* Fix test.
2024-10-03 16:25:01 +05:30
Akshat Jain edc235cfe1
WindowOperatorQueryFrameProcessor: Avoid unnecessary re-runs of runIncrementally() (#17211) 2024-10-03 15:33:50 +05:30
Akshat Jain 135ca8f6a7
WindowOperatorQueryFrameProcessor: Fix frame writer capacity issues + adhere to FrameProcessor's contract (#17209)
This PR fixes the above issue by maintaining the state of last rowId flushed to output channel, and triggering another iteration of runIncrementally() method if frame writer has rows pending flush to the output channel.

The above is done keeping in mind FrameProcessor's contract which enforces that we should write only a single frame to each output channel in any given iteration of runIncrementally().
2024-10-03 10:39:22 +05:30
Gian Merlino fbc1221837
DartTableInputSpecSlicer: Fix for TLS workers. (#17224)
We should use getHost(), which returns TLS if configured or plaintext
otherwise. getHostAndPort() returns plaintext only.
2024-10-03 11:01:11 +08:00
Gian Merlino 878adff9aa
MSQ profile for Brokers and Historicals. (#17140)
This patch adds a profile of MSQ named "Dart" that runs on Brokers and
Historicals, and which is compatible with the standard SQL query API.
For more high-level description, and notes on future work, refer to #17139.

This patch contains the following changes, grouped into packages.

Controller (org.apache.druid.msq.dart.controller):

The controller runs on Brokers. Main classes are,

- DartSqlResource, which serves /druid/v2/sql/dart/.
- DartSqlEngine and DartQueryMaker, the entry points from SQL that actually
  run the MSQ controller code.
- DartControllerContext, which configures the MSQ controller.
- DartMessageRelays, which sets up relays (see "message relays" below) to read
  messages from workers' DartControllerClients.
- DartTableInputSpecSlicer, which assigns work based on a TimelineServerView.

Worker (org.apache.druid.msq.dart.worker)

The worker runs on Historicals. Main classes are,

- DartWorkerResource, which supplies the regular MSQ WorkerResource, plus
  Dart-specific APIs.
- DartWorkerRunner, which runs MSQ worker code.
- DartWorkerContext, which configures the MSQ worker.
- DartProcessingBuffersProvider, which provides processing buffers from
  sliced-up merge buffers.
- DartDataSegmentProvider, which provides segments from the Historical's
  local cache.

Message relays (org.apache.druid.messages):

To avoid the need for Historicals to contact Brokers during a query, which
would create opportunities for queries to get stuck, all connections are
opened from Broker to Historical. This is made possible by a message relay
system, where the relay server (worker) has an outbox of messages.

The relay client (controller) connects to the outbox and retrieves messages.
Code for this system lives in the "server" package to keep it separate from
the MSQ extension and make it easier to maintain. The worker-to-controller
ControllerClient is implemented using message relays.

Other changes:

- Controller: Added the method "hasWorker". Used by the ControllerMessageListener
  to notify the appropriate controllers when a worker fails.
- WorkerResource: No longer tries to respond more than once in the
  "httpGetChannelData" API. This comes up when a response due to resolved future
  is ready at about the same time as a timeout occurs.
- MSQTaskQueryMaker: Refactor to separate out some useful functions for reuse
  in DartQueryMaker.
- SqlEngine: Add "queryContext" to "resultTypeForSelect" and "resultTypeForInsert".
  This allows the DartSqlEngine to modify result format based on whether a "fullReport"
  context parameter is set.
- LimitedOutputStream: New utility class. Used when in "fullReport" mode.
- TimelineServerView: Add getDruidServerMetadata as a performance optimization.
- CliHistorical: Add SegmentWrangler, so it can query inline data, lookups, etc.
- ServiceLocation: Add "fromUri" method, relocating some code from ServiceClientImpl.
- FixedServiceLocator: New locator for a fixed set of service locations. Useful for
  URI locations.
2024-10-01 14:38:55 -07:00
Kashif Faraz 28fead58b4
MSQ: Use task context flag useConcurrentLocks to determine task lock type (#17193) 2024-09-30 21:15:25 +05:30
Adarsh Sanjeev 95fb813b6d
Fix bug with while adding DistinctSketches (#17184)
* Fix bug with checking the incorrect key

* Add tests
2024-09-30 11:35:32 +05:30
Shivam Garg ab361747a8
Migrated commons-lang usages to commons-lang3 (#17156) 2024-09-28 10:28:11 +02:00
Gian Merlino dc223f22db
SQL: Use regular filters for time filtering in subqueries. (#17173)
* SQL: Use regular filters for time filtering in subqueries.

Using the "intervals" feature on subqueries, or any non-table, should be
avoided because it isn't a meaningful optimization in those cases, and
it's simpler for runtime implementations if they can assume all filters
are located in the regular filter object.

Two changes:

1) Fix the logic in DruidQuery.canUseIntervalFiltering. It was intended
   to return false for QueryDataSource, but actually returned true.

2) Add a validation to ScanQueryFrameProcessor to ensure that when running
   on an input channel (which would include any subquery), the query has
   "intervals" set to ONLY_ETERNITY.

Prior to this patch, the new test case in testTimeFilterOnSubquery would
throw a "Can only handle a single interval" error in the native engine,
and "QueryNotSupported" in the MSQ engine.

* Mark new case as having extra columns in decoupled mode.

* Adjust test.
2024-09-27 10:32:30 +05:30
Clint Wylie 6ee9e42f03
fix issue with ScanQueryFrameProcessor cursor build not adjusting intervals (#17168)
* fix issue with ScanQueryFrameProcessor cursor build not adjusting intervals

* all hail the robot overlords

* style bot
2024-09-26 21:09:14 +05:30
Clint Wylie d77637344d
log.warn anytime a column is relying on ArrayIngestMode.MVD (#17164)
* log.warn anytime a column is relying on ArrayIngestMode.MVD
2024-09-26 13:44:37 +05:30
Gian Merlino 2e531bb724
ScanQueryFrameProcessor: Close CursorHolders as we go along. (#17152)
* ScanQueryFrameProcessor: Close CursorHolders as we go along.

The change in #16533 added CursorHolders to the processor-level Closer.
This is problematic when running on an input channel: it means we started
keeping around all CursorHolders for all frames we process and closing them
when the channel is complete, rather than closing them as we go along.

This patch restores the prior behavior, where resources are closed as we go.

* Fix other call sites.

* Fix reference.

* Improvements.
2024-09-25 15:51:52 -07:00
Akshat Jain 446ffc325c
MSQ window functions: Fix boost column not being written to the frame in window stage (#17141)
In WindowOperatorQueryFrameProcessor, currently we were not adding the partition boost column to the ColumnSelectorFactory used to create the FrameWriter. Because of this, the boost column was not being written to the frame.
2024-09-25 09:50:42 +05:30
Adarsh Sanjeev 464b153850
Make sketch encoding configurable (#17086)
Makes sketch encoding in MSQ configurable by the user. This would allow a user to configure the sketch encoding method for a specific query.

The default is octet stream encoding.
2024-09-25 09:15:32 +05:30
George Shiqi Wu d1bfabbf4d
inter-Extension dependency support (#16973)
* update docs for kafka lookup extension to specify correct extension ordering

* fix first line

* test with extension dependencies

* save work on dependency management

* working dependency graph

* working pull

* fix style

* fix style

* remove name

* load extension dependencies recursively

* generate depenencies on classloader creation

* add check for circular dependencies

* fix style

* revert style changes

* remove mutable class loader

* clean up class heirarchy

* extensions loader test working

* add unit tests

* pr comments

* fix unit tests
2024-09-24 14:17:33 -04:00
zachjsh ba8245f114
Properly url encode schema and table names when syncing catalog (#17131)
* SQL syntax error should target USER persona

* * revert change to queryHandler and related tests, based on review comments

* * add test

* Properly encode schema and table names when syncing catalog

Previously table names and schema names were not being properly url
encododed when requested. Now they are.
2024-09-23 23:23:26 -04:00
Abhishek Radhakrishnan 37a2a12d79
rerwrite node so dynamic parameter applies to ingest node as well. (#17126) 2024-09-23 12:49:46 -07:00
Sree Charan Manamala 67d361c9bf
Window Functions : Remove enable windowing flag (#17087) 2024-09-23 08:24:26 +02:00
Akshat Jain 40414cfe78
MSQ window functions: Reject MVDs during window processing (#17036)
* MSQ window functions: Reject MVDs during window processing

* MSQ window functions: Reject MVDs during window processing

* Remove parameterization from MSQWindowTest
2024-09-23 11:39:35 +05:30
Zoltan Haindrich 2eee470f6e
Support explain in decoupled planning and log native plan consistently with DruidHook (#17101)
* enables to use DruidHook for native plan logging
* qudiem tests doesn't necessarily need to run the query to get an explain - this helps during development as if there is a runtime issue it could still be explained in the test
2024-09-20 10:53:43 +02:00
Abhishek Radhakrishnan 635e418131
Support to parse numbers in text-based input formats (#17082)
Text-based input formats like csv and tsv currently parse inputs only as strings, following the RFC4180Parser spec).
To workaround this, the web-console and other tools need to further inspect the sample data returned to sample data returned by the Druid sampler API to parse them as numbers. 

This patch introduces a new optional config, tryParseNumbers, for the csv and tsv input formats. If enabled, any numbers present in the input will be parsed in the following manner -- long data type for integer types and double for floating-point numbers, and if parsing fails for whatever reason, the input is treated as a string. By default, this configuration is set to false, so numeric strings will be treated as strings.
2024-09-19 13:21:18 -07:00
Gian Merlino 3d45f9829c
Use the whole frame when writing rows. (#17094)
* Use the whole frame when writing rows.

This patch makes the following adjustments to enable writing larger
single rows to frames:

1) RowBasedFrameWriter: Max out allocation size on the final doubling.
   i.e., if the final allocation "naturally" would be 1 MiB but the
   max frame size is 900 KiB, use 900 KiB rather than failing the 1 MiB
   allocation.

2) AppendableMemory: In reserveAdditional, release the last block if it
   is empty. This eliminates waste when a frame writer uses a
   successive-doubling approach to find the right allocation size.

3) ArenaMemoryAllocator: Reclaim memory from the last allocation when
   the last allocation is closed.

Prior to these changes, a single row could be much smaller than the
frame size and still fail to be added to the frame.

* Style.

* Fix test.
2024-09-19 00:42:03 -07:00
Gian Merlino ca0cb64ee8
MSQ: Fix calculation of suggested memory in WorkerMemoryParameters. (#17108)
The "suggested server memory" figure needs to take into account
maxConcurrentStages. The fix here does not affect the main memory
calculations, but it does affect the accuracy of error messages.
2024-09-18 16:08:35 -07:00
Adarsh Sanjeev 2f50138af9
Modify DataSegmentProvider to also return DataSegment (#17021)
Currently, TaskDataSegmentProvider fetches the DataSegment from the Coordinator while loading the segment, but just discards it later. This PR refactors this to also return the DataSegment so that it can be used by workers without a separate fetch.
2024-09-18 11:20:20 +05:30
Gian Merlino c5968aa463
MSQ: Add QueryKitSpec to encapsulate QueryKit params. (#17077)
* MSQ: Add QueryKitSpec to encapsulate QueryKit params.

This patch introduces QueryKitSpec, an object that encapsulates the
parameters to makeQueryDefinition that are consistent from call to
call. This simplifies things because we avoid passing around all the
components individually.

This patch also splits "maxWorkerCount" into "maxLeafWorkerCount" and
"maxNonLeafWorkerCount", which apply to leaf stages (no other stages as
inputs) and nonleaf stages respectively.

Finally, this patch also rovides a way for ControllerContext to supply a
QueryKitSpec to its liking. It is expected that this will be used by
controllers of quick interactive queries to set maxNonLeafWorkerCount = 1,
which will generate fanning-in query plans.

* Fix javadoc.
2024-09-17 13:37:14 -07:00
Gian Merlino 50503fe0ef
MSQ: Properly report errors that occur when starting up RunWorkOrder. (#17069)
* MSQ: Properly report errors that occur when starting up RunWorkOrder.

In #17046, an exception thrown by RunWorkOrder#startAsync would be ignored
and replaced with a generic CanceledFault. This patch fixes it by retaining
the original error.
2024-09-17 20:32:02 +05:30
Gian Merlino 8af9b4729f
TableInputSpecSlicer changes to support running on Brokers. (#17074)
* TableInputSpecSlicer changes to support running on Brokers.

Changes:

1) Rename TableInputSpecSlicer to IndexerTableInputSpecSlicer, in anticipation
   of a new implementation being added for controllers running on Brokers.

2) Allow the context to use the WorkerManager to build the TableInputSpecSlicer,
   in anticipation of Brokers wanting to use this to assign segments to servers
   that are already serving those segments.

3) Remove unused DataSegmentTimelineView interface.

4) Add additional javadoc to DataSegmentProvider.

* Style.
2024-09-17 03:51:18 -07:00
Gian Merlino c56e23ec37
Remove workerId parameter from postWorkerError. (#17072)
* Remove workerId parameter from postWorkerError.

It was redundant to MSQErrorReport#getTaskId.

* Fix javadoc.
2024-09-17 01:37:46 -07:00
Gian Merlino 2e4d596d82
MSQ: Include worker context maps in WorkOrders. (#17076)
* MSQ: Include worker context maps in WorkOrders.

This provides a mechanism to send contexts to workers in long-lived,
shared JVMs that are not part of the task system.

* Style, coverage.
2024-09-17 01:37:21 -07:00
Gian Merlino 9696f0b37c
Remove close method on MSQWarningReportPublisher. (#17071)
It didn't do anything and also wasn't called.
2024-09-16 18:38:36 +05:30
Gian Merlino 4d8015578d
Remove unused WorkerManagerClient interface. (#17073) 2024-09-16 18:00:47 +05:30
Gian Merlino 8630974157
MSQ: Wake up the main controller thread on workerError. (#17075)
This isn't necessary when using MSQWorkerTaskLauncher as the WorkerManager
implementation, because in that case, task failure also wakes up the
main thread. However, when using workers that are not task-based, we don't
want to rely on the WorkerManager for this.
2024-09-16 18:00:09 +05:30
Gian Merlino a8d15182a3
Additional tests for ChannelStageOutputReader. (#17050)
The existing tests are moved into a "WithMaximalBuffering" subclass,
and a new "WithMinimalBuffering" subclass is added to test cases
where only a single frame is buffered.
2024-09-16 12:22:32 +05:30
Gian Merlino 33cb563ff9
Move TerminalStageSpecFactory packages. (#17049)
* Move TerminalStageSpecFactory packages.

These packages are moved from the "guice" package to the "indexing.destination"
package. They make more sense here, since "guice" is reserved for Guice modules,
annotations, and providers.

* Rearrange imports.
2024-09-16 11:33:26 +05:30
Clint Wylie aa6336c5cf
add DataSchema.Builder to tidy stuff up a bit (#17065)
* add DataSchema.Builder to tidy stuff up a bit

* fixes

* fixes

* more style fixes

* review stuff
2024-09-15 11:18:34 -07:00
Akshat Jain 6ed8632420
Handle memory leaks from Mockito inline mocks (#17070) 2024-09-15 11:17:25 -07:00
Gian Merlino 4dc5942dab
BaseWorkerClientImpl: Don't attempt to recover from a closed channel. (#17052)
* BaseWorkerClientImpl: Don't attempt to recover from a closed channel.

This patch introduces an exception type "ChannelClosedForWritesException",
which allows the BaseWorkerClientImpl to avoid retrying when the local
channel has been closed. This can happen in cases of cancellation.

* Add some test coverage.

* wip

* Add test coverage.

* Style.
2024-09-15 02:10:58 -07:00
Gian Merlino 6fac267f17
MSQ: Improved worker cancellation. (#17046)
* MSQ: Improved worker cancellation.

Four changes:

1) FrameProcessorExecutor now requires that cancellationIds be registered
   with "registerCancellationId" prior to being used in "runFully" or "runAllFully".

2) FrameProcessorExecutor gains an "asExecutor" method, which allows that
   executor to be used as an executor for future callbacks in such a way
   that respects cancellationId.

3) RunWorkOrder gains a "stop" method, which cancels the current
   cancellationId and closes the current FrameContext. It blocks until
   both operations are complete.

4) Fixes a bug in RunAllFullyWidget where "processorManager.result()" was
   called outside "runAllFullyLock", which could cause it to be called
   out-of-order with "cleanup()" in case of cancellation or other error.

Together, these changes help ensure cancellation does not have races.
Once "cancel" is called for a given cancellationId, all existing processors
and running callbacks are canceled and exit in an orderly manner. Future
processors and callbacks with the same cancellationId are rejected
before being executed.

* Fix test.

* Use execute, which doesn't return, to avoid errorprone complaints.

* Fix some style stuff.

* Further enhancements.

* Fix style.
2024-09-15 01:22:28 -07:00
Gian Merlino a276871dd0
Fix call to MemoryIntrospector in IndexerControllerContext. (#17066)
This was a logical conflict between #17057 and #17048.
2024-09-14 18:10:56 -07:00
Gian Merlino fd6706cd6a
MSQ: Rework memory management. (#17057)
* MSQ: Rework memory management.

This patch reworks memory management to better support multi-threaded
workers running in shared JVMs. There are two main changes.

First, processing buffers and threads are moved from a per-JVM model to
a per-worker model. This enables queries to hold processing buffers
without blocking other concurrently-running queries. Changes:

- Introduce ProcessingBuffersSet and ProcessingBuffers to hold the
  per-worker and per-work-order processing buffers (respectively). On Peons,
  this is the JVM-wide processing pool. On Indexers, this is a per-worker
  pool of on-heap buffers. (This change fixes a bug on Indexers where
  excessive processing buffers could be used if MSQ tasks ran concurrently
  with realtime tasks.)

- Add "bufferPool" argument to GroupingEngine#process so a per-worker pool
  can be passed in.

- Add "druid.msq.task.memory.maxThreads" property, which controls the
  maximum number of processing threads to use per task. This allows usage of
  multiple processing buffers per task if admins desire.

- IndexerWorkerContext acquires processingBuffers when creating the FrameContext
  for a work order, and releases them when closing the FrameContext.

- Add "usesProcessingBuffers()" to FrameProcessorFactory so workers know
  how many sets of processing buffers are needed to run a given query.

Second, adjustments to how WorkerMemoryParameters slices up bundles, to
favor more memory for sorting and segment generation. Changes:

- Instead of using same-sized bundles for processing and for sorting,
  workers now use minimally-sized processing bundles (just enough to read
  inputs plus a little overhead). The rest is devoted to broadcast data
  buffering, sorting, and segment-building.

- Segment-building is now limited to 1 concurrent segment per work order.
  This allows each segment-building action to use more memory. Note that
  segment-building is internally multi-threaded to a degree. (Build and
  persist can run concurrently.)

- Simplify frame size calculations by removing the distinction between
  "standard" and "large" frames. The new default frame size is the same
  as the old "standard" frames, 1 MB. The original goal of of the large
  frames was to reduce the number of temporary files during sorting, but
  I think we can achieve the same thing by simply merging a larger number
  of standard frames at once.

- Remove the small worker adjustment that was added in #14117 to account
  for an extra frame involved in writing to durable storage. Instead,
  account for the extra frame whenever we are actually using durable storage.

- Cap super-sorter parallelism using the number of output partitions, rather
  than using a hard coded cap at 4. Note that in practice, so far, this cap
  has not been relevant for tasks because they have only been using a single
  processing thread anyway.

* Remove unused import.

* Fix errorprone annotation.

* Fixes for javadocs and inspections.

* Additional test coverage.

* Fix test.
2024-09-14 15:35:21 -07:00
Gian Merlino d3f86baff9
Add "targetPartitionsPerWorker" setting for MSQ. (#17048)
As we move towards multi-threaded MSQ workers, it helps for parallelism
to generate more than one partition per worker. That way, we can fully
utilize all worker threads throughout all stages.

The default value is the number of processing threads. Currently, this
is hard-coded to 1 for peons, but that is expected to change in the future.
2024-09-13 16:01:18 -07:00
Gian Merlino 654e0b444b
MSQ: Fix two issues with phase transitions. (#17053)
1) ControllerQueryKernel: Update readyToReadResults to acknowledge that sorting stages can
   go directly from READING_INPUT to RESULTS_READY.

2) WorkerStageKernel: Ignore RESULTS_COMPLETE if work is already finished, which can happen
   if the transition to FINISHED comes early due to a downstream LIMIT.
2024-09-13 15:59:41 -07:00
Gian Merlino 99e8f664a9
Add "includeAllCounters()" to WorkerContext. (#17047)
This removes the need to read it from the query context.
2024-09-13 15:47:51 -07:00