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

Steve Loughran commented on HADOOP-14520:
-----------------------------------------

Production code is looking pretty good, so I've just gone through the tests in 
detail too now. Sorry.


1. size of buffers/compaction blocks

I'm worried about what happens when large buffers have been flushed & then a 
compaction starts. The size of the buffer needed will be that of 
sum(size(blocks)), won't it? I don't see any checks on those limits, such as a 
decision to set a maximum size of a compacted block & break up compactions if 
the total block count to compact is > that.

2. Failure handling on the compaction process. Does a failure on a compaction 
download & upload in {{blockCompaction()} }need to fail the entire write 
process? If it's a transient error it could be overkill. However, if it is a 
sign that {{flush()}} isn't reliably working then the current behaviour is the 
one to run with.

3. One thing I'd like (but which won't mandate) is for the stream to count the 
#of compaction events, bytes compacted and total duration. then provide some 
@VisibleForTesting @ Unstable getters, *and print them in the {{toString()}} 
call. That would line things up for moving to FS-level instrumentation, and can 
be used immediately .

h3. {{BlockBlobAppendStream}}: 
* L349: use constant in {{StorageErrorCodeStrings}}
* Use {{org.apache.hadoop.util.DirectBufferPool}} to pool the buffers; stable 
code, uses weak refs to ensure GCs will recover free buffers from the pool.
* Make sure that {{blockCompaction}} uses a buffer from the pool too; I don't 
think it does right now.
* {{UploaderThreadFactory}}: idle thought: would it make sense to include the 
container ID or container & key in the thread? I don't know of anything else 
which does this, but it would aid thread dump diagnostics.

h3. {{SelfRenewingLease}}

L82: use the constants in {{StorageErrorCodeStrings}}

h2. Test code

* There's no concurrency test, which would be nice. Could one go into 
{{TestNativeAzureFileSystemConcurrency}}
* Maybe also think about having {{TestBlockBlobInputStream}} use this stream as 
its upload mechanism; insert some flushes through the loop and see what 
actually happens on larger scale files. The small tests, while nice and fast, 
don't check things like buffer sizing if you have large blocks to combine.


h3. {{TestNativeAzureFileSystemBlockCompaction}}


As background, I like to review tests from the following use case "its got a 
transient jenkins failure and all you have is the stack trace to debug what 
failed". Which means I expect tests to: preserve all stack traces, add as much 
diagnostics information in asserts, including text for every simple 
assertTrue/assertFalse —enough to get an idea what's wrong without pasting the 
stack in the IDE to find out which specific assert actually failed.

h4.  {{verifyFileData}} & {{verifyAppend}}:

I'm not actually sure these work properly if the created file is > the 
generated test data, and, by swallowing exceptions, they don't actually report 
underlying failures, merely trigger an assertion failure somewhere in the 
calling code. 

I'd replace these entirely with {{ContractTestUtils.verifyFileContents()}}, 
which does report failures and is widely enough used that it's considered 
stable.


h4. {{testCompaction()}}

* once the verify calls rethrow all exceptions, some of the asserts here can be 
cut
* there's a lot of copy-and-paste duplication fo the 
write/write/write/flush/verify sequences; these should be factored out into 
shared methods.
* if the stream.toString() call logs the compaction history, then includng the 
stream toString in all asserts would help diagnose problems.

h4. other 

* {{verifyBlockList}}: don't bother catching & asserting on exception, just 
throw it all the way up & let JUnit report it.
* {{testCompactionDisabled}: use try-with-resource or 
{{IOUtils.cleanupWithLogger}}.


h3. checkstyle


# Most of those "is a magic number" complaints are just about common values in 
the test...if they were pulled out into some shared variables then it'd shut up 
checkstyle
# there is that "15 minutes" constant in production. How about moving that up 
from an inline constant to a static constant "CLOSE_UPLOAD_DELAY" or similar in 
the class —so at least its obvious what the number is for/where the delay is 
chosen. At some point in the future, if ever felt to be an issue, then it could 
be made a config option, with all the trouble that ensues.
# javadoc is still unhappy.. I'm actually surprised that it's not complaining 
about all the missing "."' chars at the end of each sentence ... maybe the 
latest update to java 8.x has got javadocs complaining less. Lovely as that may 
be, we have to worry about java9 too, so please: review the diff and add them 
to the new javadoc comments.

# Probably a good time to look at the javadocs and make sure that there are 
{@code } wraps
around code snippets, variable names, etc.



> WASB: Block compaction for Azure Block Blobs
> --------------------------------------------
>
>                 Key: HADOOP-14520
>                 URL: https://issues.apache.org/jira/browse/HADOOP-14520
>             Project: Hadoop Common
>          Issue Type: Improvement
>          Components: fs/azure
>    Affects Versions: 3.0.0-alpha3
>            Reporter: Georgi Chalakov
>            Assignee: Georgi Chalakov
>         Attachments: HADOOP-14520-006.patch, HADOOP-14520-05.patch
>
>
> Block Compaction for WASB allows uploading new blocks for every hflush/hsync 
> call. When the number of blocks is above 32000, next hflush/hsync triggers 
> the block compaction process. Block compaction replaces a sequence of blocks 
> with one block. From all the sequences with total length less than 4M, 
> compaction chooses the longest one. It is a greedy algorithm that preserve 
> all potential candidates for the next round. Block Compaction for WASB 
> increases data durability and allows using block blobs instead of page blobs. 
> By default, block compaction is disabled. Similar to the configuration for 
> page blobs, the client needs to specify HDFS folders where block compaction 
> over block blobs is enabled. 
> Results for HADOOP-14520-05.patch
> tested endpoint: fs.azure.account.key.hdfs4.blob.core.windows.net
> Tests run: 707, Failures: 0, Errors: 0, Skipped: 119



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to