You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@accumulo.apache.org by Fikri Akbar <fi...@ci-mediatrac.com> on 2016/04/08 12:30:23 UTC

Fwd: Data authorization/visibility limit in Accumulo

Hi Guys,

We're a group of accumulo enthusiasts from Indonesia. We've been trying to
implement accumulo for several different type of data processing purposes.
We've got several questions regarding Accumulo, which you might help us
with. We encounter these issues when we're trying to process heavy amount
of data, our questions are as follows:

1. Let's say that I have a file in HDFS that's about 300 GB with a total
1.6 Billion rows, and each line are separated by "^". The question is, what
is the most effective way to move the data to Accumulo (with assumption
that the structure of each cell is [rowkey cf:cq vis value] => [lineNumber
raw:columnName fileName columnValue])?

2. What is the most effective way to ingest data, if we're receiving data
with the size of >1 TB on a daily basis?

3. We're currently testing the ability of Accumulo for its data-level
access control, however the issue regarding the limit of dataset
authorization occurred when the datasets reached >20,000.

For example, lets say user X has a data called one.txt. This will make user
X has authorization to one.txt (let's call it X.one.txt). Now, what if X
has more than that (one.txt, two.xt, three.txt...n.txt), this will result
in user X having multiple authorization (as much as the data or n
authorization) and apparently when we tried it for datasets >20,000 (which
user will have >20,000 authorization), we're not able to execute "get
auth". We find that this is a very crucial issue, especially if (in one
case) there's >20,000 datasets that is being granted authorization at once.

The following are error logs from our system.

*Error log in shell:*

org.apache.accumulo.core.client.AccumuloException:
org.apache.thrift.TApplicationException: Internal error processing
getUserAuthorizations
        at
org.apache.accumulo.core.client.impl.SecurityOperationsImpl.execute(SecurityOperationsImpl.java:83)
        at
org.apache.accumulo.core.client.impl.SecurityOperationsImpl.getUserAuthorizations(SecurityOperationsImpl.java:182)
        at com.msk.auxilium.table.AuxUser.setUserAuth(AuxUser.java:310)
        at
com.msk.auxilium.commons.UserSystem.getAuxUser(UserSystem.java:24)
        at com.msk.auxilium.tester.HDFSTest.main(HDFSTest.java:57)
Caused by: org.apache.thrift.TApplicationException: Internal error
processing getUserAuthorizations
        at
org.apache.thrift.TApplicationException.read(TApplicationException.java:108)
        at
org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:71)
        at
org.apache.accumulo.core.client.impl.thrift.ClientService$Client.recv_getUserAuthorizations(ClientService.java:580)
        at
org.apache.accumulo.core.client.impl.thrift.ClientService$Client.getUserAuthorizations(ClientService.java:565)
        at
org.apache.accumulo.core.client.impl.SecurityOperationsImpl$6.execute(SecurityOperationsImpl.java:185)
        at
org.apache.accumulo.core.client.impl.SecurityOperationsImpl$6.execute(SecurityOperationsImpl.java:182)
        at
org.apache.accumulo.core.client.impl.ServerClient.executeRaw(ServerClient.java:90)
        at
org.apache.accumulo.core.client.impl.SecurityOperationsImpl.execute(SecurityOperationsImpl.java:69)
        ... 4 more

*Error log in accumulo master (web)*
tserver:

Zookeeper error, will retry
	org.apache.zookeeper.KeeperException$ConnectionLossException:
KeeperErrorCode = ConnectionLoss for
/accumulo/281c3ac0-74eb-4135-bc63-3158eabe2c47/tables/1a/conf/table.split.threshold
		at org.apache.zookeeper.KeeperException.create(KeeperException.java:99)
		at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
		at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:1041)
		at org.apache.accumulo.fate.zookeeper.ZooCache$2.run(ZooCache.java:210)
		at org.apache.accumulo.fate.zookeeper.ZooCache.retry(ZooCache.java:132)
		at org.apache.accumulo.fate.zookeeper.ZooCache.get(ZooCache.java:235)
		at org.apache.accumulo.fate.zookeeper.ZooCache.get(ZooCache.java:190)
		at org.apache.accumulo.server.conf.TableConfiguration.get(TableConfiguration.java:130)
		at org.apache.accumulo.server.conf.TableConfiguration.get(TableConfiguration.java:118)
		at org.apache.accumulo.core.conf.AccumuloConfiguration.getMemoryInBytes(AccumuloConfiguration.java:100)
		at org.apache.accumulo.tserver.Tablet.findSplitRow(Tablet.java:2892)
		at org.apache.accumulo.tserver.Tablet.needsSplit(Tablet.java:3032)
		at org.apache.accumulo.tserver.TabletServer$MajorCompactor.run(TabletServer.java:2603)
		at org.apache.accumulo.core.util.LoggingRunnable.run(LoggingRunnable.java:34)
		at java.lang.Thread.run(Thread.java:745)

*garbage collector:*

Zookeeper error, will retry
	org.apache.zookeeper.KeeperException$ConnectionLossException:
KeeperErrorCode = ConnectionLoss for
/accumulo/281c3ac0-74eb-4135-bc63-3158eabe2c47/tables
		at org.apache.zookeeper.KeeperException.create(KeeperException.java:99)
		at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
		at org.apache.zookeeper.ZooKeeper.getChildren(ZooKeeper.java:1468)
		at org.apache.accumulo.fate.zookeeper.ZooCache$1.run(ZooCache.java:169)
		at org.apache.accumulo.fate.zookeeper.ZooCache.retry(ZooCache.java:132)
		at org.apache.accumulo.fate.zookeeper.ZooCache.getChildren(ZooCache.java:180)
		at org.apache.accumulo.core.client.impl.Tables.getMap(Tables.java:126)
		at org.apache.accumulo.core.client.impl.Tables.getNameToIdMap(Tables.java:197)
		at org.apache.accumulo.core.client.impl.Tables._getTableId(Tables.java:173)
		at org.apache.accumulo.core.client.impl.Tables.getTableId(Tables.java:166)
		at org.apache.accumulo.core.client.impl.ConnectorImpl.getTableId(ConnectorImpl.java:84)
		at org.apache.accumulo.core.client.impl.ConnectorImpl.createScanner(ConnectorImpl.java:151)
		at org.apache.accumulo.gc.SimpleGarbageCollector$GCEnv.getCandidates(SimpleGarbageCollector.java:278)
		at org.apache.accumulo.gc.GarbageCollectionAlgorithm.getCandidates(GarbageCollectionAlgorithm.java:238)
		at org.apache.accumulo.gc.GarbageCollectionAlgorithm.collect(GarbageCollectionAlgorithm.java:272)
		at org.apache.accumulo.gc.SimpleGarbageCollector.run(SimpleGarbageCollector.java:544)
		at org.apache.accumulo.gc.SimpleGarbageCollector.main(SimpleGarbageCollector.java:154)
		at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
		at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
		at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
		at java.lang.reflect.Method.invoke(Method.java:606)
		at org.apache.accumulo.start.Main$1.run(Main.java:141)
		at java.lang.Thread.run(Thread.java:745)


we tried finding some resources regarding this issue, but couldn't find any
that mention the limit of authorizations per user and FYI we're using
accumulo version 1.6.

Sorry for the long email :) and have a great day.

Regards,

*Fikri Akbar*
Technology


*PT Mediatrac Sistem Komunikasi*
Grha Tirtadi 2nd Floor   |   Jl. Senopati 71-73   |   Jakarta 12110   |
Indonesia   |   *M**ap* 6°13'57.37"S 106°48'42.29"E
*P* +62 21 520 2568   |   *F* +62 21 520 4180   |   *M*  +62 812 1243 4786
   |   *www.mediatrac.co.id <http://www.mediatrac.co.id>*

Re: Fwd: Data authorization/visibility limit in Accumulo

Posted by Russ Weeks <rw...@newbrightidea.com>.
> Eventually, compactions might bog you down too (depending on how you
generated the data)

Yes, I've found that this is very important. If you're using a hash-based
partitioner to distribute the work in either Spark or M/R, it's easy to get
into a situation where each tablet server is responsible for every split of
every RFile. You wind up in this weird situation where the importDirectory
call takes almost as long as generating the RFiles in the first place! I
*think* it's because importDirectory is waiting for a bunch of compactions
to reorganize the data, but I'm not 100% sure.

-Russ



On Sun, Apr 10, 2016 at 10:08 PM Dylan Hutchison <dh...@cs.washington.edu>
wrote:

> On Sun, Apr 10, 2016 at 8:32 PM, Josh Elser <jo...@gmail.com> wrote:
>
> > Dylan Hutchison wrote:
> >
> >> >  2. What is the most effective way to ingest data, if we're receiving
> >>> data
> >>>
> >>>> >>  with the size of>1 TB on a daily basis?
> >>>> >>
> >>>>
> >>> >
> >>> >  If latency is not a primary concern, creating Accumuo RFiles and
> >>> >  performing bulk ingest/bulk loading is by far the most efficient way
> >>> to
> >>> >  getting data into Accumulo. This is often done by a MapReduce job to
> >>> >  process your incoming data, create Accumulo RFiles and then bulk
> load
> >>> these
> >>> >  files into Accumulo. If you have a low latency for getting data into
> >>> >  Accumuo, waiting for a MapReduce job to complete may take too long
> to
> >>> meet
> >>> >  your required latencies.
> >>> >
> >>> >
> >>>
> >> If you need a lower latency, you still have the option of parallel
> ingest
> >> via normal BatchWriters.  Assuming good load balancing and the same
> number
> >> of ingestors as tablet servers, you should easily obtain ingest rates of
> >> 100k entries/sec/node.  With significant effort, some have pushed this
> to
> >> 400k entries/sec/node.
> >>
> >> Josh, do we have numbers on bulk ingest rates?  I'm curious what the
> best
> >> rates ever achieved are.
> >>
> >
> > Hrm. Not that I'm aware of. Generally, a bulk import is some ZooKeeper
> > operations (via FATE) and a few metadata updates per file (~3? i'm not
> > actually sure). Maybe I'm missing something?
> >
> > My hunch is that you'd run into HDFS issues in generating the data to
> > import before you'd run into Accumulo limits. Eventually, compactions
> might
> > bog you down too (depending on how you generated the data). I'm not sure
> if
> > we even have a bulk-import benchmark (akin to continuous ingest).
> >
>
> Good point: this does depend on the original data source.  If the data
> source is itself the output of a MapReduce job, then MapReducing to RFiles
> is free (in the best case).  If the data source is a 1TB file on disk, then
> it is hard to say whether MapReduce->BulkImport or BatchWriter is faster,
> without empirical evidence on both solutions.
>
> Fikri, it sounds like the conclusion is that you should determine your
> latency requirement, and try whichever method is easiest to start and fits
> your requirement.  Then you can measure performance and keep the solution
> if it works, or seek another option if not.  You can report back your
> numbers and experience to us =)
>

Re: Fwd: Data authorization/visibility limit in Accumulo

Posted by Dylan Hutchison <dh...@cs.washington.edu>.
On Sun, Apr 10, 2016 at 8:32 PM, Josh Elser <jo...@gmail.com> wrote:

> Dylan Hutchison wrote:
>
>> >  2. What is the most effective way to ingest data, if we're receiving
>>> data
>>>
>>>> >>  with the size of>1 TB on a daily basis?
>>>> >>
>>>>
>>> >
>>> >  If latency is not a primary concern, creating Accumuo RFiles and
>>> >  performing bulk ingest/bulk loading is by far the most efficient way
>>> to
>>> >  getting data into Accumulo. This is often done by a MapReduce job to
>>> >  process your incoming data, create Accumulo RFiles and then bulk load
>>> these
>>> >  files into Accumulo. If you have a low latency for getting data into
>>> >  Accumuo, waiting for a MapReduce job to complete may take too long to
>>> meet
>>> >  your required latencies.
>>> >
>>> >
>>>
>> If you need a lower latency, you still have the option of parallel ingest
>> via normal BatchWriters.  Assuming good load balancing and the same number
>> of ingestors as tablet servers, you should easily obtain ingest rates of
>> 100k entries/sec/node.  With significant effort, some have pushed this to
>> 400k entries/sec/node.
>>
>> Josh, do we have numbers on bulk ingest rates?  I'm curious what the best
>> rates ever achieved are.
>>
>
> Hrm. Not that I'm aware of. Generally, a bulk import is some ZooKeeper
> operations (via FATE) and a few metadata updates per file (~3? i'm not
> actually sure). Maybe I'm missing something?
>
> My hunch is that you'd run into HDFS issues in generating the data to
> import before you'd run into Accumulo limits. Eventually, compactions might
> bog you down too (depending on how you generated the data). I'm not sure if
> we even have a bulk-import benchmark (akin to continuous ingest).
>

Good point: this does depend on the original data source.  If the data
source is itself the output of a MapReduce job, then MapReducing to RFiles
is free (in the best case).  If the data source is a 1TB file on disk, then
it is hard to say whether MapReduce->BulkImport or BatchWriter is faster,
without empirical evidence on both solutions.

Fikri, it sounds like the conclusion is that you should determine your
latency requirement, and try whichever method is easiest to start and fits
your requirement.  Then you can measure performance and keep the solution
if it works, or seek another option if not.  You can report back your
numbers and experience to us =)

Re: Fwd: Data authorization/visibility limit in Accumulo

Posted by Josh Elser <jo...@gmail.com>.
Dylan Hutchison wrote:
>> >  2. What is the most effective way to ingest data, if we're receiving data
>>> >>  with the size of>1 TB on a daily basis?
>>> >>
>> >
>> >  If latency is not a primary concern, creating Accumuo RFiles and
>> >  performing bulk ingest/bulk loading is by far the most efficient way to
>> >  getting data into Accumulo. This is often done by a MapReduce job to
>> >  process your incoming data, create Accumulo RFiles and then bulk load these
>> >  files into Accumulo. If you have a low latency for getting data into
>> >  Accumuo, waiting for a MapReduce job to complete may take too long to meet
>> >  your required latencies.
>> >
>> >
> If you need a lower latency, you still have the option of parallel ingest
> via normal BatchWriters.  Assuming good load balancing and the same number
> of ingestors as tablet servers, you should easily obtain ingest rates of
> 100k entries/sec/node.  With significant effort, some have pushed this to
> 400k entries/sec/node.
>
> Josh, do we have numbers on bulk ingest rates?  I'm curious what the best
> rates ever achieved are.

Hrm. Not that I'm aware of. Generally, a bulk import is some ZooKeeper 
operations (via FATE) and a few metadata updates per file (~3? i'm not 
actually sure). Maybe I'm missing something?

My hunch is that you'd run into HDFS issues in generating the data to 
import before you'd run into Accumulo limits. Eventually, compactions 
might bog you down too (depending on how you generated the data). I'm 
not sure if we even have a bulk-import benchmark (akin to continuous 
ingest).

Re: Fwd: Data authorization/visibility limit in Accumulo

Posted by Dylan Hutchison <dh...@cs.washington.edu>.
On Fri, Apr 8, 2016 at 3:13 PM, Josh Elser <jo...@gmail.com> wrote:

> Hi Fikri,
>
> Welcome! You're the first Accumulo enthusiast I've heard from in Indonesia
> :)
>
> Responses inline:
>
> Fikri Akbar wrote:
>
>> Hi Guys,
>>
>> We're a group of accumulo enthusiasts from Indonesia. We've been trying to
>> implement accumulo for several different type of data processing purposes.
>> We've got several questions regarding Accumulo, which you might help us
>> with. We encounter these issues when we're trying to process heavy amount
>> of data, our questions are as follows:
>>
>> 1. Let's say that I have a file in HDFS that's about 300 GB with a total
>> 1.6 Billion rows, and each line are separated by "^". The question is,
>> what
>> is the most effective way to move the data to Accumulo (with assumption
>> that the structure of each cell is [rowkey cf:cq vis value] =>
>> [lineNumber
>> raw:columnName fileName columnValue])?
>>
>
> For a 300GB file, you likely want to use MapReduce to ingest it into
> Accumulo. You can use the AccumuloOutputFormat to write to Accumulo
> directly from a MapReduce job.
>
> Reading data whose lines are separated by a '^' will likely require some
> custom InputFormat. I'm not sure if one already exists that you can build
> from. If you can convert the '^' to a standard newline character, you can
> probably leverage the existing TextInputFormat or similar.
>
> 2. What is the most effective way to ingest data, if we're receiving data
>> with the size of>1 TB on a daily basis?
>>
>
> If latency is not a primary concern, creating Accumuo RFiles and
> performing bulk ingest/bulk loading is by far the most efficient way to
> getting data into Accumulo. This is often done by a MapReduce job to
> process your incoming data, create Accumulo RFiles and then bulk load these
> files into Accumulo. If you have a low latency for getting data into
> Accumuo, waiting for a MapReduce job to complete may take too long to meet
> your required latencies.
>
>
If you need a lower latency, you still have the option of parallel ingest
via normal BatchWriters.  Assuming good load balancing and the same number
of ingestors as tablet servers, you should easily obtain ingest rates of
100k entries/sec/node.  With significant effort, some have pushed this to
400k entries/sec/node.

Josh, do we have numbers on bulk ingest rates?  I'm curious what the best
rates ever achieved are.


> 3. We're currently testing the ability of Accumulo for its data-level
>> access control, however the issue regarding the limit of dataset
>> authorization occurred when the datasets reached>20,000.
>>
>> For example, lets say user X has a data called one.txt. This will make
>> user
>> X has authorization to one.txt (let's call it X.one.txt). Now, what if X
>> has more than that (one.txt, two.xt, three.txt...n.txt), this will result
>> in user X having multiple authorization (as much as the data or n
>> authorization) and apparently when we tried it for datasets>20,000 (which
>> user will have>20,000 authorization), we're not able to execute "get
>> auth". We find that this is a very crucial issue, especially if (in one
>> case) there's>20,000 datasets that is being granted authorization at once.
>>
>
> Accumulo's column visibilities don't directly work well in the situation
> you describe; this is likely why you are having problems. Specifically,
> because the ColumnVisibility is a part of the Accumulo Key, you cannot
> update it without removing the old Key-Value and adding a new one.
>
> As such, ColumnVisibilities work much better as a labelling system than a
> direct authorization mechanism. Does that make sense? They are a building
> block to help you build authorization, not a complete authorization system
> on their own.
>
> Authorizations for users are stored in ZooKeeper by default, which is
> probably why you were having problems with 20k+ authorizations.
>
> Can you go into some detail on what your access control requirements are?
> For example, are documents only visible to one user known at ingest time?
> Do the set of allowed users for a file change over time?
>
> Commonly, some external system that manages the current roles for a user
> is a better approach here. For some $user, you can configure Accumulo to
> query that system to get the set of authorizations that $user current has
> and query that way. With some more specifics, we can try to get you a
> better recommendation.
>
>
> The following are error logs from our system.
>>
>> *Error log in shell:*
>>
>> org.apache.accumulo.core.client.AccumuloException:
>> org.apache.thrift.TApplicationException: Internal error processing
>> getUserAuthorizations
>>          at
>>
>> org.apache.accumulo.core.client.impl.SecurityOperationsImpl.execute(SecurityOperationsImpl.java:83)
>>          at
>>
>> org.apache.accumulo.core.client.impl.SecurityOperationsImpl.getUserAuthorizations(SecurityOperationsImpl.java:182)
>>          at com.msk.auxilium.table.AuxUser.setUserAuth(AuxUser.java:310)
>>          at
>> com.msk.auxilium.commons.UserSystem.getAuxUser(UserSystem.java:24)
>>          at com.msk.auxilium.tester.HDFSTest.main(HDFSTest.java:57)
>> Caused by: org.apache.thrift.TApplicationException: Internal error
>> processing getUserAuthorizations
>>          at
>>
>> org.apache.thrift.TApplicationException.read(TApplicationException.java:108)
>>          at
>> org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:71)
>>          at
>>
>> org.apache.accumulo.core.client.impl.thrift.ClientService$Client.recv_getUserAuthorizations(ClientService.java:580)
>>          at
>>
>> org.apache.accumulo.core.client.impl.thrift.ClientService$Client.getUserAuthorizations(ClientService.java:565)
>>          at
>>
>> org.apache.accumulo.core.client.impl.SecurityOperationsImpl$6.execute(SecurityOperationsImpl.java:185)
>>          at
>>
>> org.apache.accumulo.core.client.impl.SecurityOperationsImpl$6.execute(SecurityOperationsImpl.java:182)
>>          at
>>
>> org.apache.accumulo.core.client.impl.ServerClient.executeRaw(ServerClient.java:90)
>>          at
>>
>> org.apache.accumulo.core.client.impl.SecurityOperationsImpl.execute(SecurityOperationsImpl.java:69)
>>          ... 4 more
>>
>> *Error log in accumulo master (web)*
>> tserver:
>>
>> Zookeeper error, will retry
>>         org.apache.zookeeper.KeeperException$ConnectionLossException:
>> KeeperErrorCode = ConnectionLoss for
>>
>> /accumulo/281c3ac0-74eb-4135-bc63-3158eabe2c47/tables/1a/conf/table.split.threshold
>>                 at
>> org.apache.zookeeper.KeeperException.create(KeeperException.java:99)
>>                 at
>> org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
>>                 at
>> org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:1041)
>>                 at
>> org.apache.accumulo.fate.zookeeper.ZooCache$2.run(ZooCache.java:210)
>>                 at
>> org.apache.accumulo.fate.zookeeper.ZooCache.retry(ZooCache.java:132)
>>                 at
>> org.apache.accumulo.fate.zookeeper.ZooCache.get(ZooCache.java:235)
>>                 at
>> org.apache.accumulo.fate.zookeeper.ZooCache.get(ZooCache.java:190)
>>                 at
>> org.apache.accumulo.server.conf.TableConfiguration.get(TableConfiguration.java:130)
>>                 at
>> org.apache.accumulo.server.conf.TableConfiguration.get(TableConfiguration.java:118)
>>                 at
>> org.apache.accumulo.core.conf.AccumuloConfiguration.getMemoryInBytes(AccumuloConfiguration.java:100)
>>                 at
>> org.apache.accumulo.tserver.Tablet.findSplitRow(Tablet.java:2892)
>>                 at
>> org.apache.accumulo.tserver.Tablet.needsSplit(Tablet.java:3032)
>>                 at
>> org.apache.accumulo.tserver.TabletServer$MajorCompactor.run(TabletServer.java:2603)
>>                 at
>> org.apache.accumulo.core.util.LoggingRunnable.run(LoggingRunnable.java:34)
>>                 at java.lang.Thread.run(Thread.java:745)
>>
>> *garbage collector:*
>>
>> Zookeeper error, will retry
>>         org.apache.zookeeper.KeeperException$ConnectionLossException:
>> KeeperErrorCode = ConnectionLoss for
>> /accumulo/281c3ac0-74eb-4135-bc63-3158eabe2c47/tables
>>                 at
>> org.apache.zookeeper.KeeperException.create(KeeperException.java:99)
>>                 at
>> org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
>>                 at
>> org.apache.zookeeper.ZooKeeper.getChildren(ZooKeeper.java:1468)
>>                 at
>> org.apache.accumulo.fate.zookeeper.ZooCache$1.run(ZooCache.java:169)
>>                 at
>> org.apache.accumulo.fate.zookeeper.ZooCache.retry(ZooCache.java:132)
>>                 at
>> org.apache.accumulo.fate.zookeeper.ZooCache.getChildren(ZooCache.java:180)
>>                 at
>> org.apache.accumulo.core.client.impl.Tables.getMap(Tables.java:126)
>>                 at
>> org.apache.accumulo.core.client.impl.Tables.getNameToIdMap(Tables.java:197)
>>                 at
>> org.apache.accumulo.core.client.impl.Tables._getTableId(Tables.java:173)
>>                 at
>> org.apache.accumulo.core.client.impl.Tables.getTableId(Tables.java:166)
>>                 at
>> org.apache.accumulo.core.client.impl.ConnectorImpl.getTableId(ConnectorImpl.java:84)
>>                 at
>> org.apache.accumulo.core.client.impl.ConnectorImpl.createScanner(ConnectorImpl.java:151)
>>                 at
>> org.apache.accumulo.gc.SimpleGarbageCollector$GCEnv.getCandidates(SimpleGarbageCollector.java:278)
>>                 at
>> org.apache.accumulo.gc.GarbageCollectionAlgorithm.getCandidates(GarbageCollectionAlgorithm.java:238)
>>                 at
>> org.apache.accumulo.gc.GarbageCollectionAlgorithm.collect(GarbageCollectionAlgorithm.java:272)
>>                 at
>> org.apache.accumulo.gc.SimpleGarbageCollector.run(SimpleGarbageCollector.java:544)
>>                 at
>> org.apache.accumulo.gc.SimpleGarbageCollector.main(SimpleGarbageCollector.java:154)
>>                 at sun.reflect.NativeMethodAccessorImpl.invoke0(Native
>> Method)
>>                 at
>> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
>>                 at
>> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>>                 at java.lang.reflect.Method.invoke(Method.java:606)
>>                 at org.apache.accumulo.start.Main$1.run(Main.java:141)
>>                 at java.lang.Thread.run(Thread.java:745)
>>
>>
>> we tried finding some resources regarding this issue, but couldn't find
>> any
>> that mention the limit of authorizations per user and FYI we're using
>> accumulo version 1.6.
>>
>
> Can you give the Accumulo processes more Java heap space? ZooKeeper needs
> to maintain a heartbeat with ZooKeeper servers to stay connected. These
> error messages are implying that the Accumulo process cannot run in a
> timely manner which causes it be disconnected from ZooKeeper (and the
> client will error until it can be reconnected -- this happens
> asynchronously).
>
> Also, make sure that swappiness on your nodes is set to a value less than
> 10, ideally 1 or 0. Otherwise, the operating system may swap out pages in
> memory to disk and cause you to have pauses.
>
>
> Sorry for the long email :) and have a great day.
>>
>> Regards,
>>
>> *Fikri Akbar*
>> Technology
>>
>>
>> *PT Mediatrac Sistem Komunikasi*
>> Grha Tirtadi 2nd Floor   |   Jl. Senopati 71-73   |   Jakarta 12110   |
>> Indonesia   |   *M**ap* 6°13'57.37"S 106°48'42.29"E
>> *P* +62 21 520 2568   |   *F* +62 21 520 4180   |   *M*  +62 812 1243
>> 4786
>>     |   *www.mediatrac.co.id<http://www.mediatrac.co.id>*
>>
>>

Re: Fwd: Data authorization/visibility limit in Accumulo

Posted by Josh Elser <jo...@gmail.com>.
Hi Fikri,

Welcome! You're the first Accumulo enthusiast I've heard from in 
Indonesia :)

Responses inline:

Fikri Akbar wrote:
> Hi Guys,
>
> We're a group of accumulo enthusiasts from Indonesia. We've been trying to
> implement accumulo for several different type of data processing purposes.
> We've got several questions regarding Accumulo, which you might help us
> with. We encounter these issues when we're trying to process heavy amount
> of data, our questions are as follows:
>
> 1. Let's say that I have a file in HDFS that's about 300 GB with a total
> 1.6 Billion rows, and each line are separated by "^". The question is, what
> is the most effective way to move the data to Accumulo (with assumption
> that the structure of each cell is [rowkey cf:cq vis value] =>  [lineNumber
> raw:columnName fileName columnValue])?

For a 300GB file, you likely want to use MapReduce to ingest it into 
Accumulo. You can use the AccumuloOutputFormat to write to Accumulo 
directly from a MapReduce job.

Reading data whose lines are separated by a '^' will likely require some 
custom InputFormat. I'm not sure if one already exists that you can 
build from. If you can convert the '^' to a standard newline character, 
you can probably leverage the existing TextInputFormat or similar.

> 2. What is the most effective way to ingest data, if we're receiving data
> with the size of>1 TB on a daily basis?

If latency is not a primary concern, creating Accumuo RFiles and 
performing bulk ingest/bulk loading is by far the most efficient way to 
getting data into Accumulo. This is often done by a MapReduce job to 
process your incoming data, create Accumulo RFiles and then bulk load 
these files into Accumulo. If you have a low latency for getting data 
into Accumuo, waiting for a MapReduce job to complete may take too long 
to meet your required latencies.

> 3. We're currently testing the ability of Accumulo for its data-level
> access control, however the issue regarding the limit of dataset
> authorization occurred when the datasets reached>20,000.
>
> For example, lets say user X has a data called one.txt. This will make user
> X has authorization to one.txt (let's call it X.one.txt). Now, what if X
> has more than that (one.txt, two.xt, three.txt...n.txt), this will result
> in user X having multiple authorization (as much as the data or n
> authorization) and apparently when we tried it for datasets>20,000 (which
> user will have>20,000 authorization), we're not able to execute "get
> auth". We find that this is a very crucial issue, especially if (in one
> case) there's>20,000 datasets that is being granted authorization at once.

Accumulo's column visibilities don't directly work well in the situation 
you describe; this is likely why you are having problems. Specifically, 
because the ColumnVisibility is a part of the Accumulo Key, you cannot 
update it without removing the old Key-Value and adding a new one.

As such, ColumnVisibilities work much better as a labelling system than 
a direct authorization mechanism. Does that make sense? They are a 
building block to help you build authorization, not a complete 
authorization system on their own.

Authorizations for users are stored in ZooKeeper by default, which is 
probably why you were having problems with 20k+ authorizations.

Can you go into some detail on what your access control requirements 
are? For example, are documents only visible to one user known at ingest 
time? Do the set of allowed users for a file change over time?

Commonly, some external system that manages the current roles for a user 
is a better approach here. For some $user, you can configure Accumulo to 
query that system to get the set of authorizations that $user current 
has and query that way. With some more specifics, we can try to get you 
a better recommendation.

> The following are error logs from our system.
>
> *Error log in shell:*
>
> org.apache.accumulo.core.client.AccumuloException:
> org.apache.thrift.TApplicationException: Internal error processing
> getUserAuthorizations
>          at
> org.apache.accumulo.core.client.impl.SecurityOperationsImpl.execute(SecurityOperationsImpl.java:83)
>          at
> org.apache.accumulo.core.client.impl.SecurityOperationsImpl.getUserAuthorizations(SecurityOperationsImpl.java:182)
>          at com.msk.auxilium.table.AuxUser.setUserAuth(AuxUser.java:310)
>          at
> com.msk.auxilium.commons.UserSystem.getAuxUser(UserSystem.java:24)
>          at com.msk.auxilium.tester.HDFSTest.main(HDFSTest.java:57)
> Caused by: org.apache.thrift.TApplicationException: Internal error
> processing getUserAuthorizations
>          at
> org.apache.thrift.TApplicationException.read(TApplicationException.java:108)
>          at
> org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:71)
>          at
> org.apache.accumulo.core.client.impl.thrift.ClientService$Client.recv_getUserAuthorizations(ClientService.java:580)
>          at
> org.apache.accumulo.core.client.impl.thrift.ClientService$Client.getUserAuthorizations(ClientService.java:565)
>          at
> org.apache.accumulo.core.client.impl.SecurityOperationsImpl$6.execute(SecurityOperationsImpl.java:185)
>          at
> org.apache.accumulo.core.client.impl.SecurityOperationsImpl$6.execute(SecurityOperationsImpl.java:182)
>          at
> org.apache.accumulo.core.client.impl.ServerClient.executeRaw(ServerClient.java:90)
>          at
> org.apache.accumulo.core.client.impl.SecurityOperationsImpl.execute(SecurityOperationsImpl.java:69)
>          ... 4 more
>
> *Error log in accumulo master (web)*
> tserver:
>
> Zookeeper error, will retry
> 	org.apache.zookeeper.KeeperException$ConnectionLossException:
> KeeperErrorCode = ConnectionLoss for
> /accumulo/281c3ac0-74eb-4135-bc63-3158eabe2c47/tables/1a/conf/table.split.threshold
> 		at org.apache.zookeeper.KeeperException.create(KeeperException.java:99)
> 		at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
> 		at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:1041)
> 		at org.apache.accumulo.fate.zookeeper.ZooCache$2.run(ZooCache.java:210)
> 		at org.apache.accumulo.fate.zookeeper.ZooCache.retry(ZooCache.java:132)
> 		at org.apache.accumulo.fate.zookeeper.ZooCache.get(ZooCache.java:235)
> 		at org.apache.accumulo.fate.zookeeper.ZooCache.get(ZooCache.java:190)
> 		at org.apache.accumulo.server.conf.TableConfiguration.get(TableConfiguration.java:130)
> 		at org.apache.accumulo.server.conf.TableConfiguration.get(TableConfiguration.java:118)
> 		at org.apache.accumulo.core.conf.AccumuloConfiguration.getMemoryInBytes(AccumuloConfiguration.java:100)
> 		at org.apache.accumulo.tserver.Tablet.findSplitRow(Tablet.java:2892)
> 		at org.apache.accumulo.tserver.Tablet.needsSplit(Tablet.java:3032)
> 		at org.apache.accumulo.tserver.TabletServer$MajorCompactor.run(TabletServer.java:2603)
> 		at org.apache.accumulo.core.util.LoggingRunnable.run(LoggingRunnable.java:34)
> 		at java.lang.Thread.run(Thread.java:745)
>
> *garbage collector:*
>
> Zookeeper error, will retry
> 	org.apache.zookeeper.KeeperException$ConnectionLossException:
> KeeperErrorCode = ConnectionLoss for
> /accumulo/281c3ac0-74eb-4135-bc63-3158eabe2c47/tables
> 		at org.apache.zookeeper.KeeperException.create(KeeperException.java:99)
> 		at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
> 		at org.apache.zookeeper.ZooKeeper.getChildren(ZooKeeper.java:1468)
> 		at org.apache.accumulo.fate.zookeeper.ZooCache$1.run(ZooCache.java:169)
> 		at org.apache.accumulo.fate.zookeeper.ZooCache.retry(ZooCache.java:132)
> 		at org.apache.accumulo.fate.zookeeper.ZooCache.getChildren(ZooCache.java:180)
> 		at org.apache.accumulo.core.client.impl.Tables.getMap(Tables.java:126)
> 		at org.apache.accumulo.core.client.impl.Tables.getNameToIdMap(Tables.java:197)
> 		at org.apache.accumulo.core.client.impl.Tables._getTableId(Tables.java:173)
> 		at org.apache.accumulo.core.client.impl.Tables.getTableId(Tables.java:166)
> 		at org.apache.accumulo.core.client.impl.ConnectorImpl.getTableId(ConnectorImpl.java:84)
> 		at org.apache.accumulo.core.client.impl.ConnectorImpl.createScanner(ConnectorImpl.java:151)
> 		at org.apache.accumulo.gc.SimpleGarbageCollector$GCEnv.getCandidates(SimpleGarbageCollector.java:278)
> 		at org.apache.accumulo.gc.GarbageCollectionAlgorithm.getCandidates(GarbageCollectionAlgorithm.java:238)
> 		at org.apache.accumulo.gc.GarbageCollectionAlgorithm.collect(GarbageCollectionAlgorithm.java:272)
> 		at org.apache.accumulo.gc.SimpleGarbageCollector.run(SimpleGarbageCollector.java:544)
> 		at org.apache.accumulo.gc.SimpleGarbageCollector.main(SimpleGarbageCollector.java:154)
> 		at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> 		at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
> 		at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> 		at java.lang.reflect.Method.invoke(Method.java:606)
> 		at org.apache.accumulo.start.Main$1.run(Main.java:141)
> 		at java.lang.Thread.run(Thread.java:745)
>
>
> we tried finding some resources regarding this issue, but couldn't find any
> that mention the limit of authorizations per user and FYI we're using
> accumulo version 1.6.

Can you give the Accumulo processes more Java heap space? ZooKeeper 
needs to maintain a heartbeat with ZooKeeper servers to stay connected. 
These error messages are implying that the Accumulo process cannot run 
in a timely manner which causes it be disconnected from ZooKeeper (and 
the client will error until it can be reconnected -- this happens 
asynchronously).

Also, make sure that swappiness on your nodes is set to a value less 
than 10, ideally 1 or 0. Otherwise, the operating system may swap out 
pages in memory to disk and cause you to have pauses.

> Sorry for the long email :) and have a great day.
>
> Regards,
>
> *Fikri Akbar*
> Technology
>
>
> *PT Mediatrac Sistem Komunikasi*
> Grha Tirtadi 2nd Floor   |   Jl. Senopati 71-73   |   Jakarta 12110   |
> Indonesia   |   *M**ap* 6°13'57.37"S 106°48'42.29"E
> *P* +62 21 520 2568   |   *F* +62 21 520 4180   |   *M*  +62 812 1243 4786
>     |   *www.mediatrac.co.id<http://www.mediatrac.co.id>*
>