Details
-
Bug
-
Status: Closed
-
Major
-
Resolution: Fixed
-
1.1.0
-
None
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
Attachments
- HIVE-13632.3.patch
- 14 kB
- Yongzhi Chen
- HIVE-13632.2.patch
- 12 kB
- Yongzhi Chen
- HIVE-13632.1.patch
- 8 kB
- Yongzhi Chen
Issue Links
- is related to
-
PARQUET-596 Parquet need to support empty list and empty map
- Resolved
Activity
They show empty. But I can not find a way in parquet.
writeNullForMissingFieldsAtCurrentLevel(); in the java/parquet/io/MessageColumnIO.java may cause this.
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 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?
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.
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
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
re-attach patch 2 with fixing size and length for empty map and empty list.
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
Most failures are not related.
Only two tests need change comparing values.
TestParquetHiveArrayInspector
TestAbstractParquetMapInspector
Attach patch3 for it.
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 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
patch 1 fixes the exception by avoid creating empty field.
But parquet api seems have limitations, empty array and map are stored as null.