accumulo-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Shawn Walker <accum...@shawn-walker.net>
Subject Re: Accumulo on s3
Date Tue, 26 Apr 2016 14:35:18 GMT
RFiles (Accumulo's primary data storage mechanism) are immutable and lazily
deleted (by the GC process).  Read-after-write consistency for new files
should be sufficient for them.  I suspect the only real gotchas would be:
NativeS3FileSystem has a 5G max file size, and NativeS3FileSystem is very
slow when renaming files.  One might consider using the Hadoop S3 block
filesystem instead, for better rename performance.

On the other hand, write-ahead logs simply can't function as expected atop
the NativeS3FileSystem:  A write-ahead log is an incrementally built file,
and S3 doesn't support a compatible concept of incremental writes to a
stored object.  Neither hflush()'ing or hsync()'ing a FSDataOutputStream
sourced from a NativeS3FileSystem actually makes any data available outside
the process.  The S3 block filesystem improves matters slightly, but not
enough to

This means that a tablet server death (e.g. caused by a lost Zookeeper
lock) can (almost certainly will) lead to lost mutations.  It strikes me
that this would be particularly bad for mutations against the root tablet
or metadata tablets, and can leave Accumulo in an inconsistent state.

Working around that limitation in Accumulo would likely prove somewhere
between difficult and impossible.  At the least, it might mean redesigning
the entire WAL concept.

--
Shawn Walker

On Tue, Apr 26, 2016 at 5:12 AM, chutium <tengqiu@gmail.com> wrote:

> Hi Josh,
>
> about the guarantees of s3, according to this doc from amazon:
>
> https://docs.aws.amazon.com/ElasticMapReduce/latest/ReleaseGuide/emr-plan-consistent-view.html
>
> > Amazon S3 buckets in xxx, xxx regions provide read-after-write
> consistency
> > for put requests of new objects and eventual consistency for overwrite
> put
> > and delete requests.
>
> so may be accumulo will get problem with consistency only by major
> compactions right? it seems no other operation is overwriting or deleting
> files on HDFS.
>
> let me describe our usage of accumulo on s3, basically, we want to combine
> the unlimited storage feature of s3 and the fine grained access control
> provided by accumulo.
>
> we are using "accumulo on s3" as a secured storage behind data processing
> engine (spark), data are ingested into accumulo regularly, not in real time
> (no single put, batch ingestion each X hours), most of data access use
> cases
> are batch processing, so no realtime read or write.
>
> then consistency or sync will still be a problem or not?
>
> I added some thoughts of mine in that stackoverflow thread:
> http://stackoverflow.com/a/36845743/5630352 ,  I really want to know is
> this
> possible to solve the s3 problem for our use case? because it seems until
> now, no other tools can provide such a powerful access control framework
> like accumulo.
>
> Thanks!
>
>
>
> --
> View this message in context:
> http://apache-accumulo.1065345.n5.nabble.com/Accumulo-on-s3-tp16737p16764.html
> Sent from the Developers mailing list archive at Nabble.com.
>

Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message