[jira] [Commented] (HIVE-15658) hive.ql.session.SessionState start() is not atomic, SessionState thread local variable can get into inconsistent state

2017-01-20 Thread Michal Klempa (JIRA)

[ 
https://issues.apache.org/jira/browse/HIVE-15658?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15832835#comment-15832835
 ] 

Michal Klempa commented on HIVE-15658:
--

Hi Sergey, thank you for response, unfortunately, I am not the author of the 
original code and I have problems figuring out all the places where this should 
be fixed. I analyzed all the calls of .setCurrentSession and modified those 
parts, where I was it is not paired with .detach().
If, in HiveSessionImpl the session is already valid, why there is call to 
setSessionState, though. Maybe it may be reomved.

Maybe we need more complex change of this, to centralize the point where the 
Session is created & initialized atomically and then only call this one 
procedure. Standard way of doing this is in constructors. Detaching/closing is 
then done in .close(). I do not know why this time, the .start was chosen to 
decouple construction and starting of object, and why classes using 
SessionState are calling .detach() directly and not only .close(). The outer 
API of SessionState class is not clear to me.

> hive.ql.session.SessionState start() is not atomic, SessionState thread local 
> variable can get into inconsistent state
> --
>
> Key: HIVE-15658
> URL: https://issues.apache.org/jira/browse/HIVE-15658
> Project: Hive
>  Issue Type: Bug
>  Components: API, HCatalog
>Affects Versions: 1.1.0, 1.2.1, 2.0.0, 2.0.1
> Environment: CDH5.8.0, Flume 1.6.0, Hive 1.1.0
>Reporter: Michal Klempa
> Attachments: HIVE-15658_branch-1.2_1.patch, 
> HIVE-15658_branch-2.1_1.patch
>
>
> Method start() in hive.ql.session.SessionState is supposed to setup needed 
> preconditions, like HDFS scratch directories for session.
> This happens to be not an atomic operation with setting thread local 
> variable, which can later be obtained by calling SessionState.get().
> Therefore, even is the start() method itself fails, the SessionState.get() 
> does not return null and further re-use of the thread which previously 
> invoked start() may lead to obtaining SessionState object in inconsistent 
> state.
> I have observed this using Flume Hive Sink, which uses Hive Streaming 
> interface. When the directory /tmp/hive is not writable by session user, the 
> start() method fails (throwing RuntimeException). If the thread is re-used 
> (like it is in Flume), further executions work with wrongly initialized 
> SessionState object (HDFS dirs are non-existent). In Flume, this happens to 
> me when Flume should create partition if not exists (but the code doing this 
> is in Hive Streaming).
> Steps to reproduce:
> 0. create test spooldir and allow flume to write to it, in my case 
> /home/ubuntu/flume_test, 775, ubuntu:flume
> 1. create Flume config (see attachment)
> 2. create Hive table
> {code}
> create table default.flume_test (column1 string, column2 string) partitioned 
> by (dt string) clustered by (column1) INTO 2 BUCKETS STORED AS ORC;
> {code}
> 3. start flume agent:
> {code}
> bin/flume-ng agent -n a1 -c conf -f conf/flume-config.txt
> {code}
> 4. hdfs dfs -chmod 600 /tmp/hive
> 5. put this file into spooldir:
> {code}
> echo value1,value2 > file1
> {code}
> Expected behavior:
> Exception regarding scratch dir permissions to be thrown repeatedly.
> example (note that the line numbers are wrong as Cloudera is cloning the 
> source codes here https://github.com/cloudera/flume-ng/ and here 
> https://github.com/cloudera/hive):
> {code}
> 2017-01-18 12:39:38,926 WARN org.apache.flume.sink.hive.HiveSink: sink_hive_1 
> : Failed connecting to EndPoint {metaStoreUri='thrift://n02.cdh.ideata:9083', 
> database='default', table='flume_test', partitionVals=[20170118] }
> org.apache.flume.sink.hive.HiveWriter$ConnectException: Failed connecting to 
> EndPoint {metaStoreUri='thrift://n02.cdh.ideata:9083', database='default', 
> table='flume_test', partitionVals=[20170118] } 
> at org.apache.flume.sink.hive.HiveWriter.(HiveWriter.java:99)
> at 
> org.apache.flume.sink.hive.HiveSink.getOrCreateWriter(HiveSink.java:344)
> at 
> org.apache.flume.sink.hive.HiveSink.drainOneBatch(HiveSink.java:296)
> at org.apache.flume.sink.hive.HiveSink.process(HiveSink.java:254)
> at 
> org.apache.flume.sink.DefaultSinkProcessor.process(DefaultSinkProcessor.java:68)
> at org.apache.flume.SinkRunner$PollingRunner.run(SinkRunner.java:147)
> at java.lang.Thread.run(Thread.java:745)
> Caused by: org.apache.flume.sink.hive.HiveWriter$ConnectException: Failed 
> connecting to EndPoint {metaStoreUri='thrift://n02.cdh.ideata:9083', 
> database='default', table='flume_test', partitionVals=[20170118] }
> at 
> 

[jira] [Commented] (HIVE-15658) hive.ql.session.SessionState start() is not atomic, SessionState thread local variable can get into inconsistent state

2017-01-20 Thread Sergey Shelukhin (JIRA)

[ 
https://issues.apache.org/jira/browse/HIVE-15658?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15832465#comment-15832465
 ] 

Sergey Shelukhin commented on HIVE-15658:
-

In the last catch of the SessionState.java part of the patch, can you add a 
call to tezSessionState.close, if present, protected by try-catch? 
Doesn't look like it should fail leaving it in open state, but I wonder if open 
(and esp. beginOpen) calls are atomic... we'd rather not have that leak. 

Also why is HiveSessionImpl change necessary? As far as I understand, the 
session at that point is already valid. At least, it's retained in the field at 
the same time as we detach it.

> hive.ql.session.SessionState start() is not atomic, SessionState thread local 
> variable can get into inconsistent state
> --
>
> Key: HIVE-15658
> URL: https://issues.apache.org/jira/browse/HIVE-15658
> Project: Hive
>  Issue Type: Bug
>  Components: API, HCatalog
>Affects Versions: 1.1.0, 1.2.1, 2.0.0, 2.0.1
> Environment: CDH5.8.0, Flume 1.6.0, Hive 1.1.0
>Reporter: Michal Klempa
> Attachments: HIVE-15658_branch-1.2_1.patch, 
> HIVE-15658_branch-2.1_1.patch
>
>
> Method start() in hive.ql.session.SessionState is supposed to setup needed 
> preconditions, like HDFS scratch directories for session.
> This happens to be not an atomic operation with setting thread local 
> variable, which can later be obtained by calling SessionState.get().
> Therefore, even is the start() method itself fails, the SessionState.get() 
> does not return null and further re-use of the thread which previously 
> invoked start() may lead to obtaining SessionState object in inconsistent 
> state.
> I have observed this using Flume Hive Sink, which uses Hive Streaming 
> interface. When the directory /tmp/hive is not writable by session user, the 
> start() method fails (throwing RuntimeException). If the thread is re-used 
> (like it is in Flume), further executions work with wrongly initialized 
> SessionState object (HDFS dirs are non-existent). In Flume, this happens to 
> me when Flume should create partition if not exists (but the code doing this 
> is in Hive Streaming).
> Steps to reproduce:
> 0. create test spooldir and allow flume to write to it, in my case 
> /home/ubuntu/flume_test, 775, ubuntu:flume
> 1. create Flume config (see attachment)
> 2. create Hive table
> {code}
> create table default.flume_test (column1 string, column2 string) partitioned 
> by (dt string) clustered by (column1) INTO 2 BUCKETS STORED AS ORC;
> {code}
> 3. start flume agent:
> {code}
> bin/flume-ng agent -n a1 -c conf -f conf/flume-config.txt
> {code}
> 4. hdfs dfs -chmod 600 /tmp/hive
> 5. put this file into spooldir:
> {code}
> echo value1,value2 > file1
> {code}
> Expected behavior:
> Exception regarding scratch dir permissions to be thrown repeatedly.
> example (note that the line numbers are wrong as Cloudera is cloning the 
> source codes here https://github.com/cloudera/flume-ng/ and here 
> https://github.com/cloudera/hive):
> {code}
> 2017-01-18 12:39:38,926 WARN org.apache.flume.sink.hive.HiveSink: sink_hive_1 
> : Failed connecting to EndPoint {metaStoreUri='thrift://n02.cdh.ideata:9083', 
> database='default', table='flume_test', partitionVals=[20170118] }
> org.apache.flume.sink.hive.HiveWriter$ConnectException: Failed connecting to 
> EndPoint {metaStoreUri='thrift://n02.cdh.ideata:9083', database='default', 
> table='flume_test', partitionVals=[20170118] } 
> at org.apache.flume.sink.hive.HiveWriter.(HiveWriter.java:99)
> at 
> org.apache.flume.sink.hive.HiveSink.getOrCreateWriter(HiveSink.java:344)
> at 
> org.apache.flume.sink.hive.HiveSink.drainOneBatch(HiveSink.java:296)
> at org.apache.flume.sink.hive.HiveSink.process(HiveSink.java:254)
> at 
> org.apache.flume.sink.DefaultSinkProcessor.process(DefaultSinkProcessor.java:68)
> at org.apache.flume.SinkRunner$PollingRunner.run(SinkRunner.java:147)
> at java.lang.Thread.run(Thread.java:745)
> Caused by: org.apache.flume.sink.hive.HiveWriter$ConnectException: Failed 
> connecting to EndPoint {metaStoreUri='thrift://n02.cdh.ideata:9083', 
> database='default', table='flume_test', partitionVals=[20170118] }
> at 
> org.apache.flume.sink.hive.HiveWriter.newConnection(HiveWriter.java:380)
> at org.apache.flume.sink.hive.HiveWriter.(HiveWriter.java:86)
> ... 6 more
> Caused by: java.lang.RuntimeException: java.lang.RuntimeException: The root 
> scratch dir: /tmp/hive on HDFS should be writable. Current permissions are: 
> rw---
> at 
> org.apache.hadoop.hive.ql.session.SessionState.start(SessionState.java:540)
> at 
> 

[jira] [Commented] (HIVE-15658) hive.ql.session.SessionState start() is not atomic, SessionState thread local variable can get into inconsistent state

2017-01-20 Thread Michal Klempa (JIRA)

[ 
https://issues.apache.org/jira/browse/HIVE-15658?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15831627#comment-15831627
 ] 

Michal Klempa commented on HIVE-15658:
--

Another exception type, which can cause this error to happen is, when during 
creation of directories in HDFS the InterruptedException occurs. See the stack 
trace:
{code}

2017-01-19 08:36:19,917 WARN org.apache.hadoop.ipc.Client: interrupted waiting 
to send rpc request to server
java.lang.InterruptedException
at java.util.concurrent.FutureTask.awaitDone(FutureTask.java:400)
at java.util.concurrent.FutureTask.get(FutureTask.java:187)
at 
org.apache.hadoop.ipc.Client$Connection.sendRpcRequest(Client.java:1055)
at org.apache.hadoop.ipc.Client.call(Client.java:1450)
at org.apache.hadoop.ipc.Client.call(Client.java:1408)
at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:230)
at com.sun.proxy.$Proxy24.getFileInfo(Unknown Source) 
at 
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getFileInfo(ClientNamenodeProtocolTranslatorPB.java:762)
at sun.reflect.GeneratedMethodAccessor68.invoke(Unknown Source) 
at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:606)
at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:256)
at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:104)
at com.sun.proxy.$Proxy25.getFileInfo(Unknown Source) 
at org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:2102)
at 
org.apache.hadoop.hdfs.DistributedFileSystem$19.doCall(DistributedFileSystem.java:1215)
at 
org.apache.hadoop.hdfs.DistributedFileSystem$19.doCall(DistributedFileSystem.java:1211)
at 
org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
at 
org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1211)
at org.apache.hadoop.fs.FileSystem.exists(FileSystem.java:1412)
at 
org.apache.hadoop.hive.ql.session.SessionState.createRootHDFSDir(SessionState.java:616)
at 
org.apache.hadoop.hive.ql.session.SessionState.createSessionDirs(SessionState.java:574)
at 
org.apache.hadoop.hive.ql.session.SessionState.start(SessionState.java:518)
at 
org.apache.hive.hcatalog.streaming.HiveEndPoint$ConnectionImpl.createPartitionIfNotExists(HiveEndPoint.java:358)
at 
org.apache.hive.hcatalog.streaming.HiveEndPoint$ConnectionImpl.(HiveEndPoint.java:276)
at 
org.apache.hive.hcatalog.streaming.HiveEndPoint$ConnectionImpl.(HiveEndPoint.java:243)
at 
org.apache.hive.hcatalog.streaming.HiveEndPoint.newConnectionImpl(HiveEndPoint.java:180)
at 
org.apache.hive.hcatalog.streaming.HiveEndPoint.newConnection(HiveEndPoint.java:157)
at 
org.apache.hive.hcatalog.streaming.HiveEndPoint.newConnection(HiveEndPoint.java:110)
at org.apache.flume.sink.hive.HiveWriter$8.call(HiveWriter.java:376)
at org.apache.flume.sink.hive.HiveWriter$8.call(HiveWriter.java:373)
at org.apache.flume.sink.hive.HiveWriter$11.call(HiveWriter.java:425)
at java.util.concurrent.FutureTask.run(FutureTask.java:262)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:745)
{code}

> hive.ql.session.SessionState start() is not atomic, SessionState thread local 
> variable can get into inconsistent state
> --
>
> Key: HIVE-15658
> URL: https://issues.apache.org/jira/browse/HIVE-15658
> Project: Hive
>  Issue Type: Bug
>  Components: API, HCatalog
>Affects Versions: 1.1.0, 1.2.1, 2.0.0, 2.0.1
> Environment: CDH5.8.0, Flume 1.6.0, Hive 1.1.0
>Reporter: Michal Klempa
> Attachments: HIVE-15658_branch-1.2_1.patch, 
> HIVE-15658_branch-2.1_1.patch
>
>
> Method start() in hive.ql.session.SessionState is supposed to setup needed 
> preconditions, like HDFS scratch directories for session.
> This happens to be not an atomic operation with setting thread local 
> variable, which can later be obtained by calling SessionState.get().
> Therefore, even is the start() method itself fails, the SessionState.get() 
> does not return null and further re-use of the thread which previously 
> invoked start() may lead to obtaining SessionState object in inconsistent 
> state.
> I have observed this using Flume Hive Sink, which uses Hive Streaming 
> interface. When the