Shawn -- you win the gold star for the day from me. This is exactly the fear I had, but had an inability put it into words correctly :)

Valerio/chutium -- The common scenario I have run into is that processing jobs (your use of Spark) can read data from S3 and ingest it into the database (Accumulo here, but commonly Hive or HBase for others).

One thought would be that you could use Spark to create Accumulo RFiles, store those in s3 and then get load them into Accumulo running not on s3. You could use ec2 to run Accumulo instead and bulk load your pre-created Accumulo RFiles from s3 (this should be fine, but I haven't tried it myself). It isn't quite the same as what you were hoping to get via S3, but I think it could be very close (easy to reprovision ec2 and re-import all of your current data to a new Accumulo instance).

I don't think S3 provides enough of a "real filesystem" implementation to run Accumulo natively over -- Shawn's points really drive the "why" home. This isn't something we can "fix Accumulo" to do as it would change the entire characteristics of the ssytem. Accumulo needs to be able to append data to a file and sync it to make it durable -- otherwise, Accumulo *will* eventually lose data. You may not see it in trivial testing, but I guarantee you 100% that you will run into data loss issues.

Does that make sense?

Shawn Walker wrote:
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<[email protected]>  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.


Reply via email to