Description
In Schema Evolution, the reader schema is different than the file schema which is used to evaluate predicate push down.
Attachments
Attachments
- HIVE-14214.01.patch
- 1.57 MB
- Matt McCline
- HIVE-14214.02.patch
- 425 kB
- Matt McCline
- HIVE-14214.03.patch
- 423 kB
- Matt McCline
- HIVE-14214.04.patch
- 424 kB
- Matt McCline
- HIVE-14214.05.patch
- 442 kB
- Matt McCline
- HIVE-14214.06.patch
- 433 kB
- Matt McCline
- HIVE-14214.07.patch
- 439 kB
- Matt McCline
- HIVE-14214.08.patch
- 439 kB
- Matt McCline
- HIVE-14214.09.patch
- 442 kB
- Matt McCline
- HIVE-14214.WIP.patch
- 61 kB
- Matt McCline
Issue Links
- is related to
-
HIVE-14333 ORC schema evolution from float to double changes precision and breaks filters
- Closed
- relates to
-
HIVE-14310 ORC schema evolution should not completely disable PPD
- Closed
- links to
Activity
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
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...
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
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
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
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
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.
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.
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.
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
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.
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.
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.