You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@drill.apache.org by "Deneche A. Hakim (JIRA)" <ji...@apache.org> on 2015/12/15 20:37:46 UTC

[jira] [Comment Edited] (DRILL-4190) TPCDS queries are running out of memory when hash join is disabled

    [ https://issues.apache.org/jira/browse/DRILL-4190?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15058622#comment-15058622 ] 

Deneche A. Hakim edited comment on DRILL-4190 at 12/15/15 7:37 PM:
-------------------------------------------------------------------

If I'm not mistaken, here is what's causing the query to fail:
- a sort below a merge join spilled to disk (this is important), then starts passing data downstream. This means that whenever it loads a batch from disk it uses it's own allocator (bound by the query's sort memory limit) to allocate the batch
- MergeJoin's RecordIterator seems to hold all incoming batches in memory until the operator is closed. This causes the sort allocator to hit it's allocation limit and the query fails.

The same query runs fine in 1.2.0 which suggests prior to RecordIterator, MergeJoin didn't hold all batches in memory (I asked a question on the dev list to confirm this point)

[~aah] can you please confirm if I am correct ? thanks


was (Author: adeneche):
If I'm not mistaken, here is what's causing the query to fail:
- a sort below a merge join spilled to disk (this is important), then starts passing data downstream. This means that whenever it loads a batch from disk it uses it's own allocator (bound by the query's sort memory limit) to allocate the batch
- MergeJoin's RecordIterator seems to hold all incoming batches in memory until the operator is closed. This causes the sort allocator to hit it's allocation limit and the query fails.

The same query runs fine in 1.2.0 which suggests prior to RecordIterator, MergeJoin didn't hold all batches in memory (I asked a question on the dev list to confirm this point)

> TPCDS queries are running out of memory when hash join is disabled
> ------------------------------------------------------------------
>
>                 Key: DRILL-4190
>                 URL: https://issues.apache.org/jira/browse/DRILL-4190
>             Project: Apache Drill
>          Issue Type: Bug
>          Components: Execution - Relational Operators
>    Affects Versions: 1.3.0, 1.4.0, 1.5.0
>            Reporter: Victoria Markman
>            Assignee: Deneche A. Hakim
>            Priority: Blocker
>         Attachments: 2990f5f8-ec64-1223-c1d8-97dd7e601cee.sys.drill, exception.log, query3.sql
>
>
> TPCDS queries with the latest 1.4.0 release when hash join is disabled:
> 22 queries fail with out of memory 
> 2 wrong results (I did not validate the nature of wrong result yet)
> Only query97.sql is a legitimate failure: we don't support full outer join with the merge join.
> It is important to understand what has changed between 1.2.0 and 1.4.0 that made these tests not runnable with the same configuration. 
> Same tests with the same drill configuration pass in 1.2.0 release.
> (I hope I did not make a mistake somewhere in my cluster setup :))
> {code}
> 0: jdbc:drill:schema=dfs> select * from sys.version;
> +-----------------+-------------------------------------------+---------------------------------------------------------------------+----------------------------+--------------+----------------------------+
> |     version     |                 commit_id                 |                           commit_message                            |        commit_time         | build_email  |         build_time         |
> +-----------------+-------------------------------------------+---------------------------------------------------------------------+----------------------------+--------------+----------------------------+
> | 1.4.0-SNAPSHOT  | b9068117177c3b47025f52c00f67938e0c3e4732  | DRILL-4165 Add a precondition for size of merge join record batch.  | 08.12.2015 @ 01:25:34 UTC  | Unknown      | 08.12.2015 @ 03:36:25 UTC  |
> +-----------------+-------------------------------------------+---------------------------------------------------------------------+----------------------------+--------------+----------------------------+
> 1 row selected (2.211 seconds)
> Execution Failures:
> /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query50.sql
> /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query33.sql
> /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query74.sql
> /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query68.sql
> /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query34.sql
> /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query21.sql
> /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query46.sql
> /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query91.sql
> /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query59.sql
> /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query3.sql
> /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query66.sql
> /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query84.sql
> /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query97.sql
> /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query19.sql
> /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query96.sql
> /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query43.sql
> /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query15.sql
> /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query2.sql
> /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query60.sql
> /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query79.sql
> /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query73.sql
> /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query45.sql
> Verification Failures
> /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query52.sql
> /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query40.sql
> Timeout Failures
> ----------------------------------------------------------------------------------------------------------------
> Passing tests: 3
> Execution Failures: 22
> VerificationFailures: 2
> Timeouts: 0
> Canceled: 0
> {code}
> {code}
> 0: jdbc:drill:schema=dfs> select * from sys.version;
> +-----------+----------------+-------------+-------------+------------+
> | commit_id | commit_message | commit_time | build_email | build_time |
> +-----------+----------------+-------------+-------------+------------+
> | f1100a79b4e4fbb1b58b35b0230edff137588777 | DRILL-3947: Use setSafe() for date, time, timestamp types while populating pruning vector (other types were already using setSafe). | 19.10.2015 @ 16:02:00 UTC | Unknown | 19.10.2015 @ 16:25:21 UTC |
> +-----------+----------------+-------------+-------------+------------+
> 1 row selected (2.79 seconds)
> PASS (1.543 min) /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query68.sql (connection: 1681915178)
> PASS (29.36 s) /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query34.sql (connection: 1681915178)
> PASS (3.311 min) /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query21.sql (connection: 1681915178)
> PASS (1.447 min) /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query46.sql (connection: 1681915178)
> PASS (34.53 s) /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query76.sql (connection: 1681915178)
> PASS (47.13 s) /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query91.sql (connection: 1681915178)
> PASS (1.151 min) /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query59.sql (connection: 1681915178)
> PASS (32.29 s) /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query3.sql (connection: 1681915178)
> PASS (1.939 min) /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query66.sql (connection: 1681915178)
> PASS (19.26 s) /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query84.sql (connection: 1681915178)
> PASS (1.243 min) /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query40.sql (connection: 1681915178)
> [#37] Query failed:
> oadd.org.apache.drill.common.exceptions.UserRemoteException: SYSTEM ERROR: IllegalArgumentException: Full outer join not currently supported
> [Error Id: 9a400ac2-3f1d-428c-9dc6-5f556cb520aa on atsqa4-133.qa.lab:31010]
>         at oadd.org.apache.drill.exec.rpc.user.QueryResultHandler.resultArrived(QueryResultHandler.java:118)
>         at oadd.org.apache.drill.exec.rpc.user.UserClient.handleReponse(UserClient.java:110)
>         at oadd.org.apache.drill.exec.rpc.BasicClientWithConnection.handle(BasicClientWithConnection.java:47)
>         at oadd.org.apache.drill.exec.rpc.BasicClientWithConnection.handle(BasicClientWithConnection.java:32)
>         at oadd.org.apache.drill.exec.rpc.RpcBus.handle(RpcBus.java:61)
>         at oadd.org.apache.drill.exec.rpc.RpcBus$InboundHandler.decode(RpcBus.java:233)
>         at oadd.org.apache.drill.exec.rpc.RpcBus$InboundHandler.decode(RpcBus.java:205)
>         at oadd.io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:89)
>         at oadd.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:339)
>         at oadd.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:324)
>         at oadd.io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:254)
>         at oadd.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:339)
>         at oadd.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:324)
>         at oadd.io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103)
>         at oadd.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:339)
>         at oadd.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:324)
>         at oadd.io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:242)
>         at oadd.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:339)
>         at oadd.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:324)
>         at oadd.io.netty.channel.ChannelInboundHandlerAdapter.channelRead(ChannelInboundHandlerAdapter.java:86)
>         at oadd.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:339)
>         at oadd.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:324)
>         at oadd.io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:847)
>         at oadd.io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:131)
>         at oadd.io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511)
>         at oadd.io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468)
>         at oadd.io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382)
>         at oadd.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354)
>         at oadd.io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:111)
>         at java.lang.Thread.run(Thread.java:745)
> EXECUTION_FAILURE (2.814 s) /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query97.sql (connection: 1681915178)
> PASS (57.04 s) /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query19.sql (connection: 1681915178)
> PASS (24.01 s) /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query96.sql (connection: 1681915178)
> PASS (28.77 s) /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query43.sql (connection: 1681915178)
> PASS (1.833 min) /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query93.sql (connection: 1681915178)
> PASS (38.84 s) /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query15.sql (connection: 1681915178)
> PASS (55.82 s) /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query2.sql (connection: 1681915178)
> PASS (1.308 min) /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query60.sql (connection: 1681915178)
> PASS (1.116 min) /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query79.sql (connection: 1681915178)
> PASS (27.79 s) /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query73.sql (connection: 1681915178)
> PASS (39.85 s) /root/drill-tests-new/framework/resources/Advanced/tpcds/tpcds_sf100/original/query45.sql (connection: 1681915178)
> {code}
> *Cluster configuration:*
> - 4 nodes
> - 48 GB direct memory
> - 10GB memory allocated to sort
> - timeout setup for the framework = 600 seconds
> - queries were executed one at a time
> *System settings:*
> {code}
> 0: jdbc:drill:schema=dfs> select * from sys.options where status like '%CHANGED%';
> +-------------------------------------------+----------+---------+----------+--------------+-------------+-----------+------------+
> |                   name                    |   kind   |  type   |  status  |   num_val    | string_val  | bool_val  | float_val  |
> +-------------------------------------------+----------+---------+----------+--------------+-------------+-----------+------------+
> | planner.enable_decimal_data_type          | BOOLEAN  | SYSTEM  | CHANGED  | null         | null        | true      | null       |
> | planner.enable_hashjoin                   | BOOLEAN  | SYSTEM  | CHANGED  | null         | null        | false     | null       |
> | planner.memory.max_query_memory_per_node  | LONG     | SYSTEM  | CHANGED  | 10737418240  | null        | null      | null       |
> +-------------------------------------------+----------+---------+----------+--------------+-------------+-----------+------------+
> 3 rows selected (3.464 seconds)
> {code}
> TPCDS queries that were executed from the public test framework: 
> ./run.sh -s Advanced/tpcds/tpcds_sf100/original -g smoke -t 600
> More details shortly.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)