Re: Select distinct on partitioned column requires reading all the files?

2015-02-23 Thread Gopal Vijayaraghavan
Hi, Are you sure you have hive.optimize.metadataonly=true ? I¹m not saying it will complete instantaneously (possibly even be very slow, due to the lack of a temp-table optimization of that), but it won¹t read any part of the actual table. Cheers, Gopal From: Stephen Boesch Reply-To: "user@

Re: CombineHiveInputFormat does not call getSplits on custom InputFormat

2015-02-25 Thread Gopal Vijayaraghavan
Hi, There¹s a special interface in hive-1.0, which gives more information to the input format. https://hive.apache.org/javadocs/r1.0.0/api/ql/org/apache/hadoop/hive/ql/io /CombineHiveInputFormat.AvoidSplitCombination.html But entirely skipping combination results in so many performance problems

Re: Bucket pruning

2015-03-12 Thread Gopal Vijayaraghavan
Hi, No and it¹s a shame because we¹re stuck on some compatibility details with this. The primary issue is the fact that the InputFormat is very generic and offers no way to communicate StorageDescriptor or bucketing. The split generation for something SequenceFileInputFormat lives inside MapRedu

Re: Executing HQL files from JAVA application.

2015-03-24 Thread Gopal Vijayaraghavan
Hi, Any mechanism which bypasses schema layers for SQL is a bad idea. See this example for how you can connect to HiveServer2 directly from Java - https://cwiki.apache.org/confluence/display/Hive/HiveServer2+Clients#HiveSe rver2Clients-JDBCClientSampleCode Use the JDBC driver to access HiveServ

Re: ORDER BY clause in Hive

2015-03-28 Thread Gopal Vijayaraghavan
> SELECT cust_id AS Customer_ID, > Š > ORDER BY 3 ; You¹re sorting on a constant (literal value 3). The results are what you get when you run a non-stable sort on a constant. Cheers, Gopal

Re: ORDER BY clause in Hive

2015-03-29 Thread Gopal Vijayaraghavan
Hi Lefty, Couldn¹t find the documentation for what hive.groupby.orderby.position.alias=true does. I suspect that might be what Mich was looking for (though I tend to write the column names explicitly). Cheers, Gopal From: Lefty Leverenz Reply-To: "user@hive.apache.org" Date: Sunday, March

Re: Predicate pushdown on HBase snapshots

2015-03-30 Thread Gopal Vijayaraghavan
>Looking at the current implementation on trunk, hive's hbase integration >doesn't currently seem to support predicate pushdown for queries over >HBase snapshots. Does this seem like a reasonable feature to add? >It would be nice to have relative feature parity between queries running >over snapsh

Re: Standard deviation (STDDEV) function calculation in Hive

2015-04-01 Thread Gopal Vijayaraghavan
Hi Lefty, ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java: system.registerGenericUDAF("stddev", new GenericUDAFStd()); ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java: system.registerGenericUDAF("stddev_pop", new GenericUDAFStd()); ql/src/java/org/apache/hadoop/hi

Re: Dataset for hive

2015-04-02 Thread Gopal Vijayaraghavan
> https://github.com/hortonworks/hive-testbench > > The official procedure to generate and upload the data has never worked >for me (and it looks like it's not a supported software), so it could be >a bit tricky to do it manually and on a single host. I wrote the MapReduce jobs for that (tpcds-g

Re: How efficient is memory allocation in tez.

2015-04-06 Thread Gopal Vijayaraghavan
> I have a map join in which the smaller tables together are 200 MB and >trying to have one block of main table be processed by one tez task. ... > What am I missing and is this even the right way of approaching the >problem ? You need to be more specific about the Hive version. Hive-13 needs ~6

Re: Writing ORC Files

2015-04-07 Thread Gopal Vijayaraghavan
> addRow() is called in execute(). Does something look wrong with the call? Š There is no need for an explicit flush, but addRow() is rather far below the layers of useful abstraction (and barely any seatbelts). Can you try logging the category/type fields of your inspector and make sure it is r

Re: Analyze table compute statistics on wide table taking too long

2015-04-07 Thread Gopal Vijayaraghavan
> The table also has a large Regex serde. There are no stats fast paths for Regex SerDe. The statistics computation is lifting each row into memory, parsing it and throwing it away. Most of your time would be spent in GC (check the GC time millis), due to the huge expense of the Regex Serde. F

Re: Analyze table compute statistics on wide table taking too long

2015-04-08 Thread Gopal Vijayaraghavan
> I'm happy to look into improving the Regex serde performance, any tips >on where I should start looking?. There are three things off the top of my head. First up, the matcher needs to be reused within a single scan. You can also check the groupCount exactly once for a given pattern. matcher.r

Re: How efficient is memory allocation in tez.

2015-04-10 Thread Gopal Vijayaraghavan
> wasn't being limited by memory but I tried to get the memory usage of >each tez task down so it could spawn more tasks(but it didn't) Giving >tez more or less memory didn't really improve the performance. > How would one go about find out the limiting factor on the performance >of a job. would j

Re: same query works with TEXTFILE and fails with ORC

2015-04-13 Thread Gopal Vijayaraghavan
> I¹m getting an error in Hive when executing a query on a table in ORC >format. This is not an ORC bug, this looks like a vectorization issue. Can you try comparing both query plans (³explain ²) for the Execution mode: vectorized markers? TextFile queries are not vectorized today, since you can

Re: External Table with unclosed orc files.

2015-04-14 Thread Gopal Vijayaraghavan
> What will Hive do if querying an external table containing orc files >that are still being written to? Doing that directly won¹t work at all. Because ORC files are only readable after the Footer is written out, which won¹t be for any open files. > I won¹t be able to test these scenarios till t

Re: External Table with unclosed orc files.

2015-04-14 Thread Gopal Vijayaraghavan
>0.14 . Acid tables have been a real pain for us. We don¹t believe they are >production ready. At least in our use cases, Tez crashes for assorted >reasons or only assigns 1 mapper to the partition. Having delta files and >no base files borks mapper assignments. Some of the chicken-egg problems f

Re: merge small orc files

2015-04-20 Thread Gopal Vijayaraghavan
Hi, >How to set the configuration hive-site.xml to automatically merge small >orc file (output from mapreduce job) in hive 0.14 ? Hive cannot add work-stages to a map-reduce job. Hive follows merge.mapfiles=true when Hive generates a plan, by adding more work to the plan as a conditional task.

Re: merge small orc files

2015-04-21 Thread Gopal Vijayaraghavan
>alter table concatenate do not work? I have a dynamic >partitioned table (stored as orc). I tried to alter concatenate, but it >did not work. See my test result. ORC fast concatenate does work on partitioned tables, but it doesn¹t work on bucketed tables. Bucketed tables cannot merge files, s

Re: MapredContext not available when tez enabled

2015-04-21 Thread Gopal Vijayaraghavan
> A bit digging shows that GenericUDF#configure method was not called. So >in this case, is it possible to get counters through other means, or we >have to implement Counter concept ourselves? You should be getting a TezContext object there (which inherits from MapRedContext). And the method s

Re: Parsing and moving data to ORC from HDFS

2015-04-22 Thread Gopal Vijayaraghavan
> I have about 100 TB of data, approximately 180 billion events, in my >HDFS cluster. It is my raw data stored as GZIP files. At the time of >setup this was due to "saving the data" until we figured out what to do >with it. > > After attending @t3rmin4t0r's ORC 2015 session @hadoopsummit in Brusse

Re: Parsing and moving data to ORC from HDFS

2015-04-22 Thread Gopal Vijayaraghavan
> In production we run HDP 2.2.4. Any thought when crazy stuff like bloom >filters might move to GA? I¹d say that it will be in the next release, considering it is already checked into hive-trunk. Bloom filters aren¹t too crazy today. They are written within the ORC file right next to the row-in

Re: Parsing and moving data to ORC from HDFS

2015-04-22 Thread Gopal Vijayaraghavan
> In production we run HDP 2.2.4. Any thought when crazy stuff like bloom >filters might move to GA? I¹d say that it will be in the next release, considering it is already checked into hive-trunk. Bloom filters aren¹t too crazy today. They are written within the ORC file right next to the row-in

Re: map side join

2015-04-30 Thread Gopal Vijayaraghavan
Hi, > its submitting the whole table to the job. if I use a view with the >filter > baked in, will that help? I don't want to have to jack up the JVM for >the > client/HiveServer2 to accommodate the full table. Which hive version are you using? If you¹re on a recent version like hive-1.0, this

Re: map side join

2015-04-30 Thread Gopal Vijayaraghavan
Hi, > Using CDH 5.3 - Hive 0.13. Does a view help here? Does how i format >the table help in reducing size? No, a view does not help - they are not materialized and you need hive-1.0 to have temporary table support. The only way out is if you only have 1 filter column in the system. I assume

Re: join on different data type

2015-05-04 Thread Gopal Vijayaraghavan
> If A.col1 is of DOUBLE type, > but B.col2 is of BIGINT, The automatic conversion is not acceptable according to the java language spec (section 5.1.2) https://docs.oracle.com/javase/specs/jls/se7/html/jls-5.html#jls-5.1.2 Also to be noted here is that in general, that even if you cast, you

Re: hive sql on tez run forever

2015-05-11 Thread Gopal Vijayaraghavan
Hi, > I change the sql where condition to (where t.update_time >= >'2015-05-04') , the sql can return result for a while. Because >t.update_time > >= '2015-05-04' can filter many row when table scan. But why change >where condition to > (where t.update_time >= '2015-05-04' or length(t8.end_user_i

Re: hive sql on tez run forever

2015-05-11 Thread Gopal Vijayaraghavan
en the 1st query >will produce 1 row >but the 2nd query with subselects will not. > >On 5/11/15, 10:13 AM, "Gopal Vijayaraghavan" wrote: > >>Hi, >> >>> I change the sql where condition to (where t.update_time >= >>>'2015-05-04')

Re: Order of Partition column and Non Partition column in the WHERE clause

2015-05-19 Thread Gopal Vijayaraghavan
> Would the order of partition column in the where clause matter for >performance? No, unless you have more complex predicates than an AND. There¹s one recent regression though - https://issues.apache.org/jira/browse/HIVE-10122 Which release are you on? > Also, how can I make sure that ³partiti

Re: Benefit of ORC format storing Sum, Min, Max...

2015-05-29 Thread Gopal Vijayaraghavan
> I am new to Hive, please help me understand the benefit of ORC file >format storing Sum, Min, Max values. > Whenever we try to find a sum of values in a particular column, it still >runs the MapReduce job. ORC uses row-indexes to constraint filtering. What you¹re looking at is the ORC file foo

Re: current_date function in hive

2015-06-02 Thread Gopal Vijayaraghavan
Hi, > You may try to_date(FROM_UNIXTIME(UNIX_TIMESTAMP())) That would be a very bad idea for query correctness. The current_date UDF was introduced because of that particular anti-pattern. The unix_timestamp() is evaluated when a row is read, so each row gets a slightly different timestamp when

Re: NoSuchMethodError when hive.execution.engine value its tez

2015-06-02 Thread Gopal Vijayaraghavan
> I am using *hive 1.0.0* and *apache tez 0.4.1* When I configure hive to use > tez I get an exception. Use the tez.version that hive-1.0.0 is tested with ­ 0.5.2 (https://github.com/apache/hive/blob/branch-1.0/pom.xml#L155) I suspect you¹ll get several build failures trying to build 1.0.0 agains

Re: Top N query

2015-06-03 Thread Gopal Vijayaraghavan
Hi, This particular case can be handled by a special case of the inner equijoin. > ( recommend_ratings2.movieid = T_0.movieid_0 or >((recommend_ratings2.movieid IS NULL) AND (T_0.movieid_0 IS NULL))) Can be rewritten as a null-safe equi-join (i.e where movieid <=> movieid_0, which is not SQL se

Re: When using ORC, how to confirm Stipes/row-groups are being skipped ?

2015-06-04 Thread Gopal Vijayaraghavan
Hi, > While using ORC file format, I would like to see in the logs that >stripes and/or row-groups are being skipped based on my where clause. There¹s no logging in the inner loop there. > Is that info even outputted ? If so, what do I need to enable it ? You can do a query run with the followi

Re: Top N query

2015-06-04 Thread Gopal Vijayaraghavan
> I does't understand about your solution. Could you please provide exact >query? ... GROUP BY recommend_ratings2.movieid ORDER BY Measure_0 DESC limit 2 ) T_0 ON ( recommend_ratings2.movieid <=> T_0.movieid_0) Follow the docs on the mysql impl of the space-ship operator -

Re: Hive on Tez

2015-06-10 Thread Gopal Vijayaraghavan
Hi, There¹s no relationship between number of containers and tasks ­ well the number of tasks is the maximum number of containers you can use. You can run an entire vertex containing many task attempts in one container if there are no more available ­ because of container reuse. The memory/cpu s

Re: Empty Table in MR with "union all" (created in Tez)

2015-06-10 Thread Gopal Vijayaraghavan
> set hive.execution.engine=mr; > select * from test_table; --gives empty table > Any ideas on why this must be happening? This issue does not occur when >I don¹t use a ³union all² query. What is the value of the parameter mapreduce.input.fileinputformat.input.dir.recursive in your mapre

Re: nested join issue

2015-06-11 Thread Gopal Vijayaraghavan
Hi, > I'm running into a peculiar issue with nested joins and outer select. I >see this error on 1.1.0 and 1.2.0 but not 0.13 which seems like a >regression. ... > create table events (s string, st2 string, n int, timestamp int); The issue does not seem to be happening in hive-2.0.0-SNAPSHOT, wh

Re: nested join issue

2015-06-12 Thread Gopal Vijayaraghavan
Hi > Thanks for investigating.. Trying to locate the patch that fixes this >between 1.1 and 2.0.0-SNAPSHOT. Any leads on what Jira this fix was part >of? Or what part of the code the patch is likely to be on? git bisect is the only way usually to identify these things. But before you hunt into

Re: Read error : Varchar cannot be cast to string

2015-06-17 Thread Gopal Vijayaraghavan
Hi, > Caused by: java.lang.ClassCastException: >org.apache.hadoop.hive.common.type.HiveVarchar cannot be cast to >java.lang.String >at >org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatchCtx.addPartitionCo >lsToBatch(VectorizedRowBatchCtx.java:566) Is it a partition column the one m

Re: Hive - Tez error with big join - Container expired.

2015-06-18 Thread Gopal Vijayaraghavan
> I have a pretty big Hive Query. I¹m joining over 3 Hive-Tables which >have thousands of lines each. I¹m grouping this join by several columns. Hive-on-Tez shouldn¹t have any issue even with billion of lines on a JOIN. > 0 failed, info=[Containercontainer_1434357133795_0008_01_39 finished >w

Re: Read error : Varchar cannot be cast to string

2015-06-18 Thread Gopal Vijayaraghavan
mp timestamp COMMENT '' > src_sys_cd varchar(10) COMMENT '', >batch_id string COMMENT '') >ROW FORMAT DELIMITED > FIELDS TERMINATED BY '|' >STORED AS ORC >LOCATION > '/test/test_staging_table'; > >We were loading data

Re: hive -e run tez query error

2015-06-26 Thread Gopal Vijayaraghavan
> perhaps deleteOnExit() is set somewhere fs.cache.disable settings from hdfs-site.xml are usually to blame for that. Till hive-1.0, HiveServer2 used to leak filesystem objects, so the cache was disabled. > 2015-06-25 15:54:33,673 INFO FSNamesystem.audit: allowed=true >ugi=lujian (auth:SIMPLE)

Re: Hive indexing optimization

2015-06-26 Thread Gopal Vijayaraghavan
Hi, Hive indexes won¹t really help you speed up that query right now, because of the plan it generates due to the <= clauses. > CREATETABLE ipv4table > AS > SELECT logon.IP, ipv4.Country > FROM > (SELECT * FROM logontable WHERE isIpv4(IP)) logon > LEFT OUTER JOIN > (SELECT StartIp, EndIp, Country

Re: schedule data ingestion to hive table using ftp

2015-06-30 Thread Gopal Vijayaraghavan
Hi, > So, I want to schedule data ingestion to hive from ftp. I have to >schedule a job to check for files that are getting generated and when >they get generated, move it to hdfs. There is no ³best² way unfortunately. The options start with Apache Oozie, the bog standard solution. Then there¹s

Re: Limiting outer join

2015-07-06 Thread Gopal Vijayaraghavan
> In the following query, it is possible to limit the amount of entries >returned by an outer join to a single value? I want to obtain a single >country from ipv4geotable for each entry in logontable. Yes, the PTF DENSE_RANK()/ROW_NUMBER() basically gives you that - you can read the first row out

Re: Limiting outer join

2015-07-07 Thread Gopal Vijayaraghavan
> Never mind, I got it working with UDF. I just pass the file location to >my evaluate function. Thanks! :) Nice. Would be very interested in looking at performance of such a UDF, if you have numbers before/after. I suspect it will be a magnitude or more faster than the BETWEEN/JOIN clauses. Ch

Re: Insert with dynamic partitioning from an ORC table fails

2015-07-08 Thread Gopal Vijayaraghavan
> I'm having a problem in Hive 0.13.0 using INSERT OVERWRITE with dynamic >partitioning, selecting from an ORC table to another ORC table (I don't >think the target table being ORC is significant). I think for this to be triggered in 0.13, the src and destination have to be ORC tables. > Has an

Re: Optimizing UDF

2015-07-14 Thread Gopal Vijayaraghavan
> I'm trying to optimize a UDF that runs very slowly on Hive. The UDF >takes in a 5GB table and builds a large data structure out of it to >facilitate lookups. The 5GB input is loaded into the distributed cache >with an 'add file ' command, and the UDF builds > the data structure a single time p

Re: Optimizing UDF

2015-07-14 Thread Gopal Vijayaraghavan
> I am already using Tez (sorry, forgot to mention this), and my goal is >indeed to build the instance once per container. Put a log line in your UDF init() and check if it is being called multiple times per container. If you¹re loading the data everytime, then that might be something to fix.

Re: Hive on Tez query failed with ³wrong key class"

2015-07-21 Thread Gopal Vijayaraghavan
> Query is a simple group-by on top of sequence table. ... > java.io.IOException: java.io.IOException: wrong key class: >org.apache.hadoop.io.BytesWritable is not class >org.apache.hadoop.io.NullWritable I have seen this issue when mixing Sequence files written by PIG with Sequence files written

Re: limit clause + fetch optimization

2015-07-21 Thread Gopal Vijayaraghavan
> I've been experimenting with 'select *' and 'select * limit X' in >beeline and watching the hive-server2 log to understand when a M/R job is >triggered and when not. It seems like whenever I set a limit, the job is >avoided, but with no limit, it is run. https://issues.apache.org/jira/browse/H

Re: limit clause + fetch optimization

2015-07-21 Thread Gopal Vijayaraghavan
> Just want to make sure I understand the behavior once that bug is >fixed...a 'select *' with no limit will run without a M/R job and instead >stream. Is that correct? Yes, that¹s the intended behaviour. I can help you get a fix in, if you have some time to test out my WIP patches. > That may

Re: limit clause + fetch optimization

2015-07-22 Thread Gopal Vijayaraghavan
Hi, > For your first offer of testing a patch, unfortunately we tend to run >our production software on customers' Hadoop clusters, so we can't easily >patch their Hive instances. But I'll still take you up on that if I find >some time to try it. No worries, I¹ll get to a patch sooner or later.

Re: Hive on Tez query failed with ³wrong key class"

2015-07-27 Thread Gopal Vijayaraghavan
> From the java code which creates the sequence file, it has set the key >class to NullWritable.class: > job.setOutputKeyClass(org.apache.hadoop.io.NullWritable.class); ... > I think that caused the mismatch: > wrong key class: org.apache.hadoop.io.BytesWritable is not class >org.apache.hadoop.i

Re: hive error when trying to write data to s3n

2015-08-04 Thread Gopal Vijayaraghavan
> Moving data to: >s3n://:@my_bucket/a/b/2015-07-30/.hive-staging_hiv >e_2015-08-04_18-38-47_649_1476668515119011800-1/-ext-1 > Failed with exception Wrong FS: >s3n://:@my_bucket/a/b/2015-07-30/.hive-staging_hiv >e_2015-08-04_18-38-47_649_1476668515119011800-1/-ext-10002, expected: >hdfs://s

Re: Running hive on tez locally

2015-08-10 Thread Gopal Vijayaraghavan
> How do I point Hive to tez libraries? Is it sufficient to populate >CLASSPATH environment variables with location of tez libraries ? The easiest option there is to populate the hive-config.sh with these two parameters https://github.com/t3rmin4t0r/tez-autobuild/blob/master/Makefile#L138 Once

Re: Question about PredicateTransitivePropagate

2015-08-18 Thread Gopal Vijayaraghavan
>select * from t1 join t2 on t1.col = t2.col where t1.col = 1; > Is rule PredicateTransitivePropagate supposed to propagate predicate >"t1.col = 1" to t2 via join condition t1.col = t2.col? > Assuming so, I found that the predicate "t1.col = 1" has not been pushed >down to table scan of t1, thus Pr

Re: Tez : Anyway to avoid creating subdirectories by "Insert with union all² ?

2015-08-19 Thread Gopal Vijayaraghavan
> Is there anyway to avoid creating sub-directories? Or this is by design >and can not be changed? This is because of the way file-formats generate hadoop name files without collisions. For instance, any change to that would break Parquet-MR for Tez. That's why we generate a compatible, but colli

Re: sql mapjoin very slow

2015-08-28 Thread Gopal Vijayaraghavan
> I have a question. I use hive 1.1.0 ,so hive.stats.dbclass default value >is fs. Mean store statistics > in local filesystem. Any one can tell what is the file path to store >statistics ? The statistics aren't stored in the file system long term - the final destination for stats is the metasto

Re: python libraries to execute or call hive queries

2015-08-28 Thread Gopal Vijayaraghavan
> Can anyone suggest any python libraries to call hive queries from python >scripts ? https://cwiki.apache.org/confluence/display/Hive/HiveClient#HiveClient-Pyth on Though I suspect that's out of date. https://github.com/t3rmin4t0r/amplab-benchmark/blob/master/runner/run_query .py#L604 is ro

Re: Decomposing nested Hive statements with views

2015-09-14 Thread Gopal Vijayaraghavan
> We have many HQL scripts that select from nested sub-selects. In many >cases the nesting can be a few levels deep: ... > Such queries are difficult to test as they are complex monoliths. While >we can decompose and modularise them at the column level with UDFs and >macros, it is not as evident t

Re: Decomposing nested Hive statements with views

2015-09-15 Thread Gopal Vijayaraghavan
> My goal is to create separate independent processing units (queries) >that can then be composed into larger queries. Effectively something like: Ah, modularization beyond a single statement. Views would be the right thing there as a permanent abstraction. I like CTEs during dev is because I ca

Re: mapjoin with left join

2015-09-22 Thread Gopal Vijayaraghavan
> select small.* from small s left join large l on s.id = >l.id where l.id is null; ... > We simply want to load the 81K rows in to RAM, then for each row in >large, check the small hash table and if it the row in small is not in >large, then add it to

Re: Force users to specify partition indexes in queries

2015-09-29 Thread Gopal Vijayaraghavan
> If this is not doable in Hive at the moment, I am interested in writing >a patch for it. I am not familiar with hive codebase so not sure how >complex this is. Any hints or tips would be great and if I do need to >write such a patch, I would be happy to contribute back to the source. There are

Re: Two Tables Join (One Big table and other 1gb size table)

2015-10-13 Thread Gopal Vijayaraghavan
> I tried doing stream table, but ran for long time like 3 hrs : Looks >like only 1 reducer is working on it ... > on (trim(p.pid)=trim(c.p_id) and p.source='XYZ'); In case that's devolving to a cross-product, it might be a miss in pushing down the trim() to the TableScan. Are you using hive-13?

Re: Container is running beyond physical memory limits

2015-10-13 Thread Gopal Vijayaraghavan
> is running beyond physical memory limits. Current usage: 2.0 GB of 2 GB >physical memory used; 6.6 GB of 8 GB virtual memory used. Killing >container. You need to change the yarn.nodemanager.vmem-check-enabled=false on *every* machine on your cluster & restart all NodeManagers. The VMEM check

Re: Container is running beyond physical memory limits

2015-10-13 Thread Gopal Vijayaraghavan
> Now I am rather confused about the following parameters (for example > mapreduce.reduce versus > mapreduce.map) and their correlation to each other They have no relationship with each other. They are meant for two different task types in MapReduce. In general you run fewer reducers than mapp

Re: Merging small files

2015-10-16 Thread Gopal Vijayaraghavan
> Is there a more efficient way to have Hive merge small files on the >files without running with two passes? Not entirely an efficient way, but adding a shuffle stage usually works much better as it gives you the ability to layout the files for better vectorization. Like for TPC-H, doing ETL wi

Re: need help to keep the Hive logo on wikipedia

2015-10-18 Thread Gopal Vijayaraghavan
> Rights to use the Hive logo are controlled by the Apache Software >Foundation, not by the Hive project. You might find what you need here: >http://www.apache.org/foundation/marks/contact. > (I found that link here: May I use Apache product logos on the >cover/title of published books, article

Re: Strict mode and joins

2015-10-19 Thread Gopal Vijayaraghavan
> However reality is I just want it called 1 time which is during planning >and if I flag is as deterministic this is exactly what happens so you can >do this: There was a new constant added in hive-1.2.0 named CURRENT_TIMESTAMP which works the way most people want unix_timestamp() to work. > AN

Re: Hi, Hive People urgent question about [Distribute By] function

2015-10-22 Thread Gopal Vijayaraghavan
> When applying [Distribute By] on Hive to the framework, the function >should be partitionByHash on Flink. This is to spread out all the rows >distributed by a hash key from Object Class in Java. Hive does not use the Object hashCode - the identityHashCode is inconsistent, so Object.hashCode() .

Re: Hi, Hive People urgent question about [Distribute By] function

2015-10-22 Thread Gopal Vijayaraghavan
> so do you think if we want the same result from Hive and Spark or the >other freamwork, how could we try this one ? There's a special backwards compat slow codepath that gets triggered if you do set mapred.reduce.tasks=199; (or any number) This will produce the exact same hash-code as the jav

Re: Issue with job serialization formats mangling results

2015-10-23 Thread Gopal Vijayaraghavan
>I've then created ORC and Parquet versions of this same table. The >behavior remains... select * works, any filter creates horribly >mangled results. > >To replace- throw this into a file: > >{"id":1,"order_id":8,"number":1,"broken":"#\n---\nstuff\nstuff2: >\"stuff3\"\nstuff4: '730'\nstuff5: []

Re: Hi, Hive People urgent question about [Distribute By] function

2015-10-27 Thread Gopal Vijayaraghavan
> I want to override partitionByHash function on Flink like the same way >of DBY on Hive. > I am working on implementing some benchmark system for these two system, >which could be contritbutino to Hive as well. I would be very disappointed if Flink fails to outperform Hive with a Distribute BY,

Re: Using json_tuple for Nested json Arrays

2015-10-27 Thread Gopal Vijayaraghavan
Hi, > If you have any tutorial for extracting data from complex nested json >arrays (as the example given in my previous email), please send it. 90% of working with the real world is cleansing bad data. People under-sell hive's flexibility in situations like this. This is what I do hive> comp

Re: Cross join/cartesian product explanation

2015-11-06 Thread Gopal Vijayaraghavan
> Over the last few week I¹ve been trying to use cross joins/cartesian >products and was wondering why, exactly, this all gets sent to one >reducer. All I¹ve heard or read is that Hive can¹t/doesn¹t parallelize >the job. The hashcode of the shuffle key is 0, since you need to process every row a

Re: Cross join/cartesian product explanation

2015-11-10 Thread Gopal Vijayaraghavan
>I¹m having trouble doing a cross join between two tables that are too big >for a map-side join. The actual query would help btw. Usually what is planned as a cross-join can be optimized out into a binning query with a custom UDF. In particular with 2-D geo queries with binning, which people ten

Re: export/import in hive failing with nested directory exception!

2015-11-12 Thread Gopal Vijayaraghavan
> I am doing a simple export and import test from one database in Hive to >another database in the same Hive instance. I thought this would have >been straight forward. Not if ACID transactions are involved. > Copying file: >hdfs://rhes564:9000/user/hive/warehouse/asehadoop.db/t/delta_056_00

Re: export/import in hive failing with nested directory exception!

2015-11-12 Thread Gopal Vijayaraghavan
Hi, >Thanks Gopal. Indeed table t is defined as ORC and transactional. > >Any reason why this should not work for transactional tables? The committed transactions list is actually missing from the exported metadata. So the EXPORT as it exists today is a dirty read snapshot, which is not a good t

Re: [VOTE] Hive 2.0 release plan

2015-11-13 Thread Gopal Vijayaraghavan
(+user@) +1. Cheers, Gopal On 11/13/15, 5:54 PM, "Lefty Leverenz" wrote: >The Hive bylaws require this to be submitted on the user@hive mailing list >(even though users don't get to vote). See Release Plan in Actions >. >

Re: does hive support non equality join?

2015-11-13 Thread Gopal Vijayaraghavan
> from cwiki, the answer is no. it seems supported after some test. The old syntax doesn't support it, but the new one does. select count(1) from table1, table2 where table1.x < table2.x; That said, that query won't run very fast unless one of the tables is tiny. Cheers, Gopal

Re: Changelog table

2015-11-17 Thread Gopal Vijayaraghavan
> If I have a series of entries that look like ... > { "update", {"baz" : "bar" }} Due to the way the split distribution works, you need a global ordering key for each operation. 0, "ADD", "baz", "" 1, "SET", "baz", "bar" 2, "DEL", "baz", null If you do not have updates coming in within a secon

Re: Cross join/cartesian product explanation

2015-11-17 Thread Gopal Vijayaraghavan
>It¹s really a very simple query that I¹m trying to run: >select ... >bloom_contains(a_id, b_id_bloom) That's nearly impossible to optimize directly - there is no way to limit the number of table_b rows which may match table a. More than one bloom filter can successfully match a single row f

Re: Building Spark to use for Hive on Spark

2015-11-18 Thread Gopal Vijayaraghavan
> I wanted to know why is it necessary to remove the Hive jars from the >Spark build as mentioned on this Because SparkSQL was originally based on Hive & still uses Hive AST to parse SQL. The org.apache.spark.sql.hive package contains the parser which has hard-references to the hive's internal

Re: How to capture query log and duration

2015-11-19 Thread Gopal Vijayaraghavan
> We would like to capture some information in our Hadoop Cluster. > Can anybody please suggest how we can we achieve this, any tools >available already ? Or do we need to scrub any log ? Apache Atlas is the standardized solution for deeper analytics into data ownership/usage (look at the HiveHoo

Re: How to capture query log and duration

2015-11-20 Thread Gopal Vijayaraghavan
>Can you please also let me know what argument list this script want . > >I was trying following in HDP Sandbox , but did not get JSON outout The JSON output is saved into a .zip file, if you hit ^C. > https://gist.github.com/t3rmin4t0r/e4bf835f10271b9e466e Look for a file named atsdump*.zip

Re: Low performance map join when join key types are different

2015-12-22 Thread Gopal Vijayaraghavan
> We found that when we join on two different type keys , hive will >convert all join key to Double. This is because of type coercions for BaseCompare, so that String:Integer comparisons with "<=" will work similarly to "=". > b.id to double. When the conversion occurs, map join will become very

Re: unique-id for the mapper task with tez execution engine

2015-12-22 Thread Gopal Vijayaraghavan
Hi, (x-posts to bcc:) On 12/22/15, 9:19 PM, "Amey Barve" wrote: >conf.get("mapreduce.task.id"); > >Now I want to run same hive queries with tez engine and I want to know >what >should be my unique-id. Is there any property from configuration or other >that can give me unique-id from the mapper

Re: Low performance map join when join key types are different

2015-12-22 Thread Gopal Vijayaraghavan
> But why disable mapjoin has better performance when we don't use cast to >string(user always lazy)? > > Join key values comparison in in reduce stage is more quickly? The HashMap is slower than the full-sort + sorted-merge-join. It shouldn't be, but it hits the worst-case performance for th

Re: unique-id for the mapper task with tez execution engine

2015-12-22 Thread Gopal Vijayaraghavan
Hi, > So what do you suggest to get unique-id for mapper task with tez >execution engine? > > conf.get("mapreduce.task.partition"); > > Is this correct? Yes, that is correct - but it can only be unique within a Mapper vertex. Tez plans sort of look like this for complex queries http://people.a

Re: Null Representation in Hive tables

2015-12-27 Thread Gopal Vijayaraghavan
> Your best bet is take the serde you s re using and copy it and change >the code to accept bith null types Or use a view with an IF(). For instance, I like the ease of generating groovy UDFs inline for quick-n-dirty state machines. We now do column pruning into views, so if you do not select t

Re: Hive HLL for appx count distinct

2015-12-30 Thread Gopal Vijayaraghavan
> I'm trying to explore the HLL UDF option to compute # of uniq users for >each time range (week, month, yr, etc.) and wanted to know if > its possible to just maintain HLL struct for each day and then use those >to compute the uniqs for various time > ranges using these per day structs instead of

Re: Hive HLL for appx count distinct

2015-12-30 Thread Gopal Vijayaraghavan
> In the hive-hll-udf, you seem to mention about RRD. Is that something >supported by Hive? No. RRDTool is what most people are replacing with Hive to store time series data in. Raw RRDTool files on a local disk have no availability model (i.e lose a disk, you lose data). The rollup concept howe

Re: Hive Buckets and Select queries

2015-12-31 Thread Gopal Vijayaraghavan
> and when we issue the following query, its doing a "Full table scan" > > > SELECT * FROM foo WHERE id= Bucket pruning is available in Hive-2.0 right now (HIVE-11525). Doc note pending, for 2016 when Hive-2.0 releases. Cheers, Gopal

Re: NPE when reading Parquet using Hive on Tez

2016-01-04 Thread Gopal Vijayaraghavan
> select count(*) from alexa_parquet; > Caused by: java.lang.NullPointerException >at >org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils$TypeInfoParser.tokeni >ze(TypeInfoUtils.java:274) >at >org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils$TypeInfoParser. >(TypeInfoUtils.java:29

Re: Is Hive Index officially not recommended?

2016-01-05 Thread Gopal Vijayaraghavan
> I am going to run the same query in Hive. However, I only see a table >scan below and no mention of that index. May be I am missing something >here? Hive Indexes are an incomplete feature, because they are not maintained over an ACID storage & demand FileSystem access to check for validity.

Re: Is Hive Index officially not recommended?

2016-01-05 Thread Gopal Vijayaraghavan
>So in a nutshell in Hive if "external" indexes are not used for improving >query response, what value they add and can we forget them for now? The builtin indexes - those that write data as smaller tables are only useful in a pre-columnar world, where the indexes offer a huge reduction in IO. P

Re: Impact of partitioning on certain queries

2016-01-08 Thread Gopal Vijayaraghavan
> Ok we hope that partitioning improves performance where the predicate is >on partitioned columns Nope. Partitioning *only* improves performance if your queries run with set hive.mapred.mode=strict; That's the "use strict" easy way to make sure you're writing good queries. Even then, schem

Re: Hive query hangs in reduce steps

2016-01-09 Thread Gopal Vijayaraghavan
Hi, > The job completes fine if we reduce the # of rows processed by reducing >the # of days data being processed. > > It just gets stuck after all maps are completed. We checked the logs and >it says the containers are released. Looks like you're inserting into a bucketed & partitioned table an

  1   2   3   4   >