if the values
> are different keep one entry with the later timestamp (then skip to the
> next record). I think you'll have to write a custom iterator for that.
>
> Billie
>
>
> > Naturally, I could query the database before the ingest insert. But,
> > referring to slide 19 in Adam's presentation at
> > http://people.apache.org/~afuchs/slides/accumulo_table_design.pdf, the
> > read-modify-write design is not optimal.
>
--
Corey Nolet
Senior Software Engineer
TexelTek, inc.
[Office] 301.880.7123
[Cell] 410-903-2110
Hadoop & Zookeeper put that information at the INFO level the their logs don't
they?
On Jul 12, 2012, at 9:41 PM, David Medinets wrote:
> I was thinking that someone might not want hostname and port
> information in log files for security reasons. By putting the
> 'sensitive' information in a DE
e who's name is Paul or who's name is Gary or
who's name is Lee who has Brown hair? That would mean I need to look up
everything where (name=Paul | name=Gary | name=Lee) & hairColor=Brown. Do I
need to extend the IntersectingIterator or the IndexedDocIterator and
making my own that wi
You can extend the output format to write to both and have the resulting record
writer underneath write to the correct endpoint depending on the items
submitted from the job.
On Oct 16, 2012, at 10:16 PM, Huanchen Zhang wrote:
> Hello,
>
> Hese I have a mapreduce job which needs to write t
t 10:48 PM, Huanchen Zhang wrote:
> Hello, Corey
>
> Thank you for your answer.
>
> Can I use InsertWithBatchWriter for this task ? I mean, use context.write to
> write to hdfs, use batchwriter.addMutation to write to accumulo.
>
> Huanchen
>
> On Oct 16, 2012, at
issues until trying to update my code for the new
version. If I'm doing this completely wrong, any ideas on how to make this
better?
Thanks!
--
Corey Nolet
Senior Software Engineer
TexelTek, inc.
[Office] 301.880.7123
[Cell] 410-903-2110
is no guarantee that the iterator remains intact
> through the entire scan, and it instead may be reconstructed, causing state
> to be lost. I don't think this is the case for compaction time iterators, but
> I'm not positive.
>
>
> On Thu, Jan 3, 2013 at 5:41 PM,
ough the index rows of a
>> specific "index" and build up a hashset, then init() will be called which
>> wipes away the hashset of uuids, then the further goes on to iterate through
>> the key/value rows. Keep in mind, we are talking about maybe 400k entries,
>>
t;>>
>>> The default buffer size was changed from 50M to 1M at some point.
>>> This is configured via the property table.scan.max.memory
>>>
>>> The lower buffer size will cause iterator to be reinitialized more
>>> frequently. Maybe you are seein
We have a sharded-event table that failed miserably when we accidentally tried
to merge all of the tablets together. When starting accumulo, the monitor page
says the event table (once having 43k tablets) now has 5 tablets and 1.05B
rows. There are 14.5k unassigned tablets, The tablet servers ea
The Thrift Proxy Server in 1.5 has had me excited so I started working on a C++
client API. I have the code checked in to GitHub if anyone is interested in
using or helping.
https://github.com/cjnolet/accumulo-cpp
Would be useful to implement a "-h" flag that makes displays "1.4G" or "500M"
based on the input.
On Apr 23, 2013, at 10:31 AM, Keith Turner wrote:
>
>
>
> On Tue, Apr 23, 2013 at 9:17 AM, Chris Sigman wrote:
> Thanks Keith, I thought that might be the case considering the output, but it
>
ACCUMULO-1438 and ACCUMULO-1414 will be checked in shortly. Do we have a
desired time for rolling the RC5? Next couple of days? Before the end of
the week?
On Tue, May 21, 2013 at 1:39 PM, Christopher wrote:
> ACCUMULO-1441, ACCUMULO-1436, ACCUMULO-1440 are all issues that have
> been identifie
You are correct. Include as an attachment on the ticket.
On May 29, 2013 7:48 PM, "Aaron" wrote:
> Ok, is the best way just to add as an attachment? I did see that it got
> rolled into a larger "configurator" issue..just upload the attachment to
> the ticket? First time submitting patch for Acc
Aaron,
We are currently re-working the AccumuloInputFormat for Accumulo 1.6 to
provide inputs from multiple tables (each with their own set of configured
iterators, ranges, columns). Check out ACCUMULO-391.
On Mon, Sep 16, 2013 at 11:41 AM, Aaron wrote:
> I was curious if this is possible (i
on a single table but the use case sounds interesting.
On Mon, Sep 16, 2013 at 3:55 PM, Corey Nolet wrote:
> Aaron,
>
> We are currently re-working the AccumuloInputFormat for Accumulo 1.6 to
> provide inputs from multiple tables (each with their own set of configured
> it
Matt,
This should help:
Collection> cols = Collections.singleton(new
Pair(new Text("cityOfBirth"), null));
AccumuloInputFormat.fetchColumns(job, cols);
On Wed, Jan 15, 2014 at 7:29 PM, Dickson, Matt MR <
matt.dick...@defence.gov.au> wrote:
> *UNOFFICIAL*
> Thanks Keith. I've run a simple mr
Geoffry,
What OSGi container are you using currently? The servicemix Hadoop bundle
should get you going with the Hadoop client dependencies at least [1]. It
looks like one of the servicemix guys created a Hadoop ticket for making
bundles of their jars as well [2], though it doesn't look like there
ed on ISO standards so I need to walk the
>> straight and narrow and not drop a stitch. I have code that does what I
>> need (persist any graph sight unseen) into MongoDB. I need to adapt said
>> code to Accumulo. All the above is OSGi based so it would really help if I
>>
Geoffry,
As Josh pointed out, you should only need the Hadoop libraries on the
client side to use the Text object. This means you won't have to go through
the pain of placing the xml files in your root bundles.
Did you try the JAAS export from the packages in your container? Did that
help?
I agr
gt; So far as my travails with JAAS is concerned, I did this in my bndtools
> *.bndrun file:
>
>
> -runproperties:
> org.osgi.framework.system.packages.extra=com.sun.security.auth.module
>
> I also tried:
> Import-Package: com.sun.security.auth.module
> in my bundle that calls H
Geoffry,
Unfortunately, I will not be able to provide a patch on the internet for
the work I've done, yet. I hope some of the resources I've provided can be
a good starting place for you. I can certainly be available to help you
through some of the painful problems that I went through, but there's
+1 for slf4j.
On Wed, Apr 23, 2014 at 10:51 AM, Josh Elser wrote:
> I'd love to see us move to slf4j. Hadoop is in the middle of a proposal
> about this too which sounds good to me.
>
> http://mail-archives.apache.org/mod_mbox/hadoop-common-
> dev/201404.mbox/%3CCA%2B4kjVv7N2dRR5rmdFHCpBx-K3yT7
Can we assume this data has not yet been ingested? Do you have control over
the way in which you structure your table?
On Fri, May 16, 2014 at 1:54 PM, David Medinets wrote:
> If I have the following simple set of data:
>
> NAME John
> NAME Jake
> NAME John
> NAME Mary
>
> I want to end up with
Has the table been compacted since loading the data?
Hi Russ,
I believe that the AccumuloInputFormat will use the splits on the table
you're reading to generate the MR InputSplits. The InputFormat should be
trying to run the Mappers on the same machine as the tserver serving the
data is located.
esn't require
> re-processing all the data. Still researching.
>
>
> On Fri, May 16, 2014 at 4:19 PM, Corey Nolet wrote:
>
>> Can we assume this data has not yet been ingested? Do you have control
>> over the way in which you structure your table?
>>
>>
>>
Jeff,
Unless you've got multiple different tables with different permissions to
manage for different physical Accumulo users, the connector should probably
be an instance variable in your service. It can be safely shared across all
the reads as long as the Accumulo user configured in the connector
Andrew,
Our recommendation on this has typically been to reverse the sort order of
the keys on ingest.
On Mon, Jun 30, 2014 at 12:24 PM, Andrew Wells
wrote:
> Are there currently any good practices on doing this?
>
> Especially when a rowId has a large number of Keys.
>
> --
> *Andrew George W
ur situation.
>>
>>
>> On Mon, Jun 30, 2014 at 12:26 PM, Corey Nolet wrote:
>>
>>> Andrew,
>>>
>>> Our recommendation on this has typically been to reverse the sort order
>>> of the keys on ingest.
>>>
>>>
>>> On Mo
On Calrissian, we had once considered making a lexicoder for lat/long that
really transformed the two-dimensions (lat/lon) down into a geohash based
on the z-curve.
The reason we decided against a first class data-type for this is exactly
the same reason that Anthony brings up in his previous comm
Hey John,
Could you give an example of one of the ranges you are using which causes
this to happen?
On Fri, Aug 22, 2014 at 11:02 PM, John Yost
wrote:
> Hey Everyone,
>
> The AccumuloMultiTableInputFormat is an awesome addition to the Accumulo
> API and I am really excited to start using it.
>
The table configs get serialized as base64 and placed in the job's
Configuration under the key "AccumuloInputFormat.ScanOpts.TableConfigs".
Could you verify/print what's being placed in this key in your
configuration?
On Sat, Aug 23, 2014 at 12:15 AM, JavaHokie
wrote:
> Hey Corey,
>
> Sure thi
The tests I'm running aren't using the native Hadoop libs either. If you
don't mind, a little more code as to how you are setting up your job would
be useful. That's weird the key in the config would be null. Are you using
the job.getConfiguration()?
On Sat, Aug 23, 2014 at 12:31 AM, JavaHokie
Job.getInstance(configuration) copies the configuration and makes its own.
Try doing your debug statement from earlier on job.getConfiguration() and
let's see what the base64 string looks like.
On Sat, Aug 23, 2014 at 1:00 AM, JavaHokie
wrote:
> Sure thing, here's my run method implementation:
On Sat, Aug 23, 2014 at 1:11 AM, Corey Nolet wrote:
> Job.getInstance(configuration) copies the configuration and makes its own.
> Try doing your debug statement from earlier on job.getConfiguration() and
> let's see what the base64 string looks like.
>
>
>
> On Sa
Sure. I was able to deserialize the base64 that you posted earlier and it
looks fine. The code I used to do this was like this:
byte[] serialized =
"AQEAOm9yZy5hcGFjaGUuYWNjdW11bG8uY29yZS5jbGllbnQubWFwcmVkdWNlLklucHV0VGFibGVDb25maWcCjAlmb2xsb3dpbmcBAAIAAAYGBgYxMDQ1ODeIf/8AB
That code I posted should be able to validate where you are getting hung
up. Can you try running that on the machine and seeing if it prints the
expected tables/ranges?
Also, are you running the job live? What does the configuration look like
for the job on your resource manager? Can you see if t
Awesome! I was going to recommend checking out the code last night so that
you could put some logging statements in there. You've probably noticed
this already but the MapWritable does not have static type parameters so it
dumps out the fully qualified class name so that it can instantiate it back
I'm thinking this could be a yarn.application.classpath configuration
problem in your yarn-site.xml. I meant to ask earlier- how are you building
your jar that gets deployed? Are you shading it? Using libjars?
On Sun, Aug 24, 2014 at 6:56 AM, JavaHokie
wrote:
> Hey Corey,
>
> Yah, sometimes ya
Awesome John! It's good to have this documented for future users. Keep us
updated!
On Sun, Aug 24, 2014 at 11:05 AM, JavaHokie
wrote:
> Hi Corey,
>
> Just to wrap things up, AccumuloMultipeTableInputFormat is working really
> well. This is an outstanding feature I can leverage big-time on my c
>
> On Sun, Aug 24, 2014 at 6:50 PM, Corey Nolet-2 [via Apache Accumulo] <[hidden
> email] <http://user/SendEmail.jtp?type=node&node=11303&i=0>> wrote:
>
>> Awesome John! It's good to have this documented for future users. Keep us
>> updated!
&g
The Apache Accumulo project is happy to announce its 1.6.1 release.
Version 1.6.1 is the most recent bug-fix release in its 1.6.x release line.
This version includes numerous bug fixes and performance improvements over
previous versions. Existing users of 1.6.x are encouraged to upgrade to
this ve
The Fluo project is happy to announce the 1.0.0-alpha-1 release of Fluo.
Fluo is a transaction layer that enables incremental processing on top of
Accumulo. It integrates into Yarn using Apache Twill.
This is the first release of Fluo and is not ready for production use. We
invite developers to t
Ranjan,
Storm doesn't expose locality that would aid in pulling data locally from
tablet servers on supervisor nodes so there's no real "native"
implementation of a Spout or Bolt for Accumulo. That being said, the
Accumulo BatchWriter and Scanner/BatchScanner can be used quite easily in
the Storm
Dylan,
I know your original post mentioned grabbing it through the client API but
there's not currently a way to do that. As Sean mentioned, you can do it if
you have access to the cluster. You can run the reflection Keith provided
by adding the files in $ACCUMULO_HOME/lib/ to your classpath and r
I had a ticket for that awhile back and I don't believe it was ever
completed. By default, it wants to dump out new config files for
everything- have it reusing a config file would mean not re-initializing
each time and reusing the same instance id + rfiles.
ACCUMULO-1378 was the it and it looks li
Devs,
Please consider the following candidate for Apache Accumulo 1.6.2
Branch: 1.6.2-rc2
SHA1: 34987b4c8b4d896bbf2d26be8e70f70976614c0f
Staging Repository:
https://repository.apache.org/content/repositories/orgapacheaccumulo-1020/
Source tarball:
https://repository.apache.
mulo/1.6.1_to_1.6.2/compat_report.html
* 1.6.2 -> 1.6.1 (under a semver patch increment, this should be just as
strong an assertion as the reverse)
http://people.apache.org/~busbey/compat_reports/accumulo/1.6.2_to_1.6.1/compat_report.html
On Fri, Jan 23, 2015 at 8:02 PM, Corey Nolet wrote:
The Apache Accumulo project is happy to announce its 1.6.2 release.
Version 1.6.2 is the most recent bug-fix release in its 1.6.x release line.
This version includes numerous bug fixes as well as a performance
improvement over previous versions. Existing users of 1.6.x are encouraged
to upgrade to
Andrew,
Have you considered leveraging existing SQL query layers like Hive or
Spark's SQL/DataFrames API? There are some pretty massive optimizations
involved in that API making the push-down predicates / selections pretty
easy to adapt for Accumulo.
On Mon, Apr 27, 2015 at 8:37 PM, Andrew Wells
I'm always looking for places to help out and integrate/share designs &
ideas. I look forward to chatting with you about Q4A at the hackathon
tomorrow!
Have you, by chance, seen the Spark SQL adapter for the Accumulo Recipes
Event & Entity Stores [1]? At the very least, it's a good example of usin
Vaibnav,
The difference in an OR iterator is that you will want it to return a
single key for all of the given OR terms so that the iterator in the stack
above it would see it was a single "hit". It's essentially a merge at the
key level to stop duplicate results from being returned (thus appearin
Sven,
What version of Accumulo are you running? We have a ticket for this [1]
which has had a lot of discussion on it. Christopher Tubbs mentioned that
he had gotten this to work.
[1] https://issues.apache.org/jira/browse/ACCUMULO-1378
On Wed, Sep 16, 2015 at 9:20 AM, Sven Hodapp wrote:
> Hi t
10:40 Sven Hodapp
> wrote:
>
>> Hi Corey,
>>
>> thanks for your reply and the link. Sounds good, if that will be
>> available in the future!
>> Is the code from Christopher somewhere deployed?
>>
>> Currently I'm using version 1.7
>>
>
54 matches
Mail list logo