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

2017-01-18 Thread Michal Klempa (JIRA)

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

Michal Klempa updated HIVE-15658:
-
Description: 
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 
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)
... 1 more
Caused by: 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.createRootHDFSDir(SessionState.java:625)
at 
org.apache.hadoop.hi

[jira] [Updated] (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:all-tabpanel
 ]

Michal Klempa updated HIVE-15658:
-
Attachment: HIVE-15658_branch-2.1_1.patch
HIVE-15658_branch-1.2_1.patch

> 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 
> 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.HiveEndPo

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

2018-01-23 Thread Eugene Koifman (JIRA)

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

Eugene Koifman updated HIVE-15658:
--
Component/s: Transactions

> 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, Transactions
>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
>Priority: Major
> 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 
> 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