You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues-all@impala.apache.org by "Joe McDonnell (JIRA)" <ji...@apache.org> on 2018/11/19 18:56:00 UTC

[jira] [Commented] (IMPALA-7804) Various scanner tests intermittently failing on S3 on different runs

    [ https://issues.apache.org/jira/browse/IMPALA-7804?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16692121#comment-16692121 ] 

Joe McDonnell commented on IMPALA-7804:
---------------------------------------

New failures in test_scanners.TestParquet.test_zero_rows() and test_scanners.TestParquet.test_corrupt_rle_counts(). Since a bunch of these involve test_scanners.py, IMPALA-6709 comes to mind: [https://github.com/apache/impala/commit/e27954a5aa585db23fe3c97726aa89305efa306d#diff-c0076d575312e4fb9c8d8fad0191224e]

This changed how we load files for several of these tests. It may have unforeseen s3 implications.

 

Separately, Amazon has this blurb in their s3 introduction about consistency:
{noformat}
Amazon S3 provides read-after-write consistency for PUTS of new objects in your S3 bucket in all regions with one caveat. The caveat is that if you make a HEAD or GET request to the key name (to find if the object exists) before creating the object, Amazon S3 provides eventual consistency for read-after-write.{noformat}
If I understand this correctly, whenever we check for the existence of a file, that puts us into the eventual consistency case when we write it. If the file doesn't exist and we do a direct put, it is consistent. In quite a few of our tests, we know the file won't already exist and we don't care about checking for existence. Based on the s3 output, we are definitely checking for existence before writing the file (in this case, renaming to that location, which is a copy):
{noformat}
18/11/18 05:42:51 DEBUG s3a.S3AFileSystem: Getting path status for s3a://impala-test-uswest2-1/test-warehouse/test_corrupt_rle_counts_44c21800.db/bad_rle_repeat_count.parquet  (test-warehouse/test_corrupt_rle_counts_44c21800.db/bad_rle_repeat_count.parquet)
18/11/18 05:42:51 DEBUG s3a.S3AStorageStatistics: object_metadata_requests += 1  ->  12
18/11/18 05:42:51 DEBUG s3a.S3AStorageStatistics: object_metadata_requests += 1  ->  13
18/11/18 05:42:51 DEBUG s3a.S3AStorageStatistics: object_list_requests += 1  ->  6
18/11/18 05:42:52 DEBUG s3a.S3AFileSystem: Not Found: s3a://impala-test-uswest2-1/test-warehouse/test_corrupt_rle_counts_44c21800.db/bad_rle_repeat_count.parquet
18/11/18 05:42:52 DEBUG s3a.S3AFileSystem: rename: destination path s3a://impala-test-uswest2-1/test-warehouse/test_corrupt_rle_counts_44c21800.db/bad_rle_repeat_count.parquet not found
...
18/11/18 05:42:52 DEBUG s3a.S3AFileSystem: rename: renaming file s3a://impala-test-uswest2-1/test-warehouse/test_corrupt_rle_counts_44c21800.db/bad_rle_repeat_count.parquet._COPYING_ to s3a://impala-test-uswest2-1/test-warehouse/test_corrupt_rle_counts_44c21800.db/bad_rle_repeat_count.parquet
18/11/18 05:42:52 DEBUG s3a.S3AFileSystem: copyFile test-warehouse/test_corrupt_rle_counts_44c21800.db/bad_rle_repeat_count.parquet._COPYING_ -> test-warehouse/test_corrupt_rle_counts_44c21800.db/bad_rle_repeat_count.parquet {noformat}
Impala's LOAD DATA statement checks for existence in [FileSystemUtil::relocateAllVisibleFiles()|https://github.com/apache/impala/blob/master/fe/src/main/java/org/apache/impala/common/FileSystemUtil.java#L143], and it seems like the hadoop commandline also does so. Our s3 filesystem utility based on Boto3 (tests/util/s3_util.py) has an overwrite mode that doesn't seem to do an existence check. Switching to use this filesystem util might impact our s3 consistency.

> Various scanner tests intermittently failing on S3 on different runs
> --------------------------------------------------------------------
>
>                 Key: IMPALA-7804
>                 URL: https://issues.apache.org/jira/browse/IMPALA-7804
>             Project: IMPALA
>          Issue Type: Bug
>          Components: Backend
>    Affects Versions: Impala 3.2.0
>            Reporter: David Knupp
>            Priority: Blocker
>              Labels: S3
>
> The failures have to do with getting AWS client credentials.
> *query_test/test_scanners.py:696: in test_decimal_encodings*
> _Stacktrace_
> {noformat}
> query_test/test_scanners.py:696: in test_decimal_encodings
>     self.run_test_case('QueryTest/parquet-decimal-formats', vector, unique_database)
> common/impala_test_suite.py:496: in run_test_case
>     self.__verify_results_and_errors(vector, test_section, result, use_db)
> common/impala_test_suite.py:358: in __verify_results_and_errors
>     replace_filenames_with_placeholder)
> common/test_result_verifier.py:438: in verify_raw_results
>     VERIFIER_MAP[verifier](expected, actual)
> common/test_result_verifier.py:260: in verify_query_result_is_equal
>     assert expected_results == actual_results
> E   assert Comparing QueryTestResults (expected vs actual):
> E     -255.00,-255.00,-255.00 == -255.00,-255.00,-255.00
> E     -255.00,-255.00,-255.00 != -65535.00,-65535.00,-65535.00
> E     -65535.00,-65535.00,-65535.00 != -9999999.99,-9999999.99,-9999999.99
> E     -65535.00,-65535.00,-65535.00 != 0.00,-9999999999999999.99,-999999999999999999999999999999999999.99
> E     -9999999.99,-9999999.99,-9999999.99 != 0.00,0.00,0.00
> E     -9999999.99,-9999999.99,-9999999.99 != 0.00,9999999999999999.99,999999999999999999999999999999999999.99
> E     0.00,-9999999999999999.99,-999999999999999999999999999999999999.99 != 255.00,255.00,255.00
> E     0.00,-9999999999999999.99,-999999999999999999999999999999999999.99 != 65535.00,65535.00,65535.00
> E     0.00,0.00,0.00 != 9999999.99,9999999.99,9999999.99
> E     0.00,0.00,0.00 != None
> E     0.00,9999999999999999.99,999999999999999999999999999999999999.99 != None
> E     0.00,9999999999999999.99,999999999999999999999999999999999999.99 != None
> E     255.00,255.00,255.00 != None
> E     255.00,255.00,255.00 != None
> E     65535.00,65535.00,65535.00 != None
> E     65535.00,65535.00,65535.00 != None
> E     9999999.99,9999999.99,9999999.99 != None
> E     9999999.99,9999999.99,9999999.99 != None
> E     Number of rows returned (expected vs actual): 18 != 9
> {noformat}
> _Standard Error_
> {noformat}
> SET sync_ddl=False;
> -- executing against localhost:21000
> DROP DATABASE IF EXISTS `test_huge_num_rows_76a09ef1` CASCADE;
> -- 2018-11-01 09:42:41,140 INFO     MainThread: Started query 4c4bc0e7b69d7641:130ffe7300000000
> SET sync_ddl=False;
> -- executing against localhost:21000
> CREATE DATABASE `test_huge_num_rows_76a09ef1`;
> -- 2018-11-01 09:42:42,402 INFO     MainThread: Started query e34d714d6a62cba1:2a8544d000000000
> -- 2018-11-01 09:42:42,405 INFO     MainThread: Created database "test_huge_num_rows_76a09ef1" for test ID "query_test/test_scanners.py::TestParquet::()::test_huge_num_rows[protocol: beeswax | exec_option: {'batch_size': 0, 'num_nodes': 0, 'disable_codegen_rows_threshold': 0, 'disable_codegen': True, 'abort_on_error': 1, 'debug_action': '-1:OPEN:SET_DENY_RESERVATION_PROBABILITY@1.0', 'exec_single_node_rows_threshold': 0} | table_format: parquet/none]"
> 18/11/01 09:42:43 DEBUG s3a.S3AFileSystem: Initializing S3AFileSystem for impala-test-uswest2-1
> 18/11/01 09:42:43 DEBUG s3a.S3AUtils: Propagating entries under fs.s3a.bucket.impala-test-uswest2-1.
> 18/11/01 09:42:43 WARN impl.MetricsConfig: Cannot locate configuration: tried hadoop-metrics2-s3a-file-system.properties,hadoop-metrics2.properties
> 18/11/01 09:42:43 INFO impl.MetricsSystemImpl: Scheduled Metric snapshot period at 10 second(s).
> 18/11/01 09:42:43 INFO impl.MetricsSystemImpl: s3a-file-system metrics system started
> 18/11/01 09:42:43 DEBUG s3a.S3AUtils: For URI s3a://impala-test-uswest2-1/, using credentials AWSCredentialProviderList: BasicAWSCredentialsProvider EnvironmentVariableCredentialsProvider com.amazonaws.auth.InstanceProfileCredentialsProvider@15bbf42f
> 18/11/01 09:42:43 DEBUG s3a.S3AUtils: Value of fs.s3a.connection.maximum is 1500
> 18/11/01 09:42:43 DEBUG s3a.S3AUtils: Value of fs.s3a.attempts.maximum is 20
> 18/11/01 09:42:43 DEBUG s3a.S3AUtils: Value of fs.s3a.connection.establish.timeout is 5000
> 18/11/01 09:42:43 DEBUG s3a.S3AUtils: Value of fs.s3a.connection.timeout is 200000
> 18/11/01 09:42:43 DEBUG s3a.S3AUtils: Value of fs.s3a.socket.send.buffer is 8192
> 18/11/01 09:42:43 DEBUG s3a.S3AUtils: Value of fs.s3a.socket.recv.buffer is 8192
> 18/11/01 09:42:43 DEBUG s3a.S3AFileSystem: Using User-Agent: Hadoop 3.0.0-cdh6.x-SNAPSHOT
> 18/11/01 09:42:44 DEBUG s3a.S3AUtils: Value of fs.s3a.paging.maximum is 5000
> 18/11/01 09:42:44 DEBUG s3a.S3AUtils: Value of fs.s3a.block.size is 33554432
> 18/11/01 09:42:44 DEBUG s3a.S3AUtils: Value of fs.s3a.readahead.range is 65536
> 18/11/01 09:42:44 DEBUG s3a.S3AUtils: Value of fs.s3a.max.total.tasks is 5
> 18/11/01 09:42:44 DEBUG s3a.S3AUtils: Value of fs.s3a.threads.keepalivetime is 60
> 18/11/01 09:42:44 DEBUG s3a.AWSCredentialProviderList: No credentials provided by BasicAWSCredentialsProvider: org.apache.hadoop.fs.s3a.CredentialInitializationException: Access key or secret key is null
> org.apache.hadoop.fs.s3a.CredentialInitializationException: Access key or secret key is null
> 	at org.apache.hadoop.fs.s3a.BasicAWSCredentialsProvider.getCredentials(BasicAWSCredentialsProvider.java:51)
> 	at org.apache.hadoop.fs.s3a.AWSCredentialProviderList.getCredentials(AWSCredentialProviderList.java:117)
> {noformat}
> *query_test/test_scanners.py:351: in test_huge_num_rows*
> _Stacktrace_
> {noformat}
> query_test/test_scanners.py:351: in test_huge_num_rows
>     % unique_database)
> common/impala_connection.py:170: in execute
>     return self.__beeswax_client.execute(sql_stmt, user=user)
> beeswax/impala_beeswax.py:182: in execute
>     handle = self.__execute_query(query_string.strip(), user=user)
> beeswax/impala_beeswax.py:356: in __execute_query
>     self.wait_for_finished(handle)
> beeswax/impala_beeswax.py:377: in wait_for_finished
>     raise ImpalaBeeswaxException("Query aborted:" + error_log, None)
> E   ImpalaBeeswaxException: ImpalaBeeswaxException:
> E    Query aborted:Disk I/O error: Failed to open HDFS file s3a://impala-test-uswest2-1/test-warehouse/test_huge_num_rows_76a09ef1.db/huge_num_rows/huge_num_rows.parquet
> E   Error(2): No such file or directory
> E   Root cause: FileNotFoundException: No such file or directory: s3a://impala-test-uswest2-1/test-warehouse/test_huge_num_rows_76a09ef1.db/huge_num_rows/huge_num_rows.parquet
> {noformat}
> _Standard Error_
> {noformat}
> SET sync_ddl=False;
> -- executing against localhost:21000
> DROP DATABASE IF EXISTS `test_huge_num_rows_76a09ef1` CASCADE;
> -- 2018-11-01 09:42:41,140 INFO     MainThread: Started query 4c4bc0e7b69d7641:130ffe7300000000
> SET sync_ddl=False;
> -- executing against localhost:21000
> CREATE DATABASE `test_huge_num_rows_76a09ef1`;
> -- 2018-11-01 09:42:42,402 INFO     MainThread: Started query e34d714d6a62cba1:2a8544d000000000
> -- 2018-11-01 09:42:42,405 INFO     MainThread: Created database "test_huge_num_rows_76a09ef1" for test ID "query_test/test_scanners.py::TestParquet::()::test_huge_num_rows[protocol: beeswax | exec_option: {'batch_size': 0, 'num_nodes': 0, 'disable_codegen_rows_threshold': 0, 'disable_codegen': True, 'abort_on_error': 1, 'debug_action': '-1:OPEN:SET_DENY_RESERVATION_PROBABILITY@1.0', 'exec_single_node_rows_threshold': 0} | table_format: parquet/none]"
> 18/11/01 09:42:43 DEBUG s3a.S3AFileSystem: Initializing S3AFileSystem for impala-test-uswest2-1
> 18/11/01 09:42:43 DEBUG s3a.S3AUtils: Propagating entries under fs.s3a.bucket.impala-test-uswest2-1.
> 18/11/01 09:42:43 WARN impl.MetricsConfig: Cannot locate configuration: tried hadoop-metrics2-s3a-file-system.properties,hadoop-metrics2.properties
> 18/11/01 09:42:43 INFO impl.MetricsSystemImpl: Scheduled Metric snapshot period at 10 second(s).
> 18/11/01 09:42:43 INFO impl.MetricsSystemImpl: s3a-file-system metrics system started
> 18/11/01 09:42:43 DEBUG s3a.S3AUtils: For URI s3a://impala-test-uswest2-1/, using credentials AWSCredentialProviderList: BasicAWSCredentialsProvider EnvironmentVariableCredentialsProvider com.amazonaws.auth.InstanceProfileCredentialsProvider@15bbf42f
> 18/11/01 09:42:43 DEBUG s3a.S3AUtils: Value of fs.s3a.connection.maximum is 1500
> 18/11/01 09:42:43 DEBUG s3a.S3AUtils: Value of fs.s3a.attempts.maximum is 20
> 18/11/01 09:42:43 DEBUG s3a.S3AUtils: Value of fs.s3a.connection.establish.timeout is 5000
> 18/11/01 09:42:43 DEBUG s3a.S3AUtils: Value of fs.s3a.connection.timeout is 200000
> 18/11/01 09:42:43 DEBUG s3a.S3AUtils: Value of fs.s3a.socket.send.buffer is 8192
> 18/11/01 09:42:43 DEBUG s3a.S3AUtils: Value of fs.s3a.socket.recv.buffer is 8192
> 18/11/01 09:42:43 DEBUG s3a.S3AFileSystem: Using User-Agent: Hadoop 3.0.0-cdh6.x-SNAPSHOT
> 18/11/01 09:42:44 DEBUG s3a.S3AUtils: Value of fs.s3a.paging.maximum is 5000
> 18/11/01 09:42:44 DEBUG s3a.S3AUtils: Value of fs.s3a.block.size is 33554432
> 18/11/01 09:42:44 DEBUG s3a.S3AUtils: Value of fs.s3a.readahead.range is 65536
> 18/11/01 09:42:44 DEBUG s3a.S3AUtils: Value of fs.s3a.max.total.tasks is 5
> 18/11/01 09:42:44 DEBUG s3a.S3AUtils: Value of fs.s3a.threads.keepalivetime is 60
> 18/11/01 09:42:44 DEBUG s3a.AWSCredentialProviderList: No credentials provided by BasicAWSCredentialsProvider: org.apache.hadoop.fs.s3a.CredentialInitializationException: Access key or secret key is null
> org.apache.hadoop.fs.s3a.CredentialInitializationException: Access key or secret key is null
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-all-unsubscribe@impala.apache.org
For additional commands, e-mail: issues-all-help@impala.apache.org