From 9b8eca65dcf68129470ead39362ce870ffb0bb1d Mon Sep 17 00:00:00 2001
From: Yuming Wang <wgyumg@gmail.com>
Date: Tue, 28 Feb 2017 10:13:42 +0000
Subject: [PATCH] [SPARK-19660][CORE][SQL] Replace the configuration property
 names that are deprecated in the version of Hadoop 2.6

## What changes were proposed in this pull request?

Replace all the Hadoop deprecated configuration property names according to [DeprecatedProperties](https://hadoop.apache.org/docs/r2.6.0/hadoop-project-dist/hadoop-common/DeprecatedProperties.html).

except:
https://github.com/apache/spark/blob/v2.1.0/python/pyspark/sql/tests.py#L1533
https://github.com/apache/spark/blob/v2.1.0/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala#L987
https://github.com/apache/spark/blob/v2.1.0/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala#L45
https://github.com/apache/spark/blob/v2.1.0/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala#L614

## How was this patch tested?

Existing tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #16990 from wangyum/HadoopDeprecatedProperties.
---
 R/WINDOWS.md                                  |  2 +-
 R/run-tests.sh                                |  2 +-
 appveyor.yml                                  |  2 +-
 .../io/HadoopMapReduceCommitProtocol.scala    | 10 ++--
 .../io/SparkHadoopMapReduceWriter.scala       |  2 +-
 .../org/apache/spark/rdd/HadoopRDD.scala      | 10 ++--
 .../apache/spark/rdd/PairRDDFunctions.scala   |  9 ++--
 .../scala/org/apache/spark/FileSuite.scala    |  5 +-
 docs/hardware-provisioning.md                 |  4 +-
 python/pyspark/tests.py                       | 47 ++++++++++---------
 .../spark/sql/execution/command/tables.scala  |  4 +-
 .../datasources/FileFormatWriter.scala        | 10 ++--
 .../HiveWindowFunctionQuerySuite.scala        |  8 ++--
 .../apache/spark/sql/hive/TableReader.scala   |  8 ++--
 .../hive/execution/InsertIntoHiveTable.scala  | 15 +++---
 .../apache/spark/sql/hive/test/TestHive.scala |  2 +-
 ...adoop20-2-2b9ccaa793eae0e73bf76335d3d6880} |  0
 ...mbine1-2-6142f47d3fcdd4323162014d5eb35e07} |  0
 ...mbine1-3-10266e3d5dd4c841c0d65030b1edba7c} |  0
 ...ombine1-4-9cbd6d400fb6c3cd09010e3dbd76601} |  0
 ...mbine1-5-1ba2d6f3bb3348da3fee7fab4f283f34} |  0
 ...mbine2-2-6142f47d3fcdd4323162014d5eb35e07} |  0
 ...mbine2-3-10266e3d5dd4c841c0d65030b1edba7c} |  0
 ...ombine2-4-9cbd6d400fb6c3cd09010e3dbd76601} |  0
 ...mbine2-5-1ba2d6f3bb3348da3fee7fab4f283f34} |  0
 ...oupby1-3-c8478dac3497697b4375ee35118a5c3e} |  0
 ...oupby1-5-c9cee6382b64bd3d71177527961b8be2} |  0
 ..._limit-0-be2c0b32a02a1154bfdee1a52530f387} |  0
 ...y1_map-2-be2c0b32a02a1154bfdee1a52530f387} |  0
 ...p_skew-2-be2c0b32a02a1154bfdee1a52530f387} |  0
 ...noskew-2-be2c0b32a02a1154bfdee1a52530f387} |  0
 ..._limit-0-be2c0b32a02a1154bfdee1a52530f387} |  0
 ...y2_map-2-be2c0b32a02a1154bfdee1a52530f387} |  0
 ...p_skew-2-be2c0b32a02a1154bfdee1a52530f387} |  0
 ...noskew-2-be2c0b32a02a1154bfdee1a52530f387} |  0
 ...y4_map-2-be2c0b32a02a1154bfdee1a52530f387} |  0
 ...p_skew-2-be2c0b32a02a1154bfdee1a52530f387} |  0
 ...noskew-2-be2c0b32a02a1154bfdee1a52530f387} |  0
 ...y5_map-2-be2c0b32a02a1154bfdee1a52530f387} |  0
 ...p_skew-2-be2c0b32a02a1154bfdee1a52530f387} |  0
 ...noskew-2-be2c0b32a02a1154bfdee1a52530f387} |  0
 ...y6_map-2-be2c0b32a02a1154bfdee1a52530f387} |  0
 ...p_skew-2-be2c0b32a02a1154bfdee1a52530f387} |  0
 ...noskew-2-be2c0b32a02a1154bfdee1a52530f387} |  0
 ...y7_map-3-be2c0b32a02a1154bfdee1a52530f387} |  0
 ...educer-2-be2c0b32a02a1154bfdee1a52530f387} |  0
 ...p_skew-2-be2c0b32a02a1154bfdee1a52530f387} |  0
 ...noskew-3-be2c0b32a02a1154bfdee1a52530f387} |  0
 ...educer-2-be2c0b32a02a1154bfdee1a52530f387} |  0
 ...y8_map-2-be2c0b32a02a1154bfdee1a52530f387} |  0
 ...p_skew-2-be2c0b32a02a1154bfdee1a52530f387} |  0
 ...noskew-2-be2c0b32a02a1154bfdee1a52530f387} |  0
 ...ap_ppr-2-be2c0b32a02a1154bfdee1a52530f387} |  0
 ...adoop20-0-2b9ccaa793eae0e73bf76335d3d6880} |  0
 ...cefile-0-dd959af1968381d0ed90178d349b01a7} |  0
 ...cefile-1-ddbb8d5e5dc0988bda96ac2b4aec8f94} |  0
 ...cefile-5-25715870c569b0f8c3d483e3a38b3199} |  0
 ...adoop20-1-2b9ccaa793eae0e73bf76335d3d6880} |  0
 ...oin_mr-7-6b9861b999092f1ea4fa1fd27a666af6} |  0
 ...merge2-2-6142f47d3fcdd4323162014d5eb35e07} |  0
 ...merge2-3-10266e3d5dd4c841c0d65030b1edba7c} |  0
 ... merge2-4-9cbd6d400fb6c3cd09010e3dbd76601} |  0
 ...merge2-5-1ba2d6f3bb3348da3fee7fab4f283f34} |  0
 ...rallel-0-6dc30e2de057022e63bd2a645fbec4c2} |  0
 ...press-11-25715870c569b0f8c3d483e3a38b3199} |  0
 ...mpress-5-dd959af1968381d0ed90178d349b01a7} |  0
 .../clientpositive/auto_join14_hadoop20.q     |  2 +-
 .../src/test/queries/clientpositive/bucket5.q |  2 +-
 .../clientpositive/bucket_num_reducers.q      |  2 +-
 .../bucketizedhiveinputformat.q               |  2 +-
 .../test/queries/clientpositive/combine1.q    | 10 ++--
 .../test/queries/clientpositive/combine2.q    | 10 ++--
 .../clientpositive/combine2_hadoop20.q        | 10 ++--
 .../queries/clientpositive/combine2_win.q     |  8 ++--
 .../test/queries/clientpositive/combine3.q    |  8 ++--
 .../test/queries/clientpositive/create_1.q    |  2 +-
 .../queries/clientpositive/ctas_hadoop20.q    |  2 +-
 .../test/queries/clientpositive/groupby1.q    |  4 +-
 .../queries/clientpositive/groupby1_limit.q   |  2 +-
 .../queries/clientpositive/groupby1_map.q     |  2 +-
 .../clientpositive/groupby1_map_skew.q        |  2 +-
 .../queries/clientpositive/groupby1_noskew.q  |  2 +-
 .../queries/clientpositive/groupby2_limit.q   |  2 +-
 .../queries/clientpositive/groupby2_map.q     |  2 +-
 .../groupby2_map_multi_distinct.q             |  2 +-
 .../clientpositive/groupby2_map_skew.q        |  2 +-
 .../queries/clientpositive/groupby2_noskew.q  |  2 +-
 .../groupby2_noskew_multi_distinct.q          |  2 +-
 .../queries/clientpositive/groupby3_map.q     |  2 +-
 .../groupby3_map_multi_distinct.q             |  2 +-
 .../clientpositive/groupby3_map_skew.q        |  2 +-
 .../queries/clientpositive/groupby3_noskew.q  |  2 +-
 .../groupby3_noskew_multi_distinct.q          |  2 +-
 .../queries/clientpositive/groupby4_map.q     |  2 +-
 .../clientpositive/groupby4_map_skew.q        |  2 +-
 .../queries/clientpositive/groupby4_noskew.q  |  2 +-
 .../queries/clientpositive/groupby5_map.q     |  2 +-
 .../clientpositive/groupby5_map_skew.q        |  2 +-
 .../queries/clientpositive/groupby5_noskew.q  |  2 +-
 .../queries/clientpositive/groupby6_map.q     |  2 +-
 .../clientpositive/groupby6_map_skew.q        |  2 +-
 .../queries/clientpositive/groupby6_noskew.q  |  2 +-
 .../queries/clientpositive/groupby7_map.q     |  2 +-
 .../groupby7_map_multi_single_reducer.q       |  2 +-
 .../clientpositive/groupby7_map_skew.q        |  2 +-
 .../queries/clientpositive/groupby7_noskew.q  |  2 +-
 .../groupby7_noskew_multi_single_reducer.q    |  2 +-
 .../queries/clientpositive/groupby8_map.q     |  2 +-
 .../clientpositive/groupby8_map_skew.q        |  2 +-
 .../queries/clientpositive/groupby8_noskew.q  |  2 +-
 .../queries/clientpositive/groupby_map_ppr.q  |  2 +-
 .../groupby_map_ppr_multi_distinct.q          |  2 +-
 .../queries/clientpositive/groupby_sort_1.q   |  2 +-
 .../clientpositive/groupby_sort_skew_1.q      |  2 +-
 .../queries/clientpositive/hook_context_cs.q  |  2 +-
 .../infer_bucket_sort_dyn_part.q              |  2 +-
 .../clientpositive/infer_bucket_sort_merge.q  |  2 +-
 .../infer_bucket_sort_num_buckets.q           |  2 +-
 .../queries/clientpositive/input12_hadoop20.q |  2 +-
 .../queries/clientpositive/input39_hadoop20.q |  4 +-
 .../clientpositive/input_testsequencefile.q   |  6 +--
 .../queries/clientpositive/join14_hadoop20.q  |  2 +-
 .../queries/clientpositive/leftsemijoin_mr.q  |  2 +-
 .../src/test/queries/clientpositive/merge2.q  |  8 ++--
 .../queries/clientpositive/orc_createas1.q    |  4 +-
 .../queries/clientpositive/orc_ppd_char.q     |  4 +-
 .../queries/clientpositive/orc_ppd_date.q     |  4 +-
 .../queries/clientpositive/orc_ppd_decimal.q  |  4 +-
 .../queries/clientpositive/orc_ppd_varchar.q  |  4 +-
 .../clientpositive/orc_split_elimination.q    |  4 +-
 .../test/queries/clientpositive/parallel.q    |  2 +-
 .../queries/clientpositive/parallel_orderby.q |  2 +-
 .../queries/clientpositive/rcfile_createas1.q |  4 +-
 .../clientpositive/rcfile_lazydecompress.q    |  4 +-
 .../queries/clientpositive/rcfile_merge1.q    |  2 +-
 .../queries/clientpositive/rcfile_merge2.q    |  4 +-
 .../queries/clientpositive/rcfile_merge3.q    |  4 +-
 .../queries/clientpositive/rcfile_merge4.q    |  4 +-
 .../clientpositive/sample_islocalmode_hook.q  |  8 ++--
 .../sample_islocalmode_hook_hadoop20.q        | 12 ++---
 .../queries/clientpositive/split_sample.q     | 18 +++----
 .../queries/clientpositive/stats_partscan_1.q | 10 ++--
 .../clientpositive/stats_partscan_1_23.q      | 10 ++--
 .../clientpositive/udaf_context_ngrams.q      |  2 +-
 .../test/queries/clientpositive/udaf_ngrams.q |  2 +-
 145 files changed, 213 insertions(+), 205 deletions(-)
 rename sql/hive/src/test/resources/golden/{auto_join14_hadoop20-2-db1cd54a4cb36de2087605f32e41824f => auto_join14_hadoop20-2-2b9ccaa793eae0e73bf76335d3d6880} (100%)
 rename sql/hive/src/test/resources/golden/{combine1-2-c95dc367df88c9e5cf77157f29ba2daf => combine1-2-6142f47d3fcdd4323162014d5eb35e07} (100%)
 rename sql/hive/src/test/resources/golden/{combine1-3-6e53a3ac93113f20db3a12f1dcf30e86 => combine1-3-10266e3d5dd4c841c0d65030b1edba7c} (100%)
 rename sql/hive/src/test/resources/golden/{combine1-4-84967075baa3e56fff2a23f8ab9ba076 => combine1-4-9cbd6d400fb6c3cd09010e3dbd76601} (100%)
 rename sql/hive/src/test/resources/golden/{combine1-5-2ee5d706fe3a3bcc38b795f6e94970ea => combine1-5-1ba2d6f3bb3348da3fee7fab4f283f34} (100%)
 rename sql/hive/src/test/resources/golden/{combine2-2-c95dc367df88c9e5cf77157f29ba2daf => combine2-2-6142f47d3fcdd4323162014d5eb35e07} (100%)
 rename sql/hive/src/test/resources/golden/{combine2-3-6e53a3ac93113f20db3a12f1dcf30e86 => combine2-3-10266e3d5dd4c841c0d65030b1edba7c} (100%)
 rename sql/hive/src/test/resources/golden/{combine2-4-84967075baa3e56fff2a23f8ab9ba076 => combine2-4-9cbd6d400fb6c3cd09010e3dbd76601} (100%)
 rename sql/hive/src/test/resources/golden/{combine2-5-2ee5d706fe3a3bcc38b795f6e94970ea => combine2-5-1ba2d6f3bb3348da3fee7fab4f283f34} (100%)
 rename sql/hive/src/test/resources/golden/{groupby1-3-d57ed4bbfee1ffaffaeba0a4be84c31d => groupby1-3-c8478dac3497697b4375ee35118a5c3e} (100%)
 rename sql/hive/src/test/resources/golden/{groupby1-5-dd7bf298b8c921355edd8665c6b0c168 => groupby1-5-c9cee6382b64bd3d71177527961b8be2} (100%)
 rename sql/hive/src/test/resources/golden/{groupby1_limit-0-83c59d378571a6e487aa20217bd87817 => groupby1_limit-0-be2c0b32a02a1154bfdee1a52530f387} (100%)
 rename sql/hive/src/test/resources/golden/{groupby1_map-2-83c59d378571a6e487aa20217bd87817 => groupby1_map-2-be2c0b32a02a1154bfdee1a52530f387} (100%)
 rename sql/hive/src/test/resources/golden/{groupby1_map_skew-2-83c59d378571a6e487aa20217bd87817 => groupby1_map_skew-2-be2c0b32a02a1154bfdee1a52530f387} (100%)
 rename sql/hive/src/test/resources/golden/{groupby1_noskew-2-83c59d378571a6e487aa20217bd87817 => groupby1_noskew-2-be2c0b32a02a1154bfdee1a52530f387} (100%)
 rename sql/hive/src/test/resources/golden/{groupby2_limit-0-83c59d378571a6e487aa20217bd87817 => groupby2_limit-0-be2c0b32a02a1154bfdee1a52530f387} (100%)
 rename sql/hive/src/test/resources/golden/{groupby2_map-2-83c59d378571a6e487aa20217bd87817 => groupby2_map-2-be2c0b32a02a1154bfdee1a52530f387} (100%)
 rename sql/hive/src/test/resources/golden/{groupby2_map_skew-2-83c59d378571a6e487aa20217bd87817 => groupby2_map_skew-2-be2c0b32a02a1154bfdee1a52530f387} (100%)
 rename sql/hive/src/test/resources/golden/{groupby2_noskew-2-83c59d378571a6e487aa20217bd87817 => groupby2_noskew-2-be2c0b32a02a1154bfdee1a52530f387} (100%)
 rename sql/hive/src/test/resources/golden/{groupby4_map-2-83c59d378571a6e487aa20217bd87817 => groupby4_map-2-be2c0b32a02a1154bfdee1a52530f387} (100%)
 rename sql/hive/src/test/resources/golden/{groupby4_map_skew-2-83c59d378571a6e487aa20217bd87817 => groupby4_map_skew-2-be2c0b32a02a1154bfdee1a52530f387} (100%)
 rename sql/hive/src/test/resources/golden/{groupby4_noskew-2-83c59d378571a6e487aa20217bd87817 => groupby4_noskew-2-be2c0b32a02a1154bfdee1a52530f387} (100%)
 rename sql/hive/src/test/resources/golden/{groupby5_map-2-83c59d378571a6e487aa20217bd87817 => groupby5_map-2-be2c0b32a02a1154bfdee1a52530f387} (100%)
 rename sql/hive/src/test/resources/golden/{groupby5_map_skew-2-83c59d378571a6e487aa20217bd87817 => groupby5_map_skew-2-be2c0b32a02a1154bfdee1a52530f387} (100%)
 rename sql/hive/src/test/resources/golden/{groupby5_noskew-2-83c59d378571a6e487aa20217bd87817 => groupby5_noskew-2-be2c0b32a02a1154bfdee1a52530f387} (100%)
 rename sql/hive/src/test/resources/golden/{groupby6_map-2-83c59d378571a6e487aa20217bd87817 => groupby6_map-2-be2c0b32a02a1154bfdee1a52530f387} (100%)
 rename sql/hive/src/test/resources/golden/{groupby6_map_skew-2-83c59d378571a6e487aa20217bd87817 => groupby6_map_skew-2-be2c0b32a02a1154bfdee1a52530f387} (100%)
 rename sql/hive/src/test/resources/golden/{groupby6_noskew-2-83c59d378571a6e487aa20217bd87817 => groupby6_noskew-2-be2c0b32a02a1154bfdee1a52530f387} (100%)
 rename sql/hive/src/test/resources/golden/{groupby7_map-3-83c59d378571a6e487aa20217bd87817 => groupby7_map-3-be2c0b32a02a1154bfdee1a52530f387} (100%)
 rename sql/hive/src/test/resources/golden/{groupby7_map_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 => groupby7_map_multi_single_reducer-2-be2c0b32a02a1154bfdee1a52530f387} (100%)
 rename sql/hive/src/test/resources/golden/{groupby7_map_skew-2-83c59d378571a6e487aa20217bd87817 => groupby7_map_skew-2-be2c0b32a02a1154bfdee1a52530f387} (100%)
 rename sql/hive/src/test/resources/golden/{groupby7_noskew-3-83c59d378571a6e487aa20217bd87817 => groupby7_noskew-3-be2c0b32a02a1154bfdee1a52530f387} (100%)
 rename sql/hive/src/test/resources/golden/{groupby7_noskew_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 => groupby7_noskew_multi_single_reducer-2-be2c0b32a02a1154bfdee1a52530f387} (100%)
 rename sql/hive/src/test/resources/golden/{groupby8_map-2-83c59d378571a6e487aa20217bd87817 => groupby8_map-2-be2c0b32a02a1154bfdee1a52530f387} (100%)
 rename sql/hive/src/test/resources/golden/{groupby8_map_skew-2-83c59d378571a6e487aa20217bd87817 => groupby8_map_skew-2-be2c0b32a02a1154bfdee1a52530f387} (100%)
 rename sql/hive/src/test/resources/golden/{groupby8_noskew-2-83c59d378571a6e487aa20217bd87817 => groupby8_noskew-2-be2c0b32a02a1154bfdee1a52530f387} (100%)
 rename sql/hive/src/test/resources/golden/{groupby_map_ppr-2-83c59d378571a6e487aa20217bd87817 => groupby_map_ppr-2-be2c0b32a02a1154bfdee1a52530f387} (100%)
 rename sql/hive/src/test/resources/golden/{input12_hadoop20-0-db1cd54a4cb36de2087605f32e41824f => input12_hadoop20-0-2b9ccaa793eae0e73bf76335d3d6880} (100%)
 rename sql/hive/src/test/resources/golden/{input_testsequencefile-0-68975193b30cb34102b380e647d8d5f4 => input_testsequencefile-0-dd959af1968381d0ed90178d349b01a7} (100%)
 rename sql/hive/src/test/resources/golden/{input_testsequencefile-1-1c0f3be2d837dee49312e0a80440447e => input_testsequencefile-1-ddbb8d5e5dc0988bda96ac2b4aec8f94} (100%)
 rename sql/hive/src/test/resources/golden/{input_testsequencefile-5-3708198aac609695b22e19e89306034c => input_testsequencefile-5-25715870c569b0f8c3d483e3a38b3199} (100%)
 rename sql/hive/src/test/resources/golden/{join14_hadoop20-1-db1cd54a4cb36de2087605f32e41824f => join14_hadoop20-1-2b9ccaa793eae0e73bf76335d3d6880} (100%)
 rename sql/hive/src/test/resources/golden/{leftsemijoin_mr-7-8e9c2969b999557363e40f9ebb3f6d7c => leftsemijoin_mr-7-6b9861b999092f1ea4fa1fd27a666af6} (100%)
 rename sql/hive/src/test/resources/golden/{merge2-2-c95dc367df88c9e5cf77157f29ba2daf => merge2-2-6142f47d3fcdd4323162014d5eb35e07} (100%)
 rename sql/hive/src/test/resources/golden/{merge2-3-6e53a3ac93113f20db3a12f1dcf30e86 => merge2-3-10266e3d5dd4c841c0d65030b1edba7c} (100%)
 rename sql/hive/src/test/resources/golden/{merge2-4-84967075baa3e56fff2a23f8ab9ba076 => merge2-4-9cbd6d400fb6c3cd09010e3dbd76601} (100%)
 rename sql/hive/src/test/resources/golden/{merge2-5-2ee5d706fe3a3bcc38b795f6e94970ea => merge2-5-1ba2d6f3bb3348da3fee7fab4f283f34} (100%)
 rename sql/hive/src/test/resources/golden/{parallel-0-23a4feaede17467a8cc26e4d86ec30f9 => parallel-0-6dc30e2de057022e63bd2a645fbec4c2} (100%)
 rename sql/hive/src/test/resources/golden/{rcfile_lazydecompress-11-3708198aac609695b22e19e89306034c => rcfile_lazydecompress-11-25715870c569b0f8c3d483e3a38b3199} (100%)
 rename sql/hive/src/test/resources/golden/{rcfile_lazydecompress-5-68975193b30cb34102b380e647d8d5f4 => rcfile_lazydecompress-5-dd959af1968381d0ed90178d349b01a7} (100%)

diff --git a/R/WINDOWS.md b/R/WINDOWS.md
index 1afcbfcabe..cb2eebb9ff 100644
--- a/R/WINDOWS.md
+++ b/R/WINDOWS.md
@@ -38,6 +38,6 @@ To run the SparkR unit tests on Windows, the following steps are required —ass
 
     ```
     R -e "install.packages('testthat', repos='http://cran.us.r-project.org')"
-    .\bin\spark-submit2.cmd --conf spark.hadoop.fs.default.name="file:///" R\pkg\tests\run-all.R
+    .\bin\spark-submit2.cmd --conf spark.hadoop.fs.defaultFS="file:///" R\pkg\tests\run-all.R
     ```
 
diff --git a/R/run-tests.sh b/R/run-tests.sh
index 5e4dafaf76..742a2c5ed7 100755
--- a/R/run-tests.sh
+++ b/R/run-tests.sh
@@ -23,7 +23,7 @@ FAILED=0
 LOGFILE=$FWDIR/unit-tests.out
 rm -f $LOGFILE
 
-SPARK_TESTING=1 $FWDIR/../bin/spark-submit --driver-java-options "-Dlog4j.configuration=file:$FWDIR/log4j.properties" --conf spark.hadoop.fs.default.name="file:///" $FWDIR/pkg/tests/run-all.R 2>&1 | tee -a $LOGFILE
+SPARK_TESTING=1 $FWDIR/../bin/spark-submit --driver-java-options "-Dlog4j.configuration=file:$FWDIR/log4j.properties" --conf spark.hadoop.fs.defaultFS="file:///" $FWDIR/pkg/tests/run-all.R 2>&1 | tee -a $LOGFILE
 FAILED=$((PIPESTATUS[0]||$FAILED))
 
 NUM_TEST_WARNING="$(grep -c -e 'Warnings ----------------' $LOGFILE)"
diff --git a/appveyor.yml b/appveyor.yml
index 6bc66c0ea5..5adf1b4bed 100644
--- a/appveyor.yml
+++ b/appveyor.yml
@@ -46,7 +46,7 @@ build_script:
   - cmd: mvn -DskipTests -Psparkr -Phive -Phive-thriftserver package
 
 test_script:
-  - cmd: .\bin\spark-submit2.cmd --conf spark.hadoop.fs.default.name="file:///" R\pkg\tests\run-all.R
+  - cmd: .\bin\spark-submit2.cmd --conf spark.hadoop.fs.defaultFS="file:///" R\pkg\tests\run-all.R
 
 notifications:
   - provider: Email
diff --git a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala
index 2c1b563688..22e2679913 100644
--- a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala
+++ b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala
@@ -113,11 +113,11 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String)
     val taskAttemptId = new TaskAttemptID(taskId, 0)
 
     // Set up the configuration object
-    jobContext.getConfiguration.set("mapred.job.id", jobId.toString)
-    jobContext.getConfiguration.set("mapred.tip.id", taskAttemptId.getTaskID.toString)
-    jobContext.getConfiguration.set("mapred.task.id", taskAttemptId.toString)
-    jobContext.getConfiguration.setBoolean("mapred.task.is.map", true)
-    jobContext.getConfiguration.setInt("mapred.task.partition", 0)
+    jobContext.getConfiguration.set("mapreduce.job.id", jobId.toString)
+    jobContext.getConfiguration.set("mapreduce.task.id", taskAttemptId.getTaskID.toString)
+    jobContext.getConfiguration.set("mapreduce.task.attempt.id", taskAttemptId.toString)
+    jobContext.getConfiguration.setBoolean("mapreduce.task.ismap", true)
+    jobContext.getConfiguration.setInt("mapreduce.task.partition", 0)
 
     val taskAttemptContext = new TaskAttemptContextImpl(jobContext.getConfiguration, taskAttemptId)
     committer = setupCommitter(taskAttemptContext)
diff --git a/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopMapReduceWriter.scala b/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopMapReduceWriter.scala
index 1e0a1e605c..659ad5d0ba 100644
--- a/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopMapReduceWriter.scala
+++ b/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopMapReduceWriter.scala
@@ -79,7 +79,7 @@ object SparkHadoopMapReduceWriter extends Logging {
     val committer = FileCommitProtocol.instantiate(
       className = classOf[HadoopMapReduceCommitProtocol].getName,
       jobId = stageId.toString,
-      outputPath = conf.value.get("mapred.output.dir"),
+      outputPath = conf.value.get("mapreduce.output.fileoutputformat.outputdir"),
       isAppend = false).asInstanceOf[HadoopMapReduceCommitProtocol]
     committer.setupJob(jobContext)
 
diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
index 5fa6a7ed31..4bf8ecc383 100644
--- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
@@ -365,11 +365,11 @@ private[spark] object HadoopRDD extends Logging {
     val jobID = new JobID(jobTrackerId, jobId)
     val taId = new TaskAttemptID(new TaskID(jobID, TaskType.MAP, splitId), attemptId)
 
-    conf.set("mapred.tip.id", taId.getTaskID.toString)
-    conf.set("mapred.task.id", taId.toString)
-    conf.setBoolean("mapred.task.is.map", true)
-    conf.setInt("mapred.task.partition", splitId)
-    conf.set("mapred.job.id", jobID.toString)
+    conf.set("mapreduce.task.id", taId.getTaskID.toString)
+    conf.set("mapreduce.task.attempt.id", taId.toString)
+    conf.setBoolean("mapreduce.task.ismap", true)
+    conf.setInt("mapreduce.task.partition", splitId)
+    conf.set("mapreduce.job.id", jobID.toString)
   }
 
   /**
diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
index 567a3183e2..52ce03ff8c 100644
--- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
@@ -998,7 +998,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
     job.setOutputValueClass(valueClass)
     job.setOutputFormatClass(outputFormatClass)
     val jobConfiguration = job.getConfiguration
-    jobConfiguration.set("mapred.output.dir", path)
+    jobConfiguration.set("mapreduce.output.fileoutputformat.outputdir", path)
     saveAsNewAPIHadoopDataset(jobConfiguration)
   }
 
@@ -1039,10 +1039,11 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
     conf.setOutputFormat(outputFormatClass)
     for (c <- codec) {
       hadoopConf.setCompressMapOutput(true)
-      hadoopConf.set("mapred.output.compress", "true")
+      hadoopConf.set("mapreduce.output.fileoutputformat.compress", "true")
       hadoopConf.setMapOutputCompressorClass(c)
-      hadoopConf.set("mapred.output.compression.codec", c.getCanonicalName)
-      hadoopConf.set("mapred.output.compression.type", CompressionType.BLOCK.toString)
+      hadoopConf.set("mapreduce.output.fileoutputformat.compress.codec", c.getCanonicalName)
+      hadoopConf.set("mapreduce.output.fileoutputformat.compress.type",
+        CompressionType.BLOCK.toString)
     }
 
     // Use configured output committer if already set
diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala
index a2d3177c5c..5be0121db5 100644
--- a/core/src/test/scala/org/apache/spark/FileSuite.scala
+++ b/core/src/test/scala/org/apache/spark/FileSuite.scala
@@ -401,7 +401,7 @@ class FileSuite extends SparkFunSuite with LocalSparkContext {
     job.setOutputKeyClass(classOf[String])
     job.setOutputValueClass(classOf[String])
     job.set("mapred.output.format.class", classOf[TextOutputFormat[String, String]].getName)
-    job.set("mapred.output.dir", tempDir.getPath + "/outputDataset_old")
+    job.set("mapreduce.output.fileoutputformat.outputdir", tempDir.getPath + "/outputDataset_old")
     randomRDD.saveAsHadoopDataset(job)
     assert(new File(tempDir.getPath + "/outputDataset_old/part-00000").exists() === true)
   }
@@ -415,7 +415,8 @@ class FileSuite extends SparkFunSuite with LocalSparkContext {
     job.setOutputValueClass(classOf[String])
     job.setOutputFormatClass(classOf[NewTextOutputFormat[String, String]])
     val jobConfig = job.getConfiguration
-    jobConfig.set("mapred.output.dir", tempDir.getPath + "/outputDataset_new")
+    jobConfig.set("mapreduce.output.fileoutputformat.outputdir",
+      tempDir.getPath + "/outputDataset_new")
     randomRDD.saveAsNewAPIHadoopDataset(jobConfig)
     assert(new File(tempDir.getPath + "/outputDataset_new/part-r-00000").exists() === true)
   }
diff --git a/docs/hardware-provisioning.md b/docs/hardware-provisioning.md
index bb6f616b18..896f9302ef 100644
--- a/docs/hardware-provisioning.md
+++ b/docs/hardware-provisioning.md
@@ -15,8 +15,8 @@ possible**. We recommend the following:
 * If at all possible, run Spark on the same nodes as HDFS. The simplest way is to set up a Spark
 [standalone mode cluster](spark-standalone.html) on the same nodes, and configure Spark and
 Hadoop's memory and CPU usage to avoid interference (for Hadoop, the relevant options are
-`mapred.child.java.opts` for the per-task memory and `mapred.tasktracker.map.tasks.maximum`
-and `mapred.tasktracker.reduce.tasks.maximum` for number of tasks). Alternatively, you can run
+`mapred.child.java.opts` for the per-task memory and `mapreduce.tasktracker.map.tasks.maximum`
+and `mapreduce.tasktracker.reduce.tasks.maximum` for number of tasks). Alternatively, you can run
 Hadoop and Spark on a common cluster manager like [Mesos](running-on-mesos.html) or
 [Hadoop YARN](running-on-yarn.html).
 
diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py
index e908b1e739..a2aead7e6b 100644
--- a/python/pyspark/tests.py
+++ b/python/pyspark/tests.py
@@ -1347,7 +1347,7 @@ class InputFormatTests(ReusedPySparkTestCase):
         self.assertEqual(ints, ei)
 
         hellopath = os.path.join(SPARK_HOME, "python/test_support/hello/hello.txt")
-        oldconf = {"mapred.input.dir": hellopath}
+        oldconf = {"mapreduce.input.fileinputformat.inputdir": hellopath}
         hello = self.sc.hadoopRDD("org.apache.hadoop.mapred.TextInputFormat",
                                   "org.apache.hadoop.io.LongWritable",
                                   "org.apache.hadoop.io.Text",
@@ -1366,7 +1366,7 @@ class InputFormatTests(ReusedPySparkTestCase):
         self.assertEqual(ints, ei)
 
         hellopath = os.path.join(SPARK_HOME, "python/test_support/hello/hello.txt")
-        newconf = {"mapred.input.dir": hellopath}
+        newconf = {"mapreduce.input.fileinputformat.inputdir": hellopath}
         hello = self.sc.newAPIHadoopRDD("org.apache.hadoop.mapreduce.lib.input.TextInputFormat",
                                         "org.apache.hadoop.io.LongWritable",
                                         "org.apache.hadoop.io.Text",
@@ -1515,12 +1515,12 @@ class OutputFormatTests(ReusedPySparkTestCase):
 
         conf = {
             "mapred.output.format.class": "org.apache.hadoop.mapred.SequenceFileOutputFormat",
-            "mapred.output.key.class": "org.apache.hadoop.io.IntWritable",
-            "mapred.output.value.class": "org.apache.hadoop.io.MapWritable",
-            "mapred.output.dir": basepath + "/olddataset/"
+            "mapreduce.job.output.key.class": "org.apache.hadoop.io.IntWritable",
+            "mapreduce.job.output.value.class": "org.apache.hadoop.io.MapWritable",
+            "mapreduce.output.fileoutputformat.outputdir": basepath + "/olddataset/"
         }
         self.sc.parallelize(dict_data).saveAsHadoopDataset(conf)
-        input_conf = {"mapred.input.dir": basepath + "/olddataset/"}
+        input_conf = {"mapreduce.input.fileinputformat.inputdir": basepath + "/olddataset/"}
         result = self.sc.hadoopRDD(
             "org.apache.hadoop.mapred.SequenceFileInputFormat",
             "org.apache.hadoop.io.IntWritable",
@@ -1547,14 +1547,14 @@ class OutputFormatTests(ReusedPySparkTestCase):
         self.assertEqual(result, data)
 
         conf = {
-            "mapreduce.outputformat.class":
+            "mapreduce.job.outputformat.class":
                 "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat",
-            "mapred.output.key.class": "org.apache.hadoop.io.IntWritable",
-            "mapred.output.value.class": "org.apache.hadoop.io.Text",
-            "mapred.output.dir": basepath + "/newdataset/"
+            "mapreduce.job.output.key.class": "org.apache.hadoop.io.IntWritable",
+            "mapreduce.job.output.value.class": "org.apache.hadoop.io.Text",
+            "mapreduce.output.fileoutputformat.outputdir": basepath + "/newdataset/"
         }
         self.sc.parallelize(data).saveAsNewAPIHadoopDataset(conf)
-        input_conf = {"mapred.input.dir": basepath + "/newdataset/"}
+        input_conf = {"mapreduce.input.fileinputformat.inputdir": basepath + "/newdataset/"}
         new_dataset = sorted(self.sc.newAPIHadoopRDD(
             "org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat",
             "org.apache.hadoop.io.IntWritable",
@@ -1584,16 +1584,16 @@ class OutputFormatTests(ReusedPySparkTestCase):
         self.assertEqual(result, array_data)
 
         conf = {
-            "mapreduce.outputformat.class":
+            "mapreduce.job.outputformat.class":
                 "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat",
-            "mapred.output.key.class": "org.apache.hadoop.io.IntWritable",
-            "mapred.output.value.class": "org.apache.spark.api.python.DoubleArrayWritable",
-            "mapred.output.dir": basepath + "/newdataset/"
+            "mapreduce.job.output.key.class": "org.apache.hadoop.io.IntWritable",
+            "mapreduce.job.output.value.class": "org.apache.spark.api.python.DoubleArrayWritable",
+            "mapreduce.output.fileoutputformat.outputdir": basepath + "/newdataset/"
         }
         self.sc.parallelize(array_data).saveAsNewAPIHadoopDataset(
             conf,
             valueConverter="org.apache.spark.api.python.DoubleArrayToWritableConverter")
-        input_conf = {"mapred.input.dir": basepath + "/newdataset/"}
+        input_conf = {"mapreduce.input.fileinputformat.inputdir": basepath + "/newdataset/"}
         new_dataset = sorted(self.sc.newAPIHadoopRDD(
             "org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat",
             "org.apache.hadoop.io.IntWritable",
@@ -1663,18 +1663,19 @@ class OutputFormatTests(ReusedPySparkTestCase):
 
         conf4 = {
             "mapred.output.format.class": "org.apache.hadoop.mapred.SequenceFileOutputFormat",
-            "mapred.output.key.class": "org.apache.hadoop.io.IntWritable",
-            "mapred.output.value.class": "org.apache.hadoop.io.IntWritable",
-            "mapred.output.dir": basepath + "/reserialize/dataset"}
+            "mapreduce.job.output.key.class": "org.apache.hadoop.io.IntWritable",
+            "mapreduce.job.output.value.class": "org.apache.hadoop.io.IntWritable",
+            "mapreduce.output.fileoutputformat.outputdir": basepath + "/reserialize/dataset"}
         rdd.saveAsHadoopDataset(conf4)
         result4 = sorted(self.sc.sequenceFile(basepath + "/reserialize/dataset").collect())
         self.assertEqual(result4, data)
 
-        conf5 = {"mapreduce.outputformat.class":
+        conf5 = {"mapreduce.job.outputformat.class":
                  "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat",
-                 "mapred.output.key.class": "org.apache.hadoop.io.IntWritable",
-                 "mapred.output.value.class": "org.apache.hadoop.io.IntWritable",
-                 "mapred.output.dir": basepath + "/reserialize/newdataset"}
+                 "mapreduce.job.output.key.class": "org.apache.hadoop.io.IntWritable",
+                 "mapreduce.job.output.value.class": "org.apache.hadoop.io.IntWritable",
+                 "mapreduce.output.fileoutputformat.outputdir": basepath + "/reserialize/newdataset"
+                 }
         rdd.saveAsNewAPIHadoopDataset(conf5)
         result5 = sorted(self.sc.sequenceFile(basepath + "/reserialize/newdataset").collect())
         self.assertEqual(result5, data)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
index 49407b44d7..3e80916104 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
@@ -269,8 +269,8 @@ case class LoadDataCommand(
         } else {
           // Follow Hive's behavior:
           // If no schema or authority is provided with non-local inpath,
-          // we will use hadoop configuration "fs.default.name".
-          val defaultFSConf = sparkSession.sessionState.newHadoopConf().get("fs.default.name")
+          // we will use hadoop configuration "fs.defaultFS".
+          val defaultFSConf = sparkSession.sessionState.newHadoopConf().get("fs.defaultFS")
           val defaultFS = if (defaultFSConf == null) {
             new URI("")
           } else {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala
index 644358493e..c17796811c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala
@@ -210,11 +210,11 @@ object FileFormatWriter extends Logging {
     val taskAttemptContext: TaskAttemptContext = {
       // Set up the configuration object
       val hadoopConf = description.serializableHadoopConf.value
-      hadoopConf.set("mapred.job.id", jobId.toString)
-      hadoopConf.set("mapred.tip.id", taskAttemptId.getTaskID.toString)
-      hadoopConf.set("mapred.task.id", taskAttemptId.toString)
-      hadoopConf.setBoolean("mapred.task.is.map", true)
-      hadoopConf.setInt("mapred.task.partition", 0)
+      hadoopConf.set("mapreduce.job.id", jobId.toString)
+      hadoopConf.set("mapreduce.task.id", taskAttemptId.getTaskID.toString)
+      hadoopConf.set("mapreduce.task.attempt.id", taskAttemptId.toString)
+      hadoopConf.setBoolean("mapreduce.task.ismap", true)
+      hadoopConf.setInt("mapreduce.task.partition", 0)
 
       new TaskAttemptContextImpl(hadoopConf, taskAttemptId)
     }
diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala
index 7ba5790c29..c7d953a731 100644
--- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala
+++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala
@@ -95,9 +95,9 @@ class HiveWindowFunctionQuerySuite extends HiveComparisonTest with BeforeAndAfte
     // This is used to generate golden files.
     sql("set hive.plan.serialization.format=kryo")
     // Explicitly set fs to local fs.
-    sql(s"set fs.default.name=file://$testTempDir/")
+    sql(s"set fs.defaultFS=file://$testTempDir/")
     // Ask Hive to run jobs in-process as a single map and reduce task.
-    sql("set mapred.job.tracker=local")
+    sql("set mapreduce.jobtracker.address=local")
   }
 
   override def afterAll() {
@@ -764,9 +764,9 @@ class HiveWindowFunctionQueryFileSuite
     // This is used to generate golden files.
     // sql("set hive.plan.serialization.format=kryo")
     // Explicitly set fs to local fs.
-    // sql(s"set fs.default.name=file://$testTempDir/")
+    // sql(s"set fs.defaultFS=file://$testTempDir/")
     // Ask Hive to run jobs in-process as a single map and reduce task.
-    // sql("set mapred.job.tracker=local")
+    // sql("set mapreduce.jobtracker.address=local")
   }
 
   override def afterAll() {
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
index b4b63032ab..d48702b610 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
@@ -66,14 +66,16 @@ class HadoopTableReader(
     hadoopConf: Configuration)
   extends TableReader with Logging {
 
-  // Hadoop honors "mapred.map.tasks" as hint, but will ignore when mapred.job.tracker is "local".
-  // https://hadoop.apache.org/docs/r1.0.4/mapred-default.html
+  // Hadoop honors "mapreduce.job.maps" as hint,
+  // but will ignore when mapreduce.jobtracker.address is "local".
+  // https://hadoop.apache.org/docs/r2.6.5/hadoop-mapreduce-client/hadoop-mapreduce-client-core/
+  // mapred-default.xml
   //
   // In order keep consistency with Hive, we will let it be 0 in local mode also.
   private val _minSplitsPerRDD = if (sparkSession.sparkContext.isLocal) {
     0 // will splitted based on block by default.
   } else {
-    math.max(hadoopConf.getInt("mapred.map.tasks", 1),
+    math.max(hadoopConf.getInt("mapreduce.job.maps", 1),
       sparkSession.sparkContext.defaultMinPartitions)
   }
 
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
index 5d5688ecb3..142f25defb 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
@@ -228,13 +228,16 @@ case class InsertIntoHiveTable(
     val isCompressed = hadoopConf.get("hive.exec.compress.output", "false").toBoolean
 
     if (isCompressed) {
-      // Please note that isCompressed, "mapred.output.compress", "mapred.output.compression.codec",
-      // and "mapred.output.compression.type" have no impact on ORC because it uses table properties
-      // to store compression information.
-      hadoopConf.set("mapred.output.compress", "true")
+      // Please note that isCompressed, "mapreduce.output.fileoutputformat.compress",
+      // "mapreduce.output.fileoutputformat.compress.codec", and
+      // "mapreduce.output.fileoutputformat.compress.type"
+      // have no impact on ORC because it uses table properties to store compression information.
+      hadoopConf.set("mapreduce.output.fileoutputformat.compress", "true")
       fileSinkConf.setCompressed(true)
-      fileSinkConf.setCompressCodec(hadoopConf.get("mapred.output.compression.codec"))
-      fileSinkConf.setCompressType(hadoopConf.get("mapred.output.compression.type"))
+      fileSinkConf.setCompressCodec(hadoopConf
+        .get("mapreduce.output.fileoutputformat.compress.codec"))
+      fileSinkConf.setCompressType(hadoopConf
+        .get("mapreduce.output.fileoutputformat.compress.type"))
     }
 
     val numDynamicPartitions = partition.values.count(_.isEmpty)
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
index fd13911947..efc2f00984 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
@@ -439,7 +439,7 @@ private[hive] class TestHiveSparkSession(
         foreach { udfName => FunctionRegistry.unregisterTemporaryUDF(udfName) }
 
       // Some tests corrupt this value on purpose, which breaks the RESET call below.
-      sessionState.conf.setConfString("fs.default.name", new File(".").toURI.toString)
+      sessionState.conf.setConfString("fs.defaultFS", new File(".").toURI.toString)
       // It is important that we RESET first as broken hooks that might have been set could break
       // other sql exec here.
       sessionState.metadataHive.runSqlHive("RESET")
diff --git a/sql/hive/src/test/resources/golden/auto_join14_hadoop20-2-db1cd54a4cb36de2087605f32e41824f b/sql/hive/src/test/resources/golden/auto_join14_hadoop20-2-2b9ccaa793eae0e73bf76335d3d6880
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_join14_hadoop20-2-db1cd54a4cb36de2087605f32e41824f
rename to sql/hive/src/test/resources/golden/auto_join14_hadoop20-2-2b9ccaa793eae0e73bf76335d3d6880
diff --git a/sql/hive/src/test/resources/golden/combine1-2-c95dc367df88c9e5cf77157f29ba2daf b/sql/hive/src/test/resources/golden/combine1-2-6142f47d3fcdd4323162014d5eb35e07
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine1-2-c95dc367df88c9e5cf77157f29ba2daf
rename to sql/hive/src/test/resources/golden/combine1-2-6142f47d3fcdd4323162014d5eb35e07
diff --git a/sql/hive/src/test/resources/golden/combine1-3-6e53a3ac93113f20db3a12f1dcf30e86 b/sql/hive/src/test/resources/golden/combine1-3-10266e3d5dd4c841c0d65030b1edba7c
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine1-3-6e53a3ac93113f20db3a12f1dcf30e86
rename to sql/hive/src/test/resources/golden/combine1-3-10266e3d5dd4c841c0d65030b1edba7c
diff --git a/sql/hive/src/test/resources/golden/combine1-4-84967075baa3e56fff2a23f8ab9ba076 b/sql/hive/src/test/resources/golden/combine1-4-9cbd6d400fb6c3cd09010e3dbd76601
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine1-4-84967075baa3e56fff2a23f8ab9ba076
rename to sql/hive/src/test/resources/golden/combine1-4-9cbd6d400fb6c3cd09010e3dbd76601
diff --git a/sql/hive/src/test/resources/golden/combine1-5-2ee5d706fe3a3bcc38b795f6e94970ea b/sql/hive/src/test/resources/golden/combine1-5-1ba2d6f3bb3348da3fee7fab4f283f34
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine1-5-2ee5d706fe3a3bcc38b795f6e94970ea
rename to sql/hive/src/test/resources/golden/combine1-5-1ba2d6f3bb3348da3fee7fab4f283f34
diff --git a/sql/hive/src/test/resources/golden/combine2-2-c95dc367df88c9e5cf77157f29ba2daf b/sql/hive/src/test/resources/golden/combine2-2-6142f47d3fcdd4323162014d5eb35e07
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine2-2-c95dc367df88c9e5cf77157f29ba2daf
rename to sql/hive/src/test/resources/golden/combine2-2-6142f47d3fcdd4323162014d5eb35e07
diff --git a/sql/hive/src/test/resources/golden/combine2-3-6e53a3ac93113f20db3a12f1dcf30e86 b/sql/hive/src/test/resources/golden/combine2-3-10266e3d5dd4c841c0d65030b1edba7c
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine2-3-6e53a3ac93113f20db3a12f1dcf30e86
rename to sql/hive/src/test/resources/golden/combine2-3-10266e3d5dd4c841c0d65030b1edba7c
diff --git a/sql/hive/src/test/resources/golden/combine2-4-84967075baa3e56fff2a23f8ab9ba076 b/sql/hive/src/test/resources/golden/combine2-4-9cbd6d400fb6c3cd09010e3dbd76601
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine2-4-84967075baa3e56fff2a23f8ab9ba076
rename to sql/hive/src/test/resources/golden/combine2-4-9cbd6d400fb6c3cd09010e3dbd76601
diff --git a/sql/hive/src/test/resources/golden/combine2-5-2ee5d706fe3a3bcc38b795f6e94970ea b/sql/hive/src/test/resources/golden/combine2-5-1ba2d6f3bb3348da3fee7fab4f283f34
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine2-5-2ee5d706fe3a3bcc38b795f6e94970ea
rename to sql/hive/src/test/resources/golden/combine2-5-1ba2d6f3bb3348da3fee7fab4f283f34
diff --git a/sql/hive/src/test/resources/golden/groupby1-3-d57ed4bbfee1ffaffaeba0a4be84c31d b/sql/hive/src/test/resources/golden/groupby1-3-c8478dac3497697b4375ee35118a5c3e
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby1-3-d57ed4bbfee1ffaffaeba0a4be84c31d
rename to sql/hive/src/test/resources/golden/groupby1-3-c8478dac3497697b4375ee35118a5c3e
diff --git a/sql/hive/src/test/resources/golden/groupby1-5-dd7bf298b8c921355edd8665c6b0c168 b/sql/hive/src/test/resources/golden/groupby1-5-c9cee6382b64bd3d71177527961b8be2
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby1-5-dd7bf298b8c921355edd8665c6b0c168
rename to sql/hive/src/test/resources/golden/groupby1-5-c9cee6382b64bd3d71177527961b8be2
diff --git a/sql/hive/src/test/resources/golden/groupby1_limit-0-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby1_limit-0-be2c0b32a02a1154bfdee1a52530f387
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby1_limit-0-83c59d378571a6e487aa20217bd87817
rename to sql/hive/src/test/resources/golden/groupby1_limit-0-be2c0b32a02a1154bfdee1a52530f387
diff --git a/sql/hive/src/test/resources/golden/groupby1_map-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby1_map-2-be2c0b32a02a1154bfdee1a52530f387
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby1_map-2-83c59d378571a6e487aa20217bd87817
rename to sql/hive/src/test/resources/golden/groupby1_map-2-be2c0b32a02a1154bfdee1a52530f387
diff --git a/sql/hive/src/test/resources/golden/groupby1_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby1_map_skew-2-be2c0b32a02a1154bfdee1a52530f387
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby1_map_skew-2-83c59d378571a6e487aa20217bd87817
rename to sql/hive/src/test/resources/golden/groupby1_map_skew-2-be2c0b32a02a1154bfdee1a52530f387
diff --git a/sql/hive/src/test/resources/golden/groupby1_noskew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby1_noskew-2-be2c0b32a02a1154bfdee1a52530f387
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby1_noskew-2-83c59d378571a6e487aa20217bd87817
rename to sql/hive/src/test/resources/golden/groupby1_noskew-2-be2c0b32a02a1154bfdee1a52530f387
diff --git a/sql/hive/src/test/resources/golden/groupby2_limit-0-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby2_limit-0-be2c0b32a02a1154bfdee1a52530f387
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby2_limit-0-83c59d378571a6e487aa20217bd87817
rename to sql/hive/src/test/resources/golden/groupby2_limit-0-be2c0b32a02a1154bfdee1a52530f387
diff --git a/sql/hive/src/test/resources/golden/groupby2_map-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby2_map-2-be2c0b32a02a1154bfdee1a52530f387
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby2_map-2-83c59d378571a6e487aa20217bd87817
rename to sql/hive/src/test/resources/golden/groupby2_map-2-be2c0b32a02a1154bfdee1a52530f387
diff --git a/sql/hive/src/test/resources/golden/groupby2_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby2_map_skew-2-be2c0b32a02a1154bfdee1a52530f387
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby2_map_skew-2-83c59d378571a6e487aa20217bd87817
rename to sql/hive/src/test/resources/golden/groupby2_map_skew-2-be2c0b32a02a1154bfdee1a52530f387
diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby2_noskew-2-be2c0b32a02a1154bfdee1a52530f387
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby2_noskew-2-83c59d378571a6e487aa20217bd87817
rename to sql/hive/src/test/resources/golden/groupby2_noskew-2-be2c0b32a02a1154bfdee1a52530f387
diff --git a/sql/hive/src/test/resources/golden/groupby4_map-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby4_map-2-be2c0b32a02a1154bfdee1a52530f387
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby4_map-2-83c59d378571a6e487aa20217bd87817
rename to sql/hive/src/test/resources/golden/groupby4_map-2-be2c0b32a02a1154bfdee1a52530f387
diff --git a/sql/hive/src/test/resources/golden/groupby4_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby4_map_skew-2-be2c0b32a02a1154bfdee1a52530f387
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby4_map_skew-2-83c59d378571a6e487aa20217bd87817
rename to sql/hive/src/test/resources/golden/groupby4_map_skew-2-be2c0b32a02a1154bfdee1a52530f387
diff --git a/sql/hive/src/test/resources/golden/groupby4_noskew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby4_noskew-2-be2c0b32a02a1154bfdee1a52530f387
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby4_noskew-2-83c59d378571a6e487aa20217bd87817
rename to sql/hive/src/test/resources/golden/groupby4_noskew-2-be2c0b32a02a1154bfdee1a52530f387
diff --git a/sql/hive/src/test/resources/golden/groupby5_map-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby5_map-2-be2c0b32a02a1154bfdee1a52530f387
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby5_map-2-83c59d378571a6e487aa20217bd87817
rename to sql/hive/src/test/resources/golden/groupby5_map-2-be2c0b32a02a1154bfdee1a52530f387
diff --git a/sql/hive/src/test/resources/golden/groupby5_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby5_map_skew-2-be2c0b32a02a1154bfdee1a52530f387
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby5_map_skew-2-83c59d378571a6e487aa20217bd87817
rename to sql/hive/src/test/resources/golden/groupby5_map_skew-2-be2c0b32a02a1154bfdee1a52530f387
diff --git a/sql/hive/src/test/resources/golden/groupby5_noskew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby5_noskew-2-be2c0b32a02a1154bfdee1a52530f387
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby5_noskew-2-83c59d378571a6e487aa20217bd87817
rename to sql/hive/src/test/resources/golden/groupby5_noskew-2-be2c0b32a02a1154bfdee1a52530f387
diff --git a/sql/hive/src/test/resources/golden/groupby6_map-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby6_map-2-be2c0b32a02a1154bfdee1a52530f387
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby6_map-2-83c59d378571a6e487aa20217bd87817
rename to sql/hive/src/test/resources/golden/groupby6_map-2-be2c0b32a02a1154bfdee1a52530f387
diff --git a/sql/hive/src/test/resources/golden/groupby6_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby6_map_skew-2-be2c0b32a02a1154bfdee1a52530f387
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby6_map_skew-2-83c59d378571a6e487aa20217bd87817
rename to sql/hive/src/test/resources/golden/groupby6_map_skew-2-be2c0b32a02a1154bfdee1a52530f387
diff --git a/sql/hive/src/test/resources/golden/groupby6_noskew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby6_noskew-2-be2c0b32a02a1154bfdee1a52530f387
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby6_noskew-2-83c59d378571a6e487aa20217bd87817
rename to sql/hive/src/test/resources/golden/groupby6_noskew-2-be2c0b32a02a1154bfdee1a52530f387
diff --git a/sql/hive/src/test/resources/golden/groupby7_map-3-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby7_map-3-be2c0b32a02a1154bfdee1a52530f387
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby7_map-3-83c59d378571a6e487aa20217bd87817
rename to sql/hive/src/test/resources/golden/groupby7_map-3-be2c0b32a02a1154bfdee1a52530f387
diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-2-be2c0b32a02a1154bfdee1a52530f387
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817
rename to sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-2-be2c0b32a02a1154bfdee1a52530f387
diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby7_map_skew-2-be2c0b32a02a1154bfdee1a52530f387
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby7_map_skew-2-83c59d378571a6e487aa20217bd87817
rename to sql/hive/src/test/resources/golden/groupby7_map_skew-2-be2c0b32a02a1154bfdee1a52530f387
diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-3-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby7_noskew-3-be2c0b32a02a1154bfdee1a52530f387
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby7_noskew-3-83c59d378571a6e487aa20217bd87817
rename to sql/hive/src/test/resources/golden/groupby7_noskew-3-be2c0b32a02a1154bfdee1a52530f387
diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-2-be2c0b32a02a1154bfdee1a52530f387
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817
rename to sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-2-be2c0b32a02a1154bfdee1a52530f387
diff --git a/sql/hive/src/test/resources/golden/groupby8_map-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby8_map-2-be2c0b32a02a1154bfdee1a52530f387
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby8_map-2-83c59d378571a6e487aa20217bd87817
rename to sql/hive/src/test/resources/golden/groupby8_map-2-be2c0b32a02a1154bfdee1a52530f387
diff --git a/sql/hive/src/test/resources/golden/groupby8_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby8_map_skew-2-be2c0b32a02a1154bfdee1a52530f387
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby8_map_skew-2-83c59d378571a6e487aa20217bd87817
rename to sql/hive/src/test/resources/golden/groupby8_map_skew-2-be2c0b32a02a1154bfdee1a52530f387
diff --git a/sql/hive/src/test/resources/golden/groupby8_noskew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby8_noskew-2-be2c0b32a02a1154bfdee1a52530f387
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby8_noskew-2-83c59d378571a6e487aa20217bd87817
rename to sql/hive/src/test/resources/golden/groupby8_noskew-2-be2c0b32a02a1154bfdee1a52530f387
diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby_map_ppr-2-be2c0b32a02a1154bfdee1a52530f387
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_map_ppr-2-83c59d378571a6e487aa20217bd87817
rename to sql/hive/src/test/resources/golden/groupby_map_ppr-2-be2c0b32a02a1154bfdee1a52530f387
diff --git a/sql/hive/src/test/resources/golden/input12_hadoop20-0-db1cd54a4cb36de2087605f32e41824f b/sql/hive/src/test/resources/golden/input12_hadoop20-0-2b9ccaa793eae0e73bf76335d3d6880
similarity index 100%
rename from sql/hive/src/test/resources/golden/input12_hadoop20-0-db1cd54a4cb36de2087605f32e41824f
rename to sql/hive/src/test/resources/golden/input12_hadoop20-0-2b9ccaa793eae0e73bf76335d3d6880
diff --git a/sql/hive/src/test/resources/golden/input_testsequencefile-0-68975193b30cb34102b380e647d8d5f4 b/sql/hive/src/test/resources/golden/input_testsequencefile-0-dd959af1968381d0ed90178d349b01a7
similarity index 100%
rename from sql/hive/src/test/resources/golden/input_testsequencefile-0-68975193b30cb34102b380e647d8d5f4
rename to sql/hive/src/test/resources/golden/input_testsequencefile-0-dd959af1968381d0ed90178d349b01a7
diff --git a/sql/hive/src/test/resources/golden/input_testsequencefile-1-1c0f3be2d837dee49312e0a80440447e b/sql/hive/src/test/resources/golden/input_testsequencefile-1-ddbb8d5e5dc0988bda96ac2b4aec8f94
similarity index 100%
rename from sql/hive/src/test/resources/golden/input_testsequencefile-1-1c0f3be2d837dee49312e0a80440447e
rename to sql/hive/src/test/resources/golden/input_testsequencefile-1-ddbb8d5e5dc0988bda96ac2b4aec8f94
diff --git a/sql/hive/src/test/resources/golden/input_testsequencefile-5-3708198aac609695b22e19e89306034c b/sql/hive/src/test/resources/golden/input_testsequencefile-5-25715870c569b0f8c3d483e3a38b3199
similarity index 100%
rename from sql/hive/src/test/resources/golden/input_testsequencefile-5-3708198aac609695b22e19e89306034c
rename to sql/hive/src/test/resources/golden/input_testsequencefile-5-25715870c569b0f8c3d483e3a38b3199
diff --git a/sql/hive/src/test/resources/golden/join14_hadoop20-1-db1cd54a4cb36de2087605f32e41824f b/sql/hive/src/test/resources/golden/join14_hadoop20-1-2b9ccaa793eae0e73bf76335d3d6880
similarity index 100%
rename from sql/hive/src/test/resources/golden/join14_hadoop20-1-db1cd54a4cb36de2087605f32e41824f
rename to sql/hive/src/test/resources/golden/join14_hadoop20-1-2b9ccaa793eae0e73bf76335d3d6880
diff --git a/sql/hive/src/test/resources/golden/leftsemijoin_mr-7-8e9c2969b999557363e40f9ebb3f6d7c b/sql/hive/src/test/resources/golden/leftsemijoin_mr-7-6b9861b999092f1ea4fa1fd27a666af6
similarity index 100%
rename from sql/hive/src/test/resources/golden/leftsemijoin_mr-7-8e9c2969b999557363e40f9ebb3f6d7c
rename to sql/hive/src/test/resources/golden/leftsemijoin_mr-7-6b9861b999092f1ea4fa1fd27a666af6
diff --git a/sql/hive/src/test/resources/golden/merge2-2-c95dc367df88c9e5cf77157f29ba2daf b/sql/hive/src/test/resources/golden/merge2-2-6142f47d3fcdd4323162014d5eb35e07
similarity index 100%
rename from sql/hive/src/test/resources/golden/merge2-2-c95dc367df88c9e5cf77157f29ba2daf
rename to sql/hive/src/test/resources/golden/merge2-2-6142f47d3fcdd4323162014d5eb35e07
diff --git a/sql/hive/src/test/resources/golden/merge2-3-6e53a3ac93113f20db3a12f1dcf30e86 b/sql/hive/src/test/resources/golden/merge2-3-10266e3d5dd4c841c0d65030b1edba7c
similarity index 100%
rename from sql/hive/src/test/resources/golden/merge2-3-6e53a3ac93113f20db3a12f1dcf30e86
rename to sql/hive/src/test/resources/golden/merge2-3-10266e3d5dd4c841c0d65030b1edba7c
diff --git a/sql/hive/src/test/resources/golden/merge2-4-84967075baa3e56fff2a23f8ab9ba076 b/sql/hive/src/test/resources/golden/merge2-4-9cbd6d400fb6c3cd09010e3dbd76601
similarity index 100%
rename from sql/hive/src/test/resources/golden/merge2-4-84967075baa3e56fff2a23f8ab9ba076
rename to sql/hive/src/test/resources/golden/merge2-4-9cbd6d400fb6c3cd09010e3dbd76601
diff --git a/sql/hive/src/test/resources/golden/merge2-5-2ee5d706fe3a3bcc38b795f6e94970ea b/sql/hive/src/test/resources/golden/merge2-5-1ba2d6f3bb3348da3fee7fab4f283f34
similarity index 100%
rename from sql/hive/src/test/resources/golden/merge2-5-2ee5d706fe3a3bcc38b795f6e94970ea
rename to sql/hive/src/test/resources/golden/merge2-5-1ba2d6f3bb3348da3fee7fab4f283f34
diff --git a/sql/hive/src/test/resources/golden/parallel-0-23a4feaede17467a8cc26e4d86ec30f9 b/sql/hive/src/test/resources/golden/parallel-0-6dc30e2de057022e63bd2a645fbec4c2
similarity index 100%
rename from sql/hive/src/test/resources/golden/parallel-0-23a4feaede17467a8cc26e4d86ec30f9
rename to sql/hive/src/test/resources/golden/parallel-0-6dc30e2de057022e63bd2a645fbec4c2
diff --git a/sql/hive/src/test/resources/golden/rcfile_lazydecompress-11-3708198aac609695b22e19e89306034c b/sql/hive/src/test/resources/golden/rcfile_lazydecompress-11-25715870c569b0f8c3d483e3a38b3199
similarity index 100%
rename from sql/hive/src/test/resources/golden/rcfile_lazydecompress-11-3708198aac609695b22e19e89306034c
rename to sql/hive/src/test/resources/golden/rcfile_lazydecompress-11-25715870c569b0f8c3d483e3a38b3199
diff --git a/sql/hive/src/test/resources/golden/rcfile_lazydecompress-5-68975193b30cb34102b380e647d8d5f4 b/sql/hive/src/test/resources/golden/rcfile_lazydecompress-5-dd959af1968381d0ed90178d349b01a7
similarity index 100%
rename from sql/hive/src/test/resources/golden/rcfile_lazydecompress-5-68975193b30cb34102b380e647d8d5f4
rename to sql/hive/src/test/resources/golden/rcfile_lazydecompress-5-dd959af1968381d0ed90178d349b01a7
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join14_hadoop20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join14_hadoop20.q
index 235b7c1b3f..6a9a20f320 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join14_hadoop20.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join14_hadoop20.q
@@ -5,7 +5,7 @@ set hive.auto.convert.join = true;
 
 CREATE TABLE dest1(c1 INT, c2 STRING) STORED AS TEXTFILE;
 
-set mapred.job.tracker=localhost:58;
+set mapreduce.jobtracker.address=localhost:58;
 set hive.exec.mode.local.auto=true;
 
 explain
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket5.q
index 877f8a50a0..87f6eca4dd 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket5.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket5.q
@@ -4,7 +4,7 @@ set hive.enforce.sorting = true;
 set hive.exec.reducers.max = 1;
 set hive.merge.mapfiles = true;
 set hive.merge.mapredfiles = true;
-set mapred.reduce.tasks = 2;
+set mapreduce.job.reduces = 2;
 
 -- Tests that when a multi insert inserts into a bucketed table and a table which is not bucketed
 -- the bucketed table is not merged and the table which is not bucketed is
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_num_reducers.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_num_reducers.q
index 37ae6cc7ad..84fe3919d7 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_num_reducers.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_num_reducers.q
@@ -1,6 +1,6 @@
 set hive.enforce.bucketing = true;
 set hive.exec.mode.local.auto=false;
-set mapred.reduce.tasks = 10;
+set mapreduce.job.reduces = 10;
 
 -- This test sets number of mapred tasks to 10 for a database with 50 buckets, 
 -- and uses a post-hook to confirm that 10 tasks were created
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat.q
index d2e12e82d4..ae72f98fa4 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat.q
@@ -1,5 +1,5 @@
 set hive.input.format=org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat;
-set mapred.min.split.size = 64;
+set mapreduce.input.fileinputformat.split.minsize = 64;
 
 CREATE TABLE T1(name STRING) STORED AS TEXTFILE;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine1.q
index 86abf09960..5ecfc21724 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine1.q
@@ -1,11 +1,11 @@
 set hive.exec.compress.output = true;
 set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
-set mapred.min.split.size=256;
-set mapred.min.split.size.per.node=256;
-set mapred.min.split.size.per.rack=256;
-set mapred.max.split.size=256;
+set mapreduce.input.fileinputformat.split.minsize=256;
+set mapreduce.input.fileinputformat.split.minsize.per.node=256;
+set mapreduce.input.fileinputformat.split.minsize.per.rack=256;
+set mapreduce.input.fileinputformat.split.maxsize=256;
 
-set mapred.output.compression.codec=org.apache.hadoop.io.compress.GzipCodec;
+set mapreduce.output.fileoutputformat.compress.codec=org.apache.hadoop.io.compress.GzipCodec;
 
 create table combine1_1(key string, value string) stored as textfile;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine2.q
index cfd9856f08..acd0dd5e5b 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine2.q
@@ -1,10 +1,10 @@
 USE default;
 
 set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
-set mapred.min.split.size=256;
-set mapred.min.split.size.per.node=256;
-set mapred.min.split.size.per.rack=256;
-set mapred.max.split.size=256;
+set mapreduce.input.fileinputformat.split.minsize=256;
+set mapreduce.input.fileinputformat.split.minsize.per.node=256;
+set mapreduce.input.fileinputformat.split.minsize.per.rack=256;
+set mapreduce.input.fileinputformat.split.maxsize=256;
 set hive.exec.dynamic.partition=true;
 set hive.exec.dynamic.partition.mode=nonstrict;
 set mapred.cache.shared.enabled=false;
@@ -18,7 +18,7 @@ set hive.merge.smallfiles.avgsize=0;
 create table combine2(key string) partitioned by (value string);
 
 -- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.20, 0.20S)
--- This test sets mapred.max.split.size=256 and hive.merge.smallfiles.avgsize=0
+-- This test sets mapreduce.input.fileinputformat.split.maxsize=256 and hive.merge.smallfiles.avgsize=0
 -- in an attempt to force the generation of multiple splits and multiple output files.
 -- However, Hadoop 0.20 is incapable of generating splits smaller than the block size
 -- when using CombineFileInputFormat, so only one split is generated. This has a
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine2_hadoop20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine2_hadoop20.q
index 8f9a59d497..597d3ae479 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine2_hadoop20.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine2_hadoop20.q
@@ -1,10 +1,10 @@
 USE default;
 
 set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
-set mapred.min.split.size=256;
-set mapred.min.split.size.per.node=256;
-set mapred.min.split.size.per.rack=256;
-set mapred.max.split.size=256;
+set mapreduce.input.fileinputformat.split.minsize=256;
+set mapreduce.input.fileinputformat.split.minsize.per.node=256;
+set mapreduce.input.fileinputformat.split.minsize.per.rack=256;
+set mapreduce.input.fileinputformat.split.maxsize=256;
 set hive.exec.dynamic.partition=true;
 set hive.exec.dynamic.partition.mode=nonstrict;
 set mapred.cache.shared.enabled=false;
@@ -17,7 +17,7 @@ set hive.merge.smallfiles.avgsize=0;
 create table combine2(key string) partitioned by (value string);
 
 -- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20, 0.20S)
--- This test sets mapred.max.split.size=256 and hive.merge.smallfiles.avgsize=0
+-- This test sets mapreduce.input.fileinputformat.split.maxsize=256 and hive.merge.smallfiles.avgsize=0
 -- in an attempt to force the generation of multiple splits and multiple output files.
 -- However, Hadoop 0.20 is incapable of generating splits smaller than the block size
 -- when using CombineFileInputFormat, so only one split is generated. This has a
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine2_win.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine2_win.q
index f6090bb99b..4f7174a1b6 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine2_win.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine2_win.q
@@ -1,8 +1,8 @@
 set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
-set mapred.min.split.size=256;
-set mapred.min.split.size.per.node=256;
-set mapred.min.split.size.per.rack=256;
-set mapred.max.split.size=256;
+set mapreduce.input.fileinputformat.split.minsize=256;
+set mapreduce.input.fileinputformat.split.minsize.per.node=256;
+set mapreduce.input.fileinputformat.split.minsize.per.rack=256;
+set mapreduce.input.fileinputformat.split.maxsize=256;
 set hive.exec.dynamic.partition=true;
 set hive.exec.dynamic.partition.mode=nonstrict;
 set mapred.cache.shared.enabled=false;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine3.q
index c9afc91bb4..35dd442027 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine3.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine3.q
@@ -1,9 +1,9 @@
 set hive.exec.compress.output = true;
 set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
-set mapred.min.split.size=256;
-set mapred.min.split.size.per.node=256;
-set mapred.min.split.size.per.rack=256;
-set mapred.max.split.size=256;
+set mapreduce.input.fileinputformat.split.minsize=256;
+set mapreduce.input.fileinputformat.split.minsize.per.node=256;
+set mapreduce.input.fileinputformat.split.minsize.per.rack=256;
+set mapreduce.input.fileinputformat.split.maxsize=256;
 
 
 drop table combine_3_srcpart_seq_rc;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_1.q
index f348e59022..5e51d11864 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_1.q
@@ -1,4 +1,4 @@
-set fs.default.name=invalidscheme:///;
+set fs.defaultFS=invalidscheme:///;
 
 CREATE TABLE table1 (a STRING, b STRING) STORED AS TEXTFILE;
 DESCRIBE table1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_hadoop20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_hadoop20.q
index f39689de03..979c907230 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_hadoop20.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_hadoop20.q
@@ -49,7 +49,7 @@ describe formatted nzhang_CTAS4;
 
 explain extended create table nzhang_ctas5 row format delimited fields terminated by ',' lines terminated by '\012' stored as textfile as select key, value from src sort by key, value limit 10;
 
-set mapred.job.tracker=localhost:58;
+set mapreduce.jobtracker.address=localhost:58;
 set hive.exec.mode.local.auto=true;
 
 create table nzhang_ctas5 row format delimited fields terminated by ',' lines terminated by '\012' stored as textfile as select key, value from src sort by key, value limit 10;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1.q
index 1275eab281..0d75857e54 100755
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1.q
@@ -3,12 +3,12 @@ set hive.groupby.skewindata=true;
 
 CREATE TABLE dest_g1(key INT, value DOUBLE) STORED AS TEXTFILE;
 
-set fs.default.name=invalidscheme:///;
+set fs.defaultFS=invalidscheme:///;
 
 EXPLAIN
 FROM src INSERT OVERWRITE TABLE dest_g1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key;
 
-set fs.default.name=file:///;
+set fs.defaultFS=file:///;
 
 FROM src INSERT OVERWRITE TABLE dest_g1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_limit.q
index 55133332a8..bbb2859a9d 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_limit.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_limit.q
@@ -1,4 +1,4 @@
-set mapred.reduce.tasks=31;
+set mapreduce.job.reduces=31;
 
 CREATE TABLE dest1(key INT, value DOUBLE) STORED AS TEXTFILE;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map.q
index dde37dfd47..7883d948d0 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map.q
@@ -1,6 +1,6 @@
 set hive.map.aggr=true;
 set hive.groupby.skewindata=false;
-set mapred.reduce.tasks=31;
+set mapreduce.job.reduces=31;
 
 CREATE TABLE dest1(key INT, value DOUBLE) STORED AS TEXTFILE;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map_skew.q
index f346cb7e90..a5ac3762ce 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map_skew.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map_skew.q
@@ -1,6 +1,6 @@
 set hive.map.aggr=true;
 set hive.groupby.skewindata=true;
-set mapred.reduce.tasks=31;
+set mapreduce.job.reduces=31;
 
 CREATE TABLE dest1(key INT, value DOUBLE) STORED AS TEXTFILE;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_noskew.q
index c587b5f658..6341eefb50 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_noskew.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_noskew.q
@@ -1,6 +1,6 @@
 set hive.map.aggr=false;
 set hive.groupby.skewindata=false;
-set mapred.reduce.tasks=31;
+set mapreduce.job.reduces=31;
 
 CREATE TABLE dest_g1(key INT, value DOUBLE) STORED AS TEXTFILE;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_limit.q
index 30499248ca..df4693446d 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_limit.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_limit.q
@@ -1,4 +1,4 @@
-set mapred.reduce.tasks=31;
+set mapreduce.job.reduces=31;
 
 EXPLAIN
 SELECT src.key, sum(substr(src.value,5)) FROM src GROUP BY src.key ORDER BY src.key LIMIT 5;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map.q
index 794ec758e9..7b6e175c2d 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map.q
@@ -1,6 +1,6 @@
 set hive.map.aggr=true;
 set hive.groupby.skewindata=false;
-set mapred.reduce.tasks=31;
+set mapreduce.job.reduces=31;
 
 CREATE TABLE dest1(key STRING, c1 INT, c2 STRING) STORED AS TEXTFILE;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q
index 55d1a34b3c..3aeae0d5c3 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q
@@ -1,6 +1,6 @@
 set hive.map.aggr=true;
 set hive.groupby.skewindata=false;
-set mapred.reduce.tasks=31;
+set mapreduce.job.reduces=31;
 
 CREATE TABLE dest1(key STRING, c1 INT, c2 STRING, c3 INT, c4 INT) STORED AS TEXTFILE;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_skew.q
index 39a2a178e3..998156d05f 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_skew.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_skew.q
@@ -1,6 +1,6 @@
 set hive.map.aggr=true;
 set hive.groupby.skewindata=true;
-set mapred.reduce.tasks=31;
+set mapreduce.job.reduces=31;
 
 CREATE TABLE dest1(key STRING, c1 INT, c2 STRING) STORED AS TEXTFILE;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew.q
index 6d7cb61e2d..fab4f5d097 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew.q
@@ -1,6 +1,6 @@
 set hive.map.aggr=false;
 set hive.groupby.skewindata=false;
-set mapred.reduce.tasks=31;
+set mapreduce.job.reduces=31;
 
 CREATE TABLE dest_g2(key STRING, c1 INT, c2 STRING) STORED AS TEXTFILE;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew_multi_distinct.q
index b2450c9ea0..9ef556cdc5 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew_multi_distinct.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew_multi_distinct.q
@@ -1,6 +1,6 @@
 set hive.map.aggr=false;
 set hive.groupby.skewindata=false;
-set mapred.reduce.tasks=31;
+set mapreduce.job.reduces=31;
 
 CREATE TABLE dest_g2(key STRING, c1 INT, c2 STRING, c3 INT, c4 INT) STORED AS TEXTFILE;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_map.q
index 7ecc71dfab..36ba5d89c0 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_map.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_map.q
@@ -1,6 +1,6 @@
 set hive.map.aggr=true;
 set hive.groupby.skewindata=false;
-set mapred.reduce.tasks=31;
+set mapreduce.job.reduces=31;
 
 CREATE TABLE dest1(c1 DOUBLE, c2 DOUBLE, c3 DOUBLE, c4 DOUBLE, c5 DOUBLE, c6 DOUBLE, c7 DOUBLE, c8 DOUBLE, c9 DOUBLE) STORED AS TEXTFILE;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_map_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_map_multi_distinct.q
index 50243beca9..6f0a9635a2 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_map_multi_distinct.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_map_multi_distinct.q
@@ -1,6 +1,6 @@
 set hive.map.aggr=true;
 set hive.groupby.skewindata=false;
-set mapred.reduce.tasks=31;
+set mapreduce.job.reduces=31;
 
 CREATE TABLE dest1(c1 DOUBLE, c2 DOUBLE, c3 DOUBLE, c4 DOUBLE, c5 DOUBLE, c6 DOUBLE, c7 DOUBLE, c8 DOUBLE, c9 DOUBLE, c10 DOUBLE, c11 DOUBLE) STORED AS TEXTFILE;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_map_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_map_skew.q
index 07d10c2d74..64a49e2525 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_map_skew.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_map_skew.q
@@ -1,6 +1,6 @@
 set hive.map.aggr=true;
 set hive.groupby.skewindata=true;
-set mapred.reduce.tasks=31;
+set mapreduce.job.reduces=31;
 
 CREATE TABLE dest1(c1 DOUBLE, c2 DOUBLE, c3 DOUBLE, c4 DOUBLE, c5 DOUBLE, c6 DOUBLE, c7 DOUBLE, c8 DOUBLE, c9 DOUBLE) STORED AS TEXTFILE;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_noskew.q
index d33f12c574..4fd98efd6e 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_noskew.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_noskew.q
@@ -1,7 +1,7 @@
 set hive.map.aggr=false;
 
 set hive.groupby.skewindata=false;
-set mapred.reduce.tasks=31;
+set mapreduce.job.reduces=31;
 
 CREATE TABLE dest1(c1 DOUBLE, c2 DOUBLE, c3 DOUBLE, c4 DOUBLE, c5 DOUBLE, c6 DOUBLE, c7 DOUBLE, c8 DOUBLE, c9 DOUBLE) STORED AS TEXTFILE;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_noskew_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_noskew_multi_distinct.q
index 86d8986f1d..85ee8ac43e 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_noskew_multi_distinct.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_noskew_multi_distinct.q
@@ -1,7 +1,7 @@
 set hive.map.aggr=false;
 
 set hive.groupby.skewindata=false;
-set mapred.reduce.tasks=31;
+set mapreduce.job.reduces=31;
 
 CREATE TABLE dest1(c1 DOUBLE, c2 DOUBLE, c3 DOUBLE, c4 DOUBLE, c5 DOUBLE, c6 DOUBLE, c7 DOUBLE, c8 DOUBLE, c9 DOUBLE, c10 DOUBLE, c11 DOUBLE) STORED AS TEXTFILE;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_map.q
index 8ecce23eb8..d71721875b 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_map.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_map.q
@@ -1,6 +1,6 @@
 set hive.map.aggr=true;
 set hive.groupby.skewindata=false;
-set mapred.reduce.tasks=31;
+set mapreduce.job.reduces=31;
 
 CREATE TABLE dest1(key INT) STORED AS TEXTFILE;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_map_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_map_skew.q
index eb2001c6b2..d1ecba143d 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_map_skew.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_map_skew.q
@@ -1,6 +1,6 @@
 set hive.map.aggr=true;
 set hive.groupby.skewindata=true;
-set mapred.reduce.tasks=31;
+set mapreduce.job.reduces=31;
 
 CREATE TABLE dest1(key INT) STORED AS TEXTFILE;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_noskew.q
index a1ebf90aad..63530c262c 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_noskew.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_noskew.q
@@ -1,7 +1,7 @@
 set hive.map.aggr=false;
 
 set hive.groupby.skewindata=false;
-set mapred.reduce.tasks=31;
+set mapreduce.job.reduces=31;
 
 CREATE TABLE dest1(c1 STRING) STORED AS TEXTFILE;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_map.q
index 4fd6445d79..4418bbffec 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_map.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_map.q
@@ -1,6 +1,6 @@
 set hive.map.aggr=true;
 set hive.groupby.skewindata=false;
-set mapred.reduce.tasks=31;
+set mapreduce.job.reduces=31;
 
 CREATE TABLE dest1(key INT) STORED AS TEXTFILE;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_map_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_map_skew.q
index eccd45dd5b..ef20dacf05 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_map_skew.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_map_skew.q
@@ -1,6 +1,6 @@
 set hive.map.aggr=true;
 set hive.groupby.skewindata=true;
-set mapred.reduce.tasks=31;
+set mapreduce.job.reduces=31;
 
 CREATE TABLE dest1(key INT) STORED AS TEXTFILE;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_noskew.q
index e96568b398..17b322b890 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_noskew.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_noskew.q
@@ -1,7 +1,7 @@
 set hive.map.aggr=false;
 
 set hive.groupby.skewindata=false;
-set mapred.reduce.tasks=31;
+set mapreduce.job.reduces=31;
 
 CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map.q
index ced122fae3..bef0eeee0e 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map.q
@@ -1,6 +1,6 @@
 set hive.map.aggr=true;
 set hive.groupby.skewindata=false;
-set mapred.reduce.tasks=31;
+set mapreduce.job.reduces=31;
 
 CREATE TABLE dest1(c1 STRING) STORED AS TEXTFILE;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map_skew.q
index 0d3727b052..ee93b218ac 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map_skew.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map_skew.q
@@ -1,6 +1,6 @@
 set hive.map.aggr=true;
 set hive.groupby.skewindata=true;
-set mapred.reduce.tasks=31;
+set mapreduce.job.reduces=31;
 
 CREATE TABLE dest1(c1 STRING) STORED AS TEXTFILE;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_noskew.q
index 466c13222f..72fff08dec 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_noskew.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_noskew.q
@@ -1,7 +1,7 @@
 set hive.map.aggr=false;
 
 set hive.groupby.skewindata=false;
-set mapred.reduce.tasks=31;
+set mapreduce.job.reduces=31;
 
 CREATE TABLE dest1(c1 STRING) STORED AS TEXTFILE;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map.q
index 2b8c5db41e..75149b1404 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map.q
@@ -1,7 +1,7 @@
 set hive.map.aggr=true;
 set hive.multigroupby.singlereducer=false;
 set hive.groupby.skewindata=false;
-set mapred.reduce.tasks=31;
+set mapreduce.job.reduces=31;
 
 CREATE TABLE DEST1(key INT, value STRING) STORED AS TEXTFILE;
 CREATE TABLE DEST2(key INT, value STRING) STORED AS TEXTFILE;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_multi_single_reducer.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_multi_single_reducer.q
index 5895ed4599..7c7829aac2 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_multi_single_reducer.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_multi_single_reducer.q
@@ -1,6 +1,6 @@
 set hive.map.aggr=true;
 set hive.groupby.skewindata=false;
-set mapred.reduce.tasks=31;
+set mapreduce.job.reduces=31;
 
 CREATE TABLE DEST1(key INT, value STRING) STORED AS TEXTFILE;
 CREATE TABLE DEST2(key INT, value STRING) STORED AS TEXTFILE;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_skew.q
index ee6d7bf830..905986d417 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_skew.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_skew.q
@@ -1,6 +1,6 @@
 set hive.map.aggr=true;
 set hive.groupby.skewindata=true;
-set mapred.reduce.tasks=31;
+set mapreduce.job.reduces=31;
 
 CREATE TABLE DEST1(key INT, value STRING) STORED AS TEXTFILE;
 CREATE TABLE DEST2(key INT, value STRING) STORED AS TEXTFILE;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew.q
index 8c2308e5d7..1f63453672 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew.q
@@ -1,7 +1,7 @@
 set hive.map.aggr=false;
 set hive.multigroupby.singlereducer=false;
 set hive.groupby.skewindata=false;
-set mapred.reduce.tasks=31;
+set mapreduce.job.reduces=31;
 
 CREATE TABLE DEST1(key INT, value STRING) STORED AS TEXTFILE;
 CREATE TABLE DEST2(key INT, value STRING) STORED AS TEXTFILE;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew_multi_single_reducer.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew_multi_single_reducer.q
index e673cc6162..2ce57e9807 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew_multi_single_reducer.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew_multi_single_reducer.q
@@ -1,6 +1,6 @@
 set hive.map.aggr=false;
 set hive.groupby.skewindata=false;
-set mapred.reduce.tasks=31;
+set mapreduce.job.reduces=31;
 
 CREATE TABLE DEST1(key INT, value STRING) STORED AS TEXTFILE;
 CREATE TABLE DEST2(key INT, value STRING) STORED AS TEXTFILE;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map.q
index 0252e99336..9def7d6472 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map.q
@@ -1,6 +1,6 @@
 set hive.map.aggr=true;
 set hive.groupby.skewindata=false;
-set mapred.reduce.tasks=31;
+set mapreduce.job.reduces=31;
 
 CREATE TABLE DEST1(key INT, value STRING) STORED AS TEXTFILE;
 CREATE TABLE DEST2(key INT, value STRING) STORED AS TEXTFILE;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map_skew.q
index b5e1f63a45..788bc68369 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map_skew.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map_skew.q
@@ -1,6 +1,6 @@
 set hive.map.aggr=true;
 set hive.groupby.skewindata=true;
-set mapred.reduce.tasks=31;
+set mapreduce.job.reduces=31;
 
 CREATE TABLE DEST1(key INT, value STRING) STORED AS TEXTFILE;
 CREATE TABLE DEST2(key INT, value STRING) STORED AS TEXTFILE;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_noskew.q
index da85504ca1..17885c56b3 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_noskew.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_noskew.q
@@ -1,7 +1,7 @@
 set hive.map.aggr=false;
 
 set hive.groupby.skewindata=false;
-set mapred.reduce.tasks=31;
+set mapreduce.job.reduces=31;
 
 CREATE TABLE DEST1(key INT, value STRING) STORED AS TEXTFILE;
 CREATE TABLE DEST2(key INT, value STRING) STORED AS TEXTFILE;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr.q
index 4a199365cf..9cb98aa909 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr.q
@@ -1,6 +1,6 @@
 set hive.map.aggr=true;
 set hive.groupby.skewindata=false;
-set mapred.reduce.tasks=31;
+set mapreduce.job.reduces=31;
 
 CREATE TABLE dest1(key STRING, c1 INT, c2 STRING) STORED AS TEXTFILE;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr_multi_distinct.q
index cb3ee82918..841df75af1 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr_multi_distinct.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr_multi_distinct.q
@@ -1,6 +1,6 @@
 set hive.map.aggr=true;
 set hive.groupby.skewindata=false;
-set mapred.reduce.tasks=31;
+set mapreduce.job.reduces=31;
 
 CREATE TABLE dest1(key STRING, c1 INT, c2 STRING, C3 INT, c4 INT) STORED AS TEXTFILE;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_1.q
index 7401a9ca1d..cdf4bb1cac 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_1.q
@@ -248,7 +248,7 @@ SELECT * FROM outputTbl4 ORDER BY key1, key2, key3;
 
 set hive.map.aggr=true;
 set hive.multigroupby.singlereducer=false;
-set mapred.reduce.tasks=31;
+set mapreduce.job.reduces=31;
 
 CREATE TABLE DEST1(key INT, cnt INT);
 CREATE TABLE DEST2(key INT, val STRING, cnt INT);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_skew_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_skew_1.q
index db0faa04da..1c23fad76e 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_skew_1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_skew_1.q
@@ -249,7 +249,7 @@ SELECT * FROM outputTbl4 ORDER BY key1, key2, key3;
 
 set hive.map.aggr=true;
 set hive.multigroupby.singlereducer=false;
-set mapred.reduce.tasks=31;
+set mapreduce.job.reduces=31;
 
 CREATE TABLE DEST1(key INT, cnt INT);
 CREATE TABLE DEST2(key INT, val STRING, cnt INT);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/hook_context_cs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/hook_context_cs.q
index 94ba14802f..996c9d99f0 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/hook_context_cs.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/hook_context_cs.q
@@ -5,7 +5,7 @@ ALTER TABLE vcsc ADD partition (ds='dummy') location '${system:test.tmp.dir}/Ver
 set hive.exec.pre.hooks=org.apache.hadoop.hive.ql.hooks.VerifyContentSummaryCacheHook;
 SELECT a.c, b.c FROM vcsc a JOIN vcsc b ON a.ds = 'dummy' AND b.ds = 'dummy' AND a.c = b.c;
 
-set mapred.job.tracker=local;
+set mapreduce.jobtracker.address=local;
 set hive.exec.pre.hooks = ;
 set hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.VerifyContentSummaryCacheHook;
 SELECT a.c, b.c FROM vcsc a JOIN vcsc b ON a.ds = 'dummy' AND b.ds = 'dummy' AND a.c = b.c;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q
index 728b8cc4a9..5d3c6c43c6 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q
@@ -63,7 +63,7 @@ set hive.merge.mapredfiles=true;
 set hive.merge.smallfiles.avgsize=200;
 set hive.exec.compress.output=false;
 set hive.exec.dynamic.partition=true;
-set mapred.reduce.tasks=2;
+set mapreduce.job.reduces=2;
 
 -- Tests dynamic partitions where bucketing/sorting can be inferred, but some partitions are
 -- merged and some are moved.  Currently neither should be bucketed or sorted, in the future,
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_merge.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_merge.q
index 41c1a13980..aa49b0dc64 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_merge.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_merge.q
@@ -1,7 +1,7 @@
 set hive.exec.infer.bucket.sort=true;
 set hive.exec.infer.bucket.sort.num.buckets.power.two=true;
 set hive.merge.mapredfiles=true;
-set mapred.reduce.tasks=2;
+set mapreduce.job.reduces=2;
 
 -- This tests inferring how data is bucketed/sorted from the operators in the reducer
 -- and populating that information in partitions' metadata.  In particular, those cases
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_num_buckets.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_num_buckets.q
index 2255bdb349..3a454f77bc 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_num_buckets.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_num_buckets.q
@@ -1,7 +1,7 @@
 set hive.exec.infer.bucket.sort=true;
 set hive.merge.mapfiles=false;
 set hive.merge.mapredfiles=false;
-set mapred.reduce.tasks=2;
+set mapreduce.job.reduces=2;
 
 CREATE TABLE test_table (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING);
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input12_hadoop20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input12_hadoop20.q
index 318cd378db..31e99e8d94 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input12_hadoop20.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input12_hadoop20.q
@@ -1,4 +1,4 @@
-set mapred.job.tracker=localhost:58;
+set mapreduce.jobtracker.address=localhost:58;
 set hive.exec.mode.local.auto=true;
 
 -- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20, 0.20S)
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input39_hadoop20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input39_hadoop20.q
index 29e9fae1da..362c164176 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input39_hadoop20.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input39_hadoop20.q
@@ -15,7 +15,7 @@ select key, value from src;
 
 set hive.test.mode=true;
 set hive.mapred.mode=strict;
-set mapred.job.tracker=localhost:58;
+set mapreduce.jobtracker.address=localhost:58;
 set hive.exec.mode.local.auto=true;
 
 explain
@@ -24,7 +24,7 @@ select count(1) from t1 join t2 on t1.key=t2.key where t1.ds='1' and t2.ds='1';
 select count(1) from t1 join t2 on t1.key=t2.key where t1.ds='1' and t2.ds='1';
 
 set hive.test.mode=false;
-set mapred.job.tracker;
+set mapreduce.jobtracker.address;
 
 
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_testsequencefile.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_testsequencefile.q
index d9926888ce..2b16c5cd08 100755
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_testsequencefile.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_testsequencefile.q
@@ -1,5 +1,5 @@
-set mapred.output.compress=true;
-set mapred.output.compression.type=BLOCK;
+set mapreduce.output.fileoutputformat.compress=true;
+set mapreduce.output.fileoutputformat.compress.type=BLOCK;
 
 CREATE TABLE dest4_sequencefile(key INT, value STRING) STORED AS SEQUENCEFILE;
 
@@ -10,5 +10,5 @@ INSERT OVERWRITE TABLE dest4_sequencefile SELECT src.key, src.value;
 FROM src
 INSERT OVERWRITE TABLE dest4_sequencefile SELECT src.key, src.value;
 
-set mapred.output.compress=false;
+set mapreduce.output.fileoutputformat.compress=false;
 SELECT dest4_sequencefile.* FROM dest4_sequencefile;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join14_hadoop20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join14_hadoop20.q
index a12ef1afb0..b3d75b63bd 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join14_hadoop20.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join14_hadoop20.q
@@ -2,7 +2,7 @@
 
 CREATE TABLE dest1(c1 INT, c2 STRING) STORED AS TEXTFILE;
 
-set mapred.job.tracker=localhost:58;
+set mapreduce.jobtracker.address=localhost:58;
 set hive.exec.mode.local.auto=true;
 
 EXPLAIN
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin_mr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin_mr.q
index c9ebe0e8fa..d98247b63d 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin_mr.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin_mr.q
@@ -9,7 +9,7 @@ SELECT * FROM T1;
 SELECT * FROM T2;
 
 set hive.auto.convert.join=false;
-set mapred.reduce.tasks=2;
+set mapreduce.job.reduces=2;
 
 set hive.join.emit.interval=100;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge2.q
index 8b77bd2fe1..9189e7c0d1 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge2.q
@@ -1,9 +1,9 @@
 set hive.merge.mapfiles=true;
 set hive.merge.mapredfiles=true;
-set mapred.min.split.size=256;
-set mapred.min.split.size.per.node=256;
-set mapred.min.split.size.per.rack=256;
-set mapred.max.split.size=256;
+set mapreduce.input.fileinputformat.split.minsize=256;
+set mapreduce.input.fileinputformat.split.minsize.per.node=256;
+set mapreduce.input.fileinputformat.split.minsize.per.rack=256;
+set mapreduce.input.fileinputformat.split.maxsize=256;
 
 create table test1(key int, val int);
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_createas1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_createas1.q
index 872692567b..dcb2a853ba 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_createas1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_createas1.q
@@ -1,5 +1,5 @@
-set mapred.max.split.size=100;
-set mapred.min.split.size=1;
+set mapreduce.input.fileinputformat.split.maxsize=100;
+set mapreduce.input.fileinputformat.split.minsize=1;
 
 DROP TABLE orc_createas1a;
 DROP TABLE orc_createas1b;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_char.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_char.q
index 1f5f54ae19..93f8f519cf 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_char.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_char.q
@@ -1,6 +1,6 @@
 SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
-SET mapred.min.split.size=1000;
-SET mapred.max.split.size=5000;
+SET mapreduce.input.fileinputformat.split.minsize=1000;
+SET mapreduce.input.fileinputformat.split.maxsize=5000;
 
 create table newtypesorc(c char(10), v varchar(10), d decimal(5,3), da date) stored as orc tblproperties("orc.stripe.size"="16777216"); 
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_date.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_date.q
index c34be867e4..3a74de82a4 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_date.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_date.q
@@ -1,6 +1,6 @@
 SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
-SET mapred.min.split.size=1000;
-SET mapred.max.split.size=5000;
+SET mapreduce.input.fileinputformat.split.minsize=1000;
+SET mapreduce.input.fileinputformat.split.maxsize=5000;
 
 create table newtypesorc(c char(10), v varchar(10), d decimal(5,3), da date) stored as orc tblproperties("orc.stripe.size"="16777216"); 
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_decimal.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_decimal.q
index a93590eacc..82f68a9ae5 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_decimal.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_decimal.q
@@ -1,6 +1,6 @@
 SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
-SET mapred.min.split.size=1000;
-SET mapred.max.split.size=5000;
+SET mapreduce.input.fileinputformat.split.minsize=1000;
+SET mapreduce.input.fileinputformat.split.maxsize=5000;
 
 create table newtypesorc(c char(10), v varchar(10), d decimal(5,3), da date) stored as orc tblproperties("orc.stripe.size"="16777216"); 
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_varchar.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_varchar.q
index 0fecc664e4..99f58cd73f 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_varchar.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_varchar.q
@@ -1,6 +1,6 @@
 SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
-SET mapred.min.split.size=1000;
-SET mapred.max.split.size=5000;
+SET mapreduce.input.fileinputformat.split.minsize=1000;
+SET mapreduce.input.fileinputformat.split.maxsize=5000;
 
 create table newtypesorc(c char(10), v varchar(10), d decimal(5,3), da date) stored as orc tblproperties("orc.stripe.size"="16777216"); 
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_split_elimination.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_split_elimination.q
index 54eb23e776..9aa868f9d2 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_split_elimination.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_split_elimination.q
@@ -3,8 +3,8 @@ create table orc_split_elim (userid bigint, string1 string, subtype double, deci
 load data local inpath '../../data/files/orc_split_elim.orc' into table orc_split_elim;
 
 SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
-SET mapred.min.split.size=1000;
-SET mapred.max.split.size=5000;
+SET mapreduce.input.fileinputformat.split.minsize=1000;
+SET mapreduce.input.fileinputformat.split.maxsize=5000;
 SET hive.optimize.index.filter=false;
 
 -- The above table will have 5 splits with the followings stats
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parallel.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parallel.q
index 03edeaadee..3ac6030655 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parallel.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parallel.q
@@ -1,4 +1,4 @@
-set mapred.job.name='test_parallel';
+set mapreduce.job.name='test_parallel';
 set hive.exec.parallel=true;
 set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parallel_orderby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parallel_orderby.q
index 73c3940644..777771f227 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parallel_orderby.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parallel_orderby.q
@@ -2,7 +2,7 @@ create table src5 (key string, value string);
 load data local inpath '../../data/files/kv5.txt' into table src5;
 load data local inpath '../../data/files/kv5.txt' into table src5;
 
-set mapred.reduce.tasks = 4;
+set mapreduce.job.reduces = 4;
 set hive.optimize.sampling.orderby=true;
 set hive.optimize.sampling.orderby.percent=0.66f;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_createas1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_createas1.q
index f36203724c..14e13c56b1 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_createas1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_createas1.q
@@ -1,6 +1,6 @@
 set hive.merge.rcfile.block.level=true;
-set mapred.max.split.size=100;
-set mapred.min.split.size=1;
+set mapreduce.input.fileinputformat.split.maxsize=100;
+set mapreduce.input.fileinputformat.split.minsize=1;
 
 DROP TABLE rcfile_createas1a;
 DROP TABLE rcfile_createas1b;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_lazydecompress.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_lazydecompress.q
index 7f55d10bd6..43a15a06f8 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_lazydecompress.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_lazydecompress.q
@@ -10,7 +10,7 @@ SELECT key, value FROM rcfileTableLazyDecompress where key > 238 and key < 400 O
 
 SELECT key, count(1) FROM rcfileTableLazyDecompress where key > 238 group by key ORDER BY key ASC;
 
-set mapred.output.compress=true;
+set mapreduce.output.fileoutputformat.compress=true;
 set hive.exec.compress.output=true;
 
 FROM src
@@ -22,6 +22,6 @@ SELECT key, value FROM rcfileTableLazyDecompress where key > 238 and key < 400 O
 
 SELECT key, count(1) FROM rcfileTableLazyDecompress where key > 238 group by key ORDER BY key ASC;
 
-set mapred.output.compress=false;
+set mapreduce.output.fileoutputformat.compress=false;
 set hive.exec.compress.output=false;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge1.q
index 1f6f1bd251..25071579cb 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge1.q
@@ -1,6 +1,6 @@
 set hive.merge.rcfile.block.level=false;
 set hive.exec.dynamic.partition=true;
-set mapred.max.split.size=100;
+set mapreduce.input.fileinputformat.split.maxsize=100;
 set mapref.min.split.size=1;
 
 DROP TABLE rcfile_merge1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge2.q
index 215d5ebc4a..15ffb90bf6 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge2.q
@@ -1,7 +1,7 @@
 set hive.merge.rcfile.block.level=true;
 set hive.exec.dynamic.partition=true;
-set mapred.max.split.size=100;
-set mapred.min.split.size=1;
+set mapreduce.input.fileinputformat.split.maxsize=100;
+set mapreduce.input.fileinputformat.split.minsize=1;
 
 DROP TABLE rcfile_merge2a;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge3.q
index 39fbd25646..787ab4a8d7 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge3.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge3.q
@@ -1,6 +1,6 @@
 set hive.merge.rcfile.block.level=true;
-set mapred.max.split.size=100;
-set mapred.min.split.size=1;
+set mapreduce.input.fileinputformat.split.maxsize=100;
+set mapreduce.input.fileinputformat.split.minsize=1;
 
 DROP TABLE rcfile_merge3a;
 DROP TABLE rcfile_merge3b;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge4.q
index fe6df28566..77ac381c65 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge4.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge4.q
@@ -1,6 +1,6 @@
 set hive.merge.rcfile.block.level=true;
-set mapred.max.split.size=100;
-set mapred.min.split.size=1;
+set mapreduce.input.fileinputformat.split.maxsize=100;
+set mapreduce.input.fileinputformat.split.minsize=1;
 
 DROP TABLE rcfile_merge3a;
 DROP TABLE rcfile_merge3b;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample_islocalmode_hook.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample_islocalmode_hook.q
index 12f2bcd46e..bf12ba5ed8 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample_islocalmode_hook.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample_islocalmode_hook.q
@@ -1,8 +1,8 @@
 set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
-set mapred.max.split.size=300;
-set mapred.min.split.size=300;
-set mapred.min.split.size.per.node=300;
-set mapred.min.split.size.per.rack=300;
+set mapreduce.input.fileinputformat.split.maxsize=300;
+set mapreduce.input.fileinputformat.split.minsize=300;
+set mapreduce.input.fileinputformat.split.minsize.per.node=300;
+set mapreduce.input.fileinputformat.split.minsize.per.rack=300;
 set hive.exec.mode.local.auto=true;
 set hive.merge.smallfiles.avgsize=1;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample_islocalmode_hook_hadoop20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample_islocalmode_hook_hadoop20.q
index 484e1fa617..5d1bd184d2 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample_islocalmode_hook_hadoop20.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample_islocalmode_hook_hadoop20.q
@@ -1,15 +1,15 @@
 USE default;
 
 set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
-set mapred.max.split.size=300;
-set mapred.min.split.size=300;
-set mapred.min.split.size.per.node=300;
-set mapred.min.split.size.per.rack=300;
+set mapreduce.input.fileinputformat.split.maxsize=300;
+set mapreduce.input.fileinputformat.split.minsize=300;
+set mapreduce.input.fileinputformat.split.minsize.per.node=300;
+set mapreduce.input.fileinputformat.split.minsize.per.rack=300;
 set hive.exec.mode.local.auto=true;
 set hive.merge.smallfiles.avgsize=1;
 
 -- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20, 0.20S)
--- This test sets mapred.max.split.size=300 and hive.merge.smallfiles.avgsize=1
+-- This test sets mapreduce.input.fileinputformat.split.maxsize=300 and hive.merge.smallfiles.avgsize=1
 -- in an attempt to force the generation of multiple splits and multiple output files.
 -- However, Hadoop 0.20 is incapable of generating splits smaller than the block size
 -- when using CombineFileInputFormat, so only one split is generated. This has a
@@ -25,7 +25,7 @@ create table sih_src as select key, value from sih_i_part order by key, value;
 create table sih_src2 as select key, value from sih_src order by key, value;
 
 set hive.exec.post.hooks = org.apache.hadoop.hive.ql.hooks.VerifyIsLocalModeHook ;
-set mapred.job.tracker=localhost:58;
+set mapreduce.jobtracker.address=localhost:58;
 set hive.exec.mode.local.auto.input.files.max=1;
 
 -- Sample split, running locally limited by num tasks
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/split_sample.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/split_sample.q
index 952eaf72f1..eb774f1582 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/split_sample.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/split_sample.q
@@ -1,14 +1,14 @@
 USE default;
 
 set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
-set mapred.max.split.size=300;
-set mapred.min.split.size=300;
-set mapred.min.split.size.per.node=300;
-set mapred.min.split.size.per.rack=300;
+set mapreduce.input.fileinputformat.split.maxsize=300;
+set mapreduce.input.fileinputformat.split.minsize=300;
+set mapreduce.input.fileinputformat.split.minsize.per.node=300;
+set mapreduce.input.fileinputformat.split.minsize.per.rack=300;
 set hive.merge.smallfiles.avgsize=1;
 
 -- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20)
--- This test sets mapred.max.split.size=300 and hive.merge.smallfiles.avgsize=1
+-- This test sets mapreduce.input.fileinputformat.split.maxsize=300 and hive.merge.smallfiles.avgsize=1
 -- in an attempt to force the generation of multiple splits and multiple output files.
 -- However, Hadoop 0.20 is incapable of generating splits smaller than the block size
 -- when using CombineFileInputFormat, so only one split is generated. This has a
@@ -72,10 +72,10 @@ select t1.key as k1, t2.key as k from ss_src1 tablesample(80 percent) t1 full ou
 
 -- shrink last split
 explain select count(1) from ss_src2 tablesample(1 percent);
-set mapred.max.split.size=300000;
-set mapred.min.split.size=300000;
-set mapred.min.split.size.per.node=300000;
-set mapred.min.split.size.per.rack=300000;
+set mapreduce.input.fileinputformat.split.maxsize=300000;
+set mapreduce.input.fileinputformat.split.minsize=300000;
+set mapreduce.input.fileinputformat.split.minsize.per.node=300000;
+set mapreduce.input.fileinputformat.split.minsize.per.rack=300000;
 select count(1) from ss_src2 tablesample(1 percent);
 select count(1) from ss_src2 tablesample(50 percent);
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_partscan_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_partscan_1.q
index cdf92e44cf..caf359c9e6 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_partscan_1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_partscan_1.q
@@ -2,13 +2,13 @@ set datanucleus.cache.collections=false;
 set hive.stats.autogather=false;
 set hive.exec.dynamic.partition=true;
 set hive.exec.dynamic.partition.mode=nonstrict;
-set mapred.min.split.size=256;
-set mapred.min.split.size.per.node=256;
-set mapred.min.split.size.per.rack=256;
-set mapred.max.split.size=256;
+set mapreduce.input.fileinputformat.split.minsize=256;
+set mapreduce.input.fileinputformat.split.minsize.per.node=256;
+set mapreduce.input.fileinputformat.split.minsize.per.rack=256;
+set mapreduce.input.fileinputformat.split.maxsize=256;
 
 -- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20,0.20S)
--- This test uses mapred.max.split.size/mapred.max.split.size for controlling
+-- This test uses mapreduce.input.fileinputformat.split.maxsize/mapred.max.split.size for controlling
 -- number of input splits, which is not effective in hive 0.20.
 -- stats_partscan_1_23.q is the same test with this but has different result.
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_partscan_1_23.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_partscan_1_23.q
index 1e5f360b20..07694891fd 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_partscan_1_23.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_partscan_1_23.q
@@ -2,13 +2,13 @@ set datanucleus.cache.collections=false;
 set hive.stats.autogather=false;
 set hive.exec.dynamic.partition=true;
 set hive.exec.dynamic.partition.mode=nonstrict;
-set mapred.min.split.size=256;
-set mapred.min.split.size.per.node=256;
-set mapred.min.split.size.per.rack=256;
-set mapred.max.split.size=256;
+set mapreduce.input.fileinputformat.split.minsize=256;
+set mapreduce.input.fileinputformat.split.minsize.per.node=256;
+set mapreduce.input.fileinputformat.split.minsize.per.rack=256;
+set mapreduce.input.fileinputformat.split.maxsize=256;
 
 -- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23)
--- This test uses mapred.max.split.size/mapred.max.split.size for controlling
+-- This test uses mapreduce.input.fileinputformat.split.maxsize/mapred.max.split.size for controlling
 -- number of input splits.
 -- stats_partscan_1.q is the same test with this but has different result.
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_context_ngrams.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_context_ngrams.q
index f065385688..5b5d669a7c 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_context_ngrams.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_context_ngrams.q
@@ -1,6 +1,6 @@
 CREATE TABLE kafka (contents STRING);
 LOAD DATA LOCAL INPATH '../../data/files/text-en.txt' INTO TABLE kafka;
-set mapred.reduce.tasks=1;
+set mapreduce.job.reduces=1;
 set hive.exec.reducers.max=1;
 
 SELECT context_ngrams(sentences(lower(contents)), array(null), 100, 1000).estfrequency FROM kafka;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_ngrams.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_ngrams.q
index 6a2fde52e4..39e6e30ae6 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_ngrams.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_ngrams.q
@@ -1,6 +1,6 @@
 CREATE TABLE kafka (contents STRING);
 LOAD DATA LOCAL INPATH '../../data/files/text-en.txt' INTO TABLE kafka;
-set mapred.reduce.tasks=1;
+set mapreduce.job.reduces=1;
 set hive.exec.reducers.max=1;
 
 SELECT ngrams(sentences(lower(contents)), 1, 100, 1000).estfrequency FROM kafka;
-- 
GitLab