[jira] [Commented] (HIVE-15803) msck can hang when nested partitions are present

2017-02-08 Thread Ashutosh Chauhan (JIRA)

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

Ashutosh Chauhan commented on HIVE-15803:
-

+1

> msck can hang when nested partitions are present
> 
>
> Key: HIVE-15803
> URL: https://issues.apache.org/jira/browse/HIVE-15803
> Project: Hive
>  Issue Type: Bug
>  Components: Metastore
>Reporter: Rajesh Balamohan
>Assignee: Rajesh Balamohan
>Priority: Minor
> Attachments: HIVE-15803.1.patch, HIVE-15803.2.patch, HIVE-15803.patch
>
>
> Steps to reproduce. 
> {noformat}
> CREATE TABLE `repairtable`( `col` string) PARTITIONED BY (  `p1` string,  
> `p2` string)
> hive> dfs -mkdir -p /apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b;
> hive> dfs -touchz 
> /apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b/datafile;
> hive> set hive.mv.files.thread;
> hive.mv.files.thread=15
> hive> set hive.mv.files.thread=1;
> hive> MSCK TABLE repairtable;
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (HIVE-15803) msck can hang when nested partitions are present

2017-02-07 Thread Hive QA (JIRA)

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

Hive QA commented on HIVE-15803:




Here are the results of testing the latest attachment:
https://issues.apache.org/jira/secure/attachment/12851547/HIVE-15803.2.patch

{color:green}SUCCESS:{color} +1 due to 1 test(s) being added or modified.

{color:red}ERROR:{color} -1 due to 5 failed/errored test(s), 10241 tests 
executed
*Failed tests:*
{noformat}
TestDerbyConnector - did not produce a TEST-*.xml file (likely timed out) 
(batchId=235)
org.apache.hadoop.hive.cli.TestEncryptedHDFSCliDriver.testCliDriver[encryption_join_with_different_encryption_keys]
 (batchId=159)
org.apache.hadoop.hive.cli.TestPerfCliDriver.testCliDriver[query14] 
(batchId=223)
org.apache.hadoop.hive.cli.TestPerfCliDriver.testCliDriver[query23] 
(batchId=223)
org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationDrops.testDropTable
 (batchId=210)
{noformat}

Test results: https://builds.apache.org/job/PreCommit-HIVE-Build/3432/testReport
Console output: https://builds.apache.org/job/PreCommit-HIVE-Build/3432/console
Test logs: http://104.198.109.242/logs/PreCommit-HIVE-Build-3432/

Messages:
{noformat}
Executing org.apache.hive.ptest.execution.TestCheckPhase
Executing org.apache.hive.ptest.execution.PrepPhase
Executing org.apache.hive.ptest.execution.ExecutionPhase
Executing org.apache.hive.ptest.execution.ReportingPhase
Tests exited with: TestsFailedException: 5 tests failed
{noformat}

This message is automatically generated.

ATTACHMENT ID: 12851547 - PreCommit-HIVE-Build

> msck can hang when nested partitions are present
> 
>
> Key: HIVE-15803
> URL: https://issues.apache.org/jira/browse/HIVE-15803
> Project: Hive
>  Issue Type: Bug
>  Components: Metastore
>Reporter: Rajesh Balamohan
>Assignee: Rajesh Balamohan
>Priority: Minor
> Attachments: HIVE-15803.1.patch, HIVE-15803.2.patch, HIVE-15803.patch
>
>
> Steps to reproduce. 
> {noformat}
> CREATE TABLE `repairtable`( `col` string) PARTITIONED BY (  `p1` string,  
> `p2` string)
> hive> dfs -mkdir -p /apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b;
> hive> dfs -touchz 
> /apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b/datafile;
> hive> set hive.mv.files.thread;
> hive.mv.files.thread=15
> hive> set hive.mv.files.thread=1;
> hive> MSCK TABLE repairtable;
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (HIVE-15803) msck can hang when nested partitions are present

2017-02-07 Thread Rajesh Balamohan (JIRA)

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

Rajesh Balamohan commented on HIVE-15803:
-

Thank you for sharing the patch. Deadlock would happen when multiple paths are 
there. For instance, following would deadlock with the patch.

{noformat}
DROP table repairtable;
CREATE TABLE repairtable(col STRING) PARTITIONED BY (p1 STRING, p2 STRING);
dfs -mkdir -p /apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b;
dfs -mkdir -p /apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b/;
dfs -mkdir -p /apps/hive/warehouse/test.db/repairtable/p1=cc/p2=aa/p3=bb/;
dfs -mkdir -p /apps/hive/warehouse/test.db/repairtable/p1=ccc/p2=aaa/p3=bbb/;
dfs -mkdir -p /apps/hive/warehouse/test.db/repairtable/p1=/p2=/p3=/;
dfs -mkdir -p 
/apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b/;
dfs -mkdir -p 
/apps/hive/warehouse/test.db/repairtable/p1=cc/p2=aa/p3=bb/;
dfs -mkdir -p 
/apps/hive/warehouse/test.db/repairtable/p1=ccc/p2=/p3=/;

dfs -touchz /apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b/datafile;
dfs -touchz /apps/hive/warehouse/test.db/repairtable/p1=cc/p2=aa/p3=bb/datafile;
dfs -touchz 
/apps/hive/warehouse/test.db/repairtable/p1=ccc/p2=aaa/p3=bbb/datafile;
dfs -touchz 
/apps/hive/warehouse/test.db/repairtable/p1=/p2=/p3=/datafile;
dfs -touchz 
/apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b/datafile;
dfs -touchz 
/apps/hive/warehouse/test.db/repairtable/p1=cc/p2=aa/p3=bb/datafile;
dfs -touchz 
/apps/hive/warehouse/test.db/repairtable/p1=ccc/p2=/p3=/datafile;
set hive.mv.files.thread=1;
MSCK TABLE repairtable;
{noformat}

> msck can hang when nested partitions are present
> 
>
> Key: HIVE-15803
> URL: https://issues.apache.org/jira/browse/HIVE-15803
> Project: Hive
>  Issue Type: Bug
>  Components: Metastore
>Reporter: Rajesh Balamohan
>Assignee: Rajesh Balamohan
>Priority: Minor
> Attachments: HIVE-15803.patch
>
>
> Steps to reproduce. 
> {noformat}
> CREATE TABLE `repairtable`( `col` string) PARTITIONED BY (  `p1` string,  
> `p2` string)
> hive> dfs -mkdir -p /apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b;
> hive> dfs -touchz 
> /apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b/datafile;
> hive> set hive.mv.files.thread;
> hive.mv.files.thread=15
> hive> set hive.mv.files.thread=1;
> hive> MSCK TABLE repairtable;
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (HIVE-15803) msck can hang when nested partitions are present

2017-02-07 Thread Hive QA (JIRA)

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

Hive QA commented on HIVE-15803:




Here are the results of testing the latest attachment:
https://issues.apache.org/jira/secure/attachment/12851512/HIVE-15803.patch

{color:green}SUCCESS:{color} +1 due to 1 test(s) being added or modified.

{color:red}ERROR:{color} -1 due to 4 failed/errored test(s), 10241 tests 
executed
*Failed tests:*
{noformat}
TestDerbyConnector - did not produce a TEST-*.xml file (likely timed out) 
(batchId=235)
org.apache.hadoop.hive.cli.TestEncryptedHDFSCliDriver.testCliDriver[encryption_join_with_different_encryption_keys]
 (batchId=159)
org.apache.hadoop.hive.cli.TestPerfCliDriver.testCliDriver[query14] 
(batchId=223)
org.apache.hadoop.hive.cli.TestPerfCliDriver.testCliDriver[query23] 
(batchId=223)
{noformat}

Test results: https://builds.apache.org/job/PreCommit-HIVE-Build/3425/testReport
Console output: https://builds.apache.org/job/PreCommit-HIVE-Build/3425/console
Test logs: http://104.198.109.242/logs/PreCommit-HIVE-Build-3425/

Messages:
{noformat}
Executing org.apache.hive.ptest.execution.TestCheckPhase
Executing org.apache.hive.ptest.execution.PrepPhase
Executing org.apache.hive.ptest.execution.ExecutionPhase
Executing org.apache.hive.ptest.execution.ReportingPhase
Tests exited with: TestsFailedException: 4 tests failed
{noformat}

This message is automatically generated.

ATTACHMENT ID: 12851512 - PreCommit-HIVE-Build

> msck can hang when nested partitions are present
> 
>
> Key: HIVE-15803
> URL: https://issues.apache.org/jira/browse/HIVE-15803
> Project: Hive
>  Issue Type: Bug
>  Components: Metastore
>Reporter: Rajesh Balamohan
>Assignee: Rajesh Balamohan
>Priority: Minor
> Attachments: HIVE-15803.patch
>
>
> Steps to reproduce. 
> {noformat}
> CREATE TABLE `repairtable`( `col` string) PARTITIONED BY (  `p1` string,  
> `p2` string)
> hive> dfs -mkdir -p /apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b;
> hive> dfs -touchz 
> /apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b/datafile;
> hive> set hive.mv.files.thread;
> hive.mv.files.thread=15
> hive> set hive.mv.files.thread=1;
> hive> MSCK TABLE repairtable;
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (HIVE-15803) msck can hang when nested partitions are present

2017-02-07 Thread Pengcheng Xiong (JIRA)

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

Pengcheng Xiong commented on HIVE-15803:


LGTM +1.

> msck can hang when nested partitions are present
> 
>
> Key: HIVE-15803
> URL: https://issues.apache.org/jira/browse/HIVE-15803
> Project: Hive
>  Issue Type: Bug
>  Components: Metastore
>Reporter: Rajesh Balamohan
>Assignee: Rajesh Balamohan
>Priority: Minor
> Attachments: HIVE-15803.patch
>
>
> Steps to reproduce. 
> {noformat}
> CREATE TABLE `repairtable`( `col` string) PARTITIONED BY (  `p1` string,  
> `p2` string)
> hive> dfs -mkdir -p /apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b;
> hive> dfs -touchz 
> /apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b/datafile;
> hive> set hive.mv.files.thread;
> hive.mv.files.thread=15
> hive> set hive.mv.files.thread=1;
> hive> MSCK TABLE repairtable;
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (HIVE-15803) msck can hang when nested partitions are present

2017-02-07 Thread Ashutosh Chauhan (JIRA)

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

Ashutosh Chauhan commented on HIVE-15803:
-

[~nandakumar131]  Trouble with list of pool is, its unbounded, in pathological 
case of large number of levels, we will have large number of pools each having 
its own set of threads.  I like [~pattipaka] suggestion better of using counter 
that we get sharing as much as possible with predefined number of threads. 
Would you like to create a patch with that approach?

> msck can hang when nested partitions are present
> 
>
> Key: HIVE-15803
> URL: https://issues.apache.org/jira/browse/HIVE-15803
> Project: Hive
>  Issue Type: Bug
>  Components: Metastore
>Reporter: Rajesh Balamohan
>Assignee: Rajesh Balamohan
>Priority: Minor
>
> Steps to reproduce. 
> {noformat}
> CREATE TABLE `repairtable`( `col` string) PARTITIONED BY (  `p1` string,  
> `p2` string)
> hive> dfs -mkdir -p /apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b;
> hive> dfs -touchz 
> /apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b/datafile;
> hive> set hive.mv.files.thread;
> hive.mv.files.thread=15
> hive> set hive.mv.files.thread=1;
> hive> MSCK TABLE repairtable;
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (HIVE-15803) msck can hang when nested partitions are present

2017-02-06 Thread Nandakumar (JIRA)

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

Nandakumar commented on HIVE-15803:
---

We can split the {{hive.mv.files.thread}} count and create different pool for 
each level (depth)
{code}
private List getExecutorServiceList(int 
numberOfExecutorService, int totalThreads) {
  int average = totalThreads / numberOfExecutorService;
  int remainingThreads = totalThreads;
  List serviceList = new 
ArrayList(numberOfExecutorService);
  while(numberOfExecutorService == 0) {
--numberOfExecutorService;
int numberOfThreads = numberOfExecutorService != 0 ? average : 
remainingThreads;
serviceList.add(Executors.newFixedThreadPool(numberOfThreads, new 
ThreadFactoryBuilder().setDaemon(true).setNameFormat("MSCK-GetPaths-%d").build()));
remainingThreads = remainingThreads - numberOfThreads;
  }
  return serviceList;
}
{code}

Now instead of passing  {{pool}} to {{private void checkPartitionDirs(final 
ExecutorService pool, final ConcurrentLinkedQueue basePaths, final 
Set allDirs, final FileSystem fs, final int depth, final int maxDepth)}}, 
we can pass {{List poolList}} and for accessing the pool we 
can use  {{poolList\[depth\]#submit}}

> msck can hang when nested partitions are present
> 
>
> Key: HIVE-15803
> URL: https://issues.apache.org/jira/browse/HIVE-15803
> Project: Hive
>  Issue Type: Bug
>  Components: Metastore
>Reporter: Rajesh Balamohan
>Assignee: Rajesh Balamohan
>Priority: Minor
>
> Steps to reproduce. 
> {noformat}
> CREATE TABLE `repairtable`( `col` string) PARTITIONED BY (  `p1` string,  
> `p2` string)
> hive> dfs -mkdir -p /apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b;
> hive> dfs -touchz 
> /apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b/datafile;
> hive> set hive.mv.files.thread;
> hive.mv.files.thread=15
> hive> set hive.mv.files.thread=1;
> hive> MSCK TABLE repairtable;
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (HIVE-15803) msck can hang when nested partitions are present

2017-02-06 Thread Subramanyam Pattipaka (JIRA)

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

Subramanyam Pattipaka commented on HIVE-15803:
--

[~rajesh.balamohan] and [~ashutoshc], instead of simply passing null to 
recursive call, can we maintain an atomic counter and pass null as soon we see 
number of threads in the pool are over. That way we can best utilize the 
threads.

> msck can hang when nested partitions are present
> 
>
> Key: HIVE-15803
> URL: https://issues.apache.org/jira/browse/HIVE-15803
> Project: Hive
>  Issue Type: Bug
>  Components: Metastore
>Reporter: Rajesh Balamohan
>Assignee: Rajesh Balamohan
>Priority: Minor
>
> Steps to reproduce. 
> {noformat}
> CREATE TABLE `repairtable`( `col` string) PARTITIONED BY (  `p1` string,  
> `p2` string)
> hive> dfs -mkdir -p /apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b;
> hive> dfs -touchz 
> /apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b/datafile;
> hive> set hive.mv.files.thread;
> hive.mv.files.thread=15
> hive> set hive.mv.files.thread=1;
> hive> MSCK TABLE repairtable;
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (HIVE-15803) msck can hang when nested partitions are present

2017-02-06 Thread Pengcheng Xiong (JIRA)

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

Pengcheng Xiong commented on HIVE-15803:


[~ashutoshc], i think we can revert our patch to disable reusing the pool when 
we do the BFS.

> msck can hang when nested partitions are present
> 
>
> Key: HIVE-15803
> URL: https://issues.apache.org/jira/browse/HIVE-15803
> Project: Hive
>  Issue Type: Bug
>  Components: Metastore
>Reporter: Rajesh Balamohan
>Assignee: Rajesh Balamohan
>Priority: Minor
>
> Steps to reproduce. 
> {noformat}
> CREATE TABLE `repairtable`( `col` string) PARTITIONED BY (  `p1` string,  
> `p2` string)
> hive> dfs -mkdir -p /apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b;
> hive> dfs -touchz 
> /apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b/datafile;
> hive> set hive.mv.files.thread;
> hive.mv.files.thread=15
> hive> set hive.mv.files.thread=1;
> hive> MSCK TABLE repairtable;
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (HIVE-15803) msck can hang when nested partitions are present

2017-02-06 Thread Ashutosh Chauhan (JIRA)

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

Ashutosh Chauhan commented on HIVE-15803:
-

I agree with [~rajesh.balamohan] suggestion to pass on null for thread pull in 
recursive call. It will mean parallelism will be limited to number of distinct 
dirs in first level, but that will be better than single threaded execution as 
well as hangs.
[~pxiong] What do you think?

> msck can hang when nested partitions are present
> 
>
> Key: HIVE-15803
> URL: https://issues.apache.org/jira/browse/HIVE-15803
> Project: Hive
>  Issue Type: Bug
>  Components: Metastore
>Reporter: Rajesh Balamohan
>Assignee: Rajesh Balamohan
>Priority: Minor
>
> Steps to reproduce. 
> {noformat}
> CREATE TABLE `repairtable`( `col` string) PARTITIONED BY (  `p1` string,  
> `p2` string)
> hive> dfs -mkdir -p /apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b;
> hive> dfs -touchz 
> /apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b/datafile;
> hive> set hive.mv.files.thread;
> hive.mv.files.thread=15
> hive> set hive.mv.files.thread=1;
> hive> MSCK TABLE repairtable;
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (HIVE-15803) msck can hang when nested partitions are present

2017-02-03 Thread Rajesh Balamohan (JIRA)

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

Rajesh Balamohan commented on HIVE-15803:
-

[~hagleitn] - It is not slow, it gets stuck.  

Another option is to use CachedThreadPool (which would spin up additional 
threads on need basis) instead of FixedThreadPool in executor service.  But 
again, if there are 5 partitions with nesting, it would end up throwing 
"unable to create native thread".  Passing "null" (instead of pool) 
https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveMetaStoreChecker.java#L499
 would ensure that only certain amount of threads are there doing the 
processing and any further nesting has to be handled in the same thread 
context. That would still improve the perf compared to no-thread pool scenario 
as 15 threads would be processing instead of 1.

> msck can hang when nested partitions are present
> 
>
> Key: HIVE-15803
> URL: https://issues.apache.org/jira/browse/HIVE-15803
> Project: Hive
>  Issue Type: Bug
>  Components: Metastore
>Reporter: Rajesh Balamohan
>Assignee: Rajesh Balamohan
>Priority: Minor
>
> Steps to reproduce. 
> {noformat}
> CREATE TABLE `repairtable`( `col` string) PARTITIONED BY (  `p1` string,  
> `p2` string)
> hive> dfs -mkdir -p /apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b;
> hive> dfs -touchz 
> /apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b/datafile;
> hive> set hive.mv.files.thread;
> hive.mv.files.thread=15
> hive> set hive.mv.files.thread=1;
> hive> MSCK TABLE repairtable;
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (HIVE-15803) msck can hang when nested partitions are present

2017-02-03 Thread Vihang Karajgaonkar (JIRA)

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

Vihang Karajgaonkar commented on HIVE-15803:


I agree with [~hagleitn]. The issue cannot be solved just by changing it to 
"right" number of threads. It will always happen when number of partition keys 
(depth of the recursion tree) is greater than the number of the threads. For 
instance I could reproduce the problem by setting number of threads to 3 and 
creating 4 partition keys.

{noformat}
0 jdbc:hive2://localhost:1/> set hive.mv.files.thread=3;
0: jdbc:hive2://localhost:1/> create table repairtable3(col string) 
partitioned by (p1 string, p2 string, p3 string, p4 string);
0: jdbc:hive2://localhost:1/> dfs -mkdir -p 
/user/hive/warehouse/repairtable3/p1=a/p2=b/p3=c/p4=d;
0: jdbc:hive2://localhost:1/> dfs -touchz 
/user/hive/warehouse/repairtable3/p1=a/p2=b/p3=c/p4=d/datafile;
0: jdbc:hive2://localhost:1/> msck repair table repairtable3;
{noformat}

The issue happens because each thread which processes a path X waits until some 
other thread from the pool processes the children paths of X. If the recursion 
level is deep enough eventually the pool runs out of threads to process the 
children paths.

> msck can hang when nested partitions are present
> 
>
> Key: HIVE-15803
> URL: https://issues.apache.org/jira/browse/HIVE-15803
> Project: Hive
>  Issue Type: Bug
>  Components: Metastore
>Reporter: Rajesh Balamohan
>Assignee: Rajesh Balamohan
>Priority: Minor
>
> Steps to reproduce. 
> {noformat}
> CREATE TABLE `repairtable`( `col` string) PARTITIONED BY (  `p1` string,  
> `p2` string)
> hive> dfs -mkdir -p /apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b;
> hive> dfs -touchz 
> /apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b/datafile;
> hive> set hive.mv.files.thread;
> hive.mv.files.thread=15
> hive> set hive.mv.files.thread=1;
> hive> MSCK TABLE repairtable;
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (HIVE-15803) msck can hang when nested partitions are present

2017-02-03 Thread Gunther Hagleitner (JIRA)

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

Gunther Hagleitner commented on HIVE-15803:
---

[~rajesh.balamohan] are you saying "very slow" or dead lock? I'm assuming it's 
the latter.

I don't think manually changing the setting is a solution. It would mean that 
users can run into hang/deadlock scenarios and then have to find the correct 
setting to move on. Why do we run into the hang in the first place? It seems we 
should be able to have a fixed size pool and predictable completion at the same 
time.

> msck can hang when nested partitions are present
> 
>
> Key: HIVE-15803
> URL: https://issues.apache.org/jira/browse/HIVE-15803
> Project: Hive
>  Issue Type: Bug
>  Components: Metastore
>Reporter: Rajesh Balamohan
>Assignee: Rajesh Balamohan
>Priority: Minor
>
> Steps to reproduce. 
> {noformat}
> CREATE TABLE `repairtable`( `col` string) PARTITIONED BY (  `p1` string,  
> `p2` string)
> hive> dfs -mkdir -p /apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b;
> hive> dfs -touchz 
> /apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b/datafile;
> hive> set hive.mv.files.thread;
> hive.mv.files.thread=15
> hive> set hive.mv.files.thread=1;
> hive> MSCK TABLE repairtable;
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (HIVE-15803) msck can hang when nested partitions are present

2017-02-03 Thread Pengcheng Xiong (JIRA)

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

Pengcheng Xiong commented on HIVE-15803:


If I remembered correctly, I did not share the thread pool when I do a BFS in 
my first patch. Then reviewers suggested reuse the thread pool. I changed my 
patch to reuse them. If we are using the patch here, then it means we are only 
using thread-pool for the first level of directory. I think this is a setting 
problem. I would suggest that the user should set hive.mv.files.thread=0 or 
hive.mv.files.thread=a number that is large enough. Or i would change 
hive.mv.files.thread to a boolean. If it is true, then we are going to 
automatically set it to the maximum number, else we set it to 0. How about 
this? ccing [~ashutoshc] and [~hagleitn]

> msck can hang when nested partitions are present
> 
>
> Key: HIVE-15803
> URL: https://issues.apache.org/jira/browse/HIVE-15803
> Project: Hive
>  Issue Type: Bug
>  Components: Metastore
>Reporter: Rajesh Balamohan
>Assignee: Rajesh Balamohan
>Priority: Minor
>
> Steps to reproduce. 
> {noformat}
> CREATE TABLE `repairtable`( `col` string) PARTITIONED BY (  `p1` string,  
> `p2` string)
> hive> dfs -mkdir -p /apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b;
> hive> dfs -touchz 
> /apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b/datafile;
> hive> set hive.mv.files.thread;
> hive.mv.files.thread=15
> hive> set hive.mv.files.thread=1;
> hive> MSCK TABLE repairtable;
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (HIVE-15803) msck can hang when nested partitions are present

2017-02-03 Thread Rajesh Balamohan (JIRA)

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

Rajesh Balamohan commented on HIVE-15803:
-

Stack:

{noformat}
"main" #1 prio=5 os_prio=0 tid=0x7f853401a000 nid=0x3e3e waiting on 
condition [0x7f853b0b6000]
   java.lang.Thread.State: WAITING (parking)
at sun.misc.Unsafe.park(Native Method)
- parking to wait for  <0xfdfaba30> (a 
java.util.concurrent.FutureTask)
at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
at java.util.concurrent.FutureTask.awaitDone(FutureTask.java:429)
at java.util.concurrent.FutureTask.get(FutureTask.java:191)
at 
org.apache.hadoop.hive.ql.metadata.HiveMetaStoreChecker.checkPartitionDirs(HiveMetaStoreChecker.java:523)
at 
org.apache.hadoop.hive.ql.metadata.HiveMetaStoreChecker.checkPartitionDirs(HiveMetaStoreChecker.java:424)
at 
org.apache.hadoop.hive.ql.metadata.HiveMetaStoreChecker.findUnknownPartitions(HiveMetaStoreChecker.java:315)
at 
org.apache.hadoop.hive.ql.metadata.HiveMetaStoreChecker.checkTable(HiveMetaStoreChecker.java:291)
at 
org.apache.hadoop.hive.ql.metadata.HiveMetaStoreChecker.checkTable(HiveMetaStoreChecker.java:236)
at 
org.apache.hadoop.hive.ql.metadata.HiveMetaStoreChecker.checkMetastore(HiveMetaStoreChecker.java:113)
at org.apache.hadoop.hive.ql.exec.DDLTask.msck(DDLTask.java:1759)
at org.apache.hadoop.hive.ql.exec.DDLTask.execute(DDLTask.java:378)
at org.apache.hadoop.hive.ql.exec.Task.executeTask(Task.java:160)
at 
org.apache.hadoop.hive.ql.exec.TaskRunner.runSequential(TaskRunner.java:89)
at org.apache.hadoop.hive.ql.Driver.launchTask(Driver.java:1745)
at org.apache.hadoop.hive.ql.Driver.execute(Driver.java:1491)
at org.apache.hadoop.hive.ql.Driver.runInternal(Driver.java:1289)
{noformat}

> msck can hang when nested partitions are present
> 
>
> Key: HIVE-15803
> URL: https://issues.apache.org/jira/browse/HIVE-15803
> Project: Hive
>  Issue Type: Bug
>  Components: Metastore
>Reporter: Rajesh Balamohan
>Priority: Minor
>
> Steps to reproduce. 
> {noformat}
> CREATE TABLE `repairtable`( `col` string) PARTITIONED BY (  `p1` string,  
> `p2` string)
> hive> dfs -mkdir -p /apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b;
> hive> dfs -touchz 
> /apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b/datafile;
> hive> set hive.mv.files.thread;
> hive.mv.files.thread=15
> hive> set hive.mv.files.thread=1;
> hive> MSCK TABLE repairtable;
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)