The Drill test team was able to repro this and is now filed as:
https://issues.apache.org/jira/browse/DRILL-4665

On Tue, May 10, 2016 at 8:16 AM, Aman Sinha <amansi...@apache.org> wrote:

> This is supposed to work, especially since LIKE predicate is not even on
> the partitioning column (it should work either way).  I did a quick test
> with file system tables and it works for LIKE conditions.  Not sure yet
> about Hive tables.  Could you pls file a JIRA and we'll follow up.
> Thanks.
>
> -Aman
>
> On Tue, May 10, 2016 at 1:09 AM, Shankar Mane <shankar.m...@games24x7.com>
> wrote:
>
>> Problem:
>>
>> 1. In drill, we are using hive partition table. But explain plan (same
>> query) for like and = operator differs and used all partitions in case of
>> like operator.
>> 2. If you see below drill explain plans: Like operator uses *all*
>> partitions where
>> = operator uses *only* partition filtered by log_date condition.
>>
>> FYI- We are storing our logs in hive partition table (parquet,
>> gz-compressed). Each partition is having ~15 GB data. Below is the
>> describe
>> statement output from hive:
>>
>>
>> /**************************************************************** Hive
>>
>> ************************************************************************************/
>> hive> desc hive_kafkalogs_daily ;
>> OK
>> col_name data_type comment
>> sessionid           string
>> ajaxurl             string
>>
>> log_date             string
>>
>> # Partition Information
>> # col_name             data_type           comment
>>
>> log_date             string
>>
>>
>>
>>
>> /***************************************************************** Drill
>> Plan (query with LIKE)
>>
>> ***********************************************************************************/
>>
>> explain plan for select sessionid, servertime, ajaxUrl from
>> hive.hive_kafkalogs_daily where log_date = '2016-05-09' and ajaxUrl like
>> '%utm_source%' limit 1 ;
>>
>> +------+------+
>> | text | json |
>> +------+------+
>> | 00-00    Screen
>> 00-01      Project(sessionid=[$0], servertime=[$1], ajaxUrl=[$2])
>> 00-02        SelectionVectorRemover
>> 00-03          Limit(fetch=[1])
>> 00-04            UnionExchange
>> 01-01              SelectionVectorRemover
>> 01-02                Limit(fetch=[1])
>> 01-03                  Project(sessionid=[$0], servertime=[$1],
>> ajaxUrl=[$2])
>> 01-04                    SelectionVectorRemover
>> 01-05                      Filter(condition=[AND(=($3, '2016-05-09'),
>> LIKE($2, '%utm_source%'))])
>> 01-06                        Scan(groupscan=[HiveScan
>> [table=Table(dbName:default, tableName:hive_kafkalogs_daily),
>> columns=[`sessionid`, `servertime`, `ajaxurl`, `log_date`],
>> numPartitions=29, partitions= [Partition(values:[2016-04-11]),
>> Partition(values:[2016-04-12]), Partition(values:[2016-04-13]),
>> Partition(values:[2016-04-14]), Partition(values:[2016-04-15]),
>> Partition(values:[2016-04-16]), Partition(values:[2016-04-17]),
>> Partition(values:[2016-04-18]), Partition(values:[2016-04-19]),
>> Partition(values:[2016-04-20]), Partition(values:[2016-04-21]),
>> Partition(values:[2016-04-22]), Partition(values:[2016-04-23]),
>> Partition(values:[2016-04-24]), Partition(values:[2016-04-25]),
>> Partition(values:[2016-04-26]), Partition(values:[2016-04-27]),
>> Partition(values:[2016-04-28]), Partition(values:[2016-04-29]),
>> Partition(values:[2016-04-30]), Partition(values:[2016-05-01]),
>> Partition(values:[2016-05-02]), Partition(values:[2016-05-03]),
>> Partition(values:[2016-05-04]), Partition(values:[2016-05-05]),
>> Partition(values:[2016-05-06]), Partition(values:[2016-05-07]),
>> Partition(values:[2016-05-08]), Partition(values:[2016-05-09])],
>>
>> inputDirectories=[hdfs://namenode:9000/usr/hive/warehouse/hive_kafkalogs_daily_20160411,
>> hdfs://namenode:9000/usr/hive/warehouse/hive_kafkalogs_daily_20160412,
>> hdfs://namenode:9000/usr/hive/warehouse/hive_kafkalogs_daily_20160413,
>> hdfs://namenode:9000/usr/hive/warehouse/hive_kafkalogs_daily_20160414,
>> hdfs://namenode:9000/usr/hive/warehouse/hive_kafkalogs_daily_20160415,
>> hdfs://namenode:9000/usr/hive/warehouse/hive_kafkalogs_daily_20160416,
>> hdfs://namenode:9000/usr/hive/warehouse/hive_kafkalogs_daily_20160417,
>> hdfs://namenode:9000/usr/hive/warehouse/hive_kafkalogs_daily_20160418,
>> hdfs://namenode:9000/usr/hive/warehouse/hive_kafkalogs_daily_20160419,
>> hdfs://namenode:9000/usr/hive/warehouse/hive_kafkalogs_daily_20160420,
>> hdfs://namenode:9000/usr/hive/warehouse/hive_kafkalogs_daily_20160421,
>> hdfs://namenode:9000/usr/hive/warehouse/hive_kafkalogs_daily_20160422,
>> hdfs://namenode:9000/usr/hive/warehouse/hive_kafkalogs_daily_20160423,
>> hdfs://namenode:9000/usr/hive/warehouse/hive_kafkalogs_daily_20160424,
>> hdfs://namenode:9000/usr/hive/warehouse/hive_kafkalogs_daily_20160425,
>> hdfs://namenode:9000/usr/hive/warehouse/hive_kafkalogs_daily_20160426,
>> hdfs://namenode:9000/usr/hive/warehouse/hive_kafkalogs_daily_20160427,
>> hdfs://namenode:9000/usr/hive/warehouse/hive_kafkalogs_daily_20160428,
>> hdfs://namenode:9000/usr/hive/warehouse/hive_kafkalogs_daily_20160429,
>> hdfs://namenode:9000/usr/hive/warehouse/hive_kafkalogs_daily_20160430,
>> hdfs://namenode:9000/usr/hive/warehouse/hive_kafkalogs_daily_20160501,
>> hdfs://namenode:9000/usr/hive/warehouse/hive_kafkalogs_daily_20160502,
>> hdfs://namenode:9000/usr/hive/warehouse/hive_kafkalogs_daily_20160503,
>> hdfs://namenode:9000/usr/hive/warehouse/hive_kafkalogs_daily_20160504,
>> hdfs://namenode:9000/usr/hive/warehouse/hive_kafkalogs_daily_20160505,
>> hdfs://namenode:9000/usr/hive/warehouse/hive_kafkalogs_daily_20160506,
>> hdfs://namenode:9000/usr/hive/warehouse/hive_kafkalogs_daily_20160507,
>> hdfs://namenode:9000/usr/hive/warehouse/hive_kafkalogs_daily_20160508,
>> hdfs://namenode:9000/usr/hive/warehouse/hive_kafkalogs_daily_20160509]]])
>>  | {
>>   "head" : {
>>     "version" : 1,
>>     "generator" : {
>>       "type" : "ExplainHandler",
>>       "info" : ""
>>     },
>>     "type" : "APACHE_DRILL_PHYSICAL",
>>     "options" : [ ],
>>     "queue" : 0,
>>     "resultMode" : "EXEC"
>>   },
>>   "graph" : [ {
>>     "pop" : "hive-scan",
>>     "@id" : 65542,
>>     "userName" : "hadoop",
>>     "hive-table" : {
>>       "table" : {
>>         "tableName" : "hive_kafkalogs_daily",
>>         "dbName" : "default",
>>         "owner" : "hadoop",
>>         "createTime" : 1461952920,
>>         "lastAccessTime" : 0,
>>         "retention" : 0,
>>         "sd" : {
>>           "cols" : [ {
>>             "name" : "sessionid",
>>             "type" : "string",
>>             "comment" : null
>>           }, {
>>             "name" : "servertime",
>>             "type" : "string",
>>             "comment" : null
>>           }, {
>>             "name" : "ajaxurl",
>>             "type" : "string",
>>             "comment" : null
>>           } ],
>>           "location" :
>> "hdfs://namenode:9000/usr/hive/warehouse/hive_kafkalogs_daily",
>>           "inputFormat" :
>> "org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat",
>>           "outputFormat" :
>> "org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat",
>>           "compressed" : false,
>>           "numBuckets" : -1,
>>           "serDeInfo" : {
>>             "name" : null,
>>             "serializationLib" :
>> "org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe",
>>             "parameters" : {
>>               "serialization.format" : "1"
>>             }
>>           },
>>           "sortCols" : [ ],
>>           "parameters" : { }
>>         },
>>         "partitionKeys" : [ {
>>           "name" : "log_date",
>>           "type" : "string",
>>           "comment" : null
>>         } ],
>>         "parameters" : {
>>           "EXTERNAL" : "TRUE",
>>           "transient_lastDdlTime" : "1461952920"
>>         },
>>         "viewOriginalText" : null,
>>         "viewExpandedText" : null,
>>         "tableType" : "EXTERNAL_TABLE"
>>       },
>>       "partitions" : [ {
>>         "values" : [ "2016-04-11" ],
>>         "tableName" : "hive_kafkalogs_daily",
>>         "dbName" : "default",
>>         "createTime" : 1461952941,
>>         "lastAccessTime" : 0,
>>         "sd" : {
>>           "cols" : [ {
>>             "name" : "sessionid",
>>             "type" : "string",
>>             "comment" : null
>>           |
>> +------+------+
>> 1 row selected (0.859 seconds)
>>
>>
>>
>>
>>
>>
>> /***************************************************************** Drill
>> Plan (query without LIKE)
>>
>> ***********************************************************************************/
>>
>> explain plan for select sessionid, servertime, ajaxUrl from
>> hive.hive_kafkalogs_daily where log_date = '2016-05-09' and ajaxUrl =
>> 'utm_source' limit 1 ;
>>
>> +------+------+
>> | text | json |
>> +------+------+
>> | 00-00    Screen
>> 00-01      Project(sessionid=[$0], servertime=[$1], ajaxUrl=[$2])
>> 00-02        SelectionVectorRemover
>> 00-03          Limit(fetch=[1])
>> 00-04            UnionExchange
>> 01-01              SelectionVectorRemover
>> 01-02                Limit(fetch=[1])
>> 01-03                  Project(sessionid=[$0], servertime=[$1],
>> ajaxUrl=[$2])
>> 01-04                    SelectionVectorRemover
>> 01-05                      Filter(condition=[AND(=($3, '2016-05-09'),
>> =($2,
>> 'utm_source'))])
>> 01-06                        Scan(groupscan=[HiveScan
>> [table=Table(dbName:default, tableName:hive_kafkalogs_daily),
>> columns=[`sessionid`, `servertime`, `ajaxurl`, `log_date`],
>> numPartitions=1, partitions= [Partition(values:[2016-05-09])],
>>
>> inputDirectories=[hdfs://namenode:9000/usr/hive/warehouse/hive_kafkalogs_daily_20160509]]])
>>  | {
>>   "head" : {
>>     "version" : 1,
>>     "generator" : {
>>       "type" : "ExplainHandler",
>>       "info" : ""
>>     },
>>     "type" : "APACHE_DRILL_PHYSICAL",
>>     "options" : [ ],
>>     "queue" : 0,
>>     "resultMode" : "EXEC"
>>   },
>>   "graph" : [ {
>>     "pop" : "hive-scan",
>>     "@id" : 65542,
>>     "userName" : "hadoop",
>>     "hive-table" : {
>>       "table" : {
>>         "tableName" : "hive_kafkalogs_daily",
>>         "dbName" : "default",
>>         "owner" : "hadoop",
>>         "createTime" : 1461952920,
>>         "lastAccessTime" : 0,
>>         "retention" : 0,
>>         "sd" : {
>>           "cols" : [ {
>>             "name" : "sessionid",
>>             "type" : "string",
>>             "comment" : null
>>           }, {
>>             "name" : "servertime",
>>             "type" : "string",
>>             "comment" : null
>>           }, {
>>             "name" : "ajaxurl",
>>             "type" : "string",
>>             "comment" : null
>>           } ],
>>           "location" :
>> "hdfs://namenode:9000/usr/hive/warehouse/hive_kafkalogs_daily",
>>           "inputFormat" :
>> "org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat",
>>           "outputFormat" :
>> "org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat",
>>           "compressed" : false,
>>           "numBuckets" : -1,
>>           "serDeInfo" : {
>>             "name" : null,
>>             "serializationLib" :
>> "org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe",
>>             "parameters" : {
>>               "serialization.format" : "1"
>>             }
>>           },
>>           "sortCols" : [ ],
>>           "parameters" : { }
>>         },
>>         "partitionKeys" : [ {
>>           "name" : "log_date",
>>           "type" : "string",
>>           "comment" : null
>>         } ],
>>         "parameters" : {
>>           "EXTERNAL" : "TRUE",
>>           "transient_lastDdlTime" : "1461952920"
>>         },
>>         "viewOriginalText" : null,
>>         "viewExpandedText" : null,
>>         "tableType" : "EXTERNAL_TABLE"
>>       },
>>       "partitions" : [ {
>>         "values" : [ "2016-05-09" ],
>>         "tableName" : "hive_kafkalogs_daily",
>>         "dbName" : "default",
>>         "createTime" : 1462848405,
>>         "lastAccessTime" : 0,
>>         "sd" : {
>>           "cols" : [ {
>>             "name" : "sessionid",
>>             "type" : "string",
>>             "comment" : null
>>           }, {
>>             "name" : "servertime",
>>             "type" : "string",
>>             "comment" : null
>>           }, {
>>             "name" : "ajaxurl",
>>             "type" : "string",
>>             "comment" : null
>>      |
>> +------+------+
>> 1 row selected (3.394 seconds)
>>
>
>

Reply via email to