Eugene Koifman pointed out to me that there is one other possibility.  If we 
ignore query 3 for a minute, I can explain this with the following timeline:

1) Query one requests and gets a read lock, starts its select.
2) Query two (alter table…) requests an exclusive lock, and then blocks until 
query one has completed.
3) Query four requests a read lock, and then blocks until query two has 
completed.  Once it can run it finds that part of its inputs have gone away and 
thus fails with an error.

Hive’s metadata is not part of the transactional system.  So we don’t keep the 
old dropped partition around after query two drops it.  So the error from query 
four is actually the correct thing to do.  If four ignored the fact that the 
partition had vanished and returned results minus that partition it would not 
be returning correct results as of the time its transaction started.  Because 
the underlying data has been dropped the query is now unanswerable an error is 
the best possible answer.

This situation cannot be fixed until Hive DDL operations take part in 
transactions.  I don’t know when that will happen.

However, this explanation is incomplete because it doesn’t account for query 
three.  That query should also have blocked.  The code is not supposed to allow 
“jumping” exclusive locks.  That is, query three shouldn’t be allowed to 
acquire the read lock along with query one because query two’s exclusive lock 
is in between.  It’s possible there’s a bug in that code.  It’s also possible 
that the above explanation is not what’s happening.

Would it be possible for you to turn on debug logging on your thrift metastore 
process and rerun this test and post the logs somewhere?  Apache lists strip 
attachments so you won’t be able to attach them here, you’ll have to put them 
up on paste bin or something.

Alan.

> On Jun 7, 2016, at 04:00, Igor Kuzmenko <f1she...@gmail.com> wrote:
> 
>       • hive.support.concurrency – true
>       • hive.enforce.bucketing – true (Not required as of Hive 2.0)
>       • hive.exec.dynamic.partition.mode – nonstrict
>       • hive.txn.manager – org.apache.hadoop.hive.ql.lockmgr.DbTxnManager
>       • hive.compactor.initiator.on – true (for exactly one instance of the 
> Thrift metastore service)
>       • hive.compactor.worker.threads – a positive number on at least one 
> instance of the Thrift metastore service
> I've double check all settings and restart hive. The problem remained.
> I've complete the same test with little modification in console output.
> asyncExecute("Select count(distinct in_info_msisdn) from mobile_connections 
> where dt=20151124 and msisdn_last_digit=1", 1);
> Thread.sleep(3000);
> asyncExecute("alter table mobile_connections drop if exists partition 
> (dt=20151124, msisdn_last_digit=1) purge", 2);
> Thread.sleep(3000);
> asyncExecute("Select count(distinct in_info_msisdn) from mobile_connections 
> where dt=20151124 and msisdn_last_digit=1", 3);
> Thread.sleep(3000);
> asyncExecute("Select count(distinct in_info_msisdn) from mobile_connections 
> where dt=20151124 and msisdn_last_digit=1", 4);
> 
> Like in previous test four queries executing in parallel. First, third and 
> fourth are simple selects. Second is drop partition command.
> Here's out put:
> 
> Tue Jun 07 13:45:18 MSK 2016 Start thread 1
> Tue Jun 07 13:45:21 MSK 2016 Start thread 2
> Tue Jun 07 13:45:24 MSK 2016 Start thread 3
> Tue Jun 07 13:45:27 MSK 2016 Start thread 4
> 
> Tue Jun 07 13:45:32 MSK 2016 Finish thread 1
> Tue Jun 07 13:45:32 MSK 2016 Thread 1 result: '210802'
> Tue Jun 07 13:45:32 MSK 2016 Thread 1 completed in 13810 ms
> 
> Tue Jun 07 13:45:35 MSK 2016 Finished thread 2 
> Tue Jun 07 13:45:35 MSK 2016 Thread 2 completed in 13568 ms
> 
> Tue Jun 07 13:45:39 MSK 2016 Finish tread 3
> Tue Jun 07 13:45:39 MSK 2016 Thread 3 result: '210802'
> Tue Jun 07 13:45:39 MSK 2016 Thread 3 completed in 14970 ms
> 
> java.sql.SQLException: Error while processing statement: FAILED: Execution 
> Error, return code 2 from org.apache.hadoop.hive.ql.exec.tez.TezTask. Vertex 
> failed, vertexName=Map 1, vertexId=vertex_1461923723503_0256_1_00, 
> diagnostics=[Vertex vertex_1461923723503_0256_1_00 [Map 1] killed/failed due 
> to:ROOT_INPUT_INIT_FAILURE, Vertex Input: mobile_connections initializer 
> failed, vertex=vertex_1461923723503_0256_1_00 [Map 1], 
> java.lang.RuntimeException: serious problem
>       at 
> org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.generateSplitsInfo(OrcInputFormat.java:1059)
>       at 
> org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.getSplits(OrcInputFormat.java:1086)
>       at 
> org.apache.hadoop.hive.ql.io.HiveInputFormat.addSplitsForGroup(HiveInputFormat.java:305)
>       at 
> org.apache.hadoop.hive.ql.io.HiveInputFormat.getSplits(HiveInputFormat.java:407)
>       at 
> org.apache.hadoop.hive.ql.exec.tez.HiveSplitGenerator.initialize(HiveSplitGenerator.java:155)
>       at 
> org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable$1.run(RootInputInitializerManager.java:255)
>       at 
> org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable$1.run(RootInputInitializerManager.java:248)
>       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:1657)
>       at 
> org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable.call(RootInputInitializerManager.java:248)
>       at 
> org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable.call(RootInputInitializerManager.java:235)
>       at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>       at java.lang.Thread.run(Thread.java:745)
> Caused by: java.util.concurrent.ExecutionException: 
> java.io.FileNotFoundException: File 
> hdfs://jupiter.bss:8020/apps/hive/warehouse/mobile_connections/dt=20151124/msisdn_last_digit=2
>  does not exist.
>       at java.util.concurrent.FutureTask.report(FutureTask.java:122)
>       at java.util.concurrent.FutureTask.get(FutureTask.java:192)
>       at 
> org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.generateSplitsInfo(OrcInputFormat.java:1036)
>       ... 15 more
> 
> So second thread definitely waits until first thread completes and than make 
> a partition drop. Than, somehow, after partition was droped, third query 
> completes and shows result. Fourth query doesn't complete at all, throwing 
> exception. 
> 
> 
> 
> 
> 
> 
> On Mon, Jun 6, 2016 at 8:30 PM, Alan Gates <alanfga...@gmail.com> wrote:
> Do you have the system configured to use the DbTxnManager?  See 
> https://cwiki.apache.org/confluence/display/Hive/Hive+Transactions#HiveTransactions-Configuration
>  for details on how to set this up.  The transaction manager is what manages 
> locking and makes sure that your queries don’t stomp each other.
> 
> Alan.
> 
> > On Jun 6, 2016, at 06:01, Igor Kuzmenko <f1she...@gmail.com> wrote:
> >
> > Hello, I'm trying to find a safe way to delete partition with all data it 
> > includes.
> >
> > I'm using Hive 1.2.1, Hive JDBC driver 1.2.1 and perform simple test on 
> > transactional table:
> >
> > asyncExecute("Select count(distinct in_info_msisdn) from mobile_connections 
> > where dt=20151124 and msisdn_last_digit=2", 1);
> > Thread.sleep(3000);
> > asyncExecute("alter table mobile_connections drop if exists partition 
> > (dt=20151124, msisdn_last_digit=2) purge", 2);
> > Thread.sleep(3000);
> > asyncExecute("Select count(distinct in_info_msisdn) from mobile_connections 
> > where dt=20151124 and msisdn_last_digit=2", 3);
> > Thread.sleep(3000);
> > asyncExecute("Select count(distinct in_info_msisdn) from mobile_connections 
> > where dt=20151124 and msisdn_last_digit=2", 4);
> > (full code here)
> >
> > I cretate several threads, each execute query async. First is querying 
> > partition. Second drop partition. Others are the same as first. First query 
> > takes about 10-15 seconds to complete, so "alter table" query starts before 
> > first query completes.
> > As a result i get:
> >       • First query - successfully completes
> >       • Second query - successfully completes
> >       • Third query - successfully completes
> >       • Fourth query - throw exception:
> > java.sql.SQLException: Error while processing statement: FAILED: Execution 
> > Error, return code 2 from org.apache.hadoop.hive.ql.exec.tez.TezTask. 
> > Vertex failed, vertexName=Map 1, vertexId=vertex_1461923723503_0189_1_00, 
> > diagnostics=[Vertex vertex_1461923723503_0189_1_00 [Map 1] killed/failed 
> > due to:ROOT_INPUT_INIT_FAILURE, Vertex Input: mobile_connections 
> > initializer failed, vertex=vertex_1461923723503_0189_1_00 [Map 1], 
> > java.lang.RuntimeException: serious problem
> >       at 
> > org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.generateSplitsInfo(OrcInputFormat.java:1059)
> >       at 
> > org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.getSplits(OrcInputFormat.java:1086)
> >       at 
> > org.apache.hadoop.hive.ql.io.HiveInputFormat.addSplitsForGroup(HiveInputFormat.java:305)
> >       at 
> > org.apache.hadoop.hive.ql.io.HiveInputFormat.getSplits(HiveInputFormat.java:407)
> >       at 
> > org.apache.hadoop.hive.ql.exec.tez.HiveSplitGenerator.initialize(HiveSplitGenerator.java:155)
> >       at 
> > org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable$1.run(RootInputInitializerManager.java:255)
> >       at 
> > org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable$1.run(RootInputInitializerManager.java:248)
> >       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:1657)
> >       at 
> > org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable.call(RootInputInitializerManager.java:248)
> >       at 
> > org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable.call(RootInputInitializerManager.java:235)
> >       at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> >       at 
> > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> >       at 
> > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> >       at java.lang.Thread.run(Thread.java:745)
> > Caused by: java.util.concurrent.ExecutionException: 
> > java.io.FileNotFoundException: File 
> > hdfs://jupiter.bss:8020/apps/hive/warehouse/mobile_connections/dt=20151124/msisdn_last_digit=2
> >  does not exist.
> >       at java.util.concurrent.FutureTask.report(FutureTask.java:122)
> >       at java.util.concurrent.FutureTask.get(FutureTask.java:192)
> >       at 
> > org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.generateSplitsInfo(OrcInputFormat.java:1036)
> >       ... 15 more
> > Caused by: java.io.FileNotFoundException: File 
> > hdfs://jupiter.bss:8020/apps/hive/warehouse/mobile_connections/dt=20151124/msisdn_last_digit=2
> >  does not exist.
> >       at 
> > org.apache.hadoop.hdfs.DistributedFileSystem$DirListingIterator.<init>(DistributedFileSystem.java:958)
> >       at 
> > org.apache.hadoop.hdfs.DistributedFileSystem$DirListingIterator.<init>(DistributedFileSystem.java:937)
> >       at 
> > org.apache.hadoop.hdfs.DistributedFileSystem$19.doCall(DistributedFileSystem.java:882)
> >       at 
> > org.apache.hadoop.hdfs.DistributedFileSystem$19.doCall(DistributedFileSystem.java:878)
> >       at 
> > org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
> >       at 
> > org.apache.hadoop.hdfs.DistributedFileSystem.listLocatedStatus(DistributedFileSystem.java:878)
> >       at 
> > org.apache.hadoop.fs.FileSystem.listLocatedStatus(FileSystem.java:1694)
> >       at 
> > org.apache.hadoop.hive.shims.Hadoop23Shims.listLocatedStatus(Hadoop23Shims.java:690)
> >       at 
> > org.apache.hadoop.hive.ql.io.AcidUtils.getAcidState(AcidUtils.java:366)
> >       at 
> > org.apache.hadoop.hive.ql.io.orc.OrcInputFormat$FileGenerator.call(OrcInputFormat.java:648)
> >       at 
> > org.apache.hadoop.hive.ql.io.orc.OrcInputFormat$FileGenerator.call(OrcInputFormat.java:634)
> >       ... 4 more
> > ]Vertex killed, vertexName=Reducer 3, 
> > vertexId=vertex_1461923723503_0189_1_02, diagnostics=[Vertex received Kill 
> > in INITED state., Vertex vertex_1461923723503_0189_1_02 [Reducer 3] 
> > killed/failed due to:OTHER_VERTEX_FAILURE]Vertex killed, vertexName=Reducer 
> > 2, vertexId=vertex_1461923723503_0189_1_01, diagnostics=[Vertex received 
> > Kill in INITED state., Vertex vertex_1461923723503_0189_1_01 [Reducer 2] 
> > killed/failed due to:OTHER_VERTEX_FAILURE]DAG did not succeed due to 
> > VERTEX_FAILURE. failedVertices:1 killedVertices:2
> >       at org.apache.hive.jdbc.HiveStatement.execute(HiveStatement.java:296)
> >       at Test$MyRunnable.run(Test.java:54)
> >       at java.lang.Thread.run(Thread.java:745)
> >
> > Since I'm using transactional table, I expect, that all queries, executed 
> > after partition drop, will complete successfully with no result.  Am I 
> > doing something wrong? Is there other way to drop partition with data?
> 
> 

Reply via email to