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

Hive failing on insert empty array into parquet table

Details

    Description

      The insert will fail with following stack:

      by: parquet.io.ParquetEncodingException: empty fields are illegal, the field should be ommited completely instead
      	at parquet.io.MessageColumnIO$MessageColumnIORecordConsumer.endField(MessageColumnIO.java:271)
      	at org.apache.hadoop.hive.ql.io.parquet.write.DataWritableWriter$ListDataWriter.write(DataWritableWriter.java:271)
      	at org.apache.hadoop.hive.ql.io.parquet.write.DataWritableWriter$GroupDataWriter.write(DataWritableWriter.java:199)
      	at org.apache.hadoop.hive.ql.io.parquet.write.DataWritableWriter$MessageDataWriter.write(DataWritableWriter.java:215)
      	at org.apache.hadoop.hive.ql.io.parquet.write.DataWritableWriter.write(DataWritableWriter.java:88)
      	at org.apache.hadoop.hive.ql.io.parquet.write.DataWritableWriteSupport.write(DataWritableWriteSupport.java:59)
      	at org.apache.hadoop.hive.ql.io.parquet.write.DataWritableWriteSupport.write(DataWritableWriteSupport.java:31)
      	at parquet.hadoop.InternalParquetRecordWriter.write(InternalParquetRecordWriter.java:116)
      	at parquet.hadoop.ParquetRecordWriter.write(ParquetRecordWriter.java:123)
      	at parquet.hadoop.ParquetRecordWriter.write(ParquetRecordWriter.java:42)
      	at org.apache.hadoop.hive.ql.io.parquet.write.ParquetRecordWriterWrapper.write(ParquetRecordWriterWrapper.java:111)
      	at org.apache.hadoop.hive.ql.io.parquet.write.ParquetRecordWriterWrapper.write(ParquetRecordWriterWrapper.java:124)
      	at org.apache.hadoop.hive.ql.exec.FileSinkOperator.processOp(FileSinkOperator.java:697)
      

      Reproduce:

      create table test_small (
      key string,
      arrayValues array<string>)
      stored as parquet;
      insert into table test_small select 'abcd', array() from src limit 1;
      

      Attachments

        1. HIVE-13632.3.patch
          14 kB
          Yongzhi Chen
        2. HIVE-13632.2.patch
          12 kB
          Yongzhi Chen
        3. HIVE-13632.1.patch
          8 kB
          Yongzhi Chen

        Issue Links

          Activity

            ychena Yongzhi Chen added a comment -

            patch 1 fixes the exception by avoid creating empty field.
            But parquet api seems have limitations, empty array and map are stored as null.

            ychena Yongzhi Chen added a comment - patch 1 fixes the exception by avoid creating empty field. But parquet api seems have limitations, empty array and map are stored as null.
            ychena Yongzhi Chen added a comment -

            spena, could you review the code?

            ychena Yongzhi Chen added a comment - spena , could you review the code?
            spena Sergio Peña added a comment -

            Looks good
            +1

            spena Sergio Peña added a comment - Looks good +1
            spena Sergio Peña added a comment -

            What do other file formats show with empty arrays? null or empty?

            spena Sergio Peña added a comment - What do other file formats show with empty arrays? null or empty?
            ychena Yongzhi Chen added a comment -

            They show empty. But I can not find a way in parquet.
            writeNullForMissingFieldsAtCurrentLevel(); in the java/parquet/io/MessageColumnIO.java may cause this.

            ychena Yongzhi Chen added a comment - They show empty. But I can not find a way in parquet. writeNullForMissingFieldsAtCurrentLevel(); in the java/parquet/io/MessageColumnIO.java may cause this.
            spena Sergio Peña added a comment -

            I think this should work in situations where the list does not contain values. Try it and see what it does.

            recordConsumer.startField(repeatedGroupName, 0);
            recordConsumer.startGroup()
            recordConsumer.endGroup()
            recordConsumer.endField(repeatedGroupName, 0);

            spena Sergio Peña added a comment - I think this should work in situations where the list does not contain values. Try it and see what it does. recordConsumer.startField(repeatedGroupName, 0); recordConsumer.startGroup() recordConsumer.endGroup() recordConsumer.endField(repeatedGroupName, 0);
            ychena Yongzhi Chen added a comment -

            spena, I think I tried it, the result is [null]

            ychena Yongzhi Chen added a comment - spena , I think I tried it, the result is [null]
            spena Sergio Peña added a comment -

            ychena I think we should try to be consistent with other file formats before committing this patch.
            Could you investigate what other format serde or object inspectors do in order to translate this to empty lists?

            spena Sergio Peña added a comment - ychena I think we should try to be consistent with other file formats before committing this patch. Could you investigate what other format serde or object inspectors do in order to translate this to empty lists?
            ychena Yongzhi Chen added a comment -

            Each serde is different, for example the avro, avro record has a map:
            record GenericData$Record (id=8454)
            {"key": "abcd", "arrayvalues": [], "mapvalues": {}}
            Each column name map to a value object. For empty array, it is an empty List object.
            The Serializing value is just convert the record directly by encoding:
            BinaryEncoder be = EncoderFactory.get().directBinaryEncoder((DataOutputStream)out, null);
            So it is very easy to translate.
            Our problem is, although hive know it is an empty list before Serializing to Parque, but it does not know how to tell Parque it is an empty.

            ychena Yongzhi Chen added a comment - Each serde is different, for example the avro, avro record has a map: record GenericData$Record (id=8454) {"key": "abcd", "arrayvalues": [], "mapvalues": {}} Each column name map to a value object. For empty array, it is an empty List object. The Serializing value is just convert the record directly by encoding: BinaryEncoder be = EncoderFactory.get().directBinaryEncoder((DataOutputStream)out, null); So it is very easy to translate. Our problem is, although hive know it is an empty list before Serializing to Parque, but it does not know how to tell Parque it is an empty.
            hiveqa Hive QA added a comment -

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

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

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

            TestHWISessionManager - did not produce a TEST-*.xml file
            TestMiniTezCliDriver-vector_non_string_partition.q-delete_where_non_partitioned.q-auto_sortmerge_join_16.q-and-12-more - did not produce a TEST-*.xml file
            TestMiniTezCliDriver-vectorization_16.q-vector_decimal_round.q-orc_merge6.q-and-12-more - did not produce a TEST-*.xml file
            org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_groupby1_limit
            org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_nomore_ambiguous_table_col
            org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_regexp_extract
            org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_bucket4
            org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_bucket5
            org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_bucket6
            org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_disable_merge_for_bucketing
            org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_index_bitmap3
            org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_infer_bucket_sort_map_operators
            org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_infer_bucket_sort_num_buckets
            org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_infer_bucket_sort_reducers_power_two
            org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_list_bucket_dml_10
            org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_orc_merge1
            org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_orc_merge2
            org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_orc_merge9
            org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_orc_merge_diff_fs
            org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_reduce_deduplicate
            org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_vector_outer_join1
            org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_vector_outer_join2
            org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_vector_outer_join3
            org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_vector_outer_join4
            org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_vector_outer_join5
            org.apache.hadoop.hive.cli.TestNegativeCliDriver.testNegativeCliDriver_clustern3
            org.apache.hadoop.hive.cli.TestNegativeCliDriver.testNegativeCliDriver_clustern4
            org.apache.hadoop.hive.cli.TestNegativeCliDriver.testNegativeCliDriver_nonkey_groupby
            org.apache.hadoop.hive.cli.TestNegativeCliDriver.testNegativeCliDriver_selectDistinctStarNeg_2
            org.apache.hadoop.hive.cli.TestNegativeCliDriver.testNegativeCliDriver_subquery_shared_alias
            org.apache.hadoop.hive.cli.TestNegativeCliDriver.testNegativeCliDriver_udtf_not_supported1
            org.apache.hadoop.hive.metastore.TestAuthzApiEmbedAuthorizerInRemote.org.apache.hadoop.hive.metastore.TestAuthzApiEmbedAuthorizerInRemote
            org.apache.hadoop.hive.metastore.TestFilterHooks.org.apache.hadoop.hive.metastore.TestFilterHooks
            org.apache.hadoop.hive.metastore.TestHiveMetaStoreGetMetaConf.testGetMetaConfDefault
            org.apache.hadoop.hive.metastore.TestHiveMetaStoreGetMetaConf.testGetMetaConfDefaultEmptyString
            org.apache.hadoop.hive.metastore.TestHiveMetaStoreGetMetaConf.testGetMetaConfOverridden
            org.apache.hadoop.hive.metastore.TestHiveMetaStoreGetMetaConf.testGetMetaConfUnknownPreperty
            org.apache.hadoop.hive.metastore.TestMetaStoreEndFunctionListener.testEndFunctionListener
            org.apache.hadoop.hive.metastore.TestMetaStoreEventListenerOnlyOnCommit.testEventStatus
            org.apache.hadoop.hive.metastore.TestMetaStoreInitListener.testMetaStoreInitListener
            org.apache.hadoop.hive.metastore.TestMetaStoreMetrics.org.apache.hadoop.hive.metastore.TestMetaStoreMetrics
            org.apache.hadoop.hive.metastore.TestPartitionNameWhitelistValidation.testAppendPartitionWithCommas
            org.apache.hadoop.hive.metastore.TestPartitionNameWhitelistValidation.testAppendPartitionWithUnicode
            org.apache.hadoop.hive.metastore.TestPartitionNameWhitelistValidation.testAppendPartitionWithValidCharacters
            org.apache.hadoop.hive.metastore.TestRetryingHMSHandler.testRetryingHMSHandler
            org.apache.hadoop.hive.ql.security.TestClientSideAuthorizationProvider.testSimplePrivileges
            org.apache.hadoop.hive.ql.security.TestExtendedAcls.org.apache.hadoop.hive.ql.security.TestExtendedAcls
            org.apache.hadoop.hive.ql.security.TestMetastoreAuthorizationProvider.testSimplePrivileges
            org.apache.hadoop.hive.ql.security.TestMultiAuthorizationPreEventListener.org.apache.hadoop.hive.ql.security.TestMultiAuthorizationPreEventListener
            org.apache.hadoop.hive.ql.security.TestStorageBasedClientSideAuthorizationProvider.testSimplePrivileges
            org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationProvider.testSimplePrivileges
            org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationProviderWithACL.testSimplePrivileges
            org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationReads.testReadDbFailure
            org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationReads.testReadDbSuccess
            org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationReads.testReadTableFailure
            org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationReads.testReadTableSuccess
            org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationReads.testReadTableSuccessWithReadOnly
            org.apache.hadoop.hive.thrift.TestHadoopAuthBridge23.testDelegationTokenSharedStore
            org.apache.hadoop.hive.thrift.TestHadoopAuthBridge23.testMetastoreProxyUser
            org.apache.hadoop.hive.thrift.TestHadoopAuthBridge23.testSaslWithHiveMetaStore
            org.apache.hive.hcatalog.api.repl.commands.TestCommands.org.apache.hive.hcatalog.api.repl.commands.TestCommands
            org.apache.hive.minikdc.TestJdbcWithDBTokenStore.testNegativeTokenAuth
            

            Test results: http://ec2-54-177-240-2.us-west-1.compute.amazonaws.com/job/PreCommit-HIVE-MASTER-Build/138/testReport
            Console output: http://ec2-54-177-240-2.us-west-1.compute.amazonaws.com/job/PreCommit-HIVE-MASTER-Build/138/console
            Test logs: http://ec2-50-18-27-0.us-west-1.compute.amazonaws.com/logs/PreCommit-HIVE-MASTER-Build-138/

            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: 62 tests failed
            

            This message is automatically generated.

            ATTACHMENT ID: 12801299 - 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/12801299/HIVE-13632.1.patch SUCCESS: +1 due to 1 test(s) being added or modified. ERROR: -1 due to 62 failed/errored test(s), 9966 tests executed Failed tests: TestHWISessionManager - did not produce a TEST-*.xml file TestMiniTezCliDriver-vector_non_string_partition.q-delete_where_non_partitioned.q-auto_sortmerge_join_16.q-and-12-more - did not produce a TEST-*.xml file TestMiniTezCliDriver-vectorization_16.q-vector_decimal_round.q-orc_merge6.q-and-12-more - did not produce a TEST-*.xml file org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_groupby1_limit org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_nomore_ambiguous_table_col org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_regexp_extract org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_bucket4 org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_bucket5 org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_bucket6 org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_disable_merge_for_bucketing org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_index_bitmap3 org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_infer_bucket_sort_map_operators org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_infer_bucket_sort_num_buckets org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_infer_bucket_sort_reducers_power_two org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_list_bucket_dml_10 org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_orc_merge1 org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_orc_merge2 org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_orc_merge9 org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_orc_merge_diff_fs org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_reduce_deduplicate org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_vector_outer_join1 org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_vector_outer_join2 org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_vector_outer_join3 org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_vector_outer_join4 org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_vector_outer_join5 org.apache.hadoop.hive.cli.TestNegativeCliDriver.testNegativeCliDriver_clustern3 org.apache.hadoop.hive.cli.TestNegativeCliDriver.testNegativeCliDriver_clustern4 org.apache.hadoop.hive.cli.TestNegativeCliDriver.testNegativeCliDriver_nonkey_groupby org.apache.hadoop.hive.cli.TestNegativeCliDriver.testNegativeCliDriver_selectDistinctStarNeg_2 org.apache.hadoop.hive.cli.TestNegativeCliDriver.testNegativeCliDriver_subquery_shared_alias org.apache.hadoop.hive.cli.TestNegativeCliDriver.testNegativeCliDriver_udtf_not_supported1 org.apache.hadoop.hive.metastore.TestAuthzApiEmbedAuthorizerInRemote.org.apache.hadoop.hive.metastore.TestAuthzApiEmbedAuthorizerInRemote org.apache.hadoop.hive.metastore.TestFilterHooks.org.apache.hadoop.hive.metastore.TestFilterHooks org.apache.hadoop.hive.metastore.TestHiveMetaStoreGetMetaConf.testGetMetaConfDefault org.apache.hadoop.hive.metastore.TestHiveMetaStoreGetMetaConf.testGetMetaConfDefaultEmptyString org.apache.hadoop.hive.metastore.TestHiveMetaStoreGetMetaConf.testGetMetaConfOverridden org.apache.hadoop.hive.metastore.TestHiveMetaStoreGetMetaConf.testGetMetaConfUnknownPreperty org.apache.hadoop.hive.metastore.TestMetaStoreEndFunctionListener.testEndFunctionListener org.apache.hadoop.hive.metastore.TestMetaStoreEventListenerOnlyOnCommit.testEventStatus org.apache.hadoop.hive.metastore.TestMetaStoreInitListener.testMetaStoreInitListener org.apache.hadoop.hive.metastore.TestMetaStoreMetrics.org.apache.hadoop.hive.metastore.TestMetaStoreMetrics org.apache.hadoop.hive.metastore.TestPartitionNameWhitelistValidation.testAppendPartitionWithCommas org.apache.hadoop.hive.metastore.TestPartitionNameWhitelistValidation.testAppendPartitionWithUnicode org.apache.hadoop.hive.metastore.TestPartitionNameWhitelistValidation.testAppendPartitionWithValidCharacters org.apache.hadoop.hive.metastore.TestRetryingHMSHandler.testRetryingHMSHandler org.apache.hadoop.hive.ql.security.TestClientSideAuthorizationProvider.testSimplePrivileges org.apache.hadoop.hive.ql.security.TestExtendedAcls.org.apache.hadoop.hive.ql.security.TestExtendedAcls org.apache.hadoop.hive.ql.security.TestMetastoreAuthorizationProvider.testSimplePrivileges org.apache.hadoop.hive.ql.security.TestMultiAuthorizationPreEventListener.org.apache.hadoop.hive.ql.security.TestMultiAuthorizationPreEventListener org.apache.hadoop.hive.ql.security.TestStorageBasedClientSideAuthorizationProvider.testSimplePrivileges org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationProvider.testSimplePrivileges org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationProviderWithACL.testSimplePrivileges org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationReads.testReadDbFailure org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationReads.testReadDbSuccess org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationReads.testReadTableFailure org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationReads.testReadTableSuccess org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationReads.testReadTableSuccessWithReadOnly org.apache.hadoop.hive.thrift.TestHadoopAuthBridge23.testDelegationTokenSharedStore org.apache.hadoop.hive.thrift.TestHadoopAuthBridge23.testMetastoreProxyUser org.apache.hadoop.hive.thrift.TestHadoopAuthBridge23.testSaslWithHiveMetaStore org.apache.hive.hcatalog.api.repl.commands.TestCommands.org.apache.hive.hcatalog.api.repl.commands.TestCommands org.apache.hive.minikdc.TestJdbcWithDBTokenStore.testNegativeTokenAuth Test results: http://ec2-54-177-240-2.us-west-1.compute.amazonaws.com/job/PreCommit-HIVE-MASTER-Build/138/testReport Console output: http://ec2-54-177-240-2.us-west-1.compute.amazonaws.com/job/PreCommit-HIVE-MASTER-Build/138/console Test logs: http://ec2-50-18-27-0.us-west-1.compute.amazonaws.com/logs/PreCommit-HIVE-MASTER-Build-138/ 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: 62 tests failed This message is automatically generated. ATTACHMENT ID: 12801299 - PreCommit-HIVE-MASTER-Build
            ychena Yongzhi Chen added a comment -

            The failures are not related.

            ychena Yongzhi Chen added a comment - The failures are not related.
            ychena Yongzhi Chen added a comment -

            spena, patch 2 include your fix in read parquet tables.
            Also tested all the 47 q files with parquet in the file name, all pass:
            Tests run: 47, Failures: 0, Errors: 0, Skipped: 0

            ychena Yongzhi Chen added a comment - spena , patch 2 include your fix in read parquet tables. Also tested all the 47 q files with parquet in the file name, all pass: Tests run: 47, Failures: 0, Errors: 0, Skipped: 0
            ychena Yongzhi Chen added a comment -

            re-attach patch 2 with fixing size and length for empty map and empty list.

            ychena Yongzhi Chen added a comment - re-attach patch 2 with fixing size and length for empty map and empty list.
            ychena Yongzhi Chen added a comment -

            spena, could you review the new patch? Thanks

            ychena Yongzhi Chen added a comment - spena , could you review the new patch? Thanks
            hiveqa Hive QA added a comment -

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

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

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

            TestHWISessionManager - did not produce a TEST-*.xml file
            org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_index_bitmap3
            org.apache.hadoop.hive.metastore.TestAuthzApiEmbedAuthorizerInRemote.org.apache.hadoop.hive.metastore.TestAuthzApiEmbedAuthorizerInRemote
            org.apache.hadoop.hive.metastore.TestFilterHooks.org.apache.hadoop.hive.metastore.TestFilterHooks
            org.apache.hadoop.hive.metastore.TestHiveMetaStoreGetMetaConf.testGetMetaConfDefault
            org.apache.hadoop.hive.metastore.TestHiveMetaStoreGetMetaConf.testGetMetaConfDefaultEmptyString
            org.apache.hadoop.hive.metastore.TestHiveMetaStoreGetMetaConf.testGetMetaConfOverridden
            org.apache.hadoop.hive.metastore.TestHiveMetaStoreGetMetaConf.testGetMetaConfUnknownPreperty
            org.apache.hadoop.hive.metastore.TestHiveMetaStorePartitionSpecs.testAddPartitions
            org.apache.hadoop.hive.metastore.TestHiveMetaStorePartitionSpecs.testFetchingPartitionsWithDifferentSchemas
            org.apache.hadoop.hive.metastore.TestHiveMetaStorePartitionSpecs.testGetPartitionSpecs_WithAndWithoutPartitionGrouping
            org.apache.hadoop.hive.metastore.TestMetaStoreEndFunctionListener.testEndFunctionListener
            org.apache.hadoop.hive.metastore.TestMetaStoreEventListenerOnlyOnCommit.testEventStatus
            org.apache.hadoop.hive.metastore.TestMetaStoreInitListener.testMetaStoreInitListener
            org.apache.hadoop.hive.metastore.TestMetaStoreMetrics.org.apache.hadoop.hive.metastore.TestMetaStoreMetrics
            org.apache.hadoop.hive.metastore.TestPartitionNameWhitelistValidation.testAppendPartitionWithCommas
            org.apache.hadoop.hive.metastore.TestPartitionNameWhitelistValidation.testAppendPartitionWithUnicode
            org.apache.hadoop.hive.metastore.TestPartitionNameWhitelistValidation.testAppendPartitionWithValidCharacters
            org.apache.hadoop.hive.metastore.TestRetryingHMSHandler.testRetryingHMSHandler
            org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testAlterPartition
            org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testAlterTable
            org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testAlterViewParititon
            org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testColumnStatistics
            org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testComplexTable
            org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testComplexTypeApi
            org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testConcurrentMetastores
            org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testDBOwner
            org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testDBOwnerChange
            org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testDatabase
            org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testDatabaseLocation
            org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testDatabaseLocationWithPermissionProblems
            org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testDropTable
            org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testFilterLastPartition
            org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testFilterSinglePartition
            org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testFunctionWithResources
            org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testGetConfigValue
            org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testGetTableObjects
            org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testListPartitionNames
            org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testListPartitions
            org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testNameMethods
            org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testPartition
            org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testPartitionFilter
            org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testRenamePartition
            org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testRetriableClientWithConnLifetime
            org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testSimpleFunction
            org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testSimpleTable
            org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testSimpleTypeApi
            org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testStatsFastTrivial
            org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testSynchronized
            org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testTableDatabase
            org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testTableFilter
            org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testTransactionalValidation
            org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testValidateTableCols
            org.apache.hadoop.hive.ql.exec.vector.expressions.TestVectorDateExpressions.testMultiThreadedVectorUDFDate
            org.apache.hadoop.hive.ql.io.parquet.serde.TestAbstractParquetMapInspector.testEmptyContainer
            org.apache.hadoop.hive.ql.io.parquet.serde.TestParquetHiveArrayInspector.testEmptyContainer
            org.apache.hadoop.hive.ql.security.TestClientSideAuthorizationProvider.testSimplePrivileges
            org.apache.hadoop.hive.ql.security.TestExtendedAcls.org.apache.hadoop.hive.ql.security.TestExtendedAcls
            org.apache.hadoop.hive.ql.security.TestFolderPermissions.org.apache.hadoop.hive.ql.security.TestFolderPermissions
            org.apache.hadoop.hive.ql.security.TestMetastoreAuthorizationProvider.testSimplePrivileges
            org.apache.hadoop.hive.ql.security.TestMultiAuthorizationPreEventListener.org.apache.hadoop.hive.ql.security.TestMultiAuthorizationPreEventListener
            org.apache.hadoop.hive.ql.security.TestStorageBasedClientSideAuthorizationProvider.testSimplePrivileges
            org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationDrops.testDropDatabase
            org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationDrops.testDropPartition
            org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationProvider.testSimplePrivileges
            org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationProviderWithACL.testSimplePrivileges
            org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationReads.testReadDbFailure
            org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationReads.testReadDbSuccess
            org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationReads.testReadTableFailure
            org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationReads.testReadTableSuccess
            org.apache.hadoop.hive.thrift.TestHadoopAuthBridge23.testSaslWithHiveMetaStore
            org.apache.hive.hcatalog.listener.TestDbNotificationListener.cleanupNotifs
            org.apache.hive.hcatalog.listener.TestDbNotificationListener.dropDatabase
            org.apache.hive.jdbc.TestSSL.testSSLFetchHttp
            org.apache.hive.minikdc.TestJdbcNonKrbSASLWithMiniKdc.org.apache.hive.minikdc.TestJdbcNonKrbSASLWithMiniKdc
            org.apache.hive.service.TestHS2ImpersonationWithRemoteMS.org.apache.hive.service.TestHS2ImpersonationWithRemoteMS
            

            Test results: http://ec2-54-177-240-2.us-west-1.compute.amazonaws.com/job/PreCommit-HIVE-MASTER-Build/162/testReport
            Console output: http://ec2-54-177-240-2.us-west-1.compute.amazonaws.com/job/PreCommit-HIVE-MASTER-Build/162/console
            Test logs: http://ec2-50-18-27-0.us-west-1.compute.amazonaws.com/logs/PreCommit-HIVE-MASTER-Build-162/

            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: 76 tests failed
            

            This message is automatically generated.

            ATTACHMENT ID: 12801893 - 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/12801893/HIVE-13632.2.patch SUCCESS: +1 due to 2 test(s) being added or modified. ERROR: -1 due to 76 failed/errored test(s), 10008 tests executed Failed tests: TestHWISessionManager - did not produce a TEST-*.xml file org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_index_bitmap3 org.apache.hadoop.hive.metastore.TestAuthzApiEmbedAuthorizerInRemote.org.apache.hadoop.hive.metastore.TestAuthzApiEmbedAuthorizerInRemote org.apache.hadoop.hive.metastore.TestFilterHooks.org.apache.hadoop.hive.metastore.TestFilterHooks org.apache.hadoop.hive.metastore.TestHiveMetaStoreGetMetaConf.testGetMetaConfDefault org.apache.hadoop.hive.metastore.TestHiveMetaStoreGetMetaConf.testGetMetaConfDefaultEmptyString org.apache.hadoop.hive.metastore.TestHiveMetaStoreGetMetaConf.testGetMetaConfOverridden org.apache.hadoop.hive.metastore.TestHiveMetaStoreGetMetaConf.testGetMetaConfUnknownPreperty org.apache.hadoop.hive.metastore.TestHiveMetaStorePartitionSpecs.testAddPartitions org.apache.hadoop.hive.metastore.TestHiveMetaStorePartitionSpecs.testFetchingPartitionsWithDifferentSchemas org.apache.hadoop.hive.metastore.TestHiveMetaStorePartitionSpecs.testGetPartitionSpecs_WithAndWithoutPartitionGrouping org.apache.hadoop.hive.metastore.TestMetaStoreEndFunctionListener.testEndFunctionListener org.apache.hadoop.hive.metastore.TestMetaStoreEventListenerOnlyOnCommit.testEventStatus org.apache.hadoop.hive.metastore.TestMetaStoreInitListener.testMetaStoreInitListener org.apache.hadoop.hive.metastore.TestMetaStoreMetrics.org.apache.hadoop.hive.metastore.TestMetaStoreMetrics org.apache.hadoop.hive.metastore.TestPartitionNameWhitelistValidation.testAppendPartitionWithCommas org.apache.hadoop.hive.metastore.TestPartitionNameWhitelistValidation.testAppendPartitionWithUnicode org.apache.hadoop.hive.metastore.TestPartitionNameWhitelistValidation.testAppendPartitionWithValidCharacters org.apache.hadoop.hive.metastore.TestRetryingHMSHandler.testRetryingHMSHandler org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testAlterPartition org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testAlterTable org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testAlterViewParititon org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testColumnStatistics org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testComplexTable org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testComplexTypeApi org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testConcurrentMetastores org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testDBOwner org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testDBOwnerChange org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testDatabase org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testDatabaseLocation org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testDatabaseLocationWithPermissionProblems org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testDropTable org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testFilterLastPartition org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testFilterSinglePartition org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testFunctionWithResources org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testGetConfigValue org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testGetTableObjects org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testListPartitionNames org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testListPartitions org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testNameMethods org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testPartition org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testPartitionFilter org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testRenamePartition org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testRetriableClientWithConnLifetime org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testSimpleFunction org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testSimpleTable org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testSimpleTypeApi org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testStatsFastTrivial org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testSynchronized org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testTableDatabase org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testTableFilter org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testTransactionalValidation org.apache.hadoop.hive.metastore.TestSetUGIOnOnlyServer.testValidateTableCols org.apache.hadoop.hive.ql.exec.vector.expressions.TestVectorDateExpressions.testMultiThreadedVectorUDFDate org.apache.hadoop.hive.ql.io.parquet.serde.TestAbstractParquetMapInspector.testEmptyContainer org.apache.hadoop.hive.ql.io.parquet.serde.TestParquetHiveArrayInspector.testEmptyContainer org.apache.hadoop.hive.ql.security.TestClientSideAuthorizationProvider.testSimplePrivileges org.apache.hadoop.hive.ql.security.TestExtendedAcls.org.apache.hadoop.hive.ql.security.TestExtendedAcls org.apache.hadoop.hive.ql.security.TestFolderPermissions.org.apache.hadoop.hive.ql.security.TestFolderPermissions org.apache.hadoop.hive.ql.security.TestMetastoreAuthorizationProvider.testSimplePrivileges org.apache.hadoop.hive.ql.security.TestMultiAuthorizationPreEventListener.org.apache.hadoop.hive.ql.security.TestMultiAuthorizationPreEventListener org.apache.hadoop.hive.ql.security.TestStorageBasedClientSideAuthorizationProvider.testSimplePrivileges org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationDrops.testDropDatabase org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationDrops.testDropPartition org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationProvider.testSimplePrivileges org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationProviderWithACL.testSimplePrivileges org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationReads.testReadDbFailure org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationReads.testReadDbSuccess org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationReads.testReadTableFailure org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationReads.testReadTableSuccess org.apache.hadoop.hive.thrift.TestHadoopAuthBridge23.testSaslWithHiveMetaStore org.apache.hive.hcatalog.listener.TestDbNotificationListener.cleanupNotifs org.apache.hive.hcatalog.listener.TestDbNotificationListener.dropDatabase org.apache.hive.jdbc.TestSSL.testSSLFetchHttp org.apache.hive.minikdc.TestJdbcNonKrbSASLWithMiniKdc.org.apache.hive.minikdc.TestJdbcNonKrbSASLWithMiniKdc org.apache.hive.service.TestHS2ImpersonationWithRemoteMS.org.apache.hive.service.TestHS2ImpersonationWithRemoteMS Test results: http://ec2-54-177-240-2.us-west-1.compute.amazonaws.com/job/PreCommit-HIVE-MASTER-Build/162/testReport Console output: http://ec2-54-177-240-2.us-west-1.compute.amazonaws.com/job/PreCommit-HIVE-MASTER-Build/162/console Test logs: http://ec2-50-18-27-0.us-west-1.compute.amazonaws.com/logs/PreCommit-HIVE-MASTER-Build-162/ 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: 76 tests failed This message is automatically generated. ATTACHMENT ID: 12801893 - PreCommit-HIVE-MASTER-Build
            ychena Yongzhi Chen added a comment -

            Most failures are not related.
            Only two tests need change comparing values.
            TestParquetHiveArrayInspector
            TestAbstractParquetMapInspector
            Attach patch3 for it.

            ychena Yongzhi Chen added a comment - Most failures are not related. Only two tests need change comparing values. TestParquetHiveArrayInspector TestAbstractParquetMapInspector Attach patch3 for it.
            spena Sergio Peña added a comment -

            Thanks ychena. The patch looks good.
            +1

            spena Sergio Peña added a comment - Thanks ychena . The patch looks good. +1
            ychena Yongzhi Chen added a comment -

            Thanks spena

            ychena Yongzhi Chen added a comment - Thanks spena
            hiveqa Hive QA added a comment -

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

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

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

            TestHWISessionManager - did not produce a TEST-*.xml file
            TestMiniTezCliDriver-auto_sortmerge_join_7.q-tez_union_group_by.q-orc_merge9.q-and-12-more - did not produce a TEST-*.xml file
            TestMiniTezCliDriver-dynpart_sort_optimization2.q-tez_dynpart_hashjoin_3.q-orc_vectorization_ppd.q-and-12-more - did not produce a TEST-*.xml file
            org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_index_bitmap3
            org.apache.hadoop.hive.cli.TestMiniTezCliDriver.org.apache.hadoop.hive.cli.TestMiniTezCliDriver
            org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_auto_sortmerge_join_14
            org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_auto_sortmerge_join_8
            org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_custom_input_output_format
            org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_dynamic_partition_pruning_2
            org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_groupby2
            org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_merge2
            org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_schema_evol_orc_nonvec_fetchwork_table
            org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_schema_evol_orc_nonvec_mapwork_part_all_complex
            org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_schema_evol_orc_nonvec_mapwork_table
            org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_tez_dynpart_hashjoin_1
            org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_tez_multi_union
            org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_vector_between_in
            org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_vector_char_4
            org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_vector_char_simple
            org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_vector_decimal_math_funcs
            org.apache.hadoop.hive.llap.tez.TestConverters.testFragmentSpecToTaskSpec
            org.apache.hadoop.hive.llap.tezplugins.TestLlapTaskCommunicator.testFinishableStateUpdateFailure
            org.apache.hadoop.hive.metastore.TestAuthzApiEmbedAuthorizerInRemote.org.apache.hadoop.hive.metastore.TestAuthzApiEmbedAuthorizerInRemote
            org.apache.hadoop.hive.metastore.TestFilterHooks.org.apache.hadoop.hive.metastore.TestFilterHooks
            org.apache.hadoop.hive.metastore.TestHiveMetaStoreGetMetaConf.testGetMetaConfDefault
            org.apache.hadoop.hive.metastore.TestHiveMetaStoreGetMetaConf.testGetMetaConfDefaultEmptyString
            org.apache.hadoop.hive.metastore.TestHiveMetaStoreGetMetaConf.testGetMetaConfOverridden
            org.apache.hadoop.hive.metastore.TestHiveMetaStoreGetMetaConf.testGetMetaConfUnknownPreperty
            org.apache.hadoop.hive.metastore.TestHiveMetaStorePartitionSpecs.testAddPartitions
            org.apache.hadoop.hive.metastore.TestHiveMetaStorePartitionSpecs.testFetchingPartitionsWithDifferentSchemas
            org.apache.hadoop.hive.metastore.TestHiveMetaStorePartitionSpecs.testGetPartitionSpecs_WithAndWithoutPartitionGrouping
            org.apache.hadoop.hive.metastore.TestMetaStoreEventListenerOnlyOnCommit.testEventStatus
            org.apache.hadoop.hive.metastore.TestMetaStoreInitListener.testMetaStoreInitListener
            org.apache.hadoop.hive.metastore.TestMetaStoreMetrics.org.apache.hadoop.hive.metastore.TestMetaStoreMetrics
            org.apache.hadoop.hive.metastore.TestPartitionNameWhitelistValidation.testAddPartitionWithValidPartVal
            org.apache.hadoop.hive.metastore.TestPartitionNameWhitelistValidation.testAppendPartitionWithCommas
            org.apache.hadoop.hive.metastore.TestPartitionNameWhitelistValidation.testAppendPartitionWithUnicode
            org.apache.hadoop.hive.metastore.TestPartitionNameWhitelistValidation.testAppendPartitionWithValidCharacters
            org.apache.hadoop.hive.metastore.TestRetryingHMSHandler.testRetryingHMSHandler
            org.apache.hadoop.hive.ql.lockmgr.TestDbTxnManager2.lockConflictDbTable
            org.apache.hadoop.hive.ql.security.TestClientSideAuthorizationProvider.testSimplePrivileges
            org.apache.hadoop.hive.ql.security.TestFolderPermissions.org.apache.hadoop.hive.ql.security.TestFolderPermissions
            org.apache.hadoop.hive.ql.security.TestMetastoreAuthorizationProvider.testSimplePrivileges
            org.apache.hadoop.hive.ql.security.TestMultiAuthorizationPreEventListener.org.apache.hadoop.hive.ql.security.TestMultiAuthorizationPreEventListener
            org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationDrops.testDropDatabase
            org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationDrops.testDropPartition
            org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationProvider.testSimplePrivileges
            org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationProviderWithACL.testSimplePrivileges
            org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationReads.testReadDbFailure
            org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationReads.testReadDbSuccess
            org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationReads.testReadTableFailure
            org.apache.hadoop.hive.ql.security.authorization.plugin.TestHiveOperationType.checkHiveOperationTypeMatch
            org.apache.hadoop.hive.thrift.TestHadoopAuthBridge23.testDelegationTokenSharedStore
            org.apache.hadoop.hive.thrift.TestHadoopAuthBridge23.testMetastoreProxyUser
            org.apache.hadoop.hive.thrift.TestHadoopAuthBridge23.testSaslWithHiveMetaStore
            org.apache.hive.hcatalog.listener.TestDbNotificationListener.cleanupNotifs
            org.apache.hive.hcatalog.listener.TestDbNotificationListener.dropDatabase
            org.apache.hive.hcatalog.listener.TestDbNotificationListener.sqlInsertPartition
            org.apache.hive.minikdc.TestJdbcWithDBTokenStore.testNegativeTokenAuth
            org.apache.hive.service.cli.session.TestHiveSessionImpl.testLeakOperationHandle
            org.apache.hive.spark.client.TestSparkClient.testSyncRpc
            

            Test results: http://ec2-54-177-240-2.us-west-1.compute.amazonaws.com/job/PreCommit-HIVE-MASTER-Build/180/testReport
            Console output: http://ec2-54-177-240-2.us-west-1.compute.amazonaws.com/job/PreCommit-HIVE-MASTER-Build/180/console
            Test logs: http://ec2-50-18-27-0.us-west-1.compute.amazonaws.com/logs/PreCommit-HIVE-MASTER-Build-180/

            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: 61 tests failed
            

            This message is automatically generated.

            ATTACHMENT ID: 12802001 - 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/12802001/HIVE-13632.3.patch SUCCESS: +1 due to 4 test(s) being added or modified. ERROR: -1 due to 61 failed/errored test(s), 10019 tests executed Failed tests: TestHWISessionManager - did not produce a TEST-*.xml file TestMiniTezCliDriver-auto_sortmerge_join_7.q-tez_union_group_by.q-orc_merge9.q-and-12-more - did not produce a TEST-*.xml file TestMiniTezCliDriver-dynpart_sort_optimization2.q-tez_dynpart_hashjoin_3.q-orc_vectorization_ppd.q-and-12-more - did not produce a TEST-*.xml file org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_index_bitmap3 org.apache.hadoop.hive.cli.TestMiniTezCliDriver.org.apache.hadoop.hive.cli.TestMiniTezCliDriver org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_auto_sortmerge_join_14 org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_auto_sortmerge_join_8 org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_custom_input_output_format org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_dynamic_partition_pruning_2 org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_groupby2 org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_merge2 org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_schema_evol_orc_nonvec_fetchwork_table org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_schema_evol_orc_nonvec_mapwork_part_all_complex org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_schema_evol_orc_nonvec_mapwork_table org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_tez_dynpart_hashjoin_1 org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_tez_multi_union org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_vector_between_in org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_vector_char_4 org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_vector_char_simple org.apache.hadoop.hive.cli.TestMiniTezCliDriver.testCliDriver_vector_decimal_math_funcs org.apache.hadoop.hive.llap.tez.TestConverters.testFragmentSpecToTaskSpec org.apache.hadoop.hive.llap.tezplugins.TestLlapTaskCommunicator.testFinishableStateUpdateFailure org.apache.hadoop.hive.metastore.TestAuthzApiEmbedAuthorizerInRemote.org.apache.hadoop.hive.metastore.TestAuthzApiEmbedAuthorizerInRemote org.apache.hadoop.hive.metastore.TestFilterHooks.org.apache.hadoop.hive.metastore.TestFilterHooks org.apache.hadoop.hive.metastore.TestHiveMetaStoreGetMetaConf.testGetMetaConfDefault org.apache.hadoop.hive.metastore.TestHiveMetaStoreGetMetaConf.testGetMetaConfDefaultEmptyString org.apache.hadoop.hive.metastore.TestHiveMetaStoreGetMetaConf.testGetMetaConfOverridden org.apache.hadoop.hive.metastore.TestHiveMetaStoreGetMetaConf.testGetMetaConfUnknownPreperty org.apache.hadoop.hive.metastore.TestHiveMetaStorePartitionSpecs.testAddPartitions org.apache.hadoop.hive.metastore.TestHiveMetaStorePartitionSpecs.testFetchingPartitionsWithDifferentSchemas org.apache.hadoop.hive.metastore.TestHiveMetaStorePartitionSpecs.testGetPartitionSpecs_WithAndWithoutPartitionGrouping org.apache.hadoop.hive.metastore.TestMetaStoreEventListenerOnlyOnCommit.testEventStatus org.apache.hadoop.hive.metastore.TestMetaStoreInitListener.testMetaStoreInitListener org.apache.hadoop.hive.metastore.TestMetaStoreMetrics.org.apache.hadoop.hive.metastore.TestMetaStoreMetrics org.apache.hadoop.hive.metastore.TestPartitionNameWhitelistValidation.testAddPartitionWithValidPartVal org.apache.hadoop.hive.metastore.TestPartitionNameWhitelistValidation.testAppendPartitionWithCommas org.apache.hadoop.hive.metastore.TestPartitionNameWhitelistValidation.testAppendPartitionWithUnicode org.apache.hadoop.hive.metastore.TestPartitionNameWhitelistValidation.testAppendPartitionWithValidCharacters org.apache.hadoop.hive.metastore.TestRetryingHMSHandler.testRetryingHMSHandler org.apache.hadoop.hive.ql.lockmgr.TestDbTxnManager2.lockConflictDbTable org.apache.hadoop.hive.ql.security.TestClientSideAuthorizationProvider.testSimplePrivileges org.apache.hadoop.hive.ql.security.TestFolderPermissions.org.apache.hadoop.hive.ql.security.TestFolderPermissions org.apache.hadoop.hive.ql.security.TestMetastoreAuthorizationProvider.testSimplePrivileges org.apache.hadoop.hive.ql.security.TestMultiAuthorizationPreEventListener.org.apache.hadoop.hive.ql.security.TestMultiAuthorizationPreEventListener org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationDrops.testDropDatabase org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationDrops.testDropPartition org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationProvider.testSimplePrivileges org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationProviderWithACL.testSimplePrivileges org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationReads.testReadDbFailure org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationReads.testReadDbSuccess org.apache.hadoop.hive.ql.security.TestStorageBasedMetastoreAuthorizationReads.testReadTableFailure org.apache.hadoop.hive.ql.security.authorization.plugin.TestHiveOperationType.checkHiveOperationTypeMatch org.apache.hadoop.hive.thrift.TestHadoopAuthBridge23.testDelegationTokenSharedStore org.apache.hadoop.hive.thrift.TestHadoopAuthBridge23.testMetastoreProxyUser org.apache.hadoop.hive.thrift.TestHadoopAuthBridge23.testSaslWithHiveMetaStore org.apache.hive.hcatalog.listener.TestDbNotificationListener.cleanupNotifs org.apache.hive.hcatalog.listener.TestDbNotificationListener.dropDatabase org.apache.hive.hcatalog.listener.TestDbNotificationListener.sqlInsertPartition org.apache.hive.minikdc.TestJdbcWithDBTokenStore.testNegativeTokenAuth org.apache.hive.service.cli.session.TestHiveSessionImpl.testLeakOperationHandle org.apache.hive.spark.client.TestSparkClient.testSyncRpc Test results: http://ec2-54-177-240-2.us-west-1.compute.amazonaws.com/job/PreCommit-HIVE-MASTER-Build/180/testReport Console output: http://ec2-54-177-240-2.us-west-1.compute.amazonaws.com/job/PreCommit-HIVE-MASTER-Build/180/console Test logs: http://ec2-50-18-27-0.us-west-1.compute.amazonaws.com/logs/PreCommit-HIVE-MASTER-Build-180/ 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: 61 tests failed This message is automatically generated. ATTACHMENT ID: 12802001 - PreCommit-HIVE-MASTER-Build
            ychena Yongzhi Chen added a comment -

            The failures are not related.

            ychena Yongzhi Chen added a comment - The failures are not related.
            ychena Yongzhi Chen added a comment -

            Committed to master and branch-1

            ychena Yongzhi Chen added a comment - Committed to master and branch-1
            Mr.黄 Mr.黄 added a comment -

            hiveqa Hello, I did not have this problem in Hive 2.1.1 version, but this problem reappeared in Hive 3.1.2 version, I do not understand why the lower version succeeded and the higher version failed, may I ask whether this bug will be fixed later? The following is my version information and error message:

             

            spark version: 2.4.0-cdh6.3.2
            hive version: 2.1.1-cdh.6.3.2
            scala> spark.sql("create table test STORED AS PARQUET as select map() as a")
            scala> sql("select * from test").show
            +---+                                                                           
            |  a|
            +---+
            | []|
            +---+
            
            -----------------------------------------------------------------------------------------------------------------
            spark version: 2.4.3
            hive version: 3.1.2
            scala> spark.sql("create table test STORED AS PARQUET as select map() as a")
            
            Caused by: org.apache.spark.SparkException: Task failed while writing rows.
              at org.apache.spark.sql.execution.datasources.FileFormatWriter$.org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask(FileFormatWriter.scala:257)
              at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1.apply(FileFormatWriter.scala:170)
              at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1.apply(FileFormatWriter.scala:169)
              at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
              at org.apache.spark.scheduler.Task.run(Task.scala:121)
              at org.apache.spark.executor.Executor$TaskRunner$$anonfun$10.apply(Executor.scala:408)
              at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360)
              at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:414)
              at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
              at java.lang.Thread.run(Thread.java:748)
            Caused by: java.lang.RuntimeException: Parquet record is malformed: empty fields are illegal, the field should be ommited completely instead
              at org.apache.hadoop.hive.ql.io.parquet.write.DataWritableWriter.write(DataWritableWriter.java:64)
              at org.apache.hadoop.hive.ql.io.parquet.write.DataWritableWriteSupport.write(DataWritableWriteSupport.java:59)
              at org.apache.hadoop.hive.ql.io.parquet.write.DataWritableWriteSupport.write(DataWritableWriteSupport.java:31)
              at parquet.hadoop.InternalParquetRecordWriter.write(InternalParquetRecordWriter.java:121)
              at parquet.hadoop.ParquetRecordWriter.write(ParquetRecordWriter.java:123)
              at parquet.hadoop.ParquetRecordWriter.write(ParquetRecordWriter.java:42)
              at org.apache.hadoop.hive.ql.io.parquet.write.ParquetRecordWriterWrapper.write(ParquetRecordWriterWrapper.java:111)
              at org.apache.hadoop.hive.ql.io.parquet.write.ParquetRecordWriterWrapper.write(ParquetRecordWriterWrapper.java:124)
              at org.apache.spark.sql.hive.execution.HiveOutputWriter.write(HiveFileFormat.scala:149)
              at org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.write(FileFormatDataWriter.scala:137)
              at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask$3.apply(FileFormatWriter.scala:245)
              at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask$3.apply(FileFormatWriter.scala:242)
              at org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1394)
              at org.apache.spark.sql.execution.datasources.FileFormatWriter$.org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask(FileFormatWriter.scala:248)
              ... 10 more
            Caused by: parquet.io.ParquetEncodingException: empty fields are illegal, the field should be ommited completely instead
              at parquet.io.MessageColumnIO$MessageColumnIORecordConsumer.endField(MessageColumnIO.java:244)
              at org.apache.hadoop.hive.ql.io.parquet.write.DataWritableWriter.writeMap(DataWritableWriter.java:241)
              at org.apache.hadoop.hive.ql.io.parquet.write.DataWritableWriter.writeValue(DataWritableWriter.java:116)
              at org.apache.hadoop.hive.ql.io.parquet.write.DataWritableWriter.writeGroupFields(DataWritableWriter.java:89)
              at org.apache.hadoop.hive.ql.io.parquet.write.DataWritableWriter.write(DataWritableWriter.java:60)
              ... 23 more 
            Mr.黄 Mr.黄 added a comment - hiveqa Hello, I did not have this problem in Hive 2.1.1 version, but this problem reappeared in Hive 3.1.2 version, I do not understand why the lower version succeeded and the higher version failed, may I ask whether this bug will be fixed later? The following is my version information and error message:   spark version: 2.4.0-cdh6.3.2 hive version: 2.1.1-cdh.6.3.2 scala> spark.sql( "create table test STORED AS PARQUET as select map() as a" ) scala> sql( "select * from test" ).show +---+                                                                            |  a| +---+ | []| +---+ ----------------------------------------------------------------------------------------------------------------- spark version: 2.4.3 hive version: 3.1.2 scala> spark.sql( "create table test STORED AS PARQUET as select map() as a" ) Caused by: org.apache.spark.SparkException: Task failed while writing rows.   at org.apache.spark.sql.execution.datasources.FileFormatWriter$.org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask(FileFormatWriter.scala:257)   at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1.apply(FileFormatWriter.scala:170)   at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1.apply(FileFormatWriter.scala:169)   at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)   at org.apache.spark.scheduler.Task.run(Task.scala:121)   at org.apache.spark.executor.Executor$TaskRunner$$anonfun$10.apply(Executor.scala:408)   at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360)   at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:414)   at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)   at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)   at java.lang. Thread .run( Thread .java:748) Caused by: java.lang.RuntimeException: Parquet record is malformed: empty fields are illegal, the field should be ommited completely instead   at org.apache.hadoop.hive.ql.io.parquet.write.DataWritableWriter.write(DataWritableWriter.java:64)   at org.apache.hadoop.hive.ql.io.parquet.write.DataWritableWriteSupport.write(DataWritableWriteSupport.java:59)   at org.apache.hadoop.hive.ql.io.parquet.write.DataWritableWriteSupport.write(DataWritableWriteSupport.java:31)   at parquet.hadoop.InternalParquetRecordWriter.write(InternalParquetRecordWriter.java:121)   at parquet.hadoop.ParquetRecordWriter.write(ParquetRecordWriter.java:123)   at parquet.hadoop.ParquetRecordWriter.write(ParquetRecordWriter.java:42)   at org.apache.hadoop.hive.ql.io.parquet.write.ParquetRecordWriterWrapper.write(ParquetRecordWriterWrapper.java:111)   at org.apache.hadoop.hive.ql.io.parquet.write.ParquetRecordWriterWrapper.write(ParquetRecordWriterWrapper.java:124)   at org.apache.spark.sql.hive.execution.HiveOutputWriter.write(HiveFileFormat.scala:149)   at org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.write(FileFormatDataWriter.scala:137)   at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask$3.apply(FileFormatWriter.scala:245)   at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask$3.apply(FileFormatWriter.scala:242)   at org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1394)   at org.apache.spark.sql.execution.datasources.FileFormatWriter$.org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask(FileFormatWriter.scala:248)   ... 10 more Caused by: parquet.io.ParquetEncodingException: empty fields are illegal, the field should be ommited completely instead   at parquet.io.MessageColumnIO$MessageColumnIORecordConsumer.endField(MessageColumnIO.java:244)   at org.apache.hadoop.hive.ql.io.parquet.write.DataWritableWriter.writeMap(DataWritableWriter.java:241)   at org.apache.hadoop.hive.ql.io.parquet.write.DataWritableWriter.writeValue(DataWritableWriter.java:116)   at org.apache.hadoop.hive.ql.io.parquet.write.DataWritableWriter.writeGroupFields(DataWritableWriter.java:89)   at org.apache.hadoop.hive.ql.io.parquet.write.DataWritableWriter.write(DataWritableWriter.java:60)   ... 23 more

            People

              ychena Yongzhi Chen
              ychena Yongzhi Chen
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: