You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@kudu.apache.org by fa...@tf-bic.sk, fa...@tf-bic.sk on 2018/08/03 09:12:01 UTC

Yes Balazs,  
 that was my conclusion too, after the Exchange some of the nodes had zero row to send to Kudu and some of them had 30% of the whole table.  Now the question is whats wrong with Impala, why is this exchange hurting the performance? I used /*+noshuffle*/ as a workaround

Tomas



On 2018/08/01 15:19:02, Balázs Jeszenszky <ba...@cloudera.com> wrote: 
> The worst offender is Kudu sink, e.g:
>  KuduTableSink:(Total: 2m17s, non-child: 2m17s, % non-child: 100.00%)
>            - KuduApplyTimer: 1m57s
> 
> There's big skew in fragment F01 (completion times: min:19s223ms
> max:2m45s), the two quick instances have zero as KuduApplyTimer, and the
> 19s is due to sorting.
> 
> 
> On Wed, Aug 1, 2018 at 4:16 PM, farkas@tf-bic.sk <fa...@tf-bic.sk> wrote:
> 
> >
> >
> > On 2018/07/31 20:59:28, Mike Percy <mp...@apache.org> wrote:
> > > Can you post a query profile from Impala for one of the slow insert jobs?
> > >
> > > Mike
> > >
> > > On Tue, Jul 31, 2018 at 12:56 PM Tomas Farkas <fa...@tf-bic.sk> wrote:
> > >
> > > > Hi,
> > > > wanted share with you the preliminary results of my Kudu testing on AWS
> > > > Created a set of performance tests for evaluation of different instance
> > > > types in AWS and different configurations (Kudu separated from Impala,
> > Kudu
> > > > and Impala on the same nodes); different drive (st1 and gp2) settings
> > and
> > > > here my results:
> > > >
> > > > I was quite dissapointed by the inserts in Step3 see attached sqls,
> > > >
> > > > Any hints, ideas, why this does not scale?
> > > > Thanks
> > > >
> > > >
> > > >
> > >
> >
> >
> > Impala profile:
> >
> > Query (id=7e441f58868a8585:7452021000000000):
> >   Summary:
> >     Session ID: be47b44e2dde497b:fd4a9bd251e2c897
> >     Session Type: BEESWAX
> >     Start Time: 2018-08-01 15:40:30.976897000
> >     End Time: 2018-08-01 15:43:17.236925000
> >     Query Type: DML
> >     Query State: FINISHED
> >     Query Status: OK
> >     Impala Version: impalad version 2.12.0-cdh5.15.0 RELEASE (build
> > 23f574543323301846b41fa5433690df32efe085)
> >     User: impala@KUDUTEST.DOMAIN.LOCAL
> >     Connected User: impala@KUDUTEST.DOMAIN.LOCAL
> >     Delegated User:
> >     Network Address: 10.197.7.176:45846
> >     Default Db: default
> >     Sql Statement: insert into test.test_kudu_range
> > select
> >   id,
> >   name,
> >   ban,
> >   2018,
> >   iban,
> >   eventdate,
> >   eventid,
> >   city,
> >   state,
> >   lat,
> >   lon
> > from test.test_prq
> >     Coordinator: ip-10-197-10-88.eu-west-1.compute.internal:22000
> >     Query Options (set by configuration):
> >     Query Options (set by configuration and planner): MT_DOP=0
> >     Plan:
> > ----------------
> > Max Per-Host Resource Reservation: Memory=4.00MB
> > Per-Host Resource Estimates: Memory=832.00MB
> >
> > F01:PLAN FRAGMENT [KUDU(KuduPartition(id, name, ban, 2018))] hosts=6
> > instances=6
> > |  Per-Host Resources: mem-estimate=128.00MB mem-reservation=4.00MB
> > INSERT INTO KUDU [test.test_kudu_range]
> > |  mem-estimate=0B mem-reservation=0B
> > |
> > 02:PARTIAL SORT
> > |  order by: KuduPartition(id, name, ban, 2018) ASC NULLS LAST, id ASC
> > NULLS LAST, name ASC NULLS LAST, ban ASC NULLS LAST
> > |  materialized: KuduPartition(id, name, ban, 2018)
> > |  mem-estimate=128.00MB mem-reservation=4.00MB spill-buffer=2.00MB
> > |  tuple-ids=2 row-size=248B cardinality=45000000
> > |
> > 01:EXCHANGE [KUDU(KuduPartition(id, name, ban, 2018))]
> > |  mem-estimate=0B mem-reservation=0B
> > |  tuple-ids=0 row-size=244B cardinality=45000000
> > |
> > F00:PLAN FRAGMENT [RANDOM] hosts=6 instances=6
> > Per-Host Resources: mem-estimate=704.00MB mem-reservation=0B
> > 00:SCAN HDFS [test.test_prq, RANDOM]
> >    partitions=1/1 files=24 size=4.93GB
> >    stored statistics:
> >      table: rows=45000000 size=4.93GB
> >      columns: all
> >    extrapolated-rows=disabled
> >    mem-estimate=704.00MB mem-reservation=0B
> >    tuple-ids=0 row-size=244B cardinality=45000000
> > ----------------
> >     Estimated Per-Host Mem: 872415232
> >     Per Host Min Reservation: ip-10-197-10-88.eu-west-1.compute.internal:22000(4.00
> > MB) ip-10-197-11-142.eu-west-1.compute.internal:22000(4.00 MB)
> > ip-10-197-29-94.eu-west-1.compute.internal:22000(4.00 MB)
> > ip-10-197-3-207.eu-west-1.compute.internal:22000(4.00 MB)
> > ip-10-197-30-21.eu-west-1.compute.internal:22000(4.00 MB)
> > ip-10-197-7-97.eu-west-1.compute.internal:22000(4.00 MB)
> >     Request Pool: root.impala
> >     Admission result: Admitted immediately
> >     ExecSummary:
> > Operator          #Hosts   Avg Time   Max Time   #Rows  Est. #Rows   Peak
> > Mem  Est. Peak Mem  Detail
> > ------------------------------------------------------------
> > ------------------------------------------------------------
> > ---------------
> > 02:PARTIAL SORT        6   12s495ms   26s075ms  45.00M      45.00M    3.49
> > GB      128.00 MB
> > 01:EXCHANGE            6  232.833ms  517.001ms  45.00M      45.00M   13.72
> > MB              0  KUDU(KuduPartition(id, name, ban, 2018))
> > 00:SCAN HDFS           6   96.500ms  117.000ms  45.00M      45.00M  774.11
> > MB      704.00 MB  test.test_prq
> >     Errors: Key already present in Kudu table
> > 'impala::test.test_kudu_range'. (1 of 15442 similar)
> >
> >     Query Compilation: 4.022ms
> >        - Metadata of all 2 tables cached: 380.433us (380.433us)
> >        - Analysis finished: 1.141ms (761.253us)
> >        - Value transfer graph computed: 1.254ms (113.129us)
> >        - Single node plan created: 1.780ms (525.298us)
> >        - Runtime filters computed: 1.819ms (39.444us)
> >        - Distributed plan created: 2.508ms (688.869us)
> >        - Lineage info computed: 2.678ms (169.662us)
> >        - Planning finished: 4.022ms (1.343ms)
> >     Query Timeline: 2m46s
> >        - Query submitted: 0.000ns (0.000ns)
> >        - Planning finished: 5.000ms (5.000ms)
> >        - Submit for admission: 6.000ms (1.000ms)
> >        - Completed admission: 6.000ms (0.000ns)
> >        - Ready to start on 6 backends: 6.000ms (0.000ns)
> >        - All 6 execution backends (12 fragment instances) started: 7.000ms
> > (1.000ms)
> >        - Released admission control resources: 2m45s (2m45s)
> >        - DML data written: 2m45s (1.000ms)
> >        - DML Metastore update finished: 2m45s (0.000ns)
> >        - Request finished: 2m45s (0.000ns)
> >        - Unregister query: 2m46s (431.003ms)
> >      - ComputeScanRangeAssignmentTimer: 1.000ms
> >   ImpalaServer:
> >      - ClientFetchWaitTimer: 431.003ms
> >      - MetastoreUpdateTimer: 0.000ns
> >      - RowMaterializationTimer: 0.000ns
> >   Execution Profile 7e441f58868a8585:7452021000000000:(Total: 2m45s,
> > non-child: 0.000ns, % non-child: 0.00%)
> >     Number of filters: 0
> >     Filter routing table:
> >  ID  Src. Node  Tgt. Node(s)  Target type  Partition filter  Pending
> > (Expected)  First arrived  Completed   Enabled
> > ------------------------------------------------------------
> > -------------------------------------------------------
> >
> >     Backend startup latencies: Count: 6, min / max: 0 / 1ms, 25th %-ile:
> > 1ms, 50th %-ile: 1ms, 75th %-ile: 1ms, 90th %-ile: 1ms, 95th %-ile: 1ms,
> > 99.9th %-ile: 1ms
> >     DML Stats:
> > Partition: Default
> > NumModifiedRows: 44984558
> > NumRowErrors: 15442
> >   BytesWritten: 0
> >     Per Node Peak Memory Usage: ip-10-197-10-88.eu-west-1.compute.internal:22000(704.86
> > MB) ip-10-197-11-142.eu-west-1.compute.internal:22000(3.52 GB)
> > ip-10-197-29-94.eu-west-1.compute.internal:22000(3.53 GB)
> > ip-10-197-3-207.eu-west-1.compute.internal:22000(720.28 MB)
> > ip-10-197-30-21.eu-west-1.compute.internal:22000(3.76 GB)
> > ip-10-197-7-97.eu-west-1.compute.internal:22000(675.40 MB)
> >      - FiltersReceived: 0 (0)
> >      - FinalizationTimer: 0.000ns
> >      - NumBackends: 6 (6)
> >      - NumFragmentInstances: 12 (12)
> >      - NumFragments: 2 (2)
> >     Averaged Fragment F01:(Total: 1m30s, non-child: 127.333ms, %
> > non-child: 0.14%)
> >       split sizes:  min: 0, max: 0, avg: 0, stddev: 0
> >       completion times: min:19s223ms  max:2m45s  mean: 1m30s  stddev:1m11s
> >       execution rates: min:0.00 /sec  max:0.00 /sec  mean:0.00 /sec
> > stddev:0.00 /sec
> >       num instances: 6
> >        - AverageThreadTokens: 1.00
> >        - BloomFilterBytes: 0
> >        - PeakMemoryUsage: 1.78 GB (1906470029)
> >        - PeakReservation: 1.75 GB (1878349141)
> >        - PeakUsedReservation: 0
> >        - PerHostPeakMemUsage: 2.14 GB (2301629572)
> >        - RowsProduced: 7.50M (7500000)
> >        - TotalNetworkReceiveTime: 11s040ms
> >        - TotalNetworkSendTime: 0.000ns
> >        - TotalStorageWaitTime: 0.000ns
> >        - TotalThreadsInvoluntaryContextSwitches: 755 (755)
> >        - TotalThreadsTotalWallClockTime: 1m30s
> >          - TotalThreadsSysTime: 4s416ms
> >          - TotalThreadsUserTime: 37s775ms
> >        - TotalThreadsVoluntaryContextSwitches: 5.94K (5937)
> >       Buffer pool:
> >          - AllocTime: 0.000ns
> >          - CumulativeAllocationBytes: 0
> >          - CumulativeAllocations: 0 (0)
> >          - PeakReservation: 0
> >          - PeakUnpinnedBytes: 0
> >          - PeakUsedReservation: 0
> >          - ReadIoBytes: 0
> >          - ReadIoOps: 0 (0)
> >          - ReadIoWaitTime: 0.000ns
> >          - ReservationLimit: 0
> >          - WriteIoBytes: 0
> >          - WriteIoOps: 0 (0)
> >          - WriteIoWaitTime: 0.000ns
> >       Fragment Instance Lifecycle Timings:
> >          - ExecTime: 1m21s
> >            - ExecTreeExecTime: 14s101ms
> >          - OpenTime: 9s615ms
> >            - ExecTreeOpenTime: 9s571ms
> >          - PrepareTime: 49.500ms
> >            - ExecTreePrepareTime: 0.000ns
> >       KuduTableSink:(Total: 1m7s, non-child: 1m7s, % non-child: 100.00%)
> >          - KuduApplyTimer: 57s540ms
> >          - NumRowErrors: 2.57K (2573)
> >          - PeakMemoryUsage: 20.00 MB (20971520)
> >          - RowsProcessedRate: 55.98 K/sec
> >          - TotalNumRows: 7.50M (7500000)
> >       SORT_NODE (id=2):(Total: 23s666ms, non-child: 12s495ms, % non-child:
> > 52.80%)
> >          - InMemorySortTime: 4s505ms
> >          - PeakMemoryUsage: 1.75 GB (1878365525)
> >          - RowsReturned: 7.50M (7500000)
> >          - RowsReturnedRate: 265.96 K/sec
> >          - RunsCreated: 1 (1)
> >          - SortDataSize: 1.74 GB (1873230714)
> >         Buffer pool:
> >            - AllocTime: 147.833ms
> >            - CumulativeAllocationBytes: 1.75 GB (1878349141)
> >            - CumulativeAllocations: 895 (895)
> >            - PeakReservation: 1.75 GB (1878349141)
> >            - PeakUnpinnedBytes: 0
> >            - PeakUsedReservation: 1.75 GB (1878349141)
> >            - ReadIoBytes: 0
> >            - ReadIoOps: 0 (0)
> >            - ReadIoWaitTime: 0.000ns
> >            - WriteIoBytes: 0
> >            - WriteIoOps: 0 (0)
> >            - WriteIoWaitTime: 0.000ns
> >       EXCHANGE_NODE (id=1):(Total: 11s171ms, non-child: 11s171ms, %
> > non-child: 100.00%)
> >          - ConvertRowBatchTime: 106.833ms
> >          - PeakMemoryUsage: 6.85 MB (7179227)
> >          - RowsReturned: 7.50M (7500000)
> >          - RowsReturnedRate: 2.42 M/sec
> >         Buffer pool:
> >            - AllocTime: 59.333ms
> >            - CumulativeAllocationBytes: 2.35 GB (2518056960)
> >            - CumulativeAllocations: 122.95K (122953)
> >            - PeakReservation: 6.79 MB (7124309)
> >            - PeakUnpinnedBytes: 0
> >            - PeakUsedReservation: 6.79 MB (7124309)
> >            - ReadIoBytes: 0
> >            - ReadIoOps: 0 (0)
> >            - ReadIoWaitTime: 0.000ns
> >            - WriteIoBytes: 0
> >            - WriteIoOps: 0 (0)
> >            - WriteIoWaitTime: 0.000ns
> >         Dequeue:
> >            - FirstBatchWaitTime: 9s570ms
> >            - TotalBytesDequeued: 1.77 GB (1903230714)
> >            - TotalGetBatchTime: 11s037ms
> >              - DataWaitTime: 10s938ms
> >         Enqueue:
> >            - DeserializeRowBatchTime: 1s764ms
> >            - TotalBatchesEnqueued: 61.48K (61476)
> >            - TotalBatchesReceived: 61.48K (61476)
> >            - TotalBytesReceived: 1.24 GB (1333898634)
> >            - TotalEarlySenders: 0 (0)
> >            - TotalEosReceived: 6 (6)
> >            - TotalHasDeferredRPCsTime: 1s897ms
> >            - TotalRPCsDeferred: 11.56K (11562)
> >       CodeGen:(Total: 44.000ms, non-child: 44.000ms, % non-child: 100.00%)
> >          - CodegenTime: 833.334us
> >          - CompileTime: 5.666ms
> >          - LoadTime: 0.000ns
> >          - ModuleBitcodeSize: 1.95 MB (2039944)
> >          - NumFunctions: 32 (32)
> >          - NumInstructions: 638 (638)
> >          - OptimizationTime: 24.666ms
> >          - PeakMemoryUsage: 319.00 KB (326656)
> >          - PrepareTime: 12.833ms
> >     Fragment F01:
> >       Instance 7e441f58868a8585:745202100000000a
> > (host=ip-10-197-29-94.eu-west-1.compute.internal:22000):(Total: 2m45s,
> > non-child: 130.000ms, % non-child: 0.08%)
> >         Fragment Instance Lifecycle Event Timeline: 2m45s
> >            - Prepare Finished: 53.000ms (53.000ms)
> >            - Open Finished: 108.000ms (55.000ms)
> >            - First Batch Produced: 28s518ms (28s410ms)
> >            - First Batch Sent: 28s522ms (4.000ms)
> >            - ExecInternal Finished: 2m45s (2m17s)
> >         MemoryUsage(4s000ms): 406.92 MB, 1.19 GB, 2.03 GB, 2.79 GB, 3.41
> > GB, 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB,
> > 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB,
> > 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB,
> > 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB,
> > 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB
> >         ThreadUsage(4s000ms): 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1,
> > 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1
> >          - AverageThreadTokens: 1.00
> >          - BloomFilterBytes: 0
> >          - PeakMemoryUsage: 3.53 GB (3787104739)
> >          - PeakReservation: 3.49 GB (3751804928)
> >          - PeakUsedReservation: 0
> >          - PerHostPeakMemUsage: 3.53 GB (3787104739)
> >          - RowsProduced: 15.00M (15003125)
> >          - TotalNetworkReceiveTime: 2s139ms
> >          - TotalNetworkSendTime: 0.000ns
> >          - TotalStorageWaitTime: 0.000ns
> >          - TotalThreadsInvoluntaryContextSwitches: 1.92K (1923)
> >          - TotalThreadsTotalWallClockTime: 2m45s
> >            - TotalThreadsSysTime: 8s959ms
> >            - TotalThreadsUserTime: 1m18s
> >          - TotalThreadsVoluntaryContextSwitches: 6.17K (6169)
> >         Buffer pool:
> >            - AllocTime: 0.000ns
> >            - CumulativeAllocationBytes: 0
> >            - CumulativeAllocations: 0 (0)
> >            - PeakReservation: 0
> >            - PeakUnpinnedBytes: 0
> >            - PeakUsedReservation: 0
> >            - ReadIoBytes: 0
> >            - ReadIoOps: 0 (0)
> >            - ReadIoWaitTime: 0.000ns
> >            - ReservationLimit: 0
> >            - WriteIoBytes: 0
> >            - WriteIoOps: 0 (0)
> >            - WriteIoWaitTime: 0.000ns
> >         Fragment Instance Lifecycle Timings:
> >            - ExecTime: 2m45s
> >              - ExecTreeExecTime: 28s483ms
> >            - OpenTime: 55.000ms
> >              - ExecTreeOpenTime: 7.000ms
> >            - PrepareTime: 52.000ms
> >              - ExecTreePrepareTime: 0.000ns
> >         KuduTableSink:(Total: 2m17s, non-child: 2m17s, % non-child:
> > 100.00%)
> >            - KuduApplyTimer: 1m57s
> >            - NumRowErrors: 5.16K (5159)
> >            - PeakMemoryUsage: 20.00 MB (20971520)
> >            - RowsProcessedRate: 109.38 K/sec
> >            - TotalNumRows: 15.00M (15003125)
> >         SORT_NODE (id=2):(Total: 28s481ms, non-child: 26s075ms, %
> > non-child: 91.55%)
> >           SortType: Partial
> >           ExecOption: Codegen Enabled
> >            - NumRowsPerRun: (Avg: 15.00M (15003125) ; Min: 15.00M
> > (15003125) ; Max: 15.00M (15003125) ; Number of samples: 1)
> >            - InMemorySortTime: 9s293ms
> >            - PeakMemoryUsage: 3.49 GB (3751821312)
> >            - RowsReturned: 15.00M (15003125)
> >            - RowsReturnedRate: 526.77 K/sec
> >            - RunsCreated: 1 (1)
> >            - SortDataSize: 3.49 GB (3747240938)
> >           Buffer pool:
> >              - AllocTime: 103.000ms
> >              - CumulativeAllocationBytes: 3.49 GB (3751804928)
> >              - CumulativeAllocations: 1.79K (1789)
> >              - PeakReservation: 3.49 GB (3751804928)
> >              - PeakUnpinnedBytes: 0
> >              - PeakUsedReservation: 3.49 GB (3751804928)
> >              - ReadIoBytes: 0
> >              - ReadIoOps: 0 (0)
> >              - ReadIoWaitTime: 0.000ns
> >              - WriteIoBytes: 0
> >              - WriteIoOps: 0 (0)
> >              - WriteIoWaitTime: 0.000ns
> >         EXCHANGE_NODE (id=1):(Total: 2s406ms, non-child: 517.001ms, %
> > non-child: 21.49%)
> >            - ConvertRowBatchTime: 225.000ms
> >            - PeakMemoryUsage: 13.72 MB (14390403)
> >            - RowsReturned: 15.00M (15003125)
> >            - RowsReturnedRate: 6.24 M/sec
> >           Buffer pool:
> >              - AllocTime: 124.000ms
> >              - CumulativeAllocationBytes: 4.69 GB (5037178880)
> >              - CumulativeAllocations: 245.96K (245960)
> >              - PeakReservation: 13.64 MB (14303232)
> >              - PeakUnpinnedBytes: 0
> >              - PeakUsedReservation: 13.64 MB (14303232)
> >              - ReadIoBytes: 0
> >              - ReadIoOps: 0 (0)
> >              - ReadIoWaitTime: 0.000ns
> >              - WriteIoBytes: 0
> >              - WriteIoOps: 0 (0)
> >              - WriteIoWaitTime: 0.000ns
> >           Dequeue:
> >             BytesDequeued(4s000ms): 489.70 MB, 1.28 GB, 2.12 GB, 2.89 GB,
> > 3.47 GB, 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB,
> > 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB,
> > 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB,
> > 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB,
> > 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB
> >              - FirstBatchWaitTime: 7.000ms
> >              - TotalBytesDequeued: 3.55 GB (3807253438)
> >              - TotalGetBatchTime: 2s134ms
> >                - DataWaitTime: 1s890ms
> >           Enqueue:
> >             BytesReceived(4s000ms): 347.95 MB, 917.64 MB, 1.49 GB, 2.03
> > GB, 2.43 GB, 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB,
> > 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB,
> > 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB,
> > 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB,
> > 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB
> >             DeferredQueueSize(4s000ms): 0, 1, 1, 3, 0, 0, 0, 0, 0, 0, 0,
> > 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
> > 0, 0, 0, 0, 0
> >              - DispatchTime: (Avg: 16.831us ; Min: 3.269us ; Max: 5.228ms
> > ; Number of samples: 122980)
> >              - DeserializeRowBatchTime: 3s661ms
> >              - TotalBatchesEnqueued: 122.98K (122980)
> >              - TotalBatchesReceived: 122.98K (122980)
> >              - TotalBytesReceived: 2.49 GB (2668357754)
> >              - TotalEarlySenders: 0 (0)
> >              - TotalEosReceived: 6 (6)
> >              - TotalHasDeferredRPCsTime: 6s631ms
> >              - TotalRPCsDeferred: 36.40K (36395)
> >         CodeGen:(Total: 46.000ms, non-child: 46.000ms, % non-child:
> > 100.00%)
> >            - CodegenTime: 1.000ms
> >            - CompileTime: 8.000ms
> >            - LoadTime: 0.000ns
> >            - ModuleBitcodeSize: 1.95 MB (2039944)
> >            - NumFunctions: 32 (32)
> >            - NumInstructions: 638 (638)
> >            - OptimizationTime: 26.000ms
> >            - PeakMemoryUsage: 319.00 KB (326656)
> >            - PrepareTime: 12.000ms
> >       Instance 7e441f58868a8585:7452021000000006
> > (host=ip-10-197-30-21.eu-west-1.compute.internal:22000):(Total: 2m41s,
> > non-child: 152.998ms, % non-child: 0.09%)
> >         Fragment Instance Lifecycle Event Timeline: 2m41s
> >            - Prepare Finished: 48.999ms (48.999ms)
> >            - Open Finished: 106.999ms (57.999ms)
> >            - First Batch Produced: 28s199ms (28s092ms)
> >            - First Batch Sent: 28s202ms (2.999ms)
> >            - ExecInternal Finished: 2m41s (2m12s)
> >         MemoryUsage(4s000ms): 496.29 MB, 1.27 GB, 2.10 GB, 2.86 GB, 3.45
> > GB, 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB,
> > 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB,
> > 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB,
> > 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB, 3.53 GB,
> > 3.53 GB, 3.53 GB, 3.53 GB
> >         ThreadUsage(4s000ms): 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1,
> > 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1
> >          - AverageThreadTokens: 1.00
> >          - BloomFilterBytes: 0
> >          - PeakMemoryUsage: 3.53 GB (3786981859)
> >          - PeakReservation: 3.49 GB (3751804928)
> >          - PeakUsedReservation: 0
> >          - PerHostPeakMemUsage: 3.76 GB (4037232486)
> >          - RowsProduced: 15.01M (15006663)
> >          - TotalNetworkReceiveTime: 3s501ms
> >          - TotalNetworkSendTime: 0.000ns
> >          - TotalStorageWaitTime: 0.000ns
> >          - TotalThreadsInvoluntaryContextSwitches: 1.34K (1340)
> >          - TotalThreadsTotalWallClockTime: 2m41s
> >            - TotalThreadsSysTime: 8s804ms
> >            - TotalThreadsUserTime: 1m14s
> >          - TotalThreadsVoluntaryContextSwitches: 15.37K (15372)
> >         Buffer pool:
> >            - AllocTime: 0.000ns
> >            - CumulativeAllocationBytes: 0
> >            - CumulativeAllocations: 0 (0)
> >            - PeakReservation: 0
> >            - PeakUnpinnedBytes: 0
> >            - PeakUsedReservation: 0
> >            - ReadIoBytes: 0
> >            - ReadIoOps: 0 (0)
> >            - ReadIoWaitTime: 0.000ns
> >            - ReservationLimit: 0
> >            - WriteIoBytes: 0
> >            - WriteIoOps: 0 (0)
> >            - WriteIoWaitTime: 0.000ns
> >         Fragment Instance Lifecycle Timings:
> >            - ExecTime: 2m41s
> >              - ExecTreeExecTime: 28s172ms
> >            - OpenTime: 57.999ms
> >              - ExecTreeOpenTime: 6.999ms
> >            - PrepareTime: 48.999ms
> >              - ExecTreePrepareTime: 0.000ns
> >         KuduTableSink:(Total: 2m12s, non-child: 2m12s, % non-child:
> > 100.00%)
> >            - KuduApplyTimer: 1m54s
> >            - NumRowErrors: 5.11K (5108)
> >            - PeakMemoryUsage: 20.00 MB (20971520)
> >            - RowsProcessedRate: 112.98 K/sec
> >            - TotalNumRows: 15.01M (15006663)
> >         SORT_NODE (id=2):(Total: 28s162ms, non-child: 24s414ms, %
> > non-child: 86.69%)
> >           SortType: Partial
> >           ExecOption: Codegen Enabled
> >            - NumRowsPerRun: (Avg: 15.01M (15006663) ; Min: 15.01M
> > (15006663) ; Max: 15.01M (15006663) ; Number of samples: 1)
> >            - InMemorySortTime: 8s976ms
> >            - PeakMemoryUsage: 3.49 GB (3751821312)
> >            - RowsReturned: 15.01M (15006663)
> >            - RowsReturnedRate: 532.85 K/sec
> >            - RunsCreated: 1 (1)
> >            - SortDataSize: 3.49 GB (3748143215)
> >           Buffer pool:
> >              - AllocTime: 485.996ms
> >              - CumulativeAllocationBytes: 3.49 GB (3751804928)
> >              - CumulativeAllocations: 1.79K (1789)
> >              - PeakReservation: 3.49 GB (3751804928)
> >              - PeakUnpinnedBytes: 0
> >              - PeakUsedReservation: 3.49 GB (3751804928)
> >              - ReadIoBytes: 0
> >              - ReadIoOps: 0 (0)
> >              - ReadIoWaitTime: 0.000ns
> >              - WriteIoBytes: 0
> >              - WriteIoOps: 0 (0)
> >              - WriteIoWaitTime: 0.000ns
> >         EXCHANGE_NODE (id=1):(Total: 3s747ms, non-child: 444.996ms, %
> > non-child: 11.87%)
> >            - ConvertRowBatchTime: 199.998ms
> >            - PeakMemoryUsage: 13.65 MB (14313119)
> >            - RowsReturned: 15.01M (15006663)
> >            - RowsReturnedRate: 4.00 M/sec
> >           Buffer pool:
> >              - AllocTime: 125.999ms
> >              - CumulativeAllocationBytes: 4.69 GB (5038350336)
> >              - CumulativeAllocations: 246.02K (246016)
> >              - PeakReservation: 13.52 MB (14180352)
> >              - PeakUnpinnedBytes: 0
> >              - PeakUsedReservation: 13.52 MB (14180352)
> >              - ReadIoBytes: 0
> >              - ReadIoOps: 0 (0)
> >              - ReadIoWaitTime: 0.000ns
> >              - WriteIoBytes: 0
> >              - WriteIoOps: 0 (0)
> >              - WriteIoWaitTime: 0.000ns
> >           Dequeue:
> >             BytesDequeued(4s000ms): 466.52 MB, 1.25 GB, 2.10 GB, 2.87 GB,
> > 3.47 GB, 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB,
> > 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB,
> > 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB,
> > 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB,
> > 3.55 GB, 3.55 GB, 3.55 GB, 3.55 GB
> >              - FirstBatchWaitTime: 5.999ms
> >              - TotalBytesDequeued: 3.55 GB (3808169867)
> >              - TotalGetBatchTime: 3s494ms
> >                - DataWaitTime: 3s302ms
> >           Enqueue:
> >             BytesReceived(4s000ms): 330.36 MB, 901.24 MB, 1.48 GB, 2.01
> > GB, 2.43 GB, 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB,
> > 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB,
> > 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB,
> > 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB,
> > 2.49 GB, 2.49 GB, 2.49 GB, 2.49 GB
> >             DeferredQueueSize(4s000ms): 0, 0, 1, 0, 0, 0, 0, 0, 0, 0, 0,
> > 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
> > 0, 0, 0, 0
> >              - DispatchTime: (Avg: 18.758us ; Min: 4.110us ; Max: 9.230ms
> > ; Number of samples: 123008)
> >              - DeserializeRowBatchTime: 3s491ms
> >              - TotalBatchesEnqueued: 123.01K (123008)
> >              - TotalBatchesReceived: 123.01K (123008)
> >              - TotalBytesReceived: 2.49 GB (2668950891)
> >              - TotalEarlySenders: 0 (0)
> >              - TotalEosReceived: 6 (6)
> >              - TotalHasDeferredRPCsTime: 2s186ms
> >              - TotalRPCsDeferred: 15.90K (15904)
> >         CodeGen:(Total: 50.999ms, non-child: 50.999ms, % non-child:
> > 100.00%)
> >            - CodegenTime: 999.992us
> >            - CompileTime: 4.999ms
> >            - LoadTime: 0.000ns
> >            - ModuleBitcodeSize: 1.95 MB (2039944)
> >            - NumFunctions: 32 (32)
> >            - NumInstructions: 638 (638)
> >            - OptimizationTime: 26.999ms
> >            - PeakMemoryUsage: 319.00 KB (326656)
> >            - PrepareTime: 17.999ms
> >       Instance 7e441f58868a8585:7452021000000008
> > (host=ip-10-197-11-142.eu-west-1.compute.internal:22000):(Total: 2m40s,
> > non-child: 333.001ms, % non-child: 0.21%)
> >         Fragment Instance Lifecycle Event Timeline: 2m40s
> >            - Prepare Finished: 49.000ms (49.000ms)
> >            - Open Finished: 107.000ms (58.000ms)
> >            - First Batch Produced: 27s989ms (27s882ms)
> >            - First Batch Sent: 27s992ms (3.000ms)
> >            - ExecInternal Finished: 2m40s (2m12s)
> >         MemoryUsage(4s000ms): 500.63 MB, 1.27 GB, 2.11 GB, 2.87 GB, 3.44
> > GB, 3.52 GB, 3.52 GB, 3.52 GB, 3.52 GB, 3.52 GB, 3.52 GB, 3.52 GB, 3.52 GB,
> > 3.52 GB, 3.52 GB, 3.52 GB, 3.52 GB, 3.52 GB, 3.52 GB, 3.52 GB, 3.52 GB,
> > 3.52 GB, 3.52 GB, 3.52 GB, 3.52 GB, 3.52 GB, 3.52 GB, 3.52 GB, 3.52 GB,
> > 3.52 GB, 3.52 GB, 3.52 GB, 3.52 GB, 3.52 GB, 3.52 GB, 3.52 GB, 3.52 GB,
> > 3.52 GB, 3.52 GB, 3.52 GB
> >         ThreadUsage(4s000ms): 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1,
> > 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1
> >          - AverageThreadTokens: 1.00
> >          - BloomFilterBytes: 0
> >          - PeakMemoryUsage: 3.52 GB (3782869475)
> >          - PeakReservation: 3.49 GB (3747610624)
> >          - PeakUsedReservation: 0
> >          - PerHostPeakMemUsage: 3.52 GB (3782869475)
> >          - RowsProduced: 14.99M (14990212)
> >          - TotalNetworkReceiveTime: 3s205ms
> >          - TotalNetworkSendTime: 0.000ns
> >          - TotalStorageWaitTime: 0.000ns
> >          - TotalThreadsInvoluntaryContextSwitches: 1.27K (1268)
> >          - TotalThreadsTotalWallClockTime: 2m40s
> >            - TotalThreadsSysTime: 8s733ms
> >            - TotalThreadsUserTime: 1m13s
> >          - TotalThreadsVoluntaryContextSwitches: 14.08K (14076)
> >         Buffer pool:
> >            - AllocTime: 0.000ns
> >            - CumulativeAllocationBytes: 0
> >            - CumulativeAllocations: 0 (0)
> >            - PeakReservation: 0
> >            - PeakUnpinnedBytes: 0
> >            - PeakUsedReservation: 0
> >            - ReadIoBytes: 0
> >            - ReadIoOps: 0 (0)
> >            - ReadIoWaitTime: 0.000ns
> >            - ReservationLimit: 0
> >            - WriteIoBytes: 0
> >            - WriteIoOps: 0 (0)
> >            - WriteIoWaitTime: 0.000ns
> >         Fragment Instance Lifecycle Timings:
> >            - ExecTime: 2m40s
> >              - ExecTreeExecTime: 27s952ms
> >            - OpenTime: 58.000ms
> >              - ExecTreeOpenTime: 17.000ms
> >            - PrepareTime: 48.000ms
> >              - ExecTreePrepareTime: 0.000ns
> >         KuduTableSink:(Total: 2m12s, non-child: 2m12s, % non-child:
> > 100.00%)
> >            - KuduApplyTimer: 1m53s
> >            - NumRowErrors: 5.17K (5175)
> >            - PeakMemoryUsage: 20.00 MB (20971520)
> >            - RowsProcessedRate: 113.55 K/sec
> >            - TotalNumRows: 14.99M (14990212)
> >         SORT_NODE (id=2):(Total: 27s960ms, non-child: 24s481ms, %
> > non-child: 87.56%)
> >           SortType: Partial
> >           ExecOption: Codegen Enabled
> >            - NumRowsPerRun: (Avg: 14.99M (14990212) ; Min: 14.99M
> > (14990212) ; Max: 14.99M (14990212) ; Number of samples: 1)
> >            - InMemorySortTime: 8s765ms
> >            - PeakMemoryUsage: 3.49 GB (3747627008)
> >            - RowsReturned: 14.99M (14990212)
> >            - RowsReturnedRate: 536.13 K/sec
> >            - RunsCreated: 1 (1)
> >            - SortDataSize: 3.49 GB (3744000131)
> >           Buffer pool:
> >              - AllocTime: 298.001ms
> >              - CumulativeAllocationBytes: 3.49 GB (3747610624)
> >              - CumulativeAllocations: 1.79K (1787)
> >              - PeakReservation: 3.49 GB (3747610624)
> >              - PeakUnpinnedBytes: 0
> >              - PeakUsedReservation: 3.49 GB (3747610624)
> >              - ReadIoBytes: 0
> >              - ReadIoOps: 0 (0)
> >              - ReadIoWaitTime: 0.000ns
> >              - WriteIoBytes: 0
> >              - WriteIoOps: 0 (0)
> >              - WriteIoWaitTime: 0.000ns
> >         EXCHANGE_NODE (id=1):(Total: 3s479ms, non-child: 435.002ms, %
> > non-child: 12.50%)
> >            - ConvertRowBatchTime: 216.001ms
> >            - PeakMemoryUsage: 13.71 MB (14371845)
> >            - RowsReturned: 14.99M (14990212)
> >            - RowsReturnedRate: 4.31 M/sec
> >           Buffer pool:
> >              - AllocTime: 106.000ms
> >              - CumulativeAllocationBytes: 4.69 GB (5032812544)
> >              - CumulativeAllocations: 245.75K (245746)
> >              - PeakReservation: 13.60 MB (14262272)
> >              - PeakUnpinnedBytes: 0
> >              - PeakUsedReservation: 13.60 MB (14262272)
> >              - ReadIoBytes: 0
> >              - ReadIoOps: 0 (0)
> >              - ReadIoWaitTime: 0.000ns
> >              - WriteIoBytes: 0
> >              - WriteIoOps: 0 (0)
> >              - WriteIoWaitTime: 0.000ns
> >           Dequeue:
> >             BytesDequeued(4s000ms): 472.38 MB, 1.25 GB, 2.10 GB, 2.87 GB,
> > 3.46 GB, 3.54 GB, 3.54 GB, 3.54 GB, 3.54 GB, 3.54 GB, 3.54 GB, 3.54 GB,
> > 3.54 GB, 3.54 GB, 3.54 GB, 3.54 GB, 3.54 GB, 3.54 GB, 3.54 GB, 3.54 GB,
> > 3.54 GB, 3.54 GB, 3.54 GB, 3.54 GB, 3.54 GB, 3.54 GB, 3.54 GB, 3.54 GB,
> > 3.54 GB, 3.54 GB, 3.54 GB, 3.54 GB, 3.54 GB, 3.54 GB, 3.54 GB, 3.54 GB,
> > 3.54 GB, 3.54 GB, 3.54 GB, 3.54 GB
> >              - FirstBatchWaitTime: 17.000ms
> >              - TotalBytesDequeued: 3.54 GB (3803960979)
> >              - TotalGetBatchTime: 3s198ms
> >                - DataWaitTime: 3s044ms
> >           Enqueue:
> >             BytesReceived(4s000ms): 334.01 MB, 903.81 MB, 1.48 GB, 2.02
> > GB, 2.43 GB, 2.48 GB, 2.48 GB, 2.48 GB, 2.48 GB, 2.48 GB, 2.48 GB, 2.48 GB,
> > 2.48 GB, 2.48 GB, 2.48 GB, 2.48 GB, 2.48 GB, 2.48 GB, 2.48 GB, 2.48 GB,
> > 2.48 GB, 2.48 GB, 2.48 GB, 2.48 GB, 2.48 GB, 2.48 GB, 2.48 GB, 2.48 GB,
> > 2.48 GB, 2.48 GB, 2.48 GB, 2.48 GB, 2.48 GB, 2.48 GB, 2.48 GB, 2.48 GB,
> > 2.48 GB, 2.48 GB, 2.48 GB, 2.48 GB
> >             DeferredQueueSize(4s000ms): 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
> > 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
> > 0, 0, 0, 0
> >              - DispatchTime: (Avg: 17.899us ; Min: 3.826us ; Max: 5.040ms
> > ; Number of samples: 122873)
> >              - DeserializeRowBatchTime: 3s434ms
> >              - TotalBatchesEnqueued: 122.87K (122873)
> >              - TotalBatchesReceived: 122.87K (122873)
> >              - TotalBytesReceived: 2.48 GB (2666083162)
> >              - TotalEarlySenders: 0 (0)
> >              - TotalEosReceived: 6 (6)
> >              - TotalHasDeferredRPCsTime: 2s563ms
> >              - TotalRPCsDeferred: 17.08K (17077)
> >         CodeGen:(Total: 40.000ms, non-child: 40.000ms, % non-child:
> > 100.00%)
> >            - CodegenTime: 1.000ms
> >            - CompileTime: 5.000ms
> >            - LoadTime: 0.000ns
> >            - ModuleBitcodeSize: 1.95 MB (2039944)
> >            - NumFunctions: 32 (32)
> >            - NumInstructions: 638 (638)
> >            - OptimizationTime: 23.000ms
> >            - PeakMemoryUsage: 319.00 KB (326656)
> >            - PrepareTime: 11.000ms
> >       Instance 7e441f58868a8585:745202100000000b
> > (host=ip-10-197-10-88.eu-west-1.compute.internal:22000):(Total: 19s224ms,
> > non-child: 49.000ms, % non-child: 0.25%)
> >         Fragment Instance Lifecycle Event Timeline: 19s224ms
> >            - Prepare Finished: 49.000ms (49.000ms)
> >            - Open Finished: 19s224ms (19s175ms)
> >            - First Batch Produced: 19s224ms (0.000ns)
> >            - First Batch Sent: 19s224ms (0.000ns)
> >            - ExecInternal Finished: 19s224ms (0.000ns)
> >         MemoryUsage(500.000ms): 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB,
> > 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47
> > KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB,
> > 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47
> > KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB,
> > 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB
> >         ThreadUsage(500.000ms): 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1,
> > 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1
> >          - AverageThreadTokens: 1.00
> >          - BloomFilterBytes: 0
> >          - PeakMemoryUsage: 26.02 MB (27288035)
> >          - PeakReservation: 6.00 MB (6291456)
> >          - PeakUsedReservation: 0
> >          - PerHostPeakMemUsage: 704.86 MB (739094888)
> >          - RowsProduced: 0 (0)
> >          - TotalNetworkReceiveTime: 19s135ms
> >          - TotalNetworkSendTime: 0.000ns
> >          - TotalStorageWaitTime: 0.000ns
> >          - TotalThreadsInvoluntaryContextSwitches: 0 (0)
> >          - TotalThreadsTotalWallClockTime: 19s175ms
> >            - TotalThreadsSysTime: 0.000ns
> >            - TotalThreadsUserTime: 40.199ms
> >          - TotalThreadsVoluntaryContextSwitches: 2 (2)
> >         Buffer pool:
> >            - AllocTime: 0.000ns
> >            - CumulativeAllocationBytes: 0
> >            - CumulativeAllocations: 0 (0)
> >            - PeakReservation: 0
> >            - PeakUnpinnedBytes: 0
> >            - PeakUsedReservation: 0
> >            - ReadIoBytes: 0
> >            - ReadIoOps: 0 (0)
> >            - ReadIoWaitTime: 0.000ns
> >            - ReservationLimit: 0
> >            - WriteIoBytes: 0
> >            - WriteIoOps: 0 (0)
> >            - WriteIoWaitTime: 0.000ns
> >         Fragment Instance Lifecycle Timings:
> >            - ExecTime: 0.000ns
> >              - ExecTreeExecTime: 0.000ns
> >            - OpenTime: 19s175ms
> >              - ExecTreeOpenTime: 19s135ms
> >            - PrepareTime: 49.000ms
> >              - ExecTreePrepareTime: 0.000ns
> >         KuduTableSink:
> >            - KuduApplyTimer: 0.000ns
> >            - NumRowErrors: 0 (0)
> >            - PeakMemoryUsage: 20.00 MB (20971520)
> >            - RowsProcessedRate: 0
> >            - TotalNumRows: 0 (0)
> >         SORT_NODE (id=2):(Total: 19s135ms, non-child: 0.000ns, %
> > non-child: 0.00%)
> >           SortType: Partial
> >           ExecOption: Codegen Enabled
> >            - NumRowsPerRun: (Avg: 0 (0) ; Min: 0 (0) ; Max: 0 (0) ; Number
> > of samples: 1)
> >            - InMemorySortTime: 0.000ns
> >            - PeakMemoryUsage: 6.02 MB (6307840)
> >            - RowsReturned: 0 (0)
> >            - RowsReturnedRate: 0
> >            - RunsCreated: 1 (1)
> >            - SortDataSize: 0
> >           Buffer pool:
> >              - AllocTime: 0.000ns
> >              - CumulativeAllocationBytes: 6.00 MB (6291456)
> >              - CumulativeAllocations: 3 (3)
> >              - PeakReservation: 6.00 MB (6291456)
> >              - PeakUnpinnedBytes: 0
> >              - PeakUsedReservation: 6.00 MB (6291456)
> >              - ReadIoBytes: 0
> >              - ReadIoOps: 0 (0)
> >              - ReadIoWaitTime: 0.000ns
> >              - WriteIoBytes: 0
> >              - WriteIoOps: 0 (0)
> >              - WriteIoWaitTime: 0.000ns
> >         EXCHANGE_NODE (id=1):(Total: 19s135ms, non-child: 0.000ns, %
> > non-child: 0.00%)
> >            - ConvertRowBatchTime: 0.000ns
> >            - PeakMemoryUsage: 0
> >            - RowsReturned: 0 (0)
> >            - RowsReturnedRate: 0
> >           Buffer pool:
> >              - AllocTime: 0.000ns
> >              - CumulativeAllocationBytes: 0
> >              - CumulativeAllocations: 0 (0)
> >              - PeakReservation: 0
> >              - PeakUnpinnedBytes: 0
> >              - PeakUsedReservation: 0
> >              - ReadIoBytes: 0
> >              - ReadIoOps: 0 (0)
> >              - ReadIoWaitTime: 0.000ns
> >              - WriteIoBytes: 0
> >              - WriteIoOps: 0 (0)
> >              - WriteIoWaitTime: 0.000ns
> >           Dequeue:
> >             BytesDequeued(500.000ms): 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
> > 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0
> >              - FirstBatchWaitTime: 19s135ms
> >              - TotalBytesDequeued: 0
> >              - TotalGetBatchTime: 19s135ms
> >                - DataWaitTime: 19s135ms
> >           Enqueue:
> >             BytesReceived(500.000ms): 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
> > 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0
> >             DeferredQueueSize(500.000ms): 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
> > 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
> > 0, 0
> >             - DispatchTime: 0.000ns (Number of samples: 0)
> >              - DeserializeRowBatchTime: 0.000ns
> >              - TotalBatchesEnqueued: 0 (0)
> >              - TotalBatchesReceived: 0 (0)
> >              - TotalBytesReceived: 0
> >              - TotalEarlySenders: 0 (0)
> >              - TotalEosReceived: 6 (6)
> >              - TotalHasDeferredRPCsTime: 0.000ns
> >              - TotalRPCsDeferred: 0 (0)
> >         CodeGen:(Total: 40.000ms, non-child: 40.000ms, % non-child:
> > 100.00%)
> >            - CodegenTime: 1.000ms
> >            - CompileTime: 4.000ms
> >            - LoadTime: 0.000ns
> >            - ModuleBitcodeSize: 1.95 MB (2039944)
> >            - NumFunctions: 32 (32)
> >            - NumInstructions: 638 (638)
> >            - OptimizationTime: 23.000ms
> >            - PeakMemoryUsage: 319.00 KB (326656)
> >            - PrepareTime: 12.000ms
> >       Instance 7e441f58868a8585:7452021000000007
> > (host=ip-10-197-3-207.eu-west-1.compute.internal:22000):(Total: 19s223ms,
> > non-child: 48.000ms, % non-child: 0.25%)
> >         Fragment Instance Lifecycle Event Timeline: 19s223ms
> >            - Prepare Finished: 48.000ms (48.000ms)
> >            - Open Finished: 19s223ms (19s175ms)
> >            - First Batch Produced: 19s223ms (0.000ns)
> >            - First Batch Sent: 19s223ms (0.000ns)
> >            - ExecInternal Finished: 19s223ms (0.000ns)
> >         MemoryUsage(500.000ms): 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB,
> > 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47
> > KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB,
> > 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47
> > KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB,
> > 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB
> >         ThreadUsage(500.000ms): 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1,
> > 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1
> >          - AverageThreadTokens: 1.00
> >          - BloomFilterBytes: 0
> >          - PeakMemoryUsage: 26.02 MB (27288035)
> >          - PeakReservation: 6.00 MB (6291456)
> >          - PeakUsedReservation: 0
> >          - PerHostPeakMemUsage: 720.28 MB (755267876)
> >          - RowsProduced: 0 (0)
> >          - TotalNetworkReceiveTime: 19s135ms
> >          - TotalNetworkSendTime: 0.000ns
> >          - TotalStorageWaitTime: 0.000ns
> >          - TotalThreadsInvoluntaryContextSwitches: 0 (0)
> >          - TotalThreadsTotalWallClockTime: 19s175ms
> >            - TotalThreadsSysTime: 2.013ms
> >            - TotalThreadsUserTime: 37.565ms
> >          - TotalThreadsVoluntaryContextSwitches: 3 (3)
> >         Buffer pool:
> >            - AllocTime: 0.000ns
> >            - CumulativeAllocationBytes: 0
> >            - CumulativeAllocations: 0 (0)
> >            - PeakReservation: 0
> >            - PeakUnpinnedBytes: 0
> >            - PeakUsedReservation: 0
> >            - ReadIoBytes: 0
> >            - ReadIoOps: 0 (0)
> >            - ReadIoWaitTime: 0.000ns
> >            - ReservationLimit: 0
> >            - WriteIoBytes: 0
> >            - WriteIoOps: 0 (0)
> >            - WriteIoWaitTime: 0.000ns
> >         Fragment Instance Lifecycle Timings:
> >            - ExecTime: 0.000ns
> >              - ExecTreeExecTime: 0.000ns
> >            - OpenTime: 19s175ms
> >              - ExecTreeOpenTime: 19s135ms
> >            - PrepareTime: 48.000ms
> >              - ExecTreePrepareTime: 0.000ns
> >         KuduTableSink:
> >            - KuduApplyTimer: 0.000ns
> >            - NumRowErrors: 0 (0)
> >            - PeakMemoryUsage: 20.00 MB (20971520)
> >            - RowsProcessedRate: 0
> >            - TotalNumRows: 0 (0)
> >         SORT_NODE (id=2):(Total: 19s135ms, non-child: 0.000ns, %
> > non-child: 0.00%)
> >           SortType: Partial
> >           ExecOption: Codegen Enabled
> >            - NumRowsPerRun: (Avg: 0 (0) ; Min: 0 (0) ; Max: 0 (0) ; Number
> > of samples: 1)
> >            - InMemorySortTime: 0.000ns
> >            - PeakMemoryUsage: 6.02 MB (6307840)
> >            - RowsReturned: 0 (0)
> >            - RowsReturnedRate: 0
> >            - RunsCreated: 1 (1)
> >            - SortDataSize: 0
> >           Buffer pool:
> >              - AllocTime: 0.000ns
> >              - CumulativeAllocationBytes: 6.00 MB (6291456)
> >              - CumulativeAllocations: 3 (3)
> >              - PeakReservation: 6.00 MB (6291456)
> >              - PeakUnpinnedBytes: 0
> >              - PeakUsedReservation: 6.00 MB (6291456)
> >              - ReadIoBytes: 0
> >              - ReadIoOps: 0 (0)
> >              - ReadIoWaitTime: 0.000ns
> >              - WriteIoBytes: 0
> >              - WriteIoOps: 0 (0)
> >              - WriteIoWaitTime: 0.000ns
> >         EXCHANGE_NODE (id=1):(Total: 19s135ms, non-child: 0.000ns, %
> > non-child: 0.00%)
> >            - ConvertRowBatchTime: 0.000ns
> >            - PeakMemoryUsage: 0
> >            - RowsReturned: 0 (0)
> >            - RowsReturnedRate: 0
> >           Buffer pool:
> >              - AllocTime: 0.000ns
> >              - CumulativeAllocationBytes: 0
> >              - CumulativeAllocations: 0 (0)
> >              - PeakReservation: 0
> >              - PeakUnpinnedBytes: 0
> >              - PeakUsedReservation: 0
> >              - ReadIoBytes: 0
> >              - ReadIoOps: 0 (0)
> >              - ReadIoWaitTime: 0.000ns
> >              - WriteIoBytes: 0
> >              - WriteIoOps: 0 (0)
> >              - WriteIoWaitTime: 0.000ns
> >           Dequeue:
> >             BytesDequeued(500.000ms): 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
> > 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0
> >              - FirstBatchWaitTime: 19s135ms
> >              - TotalBytesDequeued: 0
> >              - TotalGetBatchTime: 19s135ms
> >                - DataWaitTime: 19s135ms
> >           Enqueue:
> >             BytesReceived(500.000ms): 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
> > 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0
> >             DeferredQueueSize(500.000ms): 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
> > 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
> > 0, 0
> >             - DispatchTime: 0.000ns (Number of samples: 0)
> >              - DeserializeRowBatchTime: 0.000ns
> >              - TotalBatchesEnqueued: 0 (0)
> >              - TotalBatchesReceived: 0 (0)
> >              - TotalBytesReceived: 0
> >              - TotalEarlySenders: 0 (0)
> >              - TotalEosReceived: 6 (6)
> >              - TotalHasDeferredRPCsTime: 0.000ns
> >              - TotalRPCsDeferred: 0 (0)
> >         CodeGen:(Total: 40.000ms, non-child: 40.000ms, % non-child:
> > 100.00%)
> >            - CodegenTime: 0.000ns
> >            - CompileTime: 5.000ms
> >            - LoadTime: 0.000ns
> >            - ModuleBitcodeSize: 1.95 MB (2039944)
> >            - NumFunctions: 32 (32)
> >            - NumInstructions: 638 (638)
> >            - OptimizationTime: 22.000ms
> >            - PeakMemoryUsage: 319.00 KB (326656)
> >            - PrepareTime: 12.000ms
> >       Instance 7e441f58868a8585:7452021000000009
> > (host=ip-10-197-7-97.eu-west-1.compute.internal:22000):(Total: 19s222ms,
> > non-child: 50.999ms, % non-child: 0.27%)
> >         Fragment Instance Lifecycle Event Timeline: 19s222ms
> >            - Prepare Finished: 50.999ms (50.999ms)
> >            - Open Finished: 19s222ms (19s171ms)
> >            - First Batch Produced: 19s222ms (0.000ns)
> >            - First Batch Sent: 19s222ms (0.000ns)
> >            - ExecInternal Finished: 19s222ms (0.000ns)
> >         MemoryUsage(500.000ms): 335.00 KB, 16.47 KB, 16.47 KB, 16.47 KB,
> > 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47
> > KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB,
> > 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47
> > KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB,
> > 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB, 16.47 KB
> >         ThreadUsage(500.000ms): 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1,
> > 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1
> >          - AverageThreadTokens: 1.00
> >          - BloomFilterBytes: 0
> >          - PeakMemoryUsage: 26.02 MB (27288035)
> >          - PeakReservation: 6.00 MB (6291456)
> >          - PeakUsedReservation: 0
> >          - PerHostPeakMemUsage: 675.40 MB (708207971)
> >          - RowsProduced: 0 (0)
> >          - TotalNetworkReceiveTime: 19s124ms
> >          - TotalNetworkSendTime: 0.000ns
> >          - TotalStorageWaitTime: 0.000ns
> >          - TotalThreadsInvoluntaryContextSwitches: 4 (4)
> >          - TotalThreadsTotalWallClockTime: 19s171ms
> >            - TotalThreadsSysTime: 1.006ms
> >            - TotalThreadsUserTime: 45.570ms
> >          - TotalThreadsVoluntaryContextSwitches: 2 (2)
> >         Buffer pool:
> >            - AllocTime: 0.000ns
> >            - CumulativeAllocationBytes: 0
> >            - CumulativeAllocations: 0 (0)
> >            - PeakReservation: 0
> >            - PeakUnpinnedBytes: 0
> >            - PeakUsedReservation: 0
> >            - ReadIoBytes: 0
> >            - ReadIoOps: 0 (0)
> >            - ReadIoWaitTime: 0.000ns
> >            - ReservationLimit: 0
> >            - WriteIoBytes: 0
> >            - WriteIoOps: 0 (0)
> >            - WriteIoWaitTime: 0.000ns
> >         Fragment Instance Lifecycle Timings:
> >            - ExecTime: 0.000ns
> >              - ExecTreeExecTime: 0.000ns
> >            - OpenTime: 19s171ms
> >              - ExecTreeOpenTime: 19s124ms
> >            - PrepareTime: 50.999ms
> >              - ExecTreePrepareTime: 0.000ns
> >         KuduTableSink:
> >            - KuduApplyTimer: 0.000ns
> >            - NumRowErrors: 0 (0)
> >            - PeakMemoryUsage: 20.00 MB (20971520)
> >            - RowsProcessedRate: 0
> >            - TotalNumRows: 0 (0)
> >         SORT_NODE (id=2):(Total: 19s124ms, non-child: 0.000ns, %
> > non-child: 0.00%)
> >           SortType: Partial
> >           ExecOption: Codegen Enabled
> >            - NumRowsPerRun: (Avg: 0 (0) ; Min: 0 (0) ; Max: 0 (0) ; Number
> > of samples: 1)
> >            - InMemorySortTime: 0.000ns
> >            - PeakMemoryUsage: 6.02 MB (6307840)
> >            - RowsReturned: 0 (0)
> >            - RowsReturnedRate: 0
> >            - RunsCreated: 1 (1)
> >            - SortDataSize: 0
> >           Buffer pool:
> >              - AllocTime: 0.000ns
> >              - CumulativeAllocationBytes: 6.00 MB (6291456)
> >              - CumulativeAllocations: 3 (3)
> >              - PeakReservation: 6.00 MB (6291456)
> >              - PeakUnpinnedBytes: 0
> >              - PeakUsedReservation: 6.00 MB (6291456)
> >              - ReadIoBytes: 0
> >              - ReadIoOps: 0 (0)
> >              - ReadIoWaitTime: 0.000ns
> >              - WriteIoBytes: 0
> >              - WriteIoOps: 0 (0)
> >              - WriteIoWaitTime: 0.000ns
> >         EXCHANGE_NODE (id=1):(Total: 19s124ms, non-child: 0.000ns, %
> > non-child: 0.00%)
> >            - ConvertRowBatchTime: 0.000ns
> >            - PeakMemoryUsage: 0
> >            - RowsReturned: 0 (0)
> >            - RowsReturnedRate: 0
> >           Buffer pool:
> >              - AllocTime: 0.000ns
> >              - CumulativeAllocationBytes: 0
> >              - CumulativeAllocations: 0 (0)
> >              - PeakReservation: 0
> >              - PeakUnpinnedBytes: 0
> >              - PeakUsedReservation: 0
> >              - ReadIoBytes: 0
> >              - ReadIoOps: 0 (0)
> >              - ReadIoWaitTime: 0.000ns
> >              - WriteIoBytes: 0
> >              - WriteIoOps: 0 (0)
> >              - WriteIoWaitTime: 0.000ns
> >           Dequeue:
> >             BytesDequeued(500.000ms): 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
> > 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
> > 0, 0
> >              - FirstBatchWaitTime: 19s124ms
> >              - TotalBytesDequeued: 0
> >              - TotalGetBatchTime: 19s124ms
> >                - DataWaitTime: 19s124ms
> >           Enqueue:
> >             BytesReceived(500.000ms): 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
> > 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
> > 0, 0
> >             DeferredQueueSize(500.000ms): 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
> > 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
> > 0, 0, 0
> >             - DispatchTime: 0.000ns (Number of samples: 0)
> >              - DeserializeRowBatchTime: 0.000ns
> >              - TotalBatchesEnqueued: 0 (0)
> >              - TotalBatchesReceived: 0 (0)
> >              - TotalBytesReceived: 0
> >              - TotalEarlySenders: 0 (0)
> >              - TotalEosReceived: 6 (6)
> >              - TotalHasDeferredRPCsTime: 0.000ns
> >              - TotalRPCsDeferred: 0 (0)
> >         CodeGen:(Total: 46.999ms, non-child: 46.999ms, % non-child:
> > 100.00%)
> >            - CodegenTime: 999.999us
> >            - CompileTime: 6.999ms
> >            - LoadTime: 0.000ns
> >            - ModuleBitcodeSize: 1.95 MB (2039944)
> >            - NumFunctions: 32 (32)
> >            - NumInstructions: 638 (638)
> >            - OptimizationTime: 26.999ms
> >            - PeakMemoryUsage: 319.00 KB (326656)
> >            - PrepareTime: 11.999ms
> >     Averaged Fragment F00:(Total: 16s864ms, non-child: 20.666ms, %
> > non-child: 0.12%)
> >       split sizes:  min: 663.66 MB, max: 918.23 MB, avg: 841.66 MB,
> > stddev: 86.23 MB
> >       completion times: min:13s276ms  max:19s224ms  mean: 16s864ms
> > stddev:1s777ms
> >       execution rates: min:47.38 MB/sec  max:53.54 MB/sec  mean:49.97
> > MB/sec  stddev:2.15 MB/sec
> >       num instances: 6
> >        - AverageThreadTokens: 4.48
> >        - BloomFilterBytes: 0
> >        - PeakMemoryUsage: 687.06 MB (720431332)
> >        - PeakReservation: 0
> >        - PeakUsedReservation: 0
> >        - PerHostPeakMemUsage: 1.99 GB (2136129084)
> >        - RowsProduced: 7.50M (7500000)
> >        - TotalNetworkReceiveTime: 0.000ns
> >        - TotalNetworkSendTime: 5s256ms
> >        - TotalStorageWaitTime: 131.500ms
> >        - TotalThreadsInvoluntaryContextSwitches: 1.25K (1252)
> >        - TotalThreadsTotalWallClockTime: 1m16s
> >          - TotalThreadsSysTime: 591.044ms
> >          - TotalThreadsUserTime: 13s372ms
> >        - TotalThreadsVoluntaryContextSwitches: 17.10K (17097)
> >       Buffer pool:
> >          - AllocTime: 0.000ns
> >          - CumulativeAllocationBytes: 0
> >          - CumulativeAllocations: 0 (0)
> >          - PeakReservation: 0
> >          - PeakUnpinnedBytes: 0
> >          - PeakUsedReservation: 0
> >          - ReadIoBytes: 0
> >          - ReadIoOps: 0 (0)
> >          - ReadIoWaitTime: 0.000ns
> >          - ReservationLimit: 0
> >          - WriteIoBytes: 0
> >          - WriteIoOps: 0 (0)
> >          - WriteIoWaitTime: 0.000ns
> >       Fragment Instance Lifecycle Timings:
> >          - ExecTime: 16s790ms
> >            - ExecTreeExecTime: 101.333ms
> >          - OpenTime: 23.166ms
> >            - ExecTreeOpenTime: 0.000ns
> >          - PrepareTime: 50.500ms
> >            - ExecTreePrepareTime: 0.000ns
> >       KrpcDataStreamSender (dst_id=1):(Total: 16s724ms, non-child:
> > 16s724ms, % non-child: 100.00%)
> >          - EosSent: 6 (6)
> >          - NetworkThroughput: 98.51 MB/sec
> >          - PeakMemoryUsage: 185.72 KB (190176)
> >          - RowsSent: 7.50M (7500000)
> >          - RpcFailure: 0 (0)
> >          - RpcRetry: 0 (0)
> >          - SerializeBatchTime: 5s558ms
> >          - TotalBytesSent: 1.24 GB (1333898634)
> >          - UncompressedRowBatchSize: 1.77 GB (1903230714)
> >       HDFS_SCAN_NODE (id=0):(Total: 96.500ms, non-child: 96.500ms, %
> > non-child: 100.00%)
> >          - AverageHdfsReadThreadConcurrency: 0.01
> >          - AverageScannerThreadConcurrency: 3.50
> >          - BytesRead: 842.04 MB (882945552)
> >          - BytesReadDataNodeCache: 0
> >          - BytesReadLocal: 842.04 MB (882945552)
> >          - BytesReadRemoteUnexpected: 0
> >          - BytesReadShortCircuit: 842.04 MB (882945552)
> >          - CachedFileHandlesHitCount: 132 (132)
> >          - CachedFileHandlesMissCount: 0 (0)
> >          - CollectionItemsRead: 0 (0)
> >          - DecompressionTime: 1s248ms
> >          - MaxCompressedTextFileLength: 0
> >          - NumColumns: 10 (10)
> >          - NumDictFilteredRowGroups: 0 (0)
> >          - NumDisksAccessed: 2 (2)
> >          - NumRowGroups: 4 (4)
> >          - NumScannerThreadsStarted: 4 (4)
> >          - NumScannersWithNoReads: 0 (0)
> >          - NumStatsFilteredRowGroups: 0 (0)
> >          - PeakMemoryUsage: 686.69 MB (720051139)
> >          - PerReadThreadRawHdfsThroughput: 2.16 GB/sec
> >          - RemoteScanRanges: 0 (0)
> >          - RowBatchQueueGetWaitTime: 35.000ms
> >          - RowBatchQueuePutWaitTime: 57s002ms
> >          - RowsRead: 7.50M (7500000)
> >          - RowsReturned: 7.50M (7500000)
> >          - RowsReturnedRate: 80.32 M/sec
> >          - ScanRangesComplete: 4 (4)
> >          - ScannerThreadsInvoluntaryContextSwitches: 156 (156)
> >          - ScannerThreadsTotalWallClockTime: 59s411ms
> >            - MaterializeTupleTime(*): 818.668ms
> >            - ScannerThreadsSysTime: 129.643ms
> >            - ScannerThreadsUserTime: 2s185ms
> >          - ScannerThreadsVoluntaryContextSwitches: 7.27K (7270)
> >          - TotalRawHdfsOpenFileTime(*): 499.996us
> >          - TotalRawHdfsReadTime(*): 381.001ms
> >          - TotalReadThroughput: 50.04 MB/sec
> >       CodeGen:(Total: 23.166ms, non-child: 23.166ms, % non-child: 100.00%)
> >          - CodegenTime: 500.003us
> >          - CompileTime: 4.000ms
> >          - LoadTime: 0.000ns
> >          - ModuleBitcodeSize: 1.95 MB (2039944)
> >          - NumFunctions: 7 (7)
> >          - NumInstructions: 105 (105)
> >          - OptimizationTime: 5.666ms
> >          - PeakMemoryUsage: 52.50 KB (53760)
> >          - PrepareTime: 13.000ms
> >     Fragment F00:
> >       Instance 7e441f58868a8585:7452021000000000
> > (host=ip-10-197-30-21.eu-west-1.compute.internal:22000):(Total: 19s223ms,
> > non-child: 27.999ms, % non-child: 0.15%)
> >         Hdfs split stats (<volume id>:<# splits>/<split lengths>):
> > 0:2/406.63 MB 1:2/507.55 MB
> >         Fragment Instance Lifecycle Event Timeline: 19s223ms
> >            - Prepare Finished: 49.999ms (49.999ms)
> >            - Open Finished: 80.999ms (30.999ms)
> >            - First Batch Produced: 114.999ms (33.999ms)
> >            - First Batch Sent: 116.999ms (1.999ms)
> >            - ExecInternal Finished: 19s223ms (19s106ms)
> >         MemoryUsage(500.000ms): 714.21 MB, 720.67 MB, 721.47 MB, 721.63
> > MB, 721.93 MB, 722.25 MB, 722.25 MB, 716.29 MB, 708.21 MB, 708.59 MB,
> > 708.70 MB, 708.69 MB, 708.61 MB, 708.85 MB, 700.93 MB, 693.00 MB, 661.31
> > MB, 637.85 MB, 621.94 MB, 621.88 MB, 605.94 MB, 582.18 MB, 582.13 MB,
> > 582.18 MB, 570.18 MB, 560.14 MB, 481.03 MB, 475.95 MB, 476.17 MB, 459.81
> > MB, 444.04 MB, 419.97 MB, 356.36 MB, 340.45 MB, 308.42 MB, 300.68 MB,
> > 236.62 MB, 140.70 MB
> >         ThreadUsage(500.000ms): 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
> > 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 3
> >          - AverageThreadTokens: 4.68
> >          - BloomFilterBytes: 0
> >          - PeakMemoryUsage: 722.47 MB (757560400)
> >          - PeakReservation: 0
> >          - PeakUsedReservation: 0
> >          - PerHostPeakMemUsage: 3.76 GB (4037232486)
> >          - RowsProduced: 8.13M (8134450)
> >          - TotalNetworkReceiveTime: 0.000ns
> >          - TotalNetworkSendTime: 5s841ms
> >          - TotalStorageWaitTime: 134.998ms
> >          - TotalThreadsInvoluntaryContextSwitches: 2.32K (2324)
> >          - TotalThreadsTotalWallClockTime: 1m29s
> >            - TotalThreadsSysTime: 856.226ms
> >            - TotalThreadsUserTime: 15s315ms
> >          - TotalThreadsVoluntaryContextSwitches: 17.82K (17825)
> >         Buffer pool:
> >            - AllocTime: 0.000ns
> >            - CumulativeAllocationBytes: 0
> >            - CumulativeAllocations: 0 (0)
> >            - PeakReservation: 0
> >            - PeakUnpinnedBytes: 0
> >            - PeakUsedReservation: 0
> >            - ReadIoBytes: 0
> >            - ReadIoOps: 0 (0)
> >            - ReadIoWaitTime: 0.000ns
> >            - ReservationLimit: 0
> >            - WriteIoBytes: 0
> >            - WriteIoOps: 0 (0)
> >            - WriteIoWaitTime: 0.000ns
> >         Fragment Instance Lifecycle Timings:
> >            - ExecTime: 19s142ms
> >              - ExecTreeExecTime: 122.999ms
> >            - OpenTime: 30.999ms
> >              - ExecTreeOpenTime: 0.000ns
> >            - PrepareTime: 49.999ms
> >              - ExecTreePrepareTime: 0.000ns
> >         KrpcDataStreamSender (dst_id=1):(Total: 19s050ms, non-child:
> > 13s208ms, % non-child: 69.33%)
> >           BytesSent(500.000ms): 22.84 MB, 64.52 MB, 101.06 MB, 132.84 MB,
> > 170.18 MB, 195.54 MB, 231.15 MB, 267.65 MB, 293.64 MB, 315.66 MB, 342.16
> > MB, 381.38 MB, 409.68 MB, 437.93 MB, 473.86 MB, 515.57 MB, 557.72 MB,
> > 601.94 MB, 649.50 MB, 684.68 MB, 724.47 MB, 757.78 MB, 781.73 MB, 807.80
> > MB, 836.11 MB, 860.73 MB, 886.71 MB, 920.85 MB, 961.65 MB, 994.13 MB, 1.00
> > GB, 1.03 GB, 1.06 GB, 1.10 GB, 1.15 GB, 1.21 GB, 1.26 GB, 1.31 GB
> >            - EosSent: 6 (6)
> >            - NetworkThroughput: 99.22 MB/sec
> >            - PeakMemoryUsage: 185.72 KB (190176)
> >            - RowsSent: 8.13M (8134450)
> >            - RpcFailure: 0 (0)
> >            - RpcRetry: 0 (0)
> >            - SerializeBatchTime: 6s371ms
> >            - TotalBytesSent: 1.35 GB (1447999819)
> >            - UncompressedRowBatchSize: 1.93 GB (2067278508)
> >         HDFS_SCAN_NODE (id=0):(Total: 113.999ms, non-child: 113.999ms, %
> > non-child: 100.00%)
> >           Hdfs split stats (<volume id>:<# splits>/<split lengths>):
> > 0:2/406.63 MB 1:2/507.55 MB
> >           ExecOption: PARQUET Codegen Enabled, Codegen enabled: 4 out of 4
> >           Hdfs Read Thread Concurrency Bucket: 0:100% 1:0% 2:0% 3:0% 4:0%
> > 5:0% 6:0% 7:0% 8:0% 9:0% 10:0%
> >           File Formats: PARQUET/SNAPPY:4
> >           BytesRead(500.000ms): 621.00 MB, 621.00 MB, 621.00 MB, 637.00
> > MB, 653.00 MB, 653.00 MB, 653.00 MB, 678.59 MB, 694.59 MB, 718.59 MB,
> > 734.59 MB, 734.59 MB, 734.59 MB, 742.59 MB, 766.59 MB, 766.59 MB, 782.59
> > MB, 814.81 MB, 830.81 MB, 836.69 MB, 836.69 MB, 844.69 MB, 844.69 MB,
> > 860.69 MB, 863.99 MB, 868.33 MB, 871.81 MB, 879.81 MB, 895.81 MB, 895.81
> > MB, 901.16 MB, 914.51 MB, 914.51 MB, 914.51 MB, 914.57 MB, 914.57 MB,
> > 914.57 MB, 914.57 MB
> >            - FooterProcessingTime: (Avg: 999.992us ; Min: 999.992us ; Max:
> > 999.992us ; Number of samples: 4)
> >            - AverageHdfsReadThreadConcurrency: 0.00
> >            - AverageScannerThreadConcurrency: 3.68
> >            - BytesRead: 914.57 MB (958994524)
> >            - BytesReadDataNodeCache: 0
> >            - BytesReadLocal: 914.57 MB (958994524)
> >            - BytesReadRemoteUnexpected: 0
> >            - BytesReadShortCircuit: 914.57 MB (958994524)
> >            - CachedFileHandlesHitCount: 142 (142)
> >            - CachedFileHandlesMissCount: 0 (0)
> >            - CollectionItemsRead: 0 (0)
> >            - DecompressionTime: 1s404ms
> >            - MaxCompressedTextFileLength: 0
> >            - NumColumns: 10 (10)
> >            - NumDictFilteredRowGroups: 0 (0)
> >            - NumDisksAccessed: 2 (2)
> >            - NumRowGroups: 4 (4)
> >            - NumScannerThreadsStarted: 4 (4)
> >            - NumScannersWithNoReads: 0 (0)
> >            - NumStatsFilteredRowGroups: 0 (0)
> >            - PeakMemoryUsage: 722.10 MB (757173382)
> >            - PerReadThreadRawHdfsThroughput: 2.13 GB/sec
> >            - RemoteScanRanges: 0 (0)
> >            - RowBatchQueueGetWaitTime: 33.999ms
> >            - RowBatchQueuePutWaitTime: 1m7s
> >            - RowsRead: 8.13M (8134450)
> >            - RowsReturned: 8.13M (8134450)
> >            - RowsReturnedRate: 71.36 M/sec
> >            - ScanRangesComplete: 4 (4)
> >            - ScannerThreadsInvoluntaryContextSwitches: 220 (220)
> >            - ScannerThreadsTotalWallClockTime: 1m10s
> >              - MaterializeTupleTime(*): 997.992ms
> >              - ScannerThreadsSysTime: 178.638ms
> >              - ScannerThreadsUserTime: 2s517ms
> >            - ScannerThreadsVoluntaryContextSwitches: 7.90K (7898)
> >            - TotalRawHdfsOpenFileTime(*): 2.999ms
> >            - TotalRawHdfsReadTime(*): 418.996ms
> >            - TotalReadThroughput: 48.14 MB/sec
> >         CodeGen:(Total: 30.999ms, non-child: 30.999ms, % non-child:
> > 100.00%)
> >            - CodegenTime: 0.000ns
> >            - CompileTime: 4.999ms
> >            - LoadTime: 0.000ns
> >            - ModuleBitcodeSize: 1.95 MB (2039944)
> >            - NumFunctions: 7 (7)
> >            - NumInstructions: 105 (105)
> >            - OptimizationTime: 6.999ms
> >            - PeakMemoryUsage: 52.50 KB (53760)
> >            - PrepareTime: 16.999ms
> >       Instance 7e441f58868a8585:7452021000000003
> > (host=ip-10-197-7-97.eu-west-1.compute.internal:22000):(Total: 17s461ms,
> > non-child: 16.999ms, % non-child: 0.10%)
> >         Hdfs split stats (<volume id>:<# splits>/<split lengths>):
> > 1:1/106.53 MB 0:1/253.72 MB 2:2/467.03 MB
> >         Fragment Instance Lifecycle Event Timeline: 17s461ms
> >            - Prepare Finished: 51.999ms (51.999ms)
> >            - Open Finished: 73.999ms (21.999ms)
> >            - First Batch Produced: 109.999ms (35.999ms)
> >            - First Batch Sent: 111.999ms (1.999ms)
> >            - ExecInternal Finished: 17s461ms (17s349ms)
> >         MemoryUsage(500.000ms): 28.20 MB, 667.82 MB, 670.49 MB, 670.61 MB,
> > 670.68 MB, 670.84 MB, 670.97 MB, 663.27 MB, 649.43 MB, 649.40 MB, 649.51
> > MB, 649.48 MB, 649.57 MB, 641.18 MB, 641.64 MB, 617.73 MB, 582.14 MB,
> > 565.86 MB, 501.82 MB, 501.70 MB, 477.77 MB, 465.95 MB, 458.21 MB, 448.17
> > MB, 441.20 MB, 441.33 MB, 433.04 MB, 417.51 MB, 393.42 MB, 305.86 MB,
> > 298.02 MB, 228.39 MB, 207.19 MB, 183.50 MB, 135.72 MB
> >         ThreadUsage(500.000ms): 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
> > 5, 5, 5, 5, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 3, 3, 3, 3
> >          - AverageThreadTokens: 4.40
> >          - BloomFilterBytes: 0
> >          - PeakMemoryUsage: 671.38 MB (703996800)
> >          - PeakReservation: 0
> >          - PeakUsedReservation: 0
> >          - PerHostPeakMemUsage: 675.40 MB (708207971)
> >          - RowsProduced: 7.35M (7354152)
> >          - TotalNetworkReceiveTime: 0.000ns
> >          - TotalNetworkSendTime: 6s756ms
> >          - TotalStorageWaitTime: 79.999ms
> >          - TotalThreadsInvoluntaryContextSwitches: 243 (243)
> >          - TotalThreadsTotalWallClockTime: 1m15s
> >            - TotalThreadsSysTime: 445.885ms
> >            - TotalThreadsUserTime: 12s510ms
> >          - TotalThreadsVoluntaryContextSwitches: 18.23K (18232)
> >         Buffer pool:
> >            - AllocTime: 0.000ns
> >            - CumulativeAllocationBytes: 0
> >            - CumulativeAllocations: 0 (0)
> >            - PeakReservation: 0
> >            - PeakUnpinnedBytes: 0
> >            - PeakUsedReservation: 0
> >            - ReadIoBytes: 0
> >            - ReadIoOps: 0 (0)
> >            - ReadIoWaitTime: 0.000ns
> >            - ReservationLimit: 0
> >            - WriteIoBytes: 0
> >            - WriteIoOps: 0 (0)
> >            - WriteIoWaitTime: 0.000ns
> >         Fragment Instance Lifecycle Timings:
> >            - ExecTime: 17s387ms
> >              - ExecTreeExecTime: 87.999ms
> >            - OpenTime: 21.999ms
> >              - ExecTreeOpenTime: 0.000ns
> >            - PrepareTime: 51.999ms
> >              - ExecTreePrepareTime: 0.000ns
> >         KrpcDataStreamSender (dst_id=1):(Total: 17s338ms, non-child:
> > 10s580ms, % non-child: 61.02%)
> >           BytesSent(500.000ms): 0, 44.05 MB, 87.14 MB, 122.34 MB, 161.11
> > MB, 195.23 MB, 226.66 MB, 269.07 MB, 302.79 MB, 330.26 MB, 352.78 MB,
> > 389.26 MB, 427.13 MB, 452.09 MB, 481.46 MB, 516.94 MB, 561.69 MB, 602.17
> > MB, 648.44 MB, 693.18 MB, 730.98 MB, 771.50 MB, 799.23 MB, 824.71 MB,
> > 849.85 MB, 880.43 MB, 905.90 MB, 936.61 MB, 975.60 MB, 1.00 GB, 1.04 GB,
> > 1.07 GB, 1.10 GB, 1.14 GB, 1.18 GB
> >            - EosSent: 6 (6)
> >            - NetworkThroughput: 85.91 MB/sec
> >            - PeakMemoryUsage: 185.72 KB (190176)
> >            - RowsSent: 7.35M (7354152)
> >            - RpcFailure: 0 (0)
> >            - RpcRetry: 0 (0)
> >            - SerializeBatchTime: 5s306ms
> >            - TotalBytesSent: 1.22 GB (1309413246)
> >            - UncompressedRowBatchSize: 1.74 GB (1869500126)
> >         HDFS_SCAN_NODE (id=0):(Total: 83.999ms, non-child: 83.999ms, %
> > non-child: 100.00%)
> >           Hdfs split stats (<volume id>:<# splits>/<split lengths>):
> > 1:1/106.53 MB 0:1/253.72 MB 2:2/467.03 MB
> >           ExecOption: PARQUET Codegen Enabled, Codegen enabled: 4 out of 4
> >           Hdfs Read Thread Concurrency Bucket: 0:97.14% 1:0% 2:0% 3:2.857%
> > 4:0% 5:0% 6:0% 7:0% 8:0% 9:0% 10:0%
> >           File Formats: PARQUET/SNAPPY:4
> >           BytesRead(500.000ms): 4.14 MB, 576.20 MB, 576.20 MB, 576.20 MB,
> > 608.20 MB, 608.20 MB, 608.20 MB, 632.20 MB, 665.79 MB, 681.79 MB, 681.79
> > MB, 681.79 MB, 681.79 MB, 697.79 MB, 705.79 MB, 705.79 MB, 720.60 MB,
> > 744.60 MB, 760.60 MB, 766.48 MB, 782.48 MB, 793.78 MB, 793.78 MB, 797.27
> > MB, 806.25 MB, 814.25 MB, 814.25 MB, 814.25 MB, 819.60 MB, 827.60 MB,
> > 827.60 MB, 827.66 MB, 827.66 MB, 827.66 MB, 827.66 MB
> >            - FooterProcessingTime: (Avg: 999.999us ; Min: 999.999us ; Max:
> > 999.999us ; Number of samples: 4)
> >            - AverageHdfsReadThreadConcurrency: 0.09
> >            - AverageScannerThreadConcurrency: 3.40
> >            - BytesRead: 827.66 MB (867865141)
> >            - BytesReadDataNodeCache: 0
> >            - BytesReadLocal: 827.66 MB (867865141)
> >            - BytesReadRemoteUnexpected: 0
> >            - BytesReadShortCircuit: 827.66 MB (867865141)
> >            - CachedFileHandlesHitCount: 132 (132)
> >            - CachedFileHandlesMissCount: 0 (0)
> >            - CollectionItemsRead: 0 (0)
> >            - DecompressionTime: 1s181ms
> >            - MaxCompressedTextFileLength: 0
> >            - NumColumns: 10 (10)
> >            - NumDictFilteredRowGroups: 0 (0)
> >            - NumDisksAccessed: 3 (3)
> >            - NumRowGroups: 4 (4)
> >            - NumScannerThreadsStarted: 4 (4)
> >            - NumScannersWithNoReads: 0 (0)
> >            - NumStatsFilteredRowGroups: 0 (0)
> >            - PeakMemoryUsage: 670.94 MB (703529916)
> >            - PerReadThreadRawHdfsThroughput: 2.07 GB/sec
> >            - RemoteScanRanges: 0 (0)
> >            - RowBatchQueueGetWaitTime: 35.999ms
> >            - RowBatchQueuePutWaitTime: 56s280ms
> >            - RowsRead: 7.35M (7354152)
> >            - RowsReturned: 7.35M (7354152)
> >            - RowsReturnedRate: 87.55 M/sec
> >            - ScanRangesComplete: 4 (4)
> >            - ScannerThreadsInvoluntaryContextSwitches: 138 (138)
> >            - ScannerThreadsTotalWallClockTime: 58s549ms
> >              - MaterializeTupleTime(*): 813.999ms
> >              - ScannerThreadsSysTime: 111.444ms
> >              - ScannerThreadsUserTime: 2s065ms
> >            - ScannerThreadsVoluntaryContextSwitches: 7.13K (7126)
> >            - TotalRawHdfsOpenFileTime(*): 0.000ns
> >            - TotalRawHdfsReadTime(*): 390.999ms
> >            - TotalReadThroughput: 47.29 MB/sec
> >         CodeGen:(Total: 21.999ms, non-child: 21.999ms, % non-child:
> > 100.00%)
> >            - CodegenTime: 0.000ns
> >            - CompileTime: 3.999ms
> >            - LoadTime: 0.000ns
> >            - ModuleBitcodeSize: 1.95 MB (2039944)
> >            - NumFunctions: 7 (7)
> >            - NumInstructions: 105 (105)
> >            - OptimizationTime: 4.999ms
> >            - PeakMemoryUsage: 52.50 KB (53760)
> >            - PrepareTime: 12.999ms
> >       Instance 7e441f58868a8585:7452021000000001
> > (host=ip-10-197-3-207.eu-west-1.compute.internal:22000):(Total: 17s149ms,
> > non-child: 18.000ms, % non-child: 0.10%)
> >         Hdfs split stats (<volume id>:<# splits>/<split lengths>):
> > 1:2/467.06 MB 2:2/451.17 MB
> >         Fragment Instance Lifecycle Event Timeline: 17s149ms
> >            - Prepare Finished: 50.000ms (50.000ms)
> >            - Open Finished: 71.000ms (21.000ms)
> >            - First Batch Produced: 106.001ms (35.000ms)
> >            - First Batch Sent: 108.001ms (2.000ms)
> >            - ExecInternal Finished: 17s149ms (17s041ms)
> >         MemoryUsage(500.000ms): 709.14 MB, 714.39 MB, 715.27 MB, 715.40
> > MB, 715.64 MB, 715.54 MB, 715.79 MB, 715.99 MB, 716.04 MB, 715.96 MB,
> > 716.14 MB, 716.15 MB, 700.17 MB, 676.50 MB, 668.59 MB, 642.89 MB, 634.99
> > MB, 635.14 MB, 618.75 MB, 602.92 MB, 599.26 MB, 581.11 MB, 564.91 MB,
> > 550.86 MB, 543.13 MB, 519.24 MB, 503.23 MB, 447.54 MB, 375.65 MB, 359.90
> > MB, 274.91 MB, 207.42 MB, 167.55 MB, 36.20 MB
> >         ThreadUsage(500.000ms): 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
> > 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 4, 3, 3, 2
> >          - AverageThreadTokens: 4.76
> >          - BloomFilterBytes: 0
> >          - PeakMemoryUsage: 716.26 MB (751056705)
> >          - PeakReservation: 0
> >          - PeakUsedReservation: 0
> >          - PerHostPeakMemUsage: 720.28 MB (755267876)
> >          - RowsProduced: 8.18M (8182685)
> >          - TotalNetworkReceiveTime: 0.000ns
> >          - TotalNetworkSendTime: 5s229ms
> >          - TotalStorageWaitTime: 104.001ms
> >          - TotalThreadsInvoluntaryContextSwitches: 270 (270)
> >          - TotalThreadsTotalWallClockTime: 1m21s
> >            - TotalThreadsSysTime: 469.158ms
> >            - TotalThreadsUserTime: 13s793ms
> >          - TotalThreadsVoluntaryContextSwitches: 18.38K (18377)
> >         Buffer pool:
> >            - AllocTime: 0.000ns
> >            - CumulativeAllocationBytes: 0
> >            - CumulativeAllocations: 0 (0)
> >            - PeakReservation: 0
> >            - PeakUnpinnedBytes: 0
> >            - PeakUsedReservation: 0
> >            - ReadIoBytes: 0
> >            - ReadIoOps: 0 (0)
> >            - ReadIoWaitTime: 0.000ns
> >            - ReservationLimit: 0
> >            - WriteIoBytes: 0
> >            - WriteIoOps: 0 (0)
> >            - WriteIoWaitTime: 0.000ns
> >         Fragment Instance Lifecycle Timings:
> >            - ExecTime: 17s078ms
> >              - ExecTreeExecTime: 86.000ms
> >            - OpenTime: 21.000ms
> >              - ExecTreeOpenTime: 0.000ns
> >            - PrepareTime: 50.000ms
> >              - ExecTreePrepareTime: 0.000ns
> >         KrpcDataStreamSender (dst_id=1):(Total: 17s031ms, non-child:
> > 11s801ms, % non-child: 69.29%)
> >           BytesSent(500.000ms): 33.87 MB, 78.11 MB, 123.33 MB, 161.93 MB,
> > 205.16 MB, 238.71 MB, 280.35 MB, 326.33 MB, 361.42 MB, 387.00 MB, 424.14
> > MB, 464.72 MB, 500.18 MB, 537.81 MB, 584.75 MB, 626.81 MB, 667.53 MB,
> > 706.43 MB, 746.18 MB, 787.36 MB, 832.82 MB, 871.11 MB, 906.61 MB, 945.48
> > MB, 978.47 MB, 1014.22 MB, 1.03 GB, 1.07 GB, 1.13 GB, 1.17 GB, 1.21 GB,
> > 1.25 GB, 1.29 GB, 1.34 GB
> >            - EosSent: 6 (6)
> >            - NetworkThroughput: 97.38 MB/sec
> >            - PeakMemoryUsage: 185.72 KB (190176)
> >            - RowsSent: 8.18M (8182685)
> >            - RpcFailure: 0 (0)
> >            - RpcRetry: 0 (0)
> >            - SerializeBatchTime: 5s788ms
> >            - TotalBytesSent: 1.36 GB (1455381679)
> >            - UncompressedRowBatchSize: 1.93 GB (2076465996)
> >         HDFS_SCAN_NODE (id=0):(Total: 79.000ms, non-child: 79.000ms, %
> > non-child: 100.00%)
> >           Hdfs split stats (<volume id>:<# splits>/<split lengths>):
> > 1:2/467.06 MB 2:2/451.17 MB
> >           ExecOption: PARQUET Codegen Enabled, Codegen enabled: 4 out of 4
> >           Hdfs Read Thread Concurrency Bucket: 0:100% 1:0% 2:0% 3:0% 4:0%
> > 5:0% 6:0% 7:0% 8:0% 9:0% 10:0%
> >           File Formats: PARQUET/SNAPPY:4
> >           BytesRead(500.000ms): 623.91 MB, 623.91 MB, 639.91 MB, 655.91
> > MB, 655.91 MB, 663.91 MB, 703.91 MB, 743.91 MB, 749.38 MB, 749.38 MB,
> > 749.38 MB, 765.38 MB, 781.38 MB, 789.38 MB, 809.73 MB, 851.12 MB, 851.12
> > MB, 851.12 MB, 867.12 MB, 881.39 MB, 884.72 MB, 888.20 MB, 904.20 MB,
> > 905.20 MB, 905.20 MB, 905.20 MB, 905.20 MB, 918.54 MB, 918.54 MB, 918.54
> > MB, 918.61 MB, 918.61 MB, 918.61 MB, 918.61 MB
> >            - FooterProcessingTime: (Avg: 750.007us ; Min: 0.000ns ; Max:
> > 1.000ms ; Number of samples: 4)
> >            - AverageHdfsReadThreadConcurrency: 0.00
> >            - AverageScannerThreadConcurrency: 3.76
> >            - BytesRead: 918.61 MB (963230992)
> >            - BytesReadDataNodeCache: 0
> >            - BytesReadLocal: 918.61 MB (963230992)
> >            - BytesReadRemoteUnexpected: 0
> >            - BytesReadShortCircuit: 918.61 MB (963230992)
> >            - CachedFileHandlesHitCount: 143 (143)
> >            - CachedFileHandlesMissCount: 0 (0)
> >            - CollectionItemsRead: 0 (0)
> >            - DecompressionTime: 1s301ms
> >            - MaxCompressedTextFileLength: 0
> >            - NumColumns: 10 (10)
> >            - NumDictFilteredRowGroups: 0 (0)
> >            - NumDisksAccessed: 2 (2)
> >            - NumRowGroups: 4 (4)
> >            - NumScannerThreadsStarted: 4 (4)
> >            - NumScannersWithNoReads: 0 (0)
> >            - NumStatsFilteredRowGroups: 0 (0)
> >            - PeakMemoryUsage: 715.85 MB (750622592)
> >            - PerReadThreadRawHdfsThroughput: 2.34 GB/sec
> >            - RemoteScanRanges: 0 (0)
> >            - RowBatchQueueGetWaitTime: 35.000ms
> >            - RowBatchQueuePutWaitTime: 1m2s
> >            - RowsRead: 8.18M (8182685)
> >            - RowsReturned: 8.18M (8182685)
> >            - RowsReturnedRate: 103.58 M/sec
> >            - ScanRangesComplete: 4 (4)
> >            - ScannerThreadsInvoluntaryContextSwitches: 98 (98)
> >            - ScannerThreadsTotalWallClockTime: 1m4s
> >              - MaterializeTupleTime(*): 715.007ms
> >              - ScannerThreadsSysTime: 98.914ms
> >              - ScannerThreadsUserTime: 2s225ms
> >            - ScannerThreadsVoluntaryContextSwitches: 7.93K (7928)
> >            - TotalRawHdfsOpenFileTime(*): 0.000ns
> >            - TotalRawHdfsReadTime(*): 383.003ms
> >            - TotalReadThroughput: 54.04 MB/sec
> >         CodeGen:(Total: 21.000ms, non-child: 21.000ms, % non-child:
> > 100.00%)
> >            - CodegenTime: 1.000ms
> >            - CompileTime: 4.000ms
> >            - LoadTime: 0.000ns
> >            - ModuleBitcodeSize: 1.95 MB (2039944)
> >            - NumFunctions: 7 (7)
> >            - NumInstructions: 105 (105)
> >            - OptimizationTime: 5.000ms
> >            - PeakMemoryUsage: 52.50 KB (53760)
> >            - PrepareTime: 12.000ms
> >       Instance 7e441f58868a8585:7452021000000005
> > (host=ip-10-197-10-88.eu-west-1.compute.internal:22000):(Total: 17s105ms,
> > non-child: 19.000ms, % non-child: 0.11%)
> >         Hdfs split stats (<volume id>:<# splits>/<split lengths>):
> > 0:3/627.80 MB 1:1/253.66 MB
> >         Fragment Instance Lifecycle Event Timeline: 17s105ms
> >            - Prepare Finished: 51.000ms (51.000ms)
> >            - Open Finished: 72.000ms (21.000ms)
> >            - First Batch Produced: 107.000ms (35.000ms)
> >            - First Batch Sent: 109.000ms (2.000ms)
> >            - ExecInternal Finished: 17s105ms (16s996ms)
> >         MemoryUsage(500.000ms): 692.86 MB, 699.27 MB, 700.14 MB, 700.51
> > MB, 700.56 MB, 700.60 MB, 684.63 MB, 685.08 MB, 685.29 MB, 685.32 MB,
> > 685.18 MB, 677.29 MB, 661.38 MB, 637.53 MB, 613.87 MB, 605.89 MB, 589.97
> > MB, 524.56 MB, 511.58 MB, 495.58 MB, 484.73 MB, 476.73 MB, 476.90 MB,
> > 476.89 MB, 452.74 MB, 436.96 MB, 404.94 MB, 365.30 MB, 341.49 MB, 309.50
> > MB, 285.79 MB, 237.92 MB, 142.99 MB, 31.44 MB
> >         ThreadUsage(500.000ms): 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
> > 5, 5, 5, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 3, 1
> >          - AverageThreadTokens: 4.38
> >          - BloomFilterBytes: 0
> >          - PeakMemoryUsage: 700.84 MB (734883717)
> >          - PeakReservation: 0
> >          - PeakUsedReservation: 0
> >          - PerHostPeakMemUsage: 704.86 MB (739094888)
> >          - RowsProduced: 7.89M (7887894)
> >          - TotalNetworkReceiveTime: 0.000ns
> >          - TotalNetworkSendTime: 5s835ms
> >          - TotalStorageWaitTime: 183.001ms
> >          - TotalThreadsInvoluntaryContextSwitches: 231 (231)
> >          - TotalThreadsTotalWallClockTime: 1m15s
> >            - TotalThreadsSysTime: 445.228ms
> >            - TotalThreadsUserTime: 13s057ms
> >          - TotalThreadsVoluntaryContextSwitches: 18.54K (18544)
> >         Buffer pool:
> >            - AllocTime: 0.000ns
> >            - CumulativeAllocationBytes: 0
> >            - CumulativeAllocations: 0 (0)
> >            - PeakReservation: 0
> >            - PeakUnpinnedBytes: 0
> >            - PeakUsedReservation: 0
> >            - ReadIoBytes: 0
> >            - ReadIoOps: 0 (0)
> >            - ReadIoWaitTime: 0.000ns
> >            - ReservationLimit: 0
> >            - WriteIoBytes: 0
> >            - WriteIoOps: 0 (0)
> >            - WriteIoWaitTime: 0.000ns
> >         Fragment Instance Lifecycle Timings:
> >            - ExecTime: 17s033ms
> >              - ExecTreeExecTime: 86.000ms
> >            - OpenTime: 21.000ms
> >              - ExecTreeOpenTime: 0.000ns
> >            - PrepareTime: 51.000ms
> >              - ExecTreePrepareTime: 0.000ns
> >         KrpcDataStreamSender (dst_id=1):(Total: 16s982ms, non-child:
> > 11s143ms, % non-child: 65.62%)
> >           BytesSent(500.000ms): 26.93 MB, 66.51 MB, 110.22 MB, 150.25 MB,
> > 194.98 MB, 230.43 MB, 276.49 MB, 317.98 MB, 348.51 MB, 373.38 MB, 406.12
> > MB, 450.37 MB, 487.00 MB, 525.66 MB, 571.16 MB, 611.32 MB, 648.53 MB,
> > 684.93 MB, 720.33 MB, 759.24 MB, 805.05 MB, 839.81 MB, 871.11 MB, 904.63
> > MB, 939.44 MB, 970.81 MB, 1006.05 MB, 1.02 GB, 1.08 GB, 1.12 GB, 1.16 GB,
> > 1.20 GB, 1.25 GB, 1.29 GB
> >            - EosSent: 6 (6)
> >            - NetworkThroughput: 94.11 MB/sec
> >            - PeakMemoryUsage: 185.72 KB (190176)
> >            - RowsSent: 7.89M (7887894)
> >            - RpcFailure: 0 (0)
> >            - RpcRetry: 0 (0)
> >            - SerializeBatchTime: 5s436ms
> >            - TotalBytesSent: 1.30 GB (1399551209)
> >            - UncompressedRowBatchSize: 1.86 GB (1996363422)
> >         HDFS_SCAN_NODE (id=0):(Total: 83.000ms, non-child: 83.000ms, %
> > non-child: 100.00%)
> >           Hdfs split stats (<volume id>:<# splits>/<split lengths>):
> > 0:3/627.80 MB 1:1/253.66 MB
> >           ExecOption: PARQUET Codegen Enabled, Codegen enabled: 4 out of 4
> >           Hdfs Read Thread Concurrency Bucket: 0:100% 1:0% 2:0% 3:0% 4:0%
> > 5:0% 6:0% 7:0% 8:0% 9:0% 10:0%
> >           File Formats: PARQUET/SNAPPY:4
> >           BytesRead(500.000ms): 602.80 MB, 602.80 MB, 602.80 MB, 618.80
> > MB, 634.80 MB, 634.80 MB, 655.41 MB, 695.41 MB, 711.41 MB, 711.41 MB,
> > 711.41 MB, 719.41 MB, 735.41 MB, 743.41 MB, 751.41 MB, 775.41 MB, 791.41
> > MB, 803.16 MB, 803.16 MB, 827.16 MB, 831.51 MB, 836.43 MB, 844.43 MB,
> > 860.43 MB, 860.43 MB, 865.73 MB, 881.73 MB, 881.73 MB, 881.73 MB, 881.84
> > MB, 881.84 MB, 881.84 MB, 881.84 MB, 881.84 MB
> >            - FooterProcessingTime: (Avg: 750.006us ; Min: 0.000ns ; Max:
> > 1.000ms ; Number of samples: 4)
> >            - AverageHdfsReadThreadConcurrency: 0.00
> >            - AverageScannerThreadConcurrency: 3.38
> >            - BytesRead: 881.84 MB (924678748)
> >            - BytesReadDataNodeCache: 0
> >            - BytesReadLocal: 881.84 MB (924678748)
> >            - BytesReadRemoteUnexpected: 0
> >            - BytesReadShortCircuit: 881.84 MB (924678748)
> >            - CachedFileHandlesHitCount: 138 (138)
> >            - CachedFileHandlesMissCount: 0 (0)
> >            - CollectionItemsRead: 0 (0)
> >            - DecompressionTime: 1s231ms
> >            - MaxCompressedTextFileLength: 0
> >            - NumColumns: 10 (10)
> >            - NumDictFilteredRowGroups: 0 (0)
> >            - NumDisksAccessed: 2 (2)
> >            - NumRowGroups: 4 (4)
> >            - NumScannerThreadsStarted: 4 (4)
> >            - NumScannersWithNoReads: 0 (0)
> >            - NumStatsFilteredRowGroups: 0 (0)
> >            - PeakMemoryUsage: 700.39 MB (734416868)
> >            - PerReadThreadRawHdfsThroughput: 2.17 GB/sec
> >            - RemoteScanRanges: 0 (0)
> >            - RowBatchQueueGetWaitTime: 35.000ms
> >            - RowBatchQueuePutWaitTime: 56s298ms
> >            - RowsRead: 7.89M (7887894)
> >            - RowsReturned: 7.89M (7887894)
> >            - RowsReturnedRate: 95.03 M/sec
> >            - ScanRangesComplete: 4 (4)
> >            - ScannerThreadsInvoluntaryContextSwitches: 123 (123)
> >            - ScannerThreadsTotalWallClockTime: 58s719ms
> >              - MaterializeTupleTime(*): 816.006ms
> >              - ScannerThreadsSysTime: 95.015ms
> >              - ScannerThreadsUserTime: 2s161ms
> >            - ScannerThreadsVoluntaryContextSwitches: 7.65K (7647)
> >            - TotalRawHdfsOpenFileTime(*): 0.000ns
> >            - TotalRawHdfsReadTime(*): 397.002ms
> >            - TotalReadThroughput: 51.87 MB/sec
> >         CodeGen:(Total: 21.000ms, non-child: 21.000ms, % non-child:
> > 100.00%)
> >            - CodegenTime: 1.000ms
> >            - CompileTime: 4.000ms
> >            - LoadTime: 0.000ns
> >            - ModuleBitcodeSize: 1.95 MB (2039944)
> >            - NumFunctions: 7 (7)
> >            - NumInstructions: 105 (105)
> >            - OptimizationTime: 5.000ms
> >            - PeakMemoryUsage: 52.50 KB (53760)
> >            - PrepareTime: 12.000ms
> >       Instance 7e441f58868a8585:7452021000000002
> > (host=ip-10-197-11-142.eu-west-1.compute.internal:22000):(Total:
> > 16s970ms, non-child: 27.000ms, % non-child: 0.16%)
> >         Hdfs split stats (<volume id>:<# splits>/<split lengths>):
> > 1:1/167.11 MB 2:4/678.05 MB
> >         Fragment Instance Lifecycle Event Timeline: 16s971ms
> >            - Prepare Finished: 51.000ms (51.000ms)
> >            - Open Finished: 72.000ms (21.000ms)
> >            - First Batch Produced: 106.000ms (34.000ms)
> >            - First Batch Sent: 108.000ms (2.000ms)
> >            - ExecInternal Finished: 16s971ms (16s863ms)
> >         MemoryUsage(500.000ms): 765.77 MB, 772.47 MB, 772.79 MB, 773.80
> > MB, 773.89 MB, 774.25 MB, 773.85 MB, 762.22 MB, 754.42 MB, 746.55 MB,
> > 740.69 MB, 732.79 MB, 732.80 MB, 732.67 MB, 724.67 MB, 701.24 MB, 661.22
> > MB, 629.30 MB, 625.35 MB, 603.35 MB, 571.83 MB, 444.85 MB, 427.24 MB,
> > 403.24 MB, 394.97 MB, 387.30 MB, 367.22 MB, 359.05 MB, 261.78 MB, 245.86
> > MB, 198.36 MB, 113.10 MB, 81.26 MB, 7.70 MB
> >         ThreadUsage(500.000ms): 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
> > 6, 6, 6, 6, 6, 6, 6, 4, 4, 4, 4, 4, 4, 4, 3, 3, 3, 2, 2, 1
> >          - AverageThreadTokens: 4.94
> >          - BloomFilterBytes: 0
> >          - PeakMemoryUsage: 774.46 MB (812076675)
> >          - PeakReservation: 0
> >          - PeakUsedReservation: 0
> >          - PerHostPeakMemUsage: 3.40 GB (3650119082)
> >          - RowsProduced: 7.57M (7568704)
> >          - TotalNetworkReceiveTime: 0.000ns
> >          - TotalNetworkSendTime: 4s401ms
> >          - TotalStorageWaitTime: 254.001ms
> >          - TotalThreadsInvoluntaryContextSwitches: 1.65K (1652)
> >          - TotalThreadsTotalWallClockTime: 1m24s
> >            - TotalThreadsSysTime: 758.495ms
> >            - TotalThreadsUserTime: 14s414ms
> >          - TotalThreadsVoluntaryContextSwitches: 16.32K (16321)
> >         Buffer pool:
> >            - AllocTime: 0.000ns
> >            - CumulativeAllocationBytes: 0
> >            - CumulativeAllocations: 0 (0)
> >            - PeakReservation: 0
> >            - PeakUnpinnedBytes: 0
> >            - PeakUsedReservation: 0
> >            - ReadIoBytes: 0
> >            - ReadIoOps: 0 (0)
> >            - ReadIoWaitTime: 0.000ns
> >            - ReservationLimit: 0
> >            - WriteIoBytes: 0
> >            - WriteIoOps: 0 (0)
> >            - WriteIoWaitTime: 0.000ns
> >         Fragment Instance Lifecycle Timings:
> >            - ExecTime: 16s899ms
> >              - ExecTreeExecTime: 104.000ms
> >            - OpenTime: 21.000ms
> >              - ExecTreeOpenTime: 0.000ns
> >            - PrepareTime: 50.000ms
> >              - ExecTreePrepareTime: 0.000ns
> >         KrpcDataStreamSender (dst_id=1):(Total: 16s820ms, non-child:
> > 12s417ms, % non-child: 73.82%)
> >           BytesSent(500.000ms): 24.65 MB, 66.72 MB, 108.74 MB, 148.75 MB,
> > 187.00 MB, 227.95 MB, 271.53 MB, 313.40 MB, 343.59 MB, 369.08 MB, 401.56
> > MB, 443.03 MB, 485.01 MB, 529.95 MB, 576.96 MB, 617.74 MB, 653.17 MB,
> > 688.82 MB, 724.27 MB, 761.84 MB, 803.51 MB, 838.04 MB, 869.12 MB, 901.81
> > MB, 936.65 MB, 969.05 MB, 1003.70 MB, 1.01 GB, 1.06 GB, 1.10 GB, 1.14 GB,
> > 1.17 GB, 1.21 GB, 1.25 GB
> >            - EosSent: 6 (6)
> >            - NetworkThroughput: 110.72 MB/sec
> >            - PeakMemoryUsage: 185.72 KB (190176)
> >            - RowsSent: 7.57M (7568704)
> >            - RpcFailure: 0 (0)
> >            - RpcRetry: 0 (0)
> >            - SerializeBatchTime: 5s908ms
> >            - TotalBytesSent: 1.25 GB (1340563625)
> >            - UncompressedRowBatchSize: 1.78 GB (1910432069)
> >         HDFS_SCAN_NODE (id=0):(Total: 102.000ms, non-child: 102.000ms, %
> > non-child: 100.00%)
> >           Hdfs split stats (<volume id>:<# splits>/<split lengths>):
> > 1:1/167.11 MB 2:4/678.05 MB
> >           ExecOption: PARQUET Codegen Enabled, Codegen enabled: 5 out of 5
> >           Hdfs Read Thread Concurrency Bucket: 0:100% 1:0% 2:0% 3:0% 4:0%
> > 5:0% 6:0% 7:0% 8:0% 9:0% 10:0%
> >           File Formats: PARQUET/SNAPPY:5
> >           BytesRead(500.000ms): 647.74 MB, 647.74 MB, 647.74 MB, 663.74
> > MB, 663.74 MB, 687.74 MB, 687.74 MB, 695.27 MB, 707.59 MB, 707.59 MB,
> > 741.49 MB, 754.97 MB, 754.97 MB, 762.97 MB, 762.97 MB, 778.97 MB, 778.97
> > MB, 778.97 MB, 781.91 MB, 790.95 MB, 806.95 MB, 812.83 MB, 812.83 MB,
> > 812.83 MB, 827.11 MB, 827.11 MB, 829.57 MB, 837.57 MB, 837.57 MB, 845.57
> > MB, 845.57 MB, 845.63 MB, 845.63 MB, 845.63 MB
> >            - FooterProcessingTime: (Avg: 5.800ms ; Min: 0.000ns ; Max:
> > 26.000ms ; Number of samples: 5)
> >            - AverageHdfsReadThreadConcurrency: 0.00
> >            - AverageScannerThreadConcurrency: 3.94
> >            - BytesRead: 845.63 MB (886710526)
> >            - BytesReadDataNodeCache: 0
> >            - BytesReadLocal: 845.63 MB (886710526)
> >            - BytesReadRemoteUnexpected: 0
> >            - BytesReadShortCircuit: 845.63 MB (886710526)
> >            - CachedFileHandlesHitCount: 138 (138)
> >            - CachedFileHandlesMissCount: 0 (0)
> >            - CollectionItemsRead: 0 (0)
> >            - DecompressionTime: 1s374ms
> >            - MaxCompressedTextFileLength: 0
> >            - NumColumns: 10 (10)
> >            - NumDictFilteredRowGroups: 0 (0)
> >            - NumDisksAccessed: 2 (2)
> >            - NumRowGroups: 5 (5)
> >            - NumScannerThreadsStarted: 5 (5)
> >            - NumScannersWithNoReads: 0 (0)
> >            - NumStatsFilteredRowGroups: 0 (0)
> >            - PeakMemoryUsage: 774.11 MB (811708098)
> >            - PerReadThreadRawHdfsThroughput: 2.24 GB/sec
> >            - RemoteScanRanges: 0 (0)
> >            - RowBatchQueueGetWaitTime: 33.000ms
> >            - RowBatchQueuePutWaitTime: 1m4s
> >            - RowsRead: 7.57M (7568704)
> >            - RowsReturned: 7.57M (7568704)
> >            - RowsReturnedRate: 74.20 M/sec
> >            - ScanRangesComplete: 5 (5)
> >            - ScannerThreadsInvoluntaryContextSwitches: 135 (135)
> >            - ScannerThreadsTotalWallClockTime: 1m7s
> >              - MaterializeTupleTime(*): 882.004ms
> >              - ScannerThreadsSysTime: 175.091ms
> >              - ScannerThreadsUserTime: 2s317ms
> >            - ScannerThreadsVoluntaryContextSwitches: 7.35K (7351)
> >            - TotalRawHdfsOpenFileTime(*): 0.000ns
> >            - TotalRawHdfsReadTime(*): 369.002ms
> >            - TotalReadThroughput: 49.74 MB/sec
> >         CodeGen:(Total: 21.000ms, non-child: 21.000ms, % non-child:
> > 100.00%)
> >            - CodegenTime: 1.000ms
> >            - CompileTime: 3.000ms
> >            - LoadTime: 0.000ns
> >            - ModuleBitcodeSize: 1.95 MB (2039944)
> >            - NumFunctions: 7 (7)
> >            - NumInstructions: 105 (105)
> >            - OptimizationTime: 6.000ms
> >            - PeakMemoryUsage: 52.50 KB (53760)
> >            - PrepareTime: 12.000ms
> >       Instance 7e441f58868a8585:7452021000000004
> > (host=ip-10-197-29-94.eu-west-1.compute.internal:22000):(Total: 13s276ms,
> > non-child: 15.000ms, % non-child: 0.11%)
> >         Hdfs split stats (<volume id>:<# splits>/<split lengths>):
> > 1:1/197.15 MB 2:1/253.70 MB 0:1/212.81 MB
> >         Fragment Instance Lifecycle Event Timeline: 13s277ms
> >            - Prepare Finished: 51.000ms (51.000ms)
> >            - Open Finished: 74.000ms (23.000ms)
> >            - First Batch Produced: 111.000ms (37.000ms)
> >            - First Batch Sent: 114.000ms (3.000ms)
> >            - ExecInternal Finished: 13s277ms (13s163ms)
> >         MemoryUsage(500.000ms): 0, 533.62 MB, 535.88 MB, 536.01 MB, 535.96
> > MB, 536.17 MB, 536.19 MB, 536.50 MB, 536.74 MB, 536.63 MB, 528.69 MB,
> > 505.96 MB, 478.24 MB, 470.17 MB, 470.15 MB, 454.34 MB, 446.31 MB, 434.25
> > MB, 412.47 MB, 397.38 MB, 389.31 MB, 373.32 MB, 349.35 MB, 277.69 MB,
> > 190.58 MB, 119.54 MB, 79.70 MB
> >         ThreadUsage(500.000ms): 1, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4,
> > 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 3, 2, 2
> >          - AverageThreadTokens: 3.70
> >          - BloomFilterBytes: 0
> >          - PeakMemoryUsage: 536.93 MB (563013699)
> >          - PeakReservation: 0
> >          - PeakUsedReservation: 0
> >          - PerHostPeakMemUsage: 2.73 GB (2926852205)
> >          - RowsProduced: 5.87M (5872115)
> >          - TotalNetworkReceiveTime: 0.000ns
> >          - TotalNetworkSendTime: 3s474ms
> >          - TotalStorageWaitTime: 33.000ms
> >          - TotalThreadsInvoluntaryContextSwitches: 2.79K (2792)
> >          - TotalThreadsTotalWallClockTime: 50s268ms
> >            - TotalThreadsSysTime: 571.274ms
> >            - TotalThreadsUserTime: 11s141ms
> >          - TotalThreadsVoluntaryContextSwitches: 13.29K (13286)
> >         Buffer pool:
> >            - AllocTime: 0.000ns
> >            - CumulativeAllocationBytes: 0
> >            - CumulativeAllocations: 0 (0)
> >            - PeakReservation: 0
> >            - PeakUnpinnedBytes: 0
> >            - PeakUsedReservation: 0
> >            - ReadIoBytes: 0
> >            - ReadIoOps: 0 (0)
> >            - ReadIoWaitTime: 0.000ns
> >            - ReservationLimit: 0
> >            - WriteIoBytes: 0
> >            - WriteIoOps: 0 (0)
> >            - WriteIoWaitTime: 0.000ns
> >         Fragment Instance Lifecycle Timings:
> >            - ExecTime: 13s203ms
> >              - ExecTreeExecTime: 121.000ms
> >            - OpenTime: 23.000ms
> >              - ExecTreeOpenTime: 0.000ns
> >            - PrepareTime: 50.000ms
> >              - ExecTreePrepareTime: 0.000ns
> >         KrpcDataStreamSender (dst_id=1):(Total: 13s121ms, non-child:
> > 9s642ms, % non-child: 73.49%)
> >           BytesSent(500.000ms): 32.44 MB, 72.73 MB, 116.97 MB, 157.51 MB,
> > 196.82 MB, 228.57 MB, 267.48 MB, 310.26 MB, 338.82 MB, 370.78 MB, 413.11
> > MB, 446.41 MB, 480.28 MB, 516.76 MB, 558.14 MB, 591.15 MB, 625.03 MB,
> > 659.01 MB, 692.28 MB, 731.28 MB, 769.56 MB, 812.39 MB, 852.55 MB, 896.49
> > MB, 938.39 MB, 981.76 MB
> >            - EosSent: 6 (6)
> >            - NetworkThroughput: 103.73 MB/sec
> >            - PeakMemoryUsage: 185.72 KB (190176)
> >            - RowsSent: 5.87M (5872115)
> >            - RpcFailure: 0 (0)
> >            - RpcRetry: 0 (0)
> >            - SerializeBatchTime: 4s538ms
> >            - TotalBytesSent: 1001.82 MB (1050482229)
> >            - UncompressedRowBatchSize: 1.40 GB (1499344163)
> >         HDFS_SCAN_NODE (id=0):(Total: 117.000ms, non-child: 117.000ms, %
> > non-child: 100.00%)
> >           Hdfs split stats (<volume id>:<# splits>/<split lengths>):
> > 1:1/197.15 MB 2:1/253.70 MB 0:1/212.81 MB
> >           ExecOption: PARQUET Codegen Enabled, Codegen enabled: 3 out of 3
> >           Hdfs Read Thread Concurrency Bucket: 0:100% 1:0% 2:0% 3:0% 4:0%
> > 5:0% 6:0% 7:0% 8:0% 9:0% 10:0%
> >           File Formats: PARQUET/SNAPPY:3
> >           BytesRead(500.000ms): 0, 458.13 MB, 458.13 MB, 482.13 MB, 482.13
> > MB, 498.13 MB, 543.55 MB, 551.55 MB, 551.55 MB, 567.55 MB, 575.55 MB,
> > 588.81 MB, 607.23 MB, 623.23 MB, 623.23 MB, 629.39 MB, 645.39 MB, 648.70
> > MB, 649.72 MB, 650.60 MB, 658.60 MB, 658.60 MB, 658.60 MB, 663.94 MB,
> > 663.94 MB, 663.94 MB, 663.94 MB
> >            - FooterProcessingTime: (Avg: 666.669us ; Min: 0.000ns ; Max:
> > 1.000ms ; Number of samples: 3)
> >            - AverageHdfsReadThreadConcurrency: 0.00
> >            - AverageScannerThreadConcurrency: 2.81
> >            - BytesRead: 663.94 MB (696193384)
> >            - BytesReadDataNodeCache: 0
> >            - BytesReadLocal: 663.94 MB (696193384)
> >            - BytesReadRemoteUnexpected: 0
> >            - BytesReadShortCircuit: 663.94 MB (696193384)
> >            - CachedFileHandlesHitCount: 104 (104)
> >            - CachedFileHandlesMissCount: 0 (0)
> >            - CollectionItemsRead: 0 (0)
> >            - DecompressionTime: 995.003ms
> >            - MaxCompressedTextFileLength: 0
> >            - NumColumns: 10 (10)
> >            - NumDictFilteredRowGroups: 0 (0)
> >            - NumDisksAccessed: 3 (3)
> >            - NumRowGroups: 3 (3)
> >            - NumScannerThreadsStarted: 3 (3)
> >            - NumScannersWithNoReads: 0 (0)
> >            - NumStatsFilteredRowGroups: 0 (0)
> >            - PeakMemoryUsage: 536.78 MB (562855983)
> >            - PerReadThreadRawHdfsThroughput: 1.98 GB/sec
> >            - RemoteScanRanges: 0 (0)
> >            - RowBatchQueueGetWaitTime: 37.000ms
> >            - RowBatchQueuePutWaitTime: 35s136ms
> >            - RowsRead: 5.87M (5872115)
> >            - RowsReturned: 5.87M (5872115)
> >            - RowsReturnedRate: 50.19 M/sec
> >            - ScanRangesComplete: 3 (3)
> >            - ScannerThreadsInvoluntaryContextSwitches: 223 (223)
> >            - ScannerThreadsTotalWallClockTime: 37s042ms
> >              - MaterializeTupleTime(*): 687.002ms
> >              - ScannerThreadsSysTime: 118.760ms
> >              - ScannerThreadsUserTime: 1s825ms
> >            - ScannerThreadsVoluntaryContextSwitches: 5.67K (5674)
> >            - TotalRawHdfsOpenFileTime(*): 0.000ns
> >            - TotalRawHdfsReadTime(*): 327.001ms
> >            - TotalReadThroughput: 49.18 MB/sec
> >         CodeGen:(Total: 23.000ms, non-child: 23.000ms, % non-child:
> > 100.00%)
> >            - CodegenTime: 0.000ns
> >            - CompileTime: 4.000ms
> >            - LoadTime: 0.000ns
> >            - ModuleBitcodeSize: 1.95 MB (2039944)
> >            - NumFunctions: 7 (7)
> >            - NumInstructions: 105 (105)
> >            - OptimizationTime: 6.000ms
> >            - PeakMemoryUsage: 52.50 KB (53760)
> >            - PrepareTime: 12.000ms
> >
> 
> 
> 
> -- 
> Balázs Jeszenszky
>