You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by "Wei-Chiu Chuang (Jira)" <ji...@apache.org> on 2020/03/16 18:50:00 UTC

[jira] [Commented] (HADOOP-16836) Bug in widely-used helper function caused valid configuration value to fail on multiple tests, causing build failure

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

Wei-Chiu Chuang commented on HADOOP-16836:
------------------------------------------

Updated target version. Unless I am wrong, this is also a blocker for Hadoop 3.3.0.

[~brahma] fyi.

> Bug in widely-used helper function caused valid configuration value to fail on multiple tests, causing build failure
> --------------------------------------------------------------------------------------------------------------------
>
>                 Key: HADOOP-16836
>                 URL: https://issues.apache.org/jira/browse/HADOOP-16836
>             Project: Hadoop Common
>          Issue Type: Bug
>          Components: common
>    Affects Versions: 3.3.0, 3.2.1
>            Reporter: Ctest
>            Priority: Blocker
>              Labels: configuration, easyfix, patch, test
>         Attachments: HADOOP-16836-000.patch, HADOOP-16836-000.patch
>
>
> Test helper function `org.apache.hadoop.io.file.tfile.TestTFileByteArrays#readRecords(org.apache.hadoop.fs.FileSystem, org.apache.hadoop.fs.Path, int, org.apache.hadoop.conf.Configuration)` (abbreviate as `readRecords()` below) are called in 4 actively-used tests below:
>  
> {code:java}
> org.apache.hadoop.io.file.tfile.TestTFileStreams#testOneEntryMixedLengths1
> org.apache.hadoop.io.file.tfile.TestTFileStreams#testOneEntryUnknownLength
> org.apache.hadoop.io.file.tfile.TestTFileLzoCodecsStreams#testOneEntryMixedLengths1
> org.apache.hadoop.io.file.tfile.TestTFileLzoCodecsStreams#testOneEntryUnknownLength{code}
>  
> These tests first call `org.apache.hadoop.io.file.tfile.TestTFileStreams#writeRecords(int count, boolean knownKeyLength, boolean knownValueLength, boolean close)` to write `key-value` pair records in a `TFile` object, then call the helper function `readRecords()` to assert the `key` part and the `value` part of `key-value` pair records stored matched with what they wrote perviously. The `value` parts of `key-value` pairs from these tests are hardcode strings with a length of 6.
> Assertions in `readRecords()` are directly related to the value of the configuration parameter `tfile.io.chunk.size`. The formal definition of `tfile.io.chunk.size` is "Value chunk size in bytes. Default to 1MB. Values of the length less than the chunk size is guaranteed to have known value length in read time (See also TFile.Reader.Scanner.Entry.isValueLengthKnown())".
> When `tfile.io.chunk.size` is configured to a value less than the length of the `value` part of the `key-value` pairs from these 4 tests, these tests will fail, even though the configured value for `tfile.io.chunk.size` is correct in semantic.
>  
> *Consequence*
> At least 4 actively-used tests failed on correctly configured parameters. Tests used `readRecords()` could fail if the length of the hardcoded `value` part they tested is larger than the configured value of `tfile.io.chunk.size`. This caused build failure of Hadoop-Common if these tests are not skipped.
>  
> *Root Cause*
> `readRecords()` used `org.apache.hadoop.io.file.tfile.TFile.Reader.Scanner.Entry#getValueLength()` (abbreviate as `getValueLength()` below) to get the full length of the `value` part in the `key-value` pair. But `getValueLength()` can only get the full length of the `value` part when the full length is less than `tfile.io.chunk.size`, otherwise, `getValueLength()` throws an exception, causing `readRecords()` to fail, and thus resulting in failures in the aforementioned 4 tests. This is because `getValueLength()` do not know the full length of the `value` part when `value` part's size is larger than `tfile.io.chunk.size`.
>  
> *Fixes*
> `readRecords()` should instead call `org.apache.hadoop.io.file.tfile.TFile.Reader.Scanner.Entry#getValue(byte[])` (abbreviate as `getValue()` below), which returns the correct full length of the `value` part despite whether the `value` length is larger than `tfile.io.chunk.size`.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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