[ 
https://issues.apache.org/jira/browse/HADOOP-14520?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Steve Loughran updated HADOOP-14520:
------------------------------------
    Attachment: HADOOP-14520-008.patch

Review of Patch 007 (which comes with some code diffs attached in the form of 
patch 008)

I actually reviewed this by applying it to the IDE and looking @ the diffs 
there; doing the build. This means that it was easy enough to do some of the 
minor tweaks while I was reading it, with the IDE telling me what to do, and 
step around what's going on more. I *think* understand what's going on more 
now, so now I'm worrying about things like close() and the semantics of the 
flush operations.


h3. {{BlockBlobAppendStream}}

L490: {{flush()}} isn't synced, but is checking non-volatile {{closed}} field 
set in close() and elsewhere. Either {{closed}} should be volatile (easiest), 
do things with AtomicBoolean. As discussed. you don't want flush to be blocking 
writes. (More on flush later)

L162 mark UploadCommand as abstract (fixed)
L207: javadocs say "last exception", but given how it's set, it's not quite is 
it? (fixed) 
L223 javadoc: duplicate "block" (fixed)
L455 replace is null test with Preconditions.checkArgument() (done)
L595 add whitespace around "==" (done)
L599, L606, L640,  L834 use isEmpty() (done)
L363, L705, L:751 add whitespace when building up text (done)


I've factored the setting of {{firstError}} into {{maybeSetFirstError()}}; 
throwing it in {{maybeThrowFirstError}}, which
notes whether or not it has been raised.

h3. {{BlockBlobAppendStream.close()}} robustness

I think {{close()}} needs a bit of work to be somewhat more robust to things 
going wrong.

# with my patch, it only throws firstError if its not already hit a problem. 
So, say, if a write fails, you catch it and call {{close()}}, it doesn't bother 
rethrowing any followon exceptions. Is that the right thing to do? Because 
while it wont rethrow the previous exception, it means that any new exception 
raised during {{lease.free()}} isn't going to be raised 

# if an exception gets raised in close() before {{closed}} is set, operations 
like {[write()}} and flush can still be attempted. Indeed, you can even try 
calling close() again. This can explicitly happen if there's a timeout awaiting 
the thread pool to terminate, and it could also happen if there's some code 
error.

What could be done here?

# I think the {{closed}} field should go to {{AtomicBoolean}}, the first bit of 
{{close()}} is to atomically get & set it & return if close had already taken 
place.

# Maybe, when {{close()}} is entered, if the {{firstException}} has already 
been thrown, reset the field & the flag, so that any new exception raised 
during close is then picked up and thrown.

This is a complicated enough issue I'm not changing it myself, I think we need 
to look at what the possibilities of failure are and how best to report them 
*while doing a best effort shut down*.


h3. {{StreamCapabilities}}

In HADOOP-13327 I'm slowly trying to fix the hadoop codebase so that all 
streams which support {{Syncable}} support the interface
{{StreamCapabilities}}, where they can dynamically tell callers what their sync 
policy really is.

See [this 
document|https://github.com/steveloughran/hadoop/blob/s3/HADOOP-13327-outputstream-trunk/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/outputstream.md]

I've added what I think the stream capabilities should be for this new stream: 
hflush and hsync are no-ops without {{compactionEnabled}} set. But are 
{{hsync()}} and {{hflush()}} doing what they are meant to? Because with the 
current patch, {{flush()}} is a fairly expensive call (it's uploading the 
block) on whether or not compaction is enabled, and hsync and hflush are no-ops 
when compaction is disabled.

What is the behaviour you want? As {{flush()}} can get called a lot in code, 
it's in OutputStream and there's no requirements in implementations to offer 
any guarantees about visibility and durability. And if compaction is disabled, 
every time {{flush()}} is called it'll soon upload too many entries. And as its 
doing this even with compaction is off, you are in trouble. 

Proposed: 
* {{flush()}} is a no-op.
* {{hsync()}} writes up/compacts data {{compactionEnabled}} is set.
* {{hflush()}} calls {{hsync()}}
* {{hasCapabilities}} declares hsync and hflush support only 

This way those apps which really expect hflush to do a full (slow) flush, and 
hsync to do an even slower durable write get what they expect, calls of 
{{flush()}} get no guarantees of anything, which is consistent with what 
{{OpenStream.flush()}} promises.

h3. Other

* {{SelfRenewingLease}}: static import of constant string, keeps line length 
down.
* only need to use single quotes in `code` entries in .md file (done). That was 
me being confusing in the previous feedback, sorry.


Testing: tested {{TestNativeAzureFileSystemBlockCompaction}} against azure 
ireland. Note that the hasCapabilities() code merits some test coverage, 
ideally including some hsync/hflush.

> 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-008.patch, 
> HADOOP-14520-05.patch, HADOOP_14520_07.patch, HADOOP_14520_08.patch, 
> HADOOP_14520_09.patch, HADOOP-14520-patch-07-08.diff
>
>
> 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_07.patch
> tested endpoint: fs.azure.account.key.hdfs4.blob.core.windows.net
> Tests run: 777, Failures: 0, Errors: 0, Skipped: 155



--
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