It seems to be a bit faster now, completes in about 5 seconds. I've set the planner.width.max_per_node higher: alter system set `planner.width.max_per_node` = 30
I was already doing: REFRESH TABLE METADATA aws_s3.`fact/viewership_120.parquet`; So it should have been using the metadata cache file, correct? https://gist.github.com/rxcanavan/a3093dc4a66d2d03fc1987ec096c3128 Now it seems like it's scanning more files but faster 2-3 seconds, because the files are smaller. I'm going to try to do some joins now and see what kind of performance I can get. Do you think it would be worth while to compare the performance of s3 vs hdfs with my files? I'm guessing that I'd see a significant performance increase because of data locality with hdfs. Thanks for all of your help! On Mon, May 2, 2016 at 12:31 PM, Jacques Nadeau <[email protected]> wrote: > For the example you sent, most of the time is in the reading viewership > table. You can look at the "03-xx-13 - PARQUET_ROW_GROUP_SCAN" section of > the profile to see this. There is a bimodal distribution here of 1 file > versus 2 files (and you can see the runtime differences). I'd suggestion > generating more/smaller files to get maximum performance (and bumping up > the maximum width per node). > > You might also be able to prune the 3s planning time by using the parquet > metadata cache file that Drill can generate. > > -- > Jacques Nadeau > CTO and Co-Founder, Dremio > > On Mon, May 2, 2016 at 7:55 AM, Rob Canavan <[email protected]> wrote: > > > Thanks, Jacques. I've attached the profile and I'm still trying to get > > familiar with it. I had to compress it to send it. > > > > > > On Sun, May 1, 2016 at 9:37 PM, Jacques Nadeau <[email protected]> > wrote: > > > >> Hey Rob, > >> > >> Can you post your profile on gist or send it to Abhishek and myself? You > >> can get the profile by navigating to it in the web ui and then changing > >> the > >> url to .json to download the file. The key metrics to look at are what > >> level of parallelization Drill is using and how much time it is waiting > >> for > >> S3. You'll also need to look at the first start time to see how long > >> planning is taking. > >> > >> thanks, > >> Jacques > >> > >> -- > >> Jacques Nadeau > >> CTO and Co-Founder, Dremio > >> > >> On Sun, May 1, 2016 at 12:11 PM, Rob Canavan <[email protected]> > wrote: > >> > >> > Thanks Abhisek, that seems to have work. I can now join the larger > >> file to > >> > the smaller dimension. I have a some more questions regarding S3 > >> > performance that maybe you could help with. I'm doing some > performance > >> > evaluation against a postgresl data warehouse that I've built. My > test > >> is > >> > to compare the usage of an s3 backed drill platform reading parquet > >> files > >> > and the dw. I'm currently testing with ~ 250 million rows in a table > >> and > >> > doing some simple aggregations and counting. My next test will be > >> joining > >> > the table to multiple dimensions and doing star type BI queries. > >> > > >> > Currently, I'm doing a sum of a float column for all the rows in the > >> table > >> > (250m), drill is taking ~10 seconds to return where the postgres query > >> > takes 5 seconds. The thing that I notice when watching the drill bits > >> with > >> > htop is that not all of the CPUs are being used. I have 4 nodes with > 16 > >> > core each, and I'm only seeing 8 used on each node. I tried upping > the > >> > planner.width.max_per_node to a higher value (drill defaulted it to > >> 12... > >> > guessing 16*.7), but saw no change. Are there any > recommendations/best > >> > practices to make sure drill reads parquet over s3 as fast as > possible? > >> > I'm guessing the delay is caused by the network latency between by ec2 > >> > instances and s3, but it could also be that I need to learn how to > >> better > >> > tune drill. > >> > > >> > The parquet file I'm querying against is generated by Spark and is > >> > repartitioned into 60 files. I've tested against 1 large file, but > that > >> > seems a little slower. Would it be better to recreate the parquet > >> files as > >> > Drill CTAS tables? > >> > > >> > query: > >> > select sum(cast(seg_share as float)) from > >> > aws_s3.`fact/viewership_60.parquet`; > >> > > >> > > >> > Thanks again for your help! > >> > > >> > > >> > > >> > > >> > > >> > On Sat, Apr 30, 2016 at 10:56 AM, Abhishek Girish <[email protected]> > >> > wrote: > >> > > >> > > Can you check if this resolves the issue? > >> > > > >> > > > >> > > >> > http://drill.apache.org/docs/s3-storage-plugin/#quering-parquet-format-files-on-s3 > >> > > > >> > > -Abhishek > >> > > > >> > > On Saturday, April 30, 2016, Rob Canavan <[email protected]> > wrote: > >> > > > >> > > > I'm trying to join two parquet files that I have stored in S3 and > >> the > >> > > query > >> > > > keeps timing out: > >> > > > > >> > > > select * from aws_s3.`dim/market_header.parquet` a inner join > >> > > > aws_s3.n`dim/market_program.parquet` b on a.market_no = > b.market_no; > >> > > > > >> > > > I can run counts and aggs on the two tables fine: > >> > > > > >> > > > select count(*) from aws_s3.`dim/market_header.parquet`; > >> > > > +---------+ > >> > > > | EXPR$0 | > >> > > > +---------+ > >> > > > | 420 | > >> > > > +---------+ > >> > > > 1 row selected (0.984 seconds) > >> > > > > >> > > > > >> > > > select count(*) from aws_s3.`dim/market_program.parquet`; > >> > > > +----------+ > >> > > > | EXPR$0 | > >> > > > +----------+ > >> > > > | 1035318 | > >> > > > +----------+ > >> > > > 1 row selected (0.738 seconds) > >> > > > > >> > > > select sum(cast(series_no as float)) from > >> > > > aws_s3.`dim/market_program.parquet` as b limit 10; > >> > > > +--------------------+ > >> > > > | EXPR$0 | > >> > > > +--------------------+ > >> > > > | 2.072667694581E12 | > >> > > > +--------------------+ > >> > > > 1 row selected (1.63 seconds) > >> > > > > >> > > > > >> > > > When I run the query to join them, after a few minutes I get: > >> > > > > >> > > > Error: SYSTEM ERROR: ConnectionPoolTimeoutException: Timeout > waiting > >> > for > >> > > > connection from pool > >> > > > > >> > > > Fragment 0:0 > >> > > > > >> > > > [Error Id: 45a6055c-08af-4ecd-b670-8dbcf196673f on ....... > >> > > > amazonaws.com:31010] (state=,code=0) > >> > > > > >> > > > > >> > > > This is a distributed setup with 4 drillbits. 16 core each with > 64 > >> GB > >> > > > memory on each. My drill-env.sh has: > >> > > > > >> > > > DRILL_MAX_DIRECT_MEMORY="55G" > >> > > > DRILL_HEAP="4G" > >> > > > > >> > > > > >> > > > There's also a stacktrace in sqlline.log > >> > > > > >> > > > [Error Id: 45a6055c-08af-4ecd-b670-8dbcf196673f on . > >> > > > compute-1.amazonaws.com:31010] > >> > > > at > >> > > > > >> > > > > >> > > > >> > > >> > org.apache.drill.exec.rpc.user.QueryResultHandler.resultArrived(QueryResultHandler.java:119) > >> > > > [drill-java-exec-1.6.0.jar:1.6.0] > >> > > > at > >> > > > > >> > > > > >> > > > >> > > >> > org.apache.drill.exec.rpc.user.UserClient.handleReponse(UserClient.java:113) > >> > > > [drill-java-exec-1.6.0.jar:1.6.0] > >> > > > at > >> > > > > >> > > > > >> > > > >> > > >> > org.apache.drill.exec.rpc.BasicClientWithConnection.handle(BasicClientWithConnection.java:46) > >> > > > [drill-rpc-1.6.0.jar:1.6.0] > >> > > > at > >> > > > > >> > > > > >> > > > >> > > >> > org.apache.drill.exec.rpc.BasicClientWithConnection.handle(BasicClientWithConnection.java:31) > >> > > > [drill-rpc-1.6.0.jar:1.6.0] > >> > > > at org.apache.drill.exec.rpc.RpcBus.handle(RpcBus.java:67) > >> > > > [drill-rpc-1.6.0.jar:1.6.0] > >> > > > at > >> > > > org.apache.drill.exec.rpc.RpcBus$RequestEvent.run(RpcBus.java:374) > >> > > > [drill-rpc-1.6.0.jar:1.6.0] > >> > > > at > >> > > > > >> > > > > >> > > > >> > > >> > org.apache.drill.common.SerializedExecutor$RunnableProcessor.run(SerializedExecutor.java:89) > >> > > > [drill-rpc-1.6.0.jar:1.6.0] > >> > > > at > >> > > > > >> org.apache.drill.exec.rpc.RpcBus$SameExecutor.execute(RpcBus.java:252) > >> > > > [drill-rpc-1.6.0.jar:1.6.0] > >> > > > at > >> > > > > >> > > > > >> > > > >> > > >> > org.apache.drill.common.SerializedExecutor.execute(SerializedExecutor.java:123) > >> > > > [drill-rpc-1.6.0.jar:1.6.0] > >> > > > at > >> > > > > >> org.apache.drill.exec.rpc.RpcBus$InboundHandler.decode(RpcBus.java:285) > >> > > > [drill-rpc-1.6.0.jar:1.6.0] > >> > > > at > >> > > > > >> org.apache.drill.exec.rpc.RpcBus$InboundHandler.decode(RpcBus.java:257) > >> > > > [drill-rpc-1.6.0.jar:1.6.0] > >> > > > at > >> > > > > >> > > > > >> > > > >> > > >> > io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:89) > >> > > > [netty-codec-4.0.27.Final.jar:4.0.27.Final] > >> > > > at > >> > > > > >> > > > > >> > > > >> > > >> > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:339) > >> > > > [netty-transport-4.0.27.Final.jar:4.0.27.Final] > >> > > > at > >> > > > > >> > > > > >> > > > >> > > >> > io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:324) > >> > > > [netty-transport-4.0.27.Final.jar:4.0.27.Final] > >> > > > at > >> > > > > >> > > > > >> > > > >> > > >> > io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:254) > >> > > > [netty-handler-4.0.27.Final.jar:4.0.27.Final] > >> > > > at > >> > > > > >> > > > > >> > > > >> > > >> > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:339) > >> > > > [netty-transport-4.0.27.Final.jar:4.0.27.Final] > >> > > > at > >> > > > > >> > > > > >> > > > >> > > >> > io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:324) > >> > > > [netty-transport-4.0.27.Final.jar:4.0.27.Final] > >> > > > at > >> > > > > >> > > > > >> > > > >> > > >> > io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103) > >> > > > [netty-codec-4.0.27.Final.jar:4.0.27.Final] > >> > > > at > >> > > > > >> > > > > >> > > > >> > > >> > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:339) > >> > > > [netty-transport-4.0.27.Final.jar:4.0.27.Final] > >> > > > at > >> > > > > >> > > > > >> > > > >> > > >> > io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:324) > >> > > > [netty-transport-4.0.27.Final.jar:4.0.27.Final] > >> > > > at > >> > > > > >> > > > > >> > > > >> > > >> > io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:242) > >> > > > [netty-codec-4.0.27.Final.jar:4.0.27.Final] > >> > > > at > >> > > > > >> > > > > >> > > > >> > > >> > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:339) > >> > > > [netty-transport-4.0.27.Final.jar:4.0.27.Final] > >> > > > at > >> > > > > >> > > > > >> > > > >> > > >> > io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:324) > >> > > > [netty-transport-4.0.27.Final.jar:4.0.27.Final] > >> > > > at > >> > > > > >> > > > > >> > > > >> > > >> > io.netty.channel.ChannelInboundHandlerAdapter.channelRead(ChannelInboundHandlerAdapter.java:86) > >> > > > [netty-transport-4.0.27.Final.jar:4.0.27.Final] > >> > > > at > >> > > > > >> > > > > >> > > > >> > > >> > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:339) > >> > > > [netty-transport-4.0.27.Final.jar:4.0.27.Final] > >> > > > at > >> > > > > >> > > > > >> > > > >> > > >> > io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:324) > >> > > > [netty-transport-4.0.27.Final.jar:4.0.27.Final] > >> > > > at > >> > > > > >> > > > > >> > > > >> > > >> > io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:847) > >> > > > [netty-transport-4.0.27.Final.jar:4.0.27.Final] > >> > > > at > >> > > > > >> > > > > >> > > > >> > > >> > io.netty.channel.epoll.AbstractEpollStreamChannel$EpollStreamUnsafe.epollInReady(AbstractEpollStreamChannel.java:618) > >> > > > [netty-transport-native-epoll-4.0.27.Final-linux-x86_64.jar:na] > >> > > > at > >> > > > > >> > > > >> > > >> > io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:329) > >> > > > [netty-transport-native-epoll-4.0.27.Final-linux-x86_64.jar:na] > >> > > > at > >> > > > io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:250) > >> > > > [netty-transport-native-epoll-4.0.27.Final-linux-x86_64.jar:na] > >> > > > at > >> > > > > >> > > > > >> > > > >> > > >> > io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:111) > >> > > > [netty-common-4.0.27.Final.jar:4.0.27.Final] > >> > > > at java.lang.Thread.run(Thread.java:745) [na:1.7.0_80] > >> > > > > >> > > > > >> > > > I guess I'm not sure to even know where to start looking todebug > >> this > >> > > > issue, has anyone run into this problem before? > >> > > > > >> > > > > >> > > > Thanks. > >> > > > > >> > > > >> > > > >> > > -- > >> > > > >> > > Abhishek Girish > >> > > Senior Software Engineer > >> > > (408) 476-9209 > >> > > > >> > > <http://www.mapr.com/> > >> > > > >> > > >> > > > > >
