hadoop-common-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Georgi Chalakov (JIRA)" <j...@apache.org>
Subject [jira] [Comment Edited] (HADOOP-14520) WASB: Block compaction for Azure Block Blobs
Date Tue, 29 Aug 2017 23:51:00 GMT

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

Georgi Chalakov edited comment on HADOOP-14520 at 8/29/17 11:50 PM:
--------------------------------------------------------------------

HADOOP_14520_07.patch
Results : Tests run: 777, Failures: 0, Errors: 0, Skipped: 155

bq. if you are changing precondition check, I'd recommend StringUtils.isEmpty() for Preconditions.checkArgument(StringUtils.isNotEmpty(aKey));

Done.

bq. If fields aren't updated after the constructor, best to set to final (example, compactionEnabled
?).

Done.

bq. How long is downloadBlockList going to take in that constructor? More specifically: if
compaction is disabled, can that step be skipped?

downloadBlockList is used for two purposes: 1) to check for block existence 2) to download
the block list

bq. If the stream needs a byte buffer, best to use ElasticByteBufferPool as a pool of buffers.

Done.

bq. Use StorageErrorCodeStrings as the source of string constants to check for in exception
error codes.

Done.

bq. Rather than throw IOException(e), I'd prefer more specific (existing ones). That's PathIOException
and subclasses, AzureException(e), and the java.io/java.nio ones.

Done

bq. When wrapping a StorageException with another IOE, always include the toString value of
the wrapped exception. That way, the log message of the top level log retains the underlying
problem.

Done.

bq. BlockBlobAppendStream.WriteRequest retry logic will retry even on RuntimeExceptions like
IllegalArgumentException. Ideally they should be split into recoverable vs non-recoverable
ops via a RetryPolicy. Is this an issue to address here though? Overall, with the new operatins
doing retries, this may be time to embrace rety policies. Or at least create a JIRA entry
on doing so.

add*Command() will rethrow the last exception. That means the following write() or close()
will retrow stored exception. It is not going to happen right away, but the will happen before
the stream is closed()

bq. I know java.io.OutputStream is marked as single-thread only, but I know of code (hello
HBase!) which means that you must make some of the calls thread safe. HADOOP-11708/HADOOP-11710
covers this issue in CryptoOutputStream. At the very least, flush() must be synchronous with
itself, close() & maybe write()

flush() is synchronous with itself through addFlushCommand(). We do not want flush() to be
synchronous with write(). We would like while a thread waits for a flush(), other threads
to continue writing. 

bq. I'm unsure about BlockBlobAppendStream.close() waiting for up to 15 minutes for things
to complete, but looking @ other blobstore clients, I can see that they are implicitly waiting
without any timeout at all. And it's in the existing codebase. But: why was the time limit
changed from 10 min to 15? Was this based on test failures? If so, where is the guarantee
that a 15 minute wait is always sufficient.

The change to 15 min was not based on test failures. I have changed the timeout back to 10
min and added a const. 

bq. Looking at BlockBlobAppendStream thread pooling, I think having a thread pool per output
stream is expensive, especially as it has a minimum size of 4; it will ramp up fast. A pool
of min=1 max=4 might be less expensive. But really, the stream should be thinking about sharing
a pool common to the FS, relying on callbacks to notify it of completion rather than just
awaiting pool completion and a shared writeable field.

I did a some tests with YCSB and a pool of min=1, max=4. It is slower and the difference is
measurable. Considering how many output stream you usually have per FS, I would like to keep
min=4, max=4. The shared pool is a good idea, but I am afraid we would need bigger change
and at the end I am not sure we will get significant benefits. 
    
bq. I think the access/use of lastException needs to be made stronger than just a volatile,
as it means that code of the form if (lastException!=null) throw lastException isn't thread
safe. I know, it's not that harmful provided lastException is never set to null, but I'd still
like some isolated get/getAndSet/maybeThrow operations. Similarly, is lastException the best
way to propagate failure, as it means that teardown failures are going to get reported ahead
of earlier ones during the write itself. Overall, I propose using Callable<bq.  over Runnable.
Allows you to throw exceptions & return things, caller gets to pick them up when it chooses
to.

The original code was using Runnable and I didn’t know about Callable. We could create a
JIRA to do the change here and for the page blob.

bq. Can you do a patch without all the whitespace stripping? It makes the patch too big &
very brittle to cherrypick. I know the spaces are wrong, but trying to strip them in a patch
creates needless patch conflict. when the patch goes in we'll strip off whitespace on new/changed
lines. so it won't get any worse.

Done 

bq. Do try to get those line lengths under 80, including in comments. I know it seems out
of date, and I'd prefer a higher number, but current consensus is 80 except when it's really
hard to do. Supporting side-by-side diff comparison is a reason.

Done

bq. Don't use import java.util.*; for imports —it's OK for static member import, but not
for whole packages. Makes for brittleness across versions, as when a package adds a competing
class, things stop building. I'm assuming this was the IDE being helpful here. If it has autoconvert
to * on, turn it off, along with "delete trailing whitespace".

Yep, the IDE was auto converting to *. It is off now.

bq. and add trailing "." on the first sentence of javadocs, as javadoc complains in their
absence.

Done.

bq. Use try-with-resources & you can get streams closed automatically

Done

bq. index.md use `` around method names.

Done

bq. index.md Need to emphasize cost compaction: a download of segments and uplod of a compacted
replacement. Only for use when the Azure storage is on the same site/network, cost of bandwidth
& time. Do provide details on failure guarantees too, like "doesn't ever lose data".

Done

bq. It would seem good to have some FS statistics tracking compaction, e.g: #of compaction
events, bytes downloaded from compaction, bytes uploaded. These can then be used in assertions
in the tests, but, most importantly, can be looked at in production to see why things are
slow and/or whether compaction is working.

I don't think compaction is interesting, but if you can help me to find FS statistics sample,
I will add it. I've tested compaction with YCSB tests on HBase and the highest number of blocks
per blob for the WAL files was less than 1000. We turn on compaction after 32000 blocks. Block
compaction is a safe net, not a main scenario. Block compaction improves the read operations
and the write cost of Block compaction is amortized over such large number of blocks operations
(bq. 32000). 
 
bq. 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 bq.  that.

The size of the read buffer will be the maximum size of a blob block: 4M. We search for best
sequence of small blocks that can be replaced with a block less than 4M. 

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

Block Compaction is triggered when the number of blocks is above 32000. That is very rare,
but it helps minimizing the chance of getting to the point when no new write operations for
this blob are permitted. When the compaction is triggered and the compaction fail, it is better
to fail the entire write process. 

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

bq. BlockBlobAppendStream: L349: use constant in StorageErrorCodeStrings

Done

bq.  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. 
Before I created very simple buffer pool, I considered that option. The problem is that later
when we want to upload the buffer with Azure Storage Client, we cannot (I couldn't) provide
the buffer without allocating a new one and mem coping the data. 

bq. Make sure that blockCompaction uses a buffer from the pool too; I don't think it does
right now. 

Well, that was my intention, but I couldn't create OutputStream with custom buffer. I choose
the other direction where I can steal the buffer and send it back to the buffer pool. Block
compaction is a rare operation and when new buffer is allocated, it is going to be reused.
If I can find how to create OutputStream object with custom buffer, I would change it. 

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

It is a simple change “ + key”, but the log lines get too long, I think. It is not convenient
reading it in a terminal window. If you really feel that this is necessary, I will add it.


bq. SelfRenewingLease L82: use the constants in StorageErrorCodeStrings

Done. 

bq. Test code:  There's no concurrency test, which would be nice. Could one go into TestNativeAzureFileSystemConcurrency

Can you suggest an existing test that I can use as an sample? 

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

I am sorry, I don't understand how TestBlockBlobInputStream can be used here. 

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

I am not sure I understand how you recommend to change the code, but I had to similar issue
to what you described above. From five runs once was failing with "Invalid Block List". The
assert and the order of the tests helped me to find what is wrong. The order of the insert
is not arbitrary and verifyBlockList allows finding the issue very close to the point of failure.
  

bq. verifyFileData & verifyAppend: I'm not actually sure these work properly if the created
file is bq.  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.

Done.

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

Done 

bq. if the stream.toString() call logs the compaction history, then includng the stream toString
in all asserts would help diagnose problems.

Compaction history would be proportional to the number of the blocks. the default value is
32000 blocks. Even for the tests where we have small number of blocks the history is was actually
very useful for me. I need not only the history of the compaction but also the history of
the write operations. That would be too much for keeping in the stream, I think.

bq. verifyBlockList: don't bother catching & asserting on exception, just throw it all
the way up & let JUnit report it.

Done.
    
bq. testCompactionDisabled}: use try-with-resource or {{IOUtils.cleanupWithLogger.

Done.

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

Done.

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

Done.

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

I have added the missing "." at the end of each sentence.

bq. Probably a good time to look at the javadocs and make sure that there are {@code }

I haven't found issues, but if I have missed something please let me know.

bq. writeBlockRequestInternal has retry logic that returns the buffer to the pool and then
retries using the buffer that it just returned.

This was a bug. It is fixed now. 

bq. writeBlockRequestInternal is currently returning a byte array originally created by ByteArrayOutputStream
to the buffer pool. If this is not clear, look at blockCompaction where it creates ByteArrayOutputStreamInternal,
then wraps the underlying byte[] in a ByteBuffer and passes it to writeBlockRequestInternal
which returns it to the pool.

Fixed

bq. blockCompaction can be refactored to make unit testing easy. For example, extracting out
a getBlockSequenceForCompaction function that takes a block list as input and returns a sequence
of blocks to be compacted would allow a data driven unit test to run many different block
lists thru the algorithm.

I considered that but found that would be more valuable to run e2e.

bq. I recommend the following description for the blockCompaction function:

Done. 

bq. I recommend renaming BlockBlobAppendStream.bufferSize to maxBlockSize. It is the maximum
size of a block.

Done.



was (Author: georgi):
Results : Tests run: 777, Failures: 0, Errors: 0, Skipped: 155

bq. if you are changing precondition check, I'd recommend StringUtils.isEmpty() for Preconditions.checkArgument(StringUtils.isNotEmpty(aKey));

Done.

bq. If fields aren't updated after the constructor, best to set to final (example, compactionEnabled
?).

Done.

bq. How long is downloadBlockList going to take in that constructor? More specifically: if
compaction is disabled, can that step be skipped?

downloadBlockList is used for two purposes: 1) to check for block existence 2) to download
the block list

bq. If the stream needs a byte buffer, best to use ElasticByteBufferPool as a pool of buffers.

Done.

bq. Use StorageErrorCodeStrings as the source of string constants to check for in exception
error codes.

Done.

bq. Rather than throw IOException(e), I'd prefer more specific (existing ones). That's PathIOException
and subclasses, AzureException(e), and the java.io/java.nio ones.

Done

bq. When wrapping a StorageException with another IOE, always include the toString value of
the wrapped exception. That way, the log message of the top level log retains the underlying
problem.

Done.

bq. BlockBlobAppendStream.WriteRequest retry logic will retry even on RuntimeExceptions like
IllegalArgumentException. Ideally they should be split into recoverable vs non-recoverable
ops via a RetryPolicy. Is this an issue to address here though? Overall, with the new operatins
doing retries, this may be time to embrace rety policies. Or at least create a JIRA entry
on doing so.

add*Command() will rethrow the last exception. That means the following write() or close()
will retrow stored exception. It is not going to happen right away, but the will happen before
the stream is closed()

bq. I know java.io.OutputStream is marked as single-thread only, but I know of code (hello
HBase!) which means that you must make some of the calls thread safe. HADOOP-11708/HADOOP-11710
covers this issue in CryptoOutputStream. At the very least, flush() must be synchronous with
itself, close() & maybe write()

flush() is synchronous with itself through addFlushCommand(). We do not want flush() to be
synchronous with write(). We would like while a thread waits for a flush(), other threads
to continue writing. 

bq. I'm unsure about BlockBlobAppendStream.close() waiting for up to 15 minutes for things
to complete, but looking @ other blobstore clients, I can see that they are implicitly waiting
without any timeout at all. And it's in the existing codebase. But: why was the time limit
changed from 10 min to 15? Was this based on test failures? If so, where is the guarantee
that a 15 minute wait is always sufficient.

The change to 15 min was not based on test failures. I have changed the timeout back to 10
min and added a const. 

bq. Looking at BlockBlobAppendStream thread pooling, I think having a thread pool per output
stream is expensive, especially as it has a minimum size of 4; it will ramp up fast. A pool
of min=1 max=4 might be less expensive. But really, the stream should be thinking about sharing
a pool common to the FS, relying on callbacks to notify it of completion rather than just
awaiting pool completion and a shared writeable field.

I did a some tests with YCSB and a pool of min=1, max=4. It is slower and the difference is
measurable. Considering how many output stream you usually have per FS, I would like to keep
min=4, max=4. The shared pool is a good idea, but I am afraid we would need bigger change
and at the end I am not sure we will get significant benefits. 
    
bq. I think the access/use of lastException needs to be made stronger than just a volatile,
as it means that code of the form if (lastException!=null) throw lastException isn't thread
safe. I know, it's not that harmful provided lastException is never set to null, but I'd still
like some isolated get/getAndSet/maybeThrow operations. Similarly, is lastException the best
way to propagate failure, as it means that teardown failures are going to get reported ahead
of earlier ones during the write itself. Overall, I propose using Callable<bq.  over Runnable.
Allows you to throw exceptions & return things, caller gets to pick them up when it chooses
to.

The original code was using Runnable and I didn’t know about Callable. We could create a
JIRA to do the change here and for the page blob.

bq. Can you do a patch without all the whitespace stripping? It makes the patch too big &
very brittle to cherrypick. I know the spaces are wrong, but trying to strip them in a patch
creates needless patch conflict. when the patch goes in we'll strip off whitespace on new/changed
lines. so it won't get any worse.

Done 

bq. Do try to get those line lengths under 80, including in comments. I know it seems out
of date, and I'd prefer a higher number, but current consensus is 80 except when it's really
hard to do. Supporting side-by-side diff comparison is a reason.

Done

bq. Don't use import java.util.*; for imports —it's OK for static member import, but not
for whole packages. Makes for brittleness across versions, as when a package adds a competing
class, things stop building. I'm assuming this was the IDE being helpful here. If it has autoconvert
to * on, turn it off, along with "delete trailing whitespace".

Yep, the IDE was auto converting to *. It is off now.

bq. and add trailing "." on the first sentence of javadocs, as javadoc complains in their
absence.

Done.

bq. Use try-with-resources & you can get streams closed automatically

Done

bq. index.md use `` around method names.

Done

bq. index.md Need to emphasize cost compaction: a download of segments and uplod of a compacted
replacement. Only for use when the Azure storage is on the same site/network, cost of bandwidth
& time. Do provide details on failure guarantees too, like "doesn't ever lose data".

Done

bq. It would seem good to have some FS statistics tracking compaction, e.g: #of compaction
events, bytes downloaded from compaction, bytes uploaded. These can then be used in assertions
in the tests, but, most importantly, can be looked at in production to see why things are
slow and/or whether compaction is working.

I don't think compaction is interesting, but if you can help me to find FS statistics sample,
I will add it. I've tested compaction with YCSB tests on HBase and the highest number of blocks
per blob for the WAL files was less than 1000. We turn on compaction after 32000 blocks. Block
compaction is a safe net, not a main scenario. Block compaction improves the read operations
and the write cost of Block compaction is amortized over such large number of blocks operations
(bq. 32000). 
 
bq. 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 bq.  that.

The size of the read buffer will be the maximum size of a blob block: 4M. We search for best
sequence of small blocks that can be replaced with a block less than 4M. 

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

Block Compaction is triggered when the number of blocks is above 32000. That is very rare,
but it helps minimizing the chance of getting to the point when no new write operations for
this blob are permitted. When the compaction is triggered and the compaction fail, it is better
to fail the entire write process. 

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

bq. BlockBlobAppendStream: L349: use constant in StorageErrorCodeStrings

Done

bq.  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. 
Before I created very simple buffer pool, I considered that option. The problem is that later
when we want to upload the buffer with Azure Storage Client, we cannot (I couldn't) provide
the buffer without allocating a new one and mem coping the data. 

bq. Make sure that blockCompaction uses a buffer from the pool too; I don't think it does
right now. 

Well, that was my intention, but I couldn't create OutputStream with custom buffer. I choose
the other direction where I can steal the buffer and send it back to the buffer pool. Block
compaction is a rare operation and when new buffer is allocated, it is going to be reused.
If I can find how to create OutputStream object with custom buffer, I would change it. 

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

It is a simple change “ + key”, but the log lines get too long, I think. It is not convenient
reading it in a terminal window. If you really feel that this is necessary, I will add it.


bq. SelfRenewingLease L82: use the constants in StorageErrorCodeStrings

Done. 

bq. Test code:  There's no concurrency test, which would be nice. Could one go into TestNativeAzureFileSystemConcurrency

Can you suggest an existing test that I can use as an sample? 

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

I am sorry, I don't understand how TestBlockBlobInputStream can be used here. 

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

I am not sure I understand how you recommend to change the code, but I had to similar issue
to what you described above. From five runs once was failing with "Invalid Block List". The
assert and the order of the tests helped me to find what is wrong. The order of the insert
is not arbitrary and verifyBlockList allows finding the issue very close to the point of failure.
  

bq. verifyFileData & verifyAppend: I'm not actually sure these work properly if the created
file is bq.  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.

Done.

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

Done 

bq. if the stream.toString() call logs the compaction history, then includng the stream toString
in all asserts would help diagnose problems.

Compaction history would be proportional to the number of the blocks. the default value is
32000 blocks. Even for the tests where we have small number of blocks the history is was actually
very useful for me. I need not only the history of the compaction but also the history of
the write operations. That would be too much for keeping in the stream, I think.

bq. verifyBlockList: don't bother catching & asserting on exception, just throw it all
the way up & let JUnit report it.

Done.
    
bq. testCompactionDisabled}: use try-with-resource or {{IOUtils.cleanupWithLogger.

Done.

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

Done.

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

Done.

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

I have added the missing "." at the end of each sentence.

bq. Probably a good time to look at the javadocs and make sure that there are {@code }

I haven't found issues, but if I have missed something please let me know.

bq. writeBlockRequestInternal has retry logic that returns the buffer to the pool and then
retries using the buffer that it just returned.

This was a bug. It is fixed now. 

bq. writeBlockRequestInternal is currently returning a byte array originally created by ByteArrayOutputStream
to the buffer pool. If this is not clear, look at blockCompaction where it creates ByteArrayOutputStreamInternal,
then wraps the underlying byte[] in a ByteBuffer and passes it to writeBlockRequestInternal
which returns it to the pool.

Fixed

bq. blockCompaction can be refactored to make unit testing easy. For example, extracting out
a getBlockSequenceForCompaction function that takes a block list as input and returns a sequence
of blocks to be compacted would allow a data driven unit test to run many different block
lists thru the algorithm.

I considered that but found that would be more valuable to run e2e.

bq. I recommend the following description for the blockCompaction function:

Done. 

bq. I recommend renaming BlockBlobAppendStream.bufferSize to maxBlockSize. It is the maximum
size of a block.

Done.


> 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, HADOOP_14520_07.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_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: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


Mime
View raw message