You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by kg...@apache.org on 2019/07/15 10:41:11 UTC

[hive] branch master updated (a669ca4 -> 2350662)

This is an automated email from the ASF dual-hosted git repository.

kgyrtkirk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git.


    from a669ca4  HIVE-21920: Extract command authorisation from the Driver (Miklos Gergely, reviewed by Jesus Camacho Rodriguez)
     new 79a15ba  HIVE-21224: Upgrade tests JUnit3 to JUnit4 (Bruno Pusztahazi via Zoltan Haindrich, Laszlo Bodor)
     new 96e8e78  HIVE-21923: Vectorized MapJoin may miss results when only the join key is selected (Zoltan Haindrich reviewed by Jesus Camacho Rodriguez)
     new 0bc73f3  HIVE-21940: Metastore: Postgres text <-> clob mismatch for PARTITION_PARAMS/PARAM_VALUE (Laszlo Bodor via Naveen Gangam, Zoltan Haindrich)
     new cff3c62  HIVE-21965: Implement parallel processing in HiveStrictManagedMigration (Krisztian Kasa via Zoltan Haindrich)
     new 2350662  HIVE-21973: SHOW LOCKS prints the headers twice (Rajkumar Singh via Gopal V)

The 5 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../accumulo/TestAccumuloConnectionParameters.java |    1 +
 .../accumulo/TestAccumuloDefaultIndexScanner.java  |    1 +
 .../hadoop/hive/cli/TestCliDriverMethods.java      |   25 +-
 .../datetime/TestHiveSqlDateTimeFormatter.java     |   13 +-
 .../hive/common/format/datetime/package-info.java  |   22 -
 .../hadoop/hive/common/type/TestHiveBaseChar.java  |   12 +-
 .../hadoop/hive/conf/TestHiveAsyncLogging.java     |    9 +-
 .../hadoop/hive/conf/TestHiveConfRestrictList.java |   16 +-
 .../apache/hadoop/hive/conf/TestHiveLogging.java   |    9 +-
 .../hadoop/hive/contrib/mr/TestGenericMR.java      |   13 +-
 .../hadoop/hive/contrib/serde2/TestRegexSerDe.java |    7 +-
 .../druid/TestHiveDruidQueryBasedInputFormat.java  |    6 +-
 .../apache/hadoop/hive/hbase/TestHBaseSerDe.java   |   26 +-
 .../hadoop/hive/hbase/TestLazyHBaseObject.java     |   13 +-
 .../org/apache/hive/hcatalog/cli/TestPermsGrp.java |   23 +-
 .../apache/hive/hcatalog/cli/TestUseDatabase.java  |   16 +-
 .../hive/hcatalog/data/TestDefaultHCatRecord.java  |   14 +-
 .../hive/hcatalog/data/TestHCatRecordSerDe.java    |    7 +-
 .../apache/hive/hcatalog/data/TestJsonSerDe.java   |   14 +-
 .../hive/hcatalog/data/schema/TestHCatSchema.java  |   18 +-
 .../hcatalog/data/schema/TestHCatSchemaUtils.java  |   11 +-
 .../hcatalog/mapreduce/TestHCatOutputFormat.java   |   29 +-
 .../rcfile/TestRCFileMapReduceInputFormat.java     |    7 +-
 .../hcatalog/api/repl/TestReplicationTask.java     |   10 +-
 .../api/repl/commands/TestNoopCommand.java         |   10 +-
 .../apache/hive/hcatalog/templeton/TestDesc.java   |    9 +-
 .../apache/hive/hcatalog/templeton/TestServer.java |   18 +-
 ...BasedMetastoreAuthorizationProviderWithACL.java |    6 +-
 .../hive/metastore/TestMetaStoreAuthorization.java |   11 +-
 .../hive/metastore/TestMetastoreVersion.java       |   30 +-
 .../hive/metastore/security/TestDBTokenStore.java  |   15 +-
 .../security/TestZooKeeperTokenStore.java          |   30 +-
 .../metastore/tools/metatool/TestHiveMetaTool.java |   24 +-
 .../org/apache/hadoop/hive/ql/BaseTestQueries.java |    4 +-
 .../TestDDLWithRemoteMetastoreSecondNamenode.java  |   24 +-
 .../apache/hadoop/hive/ql/TestLocationQueries.java |    4 +
 .../org/apache/hadoop/hive/ql/TestMTQueries.java   |    3 +
 .../hadoop/hive/ql/history/TestHiveHistory.java    |   17 +-
 .../metadata/TestSemanticAnalyzerHookLoading.java  |   11 +-
 .../TestAuthorizationPreEventListener.java         |   22 +-
 .../TestClientSideAuthorizationProvider.java       |   24 +-
 .../TestMetastoreAuthorizationProvider.java        |   36 +-
 ...torageBasedClientSideAuthorizationProvider.java |    3 +
 ...StorageBasedMetastoreAuthorizationProvider.java |    2 +
 .../hive/serde2/TestSerdeWithFieldComments.java    |   10 +-
 .../hive/serde2/dynamic_type/TestDynamicSerDe.java |   17 +-
 .../hive/jdbc/TestJdbcWithMiniLlapArrow.java       |    5 +
 .../hive/ql/ddl/table/lock/ShowLocksOperation.java |   57 +-
 ...rMapJoinInnerBigOnlyGenerateResultOperator.java |    6 +-
 .../hadoop/hive/ql/util}/CloseableThreadLocal.java |   20 +-
 .../hive/ql/util/HiveStrictManagedMigration.java   |  629 ++--
 .../ql/util}/NamedForkJoinWorkerThreadFactory.java |    2 +-
 .../hadoop/hive/metastore/TestMetastoreExpr.java   |   22 +-
 .../hadoop/hive/ql/TxnCommandsBaseForTests.java    |   16 +-
 .../apache/hadoop/hive/ql/exec/TestExecDriver.java |   67 +-
 .../hive/ql/exec/TestExpressionEvaluator.java      |   15 +-
 .../hadoop/hive/ql/exec/TestFunctionRegistry.java  |   51 +-
 .../hadoop/hive/ql/exec/TestOperatorNames.java     |   25 +-
 .../apache/hadoop/hive/ql/exec/TestOperators.java  |   44 +-
 .../hive/ql/exec/TestPartitionKeySampler.java      |   39 +-
 .../org/apache/hadoop/hive/ql/exec/TestPlan.java   |   19 +-
 .../ql/exec/vector/TestVectorGroupByOperator.java  |    1 +
 .../hive/ql/exec/vector/TestVectorRowObject.java   |    9 +-
 .../hive/ql/exec/vector/TestVectorSerDeRow.java    |   23 +-
 .../ql/exec/vector/mapjoin/TestDebugDisplay.java   |    8 +-
 .../apache/hadoop/hive/ql/io/TestAcidUtils.java    |    3 +-
 .../hive/ql/io/TestCombineHiveInputFormat.java     |   62 +-
 .../ql/io/TestHiveBinarySearchRecordReader.java    |   15 +-
 .../hadoop/hive/ql/io/TestHiveFileFormatUtils.java |   10 +-
 .../hive/ql/io/TestHiveInputOutputBuffer.java      |   10 +-
 .../hive/ql/io/TestSymlinkTextInputFormat.java     |   21 +-
 .../ql/io/arrow/TestArrowColumnarBatchSerDe.java   |    2 +
 .../hive/ql/io/parquet/TestParquetSerDe.java       |   12 +-
 .../serde/TestAbstractParquetMapInspector.java     |   13 +-
 .../serde/TestArrayWritableObjectInspector.java    |    8 +-
 .../serde/TestDeepParquetHiveMapInspector.java     |   12 +-
 .../serde/TestParquetHiveArrayInspector.java       |   13 +-
 .../parquet/serde/TestParquetTimestampUtils.java   |   10 +-
 .../serde/TestStandardParquetHiveMapInspector.java |   12 +-
 .../hive/ql/lockmgr/TestEmbeddedLockManager.java   |   10 +-
 .../apache/hadoop/hive/ql/metadata/TestHive.java   |   36 +-
 .../hadoop/hive/ql/metadata/TestHiveRemote.java    |    6 +-
 .../TestDynamicMultiDimeCollection.java            |    4 +-
 .../TestListBucketingPrunner.java                  |    2 +-
 .../apache/hadoop/hive/ql/parse/TestEximUtil.java  |   17 +-
 .../hadoop/hive/ql/testutil/BaseScalarUdfTest.java |    8 +-
 .../hadoop/hive/ql/tool/TestLineageInfo.java       |   11 +-
 .../apache/hadoop/hive/ql/udf/TestToInteger.java   |    9 +-
 .../apache/hadoop/hive/ql/udf/TestUDFBase64.java   |   10 +-
 .../hadoop/hive/ql/udf/TestUDFBuildVersion.java    |    7 +-
 .../apache/hadoop/hive/ql/udf/TestUDFCrc32.java    |   13 +-
 .../hive/ql/udf/TestUDFDateFormatGranularity.java  |    9 +-
 .../org/apache/hadoop/hive/ql/udf/TestUDFHex.java  |   10 +-
 .../org/apache/hadoop/hive/ql/udf/TestUDFMd5.java  |   11 +-
 .../org/apache/hadoop/hive/ql/udf/TestUDFSha1.java |   11 +-
 .../org/apache/hadoop/hive/ql/udf/TestUDFUUID.java |    9 +-
 .../apache/hadoop/hive/ql/udf/TestUDFUnbase64.java |   10 +-
 .../apache/hadoop/hive/ql/udf/TestUDFUnhex.java    |   10 +-
 .../apache/hadoop/hive/ql/udf/TestUDFVersion.java  |   10 +-
 .../ql/udf/generic/TestGenericUDAFCorrelation.java |   11 +-
 .../hive/ql/udf/generic/TestGenericUDFAbs.java     |   24 +-
 .../ql/udf/generic/TestGenericUDFAddMonths.java    |   22 +-
 .../hive/ql/udf/generic/TestGenericUDFCbrt.java    |   12 +-
 .../hive/ql/udf/generic/TestGenericUDFConcat.java  |    8 +
 .../hive/ql/udf/generic/TestGenericUDFDate.java    |   15 +-
 .../hive/ql/udf/generic/TestGenericUDFDateAdd.java |   15 +-
 .../ql/udf/generic/TestGenericUDFDateDiff.java     |   13 +-
 .../ql/udf/generic/TestGenericUDFDateFormat.java   |   15 +-
 .../hive/ql/udf/generic/TestGenericUDFDateSub.java |   15 +-
 .../hive/ql/udf/generic/TestGenericUDFDecode.java  |   10 +-
 .../hive/ql/udf/generic/TestGenericUDFEncode.java  |   11 +-
 .../generic/TestGenericUDFEnforceConstraint.java   |   10 +-
 .../ql/udf/generic/TestGenericUDFFactorial.java    |   14 +-
 .../generic/TestGenericUDFFromUtcTimestamp.java    |   12 +-
 .../ql/udf/generic/TestGenericUDFGreatest.java     |   18 +-
 .../hive/ql/udf/generic/TestGenericUDFInitCap.java |   10 +-
 .../hive/ql/udf/generic/TestGenericUDFLTrim.java   |   10 +-
 .../hive/ql/udf/generic/TestGenericUDFLastDay.java |   13 +-
 .../hive/ql/udf/generic/TestGenericUDFLeast.java   |   18 +-
 .../ql/udf/generic/TestGenericUDFLevenshtein.java  |   16 +-
 .../hive/ql/udf/generic/TestGenericUDFLpad.java    |   11 +-
 .../udf/generic/TestGenericUDFMonthsBetween.java   |   15 +-
 .../hive/ql/udf/generic/TestGenericUDFNextDay.java |   14 +-
 .../hive/ql/udf/generic/TestGenericUDFQuarter.java |   15 +-
 .../hive/ql/udf/generic/TestGenericUDFQuote.java   |    9 +-
 .../hive/ql/udf/generic/TestGenericUDFRTrim.java   |   10 +-
 .../hive/ql/udf/generic/TestGenericUDFRegexp.java  |   15 +-
 .../hive/ql/udf/generic/TestGenericUDFReplace.java |   13 +-
 .../hive/ql/udf/generic/TestGenericUDFRpad.java    |   11 +-
 .../hive/ql/udf/generic/TestGenericUDFSha2.java    |   21 +-
 .../hive/ql/udf/generic/TestGenericUDFSoundex.java |   15 +-
 .../udf/generic/TestGenericUDFSubstringIndex.java  |   11 +-
 .../udf/generic/TestGenericUDFToUnixTimestamp.java |   13 +-
 .../hive/ql/udf/generic/TestGenericUDFTrim.java    |   10 +-
 .../hive/ql/udf/generic/TestGenericUDFTrunc.java   |   18 +-
 .../hive/ql/util/CloseableThreadLocalTest.java     |   16 +-
 .../ql/util/TestHiveStrictManagedMigration.java    |   78 +
 .../clientpositive/hybridgrace_hashjoin_2.q        |   24 +-
 .../llap/correlationoptimizer4.q.out               |    2 +-
 .../llap/hybridgrace_hashjoin_2.q.out              |  228 +-
 .../results/clientpositive/spark/auto_join14.q.out |    2 +-
 .../spark/bucket_map_join_tez1.q.out               | 3056 ++++++++++----------
 .../tez/hybridgrace_hashjoin_2.q.out               |  227 +-
 .../apache/hadoop/hive/serde2/TestStatsSerde.java  |   17 +-
 .../hive/serde2/TestTCTLSeparatedProtocol.java     |   15 +-
 .../binarysortable/TestBinarySortableFast.java     |   21 +-
 .../binarysortable/TestBinarySortableSerDe.java    |    8 +-
 .../columnar/TestLazyBinaryColumnarSerDe.java      |   16 +-
 .../hive/serde2/lazy/TestLazyArrayMapStruct.java   |   27 +-
 .../hadoop/hive/serde2/lazy/TestLazyPrimitive.java |   21 +-
 .../hive/serde2/lazy/TestLazySimpleFast.java       |   32 +-
 .../hive/serde2/lazy/TestLazySimpleSerDe.java      |    9 +-
 .../lazy/fast/TestLazySimpleDeserializeRead.java   |    8 +-
 .../hive/serde2/lazybinary/TestLazyBinaryFast.java |   25 +-
 .../serde2/lazybinary/TestLazyBinarySerDe.java     |   11 +-
 .../objectinspector/TestCrossMapEqualComparer.java |   13 +-
 .../objectinspector/TestFullMapEqualComparer.java  |   11 +-
 .../TestObjectInspectorConverters.java             |  536 ++--
 .../objectinspector/TestObjectInspectorUtils.java  |   12 +-
 .../TestProtocolBuffersObjectInspectors.java       |    8 +-
 .../TestReflectionObjectInspectors.java            |   11 +-
 .../TestSimpleMapEqualComparer.java                |   13 +-
 .../TestStandardObjectInspectors.java              |   16 +-
 .../TestThriftObjectInspectors.java                |   10 +-
 .../TestUnionStructObjectInspector.java            |    8 +-
 .../TestPrimitiveObjectInspectorFactory.java       |   11 +-
 .../TestPrimitiveObjectInspectorUtils.java         |   11 +-
 .../teradata/TestTeradataBinarySerdeForDate.java   |   11 +-
 .../TestTeradataBinarySerdeForDecimal.java         |   15 +-
 .../TestTeradataBinarySerdeForTimeStamp.java       |   12 +-
 .../teradata/TestTeradataBinarySerdeGeneral.java   |   13 +-
 .../hive/serde2/typeinfo/TestTypeInfoUtils.java    |   12 +-
 .../hive/http/TestJdbcJarDownloadServlet.java      |   12 +-
 .../org/apache/hive/service/TestCookieSigner.java  |    4 +-
 .../hive/service/auth/TestPlainSaslHelper.java     |   11 +-
 .../service/cli/session/TestSessionCleanup.java    |    7 +-
 .../cli/session/TestSessionGlobalInitFile.java     |    9 +-
 .../hive/service/cli/session/TestSessionHooks.java |    8 +-
 .../src/main/resources/package.jdo                 |    4 +-
 179 files changed, 4554 insertions(+), 2686 deletions(-)
 delete mode 100644 common/src/test/org/apache/hadoop/hive/common/format/datetime/package-info.java
 copy {upgrade-acid/pre-upgrade/src/main/java/org/apache/hadoop/hive/upgrade/acid => ql/src/java/org/apache/hadoop/hive/ql/util}/CloseableThreadLocal.java (72%)
 copy {upgrade-acid/pre-upgrade/src/main/java/org/apache/hadoop/hive/upgrade/acid => ql/src/java/org/apache/hadoop/hive/ql/util}/NamedForkJoinWorkerThreadFactory.java (97%)
 copy upgrade-acid/pre-upgrade/src/test/java/org/apache/hadoop/hive/upgrade/acid/TestCloseableThreadLocal.java => ql/src/test/org/apache/hadoop/hive/ql/util/CloseableThreadLocalTest.java (84%)
 create mode 100644 ql/src/test/org/apache/hadoop/hive/ql/util/TestHiveStrictManagedMigration.java


[hive] 01/05: HIVE-21224: Upgrade tests JUnit3 to JUnit4 (Bruno Pusztahazi via Zoltan Haindrich, Laszlo Bodor)

Posted by kg...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kgyrtkirk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git

commit 79a15baa18fcd81bd68b4d7f8c8352a6ea1fd7c3
Author: Bruno Pusztahazi <bp...@hortonworks.com>
AuthorDate: Mon Jul 15 10:59:14 2019 +0200

    HIVE-21224: Upgrade tests JUnit3 to JUnit4 (Bruno Pusztahazi via Zoltan Haindrich, Laszlo Bodor)
    
    Signed-off-by: Zoltan Haindrich <ki...@rxd.hu>
---
 .../accumulo/TestAccumuloConnectionParameters.java |   1 +
 .../accumulo/TestAccumuloDefaultIndexScanner.java  |   1 +
 .../hadoop/hive/cli/TestCliDriverMethods.java      |  25 +-
 .../datetime/TestHiveSqlDateTimeFormatter.java     |  13 +-
 .../hadoop/hive/common/type/TestHiveBaseChar.java  |  12 +-
 .../hadoop/hive/conf/TestHiveAsyncLogging.java     |   9 +-
 .../hadoop/hive/conf/TestHiveConfRestrictList.java |  16 +-
 .../apache/hadoop/hive/conf/TestHiveLogging.java   |   9 +-
 .../hadoop/hive/contrib/mr/TestGenericMR.java      |  13 +-
 .../hadoop/hive/contrib/serde2/TestRegexSerDe.java |   7 +-
 .../druid/TestHiveDruidQueryBasedInputFormat.java  |   6 +-
 .../apache/hadoop/hive/hbase/TestHBaseSerDe.java   |  26 +-
 .../hadoop/hive/hbase/TestLazyHBaseObject.java     |  13 +-
 .../org/apache/hive/hcatalog/cli/TestPermsGrp.java |  23 +-
 .../apache/hive/hcatalog/cli/TestUseDatabase.java  |  16 +-
 .../hive/hcatalog/data/TestDefaultHCatRecord.java  |  14 +-
 .../hive/hcatalog/data/TestHCatRecordSerDe.java    |   7 +-
 .../apache/hive/hcatalog/data/TestJsonSerDe.java   |  14 +-
 .../hive/hcatalog/data/schema/TestHCatSchema.java  |  18 +-
 .../hcatalog/data/schema/TestHCatSchemaUtils.java  |  11 +-
 .../hcatalog/mapreduce/TestHCatOutputFormat.java   |  29 +-
 .../rcfile/TestRCFileMapReduceInputFormat.java     |   7 +-
 .../hcatalog/api/repl/TestReplicationTask.java     |  10 +-
 .../api/repl/commands/TestNoopCommand.java         |  10 +-
 .../apache/hive/hcatalog/templeton/TestDesc.java   |   9 +-
 .../apache/hive/hcatalog/templeton/TestServer.java |  18 +-
 ...BasedMetastoreAuthorizationProviderWithACL.java |   6 +-
 .../hive/metastore/TestMetaStoreAuthorization.java |  11 +-
 .../hive/metastore/TestMetastoreVersion.java       |  30 +-
 .../hive/metastore/security/TestDBTokenStore.java  |  15 +-
 .../security/TestZooKeeperTokenStore.java          |  30 +-
 .../metastore/tools/metatool/TestHiveMetaTool.java |  24 +-
 .../org/apache/hadoop/hive/ql/BaseTestQueries.java |   4 +-
 .../TestDDLWithRemoteMetastoreSecondNamenode.java  |  24 +-
 .../apache/hadoop/hive/ql/TestLocationQueries.java |   4 +
 .../org/apache/hadoop/hive/ql/TestMTQueries.java   |   3 +
 .../hadoop/hive/ql/history/TestHiveHistory.java    |  17 +-
 .../metadata/TestSemanticAnalyzerHookLoading.java  |  11 +-
 .../TestAuthorizationPreEventListener.java         |  22 +-
 .../TestClientSideAuthorizationProvider.java       |  24 +-
 .../TestMetastoreAuthorizationProvider.java        |  36 +-
 ...torageBasedClientSideAuthorizationProvider.java |   3 +
 ...StorageBasedMetastoreAuthorizationProvider.java |   2 +
 .../hive/serde2/TestSerdeWithFieldComments.java    |  10 +-
 .../hive/serde2/dynamic_type/TestDynamicSerDe.java |  17 +-
 .../hive/jdbc/TestJdbcWithMiniLlapArrow.java       |   5 +
 .../hadoop/hive/metastore/TestMetastoreExpr.java   |  22 +-
 .../apache/hadoop/hive/ql/exec/TestExecDriver.java |  67 +--
 .../hive/ql/exec/TestExpressionEvaluator.java      |  15 +-
 .../hadoop/hive/ql/exec/TestFunctionRegistry.java  |  51 +-
 .../hadoop/hive/ql/exec/TestOperatorNames.java     |  25 +-
 .../apache/hadoop/hive/ql/exec/TestOperators.java  |  44 +-
 .../hive/ql/exec/TestPartitionKeySampler.java      |  39 +-
 .../org/apache/hadoop/hive/ql/exec/TestPlan.java   |  19 +-
 .../ql/exec/vector/TestVectorGroupByOperator.java  |   1 +
 .../hive/ql/exec/vector/TestVectorRowObject.java   |   9 +-
 .../hive/ql/exec/vector/TestVectorSerDeRow.java    |  23 +-
 .../ql/exec/vector/mapjoin/TestDebugDisplay.java   |   8 +-
 .../apache/hadoop/hive/ql/io/TestAcidUtils.java    |   3 +-
 .../hive/ql/io/TestCombineHiveInputFormat.java     |  62 +--
 .../ql/io/TestHiveBinarySearchRecordReader.java    |  15 +-
 .../hadoop/hive/ql/io/TestHiveFileFormatUtils.java |  10 +-
 .../hive/ql/io/TestHiveInputOutputBuffer.java      |  10 +-
 .../hive/ql/io/TestSymlinkTextInputFormat.java     |  21 +-
 .../ql/io/arrow/TestArrowColumnarBatchSerDe.java   |   2 +
 .../hive/ql/io/parquet/TestParquetSerDe.java       |  12 +-
 .../serde/TestAbstractParquetMapInspector.java     |  13 +-
 .../serde/TestArrayWritableObjectInspector.java    |   8 +-
 .../serde/TestDeepParquetHiveMapInspector.java     |  12 +-
 .../serde/TestParquetHiveArrayInspector.java       |  13 +-
 .../parquet/serde/TestParquetTimestampUtils.java   |  10 +-
 .../serde/TestStandardParquetHiveMapInspector.java |  12 +-
 .../hive/ql/lockmgr/TestEmbeddedLockManager.java   |  10 +-
 .../apache/hadoop/hive/ql/metadata/TestHive.java   |  36 +-
 .../hadoop/hive/ql/metadata/TestHiveRemote.java    |   6 +-
 .../TestDynamicMultiDimeCollection.java            |   4 +-
 .../TestListBucketingPrunner.java                  |   2 +-
 .../apache/hadoop/hive/ql/parse/TestEximUtil.java  |  17 +-
 .../hadoop/hive/ql/testutil/BaseScalarUdfTest.java |   8 +-
 .../hadoop/hive/ql/tool/TestLineageInfo.java       |  11 +-
 .../apache/hadoop/hive/ql/udf/TestToInteger.java   |   9 +-
 .../apache/hadoop/hive/ql/udf/TestUDFBase64.java   |  10 +-
 .../hadoop/hive/ql/udf/TestUDFBuildVersion.java    |   7 +-
 .../apache/hadoop/hive/ql/udf/TestUDFCrc32.java    |  13 +-
 .../hive/ql/udf/TestUDFDateFormatGranularity.java  |   9 +-
 .../org/apache/hadoop/hive/ql/udf/TestUDFHex.java  |  10 +-
 .../org/apache/hadoop/hive/ql/udf/TestUDFMd5.java  |  11 +-
 .../org/apache/hadoop/hive/ql/udf/TestUDFSha1.java |  11 +-
 .../org/apache/hadoop/hive/ql/udf/TestUDFUUID.java |   9 +-
 .../apache/hadoop/hive/ql/udf/TestUDFUnbase64.java |  10 +-
 .../apache/hadoop/hive/ql/udf/TestUDFUnhex.java    |  10 +-
 .../apache/hadoop/hive/ql/udf/TestUDFVersion.java  |  10 +-
 .../ql/udf/generic/TestGenericUDAFCorrelation.java |  11 +-
 .../hive/ql/udf/generic/TestGenericUDFAbs.java     |  24 +-
 .../ql/udf/generic/TestGenericUDFAddMonths.java    |  22 +-
 .../hive/ql/udf/generic/TestGenericUDFCbrt.java    |  12 +-
 .../hive/ql/udf/generic/TestGenericUDFConcat.java  |   8 +
 .../hive/ql/udf/generic/TestGenericUDFDate.java    |  15 +-
 .../hive/ql/udf/generic/TestGenericUDFDateAdd.java |  15 +-
 .../ql/udf/generic/TestGenericUDFDateDiff.java     |  13 +-
 .../ql/udf/generic/TestGenericUDFDateFormat.java   |  15 +-
 .../hive/ql/udf/generic/TestGenericUDFDateSub.java |  15 +-
 .../hive/ql/udf/generic/TestGenericUDFDecode.java  |  10 +-
 .../hive/ql/udf/generic/TestGenericUDFEncode.java  |  11 +-
 .../generic/TestGenericUDFEnforceConstraint.java   |  10 +-
 .../ql/udf/generic/TestGenericUDFFactorial.java    |  14 +-
 .../generic/TestGenericUDFFromUtcTimestamp.java    |  12 +-
 .../ql/udf/generic/TestGenericUDFGreatest.java     |  18 +-
 .../hive/ql/udf/generic/TestGenericUDFInitCap.java |  10 +-
 .../hive/ql/udf/generic/TestGenericUDFLTrim.java   |  10 +-
 .../hive/ql/udf/generic/TestGenericUDFLastDay.java |  13 +-
 .../hive/ql/udf/generic/TestGenericUDFLeast.java   |  18 +-
 .../ql/udf/generic/TestGenericUDFLevenshtein.java  |  16 +-
 .../hive/ql/udf/generic/TestGenericUDFLpad.java    |  11 +-
 .../udf/generic/TestGenericUDFMonthsBetween.java   |  15 +-
 .../hive/ql/udf/generic/TestGenericUDFNextDay.java |  14 +-
 .../hive/ql/udf/generic/TestGenericUDFQuarter.java |  15 +-
 .../hive/ql/udf/generic/TestGenericUDFQuote.java   |   9 +-
 .../hive/ql/udf/generic/TestGenericUDFRTrim.java   |  10 +-
 .../hive/ql/udf/generic/TestGenericUDFRegexp.java  |  15 +-
 .../hive/ql/udf/generic/TestGenericUDFReplace.java |  13 +-
 .../hive/ql/udf/generic/TestGenericUDFRpad.java    |  11 +-
 .../hive/ql/udf/generic/TestGenericUDFSha2.java    |  21 +-
 .../hive/ql/udf/generic/TestGenericUDFSoundex.java |  15 +-
 .../udf/generic/TestGenericUDFSubstringIndex.java  |  11 +-
 .../udf/generic/TestGenericUDFToUnixTimestamp.java |  13 +-
 .../hive/ql/udf/generic/TestGenericUDFTrim.java    |  10 +-
 .../hive/ql/udf/generic/TestGenericUDFTrunc.java   |  18 +-
 .../apache/hadoop/hive/serde2/TestStatsSerde.java  |  17 +-
 .../hive/serde2/TestTCTLSeparatedProtocol.java     |  15 +-
 .../binarysortable/TestBinarySortableFast.java     |  21 +-
 .../binarysortable/TestBinarySortableSerDe.java    |   8 +-
 .../columnar/TestLazyBinaryColumnarSerDe.java      |  16 +-
 .../hive/serde2/lazy/TestLazyArrayMapStruct.java   |  27 +-
 .../hadoop/hive/serde2/lazy/TestLazyPrimitive.java |  21 +-
 .../hive/serde2/lazy/TestLazySimpleFast.java       |  32 +-
 .../hive/serde2/lazy/TestLazySimpleSerDe.java      |   9 +-
 .../lazy/fast/TestLazySimpleDeserializeRead.java   |   8 +-
 .../hive/serde2/lazybinary/TestLazyBinaryFast.java |  25 +-
 .../serde2/lazybinary/TestLazyBinarySerDe.java     |  11 +-
 .../objectinspector/TestCrossMapEqualComparer.java |  13 +-
 .../objectinspector/TestFullMapEqualComparer.java  |  11 +-
 .../TestObjectInspectorConverters.java             | 536 +++++++++++----------
 .../objectinspector/TestObjectInspectorUtils.java  |  12 +-
 .../TestProtocolBuffersObjectInspectors.java       |   8 +-
 .../TestReflectionObjectInspectors.java            |  11 +-
 .../TestSimpleMapEqualComparer.java                |  13 +-
 .../TestStandardObjectInspectors.java              |  16 +-
 .../TestThriftObjectInspectors.java                |  10 +-
 .../TestUnionStructObjectInspector.java            |   8 +-
 .../TestPrimitiveObjectInspectorFactory.java       |  11 +-
 .../TestPrimitiveObjectInspectorUtils.java         |  11 +-
 .../teradata/TestTeradataBinarySerdeForDate.java   |  11 +-
 .../TestTeradataBinarySerdeForDecimal.java         |  15 +-
 .../TestTeradataBinarySerdeForTimeStamp.java       |  12 +-
 .../teradata/TestTeradataBinarySerdeGeneral.java   |  13 +-
 .../hive/serde2/typeinfo/TestTypeInfoUtils.java    |  12 +-
 .../hive/http/TestJdbcJarDownloadServlet.java      |  12 +-
 .../org/apache/hive/service/TestCookieSigner.java  |   4 +-
 .../hive/service/auth/TestPlainSaslHelper.java     |  11 +-
 .../service/cli/session/TestSessionCleanup.java    |   7 +-
 .../cli/session/TestSessionGlobalInitFile.java     |   9 +-
 .../hive/service/cli/session/TestSessionHooks.java |   8 +-
 163 files changed, 2046 insertions(+), 805 deletions(-)

diff --git a/accumulo-handler/src/test/org/apache/hadoop/hive/accumulo/TestAccumuloConnectionParameters.java b/accumulo-handler/src/test/org/apache/hadoop/hive/accumulo/TestAccumuloConnectionParameters.java
index 23be5f1..be96e60 100644
--- a/accumulo-handler/src/test/org/apache/hadoop/hive/accumulo/TestAccumuloConnectionParameters.java
+++ b/accumulo-handler/src/test/org/apache/hadoop/hive/accumulo/TestAccumuloConnectionParameters.java
@@ -100,6 +100,7 @@ public class TestAccumuloConnectionParameters {
     cnxnParams.getConnector(instance);
   }
 
+  @Test
   public void testSasl() {
     Configuration conf = new Configuration(false);
 
diff --git a/accumulo-handler/src/test/org/apache/hadoop/hive/accumulo/TestAccumuloDefaultIndexScanner.java b/accumulo-handler/src/test/org/apache/hadoop/hive/accumulo/TestAccumuloDefaultIndexScanner.java
index f3e7902..533700c 100644
--- a/accumulo-handler/src/test/org/apache/hadoop/hive/accumulo/TestAccumuloDefaultIndexScanner.java
+++ b/accumulo-handler/src/test/org/apache/hadoop/hive/accumulo/TestAccumuloDefaultIndexScanner.java
@@ -137,6 +137,7 @@ public class TestAccumuloDefaultIndexScanner {
     assertTrue("does not contain row3", ranges.contains(new Range("row3")));
   }
 
+  @Test
   public void testTooManyMatches() {
     AccumuloDefaultIndexScanner handler = buildMockHandler(2);
     List<Range> ranges = handler.getIndexRowRanges("age", new Range("10", "50"));
diff --git a/cli/src/test/org/apache/hadoop/hive/cli/TestCliDriverMethods.java b/cli/src/test/org/apache/hadoop/hive/cli/TestCliDriverMethods.java
index debcc7a..a3ea326 100644
--- a/cli/src/test/org/apache/hadoop/hive/cli/TestCliDriverMethods.java
+++ b/cli/src/test/org/apache/hadoop/hive/cli/TestCliDriverMethods.java
@@ -43,7 +43,7 @@ import java.util.Map;
 import jline.console.ConsoleReader;
 import jline.console.completer.ArgumentCompleter;
 import jline.console.completer.Completer;
-import junit.framework.TestCase;
+
 
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -56,29 +56,39 @@ import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.QueryState;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.junit.Test;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import org.junit.Before;
+import org.junit.After;
 
 
 // Cannot call class TestCliDriver since that's the name of the generated
 // code for the script-based testing
-public class TestCliDriverMethods extends TestCase {
+/**
+ * TestCliDriverMethods.
+ */
+public class TestCliDriverMethods {
 
   SecurityManager securityManager;
 
   // Some of these tests require intercepting System.exit() using the SecurityManager.
   // It is safer to  register/unregister our SecurityManager during setup/teardown instead
   // of doing it within the individual test cases.
-  @Override
+  @Before
   public void setUp() {
     securityManager = System.getSecurityManager();
     System.setSecurityManager(new NoExitSecurityManager(securityManager));
   }
 
-  @Override
+  @After
   public void tearDown() {
     System.setSecurityManager(securityManager);
   }
 
   // If the command has an associated schema, make sure it gets printed to use
+  @Test
   public void testThatCliDriverPrintsHeaderForCommandsWithSchema() {
     Schema mockSchema = mock(Schema.class);
     List<FieldSchema> fieldSchemas = new ArrayList<FieldSchema>();
@@ -93,6 +103,7 @@ public class TestCliDriverMethods extends TestCase {
   }
 
   // If the command has no schema, make sure nothing is printed
+  @Test
   public void testThatCliDriverPrintsNoHeaderForCommandsWithNoSchema() {
     Schema mockSchema = mock(Schema.class);
     when(mockSchema.getFieldSchemas()).thenReturn(null);
@@ -103,6 +114,7 @@ public class TestCliDriverMethods extends TestCase {
   }
 
   // Test that CliDriver does not strip comments starting with '--'
+  @Test
   public void testThatCliDriverDoesNotStripComments() throws Exception {
     // We need to overwrite System.out and System.err as that is what is used in ShellCmdExecutor
     // So save old values...
@@ -185,6 +197,7 @@ public class TestCliDriverMethods extends TestCase {
   }
 
 
+  @Test
   public void testGetCommandCompletor() {
     Completer[] completors = CliDriver.getCommandCompleter();
     assertEquals(2, completors.length);
@@ -205,6 +218,7 @@ public class TestCliDriverMethods extends TestCase {
 
   }
 
+  @Test
   public void testRun() throws Exception {
     // clean history
     String historyDirectory = System.getProperty("user.home");
@@ -243,6 +257,7 @@ public class TestCliDriverMethods extends TestCase {
   /**
    * Test commands exit and quit
    */
+  @Test
   public void testQuit() throws Exception {
 
     CliSessionState ss = new CliSessionState(new HiveConf());
@@ -273,6 +288,7 @@ public class TestCliDriverMethods extends TestCase {
 
   }
 
+  @Test
   public void testProcessSelectDatabase() throws Exception {
     CliSessionState sessinState = new CliSessionState(new HiveConf());
     CliSessionState.start(sessinState);
@@ -293,6 +309,7 @@ public class TestCliDriverMethods extends TestCase {
         "FAILED: ParseException line 1:4 cannot recognize input near 'database'"));
   }
 
+  @Test
   public void testprocessInitFiles() throws Exception {
     String oldHiveHome = System.getenv("HIVE_HOME");
     String oldHiveConfDir = System.getenv("HIVE_CONF_DIR");
diff --git a/common/src/test/org/apache/hadoop/hive/common/format/datetime/TestHiveSqlDateTimeFormatter.java b/common/src/test/org/apache/hadoop/hive/common/format/datetime/TestHiveSqlDateTimeFormatter.java
index 4e822d5..ac57842 100644
--- a/common/src/test/org/apache/hadoop/hive/common/format/datetime/TestHiveSqlDateTimeFormatter.java
+++ b/common/src/test/org/apache/hadoop/hive/common/format/datetime/TestHiveSqlDateTimeFormatter.java
@@ -41,14 +41,19 @@ import static java.time.temporal.ChronoField.MONTH_OF_YEAR;
 import static java.time.temporal.ChronoField.SECOND_OF_MINUTE;
 import static java.time.temporal.ChronoField.YEAR;
 
+import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
 /**
  * Tests HiveSqlDateTimeFormatter.
  */
 
-public class TestHiveSqlDateTimeFormatter extends TestCase {
+public class TestHiveSqlDateTimeFormatter {
 
   private HiveSqlDateTimeFormatter formatter;
 
+  @Test
   public void testSetPattern() {
     verifyPatternParsing(" ---yyyy-\'-:-  -,.;/MM-dd--", new ArrayList<>(List.of(
         null, // represents separator, which has no temporal field
@@ -73,6 +78,7 @@ public class TestHiveSqlDateTimeFormatter extends TestCase {
     )));
   }
 
+  @Test
   public void testSetPatternWithBadPatterns() {
     verifyBadPattern("eyyyy-ddd", true);
     verifyBadPattern("1yyyy-mm-dd", true);
@@ -102,6 +108,7 @@ public class TestHiveSqlDateTimeFormatter extends TestCase {
     verifyBadPattern("tzh", false);
   }
 
+  @Test
   public void testFormatTimestamp() {
     checkFormatTs("rr rrrr ddd", "2018-01-03 00:00:00", "18 2018 003");
     checkFormatTs("yyyy-mm-ddtsssss.ff4z", "2018-02-03 00:00:10.777777777", "2018-02-03T00010.7777Z");
@@ -118,6 +125,7 @@ public class TestHiveSqlDateTimeFormatter extends TestCase {
     assertEquals(expectedOutput, formatter.format(toTimestamp(input)));
   }
 
+  @Test
   public void testFormatDate() {
     checkFormatDate("rr rrrr ddd", "2018-01-03", "18 2018 003");
     checkFormatDate("yyyy-mm-ddtsssss.ff4z", "2018-02-03", "2018-02-03T00000.0000Z");
@@ -133,6 +141,7 @@ public class TestHiveSqlDateTimeFormatter extends TestCase {
     assertEquals(expectedOutput, formatter.format(toDate(input)));
   }
 
+  @Test
   public void testParseTimestamp() {
     String thisYearString = String.valueOf(LocalDateTime.now().getYear());
     int firstTwoDigits = getFirstTwoDigits();
@@ -205,6 +214,7 @@ public class TestHiveSqlDateTimeFormatter extends TestCase {
     assertEquals(toTimestamp(expectedOutput), formatter.parseTimestamp(input));
   }
 
+  @Test
   public void testParseDate() {
 
     String thisYearString = String.valueOf(LocalDateTime.now().getYear());
@@ -235,6 +245,7 @@ public class TestHiveSqlDateTimeFormatter extends TestCase {
     assertEquals(toDate(expectedOutput), formatter.parseDate(input));
   }
 
+  @Test
   public void testParseTimestampError() {
     verifyBadParseString("yyyy", "2019-02-03");
     verifyBadParseString("yyyy-mm-dd  ", "2019-02-03"); //separator missing
diff --git a/common/src/test/org/apache/hadoop/hive/common/type/TestHiveBaseChar.java b/common/src/test/org/apache/hadoop/hive/common/type/TestHiveBaseChar.java
index 941e9e3..fd73274 100644
--- a/common/src/test/org/apache/hadoop/hive/common/type/TestHiveBaseChar.java
+++ b/common/src/test/org/apache/hadoop/hive/common/type/TestHiveBaseChar.java
@@ -20,9 +20,15 @@ package org.apache.hadoop.hive.common.type;
 
 import java.util.Random;
 
-import junit.framework.TestCase;
 
-public class TestHiveBaseChar extends TestCase {
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import org.junit.Test;
+
+/**
+ * HiveBaseChar Test.
+ */
+public class TestHiveBaseChar {
   static Random rnd = new Random();
 
   public static int getRandomSupplementaryChar() {
@@ -62,6 +68,7 @@ public class TestHiveBaseChar extends TestCase {
     return sb.toString();
   }
 
+  @Test
   public void testStringLength() throws Exception {
     int strLen = 20;
     int[] lengths = { 15, 20, 25 };
@@ -83,6 +90,7 @@ public class TestHiveBaseChar extends TestCase {
     assertNull(HiveBaseChar.enforceMaxLength(null, 0));
   }
 
+  @Test
   public void testGetPaddedValue() {
     int strLen = 20;
     int[] lengths = { 15, 20, 25 };
diff --git a/common/src/test/org/apache/hadoop/hive/conf/TestHiveAsyncLogging.java b/common/src/test/org/apache/hadoop/hive/conf/TestHiveAsyncLogging.java
index bb23882..b2f1bb3 100644
--- a/common/src/test/org/apache/hadoop/hive/conf/TestHiveAsyncLogging.java
+++ b/common/src/test/org/apache/hadoop/hive/conf/TestHiveAsyncLogging.java
@@ -24,11 +24,14 @@ import org.apache.logging.log4j.core.async.AsyncLoggerContextSelector;
 import org.apache.logging.log4j.core.impl.Log4jContextFactory;
 import org.apache.logging.log4j.core.selector.ClassLoaderContextSelector;
 import org.apache.logging.log4j.core.selector.ContextSelector;
-import org.junit.Test;
 
-import junit.framework.TestCase;
+import static org.junit.Assert.assertTrue;
+import org.junit.Test;
 
-public class TestHiveAsyncLogging extends TestCase {
+/**
+ * HiveAsyncLogging Test.
+ */
+public class TestHiveAsyncLogging {
 
   // this test requires disruptor jar in classpath
   @Test
diff --git a/common/src/test/org/apache/hadoop/hive/conf/TestHiveConfRestrictList.java b/common/src/test/org/apache/hadoop/hive/conf/TestHiveConfRestrictList.java
index eb2f964..fd41c5b 100644
--- a/common/src/test/org/apache/hadoop/hive/conf/TestHiveConfRestrictList.java
+++ b/common/src/test/org/apache/hadoop/hive/conf/TestHiveConfRestrictList.java
@@ -17,18 +17,24 @@
  */
 package org.apache.hadoop.hive.conf;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import org.junit.Before;
 import org.junit.Test;
 
-public class TestHiveConfRestrictList extends TestCase {
+/**
+ * HiveConfRestrictList Test.
+ */
+public class TestHiveConfRestrictList {
 
   private HiveConf conf = null;
 
-  @Override
-  protected void setUp() throws Exception {
-    super.setUp();
+  @Before
+  public void setUp() throws Exception {
+
     System.setProperty(ConfVars.HIVE_CONF_RESTRICTED_LIST.varname,
         ConfVars.HIVETESTMODEPREFIX.varname);
     conf = new HiveConf();
diff --git a/common/src/test/org/apache/hadoop/hive/conf/TestHiveLogging.java b/common/src/test/org/apache/hadoop/hive/conf/TestHiveLogging.java
index f9c3283..ab56505 100644
--- a/common/src/test/org/apache/hadoop/hive/conf/TestHiveLogging.java
+++ b/common/src/test/org/apache/hadoop/hive/conf/TestHiveLogging.java
@@ -23,7 +23,11 @@ import org.apache.hadoop.hive.common.LogUtils;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hive.common.util.HiveTestUtils;
 
-import junit.framework.TestCase;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import org.junit.Test;
 
 /**
  * TestHiveLogging
@@ -31,7 +35,7 @@ import junit.framework.TestCase;
  * Test cases for HiveLogging, which is initialized in HiveConf.
  * Loads configuration files located in common/src/test/resources.
  */
-public class TestHiveLogging extends TestCase {
+public class TestHiveLogging {
   public TestHiveLogging() {
     super();
   }
@@ -74,6 +78,7 @@ public class TestHiveLogging extends TestCase {
     assertTrue(logFile + " should exist", logFile.exists());
   }
 
+  @Test
   public void testHiveLogging() throws Exception {
     // customized log4j config log file to be: /${test.tmp.dir}/TestHiveLogging/hiveLog4jTest.log
     File customLogPath = new File(new File(System.getProperty("test.tmp.dir")),
diff --git a/contrib/src/test/org/apache/hadoop/hive/contrib/mr/TestGenericMR.java b/contrib/src/test/org/apache/hadoop/hive/contrib/mr/TestGenericMR.java
index 5bc06e8..ffffda5 100644
--- a/contrib/src/test/org/apache/hadoop/hive/contrib/mr/TestGenericMR.java
+++ b/contrib/src/test/org/apache/hadoop/hive/contrib/mr/TestGenericMR.java
@@ -22,15 +22,19 @@ import java.io.StringWriter;
 import java.util.Iterator;
 import java.util.NoSuchElementException;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.util.Shell;
+import static org.junit.Assert.fail;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
 /**
  * TestGenericMR.
  *
  */
-public final class TestGenericMR extends TestCase {
+public final class TestGenericMR {
+  @Test
   public void testReduceTooFar() throws Exception {
     try {
       new GenericMR().reduce(new StringReader("a\tb\tc"), new StringWriter(),
@@ -50,6 +54,7 @@ public final class TestGenericMR extends TestCase {
     fail("Expected NoSuchElementException");
   }
 
+  @Test
   public void testEmptyMap() throws Exception {
     final StringWriter out = new StringWriter();
 
@@ -58,6 +63,7 @@ public final class TestGenericMR extends TestCase {
     assertEquals(0, out.toString().length());
   }
 
+  @Test
   public void testIdentityMap() throws Exception {
     final String in = "a\tb\nc\td";
     final StringWriter out = new StringWriter();
@@ -66,6 +72,7 @@ public final class TestGenericMR extends TestCase {
     assertEquals(in + "\n", out.toString());
   }
 
+  @Test
   public void testKVSplitMap() throws Exception {
     final String in = "k1=v1,k2=v2\nk1=v2,k2=v3";
     final String expected = "k1\tv1\nk2\tv2\nk1\tv2\nk2\tv3\n";
@@ -83,6 +90,7 @@ public final class TestGenericMR extends TestCase {
     assertEquals(expected, out.toString());
   }
 
+  @Test
   public void testIdentityReduce() throws Exception {
     final String in = "a\tb\nc\td";
     final StringWriter out = new StringWriter();
@@ -92,6 +100,7 @@ public final class TestGenericMR extends TestCase {
     assertEquals(in + "\n", out.toString());
   }
 
+  @Test
   public void testWordCountReduce() throws Exception {
     final String in = "hello\t1\nhello\t2\nokay\t4\nokay\t6\nokay\t2";
     final StringWriter out = new StringWriter();
diff --git a/contrib/src/test/org/apache/hadoop/hive/contrib/serde2/TestRegexSerDe.java b/contrib/src/test/org/apache/hadoop/hive/contrib/serde2/TestRegexSerDe.java
index 06d8aa5..2bca0a1 100644
--- a/contrib/src/test/org/apache/hadoop/hive/contrib/serde2/TestRegexSerDe.java
+++ b/contrib/src/test/org/apache/hadoop/hive/contrib/serde2/TestRegexSerDe.java
@@ -19,7 +19,7 @@ package org.apache.hadoop.hive.contrib.serde2;
 
 import java.util.Properties;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.serde.serdeConstants;
@@ -29,12 +29,14 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption;
 import org.apache.hadoop.io.Text;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
 /**
  * TestRegexSerDe.
  *
  */
-public class TestRegexSerDe extends TestCase {
+public class TestRegexSerDe {
 
   private AbstractSerDe createSerDe(String fieldNames, String fieldTypes,
       String inputRegex, String outputFormatString) throws Throwable {
@@ -52,6 +54,7 @@ public class TestRegexSerDe extends TestCase {
   /**
    * Test the LazySimpleSerDe class.
    */
+  @Test
   public void testRegexSerDe() throws Throwable {
     try {
       // Create the SerDe
diff --git a/druid-handler/src/test/org/apache/hadoop/hive/druid/TestHiveDruidQueryBasedInputFormat.java b/druid-handler/src/test/org/apache/hadoop/hive/druid/TestHiveDruidQueryBasedInputFormat.java
index 591cbe8..58f4a44 100644
--- a/druid-handler/src/test/org/apache/hadoop/hive/druid/TestHiveDruidQueryBasedInputFormat.java
+++ b/druid-handler/src/test/org/apache/hadoop/hive/druid/TestHiveDruidQueryBasedInputFormat.java
@@ -28,12 +28,13 @@ import org.apache.hadoop.hive.druid.io.HiveDruidSplit;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 
 import org.apache.druid.query.Query;
-import junit.framework.TestCase;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
 /**
  * Test Class.
  */
-@SuppressWarnings("SameParameterValue") public class TestHiveDruidQueryBasedInputFormat extends TestCase {
+@SuppressWarnings("SameParameterValue") public class TestHiveDruidQueryBasedInputFormat {
 
   private static final String
       TIMESERIES_QUERY =
@@ -181,6 +182,7 @@ import junit.framework.TestCase;
           + "\"pagingSpec\":{\"pagingIdentifiers\":{},\"threshold\":5,\"fromNext\":false},"
           + "\"context\":{\"druid.query.fetch\":true,\"queryId\":\"\"}}, [localhost:8082]}]";
 
+  @Test
   public void testTimeZone() throws Exception {
     DruidQueryBasedInputFormat input = new DruidQueryBasedInputFormat();
 
diff --git a/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestHBaseSerDe.java b/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestHBaseSerDe.java
index 59468fe..6e08d4e 100644
--- a/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestHBaseSerDe.java
+++ b/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestHBaseSerDe.java
@@ -71,11 +71,16 @@ import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.thrift.TException;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
+import static org.junit.Assert.assertNull;
+import org.junit.Test;
 
 /**
  * Tests the HBaseSerDe class.
  */
-public class TestHBaseSerDe extends TestCase {
+public class TestHBaseSerDe {
 
   static final byte[] TEST_BYTE_ARRAY = Bytes.toBytes("test");
 
@@ -161,6 +166,7 @@ public class TestHBaseSerDe extends TestCase {
   /**
    * Test the default behavior of the Lazy family of objects and object inspectors.
    */
+  @Test
   public void testHBaseSerDeI() throws SerDeException {
 
     byte [] cfa = "cola".getBytes();
@@ -246,6 +252,7 @@ public class TestHBaseSerDe extends TestCase {
     deserializeAndSerialize(serDe, r, p, expectedFieldsData);
   }
 
+  @Test
   public void testHBaseSerDeWithTimestamp() throws SerDeException {
     // Create the SerDe
     HBaseSerDe serDe = new HBaseSerDe();
@@ -401,6 +408,7 @@ public class TestHBaseSerDe extends TestCase {
     return tbl;
   }
 
+  @Test
   public void testHBaseSerDeII() throws SerDeException {
 
     byte [] cfa = "cfa".getBytes();
@@ -526,6 +534,7 @@ public class TestHBaseSerDe extends TestCase {
     return tbl;
   }
 
+  @Test
   public void testHBaseSerDeWithHiveMapToHBaseColumnFamily() throws SerDeException {
 
     byte [] cfint = "cf-int".getBytes();
@@ -680,6 +689,7 @@ public class TestHBaseSerDe extends TestCase {
     return tbl;
   }
 
+  @Test
   public void testHBaseSerDeWithHiveMapToHBaseColumnFamilyII() throws SerDeException {
 
     byte [] cfbyte = "cf-byte".getBytes();
@@ -800,6 +810,7 @@ public class TestHBaseSerDe extends TestCase {
     assertEquals("Serialized data: ", p.toString(), serializedPut.toString());
   }
 
+  @Test
   public void testHBaseSerDeWithColumnPrefixes()
       throws Exception {
     byte[] cfa = "cola".getBytes();
@@ -919,6 +930,7 @@ public class TestHBaseSerDe extends TestCase {
     }
   }
 
+  @Test
   public void testHBaseSerDeCompositeKeyWithSeparator() throws SerDeException, TException,
       IOException {
     byte[] cfa = "cola".getBytes();
@@ -967,6 +979,7 @@ public class TestHBaseSerDe extends TestCase {
     return tbl;
   }
 
+  @Test
   public void testHBaseSerDeCompositeKeyWithoutSeparator() throws SerDeException, TException,
       IOException {
     byte[] cfa = "cola".getBytes();
@@ -1041,6 +1054,7 @@ public class TestHBaseSerDe extends TestCase {
     assertEquals("Serialized put:", p.toString(), put.toString());
   }
 
+  @Test
   public void testHBaseSerDeWithAvroSchemaInline() throws SerDeException, IOException {
     byte[] cfa = "cola".getBytes();
 
@@ -1086,6 +1100,7 @@ public class TestHBaseSerDe extends TestCase {
     return tbl;
   }
 
+  @Test
   public void testHBaseSerDeWithForwardEvolvedSchema() throws SerDeException, IOException {
     byte[] cfa = "cola".getBytes();
 
@@ -1132,6 +1147,7 @@ public class TestHBaseSerDe extends TestCase {
     return tbl;
   }
 
+  @Test
   public void testHBaseSerDeWithBackwardEvolvedSchema() throws SerDeException, IOException {
     byte[] cfa = "cola".getBytes();
 
@@ -1177,6 +1193,7 @@ public class TestHBaseSerDe extends TestCase {
     return tbl;
   }
 
+  @Test
   public void testHBaseSerDeWithAvroSerClass() throws SerDeException, IOException {
     byte[] cfa = "cola".getBytes();
 
@@ -1228,6 +1245,7 @@ public class TestHBaseSerDe extends TestCase {
     return tbl;
   }
 
+  @Test
   public void testHBaseSerDeWithAvroSchemaUrl() throws SerDeException, IOException {
     byte[] cfa = "cola".getBytes();
 
@@ -1292,6 +1310,7 @@ public class TestHBaseSerDe extends TestCase {
     return tbl;
   }
 
+  @Test
   public void testHBaseSerDeWithAvroExternalSchema() throws SerDeException, IOException {
     byte[] cfa = "cola".getBytes();
 
@@ -1345,6 +1364,7 @@ public class TestHBaseSerDe extends TestCase {
     return tbl;
   }
 
+  @Test
   public void testHBaseSerDeWithHiveMapToHBaseAvroColumnFamily() throws Exception {
     byte[] cfa = "cola".getBytes();
 
@@ -1420,6 +1440,7 @@ public class TestHBaseSerDe extends TestCase {
     return tbl;
   }
 
+  @Test
   public void testHBaseSerDeCustomStructValue() throws IOException, SerDeException {
 
     byte[] cfa = "cola".getBytes();
@@ -1458,6 +1479,7 @@ public class TestHBaseSerDe extends TestCase {
    * and not the error in a production setup. The Properties.java object that is passed to the serDe
    * initializer, is passed with empty value "" for "columns.comments" key for hbase backed tables.
    */
+  @Test
   public void testEmptyColumnComment() throws SerDeException {
     HBaseSerDe serDe = new HBaseSerDe();
     Properties tbl = createPropertiesForValueStruct();
@@ -1525,7 +1547,7 @@ public class TestHBaseSerDe extends TestCase {
       assertNotNull(fieldData);
       assertEquals(expectedFieldsData[j], fieldData.toString().trim());
     }
-    
+
     assertEquals(expectedDeserializedAvroString, SerDeUtils.getJSONString(row, soi));
 
     // Now serialize
diff --git a/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestLazyHBaseObject.java b/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestLazyHBaseObject.java
index e0cf162..5d6302b 100644
--- a/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestLazyHBaseObject.java
+++ b/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestLazyHBaseObject.java
@@ -52,14 +52,20 @@ import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.fail;
+import org.junit.Test;
+
 /**
  * TestLazyHBaseObject is a test for the LazyHBaseXXX classes.
  */
-public class TestLazyHBaseObject extends TestCase {
+public class TestLazyHBaseObject {
   /**
    * Test the LazyMap class with Integer-to-String.
    * @throws SerDeException
    */
+  @Test
   public void testLazyHBaseCellMap1() throws SerDeException {
     // Map of Integer to String
     Text nullSequence = new Text("\\N");
@@ -122,6 +128,7 @@ public class TestLazyHBaseObject extends TestCase {
    * Test the LazyMap class with String-to-String.
    * @throws SerDeException
    */
+  @Test
   public void testLazyHBaseCellMap2() throws SerDeException {
     // Map of String to String
     Text nullSequence = new Text("\\N");
@@ -185,6 +192,7 @@ public class TestLazyHBaseObject extends TestCase {
    * map are stored in binary format using the appropriate LazyPrimitive objects.
    * @throws SerDeException
    */
+  @Test
   public void testLazyHBaseCellMap3() throws SerDeException {
 
     Text nullSequence = new Text("\\N");
@@ -456,6 +464,7 @@ public class TestLazyHBaseObject extends TestCase {
    * Hive fields and HBase columns.
    * @throws SerDeException
    */
+  @Test
   public void testLazyHBaseRow1() throws SerDeException {
     List<TypeInfo> fieldTypeInfos =
       TypeInfoUtils.getTypeInfosFromTypeString(
@@ -578,6 +587,7 @@ public class TestLazyHBaseObject extends TestCase {
    * an HBase column family.
    * @throws SerDeException
    */
+  @Test
   public void testLazyHBaseRow2() throws SerDeException {
     // column family is mapped to Map<string,string>
     List<TypeInfo> fieldTypeInfos =
@@ -700,6 +710,7 @@ public class TestLazyHBaseObject extends TestCase {
    * are stored in binary format in HBase.
    * @throws SerDeException
    */
+  @Test
   public void testLazyHBaseRow3() throws SerDeException {
 
     List<TypeInfo> fieldTypeInfos = TypeInfoUtils.getTypeInfosFromTypeString(
diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestPermsGrp.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestPermsGrp.java
index 3cf172b..2b57d8d 100644
--- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestPermsGrp.java
+++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestPermsGrp.java
@@ -59,9 +59,19 @@ import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import junit.framework.TestCase;
 
-public class TestPermsGrp extends TestCase {
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.fail;
+import static org.junit.Assert.assertTrue;
+import org.junit.Before;
+import org.junit.After;
+import org.junit.Test;
+
+/**
+ * TestPermsGrp.
+ */
+public class TestPermsGrp {
 
   private boolean isServerRunning = false;
   private HiveConf hcatConf;
@@ -69,13 +79,13 @@ public class TestPermsGrp extends TestCase {
   private HiveMetaStoreClient msc;
   private static final Logger LOG = LoggerFactory.getLogger(TestPermsGrp.class);
 
-  @Override
-  protected void tearDown() throws Exception {
+  @After
+  public void tearDown() throws Exception {
     System.setSecurityManager(securityManager);
   }
 
-  @Override
-  protected void setUp() throws Exception {
+  @Before
+  public void setUp() throws Exception {
 
     if (isServerRunning) {
       return;
@@ -104,6 +114,7 @@ public class TestPermsGrp extends TestCase {
     msc = new HiveMetaStoreClient(hcatConf);
   }
 
+  @Test
   public void testCustomPerms() throws Exception {
 
     String dbName = Warehouse.DEFAULT_DATABASE_NAME;
diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestUseDatabase.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestUseDatabase.java
index a8aafb1..c4896fe 100644
--- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestUseDatabase.java
+++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestUseDatabase.java
@@ -21,7 +21,7 @@ package org.apache.hive.hcatalog.cli;
 import java.io.File;
 import java.io.IOException;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.cli.CliSessionState;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -31,14 +31,21 @@ import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hive.hcatalog.cli.SemanticAnalysis.HCatSemanticAnalyzer;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import org.junit.Before;
+import org.junit.Test;
 
 /* Unit test for GitHub Howl issue #3 */
-public class TestUseDatabase extends TestCase {
+/**
+ * TestUseDatabase.
+ */
+public class TestUseDatabase {
 
   private IDriver hcatDriver;
 
-  @Override
-  protected void setUp() throws Exception {
+  @Before
+  public void setUp() throws Exception {
 
     HiveConf hcatConf = new HiveConf(this.getClass());
     hcatConf.set(ConfVars.PREEXECHOOKS.varname, "");
@@ -54,6 +61,7 @@ public class TestUseDatabase extends TestCase {
   private final String dbName = "testUseDatabase_db";
   private final String tblName = "testUseDatabase_tbl";
 
+  @Test
   public void testAlterTablePass() throws Exception {
 
     hcatDriver.run("create database " + dbName);
diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/data/TestDefaultHCatRecord.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/data/TestDefaultHCatRecord.java
index 4c34211..3dd3433 100644
--- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/data/TestDefaultHCatRecord.java
+++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/data/TestDefaultHCatRecord.java
@@ -47,13 +47,18 @@ import org.apache.hive.hcatalog.data.schema.HCatSchemaUtils;
 import org.junit.Assert;
 import junit.framework.TestCase;
 import org.apache.pig.parser.AliasMasker;
+import org.junit.Test;
 
-public class TestDefaultHCatRecord extends TestCase {
+/**
+ * TestDefaultHCatRecord.
+ */
+public class TestDefaultHCatRecord {
 
   /**
    * test that we properly serialize/deserialize HCatRecordS
    * @throws IOException
    */
+  @Test
   public void testRYW() throws IOException {
 
     File f = new File("binary.dat");
@@ -87,12 +92,14 @@ public class TestDefaultHCatRecord extends TestCase {
 
   }
 
+  @Test
   public void testCompareTo() {
     HCatRecord[] recs = getHCatRecords();
     Assert.assertTrue(HCatDataCheckUtil.compareRecords(recs[0], recs[1]) == 0);
     Assert.assertTrue(HCatDataCheckUtil.compareRecords(recs[4], recs[5]) == 0);
   }
 
+  @Test
   public void testEqualsObject() {
 
     HCatRecord[] recs = getHCatRecords();
@@ -104,6 +111,7 @@ public class TestDefaultHCatRecord extends TestCase {
    * Test get and set calls with type
    * @throws HCatException
    */
+  @Test
   public void testGetSetByType1() throws HCatException {
     HCatRecord inpRec = getHCatRecords()[0];
     HCatRecord newRec = new DefaultHCatRecord(inpRec.size());
@@ -132,6 +140,7 @@ public class TestDefaultHCatRecord extends TestCase {
    * Test get and set calls with type
    * @throws HCatException
    */
+  @Test
   public void testGetSetByType2() throws HCatException {
     HCatRecord inpRec = getGetSet2InpRec();
 
@@ -152,6 +161,7 @@ public class TestDefaultHCatRecord extends TestCase {
    * Test type specific get/set methods on HCatRecord types added in Hive 13
    * @throws HCatException
    */
+  @Test
   public void testGetSetByType3() throws HCatException {
     HCatRecord inpRec = getHCat13TypesRecord();
     HCatRecord newRec = new DefaultHCatRecord(inpRec.size());
@@ -267,7 +277,7 @@ public class TestDefaultHCatRecord extends TestCase {
     rec_6.add(getList());
     HCatRecord tup_6 = new DefaultHCatRecord(rec_6);
 
-    return new HCatRecord[]{tup_1, tup_2, tup_3, tup_4, tup_5, tup_6, getHCat13TypesRecord(), 
+    return new HCatRecord[]{tup_1, tup_2, tup_3, tup_4, tup_5, tup_6, getHCat13TypesRecord(),
             getHCat13TypesComplexRecord()};
   }
   private static HCatRecord getHCat13TypesRecord() {
diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/data/TestHCatRecordSerDe.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/data/TestHCatRecordSerDe.java
index e235b2e..07dae55 100644
--- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/data/TestHCatRecordSerDe.java
+++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/data/TestHCatRecordSerDe.java
@@ -35,8 +35,12 @@ import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
 import org.apache.hadoop.io.Writable;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.junit.Test;
 
-public class TestHCatRecordSerDe extends TestCase {
+/**
+ * TestHCatRecordSerDe.
+ */
+public class TestHCatRecordSerDe {
 
   private static final Logger LOG = LoggerFactory.getLogger(TestHCatRecordSerDe.class);
 
@@ -115,6 +119,7 @@ public class TestHCatRecordSerDe extends TestCase {
     return data;
   }
 
+  @Test
   public void testRW() throws Exception {
 
     Configuration conf = new Configuration();
diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/data/TestJsonSerDe.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/data/TestJsonSerDe.java
index 00cc1ee..583027f 100644
--- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/data/TestJsonSerDe.java
+++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/data/TestJsonSerDe.java
@@ -41,9 +41,14 @@ import org.apache.hadoop.io.Writable;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import junit.framework.TestCase;
 
-public class TestJsonSerDe extends TestCase {
+import static org.junit.Assert.assertTrue;
+import org.junit.Test;
+
+/**
+ * TestJsonSerDe.
+ */
+public class TestJsonSerDe {
 
   private static final Logger LOG = LoggerFactory.getLogger(TestJsonSerDe.class);
 
@@ -139,6 +144,7 @@ public class TestJsonSerDe extends TestCase {
     return data;
   }
 
+  @Test
   public void testRW() throws Exception {
 
     Configuration conf = new Configuration();
@@ -174,6 +180,7 @@ public class TestJsonSerDe extends TestCase {
 
   }
 
+  @Test
   public void testRobustRead() throws Exception {
     /**
      *  This test has been added to account for HCATALOG-436
@@ -264,6 +271,7 @@ public class TestJsonSerDe extends TestCase {
    * Then it should still work, and ignore the "x" and "t" field and "c" subfield of "s", and it
    * should read k as null.
    */
+  @Test
   public void testLooseJsonReadability() throws Exception {
     Configuration conf = new Configuration();
     Properties props = new Properties();
@@ -291,6 +299,7 @@ public class TestJsonSerDe extends TestCase {
 
   }
 
+  @Test
   public void testUpperCaseKey() throws Exception {
     Configuration conf = new Configuration();
     Properties props = new Properties();
@@ -320,6 +329,7 @@ public class TestJsonSerDe extends TestCase {
     return retval;
   }
 
+  @Test
   public void testMapValues() throws Exception {
     Configuration conf = new Configuration();
     Properties props = new Properties();
diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/data/schema/TestHCatSchema.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/data/schema/TestHCatSchema.java
index b71c963..2824f4d 100644
--- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/data/schema/TestHCatSchema.java
+++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/data/schema/TestHCatSchema.java
@@ -21,13 +21,23 @@ package org.apache.hive.hcatalog.data.schema;
 import java.util.ArrayList;
 import java.util.List;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hive.hcatalog.common.HCatException;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import static org.junit.Assert.assertFalse;
+import org.junit.Test;
 
 
-public class TestHCatSchema extends TestCase {
+/**
+ *
+ * TestHCatSchema.
+ */
+public class TestHCatSchema {
+  @Test
   public void testCannotAddFieldMoreThanOnce() throws HCatException {
     List<HCatFieldSchema> fieldSchemaList = new ArrayList<HCatFieldSchema>();
     fieldSchemaList.add(new HCatFieldSchema("name", HCatFieldSchema.Type.STRING, "What's your handle?"));
@@ -60,6 +70,7 @@ public class TestHCatSchema extends TestCase {
     assertEquals(2, schema.getFields().size());
   }
 
+  @Test
   public void testHashCodeEquals() throws HCatException {
     HCatFieldSchema memberID1 = new HCatFieldSchema("memberID", HCatFieldSchema.Type.INT, "as a number");
     HCatFieldSchema memberID2 = new HCatFieldSchema("memberID", HCatFieldSchema.Type.INT, "as a number");
@@ -75,6 +86,7 @@ public class TestHCatSchema extends TestCase {
     assertTrue("Expected hash codes to be equal", memberID1.hashCode() == memberID2.hashCode());
   }
 
+  @Test
   public void testCannotInstantiateSchemaWithRepeatedFieldNames() throws HCatException {
     List<HCatFieldSchema> fieldSchemaList = new ArrayList<HCatFieldSchema>();
 
@@ -94,6 +106,7 @@ public class TestHCatSchema extends TestCase {
       assertTrue(iae.getMessage().contains("Field named memberID already exists"));
     }
   }
+  @Test
   public void testRemoveAddField() throws HCatException {
     List<HCatFieldSchema> fieldSchemaList = new ArrayList<HCatFieldSchema>();
 
@@ -115,6 +128,7 @@ public class TestHCatSchema extends TestCase {
   // HIVE-5336. Re-number the position after remove such that:
   // (1) getPosition on a column always returns a value between 0..schema.size()-1
   // (2) getPosition() on 2 different columns should never give the same value.
+  @Test
   public void testRemoveAddField2() throws HCatException {
     List<HCatFieldSchema> fieldSchemaList = new ArrayList<HCatFieldSchema>();
     HCatFieldSchema memberIDField = new HCatFieldSchema("memberID", HCatFieldSchema.Type.INT, "id as number");
diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/data/schema/TestHCatSchemaUtils.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/data/schema/TestHCatSchemaUtils.java
index 3655eea..6455fad 100644
--- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/data/schema/TestHCatSchemaUtils.java
+++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/data/schema/TestHCatSchemaUtils.java
@@ -20,7 +20,7 @@ package org.apache.hive.hcatalog.data.schema;
 
 import java.io.PrintStream;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.serde.serdeConstants;
@@ -30,11 +30,17 @@ import org.apache.hive.hcatalog.common.HCatException;
 import org.apache.hive.hcatalog.data.schema.HCatFieldSchema.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
-public class TestHCatSchemaUtils extends TestCase {
+/**
+ * TestHCatSchemaUtils.
+ */
+public class TestHCatSchemaUtils {
 
   private static final Logger LOG = LoggerFactory.getLogger(TestHCatSchemaUtils.class);
 
+  @Test
   public void testSimpleOperation() throws Exception {
     String typeString = "struct<name:string,studentid:int,"
         + "contact:struct<phNo:string,email:string>,"
@@ -53,6 +59,7 @@ public class TestHCatSchemaUtils extends TestCase {
     assertEquals(hsch.get(0).getTypeString(), typeString.toLowerCase());
   }
   
+  @Test
   public void testHCatFieldSchemaConversion() throws Exception {
 	  FieldSchema stringFieldSchema = new FieldSchema("name1", serdeConstants.STRING_TYPE_NAME, "comment1");
 	  HCatFieldSchema stringHCatFieldSchema = HCatSchemaUtils.getHCatFieldSchema(stringFieldSchema);
diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatOutputFormat.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatOutputFormat.java
index 4ac01df..416a01e 100644
--- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatOutputFormat.java
+++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatOutputFormat.java
@@ -26,7 +26,7 @@ import java.util.List;
 import java.util.Map;
 
 import com.google.common.collect.Lists;
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -49,8 +49,17 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hive.hcatalog.data.schema.HCatSchema;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
-public class TestHCatOutputFormat extends TestCase {
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+import org.junit.Before;
+import org.junit.After;
+import org.junit.Test;
+
+/**
+ * TestHCatOutputFormat.
+ */
+public class TestHCatOutputFormat {
 
   private static final Logger LOG = LoggerFactory.getLogger(TestHCatOutputFormat.class);
   private HiveMetaStoreClient client;
@@ -59,9 +68,9 @@ public class TestHCatOutputFormat extends TestCase {
   private static final String dbName = "hcatOutputFormatTestDB";
   private static final String tblName = "hcatOutputFormatTestTable";
 
-  @Override
-  protected void setUp() throws Exception {
-    super.setUp();
+  @Before
+  public void setUp() throws Exception {
+
     hiveConf = new HiveConf(this.getClass());
 
     try {
@@ -74,10 +83,10 @@ public class TestHCatOutputFormat extends TestCase {
     }
   }
 
-  @Override
-  protected void tearDown() throws Exception {
+  @After
+  public void tearDown() throws Exception {
     try {
-      super.tearDown();
+
       client.dropTable(dbName, tblName);
       client.dropDatabase(dbName);
 
@@ -136,6 +145,7 @@ public class TestHCatOutputFormat extends TestCase {
 
   }
 
+  @Test
   public void testSetOutput() throws Exception {
     Configuration conf = new Configuration();
     Job job = Job.getInstance(conf, "test outputformat");
@@ -175,6 +185,7 @@ public class TestHCatOutputFormat extends TestCase {
     assertTrue(part.getSd().getLocation().contains("p1"));
   }
 
+  @Test
   public void testGetTableSchema() throws Exception {
 
     Configuration conf = new Configuration();
diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java
index b190e4b..18058dc 100644
--- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java
+++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java
@@ -24,7 +24,7 @@ import java.io.UnsupportedEncodingException;
 import java.util.List;
 import java.util.Properties;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -49,12 +49,14 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
 /**
  * TestRCFile.
  *
  */
-public class TestRCFileMapReduceInputFormat extends TestCase {
+public class TestRCFileMapReduceInputFormat {
 
   private static final Logger LOG = LoggerFactory.getLogger(TestRCFileMapReduceInputFormat.class);
 
@@ -173,6 +175,7 @@ public class TestRCFileMapReduceInputFormat extends TestCase {
   }
 
 
+  @Test
   public void testSynAndSplit() throws IOException, InterruptedException {
     splitBeforeSync();
     splitRightBeforeSync();
diff --git a/hcatalog/webhcat/java-client/src/test/java/org/apache/hive/hcatalog/api/repl/TestReplicationTask.java b/hcatalog/webhcat/java-client/src/test/java/org/apache/hive/hcatalog/api/repl/TestReplicationTask.java
index dca56ee..09dc5d8 100644
--- a/hcatalog/webhcat/java-client/src/test/java/org/apache/hive/hcatalog/api/repl/TestReplicationTask.java
+++ b/hcatalog/webhcat/java-client/src/test/java/org/apache/hive/hcatalog/api/repl/TestReplicationTask.java
@@ -18,7 +18,7 @@
  */
 package org.apache.hive.hcatalog.api.repl;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.NotificationEvent;
@@ -30,9 +30,13 @@ import org.apache.hive.hcatalog.api.repl.exim.EximReplicationTaskFactory;
 import org.apache.hive.hcatalog.common.HCatConstants;
 import org.apache.hive.hcatalog.common.HCatException;
 import org.apache.hive.hcatalog.messaging.MessageFactory;
+import static org.junit.Assert.assertTrue;
 import org.junit.Test;
 
-public class TestReplicationTask extends TestCase{
+/**
+ * TestReplicationTask.
+ */
+public class TestReplicationTask {
   private static MessageFactory msgFactory = MessageFactory.getInstance();
 
 
@@ -68,7 +72,7 @@ public class TestReplicationTask extends TestCase{
   }
 
   @Test
-  public static void testCreate() throws HCatException {
+  public void testCreate() throws HCatException {
     Table t = new Table();
     t.setDbName("testdb");
     t.setTableName("testtable");
diff --git a/hcatalog/webhcat/java-client/src/test/java/org/apache/hive/hcatalog/api/repl/commands/TestNoopCommand.java b/hcatalog/webhcat/java-client/src/test/java/org/apache/hive/hcatalog/api/repl/commands/TestNoopCommand.java
index bdabb0d..e13cf14 100644
--- a/hcatalog/webhcat/java-client/src/test/java/org/apache/hive/hcatalog/api/repl/commands/TestNoopCommand.java
+++ b/hcatalog/webhcat/java-client/src/test/java/org/apache/hive/hcatalog/api/repl/commands/TestNoopCommand.java
@@ -18,15 +18,19 @@
  */
 package org.apache.hive.hcatalog.api.repl.commands;
 
-import junit.framework.TestCase;
+
 import org.apache.hive.hcatalog.api.repl.Command;
 import org.apache.hive.hcatalog.api.repl.CommandTestUtils;
+import static org.junit.Assert.assertEquals;
 import org.junit.Test;
 
-public class TestNoopCommand extends TestCase {
+/**
+ * TestNoopCommand.
+ */
+public class TestNoopCommand {
 
   @Test
-  public static void testCommand(){
+  public void testCommand(){
     int evid = 999;
     Command testCmd = new NoopCommand(evid);
 
diff --git a/hcatalog/webhcat/svr/src/test/java/org/apache/hive/hcatalog/templeton/TestDesc.java b/hcatalog/webhcat/svr/src/test/java/org/apache/hive/hcatalog/templeton/TestDesc.java
index 9ce714e..3770c53 100644
--- a/hcatalog/webhcat/svr/src/test/java/org/apache/hive/hcatalog/templeton/TestDesc.java
+++ b/hcatalog/webhcat/svr/src/test/java/org/apache/hive/hcatalog/templeton/TestDesc.java
@@ -23,14 +23,19 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import junit.framework.TestCase;
+
 import org.codehaus.jackson.map.ObjectMapper;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
 /**
  * TestDesc - Test the desc objects that are correctly converted to
  * and from json.  This also sets every field of the TableDesc object.
  */
-public class TestDesc extends TestCase {
+public class TestDesc {
+  @Test
   public void testTableDesc()
     throws Exception
   {
diff --git a/hcatalog/webhcat/svr/src/test/java/org/apache/hive/hcatalog/templeton/TestServer.java b/hcatalog/webhcat/svr/src/test/java/org/apache/hive/hcatalog/templeton/TestServer.java
index 21a685c..4491a35 100644
--- a/hcatalog/webhcat/svr/src/test/java/org/apache/hive/hcatalog/templeton/TestServer.java
+++ b/hcatalog/webhcat/svr/src/test/java/org/apache/hive/hcatalog/templeton/TestServer.java
@@ -18,40 +18,54 @@
  */
 package org.apache.hive.hcatalog.templeton;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hive.hcatalog.templeton.mock.MockServer;
 import java.util.List;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import org.junit.Before;
+import org.junit.Test;
 
 /*
  * Test that the server code exists, and responds to basic requests.
  */
-public class TestServer extends TestCase {
+/**
+ * TestServer.
+ */
+public class TestServer {
 
   MockServer server;
 
+  @Before
   public void setUp() {
     new Main(new String[]{});         // Initialize the config
     server = new MockServer();
   }
 
+  @Test
   public void testServer() {
     assertNotNull(server);
   }
 
+  @Test
   public void testStatus() {
     assertEquals(server.status().get("status"), "ok");
   }
 
+  @Test
   public void testVersions() {
     assertEquals(server.version().get("version"), "v1");
   }
 
+  @Test
   public void testFormats() {
     assertEquals(1, server.requestFormats().size());
     assertEquals( ((List)server.requestFormats().get("responseTypes")).get(0), "application/json");
   }
 
+  @Test
   public void testVerifyPropertyParam() {
     // HIVE-15410: Though there are not restrictions to Hive table property key and it could be any
     // combination of the letters, digits and even punctuations, we support conventional property
diff --git a/itests/hive-unit-hadoop2/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationProviderWithACL.java b/itests/hive-unit-hadoop2/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationProviderWithACL.java
index c9253c1..20e2dbf 100644
--- a/itests/hive-unit-hadoop2/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationProviderWithACL.java
+++ b/itests/hive-unit-hadoop2/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationProviderWithACL.java
@@ -22,6 +22,8 @@ import static org.apache.hadoop.fs.permission.AclEntryType.GROUP;
 import static org.apache.hadoop.fs.permission.AclEntryType.OTHER;
 import static org.apache.hadoop.fs.permission.AclEntryType.USER;
 
+import org.junit.After;
+
 import java.lang.reflect.Method;
 import java.net.URI;
 import java.security.PrivilegedExceptionAction;
@@ -107,8 +109,8 @@ public class TestStorageBasedMetastoreAuthorizationProviderWithACL
     return userUgi.getShortUserName();
   }
 
-  @Override
-  protected void tearDown() throws Exception {
+  @After
+  public void tearDown() throws Exception {
     super.tearDown();
 
     if (dfs != null) {
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreAuthorization.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreAuthorization.java
index 747efd8..9ddad99 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreAuthorization.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreAuthorization.java
@@ -29,10 +29,15 @@ import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 
-import junit.framework.TestCase;
 
+import static org.junit.Assert.assertTrue;
+import org.junit.Test;
 
-public class TestMetaStoreAuthorization extends TestCase {
+
+/**
+ * TestMetaStoreAuthorization.
+ */
+public class TestMetaStoreAuthorization {
   protected HiveConf conf = new HiveConf();
 
   private int port;
@@ -43,6 +48,7 @@ public class TestMetaStoreAuthorization extends TestCase {
     conf.setTimeVar(ConfVars.METASTORE_CLIENT_CONNECT_RETRY_DELAY, 60, TimeUnit.SECONDS);
   }
 
+  @Test
   public void testIsWritable() throws Exception {
     setup();
     String testDir = System.getProperty("test.warehouse.dir", "/tmp");
@@ -66,6 +72,7 @@ public class TestMetaStoreAuthorization extends TestCase {
     }
   }
 
+  @Test
   public void testMetaStoreAuthorization() throws Exception {
     setup();
     MetaStoreTestUtils.startMetaStoreWithRetry(conf);
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetastoreVersion.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetastoreVersion.java
index 4d26f3e..88c7efa 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetastoreVersion.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetastoreVersion.java
@@ -33,9 +33,19 @@ import org.apache.hive.common.util.HiveStringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import junit.framework.TestCase;
 
-public class TestMetastoreVersion extends TestCase {
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.junit.Assert.assertEquals;
+import org.junit.Before;
+import org.junit.After;
+import org.junit.Test;
+
+/**
+ * TestMetastoreVersion.
+ */
+public class TestMetastoreVersion {
   private static final Logger LOG = LoggerFactory.getLogger(TestMetastoreVersion.class);
   protected HiveConf hiveConf;
   private IDriver driver;
@@ -43,9 +53,9 @@ public class TestMetastoreVersion extends TestCase {
   private String testMetastoreDB;
   private IMetaStoreSchemaInfo metastoreSchemaInfo;
 
-  @Override
-  protected void setUp() throws Exception {
-    super.setUp();
+  @Before
+  public void setUp() throws Exception {
+
     Field defDb = HiveMetaStore.HMSHandler.class.getDeclaredField("currentUrl");
     defDb.setAccessible(true);
     defDb.set(null, null);
@@ -68,8 +78,8 @@ public class TestMetastoreVersion extends TestCase {
         System.getProperty("test.tmp.dir", "target/tmp"), "derby");
   }
 
-  @Override
-  protected void tearDown() throws Exception {
+  @After
+  public void tearDown() throws Exception {
     File metaStoreDir = new File(testMetastoreDB);
     if (metaStoreDir.exists()) {
       FileUtils.forceDeleteOnExit(metaStoreDir);
@@ -79,6 +89,7 @@ public class TestMetastoreVersion extends TestCase {
   /***
    * Test config defaults
    */
+  @Test
   public void testDefaults() {
     System.clearProperty(HiveConf.ConfVars.METASTORE_SCHEMA_VERIFICATION.toString());
     hiveConf = new HiveConf(this.getClass());
@@ -90,6 +101,7 @@ public class TestMetastoreVersion extends TestCase {
    * Test schema verification property
    * @throws Exception
    */
+  @Test
   public void testVersionRestriction () throws Exception {
     System.setProperty(HiveConf.ConfVars.METASTORE_SCHEMA_VERIFICATION.toString(), "true");
     hiveConf = new HiveConf(this.getClass());
@@ -114,6 +126,7 @@ public class TestMetastoreVersion extends TestCase {
    * and version correctly
    * @throws Exception
    */
+  @Test
   public void testMetastoreVersion () throws Exception {
     // let the schema and version be auto created
     System.setProperty(HiveConf.ConfVars.METASTORE_SCHEMA_VERIFICATION.toString(), "false");
@@ -133,6 +146,7 @@ public class TestMetastoreVersion extends TestCase {
    * Test that with verification enabled, hive works when the correct schema is already populated
    * @throws Exception
    */
+  @Test
   public void testVersionMatching () throws Exception {
     System.setProperty(HiveConf.ConfVars.METASTORE_SCHEMA_VERIFICATION.toString(), "false");
     hiveConf = new HiveConf(this.getClass());
@@ -153,6 +167,7 @@ public class TestMetastoreVersion extends TestCase {
    * Store garbage version in metastore and verify that hive fails when verification is on
    * @throws Exception
    */
+  @Test
   public void testVersionMisMatch () throws Exception {
     System.setProperty(HiveConf.ConfVars.METASTORE_SCHEMA_VERIFICATION.toString(), "false");
     hiveConf = new HiveConf(this.getClass());
@@ -175,6 +190,7 @@ public class TestMetastoreVersion extends TestCase {
    * version
    * @throws Exception
    */
+  @Test
   public void testVersionCompatibility () throws Exception {
     System.setProperty(HiveConf.ConfVars.METASTORE_SCHEMA_VERIFICATION.toString(), "false");
     hiveConf = new HiveConf(this.getClass());
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/security/TestDBTokenStore.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/security/TestDBTokenStore.java
index 64c606c..20f95fe 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/security/TestDBTokenStore.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/security/TestDBTokenStore.java
@@ -21,7 +21,7 @@ package org.apache.hadoop.hive.metastore.security;
 import java.io.IOException;
 import java.util.List;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
 import org.apache.hadoop.hive.metastore.api.MetaException;
@@ -32,9 +32,20 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager.DelegationTokenInformation;
 import org.apache.hadoop.security.token.delegation.HiveDelegationTokenSupport;
 import org.junit.Assert;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotSame;
+import org.junit.Test;
 
-public class TestDBTokenStore extends TestCase{
+/**
+ * TestDBTokenStore.
+ */
+public class TestDBTokenStore {
 
+  @Test
   public void testDBTokenStore() throws TokenStoreException, MetaException, IOException {
 
     DelegationTokenStore ts = new DBTokenStore();
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/security/TestZooKeeperTokenStore.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/security/TestZooKeeperTokenStore.java
index 4c4cf7c..0b4f2bc 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/security/TestZooKeeperTokenStore.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/security/TestZooKeeperTokenStore.java
@@ -22,7 +22,7 @@ import java.io.File;
 import java.io.IOException;
 import java.util.List;
 
-import junit.framework.TestCase;
+
 
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
@@ -37,16 +37,28 @@ import org.apache.hadoop.security.token.delegation.HiveDelegationTokenSupport;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.data.ACL;
 import org.junit.Assert;
-
-public class TestZooKeeperTokenStore extends TestCase {
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.fail;
+import org.junit.Before;
+import org.junit.After;
+import org.junit.Test;
+
+/**
+ * TestZooKeeperTokenStore.
+ */
+public class TestZooKeeperTokenStore {
 
   private MiniZooKeeperCluster zkCluster = null;
   private CuratorFramework zkClient = null;
   private int zkPort = -1;
   private ZooKeeperTokenStore ts;
 
-  @Override
-  protected void setUp() throws Exception {
+  @Before
+  public void setUp() throws Exception {
     File zkDataDir = new File(System.getProperty("test.tmp.dir"));
     if (this.zkCluster != null) {
       throw new IOException("Cluster already running");
@@ -59,8 +71,8 @@ public class TestZooKeeperTokenStore extends TestCase {
     this.zkClient.start();
   }
 
-  @Override
-  protected void tearDown() throws Exception {
+  @After
+  public void tearDown() throws Exception {
     this.zkClient.close();
     if (ts != null) {
       ts.close();
@@ -77,6 +89,7 @@ public class TestZooKeeperTokenStore extends TestCase {
     return conf;
   }
 
+  @Test
   public void testTokenStorage() throws Exception {
     String ZK_PATH = "/zktokenstore-testTokenStorage";
     ts = new ZooKeeperTokenStore();
@@ -126,6 +139,7 @@ public class TestZooKeeperTokenStore extends TestCase {
     assertNull(ts.getToken(tokenId));
   }
 
+  @Test
   public void testAclNoAuth() throws Exception {
     String ZK_PATH = "/zktokenstore-testAclNoAuth";
     Configuration conf = createConf(ZK_PATH);
@@ -143,6 +157,7 @@ public class TestZooKeeperTokenStore extends TestCase {
     }
   }
 
+  @Test
   public void testAclInvalid() throws Exception {
     String ZK_PATH = "/zktokenstore-testAclInvalid";
     String aclString = "sasl:hive/host@TEST.DOMAIN:cdrwa, fail-parse-ignored";
@@ -164,6 +179,7 @@ public class TestZooKeeperTokenStore extends TestCase {
     }
   }
 
+  @Test
   public void testAclPositive() throws Exception {
     String ZK_PATH = "/zktokenstore-testAcl";
     Configuration conf = createConf(ZK_PATH);
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/tools/metatool/TestHiveMetaTool.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/tools/metatool/TestHiveMetaTool.java
index ba344d4..81b7ff0 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/tools/metatool/TestHiveMetaTool.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/tools/metatool/TestHiveMetaTool.java
@@ -26,7 +26,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
@@ -39,9 +39,14 @@ import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.avro.AvroSerdeUtils.AvroTableProperties;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.thrift.TException;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+import org.junit.Before;
+import org.junit.After;
+import org.junit.Test;
 
 /** Integration tests for the HiveMetaTool program. */
-public class TestHiveMetaTool extends TestCase {
+public class TestHiveMetaTool {
   private static final String DB_NAME = "TestHiveMetaToolDB";
   private static final String TABLE_NAME = "simpleTbl";
   private static final String LOCATION = "hdfs://nn.example.com/";
@@ -53,9 +58,9 @@ public class TestHiveMetaTool extends TestCase {
   private HiveMetaStoreClient client;
   private OutputStream os;
 
-  @Override
-  protected void setUp() throws Exception {
-    super.setUp();
+  @Before
+  public void setUp() throws Exception {
+
 
     try {
       os = new ByteArrayOutputStream();
@@ -109,6 +114,7 @@ public class TestHiveMetaTool extends TestCase {
     client.createTable(tbl);
   }
 
+  @Test
   public void testListFSRoot() throws Exception {
     HiveMetaTool.main(new String[] {"-listFSRoot"});
     String out = os.toString();
@@ -116,6 +122,7 @@ public class TestHiveMetaTool extends TestCase {
         out.contains(client.getDatabase(DB_NAME).getLocationUri()));
   }
 
+  @Test
   public void testExecuteJDOQL() throws Exception {
     HiveMetaTool.main(
         new String[] {"-executeJDOQL", "select locationUri from org.apache.hadoop.hive.metastore.model.MDatabase"});
@@ -124,6 +131,7 @@ public class TestHiveMetaTool extends TestCase {
         out.contains(client.getDatabase(DB_NAME).getLocationUri()));
   }
 
+  @Test
   public void testUpdateFSRootLocation() throws Exception {
     checkAvroSchemaURLProps(AVRO_URI);
 
@@ -140,12 +148,12 @@ public class TestHiveMetaTool extends TestCase {
     assertEquals(expectedUri, table.getSd().getParameters().get(AvroTableProperties.SCHEMA_URL.getPropName()));
   }
 
-  @Override
-  protected void tearDown() throws Exception {
+  @After
+  public void tearDown() throws Exception {
     try {
       client.dropTable(DB_NAME, TABLE_NAME);
       client.dropDatabase(DB_NAME);
-      super.tearDown();
+
       client.close();
     } catch (Throwable e) {
       System.err.println("Unable to close metastore");
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/BaseTestQueries.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/BaseTestQueries.java
index 14951cd..dffaa25 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/BaseTestQueries.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/BaseTestQueries.java
@@ -20,12 +20,12 @@ package org.apache.hadoop.hive.ql;
 
 import java.io.File;
 
-import junit.framework.TestCase;
+
 
 /**
  * Base class for testing queries.
  */
-public abstract class BaseTestQueries extends TestCase {
+public abstract class BaseTestQueries {
 
   protected final String inpDir = System
       .getProperty("hive.root") + "/ql/src/test/queries/clientpositive";
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestDDLWithRemoteMetastoreSecondNamenode.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestDDLWithRemoteMetastoreSecondNamenode.java
index a56802c..5b1c903 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestDDLWithRemoteMetastoreSecondNamenode.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestDDLWithRemoteMetastoreSecondNamenode.java
@@ -20,7 +20,7 @@ import java.io.IOException;
 import java.net.URI;
 import java.util.HashMap;
 import junit.framework.JUnit4TestAdapter;
-import junit.framework.TestCase;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
@@ -33,12 +33,18 @@ import org.apache.hadoop.hive.ql.exec.mr.ExecDriver;
 import org.apache.hadoop.hive.ql.metadata.*;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.session.SessionState;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import org.junit.Before;
+import org.junit.After;
+import org.junit.Test;
 
 /**
  * Tests DDL with remote metastore service and second namenode (HIVE-6374)
  *
  */
-public class TestDDLWithRemoteMetastoreSecondNamenode extends TestCase {
+public class TestDDLWithRemoteMetastoreSecondNamenode {
   static HiveConf conf;
 
   private static final String Database1Name = "db1_nondefault_nn";
@@ -63,9 +69,9 @@ public class TestDDLWithRemoteMetastoreSecondNamenode extends TestCase {
   private static int tests = 0;
   private static Boolean isInitialized = false;
 
-  @Override
-  protected void setUp() throws Exception {
-    super.setUp();
+  @Before
+  public void setUp() throws Exception {
+
     if (tests > 0) {
       return;
     }
@@ -123,9 +129,9 @@ public class TestDDLWithRemoteMetastoreSecondNamenode extends TestCase {
     }
   }
 
-  @Override
-  protected void tearDown() throws Exception {
-    super.tearDown();
+  @After
+  public void tearDown() throws Exception {
+
     if (--tests == 0) {
       cleanup();
       shutdownMiniDfs();
@@ -249,6 +255,7 @@ public class TestDDLWithRemoteMetastoreSecondNamenode extends TestCase {
     }
   }
 
+  @Test
   public void testAlterPartitionSetLocationNonDefaultNameNode() throws Exception {
     assertTrue("Test suite should have been initialized", isInitialized);
     String tableLocation = tmppathFs2 + "/" + "test_set_part_loc";
@@ -258,6 +265,7 @@ public class TestDDLWithRemoteMetastoreSecondNamenode extends TestCase {
     alterPartitionAndCheck(table, "p", "p1", "/tmp/test/2");
   }
 
+  @Test
   public void testCreateDatabaseWithTableNonDefaultNameNode() throws Exception {
     assertTrue("Test suite should be initialied", isInitialized );
     final String tableLocation = tmppathFs2 + "/" + Table3Name;
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestLocationQueries.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestLocationQueries.java
index 38bdcb1..eb3b935 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestLocationQueries.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestLocationQueries.java
@@ -26,6 +26,9 @@ import java.util.regex.Pattern;
 
 import org.apache.hadoop.hive.ql.QTestMiniClusters.MiniClusterType;
 
+import org.junit.Test;
+import static org.junit.Assert.fail;
+
 /**
  * Suite for testing location. e.g. if "alter table alter partition
  * location" is run, do the partitions end up in the correct location.
@@ -109,6 +112,7 @@ public class TestLocationQueries extends BaseTestQueries {
    * the path should end in "parta" and not "dt=a" (the default).
    *
    */
+  @Test
   public void testAlterTablePartitionLocation_alter5() throws Exception {
     String[] testNames = new String[] {"alter5.q"};
 
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestMTQueries.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestMTQueries.java
index e336712..3e0cdac 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestMTQueries.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestMTQueries.java
@@ -19,6 +19,8 @@
 package org.apache.hadoop.hive.ql;
 
 import java.io.File;
+import org.junit.Test;
+import static org.junit.Assert.fail;
 
 /**
  * Suite for testing running of queries in multi-threaded mode.
@@ -32,6 +34,7 @@ public class TestMTQueries extends BaseTestQueries {
     }
   }
 
+  @Test
   public void testMTQueries1() throws Exception {
     String[] testNames = new String[] {"join2.q", "groupby1.q", "input1.q", "input19.q"};
 
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/history/TestHiveHistory.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/history/TestHiveHistory.java
index 5fd0ef9..3c40983 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/history/TestHiveHistory.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/history/TestHiveHistory.java
@@ -24,7 +24,7 @@ import java.lang.reflect.Proxy;
 import java.util.LinkedList;
 import java.util.Map;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -46,12 +46,17 @@ import org.apache.hadoop.hive.ql.plan.LoadTableDesc.LoadFileType;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.ql.tools.LineageInfo;
 import org.apache.hadoop.mapred.TextInputFormat;
+import static org.junit.Assert.fail;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import org.junit.Before;
+import org.junit.Test;
 
 /**
  * TestHiveHistory.
  *
  */
-public class TestHiveHistory extends TestCase {
+public class TestHiveHistory {
 
   static HiveConf conf;
 
@@ -63,8 +68,8 @@ public class TestHiveHistory extends TestCase {
    * intialize the tables
    */
 
-  @Override
-  protected void setUp() {
+  @Before
+  public void setUp() {
     try {
       conf = new HiveConf(HiveHistory.class);
       SessionState.start(conf);
@@ -120,6 +125,7 @@ public class TestHiveHistory extends TestCase {
   /**
    * Check history file output for this query.
    */
+  @Test
   public void testSimpleQuery() {
     new LineageInfo();
     try {
@@ -174,6 +180,7 @@ public class TestHiveHistory extends TestCase {
     }
   }
 
+  @Test
   public void testQueryloglocParentDirNotExist() throws Exception {
     String parentTmpDir = tmpdir + "/HIVE2654";
     Path parentDirPath = new Path(parentTmpDir);
@@ -203,6 +210,7 @@ public class TestHiveHistory extends TestCase {
    * Check if HiveHistoryImpl class is returned when hive history is enabled
    * @throws Exception
    */
+  @Test
   public void testHiveHistoryConfigEnabled() throws Exception {
       HiveConf conf = new HiveConf(SessionState.class);
       conf.setBoolVar(ConfVars.HIVE_SESSION_HISTORY_ENABLED, true);
@@ -216,6 +224,7 @@ public class TestHiveHistory extends TestCase {
    * Check if HiveHistory class is a Proxy class when hive history is disabled
    * @throws Exception
    */
+  @Test
   public void testHiveHistoryConfigDisabled() throws Exception {
     HiveConf conf = new HiveConf(SessionState.class);
     conf.setBoolVar(ConfVars.HIVE_SESSION_HISTORY_ENABLED, false);
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/metadata/TestSemanticAnalyzerHookLoading.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/metadata/TestSemanticAnalyzerHookLoading.java
index 58ac4ac..8cdd75b 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/metadata/TestSemanticAnalyzerHookLoading.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/metadata/TestSemanticAnalyzerHookLoading.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hive.ql.metadata;
 
 import java.util.Map;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
@@ -29,9 +29,16 @@ import org.apache.hadoop.hive.ql.DriverFactory;
 import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.session.SessionState;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import org.junit.Test;
 
-public class TestSemanticAnalyzerHookLoading extends TestCase {
+/**
+ * TestSemanticAnalyzerHookLoading.
+ */
+public class TestSemanticAnalyzerHookLoading {
 
+  @Test
   public void testHookLoading() throws Exception{
 
     HiveConf conf = new HiveConf(this.getClass());
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestAuthorizationPreEventListener.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestAuthorizationPreEventListener.java
index 1f6ec27..b78c348 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestAuthorizationPreEventListener.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestAuthorizationPreEventListener.java
@@ -22,8 +22,6 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 
-import junit.framework.TestCase;
-
 import org.apache.hadoop.hive.cli.CliSessionState;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
@@ -40,20 +38,24 @@ import org.apache.hadoop.hive.ql.security.authorization.AuthorizationPreEventLis
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.junit.Assert;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import org.junit.Before;
+import org.junit.After;
+import org.junit.Test;
+
 /**
  * TestAuthorizationPreEventListener. Test case for
  * {@link org.apache.hadoop.hive.ql.security.authorization.AuthorizationPreEventListener} and
  * {@link org.apache.hadoop.hive.metastore.MetaStorePreEventListener}
  */
-public class TestAuthorizationPreEventListener extends TestCase {
+public class TestAuthorizationPreEventListener {
   private HiveConf clientHiveConf;
   private HiveMetaStoreClient msc;
   private IDriver driver;
 
-  @Override
-  protected void setUp() throws Exception {
-
-    super.setUp();
+  @Before
+  public void setUp() throws Exception {
 
     System.setProperty(HiveConf.ConfVars.METASTORE_PRE_EVENT_LISTENERS.varname,
         AuthorizationPreEventListener.class.getName());
@@ -79,9 +81,8 @@ public class TestAuthorizationPreEventListener extends TestCase {
     driver = DriverFactory.newDriver(clientHiveConf);
   }
 
-  @Override
-  protected void tearDown() throws Exception {
-    super.tearDown();
+  @After
+  public void tearDown() throws Exception {
   }
 
   private void validateCreateDb(Database expectedDb, Database actualDb) {
@@ -164,6 +165,7 @@ public class TestAuthorizationPreEventListener extends TestCase {
     assertEquals(expectedDb, actualDb);
   }
 
+  @Test
   public void testListener() throws Exception {
     String dbName = "hive3705";
     String tblName = "tmptbl";
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestClientSideAuthorizationProvider.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestClientSideAuthorizationProvider.java
index ce55f91..b284777 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestClientSideAuthorizationProvider.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestClientSideAuthorizationProvider.java
@@ -21,7 +21,7 @@ package org.apache.hadoop.hive.ql.security;
 import java.util.ArrayList;
 import java.util.List;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.cli.CliSessionState;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -37,12 +37,19 @@ import org.apache.hadoop.hive.ql.security.authorization.DefaultHiveAuthorization
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.shims.Utils;
 import org.apache.hadoop.security.UserGroupInformation;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import org.junit.Before;
+import org.junit.After;
+import org.junit.Test;
 
 /**
  * TestClientSideAuthorizationProvider : Simple base test for client side
  * Authorization Providers. By default, tests DefaultHiveAuthorizationProvider
  */
-public class TestClientSideAuthorizationProvider extends TestCase {
+public class TestClientSideAuthorizationProvider {
   protected HiveConf clientHiveConf;
   protected HiveMetaStoreClient msc;
   protected IDriver driver;
@@ -54,10 +61,10 @@ public class TestClientSideAuthorizationProvider extends TestCase {
   }
 
 
-  @Override
-  protected void setUp() throws Exception {
+  @Before
+  public void setUp() throws Exception {
+
 
-    super.setUp();
 
     // Turn off metastore-side authorization
     System.setProperty(HiveConf.ConfVars.METASTORE_PRE_EVENT_LISTENERS.varname,
@@ -89,9 +96,9 @@ public class TestClientSideAuthorizationProvider extends TestCase {
     driver = DriverFactory.newDriver(clientHiveConf);
   }
 
-  @Override
-  protected void tearDown() throws Exception {
-    super.tearDown();
+  @After
+  public void tearDown() throws Exception {
+
   }
 
   private void validateCreateDb(Database expectedDb, String dbName) {
@@ -112,6 +119,7 @@ public class TestClientSideAuthorizationProvider extends TestCase {
     return "smp_cl_tbl";
   }
 
+  @Test
   public void testSimplePrivileges() throws Exception {
     String dbName = getTestDbName();
     String tblName = getTestTableName();
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMetastoreAuthorizationProvider.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMetastoreAuthorizationProvider.java
index 0fc677b..10a1ac2 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMetastoreAuthorizationProvider.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMetastoreAuthorizationProvider.java
@@ -24,7 +24,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Collections;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.cli.CliSessionState;
@@ -53,6 +53,12 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.Assert;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import org.junit.Before;
+import org.junit.After;
+import org.junit.Test;
 
 /**
  * TestHiveMetastoreAuthorizationProvider. Test case for
@@ -69,7 +75,7 @@ import org.slf4j.LoggerFactory;
  * This test is also intended to be extended to provide tests for other
  * authorization providers like StorageBasedAuthorizationProvider
  */
-public class TestMetastoreAuthorizationProvider extends TestCase {
+public class TestMetastoreAuthorizationProvider {
   private static final Logger LOG = LoggerFactory.getLogger(TestMetastoreAuthorizationProvider.class);
 
   protected HiveConf clientHiveConf;
@@ -90,10 +96,10 @@ public class TestMetastoreAuthorizationProvider extends TestCase {
     return null;
   }
 
-  @Override
-  protected void setUp() throws Exception {
+  @Before
+  public void setUp() throws Exception {
+
 
-    super.setUp();
 
     // Turn on metastore-side authorization
     System.setProperty(HiveConf.ConfVars.METASTORE_PRE_EVENT_LISTENERS.varname,
@@ -135,9 +141,9 @@ public class TestMetastoreAuthorizationProvider extends TestCase {
     System.setProperty(HiveConf.ConfVars.HIVE_METASTORE_AUTHORIZATION_AUTH_READS.varname, "false");
   }
 
-  @Override
-  protected void tearDown() throws Exception {
-    super.tearDown();
+  @After
+  public void tearDown() throws Exception {
+
   }
 
   private void validateCreateDb(Database expectedDb, String dbName) {
@@ -166,38 +172,31 @@ public class TestMetastoreAuthorizationProvider extends TestCase {
     return ugi.getUserName();
   }
 
+  @Test
   public void testSimplePrivileges() throws Exception {
     if (!isTestEnabled()) {
       System.out.println("Skipping test " + this.getClass().getName());
       return;
     }
-
     String dbName = getTestDbName();
     String tblName = getTestTableName();
     String userName = setupUser();
-
     allowCreateDatabase(userName);
-
     CommandProcessorResponse ret = driver.run("create database " + dbName);
     assertEquals(0,ret.getResponseCode());
     Database db = msc.getDatabase(dbName);
     String dbLocn = db.getLocationUri();
-
     validateCreateDb(db,dbName);
     disallowCreateInDb(dbName, userName, dbLocn);
-
     disallowCreateDatabase(userName);
-
     driver.run("use " + dbName);
     ret = driver.run(
         String.format("create table %s (a string) partitioned by (b string)", tblName));
-
     assertEquals(1,ret.getResponseCode());
 
     // Even if table location is specified table creation should fail
     String tblNameLoc = tblName + "_loc";
     String tblLocation = new Path(dbLocn).getParent().toUri() + "/" + tblNameLoc;
-
     driver.run("use " + dbName);
     ret = driver.run(
         String.format("create table %s (a string) partitioned by (b string) location '" +
@@ -205,10 +204,8 @@ public class TestMetastoreAuthorizationProvider extends TestCase {
     assertEquals(1, ret.getResponseCode());
 
     // failure from not having permissions to create table
-
     ArrayList<FieldSchema> fields = new ArrayList<FieldSchema>(2);
     fields.add(new FieldSchema("a", serdeConstants.STRING_TYPE_NAME, ""));
-
     Table ttbl = new Table();
     ttbl.setDbName(dbName);
     ttbl.setTableName(tblName);
@@ -237,7 +234,6 @@ public class TestMetastoreAuthorizationProvider extends TestCase {
     assertNoPrivileges(me);
 
     allowCreateInDb(dbName, userName, dbLocn);
-
     driver.run("use " + dbName);
     ret = driver.run(
         String.format("create table %s (a string) partitioned by (b string)", tblName));
@@ -246,7 +242,6 @@ public class TestMetastoreAuthorizationProvider extends TestCase {
     Table tbl = msc.getTable(dbName, tblName);
     Assert.assertTrue(tbl.isSetId());
     tbl.unsetId();
-
     validateCreateTable(tbl,tblName, dbName);
 
     // Table creation should succeed even if location is specified
@@ -338,7 +333,6 @@ public class TestMetastoreAuthorizationProvider extends TestCase {
     disallowDropOnTable(tblName, userName, tbl.getSd().getLocation());
     ret = driver.run("drop table "+tbl.getTableName());
     assertEquals(1,ret.getResponseCode());
-
   }
 
   protected void allowCreateDatabase(String userName)
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedClientSideAuthorizationProvider.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedClientSideAuthorizationProvider.java
index 086ef9e..d0645d7 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedClientSideAuthorizationProvider.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedClientSideAuthorizationProvider.java
@@ -25,6 +25,9 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.security.authorization.StorageBasedAuthorizationProvider;
 
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
 /**
  * TestStorageBasedClientSideAuthorizationProvider : Overrides
  * TestClientSideAuthorizationProvider to test StorageBasedAuthorizationProvider
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationProvider.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationProvider.java
index 0783e61..fe8e15f 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationProvider.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationProvider.java
@@ -27,6 +27,8 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.ql.security.authorization.StorageBasedAuthorizationProvider;
 
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
 /**
  * TestStorageBasedMetastoreAuthorizationProvider. Test case for
  * StorageBasedAuthorizationProvider, by overriding methods defined in
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/serde2/TestSerdeWithFieldComments.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/serde2/TestSerdeWithFieldComments.java
index b398712..b31e5b8 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/serde2/TestSerdeWithFieldComments.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/serde2/TestSerdeWithFieldComments.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hive.serde2;
 
-import junit.framework.TestCase;
+
 import org.apache.hadoop.hive.metastore.HiveMetaStoreUtils;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.MetaException;
@@ -30,8 +30,13 @@ import java.util.List;
 
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
-public class TestSerdeWithFieldComments extends TestCase {
+/**
+ * TestSerdeWithFieldComments.
+ */
+public class TestSerdeWithFieldComments {
 
   private StructField mockedStructField(String name, String oiTypeName,
                                         String comment) {
@@ -47,6 +52,7 @@ public class TestSerdeWithFieldComments extends TestCase {
     return m;
   }
 
+  @Test
   public void testFieldComments() throws MetaException, SerDeException {
     StructObjectInspector mockSOI = mock(StructObjectInspector.class);
     when(mockSOI.getCategory()).thenReturn(ObjectInspector.Category.STRUCT);
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/serde2/dynamic_type/TestDynamicSerDe.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/serde2/dynamic_type/TestDynamicSerDe.java
index 9ef678c..8fbd4d4 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/serde2/dynamic_type/TestDynamicSerDe.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/serde2/dynamic_type/TestDynamicSerDe.java
@@ -26,19 +26,23 @@ import java.util.Properties;
 import java.util.Random;
 import java.util.Map.Entry;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.thrift.TCTLSeparatedProtocol;
 import org.apache.hadoop.io.BytesWritable;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import static org.junit.Assert.assertTrue;
+import org.junit.Test;
 
 /**
  * TestDynamicSerDe.
  *
  */
-public class TestDynamicSerDe extends TestCase {
+public class TestDynamicSerDe {
 
   public static HashMap<String, String> makeHashMap(String... params) {
     HashMap<String, String> r = new HashMap<String, String>();
@@ -48,6 +52,7 @@ public class TestDynamicSerDe extends TestCase {
     return r;
   }
 
+  @Test
   public void testDynamicSerDe() throws Throwable {
     try {
 
@@ -284,6 +289,7 @@ public class TestDynamicSerDe extends TestCase {
     }
   }
 
+  @Test
   public void testTBinarySortableProtocol() throws Throwable {
     try {
 
@@ -402,6 +408,7 @@ public class TestDynamicSerDe extends TestCase {
     }
   }
 
+  @Test
   public void testConfigurableTCTLSeparated() throws Throwable {
     try {
 
@@ -481,6 +488,7 @@ public class TestDynamicSerDe extends TestCase {
    * Tests a single null list within a struct with return nulls on.
    */
 
+  @Test
   public void testNulls1() throws Throwable {
     try {
 
@@ -533,6 +541,7 @@ public class TestDynamicSerDe extends TestCase {
    * Tests all elements of a struct being null with return nulls on.
    */
 
+  @Test
   public void testNulls2() throws Throwable {
     try {
 
@@ -591,6 +600,7 @@ public class TestDynamicSerDe extends TestCase {
    * Tests map and list being empty with return nulls on.
    */
 
+  @Test
   public void testNulls3() throws Throwable {
     try {
 
@@ -649,6 +659,7 @@ public class TestDynamicSerDe extends TestCase {
    * Tests map and list null/empty with return nulls *off*.
    */
 
+  @Test
   public void testNulls4() throws Throwable {
     try {
 
@@ -709,6 +720,7 @@ public class TestDynamicSerDe extends TestCase {
    * Tests map and list null/empty with return nulls *off*.
    */
 
+  @Test
   public void testStructsinStructs() throws Throwable {
     try {
 
@@ -775,6 +787,7 @@ public class TestDynamicSerDe extends TestCase {
 
   }
 
+  @Test
   public void testSkip() throws Throwable {
     try {
 
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlapArrow.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlapArrow.java
index 1b088e2..68a515c 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlapArrow.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlapArrow.java
@@ -122,6 +122,7 @@ public class TestJdbcWithMiniLlapArrow extends BaseJdbcWithMiniLlap {
 
   // Currently MAP type is not supported. Add it back when Arrow 1.0 is released.
   // See: SPARK-21187
+  @Test
   @Override
   public void testDataTypes() throws Exception {
     createDataTypesTable("datatypes");
@@ -404,6 +405,7 @@ public class TestJdbcWithMiniLlapArrow extends BaseJdbcWithMiniLlap {
     testKillQueryByTagOwner();
   }
 
+  @Test
   public void testKillQueryById() throws Exception {
     ExceptionHolder tExecuteHolder = new ExceptionHolder();
     ExceptionHolder tKillHolder = new ExceptionHolder();
@@ -413,6 +415,7 @@ public class TestJdbcWithMiniLlapArrow extends BaseJdbcWithMiniLlap {
     assertNull("tCancel", tKillHolder.throwable);
   }
 
+  @Test
   public void testKillQueryByTagNegative() throws Exception {
     ExceptionHolder tExecuteHolder = new ExceptionHolder();
     ExceptionHolder tKillHolder = new ExceptionHolder();
@@ -422,6 +425,7 @@ public class TestJdbcWithMiniLlapArrow extends BaseJdbcWithMiniLlap {
     assertTrue(tKillHolder.throwable.getMessage(), tKillHolder.throwable.getMessage().contains("No privilege"));
   }
 
+  @Test
   public void testKillQueryByTagAdmin() throws Exception {
     ExceptionHolder tExecuteHolder = new ExceptionHolder();
     ExceptionHolder tKillHolder = new ExceptionHolder();
@@ -430,6 +434,7 @@ public class TestJdbcWithMiniLlapArrow extends BaseJdbcWithMiniLlap {
     assertNull("tCancel", tKillHolder.throwable);
   }
 
+  @Test
   public void testKillQueryByTagOwner() throws Exception {
     ExceptionHolder tExecuteHolder = new ExceptionHolder();
     ExceptionHolder tKillHolder = new ExceptionHolder();
diff --git a/ql/src/test/org/apache/hadoop/hive/metastore/TestMetastoreExpr.java b/ql/src/test/org/apache/hadoop/hive/metastore/TestMetastoreExpr.java
index 1becbb8..c861107 100644
--- a/ql/src/test/org/apache/hadoop/hive/metastore/TestMetastoreExpr.java
+++ b/ql/src/test/org/apache/hadoop/hive/metastore/TestMetastoreExpr.java
@@ -50,7 +50,12 @@ import org.apache.thrift.TException;
 
 import com.google.common.collect.Lists;
 
-import junit.framework.TestCase;
+
+import static org.junit.Assert.fail;
+import static org.junit.Assert.assertEquals;
+import org.junit.Before;
+import org.junit.After;
+import org.junit.Test;
 
 /**
  * Tests hive metastore expression support. This should be moved in metastore module
@@ -60,13 +65,13 @@ import junit.framework.TestCase;
  * it doesn't test all the edge cases of the filter (if classes were merged, perhaps the
  * filter test could be rolled into it); assumption is that they use the same path in SQL/JDO.
  */
-public class TestMetastoreExpr extends TestCase {
+public class TestMetastoreExpr {
   protected static HiveMetaStoreClient client;
 
-  @Override
-  protected void tearDown() throws Exception {
+  @After
+  public void tearDown() throws Exception {
     try {
-      super.tearDown();
+
       client.close();
     } catch (Throwable e) {
       System.err.println("Unable to close metastore");
@@ -75,9 +80,9 @@ public class TestMetastoreExpr extends TestCase {
     }
   }
 
-  @Override
-  protected void setUp() throws Exception {
-    super.setUp();
+  @Before
+  public void setUp() throws Exception {
+
     try {
       client = new HiveMetaStoreClient(new HiveConf(this.getClass()));
     } catch (Throwable e) {
@@ -98,6 +103,7 @@ public class TestMetastoreExpr extends TestCase {
     }
   }
 
+  @Test
   public void testPartitionExpr() throws Exception {
     String dbName = "filterdb";
     String tblName = "filtertbl";
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java
index 78f2585..a78fdfc 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java
@@ -24,7 +24,7 @@ import java.util.ArrayList;
 import java.util.LinkedList;
 import java.util.List;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.slf4j.Logger;
@@ -63,18 +63,21 @@ import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.mapred.TextInputFormat;
+import static org.junit.Assert.assertEquals;
+import org.junit.Before;
+import org.junit.Test;
 
 /**
  * Mimics the actual query compiler in generating end to end plans and testing
  * them out.
  *
  */
-public class TestExecDriver extends TestCase {
+public class TestExecDriver {
 
   static QueryState queryState;
   static HiveConf conf;
 
-  private static final String tmpdir;
+  private static final String TMPDIR;
   private static final Logger LOG = LoggerFactory.getLogger(TestExecDriver.class);
   private static final Path tmppath;
   private static Hive db;
@@ -92,19 +95,19 @@ public class TestExecDriver extends TestCase {
 
       SessionState.start(conf);
 
-      tmpdir = System.getProperty("test.tmp.dir");
-      tmppath = new Path(tmpdir);
+      TMPDIR = System.getProperty("test.tmp.dir");
+      tmppath = new Path(TMPDIR);
 
       fs = FileSystem.get(conf);
       if (fs.exists(tmppath) &&
           !ShimLoader.getHadoopShims().isDirectory(fs.getFileStatus(tmppath))) {
-        throw new RuntimeException(tmpdir + " exists but is not a directory");
+        throw new RuntimeException(TMPDIR + " exists but is not a directory");
       }
 
       if (!fs.exists(tmppath)) {
         if (!fs.mkdirs(tmppath)) {
           throw new RuntimeException("Could not make scratch directory "
-              + tmpdir);
+              + TMPDIR);
         }
       }
       LOG.info("Directory of actual files: " + tmppath);
@@ -142,7 +145,7 @@ public class TestExecDriver extends TestCase {
         db.createTable(src, cols, null, TextInputFormat.class,
             HiveIgnoreKeyTextOutputFormat.class);
         db.loadTable(hadoopDataFile[i], src, LoadFileType.KEEP_EXISTING,
-           true, false, false, true, null, 0, false);
+            true, false, false, true, null, 0, false);
         i++;
       }
 
@@ -153,8 +156,8 @@ public class TestExecDriver extends TestCase {
 
   MapredWork mr;
 
-  @Override
-  protected void setUp() {
+  @Before
+  public void setUp() {
     mr = PlanUtils.getMapRedWork();
     ctx = new CompilationOpContext();
   }
@@ -168,18 +171,18 @@ public class TestExecDriver extends TestCase {
     String testFileDir = conf.get("test.data.files");
 
     // inbuilt assumption that the testdir has only one output file.
-    Path di_test = new Path(tmppath, testdir);
-    if (!fs.exists(di_test)) {
-      throw new RuntimeException(tmpdir + File.separator + testdir + " does not exist");
+    Path diTest = new Path(tmppath, testdir);
+    if (!fs.exists(diTest)) {
+      throw new RuntimeException(TMPDIR + File.separator + testdir + " does not exist");
     }
-    if (!ShimLoader.getHadoopShims().isDirectory(fs.getFileStatus(di_test))) {
-      throw new RuntimeException(tmpdir + File.separator + testdir + " is not a directory");
+    if (!ShimLoader.getHadoopShims().isDirectory(fs.getFileStatus(diTest))) {
+      throw new RuntimeException(TMPDIR + File.separator + testdir + " is not a directory");
     }
-    FSDataInputStream fi_test = fs.open((fs.listStatus(di_test))[0].getPath());
+    FSDataInputStream fiTest = fs.open((fs.listStatus(diTest))[0].getPath());
 
-    FileInputStream fi_gold = new FileInputStream(new File(testFileDir,datafile));
-    if (!Utilities.contentsEqual(fi_gold, fi_test, false)) {
-      LOG.error(di_test.toString() + " does not match " + datafile);
+    FileInputStream fiGold = new FileInputStream(new File(testFileDir, datafile));
+    if (!Utilities.contentsEqual(fiGold, fiTest, false)) {
+      LOG.error(diTest.toString() + " does not match " + datafile);
       assertEquals(false, true);
     }
   }
@@ -213,7 +216,7 @@ public class TestExecDriver extends TestCase {
   @SuppressWarnings("unchecked")
   private void populateMapPlan1(Table src) throws Exception {
 
-    Operator<FileSinkDesc> op2 = OperatorFactory.get(ctx, new FileSinkDesc(new Path(tmpdir + File.separator
+    Operator<FileSinkDesc> op2 = OperatorFactory.get(ctx, new FileSinkDesc(new Path(TMPDIR + File.separator
         + "mapplan1.out"), Utilities.defaultTd, true));
     Operator<FilterDesc> op1 = OperatorFactory.get(getTestFilterDesc("key"), op2);
 
@@ -223,7 +226,7 @@ public class TestExecDriver extends TestCase {
   @SuppressWarnings("unchecked")
   private void populateMapPlan2(Table src) throws Exception {
 
-    Operator<FileSinkDesc> op3 = OperatorFactory.get(ctx, new FileSinkDesc(new Path(tmpdir + File.separator
+    Operator<FileSinkDesc> op3 = OperatorFactory.get(ctx, new FileSinkDesc(new Path(TMPDIR + File.separator
         + "mapplan2.out"), Utilities.defaultTd, false));
 
     Operator<ScriptDesc> op2 = OperatorFactory.get(new ScriptDesc("cat",
@@ -259,7 +262,7 @@ public class TestExecDriver extends TestCase {
     mr.setReduceWork(rWork);
 
     // reduce side work
-    Operator<FileSinkDesc> op3 = OperatorFactory.get(ctx, new FileSinkDesc(new Path(tmpdir + File.separator
+    Operator<FileSinkDesc> op3 = OperatorFactory.get(ctx, new FileSinkDesc(new Path(TMPDIR + File.separator
         + "mapredplan1.out"), Utilities.defaultTd, false));
 
     List<ExprNodeDesc> cols = new ArrayList<ExprNodeDesc>();
@@ -292,7 +295,7 @@ public class TestExecDriver extends TestCase {
     mr.setReduceWork(rWork);
 
     // reduce side work
-    Operator<FileSinkDesc> op4 = OperatorFactory.get(ctx, new FileSinkDesc(new Path(tmpdir + File.separator
+    Operator<FileSinkDesc> op4 = OperatorFactory.get(ctx, new FileSinkDesc(new Path(TMPDIR + File.separator
         + "mapredplan2.out"), Utilities.defaultTd, false));
 
     Operator<FilterDesc> op3 = OperatorFactory.get(getTestFilterDesc("0"), op4);
@@ -338,7 +341,7 @@ public class TestExecDriver extends TestCase {
     rWork.getTagToValueDesc().add(op2.getConf().getValueSerializeInfo());
 
     // reduce side work
-    Operator<FileSinkDesc> op4 = OperatorFactory.get(ctx, new FileSinkDesc(new Path(tmpdir + File.separator
+    Operator<FileSinkDesc> op4 = OperatorFactory.get(ctx, new FileSinkDesc(new Path(TMPDIR + File.separator
         + "mapredplan3.out"), Utilities.defaultTd, false));
 
     Operator<SelectDesc> op5 = OperatorFactory.get(new SelectDesc(Utilities
@@ -381,7 +384,7 @@ public class TestExecDriver extends TestCase {
     mr.setReduceWork(rWork);
 
     // reduce side work
-    Operator<FileSinkDesc> op3 = OperatorFactory.get(ctx, new FileSinkDesc(new Path(tmpdir + File.separator
+    Operator<FileSinkDesc> op3 = OperatorFactory.get(ctx, new FileSinkDesc(new Path(TMPDIR + File.separator
         + "mapredplan4.out"), Utilities.defaultTd, false));
     List<ExprNodeDesc> cols = new ArrayList<ExprNodeDesc>();
     cols.add(getStringColumn(Utilities.ReduceField.KEY + ".reducesinkkey" + 0));
@@ -420,7 +423,7 @@ public class TestExecDriver extends TestCase {
     rWork.getTagToValueDesc().add(op0.getConf().getValueSerializeInfo());
 
     // reduce side work
-    Operator<FileSinkDesc> op3 = OperatorFactory.get(ctx, new FileSinkDesc(new Path(tmpdir + File.separator
+    Operator<FileSinkDesc> op3 = OperatorFactory.get(ctx, new FileSinkDesc(new Path(TMPDIR + File.separator
         + "mapredplan5.out"), Utilities.defaultTd, false));
 
     List<ExprNodeDesc> cols = new ArrayList<ExprNodeDesc>();
@@ -462,7 +465,7 @@ public class TestExecDriver extends TestCase {
     rWork.getTagToValueDesc().add(op1.getConf().getValueSerializeInfo());
 
     // reduce side work
-    Operator<FileSinkDesc> op3 = OperatorFactory.get(ctx, new FileSinkDesc(new Path(tmpdir + File.separator
+    Operator<FileSinkDesc> op3 = OperatorFactory.get(ctx, new FileSinkDesc(new Path(TMPDIR + File.separator
         + "mapredplan6.out"), Utilities.defaultTd, false));
 
     Operator<FilterDesc> op2 = OperatorFactory.get(getTestFilterDesc("0"), op3);
@@ -478,7 +481,7 @@ public class TestExecDriver extends TestCase {
   private void executePlan() throws Exception {
     String testName = new Exception().getStackTrace()[1].getMethodName();
     MapRedTask mrtask = new MapRedTask();
-    DriverContext dctx = new DriverContext ();
+    DriverContext dctx = new DriverContext();
     mrtask.setWork(mr);
     mrtask.initialize(queryState, null, dctx, null);
     int exitVal =  mrtask.execute(dctx);
@@ -491,6 +494,7 @@ public class TestExecDriver extends TestCase {
     LOG.info(testName + " execution completed successfully");
   }
 
+  @Test
   public void testMapPlan1() throws Exception {
 
     LOG.info("Beginning testMapPlan1");
@@ -499,6 +503,7 @@ public class TestExecDriver extends TestCase {
     fileDiff("lt100.txt.deflate", "mapplan1.out");
   }
 
+  @Test
   public void testMapPlan2() throws Exception {
 
     LOG.info("Beginning testMapPlan2");
@@ -507,6 +512,7 @@ public class TestExecDriver extends TestCase {
     fileDiff("lt100.txt", "mapplan2.out");
   }
 
+  @Test
   public void testMapRedPlan1() throws Exception {
 
     LOG.info("Beginning testMapRedPlan1");
@@ -516,6 +522,7 @@ public class TestExecDriver extends TestCase {
     fileDiff("kv1.val.sorted.txt", "mapredplan1.out");
   }
 
+  @Test
   public void testMapRedPlan2() throws Exception {
 
     LOG.info("Beginning testMapPlan2");
@@ -525,6 +532,7 @@ public class TestExecDriver extends TestCase {
     fileDiff("lt100.sorted.txt", "mapredplan2.out");
   }
 
+  @Test
   public void testMapRedPlan3() throws Exception {
 
     LOG.info("Beginning testMapPlan3");
@@ -534,6 +542,7 @@ public class TestExecDriver extends TestCase {
     fileDiff("kv1kv2.cogroup.txt", "mapredplan3.out");
   }
 
+  @Test
   public void testMapRedPlan4() throws Exception {
 
     LOG.info("Beginning testMapPlan4");
@@ -543,6 +552,7 @@ public class TestExecDriver extends TestCase {
     fileDiff("kv1.string-sorted.txt", "mapredplan4.out");
   }
 
+  @Test
   public void testMapRedPlan5() throws Exception {
 
     LOG.info("Beginning testMapPlan5");
@@ -552,6 +562,7 @@ public class TestExecDriver extends TestCase {
     fileDiff("kv1.string-sorted.txt", "mapredplan5.out");
   }
 
+  @Test
   public void testMapRedPlan6() throws Exception {
 
     LOG.info("Beginning testMapPlan6");
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExpressionEvaluator.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExpressionEvaluator.java
index 32f1499..34fe2b9 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExpressionEvaluator.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExpressionEvaluator.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hive.ql.exec;
 
 import java.util.ArrayList;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -40,12 +40,15 @@ import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
 import org.apache.hadoop.io.Text;
+import static org.junit.Assert.assertEquals;
+import org.junit.Before;
+import org.junit.Test;
 
 /**
  * TestExpressionEvaluator.
  *
  */
-public class TestExpressionEvaluator extends TestCase {
+public class TestExpressionEvaluator {
 
   // this is our row to test expressions on
   protected InspectableObject r;
@@ -97,10 +100,11 @@ public class TestExpressionEvaluator extends TestCase {
     }
   }
 
-  @Override
-  protected void setUp() {
+  @Before
+  public void setUp() {
   }
 
+  @Test
   public void testExprNodeColumnEvaluator() throws Throwable {
     try {
       // get a evaluator for a simple field expression
@@ -136,6 +140,7 @@ public class TestExpressionEvaluator extends TestCase {
         children);
   }
 
+  @Test
   public void testExprNodeFuncEvaluator() throws Throwable {
     try {
       // get a evaluator for a string concatenation expression
@@ -161,6 +166,7 @@ public class TestExpressionEvaluator extends TestCase {
     }
   }
 
+  @Test
   public void testExprNodeConversionEvaluator() throws Throwable {
     try {
       // get a evaluator for a string concatenation expression
@@ -204,6 +210,7 @@ public class TestExpressionEvaluator extends TestCase {
         + " seconds/million call.");
   }
 
+  @Test
   public void testExprNodeSpeed() throws Throwable {
     try {
       int basetimes = 100000;
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java
index 40d60f3..dead3ec 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java
@@ -23,7 +23,7 @@ import java.util.ArrayList;
 import java.util.LinkedList;
 import java.util.List;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.common.type.HiveVarchar;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -50,9 +50,21 @@ import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Text;
 import org.junit.Assert;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import org.junit.Before;
+import org.junit.After;
+import org.junit.Test;
+
+/**
+ * FunctionRegistry Test.
+ */
+public class TestFunctionRegistry {
 
-public class TestFunctionRegistry extends TestCase {
-
+  /**
+   * Test UDF class.
+   */
   public class TestUDF {
     public void same(DoubleWritable x, DoubleWritable y) {}
     public void same(HiveDecimalWritable x, HiveDecimalWritable y) {}
@@ -76,8 +88,8 @@ public class TestFunctionRegistry extends TestCase {
   TypeInfo char5;
   TypeInfo char10;
 
-  @Override
-  protected void setUp() {
+  @Before
+  public void setUp() {
     String maxVarcharTypeName = "varchar(" + HiveVarchar.MAX_VARCHAR_LENGTH + ")";
     maxVarchar = TypeInfoFactory.getPrimitiveTypeInfo(maxVarcharTypeName);
     varchar10 = TypeInfoFactory.getPrimitiveTypeInfo("varchar(10)");
@@ -91,6 +103,7 @@ public class TestFunctionRegistry extends TestCase {
     assertEquals(convertible, TypeInfoUtils.implicitConvertible(a, b));
   }
 
+  @Test
   public void testImplicitConversion() {
     implicit(TypeInfoFactory.intTypeInfo, TypeInfoFactory.decimalTypeInfo, true);
     implicit(TypeInfoFactory.longTypeInfo, TypeInfoFactory.decimalTypeInfo, true);
@@ -149,6 +162,7 @@ public class TestFunctionRegistry extends TestCase {
     }
   }
 
+  @Test
   public void testTypeAffinity() {
     // Prefer numeric type arguments over other method signatures
     typeAffinity("typeaffinity1", TypeInfoFactory.shortTypeInfo, 1, DoubleWritable.class);
@@ -191,6 +205,7 @@ public class TestFunctionRegistry extends TestCase {
     assertEquals(b, result.getParameterTypes()[1]);
   }
 
+  @Test
   public void testGetMethodInternal() {
 
     verify(TestUDF.class, "same", TypeInfoFactory.intTypeInfo, TypeInfoFactory.intTypeInfo,
@@ -226,9 +241,10 @@ public class TestFunctionRegistry extends TestCase {
   }
 
   private void common(TypeInfo a, TypeInfo b, TypeInfo result) {
-    assertEquals(result, FunctionRegistry.getCommonClass(a,b));
+    assertEquals(result, FunctionRegistry.getCommonClass(a, b));
   }
 
+  @Test
   public void testCommonClass() {
     common(TypeInfoFactory.intTypeInfo, TypeInfoFactory.decimalTypeInfo,
            TypeInfoFactory.decimalTypeInfo);
@@ -248,9 +264,10 @@ public class TestFunctionRegistry extends TestCase {
   }
 
   private void comparison(TypeInfo a, TypeInfo b, TypeInfo result) {
-    assertEquals(result, FunctionRegistry.getCommonClassForComparison(a,b));
+    assertEquals(result, FunctionRegistry.getCommonClassForComparison(a, b));
   }
 
+  @Test
   public void testCommonClassComparison() {
     comparison(TypeInfoFactory.intTypeInfo, TypeInfoFactory.decimalTypeInfo,
                TypeInfoFactory.decimalTypeInfo);
@@ -274,7 +291,7 @@ public class TestFunctionRegistry extends TestCase {
         TypeInfoFactory.doubleTypeInfo);
     comparison(TypeInfoFactory.timestampTypeInfo, TypeInfoFactory.intTypeInfo,
         TypeInfoFactory.doubleTypeInfo);
-   comparison(TypeInfoFactory.timestampTypeInfo, TypeInfoFactory.dateTypeInfo,
+    comparison(TypeInfoFactory.timestampTypeInfo, TypeInfoFactory.dateTypeInfo,
         TypeInfoFactory.timestampTypeInfo);
 
     comparison(TypeInfoFactory.stringTypeInfo, varchar10, TypeInfoFactory.stringTypeInfo);
@@ -290,6 +307,7 @@ public class TestFunctionRegistry extends TestCase {
   /**
    * Method to print out the comparison/conversion behavior for data types.
    */
+  @Test
   public void testPrintTypeCompatibility() {
     if (true) {
       return;
@@ -334,9 +352,10 @@ public class TestFunctionRegistry extends TestCase {
   }
 
   private void unionAll(TypeInfo a, TypeInfo b, TypeInfo result) {
-    assertEquals(result, FunctionRegistry.getCommonClassForUnionAll(a,b));
+    assertEquals(result, FunctionRegistry.getCommonClassForUnionAll(a, b));
   }
 
+  @Test
   public void testCommonClassUnionAll() {
     unionAll(TypeInfoFactory.doubleTypeInfo, TypeInfoFactory.intTypeInfo,
         TypeInfoFactory.doubleTypeInfo);
@@ -364,6 +383,7 @@ public class TestFunctionRegistry extends TestCase {
 
   }
 
+  @Test
   public void testGetTypeInfoForPrimitiveCategory() {
     // varchar should take string length into account.
     // varchar(5), varchar(10) => varchar(10)
@@ -394,10 +414,11 @@ public class TestFunctionRegistry extends TestCase {
         PrimitiveCategory.DOUBLE));
   }
 
-  @Override
-  protected void tearDown() {
+  @After
+  public void tearDown() {
   }
 
+  @Test
   public void testIsRankingFunction() throws Exception {
     Assert.assertTrue(FunctionRegistry.isRankingFunction("rank"));
     Assert.assertTrue(FunctionRegistry.isRankingFunction("dense_rank"));
@@ -406,6 +427,7 @@ public class TestFunctionRegistry extends TestCase {
     Assert.assertFalse(FunctionRegistry.isRankingFunction("min"));
   }
 
+  @Test
   public void testImpliesOrder() throws Exception {
     Assert.assertTrue(FunctionRegistry.impliesOrder("rank"));
     Assert.assertTrue(FunctionRegistry.impliesOrder("dense_rank"));
@@ -418,6 +440,7 @@ public class TestFunctionRegistry extends TestCase {
     Assert.assertFalse(FunctionRegistry.impliesOrder("min"));
   }
 
+  @Test
   public void testRegisterTemporaryFunctions() throws Exception {
     FunctionResource[] emptyResources = new FunctionResource[] {};
 
@@ -432,7 +455,7 @@ public class TestFunctionRegistry extends TestCase {
     assertFalse(functionInfo.isNative());
 
     // GenericUDAF
-    FunctionRegistry.registerTemporaryUDF("tmp_max",GenericUDAFMax.class, emptyResources);
+    FunctionRegistry.registerTemporaryUDF("tmp_max", GenericUDAFMax.class, emptyResources);
     functionInfo = FunctionRegistry.getFunctionInfo("tmp_max");
     assertFalse(functionInfo.isNative());
     functionInfo = FunctionRegistry.getWindowFunctionInfo("tmp_max");
@@ -444,6 +467,7 @@ public class TestFunctionRegistry extends TestCase {
     assertFalse(functionInfo.isNative());
   }
 
+  @Test
   public void testRegisterPermanentFunction() throws Exception {
     FunctionResource[] emptyResources = new FunctionResource[] {};
 
@@ -485,12 +509,14 @@ public class TestFunctionRegistry extends TestCase {
     assertFalse(functionInfo.isBuiltIn());
   }
 
+  @Test
   public void testBuiltInFunction() throws Exception {
     FunctionInfo functionInfo = FunctionRegistry.getFunctionInfo("ln");
     assertTrue(functionInfo.isBuiltIn());
     assertTrue(functionInfo.isNative());
   }
 
+  @Test
   public void testIsPermanentFunction() throws Exception {
     // Setup exprNode
     GenericUDF udf = new GenericUDFCurrentTimestamp();
@@ -530,6 +556,7 @@ public class TestFunctionRegistry extends TestCase {
     assertFalse(FunctionRegistry.isConsistentWithinQuery(udf));
   }
 
+  @Test
   public void testDeterminism() throws Exception {
     checkDeterministicFn(getUDF("+"));
     checkDeterministicFn(getUDF("ascii"));
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperatorNames.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperatorNames.java
index de3ecb0..530ff91 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperatorNames.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperatorNames.java
@@ -18,7 +18,7 @@
  
 package org.apache.hadoop.hive.ql.exec;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.ql.exec.vector.VectorAppMasterEventOperator;
 import org.apache.hadoop.hive.ql.exec.vector.VectorFilterOperator;
@@ -32,6 +32,9 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorSelectOperator;
 import org.apache.hadoop.hive.ql.exec.vector.VectorSparkHashTableSinkOperator;
 import org.apache.hadoop.hive.ql.exec.vector.VectorSparkPartitionPruningSinkOperator;
 import org.apache.hadoop.hive.ql.parse.spark.SparkPartitionPruningSinkOperator;
+import static org.junit.Assert.assertEquals;
+import org.junit.Before;
+import org.junit.After;
 import org.junit.Test;
 
 /*
@@ -50,20 +53,20 @@ import org.junit.Test;
  * License for the specific language governing permissions and limitations under
  * the License.
  */
-public class TestOperatorNames extends TestCase {
 
-  public TestOperatorNames(String name) {
-    super(name);
-  }
+/**
+ * OperatorNames Test.
+ */
+public class TestOperatorNames {
+
+  @Before
+  public void setUp() throws Exception {
 
-  @Override
-  protected void setUp() throws Exception {
-    super.setUp();
   }
 
-  @Override
-  protected void tearDown() throws Exception {
-    super.tearDown();
+  @After
+  public void tearDown() throws Exception {
+
   }
 
   /*
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperators.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperators.java
index a363b22..0f83310 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperators.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperators.java
@@ -62,23 +62,26 @@ import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.TextInputFormat;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.junit.Assert;
-import org.junit.Test;
 import org.mockito.Mockito;
 import static org.mockito.Mockito.when;
 
-import junit.framework.TestCase;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import org.junit.Before;
+import org.junit.Test;
 
 /**
  * TestOperators.
  *
  */
-public class TestOperators extends TestCase {
+public class TestOperators {
 
   // this is our row to test expressions on
   protected InspectableObject[] r;
 
-  @Override
-  protected void setUp() {
+  @Before
+  public void setUp() {
     r = new InspectableObject[5];
     ArrayList<String> names = new ArrayList<String>(3);
     names.add("col0");
@@ -108,7 +111,7 @@ public class TestOperators extends TestCase {
     }
   }
 
-  private void testTaskIds(String [] taskIds, String expectedAttemptId, String expectedTaskId) {
+  private void testTaskIds(String[] taskIds, String expectedAttemptId, String expectedTaskId) {
     Configuration conf = new JobConf(TestOperators.class);
     for (String one: taskIds) {
       conf.set("mapred.task.id", one);
@@ -116,8 +119,8 @@ public class TestOperators extends TestCase {
       assertEquals(expectedAttemptId, attemptId);
       assertEquals(Utilities.getTaskIdFromFilename(attemptId), expectedTaskId);
       assertEquals(Utilities.getTaskIdFromFilename(attemptId + ".gz"), expectedTaskId);
-      assertEquals(Utilities.getTaskIdFromFilename
-                   (Utilities.toTempPath(new Path(attemptId + ".gz")).toString()), expectedTaskId);
+      assertEquals(Utilities.getTaskIdFromFilename(
+                   Utilities.toTempPath(new Path(attemptId + ".gz")).toString()), expectedTaskId);
     }
   }
 
@@ -126,26 +129,27 @@ public class TestOperators extends TestCase {
    * file naming libraries
    * The old test was deactivated as part of hive-405
    */
+  @Test
   public void testFileSinkOperator() throws Throwable {
 
     try {
-      testTaskIds (new String [] {
+      testTaskIds(new String[] {
           "attempt_200707121733_0003_m_000005_0",
           "attempt_local_0001_m_000005_0",
           "task_200709221812_0001_m_000005_0",
           "task_local_0001_m_000005_0"
-        }, "000005_0", "000005");
+          }, "000005_0", "000005");
 
-      testTaskIds (new String [] {
+      testTaskIds(new String[] {
           "job_local_0001_map_000005",
           "job_local_0001_reduce_000005",
-        }, "000005", "000005");
+          }, "000005", "000005");
 
-      testTaskIds (new String [] {"1234567"},
+      testTaskIds(new String[] {"1234567"},
                    "1234567", "1234567");
 
-      assertEquals(Utilities.getTaskIdFromFilename
-                   ("/mnt/dev005/task_local_0001_m_000005_0"),
+      assertEquals(Utilities.getTaskIdFromFilename(
+                   "/mnt/dev005/task_local_0001_m_000005_0"),
                    "000005");
 
       System.out.println("FileSink Operator ok");
@@ -160,6 +164,7 @@ public class TestOperators extends TestCase {
    *  variables. But environment variables have some system limitations and we have to check
    *  job configuration properties firstly. This test checks that staff.
    */
+  @Test
   public void testScriptOperatorEnvVarsProcessing() throws Throwable {
     try {
       ScriptOperator scriptOperator = new ScriptOperator(new CompilationOpContext());
@@ -173,7 +178,7 @@ public class TestOperators extends TestCase {
       assertEquals("value", scriptOperator.safeEnvVarValue("value", "name", true));
 
       //Environment Variables long values
-      char [] array = new char[20*1024+1];
+      char[] array = new char[20*1024+1];
       Arrays.fill(array, 'a');
       String hugeEnvVar = new String(array);
       assertEquals(20*1024+1, hugeEnvVar.length());
@@ -200,6 +205,7 @@ public class TestOperators extends TestCase {
     }
   }
 
+  @Test
   public void testScriptOperatorBlacklistedEnvVarsProcessing() {
     ScriptOperator scriptOperator = new ScriptOperator(new CompilationOpContext());
 
@@ -215,6 +221,7 @@ public class TestOperators extends TestCase {
     Assert.assertTrue(env.containsKey("barfoo"));
   }
 
+  @Test
   public void testScriptOperator() throws Throwable {
     try {
       System.out.println("Testing Script Operator");
@@ -289,6 +296,7 @@ public class TestOperators extends TestCase {
     }
   }
 
+  @Test
   public void testMapOperator() throws Throwable {
     try {
       System.out.println("Testing Map Operator");
@@ -310,7 +318,7 @@ public class TestOperators extends TestCase {
       TableDesc td = Utilities.defaultTd;
       PartitionDesc pd = new PartitionDesc(td, null);
       LinkedHashMap<Path, org.apache.hadoop.hive.ql.plan.PartitionDesc> pathToPartitionInfo =
-        new LinkedHashMap<>();
+          new LinkedHashMap<>();
       pathToPartitionInfo.put(new Path("hdfs:///testDir"), pd);
 
       // initialize aliasToWork
@@ -323,7 +331,7 @@ public class TestOperators extends TestCase {
           .get(ctx, CollectDesc.class);
       cdop2.setConf(cd);
       LinkedHashMap<String, Operator<? extends OperatorDesc>> aliasToWork =
-        new LinkedHashMap<String, Operator<? extends OperatorDesc>>();
+          new LinkedHashMap<String, Operator<? extends OperatorDesc>>();
       aliasToWork.put("a", cdop1);
       aliasToWork.put("b", cdop2);
 
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestPartitionKeySampler.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestPartitionKeySampler.java
index 206bf08..c6d734f 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestPartitionKeySampler.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestPartitionKeySampler.java
@@ -18,46 +18,53 @@
 
 package org.apache.hadoop.hive.ql.exec;
 
-import junit.framework.TestCase;
+
 
 import java.util.Arrays;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import org.junit.Test;
 
-public class TestPartitionKeySampler extends TestCase {
+/**
+ * PartitionKeySampler Test.
+ */
+public class TestPartitionKeySampler {
 
-  private static final byte[] _100 = "100".getBytes();
-  private static final byte[] _200 = "200".getBytes();
-  private static final byte[] _300 = "300".getBytes();
-  private static final byte[] _400 = "400".getBytes();
+  private static final byte[] tmp100 = "100".getBytes();
+  private static final byte[] tmp200 = "200".getBytes();
+  private static final byte[] tmp300 = "300".getBytes();
+  private static final byte[] tmp400 = "400".getBytes();
 
   // current random sampling implementation in InputSampler always returns
   // value of index 3, 5, 8, which can be same with previous partition key.
   // That induces "Split points are out of order" exception in TotalOrderPartitioner causing HIVE-7699
+  @Test
   public void test() throws Throwable {
     byte[][] sampled;
     sampled = new byte[][] {
-        _100, _100, _100, _100, _100, _100, _100, _100, _100, _100
+        tmp100, tmp100, tmp100, tmp100, tmp100, tmp100, tmp100, tmp100, tmp100, tmp100
     };
-    assertKeys(sampled, _100); // 3
+    assertKeys(sampled, tmp100); // 3
 
     sampled = new byte[][] {
-        _100, _100, _100, _100, _100, _100, _100, _100, _200, _200
+        tmp100, tmp100, tmp100, tmp100, tmp100, tmp100, tmp100, tmp100, tmp200, tmp200
     };
-    assertKeys(sampled, _100, _200); // 3, 8
+    assertKeys(sampled, tmp100, tmp200); // 3, 8
 
     sampled = new byte[][] {
-        _100, _100, _100, _100 , _200, _200, _200, _300, _300, _300
+        tmp100, tmp100, tmp100, tmp100, tmp200, tmp200, tmp200, tmp300, tmp300, tmp300
     };
-    assertKeys(sampled, _100, _200, _300); // 3, 5, 8
+    assertKeys(sampled, tmp100, tmp200, tmp300); // 3, 5, 8
 
     sampled = new byte[][] {
-        _100, _200, _200, _200, _200, _200, _200, _300, _300, _400
+        tmp100, tmp200, tmp200, tmp200, tmp200, tmp200, tmp200, tmp300, tmp300, tmp400
     };
-    assertKeys(sampled, _200, _300, _400); // 3, 7, 9
+    assertKeys(sampled, tmp200, tmp300, tmp400); // 3, 7, 9
 
     sampled = new byte[][] {
-        _100, _200, _300, _400, _400, _400, _400, _400, _400, _400
+        tmp100, tmp200, tmp300, tmp400, tmp400, tmp400, tmp400, tmp400, tmp400, tmp400
     };
-    assertKeys(sampled, _400);  // 3
+    assertKeys(sampled, tmp400);  // 3
   }
 
   private void assertKeys(byte[][] sampled, byte[]... expected) {
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestPlan.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestPlan.java
index d8e4347..ca22734 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestPlan.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestPlan.java
@@ -38,25 +38,28 @@ import org.apache.hadoop.hive.ql.plan.TableDesc;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.mapred.JobConf;
 
-import junit.framework.TestCase;
+
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
 /**
  * TestPlan.
  *
  */
-public class TestPlan extends TestCase {
+public class TestPlan {
 
+  @Test
   public void testPlan() throws Exception {
 
-    final String F1 = "#affiliations";
-    final String F2 = "friends[0].friendid";
+    final String f1 = "#affiliations";
+    final String f2 = "friends[0].friendid";
 
     try {
       // initialize a complete map reduce configuration
       ExprNodeDesc expr1 = new ExprNodeColumnDesc(
-          TypeInfoFactory.stringTypeInfo, F1, "", false);
+          TypeInfoFactory.stringTypeInfo, f1, "", false);
       ExprNodeDesc expr2 = new ExprNodeColumnDesc(
-          TypeInfoFactory.stringTypeInfo, F2, "", false);
+          TypeInfoFactory.stringTypeInfo, f2, "", false);
       ExprNodeDesc filterExpr = TypeCheckProcFactory.DefaultExprProcessor
           .getFuncExprNodeDesc("==", expr1, expr2);
 
@@ -75,7 +78,7 @@ public class TestPlan extends TestCase {
       pt.put(new Path("/tmp/testfolder"), partDesc);
 
       LinkedHashMap<String, Operator<? extends OperatorDesc>> ao =
-        new LinkedHashMap<String, Operator<? extends OperatorDesc>>();
+          new LinkedHashMap<String, Operator<? extends OperatorDesc>>();
       ao.put("a", op);
 
       MapredWork mrwork = new MapredWork();
@@ -94,7 +97,7 @@ public class TestPlan extends TestCase {
 
       job.set("fs.default.name", "file:///");
       Utilities.setMapRedWork(job, mrwork, new Path(System.getProperty("java.io.tmpdir") + File.separator +
-        System.getProperty("user.name") + File.separator + "hive"));
+          System.getProperty("user.name") + File.separator + "hive"));
       MapredWork mrwork2 = Utilities.getMapRedWork(job);
       Utilities.clearWork(job);
 
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorGroupByOperator.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorGroupByOperator.java
index 278f167..f2d95bb 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorGroupByOperator.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorGroupByOperator.java
@@ -584,6 +584,7 @@ public class TestVectorGroupByOperator {
         buildHashMap((byte)1, 2L, null, 1L));
   }
 
+  @Test
   public void testDoubleValueTypeCountOneKey() throws HiveException {
     testKeyTypeAggregate(
         "count",
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorRowObject.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorRowObject.java
index 1f2b171..92ca0f6 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorRowObject.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorRowObject.java
@@ -23,12 +23,14 @@ import java.util.Random;
 
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 
-import junit.framework.TestCase;
+
+import static org.junit.Assert.fail;
+import org.junit.Test;
 
 /**
  * Unit test for the vectorized conversion to and from row object[].
  */
-public class TestVectorRowObject extends TestCase {
+public class TestVectorRowObject {
 
   void examineBatch(VectorizedRowBatch batch, VectorExtractRow vectorExtractRow,
               Object[][] randomRows, int firstRandomRowIndex ) {
@@ -98,6 +100,7 @@ public class TestVectorRowObject extends TestCase {
     }
   }
 
+  @Test
   public void testVectorRowObject() throws Throwable {
 
     try {
@@ -118,4 +121,4 @@ public class TestVectorRowObject extends TestCase {
       throw e;
     }
   }
-}
\ No newline at end of file
+}
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorSerDeRow.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorSerDeRow.java
index 8b1b612..57bac7f 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorSerDeRow.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorSerDeRow.java
@@ -54,12 +54,15 @@ import org.apache.hadoop.hive.serde2.io.HiveCharWritable;
 import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
 import org.apache.hadoop.io.Text;
 
-import junit.framework.TestCase;
+
+import static org.junit.Assert.fail;
+import static org.junit.Assert.assertTrue;
+import org.junit.Test;
 
 /**
  * Unit test for the vectorized serialize and deserialize row.
  */
-public class TestVectorSerDeRow extends TestCase {
+public class TestVectorSerDeRow {
 
   public static enum SerializationType {
     NONE,
@@ -77,7 +80,7 @@ public class TestVectorSerDeRow extends TestCase {
       Object complexFieldObj = VectorVerifyFast.deserializeReadComplexType(deserializeRead, typeInfo);
       if (expectedObject == null) {
         if (complexFieldObj != null) {
-          TestCase.fail("Field reports not null but object is null (class " + complexFieldObj.getClass().getName() +
+          fail("Field reports not null but object is null (class " + complexFieldObj.getClass().getName() +
               ", " + complexFieldObj.toString() + ")");
         }
       } else {
@@ -89,12 +92,12 @@ public class TestVectorSerDeRow extends TestCase {
               return;
             }
           }
-          TestCase.fail("Field reports null but object is not null (class " + expectedObject.getClass().getName() +
+          fail("Field reports null but object is not null (class " + expectedObject.getClass().getName() +
               ", " + expectedObject.toString() + ")");
         }
       }
       if (!VerifyLazy.lazyCompare(typeInfo, complexFieldObj, expectedObject)) {
-        TestCase.fail("Comparision failed typeInfo " + typeInfo.toString());
+        fail("Comparision failed typeInfo " + typeInfo.toString());
       }
     }
   }
@@ -111,7 +114,7 @@ public class TestVectorSerDeRow extends TestCase {
       TypeInfo typeInfo = typeInfos[i];
       verifyRead(deserializeRead, typeInfo, expected);
     }
-    TestCase.assertTrue(deserializeRead.isEndOfInputReached());
+    assertTrue(deserializeRead.isEndOfInputReached());
   }
 
   void serializeBatch(
@@ -555,21 +558,25 @@ public class TestVectorSerDeRow extends TestCase {
     }
   }
 
+  @Test
   public void testVectorBinarySortableSerializeRow() throws Throwable {
     Random r = new Random(8732);
     testVectorSerializeRow(r, SerializationType.BINARY_SORTABLE);
   }
 
+  @Test
   public void testVectorLazyBinarySerializeRow() throws Throwable {
     Random r = new Random(8732);
     testVectorSerializeRow(r, SerializationType.LAZY_BINARY);
   }
 
+  @Test
   public void testVectorLazySimpleSerializeRow() throws Throwable {
     Random r = new Random(8732);
     testVectorSerializeRow(r, SerializationType.LAZY_SIMPLE);
   }
  
+  @Test
   public void testVectorBinarySortableDeserializeRow() throws Throwable {
     Random r = new Random(8732);
     testVectorDeserializeRow(r,
@@ -621,6 +628,7 @@ public class TestVectorSerDeRow extends TestCase {
         /* useExternalBuffer */ true);
   }
 
+  @Test
   public void testVectorLazyBinaryDeserializeRow() throws Throwable {
     Random r = new Random(8732);
     testVectorDeserializeRow(r,
@@ -636,6 +644,7 @@ public class TestVectorSerDeRow extends TestCase {
         /* useExternalBuffer */ true);
   }
 
+  @Test
   public void testVectorLazySimpleDeserializeRow() throws Throwable {
     Random r = new Random(8732);
     testVectorDeserializeRow(r,
@@ -662,4 +671,4 @@ public class TestVectorSerDeRow extends TestCase {
         /* alternate2 = unused */ false,
         /* useExternalBuffer */ true);
   }
-}
\ No newline at end of file
+}
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/TestDebugDisplay.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/TestDebugDisplay.java
index 942d1b5..d7536e9 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/TestDebugDisplay.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/TestDebugDisplay.java
@@ -18,13 +18,15 @@
 
 package org.apache.hadoop.hive.ql.exec.vector.mapjoin;
 
-import junit.framework.TestCase;
+
+import org.junit.Test;
 
 /**
  * Unit test for the vectorized conversion to and from row object[].
  */
-public class TestDebugDisplay extends TestCase {
+public class TestDebugDisplay {
 
+  @Test
   public void testDebugDisplay() throws Throwable {
 
   try {
@@ -60,4 +62,4 @@ public class TestDebugDisplay extends TestCase {
     throw e;
   }
   }
-}
\ No newline at end of file
+}
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/TestAcidUtils.java b/ql/src/test/org/apache/hadoop/hive/ql/io/TestAcidUtils.java
index c5faec5..ea31557 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/TestAcidUtils.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/TestAcidUtils.java
@@ -651,6 +651,7 @@ public class TestAcidUtils {
   /**
    * See {@link TestOrcRawRecordMerger#testGetLogicalLength()}
    */
+  @Test
   public void testGetLogicalLength() throws Exception {
   }
-}
\ No newline at end of file
+}
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/TestCombineHiveInputFormat.java b/ql/src/test/org/apache/hadoop/hive/ql/io/TestCombineHiveInputFormat.java
index 07cef93..e75412d 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/TestCombineHiveInputFormat.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/TestCombineHiveInputFormat.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hive.ql.io;
 
-import junit.framework.TestCase;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -35,42 +35,44 @@ import org.apache.hadoop.util.ReflectionUtils;
 import java.io.IOException;
 import java.util.LinkedHashMap;
 import java.util.Set;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
 /**
  * Unittest for CombineHiveInputFormat.
  */
-public class TestCombineHiveInputFormat extends TestCase {
-    public void testAvoidSplitCombination() throws Exception {
-        Configuration conf = new Configuration();
-        JobConf job = new JobConf(conf);
+public class TestCombineHiveInputFormat {
+  @Test
+  public void testAvoidSplitCombination() throws Exception {
+    Configuration conf = new Configuration();
+    JobConf job = new JobConf(conf);
 
-        TableDesc tblDesc = Utilities.defaultTd;
-        tblDesc.setInputFileFormatClass(TestSkipCombineInputFormat.class);
-        PartitionDesc partDesc = new PartitionDesc(tblDesc, null);
-        LinkedHashMap<Path, PartitionDesc> pt = new LinkedHashMap<>();
-        pt.put(new Path("/tmp/testfolder1"), partDesc);
-        pt.put(new Path("/tmp/testfolder2"), partDesc);
-        MapredWork mrwork = new MapredWork();
-        mrwork.getMapWork().setPathToPartitionInfo(pt);
-        Path mapWorkPath = new Path("/tmp/" + System.getProperty("user.name"), "hive");
-        Utilities.setMapRedWork(conf, mrwork,
-            mapWorkPath);
+    TableDesc tblDesc = Utilities.defaultTd;
+    tblDesc.setInputFileFormatClass(TestSkipCombineInputFormat.class);
+    PartitionDesc partDesc = new PartitionDesc(tblDesc, null);
+    LinkedHashMap<Path, PartitionDesc> pt = new LinkedHashMap<>();
+    pt.put(new Path("/tmp/testfolder1"), partDesc);
+    pt.put(new Path("/tmp/testfolder2"), partDesc);
+    MapredWork mrwork = new MapredWork();
+    mrwork.getMapWork().setPathToPartitionInfo(pt);
+    Path mapWorkPath = new Path("/tmp/" + System.getProperty("user.name"), "hive");
+    Utilities.setMapRedWork(conf, mrwork, mapWorkPath);
 
-        try {
-            Path[] paths = new Path[2];
-            paths[0] = new Path("/tmp/testfolder1");
-            paths[1] = new Path("/tmp/testfolder2");
-            CombineHiveInputFormat combineInputFormat =
-                ReflectionUtils.newInstance(CombineHiveInputFormat.class, conf);
-            combineInputFormat.pathToPartitionInfo =
-                Utilities.getMapWork(conf).getPathToPartitionInfo();
-            Set results = combineInputFormat.getNonCombinablePathIndices(job, paths, 2);
-            assertEquals("Should have both path indices in the results set", 2, results.size());
-        } finally {
-            // Cleanup the mapwork path
-            FileSystem.get(conf).delete(mapWorkPath, true);
-        }
+    try {
+      Path[] paths = new Path[2];
+      paths[0] = new Path("/tmp/testfolder1");
+      paths[1] = new Path("/tmp/testfolder2");
+      CombineHiveInputFormat combineInputFormat =
+          ReflectionUtils.newInstance(CombineHiveInputFormat.class, conf);
+      combineInputFormat.pathToPartitionInfo =
+          Utilities.getMapWork(conf).getPathToPartitionInfo();
+      Set results = combineInputFormat.getNonCombinablePathIndices(job, paths, 2);
+      assertEquals("Should have both path indices in the results set", 2, results.size());
+    } finally {
+      // Cleanup the mapwork path
+      FileSystem.get(conf).delete(mapWorkPath, true);
     }
+  }
 
     public static class TestSkipCombineInputFormat extends FileInputFormat
         implements CombineHiveInputFormat.AvoidSplitCombination {
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/TestHiveBinarySearchRecordReader.java b/ql/src/test/org/apache/hadoop/hive/ql/io/TestHiveBinarySearchRecordReader.java
index 3c53878..89fff47 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/TestHiveBinarySearchRecordReader.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/TestHiveBinarySearchRecordReader.java
@@ -29,7 +29,6 @@ import java.io.IOException;
 import java.util.LinkedHashMap;
 
 import org.junit.Assert;
-import junit.framework.TestCase;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.ql.exec.Utilities;
@@ -49,12 +48,13 @@ import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.RecordReader;
 import org.mockito.InOrder;
+import org.junit.Test;
 
 /**
  * TestHiveBinarySearchRecordReader.
  *
  */
-public class TestHiveBinarySearchRecordReader extends TestCase {
+public class TestHiveBinarySearchRecordReader {
 
   private RCFileRecordReader rcfReader;
   private JobConf conf;
@@ -152,6 +152,7 @@ public class TestHiveBinarySearchRecordReader extends TestCase {
      return hbsReader.next(hbsReader.createKey(), hbsReader.createValue());
   }
 
+  @Test
   public void testNonLinearGreaterThan() throws Exception {
     init();
     Assert.assertTrue(executeDoNext());
@@ -165,6 +166,7 @@ public class TestHiveBinarySearchRecordReader extends TestCase {
     verify(rcfReader).sync(25);
   }
 
+  @Test
   public void testNonLinearLessThan() throws Exception {
     init();
     Assert.assertTrue(executeDoNext());
@@ -178,6 +180,7 @@ public class TestHiveBinarySearchRecordReader extends TestCase {
     verify(rcfReader).sync(75);
   }
 
+  @Test
   public void testNonLinearEqualTo() throws Exception {
     init();
     Assert.assertTrue(executeDoNext());
@@ -191,6 +194,7 @@ public class TestHiveBinarySearchRecordReader extends TestCase {
     verify(rcfReader).sync(25);
   }
 
+  @Test
   public void testHitLastBlock() throws Exception {
     init();
     Assert.assertTrue(executeDoNext());
@@ -209,6 +213,7 @@ public class TestHiveBinarySearchRecordReader extends TestCase {
     Assert.assertFalse(ioContext.isBinarySearching());
   }
 
+  @Test
   public void testHitSamePositionTwice() throws Exception {
     init();
     Assert.assertTrue(executeDoNext());
@@ -225,6 +230,7 @@ public class TestHiveBinarySearchRecordReader extends TestCase {
     Assert.assertFalse(ioContext.isBinarySearching());
   }
 
+  @Test
   public void testResetRange() throws Exception {
     init();
     InOrder inOrder = inOrder(rcfReader);
@@ -247,6 +253,7 @@ public class TestHiveBinarySearchRecordReader extends TestCase {
     Assert.assertFalse(ioContext.shouldEndBinarySearch());
   }
 
+  @Test
   public void testEqualOpClass() throws Exception {
     init();
     ioContext.setGenericUDFClassName(GenericUDFOPEqual.class.getName());
@@ -261,6 +268,7 @@ public class TestHiveBinarySearchRecordReader extends TestCase {
     Assert.assertFalse(executeDoNext());
   }
 
+  @Test
   public void testLessThanOpClass() throws Exception {
     init();
     ioContext.setGenericUDFClassName(GenericUDFOPLessThan.class.getName());
@@ -274,6 +282,7 @@ public class TestHiveBinarySearchRecordReader extends TestCase {
     Assert.assertFalse(executeDoNext());
   }
 
+  @Test
   public void testLessThanOrEqualOpClass() throws Exception {
     init();
     ioContext.setGenericUDFClassName(GenericUDFOPEqualOrLessThan.class.getName());
@@ -287,6 +296,7 @@ public class TestHiveBinarySearchRecordReader extends TestCase {
     Assert.assertFalse(executeDoNext());
   }
 
+  @Test
   public void testGreaterThanOpClass() throws Exception {
     init();
     ioContext.setGenericUDFClassName(GenericUDFOPGreaterThan.class.getName());
@@ -301,6 +311,7 @@ public class TestHiveBinarySearchRecordReader extends TestCase {
     Assert.assertTrue(executeDoNext());
   }
 
+  @Test
   public void testGreaterThanOrEqualOpClass() throws Exception {
     init();
     ioContext.setGenericUDFClassName(GenericUDFOPEqualOrGreaterThan.class.getName());
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/TestHiveFileFormatUtils.java b/ql/src/test/org/apache/hadoop/hive/ql/io/TestHiveFileFormatUtils.java
index db1571c..aeeda27 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/TestHiveFileFormatUtils.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/TestHiveFileFormatUtils.java
@@ -22,13 +22,19 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.ql.plan.PartitionDesc;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
-public class TestHiveFileFormatUtils extends TestCase {
+/**
+ * TestHiveFileFormatUtils.
+ */
+public class TestHiveFileFormatUtils {
 
+  @Test
   public void testGetPartitionDescFromPathRecursively() throws IOException {
 
     PartitionDesc partDesc_3 = new PartitionDesc();
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/TestHiveInputOutputBuffer.java b/ql/src/test/org/apache/hadoop/hive/ql/io/TestHiveInputOutputBuffer.java
index 38d64aa..94c8651 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/TestHiveInputOutputBuffer.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/TestHiveInputOutputBuffer.java
@@ -17,21 +17,20 @@
  */
 package org.apache.hadoop.hive.ql.io;
 
-import static org.junit.Assert.assertArrayEquals;
-
 import java.io.DataOutput;
 import java.io.IOException;
 import java.util.Random;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertTrue;
 import org.junit.Test;
 
-import junit.framework.TestCase;
-
 /**
  * TestHiveInputOutputBuffer.
  *
  */
-public class TestHiveInputOutputBuffer extends TestCase {
+public class TestHiveInputOutputBuffer {
 
   private static final int numCases = 14; 
   
@@ -39,6 +38,7 @@ public class TestHiveInputOutputBuffer extends TestCase {
   private static final String asciiLine2 = "Line two";
   private static final String asciiString = asciiLine1 + "\n" + asciiLine2 + "\r\n";
 
+  @Test
   public void testReadAndWrite() throws IOException {
     String testString = "test_hive_input_output_number_0";
     byte[] string_bytes = testString.getBytes();
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/TestSymlinkTextInputFormat.java b/ql/src/test/org/apache/hadoop/hive/ql/io/TestSymlinkTextInputFormat.java
index 36f2505..005d420 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/TestSymlinkTextInputFormat.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/TestSymlinkTextInputFormat.java
@@ -24,7 +24,7 @@ import java.util.ArrayList;
 import java.util.LinkedHashMap;
 import java.util.List;
 
-import junit.framework.TestCase;
+
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -50,12 +50,17 @@ import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.RecordReader;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.util.ReflectionUtils;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import org.junit.Before;
+import org.junit.After;
+import org.junit.Test;
 
 /**
  * Unittest for SymlinkTextInputFormat.
  */
 @SuppressWarnings("deprecation")
-public class TestSymlinkTextInputFormat extends TestCase {
+public class TestSymlinkTextInputFormat {
   private static final Logger log =
       LoggerFactory.getLogger(TestSymlinkTextInputFormat.class);
 
@@ -69,8 +74,8 @@ public class TestSymlinkTextInputFormat extends TestCase {
   private Path dataDir2;
   private Path symlinkDir;
 
-  @Override
-  protected void setUp() throws IOException {
+  @Before
+  public void setUp() throws IOException {
     conf = new Configuration();
     job = new JobConf(conf);
 
@@ -94,8 +99,8 @@ public class TestSymlinkTextInputFormat extends TestCase {
     symlinkDir = new Path(testDir, "symlinkdir");
   }
 
-  @Override
-  protected void tearDown() throws IOException {
+  @After
+  public void tearDown() throws IOException {
     fileSystem.delete(testDir, true);
   }
 
@@ -104,6 +109,7 @@ public class TestSymlinkTextInputFormat extends TestCase {
    * file, and then create one symlink file containing these 2 files. Normally
    * without combine, it will return at least 2 splits
    */
+  @Test
   public void testCombine() throws Exception {
     JobConf newJob = new JobConf(job);
     FileSystem fs = dataDir1.getFileSystem(newJob);
@@ -205,6 +211,7 @@ public class TestSymlinkTextInputFormat extends TestCase {
    * Test scenario: Two data directories, one symlink file that contains two
    * paths each point to a file in one of data directories.
    */
+  @Test
   public void testAccuracy1() throws IOException {
     // First data dir, contains 2 files.
 
@@ -286,6 +293,7 @@ public class TestSymlinkTextInputFormat extends TestCase {
    *
    * Expected: Should return empty result set without any exception.
    */
+  @Test
   public void testAccuracy2() throws IOException {
     fileSystem.mkdirs(symlinkDir);
 
@@ -326,6 +334,7 @@ public class TestSymlinkTextInputFormat extends TestCase {
    * Scenario: No job input paths.
    * Expected: IOException with proper message.
    */
+  @Test
   public void testFailure() {
     SymlinkTextInputFormat inputFormat = new SymlinkTextInputFormat();
 
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/arrow/TestArrowColumnarBatchSerDe.java b/ql/src/test/org/apache/hadoop/hive/ql/io/arrow/TestArrowColumnarBatchSerDe.java
index f1cb6df..be15197 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/arrow/TestArrowColumnarBatchSerDe.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/arrow/TestArrowColumnarBatchSerDe.java
@@ -859,6 +859,7 @@ public class TestArrowColumnarBatchSerDe {
     }
   }
 
+  @Test
   public void testMapDecimal() throws SerDeException {
     String[][] schema = {
         {"decimal_map", "map<string,decimal(38,10)>"},
@@ -867,6 +868,7 @@ public class TestArrowColumnarBatchSerDe {
     initAndSerializeAndDeserialize(schema, toMap(DECIMAL_ROWS));
   }
 
+  @Test
   public void testListDecimal() throws SerDeException {
     String[][] schema = {
         {"decimal_list", "array<decimal(38,10)>"},
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestParquetSerDe.java b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestParquetSerDe.java
index 8d32f43..ec9ea88 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestParquetSerDe.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestParquetSerDe.java
@@ -15,7 +15,7 @@ package org.apache.hadoop.hive.ql.io.parquet;
 
 import java.util.Properties;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe;
@@ -35,9 +35,16 @@ import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Writable;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import org.junit.Test;
 
-public class TestParquetSerDe extends TestCase {
+/**
+ * TestParquetSerDe.
+ */
+public class TestParquetSerDe {
 
+  @Test
   public void testParquetHiveSerDe() throws Throwable {
     try {
       // Create the SerDe
@@ -85,6 +92,7 @@ public class TestParquetSerDe extends TestCase {
     }
   }
 
+  @Test
   public void testParquetHiveSerDeComplexTypes() throws Throwable {
     // Initialize
     ParquetHiveSerDe serDe = new ParquetHiveSerDe();
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestAbstractParquetMapInspector.java b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestAbstractParquetMapInspector.java
index 586284f..1f06571 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestAbstractParquetMapInspector.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestAbstractParquetMapInspector.java
@@ -16,16 +16,23 @@ package org.apache.hadoop.hive.ql.io.parquet.serde;
 import java.util.HashMap;
 import java.util.Map;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.io.ArrayWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Writable;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertNotNull;
+import org.junit.Before;
 import org.junit.Test;
 
-public class TestAbstractParquetMapInspector extends TestCase {
+/**
+ * AbstractParquetMapInspector Test.
+ */
+public class TestAbstractParquetMapInspector {
 
   class TestableAbstractParquetMapInspector extends AbstractParquetMapInspector {
 
@@ -40,7 +47,7 @@ public class TestAbstractParquetMapInspector extends TestCase {
   }
   private TestableAbstractParquetMapInspector inspector;
 
-  @Override
+  @Before
   public void setUp() {
     inspector = new TestableAbstractParquetMapInspector(PrimitiveObjectInspectorFactory.javaIntObjectInspector,
             PrimitiveObjectInspectorFactory.javaIntObjectInspector);
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestArrayWritableObjectInspector.java b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestArrayWritableObjectInspector.java
index a3234a5..3e8b19a 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestArrayWritableObjectInspector.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestArrayWritableObjectInspector.java
@@ -24,12 +24,13 @@ import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.junit.Assert;
 
-import junit.framework.TestCase;
+import org.junit.Test;
+
 
 /**
  * Tests for ArrayWritableObjectInspector. At the moment only behavior related to HIVE-21796 covered.
  */
-public class TestArrayWritableObjectInspector extends TestCase {
+public class TestArrayWritableObjectInspector {
 
   private StructTypeInfo nestOnce(TypeInfo nestedType) {
     return (StructTypeInfo) TypeInfoFactory.getStructTypeInfo(Arrays.asList("value"), Arrays.asList(nestedType));
@@ -44,6 +45,7 @@ public class TestArrayWritableObjectInspector extends TestCase {
   }
 
   /** Regression tests for HIVE-21796: equals and hash takes forever if HIVE-21796 is reverted / reintroduced. */
+  @Test
   public void testIdenticalInspectorsEquals() {
     StructTypeInfo nestedStruct = createNestedStruct(TypeInfoFactory.getPrimitiveTypeInfo("int"), 1000);
     ArrayWritableObjectInspector inspectorX = new ArrayWritableObjectInspector(true, nestedStruct, null);
@@ -53,6 +55,7 @@ public class TestArrayWritableObjectInspector extends TestCase {
   }
 
   /** Regression tests for HIVE-21796: equals and hash takes forever if HIVE-21796 is reverted / reintroduced. */
+  @Test
   public void testEqualInspectorsEquals() {
     StructTypeInfo nestedStructX = createNestedStruct(TypeInfoFactory.getPrimitiveTypeInfo("int"), 100);
     StructTypeInfo nestedStructY = createNestedStruct(TypeInfoFactory.getPrimitiveTypeInfo("int"), 100);
@@ -63,6 +66,7 @@ public class TestArrayWritableObjectInspector extends TestCase {
   }
 
   /** Regression tests for HIVE-21796: equals and hash takes forever if HIVE-21796 is reverted / reintroduced. */
+  @Test
   public void testDifferentInspectorsEquals() {
     StructTypeInfo nestedStructX = createNestedStruct(TypeInfoFactory.getPrimitiveTypeInfo("int"), 100);
     StructTypeInfo nestedStructY = createNestedStruct(TypeInfoFactory.getPrimitiveTypeInfo("bigint"), 100);
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestDeepParquetHiveMapInspector.java b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestDeepParquetHiveMapInspector.java
index d0756e4..b788541 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestDeepParquetHiveMapInspector.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestDeepParquetHiveMapInspector.java
@@ -16,7 +16,7 @@ package org.apache.hadoop.hive.ql.io.parquet.serde;
 import java.util.HashMap;
 import java.util.Map;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.ql.io.parquet.serde.primitive.ParquetPrimitiveInspectorFactory;
 import org.apache.hadoop.hive.serde2.io.ShortWritable;
@@ -24,13 +24,19 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectIn
 import org.apache.hadoop.io.ArrayWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Writable;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertEquals;
+import org.junit.Before;
 import org.junit.Test;
 
-public class TestDeepParquetHiveMapInspector extends TestCase {
+/**
+ * DeepParquetHiveMapInspector Test.
+ */
+public class TestDeepParquetHiveMapInspector {
 
   private DeepParquetHiveMapInspector inspector;
 
-  @Override
+  @Before
   public void setUp() {
     inspector = new DeepParquetHiveMapInspector(ParquetPrimitiveInspectorFactory.parquetShortInspector,
             PrimitiveObjectInspectorFactory.javaIntObjectInspector);
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestParquetHiveArrayInspector.java b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestParquetHiveArrayInspector.java
index 1b3de04..8fd44f6 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestParquetHiveArrayInspector.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestParquetHiveArrayInspector.java
@@ -16,19 +16,26 @@ package org.apache.hadoop.hive.ql.io.parquet.serde;
 import java.util.ArrayList;
 import java.util.List;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.io.ArrayWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Writable;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertNotNull;
+import org.junit.Before;
 import org.junit.Test;
 
-public class TestParquetHiveArrayInspector extends TestCase {
+/**
+ * TestParquetHiveArrayInspector.
+ */
+public class TestParquetHiveArrayInspector {
 
   private ParquetHiveArrayInspector inspector;
 
-  @Override
+  @Before
   public void setUp() {
     inspector = new ParquetHiveArrayInspector(PrimitiveObjectInspectorFactory.javaIntObjectInspector);
   }
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestParquetTimestampUtils.java b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestParquetTimestampUtils.java
index e745af9..9ea7850 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestParquetTimestampUtils.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestParquetTimestampUtils.java
@@ -25,18 +25,20 @@ import org.apache.hadoop.hive.ql.io.parquet.timestamp.NanoTimeUtils;
 
 import org.junit.Assert;
 import junit.framework.TestCase;
+import org.junit.Test;
 
 
 
 /**
  * Tests util-libraries used for parquet-timestamp.
  */
-public class TestParquetTimestampUtils extends TestCase {
+public class TestParquetTimestampUtils {
 
   public static final ZoneId GMT = ZoneId.of("GMT");
   public static final ZoneId US_PACIFIC = ZoneId.of("US/Pacific");
   public static final ZoneId NEW_YORK = ZoneId.of("America/New_York");
 
+  @Test
   public void testJulianDay() {
     //check if May 23, 1968 is Julian Day 2440000
     Calendar cal = Calendar.getInstance();
@@ -112,6 +114,7 @@ public class TestParquetTimestampUtils extends TestCase {
     Assert.assertEquals(nt2.getJulianDay() - nt1.getJulianDay(), 730517);
 }
 
+  @Test
   public void testNanos() {
     //case 1: 01:01:01.0000000001
     Calendar cal = Calendar.getInstance();
@@ -175,6 +178,7 @@ public class TestParquetTimestampUtils extends TestCase {
     Assert.assertEquals(ts1, NanoTimeUtils.getTimestamp(n3, false, GMT));
   }
 
+  @Test
   public void testTimezone() {
     Calendar cal = Calendar.getInstance();
     cal.set(Calendar.YEAR,  1968);
@@ -202,14 +206,17 @@ public class TestParquetTimestampUtils extends TestCase {
     Assert.assertEquals(nt.getJulianDay(), 2440001);
   }
 
+  @Test
   public void testTimezoneValues() {
     valueTest(false);
   }
 
+  @Test
   public void testTimezonelessValues() {
     valueTest(true);
   }
 
+  @Test
   public void testTimezoneless() {
     Timestamp ts1 = Timestamp.valueOf("2011-01-01 00:30:30.111111111");
     NanoTime nt1 = NanoTimeUtils.getNanoTime(ts1, true);
@@ -262,6 +269,7 @@ public class TestParquetTimestampUtils extends TestCase {
     Assert.assertEquals(tsString, tsFetched.toString());
   }
 
+  @Test
   public void testConvertTimestampToZone() {
     Timestamp ts = Timestamp.valueOf("2018-01-01 00:00:00");
     Timestamp ts1 = TimestampTZUtil.convertTimestampToZone(ts, NEW_YORK, US_PACIFIC);
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestStandardParquetHiveMapInspector.java b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestStandardParquetHiveMapInspector.java
index 583b654..37383ec 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestStandardParquetHiveMapInspector.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/serde/TestStandardParquetHiveMapInspector.java
@@ -16,20 +16,26 @@ package org.apache.hadoop.hive.ql.io.parquet.serde;
 import java.util.HashMap;
 import java.util.Map;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.serde2.io.ShortWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.io.ArrayWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Writable;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertEquals;
+import org.junit.Before;
 import org.junit.Test;
 
-public class TestStandardParquetHiveMapInspector extends TestCase {
+/**
+ * TestStandardParquetHiveMapInspector.
+ */
+public class TestStandardParquetHiveMapInspector {
 
   private StandardParquetHiveMapInspector inspector;
 
-  @Override
+  @Before
   public void setUp() {
     inspector = new StandardParquetHiveMapInspector(PrimitiveObjectInspectorFactory.javaIntObjectInspector,
             PrimitiveObjectInspectorFactory.javaIntObjectInspector);
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestEmbeddedLockManager.java b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestEmbeddedLockManager.java
index 24eb237..4a88551 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestEmbeddedLockManager.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestEmbeddedLockManager.java
@@ -18,17 +18,23 @@
 
 package org.apache.hadoop.hive.ql.lockmgr;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.lockmgr.HiveLockObject.HiveLockObjectData;
 import org.junit.Assert;
+import org.junit.Test;
 
-public class TestEmbeddedLockManager extends TestCase {
+/**
+ * TestEmbeddedLockManager.
+ *
+ */
+public class TestEmbeddedLockManager {
 
   private int counter;
   private HiveConf conf = new HiveConf();
 
+  @Test
   public void testLocking() throws LockException {
     HiveConf conf = new HiveConf();
     conf.set("hive.lock.numretries", "0");
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java
index 8d55fec..230fac3 100755
--- a/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java
@@ -71,19 +71,28 @@ import org.slf4j.LoggerFactory;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 
-import junit.framework.TestCase;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.fail;
+import org.junit.Before;
+import org.junit.After;
+import org.junit.Test;
 
 /**
  * TestHive.
  *
  */
-public class TestHive extends TestCase {
+public class TestHive {
   protected Hive hm;
   protected HiveConf hiveConf;
 
-  @Override
-  protected void setUp() throws Exception {
-    super.setUp();
+  @Before
+  public void setUp() throws Exception {
+
     hiveConf = new HiveConf(this.getClass());
     hm = setUpImpl(hiveConf);
   }
@@ -104,10 +113,10 @@ public class TestHive extends TestCase {
     }
   }
 
-  @Override
-  protected void tearDown() throws Exception {
+  @After
+  public void tearDown() throws Exception {
     try {
-      super.tearDown();
+
       // disable trash
       hiveConf.setFloat("fs.trash.checkpoint.interval", 30);  // FS_TRASH_CHECKPOINT_INTERVAL_KEY (hadoop-2)
       hiveConf.setFloat("fs.trash.interval", 30);             // FS_TRASH_INTERVAL_KEY (hadoop-2)
@@ -121,6 +130,7 @@ public class TestHive extends TestCase {
     }
   }
 
+  @Test
   public void testTable() throws Throwable {
     try {
       // create a simple table and test create, drop, get
@@ -220,6 +230,7 @@ public class TestHive extends TestCase {
    *
    * @throws Throwable
    */
+  @Test
   public void testThriftTable() throws Throwable {
     String tableName = "table_for_test_thrifttable";
     try {
@@ -263,6 +274,7 @@ public class TestHive extends TestCase {
    *
    * @throws Throwable
    */
+  @Test
   public void testMetaStoreApiTiming() throws Throwable {
     // Get the RootLogger which, if you don't have log4j2-test.properties defined, will only log ERRORs
     Logger logger = LogManager.getLogger("hive.ql.metadata.Hive");
@@ -372,6 +384,7 @@ public class TestHive extends TestCase {
    * Test basic Hive class interaction, that:
    * - We can have different Hive objects throughout the lifetime of this thread.
    */
+  @Test
   public void testHiveCloseCurrent() throws Throwable {
     Hive hive1 = Hive.get();
     Hive.closeCurrent();
@@ -380,6 +393,7 @@ public class TestHive extends TestCase {
     assertTrue(hive1 != hive2);
   }
 
+  @Test
   public void testGetAndDropTables() throws Throwable {
     try {
       String dbName = "db_for_testgettables";
@@ -432,6 +446,7 @@ public class TestHive extends TestCase {
     }
   }
 
+  @Test
   public void testWmNamespaceHandling() throws Throwable {
     HiveConf hiveConf = new HiveConf(this.getClass());
     Hive hm = setUpImpl(hiveConf);
@@ -478,6 +493,7 @@ public class TestHive extends TestCase {
     hm2.alterResourcePlan("hm", changes, true, false, false);
   }
 
+  @Test
   public void testDropTableTrash() throws Throwable {
     if (!ShimLoader.getHadoopShims().supportTrashFeature()) {
       return; // it's hadoop-1
@@ -590,6 +606,7 @@ public class TestHive extends TestCase {
    * 2. Drop partitions with PURGE, and check that the data is moved to Trash.
    * @throws Exception on failure.
    */
+  @Test
   public void testDropPartitionsWithPurge() throws Exception {
     String dbName = Warehouse.DEFAULT_DATABASE_NAME;
     String tableName = "table_for_testDropPartitionsWithPurge";
@@ -652,6 +669,7 @@ public class TestHive extends TestCase {
    * Test that tables set up with auto-purge skip trash-directory when tables/partitions are dropped.
    * @throws Throwable
    */
+  @Test
   public void testAutoPurgeTablesAndPartitions() throws Throwable {
 
     String dbName = Warehouse.DEFAULT_DATABASE_NAME;
@@ -704,6 +722,7 @@ public class TestHive extends TestCase {
     }
   }
 
+  @Test
   public void testPartition() throws Throwable {
     try {
       String tableName = "table_for_testpartition";
@@ -752,6 +771,7 @@ public class TestHive extends TestCase {
     }
   }
 
+  @Test
   public void testHiveRefreshOnConfChange() throws Throwable{
     Hive prevHiveObj = Hive.get();
     prevHiveObj.getDatabaseCurrent();
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveRemote.java b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveRemote.java
index eccca48..df3b2a5 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveRemote.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveRemote.java
@@ -25,6 +25,8 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
 import org.apache.hadoop.util.StringUtils;
 
+import org.junit.Before;
+
 /**
  *
  * TestHiveRemote.
@@ -36,8 +38,8 @@ public class TestHiveRemote extends TestHive {
   /**
    * Start a remote metastore and initialize a Hive object pointing at it.
    */
-  @Override
-  protected void setUp() throws Exception {
+  @Before
+  public void setUp() throws Exception {
     super.setUp();
     hiveConf = new HiveConf(this.getClass());
     hiveConf
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/optimizer/listbucketingpruner/TestDynamicMultiDimeCollection.java b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/listbucketingpruner/TestDynamicMultiDimeCollection.java
index f0586c2..94a7d96 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/optimizer/listbucketingpruner/TestDynamicMultiDimeCollection.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/listbucketingpruner/TestDynamicMultiDimeCollection.java
@@ -21,7 +21,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.junit.Assert;
@@ -32,7 +32,7 @@ import org.junit.Test;
  * Test {@link DynamicMultiDimeContainer}
  *
  */
-public class TestDynamicMultiDimeCollection extends TestCase {
+public class TestDynamicMultiDimeCollection {
   private static String DEF_DIR = "default";
 
   @Test
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/optimizer/listbucketingpruner/TestListBucketingPrunner.java b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/listbucketingpruner/TestListBucketingPrunner.java
index de65ec4..0383aa1 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/optimizer/listbucketingpruner/TestListBucketingPrunner.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/listbucketingpruner/TestListBucketingPrunner.java
@@ -27,7 +27,7 @@ import org.junit.Test;
  * Test {@link ListBucketingPruner}
  *
  */
-public class TestListBucketingPrunner extends TestCase {
+public class TestListBucketingPrunner {
 
   @Test
   public void testSkipSkewedDirectory1() {
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestEximUtil.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestEximUtil.java
index d4ae060..e17ee2e 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestEximUtil.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestEximUtil.java
@@ -18,22 +18,27 @@
 
 package org.apache.hadoop.hive.ql.parse;
 
-import junit.framework.TestCase;
+
+import static org.junit.Assert.fail;
+import org.junit.Before;
+import org.junit.After;
+import org.junit.Test;
 
 /**
  * TestEximUtil.
  *
  */
-public class TestEximUtil extends TestCase {
+public class TestEximUtil {
 
-  @Override
-  protected void setUp() {
+  @Before
+  public void setUp() {
   }
 
-  @Override
-  protected void tearDown() {
+  @After
+  public void tearDown() {
   }
 
+  @Test
   public void testCheckCompatibility() throws SemanticException {
 
     // backward/forward compatible
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/testutil/BaseScalarUdfTest.java b/ql/src/test/org/apache/hadoop/hive/ql/testutil/BaseScalarUdfTest.java
index dae9864..ec0bc89 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/testutil/BaseScalarUdfTest.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/testutil/BaseScalarUdfTest.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hive.ql.testutil;
 
 import java.util.List;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.CollectOperator;
@@ -36,6 +36,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.mapred.JobConf;
 import org.junit.Ignore;
 
+import org.junit.Test;
 /**
  *
  * Provides a base environment for testing scalar UDF's. Users should extend this class
@@ -45,7 +46,7 @@ import org.junit.Ignore;
  */
 @SuppressWarnings("deprecation")
 @Ignore
-public abstract class BaseScalarUdfTest extends TestCase {
+public abstract class BaseScalarUdfTest {
 
   /**
    * The data from this method will be fed through the
@@ -79,6 +80,7 @@ public abstract class BaseScalarUdfTest extends TestCase {
    * and if every row is the expected result the method completes without asserting.
    * @throws HiveException
    */
+  @Test
   public final void testUdf() throws HiveException {
     InspectableObject [] data = getBaseTable();
     List<ExprNodeDesc> expressionList = getExpressionList();
@@ -92,4 +94,4 @@ public abstract class BaseScalarUdfTest extends TestCase {
     OperatorTestUtils.assertResults(op, cdop, data, getExpectedResult());
   }
 
-}
\ No newline at end of file
+}
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/tool/TestLineageInfo.java b/ql/src/test/org/apache/hadoop/hive/ql/tool/TestLineageInfo.java
index 4cf72dd..5b17e8a 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/tool/TestLineageInfo.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/tool/TestLineageInfo.java
@@ -20,15 +20,17 @@ package org.apache.hadoop.hive.ql.tool;
 
 import java.util.TreeSet;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.ql.tools.LineageInfo;
+import static org.junit.Assert.fail;
+import org.junit.Test;
 
 /**
  * TestLineageInfo.
  *
  */
-public class TestLineageInfo extends TestCase {
+public class TestLineageInfo {
 
   /**
    * Checks whether the test outputs match the expected outputs.
@@ -50,6 +52,7 @@ public class TestLineageInfo extends TestCase {
     }
   }
 
+  @Test
   public void testSimpleQuery() {
     LineageInfo lep = new LineageInfo();
     try {
@@ -67,6 +70,7 @@ public class TestLineageInfo extends TestCase {
     }
   }
 
+  @Test
   public void testSimpleQuery2() {
     LineageInfo lep = new LineageInfo();
     try {
@@ -84,6 +88,7 @@ public class TestLineageInfo extends TestCase {
     }
   }
 
+  @Test
   public void testSimpleQuery3() {
     LineageInfo lep = new LineageInfo();
     try {
@@ -102,6 +107,7 @@ public class TestLineageInfo extends TestCase {
     }
   }
 
+  @Test
   public void testSimpleQuery4() {
     LineageInfo lep = new LineageInfo();
     try {
@@ -117,6 +123,7 @@ public class TestLineageInfo extends TestCase {
     }
   }
 
+  @Test
   public void testSimpleQuery5() {
     LineageInfo lep = new LineageInfo();
     try {
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/TestToInteger.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/TestToInteger.java
index 3549f21..e4eb423 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/TestToInteger.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/TestToInteger.java
@@ -18,13 +18,18 @@
 
 package org.apache.hadoop.hive.ql.udf;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Text;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
 import org.junit.Test;
 
-public class TestToInteger extends TestCase{
+/**
+ * TestToInteger.
+ */
+public class TestToInteger{
 
   @Test
   public void testTextToInteger() throws Exception{
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFBase64.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFBase64.java
index 7aed203..6a59927 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFBase64.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFBase64.java
@@ -18,12 +18,18 @@
 
 package org.apache.hadoop.hive.ql.udf;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.Text;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
-public class TestUDFBase64 extends TestCase {
+/**
+ * TestUDFBase64.
+ */
+public class TestUDFBase64 {
+  @Test
   public void testBase64Conversion(){
     byte[] bytes = "string".getBytes();
     // Let's make sure we only read the relevant part of the writable in case of reuse
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFBuildVersion.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFBuildVersion.java
index d54afbd..cdc3baf 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFBuildVersion.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFBuildVersion.java
@@ -21,12 +21,15 @@ package org.apache.hadoop.hive.ql.udf;
 import org.apache.hadoop.io.Text;
 import org.apache.hive.common.util.HiveVersionInfo;
 
-import junit.framework.TestCase;
+
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
 /**
  * Unit Test Case for UDFBuildVersion.
  */
-public class TestUDFBuildVersion extends TestCase {
+public class TestUDFBuildVersion {
+  @Test
   public void testVersion() {
     UDFBuildVersion udf = new UDFBuildVersion();
     Text result = udf.evaluate();
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFCrc32.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFCrc32.java
index f24a079..4c1e244 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFCrc32.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFCrc32.java
@@ -17,15 +17,23 @@
  */
 package org.apache.hadoop.hive.ql.udf;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
-public class TestUDFCrc32 extends TestCase {
+/**
+ * TestUDFCrc32.
+ */
+public class TestUDFCrc32 {
 
+  @Test
   public void testCrc32Str() throws HiveException {
     UDFCrc32 udf = new UDFCrc32();
 
@@ -38,6 +46,7 @@ public class TestUDFCrc32 extends TestCase {
     runAndVerifyStr(null, null, udf);
   }
 
+  @Test
   public void testCrc32Bin() throws HiveException {
     UDFCrc32 udf = new UDFCrc32();
 
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFDateFormatGranularity.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFDateFormatGranularity.java
index 4770ab7..75bf865 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFDateFormatGranularity.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFDateFormatGranularity.java
@@ -24,11 +24,14 @@ import org.apache.hadoop.hive.common.type.Timestamp;
 import org.apache.hadoop.hive.common.type.TimestampTZ;
 import org.apache.hadoop.hive.serde2.io.TimestampLocalTZWritable;
 import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
-import org.junit.Test;
 
-import junit.framework.TestCase;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
-public class TestUDFDateFormatGranularity extends TestCase {
+/**
+ * TestUDFDateFormatGranularity.
+ */
+public class TestUDFDateFormatGranularity {
 
   // Timestamp values are PST (timezone for tests is set to PST by default)
 
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFHex.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFHex.java
index 37ad235..f5e1fb3 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFHex.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFHex.java
@@ -18,12 +18,18 @@
 
 package org.apache.hadoop.hive.ql.udf;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.Text;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
-public class TestUDFHex extends TestCase {
+/**
+ * TestUDFHex.
+ */
+public class TestUDFHex {
+  @Test
   public void testHexConversion(){
     byte[] bytes = "string".getBytes();
     // Let's make sure we only read the relevant part of the writable in case of reuse
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFMd5.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFMd5.java
index d624617..c9ca4dc 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFMd5.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFMd5.java
@@ -17,14 +17,20 @@
  */
 package org.apache.hadoop.hive.ql.udf;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.Text;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
-public class TestUDFMd5 extends TestCase {
+/**
+ * TestUDFMd5.
+ */
+public class TestUDFMd5 {
 
+  @Test
   public void testMD5Str() throws HiveException {
     UDFMd5 udf = new UDFMd5();
 
@@ -34,6 +40,7 @@ public class TestUDFMd5 extends TestCase {
     runAndVerifyStr(null, null, udf);
   }
 
+  @Test
   public void testMD5Bin() throws HiveException {
     UDFMd5 udf = new UDFMd5();
 
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFSha1.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFSha1.java
index 3dbf30d..4c3c399 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFSha1.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFSha1.java
@@ -17,14 +17,20 @@
  */
 package org.apache.hadoop.hive.ql.udf;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.Text;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
-public class TestUDFSha1 extends TestCase {
+/**
+ * TestUDFSha1.
+ */
+public class TestUDFSha1 {
 
+  @Test
   public void testSha1Str() throws HiveException {
     UDFSha1 udf = new UDFSha1();
 
@@ -34,6 +40,7 @@ public class TestUDFSha1 extends TestCase {
     runAndVerifyStr(null, null, udf);
   }
 
+  @Test
   public void testSha1Bin() throws HiveException {
     UDFSha1 udf = new UDFSha1();
 
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFUUID.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFUUID.java
index 843f7bd..64390cc 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFUUID.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFUUID.java
@@ -18,13 +18,18 @@
 
 package org.apache.hadoop.hive.ql.udf;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertEquals;
 import org.junit.Test;
 
-public class TestUDFUUID extends TestCase {
+/**
+ * TestUDFUUID.
+ */
+public class TestUDFUUID {
   @Test
   public void testUUID() throws Exception {
     UDFUUID udf = new UDFUUID();
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFUnbase64.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFUnbase64.java
index d894086..5103ed0 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFUnbase64.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFUnbase64.java
@@ -18,12 +18,18 @@
 
 package org.apache.hadoop.hive.ql.udf;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.Text;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
-public class TestUDFUnbase64 extends TestCase {
+/**
+ * TestUDFUnbase64.
+ */
+public class TestUDFUnbase64 {
+  @Test
   public void testUnbase64Conversion(){
     Text base64 = new Text();
     // Let's make sure we only read the relevant part of the writable in case of reuse
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFUnhex.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFUnhex.java
index a77ff9b..3cf6654 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFUnhex.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFUnhex.java
@@ -18,11 +18,17 @@
 
 package org.apache.hadoop.hive.ql.udf;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.io.Text;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
-public class TestUDFUnhex extends TestCase {
+/**
+ * TestUDFUnhex.
+ */
+public class TestUDFUnhex {
+  @Test
   public void testUnhexConversion(){
     Text hex = new Text();
     // Let's make sure we only read the relevant part of the writable in case of reuse
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFVersion.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFVersion.java
index f9dd1ba..3825115 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFVersion.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFVersion.java
@@ -18,12 +18,18 @@
 
 package org.apache.hadoop.hive.ql.udf;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.io.Text;
 import org.apache.hive.common.util.HiveVersionInfo;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
-public class TestUDFVersion extends TestCase {
+/**
+ * TestUDFVersion.
+ */
+public class TestUDFVersion {
+  @Test
   public void testVersion(){
     UDFVersion udf = new UDFVersion();
     Text result = udf.evaluate();
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDAFCorrelation.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDAFCorrelation.java
index 1f7ed4a..66b069b 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDAFCorrelation.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDAFCorrelation.java
@@ -18,15 +18,22 @@
 
 package org.apache.hadoop.hive.ql.udf.generic;
 
-import junit.framework.TestCase;
+
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
-public class TestGenericUDAFCorrelation extends TestCase {
+/**
+ * TestGenericUDAFCorrelation.
+ *
+ */
+public class TestGenericUDAFCorrelation {
 
+  @Test
   public void testCorr() throws HiveException {
     GenericUDAFCorrelation corr = new GenericUDAFCorrelation();
     GenericUDAFEvaluator eval1 = corr.getEvaluator(
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFAbs.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFAbs.java
index 0bcf3f2..a0189a5 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFAbs.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFAbs.java
@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.hive.ql.udf.generic;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -35,9 +35,16 @@ import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import org.junit.Test;
 
-public class TestGenericUDFAbs extends TestCase {
+/**
+ * TestGenericUDFAbs.
+ */
+public class TestGenericUDFAbs {
 
+  @Test
   public void testInt() throws HiveException {
     GenericUDFAbs udf = new GenericUDFAbs();
     ObjectInspector valueOI = PrimitiveObjectInspectorFactory.writableIntObjectInspector;
@@ -57,6 +64,7 @@ public class TestGenericUDFAbs extends TestCase {
     assertEquals("abs() test for INT failed ", 107, output.get());
   }
 
+  @Test
   public void testLong() throws HiveException {
     GenericUDFAbs udf = new GenericUDFAbs();
     ObjectInspector valueOI = PrimitiveObjectInspectorFactory.writableLongObjectInspector;
@@ -76,6 +84,7 @@ public class TestGenericUDFAbs extends TestCase {
     assertEquals("abs() test for LONG failed ", 107, output.get());
   }
 
+  @Test
   public void testDouble() throws HiveException {
     GenericUDFAbs udf = new GenericUDFAbs();
     ObjectInspector valueOI = PrimitiveObjectInspectorFactory.writableDoubleObjectInspector;
@@ -86,15 +95,16 @@ public class TestGenericUDFAbs extends TestCase {
     DeferredObject[] args = {valueObj};
     DoubleWritable output = (DoubleWritable) udf.evaluate(args);
 
-    assertEquals("abs() test for Double failed ", 107.78, output.get());
+    assertEquals("abs() test for Double failed ", 107.78, output.get(), 1e-15);
 
     valueObj = new DeferredJavaObject(new DoubleWritable(-107.78));
     args[0] = valueObj;
     output = (DoubleWritable) udf.evaluate(args);
 
-    assertEquals("abs() test for Double failed ", 107.78, output.get());
+    assertEquals("abs() test for Double failed ", 107.78, output.get(), 1e-15);
   }
 
+  @Test
   public void testFloat() throws HiveException {
     GenericUDFAbs udf = new GenericUDFAbs();
     ObjectInspector valueOI = PrimitiveObjectInspectorFactory.writableFloatObjectInspector;
@@ -116,6 +126,7 @@ public class TestGenericUDFAbs extends TestCase {
   }
 
 
+  @Test
   public void testText() throws HiveException {
     GenericUDFAbs udf = new GenericUDFAbs();
     ObjectInspector valueOI = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
@@ -141,6 +152,7 @@ public class TestGenericUDFAbs extends TestCase {
     assertEquals("abs() test for String failed ", null, output);
   }
 
+  @Test
   public void testHiveDecimal() throws HiveException {
     GenericUDFAbs udf = new GenericUDFAbs();
     int prec = 12;
@@ -160,14 +172,14 @@ public class TestGenericUDFAbs extends TestCase {
     HiveDecimalWritable output = (HiveDecimalWritable) udf.evaluate(args);
 
     assertEquals("abs() test for HiveDecimal failed ", 107.123456789, output.getHiveDecimal()
-        .doubleValue());
+        .doubleValue(), 1e-15);
 
     valueObj = new DeferredJavaObject(new HiveDecimalWritable(HiveDecimal.create("-107.123456789")));
     args[0] = valueObj;
     output = (HiveDecimalWritable) udf.evaluate(args);
 
     assertEquals("abs() test for HiveDecimal failed ", 107.123456789, output.getHiveDecimal()
-        .doubleValue());
+        .doubleValue(), 1e-15);
 
     // null input
     args[0] = new DeferredJavaObject(null);
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFAddMonths.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFAddMonths.java
index 7c2ee15..eb104bd 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFAddMonths.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFAddMonths.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hive.ql.udf.generic;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.common.type.Timestamp;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
@@ -32,14 +32,22 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectIn
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Text;
+import static org.junit.Assert.fail;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
-public class TestGenericUDFAddMonths extends TestCase {
+/**
+ * TestGenericUDFAddMonths.
+ */
+public class TestGenericUDFAddMonths {
 
   private final Text fmtTextWithTime = new Text("YYYY-MM-dd HH:mm:ss");
   private final Text fmtTextWithTimeAndms = new Text("YYYY-MM-dd HH:mm:ss.SSS");
   private final Text fmtTextWithoutTime = new Text("YYYY-MM-dd");
   private final Text fmtTextInvalid = new Text("YYYY-abcdz");
 
+  @Test
   public void testAddMonthsInt() throws HiveException {
     GenericUDFAddMonths udf = new GenericUDFAddMonths();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
@@ -72,6 +80,7 @@ public class TestGenericUDFAddMonths extends TestCase {
     runAndVerify("2016-02-29 10:30:00", -1, fmtTextWithoutTime, "2016-01-31", udf);
   }
 
+  @Test
   public void testAddMonthsStringWithTime() throws HiveException {
     GenericUDFAddMonths udf = new GenericUDFAddMonths();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
@@ -87,6 +96,7 @@ public class TestGenericUDFAddMonths extends TestCase {
     runAndVerify("2017-12-31 14:15:16.001", 2, fmtTextWithTime, "2018-02-28 14:15:16", udf);
   }
 
+  @Test
   public void testAddMonthsInvalidFormatter() throws HiveException {
     GenericUDFAddMonths udf = new GenericUDFAddMonths();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
@@ -103,6 +113,7 @@ public class TestGenericUDFAddMonths extends TestCase {
       //test success if exception caught
     }
   }
+  @Test
   public void testAddMonthsStringWithTimeWithms() throws HiveException {
     GenericUDFAddMonths udf = new GenericUDFAddMonths();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
@@ -121,6 +132,7 @@ public class TestGenericUDFAddMonths extends TestCase {
     runAndVerify("2017-12-31 14:15:16", 2, fmtTextWithTimeAndms, "2018-02-28 14:15:16.000", udf);
   }
 
+  @Test
   public void testAddMonthsWithNullFormatter() throws HiveException {
     GenericUDFAddMonths udf = new GenericUDFAddMonths();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
@@ -136,6 +148,7 @@ public class TestGenericUDFAddMonths extends TestCase {
     runAndVerify("2017-12-31", 2, null, "2018-02-28",
         udf);
   }
+  @Test
   public void testAddMonthsTimestamp() throws HiveException {
     GenericUDFAddMonths udf = new GenericUDFAddMonths();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableTimestampObjectInspector;
@@ -150,6 +163,7 @@ public class TestGenericUDFAddMonths extends TestCase {
     runAndVerify(Timestamp.valueOf("2017-12-31 14:15:16"), 2, fmtTextWithTime, "2018-02-28 14:15:16", udf);
   }
 
+  @Test
   public void testWrongDateStr() throws HiveException {
     GenericUDFAddMonths udf = new GenericUDFAddMonths();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
@@ -162,6 +176,7 @@ public class TestGenericUDFAddMonths extends TestCase {
     runAndVerify("2014-01", 1, null, udf);
   }
 
+  @Test
   public void testWrongTsStr() throws HiveException {
     GenericUDFAddMonths udf = new GenericUDFAddMonths();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
@@ -176,6 +191,7 @@ public class TestGenericUDFAddMonths extends TestCase {
     runAndVerify("2014-01-31T10:30:00", 1, "2014-02-28", udf);
   }
 
+  @Test
   public void testAddMonthsShort() throws HiveException {
     GenericUDFAddMonths udf = new GenericUDFAddMonths();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
@@ -187,6 +203,7 @@ public class TestGenericUDFAddMonths extends TestCase {
     runAndVerify("2014-01-14", (short) 1, "2014-02-14", udf);
   }
 
+  @Test
   public void testAddMonthsByte() throws HiveException {
     GenericUDFAddMonths udf = new GenericUDFAddMonths();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
@@ -198,6 +215,7 @@ public class TestGenericUDFAddMonths extends TestCase {
     runAndVerify("2014-01-14", (byte) 1, "2014-02-14", udf);
   }
 
+  @Test
   public void testAddMonthsLong() throws HiveException {
     @SuppressWarnings("resource")
     GenericUDFAddMonths udf = new GenericUDFAddMonths();
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFCbrt.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFCbrt.java
index 8105220..6de3789 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFCbrt.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFCbrt.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hive.ql.udf.generic;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredJavaObject;
@@ -25,9 +25,17 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredObject;
 import org.apache.hadoop.hive.serde2.io.DoubleWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
-public class TestGenericUDFCbrt extends TestCase {
+/**
+ * TestGenericUDFCbrt.
+ */
+public class TestGenericUDFCbrt {
 
+  @Test
   public void testCbrt() throws HiveException {
     GenericUDFCbrt udf = new GenericUDFCbrt();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableDoubleObjectInspector;
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFConcat.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFConcat.java
index 54c3cf2..11ba426 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFConcat.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFConcat.java
@@ -30,6 +30,9 @@ import org.apache.hadoop.hive.ql.testutil.OperatorTestUtils;
 import org.apache.hadoop.hive.serde2.objectinspector.InspectableObject;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 
+import org.junit.Test;
+import static org.junit.Assert.*;
+
 public class TestGenericUDFConcat extends BaseScalarUdfTest {
 
   @Override
@@ -69,4 +72,9 @@ public class TestGenericUDFConcat extends BaseScalarUdfTest {
     return earr;
   }
 
+  @Test
+  public void testDummy() {
+    assertTrue(true);
+  }
+
 }
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFDate.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFDate.java
index dcb4d9c..728e49b 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFDate.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFDate.java
@@ -32,9 +32,17 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.io.Text;
 
-import junit.framework.TestCase;
 
-public class TestGenericUDFDate extends TestCase {
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.fail;
+import org.junit.Test;
+
+/**
+ * TestGenericUDFDate.
+ */
+public class TestGenericUDFDate {
+  @Test
   public void testStringToDate() throws HiveException {
     GenericUDFDate udf = new GenericUDFDate();
     ObjectInspector valueOI = PrimitiveObjectInspectorFactory.javaStringObjectInspector;
@@ -53,6 +61,7 @@ public class TestGenericUDFDate extends TestCase {
     assertNull("to_date() with null STRING", output);
   }
 
+  @Test
   public void testTimestampToDate() throws HiveException {
     GenericUDFDate udf = new GenericUDFDate();
     ObjectInspector valueOI = PrimitiveObjectInspectorFactory.writableTimestampObjectInspector;
@@ -72,6 +81,7 @@ public class TestGenericUDFDate extends TestCase {
     assertNull("to_date() with null TIMESTAMP", output);
   }
 
+  @Test
   public void testDateWritablepToDate() throws HiveException {
     GenericUDFDate udf = new GenericUDFDate();
     ObjectInspector valueOI = PrimitiveObjectInspectorFactory.writableDateObjectInspector;
@@ -90,6 +100,7 @@ public class TestGenericUDFDate extends TestCase {
     assertNull("to_date() with null DATE", output);
   }
 
+  @Test
   public void testVoidToDate() throws HiveException {
     GenericUDFDate udf = new GenericUDFDate();
     ObjectInspector valueOI = PrimitiveObjectInspectorFactory.writableVoidObjectInspector;
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFDateAdd.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFDateAdd.java
index 8b45a62..a03a49e 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFDateAdd.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFDateAdd.java
@@ -31,9 +31,16 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.io.Text;
 
-import junit.framework.TestCase;
 
-public class TestGenericUDFDateAdd extends TestCase {
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import org.junit.Test;
+
+/**
+ * TestGenericUDFDateAdd.
+ */
+public class TestGenericUDFDateAdd {
+  @Test
   public void testStringToDate() throws HiveException {
     GenericUDFDateAdd udf = new GenericUDFDateAdd();
     ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory.javaStringObjectInspector;
@@ -59,6 +66,7 @@ public class TestGenericUDFDateAdd extends TestCase {
     assertNull("date_add() both args null", udf.evaluate(args));
   }
 
+  @Test
   public void testTimestampToDate() throws HiveException {
     GenericUDFDateAdd udf = new GenericUDFDateAdd();
     ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory.writableTimestampObjectInspector;
@@ -85,6 +93,7 @@ public class TestGenericUDFDateAdd extends TestCase {
     assertNull("date_add() both args null", udf.evaluate(args));
   }
 
+  @Test
   public void testDateWritablepToDate() throws HiveException {
     GenericUDFDateAdd udf = new GenericUDFDateAdd();
     ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory.writableDateObjectInspector;
@@ -111,6 +120,7 @@ public class TestGenericUDFDateAdd extends TestCase {
     assertNull("date_add() both args null", udf.evaluate(args));
   }
 
+  @Test
   public void testByteDataTypeAsDays() throws HiveException {
     GenericUDFDateAdd udf = new GenericUDFDateAdd();
     ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory.writableDateObjectInspector;
@@ -126,6 +136,7 @@ public class TestGenericUDFDateAdd extends TestCase {
     assertEquals("date_add() test for BYTE failed ", "0109-06-24", output.toString());
   }
 
+  @Test
   public void testShortDataTypeAsDays() throws HiveException {
     GenericUDFDateAdd udf = new GenericUDFDateAdd();
     ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory.writableDateObjectInspector;
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFDateDiff.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFDateDiff.java
index ea183d4..86b914d 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFDateDiff.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFDateDiff.java
@@ -32,9 +32,16 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectIn
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Text;
 
-import junit.framework.TestCase;
 
-public class TestGenericUDFDateDiff extends TestCase {
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import org.junit.Test;
+
+/**
+ * TestGenericUDFDateDiff.
+ */
+public class TestGenericUDFDateDiff {
+  @Test
   public void testStringToDate() throws HiveException {
     GenericUDFDateDiff udf = new GenericUDFDateDiff();
     ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory.javaStringObjectInspector;
@@ -60,6 +67,7 @@ public class TestGenericUDFDateDiff extends TestCase {
     assertNull("date_add() both args null", udf.evaluate(args));
   }
 
+  @Test
   public void testTimestampToDate() throws HiveException {
     GenericUDFDateDiff udf = new GenericUDFDateDiff();
     ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory.writableTimestampObjectInspector;
@@ -87,6 +95,7 @@ public class TestGenericUDFDateDiff extends TestCase {
     assertNull("date_add() both args null", udf.evaluate(args));
   }
 
+  @Test
   public void testDateWritablepToDate() throws HiveException {
     GenericUDFDateDiff udf = new GenericUDFDateDiff();
     ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory.writableDateObjectInspector;
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFDateFormat.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFDateFormat.java
index 6a3cdda..f0a5d3f 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFDateFormat.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFDateFormat.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hive.ql.udf.generic;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.common.type.Date;
 import org.apache.hadoop.hive.common.type.Timestamp;
@@ -30,9 +30,15 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.io.Text;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
-public class TestGenericUDFDateFormat extends TestCase {
+/**
+ * TestGenericUDFDateFormat.
+ */
+public class TestGenericUDFDateFormat {
 
+  @Test
   public void testDateFormatStr() throws HiveException {
     GenericUDFDateFormat udf = new GenericUDFDateFormat();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
@@ -64,6 +70,7 @@ public class TestGenericUDFDateFormat extends TestCase {
     runAndVerifyStr("2015-04-12 10", fmtText, "Sunday", udf);
   }
 
+  @Test
   public void testWrongDateStr() throws HiveException {
     GenericUDFDateFormat udf = new GenericUDFDateFormat();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
@@ -79,6 +86,7 @@ public class TestGenericUDFDateFormat extends TestCase {
     runAndVerifyStr(null, fmtText, null, udf);
   }
 
+  @Test
   public void testDateFormatDate() throws HiveException {
     GenericUDFDateFormat udf = new GenericUDFDateFormat();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableDateObjectInspector;
@@ -99,6 +107,7 @@ public class TestGenericUDFDateFormat extends TestCase {
     runAndVerifyDate("2015-04-12", fmtText, "Sunday", udf);
   }
 
+  @Test
   public void testDateFormatTs() throws HiveException {
     GenericUDFDateFormat udf = new GenericUDFDateFormat();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableTimestampObjectInspector;
@@ -120,6 +129,7 @@ public class TestGenericUDFDateFormat extends TestCase {
     runAndVerifyTs("2015-04-12 10:30:45", fmtText, "Sunday", udf);
   }
 
+  @Test
   public void testNullFmt() throws HiveException {
     GenericUDFDateFormat udf = new GenericUDFDateFormat();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
@@ -133,6 +143,7 @@ public class TestGenericUDFDateFormat extends TestCase {
     runAndVerifyStr("2015-04-05", fmtText, null, udf);
   }
 
+  @Test
   public void testWrongFmt() throws HiveException {
     GenericUDFDateFormat udf = new GenericUDFDateFormat();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFDateSub.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFDateSub.java
index b70a1dc..8c86eb5 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFDateSub.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFDateSub.java
@@ -31,9 +31,16 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.io.Text;
 
-import junit.framework.TestCase;
 
-public class TestGenericUDFDateSub extends TestCase {
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import org.junit.Test;
+
+/**
+ * TestGenericUDFDateSub.
+ */
+public class TestGenericUDFDateSub {
+  @Test
   public void testStringToDate() throws HiveException {
     GenericUDFDateSub udf = new GenericUDFDateSub();
     ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory.javaStringObjectInspector;
@@ -59,6 +66,7 @@ public class TestGenericUDFDateSub extends TestCase {
     assertNull("date_add() both args null", udf.evaluate(args));
   }
 
+  @Test
   public void testTimestampToDate() throws HiveException {
     GenericUDFDateSub udf = new GenericUDFDateSub();
     ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory.writableTimestampObjectInspector;
@@ -85,6 +93,7 @@ public class TestGenericUDFDateSub extends TestCase {
     assertNull("date_add() both args null", udf.evaluate(args));
   }
 
+  @Test
   public void testDateWritablepToDate() throws HiveException {
     GenericUDFDateSub udf = new GenericUDFDateSub();
     ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory.writableDateObjectInspector;
@@ -111,6 +120,7 @@ public class TestGenericUDFDateSub extends TestCase {
     assertNull("date_add() both args null", udf.evaluate(args));
   }
 
+  @Test
   public void testByteDataTypeAsDays() throws HiveException {
     GenericUDFDateSub udf = new GenericUDFDateSub();
     ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory.writableDateObjectInspector;
@@ -126,6 +136,7 @@ public class TestGenericUDFDateSub extends TestCase {
     assertEquals("date_add() test for BYTE failed ", "0109-06-16", output.toString());
   }
 
+  @Test
   public void testShortDataTypeAsDays() throws HiveException {
     GenericUDFDateSub udf = new GenericUDFDateSub();
     ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory.writableDateObjectInspector;
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFDecode.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFDecode.java
index 1efbe05..e02d2ff 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFDecode.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFDecode.java
@@ -20,15 +20,21 @@ package org.apache.hadoop.hive.ql.udf.generic;
 
 import java.io.UnsupportedEncodingException;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredJavaObject;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredObject;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
-public class TestGenericUDFDecode extends TestCase {
+/**
+ * TestGenericUDFDecode.
+ */
+public class TestGenericUDFDecode {
+  @Test
   public void testDecode() throws UnsupportedEncodingException, HiveException {
     String[] charsetNames = {"US-ASCII", "ISO-8859-1", "UTF-8", "UTF-16BE", "UTF-16LE", "UTF-16"};
     for (String charsetName : charsetNames){
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFEncode.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFEncode.java
index d7884f7..cfead43 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFEncode.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFEncode.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hive.ql.udf.generic;
 
 import java.io.UnsupportedEncodingException;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredJavaObject;
@@ -28,8 +28,15 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredObject;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.io.BytesWritable;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
-public class TestGenericUDFEncode extends TestCase {
+/**
+ * TestGenericUDFEncode.
+ */
+public class TestGenericUDFEncode {
+  @Test
   public void testEncode() throws UnsupportedEncodingException, HiveException{
     String[] charsetNames = {"US-ASCII", "ISO-8859-1", "UTF-8", "UTF-16BE", "UTF-16LE", "UTF-16"};
     for (String charsetName : charsetNames){
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFEnforceConstraint.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFEnforceConstraint.java
index a0da723..0509da7 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFEnforceConstraint.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFEnforceConstraint.java
@@ -18,18 +18,22 @@
 
 package org.apache.hadoop.hive.ql.udf.generic;
 
-import junit.framework.TestCase;
+
 import org.apache.hadoop.hive.ql.exec.errors.DataConstraintViolationError;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.io.BooleanWritable;
+import static org.junit.Assert.fail;
+import static org.junit.Assert.assertTrue;
+import org.junit.Test;
 
 /**
  * Test class for {@link GenericUDFEnforceConstraint}.
  */
-public class TestGenericUDFEnforceConstraint extends TestCase {
+public class TestGenericUDFEnforceConstraint {
 
+  @Test
   public void testNull() throws HiveException {
     try {
       GenericUDFEnforceConstraint udf = new GenericUDFEnforceConstraint();
@@ -47,6 +51,7 @@ public class TestGenericUDFEnforceConstraint extends TestCase {
     }
   }
 
+  @Test
   public void testInvalidArgumentsLength() throws HiveException {
     try {
       GenericUDFEnforceConstraint udf = new GenericUDFEnforceConstraint();
@@ -61,6 +66,7 @@ public class TestGenericUDFEnforceConstraint extends TestCase {
     }
   }
 
+  @Test
   public void testCorrect() throws HiveException {
     GenericUDFEnforceConstraint udf = new GenericUDFEnforceConstraint();
     ObjectInspector valueOI = PrimitiveObjectInspectorFactory.writableBooleanObjectInspector;
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFFactorial.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFFactorial.java
index 7378076..fd31cbc 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFFactorial.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFFactorial.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hive.ql.udf.generic;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -27,9 +27,18 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
-public class TestGenericUDFFactorial extends TestCase {
+/**
+ * TestGenericUDFFactorial.
+ */
+public class TestGenericUDFFactorial {
 
+  @Test
   public void testFactorial() throws HiveException {
     GenericUDFFactorial udf = new GenericUDFFactorial();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableIntObjectInspector;
@@ -48,6 +57,7 @@ public class TestGenericUDFFactorial extends TestCase {
     runAndVerify(null, null, udf);
   }
 
+  @Test
   public void testWrongInputType() throws HiveException {
     @SuppressWarnings("resource")
     GenericUDFFactorial udf = new GenericUDFFactorial();
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFFromUtcTimestamp.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFFromUtcTimestamp.java
index bb9918c..59c1c49 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFFromUtcTimestamp.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFFromUtcTimestamp.java
@@ -26,9 +26,15 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.io.Text;
 
-import junit.framework.TestCase;
 
-public class TestGenericUDFFromUtcTimestamp extends TestCase {
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
+
+/**
+ * TestGenericUDFFromUtcTimestamp.
+ */
+public class TestGenericUDFFromUtcTimestamp {
   public static void runAndVerify(GenericUDF udf,
       Object arg1, Object arg2, Object expected) throws HiveException {
     DeferredObject[] args = { new DeferredJavaObject(arg1), new DeferredJavaObject(arg2) };
@@ -41,6 +47,7 @@ public class TestGenericUDFFromUtcTimestamp extends TestCase {
     }
   }
 
+  @Test
   public void testFromUtcTimestamp() throws Exception {
     ObjectInspector valueOI = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
     GenericUDFFromUtcTimestamp udf = new GenericUDFFromUtcTimestamp();
@@ -63,6 +70,7 @@ public class TestGenericUDFFromUtcTimestamp extends TestCase {
         Timestamp.valueOf("2015-03-28 18:00:00.123456789"));
   }
 
+  @Test
   public void testToUtcTimestamp() throws Exception {
     ObjectInspector valueOI = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
     GenericUDFToUtcTimestamp udf = new GenericUDFToUtcTimestamp();
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFGreatest.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFGreatest.java
index 9787454..dfd2278 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFGreatest.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFGreatest.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hive.ql.udf.generic;
 
-import junit.framework.TestCase;
+
 import org.apache.hadoop.hive.common.type.Date;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -32,9 +32,16 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectIn
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
-public class TestGenericUDFGreatest extends TestCase {
+/**
+ * TestGenericUDFGreatest.
+ */
+public class TestGenericUDFGreatest {
 
+  @Test
   public void testOneArg() throws HiveException {
     @SuppressWarnings("resource")
     GenericUDFGreatest udf = new GenericUDFGreatest();
@@ -50,6 +57,7 @@ public class TestGenericUDFGreatest extends TestCase {
     assertNotNull("greatest() test ", ex);
   }
 
+  @Test
   public void testVoids() throws HiveException {
     GenericUDFGreatest udf = new GenericUDFGreatest();
     ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory.writableVoidObjectInspector;
@@ -60,6 +68,7 @@ public class TestGenericUDFGreatest extends TestCase {
     runAndVerify(new Object[] { null, 1, "test"}, null, udf);
   }
 
+  @Test
   public void testGreatestMixed() throws HiveException {
     GenericUDFGreatest udf = new GenericUDFGreatest();
     ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory.writableIntObjectInspector;
@@ -72,6 +81,7 @@ public class TestGenericUDFGreatest extends TestCase {
   }
 
 
+  @Test
   public void testGreatestStr() throws HiveException {
     GenericUDFGreatest udf = new GenericUDFGreatest();
     ObjectInspector[] arguments = new ObjectInspector[3];
@@ -100,6 +110,7 @@ public class TestGenericUDFGreatest extends TestCase {
     runAndVerify(new String[] { null, null, null }, null, udf);
   }
 
+  @Test
   public void testGreatestInt() throws HiveException {
     GenericUDFGreatest udf = new GenericUDFGreatest();
     ObjectInspector[] arguments = new ObjectInspector[3];
@@ -122,6 +133,7 @@ public class TestGenericUDFGreatest extends TestCase {
     runAndVerify(new Integer[] { null, null, null }, null, udf);
   }
 
+  @Test
   public void testGreatestDouble() throws HiveException {
     GenericUDFGreatest udf = new GenericUDFGreatest();
     ObjectInspector[] arguments = new ObjectInspector[3];
@@ -144,6 +156,7 @@ public class TestGenericUDFGreatest extends TestCase {
     runAndVerify(new Double[] { null, null, null }, null, udf);
   }
 
+  @Test
   public void testGreatestDate() throws HiveException {
     GenericUDFGreatest udf = new GenericUDFGreatest();
     ObjectInspector[] arguments = new ObjectInspector[3];
@@ -166,6 +179,7 @@ public class TestGenericUDFGreatest extends TestCase {
     runAndVerify(new Date[] { null, null, null }, null, udf);
   }
 
+  @Test
   public void testGreatestIntTypes() throws HiveException {
     GenericUDFGreatest udf = new GenericUDFGreatest();
     ObjectInspector[] arguments = new ObjectInspector[4];
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFInitCap.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFInitCap.java
index c871e8d..3090724 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFInitCap.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFInitCap.java
@@ -25,10 +25,16 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.io.Text;
 
-import junit.framework.TestCase;
 
-public class TestGenericUDFInitCap extends TestCase {
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
+/**
+ * TestGenericUDFInitCap.
+ */
+public class TestGenericUDFInitCap {
+
+  @Test
   public void testInitCap() throws HiveException {
     GenericUDFInitCap udf = new GenericUDFInitCap();
     ObjectInspector valueOI = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFLTrim.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFLTrim.java
index 747f12c..207788b 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFLTrim.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFLTrim.java
@@ -26,10 +26,16 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.io.Text;
 
-import junit.framework.TestCase;
 
-public class TestGenericUDFLTrim extends TestCase {
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
+/**
+ * TestGenericUDFLTrim.
+ */
+public class TestGenericUDFLTrim {
+
+  @Test
   public void testTrim() throws HiveException {
     GenericUDFLTrim udf = new GenericUDFLTrim();
     ObjectInspector valueOI = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFLastDay.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFLastDay.java
index 972ab35..3cbcbdb 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFLastDay.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFLastDay.java
@@ -26,10 +26,16 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.io.Text;
 
-import junit.framework.TestCase;
 
-public class TestGenericUDFLastDay extends TestCase {
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
+/**
+ * TestGenericUDFLastDay.
+ */
+public class TestGenericUDFLastDay {
+
+  @Test
   public void testLastDay() throws HiveException {
     GenericUDFLastDay udf = new GenericUDFLastDay();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
@@ -63,6 +69,7 @@ public class TestGenericUDFLastDay extends TestCase {
     runAndVerifyTs("1966-01-31 23:59:59", "1966-01-31", udf);
   }
 
+  @Test
   public void testWrongDateStr() throws HiveException {
     GenericUDFLastDay udf = new GenericUDFLastDay();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
@@ -76,6 +83,7 @@ public class TestGenericUDFLastDay extends TestCase {
     runAndVerify(null, null, udf);
   }
 
+  @Test
   public void testWrongTsStr() throws HiveException {
     GenericUDFLastDay udf = new GenericUDFLastDay();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
@@ -89,6 +97,7 @@ public class TestGenericUDFLastDay extends TestCase {
     runAndVerify("2016-02-28T10:30:45", null, udf);
   }
 
+  @Test
   public void testLastDayTs() throws HiveException {
     GenericUDFLastDay udf = new GenericUDFLastDay();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableTimestampObjectInspector;
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFLeast.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFLeast.java
index cccc70e..fc2d974 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFLeast.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFLeast.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hive.ql.udf.generic;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.common.type.Date;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
@@ -33,9 +33,16 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectIn
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
-public class TestGenericUDFLeast extends TestCase {
+/**
+ * TestGenericUDFLeast.
+ */
+public class TestGenericUDFLeast {
 
+  @Test
   public void testOneArg() throws HiveException {
     @SuppressWarnings("resource")
     GenericUDFLeast udf = new GenericUDFLeast();
@@ -51,6 +58,7 @@ public class TestGenericUDFLeast extends TestCase {
     assertNotNull("least() test ", ex);
   }
 
+  @Test
   public void testVoids() throws HiveException {
     GenericUDFGreatest udf = new GenericUDFGreatest();
     ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory.writableVoidObjectInspector;
@@ -61,6 +69,7 @@ public class TestGenericUDFLeast extends TestCase {
     runAndVerify(new Object[] { null, 1, "test"}, null, udf);
   }
 
+  @Test
   public void testLeastTypes() throws HiveException {
     GenericUDFGreatest udf = new GenericUDFGreatest();
     ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory.writableIntObjectInspector;
@@ -72,6 +81,7 @@ public class TestGenericUDFLeast extends TestCase {
     runAndVerify(new Object[] { 1, 11.1, Date.valueOf("2015-03-20"), "test"}, "test", udf);  //string comparisons
   }
 
+  @Test
   public void testLeastStr() throws HiveException {
     GenericUDFLeast udf = new GenericUDFLeast();
     ObjectInspector[] arguments = new ObjectInspector[3];
@@ -100,6 +110,7 @@ public class TestGenericUDFLeast extends TestCase {
     runAndVerify(new String[] { null, null, null }, null, udf);
   }
 
+  @Test
   public void testLeastInt() throws HiveException {
     GenericUDFLeast udf = new GenericUDFLeast();
     ObjectInspector[] arguments = new ObjectInspector[3];
@@ -122,6 +133,7 @@ public class TestGenericUDFLeast extends TestCase {
     runAndVerify(new Integer[] { null, null, null }, null, udf);
   }
 
+  @Test
   public void testLeastDouble() throws HiveException {
     GenericUDFLeast udf = new GenericUDFLeast();
     ObjectInspector[] arguments = new ObjectInspector[3];
@@ -144,6 +156,7 @@ public class TestGenericUDFLeast extends TestCase {
     runAndVerify(new Double[] { null, null, null }, null, udf);
   }
 
+  @Test
   public void testLeastDate() throws HiveException {
     GenericUDFLeast udf = new GenericUDFLeast();
     ObjectInspector[] arguments = new ObjectInspector[3];
@@ -166,6 +179,7 @@ public class TestGenericUDFLeast extends TestCase {
     runAndVerify(new Date[] { null, null, null }, null, udf);
   }
 
+  @Test
   public void testLeastIntTypes() throws HiveException {
     GenericUDFLeast udf = new GenericUDFLeast();
     ObjectInspector[] arguments = new ObjectInspector[4];
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFLevenshtein.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFLevenshtein.java
index 5e43386..79d4694 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFLevenshtein.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFLevenshtein.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hive.ql.udf.generic;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.ql.exec.UDFArgumentLengthException;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
@@ -28,9 +28,18 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Text;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertNotNull;
+import org.junit.Test;
 
-public class TestGenericUDFLevenshtein extends TestCase {
+/**
+ * TestGenericUDFLevenshtein.
+ */
+public class TestGenericUDFLevenshtein {
 
+  @Test
   public void testLevenshtein() throws HiveException {
     GenericUDFLevenshtein udf = new GenericUDFLevenshtein();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
@@ -50,6 +59,7 @@ public class TestGenericUDFLevenshtein extends TestCase {
     runAndVerify(null, null, null, udf);
   }
 
+  @Test
   public void testLevenshteinWrongType0() throws HiveException {
     @SuppressWarnings("resource")
     GenericUDFLevenshtein udf = new GenericUDFLevenshtein();
@@ -66,6 +76,7 @@ public class TestGenericUDFLevenshtein extends TestCase {
     }
   }
 
+  @Test
   public void testLevenshteinWrongType1() throws HiveException {
     @SuppressWarnings("resource")
     GenericUDFLevenshtein udf = new GenericUDFLevenshtein();
@@ -82,6 +93,7 @@ public class TestGenericUDFLevenshtein extends TestCase {
     }
   }
 
+  @Test
   public void testLevenshteinWrongLength() throws HiveException {
     @SuppressWarnings("resource")
     GenericUDFLevenshtein udf = new GenericUDFLevenshtein();
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFLpad.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFLpad.java
index 52c7ad5..ad8f3e2 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFLpad.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFLpad.java
@@ -27,10 +27,17 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectIn
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Text;
 
-import junit.framework.TestCase;
 
-public class TestGenericUDFLpad extends TestCase {
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import org.junit.Test;
 
+/**
+ * TestGenericUDFLpad.
+ */
+public class TestGenericUDFLpad {
+
+  @Test
   public void testLpad() throws HiveException {
     GenericUDFLpad udf = new GenericUDFLpad();
     ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFMonthsBetween.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFMonthsBetween.java
index e9f32a1..707e841 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFMonthsBetween.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFMonthsBetween.java
@@ -31,10 +31,18 @@ import org.apache.hadoop.io.BooleanWritable;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.Text;
 
-import junit.framework.TestCase;
 
-public class TestGenericUDFMonthsBetween extends TestCase {
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
+/**
+ * TestGenericUDFMonthsBetween.
+ */
+public class TestGenericUDFMonthsBetween {
+
+  @Test
   public void testMonthsBetweenForString() throws HiveException {
     // Default run
     GenericUDFMonthsBetween udf = new GenericUDFMonthsBetween();
@@ -58,6 +66,7 @@ public class TestGenericUDFMonthsBetween extends TestCase {
     testMonthsBetweenForString(udf);
   }
 
+  @Test
   public void testWrongDateStr() throws HiveException {
     GenericUDFMonthsBetween udf = new GenericUDFMonthsBetween();
     ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
@@ -126,6 +135,7 @@ public class TestGenericUDFMonthsBetween extends TestCase {
 
 
 
+  @Test
   public void testMonthsBetweenForTimestamp() throws HiveException {
     GenericUDFMonthsBetween udf = new GenericUDFMonthsBetween();
     ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory.writableTimestampObjectInspector;
@@ -183,6 +193,7 @@ public class TestGenericUDFMonthsBetween extends TestCase {
     runTestTs("2003-04-23 23:59:59", "2003-03-24 00:00:00", 0.99999963, udf);
   }
 
+  @Test
   public void testMonthsBetweenForDate() throws HiveException {
     GenericUDFMonthsBetween udf = new GenericUDFMonthsBetween();
     ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory.writableDateObjectInspector;
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFNextDay.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFNextDay.java
index c211fdd..4acfe61 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFNextDay.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFNextDay.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hive.ql.udf.generic;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -26,9 +26,16 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredObject;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.io.Text;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
-public class TestGenericUDFNextDay extends TestCase {
+/**
+ * TestGenericUDFNextDay.
+ */
+public class TestGenericUDFNextDay {
 
+  @Test
   public void testNextDay() throws HiveException {
     GenericUDFNextDay udf = new GenericUDFNextDay();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
@@ -70,6 +77,7 @@ public class TestGenericUDFNextDay extends TestCase {
     runAndVerify(null, null, null, udf);
   }
 
+  @Test
   public void testNotValidValues() throws Exception {
     GenericUDFNextDay udf = new GenericUDFNextDay();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
@@ -88,6 +96,7 @@ public class TestGenericUDFNextDay extends TestCase {
     runAndVerify("2015-01-14T14:04:34", "SAT", null, udf);
   }
 
+  @Test
   public void testNextDayErrorArg1() throws HiveException {
     @SuppressWarnings("resource")
     GenericUDFNextDay udf = new GenericUDFNextDay();
@@ -105,6 +114,7 @@ public class TestGenericUDFNextDay extends TestCase {
     }
   }
 
+  @Test
   public void testNextDayErrorArg2() throws HiveException {
     @SuppressWarnings("resource")
     GenericUDFNextDay udf = new GenericUDFNextDay();
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFQuarter.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFQuarter.java
index 5e6c14e..0dcc41a 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFQuarter.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFQuarter.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hive.ql.udf.generic;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.common.type.Date;
 import org.apache.hadoop.hive.common.type.Timestamp;
@@ -30,9 +30,17 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Text;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
-public class TestGenericUDFQuarter extends TestCase {
+/**
+ * TestGenericUDFQuarter.
+ */
+public class TestGenericUDFQuarter {
 
+  @Test
   public void testQuarterStr() throws HiveException {
     GenericUDFQuarter udf = new GenericUDFQuarter();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
@@ -81,6 +89,7 @@ public class TestGenericUDFQuarter extends TestCase {
     runAndVerifyStr("1966-12-31 23:59:59.999", 4, udf);
   }
 
+  @Test
   public void testWrongDateStr() throws HiveException {
     GenericUDFQuarter udf = new GenericUDFQuarter();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
@@ -94,6 +103,7 @@ public class TestGenericUDFQuarter extends TestCase {
     runAndVerifyStr(null, null, udf);
   }
 
+  @Test
   public void testQuarterDt() throws HiveException {
     GenericUDFQuarter udf = new GenericUDFQuarter();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableDateObjectInspector;
@@ -120,6 +130,7 @@ public class TestGenericUDFQuarter extends TestCase {
     runAndVerifyDt("1966-12-31", 4, udf);
   }
 
+  @Test
   public void testQuarterTs() throws HiveException {
     GenericUDFQuarter udf = new GenericUDFQuarter();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableTimestampObjectInspector;
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFQuote.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFQuote.java
index 4bb4215..4eff009 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFQuote.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFQuote.java
@@ -24,12 +24,17 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.io.Text;
 
-import junit.framework.TestCase;
+import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
 /**
  * Source for TestGenericUDFQuote.
  */
-public class TestGenericUDFQuote extends TestCase {
+public class TestGenericUDFQuote {
   public TestGenericUDFQuote() {}
+
+  @Test
   public void testQuote() throws HiveException {
     GenericUDFQuote udf = new GenericUDFQuote();
     ObjectInspector valueOI = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFRTrim.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFRTrim.java
index 56356e4..84eae07 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFRTrim.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFRTrim.java
@@ -26,10 +26,16 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.io.Text;
 
-import junit.framework.TestCase;
 
-public class TestGenericUDFRTrim extends TestCase {
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
+/**
+ * TestGenericUDFRTrim.
+ */
+public class TestGenericUDFRTrim {
+
+  @Test
   public void testTrim() throws HiveException {
     GenericUDFRTrim udf = new GenericUDFRTrim();
     ObjectInspector valueOI = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFRegexp.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFRegexp.java
index 1cc0fa6..d6abd20 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFRegexp.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFRegexp.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hive.ql.udf.generic;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredJavaObject;
@@ -27,9 +27,17 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectIn
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.io.BooleanWritable;
 import org.apache.hadoop.io.Text;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
-public class TestGenericUDFRegexp extends TestCase {
+/**
+ * TestGenericUDFRegexp.
+ */
+public class TestGenericUDFRegexp {
 
+  @Test
   public void testConstant() throws HiveException {
     GenericUDFRegExp udf = new GenericUDFRegExp();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
@@ -48,6 +56,7 @@ public class TestGenericUDFRegexp extends TestCase {
     runAndVerifyConst(null, regexText, null, udf);
   }
 
+  @Test
   public void testEmptyConstant() throws HiveException {
     GenericUDFRegExp udf = new GenericUDFRegExp();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
@@ -65,6 +74,7 @@ public class TestGenericUDFRegexp extends TestCase {
     runAndVerifyConst(null, regexText, null, udf);
   }
 
+  @Test
   public void testNullConstant() throws HiveException {
     GenericUDFRegExp udf = new GenericUDFRegExp();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
@@ -81,6 +91,7 @@ public class TestGenericUDFRegexp extends TestCase {
     runAndVerifyConst(null, regexText, null, udf);
   }
 
+  @Test
   public void testNonConstant() throws HiveException {
     GenericUDFRegExp udf = new GenericUDFRegExp();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFReplace.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFReplace.java
index 2cf05b3..df0056c 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFReplace.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFReplace.java
@@ -17,13 +17,22 @@
  */
 package org.apache.hadoop.hive.ql.udf.generic;
 
-import junit.framework.TestCase;
+
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.udf.UDFReplace;
 import org.apache.hadoop.io.Text;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
-public class TestGenericUDFReplace extends TestCase {
+/**
+ * TestGenericUDFReplace.
+ *
+ */
+public class TestGenericUDFReplace {
 
+  @Test
   public void testReplace() throws HiveException {
     UDFReplace udf = new UDFReplace();
 
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFRpad.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFRpad.java
index 967f798..0007a81 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFRpad.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFRpad.java
@@ -25,10 +25,17 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectIn
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Text;
 
-import junit.framework.TestCase;
 
-public class TestGenericUDFRpad extends TestCase {
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import org.junit.Test;
 
+/**
+ * TestGenericUDFRpad.
+ */
+public class TestGenericUDFRpad {
+
+  @Test
   public void testRpad() throws HiveException {
     GenericUDFRpad udf = new GenericUDFRpad();
     ObjectInspector valueOI1 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFSha2.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFSha2.java
index 777dfa9..66f2abc 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFSha2.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFSha2.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hive.ql.udf.generic;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredJavaObject;
@@ -28,9 +28,15 @@ import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Text;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
-public class TestGenericUDFSha2 extends TestCase {
+/**
+ * TestGenericUDFSha2.
+ */
+public class TestGenericUDFSha2 {
 
+  @Test
   public void testSha0Str() throws HiveException {
     GenericUDFSha2 udf = new GenericUDFSha2();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
@@ -49,6 +55,7 @@ public class TestGenericUDFSha2 extends TestCase {
     runAndVerifyStr(null, lenWr, null, udf);
   }
 
+  @Test
   public void testSha0Bin() throws HiveException {
     GenericUDFSha2 udf = new GenericUDFSha2();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
@@ -67,6 +74,7 @@ public class TestGenericUDFSha2 extends TestCase {
     runAndVerifyBin(null, lenWr, null, udf);
   }
 
+  @Test
   public void testSha200Str() throws HiveException {
     GenericUDFSha2 udf = new GenericUDFSha2();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
@@ -80,6 +88,7 @@ public class TestGenericUDFSha2 extends TestCase {
     runAndVerifyStr("ABC", lenWr, null, udf);
   }
 
+  @Test
   public void testSha200Bin() throws HiveException {
     GenericUDFSha2 udf = new GenericUDFSha2();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
@@ -93,6 +102,7 @@ public class TestGenericUDFSha2 extends TestCase {
     runAndVerifyBin(new byte[] { 65, 66, 67 }, lenWr, null, udf);
   }
 
+  @Test
   public void testSha256Str() throws HiveException {
     GenericUDFSha2 udf = new GenericUDFSha2();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
@@ -111,6 +121,7 @@ public class TestGenericUDFSha2 extends TestCase {
     runAndVerifyStr(null, lenWr, null, udf);
   }
 
+  @Test
   public void testSha256Bin() throws HiveException {
     GenericUDFSha2 udf = new GenericUDFSha2();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
@@ -129,6 +140,7 @@ public class TestGenericUDFSha2 extends TestCase {
     runAndVerifyBin(null, lenWr, null, udf);
   }
 
+  @Test
   public void testSha384Str() throws HiveException {
     GenericUDFSha2 udf = new GenericUDFSha2();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
@@ -153,6 +165,7 @@ public class TestGenericUDFSha2 extends TestCase {
     runAndVerifyStr(null, lenWr, null, udf);
   }
 
+  @Test
   public void testSha384Bin() throws HiveException {
     GenericUDFSha2 udf = new GenericUDFSha2();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
@@ -177,6 +190,7 @@ public class TestGenericUDFSha2 extends TestCase {
     runAndVerifyBin(null, lenWr, null, udf);
   }
 
+  @Test
   public void testSha512Str() throws HiveException {
     GenericUDFSha2 udf = new GenericUDFSha2();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
@@ -201,6 +215,7 @@ public class TestGenericUDFSha2 extends TestCase {
     runAndVerifyStr(null, lenWr, null, udf);
   }
 
+  @Test
   public void testSha512Bin() throws HiveException {
     GenericUDFSha2 udf = new GenericUDFSha2();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
@@ -225,6 +240,7 @@ public class TestGenericUDFSha2 extends TestCase {
     runAndVerifyBin(null, lenWr, null, udf);
   }
 
+  @Test
   public void testShaNullStr() throws HiveException {
     GenericUDFSha2 udf = new GenericUDFSha2();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
@@ -238,6 +254,7 @@ public class TestGenericUDFSha2 extends TestCase {
     runAndVerifyStr("ABC", lenWr, null, udf);
   }
 
+  @Test
   public void testShaNullBin() throws HiveException {
     GenericUDFSha2 udf = new GenericUDFSha2();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFSoundex.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFSoundex.java
index e35f2d8..c6a4f70 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFSoundex.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFSoundex.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hive.ql.udf.generic;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.ql.exec.UDFArgumentLengthException;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
@@ -27,9 +27,18 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredObject;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.io.Text;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertNotNull;
+import org.junit.Test;
 
-public class TestGenericUDFSoundex extends TestCase {
+/**
+ * TestGenericUDFSoundex.
+ */
+public class TestGenericUDFSoundex {
 
+  @Test
   public void testSoundex() throws HiveException {
     GenericUDFSoundex udf = new GenericUDFSoundex();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
@@ -53,6 +62,7 @@ public class TestGenericUDFSoundex extends TestCase {
     runAndVerify("\u3500\u3501\u3502\u3503", null, udf);
   }
 
+  @Test
   public void testSoundexWrongType0() throws HiveException {
     @SuppressWarnings("resource")
     GenericUDFSoundex udf = new GenericUDFSoundex();
@@ -68,6 +78,7 @@ public class TestGenericUDFSoundex extends TestCase {
     }
   }
 
+  @Test
   public void testSoundexWrongLength() throws HiveException {
     @SuppressWarnings("resource")
     GenericUDFSoundex udf = new GenericUDFSoundex();
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFSubstringIndex.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFSubstringIndex.java
index 22ee3d1..31b2282 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFSubstringIndex.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFSubstringIndex.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hive.ql.udf.generic;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredJavaObject;
@@ -27,9 +27,15 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectIn
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Text;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
-public class TestGenericUDFSubstringIndex extends TestCase {
+/**
+ * TestGenericUDFSubstringIndex.
+ */
+public class TestGenericUDFSubstringIndex {
 
+  @Test
   public void testSubstringIndex() throws HiveException {
     GenericUDFSubstringIndex udf = new GenericUDFSubstringIndex();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
@@ -62,6 +68,7 @@ public class TestGenericUDFSubstringIndex extends TestCase {
     runAndVerify("www.apache.org", ".", null, null, udf);
   }
 
+  @Test
   public void testSubstringIndexConst() throws HiveException {
     GenericUDFSubstringIndex udf = new GenericUDFSubstringIndex();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFToUnixTimestamp.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFToUnixTimestamp.java
index 61623d5..45b60db 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFToUnixTimestamp.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFToUnixTimestamp.java
@@ -30,9 +30,15 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectIn
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 
-import junit.framework.TestCase;
 
-public class TestGenericUDFToUnixTimestamp extends TestCase {
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
+
+/**
+ * TestGenericUDFToUnixTimestamp.
+ */
+public class TestGenericUDFToUnixTimestamp {
 
   public static void runAndVerify(GenericUDFToUnixTimeStamp udf,
       Object arg, Object expected) throws HiveException {
@@ -56,6 +62,7 @@ public class TestGenericUDFToUnixTimestamp extends TestCase {
     }
   }
 
+  @Test
   public void testTimestamp() throws HiveException {
     GenericUDFToUnixTimeStamp udf = new GenericUDFToUnixTimeStamp();
     ObjectInspector valueOI = PrimitiveObjectInspectorFactory.writableTimestampObjectInspector;
@@ -76,6 +83,7 @@ public class TestGenericUDFToUnixTimestamp extends TestCase {
     runAndVerify(udf, null, null);
   }
 
+  @Test
   public void testDate() throws HiveException {
     GenericUDFToUnixTimeStamp udf = new GenericUDFToUnixTimeStamp();
     ObjectInspector valueOI = PrimitiveObjectInspectorFactory.writableDateObjectInspector;
@@ -91,6 +99,7 @@ public class TestGenericUDFToUnixTimestamp extends TestCase {
     runAndVerify(udf, null, null);
   }
 
+  @Test
   public void testString() throws HiveException {
     GenericUDFToUnixTimeStamp udf1 = new GenericUDFToUnixTimeStamp();
     ObjectInspector valueOI = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFTrim.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFTrim.java
index 7d6ecb2..414425f 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFTrim.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFTrim.java
@@ -26,10 +26,16 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.io.Text;
 
-import junit.framework.TestCase;
 
-public class TestGenericUDFTrim extends TestCase {
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
+/**
+ * TestGenericUDFTrim.
+ */
+public class TestGenericUDFTrim {
+
+  @Test
   public void testTrim() throws HiveException {
     GenericUDFTrim udf = new GenericUDFTrim();
     ObjectInspector valueOI = PrimitiveObjectInspectorFactory.writableStringObjectInspector;
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFTrunc.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFTrunc.java
index 86511a6..f82420e 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFTrunc.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFTrunc.java
@@ -29,10 +29,16 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.io.Text;
 
-import junit.framework.TestCase;
 
-public class TestGenericUDFTrunc extends TestCase {
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
+/**
+ * TestGenericUDFTrunc.
+ */
+public class TestGenericUDFTrunc {
+
+  @Test
   public void testStringToDateWithMonthFormat() throws HiveException {
     GenericUDFTrunc udf = new GenericUDFTrunc();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.javaStringObjectInspector;
@@ -111,6 +117,7 @@ public class TestGenericUDFTrunc extends TestCase {
     runAndVerify("2016-02-01", udf, initArgs, evalArgs);
   }
 
+  @Test
   public void testStringToDateWithQuarterFormat() throws HiveException {
     GenericUDFTrunc udf = new GenericUDFTrunc();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.javaStringObjectInspector;
@@ -213,6 +220,7 @@ public class TestGenericUDFTrunc extends TestCase {
     runAndVerify("2016-10-01", udf, initArgs, evalArgs);
   }
 
+  @Test
   public void testStringToDateWithYearFormat() throws HiveException {
     GenericUDFTrunc udf = new GenericUDFTrunc();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.javaStringObjectInspector;
@@ -291,6 +299,7 @@ public class TestGenericUDFTrunc extends TestCase {
     runAndVerify("2016-01-01", udf, initArgs, evalArgs);
   }
 
+  @Test
   public void testTimestampToDateWithMonthFormat() throws HiveException {
     GenericUDFTrunc udf = new GenericUDFTrunc();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableTimestampObjectInspector;
@@ -385,6 +394,7 @@ public class TestGenericUDFTrunc extends TestCase {
     runAndVerify("2016-02-01", udf, initArgs, evalArgs);
   }
 
+  @Test
   public void testTimestampToDateWithQuarterFormat() throws HiveException {
     GenericUDFTrunc udf = new GenericUDFTrunc();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableTimestampObjectInspector;
@@ -509,6 +519,7 @@ public class TestGenericUDFTrunc extends TestCase {
     runAndVerify("2016-10-01", udf, initArgs, evalArgs);
   }
 
+  @Test
   public void testTimestampToDateWithYearFormat() throws HiveException {
     GenericUDFTrunc udf = new GenericUDFTrunc();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableTimestampObjectInspector;
@@ -603,6 +614,7 @@ public class TestGenericUDFTrunc extends TestCase {
     runAndVerify("2016-01-01", udf, initArgs, evalArgs);
   }
 
+  @Test
   public void testDateWritableToDateWithMonthFormat() throws HiveException {
     GenericUDFTrunc udf = new GenericUDFTrunc();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableDateObjectInspector;
@@ -648,6 +660,7 @@ public class TestGenericUDFTrunc extends TestCase {
     runAndVerify("2016-02-01", udf, initArgs, evalArgs);
   }
 
+  @Test
   public void testDateWritableToDateWithQuarterFormat() throws HiveException {
     GenericUDFTrunc udf = new GenericUDFTrunc();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableDateObjectInspector;
@@ -705,6 +718,7 @@ public class TestGenericUDFTrunc extends TestCase {
     runAndVerify("2016-10-01", udf, initArgs, evalArgs);
   }
 
+  @Test
   public void testDateWritableToDateWithYearFormat() throws HiveException {
     GenericUDFTrunc udf = new GenericUDFTrunc();
     ObjectInspector valueOI0 = PrimitiveObjectInspectorFactory.writableDateObjectInspector;
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/TestStatsSerde.java b/serde/src/test/org/apache/hadoop/hive/serde2/TestStatsSerde.java
index 66b810a..be2d09b 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/TestStatsSerde.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/TestStatsSerde.java
@@ -23,7 +23,7 @@ import java.util.List;
 import java.util.Properties;
 import java.util.Random;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
@@ -45,17 +45,20 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
 import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.Text;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotSame;
+import org.junit.Test;
 
-public class TestStatsSerde extends TestCase {
-
-  public TestStatsSerde(String name) {
-    super(name);
-  }
+/**
+ * StatsSerde Test.
+ */
+public class TestStatsSerde {
 
   /**
    * Test LazySimpleSerDe
    */
 
+  @Test
   public void testLazySimpleSerDe() throws Throwable {
     try {
       // Create the SerDe
@@ -100,6 +103,7 @@ public class TestStatsSerde extends TestCase {
    * Test LazyBinarySerDe
    */
 
+  @Test
   public void testLazyBinarySerDe() throws Throwable {
     try {
       System.out.println("test: testLazyBinarySerDe");
@@ -165,6 +169,7 @@ public class TestStatsSerde extends TestCase {
    * Test ColumnarSerDe
    */
 
+  @Test
   public void testColumnarSerDe() throws Throwable {
     try {
       System.out.println("test: testColumnarSerde");
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/TestTCTLSeparatedProtocol.java b/serde/src/test/org/apache/hadoop/hive/serde2/TestTCTLSeparatedProtocol.java
index 9d72a1a..177944c 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/TestTCTLSeparatedProtocol.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/TestTCTLSeparatedProtocol.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hive.serde2;
 
 import java.util.Properties;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.serde.serdeConstants;
@@ -32,16 +32,22 @@ import org.apache.thrift.protocol.TStruct;
 import org.apache.thrift.transport.TMemoryBuffer;
 import org.apache.thrift.transport.TTransport;
 import org.apache.thrift.transport.TTransportException;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import org.junit.Test;
 
 /**
  * TestTCTLSeparatedProtocol.
  *
  */
-public class TestTCTLSeparatedProtocol extends TestCase {
+public class TestTCTLSeparatedProtocol {
 
   public TestTCTLSeparatedProtocol() throws Exception {
   }
 
+  @Test
   public void testReads() throws Exception {
     TMemoryBuffer trans = new TMemoryBuffer(1024);
     String foo = "Hello";
@@ -114,6 +120,7 @@ public class TestTCTLSeparatedProtocol extends TestCase {
     prot.readStructEnd();
   }
 
+  @Test
   public void testWrites() throws Exception {
     TMemoryBuffer trans = new TMemoryBuffer(1024);
     TCTLSeparatedProtocol prot = new TCTLSeparatedProtocol(trans, 1024);
@@ -236,6 +243,7 @@ public class TestTCTLSeparatedProtocol extends TestCase {
     prot.readStructEnd();
   }
 
+  @Test
   public void testQuotedWrites() throws Exception {
     TMemoryBuffer trans = new TMemoryBuffer(4096);
     TCTLSeparatedProtocol prot = new TCTLSeparatedProtocol(trans, 4096);
@@ -308,6 +316,7 @@ public class TestTCTLSeparatedProtocol extends TestCase {
    * with a more TRegexLike protocol, but for this case, TCTLSeparatedProtocol
    * can do it.
    */
+  @Test
   public void test1ApacheLogFormat() throws Exception {
     final String sample =
       "127.0.0.1 - frank [10/Oct/2000:13:55:36 -0700] \"GET /apache_pb.gif HTTP/1.0\" 200 2326";
@@ -382,6 +391,7 @@ public class TestTCTLSeparatedProtocol extends TestCase {
     prot.readStructEnd();
   }
 
+  @Test
   public void testNulls() throws Exception {
     TMemoryBuffer trans = new TMemoryBuffer(1024);
     TCTLSeparatedProtocol prot = new TCTLSeparatedProtocol(trans, 10);
@@ -476,6 +486,7 @@ public class TestTCTLSeparatedProtocol extends TestCase {
     assertTrue(ret1 == 0);
   }
 
+  @Test
   public void testShouldThrowRunTimeExceptionIfUnableToInitializeTokenizer() throws Exception {
     TCTLSeparatedProtocol separatedProtocol = new TCTLSeparatedProtocol(new TTransport() {
       @Override
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableFast.java b/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableFast.java
index 62741d3..cb48fef 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableFast.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableFast.java
@@ -39,10 +39,12 @@ import org.apache.hadoop.hive.serde2.objectinspector.UnionObject;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.io.BytesWritable;
 
-import junit.framework.TestCase;
 import org.junit.Assert;
+import static org.junit.Assert.fail;
+import static org.junit.Assert.assertTrue;
+import org.junit.Test;
 
-public class TestBinarySortableFast extends TestCase {
+public class TestBinarySortableFast {
 
   private static String debugDetailedReadPositionString;
   private static StackTraceElement[] debugStackTrace;
@@ -140,7 +142,7 @@ public class TestBinarySortableFast extends TestCase {
         }
       }
       if (writeColumnCount == columnCount) {
-        TestCase.assertTrue(binarySortableDeserializeRead.isEndOfInputReached());
+        assertTrue(binarySortableDeserializeRead.isEndOfInputReached());
       }
 
       /*
@@ -306,7 +308,7 @@ public class TestBinarySortableFast extends TestCase {
         }
       }
       if (writeColumnCount == columnCount) {
-        TestCase.assertTrue(binarySortableDeserializeRead.isEndOfInputReached());
+        assertTrue(binarySortableDeserializeRead.isEndOfInputReached());
       }
     }
   }
@@ -319,7 +321,7 @@ public class TestBinarySortableFast extends TestCase {
       Object complexFieldObj = VerifyFast.deserializeReadComplexType(binarySortableDeserializeRead, typeInfo);
       if (expectedObject == null) {
         if (complexFieldObj != null) {
-          TestCase.fail("Field reports not null but object is null (class " + complexFieldObj.getClass().getName() +
+          fail("Field reports not null but object is null (class " + complexFieldObj.getClass().getName() +
               ", " + complexFieldObj.toString() + ")");
         }
       } else {
@@ -331,12 +333,12 @@ public class TestBinarySortableFast extends TestCase {
               return;
             }
           }
-          TestCase.fail("Field reports null but object is not null (class " + expectedObject.getClass().getName() +
+          fail("Field reports null but object is not null (class " + expectedObject.getClass().getName() +
               ", " + expectedObject.toString() + ")");
         }
       }
       if (!VerifyLazy.lazyCompare(typeInfo, complexFieldObj, expectedObject)) {
-        TestCase.fail("Comparision failed typeInfo " + typeInfo.toString());
+        fail("Comparision failed typeInfo " + typeInfo.toString());
       }
     }
   }
@@ -500,14 +502,17 @@ public class TestBinarySortableFast extends TestCase {
     }
   }
 
+  @Test
   public void testBinarySortableFastPrimitive() throws Throwable {
     testBinarySortableFast(SerdeRandomRowSource.SupportedTypes.PRIMITIVE, 0);
   }
 
+  @Test
   public void testBinarySortableFastComplexDepthOne() throws Throwable {
     testBinarySortableFast(SerdeRandomRowSource.SupportedTypes.ALL_EXCEPT_MAP, 1);
   }
 
+  @Test
   public void testBinarySortableFastComplexDepthFour() throws Throwable {
     testBinarySortableFast(SerdeRandomRowSource.SupportedTypes.ALL_EXCEPT_MAP, 4);
   }
@@ -519,4 +524,4 @@ public class TestBinarySortableFast extends TestCase {
     }
     return sb.toString();
   }
-}
\ No newline at end of file
+}
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableSerDe.java b/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableSerDe.java
index e4e0213..e64dc80 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableSerDe.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableSerDe.java
@@ -34,13 +34,16 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
 import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
 import org.apache.hadoop.io.BytesWritable;
 
-import junit.framework.TestCase;
+
+import static org.junit.Assert.fail;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
 /**
  * TestBinarySortableSerDe.
  *
  */
-public class TestBinarySortableSerDe extends TestCase {
+public class TestBinarySortableSerDe {
 
   private static final String DECIMAL_CHARS = "0123456789";
 
@@ -135,6 +138,7 @@ public class TestBinarySortableSerDe extends TestCase {
     }
   }
 
+  @Test
   public void testBinarySortableSerDe() throws Throwable {
     try {
 
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/columnar/TestLazyBinaryColumnarSerDe.java b/serde/src/test/org/apache/hadoop/hive/serde2/columnar/TestLazyBinaryColumnarSerDe.java
index 6838138..81ddc59 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/columnar/TestLazyBinaryColumnarSerDe.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/columnar/TestLazyBinaryColumnarSerDe.java
@@ -40,8 +40,14 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
 import org.apache.hadoop.hive.serde2.objectinspector.SimpleMapEqualComparer;
 import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
 import org.apache.hadoop.io.LongWritable;
+import static org.junit.Assert.fail;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
-public class TestLazyBinaryColumnarSerDe extends TestCase {
+/**
+ * LazyBinaryColumnarSerDe Test.
+ */
+public class TestLazyBinaryColumnarSerDe {
 
   private static class InnerStruct {
     public InnerStruct(Integer i, Long l) {
@@ -66,6 +72,7 @@ public class TestLazyBinaryColumnarSerDe extends TestCase {
     InnerStruct mStruct;
   }
 
+  @Test
   public void testSerDe() throws SerDeException {
     StructObjectInspector oi = (StructObjectInspector) ObjectInspectorFactory
         .getReflectionObjectInspector(OuterStruct.class, ObjectInspectorOptions.JAVA);
@@ -107,6 +114,7 @@ public class TestLazyBinaryColumnarSerDe extends TestCase {
     }
   }
 
+  @Test
   public void testSerDeEmpties() throws SerDeException {
     StructObjectInspector oi = (StructObjectInspector) ObjectInspectorFactory
         .getReflectionObjectInspector(OuterStruct.class, ObjectInspectorOptions.JAVA);
@@ -142,6 +150,7 @@ public class TestLazyBinaryColumnarSerDe extends TestCase {
     }
   }
 
+  @Test
   public void testLazyBinaryColumnarSerDeWithEmptyBinary() throws SerDeException {
     StructObjectInspector oi = (StructObjectInspector) ObjectInspectorFactory
         .getReflectionObjectInspector(OuterStruct.class, ObjectInspectorOptions.JAVA);
@@ -175,6 +184,7 @@ public class TestLazyBinaryColumnarSerDe extends TestCase {
     assert false;
   }
 
+  @Test
   public void testSerDeOuterNulls() throws SerDeException {
     StructObjectInspector oi = (StructObjectInspector) ObjectInspectorFactory
         .getReflectionObjectInspector(OuterStruct.class, ObjectInspectorOptions.JAVA);
@@ -199,6 +209,7 @@ public class TestLazyBinaryColumnarSerDe extends TestCase {
     }
   }
 
+  @Test
   public void testSerDeInnerNulls() throws SerDeException {
     StructObjectInspector oi = (StructObjectInspector) ObjectInspectorFactory
         .getReflectionObjectInspector(OuterStruct.class, ObjectInspectorOptions.JAVA);
@@ -261,6 +272,7 @@ public class TestLazyBinaryColumnarSerDe extends TestCase {
    * in the new schema, and seeing if this serde can to read both types of data from the resultant table.
    * @throws SerDeException
    */
+  @Test
   public void testHandlingAlteredSchemas() throws SerDeException {
     StructObjectInspector oi = (StructObjectInspector) ObjectInspectorFactory
         .getReflectionObjectInspector(BeforeStruct.class,
@@ -322,4 +334,4 @@ public class TestLazyBinaryColumnarSerDe extends TestCase {
     Assert.assertEquals(((LongWritable) objs2.get(1)).get(), 12L);
     Assert.assertEquals(((LongWritable) objs2.get(2)).get(), 13L);
   }
-}
\ No newline at end of file
+}
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/lazy/TestLazyArrayMapStruct.java b/serde/src/test/org/apache/hadoop/hive/serde2/lazy/TestLazyArrayMapStruct.java
index 730764e..d21a86b 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/lazy/TestLazyArrayMapStruct.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/lazy/TestLazyArrayMapStruct.java
@@ -25,7 +25,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Properties;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.serde2.SerDeException;
@@ -41,12 +41,17 @@ import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Text;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import org.junit.Test;
 
 /**
  * Tests LazyArray, LazyMap, LazyStruct and LazyUnion
  *
  */
-public class TestLazyArrayMapStruct extends TestCase {
+public class TestLazyArrayMapStruct {
 
   // nesting level limits
   static final int EXTENDED_LEVEL_THRESHOLD = 24;
@@ -55,6 +60,7 @@ public class TestLazyArrayMapStruct extends TestCase {
   /**
    * Test the LazyArray class.
    */
+  @Test
   public void testLazyArray() throws Throwable {
     try {
       // Array of Byte
@@ -133,6 +139,7 @@ public class TestLazyArrayMapStruct extends TestCase {
   /**
    * Test the LazyMap class.
    */
+  @Test
   public void testLazyMap() throws Throwable {
     try {
       {
@@ -199,6 +206,7 @@ public class TestLazyArrayMapStruct extends TestCase {
    * where '[' and  ']' don't exist, only for notation purpose,
    * STX with value of 2 as entry separator, ETX with 3 as key/value separator
    * */
+  @Test
   public void testLazyMapWithBadEntries() throws Throwable {
     try {
       {
@@ -304,6 +312,7 @@ public class TestLazyArrayMapStruct extends TestCase {
   /**
    * Test the LazyMap class.
    */
+  @Test
   public void testLazyMapWithDuplicateKeys() throws Throwable {
     try {
       {
@@ -367,6 +376,7 @@ public class TestLazyArrayMapStruct extends TestCase {
   /**
    * Test the LazyStruct class.
    */
+  @Test
   public void testLazyStruct() throws Throwable {
     try {
       {
@@ -448,6 +458,7 @@ public class TestLazyArrayMapStruct extends TestCase {
   /**
    * Test the LazyUnion class.
    */
+  @Test
   public void testLazyUnion() throws Throwable {
     try {
       {
@@ -516,6 +527,7 @@ public class TestLazyArrayMapStruct extends TestCase {
   /**
    * Test the LazyArray class with multiple levels of nesting
    */
+  @Test
   public void testLazyArrayNested() throws Throwable {
     for(int i = 2; i < EXTENDED_LEVEL_THRESHOLD; i++ ){
       testNestedinArrayAtLevelExtended(i, ObjectInspector.Category.LIST);
@@ -525,6 +537,7 @@ public class TestLazyArrayMapStruct extends TestCase {
   /**
    * Test the LazyArray class with multiple levels of nesting
    */
+  @Test
   public void testLazyArrayNestedExceedLimit() throws Throwable {
     checkExtendedLimitExceeded(EXTENDED_LEVEL_THRESHOLD, ObjectInspector.Category.LIST);
   }
@@ -543,6 +556,7 @@ public class TestLazyArrayMapStruct extends TestCase {
    * Test the LazyArray class with multiple levels of nesting, when nesting
    * levels are not extended
    */
+  @Test
   public void testLazyArrayNestedExceedLimitNotExtended() throws Throwable {
     checkNotExtendedLimitExceeded(DEFAULT_LEVEL_THRESHOLD,
         ObjectInspector.Category.LIST);
@@ -552,6 +566,7 @@ public class TestLazyArrayMapStruct extends TestCase {
    * Test the LazyMap class with multiple levels of nesting, when nesting
    * levels are not extended
    */
+  @Test
   public void testLazyMapNestedExceedLimitNotExtended() throws Throwable {
     checkNotExtendedLimitExceeded(DEFAULT_LEVEL_THRESHOLD-1,
         ObjectInspector.Category.MAP);
@@ -561,6 +576,7 @@ public class TestLazyArrayMapStruct extends TestCase {
    * Test the LazyMap class with multiple levels of nesting, when nesting
    * levels are not extended
    */
+  @Test
   public void testLazyStructNestedExceedLimitNotExtended() throws Throwable {
     checkNotExtendedLimitExceeded(DEFAULT_LEVEL_THRESHOLD,
         ObjectInspector.Category.STRUCT);
@@ -570,6 +586,7 @@ public class TestLazyArrayMapStruct extends TestCase {
    * Test the LazyMap class with multiple levels of nesting, when nesting
    * levels are not extended
    */
+  @Test
   public void testLazyUnionNestedExceedLimitNotExtended() throws Throwable {
     checkNotExtendedLimitExceeded(DEFAULT_LEVEL_THRESHOLD,
         ObjectInspector.Category.UNION);
@@ -588,6 +605,7 @@ public class TestLazyArrayMapStruct extends TestCase {
   /**
    * Test the LazyMap class with multiple levels of nesting
    */
+  @Test
   public void testLazyMapNested() throws Throwable {
     //map max nesting level is one less because it uses an additional separator
     for(int i = 2; i < EXTENDED_LEVEL_THRESHOLD - 1; i++ ){
@@ -598,6 +616,7 @@ public class TestLazyArrayMapStruct extends TestCase {
   /**
    * Test the LazyMap class with multiple levels of nesting
    */
+  @Test
   public void testLazyMapNestedExceedLimit() throws Throwable {
     //map max nesting level is one less because it uses an additional separator
     checkExtendedLimitExceeded(EXTENDED_LEVEL_THRESHOLD - 1, ObjectInspector.Category.MAP);
@@ -606,6 +625,7 @@ public class TestLazyArrayMapStruct extends TestCase {
   /**
    * Test the LazyUnion class with multiple levels of nesting
    */
+  @Test
   public void testLazyUnionNested() throws Throwable {
     for(int i = 2; i < EXTENDED_LEVEL_THRESHOLD; i++ ){
      testNestedinArrayAtLevelExtended(i, ObjectInspector.Category.UNION);
@@ -615,6 +635,7 @@ public class TestLazyArrayMapStruct extends TestCase {
   /**
    * Test the LazyUnion class with multiple levels of nesting
    */
+  @Test
   public void testLazyUnionNestedExceedLimit() throws Throwable {
     checkExtendedLimitExceeded(EXTENDED_LEVEL_THRESHOLD, ObjectInspector.Category.UNION);
   }
@@ -622,6 +643,7 @@ public class TestLazyArrayMapStruct extends TestCase {
   /**
    * Test the LazyStruct class with multiple levels of nesting
    */
+  @Test
   public void testLazyStructNested() throws Throwable {
     for(int i = 2; i < EXTENDED_LEVEL_THRESHOLD; i++ ){
      testNestedinArrayAtLevelExtended(i, ObjectInspector.Category.STRUCT);
@@ -645,6 +667,7 @@ public class TestLazyArrayMapStruct extends TestCase {
   /**
    * Test the LazyStruct class with multiple levels of nesting
    */
+  @Test
   public void testLazyStructNestedExceedLimit() throws Throwable {
     checkExtendedLimitExceeded(EXTENDED_LEVEL_THRESHOLD, ObjectInspector.Category.STRUCT);
   }
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/lazy/TestLazyPrimitive.java b/serde/src/test/org/apache/hadoop/hive/serde2/lazy/TestLazyPrimitive.java
index 79bf5fb..828f17f 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/lazy/TestLazyPrimitive.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/lazy/TestLazyPrimitive.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hive.serde2.lazy;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.common.type.Date;
 import org.apache.hadoop.hive.common.type.Timestamp;
@@ -30,12 +30,16 @@ import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import org.junit.Test;
 
 /**
  * TestLazyPrimitive.
  *
  */
-public class TestLazyPrimitive extends TestCase {
+public class TestLazyPrimitive {
 
   /**
    * Initialize the LazyObject with the parameters, wrapping the byte[]
@@ -51,6 +55,7 @@ public class TestLazyPrimitive extends TestCase {
   /**
    * Test the LazyByte class.
    */
+  @Test
   public void testLazyByte() throws Throwable {
     try {
       LazyByte b = new LazyByte(
@@ -94,6 +99,7 @@ public class TestLazyPrimitive extends TestCase {
   /**
    * Test the LazyShort class.
    */
+  @Test
   public void testLazyShort() throws Throwable {
     try {
       LazyShort b = new LazyShort(
@@ -141,6 +147,7 @@ public class TestLazyPrimitive extends TestCase {
   /**
    * Test the LazyInteger class.
    */
+  @Test
   public void testLazyInteger() throws Throwable {
     try {
       LazyInteger b = new LazyInteger(
@@ -196,6 +203,7 @@ public class TestLazyPrimitive extends TestCase {
   /**
    * Test the LazyLong class.
    */
+  @Test
   public void testLazyLong() throws Throwable {
     try {
       LazyLong b = new LazyLong(
@@ -259,6 +267,7 @@ public class TestLazyPrimitive extends TestCase {
   /**
    * Test the LazyDouble class.
    */
+  @Test
   public void testLazyDouble() throws Throwable {
     try {
       LazyDouble b = new LazyDouble(
@@ -359,6 +368,7 @@ public class TestLazyPrimitive extends TestCase {
   /**
    * Test the LazyString class.
    */
+  @Test
   public void testLazyString() throws Throwable {
     try {
       LazyString b = new LazyString(LazyPrimitiveObjectInspectorFactory
@@ -376,6 +386,7 @@ public class TestLazyPrimitive extends TestCase {
     }
   }
 
+  @Test
   public void testLazyBinary() {
     LazyBinary ba = new LazyBinary(LazyPrimitiveObjectInspectorFactory.LAZY_BINARY_OBJECT_INSPECTOR);
     initLazyObject(ba, new byte[] {}, 0, 0);
@@ -390,6 +401,7 @@ public class TestLazyPrimitive extends TestCase {
     assertEquals(new BytesWritable(new byte[] {'\n'}), ba.getWritableObject());
   }
 
+  @Test
   public void testLazyTimestamp() throws Throwable {
     LazyTimestamp t = new LazyTimestamp(LazyPrimitiveObjectInspectorFactory.LAZY_TIMESTAMP_OBJECT_INSPECTOR);
     String nullDate = "NULL";
@@ -408,6 +420,7 @@ public class TestLazyPrimitive extends TestCase {
     assertEquals(true, t.isNull);
   }
 
+  @Test
   public void testLazyDate() throws Throwable {
     LazyDate t = new LazyDate(LazyPrimitiveObjectInspectorFactory.LAZY_DATE_OBJECT_INSPECTOR);
     String nullDate = "NULL";
@@ -427,6 +440,7 @@ public class TestLazyPrimitive extends TestCase {
 
   }
 
+  @Test
   public void testLazyIntegerWrite() throws Throwable {
     try {
       ByteStream.Output out = new ByteStream.Output();
@@ -447,6 +461,7 @@ public class TestLazyPrimitive extends TestCase {
     }
   }
 
+  @Test
   public void testLazyLongWrite() throws Throwable {
     try {
       ByteStream.Output out = new ByteStream.Output();
@@ -501,6 +516,7 @@ public class TestLazyPrimitive extends TestCase {
     }
   }
 
+  @Test
   public void testLazyIntWithSpaces() throws Throwable {
     Object[][] casesWithoutSpaces = {
         {"0", 0},
@@ -583,6 +599,7 @@ public class TestLazyPrimitive extends TestCase {
     }
   }
 
+  @Test
   public void testLazyLongWithSpaces() throws Throwable {
     Object[][] casesWithoutSpaces = {
         {"0", 0},
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/lazy/TestLazySimpleFast.java b/serde/src/test/org/apache/hadoop/hive/serde2/lazy/TestLazySimpleFast.java
index 34b51c8..6256168 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/lazy/TestLazySimpleFast.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/lazy/TestLazySimpleFast.java
@@ -46,9 +46,15 @@ import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.Text;
 
-import junit.framework.TestCase;
 
-public class TestLazySimpleFast extends TestCase {
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import org.junit.Test;
+
+/**
+ * LazySimpleFast Test.
+ */
+public class TestLazySimpleFast {
 
   private void testLazySimpleFast(
     SerdeRandomRowSource source, Object[][] rows,
@@ -122,7 +128,7 @@ public class TestLazySimpleFast extends TestCase {
         }
       }
       if (writeColumnCount == columnCount) {
-        TestCase.assertTrue(lazySimpleDeserializeRead.isEndOfInputReached());
+        assertTrue(lazySimpleDeserializeRead.isEndOfInputReached());
       }
     }
 
@@ -197,7 +203,7 @@ public class TestLazySimpleFast extends TestCase {
         }
       }
       if (writeColumnCount == columnCount) {
-        TestCase.assertTrue(lazySimpleDeserializeRead.isEndOfInputReached());
+        assertTrue(lazySimpleDeserializeRead.isEndOfInputReached());
       }
     }
   }
@@ -209,7 +215,7 @@ public class TestLazySimpleFast extends TestCase {
     } else {
       Object complexFieldObj = VerifyFast.deserializeReadComplexType(lazySimpleDeserializeRead, typeInfo);
       if (complexFieldObj != null) {
-        TestCase.fail("Field report not null but object is null");
+        fail("Field report not null but object is null");
       }
     }
   }
@@ -222,7 +228,8 @@ public class TestLazySimpleFast extends TestCase {
       Object complexFieldObj = VerifyFast.deserializeReadComplexType(lazySimpleDeserializeRead, typeInfo);
       if (expectedObject == null) {
         if (complexFieldObj != null) {
-          TestCase.fail("Field reports not null but object is null (class " + complexFieldObj.getClass().getName() + ", " + complexFieldObj.toString() + ")");
+          fail("Field reports not null but object is null (class " + complexFieldObj.getClass().getName() +
+              ", " + complexFieldObj.toString() + ")");
         }
       } else {
         if (complexFieldObj == null) {
@@ -233,11 +240,12 @@ public class TestLazySimpleFast extends TestCase {
               return;
             }
           }
-          TestCase.fail("Field reports null but object is not null (class " + expectedObject.getClass().getName() + ", " + expectedObject.toString() + ")");
+          fail("Field reports null but object is not null (class " + expectedObject.getClass().getName() +
+              ", " + expectedObject.toString() + ")");
         }
       }
       if (!VerifyLazy.lazyCompare(typeInfo, complexFieldObj, expectedObject)) {
-        TestCase.fail("Comparision failed typeInfo " + typeInfo.toString());
+        fail("Comparision failed typeInfo " + typeInfo.toString());
       }
     }
   }
@@ -381,18 +389,22 @@ public class TestLazySimpleFast extends TestCase {
     }
   }
 
+  @Test
   public void testLazyBinarySimplePrimitive() throws Throwable {
     testLazySimpleFast(SerdeRandomRowSource.SupportedTypes.PRIMITIVE, 0);
   }
 
+  @Test
   public void testLazyBinarySimpleComplexDepthOne() throws Throwable {
     testLazySimpleFast(SerdeRandomRowSource.SupportedTypes.ALL, 1);
   }
 
+  @Test
   public void testLazyBinarySimpleComplexDepthFour() throws Throwable {
     testLazySimpleFast(SerdeRandomRowSource.SupportedTypes.ALL, 4);
   }
 
+  @Test
   public void testLazySimpleDeserializeRowEmptyArray() throws Throwable {
     HiveConf hconf = new HiveConf();
 
@@ -417,6 +429,6 @@ public class TestLazySimpleFast extends TestCase {
     deserializeRead.set(bytes,  0, bytes.length);
     verifyRead(deserializeRead, typeInfos[0], Collections.emptyList());
     verifyRead(deserializeRead, typeInfos[1], Collections.emptyList());
-    TestCase.assertTrue(deserializeRead.isEndOfInputReached());
+    assertTrue(deserializeRead.isEndOfInputReached());
   }
-}
\ No newline at end of file
+}
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/lazy/TestLazySimpleSerDe.java b/serde/src/test/org/apache/hadoop/hive/serde2/lazy/TestLazySimpleSerDe.java
index d289205..c697dcf 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/lazy/TestLazySimpleSerDe.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/lazy/TestLazySimpleSerDe.java
@@ -22,7 +22,7 @@ import java.util.List;
 import java.util.Properties;
 import java.util.Random;
 
-import junit.framework.TestCase;
+
 
 import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.conf.Configuration;
@@ -46,16 +46,18 @@ import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
+import static org.junit.Assert.assertEquals;
 
 /**
  * TestLazySimpleSerDe.
  *
  */
-public class TestLazySimpleSerDe extends TestCase {
+public class TestLazySimpleSerDe {
 
   /**
    * Test the LazySimpleSerDe class.
    */
+  @Test
   public void testLazySimpleSerDe() throws Throwable {
     try {
       // Create the SerDe
@@ -92,6 +94,7 @@ public class TestLazySimpleSerDe extends TestCase {
   /**
    * Test the LazySimpleSerDe class with LastColumnTakesRest option.
    */
+  @Test
   public void testLazySimpleSerDeLastColumnTakesRest() throws Throwable {
     try {
       // Create the SerDe
@@ -121,6 +124,7 @@ public class TestLazySimpleSerDe extends TestCase {
   /**
    * Test the LazySimpleSerDe class with extra columns.
    */
+  @Test
   public void testLazySimpleSerDeExtraColumns() throws Throwable {
     try {
       // Create the SerDe
@@ -149,6 +153,7 @@ public class TestLazySimpleSerDe extends TestCase {
   /**
    * Test the LazySimpleSerDe class with missing columns.
    */
+  @Test
   public void testLazySimpleSerDeMissingColumns() throws Throwable {
     try {
       // Create the SerDe
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/lazy/fast/TestLazySimpleDeserializeRead.java b/serde/src/test/org/apache/hadoop/hive/serde2/lazy/fast/TestLazySimpleDeserializeRead.java
index a7873f2..196d2b6 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/lazy/fast/TestLazySimpleDeserializeRead.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/lazy/fast/TestLazySimpleDeserializeRead.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hive.serde2.lazy.fast;
 
-import junit.framework.TestCase;
+
 
 import java.util.Properties;
 
@@ -28,17 +28,21 @@ import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.io.Text;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
 /**
  * Unit tests for LazySimpleDeserializeRead.
  *
  */
-public class TestLazySimpleDeserializeRead extends TestCase {
+public class TestLazySimpleDeserializeRead {
 
   /**
    * Test for escaping.
    *
    */
+  @Test
   public void testEscaping() throws Exception {
     HiveConf hconf = new HiveConf();
 
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/lazybinary/TestLazyBinaryFast.java b/serde/src/test/org/apache/hadoop/hive/serde2/lazybinary/TestLazyBinaryFast.java
index e95c6eb..ae23809 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/lazybinary/TestLazyBinaryFast.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/lazybinary/TestLazyBinaryFast.java
@@ -22,7 +22,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Random;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.serde2.ByteStream.Output;
 import org.apache.hadoop.hive.serde2.AbstractSerDe;
@@ -39,8 +39,14 @@ import org.apache.hadoop.hive.serde2.objectinspector.UnionObject;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.Writable;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import org.junit.Test;
 
-public class TestLazyBinaryFast extends TestCase {
+/**
+ * TestLazyBinaryFast.
+ */
+public class TestLazyBinaryFast {
 
   private void testLazyBinaryFast(
       SerdeRandomRowSource source, Object[][] rows,
@@ -51,8 +57,8 @@ public class TestLazyBinaryFast extends TestCase {
 
     int rowCount = rows.length;
     int columnCount = typeInfos.length;
-
     boolean[] columnsToInclude = null;
+
     if (useIncludeColumns) {
       columnsToInclude = new boolean[columnCount];
       for (int i = 0; i < columnCount; i++) {
@@ -111,7 +117,7 @@ public class TestLazyBinaryFast extends TestCase {
         }
       }
       if (writeColumnCount == columnCount) {
-        TestCase.assertTrue(lazyBinaryDeserializeRead.isEndOfInputReached());
+        assertTrue(lazyBinaryDeserializeRead.isEndOfInputReached());
       }
     }
 
@@ -204,7 +210,7 @@ public class TestLazyBinaryFast extends TestCase {
         }
       }
       if (writeColumnCount == columnCount) {
-        TestCase.assertTrue(lazyBinaryDeserializeRead.isEndOfInputReached());
+        assertTrue(lazyBinaryDeserializeRead.isEndOfInputReached());
       }
     }
   }
@@ -217,7 +223,7 @@ public class TestLazyBinaryFast extends TestCase {
       Object complexFieldObj = VerifyFast.deserializeReadComplexType(lazyBinaryDeserializeRead, typeInfo);
       if (expectedObject == null) {
         if (complexFieldObj != null) {
-          TestCase.fail("Field reports not null but object is null (class " + complexFieldObj.getClass().getName() +
+          fail("Field reports not null but object is null (class " + complexFieldObj.getClass().getName() +
               ", " + complexFieldObj.toString() + ")");
         }
       } else {
@@ -229,12 +235,12 @@ public class TestLazyBinaryFast extends TestCase {
               return;
             }
           }
-          TestCase.fail("Field reports null but object is not null (class " + expectedObject.getClass().getName() +
+          fail("Field reports null but object is not null (class " + expectedObject.getClass().getName() +
               ", " + expectedObject.toString() + ")");
         }
       }
       if (!VerifyLazy.lazyCompare(typeInfo, complexFieldObj, expectedObject)) {
-        TestCase.fail("Comparision failed typeInfo " + typeInfo.toString());
+        fail("Comparision failed typeInfo " + typeInfo.toString());
       }
     }
   }
@@ -335,14 +341,17 @@ public class TestLazyBinaryFast extends TestCase {
     }
   }
 
+  @Test
   public void testLazyBinaryFastPrimitive() throws Throwable {
     testLazyBinaryFast(SerdeRandomRowSource.SupportedTypes.PRIMITIVE, 0);
   }
 
+  @Test
   public void testLazyBinaryFastComplexDepthOne() throws Throwable {
     testLazyBinaryFast(SerdeRandomRowSource.SupportedTypes.ALL, 1);
   }
 
+  @Test
   public void testLazyBinaryFastComplexDepthFour() throws Throwable {
     testLazyBinaryFast(SerdeRandomRowSource.SupportedTypes.ALL, 4);
   }
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/lazybinary/TestLazyBinarySerDe.java b/serde/src/test/org/apache/hadoop/hive/serde2/lazybinary/TestLazyBinarySerDe.java
index d120d27..60c95ff 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/lazybinary/TestLazyBinarySerDe.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/lazybinary/TestLazyBinarySerDe.java
@@ -27,7 +27,7 @@ import java.util.Map;
 import java.util.Properties;
 import java.util.Random;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
@@ -58,12 +58,15 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectIn
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableBinaryObjectInspector;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.io.BytesWritable;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import org.junit.Test;
 
 /**
  * TestLazyBinarySerDe.
  *
  */
-public class TestLazyBinarySerDe extends TestCase {
+public class TestLazyBinarySerDe {
 
   /**
    * Generate a random struct array.
@@ -424,6 +427,7 @@ public class TestLazyBinarySerDe extends TestCase {
    *
    * @throws Throwable
    */
+  @Test
   public void testLazyBinarySerDe() throws Throwable {
     try {
 
@@ -481,6 +485,7 @@ public class TestLazyBinarySerDe extends TestCase {
    * JavaBinaryObjectInspector from input BytesWritable
    * @throws Throwable
    */
+  @Test
   public void testJavaBinaryObjectInspector() throws Throwable {
     BytesWritable bW = getInputBytesWritable();
 
@@ -501,6 +506,7 @@ public class TestLazyBinarySerDe extends TestCase {
    * WritableBinaryObjectInspector from input BytesWritable
    * @throws Throwable
    */
+  @Test
   public void testWritableBinaryObjectInspector() throws Throwable {
     BytesWritable bW = getInputBytesWritable();
 
@@ -520,6 +526,7 @@ public class TestLazyBinarySerDe extends TestCase {
    * LazyBinaryObjectInspector from input BytesWritable
    * @throws Throwable
    */
+  @Test
   public void testLazyBinaryObjectInspector() throws Throwable {
 
     //create input ByteArrayRef
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestCrossMapEqualComparer.java b/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestCrossMapEqualComparer.java
index 1598cdc..b343406 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestCrossMapEqualComparer.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestCrossMapEqualComparer.java
@@ -32,9 +32,15 @@ import org.apache.hadoop.hive.serde2.lazy.LazySerDeParameters;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions;
 import org.apache.hadoop.io.Text;
 
-import junit.framework.TestCase;
 
-public class TestCrossMapEqualComparer extends TestCase {
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import org.junit.Test;
+
+/**
+ * CrossMapEqualComparer Test.
+ */
+public class TestCrossMapEqualComparer {
 
   public static class IntegerStringMapHolder {
     Map<Integer, String> mMap;
@@ -44,6 +50,7 @@ public class TestCrossMapEqualComparer extends TestCase {
     }
   }
 
+  @Test
   public void testSameType() {
     // empty maps
     IntegerStringMapHolder o1 = new IntegerStringMapHolder();
@@ -88,6 +95,7 @@ public class TestCrossMapEqualComparer extends TestCase {
     return serde.deserialize(t);
   }
 
+  @Test
   public void testCompatibleType() throws SerDeException, IOException {
     // empty maps
     TextStringMapHolder o1 = new TextStringMapHolder();
@@ -141,6 +149,7 @@ public class TestCrossMapEqualComparer extends TestCase {
     return serde.deserialize(t);
   }
 
+  @Test
   public void testIncompatibleType() throws SerDeException, IOException {
     // empty maps
     StringTextMapHolder o1 = new StringTextMapHolder();
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestFullMapEqualComparer.java b/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestFullMapEqualComparer.java
index 41c010b..6ccf212 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestFullMapEqualComparer.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestFullMapEqualComparer.java
@@ -20,11 +20,16 @@ package org.apache.hadoop.hive.serde2.objectinspector;
 import java.util.Map;
 import java.util.TreeMap;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions;
+import static org.junit.Assert.assertTrue;
+import org.junit.Test;
 
-public class TestFullMapEqualComparer extends TestCase {
+/**
+ * FullMapEqualComparer Test.
+ */
+public class TestFullMapEqualComparer {
 
   public static class IntegerIntegerMapHolder {
     Map<Integer, Integer> mMap;
@@ -34,6 +39,7 @@ public class TestFullMapEqualComparer extends TestCase {
     }
   }
 
+  @Test
   public void testAntiSymmetry() {
     IntegerIntegerMapHolder o1 = new IntegerIntegerMapHolder();
     IntegerIntegerMapHolder o2 = new IntegerIntegerMapHolder();
@@ -66,6 +72,7 @@ public class TestFullMapEqualComparer extends TestCase {
 
   }
 
+  @Test
   public void testTransitivity() {
     IntegerIntegerMapHolder o1 = new IntegerIntegerMapHolder();
     IntegerIntegerMapHolder o2 = new IntegerIntegerMapHolder();
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestObjectInspectorConverters.java b/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestObjectInspectorConverters.java
index 175d453..f829f94 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestObjectInspectorConverters.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestObjectInspectorConverters.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hive.serde2.objectinspector;
 import java.util.ArrayList;
 import java.util.List;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.common.type.HiveChar;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
@@ -42,177 +42,169 @@ import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
 /**
  * TestObjectInspectorConverters.
  *
  */
-public class TestObjectInspectorConverters extends TestCase {
+public class TestObjectInspectorConverters {
 
+  @Test
   public void testObjectInspectorConverters() throws Throwable {
     try {
       // Boolean
-      Converter booleanConverter = ObjectInspectorConverters.getConverter(
-          PrimitiveObjectInspectorFactory.javaIntObjectInspector,
-          PrimitiveObjectInspectorFactory.writableBooleanObjectInspector);
-      assertEquals("BooleanConverter", new BooleanWritable(false),
-          booleanConverter.convert(Integer.valueOf(0)));
-      assertEquals("BooleanConverter", new BooleanWritable(true),
-          booleanConverter.convert(Integer.valueOf(1)));
-      assertEquals("BooleanConverter", null, booleanConverter.convert(null));
+      convertBoolean();
 
       // Byte
-      Converter byteConverter = ObjectInspectorConverters.getConverter(
-          PrimitiveObjectInspectorFactory.javaIntObjectInspector,
-          PrimitiveObjectInspectorFactory.writableByteObjectInspector);
-      assertEquals("ByteConverter", new ByteWritable((byte) 0), byteConverter
-          .convert(Integer.valueOf(0)));
-      assertEquals("ByteConverter", new ByteWritable((byte) 1), byteConverter
-          .convert(Integer.valueOf(1)));
-      assertEquals("ByteConverter", null, byteConverter.convert(null));
+      convertByte();
 
       // Short
-      Converter shortConverter = ObjectInspectorConverters.getConverter(
-          PrimitiveObjectInspectorFactory.javaIntObjectInspector,
-          PrimitiveObjectInspectorFactory.writableShortObjectInspector);
-      assertEquals("ShortConverter", new ShortWritable((short) 0),
-          shortConverter.convert(Integer.valueOf(0)));
-      assertEquals("ShortConverter", new ShortWritable((short) 1),
-          shortConverter.convert(Integer.valueOf(1)));
-      assertEquals("ShortConverter", null, shortConverter.convert(null));
+      convertShort();
 
       // Int
-      Converter intConverter = ObjectInspectorConverters.getConverter(
-          PrimitiveObjectInspectorFactory.javaIntObjectInspector,
-          PrimitiveObjectInspectorFactory.writableIntObjectInspector);
-      assertEquals("IntConverter", new IntWritable(0), intConverter
-          .convert(Integer.valueOf(0)));
-      assertEquals("IntConverter", new IntWritable(1), intConverter
-          .convert(Integer.valueOf(1)));
-      assertEquals("IntConverter", null, intConverter.convert(null));
+      convertInt();
 
       // Long
-      Converter longConverter = ObjectInspectorConverters.getConverter(
-          PrimitiveObjectInspectorFactory.javaIntObjectInspector,
-          PrimitiveObjectInspectorFactory.writableLongObjectInspector);
-      assertEquals("LongConverter", new LongWritable(0), longConverter
-          .convert(Integer.valueOf(0)));
-      assertEquals("LongConverter", new LongWritable(1), longConverter
-          .convert(Integer.valueOf(1)));
-      assertEquals("LongConverter", null, longConverter.convert(null));
+      convertLong();
 
       // Float
-      Converter floatConverter = ObjectInspectorConverters.getConverter(
-          PrimitiveObjectInspectorFactory.javaIntObjectInspector,
-          PrimitiveObjectInspectorFactory.writableFloatObjectInspector);
-      assertEquals("LongConverter", new FloatWritable(0), floatConverter
-          .convert(Integer.valueOf(0)));
-      assertEquals("LongConverter", new FloatWritable(1), floatConverter
-          .convert(Integer.valueOf(1)));
-      assertEquals("LongConverter", null, floatConverter.convert(null));
+      convertFloat();
 
       // Double
-      Converter doubleConverter = ObjectInspectorConverters.getConverter(
-          PrimitiveObjectInspectorFactory.javaIntObjectInspector,
-          PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
-      assertEquals("DoubleConverter", new DoubleWritable(0), doubleConverter
-          .convert(Integer.valueOf(0)));
-      assertEquals("DoubleConverter", new DoubleWritable(1), doubleConverter
-          .convert(Integer.valueOf(1)));
-      assertEquals("DoubleConverter", null, doubleConverter.convert(null));
+      convertDouble();
 
       // Char
-      Converter charConverter = ObjectInspectorConverters.getConverter(
-        PrimitiveObjectInspectorFactory.javaBooleanObjectInspector,
-        PrimitiveObjectInspectorFactory.javaHiveCharObjectInspector);
-      assertEquals("CharConverter", new HiveChar("TRUE", -1), charConverter
-        .convert(Boolean.valueOf(true)));
-      assertEquals("CharConverter", new HiveChar("FALSE", -1), charConverter
-        .convert(Boolean.valueOf(false)));
+	  convertChar();
 
-      charConverter = ObjectInspectorConverters.getConverter(
-        PrimitiveObjectInspectorFactory.javaBooleanObjectInspector,
-        PrimitiveObjectInspectorFactory.writableHiveCharObjectInspector);
-      assertEquals("CharConverter", new HiveCharWritable(new HiveChar("TRUE", -1)), charConverter
-        .convert(Boolean.valueOf(true)));
-      assertEquals("CharConverter", new HiveCharWritable(new HiveChar("FALSE", -1)), charConverter
-        .convert(Boolean.valueOf(false)));
+      // VarChar
+	  convertVarChar();
 
-      charConverter = ObjectInspectorConverters.getConverter(
-        PrimitiveObjectInspectorFactory.javaIntObjectInspector,
-        PrimitiveObjectInspectorFactory.javaHiveCharObjectInspector);
-      assertEquals("CharConverter", new HiveChar("0", -1), charConverter
-        .convert(Integer.valueOf(0)));
-      assertEquals("CharConverter", new HiveChar("1", -1), charConverter
-        .convert(Integer.valueOf(1)));
+      // Text
+      convertText();
 
-      charConverter = ObjectInspectorConverters.getConverter(
-        PrimitiveObjectInspectorFactory.javaIntObjectInspector,
-        PrimitiveObjectInspectorFactory.writableHiveCharObjectInspector);
-      assertEquals("CharConverter", new HiveCharWritable(new HiveChar("0", -1)), charConverter
-        .convert(Integer.valueOf(0)));
-      assertEquals("CharConverter", new HiveCharWritable(new HiveChar("1", -1)), charConverter
-        .convert(Integer.valueOf(1)));
+      // Binary
+      converBinary();
 
-      charConverter = ObjectInspectorConverters.getConverter(
-        PrimitiveObjectInspectorFactory.javaStringObjectInspector,
-        PrimitiveObjectInspectorFactory.javaHiveCharObjectInspector);
-      assertEquals("CharConverter", new HiveChar("hive", -1), charConverter
-        .convert(String.valueOf("hive")));
+      // Union
+      convertUnion();
 
-      charConverter = ObjectInspectorConverters.getConverter(
-        PrimitiveObjectInspectorFactory.javaStringObjectInspector,
-        PrimitiveObjectInspectorFactory.writableHiveCharObjectInspector);
-      assertEquals("CharConverter", new HiveCharWritable(new HiveChar("hive", -1)), charConverter
-        .convert(String.valueOf("hive")));
+    } catch (Throwable e) {
+      e.printStackTrace();
+      throw e;
+    }
 
-      // VarChar
-      Converter varcharConverter = ObjectInspectorConverters.getConverter(
-        PrimitiveObjectInspectorFactory.javaBooleanObjectInspector,
-        PrimitiveObjectInspectorFactory.javaHiveVarcharObjectInspector);
-      assertEquals("VarCharConverter", new HiveVarchar("TRUE", -1), varcharConverter
-        .convert(Boolean.valueOf(true)));
-      assertEquals("VarCharConverter", new HiveVarchar("FALSE", -1), varcharConverter
-        .convert(Boolean.valueOf(false)));
+  }
 
-      varcharConverter = ObjectInspectorConverters.getConverter(
-        PrimitiveObjectInspectorFactory.javaBooleanObjectInspector,
-        PrimitiveObjectInspectorFactory.writableHiveVarcharObjectInspector);
-      assertEquals("VarCharConverter", new HiveVarcharWritable(new HiveVarchar("TRUE", -1)), varcharConverter
-        .convert(Boolean.valueOf(true)));
-      assertEquals("VarCharConverter", new HiveVarcharWritable(new HiveVarchar("FALSE", -1)), varcharConverter
-        .convert(Boolean.valueOf(false)));
+private void convertUnion() {
+	ArrayList<String> fieldNames = new ArrayList<String>();
+      fieldNames.add("firstInteger");
+      fieldNames.add("secondString");
+      fieldNames.add("thirdBoolean");
+      ArrayList<ObjectInspector> fieldObjectInspectors = new ArrayList<ObjectInspector>();
+      fieldObjectInspectors
+          .add(PrimitiveObjectInspectorFactory.javaIntObjectInspector);
+      fieldObjectInspectors
+          .add(PrimitiveObjectInspectorFactory.javaStringObjectInspector);
+      fieldObjectInspectors
+          .add(PrimitiveObjectInspectorFactory.javaBooleanObjectInspector);
 
-      varcharConverter = ObjectInspectorConverters.getConverter(
-        PrimitiveObjectInspectorFactory.javaIntObjectInspector,
-        PrimitiveObjectInspectorFactory.javaHiveVarcharObjectInspector);
-      assertEquals("VarCharConverter", new HiveVarchar("0", -1), varcharConverter
-        .convert(Integer.valueOf(0)));
-      assertEquals("VarCharConverter", new HiveVarchar("1", -1), varcharConverter
-        .convert(Integer.valueOf(1)));
+      ArrayList<String> fieldNames2 = new ArrayList<String>();
+      fieldNames2.add("firstString");
+      fieldNames2.add("secondInteger");
+      fieldNames2.add("thirdBoolean");
+      ArrayList<ObjectInspector> fieldObjectInspectors2 = new ArrayList<ObjectInspector>();
+      fieldObjectInspectors2
+          .add(PrimitiveObjectInspectorFactory.javaStringObjectInspector);
+      fieldObjectInspectors2
+          .add(PrimitiveObjectInspectorFactory.javaIntObjectInspector);
+      fieldObjectInspectors2
+          .add(PrimitiveObjectInspectorFactory.javaBooleanObjectInspector);
 
-      varcharConverter = ObjectInspectorConverters.getConverter(
-        PrimitiveObjectInspectorFactory.javaIntObjectInspector,
-        PrimitiveObjectInspectorFactory.writableHiveVarcharObjectInspector);
-      assertEquals("VarCharConverter", new HiveVarcharWritable(new HiveVarchar("0", -1)), varcharConverter
-        .convert(Integer.valueOf(0)));
-      assertEquals("VarCharConverter", new HiveVarcharWritable(new HiveVarchar("1", -1)), varcharConverter
-        .convert(Integer.valueOf(1)));
+      Converter unionConverter0 = ObjectInspectorConverters.getConverter(ObjectInspectorFactory.getStandardUnionObjectInspector(fieldObjectInspectors),
+          ObjectInspectorFactory.getStandardUnionObjectInspector(fieldObjectInspectors2));
 
-      varcharConverter = ObjectInspectorConverters.getConverter(
-        PrimitiveObjectInspectorFactory.javaStringObjectInspector,
-        PrimitiveObjectInspectorFactory.javaHiveVarcharObjectInspector);
-      assertEquals("VarCharConverter", new HiveVarchar("hive", -1), varcharConverter
-        .convert(String.valueOf("hive")));
+      Object convertedObject0 = unionConverter0.convert(new StandardUnion((byte)0, 1));
+      StandardUnion expectedObject0 = new StandardUnion();
+      expectedObject0.setTag((byte) 0);
+      expectedObject0.setObject("1");
 
-      varcharConverter = ObjectInspectorConverters.getConverter(
-        PrimitiveObjectInspectorFactory.javaStringObjectInspector,
-        PrimitiveObjectInspectorFactory.writableHiveVarcharObjectInspector);
-      assertEquals("VarCharConverter", new HiveVarcharWritable(new HiveVarchar("hive", -1)), varcharConverter
-        .convert(String.valueOf("hive")));
+      assertEquals(expectedObject0, convertedObject0);
 
-      // Text
-      Converter textConverter = ObjectInspectorConverters.getConverter(
+      Converter unionConverter1 = ObjectInspectorConverters.getConverter(ObjectInspectorFactory.getStandardUnionObjectInspector(fieldObjectInspectors),
+		  ObjectInspectorFactory.getStandardUnionObjectInspector(fieldObjectInspectors2));
+
+      Object convertedObject1 = unionConverter1.convert(new StandardUnion((byte)1, "1"));
+      StandardUnion expectedObject1 = new StandardUnion();
+      expectedObject1.setTag((byte) 1);
+      expectedObject1.setObject(1);
+
+      assertEquals(expectedObject1, convertedObject1);
+
+      Converter unionConverter2 = ObjectInspectorConverters.getConverter(ObjectInspectorFactory.getStandardUnionObjectInspector(fieldObjectInspectors),
+          ObjectInspectorFactory.getStandardUnionObjectInspector(fieldObjectInspectors2));
+
+      Object convertedObject2 = unionConverter2.convert(new StandardUnion((byte)2, true));
+      StandardUnion expectedObject2 = new StandardUnion();
+      expectedObject2.setTag((byte) 2);
+      expectedObject2.setObject(true);
+
+      assertEquals(expectedObject2, convertedObject2);
+
+      // Union (extra fields)
+      ArrayList<String> fieldNamesExtra = new ArrayList<String>();
+      fieldNamesExtra.add("firstInteger");
+      fieldNamesExtra.add("secondString");
+      fieldNamesExtra.add("thirdBoolean");
+      ArrayList<ObjectInspector> fieldObjectInspectorsExtra = new ArrayList<ObjectInspector>();
+      fieldObjectInspectorsExtra
+          .add(PrimitiveObjectInspectorFactory.javaIntObjectInspector);
+      fieldObjectInspectorsExtra
+          .add(PrimitiveObjectInspectorFactory.javaStringObjectInspector);
+      fieldObjectInspectorsExtra
+          .add(PrimitiveObjectInspectorFactory.javaBooleanObjectInspector);
+
+      ArrayList<String> fieldNamesExtra2 = new ArrayList<String>();
+      fieldNamesExtra2.add("firstString");
+      fieldNamesExtra2.add("secondInteger");
+      ArrayList<ObjectInspector> fieldObjectInspectorsExtra2 = new ArrayList<ObjectInspector>();
+      fieldObjectInspectorsExtra2
+          .add(PrimitiveObjectInspectorFactory.javaStringObjectInspector);
+      fieldObjectInspectorsExtra2
+          .add(PrimitiveObjectInspectorFactory.javaIntObjectInspector);
+
+      Converter unionConverterExtra = ObjectInspectorConverters.getConverter(ObjectInspectorFactory.getStandardUnionObjectInspector(fieldObjectInspectorsExtra),
+          ObjectInspectorFactory.getStandardUnionObjectInspector(fieldObjectInspectorsExtra2));
+
+      Object convertedObjectExtra = unionConverterExtra.convert(new StandardUnion((byte)2, true));
+      StandardUnion expectedObjectExtra = new StandardUnion();
+      expectedObjectExtra.setTag((byte) -1);
+      expectedObjectExtra.setObject(null);
+
+      assertEquals(expectedObjectExtra, convertedObjectExtra); // we should get back null
+}
+
+private void converBinary() {
+	Converter baConverter = ObjectInspectorConverters.getConverter(
+          PrimitiveObjectInspectorFactory.javaStringObjectInspector,
+          PrimitiveObjectInspectorFactory.writableBinaryObjectInspector);
+      assertEquals("BAConverter", new BytesWritable(new byte[]
+          {(byte)'h', (byte)'i',(byte)'v',(byte)'e'}),
+          baConverter.convert("hive"));
+      assertEquals("BAConverter", null, baConverter.convert(null));
+
+      baConverter = ObjectInspectorConverters.getConverter(
+          PrimitiveObjectInspectorFactory.writableStringObjectInspector,
+          PrimitiveObjectInspectorFactory.writableBinaryObjectInspector);
+      assertEquals("BAConverter", new BytesWritable(new byte[]
+          {(byte)'h', (byte)'i',(byte)'v',(byte)'e'}),
+          baConverter.convert(new Text("hive")));
+      assertEquals("BAConverter", null, baConverter.convert(null));
+}
+
+private void convertText() {
+	Converter textConverter = ObjectInspectorConverters.getConverter(
           PrimitiveObjectInspectorFactory.javaIntObjectInspector,
           PrimitiveObjectInspectorFactory.writableStringObjectInspector);
       assertEquals("TextConverter", new Text("0"), textConverter
@@ -249,6 +241,52 @@ public class TestObjectInspectorConverters extends TestCase {
       assertEquals("TextConverter", new Text("100.001000000000000000"), textConverter
 	  .convert(HiveDecimal.create("100.001")));
       assertEquals("TextConverter", null, textConverter.convert(null));
+}
+
+private void convertVarChar() {
+	Converter varcharConverter = ObjectInspectorConverters.getConverter(
+        PrimitiveObjectInspectorFactory.javaBooleanObjectInspector,
+        PrimitiveObjectInspectorFactory.javaHiveVarcharObjectInspector);
+      assertEquals("VarCharConverter", new HiveVarchar("TRUE", -1), varcharConverter
+        .convert(Boolean.valueOf(true)));
+      assertEquals("VarCharConverter", new HiveVarchar("FALSE", -1), varcharConverter
+        .convert(Boolean.valueOf(false)));
+
+      varcharConverter = ObjectInspectorConverters.getConverter(
+        PrimitiveObjectInspectorFactory.javaBooleanObjectInspector,
+        PrimitiveObjectInspectorFactory.writableHiveVarcharObjectInspector);
+      assertEquals("VarCharConverter", new HiveVarcharWritable(new HiveVarchar("TRUE", -1)), varcharConverter
+        .convert(Boolean.valueOf(true)));
+      assertEquals("VarCharConverter", new HiveVarcharWritable(new HiveVarchar("FALSE", -1)), varcharConverter
+        .convert(Boolean.valueOf(false)));
+
+      varcharConverter = ObjectInspectorConverters.getConverter(
+        PrimitiveObjectInspectorFactory.javaIntObjectInspector,
+        PrimitiveObjectInspectorFactory.javaHiveVarcharObjectInspector);
+      assertEquals("VarCharConverter", new HiveVarchar("0", -1), varcharConverter
+        .convert(Integer.valueOf(0)));
+      assertEquals("VarCharConverter", new HiveVarchar("1", -1), varcharConverter
+        .convert(Integer.valueOf(1)));
+
+      varcharConverter = ObjectInspectorConverters.getConverter(
+        PrimitiveObjectInspectorFactory.javaIntObjectInspector,
+        PrimitiveObjectInspectorFactory.writableHiveVarcharObjectInspector);
+      assertEquals("VarCharConverter", new HiveVarcharWritable(new HiveVarchar("0", -1)), varcharConverter
+        .convert(Integer.valueOf(0)));
+      assertEquals("VarCharConverter", new HiveVarcharWritable(new HiveVarchar("1", -1)), varcharConverter
+        .convert(Integer.valueOf(1)));
+
+      varcharConverter = ObjectInspectorConverters.getConverter(
+        PrimitiveObjectInspectorFactory.javaStringObjectInspector,
+        PrimitiveObjectInspectorFactory.javaHiveVarcharObjectInspector);
+      assertEquals("VarCharConverter", new HiveVarchar("hive", -1), varcharConverter
+        .convert(String.valueOf("hive")));
+
+      varcharConverter = ObjectInspectorConverters.getConverter(
+        PrimitiveObjectInspectorFactory.javaStringObjectInspector,
+        PrimitiveObjectInspectorFactory.writableHiveVarcharObjectInspector);
+      assertEquals("VarCharConverter", new HiveVarcharWritable(new HiveVarchar("hive", -1)), varcharConverter
+        .convert(String.valueOf("hive")));
 
       // Varchar
       PrimitiveTypeInfo varchar5TI =
@@ -271,6 +309,52 @@ public class TestObjectInspectorConverters extends TestCase {
           varchar30OI);
       assertEquals("VarcharConverter", "100.001000000000000000",
           varcharConverter.convert(HiveDecimal.create("100.001")).toString());
+}
+
+private void convertChar() {
+	Converter charConverter = ObjectInspectorConverters.getConverter(
+        PrimitiveObjectInspectorFactory.javaBooleanObjectInspector,
+        PrimitiveObjectInspectorFactory.javaHiveCharObjectInspector);
+      assertEquals("CharConverter", new HiveChar("TRUE", -1), charConverter
+        .convert(Boolean.valueOf(true)));
+      assertEquals("CharConverter", new HiveChar("FALSE", -1), charConverter
+        .convert(Boolean.valueOf(false)));
+
+      charConverter = ObjectInspectorConverters.getConverter(
+        PrimitiveObjectInspectorFactory.javaBooleanObjectInspector,
+        PrimitiveObjectInspectorFactory.writableHiveCharObjectInspector);
+      assertEquals("CharConverter", new HiveCharWritable(new HiveChar("TRUE", -1)), charConverter
+        .convert(Boolean.valueOf(true)));
+      assertEquals("CharConverter", new HiveCharWritable(new HiveChar("FALSE", -1)), charConverter
+        .convert(Boolean.valueOf(false)));
+
+      charConverter = ObjectInspectorConverters.getConverter(
+        PrimitiveObjectInspectorFactory.javaIntObjectInspector,
+        PrimitiveObjectInspectorFactory.javaHiveCharObjectInspector);
+      assertEquals("CharConverter", new HiveChar("0", -1), charConverter
+        .convert(Integer.valueOf(0)));
+      assertEquals("CharConverter", new HiveChar("1", -1), charConverter
+        .convert(Integer.valueOf(1)));
+
+      charConverter = ObjectInspectorConverters.getConverter(
+        PrimitiveObjectInspectorFactory.javaIntObjectInspector,
+        PrimitiveObjectInspectorFactory.writableHiveCharObjectInspector);
+      assertEquals("CharConverter", new HiveCharWritable(new HiveChar("0", -1)), charConverter
+        .convert(Integer.valueOf(0)));
+      assertEquals("CharConverter", new HiveCharWritable(new HiveChar("1", -1)), charConverter
+        .convert(Integer.valueOf(1)));
+
+      charConverter = ObjectInspectorConverters.getConverter(
+        PrimitiveObjectInspectorFactory.javaStringObjectInspector,
+        PrimitiveObjectInspectorFactory.javaHiveCharObjectInspector);
+      assertEquals("CharConverter", new HiveChar("hive", -1), charConverter
+        .convert(String.valueOf("hive")));
+
+      charConverter = ObjectInspectorConverters.getConverter(
+        PrimitiveObjectInspectorFactory.javaStringObjectInspector,
+        PrimitiveObjectInspectorFactory.writableHiveCharObjectInspector);
+      assertEquals("CharConverter", new HiveCharWritable(new HiveChar("hive", -1)), charConverter
+        .convert(String.valueOf("hive")));
 
       // Char
       PrimitiveTypeInfo char5TI =
@@ -293,118 +377,86 @@ public class TestObjectInspectorConverters extends TestCase {
           char30OI);
       assertEquals("CharConverter", "100.001000000000000000        ",
           charConverter.convert(HiveDecimal.create("100.001")).toString());
+}
 
-      // Binary
-      Converter baConverter = ObjectInspectorConverters.getConverter(
-          PrimitiveObjectInspectorFactory.javaStringObjectInspector,
-          PrimitiveObjectInspectorFactory.writableBinaryObjectInspector);
-      assertEquals("BAConverter", new BytesWritable(new byte[]
-          {(byte)'h', (byte)'i',(byte)'v',(byte)'e'}),
-          baConverter.convert("hive"));
-      assertEquals("BAConverter", null, baConverter.convert(null));
-
-      baConverter = ObjectInspectorConverters.getConverter(
-          PrimitiveObjectInspectorFactory.writableStringObjectInspector,
-          PrimitiveObjectInspectorFactory.writableBinaryObjectInspector);
-      assertEquals("BAConverter", new BytesWritable(new byte[]
-          {(byte)'h', (byte)'i',(byte)'v',(byte)'e'}),
-          baConverter.convert(new Text("hive")));
-      assertEquals("BAConverter", null, baConverter.convert(null));
-
-      // Union
-      ArrayList<String> fieldNames = new ArrayList<String>();
-      fieldNames.add("firstInteger");
-      fieldNames.add("secondString");
-      fieldNames.add("thirdBoolean");
-      ArrayList<ObjectInspector> fieldObjectInspectors = new ArrayList<ObjectInspector>();
-      fieldObjectInspectors
-          .add(PrimitiveObjectInspectorFactory.javaIntObjectInspector);
-      fieldObjectInspectors
-          .add(PrimitiveObjectInspectorFactory.javaStringObjectInspector);
-      fieldObjectInspectors
-          .add(PrimitiveObjectInspectorFactory.javaBooleanObjectInspector);
-
-      ArrayList<String> fieldNames2 = new ArrayList<String>();
-      fieldNames2.add("firstString");
-      fieldNames2.add("secondInteger");
-      fieldNames2.add("thirdBoolean");
-      ArrayList<ObjectInspector> fieldObjectInspectors2 = new ArrayList<ObjectInspector>();
-      fieldObjectInspectors2
-          .add(PrimitiveObjectInspectorFactory.javaStringObjectInspector);
-      fieldObjectInspectors2
-          .add(PrimitiveObjectInspectorFactory.javaIntObjectInspector);
-      fieldObjectInspectors2
-          .add(PrimitiveObjectInspectorFactory.javaBooleanObjectInspector);
-
-      Converter unionConverter0 = ObjectInspectorConverters.getConverter(ObjectInspectorFactory.getStandardUnionObjectInspector(fieldObjectInspectors),
-          ObjectInspectorFactory.getStandardUnionObjectInspector(fieldObjectInspectors2));
-
-      Object convertedObject0 = unionConverter0.convert(new StandardUnion((byte)0, 1));
-      StandardUnion expectedObject0 = new StandardUnion();
-      expectedObject0.setTag((byte) 0);
-      expectedObject0.setObject("1");
-
-      assertEquals(expectedObject0, convertedObject0);
-
-      Converter unionConverter1 = ObjectInspectorConverters.getConverter(ObjectInspectorFactory.getStandardUnionObjectInspector(fieldObjectInspectors),
-		  ObjectInspectorFactory.getStandardUnionObjectInspector(fieldObjectInspectors2));
-
-      Object convertedObject1 = unionConverter1.convert(new StandardUnion((byte)1, "1"));
-      StandardUnion expectedObject1 = new StandardUnion();
-      expectedObject1.setTag((byte) 1);
-      expectedObject1.setObject(1);
-
-      assertEquals(expectedObject1, convertedObject1);
-
-      Converter unionConverter2 = ObjectInspectorConverters.getConverter(ObjectInspectorFactory.getStandardUnionObjectInspector(fieldObjectInspectors),
-          ObjectInspectorFactory.getStandardUnionObjectInspector(fieldObjectInspectors2));
-
-      Object convertedObject2 = unionConverter2.convert(new StandardUnion((byte)2, true));
-      StandardUnion expectedObject2 = new StandardUnion();
-      expectedObject2.setTag((byte) 2);
-      expectedObject2.setObject(true);
-
-      assertEquals(expectedObject2, convertedObject2);
-
-      // Union (extra fields)
-      ArrayList<String> fieldNamesExtra = new ArrayList<String>();
-      fieldNamesExtra.add("firstInteger");
-      fieldNamesExtra.add("secondString");
-      fieldNamesExtra.add("thirdBoolean");
-      ArrayList<ObjectInspector> fieldObjectInspectorsExtra = new ArrayList<ObjectInspector>();
-      fieldObjectInspectorsExtra
-          .add(PrimitiveObjectInspectorFactory.javaIntObjectInspector);
-      fieldObjectInspectorsExtra
-          .add(PrimitiveObjectInspectorFactory.javaStringObjectInspector);
-      fieldObjectInspectorsExtra
-          .add(PrimitiveObjectInspectorFactory.javaBooleanObjectInspector);
+private void convertDouble() {
+	Converter doubleConverter = ObjectInspectorConverters.getConverter(
+          PrimitiveObjectInspectorFactory.javaIntObjectInspector,
+          PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
+      assertEquals("DoubleConverter", new DoubleWritable(0), doubleConverter
+          .convert(Integer.valueOf(0)));
+      assertEquals("DoubleConverter", new DoubleWritable(1), doubleConverter
+          .convert(Integer.valueOf(1)));
+      assertEquals("DoubleConverter", null, doubleConverter.convert(null));
+}
 
-      ArrayList<String> fieldNamesExtra2 = new ArrayList<String>();
-      fieldNamesExtra2.add("firstString");
-      fieldNamesExtra2.add("secondInteger");
-      ArrayList<ObjectInspector> fieldObjectInspectorsExtra2 = new ArrayList<ObjectInspector>();
-      fieldObjectInspectorsExtra2
-          .add(PrimitiveObjectInspectorFactory.javaStringObjectInspector);
-      fieldObjectInspectorsExtra2
-          .add(PrimitiveObjectInspectorFactory.javaIntObjectInspector);
+private void convertFloat() {
+	Converter floatConverter = ObjectInspectorConverters.getConverter(
+          PrimitiveObjectInspectorFactory.javaIntObjectInspector,
+          PrimitiveObjectInspectorFactory.writableFloatObjectInspector);
+      assertEquals("LongConverter", new FloatWritable(0), floatConverter
+          .convert(Integer.valueOf(0)));
+      assertEquals("LongConverter", new FloatWritable(1), floatConverter
+          .convert(Integer.valueOf(1)));
+      assertEquals("LongConverter", null, floatConverter.convert(null));
+}
 
-      Converter unionConverterExtra = ObjectInspectorConverters.getConverter(ObjectInspectorFactory.getStandardUnionObjectInspector(fieldObjectInspectorsExtra),
-          ObjectInspectorFactory.getStandardUnionObjectInspector(fieldObjectInspectorsExtra2));
+private void convertLong() {
+	Converter longConverter = ObjectInspectorConverters.getConverter(
+          PrimitiveObjectInspectorFactory.javaIntObjectInspector,
+          PrimitiveObjectInspectorFactory.writableLongObjectInspector);
+      assertEquals("LongConverter", new LongWritable(0), longConverter
+          .convert(Integer.valueOf(0)));
+      assertEquals("LongConverter", new LongWritable(1), longConverter
+          .convert(Integer.valueOf(1)));
+      assertEquals("LongConverter", null, longConverter.convert(null));
+}
 
-      Object convertedObjectExtra = unionConverterExtra.convert(new StandardUnion((byte)2, true));
-      StandardUnion expectedObjectExtra = new StandardUnion();
-      expectedObjectExtra.setTag((byte) -1);
-      expectedObjectExtra.setObject(null);
+private void convertInt() {
+	Converter intConverter = ObjectInspectorConverters.getConverter(
+          PrimitiveObjectInspectorFactory.javaIntObjectInspector,
+          PrimitiveObjectInspectorFactory.writableIntObjectInspector);
+      assertEquals("IntConverter", new IntWritable(0), intConverter
+          .convert(Integer.valueOf(0)));
+      assertEquals("IntConverter", new IntWritable(1), intConverter
+          .convert(Integer.valueOf(1)));
+      assertEquals("IntConverter", null, intConverter.convert(null));
+}
 
-      assertEquals(expectedObjectExtra, convertedObjectExtra); // we should get back null
+private void convertShort() {
+	Converter shortConverter = ObjectInspectorConverters.getConverter(
+          PrimitiveObjectInspectorFactory.javaIntObjectInspector,
+          PrimitiveObjectInspectorFactory.writableShortObjectInspector);
+      assertEquals("ShortConverter", new ShortWritable((short) 0),
+          shortConverter.convert(Integer.valueOf(0)));
+      assertEquals("ShortConverter", new ShortWritable((short) 1),
+          shortConverter.convert(Integer.valueOf(1)));
+      assertEquals("ShortConverter", null, shortConverter.convert(null));
+}
 
-    } catch (Throwable e) {
-      e.printStackTrace();
-      throw e;
-    }
+private void convertByte() {
+	Converter byteConverter = ObjectInspectorConverters.getConverter(
+          PrimitiveObjectInspectorFactory.javaIntObjectInspector,
+          PrimitiveObjectInspectorFactory.writableByteObjectInspector);
+      assertEquals("ByteConverter", new ByteWritable((byte) 0), byteConverter
+          .convert(Integer.valueOf(0)));
+      assertEquals("ByteConverter", new ByteWritable((byte) 1), byteConverter
+          .convert(Integer.valueOf(1)));
+      assertEquals("ByteConverter", null, byteConverter.convert(null));
+}
 
-  }
+private void convertBoolean() {
+	Converter booleanConverter = ObjectInspectorConverters.getConverter(
+          PrimitiveObjectInspectorFactory.javaIntObjectInspector,
+          PrimitiveObjectInspectorFactory.writableBooleanObjectInspector);
+      assertEquals("BooleanConverter", new BooleanWritable(false),
+          booleanConverter.convert(Integer.valueOf(0)));
+      assertEquals("BooleanConverter", new BooleanWritable(true),
+          booleanConverter.convert(Integer.valueOf(1)));
+      assertEquals("BooleanConverter", null, booleanConverter.convert(null));
+}
 
+  @Test
   public void testGetConvertedOI() throws Throwable {
     // Try with types that have type params
     PrimitiveTypeInfo varchar5TI =
@@ -422,4 +474,4 @@ public class TestObjectInspectorConverters extends TestCase {
     VarcharTypeInfo vcParams = (VarcharTypeInfo) poi.getTypeInfo();
     assertEquals("varchar length doesn't match", 5, vcParams.getLength());
   }
-}
\ No newline at end of file
+}
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestObjectInspectorUtils.java b/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestObjectInspectorUtils.java
index 5f90e22..b307b4d 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestObjectInspectorUtils.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestObjectInspectorUtils.java
@@ -21,7 +21,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.common.type.Date;
 import org.apache.hadoop.hive.common.type.Timestamp;
@@ -29,13 +29,17 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.hive.serde2.thrift.test.Complex;
 import org.apache.hadoop.hive.serde2.thrift.test.IntString;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import org.junit.Test;
 
 /**
  * TestObjectInspectorUtils.
  *
  */
-public class TestObjectInspectorUtils extends TestCase {
+public class TestObjectInspectorUtils {
 
+  @Test
   public void testCompareFloatingNumberSignedZero() {
     PrimitiveObjectInspector doubleOI = PrimitiveObjectInspectorFactory
         .getPrimitiveJavaObjectInspector(PrimitiveObjectInspector.PrimitiveCategory.DOUBLE);
@@ -58,6 +62,7 @@ public class TestObjectInspectorUtils extends TestCase {
     assertEquals(0, ObjectInspectorUtils.compare(f2, floatOI, f2, floatOI));
   }
 
+  @Test
   public void testObjectInspectorUtils() throws Throwable {
     try {
       ObjectInspector oi1 = ObjectInspectorFactory
@@ -133,7 +138,7 @@ public class TestObjectInspectorUtils extends TestCase {
     }
 
   }
-
+  @Test
   public void testBucketIdGeneration() {
     ArrayList<String> fieldNames = new ArrayList<String>();
     fieldNames.add("firstInteger");
@@ -160,6 +165,7 @@ public class TestObjectInspectorUtils extends TestCase {
     assertEquals("", bucketId, ObjectInspectorUtils.getBucketNumber(hashCode, 16));
   }
 
+  @Test
   public void testBucketHashGeneration() {
     ArrayList<String> fieldNames = new ArrayList<String>();
     fieldNames.add("d");
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestProtocolBuffersObjectInspectors.java b/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestProtocolBuffersObjectInspectors.java
index b4efdf8..3742065 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestProtocolBuffersObjectInspectors.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestProtocolBuffersObjectInspectors.java
@@ -21,19 +21,23 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.hive.serde2.proto.test.Complexpb.Complex;
 import org.apache.hadoop.hive.serde2.proto.test.Complexpb.IntString;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import org.junit.Test;
 
 /**
  * TestProtocolBuffersObjectInspectors.
  *
  */
-public class TestProtocolBuffersObjectInspectors extends TestCase {
+public class TestProtocolBuffersObjectInspectors {
 
+  @Test
   public void testProtocolBuffersObjectInspectors() throws Throwable {
 
     try {
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestReflectionObjectInspectors.java b/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestReflectionObjectInspectors.java
index 2faf340..251b31a 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestReflectionObjectInspectors.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestReflectionObjectInspectors.java
@@ -35,17 +35,20 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaConstantStrin
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaStringObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.hive.serde2.thrift.test.Complex;
-import org.junit.Test;
-
 import com.google.common.collect.Lists;
 
-import junit.framework.TestCase;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import org.junit.Test;
 
 /**
  * TestReflectionObjectInspectors.
  *
  */
-public class TestReflectionObjectInspectors extends TestCase {
+public class TestReflectionObjectInspectors {
 
   @Test
   public void testReflectionObjectInspectors() throws Throwable {
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestSimpleMapEqualComparer.java b/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestSimpleMapEqualComparer.java
index de5ae68..8048f37 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestSimpleMapEqualComparer.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestSimpleMapEqualComparer.java
@@ -32,9 +32,15 @@ import org.apache.hadoop.hive.serde2.lazy.LazySerDeParameters;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions;
 import org.apache.hadoop.io.Text;
 
-import junit.framework.TestCase;
 
-public class TestSimpleMapEqualComparer extends TestCase {
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import org.junit.Test;
+
+/**
+ * SimpleMapEqualComparer Test.
+ */
+public class TestSimpleMapEqualComparer {
 
   public static class IntegerStringMapHolder {
     Map<Integer, String> mMap;
@@ -44,6 +50,7 @@ public class TestSimpleMapEqualComparer extends TestCase {
     }
   }
 
+  @Test
   public void testSameType() {
     // empty maps
     IntegerStringMapHolder o1 = new IntegerStringMapHolder();
@@ -88,6 +95,7 @@ public class TestSimpleMapEqualComparer extends TestCase {
     return serde.deserialize(t);
   }
 
+  @Test
   public void testCompatibleType() throws SerDeException, IOException {
     // empty maps
     TextStringMapHolder o1 = new TextStringMapHolder();
@@ -141,6 +149,7 @@ public class TestSimpleMapEqualComparer extends TestCase {
     return serde.deserialize(t);
   }
 
+  @Test
   public void testIncompatibleType() throws SerDeException, IOException {
     // empty maps
     StringTextMapHolder o1 = new StringTextMapHolder();
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestStandardObjectInspectors.java b/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestStandardObjectInspectors.java
index 2c488b0..04ffbb8 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestStandardObjectInspectors.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestStandardObjectInspectors.java
@@ -21,7 +21,7 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.common.type.HiveChar;
 import org.apache.hadoop.hive.common.type.HiveVarchar;
@@ -47,12 +47,18 @@ import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Text;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import org.junit.Test;
 
 /**
  * TestStandardObjectInspectors.
  *
  */
-public class TestStandardObjectInspectors extends TestCase {
+public class TestStandardObjectInspectors {
 
   void doTestStandardPrimitiveObjectInspector(Class<?> writableClass,
       Class<?> javaClass) throws Throwable {
@@ -82,6 +88,7 @@ public class TestStandardObjectInspectors extends TestCase {
     }
   }
 
+  @Test
   public void testStandardPrimitiveObjectInspector() throws Throwable {
     try {
       doTestStandardPrimitiveObjectInspector(NullWritable.class, Void.class);
@@ -136,6 +143,7 @@ public class TestStandardObjectInspectors extends TestCase {
     }
   }
 
+  @Test
   public void testJavaPrimitiveObjectInspector() throws Throwable {
     try {
       doTestJavaPrimitiveObjectInspector(NullWritable.class, Void.class, null);
@@ -163,6 +171,7 @@ public class TestStandardObjectInspectors extends TestCase {
     }
   }
 
+  @Test
   public void testStandardListObjectInspector() throws Throwable {
     try {
       StandardListObjectInspector loi1 = ObjectInspectorFactory
@@ -218,6 +227,7 @@ public class TestStandardObjectInspectors extends TestCase {
 
   }
 
+  @Test
   public void testStandardMapObjectInspector() throws Throwable {
     try {
       StandardMapObjectInspector moi1 = ObjectInspectorFactory
@@ -279,6 +289,7 @@ public class TestStandardObjectInspectors extends TestCase {
   }
 
   @SuppressWarnings("unchecked")
+  @Test
   public void testStandardStructObjectInspector() throws Throwable {
     try {
       // Test StandardObjectInspector both with field comments and without
@@ -383,6 +394,7 @@ public class TestStandardObjectInspectors extends TestCase {
   }
 
   @SuppressWarnings("unchecked")
+  @Test
   public void testStandardUnionObjectInspector() throws Throwable {
     try {
       ArrayList<ObjectInspector> objectInspectors = new ArrayList<ObjectInspector>();
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestThriftObjectInspectors.java b/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestThriftObjectInspectors.java
index de291fa..9fefd66 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestThriftObjectInspectors.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestThriftObjectInspectors.java
@@ -30,14 +30,19 @@ import org.apache.hadoop.hive.serde2.thrift.test.IntString;
 import org.apache.hadoop.hive.serde2.thrift.test.PropValueUnion;
 import org.apache.hadoop.hive.serde2.thrift.test.SetIntString;
 
-import junit.framework.TestCase;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertNotNull;
+import org.junit.Test;
 
 /**
  * TestThriftObjectInspectors.
  *
  */
-public class TestThriftObjectInspectors extends TestCase {
+public class TestThriftObjectInspectors {
 
+  @Test
   public void testThriftObjectInspectors() throws Throwable {
 
     try {
@@ -129,6 +134,7 @@ public class TestThriftObjectInspectors extends TestCase {
   }
 
   @SuppressWarnings("unchecked")
+  @Test
   public void testThriftSetObjectInspector() throws Throwable {
 
     try {
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestUnionStructObjectInspector.java b/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestUnionStructObjectInspector.java
index 6e0cf47..a9d6913 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestUnionStructObjectInspector.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestUnionStructObjectInspector.java
@@ -21,17 +21,21 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import org.junit.Test;
 
 /**
  * TestUnionStructObjectInspector.
  *
  */
-public class TestUnionStructObjectInspector extends TestCase {
+public class TestUnionStructObjectInspector {
 
+  @Test
   public void testUnionStructObjectInspector() throws Throwable {
     try {
       ArrayList<String> fieldNames1 = new ArrayList<String>();
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/primitive/TestPrimitiveObjectInspectorFactory.java b/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/primitive/TestPrimitiveObjectInspectorFactory.java
index 022b642..1b9237b 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/primitive/TestPrimitiveObjectInspectorFactory.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/primitive/TestPrimitiveObjectInspectorFactory.java
@@ -18,13 +18,19 @@
 
 package org.apache.hadoop.hive.serde2.objectinspector.primitive;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
-public class TestPrimitiveObjectInspectorFactory extends TestCase {
+/**
+ * PrimitiveObjectInspectorFactory Test.
+ */
+public class TestPrimitiveObjectInspectorFactory {
 
+  @Test
   public void testGetPrimitiveWritableObjectInspector() {
     // even without type params, return a default OI for varchar
     PrimitiveObjectInspector poi = PrimitiveObjectInspectorFactory
@@ -37,6 +43,7 @@ public class TestPrimitiveObjectInspectorFactory extends TestCase {
     assertEquals(poi, PrimitiveObjectInspectorFactory.writableHiveCharObjectInspector);
   }
 
+  @Test
   public void testGetPrimitiveJavaObjectInspector() {
     // even without type params, return a default OI for varchar
     PrimitiveObjectInspector poi = PrimitiveObjectInspectorFactory
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/primitive/TestPrimitiveObjectInspectorUtils.java b/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/primitive/TestPrimitiveObjectInspectorUtils.java
index c731a57..25c3f60 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/primitive/TestPrimitiveObjectInspectorUtils.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/primitive/TestPrimitiveObjectInspectorUtils.java
@@ -31,11 +31,16 @@ import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils.PrimitiveGrouping;
 import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
-import org.junit.Test;
 
-import junit.framework.TestCase;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import org.junit.Test;
 
-public class TestPrimitiveObjectInspectorUtils extends TestCase {
+/**
+ * PrimitiveObjectInspectorUtils Test.
+ */
+public class TestPrimitiveObjectInspectorUtils {
 
   @Test
   public void testGetPrimitiveGrouping() {
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/teradata/TestTeradataBinarySerdeForDate.java b/serde/src/test/org/apache/hadoop/hive/serde2/teradata/TestTeradataBinarySerdeForDate.java
index af81fe3..6d5a78b 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/teradata/TestTeradataBinarySerdeForDate.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/teradata/TestTeradataBinarySerdeForDate.java
@@ -19,7 +19,7 @@
 package org.apache.hadoop.hive.serde2.teradata;
 
 import com.google.common.io.BaseEncoding;
-import junit.framework.TestCase;
+
 import org.apache.hadoop.hive.common.type.Date;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.io.DateWritableV2;
@@ -29,21 +29,25 @@ import org.junit.Assert;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Properties;
+import org.junit.Before;
+import org.junit.Test;
 
 /**
  * Test the data type DATE for Teradata binary format.
  */
-public class TestTeradataBinarySerdeForDate extends TestCase {
+public class TestTeradataBinarySerdeForDate {
 
   private final TeradataBinarySerde serde = new TeradataBinarySerde();
   private final Properties props = new Properties();
 
-  protected void setUp() throws Exception {
+  @Before
+  public void setUp() throws Exception {
     props.setProperty(serdeConstants.LIST_COLUMNS, "TD_DATE");
     props.setProperty(serdeConstants.LIST_COLUMN_TYPES, "date");
     serde.initialize(null, props);
   }
 
+  @Test
   public void testTimestampBefore1900() throws Exception {
 
     //0060-01-01
@@ -59,6 +63,7 @@ public class TestTeradataBinarySerdeForDate extends TestCase {
     Assert.assertTrue(Arrays.equals(in.copyBytes(), res.copyBytes()));
   }
 
+  @Test
   public void testTimestampAfter1900() throws Exception {
 
     //9999-01-01
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/teradata/TestTeradataBinarySerdeForDecimal.java b/serde/src/test/org/apache/hadoop/hive/serde2/teradata/TestTeradataBinarySerdeForDecimal.java
index 6abdd3f..28e717e 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/teradata/TestTeradataBinarySerdeForDecimal.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/teradata/TestTeradataBinarySerdeForDecimal.java
@@ -19,7 +19,7 @@
 package org.apache.hadoop.hive.serde2.teradata;
 
 import com.google.common.io.BaseEncoding;
-import junit.framework.TestCase;
+
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.io.BytesWritable;
@@ -28,22 +28,26 @@ import org.junit.Assert;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Properties;
+import org.junit.Before;
+import org.junit.Test;
 
 /**
  * Test the data type DECIMAL for Teradata binary format.
  */
-public class TestTeradataBinarySerdeForDecimal extends TestCase {
+public class TestTeradataBinarySerdeForDecimal {
 
   private final TeradataBinarySerde serde = new TeradataBinarySerde();
   private final Properties props = new Properties();
 
-  protected void setUp() throws Exception {
+  @Before
+  public void setUp() throws Exception {
     props.setProperty(serdeConstants.LIST_COLUMNS, "TD_DECIMAL");
     props.setProperty(serdeConstants.LIST_COLUMN_TYPES, "decimal(9,5)");
 
     serde.initialize(null, props);
   }
 
+  @Test
   public void testPositiveFraction() throws Exception {
     BytesWritable in = new BytesWritable(BaseEncoding.base16().lowerCase().decode("0064000000"));
 
@@ -54,6 +58,7 @@ public class TestTeradataBinarySerdeForDecimal extends TestCase {
     Assert.assertTrue(Arrays.equals(in.copyBytes(), res.copyBytes()));
   }
 
+  @Test
   public void testNegativeFraction() throws Exception {
     BytesWritable in = new BytesWritable(BaseEncoding.base16().lowerCase().decode("009cffffff"));
 
@@ -64,6 +69,7 @@ public class TestTeradataBinarySerdeForDecimal extends TestCase {
     Assert.assertTrue(Arrays.equals(in.copyBytes(), res.copyBytes()));
   }
 
+  @Test
   public void testPositiveNumber1() throws Exception {
     BytesWritable in = new BytesWritable(BaseEncoding.base16().lowerCase().decode("00a0860100"));
 
@@ -74,6 +80,7 @@ public class TestTeradataBinarySerdeForDecimal extends TestCase {
     Assert.assertTrue(Arrays.equals(in.copyBytes(), res.copyBytes()));
   }
 
+  @Test
   public void testNegativeNumber1() throws Exception {
     BytesWritable in = new BytesWritable(BaseEncoding.base16().lowerCase().decode("006079feff"));
 
@@ -84,6 +91,7 @@ public class TestTeradataBinarySerdeForDecimal extends TestCase {
     Assert.assertTrue(Arrays.equals(in.copyBytes(), res.copyBytes()));
   }
 
+  @Test
   public void testPositiveNumber2() throws Exception {
     BytesWritable in = new BytesWritable(BaseEncoding.base16().lowerCase().decode("0080969800"));
 
@@ -94,6 +102,7 @@ public class TestTeradataBinarySerdeForDecimal extends TestCase {
     Assert.assertTrue(Arrays.equals(in.copyBytes(), res.copyBytes()));
   }
 
+  @Test
   public void testNegativeNumber2() throws Exception {
     BytesWritable in = new BytesWritable(BaseEncoding.base16().lowerCase().decode("000065c4e0"));
 
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/teradata/TestTeradataBinarySerdeForTimeStamp.java b/serde/src/test/org/apache/hadoop/hive/serde2/teradata/TestTeradataBinarySerdeForTimeStamp.java
index a6cf2c1..fa5d1ca 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/teradata/TestTeradataBinarySerdeForTimeStamp.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/teradata/TestTeradataBinarySerdeForTimeStamp.java
@@ -19,7 +19,7 @@
 package org.apache.hadoop.hive.serde2.teradata;
 
 import com.google.common.io.BaseEncoding;
-import junit.framework.TestCase;
+
 import org.apache.hadoop.hive.common.type.Timestamp;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
@@ -29,20 +29,24 @@ import org.junit.Assert;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Properties;
+import org.junit.Before;
+import org.junit.Test;
 
 /**
  * Test the data type TIMESTAMP for Teradata binary format.
  */
-public class TestTeradataBinarySerdeForTimeStamp extends TestCase {
+public class TestTeradataBinarySerdeForTimeStamp {
 
   private final TeradataBinarySerde serde = new TeradataBinarySerde();
   private final Properties props = new Properties();
 
-  protected void setUp() throws Exception {
+  @Before
+  public void setUp() throws Exception {
     props.setProperty(serdeConstants.LIST_COLUMNS, "TD_TIMESTAMP");
     props.setProperty(serdeConstants.LIST_COLUMN_TYPES, "timestamp");
   }
 
+  @Test
   public void testTimestampPrecision6() throws Exception {
     props.setProperty(TeradataBinarySerde.TD_TIMESTAMP_PRECISION, "6");
     serde.initialize(null, props);
@@ -65,6 +69,7 @@ public class TestTeradataBinarySerdeForTimeStamp extends TestCase {
     Assert.assertTrue(Arrays.equals(in.copyBytes(), res.copyBytes()));
   }
 
+  @Test
   public void testTimestampPrecision0() throws Exception {
     props.setProperty(TeradataBinarySerde.TD_TIMESTAMP_PRECISION, "0");
     serde.initialize(null, props);
@@ -87,6 +92,7 @@ public class TestTeradataBinarySerdeForTimeStamp extends TestCase {
     Assert.assertTrue(Arrays.equals(in.copyBytes(), res.copyBytes()));
   }
 
+  @Test
   public void testTimestampPrecision3() throws Exception {
     props.setProperty(TeradataBinarySerde.TD_TIMESTAMP_PRECISION, "3");
     serde.initialize(null, props);
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/teradata/TestTeradataBinarySerdeGeneral.java b/serde/src/test/org/apache/hadoop/hive/serde2/teradata/TestTeradataBinarySerdeGeneral.java
index c50ef70..e392a98 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/teradata/TestTeradataBinarySerdeGeneral.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/teradata/TestTeradataBinarySerdeGeneral.java
@@ -19,7 +19,7 @@
 package org.apache.hadoop.hive.serde2.teradata;
 
 import com.google.common.io.BaseEncoding;
-import junit.framework.TestCase;
+
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.io.ByteWritable;
 import org.apache.hadoop.hive.serde2.io.DateWritableV2;
@@ -37,16 +37,19 @@ import org.junit.Assert;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Properties;
+import org.junit.Before;
+import org.junit.Test;
 
 /**
  * Test all the data types supported for Teradata Binary Format.
  */
-public class TestTeradataBinarySerdeGeneral extends TestCase {
+public class TestTeradataBinarySerdeGeneral {
 
   private final TeradataBinarySerde serde = new TeradataBinarySerde();
   private final Properties props = new Properties();
 
-  protected void setUp() throws Exception {
+  @Before
+  public void setUp() throws Exception {
     props.setProperty(serdeConstants.LIST_COLUMNS,
         "TD_CHAR, TD_VARCHAR, TD_BIGINT, TD_INT, TD_SMALLINT, TD_BYTEINT, "
             + "TD_FLOAT,TD_DECIMAL,TD_DATE, TD_TIMESTAMP, TD_VARBYTE");
@@ -56,6 +59,7 @@ public class TestTeradataBinarySerdeGeneral extends TestCase {
     serde.initialize(null, props);
   }
 
+  @Test
   public void testDeserializeAndSerialize() throws Exception {
     BytesWritable in = new BytesWritable(BaseEncoding.base16().lowerCase().decode(
         "00004e6f762020202020201b006120646179203d2031312f31312f31312020202020202020203435ec10000000000000c5feffff"
@@ -81,6 +85,7 @@ public class TestTeradataBinarySerdeGeneral extends TestCase {
     Assert.assertTrue(Arrays.equals(in.copyBytes(), res.copyBytes()));
   }
 
+  @Test
   public void testDeserializeAndSerializeWithNull() throws Exception {
     //null bitmap: 0160 -> 00000001 01100000, 7th, 9th, 10th is null
     BytesWritable in = new BytesWritable(BaseEncoding.base16().lowerCase().decode(
@@ -98,6 +103,7 @@ public class TestTeradataBinarySerdeGeneral extends TestCase {
     Assert.assertTrue(Arrays.equals(in.copyBytes(), res.copyBytes()));
   }
 
+  @Test
   public void testDeserializeAndSerializeAllNull() throws Exception {
     BytesWritable in = new BytesWritable(BaseEncoding.base16().lowerCase().decode(
         "ffe0202020202020202020000000000000000000000000000000000000000000000000000000000000000000000000000000000"
@@ -119,6 +125,7 @@ public class TestTeradataBinarySerdeGeneral extends TestCase {
     Assert.assertTrue(Arrays.equals(in.copyBytes(), res.copyBytes()));
   }
 
+  @Test
   public void testDeserializeCorruptedRecord() throws Exception {
     BytesWritable in = new BytesWritable(BaseEncoding.base16().lowerCase().decode(
         "00004e6f762020202020201b006120646179203d2031312f31312f31312020202020202020203435ec10000000000000c5feff"
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/typeinfo/TestTypeInfoUtils.java b/serde/src/test/org/apache/hadoop/hive/serde2/typeinfo/TestTypeInfoUtils.java
index 5f8ff7b..db25db2 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/typeinfo/TestTypeInfoUtils.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/typeinfo/TestTypeInfoUtils.java
@@ -18,12 +18,17 @@
 
 package org.apache.hadoop.hive.serde2.typeinfo;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
-public class TestTypeInfoUtils extends TestCase {
+/**
+ * TypeInfoUtils Test.
+ */
+public class TestTypeInfoUtils {
 
   static void parseTypeString(String typeString, boolean exceptionExpected) {
     boolean caughtException = false;
@@ -35,6 +40,7 @@ public class TestTypeInfoUtils extends TestCase {
     assertEquals("parsing typestring " + typeString, exceptionExpected, caughtException);
   }
 
+  @Test
   public void testTypeInfoParser() {
     String[] validTypeStrings = {
         "int",
@@ -65,6 +71,7 @@ public class TestTypeInfoUtils extends TestCase {
     }
   }
 
+  @Test
   public void testQualifiedTypeNoParams() {
     boolean caughtException = false;
     try {
@@ -94,6 +101,7 @@ public class TestTypeInfoUtils extends TestCase {
     }
   }
 
+  @Test
   public void testDecimal() {
     DecimalTestCase[] testCases = {
         new DecimalTestCase("decimal", 10, 0),
diff --git a/service/src/test/org/apache/hive/http/TestJdbcJarDownloadServlet.java b/service/src/test/org/apache/hive/http/TestJdbcJarDownloadServlet.java
index f7ad1fd..5217610 100644
--- a/service/src/test/org/apache/hive/http/TestJdbcJarDownloadServlet.java
+++ b/service/src/test/org/apache/hive/http/TestJdbcJarDownloadServlet.java
@@ -23,14 +23,18 @@ import java.io.IOException;
 
 import javax.servlet.http.HttpServletResponse;
 
-import org.junit.Test;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 
-import junit.framework.TestCase;
 
-public class TestJdbcJarDownloadServlet extends TestCase {
+import org.junit.Test;
+
+/**
+ * TestJdbcJarDownloadServlet.
+ *
+ */
+public class TestJdbcJarDownloadServlet {
   
   @Test
   public void testNoFileFound() throws IOException {
@@ -43,4 +47,4 @@ public class TestJdbcJarDownloadServlet extends TestCase {
     verify(mockResponse, times(1)).setContentType("application/java-archive");
     verify(mockResponse, times(1)).sendError(HttpServletResponse.SC_NOT_FOUND);
   }
-}
\ No newline at end of file
+}
diff --git a/service/src/test/org/apache/hive/service/TestCookieSigner.java b/service/src/test/org/apache/hive/service/TestCookieSigner.java
index b1aa0d8..aec6d47 100644
--- a/service/src/test/org/apache/hive/service/TestCookieSigner.java
+++ b/service/src/test/org/apache/hive/service/TestCookieSigner.java
@@ -20,7 +20,7 @@ package org.apache.hive.service;
 
 import java.util.Random;
 
-import junit.framework.TestCase;
+
 
 import org.junit.After;
 import org.junit.Before;
@@ -30,7 +30,7 @@ import org.junit.Test;
  * CLIServiceTest.
  *
  */
-public class TestCookieSigner extends TestCase {
+public class TestCookieSigner {
 
   protected static CookieSigner cs;
   private static final Random RAN = new Random();
diff --git a/service/src/test/org/apache/hive/service/auth/TestPlainSaslHelper.java b/service/src/test/org/apache/hive/service/auth/TestPlainSaslHelper.java
index 8bfa7dc..e4b0ba0 100644
--- a/service/src/test/org/apache/hive/service/auth/TestPlainSaslHelper.java
+++ b/service/src/test/org/apache/hive/service/auth/TestPlainSaslHelper.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hive.service.auth;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
@@ -25,13 +25,20 @@ import org.apache.hive.service.cli.CLIService;
 import org.apache.hive.service.cli.thrift.ThriftCLIService;
 import org.apache.hive.service.cli.thrift.ThriftBinaryCLIService;
 import org.apache.thrift.TProcessorFactory;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
 
-public class TestPlainSaslHelper extends TestCase {
+/**
+ * TestPlainSaslHelper.
+ */
+public class TestPlainSaslHelper {
 
   /**
    * Test setting {@link HiveConf.ConfVars}} config parameter
    *   HIVE_SERVER2_ENABLE_DOAS for unsecure mode
    */
+  @Test
   public void testDoAsSetting(){
 
     HiveConf hconf = new HiveConf();
diff --git a/service/src/test/org/apache/hive/service/cli/session/TestSessionCleanup.java b/service/src/test/org/apache/hive/service/cli/session/TestSessionCleanup.java
index 487a5d4..83748c3 100644
--- a/service/src/test/org/apache/hive/service/cli/session/TestSessionCleanup.java
+++ b/service/src/test/org/apache/hive/service/cli/session/TestSessionCleanup.java
@@ -25,7 +25,7 @@ import java.util.Collections;
 import java.util.HashSet;
 import java.util.Set;
 
-import junit.framework.TestCase;
+
 
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
@@ -35,7 +35,10 @@ import org.apache.hive.service.cli.thrift.ThriftCLIServiceClient;
 import org.junit.Assert;
 import org.junit.Test;
 
-public class TestSessionCleanup extends TestCase {
+/**
+ * TestSessionCleanup.
+ */
+public class TestSessionCleanup {
 
   @Test
   // This is to test session temporary files are cleaned up after HIVE-11768
diff --git a/service/src/test/org/apache/hive/service/cli/session/TestSessionGlobalInitFile.java b/service/src/test/org/apache/hive/service/cli/session/TestSessionGlobalInitFile.java
index 9d00ec4..4a3803b 100644
--- a/service/src/test/org/apache/hive/service/cli/session/TestSessionGlobalInitFile.java
+++ b/service/src/test/org/apache/hive/service/cli/session/TestSessionGlobalInitFile.java
@@ -22,7 +22,7 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
 
-import junit.framework.TestCase;
+
 
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -38,7 +38,10 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-public class TestSessionGlobalInitFile extends TestCase {
+/**
+ * TestSessionGlobalInitFile.
+ */
+public class TestSessionGlobalInitFile {
 
   private FakeEmbeddedThriftBinaryCLIService service;
   private ThriftCLIServiceClient client;
@@ -65,7 +68,7 @@ public class TestSessionGlobalInitFile extends TestCase {
 
   @Before
   public void setUp() throws Exception {
-    super.setUp();
+
 
     // create and put .hiverc sample file to default directory
     initFile = File.createTempFile("test", "hive");
diff --git a/service/src/test/org/apache/hive/service/cli/session/TestSessionHooks.java b/service/src/test/org/apache/hive/service/cli/session/TestSessionHooks.java
index 6daf757..d687a7b 100644
--- a/service/src/test/org/apache/hive/service/cli/session/TestSessionHooks.java
+++ b/service/src/test/org/apache/hive/service/cli/session/TestSessionHooks.java
@@ -36,7 +36,10 @@ import org.apache.hive.service.cli.thrift.ThriftCLIServiceClient;
 import org.junit.Before;
 import org.junit.Test;
 
-public class TestSessionHooks extends TestCase {
+/**
+ * TestSessionHooks.
+ */
+public class TestSessionHooks {
 
   private static String sessionUserName = "user1";
   private EmbeddedThriftBinaryCLIService service;
@@ -56,10 +59,9 @@ public class TestSessionHooks extends TestCase {
     }
   }
 
-  @Override
   @Before
   public void setUp() throws Exception {
-    super.setUp();
+
     SessionHookTest.runCount.set(0);
     System.setProperty(ConfVars.HIVE_SERVER2_SESSION_HOOK.varname,
         TestSessionHooks.SessionHookTest.class.getName());


[hive] 03/05: HIVE-21940: Metastore: Postgres text <-> clob mismatch for PARTITION_PARAMS/PARAM_VALUE (Laszlo Bodor via Naveen Gangam, Zoltan Haindrich)

Posted by kg...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kgyrtkirk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git

commit 0bc73f329357744f32d8c206c7c9c0a64ccaf2ce
Author: Laszlo Bodor <bo...@gmail.com>
AuthorDate: Mon Jul 15 11:32:45 2019 +0200

    HIVE-21940: Metastore: Postgres text <-> clob mismatch for PARTITION_PARAMS/PARAM_VALUE (Laszlo Bodor via Naveen Gangam, Zoltan Haindrich)
    
    Signed-off-by: Zoltan Haindrich <ki...@rxd.hu>
---
 standalone-metastore/metastore-server/src/main/resources/package.jdo | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/standalone-metastore/metastore-server/src/main/resources/package.jdo b/standalone-metastore/metastore-server/src/main/resources/package.jdo
index 8ad2f21..4586f22 100644
--- a/standalone-metastore/metastore-server/src/main/resources/package.jdo
+++ b/standalone-metastore/metastore-server/src/main/resources/package.jdo
@@ -503,7 +503,7 @@
            <column name="PARAM_KEY" length="256" jdbc-type="VARCHAR"/>
         </key>
         <value>
-           <column name="PARAM_VALUE" jdbc-type="CLOB"/>
+           <column name="PARAM_VALUE" length="32672" jdbc-type="VARCHAR"/>
         </value>
       </field>
       <field name="writeId">
@@ -553,7 +553,7 @@
            <column name="PARAM_KEY" length="256" jdbc-type="VARCHAR"/>
         </key>
         <value>
-           <column name="PARAM_VALUE" jdbc-type="CLOB"/>
+           <column name="PARAM_VALUE" length="32672" jdbc-type="VARCHAR"/>
         </value>
       </field>
     </class>


[hive] 02/05: HIVE-21923: Vectorized MapJoin may miss results when only the join key is selected (Zoltan Haindrich reviewed by Jesus Camacho Rodriguez)

Posted by kg...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kgyrtkirk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git

commit 96e8e78e8f7dadbb8527f054c21e2bb5c90fc8c9
Author: Zoltan Haindrich <ki...@rxd.hu>
AuthorDate: Mon Jul 15 11:32:04 2019 +0200

    HIVE-21923: Vectorized MapJoin may miss results when only the join key is selected (Zoltan Haindrich reviewed by Jesus Camacho Rodriguez)
    
    Signed-off-by: Zoltan Haindrich <ki...@rxd.hu>
---
 .../hive/common/format/datetime/package-info.java  |   22 -
 ...rMapJoinInnerBigOnlyGenerateResultOperator.java |    6 +-
 .../clientpositive/hybridgrace_hashjoin_2.q        |   24 +-
 .../llap/correlationoptimizer4.q.out               |    2 +-
 .../llap/hybridgrace_hashjoin_2.q.out              |  228 +-
 .../results/clientpositive/spark/auto_join14.q.out |    2 +-
 .../spark/bucket_map_join_tez1.q.out               | 3056 ++++++++++----------
 .../tez/hybridgrace_hashjoin_2.q.out               |  227 +-
 8 files changed, 1960 insertions(+), 1607 deletions(-)

diff --git a/common/src/test/org/apache/hadoop/hive/common/format/datetime/package-info.java b/common/src/test/org/apache/hadoop/hive/common/format/datetime/package-info.java
deleted file mode 100644
index 70ee426..0000000
--- a/common/src/test/org/apache/hadoop/hive/common/format/datetime/package-info.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * Tests formatting and parsing of datetime objects.
- */
-package org.apache.hadoop.hive.common.format.datetime;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyGenerateResultOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyGenerateResultOperator.java
index 35ddddd..ef6ad69 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyGenerateResultOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyGenerateResultOperator.java
@@ -252,12 +252,16 @@ public abstract class VectorMapJoinInnerBigOnlyGenerateResultOperator
 
       for (long l = 0; l < count; l++) {
 
-        // Copy the BigTable values into the overflow batch. Since the overflow batch may
+        // Copy the values into the overflow batch. Since the overflow batch may
         // not get flushed here, we must copy by value.
         if (bigTableRetainedVectorCopy != null) {
           bigTableRetainedVectorCopy.copyByValue(batch, batchIndex,
                                                  overflowBatch, overflowBatch.size);
         }
+        if (nonOuterSmallTableKeyVectorCopy != null) {
+          nonOuterSmallTableKeyVectorCopy.copyByValue(batch, batchIndex,
+              overflowBatch, overflowBatch.size);
+        }
 
         overflowBatch.size++;
         if (overflowBatch.size == overflowBatch.DEFAULT_SIZE) {
diff --git a/ql/src/test/queries/clientpositive/hybridgrace_hashjoin_2.q b/ql/src/test/queries/clientpositive/hybridgrace_hashjoin_2.q
index d989ca7..6ed771b 100644
--- a/ql/src/test/queries/clientpositive/hybridgrace_hashjoin_2.q
+++ b/ql/src/test/queries/clientpositive/hybridgrace_hashjoin_2.q
@@ -4,8 +4,7 @@
 set hive.mapred.mode=nonstrict;
 set hive.explain.user=false;
 -- Hybrid Grace Hash Join
--- Test n-way join
-SELECT 1;
+SELECT 'Test n-way join';
 
 set hive.auto.convert.join=true;
 set hive.auto.convert.join.noconditionaltask=true;
@@ -13,8 +12,7 @@ set hive.auto.convert.join.noconditionaltask.size=10000000;
 set hive.cbo.enable=false;
 
 
--- 3-way mapjoin (1 big table, 2 small tables)
-SELECT 1;
+SELECT '3-way mapjoin (1 big table, 2 small tables)';
 
 set hive.mapjoin.hybridgrace.hashtable=false;
 
@@ -34,13 +32,17 @@ SELECT COUNT(*)
 FROM src1 x JOIN srcpart z ON (x.key = z.key)
 JOIN src y ON (y.key = x.key);
 
+EXPLAIN  ANALYZE SELECT COUNT(*)
+FROM src1 x JOIN srcpart z ON (x.key = z.key)
+JOIN srcpart w ON (x.key = w.key)
+JOIN src y ON (y.key = x.key);
+
 SELECT COUNT(*)
 FROM src1 x JOIN srcpart z ON (x.key = z.key)
 JOIN src y ON (y.key = x.key);
 
 
--- 4-way mapjoin (1 big table, 3 small tables)
-SELECT 1;
+SELECT '4-way mapjoin (1 big table, 3 small tables)';
 
 set hive.mapjoin.hybridgrace.hashtable=false;
 
@@ -50,7 +52,7 @@ FROM src1 x JOIN srcpart z ON (x.key = z.key)
 JOIN srcpart w ON (x.key = w.key)
 JOIN src y ON (y.key = x.key);
 
-SELECT COUNT(*)
+SELECT assert_true(5680 = COUNT(*))
 FROM src1 x JOIN srcpart z ON (x.key = z.key)
 JOIN srcpart w ON (x.key = w.key)
 JOIN src y ON (y.key = x.key);
@@ -63,14 +65,13 @@ FROM src1 x JOIN srcpart z ON (x.key = z.key)
 JOIN srcpart w ON (x.key = w.key)
 JOIN src y ON (y.key = x.key);
 
-SELECT COUNT(*)
+SELECT assert_true(5680 = COUNT(*))
 FROM src1 x JOIN srcpart z ON (x.key = z.key)
 JOIN srcpart w ON (x.key = w.key)
 JOIN src y ON (y.key = x.key);
 
 
--- 2 sets of 3-way mapjoin under 2 different tasks
-SELECT 1;
+SELECT '2 sets of 3-way mapjoin under 2 different tasks';
 
 set hive.mapjoin.hybridgrace.hashtable=false;
 
@@ -111,8 +112,7 @@ FROM src1 x JOIN srcpart z ON (x.value = z.value)
 JOIN src y ON (y.value = x.value);
 
 
--- A chain of 2 sets of 3-way mapjoin under the same task
-SELECT 1;
+SELECT 'A chain of 2 sets of 3-way mapjoin under the same task';
 
 set hive.mapjoin.hybridgrace.hashtable=false;
 
diff --git a/ql/src/test/results/clientpositive/llap/correlationoptimizer4.q.out b/ql/src/test/results/clientpositive/llap/correlationoptimizer4.q.out
index 45a646c..71aaa10 100644
--- a/ql/src/test/results/clientpositive/llap/correlationoptimizer4.q.out
+++ b/ql/src/test/results/clientpositive/llap/correlationoptimizer4.q.out
@@ -721,7 +721,7 @@ POSTHOOK: Input: default@t1_n146
 POSTHOOK: Input: default@t2_n86
 POSTHOOK: Input: default@t3_n34
 #### A masked pattern was here ####
-5	2
+13	10
 PREHOOK: query: EXPLAIN VECTORIZATION
 SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt))
 FROM (SELECT x.key AS key, count(1) AS cnt
diff --git a/ql/src/test/results/clientpositive/llap/hybridgrace_hashjoin_2.q.out b/ql/src/test/results/clientpositive/llap/hybridgrace_hashjoin_2.q.out
index 1ddc1ea..44c4568 100644
--- a/ql/src/test/results/clientpositive/llap/hybridgrace_hashjoin_2.q.out
+++ b/ql/src/test/results/clientpositive/llap/hybridgrace_hashjoin_2.q.out
@@ -1,21 +1,21 @@
-PREHOOK: query: SELECT 1
+PREHOOK: query: SELECT 'Test n-way join'
 PREHOOK: type: QUERY
 PREHOOK: Input: _dummy_database@_dummy_table
 #### A masked pattern was here ####
-POSTHOOK: query: SELECT 1
+POSTHOOK: query: SELECT 'Test n-way join'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: _dummy_database@_dummy_table
 #### A masked pattern was here ####
-1
-PREHOOK: query: SELECT 1
+Test n-way join
+PREHOOK: query: SELECT '3-way mapjoin (1 big table, 2 small tables)'
 PREHOOK: type: QUERY
 PREHOOK: Input: _dummy_database@_dummy_table
 #### A masked pattern was here ####
-POSTHOOK: query: SELECT 1
+POSTHOOK: query: SELECT '3-way mapjoin (1 big table, 2 small tables)'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: _dummy_database@_dummy_table
 #### A masked pattern was here ####
-1
+3-way mapjoin (1 big table, 2 small tables)
 PREHOOK: query: EXPLAIN
 SELECT COUNT(*)
 FROM src1 x JOIN srcpart z ON (x.key = z.key)
@@ -309,6 +309,192 @@ STAGE PLANS:
 
 PREHOOK: query: SELECT COUNT(*)
 FROM src1 x JOIN srcpart z ON (x.key = z.key)
+JOIN srcpart w ON (x.key = w.key)
+JOIN src y ON (y.key = x.key)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Input: default@src1
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT COUNT(*)
+FROM src1 x JOIN srcpart z ON (x.key = z.key)
+JOIN srcpart w ON (x.key = w.key)
+JOIN src y ON (y.key = x.key)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@src1
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+#### A masked pattern was here ####
+PREHOOK: query: EXPLAIN  ANALYZE SELECT COUNT(*)
+FROM src1 x JOIN srcpart z ON (x.key = z.key)
+JOIN srcpart w ON (x.key = w.key)
+JOIN src y ON (y.key = x.key)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Input: default@src1
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+#### A masked pattern was here ####
+POSTHOOK: query: EXPLAIN  ANALYZE SELECT COUNT(*)
+FROM src1 x JOIN srcpart z ON (x.key = z.key)
+JOIN srcpart w ON (x.key = w.key)
+JOIN src y ON (y.key = x.key)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@src1
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 2 <- Map 1 (BROADCAST_EDGE)
+        Map 3 <- Map 2 (BROADCAST_EDGE), Map 5 (BROADCAST_EDGE)
+        Reducer 4 <- Map 3 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: x
+                  filterExpr: key is not null (type: boolean)
+                  Statistics: Num rows: 25/25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 25/25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      key expressions: key (type: string)
+                      sort order: +
+                      Map-reduce partition columns: key (type: string)
+                      Statistics: Num rows: 25/25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 2 
+            Map Operator Tree:
+                TableScan
+                  alias: z
+                  filterExpr: key is not null (type: boolean)
+                  Statistics: Num rows: 2000/2000 Data size: 174000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 2000/2000 Data size: 174000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Map Join Operator
+                      condition map:
+                           Inner Join 0 to 1
+                      keys:
+                        0 key (type: string)
+                        1 key (type: string)
+                      outputColumnNames: _col0
+                      input vertices:
+                        0 Map 1
+                      Statistics: Num rows: 158/148 Data size: 13588 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 158/148 Data size: 13588 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 3 
+            Map Operator Tree:
+                TableScan
+                  alias: w
+                  filterExpr: key is not null (type: boolean)
+                  Statistics: Num rows: 2000/2000 Data size: 174000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 2000/2000 Data size: 174000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Map Join Operator
+                      condition map:
+                           Inner Join 0 to 1
+                      keys:
+                        0 _col0 (type: string)
+                        1 key (type: string)
+                      outputColumnNames: _col0
+                      input vertices:
+                        0 Map 2
+                      Statistics: Num rows: 1000/1712 Data size: 86000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Map Join Operator
+                        condition map:
+                             Inner Join 0 to 1
+                        keys:
+                          0 _col0 (type: string)
+                          1 key (type: string)
+                        input vertices:
+                          1 Map 5
+                        Statistics: Num rows: 1582/5680 Data size: 12656 Basic stats: COMPLETE Column stats: COMPLETE
+                        Group By Operator
+                          aggregations: count()
+                          minReductionHashAggr: 0.99
+                          mode: hash
+                          outputColumnNames: _col0
+                          Statistics: Num rows: 1/1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                          Reduce Output Operator
+                            sort order: 
+                            Statistics: Num rows: 1/1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: bigint)
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: y
+                  filterExpr: key is not null (type: boolean)
+                  Statistics: Num rows: 500/500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 500/500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      key expressions: key (type: string)
+                      sort order: +
+                      Map-reduce partition columns: key (type: string)
+                      Statistics: Num rows: 500/500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Reducer 4 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1/1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1/1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT COUNT(*)
+FROM src1 x JOIN srcpart z ON (x.key = z.key)
 JOIN src y ON (y.key = x.key)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
@@ -332,15 +518,15 @@ POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
 POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
 #### A masked pattern was here ####
 428
-PREHOOK: query: SELECT 1
+PREHOOK: query: SELECT '4-way mapjoin (1 big table, 3 small tables)'
 PREHOOK: type: QUERY
 PREHOOK: Input: _dummy_database@_dummy_table
 #### A masked pattern was here ####
-POSTHOOK: query: SELECT 1
+POSTHOOK: query: SELECT '4-way mapjoin (1 big table, 3 small tables)'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: _dummy_database@_dummy_table
 #### A masked pattern was here ####
-1
+4-way mapjoin (1 big table, 3 small tables)
 PREHOOK: query: EXPLAIN
 SELECT COUNT(*)
 FROM src1 x JOIN srcpart z ON (x.key = z.key)
@@ -503,7 +689,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-PREHOOK: query: SELECT COUNT(*)
+PREHOOK: query: SELECT assert_true(5680 = COUNT(*))
 FROM src1 x JOIN srcpart z ON (x.key = z.key)
 JOIN srcpart w ON (x.key = w.key)
 JOIN src y ON (y.key = x.key)
@@ -516,7 +702,7 @@ PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
 PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
 PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
 #### A masked pattern was here ####
-POSTHOOK: query: SELECT COUNT(*)
+POSTHOOK: query: SELECT assert_true(5680 = COUNT(*))
 FROM src1 x JOIN srcpart z ON (x.key = z.key)
 JOIN srcpart w ON (x.key = w.key)
 JOIN src y ON (y.key = x.key)
@@ -529,7 +715,7 @@ POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
 POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
 POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
 #### A masked pattern was here ####
-0
+NULL
 PREHOOK: query: EXPLAIN
 SELECT COUNT(*)
 FROM src1 x JOIN srcpart z ON (x.key = z.key)
@@ -692,7 +878,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-PREHOOK: query: SELECT COUNT(*)
+PREHOOK: query: SELECT assert_true(5680 = COUNT(*))
 FROM src1 x JOIN srcpart z ON (x.key = z.key)
 JOIN srcpart w ON (x.key = w.key)
 JOIN src y ON (y.key = x.key)
@@ -705,7 +891,7 @@ PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
 PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
 PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
 #### A masked pattern was here ####
-POSTHOOK: query: SELECT COUNT(*)
+POSTHOOK: query: SELECT assert_true(5680 = COUNT(*))
 FROM src1 x JOIN srcpart z ON (x.key = z.key)
 JOIN srcpart w ON (x.key = w.key)
 JOIN src y ON (y.key = x.key)
@@ -718,16 +904,16 @@ POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
 POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
 POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
 #### A masked pattern was here ####
-0
-PREHOOK: query: SELECT 1
+NULL
+PREHOOK: query: SELECT '2 sets of 3-way mapjoin under 2 different tasks'
 PREHOOK: type: QUERY
 PREHOOK: Input: _dummy_database@_dummy_table
 #### A masked pattern was here ####
-POSTHOOK: query: SELECT 1
+POSTHOOK: query: SELECT '2 sets of 3-way mapjoin under 2 different tasks'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: _dummy_database@_dummy_table
 #### A masked pattern was here ####
-1
+2 sets of 3-way mapjoin under 2 different tasks
 PREHOOK: query: EXPLAIN
 SELECT COUNT(*)
 FROM src1 x JOIN srcpart z ON (x.key = z.key)
@@ -1308,15 +1494,15 @@ POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
 #### A masked pattern was here ####
 452
 428
-PREHOOK: query: SELECT 1
+PREHOOK: query: SELECT 'A chain of 2 sets of 3-way mapjoin under the same task'
 PREHOOK: type: QUERY
 PREHOOK: Input: _dummy_database@_dummy_table
 #### A masked pattern was here ####
-POSTHOOK: query: SELECT 1
+POSTHOOK: query: SELECT 'A chain of 2 sets of 3-way mapjoin under the same task'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: _dummy_database@_dummy_table
 #### A masked pattern was here ####
-1
+A chain of 2 sets of 3-way mapjoin under the same task
 PREHOOK: query: EXPLAIN
 SELECT COUNT(*)
 FROM src1 x
diff --git a/ql/src/test/results/clientpositive/spark/auto_join14.q.out b/ql/src/test/results/clientpositive/spark/auto_join14.q.out
index 0c80c13..64ddc35 100644
--- a/ql/src/test/results/clientpositive/spark/auto_join14.q.out
+++ b/ql/src/test/results/clientpositive/spark/auto_join14.q.out
@@ -139,4 +139,4 @@ POSTHOOK: query: SELECT sum(hash(dest1_n83.c1,dest1_n83.c2)) FROM dest1_n83
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@dest1_n83
 #### A masked pattern was here ####
-404540072956
+404554174174
diff --git a/ql/src/test/results/clientpositive/spark/bucket_map_join_tez1.q.out b/ql/src/test/results/clientpositive/spark/bucket_map_join_tez1.q.out
index 4ee669f..e026131 100644
--- a/ql/src/test/results/clientpositive/spark/bucket_map_join_tez1.q.out
+++ b/ql/src/test/results/clientpositive/spark/bucket_map_join_tez1.q.out
@@ -5146,404 +5146,120 @@ POSTHOOK: Input: default@tab_part_n9@ds=2008-04-08
 0	val_0	val_0
 0	val_0	val_0
 0	val_0	val_0
-0	val_103	val_103
-0	val_103	val_103
-0	val_103	val_103
-0	val_103	val_103
-0	val_118	val_118
-0	val_118	val_118
-0	val_118	val_118
-0	val_118	val_118
-0	val_125	val_125
-0	val_125	val_125
-0	val_125	val_125
-0	val_125	val_125
-0	val_129	val_129
-0	val_129	val_129
-0	val_129	val_129
-0	val_129	val_129
-0	val_134	val_134
-0	val_134	val_134
-0	val_134	val_134
-0	val_134	val_134
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_149	val_149
-0	val_149	val_149
-0	val_149	val_149
-0	val_149	val_149
-0	val_15	val_15
-0	val_15	val_15
-0	val_15	val_15
-0	val_15	val_15
-0	val_152	val_152
-0	val_152	val_152
-0	val_152	val_152
-0	val_152	val_152
-0	val_165	val_165
-0	val_165	val_165
-0	val_165	val_165
-0	val_165	val_165
-0	val_167	val_167
-0	val_167	val_167
-0	val_167	val_167
-0	val_167	val_167
-0	val_167	val_167
-0	val_167	val_167
-0	val_167	val_167
-0	val_167	val_167
-0	val_167	val_167
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_172	val_172
-0	val_172	val_172
-0	val_172	val_172
-0	val_172	val_172
-0	val_174	val_174
-0	val_174	val_174
-0	val_174	val_174
-0	val_174	val_174
-0	val_176	val_176
-0	val_176	val_176
-0	val_176	val_176
-0	val_176	val_176
-0	val_187	val_187
-0	val_187	val_187
-0	val_187	val_187
-0	val_187	val_187
-0	val_187	val_187
-0	val_187	val_187
-0	val_187	val_187
-0	val_187	val_187
-0	val_187	val_187
-0	val_200	val_200
-0	val_200	val_200
-0	val_200	val_200
-0	val_200	val_200
-0	val_208	val_208
-0	val_208	val_208
-0	val_208	val_208
-0	val_208	val_208
-0	val_208	val_208
-0	val_208	val_208
-0	val_208	val_208
-0	val_208	val_208
-0	val_208	val_208
-0	val_213	val_213
-0	val_213	val_213
-0	val_213	val_213
-0	val_213	val_213
-0	val_217	val_217
-0	val_217	val_217
-0	val_217	val_217
-0	val_217	val_217
-0	val_219	val_219
-0	val_219	val_219
-0	val_219	val_219
-0	val_219	val_219
-0	val_224	val_224
-0	val_224	val_224
-0	val_224	val_224
-0	val_224	val_224
-0	val_233	val_233
-0	val_233	val_233
-0	val_233	val_233
-0	val_233	val_233
-0	val_237	val_237
-0	val_237	val_237
-0	val_237	val_237
-0	val_237	val_237
-0	val_239	val_239
-0	val_239	val_239
-0	val_239	val_239
-0	val_239	val_239
-0	val_24	val_24
-0	val_24	val_24
-0	val_24	val_24
-0	val_24	val_24
-0	val_242	val_242
-0	val_242	val_242
-0	val_242	val_242
-0	val_242	val_242
-0	val_255	val_255
-0	val_255	val_255
-0	val_255	val_255
-0	val_255	val_255
-0	val_26	val_26
-0	val_26	val_26
-0	val_26	val_26
-0	val_26	val_26
-0	val_273	val_273
-0	val_273	val_273
-0	val_273	val_273
-0	val_273	val_273
-0	val_273	val_273
-0	val_273	val_273
-0	val_273	val_273
-0	val_273	val_273
-0	val_273	val_273
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_280	val_280
-0	val_280	val_280
-0	val_280	val_280
-0	val_280	val_280
-0	val_282	val_282
-0	val_282	val_282
-0	val_282	val_282
-0	val_282	val_282
-0	val_288	val_288
-0	val_288	val_288
-0	val_288	val_288
-0	val_288	val_288
-0	val_307	val_307
-0	val_307	val_307
-0	val_307	val_307
-0	val_307	val_307
-0	val_309	val_309
-0	val_309	val_309
-0	val_309	val_309
-0	val_309	val_309
-0	val_316	val_316
-0	val_316	val_316
-0	val_316	val_316
-0	val_316	val_316
-0	val_316	val_316
-0	val_316	val_316
-0	val_316	val_316
-0	val_316	val_316
-0	val_316	val_316
-0	val_318	val_318
-0	val_318	val_318
-0	val_318	val_318
-0	val_318	val_318
-0	val_318	val_318
-0	val_318	val_318
-0	val_318	val_318
-0	val_318	val_318
-0	val_318	val_318
-0	val_321	val_321
-0	val_321	val_321
-0	val_321	val_321
-0	val_321	val_321
-0	val_325	val_325
-0	val_325	val_325
-0	val_325	val_325
-0	val_325	val_325
-0	val_327	val_327
-0	val_327	val_327
-0	val_327	val_327
-0	val_327	val_327
-0	val_327	val_327
-0	val_327	val_327
-0	val_327	val_327
-0	val_327	val_327
-0	val_327	val_327
-0	val_35	val_35
-0	val_35	val_35
-0	val_35	val_35
-0	val_35	val_35
-0	val_35	val_35
-0	val_35	val_35
-0	val_35	val_35
-0	val_35	val_35
-0	val_35	val_35
-0	val_367	val_367
-0	val_367	val_367
-0	val_367	val_367
-0	val_367	val_367
-0	val_369	val_369
-0	val_369	val_369
-0	val_369	val_369
-0	val_369	val_369
-0	val_369	val_369
-0	val_369	val_369
-0	val_369	val_369
-0	val_369	val_369
-0	val_369	val_369
-0	val_37	val_37
-0	val_37	val_37
-0	val_37	val_37
-0	val_37	val_37
-0	val_396	val_396
-0	val_396	val_396
-0	val_396	val_396
-0	val_396	val_396
-0	val_396	val_396
-0	val_396	val_396
-0	val_396	val_396
-0	val_396	val_396
-0	val_396	val_396
-0	val_404	val_404
-0	val_404	val_404
-0	val_404	val_404
-0	val_404	val_404
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_413	val_413
-0	val_413	val_413
-0	val_413	val_413
-0	val_413	val_413
-0	val_417	val_417
-0	val_417	val_417
-0	val_417	val_417
-0	val_417	val_417
-0	val_417	val_417
-0	val_417	val_417
-0	val_417	val_417
-0	val_417	val_417
-0	val_417	val_417
-0	val_42	val_42
-0	val_42	val_42
-0	val_42	val_42
-0	val_42	val_42
-0	val_424	val_424
-0	val_424	val_424
-0	val_424	val_424
-0	val_424	val_424
-0	val_431	val_431
-0	val_431	val_431
-0	val_431	val_431
-0	val_431	val_431
-0	val_431	val_431
-0	val_431	val_431
-0	val_431	val_431
-0	val_431	val_431
-0	val_431	val_431
-0	val_439	val_439
-0	val_439	val_439
-0	val_439	val_439
-0	val_439	val_439
-0	val_459	val_459
-0	val_459	val_459
-0	val_459	val_459
-0	val_459	val_459
-0	val_462	val_462
-0	val_462	val_462
-0	val_462	val_462
-0	val_462	val_462
-0	val_466	val_466
-0	val_466	val_466
-0	val_466	val_466
-0	val_466	val_466
-0	val_466	val_466
-0	val_466	val_466
-0	val_466	val_466
-0	val_466	val_466
-0	val_466	val_466
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_480	val_480
-0	val_480	val_480
-0	val_480	val_480
-0	val_480	val_480
-0	val_480	val_480
-0	val_480	val_480
-0	val_480	val_480
-0	val_480	val_480
-0	val_480	val_480
-0	val_51	val_51
-0	val_51	val_51
-0	val_51	val_51
-0	val_51	val_51
-0	val_84	val_84
-0	val_84	val_84
-0	val_84	val_84
-0	val_84	val_84
-0	val_95	val_95
-0	val_95	val_95
-0	val_95	val_95
-0	val_95	val_95
-0	val_97	val_97
-0	val_97	val_97
-0	val_97	val_97
-0	val_97	val_97
+103	val_103	val_103
+103	val_103	val_103
+103	val_103	val_103
+103	val_103	val_103
 105	val_105	val_105
 11	val_11	val_11
 114	val_114	val_114
 116	val_116	val_116
+118	val_118	val_118
+118	val_118	val_118
+118	val_118	val_118
+118	val_118	val_118
+125	val_125	val_125
+125	val_125	val_125
+125	val_125	val_125
+125	val_125	val_125
+129	val_129	val_129
+129	val_129	val_129
+129	val_129	val_129
+129	val_129	val_129
+134	val_134	val_134
+134	val_134	val_134
+134	val_134	val_134
+134	val_134	val_134
 136	val_136	val_136
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
 143	val_143	val_143
 145	val_145	val_145
+149	val_149	val_149
+149	val_149	val_149
+149	val_149	val_149
+149	val_149	val_149
+15	val_15	val_15
+15	val_15	val_15
+15	val_15	val_15
+15	val_15	val_15
 150	val_150	val_150
+152	val_152	val_152
+152	val_152	val_152
+152	val_152	val_152
+152	val_152	val_152
 156	val_156	val_156
 158	val_158	val_158
 163	val_163	val_163
+165	val_165	val_165
+165	val_165	val_165
+165	val_165	val_165
+165	val_165	val_165
+167	val_167	val_167
+167	val_167	val_167
+167	val_167	val_167
+167	val_167	val_167
+167	val_167	val_167
+167	val_167	val_167
+167	val_167	val_167
+167	val_167	val_167
+167	val_167	val_167
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
 17	val_17	val_17
 170	val_170	val_170
+172	val_172	val_172
+172	val_172	val_172
+172	val_172	val_172
+172	val_172	val_172
+174	val_174	val_174
+174	val_174	val_174
+174	val_174	val_174
+174	val_174	val_174
+176	val_176	val_176
+176	val_176	val_176
+176	val_176	val_176
+176	val_176	val_176
 178	val_178	val_178
 181	val_181	val_181
 183	val_183	val_183
+187	val_187	val_187
+187	val_187	val_187
+187	val_187	val_187
+187	val_187	val_187
+187	val_187	val_187
+187	val_187	val_187
+187	val_187	val_187
+187	val_187	val_187
+187	val_187	val_187
 189	val_189	val_189
 19	val_19	val_19
 190	val_190	val_190
@@ -5552,45 +5268,271 @@ POSTHOOK: Input: default@tab_part_n9@ds=2008-04-08
 196	val_196	val_196
 2	val_2	val_2
 20	val_20	val_20
+200	val_200	val_200
+200	val_200	val_200
+200	val_200	val_200
+200	val_200	val_200
 202	val_202	val_202
+208	val_208	val_208
+208	val_208	val_208
+208	val_208	val_208
+208	val_208	val_208
+208	val_208	val_208
+208	val_208	val_208
+208	val_208	val_208
+208	val_208	val_208
+208	val_208	val_208
+213	val_213	val_213
+213	val_213	val_213
+213	val_213	val_213
+213	val_213	val_213
+217	val_217	val_217
+217	val_217	val_217
+217	val_217	val_217
+217	val_217	val_217
+219	val_219	val_219
+219	val_219	val_219
+219	val_219	val_219
+219	val_219	val_219
 222	val_222	val_222
+224	val_224	val_224
+224	val_224	val_224
+224	val_224	val_224
+224	val_224	val_224
 226	val_226	val_226
 228	val_228	val_228
+233	val_233	val_233
+233	val_233	val_233
+233	val_233	val_233
+233	val_233	val_233
 235	val_235	val_235
+237	val_237	val_237
+237	val_237	val_237
+237	val_237	val_237
+237	val_237	val_237
+239	val_239	val_239
+239	val_239	val_239
+239	val_239	val_239
+239	val_239	val_239
+24	val_24	val_24
+24	val_24	val_24
+24	val_24	val_24
+24	val_24	val_24
+242	val_242	val_242
+242	val_242	val_242
+242	val_242	val_242
+242	val_242	val_242
 244	val_244	val_244
 248	val_248	val_248
+255	val_255	val_255
+255	val_255	val_255
+255	val_255	val_255
+255	val_255	val_255
 257	val_257	val_257
+26	val_26	val_26
+26	val_26	val_26
+26	val_26	val_26
+26	val_26	val_26
 260	val_260	val_260
 262	val_262	val_262
 266	val_266	val_266
+273	val_273	val_273
+273	val_273	val_273
+273	val_273	val_273
+273	val_273	val_273
+273	val_273	val_273
+273	val_273	val_273
+273	val_273	val_273
+273	val_273	val_273
+273	val_273	val_273
 275	val_275	val_275
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
 28	val_28	val_28
+280	val_280	val_280
+280	val_280	val_280
+280	val_280	val_280
+280	val_280	val_280
+282	val_282	val_282
+282	val_282	val_282
+282	val_282	val_282
+282	val_282	val_282
 284	val_284	val_284
 286	val_286	val_286
+288	val_288	val_288
+288	val_288	val_288
+288	val_288	val_288
+288	val_288	val_288
 291	val_291	val_291
 305	val_305	val_305
+307	val_307	val_307
+307	val_307	val_307
+307	val_307	val_307
+307	val_307	val_307
+309	val_309	val_309
+309	val_309	val_309
+309	val_309	val_309
+309	val_309	val_309
 310	val_310	val_310
+316	val_316	val_316
+316	val_316	val_316
+316	val_316	val_316
+316	val_316	val_316
+316	val_316	val_316
+316	val_316	val_316
+316	val_316	val_316
+316	val_316	val_316
+316	val_316	val_316
+318	val_318	val_318
+318	val_318	val_318
+318	val_318	val_318
+318	val_318	val_318
+318	val_318	val_318
+318	val_318	val_318
+318	val_318	val_318
+318	val_318	val_318
+318	val_318	val_318
+321	val_321	val_321
+321	val_321	val_321
+321	val_321	val_321
+321	val_321	val_321
 323	val_323	val_323
+325	val_325	val_325
+325	val_325	val_325
+325	val_325	val_325
+325	val_325	val_325
+327	val_327	val_327
+327	val_327	val_327
+327	val_327	val_327
+327	val_327	val_327
+327	val_327	val_327
+327	val_327	val_327
+327	val_327	val_327
+327	val_327	val_327
+327	val_327	val_327
 33	val_33	val_33
 332	val_332	val_332
 336	val_336	val_336
 338	val_338	val_338
 341	val_341	val_341
 345	val_345	val_345
+35	val_35	val_35
+35	val_35	val_35
+35	val_35	val_35
+35	val_35	val_35
+35	val_35	val_35
+35	val_35	val_35
+35	val_35	val_35
+35	val_35	val_35
+35	val_35	val_35
 356	val_356	val_356
 365	val_365	val_365
+367	val_367	val_367
+367	val_367	val_367
+367	val_367	val_367
+367	val_367	val_367
+369	val_369	val_369
+369	val_369	val_369
+369	val_369	val_369
+369	val_369	val_369
+369	val_369	val_369
+369	val_369	val_369
+369	val_369	val_369
+369	val_369	val_369
+369	val_369	val_369
+37	val_37	val_37
+37	val_37	val_37
+37	val_37	val_37
+37	val_37	val_37
 374	val_374	val_374
 378	val_378	val_378
 389	val_389	val_389
 392	val_392	val_392
 394	val_394	val_394
+396	val_396	val_396
+396	val_396	val_396
+396	val_396	val_396
+396	val_396	val_396
+396	val_396	val_396
+396	val_396	val_396
+396	val_396	val_396
+396	val_396	val_396
+396	val_396	val_396
 4	val_4	val_4
 400	val_400	val_400
 402	val_402	val_402
+404	val_404	val_404
+404	val_404	val_404
+404	val_404	val_404
+404	val_404	val_404
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
 411	val_411	val_411
+413	val_413	val_413
+413	val_413	val_413
+413	val_413	val_413
+413	val_413	val_413
+417	val_417	val_417
+417	val_417	val_417
+417	val_417	val_417
+417	val_417	val_417
+417	val_417	val_417
+417	val_417	val_417
+417	val_417	val_417
+417	val_417	val_417
+417	val_417	val_417
 419	val_419	val_419
+42	val_42	val_42
+42	val_42	val_42
+42	val_42	val_42
+42	val_42	val_42
+424	val_424	val_424
+424	val_424	val_424
+424	val_424	val_424
+424	val_424	val_424
+431	val_431	val_431
+431	val_431	val_431
+431	val_431	val_431
+431	val_431	val_431
+431	val_431	val_431
+431	val_431	val_431
+431	val_431	val_431
+431	val_431	val_431
+431	val_431	val_431
 435	val_435	val_435
 437	val_437	val_437
+439	val_439	val_439
+439	val_439	val_439
+439	val_439	val_439
+439	val_439	val_439
 44	val_44	val_44
 444	val_444	val_444
 446	val_446	val_446
@@ -5598,16 +5540,62 @@ POSTHOOK: Input: default@tab_part_n9@ds=2008-04-08
 453	val_453	val_453
 455	val_455	val_455
 457	val_457	val_457
+459	val_459	val_459
+459	val_459	val_459
+459	val_459	val_459
+459	val_459	val_459
 460	val_460	val_460
+462	val_462	val_462
+462	val_462	val_462
+462	val_462	val_462
+462	val_462	val_462
+466	val_466	val_466
+466	val_466	val_466
+466	val_466	val_466
+466	val_466	val_466
+466	val_466	val_466
+466	val_466	val_466
+466	val_466	val_466
+466	val_466	val_466
+466	val_466	val_466
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
 475	val_475	val_475
 477	val_477	val_477
 479	val_479	val_479
+480	val_480	val_480
+480	val_480	val_480
+480	val_480	val_480
+480	val_480	val_480
+480	val_480	val_480
+480	val_480	val_480
+480	val_480	val_480
+480	val_480	val_480
+480	val_480	val_480
 482	val_482	val_482
 484	val_484	val_484
 491	val_491	val_491
 493	val_493	val_493
 495	val_495	val_495
 497	val_497	val_497
+51	val_51	val_51
+51	val_51	val_51
+51	val_51	val_51
+51	val_51	val_51
 53	val_53	val_53
 57	val_57	val_57
 64	val_64	val_64
@@ -5616,7 +5604,19 @@ POSTHOOK: Input: default@tab_part_n9@ds=2008-04-08
 8	val_8	val_8
 80	val_80	val_80
 82	val_82	val_82
+84	val_84	val_84
+84	val_84	val_84
+84	val_84	val_84
+84	val_84	val_84
 86	val_86	val_86
+95	val_95	val_95
+95	val_95	val_95
+95	val_95	val_95
+95	val_95	val_95
+97	val_97	val_97
+97	val_97	val_97
+97	val_97	val_97
+97	val_97	val_97
 PREHOOK: query: explain select a.key, a.value, b.value
         from tab_n8 a join tab_part_n9 b on a.key = b.key and a.value = b.value
 PREHOOK: type: QUERY
@@ -5739,404 +5739,120 @@ POSTHOOK: Input: default@tab_part_n9@ds=2008-04-08
 0	val_0	val_0
 0	val_0	val_0
 0	val_0	val_0
-0	val_103	val_103
-0	val_103	val_103
-0	val_103	val_103
-0	val_103	val_103
-0	val_118	val_118
-0	val_118	val_118
-0	val_118	val_118
-0	val_118	val_118
-0	val_125	val_125
-0	val_125	val_125
-0	val_125	val_125
-0	val_125	val_125
-0	val_129	val_129
-0	val_129	val_129
-0	val_129	val_129
-0	val_129	val_129
-0	val_134	val_134
-0	val_134	val_134
-0	val_134	val_134
-0	val_134	val_134
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_149	val_149
-0	val_149	val_149
-0	val_149	val_149
-0	val_149	val_149
-0	val_15	val_15
-0	val_15	val_15
-0	val_15	val_15
-0	val_15	val_15
-0	val_152	val_152
-0	val_152	val_152
-0	val_152	val_152
-0	val_152	val_152
-0	val_165	val_165
-0	val_165	val_165
-0	val_165	val_165
-0	val_165	val_165
-0	val_167	val_167
-0	val_167	val_167
-0	val_167	val_167
-0	val_167	val_167
-0	val_167	val_167
-0	val_167	val_167
-0	val_167	val_167
-0	val_167	val_167
-0	val_167	val_167
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_172	val_172
-0	val_172	val_172
-0	val_172	val_172
-0	val_172	val_172
-0	val_174	val_174
-0	val_174	val_174
-0	val_174	val_174
-0	val_174	val_174
-0	val_176	val_176
-0	val_176	val_176
-0	val_176	val_176
-0	val_176	val_176
-0	val_187	val_187
-0	val_187	val_187
-0	val_187	val_187
-0	val_187	val_187
-0	val_187	val_187
-0	val_187	val_187
-0	val_187	val_187
-0	val_187	val_187
-0	val_187	val_187
-0	val_200	val_200
-0	val_200	val_200
-0	val_200	val_200
-0	val_200	val_200
-0	val_208	val_208
-0	val_208	val_208
-0	val_208	val_208
-0	val_208	val_208
-0	val_208	val_208
-0	val_208	val_208
-0	val_208	val_208
-0	val_208	val_208
-0	val_208	val_208
-0	val_213	val_213
-0	val_213	val_213
-0	val_213	val_213
-0	val_213	val_213
-0	val_217	val_217
-0	val_217	val_217
-0	val_217	val_217
-0	val_217	val_217
-0	val_219	val_219
-0	val_219	val_219
-0	val_219	val_219
-0	val_219	val_219
-0	val_224	val_224
-0	val_224	val_224
-0	val_224	val_224
-0	val_224	val_224
-0	val_233	val_233
-0	val_233	val_233
-0	val_233	val_233
-0	val_233	val_233
-0	val_237	val_237
-0	val_237	val_237
-0	val_237	val_237
-0	val_237	val_237
-0	val_239	val_239
-0	val_239	val_239
-0	val_239	val_239
-0	val_239	val_239
-0	val_24	val_24
-0	val_24	val_24
-0	val_24	val_24
-0	val_24	val_24
-0	val_242	val_242
-0	val_242	val_242
-0	val_242	val_242
-0	val_242	val_242
-0	val_255	val_255
-0	val_255	val_255
-0	val_255	val_255
-0	val_255	val_255
-0	val_26	val_26
-0	val_26	val_26
-0	val_26	val_26
-0	val_26	val_26
-0	val_273	val_273
-0	val_273	val_273
-0	val_273	val_273
-0	val_273	val_273
-0	val_273	val_273
-0	val_273	val_273
-0	val_273	val_273
-0	val_273	val_273
-0	val_273	val_273
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_280	val_280
-0	val_280	val_280
-0	val_280	val_280
-0	val_280	val_280
-0	val_282	val_282
-0	val_282	val_282
-0	val_282	val_282
-0	val_282	val_282
-0	val_288	val_288
-0	val_288	val_288
-0	val_288	val_288
-0	val_288	val_288
-0	val_307	val_307
-0	val_307	val_307
-0	val_307	val_307
-0	val_307	val_307
-0	val_309	val_309
-0	val_309	val_309
-0	val_309	val_309
-0	val_309	val_309
-0	val_316	val_316
-0	val_316	val_316
-0	val_316	val_316
-0	val_316	val_316
-0	val_316	val_316
-0	val_316	val_316
-0	val_316	val_316
-0	val_316	val_316
-0	val_316	val_316
-0	val_318	val_318
-0	val_318	val_318
-0	val_318	val_318
-0	val_318	val_318
-0	val_318	val_318
-0	val_318	val_318
-0	val_318	val_318
-0	val_318	val_318
-0	val_318	val_318
-0	val_321	val_321
-0	val_321	val_321
-0	val_321	val_321
-0	val_321	val_321
-0	val_325	val_325
-0	val_325	val_325
-0	val_325	val_325
-0	val_325	val_325
-0	val_327	val_327
-0	val_327	val_327
-0	val_327	val_327
-0	val_327	val_327
-0	val_327	val_327
-0	val_327	val_327
-0	val_327	val_327
-0	val_327	val_327
-0	val_327	val_327
-0	val_35	val_35
-0	val_35	val_35
-0	val_35	val_35
-0	val_35	val_35
-0	val_35	val_35
-0	val_35	val_35
-0	val_35	val_35
-0	val_35	val_35
-0	val_35	val_35
-0	val_367	val_367
-0	val_367	val_367
-0	val_367	val_367
-0	val_367	val_367
-0	val_369	val_369
-0	val_369	val_369
-0	val_369	val_369
-0	val_369	val_369
-0	val_369	val_369
-0	val_369	val_369
-0	val_369	val_369
-0	val_369	val_369
-0	val_369	val_369
-0	val_37	val_37
-0	val_37	val_37
-0	val_37	val_37
-0	val_37	val_37
-0	val_396	val_396
-0	val_396	val_396
-0	val_396	val_396
-0	val_396	val_396
-0	val_396	val_396
-0	val_396	val_396
-0	val_396	val_396
-0	val_396	val_396
-0	val_396	val_396
-0	val_404	val_404
-0	val_404	val_404
-0	val_404	val_404
-0	val_404	val_404
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_413	val_413
-0	val_413	val_413
-0	val_413	val_413
-0	val_413	val_413
-0	val_417	val_417
-0	val_417	val_417
-0	val_417	val_417
-0	val_417	val_417
-0	val_417	val_417
-0	val_417	val_417
-0	val_417	val_417
-0	val_417	val_417
-0	val_417	val_417
-0	val_42	val_42
-0	val_42	val_42
-0	val_42	val_42
-0	val_42	val_42
-0	val_424	val_424
-0	val_424	val_424
-0	val_424	val_424
-0	val_424	val_424
-0	val_431	val_431
-0	val_431	val_431
-0	val_431	val_431
-0	val_431	val_431
-0	val_431	val_431
-0	val_431	val_431
-0	val_431	val_431
-0	val_431	val_431
-0	val_431	val_431
-0	val_439	val_439
-0	val_439	val_439
-0	val_439	val_439
-0	val_439	val_439
-0	val_459	val_459
-0	val_459	val_459
-0	val_459	val_459
-0	val_459	val_459
-0	val_462	val_462
-0	val_462	val_462
-0	val_462	val_462
-0	val_462	val_462
-0	val_466	val_466
-0	val_466	val_466
-0	val_466	val_466
-0	val_466	val_466
-0	val_466	val_466
-0	val_466	val_466
-0	val_466	val_466
-0	val_466	val_466
-0	val_466	val_466
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_480	val_480
-0	val_480	val_480
-0	val_480	val_480
-0	val_480	val_480
-0	val_480	val_480
-0	val_480	val_480
-0	val_480	val_480
-0	val_480	val_480
-0	val_480	val_480
-0	val_51	val_51
-0	val_51	val_51
-0	val_51	val_51
-0	val_51	val_51
-0	val_84	val_84
-0	val_84	val_84
-0	val_84	val_84
-0	val_84	val_84
-0	val_95	val_95
-0	val_95	val_95
-0	val_95	val_95
-0	val_95	val_95
-0	val_97	val_97
-0	val_97	val_97
-0	val_97	val_97
-0	val_97	val_97
+103	val_103	val_103
+103	val_103	val_103
+103	val_103	val_103
+103	val_103	val_103
 105	val_105	val_105
 11	val_11	val_11
 114	val_114	val_114
 116	val_116	val_116
+118	val_118	val_118
+118	val_118	val_118
+118	val_118	val_118
+118	val_118	val_118
+125	val_125	val_125
+125	val_125	val_125
+125	val_125	val_125
+125	val_125	val_125
+129	val_129	val_129
+129	val_129	val_129
+129	val_129	val_129
+129	val_129	val_129
+134	val_134	val_134
+134	val_134	val_134
+134	val_134	val_134
+134	val_134	val_134
 136	val_136	val_136
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
 143	val_143	val_143
 145	val_145	val_145
+149	val_149	val_149
+149	val_149	val_149
+149	val_149	val_149
+149	val_149	val_149
+15	val_15	val_15
+15	val_15	val_15
+15	val_15	val_15
+15	val_15	val_15
 150	val_150	val_150
+152	val_152	val_152
+152	val_152	val_152
+152	val_152	val_152
+152	val_152	val_152
 156	val_156	val_156
 158	val_158	val_158
 163	val_163	val_163
+165	val_165	val_165
+165	val_165	val_165
+165	val_165	val_165
+165	val_165	val_165
+167	val_167	val_167
+167	val_167	val_167
+167	val_167	val_167
+167	val_167	val_167
+167	val_167	val_167
+167	val_167	val_167
+167	val_167	val_167
+167	val_167	val_167
+167	val_167	val_167
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
 17	val_17	val_17
 170	val_170	val_170
+172	val_172	val_172
+172	val_172	val_172
+172	val_172	val_172
+172	val_172	val_172
+174	val_174	val_174
+174	val_174	val_174
+174	val_174	val_174
+174	val_174	val_174
+176	val_176	val_176
+176	val_176	val_176
+176	val_176	val_176
+176	val_176	val_176
 178	val_178	val_178
 181	val_181	val_181
 183	val_183	val_183
+187	val_187	val_187
+187	val_187	val_187
+187	val_187	val_187
+187	val_187	val_187
+187	val_187	val_187
+187	val_187	val_187
+187	val_187	val_187
+187	val_187	val_187
+187	val_187	val_187
 189	val_189	val_189
 19	val_19	val_19
 190	val_190	val_190
@@ -6145,45 +5861,271 @@ POSTHOOK: Input: default@tab_part_n9@ds=2008-04-08
 196	val_196	val_196
 2	val_2	val_2
 20	val_20	val_20
+200	val_200	val_200
+200	val_200	val_200
+200	val_200	val_200
+200	val_200	val_200
 202	val_202	val_202
+208	val_208	val_208
+208	val_208	val_208
+208	val_208	val_208
+208	val_208	val_208
+208	val_208	val_208
+208	val_208	val_208
+208	val_208	val_208
+208	val_208	val_208
+208	val_208	val_208
+213	val_213	val_213
+213	val_213	val_213
+213	val_213	val_213
+213	val_213	val_213
+217	val_217	val_217
+217	val_217	val_217
+217	val_217	val_217
+217	val_217	val_217
+219	val_219	val_219
+219	val_219	val_219
+219	val_219	val_219
+219	val_219	val_219
 222	val_222	val_222
+224	val_224	val_224
+224	val_224	val_224
+224	val_224	val_224
+224	val_224	val_224
 226	val_226	val_226
 228	val_228	val_228
+233	val_233	val_233
+233	val_233	val_233
+233	val_233	val_233
+233	val_233	val_233
 235	val_235	val_235
+237	val_237	val_237
+237	val_237	val_237
+237	val_237	val_237
+237	val_237	val_237
+239	val_239	val_239
+239	val_239	val_239
+239	val_239	val_239
+239	val_239	val_239
+24	val_24	val_24
+24	val_24	val_24
+24	val_24	val_24
+24	val_24	val_24
+242	val_242	val_242
+242	val_242	val_242
+242	val_242	val_242
+242	val_242	val_242
 244	val_244	val_244
 248	val_248	val_248
+255	val_255	val_255
+255	val_255	val_255
+255	val_255	val_255
+255	val_255	val_255
 257	val_257	val_257
+26	val_26	val_26
+26	val_26	val_26
+26	val_26	val_26
+26	val_26	val_26
 260	val_260	val_260
 262	val_262	val_262
 266	val_266	val_266
+273	val_273	val_273
+273	val_273	val_273
+273	val_273	val_273
+273	val_273	val_273
+273	val_273	val_273
+273	val_273	val_273
+273	val_273	val_273
+273	val_273	val_273
+273	val_273	val_273
 275	val_275	val_275
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
 28	val_28	val_28
+280	val_280	val_280
+280	val_280	val_280
+280	val_280	val_280
+280	val_280	val_280
+282	val_282	val_282
+282	val_282	val_282
+282	val_282	val_282
+282	val_282	val_282
 284	val_284	val_284
 286	val_286	val_286
+288	val_288	val_288
+288	val_288	val_288
+288	val_288	val_288
+288	val_288	val_288
 291	val_291	val_291
 305	val_305	val_305
+307	val_307	val_307
+307	val_307	val_307
+307	val_307	val_307
+307	val_307	val_307
+309	val_309	val_309
+309	val_309	val_309
+309	val_309	val_309
+309	val_309	val_309
 310	val_310	val_310
+316	val_316	val_316
+316	val_316	val_316
+316	val_316	val_316
+316	val_316	val_316
+316	val_316	val_316
+316	val_316	val_316
+316	val_316	val_316
+316	val_316	val_316
+316	val_316	val_316
+318	val_318	val_318
+318	val_318	val_318
+318	val_318	val_318
+318	val_318	val_318
+318	val_318	val_318
+318	val_318	val_318
+318	val_318	val_318
+318	val_318	val_318
+318	val_318	val_318
+321	val_321	val_321
+321	val_321	val_321
+321	val_321	val_321
+321	val_321	val_321
 323	val_323	val_323
+325	val_325	val_325
+325	val_325	val_325
+325	val_325	val_325
+325	val_325	val_325
+327	val_327	val_327
+327	val_327	val_327
+327	val_327	val_327
+327	val_327	val_327
+327	val_327	val_327
+327	val_327	val_327
+327	val_327	val_327
+327	val_327	val_327
+327	val_327	val_327
 33	val_33	val_33
 332	val_332	val_332
 336	val_336	val_336
 338	val_338	val_338
 341	val_341	val_341
 345	val_345	val_345
+35	val_35	val_35
+35	val_35	val_35
+35	val_35	val_35
+35	val_35	val_35
+35	val_35	val_35
+35	val_35	val_35
+35	val_35	val_35
+35	val_35	val_35
+35	val_35	val_35
 356	val_356	val_356
 365	val_365	val_365
+367	val_367	val_367
+367	val_367	val_367
+367	val_367	val_367
+367	val_367	val_367
+369	val_369	val_369
+369	val_369	val_369
+369	val_369	val_369
+369	val_369	val_369
+369	val_369	val_369
+369	val_369	val_369
+369	val_369	val_369
+369	val_369	val_369
+369	val_369	val_369
+37	val_37	val_37
+37	val_37	val_37
+37	val_37	val_37
+37	val_37	val_37
 374	val_374	val_374
 378	val_378	val_378
 389	val_389	val_389
 392	val_392	val_392
 394	val_394	val_394
+396	val_396	val_396
+396	val_396	val_396
+396	val_396	val_396
+396	val_396	val_396
+396	val_396	val_396
+396	val_396	val_396
+396	val_396	val_396
+396	val_396	val_396
+396	val_396	val_396
 4	val_4	val_4
 400	val_400	val_400
 402	val_402	val_402
+404	val_404	val_404
+404	val_404	val_404
+404	val_404	val_404
+404	val_404	val_404
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
 411	val_411	val_411
+413	val_413	val_413
+413	val_413	val_413
+413	val_413	val_413
+413	val_413	val_413
+417	val_417	val_417
+417	val_417	val_417
+417	val_417	val_417
+417	val_417	val_417
+417	val_417	val_417
+417	val_417	val_417
+417	val_417	val_417
+417	val_417	val_417
+417	val_417	val_417
 419	val_419	val_419
+42	val_42	val_42
+42	val_42	val_42
+42	val_42	val_42
+42	val_42	val_42
+424	val_424	val_424
+424	val_424	val_424
+424	val_424	val_424
+424	val_424	val_424
+431	val_431	val_431
+431	val_431	val_431
+431	val_431	val_431
+431	val_431	val_431
+431	val_431	val_431
+431	val_431	val_431
+431	val_431	val_431
+431	val_431	val_431
+431	val_431	val_431
 435	val_435	val_435
 437	val_437	val_437
+439	val_439	val_439
+439	val_439	val_439
+439	val_439	val_439
+439	val_439	val_439
 44	val_44	val_44
 444	val_444	val_444
 446	val_446	val_446
@@ -6191,16 +6133,62 @@ POSTHOOK: Input: default@tab_part_n9@ds=2008-04-08
 453	val_453	val_453
 455	val_455	val_455
 457	val_457	val_457
+459	val_459	val_459
+459	val_459	val_459
+459	val_459	val_459
+459	val_459	val_459
 460	val_460	val_460
+462	val_462	val_462
+462	val_462	val_462
+462	val_462	val_462
+462	val_462	val_462
+466	val_466	val_466
+466	val_466	val_466
+466	val_466	val_466
+466	val_466	val_466
+466	val_466	val_466
+466	val_466	val_466
+466	val_466	val_466
+466	val_466	val_466
+466	val_466	val_466
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
 475	val_475	val_475
 477	val_477	val_477
 479	val_479	val_479
+480	val_480	val_480
+480	val_480	val_480
+480	val_480	val_480
+480	val_480	val_480
+480	val_480	val_480
+480	val_480	val_480
+480	val_480	val_480
+480	val_480	val_480
+480	val_480	val_480
 482	val_482	val_482
 484	val_484	val_484
 491	val_491	val_491
 493	val_493	val_493
 495	val_495	val_495
 497	val_497	val_497
+51	val_51	val_51
+51	val_51	val_51
+51	val_51	val_51
+51	val_51	val_51
 53	val_53	val_53
 57	val_57	val_57
 64	val_64	val_64
@@ -6209,7 +6197,19 @@ POSTHOOK: Input: default@tab_part_n9@ds=2008-04-08
 8	val_8	val_8
 80	val_80	val_80
 82	val_82	val_82
+84	val_84	val_84
+84	val_84	val_84
+84	val_84	val_84
+84	val_84	val_84
 86	val_86	val_86
+95	val_95	val_95
+95	val_95	val_95
+95	val_95	val_95
+95	val_95	val_95
+97	val_97	val_97
+97	val_97	val_97
+97	val_97	val_97
+97	val_97	val_97
 PREHOOK: query: CREATE TABLE tab2_n4(key int, value string) PARTITIONED BY(ds STRING) STORED AS TEXTFILE
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
@@ -6368,404 +6368,120 @@ POSTHOOK: Input: default@tab_part_n9@ds=2008-04-08
 0	val_0	val_0
 0	val_0	val_0
 0	val_0	val_0
-0	val_103	val_103
-0	val_103	val_103
-0	val_103	val_103
-0	val_103	val_103
-0	val_118	val_118
-0	val_118	val_118
-0	val_118	val_118
-0	val_118	val_118
-0	val_125	val_125
-0	val_125	val_125
-0	val_125	val_125
-0	val_125	val_125
-0	val_129	val_129
-0	val_129	val_129
-0	val_129	val_129
-0	val_129	val_129
-0	val_134	val_134
-0	val_134	val_134
-0	val_134	val_134
-0	val_134	val_134
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_149	val_149
-0	val_149	val_149
-0	val_149	val_149
-0	val_149	val_149
-0	val_15	val_15
-0	val_15	val_15
-0	val_15	val_15
-0	val_15	val_15
-0	val_152	val_152
-0	val_152	val_152
-0	val_152	val_152
-0	val_152	val_152
-0	val_165	val_165
-0	val_165	val_165
-0	val_165	val_165
-0	val_165	val_165
-0	val_167	val_167
-0	val_167	val_167
-0	val_167	val_167
-0	val_167	val_167
-0	val_167	val_167
-0	val_167	val_167
-0	val_167	val_167
-0	val_167	val_167
-0	val_167	val_167
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_172	val_172
-0	val_172	val_172
-0	val_172	val_172
-0	val_172	val_172
-0	val_174	val_174
-0	val_174	val_174
-0	val_174	val_174
-0	val_174	val_174
-0	val_176	val_176
-0	val_176	val_176
-0	val_176	val_176
-0	val_176	val_176
-0	val_187	val_187
-0	val_187	val_187
-0	val_187	val_187
-0	val_187	val_187
-0	val_187	val_187
-0	val_187	val_187
-0	val_187	val_187
-0	val_187	val_187
-0	val_187	val_187
-0	val_200	val_200
-0	val_200	val_200
-0	val_200	val_200
-0	val_200	val_200
-0	val_208	val_208
-0	val_208	val_208
-0	val_208	val_208
-0	val_208	val_208
-0	val_208	val_208
-0	val_208	val_208
-0	val_208	val_208
-0	val_208	val_208
-0	val_208	val_208
-0	val_213	val_213
-0	val_213	val_213
-0	val_213	val_213
-0	val_213	val_213
-0	val_217	val_217
-0	val_217	val_217
-0	val_217	val_217
-0	val_217	val_217
-0	val_219	val_219
-0	val_219	val_219
-0	val_219	val_219
-0	val_219	val_219
-0	val_224	val_224
-0	val_224	val_224
-0	val_224	val_224
-0	val_224	val_224
-0	val_233	val_233
-0	val_233	val_233
-0	val_233	val_233
-0	val_233	val_233
-0	val_237	val_237
-0	val_237	val_237
-0	val_237	val_237
-0	val_237	val_237
-0	val_239	val_239
-0	val_239	val_239
-0	val_239	val_239
-0	val_239	val_239
-0	val_24	val_24
-0	val_24	val_24
-0	val_24	val_24
-0	val_24	val_24
-0	val_242	val_242
-0	val_242	val_242
-0	val_242	val_242
-0	val_242	val_242
-0	val_255	val_255
-0	val_255	val_255
-0	val_255	val_255
-0	val_255	val_255
-0	val_26	val_26
-0	val_26	val_26
-0	val_26	val_26
-0	val_26	val_26
-0	val_273	val_273
-0	val_273	val_273
-0	val_273	val_273
-0	val_273	val_273
-0	val_273	val_273
-0	val_273	val_273
-0	val_273	val_273
-0	val_273	val_273
-0	val_273	val_273
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_280	val_280
-0	val_280	val_280
-0	val_280	val_280
-0	val_280	val_280
-0	val_282	val_282
-0	val_282	val_282
-0	val_282	val_282
-0	val_282	val_282
-0	val_288	val_288
-0	val_288	val_288
-0	val_288	val_288
-0	val_288	val_288
-0	val_307	val_307
-0	val_307	val_307
-0	val_307	val_307
-0	val_307	val_307
-0	val_309	val_309
-0	val_309	val_309
-0	val_309	val_309
-0	val_309	val_309
-0	val_316	val_316
-0	val_316	val_316
-0	val_316	val_316
-0	val_316	val_316
-0	val_316	val_316
-0	val_316	val_316
-0	val_316	val_316
-0	val_316	val_316
-0	val_316	val_316
-0	val_318	val_318
-0	val_318	val_318
-0	val_318	val_318
-0	val_318	val_318
-0	val_318	val_318
-0	val_318	val_318
-0	val_318	val_318
-0	val_318	val_318
-0	val_318	val_318
-0	val_321	val_321
-0	val_321	val_321
-0	val_321	val_321
-0	val_321	val_321
-0	val_325	val_325
-0	val_325	val_325
-0	val_325	val_325
-0	val_325	val_325
-0	val_327	val_327
-0	val_327	val_327
-0	val_327	val_327
-0	val_327	val_327
-0	val_327	val_327
-0	val_327	val_327
-0	val_327	val_327
-0	val_327	val_327
-0	val_327	val_327
-0	val_35	val_35
-0	val_35	val_35
-0	val_35	val_35
-0	val_35	val_35
-0	val_35	val_35
-0	val_35	val_35
-0	val_35	val_35
-0	val_35	val_35
-0	val_35	val_35
-0	val_367	val_367
-0	val_367	val_367
-0	val_367	val_367
-0	val_367	val_367
-0	val_369	val_369
-0	val_369	val_369
-0	val_369	val_369
-0	val_369	val_369
-0	val_369	val_369
-0	val_369	val_369
-0	val_369	val_369
-0	val_369	val_369
-0	val_369	val_369
-0	val_37	val_37
-0	val_37	val_37
-0	val_37	val_37
-0	val_37	val_37
-0	val_396	val_396
-0	val_396	val_396
-0	val_396	val_396
-0	val_396	val_396
-0	val_396	val_396
-0	val_396	val_396
-0	val_396	val_396
-0	val_396	val_396
-0	val_396	val_396
-0	val_404	val_404
-0	val_404	val_404
-0	val_404	val_404
-0	val_404	val_404
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_413	val_413
-0	val_413	val_413
-0	val_413	val_413
-0	val_413	val_413
-0	val_417	val_417
-0	val_417	val_417
-0	val_417	val_417
-0	val_417	val_417
-0	val_417	val_417
-0	val_417	val_417
-0	val_417	val_417
-0	val_417	val_417
-0	val_417	val_417
-0	val_42	val_42
-0	val_42	val_42
-0	val_42	val_42
-0	val_42	val_42
-0	val_424	val_424
-0	val_424	val_424
-0	val_424	val_424
-0	val_424	val_424
-0	val_431	val_431
-0	val_431	val_431
-0	val_431	val_431
-0	val_431	val_431
-0	val_431	val_431
-0	val_431	val_431
-0	val_431	val_431
-0	val_431	val_431
-0	val_431	val_431
-0	val_439	val_439
-0	val_439	val_439
-0	val_439	val_439
-0	val_439	val_439
-0	val_459	val_459
-0	val_459	val_459
-0	val_459	val_459
-0	val_459	val_459
-0	val_462	val_462
-0	val_462	val_462
-0	val_462	val_462
-0	val_462	val_462
-0	val_466	val_466
-0	val_466	val_466
-0	val_466	val_466
-0	val_466	val_466
-0	val_466	val_466
-0	val_466	val_466
-0	val_466	val_466
-0	val_466	val_466
-0	val_466	val_466
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_480	val_480
-0	val_480	val_480
-0	val_480	val_480
-0	val_480	val_480
-0	val_480	val_480
-0	val_480	val_480
-0	val_480	val_480
-0	val_480	val_480
-0	val_480	val_480
-0	val_51	val_51
-0	val_51	val_51
-0	val_51	val_51
-0	val_51	val_51
-0	val_84	val_84
-0	val_84	val_84
-0	val_84	val_84
-0	val_84	val_84
-0	val_95	val_95
-0	val_95	val_95
-0	val_95	val_95
-0	val_95	val_95
-0	val_97	val_97
-0	val_97	val_97
-0	val_97	val_97
-0	val_97	val_97
+103	val_103	val_103
+103	val_103	val_103
+103	val_103	val_103
+103	val_103	val_103
 105	val_105	val_105
 11	val_11	val_11
 114	val_114	val_114
 116	val_116	val_116
+118	val_118	val_118
+118	val_118	val_118
+118	val_118	val_118
+118	val_118	val_118
+125	val_125	val_125
+125	val_125	val_125
+125	val_125	val_125
+125	val_125	val_125
+129	val_129	val_129
+129	val_129	val_129
+129	val_129	val_129
+129	val_129	val_129
+134	val_134	val_134
+134	val_134	val_134
+134	val_134	val_134
+134	val_134	val_134
 136	val_136	val_136
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
 143	val_143	val_143
 145	val_145	val_145
+149	val_149	val_149
+149	val_149	val_149
+149	val_149	val_149
+149	val_149	val_149
+15	val_15	val_15
+15	val_15	val_15
+15	val_15	val_15
+15	val_15	val_15
 150	val_150	val_150
+152	val_152	val_152
+152	val_152	val_152
+152	val_152	val_152
+152	val_152	val_152
 156	val_156	val_156
 158	val_158	val_158
 163	val_163	val_163
+165	val_165	val_165
+165	val_165	val_165
+165	val_165	val_165
+165	val_165	val_165
+167	val_167	val_167
+167	val_167	val_167
+167	val_167	val_167
+167	val_167	val_167
+167	val_167	val_167
+167	val_167	val_167
+167	val_167	val_167
+167	val_167	val_167
+167	val_167	val_167
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
 17	val_17	val_17
 170	val_170	val_170
+172	val_172	val_172
+172	val_172	val_172
+172	val_172	val_172
+172	val_172	val_172
+174	val_174	val_174
+174	val_174	val_174
+174	val_174	val_174
+174	val_174	val_174
+176	val_176	val_176
+176	val_176	val_176
+176	val_176	val_176
+176	val_176	val_176
 178	val_178	val_178
 181	val_181	val_181
 183	val_183	val_183
+187	val_187	val_187
+187	val_187	val_187
+187	val_187	val_187
+187	val_187	val_187
+187	val_187	val_187
+187	val_187	val_187
+187	val_187	val_187
+187	val_187	val_187
+187	val_187	val_187
 189	val_189	val_189
 19	val_19	val_19
 190	val_190	val_190
@@ -6774,45 +6490,271 @@ POSTHOOK: Input: default@tab_part_n9@ds=2008-04-08
 196	val_196	val_196
 2	val_2	val_2
 20	val_20	val_20
+200	val_200	val_200
+200	val_200	val_200
+200	val_200	val_200
+200	val_200	val_200
 202	val_202	val_202
+208	val_208	val_208
+208	val_208	val_208
+208	val_208	val_208
+208	val_208	val_208
+208	val_208	val_208
+208	val_208	val_208
+208	val_208	val_208
+208	val_208	val_208
+208	val_208	val_208
+213	val_213	val_213
+213	val_213	val_213
+213	val_213	val_213
+213	val_213	val_213
+217	val_217	val_217
+217	val_217	val_217
+217	val_217	val_217
+217	val_217	val_217
+219	val_219	val_219
+219	val_219	val_219
+219	val_219	val_219
+219	val_219	val_219
 222	val_222	val_222
+224	val_224	val_224
+224	val_224	val_224
+224	val_224	val_224
+224	val_224	val_224
 226	val_226	val_226
 228	val_228	val_228
+233	val_233	val_233
+233	val_233	val_233
+233	val_233	val_233
+233	val_233	val_233
 235	val_235	val_235
+237	val_237	val_237
+237	val_237	val_237
+237	val_237	val_237
+237	val_237	val_237
+239	val_239	val_239
+239	val_239	val_239
+239	val_239	val_239
+239	val_239	val_239
+24	val_24	val_24
+24	val_24	val_24
+24	val_24	val_24
+24	val_24	val_24
+242	val_242	val_242
+242	val_242	val_242
+242	val_242	val_242
+242	val_242	val_242
 244	val_244	val_244
 248	val_248	val_248
+255	val_255	val_255
+255	val_255	val_255
+255	val_255	val_255
+255	val_255	val_255
 257	val_257	val_257
+26	val_26	val_26
+26	val_26	val_26
+26	val_26	val_26
+26	val_26	val_26
 260	val_260	val_260
 262	val_262	val_262
 266	val_266	val_266
+273	val_273	val_273
+273	val_273	val_273
+273	val_273	val_273
+273	val_273	val_273
+273	val_273	val_273
+273	val_273	val_273
+273	val_273	val_273
+273	val_273	val_273
+273	val_273	val_273
 275	val_275	val_275
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
 28	val_28	val_28
+280	val_280	val_280
+280	val_280	val_280
+280	val_280	val_280
+280	val_280	val_280
+282	val_282	val_282
+282	val_282	val_282
+282	val_282	val_282
+282	val_282	val_282
 284	val_284	val_284
 286	val_286	val_286
+288	val_288	val_288
+288	val_288	val_288
+288	val_288	val_288
+288	val_288	val_288
 291	val_291	val_291
 305	val_305	val_305
+307	val_307	val_307
+307	val_307	val_307
+307	val_307	val_307
+307	val_307	val_307
+309	val_309	val_309
+309	val_309	val_309
+309	val_309	val_309
+309	val_309	val_309
 310	val_310	val_310
+316	val_316	val_316
+316	val_316	val_316
+316	val_316	val_316
+316	val_316	val_316
+316	val_316	val_316
+316	val_316	val_316
+316	val_316	val_316
+316	val_316	val_316
+316	val_316	val_316
+318	val_318	val_318
+318	val_318	val_318
+318	val_318	val_318
+318	val_318	val_318
+318	val_318	val_318
+318	val_318	val_318
+318	val_318	val_318
+318	val_318	val_318
+318	val_318	val_318
+321	val_321	val_321
+321	val_321	val_321
+321	val_321	val_321
+321	val_321	val_321
 323	val_323	val_323
+325	val_325	val_325
+325	val_325	val_325
+325	val_325	val_325
+325	val_325	val_325
+327	val_327	val_327
+327	val_327	val_327
+327	val_327	val_327
+327	val_327	val_327
+327	val_327	val_327
+327	val_327	val_327
+327	val_327	val_327
+327	val_327	val_327
+327	val_327	val_327
 33	val_33	val_33
 332	val_332	val_332
 336	val_336	val_336
 338	val_338	val_338
 341	val_341	val_341
 345	val_345	val_345
+35	val_35	val_35
+35	val_35	val_35
+35	val_35	val_35
+35	val_35	val_35
+35	val_35	val_35
+35	val_35	val_35
+35	val_35	val_35
+35	val_35	val_35
+35	val_35	val_35
 356	val_356	val_356
 365	val_365	val_365
+367	val_367	val_367
+367	val_367	val_367
+367	val_367	val_367
+367	val_367	val_367
+369	val_369	val_369
+369	val_369	val_369
+369	val_369	val_369
+369	val_369	val_369
+369	val_369	val_369
+369	val_369	val_369
+369	val_369	val_369
+369	val_369	val_369
+369	val_369	val_369
+37	val_37	val_37
+37	val_37	val_37
+37	val_37	val_37
+37	val_37	val_37
 374	val_374	val_374
 378	val_378	val_378
 389	val_389	val_389
 392	val_392	val_392
 394	val_394	val_394
+396	val_396	val_396
+396	val_396	val_396
+396	val_396	val_396
+396	val_396	val_396
+396	val_396	val_396
+396	val_396	val_396
+396	val_396	val_396
+396	val_396	val_396
+396	val_396	val_396
 4	val_4	val_4
 400	val_400	val_400
 402	val_402	val_402
+404	val_404	val_404
+404	val_404	val_404
+404	val_404	val_404
+404	val_404	val_404
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
 411	val_411	val_411
+413	val_413	val_413
+413	val_413	val_413
+413	val_413	val_413
+413	val_413	val_413
+417	val_417	val_417
+417	val_417	val_417
+417	val_417	val_417
+417	val_417	val_417
+417	val_417	val_417
+417	val_417	val_417
+417	val_417	val_417
+417	val_417	val_417
+417	val_417	val_417
 419	val_419	val_419
+42	val_42	val_42
+42	val_42	val_42
+42	val_42	val_42
+42	val_42	val_42
+424	val_424	val_424
+424	val_424	val_424
+424	val_424	val_424
+424	val_424	val_424
+431	val_431	val_431
+431	val_431	val_431
+431	val_431	val_431
+431	val_431	val_431
+431	val_431	val_431
+431	val_431	val_431
+431	val_431	val_431
+431	val_431	val_431
+431	val_431	val_431
 435	val_435	val_435
 437	val_437	val_437
+439	val_439	val_439
+439	val_439	val_439
+439	val_439	val_439
+439	val_439	val_439
 44	val_44	val_44
 444	val_444	val_444
 446	val_446	val_446
@@ -6820,16 +6762,62 @@ POSTHOOK: Input: default@tab_part_n9@ds=2008-04-08
 453	val_453	val_453
 455	val_455	val_455
 457	val_457	val_457
+459	val_459	val_459
+459	val_459	val_459
+459	val_459	val_459
+459	val_459	val_459
 460	val_460	val_460
+462	val_462	val_462
+462	val_462	val_462
+462	val_462	val_462
+462	val_462	val_462
+466	val_466	val_466
+466	val_466	val_466
+466	val_466	val_466
+466	val_466	val_466
+466	val_466	val_466
+466	val_466	val_466
+466	val_466	val_466
+466	val_466	val_466
+466	val_466	val_466
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
 475	val_475	val_475
 477	val_477	val_477
 479	val_479	val_479
+480	val_480	val_480
+480	val_480	val_480
+480	val_480	val_480
+480	val_480	val_480
+480	val_480	val_480
+480	val_480	val_480
+480	val_480	val_480
+480	val_480	val_480
+480	val_480	val_480
 482	val_482	val_482
 484	val_484	val_484
 491	val_491	val_491
 493	val_493	val_493
 495	val_495	val_495
 497	val_497	val_497
+51	val_51	val_51
+51	val_51	val_51
+51	val_51	val_51
+51	val_51	val_51
 53	val_53	val_53
 57	val_57	val_57
 64	val_64	val_64
@@ -6838,7 +6826,19 @@ POSTHOOK: Input: default@tab_part_n9@ds=2008-04-08
 8	val_8	val_8
 80	val_80	val_80
 82	val_82	val_82
+84	val_84	val_84
+84	val_84	val_84
+84	val_84	val_84
+84	val_84	val_84
 86	val_86	val_86
+95	val_95	val_95
+95	val_95	val_95
+95	val_95	val_95
+95	val_95	val_95
+97	val_97	val_97
+97	val_97	val_97
+97	val_97	val_97
+97	val_97	val_97
 PREHOOK: query: explain select a.key, a.value, b.value
         from tab2_n4 a join tab_part_n9 b on a.key = b.key and a.value = b.value
 PREHOOK: type: QUERY
@@ -6961,404 +6961,120 @@ POSTHOOK: Input: default@tab_part_n9@ds=2008-04-08
 0	val_0	val_0
 0	val_0	val_0
 0	val_0	val_0
-0	val_103	val_103
-0	val_103	val_103
-0	val_103	val_103
-0	val_103	val_103
-0	val_118	val_118
-0	val_118	val_118
-0	val_118	val_118
-0	val_118	val_118
-0	val_125	val_125
-0	val_125	val_125
-0	val_125	val_125
-0	val_125	val_125
-0	val_129	val_129
-0	val_129	val_129
-0	val_129	val_129
-0	val_129	val_129
-0	val_134	val_134
-0	val_134	val_134
-0	val_134	val_134
-0	val_134	val_134
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_138	val_138
-0	val_149	val_149
-0	val_149	val_149
-0	val_149	val_149
-0	val_149	val_149
-0	val_15	val_15
-0	val_15	val_15
-0	val_15	val_15
-0	val_15	val_15
-0	val_152	val_152
-0	val_152	val_152
-0	val_152	val_152
-0	val_152	val_152
-0	val_165	val_165
-0	val_165	val_165
-0	val_165	val_165
-0	val_165	val_165
-0	val_167	val_167
-0	val_167	val_167
-0	val_167	val_167
-0	val_167	val_167
-0	val_167	val_167
-0	val_167	val_167
-0	val_167	val_167
-0	val_167	val_167
-0	val_167	val_167
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_169	val_169
-0	val_172	val_172
-0	val_172	val_172
-0	val_172	val_172
-0	val_172	val_172
-0	val_174	val_174
-0	val_174	val_174
-0	val_174	val_174
-0	val_174	val_174
-0	val_176	val_176
-0	val_176	val_176
-0	val_176	val_176
-0	val_176	val_176
-0	val_187	val_187
-0	val_187	val_187
-0	val_187	val_187
-0	val_187	val_187
-0	val_187	val_187
-0	val_187	val_187
-0	val_187	val_187
-0	val_187	val_187
-0	val_187	val_187
-0	val_200	val_200
-0	val_200	val_200
-0	val_200	val_200
-0	val_200	val_200
-0	val_208	val_208
-0	val_208	val_208
-0	val_208	val_208
-0	val_208	val_208
-0	val_208	val_208
-0	val_208	val_208
-0	val_208	val_208
-0	val_208	val_208
-0	val_208	val_208
-0	val_213	val_213
-0	val_213	val_213
-0	val_213	val_213
-0	val_213	val_213
-0	val_217	val_217
-0	val_217	val_217
-0	val_217	val_217
-0	val_217	val_217
-0	val_219	val_219
-0	val_219	val_219
-0	val_219	val_219
-0	val_219	val_219
-0	val_224	val_224
-0	val_224	val_224
-0	val_224	val_224
-0	val_224	val_224
-0	val_233	val_233
-0	val_233	val_233
-0	val_233	val_233
-0	val_233	val_233
-0	val_237	val_237
-0	val_237	val_237
-0	val_237	val_237
-0	val_237	val_237
-0	val_239	val_239
-0	val_239	val_239
-0	val_239	val_239
-0	val_239	val_239
-0	val_24	val_24
-0	val_24	val_24
-0	val_24	val_24
-0	val_24	val_24
-0	val_242	val_242
-0	val_242	val_242
-0	val_242	val_242
-0	val_242	val_242
-0	val_255	val_255
-0	val_255	val_255
-0	val_255	val_255
-0	val_255	val_255
-0	val_26	val_26
-0	val_26	val_26
-0	val_26	val_26
-0	val_26	val_26
-0	val_273	val_273
-0	val_273	val_273
-0	val_273	val_273
-0	val_273	val_273
-0	val_273	val_273
-0	val_273	val_273
-0	val_273	val_273
-0	val_273	val_273
-0	val_273	val_273
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_277	val_277
-0	val_280	val_280
-0	val_280	val_280
-0	val_280	val_280
-0	val_280	val_280
-0	val_282	val_282
-0	val_282	val_282
-0	val_282	val_282
-0	val_282	val_282
-0	val_288	val_288
-0	val_288	val_288
-0	val_288	val_288
-0	val_288	val_288
-0	val_307	val_307
-0	val_307	val_307
-0	val_307	val_307
-0	val_307	val_307
-0	val_309	val_309
-0	val_309	val_309
-0	val_309	val_309
-0	val_309	val_309
-0	val_316	val_316
-0	val_316	val_316
-0	val_316	val_316
-0	val_316	val_316
-0	val_316	val_316
-0	val_316	val_316
-0	val_316	val_316
-0	val_316	val_316
-0	val_316	val_316
-0	val_318	val_318
-0	val_318	val_318
-0	val_318	val_318
-0	val_318	val_318
-0	val_318	val_318
-0	val_318	val_318
-0	val_318	val_318
-0	val_318	val_318
-0	val_318	val_318
-0	val_321	val_321
-0	val_321	val_321
-0	val_321	val_321
-0	val_321	val_321
-0	val_325	val_325
-0	val_325	val_325
-0	val_325	val_325
-0	val_325	val_325
-0	val_327	val_327
-0	val_327	val_327
-0	val_327	val_327
-0	val_327	val_327
-0	val_327	val_327
-0	val_327	val_327
-0	val_327	val_327
-0	val_327	val_327
-0	val_327	val_327
-0	val_35	val_35
-0	val_35	val_35
-0	val_35	val_35
-0	val_35	val_35
-0	val_35	val_35
-0	val_35	val_35
-0	val_35	val_35
-0	val_35	val_35
-0	val_35	val_35
-0	val_367	val_367
-0	val_367	val_367
-0	val_367	val_367
-0	val_367	val_367
-0	val_369	val_369
-0	val_369	val_369
-0	val_369	val_369
-0	val_369	val_369
-0	val_369	val_369
-0	val_369	val_369
-0	val_369	val_369
-0	val_369	val_369
-0	val_369	val_369
-0	val_37	val_37
-0	val_37	val_37
-0	val_37	val_37
-0	val_37	val_37
-0	val_396	val_396
-0	val_396	val_396
-0	val_396	val_396
-0	val_396	val_396
-0	val_396	val_396
-0	val_396	val_396
-0	val_396	val_396
-0	val_396	val_396
-0	val_396	val_396
-0	val_404	val_404
-0	val_404	val_404
-0	val_404	val_404
-0	val_404	val_404
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_406	val_406
-0	val_413	val_413
-0	val_413	val_413
-0	val_413	val_413
-0	val_413	val_413
-0	val_417	val_417
-0	val_417	val_417
-0	val_417	val_417
-0	val_417	val_417
-0	val_417	val_417
-0	val_417	val_417
-0	val_417	val_417
-0	val_417	val_417
-0	val_417	val_417
-0	val_42	val_42
-0	val_42	val_42
-0	val_42	val_42
-0	val_42	val_42
-0	val_424	val_424
-0	val_424	val_424
-0	val_424	val_424
-0	val_424	val_424
-0	val_431	val_431
-0	val_431	val_431
-0	val_431	val_431
-0	val_431	val_431
-0	val_431	val_431
-0	val_431	val_431
-0	val_431	val_431
-0	val_431	val_431
-0	val_431	val_431
-0	val_439	val_439
-0	val_439	val_439
-0	val_439	val_439
-0	val_439	val_439
-0	val_459	val_459
-0	val_459	val_459
-0	val_459	val_459
-0	val_459	val_459
-0	val_462	val_462
-0	val_462	val_462
-0	val_462	val_462
-0	val_462	val_462
-0	val_466	val_466
-0	val_466	val_466
-0	val_466	val_466
-0	val_466	val_466
-0	val_466	val_466
-0	val_466	val_466
-0	val_466	val_466
-0	val_466	val_466
-0	val_466	val_466
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_468	val_468
-0	val_480	val_480
-0	val_480	val_480
-0	val_480	val_480
-0	val_480	val_480
-0	val_480	val_480
-0	val_480	val_480
-0	val_480	val_480
-0	val_480	val_480
-0	val_480	val_480
-0	val_51	val_51
-0	val_51	val_51
-0	val_51	val_51
-0	val_51	val_51
-0	val_84	val_84
-0	val_84	val_84
-0	val_84	val_84
-0	val_84	val_84
-0	val_95	val_95
-0	val_95	val_95
-0	val_95	val_95
-0	val_95	val_95
-0	val_97	val_97
-0	val_97	val_97
-0	val_97	val_97
-0	val_97	val_97
+103	val_103	val_103
+103	val_103	val_103
+103	val_103	val_103
+103	val_103	val_103
 105	val_105	val_105
 11	val_11	val_11
 114	val_114	val_114
 116	val_116	val_116
+118	val_118	val_118
+118	val_118	val_118
+118	val_118	val_118
+118	val_118	val_118
+125	val_125	val_125
+125	val_125	val_125
+125	val_125	val_125
+125	val_125	val_125
+129	val_129	val_129
+129	val_129	val_129
+129	val_129	val_129
+129	val_129	val_129
+134	val_134	val_134
+134	val_134	val_134
+134	val_134	val_134
+134	val_134	val_134
 136	val_136	val_136
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
+138	val_138	val_138
 143	val_143	val_143
 145	val_145	val_145
+149	val_149	val_149
+149	val_149	val_149
+149	val_149	val_149
+149	val_149	val_149
+15	val_15	val_15
+15	val_15	val_15
+15	val_15	val_15
+15	val_15	val_15
 150	val_150	val_150
+152	val_152	val_152
+152	val_152	val_152
+152	val_152	val_152
+152	val_152	val_152
 156	val_156	val_156
 158	val_158	val_158
 163	val_163	val_163
+165	val_165	val_165
+165	val_165	val_165
+165	val_165	val_165
+165	val_165	val_165
+167	val_167	val_167
+167	val_167	val_167
+167	val_167	val_167
+167	val_167	val_167
+167	val_167	val_167
+167	val_167	val_167
+167	val_167	val_167
+167	val_167	val_167
+167	val_167	val_167
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
+169	val_169	val_169
 17	val_17	val_17
 170	val_170	val_170
+172	val_172	val_172
+172	val_172	val_172
+172	val_172	val_172
+172	val_172	val_172
+174	val_174	val_174
+174	val_174	val_174
+174	val_174	val_174
+174	val_174	val_174
+176	val_176	val_176
+176	val_176	val_176
+176	val_176	val_176
+176	val_176	val_176
 178	val_178	val_178
 181	val_181	val_181
 183	val_183	val_183
+187	val_187	val_187
+187	val_187	val_187
+187	val_187	val_187
+187	val_187	val_187
+187	val_187	val_187
+187	val_187	val_187
+187	val_187	val_187
+187	val_187	val_187
+187	val_187	val_187
 189	val_189	val_189
 19	val_19	val_19
 190	val_190	val_190
@@ -7367,45 +7083,271 @@ POSTHOOK: Input: default@tab_part_n9@ds=2008-04-08
 196	val_196	val_196
 2	val_2	val_2
 20	val_20	val_20
+200	val_200	val_200
+200	val_200	val_200
+200	val_200	val_200
+200	val_200	val_200
 202	val_202	val_202
+208	val_208	val_208
+208	val_208	val_208
+208	val_208	val_208
+208	val_208	val_208
+208	val_208	val_208
+208	val_208	val_208
+208	val_208	val_208
+208	val_208	val_208
+208	val_208	val_208
+213	val_213	val_213
+213	val_213	val_213
+213	val_213	val_213
+213	val_213	val_213
+217	val_217	val_217
+217	val_217	val_217
+217	val_217	val_217
+217	val_217	val_217
+219	val_219	val_219
+219	val_219	val_219
+219	val_219	val_219
+219	val_219	val_219
 222	val_222	val_222
+224	val_224	val_224
+224	val_224	val_224
+224	val_224	val_224
+224	val_224	val_224
 226	val_226	val_226
 228	val_228	val_228
+233	val_233	val_233
+233	val_233	val_233
+233	val_233	val_233
+233	val_233	val_233
 235	val_235	val_235
+237	val_237	val_237
+237	val_237	val_237
+237	val_237	val_237
+237	val_237	val_237
+239	val_239	val_239
+239	val_239	val_239
+239	val_239	val_239
+239	val_239	val_239
+24	val_24	val_24
+24	val_24	val_24
+24	val_24	val_24
+24	val_24	val_24
+242	val_242	val_242
+242	val_242	val_242
+242	val_242	val_242
+242	val_242	val_242
 244	val_244	val_244
 248	val_248	val_248
+255	val_255	val_255
+255	val_255	val_255
+255	val_255	val_255
+255	val_255	val_255
 257	val_257	val_257
+26	val_26	val_26
+26	val_26	val_26
+26	val_26	val_26
+26	val_26	val_26
 260	val_260	val_260
 262	val_262	val_262
 266	val_266	val_266
+273	val_273	val_273
+273	val_273	val_273
+273	val_273	val_273
+273	val_273	val_273
+273	val_273	val_273
+273	val_273	val_273
+273	val_273	val_273
+273	val_273	val_273
+273	val_273	val_273
 275	val_275	val_275
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
+277	val_277	val_277
 28	val_28	val_28
+280	val_280	val_280
+280	val_280	val_280
+280	val_280	val_280
+280	val_280	val_280
+282	val_282	val_282
+282	val_282	val_282
+282	val_282	val_282
+282	val_282	val_282
 284	val_284	val_284
 286	val_286	val_286
+288	val_288	val_288
+288	val_288	val_288
+288	val_288	val_288
+288	val_288	val_288
 291	val_291	val_291
 305	val_305	val_305
+307	val_307	val_307
+307	val_307	val_307
+307	val_307	val_307
+307	val_307	val_307
+309	val_309	val_309
+309	val_309	val_309
+309	val_309	val_309
+309	val_309	val_309
 310	val_310	val_310
+316	val_316	val_316
+316	val_316	val_316
+316	val_316	val_316
+316	val_316	val_316
+316	val_316	val_316
+316	val_316	val_316
+316	val_316	val_316
+316	val_316	val_316
+316	val_316	val_316
+318	val_318	val_318
+318	val_318	val_318
+318	val_318	val_318
+318	val_318	val_318
+318	val_318	val_318
+318	val_318	val_318
+318	val_318	val_318
+318	val_318	val_318
+318	val_318	val_318
+321	val_321	val_321
+321	val_321	val_321
+321	val_321	val_321
+321	val_321	val_321
 323	val_323	val_323
+325	val_325	val_325
+325	val_325	val_325
+325	val_325	val_325
+325	val_325	val_325
+327	val_327	val_327
+327	val_327	val_327
+327	val_327	val_327
+327	val_327	val_327
+327	val_327	val_327
+327	val_327	val_327
+327	val_327	val_327
+327	val_327	val_327
+327	val_327	val_327
 33	val_33	val_33
 332	val_332	val_332
 336	val_336	val_336
 338	val_338	val_338
 341	val_341	val_341
 345	val_345	val_345
+35	val_35	val_35
+35	val_35	val_35
+35	val_35	val_35
+35	val_35	val_35
+35	val_35	val_35
+35	val_35	val_35
+35	val_35	val_35
+35	val_35	val_35
+35	val_35	val_35
 356	val_356	val_356
 365	val_365	val_365
+367	val_367	val_367
+367	val_367	val_367
+367	val_367	val_367
+367	val_367	val_367
+369	val_369	val_369
+369	val_369	val_369
+369	val_369	val_369
+369	val_369	val_369
+369	val_369	val_369
+369	val_369	val_369
+369	val_369	val_369
+369	val_369	val_369
+369	val_369	val_369
+37	val_37	val_37
+37	val_37	val_37
+37	val_37	val_37
+37	val_37	val_37
 374	val_374	val_374
 378	val_378	val_378
 389	val_389	val_389
 392	val_392	val_392
 394	val_394	val_394
+396	val_396	val_396
+396	val_396	val_396
+396	val_396	val_396
+396	val_396	val_396
+396	val_396	val_396
+396	val_396	val_396
+396	val_396	val_396
+396	val_396	val_396
+396	val_396	val_396
 4	val_4	val_4
 400	val_400	val_400
 402	val_402	val_402
+404	val_404	val_404
+404	val_404	val_404
+404	val_404	val_404
+404	val_404	val_404
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
+406	val_406	val_406
 411	val_411	val_411
+413	val_413	val_413
+413	val_413	val_413
+413	val_413	val_413
+413	val_413	val_413
+417	val_417	val_417
+417	val_417	val_417
+417	val_417	val_417
+417	val_417	val_417
+417	val_417	val_417
+417	val_417	val_417
+417	val_417	val_417
+417	val_417	val_417
+417	val_417	val_417
 419	val_419	val_419
+42	val_42	val_42
+42	val_42	val_42
+42	val_42	val_42
+42	val_42	val_42
+424	val_424	val_424
+424	val_424	val_424
+424	val_424	val_424
+424	val_424	val_424
+431	val_431	val_431
+431	val_431	val_431
+431	val_431	val_431
+431	val_431	val_431
+431	val_431	val_431
+431	val_431	val_431
+431	val_431	val_431
+431	val_431	val_431
+431	val_431	val_431
 435	val_435	val_435
 437	val_437	val_437
+439	val_439	val_439
+439	val_439	val_439
+439	val_439	val_439
+439	val_439	val_439
 44	val_44	val_44
 444	val_444	val_444
 446	val_446	val_446
@@ -7413,16 +7355,62 @@ POSTHOOK: Input: default@tab_part_n9@ds=2008-04-08
 453	val_453	val_453
 455	val_455	val_455
 457	val_457	val_457
+459	val_459	val_459
+459	val_459	val_459
+459	val_459	val_459
+459	val_459	val_459
 460	val_460	val_460
+462	val_462	val_462
+462	val_462	val_462
+462	val_462	val_462
+462	val_462	val_462
+466	val_466	val_466
+466	val_466	val_466
+466	val_466	val_466
+466	val_466	val_466
+466	val_466	val_466
+466	val_466	val_466
+466	val_466	val_466
+466	val_466	val_466
+466	val_466	val_466
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
+468	val_468	val_468
 475	val_475	val_475
 477	val_477	val_477
 479	val_479	val_479
+480	val_480	val_480
+480	val_480	val_480
+480	val_480	val_480
+480	val_480	val_480
+480	val_480	val_480
+480	val_480	val_480
+480	val_480	val_480
+480	val_480	val_480
+480	val_480	val_480
 482	val_482	val_482
 484	val_484	val_484
 491	val_491	val_491
 493	val_493	val_493
 495	val_495	val_495
 497	val_497	val_497
+51	val_51	val_51
+51	val_51	val_51
+51	val_51	val_51
+51	val_51	val_51
 53	val_53	val_53
 57	val_57	val_57
 64	val_64	val_64
@@ -7431,4 +7419,16 @@ POSTHOOK: Input: default@tab_part_n9@ds=2008-04-08
 8	val_8	val_8
 80	val_80	val_80
 82	val_82	val_82
+84	val_84	val_84
+84	val_84	val_84
+84	val_84	val_84
+84	val_84	val_84
 86	val_86	val_86
+95	val_95	val_95
+95	val_95	val_95
+95	val_95	val_95
+95	val_95	val_95
+97	val_97	val_97
+97	val_97	val_97
+97	val_97	val_97
+97	val_97	val_97
diff --git a/ql/src/test/results/clientpositive/tez/hybridgrace_hashjoin_2.q.out b/ql/src/test/results/clientpositive/tez/hybridgrace_hashjoin_2.q.out
index 8e9bd05..d5f2e1e 100644
--- a/ql/src/test/results/clientpositive/tez/hybridgrace_hashjoin_2.q.out
+++ b/ql/src/test/results/clientpositive/tez/hybridgrace_hashjoin_2.q.out
@@ -1,21 +1,21 @@
-PREHOOK: query: SELECT 1
+PREHOOK: query: SELECT 'Test n-way join'
 PREHOOK: type: QUERY
 PREHOOK: Input: _dummy_database@_dummy_table
 PREHOOK: Output: hdfs://### HDFS PATH ###
-POSTHOOK: query: SELECT 1
+POSTHOOK: query: SELECT 'Test n-way join'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: _dummy_database@_dummy_table
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-1
-PREHOOK: query: SELECT 1
+Test n-way join
+PREHOOK: query: SELECT '3-way mapjoin (1 big table, 2 small tables)'
 PREHOOK: type: QUERY
 PREHOOK: Input: _dummy_database@_dummy_table
 PREHOOK: Output: hdfs://### HDFS PATH ###
-POSTHOOK: query: SELECT 1
+POSTHOOK: query: SELECT '3-way mapjoin (1 big table, 2 small tables)'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: _dummy_database@_dummy_table
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-1
+3-way mapjoin (1 big table, 2 small tables)
 PREHOOK: query: EXPLAIN
 SELECT COUNT(*)
 FROM src1 x JOIN srcpart z ON (x.key = z.key)
@@ -305,6 +305,191 @@ STAGE PLANS:
 
 PREHOOK: query: SELECT COUNT(*)
 FROM src1 x JOIN srcpart z ON (x.key = z.key)
+JOIN srcpart w ON (x.key = w.key)
+JOIN src y ON (y.key = x.key)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Input: default@src1
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: SELECT COUNT(*)
+FROM src1 x JOIN srcpart z ON (x.key = z.key)
+JOIN srcpart w ON (x.key = w.key)
+JOIN src y ON (y.key = x.key)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@src1
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+PREHOOK: query: EXPLAIN  ANALYZE SELECT COUNT(*)
+FROM src1 x JOIN srcpart z ON (x.key = z.key)
+JOIN srcpart w ON (x.key = w.key)
+JOIN src y ON (y.key = x.key)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Input: default@src1
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: EXPLAIN  ANALYZE SELECT COUNT(*)
+FROM src1 x JOIN srcpart z ON (x.key = z.key)
+JOIN srcpart w ON (x.key = w.key)
+JOIN src y ON (y.key = x.key)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@src1
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 2 <- Map 1 (BROADCAST_EDGE)
+        Map 3 <- Map 2 (BROADCAST_EDGE), Map 5 (BROADCAST_EDGE)
+        Reducer 4 <- Map 3 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: x
+                  filterExpr: key is not null (type: boolean)
+                  Statistics: Num rows: 25/25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 25/25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      key expressions: key (type: string)
+                      sort order: +
+                      Map-reduce partition columns: key (type: string)
+                      Statistics: Num rows: 25/25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized
+        Map 2 
+            Map Operator Tree:
+                TableScan
+                  alias: z
+                  filterExpr: key is not null (type: boolean)
+                  Statistics: Num rows: 2000/2000 Data size: 174000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 2000/2000 Data size: 174000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Map Join Operator
+                      condition map:
+                           Inner Join 0 to 1
+                      keys:
+                        0 key (type: string)
+                        1 key (type: string)
+                      outputColumnNames: _col0
+                      input vertices:
+                        0 Map 1
+                      Statistics: Num rows: 158/148 Data size: 13588 Basic stats: COMPLETE Column stats: COMPLETE
+                      HybridGraceHashJoin: true
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 158/148 Data size: 13588 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized
+        Map 3 
+            Map Operator Tree:
+                TableScan
+                  alias: w
+                  filterExpr: key is not null (type: boolean)
+                  Statistics: Num rows: 2000/2000 Data size: 174000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 2000/2000 Data size: 174000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Map Join Operator
+                      condition map:
+                           Inner Join 0 to 1
+                      keys:
+                        0 _col0 (type: string)
+                        1 key (type: string)
+                      outputColumnNames: _col0
+                      input vertices:
+                        0 Map 2
+                      Statistics: Num rows: 1000/1712 Data size: 86000 Basic stats: COMPLETE Column stats: COMPLETE
+                      HybridGraceHashJoin: true
+                      Map Join Operator
+                        condition map:
+                             Inner Join 0 to 1
+                        keys:
+                          0 _col0 (type: string)
+                          1 key (type: string)
+                        input vertices:
+                          1 Map 5
+                        Statistics: Num rows: 1582/5680 Data size: 12656 Basic stats: COMPLETE Column stats: COMPLETE
+                        HybridGraceHashJoin: true
+                        Group By Operator
+                          aggregations: count()
+                          minReductionHashAggr: 0.99
+                          mode: hash
+                          outputColumnNames: _col0
+                          Statistics: Num rows: 1/1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                          Reduce Output Operator
+                            sort order: 
+                            Statistics: Num rows: 1/1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: bigint)
+            Execution mode: vectorized
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: y
+                  filterExpr: key is not null (type: boolean)
+                  Statistics: Num rows: 500/500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 500/500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      key expressions: key (type: string)
+                      sort order: +
+                      Map-reduce partition columns: key (type: string)
+                      Statistics: Num rows: 500/500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized
+        Reducer 4 
+            Execution mode: vectorized
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1/1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1/1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT COUNT(*)
+FROM src1 x JOIN srcpart z ON (x.key = z.key)
 JOIN src y ON (y.key = x.key)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
@@ -328,15 +513,15 @@ POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
 POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 428
-PREHOOK: query: SELECT 1
+PREHOOK: query: SELECT '4-way mapjoin (1 big table, 3 small tables)'
 PREHOOK: type: QUERY
 PREHOOK: Input: _dummy_database@_dummy_table
 PREHOOK: Output: hdfs://### HDFS PATH ###
-POSTHOOK: query: SELECT 1
+POSTHOOK: query: SELECT '4-way mapjoin (1 big table, 3 small tables)'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: _dummy_database@_dummy_table
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-1
+4-way mapjoin (1 big table, 3 small tables)
 PREHOOK: query: EXPLAIN
 SELECT COUNT(*)
 FROM src1 x JOIN srcpart z ON (x.key = z.key)
@@ -495,7 +680,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-PREHOOK: query: SELECT COUNT(*)
+PREHOOK: query: SELECT assert_true(5680 = COUNT(*))
 FROM src1 x JOIN srcpart z ON (x.key = z.key)
 JOIN srcpart w ON (x.key = w.key)
 JOIN src y ON (y.key = x.key)
@@ -508,7 +693,7 @@ PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
 PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
 PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
 PREHOOK: Output: hdfs://### HDFS PATH ###
-POSTHOOK: query: SELECT COUNT(*)
+POSTHOOK: query: SELECT assert_true(5680 = COUNT(*))
 FROM src1 x JOIN srcpart z ON (x.key = z.key)
 JOIN srcpart w ON (x.key = w.key)
 JOIN src y ON (y.key = x.key)
@@ -521,7 +706,7 @@ POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
 POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
 POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-0
+NULL
 PREHOOK: query: EXPLAIN
 SELECT COUNT(*)
 FROM src1 x JOIN srcpart z ON (x.key = z.key)
@@ -683,7 +868,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-PREHOOK: query: SELECT COUNT(*)
+PREHOOK: query: SELECT assert_true(5680 = COUNT(*))
 FROM src1 x JOIN srcpart z ON (x.key = z.key)
 JOIN srcpart w ON (x.key = w.key)
 JOIN src y ON (y.key = x.key)
@@ -696,7 +881,7 @@ PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
 PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
 PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
 PREHOOK: Output: hdfs://### HDFS PATH ###
-POSTHOOK: query: SELECT COUNT(*)
+POSTHOOK: query: SELECT assert_true(5680 = COUNT(*))
 FROM src1 x JOIN srcpart z ON (x.key = z.key)
 JOIN srcpart w ON (x.key = w.key)
 JOIN src y ON (y.key = x.key)
@@ -709,16 +894,16 @@ POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
 POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
 POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-0
-PREHOOK: query: SELECT 1
+NULL
+PREHOOK: query: SELECT '2 sets of 3-way mapjoin under 2 different tasks'
 PREHOOK: type: QUERY
 PREHOOK: Input: _dummy_database@_dummy_table
 PREHOOK: Output: hdfs://### HDFS PATH ###
-POSTHOOK: query: SELECT 1
+POSTHOOK: query: SELECT '2 sets of 3-way mapjoin under 2 different tasks'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: _dummy_database@_dummy_table
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-1
+2 sets of 3-way mapjoin under 2 different tasks
 PREHOOK: query: EXPLAIN
 SELECT COUNT(*)
 FROM src1 x JOIN srcpart z ON (x.key = z.key)
@@ -1291,15 +1476,15 @@ POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 428
 452
-PREHOOK: query: SELECT 1
+PREHOOK: query: SELECT 'A chain of 2 sets of 3-way mapjoin under the same task'
 PREHOOK: type: QUERY
 PREHOOK: Input: _dummy_database@_dummy_table
 PREHOOK: Output: hdfs://### HDFS PATH ###
-POSTHOOK: query: SELECT 1
+POSTHOOK: query: SELECT 'A chain of 2 sets of 3-way mapjoin under the same task'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: _dummy_database@_dummy_table
 POSTHOOK: Output: hdfs://### HDFS PATH ###
-1
+A chain of 2 sets of 3-way mapjoin under the same task
 PREHOOK: query: EXPLAIN
 SELECT COUNT(*)
 FROM src1 x


[hive] 04/05: HIVE-21965: Implement parallel processing in HiveStrictManagedMigration (Krisztian Kasa via Zoltan Haindrich)

Posted by kg...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kgyrtkirk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git

commit cff3c626e201f1ab6efb2fd58a28fa80ccd83104
Author: Krisztian Kasa <kk...@cloudera.com>
AuthorDate: Mon Jul 15 11:32:50 2019 +0200

    HIVE-21965: Implement parallel processing in HiveStrictManagedMigration (Krisztian Kasa via Zoltan Haindrich)
    
    Signed-off-by: Zoltan Haindrich <ki...@rxd.hu>
---
 .../hadoop/hive/ql/util/CloseableThreadLocal.java  |  59 ++
 .../hive/ql/util/HiveStrictManagedMigration.java   | 629 ++++++++++++++-------
 .../ql/util/NamedForkJoinWorkerThreadFactory.java  |  40 ++
 .../hadoop/hive/ql/TxnCommandsBaseForTests.java    |  16 +-
 .../hive/ql/util/CloseableThreadLocalTest.java     |  86 +++
 .../ql/util/TestHiveStrictManagedMigration.java    |  78 +++
 6 files changed, 683 insertions(+), 225 deletions(-)

diff --git a/ql/src/java/org/apache/hadoop/hive/ql/util/CloseableThreadLocal.java b/ql/src/java/org/apache/hadoop/hive/ql/util/CloseableThreadLocal.java
new file mode 100644
index 0000000..efffd38
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/util/CloseableThreadLocal.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.util;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Supplier;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class has similar functionality as {@link ThreadLocal}.
+ * Plus it provides a close function to clean up resources in all threads where the resource was initialized.
+ * @param <T> - type of resource, it must implement {@link AutoCloseable}
+ */
+public class CloseableThreadLocal<T extends AutoCloseable> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CloseableThreadLocal.class);
+
+  private final ConcurrentHashMap<Thread, T> threadLocalMap;
+  private final Supplier<T> initialValue;
+
+  public CloseableThreadLocal(Supplier<T> initialValue, int poolSize) {
+    this.initialValue = initialValue;
+    threadLocalMap = new ConcurrentHashMap<>(poolSize);
+  }
+
+  public T get() {
+    return threadLocalMap.computeIfAbsent(Thread.currentThread(), thread -> initialValue.get());
+  }
+
+  public void close() {
+    threadLocalMap.values().forEach(this::closeQuietly);
+  }
+
+  private void closeQuietly(AutoCloseable autoCloseable) {
+    try {
+      autoCloseable.close();
+    } catch (Exception e) {
+      LOG.warn("Error while closing resource.", e);
+    }
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/util/HiveStrictManagedMigration.java b/ql/src/java/org/apache/hadoop/hive/ql/util/HiveStrictManagedMigration.java
index 80025b7..42c4158 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/util/HiveStrictManagedMigration.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/util/HiveStrictManagedMigration.java
@@ -19,10 +19,15 @@
 package org.apache.hadoop.hive.ql.util;
 
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ForkJoinPool;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.GnuParser;
@@ -39,21 +44,17 @@ import org.apache.hadoop.hive.common.LogUtils;
 import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.common.cli.CommonCliOptions;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.TableType;
+import org.apache.hadoop.hive.metastore.TransactionalValidationListener;
+import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
 import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
 import org.apache.hadoop.hive.metastore.api.Partition;
-import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.TableValidWriteIds;
-import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
-import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
-import org.apache.hadoop.hive.metastore.IMetaStoreClient;
-import org.apache.hadoop.hive.metastore.TableType;
-import org.apache.hadoop.hive.metastore.TransactionalValidationListener;
-import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.txn.TxnCommonUtils;
 import org.apache.hadoop.hive.metastore.txn.TxnUtils;
 import org.apache.hadoop.hive.metastore.utils.HiveStrictManagedUtils;
@@ -61,8 +62,6 @@ import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.ql.parse.HiveParser.switchDatabaseStatement_return;
-import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.shims.HadoopShims;
 import org.apache.hadoop.hive.shims.ShimLoader;
@@ -71,6 +70,7 @@ import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
 
 public class HiveStrictManagedMigration {
@@ -85,24 +85,28 @@ public class HiveStrictManagedMigration {
     MANAGED    // Migrate tables as managed transactional tables
   }
 
-  static class RunOptions {
-    String dbRegex;
-    String tableRegex;
-    String oldWarehouseRoot;
-    TableMigrationOption migrationOption;
-    boolean shouldModifyManagedTableLocation;
-    boolean shouldModifyManagedTableOwner;
-    boolean shouldModifyManagedTablePermissions;
-    boolean dryRun;
-
-    public RunOptions(String dbRegex,
-        String tableRegex,
-        String oldWarehouseRoot,
-        TableMigrationOption migrationOption,
-        boolean shouldModifyManagedTableLocation,
-        boolean shouldModifyManagedTableOwner,
-        boolean shouldModifyManagedTablePermissions,
-        boolean dryRun) {
+  private static class RunOptions {
+    final String dbRegex;
+    final String tableRegex;
+    final String oldWarehouseRoot;
+    final TableMigrationOption migrationOption;
+    final boolean shouldModifyManagedTableLocation;
+    final boolean shouldModifyManagedTableOwner;
+    final boolean shouldModifyManagedTablePermissions;
+    final boolean dryRun;
+    final TableType tableType;
+    final int tablePoolSize;
+
+    RunOptions(String dbRegex,
+               String tableRegex,
+               String oldWarehouseRoot,
+               TableMigrationOption migrationOption,
+               boolean shouldModifyManagedTableLocation,
+               boolean shouldModifyManagedTableOwner,
+               boolean shouldModifyManagedTablePermissions,
+               boolean dryRun,
+               TableType tableType,
+               int tablePoolSize) {
       super();
       this.dbRegex = dbRegex;
       this.tableRegex = tableRegex;
@@ -112,6 +116,67 @@ public class HiveStrictManagedMigration {
       this.shouldModifyManagedTableOwner = shouldModifyManagedTableOwner;
       this.shouldModifyManagedTablePermissions = shouldModifyManagedTablePermissions;
       this.dryRun = dryRun;
+      this.tableType = tableType;
+      this.tablePoolSize = tablePoolSize;
+    }
+
+    public RunOptions setShouldModifyManagedTableLocation(boolean shouldModifyManagedTableLocation) {
+      return new RunOptions(
+              this.dbRegex,
+              this.tableRegex,
+              this.oldWarehouseRoot,
+              this.migrationOption,
+              shouldModifyManagedTableLocation,
+              this.shouldModifyManagedTableOwner,
+              this.shouldModifyManagedTablePermissions,
+              this.dryRun,
+              this.tableType,
+              this.tablePoolSize);
+    }
+
+    @Override
+    public String toString() {
+      return "RunOptions{" +
+              "dbRegex='" + dbRegex + '\'' +
+              ", tableRegex='" + tableRegex + '\'' +
+              ", oldWarehouseRoot='" + oldWarehouseRoot + '\'' +
+              ", migrationOption=" + migrationOption +
+              ", shouldModifyManagedTableLocation=" + shouldModifyManagedTableLocation +
+              ", shouldModifyManagedTableOwner=" + shouldModifyManagedTableOwner +
+              ", shouldModifyManagedTablePermissions=" + shouldModifyManagedTablePermissions +
+              ", dryRun=" + dryRun +
+              ", tableType=" + tableType +
+              ", tablePoolSize=" + tablePoolSize +
+              '}';
+    }
+  }
+
+  private static class OwnerPermsOptions {
+    final String ownerName;
+    final String groupName;
+    final FsPermission dirPerms;
+    final FsPermission filePerms;
+
+    OwnerPermsOptions(String ownerName, String groupName, FsPermission dirPerms, FsPermission filePerms) {
+      this.ownerName = ownerName;
+      this.groupName = groupName;
+      this.dirPerms = dirPerms;
+      this.filePerms = filePerms;
+    }
+  }
+
+  private static class WarehouseRootCheckResult {
+    final boolean shouldModifyManagedTableLocation;
+    final Path curWhRootPath;
+    final HadoopShims.HdfsEncryptionShim encryptionShim;
+
+    WarehouseRootCheckResult(
+            boolean shouldModifyManagedTableLocation,
+            Path curWhRootPath,
+            HadoopShims.HdfsEncryptionShim encryptionShim) {
+      this.shouldModifyManagedTableLocation = shouldModifyManagedTableLocation;
+      this.curWhRootPath = curWhRootPath;
+      this.encryptionShim = encryptionShim;
     }
   }
 
@@ -136,7 +201,15 @@ public class HiveStrictManagedMigration {
     int rc = 0;
     HiveStrictManagedMigration migration = null;
     try {
-      migration = new HiveStrictManagedMigration(runOptions);
+      HiveConf conf = hiveConf == null ? new HiveConf() : hiveConf;
+      WarehouseRootCheckResult warehouseRootCheckResult = checkOldWarehouseRoot(runOptions, conf);
+      runOptions = runOptions.setShouldModifyManagedTableLocation(
+              warehouseRootCheckResult.shouldModifyManagedTableLocation);
+      boolean createExternalDirsForDbs = checkExternalWarehouseDir(conf);
+      OwnerPermsOptions ownerPermsOptions = checkOwnerPermsOptions(runOptions, conf);
+
+      migration = new HiveStrictManagedMigration(
+              conf, runOptions, createExternalDirsForDbs, ownerPermsOptions, warehouseRootCheckResult);
       migration.run();
     } catch (Exception err) {
       LOG.error("Failed with error", err);
@@ -148,7 +221,9 @@ public class HiveStrictManagedMigration {
     }
 
     // TODO: Something is preventing the process from terminating after main(), adding exit() as hacky solution.
-    System.exit(rc);
+    if (hiveConf == null) {
+      System.exit(rc);
+    }
   }
 
   static Options createOptions() {
@@ -156,66 +231,84 @@ public class HiveStrictManagedMigration {
 
     // -hiveconf x=y
     result.addOption(OptionBuilder
-        .withValueSeparator()
-        .hasArgs(2)
-        .withArgName("property=value")
-        .withLongOpt("hiveconf")
-        .withDescription("Use value for given property")
-        .create());
+            .withValueSeparator()
+            .hasArgs(2)
+            .withArgName("property=value")
+            .withLongOpt("hiveconf")
+            .withDescription("Use value for given property")
+            .create());
 
     result.addOption(OptionBuilder
-        .withLongOpt("dryRun")
-        .withDescription("Show what migration actions would be taken without actually running commands")
-        .create());
+            .withLongOpt("dryRun")
+            .withDescription("Show what migration actions would be taken without actually running commands")
+            .create());
 
     result.addOption(OptionBuilder
-        .withLongOpt("dbRegex")
-        .withDescription("Regular expression to match database names on which this tool will be run")
-        .hasArg()
-        .create('d'));
+            .withLongOpt("dbRegex")
+            .withDescription("Regular expression to match database names on which this tool will be run")
+            .hasArg()
+            .create('d'));
 
     result.addOption(OptionBuilder
-        .withLongOpt("tableRegex")
-        .withDescription("Regular expression to match table names on which this tool will be run")
-        .hasArg()
-        .create('t'));
+            .withLongOpt("tableRegex")
+            .withDescription("Regular expression to match table names on which this tool will be run")
+            .hasArg()
+            .create('t'));
 
     result.addOption(OptionBuilder
-        .withLongOpt("oldWarehouseRoot")
-        .withDescription("Location of the previous warehouse root")
-        .hasArg()
-        .create());
+            .withLongOpt("oldWarehouseRoot")
+            .withDescription("Location of the previous warehouse root")
+            .hasArg()
+            .create());
 
     result.addOption(OptionBuilder
-        .withLongOpt("migrationOption")
-        .withDescription("Table migration option (automatic|external|managed|validate|none)")
-        .hasArg()
-        .create('m'));
+            .withLongOpt("migrationOption")
+            .withDescription("Table migration option (automatic|external|managed|validate|none)")
+            .hasArg()
+            .create('m'));
 
     result.addOption(OptionBuilder
-        .withLongOpt("shouldModifyManagedTableLocation")
-        .withDescription("Whether managed tables should have their data moved from the old warehouse path to the current warehouse path")
-        .create());
+            .withLongOpt("shouldModifyManagedTableLocation")
+            .withDescription("Whether managed tables should have their data moved from " +
+                    "the old warehouse path to the current warehouse path")
+            .create());
 
     result.addOption(OptionBuilder
-        .withLongOpt("shouldModifyManagedTableOwner")
-        .withDescription("Whether managed tables should have their directory owners changed to the hive user")
-        .create());
+            .withLongOpt("shouldModifyManagedTableOwner")
+            .withDescription("Whether managed tables should have their directory owners changed to the hive user")
+            .create());
 
     result.addOption(OptionBuilder
-        .withLongOpt("shouldModifyManagedTablePermissions")
-        .withDescription("Whether managed tables should have their directory permissions changed to conform to strict managed tables mode")
-        .create());
+            .withLongOpt("shouldModifyManagedTablePermissions")
+            .withDescription("Whether managed tables should have their directory permissions changed to conform to " +
+                    "strict managed tables mode")
+            .create());
 
     result.addOption(OptionBuilder
-        .withLongOpt("modifyManagedTables")
-        .withDescription("This setting enables the shouldModifyManagedTableLocation, shouldModifyManagedTableOwner, shouldModifyManagedTablePermissions options")
-        .create());
+            .withLongOpt("modifyManagedTables")
+            .withDescription("This setting enables the shouldModifyManagedTableLocation, " +
+                    "shouldModifyManagedTableOwner, shouldModifyManagedTablePermissions options")
+            .create());
 
     result.addOption(OptionBuilder
-        .withLongOpt("help")
-        .withDescription("print help message")
-        .create('h'));
+            .withLongOpt("help")
+            .withDescription("print help message")
+            .create('h'));
+
+    result.addOption(OptionBuilder
+            .withLongOpt("tablePoolSize")
+            .withDescription("Number of threads to process tables.")
+            .hasArg()
+            .create("tn"));
+
+    result.addOption(OptionBuilder
+            .withLongOpt("tableType")
+            .withDescription(String.format("Table type to match tables on which this tool will be run. " +
+                            "Possible values: %s Default: all tables",
+                    Arrays.stream(TableType.values()).map(Enum::name).collect(Collectors.joining("|"))))
+            .hasArg()
+            .withArgName("table type")
+            .create("tt"));
 
     return result;
   }
@@ -251,6 +344,22 @@ public class HiveStrictManagedMigration {
     String oldWarehouseRoot = cli.getOptionValue("oldWarehouseRoot");
     boolean dryRun = cli.hasOption("dryRun");
 
+    String tableTypeText = cli.getOptionValue("tableType");
+
+    int defaultPoolSize = Runtime.getRuntime().availableProcessors() / 2;
+    if (defaultPoolSize < 1) {
+      defaultPoolSize = 1;
+    }
+
+    int databasePoolSize = getIntOptionValue(cli, "databasePoolSize", defaultPoolSize);
+    if (databasePoolSize < 1) {
+      throw new IllegalArgumentException("Please specify a positive integer option value for databasePoolSize");
+    }
+    int tablePoolSize = getIntOptionValue(cli, "tablePoolSize", defaultPoolSize);
+    if (tablePoolSize < 1) {
+      throw new IllegalArgumentException("Please specify a positive integer option value for tablePoolSize");
+    }
+
     RunOptions runOpts = new RunOptions(
         dbRegex,
         tableRegex,
@@ -259,75 +368,137 @@ public class HiveStrictManagedMigration {
         shouldModifyManagedTableLocation,
         shouldModifyManagedTableOwner,
         shouldModifyManagedTablePermissions,
-        dryRun);
+        dryRun,
+        tableTypeText == null ? null : TableType.valueOf(tableTypeText),
+        tablePoolSize);
     return runOpts;
   }
 
-  private RunOptions runOptions;
-  private HiveConf conf;
-  private HiveMetaStoreClient hms;
-  private boolean failedValidationChecks;
-  private boolean failuresEncountered;
-  private Warehouse wh;
-  private Warehouse oldWh;
-  private String ownerName;
-  private String groupName;
-  private FsPermission dirPerms;
-  private FsPermission filePerms;
-  private boolean createExternalDirsForDbs;
-  Path curWhRootPath;
-  private HadoopShims.HdfsEncryptionShim encryptionShim;
-
-  HiveStrictManagedMigration(RunOptions runOptions) {
-    this.runOptions = runOptions;
-    this.conf = new HiveConf();
+  private static int getIntOptionValue(CommandLine commandLine, String optionName, int defaultValue) {
+    if (commandLine.hasOption(optionName)) {
+      try {
+        return Integer.parseInt(commandLine.getOptionValue(optionName));
+      } catch (NumberFormatException e) {
+        throw new IllegalArgumentException("Please specify a positive integer option value for " + optionName, e);
+      }
+    }
+    return defaultValue;
   }
 
-  void run() throws Exception {
-    wh = new Warehouse(conf);
-    checkOldWarehouseRoot();
-    checkExternalWarehouseDir();
-    checkOwnerPermsOptions();
-
-    hms = new HiveMetaStoreClient(conf);//MetaException
-    try {
-      List<String> databases = hms.getAllDatabases();//TException
-      LOG.info("Found {} databases", databases.size());
-      for (String dbName : databases) {
-        if (dbName.matches(runOptions.dbRegex)) {
-          try {
-            processDatabase(dbName);
-          } catch (Exception err) {
-            LOG.error("Error processing database " + dbName, err);
-            failuresEncountered = true;
-          }
+  private final HiveConf conf;
+  private RunOptions runOptions;
+  private final boolean createExternalDirsForDbs;
+  private final Path curWhRootPath;
+  private final HadoopShims.HdfsEncryptionShim encryptionShim;
+  private final String ownerName;
+  private final String groupName;
+  private final FsPermission dirPerms;
+  private final FsPermission filePerms;
+
+  private CloseableThreadLocal<HiveMetaStoreClient> hms;
+  private ThreadLocal<Warehouse> wh;
+  private ThreadLocal<Warehouse> oldWh;
+  private CloseableThreadLocal<HiveUpdater> hiveUpdater;
+
+  private AtomicBoolean failuresEncountered;
+  private AtomicBoolean failedValidationChecks;
+
+  HiveStrictManagedMigration(HiveConf conf, RunOptions runOptions, boolean createExternalDirsForDbs,
+                             OwnerPermsOptions ownerPermsOptions, WarehouseRootCheckResult warehouseRootCheckResult) {
+    this.conf = conf;
+    this.runOptions = runOptions;
+    this.createExternalDirsForDbs = createExternalDirsForDbs;
+    this.ownerName = ownerPermsOptions.ownerName;
+    this.groupName = ownerPermsOptions.groupName;
+    this.dirPerms = ownerPermsOptions.dirPerms;
+    this.filePerms = ownerPermsOptions.filePerms;
+    this.curWhRootPath = warehouseRootCheckResult.curWhRootPath;
+    this.encryptionShim = warehouseRootCheckResult.encryptionShim;
+
+    this.hms = new CloseableThreadLocal<>(() -> {
+      try {
+        HiveMetaStoreClient hiveMetaStoreClient = new HiveMetaStoreClient(conf);
+        if (hiveConf != null) {
+          SessionState ss = SessionState.start(conf);
+          ss.applyAuthorizationPolicy();
         }
+        return hiveMetaStoreClient;
+      } catch (Exception e) {
+        throw new RuntimeException(e);
       }
-      LOG.info("Done processing databases.");
-    } finally {
-      hms.close();
+    }, runOptions.tablePoolSize);
+    wh = ThreadLocal.withInitial(() -> {
+      try {
+        return new Warehouse(conf);
+      } catch (MetaException e) {
+        throw new RuntimeException(e);
+      }
+    });
+    if (runOptions.shouldModifyManagedTableLocation) {
+      Configuration oldConf = new Configuration(conf);
+      HiveConf.setVar(oldConf, HiveConf.ConfVars.METASTOREWAREHOUSE, runOptions.oldWarehouseRoot);
+
+      oldWh = ThreadLocal.withInitial(() -> {
+        try {
+          return new Warehouse(oldConf);
+        } catch (MetaException e) {
+          throw new RuntimeException(e);
+        }
+      });
     }
+    this.hiveUpdater = new CloseableThreadLocal<>(() -> {
+      try {
+        return new HiveUpdater(conf, true);
+      } catch (HiveException e) {
+        throw new RuntimeException(e);
+      }
+    }, runOptions.tablePoolSize);
 
-    if (failuresEncountered) {
+    this.failuresEncountered = new AtomicBoolean(false);
+    this.failedValidationChecks = new AtomicBoolean(false);
+  }
+
+  void run() throws Exception {
+    LOG.info("Starting with {}", runOptions);
+
+    List<String> databases = hms.get().getDatabases(runOptions.dbRegex); //TException
+    LOG.info("Found {} databases", databases.size());
+    ForkJoinPool tablePool = new ForkJoinPool(
+            runOptions.tablePoolSize,
+            new NamedForkJoinWorkerThreadFactory("Table-"),
+            getUncaughtExceptionHandler(),
+            false);
+    databases.forEach(dbName -> processDatabase(dbName, tablePool));
+    LOG.info("Done processing databases.");
+
+    if (failuresEncountered.get()) {
       throw new HiveException("One or more failures encountered during processing.");
     }
-    if (failedValidationChecks) {
+    if (failedValidationChecks.get()) {
       throw new HiveException("One or more tables failed validation checks for strict managed table mode.");
     }
   }
 
-  void checkOldWarehouseRoot() throws IOException, MetaException {
+  private Thread.UncaughtExceptionHandler getUncaughtExceptionHandler() {
+    return (t, e) -> LOG.error(String.format("Thread %s exited with error", t.getName()), e);
+  }
+
+  static WarehouseRootCheckResult checkOldWarehouseRoot(RunOptions runOptions, HiveConf conf) throws IOException {
+    boolean shouldModifyManagedTableLocation = runOptions.shouldModifyManagedTableLocation;
+    Path curWhRootPath = null;
+    HadoopShims.HdfsEncryptionShim encryptionShim = null;
+
     if (runOptions.shouldModifyManagedTableLocation) {
       if (runOptions.oldWarehouseRoot == null) {
         LOG.info("oldWarehouseRoot is not specified. Disabling shouldModifyManagedTableLocation");
-        runOptions.shouldModifyManagedTableLocation = false;
+        shouldModifyManagedTableLocation = false;
       } else {
         String curWarehouseRoot = HiveConf.getVar(conf, HiveConf.ConfVars.METASTOREWAREHOUSE);
         if (arePathsEqual(conf, runOptions.oldWarehouseRoot, curWarehouseRoot)) {
           LOG.info("oldWarehouseRoot is the same as the current warehouse root {}."
               + " Disabling shouldModifyManagedTableLocation",
               runOptions.oldWarehouseRoot);
-          runOptions.shouldModifyManagedTableLocation = false;
+          shouldModifyManagedTableLocation = false;
         } else {
           Path oldWhRootPath = new Path(runOptions.oldWarehouseRoot);
           curWhRootPath = new Path(curWarehouseRoot);
@@ -339,18 +510,18 @@ public class HiveStrictManagedMigration {
             LOG.info("oldWarehouseRoot {} has a different FS than the current warehouse root {}."
                 + " Disabling shouldModifyManagedTableLocation",
                 runOptions.oldWarehouseRoot, curWarehouseRoot);
-            runOptions.shouldModifyManagedTableLocation = false;
+            shouldModifyManagedTableLocation = false;
           } else {
             if (!isHdfs(oldWhRootFs)) {
               LOG.info("Warehouse is using non-HDFS FileSystem {}. Disabling shouldModifyManagedTableLocation",
                   oldWhRootFs.getUri());
-              runOptions.shouldModifyManagedTableLocation = false;
+              shouldModifyManagedTableLocation = false;
             } else {
               encryptionShim = ShimLoader.getHadoopShims().createHdfsEncryptionShim(oldWhRootFs, conf);
               if (!hasEquivalentEncryption(encryptionShim, oldWhRootPath, curWhRootPath)) {
                 LOG.info("oldWarehouseRoot {} and current warehouse root {} have different encryption zones." +
                     " Disabling shouldModifyManagedTableLocation", oldWhRootPath, curWhRootPath);
-                runOptions.shouldModifyManagedTableLocation = false;
+                shouldModifyManagedTableLocation = false;
               }
             }
           }
@@ -358,14 +529,15 @@ public class HiveStrictManagedMigration {
       }
     }
 
-    if (runOptions.shouldModifyManagedTableLocation) {
-      Configuration oldWhConf = new Configuration(conf);
-      HiveConf.setVar(oldWhConf, HiveConf.ConfVars.METASTOREWAREHOUSE, runOptions.oldWarehouseRoot);
-      oldWh = new Warehouse(oldWhConf);
-    }
+    return new WarehouseRootCheckResult(shouldModifyManagedTableLocation, curWhRootPath, encryptionShim);
   }
 
-  void checkOwnerPermsOptions() {
+  static OwnerPermsOptions checkOwnerPermsOptions(RunOptions runOptions, HiveConf conf) {
+    String ownerName = null;
+    String groupName = null;
+    FsPermission dirPerms = null;
+    FsPermission filePerms = null;
+
     if (runOptions.shouldModifyManagedTableOwner) {
       ownerName = conf.get("strict.managed.tables.migration.owner", "hive");
       groupName = conf.get("strict.managed.tables.migration.group", null);
@@ -380,61 +552,72 @@ public class HiveStrictManagedMigration {
         filePerms = new FsPermission(filePermsString);
       }
     }
+
+    return new OwnerPermsOptions(ownerName, groupName, dirPerms, filePerms);
   }
 
-  void checkExternalWarehouseDir() {
+  static boolean checkExternalWarehouseDir(HiveConf conf) {
     String externalWarehouseDir = conf.getVar(HiveConf.ConfVars.HIVE_METASTORE_WAREHOUSE_EXTERNAL);
-    if (externalWarehouseDir != null && !externalWarehouseDir.isEmpty()) {
-      createExternalDirsForDbs = true;
-    }
+    return externalWarehouseDir != null && !externalWarehouseDir.isEmpty();
   }
 
-  void processDatabase(String dbName) throws IOException, HiveException, MetaException, TException {
-    LOG.info("Processing database {}", dbName);
-    Database dbObj = hms.getDatabase(dbName);
+  void processDatabase(String dbName, ForkJoinPool tablePool) {
+    try {
+      LOG.info("Processing database {}", dbName);
+      Database dbObj = hms.get().getDatabase(dbName);
 
-    boolean modifyDefaultManagedLocation = shouldModifyDatabaseLocation(dbObj);
-    if (modifyDefaultManagedLocation) {
-      Path newDefaultDbLocation = wh.getDefaultDatabasePath(dbName);
+      boolean modifyDefaultManagedLocation = shouldModifyDatabaseLocation(dbObj);
+      if (modifyDefaultManagedLocation) {
+        Path newDefaultDbLocation = wh.get().getDefaultDatabasePath(dbName);
 
-      LOG.info("Changing location of database {} to {}", dbName, newDefaultDbLocation);
-      if (!runOptions.dryRun) {
-        FileSystem fs = newDefaultDbLocation.getFileSystem(conf);
-        FileUtils.mkdir(fs, newDefaultDbLocation, conf);
-        // Set appropriate owner/perms of the DB dir only, no need to recurse
-        checkAndSetFileOwnerPermissions(fs, newDefaultDbLocation,
-            ownerName, groupName, dirPerms, null, runOptions.dryRun, false);
+        LOG.info("Changing location of database {} to {}", dbName, newDefaultDbLocation);
+        if (!runOptions.dryRun) {
+          FileSystem fs = newDefaultDbLocation.getFileSystem(conf);
+          FileUtils.mkdir(fs, newDefaultDbLocation, conf);
+          // Set appropriate owner/perms of the DB dir only, no need to recurse
+          checkAndSetFileOwnerPermissions(fs, newDefaultDbLocation,
+                  ownerName, groupName, dirPerms, null, runOptions.dryRun, false);
+        }
       }
-    }
 
-    if (createExternalDirsForDbs) {
-      createExternalDbDir(dbObj);
-    }
+      if (createExternalDirsForDbs) {
+        createExternalDbDir(dbObj);
+      }
 
-    boolean errorsInThisDb = false;
-    List<String> tableNames = hms.getTables(dbName, runOptions.tableRegex);
-    for (String tableName : tableNames) {
-      // If we did not change the DB location, there is no need to move the table directories.
-      try {
-        processTable(dbObj, tableName, modifyDefaultManagedLocation);
-      } catch (Exception err) {
-        LOG.error("Error processing table " + getQualifiedName(dbObj.getName(), tableName), err);
-        failuresEncountered = true;
-        errorsInThisDb = true;
+      List<String> tableNames;
+      if (runOptions.tableType == null) {
+        tableNames = hms.get().getTables(dbName, runOptions.tableRegex);
+        LOG.debug("found {} tables in {}", tableNames.size(), dbName);
+      } else {
+        tableNames = hms.get().getTables(dbName, runOptions.tableRegex, runOptions.tableType);
+        LOG.debug("found {} {}s in {}", tableNames.size(), runOptions.tableType.name(), dbName);
       }
-    }
 
-    // Finally update the DB location. This would prevent subsequent runs of the migration from processing this DB.
-    if (modifyDefaultManagedLocation) {
+      boolean errorsInThisDb = !tablePool.submit(() -> tableNames.parallelStream()
+              .map(tableName -> processTable(dbObj, tableName, modifyDefaultManagedLocation))
+              .reduce(true, (aBoolean, aBoolean2) -> aBoolean && aBoolean2)).get();
       if (errorsInThisDb) {
-        LOG.error("Not updating database location for {} since an error was encountered. The migration must be run again for this database.",
-                dbObj.getName());
-      } else {
-        Path newDefaultDbLocation = wh.getDefaultDatabasePath(dbName);
-        // dbObj after this call would have the new DB location.
-        // Keep that in mind if anything below this requires the old DB path.
-        getHiveUpdater().updateDbLocation(dbObj, newDefaultDbLocation);
+        failuresEncountered.set(true);
+      }
+
+      // Finally update the DB location. This would prevent subsequent runs of the migration from processing this DB.
+      if (modifyDefaultManagedLocation) {
+        if (errorsInThisDb) {
+          LOG.error("Not updating database location for {} since an error was encountered. " +
+                          "The migration must be run again for this database.", dbObj.getName());
+        } else {
+          Path newDefaultDbLocation = wh.get().getDefaultDatabasePath(dbName);
+          // dbObj after this call would have the new DB location.
+          // Keep that in mind if anything below this requires the old DB path.
+          hiveUpdater.get().updateDbLocation(dbObj, newDefaultDbLocation);
+        }
       }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.info("Cancel processing " + dbName, e);
+    } catch (TException | IOException | HiveException | ExecutionException ex) {
+      LOG.error("Error processing database " + dbName, ex);
+      failuresEncountered.set(true);
     }
   }
 
@@ -464,44 +647,55 @@ public class HiveStrictManagedMigration {
     return false;
   }
 
-  void processTable(Database dbObj, String tableName, boolean modifyDefaultManagedLocation)
-      throws HiveException, IOException, TException {
-    String dbName = dbObj.getName();
-    LOG.debug("Processing table {}", getQualifiedName(dbName, tableName));
+  boolean processTable(Database dbObj, String tableName, boolean modifyDefaultManagedLocation) {
+    try {
+      String dbName = dbObj.getName();
+      LOG.debug("Processing table {}", getQualifiedName(dbName, tableName));
 
-    Table tableObj = hms.getTable(dbName, tableName);
-    TableType tableType = TableType.valueOf(tableObj.getTableType());
+      Table tableObj = hms.get().getTable(dbName, tableName);
+      TableType tableType = TableType.valueOf(tableObj.getTableType());
 
-    TableMigrationOption migrationOption = runOptions.migrationOption;
-    if (migrationOption == TableMigrationOption.AUTOMATIC) {
-      migrationOption = determineMigrationTypeAutomatically(tableObj, tableType, ownerName, conf, hms, null);
-    }
+      TableMigrationOption migrationOption = runOptions.migrationOption;
+      if (migrationOption == TableMigrationOption.AUTOMATIC) {
+        migrationOption = determineMigrationTypeAutomatically(
+                tableObj, tableType, ownerName, conf, hms.get(), null);
+      }
 
-    failedValidationChecks = migrateTable(tableObj, tableType, migrationOption, runOptions.dryRun,
-            getHiveUpdater(), hms, conf);
+      boolean failedValidationCheck = migrateTable(tableObj, tableType, migrationOption, runOptions.dryRun,
+              hiveUpdater.get(), hms.get(), conf);
 
-    if (!failedValidationChecks && (TableType.valueOf(tableObj.getTableType()) == TableType.MANAGED_TABLE)) {
-      Path tablePath = new Path(tableObj.getSd().getLocation());
-      if (modifyDefaultManagedLocation && shouldModifyTableLocation(dbObj, tableObj)) {
-        Path newTablePath = wh.getDnsPath(
-            new Path(wh.getDefaultDatabasePath(dbName),
-                MetaStoreUtils.encodeTableName(tableName.toLowerCase())));
-        moveTableData(dbObj, tableObj, newTablePath);
-        if (!runOptions.dryRun) {
-          // File ownership/permission checks should be done on the new table path.
-          tablePath = newTablePath;
-        }
+      if (failedValidationCheck) {
+        this.failedValidationChecks.set(true);
+        return true;
       }
 
-      if (runOptions.shouldModifyManagedTableOwner || runOptions.shouldModifyManagedTablePermissions) {
-        FileSystem fs = tablePath.getFileSystem(conf);
-        if (isHdfs(fs)) {
-          // TODO: what about partitions not in the default location?
-          checkAndSetFileOwnerPermissions(fs, tablePath,
-              ownerName, groupName, dirPerms, filePerms, runOptions.dryRun, true);
+      if (TableType.valueOf(tableObj.getTableType()) == TableType.MANAGED_TABLE) {
+        Path tablePath = new Path(tableObj.getSd().getLocation());
+        if (modifyDefaultManagedLocation && shouldModifyTableLocation(dbObj, tableObj)) {
+          Path newTablePath = wh.get().getDnsPath(
+                  new Path(wh.get().getDefaultDatabasePath(dbName),
+                          MetaStoreUtils.encodeTableName(tableName.toLowerCase())));
+          moveTableData(dbObj, tableObj, newTablePath);
+          if (!runOptions.dryRun) {
+            // File ownership/permission checks should be done on the new table path.
+            tablePath = newTablePath;
+          }
+        }
+
+        if (runOptions.shouldModifyManagedTableOwner || runOptions.shouldModifyManagedTablePermissions) {
+          FileSystem fs = tablePath.getFileSystem(conf);
+          if (isHdfs(fs)) {
+            // TODO: what about partitions not in the default location?
+            checkAndSetFileOwnerPermissions(fs, tablePath,
+                    ownerName, groupName, dirPerms, filePerms, runOptions.dryRun, true);
+          }
         }
       }
+    } catch (Exception ex) {
+      LOG.error("Error processing table " + getQualifiedName(dbObj.getName(), tableName), ex);
+      return false;
     }
+    return true;
   }
 
   boolean shouldModifyDatabaseLocation(Database dbObj) throws IOException, MetaException {
@@ -510,7 +704,7 @@ public class HiveStrictManagedMigration {
       // Check if the database location is in the default location based on the old warehouse root.
       // If so then change the database location to the default based on the current warehouse root.
       String dbLocation = dbObj.getLocationUri();
-      Path oldDefaultDbLocation = oldWh.getDefaultDatabasePath(dbName);
+      Path oldDefaultDbLocation = oldWh.get().getDefaultDatabasePath(dbName);
       if (arePathsEqual(conf, dbLocation, oldDefaultDbLocation.toString())) {
         if (hasEquivalentEncryption(encryptionShim, oldDefaultDbLocation, curWhRootPath)) {
           return true;
@@ -529,7 +723,7 @@ public class HiveStrictManagedMigration {
     // If so then change the table location to the default based on the current warehouse root.
     // The existing table directory will also be moved to the new default database directory.
     String tableLocation = tableObj.getSd().getLocation();
-    Path oldDefaultTableLocation = oldWh.getDefaultTablePath(dbObj, tableObj.getTableName());
+    Path oldDefaultTableLocation = oldWh.get().getDefaultTablePath(dbObj, tableObj.getTableName());
     if (arePathsEqual(conf, tableLocation, oldDefaultTableLocation.toString())) {
       if (hasEquivalentEncryption(encryptionShim, oldDefaultTableLocation, curWhRootPath)) {
         return true;
@@ -545,7 +739,7 @@ public class HiveStrictManagedMigration {
       throws IOException, MetaException {
     String tableName = tableObj.getTableName();
     String partLocation = partObj.getSd().getLocation();
-    Path oldDefaultPartLocation = oldWh.getDefaultPartitionPath(dbObj, tableObj, partSpec);
+    Path oldDefaultPartLocation = oldWh.get().getDefaultPartitionPath(dbObj, tableObj, partSpec);
     if (arePathsEqual(conf, partLocation, oldDefaultPartLocation.toString())) {
       if (hasEquivalentEncryption(encryptionShim, oldDefaultPartLocation, curWhRootPath)) {
         return true;
@@ -558,7 +752,7 @@ public class HiveStrictManagedMigration {
   }
 
   void createExternalDbDir(Database dbObj) throws IOException, MetaException {
-    Path externalTableDbPath = wh.getDefaultExternalDatabasePath(dbObj.getName());
+    Path externalTableDbPath = wh.get().getDefaultExternalDatabasePath(dbObj.getName());
     FileSystem fs = externalTableDbPath.getFileSystem(conf);
     if (!fs.exists(externalTableDbPath)) {
       String dbOwner = ownerName;
@@ -621,19 +815,19 @@ public class HiveStrictManagedMigration {
     // locations to be in sync.
 
     if (isPartitionedTable(tableObj)) {
-      List<String> partNames = hms.listPartitionNames(dbName, tableName, Short.MAX_VALUE);
+      List<String> partNames = hms.get().listPartitionNames(dbName, tableName, Short.MAX_VALUE);
       // TODO: Fetch partitions in batches?
       // TODO: Threadpool to process partitions?
       for (String partName : partNames) {
-        Partition partObj = hms.getPartition(dbName, tableName, partName);
+        Partition partObj = hms.get().getPartition(dbName, tableName, partName);
         Map<String, String> partSpec =
             Warehouse.makeSpecFromValues(tableObj.getPartitionKeys(), partObj.getValues());
         if (shouldModifyPartitionLocation(dbObj, tableObj, partObj, partSpec)) {
           // Table directory (which includes the partition directory) has already been moved,
           // just update the partition location in the metastore.
           if (!runOptions.dryRun) {
-            Path newPartPath = wh.getPartitionPath(newTablePath, partSpec);
-            getHiveUpdater().updatePartitionLocation(dbName, tableObj, partName, partObj, newPartPath);
+            Path newPartPath = wh.get().getPartitionPath(newTablePath, partSpec);
+            hiveUpdater.get().updatePartitionLocation(dbName, tableObj, partName, partObj, newPartPath);
           }
         }
       }
@@ -642,7 +836,7 @@ public class HiveStrictManagedMigration {
     // Finally update the table location. This would prevent this tool from processing this table again
     // on subsequent runs of the migration.
     if (!runOptions.dryRun) {
-      getHiveUpdater().updateTableLocation(tableObj, newTablePath);
+      hiveUpdater.get().updateTableLocation(tableObj, newTablePath);
     }
   }
 
@@ -907,6 +1101,7 @@ public class HiveStrictManagedMigration {
   }
 
   void cleanup() {
+    hms.close();
     if (hiveUpdater != null) {
       runAndLogErrors(() -> hiveUpdater.close());
       hiveUpdater = null;
@@ -917,13 +1112,6 @@ public class HiveStrictManagedMigration {
     return new HiveUpdater(conf, false);
   }
 
-  HiveUpdater getHiveUpdater() throws HiveException {
-    if (hiveUpdater == null) {
-      hiveUpdater = new HiveUpdater(conf, true);
-    }
-    return hiveUpdater;
-  }
-
   private static final class TxnCtx {
     public final long writeId;
     public final String validWriteIds;
@@ -936,7 +1124,7 @@ public class HiveStrictManagedMigration {
     }
   }
 
-  public static class HiveUpdater {
+  private static class HiveUpdater implements AutoCloseable {
     Hive hive;
     boolean doFileRename;
 
@@ -946,9 +1134,10 @@ public class HiveStrictManagedMigration {
       doFileRename = fileRename;
     }
 
-    void close() {
+    @Override
+    public void close() {
       if (hive != null) {
-        runAndLogErrors(() -> Hive.closeCurrent());
+        runAndLogErrors(Hive::closeCurrent);
         hive = null;
       }
     }
@@ -1122,8 +1311,6 @@ public class HiveStrictManagedMigration {
     }
   }
 
-  HiveUpdater hiveUpdater;
-
   interface ThrowableRunnable {
     void run() throws Exception;
   }
@@ -1157,7 +1344,7 @@ public class HiveStrictManagedMigration {
   }
 
   static boolean isHdfs(FileSystem fs) {
-    return fs.getScheme().equals("hdfs");
+    return scheme.equals(fs.getScheme());
   }
 
   static String getQualifiedName(Table tableObj) {
@@ -1335,4 +1522,12 @@ public class HiveStrictManagedMigration {
     }
     return true;
   }
+
+  /**
+   * can set it from tests to test when config needs something other than default values.
+   */
+  @VisibleForTesting
+  static HiveConf hiveConf = null;
+  @VisibleForTesting
+  static String scheme = "hdfs";
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/util/NamedForkJoinWorkerThreadFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/util/NamedForkJoinWorkerThreadFactory.java
new file mode 100644
index 0000000..5b6eecc
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/util/NamedForkJoinWorkerThreadFactory.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.util;
+
+import java.util.concurrent.ForkJoinPool;
+import java.util.concurrent.ForkJoinWorkerThread;
+
+/**
+ * This class allows specifying a prefix for ForkJoinPool thread names.
+ */
+public class NamedForkJoinWorkerThreadFactory implements ForkJoinPool.ForkJoinWorkerThreadFactory {
+
+  NamedForkJoinWorkerThreadFactory(String namePrefix) {
+    this.namePrefix = namePrefix;
+  }
+
+  private final String namePrefix;
+
+  @Override
+  public ForkJoinWorkerThread newThread(ForkJoinPool pool) {
+    ForkJoinWorkerThread worker = ForkJoinPool.defaultForkJoinWorkerThreadFactory.newThread(pool);
+    worker.setName(namePrefix + worker.getName());
+    return worker;
+  }
+}
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java b/ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java
index 5f39fdc..7039b89 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java
@@ -17,6 +17,13 @@
  */
 package org.apache.hadoop.hive.ql;
 
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
@@ -40,13 +47,6 @@ import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.File;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicBoolean;
-
 public abstract class TxnCommandsBaseForTests {
   private static final Logger LOG = LoggerFactory.getLogger(TxnCommandsBaseForTests.class);
   //bucket count for test tables; set it to 1 for easier debugging
@@ -55,7 +55,7 @@ public abstract class TxnCommandsBaseForTests {
   public TestName testName = new TestName();
   protected HiveConf hiveConf;
   Driver d;
-  enum Table {
+  public enum Table {
     ACIDTBL("acidTbl"),
     ACIDTBLPART("acidTblPart"),
     ACIDTBL2("acidTbl2"),
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/util/CloseableThreadLocalTest.java b/ql/src/test/org/apache/hadoop/hive/ql/util/CloseableThreadLocalTest.java
new file mode 100644
index 0000000..25a59b7
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/util/CloseableThreadLocalTest.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.util;
+
+import static org.hamcrest.CoreMatchers.not;
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertThat;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import org.junit.Test;
+
+public class CloseableThreadLocalTest {
+
+  private static class AutoCloseableStub implements AutoCloseable {
+
+    private boolean closed = false;
+
+    public boolean isClosed() {
+      return closed;
+    }
+
+    @Override
+    public void close() throws Exception {
+      closed = true;
+    }
+  }
+
+  @Test
+  public void testResourcesAreInitiallyNotClosed() {
+    CloseableThreadLocal<AutoCloseableStub> closeableThreadLocal =
+            new CloseableThreadLocal<>(AutoCloseableStub::new, 1);
+
+    assertThat(closeableThreadLocal.get().isClosed(), is(false));
+  }
+
+  @Test
+  public void testAfterCallingCloseAllInstancesAreClosed() throws ExecutionException, InterruptedException {
+    CloseableThreadLocal<AutoCloseableStub> closeableThreadLocal =
+            new CloseableThreadLocal<>(AutoCloseableStub::new, 2);
+
+    AutoCloseableStub asyncInstance = CompletableFuture.supplyAsync(closeableThreadLocal::get).get();
+    AutoCloseableStub syncInstance = closeableThreadLocal.get();
+
+    closeableThreadLocal.close();
+
+    assertThat(asyncInstance.isClosed(), is(true));
+    assertThat(syncInstance.isClosed(), is(true));
+  }
+
+  @Test
+  public void testSubsequentGetsInTheSameThreadGivesBackTheSameObject() {
+    CloseableThreadLocal<AutoCloseableStub> closeableThreadLocal =
+            new CloseableThreadLocal<>(AutoCloseableStub::new, 2);
+
+    AutoCloseableStub ref1 = closeableThreadLocal.get();
+    AutoCloseableStub ref2 = closeableThreadLocal.get();
+    assertThat(ref1, is(ref2));
+  }
+
+  @Test
+  public void testDifferentThreadsHasDifferentInstancesOfTheResource() throws ExecutionException, InterruptedException {
+    CloseableThreadLocal<AutoCloseableStub> closeableThreadLocal =
+            new CloseableThreadLocal<>(AutoCloseableStub::new, 2);
+
+    AutoCloseableStub asyncInstance = CompletableFuture.supplyAsync(closeableThreadLocal::get).get();
+    AutoCloseableStub syncInstance = closeableThreadLocal.get();
+    assertThat(asyncInstance, is(not(syncInstance)));
+  }
+}
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/util/TestHiveStrictManagedMigration.java b/ql/src/test/org/apache/hadoop/hive/ql/util/TestHiveStrictManagedMigration.java
new file mode 100644
index 0000000..057135b
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/util/TestHiveStrictManagedMigration.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.util;
+
+import static org.apache.hadoop.hive.ql.TxnCommandsBaseForTests.Table.ACIDTBL;
+import static org.apache.hadoop.hive.ql.TxnCommandsBaseForTests.Table.ACIDTBLPART;
+import static org.apache.hadoop.hive.ql.TxnCommandsBaseForTests.Table.NONACIDNONBUCKET;
+import static org.apache.hadoop.hive.ql.TxnCommandsBaseForTests.Table.NONACIDORCTBL;
+import static org.apache.hadoop.hive.ql.TxnCommandsBaseForTests.Table.NONACIDORCTBL2;
+
+import java.io.File;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.TxnCommandsBaseForTests;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestHiveStrictManagedMigration extends TxnCommandsBaseForTests {
+  private static final String TEST_DATA_DIR = new File(System.getProperty("java.io.tmpdir") +
+  File.separator + TestHiveStrictManagedMigration.class.getCanonicalName() + "-" + System.currentTimeMillis()
+          ).getPath().replaceAll("\\\\", "/");
+
+  @Test
+  public void testUpgrade() throws Exception {
+    int[][] data = {{1, 2}, {3, 4}, {5, 6}};
+    runStatementOnDriver("DROP TABLE IF EXISTS test.TAcid");
+    runStatementOnDriver("DROP DATABASE IF EXISTS test");
+
+    runStatementOnDriver("CREATE DATABASE test");
+    runStatementOnDriver(
+            "CREATE TABLE test.TAcid (a int, b int) CLUSTERED BY (b) INTO 2 BUCKETS STORED AS orc TBLPROPERTIES" +
+                    " ('transactional'='true')");
+    runStatementOnDriver("INSERT INTO test.TAcid" + makeValuesClause(data));
+
+    runStatementOnDriver(
+            "CREATE EXTERNAL TABLE texternal (a int, b int)");
+
+    String oldWarehouse = getWarehouseDir();
+    String[] args = {"--hiveconf", "hive.strict.managed.tables=true", "-m",  "automatic", "--modifyManagedTables",
+            "--oldWarehouseRoot", oldWarehouse};
+    HiveConf newConf = new HiveConf(hiveConf);
+    File newWarehouseDir = new File(getTestDataDir(), "newWarehouse");
+    newConf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, newWarehouseDir.getAbsolutePath());
+    newConf.set("strict.managed.tables.migration.owner", System.getProperty("user.name"));
+    HiveStrictManagedMigration.hiveConf = newConf;
+    HiveStrictManagedMigration.scheme = "file";
+    HiveStrictManagedMigration.main(args);
+
+    Assert.assertTrue(newWarehouseDir.exists());
+    Assert.assertTrue(new File(newWarehouseDir, ACIDTBL.toString().toLowerCase()).exists());
+    Assert.assertTrue(new File(newWarehouseDir, ACIDTBLPART.toString().toLowerCase()).exists());
+    Assert.assertTrue(new File(newWarehouseDir, NONACIDNONBUCKET.toString().toLowerCase()).exists());
+    Assert.assertTrue(new File(newWarehouseDir, NONACIDORCTBL.toString().toLowerCase()).exists());
+    Assert.assertTrue(new File(newWarehouseDir, NONACIDORCTBL2.toString().toLowerCase()).exists());
+    Assert.assertTrue(new File(new File(newWarehouseDir, "test.db"), "tacid").exists());
+    Assert.assertTrue(new File(oldWarehouse, "texternal").exists());
+  }
+
+  @Override
+  protected String getTestDataDir() {
+    return TEST_DATA_DIR;
+  }
+}


[hive] 05/05: HIVE-21973: SHOW LOCKS prints the headers twice (Rajkumar Singh via Gopal V)

Posted by kg...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kgyrtkirk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git

commit 2350662ee4fa17332bf546fbebee5cde9714717e
Author: Rajkumar Singh <ra...@gmail.com>
AuthorDate: Mon Jul 15 11:37:13 2019 +0200

    HIVE-21973: SHOW LOCKS prints the headers twice (Rajkumar Singh via Gopal V)
    
    Signed-off-by: Zoltan Haindrich <ki...@rxd.hu>
---
 .../hive/ql/ddl/table/lock/ShowLocksOperation.java | 57 ++++++++++++----------
 1 file changed, 30 insertions(+), 27 deletions(-)

diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/ShowLocksOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/ShowLocksOperation.java
index 3457284..2636530 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/ShowLocksOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/ShowLocksOperation.java
@@ -183,33 +183,36 @@ public class ShowLocksOperation extends DDLOperation<ShowLocksDesc> {
   }
 
   public static void dumpLockInfo(DataOutputStream os, ShowLocksResponse response) throws IOException {
-    // Write a header
-    os.writeBytes("Lock ID");
-    os.write(Utilities.tabCode);
-    os.writeBytes("Database");
-    os.write(Utilities.tabCode);
-    os.writeBytes("Table");
-    os.write(Utilities.tabCode);
-    os.writeBytes("Partition");
-    os.write(Utilities.tabCode);
-    os.writeBytes("State");
-    os.write(Utilities.tabCode);
-    os.writeBytes("Blocked By");
-    os.write(Utilities.tabCode);
-    os.writeBytes("Type");
-    os.write(Utilities.tabCode);
-    os.writeBytes("Transaction ID");
-    os.write(Utilities.tabCode);
-    os.writeBytes("Last Heartbeat");
-    os.write(Utilities.tabCode);
-    os.writeBytes("Acquired At");
-    os.write(Utilities.tabCode);
-    os.writeBytes("User");
-    os.write(Utilities.tabCode);
-    os.writeBytes("Hostname");
-    os.write(Utilities.tabCode);
-    os.writeBytes("Agent Info");
-    os.write(Utilities.newLineCode);
+    SessionState sessionState = SessionState.get();
+    // Write a header for CliDriver
+    if(!sessionState.isHiveServerQuery()) {
+      os.writeBytes("Lock ID");
+      os.write(Utilities.tabCode);
+      os.writeBytes("Database");
+      os.write(Utilities.tabCode);
+      os.writeBytes("Table");
+      os.write(Utilities.tabCode);
+      os.writeBytes("Partition");
+      os.write(Utilities.tabCode);
+      os.writeBytes("State");
+      os.write(Utilities.tabCode);
+      os.writeBytes("Blocked By");
+      os.write(Utilities.tabCode);
+      os.writeBytes("Type");
+      os.write(Utilities.tabCode);
+      os.writeBytes("Transaction ID");
+      os.write(Utilities.tabCode);
+      os.writeBytes("Last Heartbeat");
+      os.write(Utilities.tabCode);
+      os.writeBytes("Acquired At");
+      os.write(Utilities.tabCode);
+      os.writeBytes("User");
+      os.write(Utilities.tabCode);
+      os.writeBytes("Hostname");
+      os.write(Utilities.tabCode);
+      os.writeBytes("Agent Info");
+      os.write(Utilities.newLineCode);
+    }
 
     List<ShowLocksResponseElement> locks = response.getLocks();
     if (locks != null) {