Re: Time to Remove Hive-on-Spark

2022-02-10 Thread Zoltan Haindrich

Hey,

I think there is no real interest in this feature; we don't have users/contributors backing it - last development was around 2018 October; there were ~2 bugfix commits ever 
since that...we should stop carrying dead weight...another 2 weeks went by since Stamatis have reminded us that after 1.5 years(!) nothing have changed.


+1 on removing it

cheers,
Zoltan

you may inspect some of the recent changes with:
git log -c `find . -type f -path '**/spark/**'|grep -v xml|grep -v 
properties|grep -v q.out`


On 1/28/22 2:32 PM, Stamatis Zampetakis wrote:

Hi team,

Almost one year has passed since the last exchange in this discussion and
if I am not wrong there has been no effort to revive Hive-on-Spark. To be
more precise, I don't think I have seen any Spark related JIRA for quite
some time now and although I don't want to rush into conclusions, there
does not seem to be any community member involved in maintaining or adding
new features in this part of the code.

Keeping dead code in the repository does not do any good to the project and
puts a non-negligible burden to future maintainers.

Clearly, we cannot make a new Hive release where a major feature is
completely untested so either someone commits to re-enable/fix the
respective tests soon or we move forward the work started by David and drop
support for Hive-on-Spark.

I would like to ask the community if there is anyone who can take up this
maintenance task and enable/fix Spark related tests in the next month or so?

Best,
Stamatis

On Sat, Feb 27, 2021 at 4:17 AM Edward Capriolo 
wrote:


I do not know how it works for most of the world. But in cloudera where the
TEZ options were never popular hive-on-spark represents a solid way to get
things done for small datasets lower latency.

As for the spark adoption. You know a while ago I came up with some ways to
make hive more  spark like. One of them was a found a way to make "compile"
a hive keyword so folks could build UDFs on the fly. It was such an
uphil climb. Folks found a way to make it disabled by default for security.
Then later when things moved from CLI to beeline it was like the ONLY thing
that I found not ported. Like it was extremely frustrating.






On Mon, Jul 27, 2020 at 3:19 PM David  wrote:


Hello  Xuefu,

I am not part of the Cloudera Hive product team,  though I volunteer to
work on small projects from time to time.  Perhaps someone from that team
can chime in with some of their thoughts, but personally, I think that in
the long run, there will be more of a merge between Hive-on-Spark and

other

Spark-native offerings.  I'm not sure what the differentiation will be
going forward.  With that said, are there any developers on this mailing
list who are willing to take on the maintenance effort of keeping HoS
moving forward?

http://www.russellspitzer.com/2017/05/19/Spark-Sql-Thriftserver/



https://docs.cloudera.com/HDPDocuments/HDP2/HDP-2.6.4/bk_spark-component-guide/content/config-sts.html



Thanks.

On Thu, Jul 23, 2020 at 12:35 PM Xuefu Zhang  wrote:


Previous reasoning seemed to suggest a lack of user adoption. Now we

are

concerned about ongoing maintenance effort. Both are valid

considerations.

However, I think we should have ways to find out the answers.

Therefore,

I

suggest the following be carried out:

1. Send out the proposal (removing Hive on Spark) to users including
user@hive.apache.org and get their feedback.
2. Ask if any developers on this mailing list are willing to take on

the

maintenance effort.

I'm concerned about user impact because I can still see issues being
reported on HoS from time to time. I'm more concerned about the future

of

Hive if we narrow Hive neutrality on execution engines, which will

possibly

force more Hive users to migrate to other alternatives such as Spark

SQL,

which is already eroding Hive's user base.

Being open and neutral used to be Hive's most admired strengths.

Thanks,
Xuefu


On Wed, Jul 22, 2020 at 8:46 AM Alan Gates 

wrote:



An important point here is I don't believe David is proposing to

remove

Hive on Spark from the 2 or 3 lines, but only from trunk.  Continuing

to

support it in existing 2 and 3 lines makes sense, but since no one

has

maintained it on trunk for some time and it does not work with many

of

the

newer features it should be removed from trunk.

Alan.

On Tue, Jul 21, 2020 at 4:10 PM Chao Sun  wrote:


Thanks David. FWIW Uber is still running Hive on Spark (2.3.4) on a

very

large scale in production right now and I don't think we have any

plan

to

change it soon.



On Tue, Jul 21, 2020 at 11:28 AM David  wrote:


Hello,

Thanks for the feedback.

Just a quick recap: I did propose this @dev and I received

unanimous

+1's

from the community.  After a couple months, I created the PR.

Certainly open to discussion, but there hasn't been any

discussion

thus

far

because there have been no objections until this point.

HoS has low adoption, heavy technical debt, and the manner i

Re: CBO doc version & log parse issue

2021-12-15 Thread Zoltan Haindrich

Hey,

by using an UDTF Hive should also be prepared that it will return more rows 
from that single row - which is not true in this case: because 1 log entry will 
be 1 result row.
Note that the CBO does not seem to support UDTFs (I've tried apache/master with 
a lateral view right now - and it failed).
I think most likely the biggest problem you are facing is that the cbo doesn't 
run on your query at all.

create table tx2 as select named_struct('i',1,'str','bla') as s;
explain select * from tx2 lateral view inline(array(s)) s1;
...
| Plan not optimized by CBO. |
...

Have you tried returning with a struct?
explain select s.i,s.str from tx2 t;
| Plan not optimized by CBO. |

for your case you should repeat the udf call multiple times
explain select 
named_struct('i',1,'str','bla').i,named_struct('i',1,'str','bla').str ;
or
explain select s.i,s.str from (select named_struct('i',1,'str','bla') s) t;


cheers,
Zoltan


On 12/13/21 12:49 PM, sam wrote:

Hi team,

I am trying to learn the CBO of hive because I need to make some performance 
tuning for my ETL job.

I find a confluence doc below, but I am not sure if it is the newest version, 
can anyone help to confirm that?
https://cwiki.apache.org/confluence/display/Hive/Cost-based+optimization+in+Hive 


Another question is that we develop some UDTF help us to parse log like:
select
my-udtf(log) as (id
,name
,time)
from tb_log
So do you have any other better idea for this scenario?

BTW, the version of Hive we used is above 3.0. My data increase by PB every day.


Thanks in advance,

Samuel









Recent log4j vulnerabilities

2021-12-15 Thread Zoltan Haindrich

Hello all!

In the recent week there were 2 new log4j vulnerabilities discovered (CVE-2021-45046, CVE-2021-44228) - and since we use log4j in Hive; existing installations might be 
affected as well.


Doing a new Hive release on any existing line would probably need a longer timeframe - and doing an upgrade would probably cause further problems for existing installation; 
for now I'll try to give some help to help in patching existing clusters.


My understanding is that both CVE can be fixed by following one of these 
options:
* remove the JndiLookup.class from the affected jars
* replace the jar with the 2.16.0 version

To identify the affected jars; you could run this script - which will ignore 
2.16.0 if there is any:

pat=org/apache/logging/log4j/core/lookup/JndiLookup.class mc=org/apache/logging/log4j/core/pattern/MessagePatternConverter.class && find . -name '*.jar'|xargs -n1 -IJAR 
unzip -t JAR |fgrep -f <(echo "$pat";echo 'Archive:')|grep -B1 "$pat"|grep '^Archive:'|cut -d '/' -f2-|xargs -n1 -IJAR bash -c 'unzip -p JAR $mc|md5sum|paste - <(echo 
JAR)'|fgrep -vf <(echo 374fa1c796465d8f542bb85243240555 )


You could remove the JndiLookup.class from the identified jars with something 
similar to this:
zip -q -d log4j-core-*.jar org/apache/logging/log4j/core/lookup/JndiLookup.class

To validate if you are still affected or not:
* generate a token on https://canarytokens.org/
* try with queries like (replace your token):
set hive.fetch.task.conversion=none;
create table aa (a string) location 
'file:///dfs${jndi:ldap:canarytokens.com/a}';
select '${jndi:ldap://canarytokens.com/a}';

cheers,
Zoltan


Re: Hive variable behaviour

2021-11-30 Thread Zoltan Haindrich

Hey Elango!

Variables are not evaluated at assignment time - instead they are evaluated at 
the time of usage.

If you want to save the current time somehow I think you could:
* put the current_timestamp into a temporary table:
  create temporary table start_time as select current_timestamp() as t;
* not sure about your use case; but if you need the start date of the session 
you could set it as an external variable when you start the session:
  beeline -hivevar start_time=`date "+%Y-%m-%d %H:%M:%S"
  select current_timestamp() - cast('${hivevar:start_time}' as timestamp);
* consider using hplsql I think with that it might be possible to store it in a 
variable; however hplsql approaches things a bit differently...
  I can't give you an example how to do this right the top of my head


cheers,
Zoltan

On 11/30/21 7:03 AM, elango vaidyanathan wrote:



Hi All, any updates on this?

Thanks,
Elango

On Fri, Nov 26, 2021, 11:09 AM elango vaidyanathan mailto:elango...@gmail.com>> wrote:


Hi all,

I am trying to create a hive variable and store the current time stamp. 
Then run some complex select queries and then create another variable to store 
current timestamp
and then find the runtime.
The problem I am facing is every time I use the variable it gives me the 
current time instead of the assigned value. It just runs the current timestamp 
function every
time. Below is the code snippet.

Code

set hivevar:start_tm=current_timestamp();

select ${hivevar:start_tm};

select query;__

select ${hivevar:start_tm};


Here the start_tm variable calculates every time the current time and 
returns always the current time. How can I just store the value once and use 
that value for this
variable. Please clarify.


Thanks,
Elango



Re: Hive meetup on March 17

2021-03-17 Thread Zoltan Haindrich

Hey All!

We have our first online Hive meetup today!

We will start at 5pm UTC for other timezones see on this site:
https://www.timeanddate.com/worldclock/meetingdetails.html?year=2021&month=3&day=17&hour=17&min=0&sec=0&p1=50&p2=137&p3=136&p4=70&p5=176

If you don't yet have the meeting url - it will be held in a zoom room at:
https://cloudera.zoom.us/j/91452267238
Most likely there will be a recording of it - which will be shared afterwards.

I was thinking to use Github discussions to (also) ask questions during the event - because it could help untangle "question time" from "answer time"; we may of course 
choose not to use it - but I've experimented with it and if we add discussions to the "Q&A" section we may even answer it - and people thinking about the same thing may 
extend the question by adding further comments...or just vote on the question...

not sure how well it will work - might worth a try!
I've set it up on my own fork for now: 
https://github.com/kgyrtkirk/hive/discussions

The meetup url is here:
https://www.meetup.com/Hive-User-Group-Meeting/events/276886707

Meet you there!

cheers,
Zoltan

On 3/16/21 3:29 PM, Zoltan Haindrich wrote:

Hey All!

Our meetup is also available as a meetup.com event:
https://www.meetup.com/Hive-User-Group-Meeting/events/276886707/

In case you want to add it to the calendar or something... :)

cheers,
Zoltan


On 3/11/21 3:00 PM, Zoltan Haindrich wrote:

Hey All!

I would like to invite you to our (first?) online Hive meetup! It will be held 
on March 17. 17:00 UTC
I'll send out a zoom url before the event starts!

The planned topics are accessible here:
https://docs.google.com/document/d/12jaWa7e6jvVjUaxoMWNJcjvTjnNoqwdCAMyswY1OiUg/edit?usp=sharing

Meet you there!

cheers,
Zoltan






Re: Hive meetup on March 17

2021-03-16 Thread Zoltan Haindrich

Hey All!

Our meetup is also available as a meetup.com event:
https://www.meetup.com/Hive-User-Group-Meeting/events/276886707/

In case you want to add it to the calendar or something... :)

cheers,
Zoltan


On 3/11/21 3:00 PM, Zoltan Haindrich wrote:

Hey All!

I would like to invite you to our (first?) online Hive meetup! It will be held 
on March 17. 17:00 UTC
I'll send out a zoom url before the event starts!

The planned topics are accessible here:
https://docs.google.com/document/d/12jaWa7e6jvVjUaxoMWNJcjvTjnNoqwdCAMyswY1OiUg/edit?usp=sharing

Meet you there!

cheers,
Zoltan






Hive meetup on March 17

2021-03-11 Thread Zoltan Haindrich

Hey All!

I would like to invite you to our (first?) online Hive meetup! It will be held 
on March 17. 17:00 UTC
I'll send out a zoom url before the event starts!

The planned topics are accessible here:
https://docs.google.com/document/d/12jaWa7e6jvVjUaxoMWNJcjvTjnNoqwdCAMyswY1OiUg/edit?usp=sharing

Meet you there!

cheers,
Zoltan






Re: Any plan for new hive 3 or 4 release?

2021-02-22 Thread Zoltan Haindrich

Hey Michel!

Yes it was a long time ago we had a release; we have quite a few new features 
in master.
I think we are scaring people for some time now that we will be dropping MR 
support...I think we should do that.

I would really like to see a new Hive release in the near future as well - 
there is no way for users to even try out new features.
I was planning to add nightly builds to package the latest master's state into a deployable artifact - I think a service like may help pretest our next release; I think it 
won't take much to do it so I'll probably throw it together in the next couple days!


cheers,
Zoltan

On 2/21/21 2:27 PM, Michel Sumbul wrote:

Hi Guys,

If I'm not wrong, the last release of Hive 3.x is 18 months old.
I wanted to ask if you had any roadmap / plan to release a new version of
Hive 3.x or Hive 4?

Thanks,
Michel



Re: Execption In SELECT Query

2020-07-07 Thread Zoltan Haindrich

Hey Harshita!

I was not able to reproduce your issue with the below versions.
from the error I would guess you may have incompatible guava versions loaded...

I think you may also try Hadoop 3.1.2 for Hive 3.1.2

cheers,
Zoltan


On 7/6/20 7:53 PM, harshita vishwakarma wrote:

Hi All,
Facing below error while executing 'select' query in hive:

/Exception in thread "main" java.lang.IncompatibleClassChangeError: Class com.google.common.collect.ImmutableSortedMap does not implement the requested interface 
java.util.NavigableMap/

/
/
/Hadoop Version->2.9.2/
/Hive Version->3.1.2/
/
/
/I have created a table:/

CREATE TABLE ClientData (id INT, name STRING, dept STRING, yoj INT);

and then Loaded data into it from local file system

LOAD DATA LOCAL INPATH 'F:/TestingData/ClientDataHive.txt' INTO table 
ClientData;

Now when I perform select * from ClientData, it throws the below error.

image.png
/
/
/Can you please help out here./
/
/
/Thanks,/
/Harshita./


Re: Help needed for count(*) query error on Hive 3.1.2 & Tez 0.9.2

2020-07-01 Thread Zoltan Haindrich

Hey Eugene!

I don't see any hints in these outputs what could be the issue...have you 
checked the tez container logs?

cheers,
Zoltan


On 7/1/20 9:58 AM, Eugene Chung wrote:

Hi,

I want to know how to investigate the count(*) query error on Hive 3.1.2 & Tez 
0.9.2, which is 'being failed for too many output errors' in the Mapper.

The query is just simple like "select count(*) from MY_DB.ORC_TABLE where 
part_date='2020-06-30';" where ORC files of MY_DB.ORC_TABLE are bucketed.

But the query for the same table (ORC files) is running normally on Hive 2.3.2 
& Tez 0.9.1.


The error on Hive 3.1.2 is like below;

|-- VERTICES MODE STATUS TOTAL COMPLETED RUNNING PENDING FAILED KILLED 
-- Map 1 .. container RUNNING 75 22 0 53 38 0 Reducer 2 container INITED 1 0 0 1 
0 0 -- VERTICES: 00/02 [===>>---] 28% ELAPSED TIME: 10.44 s 
-- 20/07/01 15:36:49 ERROR SessionState: Status: Failed 20/07/01 15:36:49 ERROR 
SessionState: Vertex failed, vertexName=Map 1, vertexId=vertex_1591769205146_436476_1_00, diagnostics=[Task failed, taskId=task_1591769205146_436476_1_00_55, 
diagnostics=[TaskAttempt 0 failed, info=[attempt_1591769205146_436476_1_00_55_0 being failed for too many output errors. failureFraction=1.0, 
MAX_ALLOWED_OUTPUT_FAILURES_FRACTION=0.1, uniquefailedOutputReports=1, MAX_ALLOWED_OUTPUT_FAILURES=10, MAX_ALLOWED_TIME_FOR_TASK_READ_ERROR_SEC=300, readErrorTimespan=0], 
TaskAttempt 1 failed, info=[attempt_1591769205146_436476_1_00_55_1 being failed for too many output errors. failureFraction=1.0, 
MAX_ALLOWED_OUTPUT_FAILURES_FRACTION=0.1, uniquefailedOutputReports=1, MAX_ALLOWED_OUTPUT_FAILURES=10, MAX_ALLOWED_TIME_FOR_TASK_READ_ERROR_SEC=300, readErrorTimespan=0], 
TaskAttempt 2 failed, info=[attempt_1591769205146_436476_1_00_55_2 being failed for too many output errors. failureFraction=1.0, 
MAX_ALLOWED_OUTPUT_FAILURES_FRACTION=0.1, uniquefailedOutputReports=1, MAX_ALLOWED_OUTPUT_FAILURES=10, MAX_ALLOWED_TIME_FOR_TASK_READ_ERROR_SEC=300, readErrorTimespan=0], 
TaskAttempt 3 failed, info=[attempt_1591769205146_436476_1_00_55_3 being failed for too many output errors. failureFraction=1.0, 
MAX_ALLOWED_OUTPUT_FAILURES_FRACTION=0.1, uniquefailedOutputReports=1, MAX_ALLOWED_OUTPUT_FAILURES=10, MAX_ALLOWED_TIME_FOR_TASK_READ_ERROR_SEC=300, readErrorTimespan=0]], 
Vertex did not succeed due to OWN_TASK_FAILURE, failedTasks:1 killedTasks:52, Vertex vertex_1591769205146_436476_1_00 [Map 1] killed/failed due to:OWN_TASK_FAILURE] 
20/07/01 15:36:49 ERROR SessionState: Vertex killed, vertexName=Reducer 2, vertexId=vertex_1591769205146_436476_1_01, diagnostics=[Vertex received Kill while in RUNNING 
state., Vertex did not succeed due to OTHER_VERTEX_FAILURE, failedTasks:0 killedTasks:1, Vertex vertex_1591769205146_436476_1_01 [Reducer 2] killed/failed due 
to:OTHER_VERTEX_FAILURE] 20/07/01 15:36:49 ERROR SessionState: DAG did not succeed due to VERTEX_FAILURE. failedVertices:1 killedVertices:1 20/07/01 15:36:49 ERROR 
ql.Driver: FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.tez.TezTask. Vertex failed, vertexName=Map 1, 
vertexId=vertex_1591769205146_436476_1_00, diagnostics=[Task failed, taskId=task_1591769205146_436476_1_00_55, diagnostics=[TaskAttempt 0 failed, 
info=[attempt_1591769205146_436476_1_00_55_0 being failed for too many output errors. failureFraction=1.0, MAX_ALLOWED_OUTPUT_FAILURES_FRACTION=0.1, 
uniquefailedOutputReports=1, MAX_ALLOWED_OUTPUT_FAILURES=10, MAX_ALLOWED_TIME_FOR_TASK_READ_ERROR_SEC=300, readErrorTimespan=0], TaskAttempt 1 failed, 
info=[attempt_1591769205146_436476_1_00_55_1 being failed for too many output errors. failureFraction=1.0, MAX_ALLOWED_OUTPUT_FAILURES_FRACTION=0.1, 
uniquefailedOutputReports=1, MAX_ALLOWED_OUTPUT_FAILURES=10, MAX_ALLOWED_TIME_FOR_TASK_READ_ERROR_SEC=300, readErrorTimespan=0], TaskAttempt 2 failed, 
info=[attempt_1591769205146_436476_1_00_55_2 being failed for too many output errors. failureFraction=1.0, MAX_ALLOWED_OUTPUT_FAILURES_FRACTION=0.1, 
uniquefailedOutputReports=1, MAX_ALLOWED_OUTPUT_FAILURES=10, MAX_ALLOWED_TIME_FOR_TASK_READ_ERROR_SEC=300, readErrorTimespan=0], TaskAttempt 3 failed, 
info=[attempt_1591769205146_436476_1_00_55_3 being failed for too many output errors. failureFraction=1.0, MAX_ALLOWED_OUTPUT_FAILURES_FRACTION=0.1, 
uniquefailedOutputReports=1, MAX_ALLOWED_OUTPUT_FAILURES=10, MAX_ALLOWED_TIME_FOR_TASK_READ_ERROR_SEC=300, readErrorTimespan=0]], Vertex did not succeed due to 
OWN_TASK_FAILURE, failedTasks:1 killedTasks:52, Vertex vertex_1591769205146_436476_1_00 [Map 1] killed/failed due

Re: Measuring the execution time of Hive queries through Ambari

2020-06-30 Thread Zoltan Haindrich

Hey Mich!

I don't know which version you use (HDP-3+?) - but you might want to see if "Data 
Analytics Studio" is available for that version; it could give similar insights as 
TezUI had.

cheers,
Zoltan

On 6/22/20 2:23 PM, Mich Talebzadeh wrote:

Hi  Julien.

It is as I see is standard Ambari. Has TEZ UI but when I run the query and 
check TEZ UI it says TEZ view is not deployed!

Thanks



LinkedIn 
/https://www.linkedin.com/profile/view?id=AAEWh2gBxianrbJd6zP6AcPCCdOABUrV8Pw/



*Disclaimer:* Use it at your own risk.Any and all responsibility for any loss, damage or destruction of data or any other property which may arise from relying on this 
email's technical content is explicitly disclaimed. The author will in no case be liable for any monetary damages arising from such loss, damage or destruction.




On Mon, 22 Jun 2020 at 13:10, Julien Tane mailto:j...@solute.de>> wrote:

Mich,


When you say, that you are using ambari to connect to hvie what do you mean 
by that.

Unless you added a view in ambari to perform query (as far as I know, not 
in the vanilla ambari)


One thing you could more or less do is use the tez.ui (assuming you are 
using tez)

but here again this is not in the standard ambari (at least not the newer 
versions)

one other possibility (depending on how you configured yarn) would be to 
use the yarn ui

which should be accessible in the Yarn Tab from your ambari... But here, it 
kinds of depends

on how you configured your system.


Kind Regards,


J



Julien Tane
Big Data Engineer
Tel.+49 721 98993-393
Fax +49 721 98993-66
E-Mail  j...@solute.de 

solute GmbH
Zeppelinstraße 15
76185 Karlsruhe
Germany

Logo Solute

Marken der solute GmbH | brands of solute GmbH
Marken

Geschäftsführer | Managing Director: Dr. Thilo Gans, Bernd Vermaaten
Webseite | www.solute.de 
Sitz | Registered Office: Karlsruhe
Registergericht | Register Court: Amtsgericht Mannheim
Registernummer | Register No.: HRB 110579
USt-ID | VAT ID: DE234663798
*Informationen zum Datenschutz | Information about privacy policy*
https://www.solute.de/ger/datenschutz/grundsaetze-der-datenverarbeitung.php



*Von:* Mich Talebzadeh mailto:mich.talebza...@gmail.com>>
*Gesendet:* Montag, 22. Juni 2020 12:57:27
*An:* user
*Betreff:* Measuring the execution time of Hive queries through Ambari
Hi,

Using Ambari to connect to Hive, is there any way of measuring the query 
time?

Please be aware that this is through Ambari not through beeline etc.

The tool we have at the moment is Ambari to Prod.

We do not have any other luxury!

Thanks




LinkedIn 
/https://www.linkedin.com/profile/view?id=AAEWh2gBxianrbJd6zP6AcPCCdOABUrV8Pw/



*Disclaimer:* Use it at your own risk.Any and all responsibility for any 
loss, damage or destruction of data or any other property which may arise from 
relying on this
email's technical content is explicitly disclaimed. The author will in no 
case be liable for any monetary damages arising from such loss, damage or 
destruction.



Re: Issues with aggregating on map values

2020-02-12 Thread Zoltan Haindrich

Hey Nakul!

It's not clear which version you are using;
I've checked this issue on apache/master and the 3.1.2 release - and both of 
them returned accurate results.
You could execute: 'select version()' ; or run 'hive --version' in a commandline

cheers,
Zoltan

On 2/11/20 11:38 AM, Nakul Khanna (BLOOMBERG/ LONDON) wrote:

Creating the table

CREATE TABLE foo
(tags MAP, size int);
INSERT INTO foo VALUES
(map("country", "USA"), 10),
(map("country", "USA"), 20),
(map("country", "UK"), 30);

SELECT * FROM foo

++---+
| foo.tags | foo.size |
++---+
| {"country":"USA"} | 10 |
| {"country":"USA"} | 20 |
| {"country":"UK"} | 30 |
++---+

Aggregating the Table

SELECT DISTINCT tags["country"] from foo;

+---+
| _c0 |
+---+
| USA |
| NULL |
+---+

SELECT tags["country"], sum(size) FROM foo GROUP BY tags["country"];

+---+--+
| _c0 | _c1 |
+---+--+
| USA | 10 |
| NULL | 50 |
+---+--+

And even more strangely, with a subquery:

SELECT flattened.country, sum(flattened.size)
FROM (
SELECT tags["country"] as country, size
FROM foo
WHERE tags["country"] IS NOT NULL
) as flattened
GROUP BY flattened.country;

++--+
| flattened.country | _c1 |
++--+
| USA | 10 |
++--+

---

Is there any way to stop this from happening and get the correct aggregation behaviour? The only method I've found is to create a new table using the query, write that to 
disk and then do the aggregation on that.


Re: Is there any way to find Hive query to Datanucleus queries mapping

2020-02-10 Thread Zoltan Haindrich

Hey Chinna!

I don't think a mapping like that is easy to get...I would rather try to narrow 
down to a single call which consumes most of the time.
There is a log message which can help you get to the most relevant metastore 
call:
https://github.com/apache/hive/blob/0d9deba3c15038df4c64ea9b8494d554eb8eea2f/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java#L5405

cheers,
Zoltan

On 2/10/20 1:07 PM, Chinna Rao Lalam wrote:

Hi All,

Is there any way to find Hive query to Datanucleus queries mapping.

"select * from table" this hive query will generate multiple Datanucleus
queries and execute on configured DB.
In our DB some of the queries are running slow, So we want to see
hivequery->datanucleus query mapping to find out which hive query of
datanucleus query is running slow.

If we enable Datanucleus debug log we can see generated queries but not
mapping.

Thanks
Chinna



Write access to the Hive wiki

2020-01-22 Thread Zoltan Haindrich

Hello,

I was using a different login for the wiki - which seem to have stopped 
working; right now my ldap login (kgyrtkirk) logs me in; but I'm not allowed to 
edit the wiki pages.
Could someone please help me get edit permissions?

Thank you,
Zoltan


Re: If Hive Metastore is compatibility with MariaDB version 10.x.?

2020-01-20 Thread Zoltan Haindrich

Hello,

Locally I use Mariadb 10.4.8 when I validate metastore schema/etc changes. So 
far, I've not uncovered any issues with it...

I'm planning to integrate some kind of smoke tests against all the supported 
DBs to help uncover metastore related issues earlier.
To evaluate that we have everything working with mysql; it might be usefull to 
see how well our existing tests behave on that version.
For that purpose the following things might be usefull:
* there is a way to run qtest-s against different metastores: see HIVE-21954
* the hive-dev-box can be used to configure and launch hive with 
postgres/mysql/ora/mssql https://github.com/kgyrtkirk/hive-dev-box

cheers,
Zoltan


On 1/17/20 1:28 PM, Oleksiy S wrote:

Hi all.

Could you please help? Customer asked if Hive Metastore is compatible with 
MariaDB version 10.x. He is going to use 10.4.10-MariaDB MariaDB Server.

--
Oleksiy


Re: Regarding hive.exec.parallel

2019-05-08 Thread Zoltan Haindrich

Hey Varun!

I'm not sure about you actual query; but hive.exec.parallel enable to execute 
stages in parallel
The full tez dag is usually "one stage" of the execution (but you should take a 
look at the explain).
If you would be using mr engine there might have been some speedup; but in case of tez the parallel execution of independent tasks are happening inside tez independently 
from this setting.


cheers,
Zoltan

On 5/7/19 9:00 PM, Varun Rao wrote:

Hello,

We were wondering what the benefits are of setting hive.exec.parallel to true. I know that this will execute any possible tasks in parallel. For example MapReduce stages, 
sampling stages, merge stages, limit stages etc will be executed in parallel allowing for the overall job to be completed more quickly. However, my co worker and I decided 
to run a 400 line long query in separate Tez sessions. I would set hive.exec.parallel=true and he would set it to false. However, we see almost no improvement in speed. I 
am assuming that is because the stages of my query is dependent on one another (join a with b with c with d). Is this  the case? Can you give me examples of queries where 
there would be improvement in speed (perhaps in TPC-DS) when one sets hive.exec.parallel to true


Thanks

Yours Truly,
Varun Rao


Re: How to implement partitioned external table.

2019-04-11 Thread Zoltan Haindrich




On 4/4/19 10:22 AM, PengHui Li wrote:

Hi guys,

I am integrating hive and pulsar(http://pulsar.apache.org ) by HiveStorageHandler and HiveMetaHook, I want to add a feature can divide the data 
into several parts(pulsar topics) when use hive `PARTITIONED BY`. But  don't know how to implement it based on HiveStorageHandler and HiveMetaHook.


I think you should be able to access the table's properties from the 
StorageHandler (and get access to the pulsar server address/etc from there).

About supporting topics: I think instead of adding some features to support 
"partitioned by"
the storage handler could get into predicate push down...by making the topic a 
column.
To get some ideas how to do that I would first take a look at the jdbc storage 
handler(or hbase).

note: I think this topic might better fit the developer list.

cheers,
Zoltan


Re: issue for the "select count(*) from table"

2018-09-05 Thread Zoltan Haindrich

Hello,

I think you might have loaded data by using an external tool into the table 
location; you should run:
analyze table table1 compute statistics ;
or
analyze table table1 compute statistics for columns;

And/or disable hive.optimize.metadataonly - but having bad statistics is not 
good at all...

I would also recommend to use an "external" table in case some other program 
(beyond hive) is also writing to the table location.

cheers,
Zoltan

On 09/05/2018 11:46 AM, cang...@tsign.cn wrote:

hello,
  1. i encounter a wired problem. “select count(*) from table1”return me "0", 
but there are a lot of data in the 'table1'.
  i check the metastore tables(mysql). the "table_params" show both "numFiles'='0" and 'numRows=0 for "table1" .  but the "select count(*) from talbe1" is ok just a few 
days ago.
  but rigth now ,some table is ok ,some's bad. so i'm confuse about the behavior of "count(*)". (all of them have no partition used. and the hive vesion is 2.1.1000 in HDP. 
and the hive.stats.autogather is true. and



Interactive Query is no

)

2. one more problem. does "ALTER TABLE table1 SET TBLPROPERTIES ('numRows'='0');" effect 
the "select count(*)  from table1"? i'm suppose this would break the data consistency

some pics for problem 1:



*芮波*
Rober

phone: 13282023300
e-mail: cang...@tsign.cn 



Re: Hook is not getting called on running query from beeline

2018-08-23 Thread Zoltan Haindrich

Hello

There is an interesting message after connecting: "Connected to: Spark SQL (version 
2.1.0.2.6.0.3-8)".
It seems you are not using Hive...please ask about this on a Spark related list.

cheers,
Zoltan

On 08/22/2018 08:43 PM, Chetan Kothari wrote:

It seems --hiveconf is not getting picked from beeline.

So even if I give wrong hook class name in following command, I am able to 
connect and execute queries.

But hook is not getting called.

beeline -u "jdbc:hive2://bdcsceforqa-bdcsce-1.dfml.ucfc2z3b.usdv1.abccloud.com:10001/;transportMode=http;httpPath=cliservice" --hiveconf 
hive.exec.post.hooks=abcc.lineage.listeners.TestPostExecutionHook


Connected to: Spark SQL (version 2.1.0.2.6.0.3-8)

Driver: Hive JDBC (version 1.2.1000.2.4.2.0-258)

Transaction isolation: TRANSACTION_REPEATABLE_READ

Beeline version 1.2.1000.2.4.2.0-258 by Apache Hive

Regards

Chetan

-Original Message-
From: Zoltan Haindrich [mailto:k...@rxd.hu]
Sent: Wednesday, August 22, 2018 8:51 PM
To: user@hive.apache.org; Chetan Kothari
Subject: Re: Hook is not getting called on running query from beeline

Hello

I was wondering about that the property is not getting picked up from beeline; 
but it looks ok to me:

$ beeline -u 'jdbc:hive2://...' --hiveconf hive.exec.post.hooks=asd -e 'select 
1'

Connected to: Apache Hive (version 1.2.1000.2.6.1.0-129)

Driver: Hive JDBC (version 1.2.1000.2.6.1.0-129) Transaction isolation: 
TRANSACTION_REPEATABLE_READ

Error: Error while processing statement: FAILED: Hive Internal Error: 
java.lang.ClassNotFoundException(asd) (state=08S01,code=12)

Which version of hive are you using?

Try first with a non-existent classname. Executing the hooks usually leave some 
messages in the logs as well...

cheers,

Zoltan

On 08/22/2018 11:22 AM, Chetan Kothari wrote:

 > Thanks Tanvi for your inputs.

 >

 > Hook is not getting called from beeline/JDBC for any queries.

 >

 > I ensured that hive-site.xml is in classpath and also tried following but 
still hook is not getting called.

 >

 > beeline -u "jdbc:hive2://example.com:2181

 > <http://example.com:2181>,example.com:2181

 > <http://example.com:2181>,example:2181/;serviceDiscoveryMode=zooKeeper

 > ;zooKeeperNamespace=hiveserver2" *--hiveconf

 > hive.exec.post.hooks=*

 >

 > Regards

 >

 > Chetan

 >

 > *From:*Tanvi Thacker [mailto:tanvithack...@gmail.com]

 > *Sent:* Wednesday, August 22, 2018 9:29 AM

 > *To:* user@hive.apache.org <mailto:user@hive.apache.org>

 > *Subject:* Re: Hook is not getting called on running query from

 > beeline

 >

 > Is the hook getting called from beeline/JDBC for queries other than CTAS?

 > can you make sure that your configuration is correctly loaded by

 > beeline or JDBC code? you may check hive-site.xml is in the classpath

 > or add a property when requesting a connection from DriverManager or

 > you may also try like this

 >

 > beeline -u "jdbc:hive2://example.com:2181

 > <http://example.com:2181>,example.com:2181

 > <http://example.com:2181>,example:2181/;serviceDiscoveryMode=zooKeeper

 > ;zooKeeperNamespace=hiveserver2" *--hiveconf

 > hive.exec.post.hooks=*

 >

 > Regards,

 >

 > Tanvi Thacker

 >

 > On Tue, Aug 21, 2018 at 9:00 AM Chetan Kothari mailto:chetan.koth...@oracle.com 
<mailto:chetan.koth...@oracle.com%20%3cmailto:chetan.koth...@oracle.com>>> wrote:


 >

 > Any inputs on following issue will be helpful

 >

 > Regards

 >

 > Chetan

 >

 > *From:* Chetan Kothari

 > *Sent:* Tuesday, August 21, 2018 6:27 PM

 > *To:* user@hive.apache.org <mailto:user@hive.apache.org> 
<mailto:user@hive.apache.org>

 > *Subject:* Hook is not getting called on running query from

 > beeline

 >

 > Hi

 >

 > I have written custom PostExecutionHook which implements 
org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext to capture lineage of hive 
queries like create table as

 > select etc.

 >

 > This hook get's called when I run CTAS query o Hive Shell. But it is not 
getting called when I run same query from beeline or programmatically using JDBC 
Code.

 >

 > Any inputs on why hook is not getting called when I run query from 
beeline or programmatically using JDBC Code.

 >

 > Regards

 >

 > Chetan

 >



Re: Hook is not getting called on running query from beeline

2018-08-22 Thread Zoltan Haindrich

Hello

I was wondering about that the property is not getting picked up from beeline; 
but it looks ok to me:

$ beeline -u 'jdbc:hive2://...' --hiveconf hive.exec.post.hooks=asd -e 'select 
1'
Connected to: Apache Hive (version 1.2.1000.2.6.1.0-129)
Driver: Hive JDBC (version 1.2.1000.2.6.1.0-129)
Transaction isolation: TRANSACTION_REPEATABLE_READ
Error: Error while processing statement: FAILED: Hive Internal Error: 
java.lang.ClassNotFoundException(asd) (state=08S01,code=12)

Which version of hive are you using?
Try first with a non-existent classname. Executing the hooks usually leave some 
messages in the logs as well...

cheers,
Zoltan


On 08/22/2018 11:22 AM, Chetan Kothari wrote:

Thanks Tanvi for your inputs.

Hook is not getting called from beeline/JDBC for any queries.

I ensured that hive-site.xml is in classpath and also tried following but still 
hook is not getting called.

beeline -u "jdbc:hive2://example.com:2181 ,example.com:2181 
,example:2181/;serviceDiscoveryMode=zooKeeper;zooKeeperNamespace=hiveserver2" *--hiveconf hive.exec.post.hooks=*


Regards

Chetan

*From:*Tanvi Thacker [mailto:tanvithack...@gmail.com]
*Sent:* Wednesday, August 22, 2018 9:29 AM
*To:* user@hive.apache.org
*Subject:* Re: Hook is not getting called on running query from beeline

Is the hook getting called from beeline/JDBC for queries other than CTAS?
can you make sure that your configuration is correctly loaded by beeline or JDBC code? you may check hive-site.xml is in the classpath or add a property when requesting a 
connection from DriverManager or you may also try like this


beeline -u "jdbc:hive2://example.com:2181 ,example.com:2181 
,example:2181/;serviceDiscoveryMode=zooKeeper;zooKeeperNamespace=hiveserver2" *--hiveconf hive.exec.post.hooks=*


Regards,

Tanvi Thacker

On Tue, Aug 21, 2018 at 9:00 AM Chetan Kothari mailto:chetan.koth...@oracle.com>> wrote:

Any inputs on following issue will be helpful

Regards

Chetan

*From:* Chetan Kothari
*Sent:* Tuesday, August 21, 2018 6:27 PM
*To:* user@hive.apache.org 
*Subject:* Hook is not getting called on running query from beeline

Hi

I have written custom PostExecutionHook which implements 
org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext to capture lineage of 
hive queries like create table as
select etc.

This hook get's called when I run CTAS query o Hive Shell. But it is not 
getting called when I run same query from beeline or programmatically using 
JDBC Code.

Any inputs on why hook is not getting called when I run query from beeline 
or programmatically using JDBC Code.

Regards

Chetan



Re: Hive generating different DAGs from the same query

2018-07-13 Thread Zoltan Haindrich

Hello Sungwoo!

I think its possible that reoptimization is kicking in, because the first 
execution have bumped into an exception.

I think the plans should not be changing permanently; unless 
"hive.query.reexecution.stats.persist.scope" is set to a wider scope than query.

To check that indeed reoptimization is happening(or not) look for:

cat > patterns << EOF
org.apache.hadoop.hive.ql.exec.mapjoin.MapJoinMemoryExhaustionError
reexec
Driver.java:execute
SessionState.java:printError
EOF

cat patterns

fgrep -Ff patterns --color=yes /var/log/hive/hiveserver2.log | grep -v DEBUG

cheers,
Zoltan

On 07/11/2018 10:40 AM, Sungwoo Park wrote:

Hello,

I am running the TPC-DS benchmark using Hive 3.0, and I find that Hive sometimes produces different DAGs from the same query. These are the two scenarios for the 
experiment. The execution engine is tez, and the TPC-DS scale factor is 3TB.


1. Run query 19 to query 24 sequentially in the same session. The first part of 
query 24 takes about 156 seconds:

100 rows selected (58.641 seconds) <-- query 19
100 rows selected (16.117 seconds)
100 rows selected (9.841 seconds)
100 rows selected (35.195 seconds)
1 row selected (258.441 seconds)
59 rows selected (213.156 seconds)
4,643 rows selected (156.982 seconds) <-- the first part of query 24
1,656 rows selected (136.382 seconds)

2. Now run query 1 to query 24 sequentially in the same session. This time the 
first part of query 24 takes more than 1000 seconds:

100 rows selected (94.981 seconds) <-- query 1
2,513 rows selected (30.804 seconds)
100 rows selected (11.076 seconds)
100 rows selected (225.646 seconds)
100 rows selected (44.186 seconds)
52 rows selected (11.436 seconds)
100 rows selected (21.968 seconds)
11 rows selected (14.05 seconds)
1 row selected (35.619 seconds)
100 rows selected (27.062 seconds)
100 rows selected (134.098 seconds)
100 rows selected (7.65 seconds)
1 row selected (14.54 seconds)
100 rows selected (143.965 seconds)
100 rows selected (101.676 seconds)
100 rows selected (19.742 seconds)
1 row selected (245.381 seconds)
100 rows selected (71.617 seconds)
100 rows selected (23.017 seconds)
100 rows selected (10.888 seconds)
100 rows selected (11.149 seconds)
100 rows selected (7.919 seconds)
100 rows selected (29.527 seconds)
1 row selected (220.516 seconds)
59 rows selected (204.363 seconds)
4,643 rows selected (1008.514 seconds) <-- the first part of query 24
1,656 rows selected (141.279 seconds)

Here are a few findings from the experiment:

1. The two DAGs for the first part of query 24 are quite similar, but actually different. The DAG from the first scenario contains 17 vertices, whereas the DAG from the 
second scenario contains 18 vertices, skipping some part of map-side join that is performed in the first scenario.


2. The configuration (HiveConf) inside HiveServer2 is precisely the same before 
running the first part of query 24 (except for minor keys).

So, I wonder how Hive can produce different DAGs from the same query. For example, is there some internal configuration key in HiveConf that enables/disables some 
optimization depending on the accumulate statistics in HiveServer2? (I haven't tested it yet, but I can also test with Hive 2.x.)


Thank you in advance,

--- Sungwoo Park



hive wiki edit permission

2016-06-01 Thread Zoltan Haindrich

Hi,

I would like to update the documentation - and add the new altertate way 
to use ide-s with hive (HIVE-13490)

my wiki login is: kirk

regards,
Zoltan