OK this seems to work.

   1. Create the target table first
   2.  Populate afterwards

 I first created the target table with

hive> create table test.dummy as select * from oraclehadoop.dummy where 1 =
2;

 Then did  INSERT/SELECT and tried to drop the target table when DML
(INSERT/SELECT) was going on

Now the process 6856 (drop table ..)  is waiting for the locks to be
released which is correct


Lock ID Database        Table   Partition       State   Type    Transaction
ID  Last Hearbeat   Acquired At     User    Hostname
6855    test    dummy   NULL    ACQUIRED        SHARED_READ     NULL
1465425703092   1465425703054   hduser  rhes564
6855    oraclehadoop    dummy   NULL    ACQUIRED        SHARED_READ
NULL    1465425703092   1465425703056   hduser  rhes564
6856    test    dummy   NULL    WAITING EXCLUSIVE       NULL
1465425820073   NULL    hduser  rhes564

Sounds like with Hive there is the issue with DDL + DML locks applied in a
single transaction i.e. --> create table A as select * from b

HTH

Dr Mich Talebzadeh



LinkedIn * 
https://www.linkedin.com/profile/view?id=AAEAAAAWh2gBxianrbJd6zP6AcPCCdOABUrV8Pw
<https://www.linkedin.com/profile/view?id=AAEAAAAWh2gBxianrbJd6zP6AcPCCdOABUrV8Pw>*



http://talebzadehmich.wordpress.com



On 8 June 2016 at 23:35, Eugene Koifman <ekoif...@hortonworks.com> wrote:

> if you split “create table test.dummy as select * from oraclehadoop.dummy;
> ”
> into create table statement, followed by insert into test.dummy as select…
> you should see the behavior you expect with Hive.
> Drop statement will block while insert is running.
>
> Eugene
>
> From: Mich Talebzadeh <mich.talebza...@gmail.com>
> Reply-To: "u...@hive.apache.org" <u...@hive.apache.org>
> Date: Wednesday, June 8, 2016 at 3:12 PM
> To: Michael Segel <michael_se...@hotmail.com>
> Cc: David Newberger <david.newber...@wandcorp.com>, "u...@hive.apache.org"
> <u...@hive.apache.org>, "user @spark" <user@spark.apache.org>
> Subject: Re: Creating a Hive table through Spark and potential locking
> issue (a bug)
>
> Hive version is 2
>
> We can discuss all sorts of scenarios.  However, Hivek is pretty good at
> applying the locks at both the table and partition level. The idea of
> having a metadata is to enforce these rules.
>
> [image: Inline images 1]
>
> For example above inserting from source to target table partitioned (year,
> month) shows that locks are applied correctly
>
> This is Hive running on Spark engine. The crucial point is that Hive
> accesses its metadata and updates its hive_locks table. Again one can see
> from data held in that table in metadata
>
> [image: Inline images 2]
>
> So I think there is a genuine issue here
>
> HTH
>
>
> Dr Mich Talebzadeh
>
>
>
> LinkedIn * 
> https://www.linkedin.com/profile/view?id=AAEAAAAWh2gBxianrbJd6zP6AcPCCdOABUrV8Pw
> <https://www.linkedin.com/profile/view?id=AAEAAAAWh2gBxianrbJd6zP6AcPCCdOABUrV8Pw>*
>
>
>
> http://talebzadehmich.wordpress.com
>
>
>
> On 8 June 2016 at 22:36, Michael Segel <michael_se...@hotmail.com> wrote:
>
>> Hi,
>>
>> Lets take a step back…
>>
>> Which version of Hive?
>>
>> Hive recently added transaction support so you have to know your
>> isolation level.
>>
>> Also are you running spark as your execution engine, or are you talking
>> about a spark app running w a hive context and then you drop the table from
>> within a Hive shell while the spark app is still running?
>>
>> You also have two different things happening… you’re mixing a DDL with a
>> query.  How does hive know you have another app reading from the table?
>> I mean what happens when you try a select * from foo; and in another
>> shell try dropping foo?  and if you want to simulate a m/r job add
>> something like an order by 1 clause.
>>
>> HTH
>>
>> -Mike
>>
>>
>>
>> On Jun 8, 2016, at 1:44 PM, Mich Talebzadeh <mich.talebza...@gmail.com>
>> wrote:
>>
>> Hi,
>>
>> The idea of accessing Hive metada is to be aware of concurrency.
>>
>>
>>  In generall if I do the following In Hive
>>
>>
>> hive> create table test.dummy as select * from oraclehadoop.dummy;
>>
>>
>> We can see that hive applies the locks in Hive
>>
>>
>> <image.png>
>>
>>
>>
>>
>>
>> However, there seems to be an issue. *I do not see any exclusive lock on
>> the target table* (i.e. test.dummy). The locking type SHARED_READ on
>> source table oraclehadoop.dummy looks OK
>>
>>
>>  One can see the locks  in Hive database
>>
>>
>>
>>
>> <image.png>
>>
>>
>>
>>
>> So there are few issues here:
>>
>>
>>    1. With Hive -> The source table is locked as SHARED_READ
>>    2. With Spark --> No locks at all
>>    3. With HIVE --> No locks on the target table
>>    4. With Spark --> No locks at all
>>
>>  HTH
>>
>>
>>
>>
>>
>>
>>
>>
>>
>>
>>
>> Dr Mich Talebzadeh
>>
>>
>> LinkedIn * 
>> https://www.linkedin.com/profile/view?id=AAEAAAAWh2gBxianrbJd6zP6AcPCCdOABUrV8Pw
>> <https://www.linkedin.com/profile/view?id=AAEAAAAWh2gBxianrbJd6zP6AcPCCdOABUrV8Pw>*
>>
>>
>> http://talebzadehmich.wordpress.com
>>
>>
>>
>> On 8 June 2016 at 20:22, David Newberger <david.newber...@wandcorp.com>
>> wrote:
>>
>>> Could you be looking at 2 jobs trying to use the same file and one
>>> getting to it before the other and finally removing it?
>>>
>>>
>>>
>>> *David Newberger*
>>>
>>>
>>>
>>> *From:* Mich Talebzadeh [mailto:mich.talebza...@gmail.com]
>>> *Sent:* Wednesday, June 8, 2016 1:33 PM
>>> *To:* user; user @spark
>>> *Subject:* Creating a Hive table through Spark and potential locking
>>> issue (a bug)
>>>
>>>
>>>
>>>
>>>
>>> Hi,
>>>
>>>
>>>
>>> I noticed an issue with Spark creating and populating a Hive table.
>>>
>>>
>>>
>>> The process as I see is as follows:
>>>
>>>
>>>
>>>    1. Spark creates the Hive table. In this case an ORC table in a Hive
>>>    Database
>>>    2. Spark uses JDBC connection to get data out from an Oracle
>>>    3. I create a temp table in Spark through (registerTempTable)
>>>    4. Spark populates that table. That table is actually created in
>>>
>>>        hdfs dfs -ls /tmp/hive/hduser
>>>
>>>        drwx------   - hduser supergroup
>>>
>>>        /tmp/hive/hduser/b1ea6829-790f-4b37-a0ff-3ed218388059
>>>
>>>
>>>
>>>
>>>
>>>    1. However, The original table itself does not have any locking on
>>>    it!
>>>    2. I log in into Hive and drop that table
>>>
>>> 3.hive> drop table dummy;
>>>
>>> OK
>>>
>>>
>>>
>>>    1.  That table is dropped OK
>>>    2. Spark crashes with message
>>>
>>> Started at
>>> [08/06/2016 18:37:53.53]
>>> 16/06/08 19:13:46 ERROR Executor: Exception in task 0.0 in stage 1.0
>>> (TID 1)
>>>
>>> org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException):
>>> No lease on
>>> /user/hive/warehouse/oraclehadoop.db/dummy/.hive-staging_hive_2016-06-08_18-38-08_804_3299712811201460314-1/-ext-10000/_temporary/0/_temporary/attempt_201606081838_0001_m_000000_0/part-00000
>>> (inode 831621): File does not exist. Holder
>>> DFSClient_NONMAPREDUCE_-1836386597_1 does not have any open files.
>>>         at
>>> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkLease(FSNamesystem.java:3516)
>>>         at
>>> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.analyzeFileState(FSNamesystem.java:3313)
>>>         at
>>> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getAdditionalBlock(FSNamesystem.java:3169)
>>>         at
>>> org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.addBlock(NameNodeRpcServer.java:641)
>>>         at
>>> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.addBlock(ClientNamenodeProtocolServerSideTranslatorPB.java:482)
>>>         at
>>> org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
>>>         at
>>> org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:619)
>>>         at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:962)
>>>         at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2039)
>>>         at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2035)
>>>         at java.security.AccessController.doPrivileged(Native Method)
>>>         at javax.security.auth.Subject.doAs(Subject.java:422)
>>>         at
>>> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1628)
>>>         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2033)
>>>
>>>         at org.apache.hadoop.ipc.Client.call(Client.java:1468)
>>>         at org.apache.hadoop.ipc.Client.call(Client.java:1399)
>>>         at
>>> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:232)
>>>         at com.sun.proxy.$Proxy22.addBlock(Unknown Source)
>>>         at
>>> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.addBlock(ClientNamenodeProtocolTranslatorPB.java:399)
>>>         at sun.reflect.GeneratedMethodAccessor16.invoke(Unknown Source)
>>>         at
>>> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>>>         at java.lang.reflect.Method.invoke(Method.java:498)
>>>         at
>>> org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:187)
>>>         at
>>> org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
>>>         at com.sun.proxy.$Proxy23.addBlock(Unknown Source)
>>>         at
>>> org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.locateFollowingBlock(DFSOutputStream.java:1532)
>>>         at
>>> org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.nextBlockOutputStream(DFSOutputStream.java:1349)
>>>         at
>>> org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.run(DFSOutputStream.java:588)
>>> 16/06/08 19:13:46 ERROR TaskSetManager: Task 0 in stage 1.0 failed 1
>>> times; aborting job
>>>
>>>
>>>
>>> Suggested solution.
>>>
>>> In a concurrent env, Spark should apply locks in order to prevent such
>>> operations. Locks are kept in Hive meta data table HIVE_LOCKS
>>>
>>>
>>>
>>> HTH
>>>
>>> Dr Mich Talebzadeh
>>>
>>> LinkedIn  
>>> *https://www.linkedin.com/profile/view?id=AAEAAAAWh2gBxianrbJd6zP6AcPCCdOABUrV8Pw
>>> <https://www.linkedin.com/profile/view?id=AAEAAAAWh2gBxianrbJd6zP6AcPCCdOABUrV8Pw>*
>>>
>>> http://talebzadehmich.wordpress.com
>>>
>>>
>>
>>
>>
>

Reply via email to