Uploaded image for project: 'Hive'
  1. Hive
  2. HIVE-14214

ORC Schema Evolution and Predicate Push Down do not work together (no rows returned)

Details

    • Bug
    • Status: Closed
    • Critical
    • Resolution: Fixed
    • None
    • 2.1.1, 2.2.0
    • Hive
    • None

    Description

      In Schema Evolution, the reader schema is different than the file schema which is used to evaluate predicate push down.

      Attachments

        1. HIVE-14214.01.patch
          1.57 MB
          Matt McCline
        2. HIVE-14214.02.patch
          425 kB
          Matt McCline
        3. HIVE-14214.03.patch
          423 kB
          Matt McCline
        4. HIVE-14214.04.patch
          424 kB
          Matt McCline
        5. HIVE-14214.05.patch
          442 kB
          Matt McCline
        6. HIVE-14214.06.patch
          433 kB
          Matt McCline
        7. HIVE-14214.07.patch
          439 kB
          Matt McCline
        8. HIVE-14214.08.patch
          439 kB
          Matt McCline
        9. HIVE-14214.09.patch
          442 kB
          Matt McCline
        10. HIVE-14214.WIP.patch
          61 kB
          Matt McCline

        Issue Links

          Activity

            mmccline Matt McCline added a comment -

            Work-In-Progress patch.

            Areas that need work:
            1) Using readerTypes to check and not do PPD in LocalCache.
            2) OrcFileFormatProxy.applySargToMetadata – need to pass down or generate included boolean array.
            3) HiveMetaStore.get_file_metadata_by_expr – need to deserialize the readerTypes. And, modify the Thrift source to add the readerTypes.

            And add unit tests and/or Q file tests.

            mmccline Matt McCline added a comment - Work-In-Progress patch. Areas that need work: 1) Using readerTypes to check and not do PPD in LocalCache. 2) OrcFileFormatProxy.applySargToMetadata – need to pass down or generate included boolean array. 3) HiveMetaStore.get_file_metadata_by_expr – need to deserialize the readerTypes. And, modify the Thrift source to add the readerTypes. And add unit tests and/or Q file tests.
            sershe Sergey Shelukhin added a comment - - edited

            HBase metadata PPD feature is not complete right now (or rather it should work and is somewhat usable but it's alpha with HBase metastore and there's convenience and testing to be added). Unless I am missing something it should be ok to break it as long as it breaks in a reasonable manner (e.g. throws suggesting the user not enable it, it's off by default), and there's a followup JIRA

            sershe Sergey Shelukhin added a comment - - edited HBase metadata PPD feature is not complete right now (or rather it should work and is somewhat usable but it's alpha with HBase metastore and there's convenience and testing to be added). Unless I am missing something it should be ok to break it as long as it breaks in a reasonable manner (e.g. throws suggesting the user not enable it, it's off by default), and there's a followup JIRA
            mmccline Matt McCline added a comment - - edited

            Thanks Sergey for the advice.

            Patch #1 still needs a test for split elimination (i.e. test new code in OrcInputFormat.SplitGenerator.callInternal)

            And, even at 10k, the 1/10th version of all100k is probably too big...

            mmccline Matt McCline added a comment - - edited Thanks Sergey for the advice. Patch #1 still needs a test for split elimination (i.e. test new code in OrcInputFormat.SplitGenerator.callInternal) And, even at 10k, the 1/10th version of all100k is probably too big...
            hiveqa Hive QA added a comment -

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

            SUCCESS: +1 due to 2 test(s) being added or modified.

            ERROR: -1 due to 101 failed/errored test(s), 10327 tests executed
            Failed tests:

            org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_acid_globallimit
            org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_list_bucket_dml_13
            org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_parquet_join
            org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_stats_list_bucket
            org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_subquery_multiinsert
            org.apache.hadoop.hive.cli.TestHBaseCliDriver.testCliDriver_hbase_ppd_join
            org.apache.hadoop.hive.cli.TestHBaseCliDriver.testCliDriver_hbase_ppd_key_range
            org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_acid_globallimit
            org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_mergejoin
            org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_orc_merge5
            org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_orc_merge6
            org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_orc_merge7
            org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_orc_merge_incompat2
            org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_orc_ppd_basic
            org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_orc_vectorization_ppd
            org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_vectorization_pushdown
            org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_vectorized_distinct_gby
            org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_vectorized_dynamic_partition_pruning
            org.apache.hadoop.hive.llap.daemon.impl.TestLlapTokenChecker.testCheckPermissions
            org.apache.hadoop.hive.llap.daemon.impl.TestLlapTokenChecker.testGetToken
            org.apache.hadoop.hive.metastore.TestMetaStoreMetrics.testConnections
            org.apache.hadoop.hive.ql.TestTxnCommands2.testNonAcidToAcidConversion2
            org.apache.hadoop.hive.ql.TestTxnCommands2.testNonAcidToAcidConversion3
            org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testACIDReaderFooterSerialize
            org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testACIDReaderFooterSerializeWithDeltas
            org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testACIDReaderNoFooterSerialize
            org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testACIDReaderNoFooterSerializeWithDeltas
            org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testDefaultTypes
            org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testDoAs
            org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testEmptyFile
            org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testInOutFormat
            org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testMROutput
            org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testNonVectorReaderFooterSerialize
            org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testNonVectorReaderNoFooterSerialize
            org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testSplitElimination
            org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testSplitEliminationNullStats
            org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testSplitGenReadOps
            org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testSplitGenReadOpsLocalCache
            org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testSplitGenReadOpsLocalCacheChangeFileLen
            org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testSplitGenReadOpsLocalCacheChangeModificationTime
            org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testVectorReaderFooterSerialize
            org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testVectorReaderNoFooterSerialize
            org.apache.hadoop.hive.ql.io.orc.TestNewInputOutputFormat.testNewInputFormat
            org.apache.hadoop.hive.ql.io.orc.TestNewInputOutputFormat.testNewInputFormatPruning
            org.apache.hadoop.hive.ql.io.orc.TestOrcRawRecordMerger.testRecordReaderDelta
            org.apache.hadoop.hive.ql.io.orc.TestOrcRawRecordMerger.testRecordReaderIncompleteDelta
            org.apache.hadoop.hive.ql.io.orc.TestOrcRawRecordMerger.testRecordReaderNewBaseAndDelta
            org.apache.hadoop.hive.ql.io.orc.TestOrcRawRecordMerger.testRecordReaderOldBaseAndDelta
            org.apache.hadoop.hive.ql.io.orc.TestOrcSplitElimination.testExternalFooterCache
            org.apache.hadoop.hive.ql.io.orc.TestOrcSplitElimination.testExternalFooterCachePpd
            org.apache.hadoop.hive.ql.io.orc.TestOrcSplitElimination.testSplitEliminationComplexExpr
            org.apache.hadoop.hive.ql.io.orc.TestOrcSplitElimination.testSplitEliminationLargeMaxSplit
            org.apache.hadoop.hive.ql.io.orc.TestOrcSplitElimination.testSplitEliminationSmallMaxSplit
            org.apache.hive.hcatalog.mapreduce.TestHCatDynamicPartitioned.testHCatDynamicPartitionedTableMultipleTask[3]
            org.apache.hive.hcatalog.mapreduce.TestHCatDynamicPartitioned.testHCatDynamicPartitionedTable[3]
            org.apache.hive.hcatalog.mapreduce.TestHCatExternalDynamicPartitioned.testHCatDynamicPartitionedTableMultipleTask[3]
            org.apache.hive.hcatalog.mapreduce.TestHCatExternalDynamicPartitioned.testHCatDynamicPartitionedTable[3]
            org.apache.hive.hcatalog.mapreduce.TestHCatExternalDynamicPartitioned.testHCatExternalDynamicCustomLocation[3]
            org.apache.hive.hcatalog.mapreduce.TestHCatExternalNonPartitioned.testHCatNonPartitionedTable[3]
            org.apache.hive.hcatalog.mapreduce.TestHCatExternalPartitioned.testHCatPartitionedTable[3]
            org.apache.hive.hcatalog.mapreduce.TestHCatMutableDynamicPartitioned.testHCatDynamicPartitionedTableMultipleTask[3]
            org.apache.hive.hcatalog.mapreduce.TestHCatMutableDynamicPartitioned.testHCatDynamicPartitionedTable[3]
            org.apache.hive.hcatalog.mapreduce.TestHCatMutableNonPartitioned.testHCatNonPartitionedTable[3]
            org.apache.hive.hcatalog.mapreduce.TestHCatMutablePartitioned.testHCatPartitionedTable[3]
            org.apache.hive.hcatalog.mapreduce.TestHCatNonPartitioned.testHCatNonPartitionedTable[3]
            org.apache.hive.hcatalog.mapreduce.TestHCatPartitioned.testHCatPartitionedTable[3]
            org.apache.hive.hcatalog.pig.TestE2EScenarios.testReadOrcAndRCFromPig
            org.apache.hive.hcatalog.pig.TestHCatLoader.testColumnarStorePushdown2[3]
            org.apache.hive.hcatalog.pig.TestHCatLoader.testColumnarStorePushdown[3]
            org.apache.hive.hcatalog.pig.TestHCatLoader.testProjectionsBasic[3]
            org.apache.hive.hcatalog.pig.TestHCatLoader.testReadDataBasic[3]
            org.apache.hive.hcatalog.pig.TestHCatLoader.testReadPartitionedBasic[3]
            org.apache.hive.hcatalog.pig.TestHCatLoaderComplexSchema.testMapNullKey[3]
            org.apache.hive.hcatalog.pig.TestHCatLoaderComplexSchema.testMapWithComplexData[3]
            org.apache.hive.hcatalog.pig.TestHCatLoaderComplexSchema.testSyntheticComplexSchema[3]
            org.apache.hive.hcatalog.pig.TestHCatLoaderComplexSchema.testTupleInBagInTupleInBag[3]
            org.apache.hive.hcatalog.pig.TestHCatStorer.testDateCharTypes[3]
            org.apache.hive.hcatalog.pig.TestHCatStorer.testPartColsInData[3]
            org.apache.hive.hcatalog.pig.TestHCatStorer.testStoreFuncAllSimpleTypes[3]
            org.apache.hive.hcatalog.pig.TestHCatStorer.testStoreInPartiitonedTbl[3]
            org.apache.hive.hcatalog.pig.TestHCatStorer.testWriteChar[3]
            org.apache.hive.hcatalog.pig.TestHCatStorer.testWriteDate2[3]
            org.apache.hive.hcatalog.pig.TestHCatStorer.testWriteDate3[3]
            org.apache.hive.hcatalog.pig.TestHCatStorer.testWriteDate[3]
            org.apache.hive.hcatalog.pig.TestHCatStorer.testWriteDecimalXY[3]
            org.apache.hive.hcatalog.pig.TestHCatStorer.testWriteDecimalX[3]
            org.apache.hive.hcatalog.pig.TestHCatStorer.testWriteDecimal[3]
            org.apache.hive.hcatalog.pig.TestHCatStorer.testWriteSmallint[3]
            org.apache.hive.hcatalog.pig.TestHCatStorer.testWriteTimestamp[3]
            org.apache.hive.hcatalog.pig.TestHCatStorer.testWriteTinyint[3]
            org.apache.hive.hcatalog.pig.TestHCatStorer.testWriteVarchar[3]
            org.apache.hive.hcatalog.streaming.TestStreaming.testInterleavedTransactionBatchCommits
            org.apache.hive.hcatalog.streaming.TestStreaming.testMultipleTransactionBatchCommits
            org.apache.hive.hcatalog.streaming.TestStreaming.testTransactionBatchAbortAndCommit
            org.apache.hive.hcatalog.streaming.TestStreaming.testTransactionBatchCommit_Delimited
            org.apache.hive.hcatalog.streaming.TestStreaming.testTransactionBatchCommit_DelimitedUGI
            org.apache.hive.hcatalog.streaming.TestStreaming.testTransactionBatchCommit_Json
            org.apache.hive.hcatalog.streaming.mutate.TestMutations.testMulti
            org.apache.hive.hcatalog.streaming.mutate.TestMutations.testTransactionBatchCommitPartitioned
            org.apache.hive.hcatalog.streaming.mutate.TestMutations.testTransactionBatchCommitUnpartitioned
            org.apache.hive.hcatalog.streaming.mutate.TestMutations.testUpdatesAndDeletes
            

            Test results: https://builds.apache.org/job/PreCommit-HIVE-MASTER-Build/536/testReport
            Console output: https://builds.apache.org/job/PreCommit-HIVE-MASTER-Build/536/console
            Test logs: http://ec2-204-236-174-241.us-west-1.compute.amazonaws.com/logs/PreCommit-HIVE-MASTER-Build-536/

            Messages:

            Executing org.apache.hive.ptest.execution.TestCheckPhase
            Executing org.apache.hive.ptest.execution.PrepPhase
            Executing org.apache.hive.ptest.execution.ExecutionPhase
            Executing org.apache.hive.ptest.execution.ReportingPhase
            Tests exited with: TestsFailedException: 101 tests failed
            

            This message is automatically generated.

            ATTACHMENT ID: 12818138 - PreCommit-HIVE-MASTER-Build

            hiveqa Hive QA added a comment - Here are the results of testing the latest attachment: https://issues.apache.org/jira/secure/attachment/12818138/HIVE-14214.01.patch SUCCESS: +1 due to 2 test(s) being added or modified. ERROR: -1 due to 101 failed/errored test(s), 10327 tests executed Failed tests: org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_acid_globallimit org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_list_bucket_dml_13 org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_parquet_join org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_stats_list_bucket org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_subquery_multiinsert org.apache.hadoop.hive.cli.TestHBaseCliDriver.testCliDriver_hbase_ppd_join org.apache.hadoop.hive.cli.TestHBaseCliDriver.testCliDriver_hbase_ppd_key_range org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_acid_globallimit org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_mergejoin org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_orc_merge5 org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_orc_merge6 org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_orc_merge7 org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_orc_merge_incompat2 org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_orc_ppd_basic org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_orc_vectorization_ppd org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_vectorization_pushdown org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_vectorized_distinct_gby org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_vectorized_dynamic_partition_pruning org.apache.hadoop.hive.llap.daemon.impl.TestLlapTokenChecker.testCheckPermissions org.apache.hadoop.hive.llap.daemon.impl.TestLlapTokenChecker.testGetToken org.apache.hadoop.hive.metastore.TestMetaStoreMetrics.testConnections org.apache.hadoop.hive.ql.TestTxnCommands2.testNonAcidToAcidConversion2 org.apache.hadoop.hive.ql.TestTxnCommands2.testNonAcidToAcidConversion3 org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testACIDReaderFooterSerialize org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testACIDReaderFooterSerializeWithDeltas org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testACIDReaderNoFooterSerialize org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testACIDReaderNoFooterSerializeWithDeltas org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testDefaultTypes org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testDoAs org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testEmptyFile org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testInOutFormat org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testMROutput org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testNonVectorReaderFooterSerialize org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testNonVectorReaderNoFooterSerialize org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testSplitElimination org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testSplitEliminationNullStats org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testSplitGenReadOps org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testSplitGenReadOpsLocalCache org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testSplitGenReadOpsLocalCacheChangeFileLen org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testSplitGenReadOpsLocalCacheChangeModificationTime org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testVectorReaderFooterSerialize org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testVectorReaderNoFooterSerialize org.apache.hadoop.hive.ql.io.orc.TestNewInputOutputFormat.testNewInputFormat org.apache.hadoop.hive.ql.io.orc.TestNewInputOutputFormat.testNewInputFormatPruning org.apache.hadoop.hive.ql.io.orc.TestOrcRawRecordMerger.testRecordReaderDelta org.apache.hadoop.hive.ql.io.orc.TestOrcRawRecordMerger.testRecordReaderIncompleteDelta org.apache.hadoop.hive.ql.io.orc.TestOrcRawRecordMerger.testRecordReaderNewBaseAndDelta org.apache.hadoop.hive.ql.io.orc.TestOrcRawRecordMerger.testRecordReaderOldBaseAndDelta org.apache.hadoop.hive.ql.io.orc.TestOrcSplitElimination.testExternalFooterCache org.apache.hadoop.hive.ql.io.orc.TestOrcSplitElimination.testExternalFooterCachePpd org.apache.hadoop.hive.ql.io.orc.TestOrcSplitElimination.testSplitEliminationComplexExpr org.apache.hadoop.hive.ql.io.orc.TestOrcSplitElimination.testSplitEliminationLargeMaxSplit org.apache.hadoop.hive.ql.io.orc.TestOrcSplitElimination.testSplitEliminationSmallMaxSplit org.apache.hive.hcatalog.mapreduce.TestHCatDynamicPartitioned.testHCatDynamicPartitionedTableMultipleTask[3] org.apache.hive.hcatalog.mapreduce.TestHCatDynamicPartitioned.testHCatDynamicPartitionedTable[3] org.apache.hive.hcatalog.mapreduce.TestHCatExternalDynamicPartitioned.testHCatDynamicPartitionedTableMultipleTask[3] org.apache.hive.hcatalog.mapreduce.TestHCatExternalDynamicPartitioned.testHCatDynamicPartitionedTable[3] org.apache.hive.hcatalog.mapreduce.TestHCatExternalDynamicPartitioned.testHCatExternalDynamicCustomLocation[3] org.apache.hive.hcatalog.mapreduce.TestHCatExternalNonPartitioned.testHCatNonPartitionedTable[3] org.apache.hive.hcatalog.mapreduce.TestHCatExternalPartitioned.testHCatPartitionedTable[3] org.apache.hive.hcatalog.mapreduce.TestHCatMutableDynamicPartitioned.testHCatDynamicPartitionedTableMultipleTask[3] org.apache.hive.hcatalog.mapreduce.TestHCatMutableDynamicPartitioned.testHCatDynamicPartitionedTable[3] org.apache.hive.hcatalog.mapreduce.TestHCatMutableNonPartitioned.testHCatNonPartitionedTable[3] org.apache.hive.hcatalog.mapreduce.TestHCatMutablePartitioned.testHCatPartitionedTable[3] org.apache.hive.hcatalog.mapreduce.TestHCatNonPartitioned.testHCatNonPartitionedTable[3] org.apache.hive.hcatalog.mapreduce.TestHCatPartitioned.testHCatPartitionedTable[3] org.apache.hive.hcatalog.pig.TestE2EScenarios.testReadOrcAndRCFromPig org.apache.hive.hcatalog.pig.TestHCatLoader.testColumnarStorePushdown2[3] org.apache.hive.hcatalog.pig.TestHCatLoader.testColumnarStorePushdown[3] org.apache.hive.hcatalog.pig.TestHCatLoader.testProjectionsBasic[3] org.apache.hive.hcatalog.pig.TestHCatLoader.testReadDataBasic[3] org.apache.hive.hcatalog.pig.TestHCatLoader.testReadPartitionedBasic[3] org.apache.hive.hcatalog.pig.TestHCatLoaderComplexSchema.testMapNullKey[3] org.apache.hive.hcatalog.pig.TestHCatLoaderComplexSchema.testMapWithComplexData[3] org.apache.hive.hcatalog.pig.TestHCatLoaderComplexSchema.testSyntheticComplexSchema[3] org.apache.hive.hcatalog.pig.TestHCatLoaderComplexSchema.testTupleInBagInTupleInBag[3] org.apache.hive.hcatalog.pig.TestHCatStorer.testDateCharTypes[3] org.apache.hive.hcatalog.pig.TestHCatStorer.testPartColsInData[3] org.apache.hive.hcatalog.pig.TestHCatStorer.testStoreFuncAllSimpleTypes[3] org.apache.hive.hcatalog.pig.TestHCatStorer.testStoreInPartiitonedTbl[3] org.apache.hive.hcatalog.pig.TestHCatStorer.testWriteChar[3] org.apache.hive.hcatalog.pig.TestHCatStorer.testWriteDate2[3] org.apache.hive.hcatalog.pig.TestHCatStorer.testWriteDate3[3] org.apache.hive.hcatalog.pig.TestHCatStorer.testWriteDate[3] org.apache.hive.hcatalog.pig.TestHCatStorer.testWriteDecimalXY[3] org.apache.hive.hcatalog.pig.TestHCatStorer.testWriteDecimalX[3] org.apache.hive.hcatalog.pig.TestHCatStorer.testWriteDecimal[3] org.apache.hive.hcatalog.pig.TestHCatStorer.testWriteSmallint[3] org.apache.hive.hcatalog.pig.TestHCatStorer.testWriteTimestamp[3] org.apache.hive.hcatalog.pig.TestHCatStorer.testWriteTinyint[3] org.apache.hive.hcatalog.pig.TestHCatStorer.testWriteVarchar[3] org.apache.hive.hcatalog.streaming.TestStreaming.testInterleavedTransactionBatchCommits org.apache.hive.hcatalog.streaming.TestStreaming.testMultipleTransactionBatchCommits org.apache.hive.hcatalog.streaming.TestStreaming.testTransactionBatchAbortAndCommit org.apache.hive.hcatalog.streaming.TestStreaming.testTransactionBatchCommit_Delimited org.apache.hive.hcatalog.streaming.TestStreaming.testTransactionBatchCommit_DelimitedUGI org.apache.hive.hcatalog.streaming.TestStreaming.testTransactionBatchCommit_Json org.apache.hive.hcatalog.streaming.mutate.TestMutations.testMulti org.apache.hive.hcatalog.streaming.mutate.TestMutations.testTransactionBatchCommitPartitioned org.apache.hive.hcatalog.streaming.mutate.TestMutations.testTransactionBatchCommitUnpartitioned org.apache.hive.hcatalog.streaming.mutate.TestMutations.testUpdatesAndDeletes Test results: https://builds.apache.org/job/PreCommit-HIVE-MASTER-Build/536/testReport Console output: https://builds.apache.org/job/PreCommit-HIVE-MASTER-Build/536/console Test logs: http://ec2-204-236-174-241.us-west-1.compute.amazonaws.com/logs/PreCommit-HIVE-MASTER-Build-536/ Messages: Executing org.apache.hive.ptest.execution.TestCheckPhase Executing org.apache.hive.ptest.execution.PrepPhase Executing org.apache.hive.ptest.execution.ExecutionPhase Executing org.apache.hive.ptest.execution.ReportingPhase Tests exited with: TestsFailedException: 101 tests failed This message is automatically generated. ATTACHMENT ID: 12818138 - PreCommit-HIVE-MASTER-Build
            mmccline Matt McCline added a comment -

            (Tolerate no MapWork)

            mmccline Matt McCline added a comment - (Tolerate no MapWork)
            hiveqa Hive QA added a comment -

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

            ERROR: -1 due to build exiting with an error

            Test results: https://builds.apache.org/job/PreCommit-HIVE-MASTER-Build/554/testReport
            Console output: https://builds.apache.org/job/PreCommit-HIVE-MASTER-Build/554/console
            Test logs: http://ec2-204-236-174-241.us-west-1.compute.amazonaws.com/logs/PreCommit-HIVE-MASTER-Build-554/

            Messages:

            Executing org.apache.hive.ptest.execution.TestCheckPhase
            Executing org.apache.hive.ptest.execution.PrepPhase
            Tests exited with: NonZeroExitCodeException
            Command 'bash /data/hive-ptest/working/scratch/source-prep.sh' failed with exit status 1 and output '+ [[ -n /usr/java/jdk1.8.0_25 ]]
            + export JAVA_HOME=/usr/java/jdk1.8.0_25
            + JAVA_HOME=/usr/java/jdk1.8.0_25
            + export PATH=/usr/java/jdk1.8.0_25/bin/:/usr/local/bin:/usr/bin:/bin:/usr/local/games:/usr/games
            + PATH=/usr/java/jdk1.8.0_25/bin/:/usr/local/bin:/usr/bin:/bin:/usr/local/games:/usr/games
            + export 'ANT_OPTS=-Xmx1g -XX:MaxPermSize=256m '
            + ANT_OPTS='-Xmx1g -XX:MaxPermSize=256m '
            + export 'M2_OPTS=-Xmx1g -XX:MaxPermSize=256m -Dhttp.proxyHost=localhost -Dhttp.proxyPort=3128'
            + M2_OPTS='-Xmx1g -XX:MaxPermSize=256m -Dhttp.proxyHost=localhost -Dhttp.proxyPort=3128'
            + cd /data/hive-ptest/working/
            + tee /data/hive-ptest/logs/PreCommit-HIVE-MASTER-Build-554/source-prep.txt
            + [[ false == \t\r\u\e ]]
            + mkdir -p maven ivy
            + [[ git = \s\v\n ]]
            + [[ git = \g\i\t ]]
            + [[ -z master ]]
            + [[ -d apache-github-source-source ]]
            + [[ ! -d apache-github-source-source/.git ]]
            + [[ ! -d apache-github-source-source ]]
            + cd apache-github-source-source
            + git fetch origin
            + git reset --hard HEAD
            HEAD is now at 9764966 HIVE-14226 : Invalid check on an ASTNode#toStringTree in CalcitePlanner (Barna Zsombor Klara via Jesus Camacho Rodriguez)
            + git clean -f -d
            Removing common/src/test/org/apache/hadoop/hive/common/TestFileUtils.java
            + git checkout master
            Already on 'master'
            Your branch is up-to-date with 'origin/master'.
            + git reset --hard origin/master
            HEAD is now at 9764966 HIVE-14226 : Invalid check on an ASTNode#toStringTree in CalcitePlanner (Barna Zsombor Klara via Jesus Camacho Rodriguez)
            + git merge --ff-only origin/master
            Already up-to-date.
            + git gc
            + patchCommandPath=/data/hive-ptest/working/scratch/smart-apply-patch.sh
            + patchFilePath=/data/hive-ptest/working/scratch/build.patch
            + [[ -f /data/hive-ptest/working/scratch/build.patch ]]
            + chmod +x /data/hive-ptest/working/scratch/smart-apply-patch.sh
            + /data/hive-ptest/working/scratch/smart-apply-patch.sh /data/hive-ptest/working/scratch/build.patch
            The patch does not appear to apply with p0, p1, or p2
            + exit 1
            '
            

            This message is automatically generated.

            ATTACHMENT ID: 12818441 - PreCommit-HIVE-MASTER-Build

            hiveqa Hive QA added a comment - Here are the results of testing the latest attachment: https://issues.apache.org/jira/secure/attachment/12818441/HIVE-14214.02.patch ERROR: -1 due to build exiting with an error Test results: https://builds.apache.org/job/PreCommit-HIVE-MASTER-Build/554/testReport Console output: https://builds.apache.org/job/PreCommit-HIVE-MASTER-Build/554/console Test logs: http://ec2-204-236-174-241.us-west-1.compute.amazonaws.com/logs/PreCommit-HIVE-MASTER-Build-554/ Messages: Executing org.apache.hive.ptest.execution.TestCheckPhase Executing org.apache.hive.ptest.execution.PrepPhase Tests exited with: NonZeroExitCodeException Command 'bash /data/hive-ptest/working/scratch/source-prep.sh' failed with exit status 1 and output '+ [[ -n /usr/java/jdk1.8.0_25 ]] + export JAVA_HOME=/usr/java/jdk1.8.0_25 + JAVA_HOME=/usr/java/jdk1.8.0_25 + export PATH=/usr/java/jdk1.8.0_25/bin/:/usr/local/bin:/usr/bin:/bin:/usr/local/games:/usr/games + PATH=/usr/java/jdk1.8.0_25/bin/:/usr/local/bin:/usr/bin:/bin:/usr/local/games:/usr/games + export 'ANT_OPTS=-Xmx1g -XX:MaxPermSize=256m ' + ANT_OPTS='-Xmx1g -XX:MaxPermSize=256m ' + export 'M2_OPTS=-Xmx1g -XX:MaxPermSize=256m -Dhttp.proxyHost=localhost -Dhttp.proxyPort=3128' + M2_OPTS='-Xmx1g -XX:MaxPermSize=256m -Dhttp.proxyHost=localhost -Dhttp.proxyPort=3128' + cd /data/hive-ptest/working/ + tee /data/hive-ptest/logs/PreCommit-HIVE-MASTER-Build-554/source-prep.txt + [[ false == \t\r\u\e ]] + mkdir -p maven ivy + [[ git = \s\v\n ]] + [[ git = \g\i\t ]] + [[ -z master ]] + [[ -d apache-github-source-source ]] + [[ ! -d apache-github-source-source/.git ]] + [[ ! -d apache-github-source-source ]] + cd apache-github-source-source + git fetch origin + git reset --hard HEAD HEAD is now at 9764966 HIVE-14226 : Invalid check on an ASTNode#toStringTree in CalcitePlanner (Barna Zsombor Klara via Jesus Camacho Rodriguez) + git clean -f -d Removing common/src/test/org/apache/hadoop/hive/common/TestFileUtils.java + git checkout master Already on 'master' Your branch is up-to-date with 'origin/master'. + git reset --hard origin/master HEAD is now at 9764966 HIVE-14226 : Invalid check on an ASTNode#toStringTree in CalcitePlanner (Barna Zsombor Klara via Jesus Camacho Rodriguez) + git merge --ff-only origin/master Already up-to-date. + git gc + patchCommandPath=/data/hive-ptest/working/scratch/smart-apply-patch.sh + patchFilePath=/data/hive-ptest/working/scratch/build.patch + [[ -f /data/hive-ptest/working/scratch/build.patch ]] + chmod +x /data/hive-ptest/working/scratch/smart-apply-patch.sh + /data/hive-ptest/working/scratch/smart-apply-patch.sh /data/hive-ptest/working/scratch/build.patch The patch does not appear to apply with p0, p1, or p2 + exit 1 ' This message is automatically generated. ATTACHMENT ID: 12818441 - PreCommit-HIVE-MASTER-Build
            hiveqa Hive QA added a comment -

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

            SUCCESS: +1 due to 2 test(s) being added or modified.

            ERROR: -1 due to 15 failed/errored test(s), 10333 tests executed
            Failed tests:

            org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_acid_globallimit
            org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_parquet_join
            org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_stats_list_bucket
            org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_subquery_multiinsert
            org.apache.hadoop.hive.cli.TestHBaseCliDriver.testCliDriver_hbase_ppd_join
            org.apache.hadoop.hive.cli.TestHBaseCliDriver.testCliDriver_hbase_ppd_key_range
            org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_acid_globallimit
            org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_vectorized_distinct_gby
            org.apache.hadoop.hive.llap.daemon.impl.TestLlapTokenChecker.testCheckPermissions
            org.apache.hadoop.hive.llap.daemon.impl.TestLlapTokenChecker.testGetToken
            org.apache.hadoop.hive.metastore.TestMetaStoreMetrics.testConnections
            org.apache.hadoop.hive.ql.TestTxnCommands2.testNonAcidToAcidConversion2
            org.apache.hadoop.hive.ql.TestTxnCommands2.testNonAcidToAcidConversion3
            org.apache.hadoop.hive.ql.io.orc.TestOrcSplitElimination.testExternalFooterCache
            org.apache.hadoop.hive.ql.io.orc.TestOrcSplitElimination.testExternalFooterCachePpd
            

            Test results: https://builds.apache.org/job/PreCommit-HIVE-MASTER-Build/556/testReport
            Console output: https://builds.apache.org/job/PreCommit-HIVE-MASTER-Build/556/console
            Test logs: http://ec2-204-236-174-241.us-west-1.compute.amazonaws.com/logs/PreCommit-HIVE-MASTER-Build-556/

            Messages:

            Executing org.apache.hive.ptest.execution.TestCheckPhase
            Executing org.apache.hive.ptest.execution.PrepPhase
            Executing org.apache.hive.ptest.execution.ExecutionPhase
            Executing org.apache.hive.ptest.execution.ReportingPhase
            Tests exited with: TestsFailedException: 15 tests failed
            

            This message is automatically generated.

            ATTACHMENT ID: 12818466 - PreCommit-HIVE-MASTER-Build

            hiveqa Hive QA added a comment - Here are the results of testing the latest attachment: https://issues.apache.org/jira/secure/attachment/12818466/HIVE-14214.03.patch SUCCESS: +1 due to 2 test(s) being added or modified. ERROR: -1 due to 15 failed/errored test(s), 10333 tests executed Failed tests: org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_acid_globallimit org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_parquet_join org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_stats_list_bucket org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_subquery_multiinsert org.apache.hadoop.hive.cli.TestHBaseCliDriver.testCliDriver_hbase_ppd_join org.apache.hadoop.hive.cli.TestHBaseCliDriver.testCliDriver_hbase_ppd_key_range org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_acid_globallimit org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_vectorized_distinct_gby org.apache.hadoop.hive.llap.daemon.impl.TestLlapTokenChecker.testCheckPermissions org.apache.hadoop.hive.llap.daemon.impl.TestLlapTokenChecker.testGetToken org.apache.hadoop.hive.metastore.TestMetaStoreMetrics.testConnections org.apache.hadoop.hive.ql.TestTxnCommands2.testNonAcidToAcidConversion2 org.apache.hadoop.hive.ql.TestTxnCommands2.testNonAcidToAcidConversion3 org.apache.hadoop.hive.ql.io.orc.TestOrcSplitElimination.testExternalFooterCache org.apache.hadoop.hive.ql.io.orc.TestOrcSplitElimination.testExternalFooterCachePpd Test results: https://builds.apache.org/job/PreCommit-HIVE-MASTER-Build/556/testReport Console output: https://builds.apache.org/job/PreCommit-HIVE-MASTER-Build/556/console Test logs: http://ec2-204-236-174-241.us-west-1.compute.amazonaws.com/logs/PreCommit-HIVE-MASTER-Build-556/ Messages: Executing org.apache.hive.ptest.execution.TestCheckPhase Executing org.apache.hive.ptest.execution.PrepPhase Executing org.apache.hive.ptest.execution.ExecutionPhase Executing org.apache.hive.ptest.execution.ReportingPhase Tests exited with: TestsFailedException: 15 tests failed This message is automatically generated. ATTACHMENT ID: 12818466 - PreCommit-HIVE-MASTER-Build
            hiveqa Hive QA added a comment -

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

            SUCCESS: +1 due to 2 test(s) being added or modified.

            ERROR: -1 due to 18 failed/errored test(s), 10338 tests executed
            Failed tests:

            org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_acid_globallimit
            org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_list_bucket_dml_12
            org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_list_bucket_dml_13
            org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_parquet_join
            org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_stats_list_bucket
            org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_subquery_multiinsert
            org.apache.hadoop.hive.cli.TestHBaseCliDriver.testCliDriver_hbase_ppd_join
            org.apache.hadoop.hive.cli.TestHBaseCliDriver.testCliDriver_hbase_ppd_key_range
            org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_acid_globallimit
            org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_vectorized_distinct_gby
            org.apache.hadoop.hive.llap.daemon.impl.TestLlapTokenChecker.testCheckPermissions
            org.apache.hadoop.hive.llap.daemon.impl.TestLlapTokenChecker.testGetToken
            org.apache.hadoop.hive.llap.daemon.impl.TestTaskExecutorService.testWaitQueuePreemption
            org.apache.hadoop.hive.metastore.TestMetaStoreMetrics.testConnections
            org.apache.hadoop.hive.ql.TestTxnCommands2.testNonAcidToAcidConversion2
            org.apache.hadoop.hive.ql.TestTxnCommands2.testNonAcidToAcidConversion3
            org.apache.hadoop.hive.ql.io.orc.TestOrcSplitElimination.testExternalFooterCache
            org.apache.hadoop.hive.ql.io.orc.TestOrcSplitElimination.testExternalFooterCachePpd
            

            Test results: https://builds.apache.org/job/PreCommit-HIVE-MASTER-Build/576/testReport
            Console output: https://builds.apache.org/job/PreCommit-HIVE-MASTER-Build/576/console
            Test logs: http://ec2-204-236-174-241.us-west-1.compute.amazonaws.com/logs/PreCommit-HIVE-MASTER-Build-576/

            Messages:

            Executing org.apache.hive.ptest.execution.TestCheckPhase
            Executing org.apache.hive.ptest.execution.PrepPhase
            Executing org.apache.hive.ptest.execution.ExecutionPhase
            Executing org.apache.hive.ptest.execution.ReportingPhase
            Tests exited with: TestsFailedException: 18 tests failed
            

            This message is automatically generated.

            ATTACHMENT ID: 12818736 - PreCommit-HIVE-MASTER-Build

            hiveqa Hive QA added a comment - Here are the results of testing the latest attachment: https://issues.apache.org/jira/secure/attachment/12818736/HIVE-14214.04.patch SUCCESS: +1 due to 2 test(s) being added or modified. ERROR: -1 due to 18 failed/errored test(s), 10338 tests executed Failed tests: org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_acid_globallimit org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_list_bucket_dml_12 org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_list_bucket_dml_13 org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_parquet_join org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_stats_list_bucket org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_subquery_multiinsert org.apache.hadoop.hive.cli.TestHBaseCliDriver.testCliDriver_hbase_ppd_join org.apache.hadoop.hive.cli.TestHBaseCliDriver.testCliDriver_hbase_ppd_key_range org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_acid_globallimit org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_vectorized_distinct_gby org.apache.hadoop.hive.llap.daemon.impl.TestLlapTokenChecker.testCheckPermissions org.apache.hadoop.hive.llap.daemon.impl.TestLlapTokenChecker.testGetToken org.apache.hadoop.hive.llap.daemon.impl.TestTaskExecutorService.testWaitQueuePreemption org.apache.hadoop.hive.metastore.TestMetaStoreMetrics.testConnections org.apache.hadoop.hive.ql.TestTxnCommands2.testNonAcidToAcidConversion2 org.apache.hadoop.hive.ql.TestTxnCommands2.testNonAcidToAcidConversion3 org.apache.hadoop.hive.ql.io.orc.TestOrcSplitElimination.testExternalFooterCache org.apache.hadoop.hive.ql.io.orc.TestOrcSplitElimination.testExternalFooterCachePpd Test results: https://builds.apache.org/job/PreCommit-HIVE-MASTER-Build/576/testReport Console output: https://builds.apache.org/job/PreCommit-HIVE-MASTER-Build/576/console Test logs: http://ec2-204-236-174-241.us-west-1.compute.amazonaws.com/logs/PreCommit-HIVE-MASTER-Build-576/ Messages: Executing org.apache.hive.ptest.execution.TestCheckPhase Executing org.apache.hive.ptest.execution.PrepPhase Executing org.apache.hive.ptest.execution.ExecutionPhase Executing org.apache.hive.ptest.execution.ReportingPhase Tests exited with: TestsFailedException: 18 tests failed This message is automatically generated. ATTACHMENT ID: 12818736 - PreCommit-HIVE-MASTER-Build
            mmccline Matt McCline added a comment -

            Ok, review board created. New patch submitted for Hive QA.

            mmccline Matt McCline added a comment - Ok, review board created. New patch submitted for Hive QA.

            Left some comments in RB.

            I think many of the complications in the patch can be avoided if we just provide a Reader api that returns true if there is conversion. Reader.hasConversion(). Behind the scenes we should do all the magic of determining if the conversion is required based on reader schema, file schema and included. In OrcInputFormat the only place we need to disable PPD is in ETL strategy, which creates ORC reader. If this reader returns hasConversion() then we should disable PPD. Similarly for task side.

            prasanth_j Prasanth Jayachandran added a comment - Left some comments in RB. I think many of the complications in the patch can be avoided if we just provide a Reader api that returns true if there is conversion. Reader.hasConversion(). Behind the scenes we should do all the magic of determining if the conversion is required based on reader schema, file schema and included. In OrcInputFormat the only place we need to disable PPD is in ETL strategy, which creates ORC reader. If this reader returns hasConversion() then we should disable PPD. Similarly for task side.
            omalley Owen O'Malley added a comment -

            I'm looking through the patch too.

            We will need unit tests for this patch since it changes the ORC module.

            I'm also worried that by fixing the Hive layer above ORC, we'll expose the other users to the same problem. I think we need to fix the ppd evaluation within ORC so that other clients also work correctly.

            omalley Owen O'Malley added a comment - I'm looking through the patch too. We will need unit tests for this patch since it changes the ORC module. I'm also worried that by fixing the Hive layer above ORC, we'll expose the other users to the same problem. I think we need to fix the ppd evaluation within ORC so that other clients also work correctly.
            mmccline Matt McCline added a comment -

            Agreed except I don't have time to rework it right now. Currently the RecordReader has the SchemaEvolution not the Reader. I'll create another JIRA for reworking this later.

            mmccline Matt McCline added a comment - Agreed except I don't have time to rework it right now. Currently the RecordReader has the SchemaEvolution not the Reader. I'll create another JIRA for reworking this later.

            I've spoken to prasanth_j offline. He's willing to take up the refactoring and enablement of safe PPD in HIVE-14310, based on this patch. With this in mind, I'm +1 on this patch.

            hagleitn Gunther Hagleitner added a comment - I've spoken to prasanth_j offline. He's willing to take up the refactoring and enablement of safe PPD in HIVE-14310 , based on this patch. With this in mind, I'm +1 on this patch.
            mmccline Matt McCline added a comment -

            Rebased due to recent SchemaEvolution.java change.

            mmccline Matt McCline added a comment - Rebased due to recent SchemaEvolution.java change.
            hiveqa Hive QA added a comment -

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

            SUCCESS: +1 due to 7 test(s) being added or modified.

            ERROR: -1 due to 13 failed/errored test(s), 10349 tests executed
            Failed tests:

            TestMsgBusConnection - did not produce a TEST-*.xml file
            org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_acid_globallimit
            org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_list_bucket_dml_12
            org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_list_bucket_dml_13
            org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_stats_list_bucket
            org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_subquery_multiinsert
            org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_acid_globallimit
            org.apache.hadoop.hive.llap.daemon.impl.TestLlapTokenChecker.testCheckPermissions
            org.apache.hadoop.hive.llap.daemon.impl.TestLlapTokenChecker.testGetToken
            org.apache.hadoop.hive.metastore.TestMetaStoreMetrics.testConnections
            org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testVectorizationWithAcid
            org.apache.hadoop.hive.ql.io.orc.TestOrcRawRecordMerger.testRecordReaderIncompleteDelta
            org.apache.hadoop.hive.ql.io.orc.TestOrcRawRecordMerger.testRecordReaderNewBaseAndDelta
            

            Test results: https://builds.apache.org/job/PreCommit-HIVE-MASTER-Build/596/testReport
            Console output: https://builds.apache.org/job/PreCommit-HIVE-MASTER-Build/596/console
            Test logs: http://ec2-204-236-174-241.us-west-1.compute.amazonaws.com/logs/PreCommit-HIVE-MASTER-Build-596/

            Messages:

            Executing org.apache.hive.ptest.execution.TestCheckPhase
            Executing org.apache.hive.ptest.execution.PrepPhase
            Executing org.apache.hive.ptest.execution.ExecutionPhase
            Executing org.apache.hive.ptest.execution.ReportingPhase
            Tests exited with: TestsFailedException: 13 tests failed
            

            This message is automatically generated.

            ATTACHMENT ID: 12819507 - PreCommit-HIVE-MASTER-Build

            hiveqa Hive QA added a comment - Here are the results of testing the latest attachment: https://issues.apache.org/jira/secure/attachment/12819507/HIVE-14214.08.patch SUCCESS: +1 due to 7 test(s) being added or modified. ERROR: -1 due to 13 failed/errored test(s), 10349 tests executed Failed tests: TestMsgBusConnection - did not produce a TEST-*.xml file org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_acid_globallimit org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_list_bucket_dml_12 org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_list_bucket_dml_13 org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_stats_list_bucket org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_subquery_multiinsert org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_acid_globallimit org.apache.hadoop.hive.llap.daemon.impl.TestLlapTokenChecker.testCheckPermissions org.apache.hadoop.hive.llap.daemon.impl.TestLlapTokenChecker.testGetToken org.apache.hadoop.hive.metastore.TestMetaStoreMetrics.testConnections org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat.testVectorizationWithAcid org.apache.hadoop.hive.ql.io.orc.TestOrcRawRecordMerger.testRecordReaderIncompleteDelta org.apache.hadoop.hive.ql.io.orc.TestOrcRawRecordMerger.testRecordReaderNewBaseAndDelta Test results: https://builds.apache.org/job/PreCommit-HIVE-MASTER-Build/596/testReport Console output: https://builds.apache.org/job/PreCommit-HIVE-MASTER-Build/596/console Test logs: http://ec2-204-236-174-241.us-west-1.compute.amazonaws.com/logs/PreCommit-HIVE-MASTER-Build-596/ Messages: Executing org.apache.hive.ptest.execution.TestCheckPhase Executing org.apache.hive.ptest.execution.PrepPhase Executing org.apache.hive.ptest.execution.ExecutionPhase Executing org.apache.hive.ptest.execution.ReportingPhase Tests exited with: TestsFailedException: 13 tests failed This message is automatically generated. ATTACHMENT ID: 12819507 - PreCommit-HIVE-MASTER-Build
            omalley Owen O'Malley added a comment - - edited

            This patch still introduces the redundant SchemaEvolution.readerFileTypes, which exactly duplicates the values in SchemaEvolution.readerToFile.

            It also has spurious trailing spaces at the end of some of the qfile tests.

            omalley Owen O'Malley added a comment - - edited This patch still introduces the redundant SchemaEvolution.readerFileTypes, which exactly duplicates the values in SchemaEvolution.readerToFile. It also has spurious trailing spaces at the end of some of the qfile tests.
            omalley Owen O'Malley added a comment -

            I should point out that an array is a better type than the Map.

            omalley Owen O'Malley added a comment - I should point out that an array is a better type than the Map.
            mmccline Matt McCline added a comment -

            Stil have 2 issues with "mvn test -Dtest=TestOrcRawRecordMerger"

            mmccline Matt McCline added a comment - Stil have 2 issues with "mvn test -Dtest=TestOrcRawRecordMerger"
            mmccline Matt McCline added a comment -

            An internal ptest pass succeeded with no related failures (#153). Committed to master.

            mmccline Matt McCline added a comment - An internal ptest pass succeeded with no related failures (#153). Committed to master.

            mmccline Can you plz backport this to branch-2.1? This is required for HIVE-14333 backport.

            prasanth_j Prasanth Jayachandran added a comment - mmccline Can you plz backport this to branch-2.1? This is required for HIVE-14333 backport.
            mmccline Matt McCline added a comment -

            Also committed to branch-2.1

            mmccline Matt McCline added a comment - Also committed to branch-2.1

            People

              mmccline Matt McCline
              mmccline Matt McCline
              Votes:
              0 Vote for this issue
              Watchers:
              7 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: