From d2eaf147599a0fed7069c529b6b3814914919a23 Mon Sep 17 00:00:00 2001 From: Ashutosh Chauhan Date: Tue, 7 Feb 2017 16:09:38 -0800 Subject: [PATCH] HIVE-15803 : msck can hang when nested partitions are present --- .../hive/ql/metadata/HiveMetaStoreChecker.java | 13 ++++++----- .../queries/clientpositive/msck_repair_batchsize.q | 10 +++++++++ .../clientpositive/msck_repair_batchsize.q.out | 25 ++++++++++++++++++++++ 3 files changed, 43 insertions(+), 5 deletions(-) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveMetaStoreChecker.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveMetaStoreChecker.java index 57f731f..ca1eb7e 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveMetaStoreChecker.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveMetaStoreChecker.java @@ -303,7 +303,7 @@ void checkTable(Table table, List parts, * Result object * @throws IOException * Thrown if we fail at fetching listings from the fs. - * @throws HiveException + * @throws HiveException */ void findUnknownPartitions(Table table, Set partPaths, CheckResult result) throws IOException, HiveException { @@ -400,17 +400,20 @@ static String getPartitionName(Path tablePath, Path partitionPath, * Specify how deep the search goes. * @throws IOException * Thrown if we can't get lists from the fs. - * @throws HiveException + * @throws HiveException */ private void checkPartitionDirs(Path basePath, Set allDirs, int maxDepth) throws IOException, HiveException { ConcurrentLinkedQueue basePaths = new ConcurrentLinkedQueue<>(); basePaths.add(basePath); - Set dirSet = Collections.newSetFromMap(new ConcurrentHashMap()); + Set dirSet = Collections.newSetFromMap(new ConcurrentHashMap()); // Here we just reuse the THREAD_COUNT configuration for // HIVE_MOVE_FILES_THREAD_COUNT - final ExecutorService pool = conf.getInt(ConfVars.HIVE_MOVE_FILES_THREAD_COUNT.varname, 25) > 0 ? Executors - .newFixedThreadPool(conf.getInt(ConfVars.HIVE_MOVE_FILES_THREAD_COUNT.varname, 25), + int poolSize = conf.getInt(ConfVars.HIVE_MOVE_FILES_THREAD_COUNT.varname, 15); + // we need atleast as many threads as # of partition levels, else will run of threads + // during BFS + poolSize = poolSize == 0 ? poolSize : Math.max(poolSize, maxDepth+1); + final ExecutorService pool = poolSize > 0 ? Executors.newFixedThreadPool(poolSize, new ThreadFactoryBuilder().setDaemon(true).setNameFormat("MSCK-GetPaths-%d").build()) : null; if (pool == null) { diff --git a/ql/src/test/queries/clientpositive/msck_repair_batchsize.q b/ql/src/test/queries/clientpositive/msck_repair_batchsize.q index 06e4507..e56e97a 100644 --- a/ql/src/test/queries/clientpositive/msck_repair_batchsize.q +++ b/ql/src/test/queries/clientpositive/msck_repair_batchsize.q @@ -20,3 +20,13 @@ MSCK REPAIR TABLE default.repairtable; MSCK TABLE repairtable; DROP TABLE default.repairtable; + + +dfs ${system:test.dfs.mkdir} -p ${system:test.tmp.dir}/apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b; +CREATE TABLE `repairtable`( `col` string) PARTITIONED BY ( `p1` string, `p2` string) location '${system:test.tmp.dir}/apps/hive/warehouse/test.db/repairtable/'; + +dfs -touchz ${system:test.tmp.dir}/apps/hive/warehouse/test.db/repairtable/p1=c/p2=a/p3=b/datafile; +set hive.mv.files.thread=1; +MSCK TABLE repairtable; + +DROP TABLE default.repairtable; diff --git a/ql/src/test/results/clientpositive/msck_repair_batchsize.q.out b/ql/src/test/results/clientpositive/msck_repair_batchsize.q.out index a0180b7..ba99024 100644 --- a/ql/src/test/results/clientpositive/msck_repair_batchsize.q.out +++ b/ql/src/test/results/clientpositive/msck_repair_batchsize.q.out @@ -47,3 +47,28 @@ POSTHOOK: query: DROP TABLE default.repairtable POSTHOOK: type: DROPTABLE POSTHOOK: Input: default@repairtable POSTHOOK: Output: default@repairtable +#### A masked pattern was here #### +PREHOOK: type: CREATETABLE +#### A masked pattern was here #### +PREHOOK: Output: database:default +PREHOOK: Output: default@repairtable +#### A masked pattern was here #### +POSTHOOK: type: CREATETABLE +#### A masked pattern was here #### +POSTHOOK: Output: database:default +POSTHOOK: Output: default@repairtable +PREHOOK: query: MSCK TABLE repairtable +PREHOOK: type: MSCK +PREHOOK: Output: default@repairtable +POSTHOOK: query: MSCK TABLE repairtable +POSTHOOK: type: MSCK +POSTHOOK: Output: default@repairtable +Partitions not in metastore: repairtable:p1=c/p2=a +PREHOOK: query: DROP TABLE default.repairtable +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@repairtable +PREHOOK: Output: default@repairtable +POSTHOOK: query: DROP TABLE default.repairtable +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@repairtable +POSTHOOK: Output: default@repairtable -- 1.7.12.4 (Apple Git-37)