You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@accumulo.apache.org by vdelmeglio <va...@zalando.de> on 2016/04/25 10:54:41 UTC

Accumulo on s3

Hi everyone,

I recently got this answer on stackoverflow (link:
http://stackoverflow.com/questions/36602719/accumulo-cluster-in-aws-with-s3-not-really-stable/36772874#36772874):


>  Yes, I would expect that running Accumulo with S3 would result in
> problems. Even though S3 has a FileSystem implementation, it does not
> behave like a normal file system. Some examples of the differences are
> that operations we would expect to be atomic are not atomic in S3,
> exceptions may mean different things than we expect, and we assume our
> view of files and their metadata is consistent rather than the eventual
> consistency S3 provides.
> 
> It's possible these issues could be mitigated if we made some
> modifications to the Accumulo code, but as far as I know no one has tried
> running Accumulo on S3 to figure out the problems and whether those could
> be fixed or not. 

Since we're currently running an accumulo cluster on aws with s3 for
evaluation purpose, this answer make me wonder, should someone explain me
why running accumulo on s3 is not a good idea? in the specific, which
operations are expected to be atomic on accumulo?

Is there eventually a roadmap for s3 compatibility?

Thanks!
Valerio



--
View this message in context: http://apache-accumulo.1065345.n5.nabble.com/Accumulo-on-s3-tp16737.html
Sent from the Developers mailing list archive at Nabble.com.

Re: Accumulo on s3

Posted by William Slacum <ws...@gmail.com>.
Ephemeral storage & EBS are more friendly. Ephemeral storage is generally
the fastest and most HDFS-friendly.

On Mon, Apr 25, 2016 at 1:13 PM, Dylan Hutchison <dhutchis@cs.washington.edu
> wrote:

> Hey Josh,
>
> Are there other platforms on AWS (or another cloud provider) that
> Accumulo/HDFS are friendly to run on?  I thought I remembered you and
> others running the agitation tests on Amazon instances during
> release-testing time.  If there are alternatives, what advantages would S3
> have over the current method?
>
> On Mon, Apr 25, 2016 at 8:09 AM, Josh Elser <jo...@gmail.com> wrote:
>
> > I'm not sure on the guarantees of s3 (much less the s3 or s3a Hadoop
> > FileSystem implementations), but, historically, the common issue is
> > lacking/incorrect implementations of sync(). For durability (read-as: not
> > losing your data), Accumulo *must* know that when it calls sync() on a
> > file, the data is persisted.
> >
> > I don't know definitively what S3 guarantees (or asserts to guarantee),
> > but I would be very afraid until I ran some testing (we have one good
> test
> > in Accumulo that can run for days and verify data integrity called
> > continuous ingest).
> >
> > You might have luck reaching out to the Hadoop community to get some
> > understanding from them about what can reasonably be expected with the
> > current S3 FileSystem implementations, and then run your own tests to
> make
> > sure that data is not lost.
> >
> >
> > vdelmeglio wrote:
> >
> >> Hi everyone,
> >>
> >> I recently got this answer on stackoverflow (link:
> >>
> >>
> http://stackoverflow.com/questions/36602719/accumulo-cluster-in-aws-with-s3-not-really-stable/36772874#36772874
> >> ):
> >>
> >>
> >>   Yes, I would expect that running Accumulo with S3 would result in
> >>> problems. Even though S3 has a FileSystem implementation, it does not
> >>> behave like a normal file system. Some examples of the differences are
> >>> that operations we would expect to be atomic are not atomic in S3,
> >>> exceptions may mean different things than we expect, and we assume our
> >>> view of files and their metadata is consistent rather than the eventual
> >>> consistency S3 provides.
> >>>
> >>> It's possible these issues could be mitigated if we made some
> >>> modifications to the Accumulo code, but as far as I know no one has
> tried
> >>> running Accumulo on S3 to figure out the problems and whether those
> could
> >>> be fixed or not.
> >>>
> >>
> >> Since we're currently running an accumulo cluster on aws with s3 for
> >> evaluation purpose, this answer make me wonder, should someone explain
> me
> >> why running accumulo on s3 is not a good idea? in the specific, which
> >> operations are expected to be atomic on accumulo?
> >>
> >> Is there eventually a roadmap for s3 compatibility?
> >>
> >> Thanks!
> >> Valerio
> >>
> >>
> >>
> >> --
> >> View this message in context:
> >>
> http://apache-accumulo.1065345.n5.nabble.com/Accumulo-on-s3-tp16737.html
> >> Sent from the Developers mailing list archive at Nabble.com.
> >>
> >
>

Re: Accumulo on s3

Posted by Josh Elser <jo...@gmail.com>.
Yeah, ec2's EBS and ephemeral storage are fine AFAIK. I just don't know 
much anything at all about S3 (which might be why I'm inherently so 
pessimistic about it working :P).

Dylan Hutchison wrote:
> Hey Josh,
>
> Are there other platforms on AWS (or another cloud provider) that
> Accumulo/HDFS are friendly to run on?  I thought I remembered you and
> others running the agitation tests on Amazon instances during
> release-testing time.  If there are alternatives, what advantages would S3
> have over the current method?
>
> On Mon, Apr 25, 2016 at 8:09 AM, Josh Elser<jo...@gmail.com>  wrote:
>
>> I'm not sure on the guarantees of s3 (much less the s3 or s3a Hadoop
>> FileSystem implementations), but, historically, the common issue is
>> lacking/incorrect implementations of sync(). For durability (read-as: not
>> losing your data), Accumulo *must* know that when it calls sync() on a
>> file, the data is persisted.
>>
>> I don't know definitively what S3 guarantees (or asserts to guarantee),
>> but I would be very afraid until I ran some testing (we have one good test
>> in Accumulo that can run for days and verify data integrity called
>> continuous ingest).
>>
>> You might have luck reaching out to the Hadoop community to get some
>> understanding from them about what can reasonably be expected with the
>> current S3 FileSystem implementations, and then run your own tests to make
>> sure that data is not lost.
>>
>>
>> vdelmeglio wrote:
>>
>>> Hi everyone,
>>>
>>> I recently got this answer on stackoverflow (link:
>>>
>>> http://stackoverflow.com/questions/36602719/accumulo-cluster-in-aws-with-s3-not-really-stable/36772874#36772874
>>> ):
>>>
>>>
>>>    Yes, I would expect that running Accumulo with S3 would result in
>>>> problems. Even though S3 has a FileSystem implementation, it does not
>>>> behave like a normal file system. Some examples of the differences are
>>>> that operations we would expect to be atomic are not atomic in S3,
>>>> exceptions may mean different things than we expect, and we assume our
>>>> view of files and their metadata is consistent rather than the eventual
>>>> consistency S3 provides.
>>>>
>>>> It's possible these issues could be mitigated if we made some
>>>> modifications to the Accumulo code, but as far as I know no one has tried
>>>> running Accumulo on S3 to figure out the problems and whether those could
>>>> be fixed or not.
>>>>
>>> Since we're currently running an accumulo cluster on aws with s3 for
>>> evaluation purpose, this answer make me wonder, should someone explain me
>>> why running accumulo on s3 is not a good idea? in the specific, which
>>> operations are expected to be atomic on accumulo?
>>>
>>> Is there eventually a roadmap for s3 compatibility?
>>>
>>> Thanks!
>>> Valerio
>>>
>>>
>>>
>>> --
>>> View this message in context:
>>> http://apache-accumulo.1065345.n5.nabble.com/Accumulo-on-s3-tp16737.html
>>> Sent from the Developers mailing list archive at Nabble.com.
>>>
>

Re: Accumulo on s3

Posted by Dylan Hutchison <dh...@cs.washington.edu>.
Hey Josh,

Are there other platforms on AWS (or another cloud provider) that
Accumulo/HDFS are friendly to run on?  I thought I remembered you and
others running the agitation tests on Amazon instances during
release-testing time.  If there are alternatives, what advantages would S3
have over the current method?

On Mon, Apr 25, 2016 at 8:09 AM, Josh Elser <jo...@gmail.com> wrote:

> I'm not sure on the guarantees of s3 (much less the s3 or s3a Hadoop
> FileSystem implementations), but, historically, the common issue is
> lacking/incorrect implementations of sync(). For durability (read-as: not
> losing your data), Accumulo *must* know that when it calls sync() on a
> file, the data is persisted.
>
> I don't know definitively what S3 guarantees (or asserts to guarantee),
> but I would be very afraid until I ran some testing (we have one good test
> in Accumulo that can run for days and verify data integrity called
> continuous ingest).
>
> You might have luck reaching out to the Hadoop community to get some
> understanding from them about what can reasonably be expected with the
> current S3 FileSystem implementations, and then run your own tests to make
> sure that data is not lost.
>
>
> vdelmeglio wrote:
>
>> Hi everyone,
>>
>> I recently got this answer on stackoverflow (link:
>>
>> http://stackoverflow.com/questions/36602719/accumulo-cluster-in-aws-with-s3-not-really-stable/36772874#36772874
>> ):
>>
>>
>>   Yes, I would expect that running Accumulo with S3 would result in
>>> problems. Even though S3 has a FileSystem implementation, it does not
>>> behave like a normal file system. Some examples of the differences are
>>> that operations we would expect to be atomic are not atomic in S3,
>>> exceptions may mean different things than we expect, and we assume our
>>> view of files and their metadata is consistent rather than the eventual
>>> consistency S3 provides.
>>>
>>> It's possible these issues could be mitigated if we made some
>>> modifications to the Accumulo code, but as far as I know no one has tried
>>> running Accumulo on S3 to figure out the problems and whether those could
>>> be fixed or not.
>>>
>>
>> Since we're currently running an accumulo cluster on aws with s3 for
>> evaluation purpose, this answer make me wonder, should someone explain me
>> why running accumulo on s3 is not a good idea? in the specific, which
>> operations are expected to be atomic on accumulo?
>>
>> Is there eventually a roadmap for s3 compatibility?
>>
>> Thanks!
>> Valerio
>>
>>
>>
>> --
>> View this message in context:
>> http://apache-accumulo.1065345.n5.nabble.com/Accumulo-on-s3-tp16737.html
>> Sent from the Developers mailing list archive at Nabble.com.
>>
>

Re: Accumulo on s3

Posted by David Medinets <da...@gmail.com>.
What if the compaction process was changed to create S3 files? All incoming
mutations would stay be written to HDFS but compaction would generate S3
files.

On Wed, Apr 27, 2016 at 10:09 AM, Shawn Walker <ac...@shawn-walker.net>
wrote:

> Following up on my earlier description, I had a thought toward Valerio's
> goals.  It might be possible to modify Accumulo to support a hybrid setup.
> We could arrange to store RFiles in a NativeS3Filesystem, and store
> everything else (WALs, etc) in a HDFS.  I believe this could be implemented
> by extending the multiple volume support to be aware of what type of file
> was being created, and then creating an appropriate VolumeChooser.
>
> While not exactly what Valerio is asking after, such a setup shoud mostly
> meet his goals.  The bulk of an Accumulo instance's data should be in its
> RFiles, while the size of "everything else" should be relatively small.
> The needed amount of HDFS space should depend on the cluster load (ingest
> rate, scans active, etc) instead of the amount of data stored.  As such, a
> cluster with a more-or-less fixed load should be able to ingest arbitrarily
> much data without needing ever-growing HDFS space.
>
> I'm still dubious of the general concept, though.  While my proposal should
> limit the need to scale a HDFS cluster with data size, one would still need
> to grow the Accumulo cluster with the data size.  Certainly a single
> tserver would be overwhelmed if it was asked to manage millions of tablets
> by itself.  Given the need to grow the cluster of tservers, it might make
> just as much sense to grow an HDFS cluster at the same time, instead of
> maintaining a hybrid S3 / HDFS storage solution.
>
> Going a different direction, it might be possible to enhance Hadoop and
> implement a more fully featured filesystem atop S3 with the help of a
> consistent store (e.g. Zookeeper) for bookkeeping.
>
> --
>
> Shawn Walker
>
>
> On Apr 26, 2016 10:59 AM, "Josh Elser" <jo...@gmail.com> wrote:
>
> > 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<te...@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.
> >>>
> >>>
> >>
>

Re: Accumulo on s3

Posted by Christopher <ct...@apache.org>.
getWalDirs() is used only in legacy code intended to ease upgrades from
1.4.x to 1.5.x. It probably has no value in the current code, as I'm not
sure we've even tested upgrades from 1.4.x to 1.7.x.

On Thu, May 12, 2016 at 11:03 AM Shawn Walker <ac...@shawn-walker.net>
wrote:

> Teng Qiu,
>
> I don't believe your proposed change would suffice.  I haven't spent much
> time exploring how one might implement my suggestion.  Two things:
>
> 1. I've just recently started working with Accumulo; as such I'm not
> certain that WALs are the only files Accumulo uses that need more
> capability than S3 can provide.  Just that it should be reasonable to store
> RFiles in S3, modulo some concerns about renaming.
>
> 2. Changing getWalDirs() would not be effective, as this function isn't
> used everywhere that matters. Indeed, my (quick) skim of the source
> indicates that getWalDirs() isn't used anywhere that matters.  For example,
> the ultimate file location for a tserver's WAL looks to be decided at
> [tserver
> - DfsLogger.open(...):379]
> <
> https://github.com/apache/accumulo/blob/rel/1.7.1/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java#L379
> >
> which doesn't call ServerConstants.getWalDirs().
>
> Instead, I was suggesting changing the signature of [server/base -
> VolumeManager.choose(...):161]
> <
> https://github.com/apache/accumulo/blob/rel/1.7.1/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManager.java#L161
> >
> to take an additional parameter indicating the purpose of the volume (RFile
> storage, WAL storage, temporary storage, etc), and passing that information
> along to the configured VolumeChooser by modifying [server/base -
> VolumeChooserEnvironment]
> <
> https://github.com/apache/accumulo/blob/rel/1.7.1/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooserEnvironment.java
> >.
> The ultimate goal would be to allow a custom VolumeChooser to better choose
> volumes based on what type of use the volume would need to support.
>
> I'm not entirely certain this tactic would work, but it would be where I
> might start.
>
> --
> Shawn Walker
>
>
> On Tue, May 10, 2016 at 11:49 AM, chutium <te...@gmail.com> wrote:
>
> > Hi Shawn, thanks for the detailed reply, we took a look into accumulo
> code,
> > and found the constant WAL_DIR defined in ServerConstants:
> >
> >
> >
> https://github.com/apache/accumulo/blob/rel/1.7.1/server/base/src/main/java/org/apache/accumulo/server/ServerConstants.java#L139
> >
> >
> https://github.com/apache/accumulo/blob/rel/1.7.1/server/base/src/main/java/org/apache/accumulo/server/ServerConstants.java#L151
> > (we are using 1.7.1)
> >
> > is that means, we can simply modify the return value of getWalDirs() to
> > change the location of WAL logs?
> >
> > Thanks
> >
> >
> >
> > --
> > View this message in context:
> >
> http://apache-accumulo.1065345.n5.nabble.com/Accumulo-on-s3-tp16737p16872.html
> > Sent from the Developers mailing list archive at Nabble.com.
> >
>

Re: Accumulo on s3

Posted by Shawn Walker <ac...@shawn-walker.net>.
Teng Qiu,

I don't believe your proposed change would suffice.  I haven't spent much
time exploring how one might implement my suggestion.  Two things:

1. I've just recently started working with Accumulo; as such I'm not
certain that WALs are the only files Accumulo uses that need more
capability than S3 can provide.  Just that it should be reasonable to store
RFiles in S3, modulo some concerns about renaming.

2. Changing getWalDirs() would not be effective, as this function isn't
used everywhere that matters. Indeed, my (quick) skim of the source
indicates that getWalDirs() isn't used anywhere that matters.  For example,
the ultimate file location for a tserver's WAL looks to be decided at [tserver
- DfsLogger.open(...):379]
<https://github.com/apache/accumulo/blob/rel/1.7.1/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java#L379>
which doesn't call ServerConstants.getWalDirs().

Instead, I was suggesting changing the signature of [server/base -
VolumeManager.choose(...):161]
<https://github.com/apache/accumulo/blob/rel/1.7.1/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManager.java#L161>
to take an additional parameter indicating the purpose of the volume (RFile
storage, WAL storage, temporary storage, etc), and passing that information
along to the configured VolumeChooser by modifying [server/base -
VolumeChooserEnvironment]
<https://github.com/apache/accumulo/blob/rel/1.7.1/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooserEnvironment.java>.
The ultimate goal would be to allow a custom VolumeChooser to better choose
volumes based on what type of use the volume would need to support.

I'm not entirely certain this tactic would work, but it would be where I
might start.

--
Shawn Walker


On Tue, May 10, 2016 at 11:49 AM, chutium <te...@gmail.com> wrote:

> Hi Shawn, thanks for the detailed reply, we took a look into accumulo code,
> and found the constant WAL_DIR defined in ServerConstants:
>
>
> https://github.com/apache/accumulo/blob/rel/1.7.1/server/base/src/main/java/org/apache/accumulo/server/ServerConstants.java#L139
>
> https://github.com/apache/accumulo/blob/rel/1.7.1/server/base/src/main/java/org/apache/accumulo/server/ServerConstants.java#L151
> (we are using 1.7.1)
>
> is that means, we can simply modify the return value of getWalDirs() to
> change the location of WAL logs?
>
> Thanks
>
>
>
> --
> View this message in context:
> http://apache-accumulo.1065345.n5.nabble.com/Accumulo-on-s3-tp16737p16872.html
> Sent from the Developers mailing list archive at Nabble.com.
>

Re: Accumulo on s3

Posted by chutium <te...@gmail.com>.
Hi Shawn, thanks for the detailed reply, we took a look into accumulo code,
and found the constant WAL_DIR defined in ServerConstants:

https://github.com/apache/accumulo/blob/rel/1.7.1/server/base/src/main/java/org/apache/accumulo/server/ServerConstants.java#L139
https://github.com/apache/accumulo/blob/rel/1.7.1/server/base/src/main/java/org/apache/accumulo/server/ServerConstants.java#L151
(we are using 1.7.1)

is that means, we can simply modify the return value of getWalDirs() to
change the location of WAL logs?

Thanks



--
View this message in context: http://apache-accumulo.1065345.n5.nabble.com/Accumulo-on-s3-tp16737p16872.html
Sent from the Developers mailing list archive at Nabble.com.

Re: Accumulo on s3

Posted by Shawn Walker <ac...@shawn-walker.net>.
Following up on my earlier description, I had a thought toward Valerio's
goals.  It might be possible to modify Accumulo to support a hybrid setup.
We could arrange to store RFiles in a NativeS3Filesystem, and store
everything else (WALs, etc) in a HDFS.  I believe this could be implemented
by extending the multiple volume support to be aware of what type of file
was being created, and then creating an appropriate VolumeChooser.

While not exactly what Valerio is asking after, such a setup shoud mostly
meet his goals.  The bulk of an Accumulo instance's data should be in its
RFiles, while the size of "everything else" should be relatively small.
The needed amount of HDFS space should depend on the cluster load (ingest
rate, scans active, etc) instead of the amount of data stored.  As such, a
cluster with a more-or-less fixed load should be able to ingest arbitrarily
much data without needing ever-growing HDFS space.

I'm still dubious of the general concept, though.  While my proposal should
limit the need to scale a HDFS cluster with data size, one would still need
to grow the Accumulo cluster with the data size.  Certainly a single
tserver would be overwhelmed if it was asked to manage millions of tablets
by itself.  Given the need to grow the cluster of tservers, it might make
just as much sense to grow an HDFS cluster at the same time, instead of
maintaining a hybrid S3 / HDFS storage solution.

Going a different direction, it might be possible to enhance Hadoop and
implement a more fully featured filesystem atop S3 with the help of a
consistent store (e.g. Zookeeper) for bookkeeping.

--

Shawn Walker


On Apr 26, 2016 10:59 AM, "Josh Elser" <jo...@gmail.com> wrote:

> 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<te...@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.
>>>
>>>
>>

Re: Accumulo on s3

Posted by Josh Elser <jo...@gmail.com>.
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<te...@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.
>>
>

Re: Accumulo on s3

Posted by Shawn Walker <ac...@shawn-walker.net>.
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 <te...@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.
>

Re: Accumulo on s3

Posted by chutium <te...@gmail.com>.
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.

Re: Accumulo on s3

Posted by Josh Elser <jo...@gmail.com>.
I'm not sure on the guarantees of s3 (much less the s3 or s3a Hadoop 
FileSystem implementations), but, historically, the common issue is 
lacking/incorrect implementations of sync(). For durability (read-as: 
not losing your data), Accumulo *must* know that when it calls sync() on 
a file, the data is persisted.

I don't know definitively what S3 guarantees (or asserts to guarantee), 
but I would be very afraid until I ran some testing (we have one good 
test in Accumulo that can run for days and verify data integrity called 
continuous ingest).

You might have luck reaching out to the Hadoop community to get some 
understanding from them about what can reasonably be expected with the 
current S3 FileSystem implementations, and then run your own tests to 
make sure that data is not lost.

vdelmeglio wrote:
> Hi everyone,
>
> I recently got this answer on stackoverflow (link:
> http://stackoverflow.com/questions/36602719/accumulo-cluster-in-aws-with-s3-not-really-stable/36772874#36772874):
>
>
>>   Yes, I would expect that running Accumulo with S3 would result in
>> problems. Even though S3 has a FileSystem implementation, it does not
>> behave like a normal file system. Some examples of the differences are
>> that operations we would expect to be atomic are not atomic in S3,
>> exceptions may mean different things than we expect, and we assume our
>> view of files and their metadata is consistent rather than the eventual
>> consistency S3 provides.
>>
>> It's possible these issues could be mitigated if we made some
>> modifications to the Accumulo code, but as far as I know no one has tried
>> running Accumulo on S3 to figure out the problems and whether those could
>> be fixed or not.
>
> Since we're currently running an accumulo cluster on aws with s3 for
> evaluation purpose, this answer make me wonder, should someone explain me
> why running accumulo on s3 is not a good idea? in the specific, which
> operations are expected to be atomic on accumulo?
>
> Is there eventually a roadmap for s3 compatibility?
>
> Thanks!
> Valerio
>
>
>
> --
> View this message in context: http://apache-accumulo.1065345.n5.nabble.com/Accumulo-on-s3-tp16737.html
> Sent from the Developers mailing list archive at Nabble.com.