public interface TableOperations
Modifier and Type | Method and Description |
---|---|
int |
addConstraint(String tableName,
String constraintClassName)
Add a new constraint to a table.
|
void |
addSplits(String tableName,
SortedSet<org.apache.hadoop.io.Text> partitionKeys)
Ensures that tablets are split along a set of keys.
|
void |
attachIterator(String tableName,
IteratorSetting setting)
Add an iterator to a table on all scopes.
|
void |
attachIterator(String tableName,
IteratorSetting setting,
EnumSet<org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope> scopes)
Add an iterator to a table on the given scopes.
|
void |
cancelCompaction(String tableName)
Cancels a user initiated major compaction of a table initiated with
compact(String, Text, Text, boolean, boolean) or
compact(String, Text, Text, List, boolean, boolean) . |
void |
checkIteratorConflicts(String tableName,
IteratorSetting setting,
EnumSet<org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope> scopes)
Check whether a given iterator configuration conflicts with existing configuration; in particular, determine if the name or priority are already in use for
the specified scopes.
|
void |
clearLocatorCache(String tableName)
Clears the tablet locator cache for a specified table
|
void |
clone(String srcTableName,
String newTableName,
boolean flush,
Map<String,String> propertiesToSet,
Set<String> propertiesToExclude)
Clone a table from an existing table.
|
void |
compact(String tableName,
CompactionConfig config)
Starts a full major compaction of the tablets in the range (start, end].
|
void |
compact(String tableName,
org.apache.hadoop.io.Text start,
org.apache.hadoop.io.Text end,
boolean flush,
boolean wait)
Starts a full major compaction of the tablets in the range (start, end].
|
void |
compact(String tableName,
org.apache.hadoop.io.Text start,
org.apache.hadoop.io.Text end,
List<IteratorSetting> iterators,
boolean flush,
boolean wait)
Starts a full major compaction of the tablets in the range (start, end].
|
void |
create(String tableName)
Create a table with no special configuration
|
void |
create(String tableName,
boolean limitVersion)
Deprecated.
since 1.7.0; use
create(String, NewTableConfiguration) instead. |
void |
create(String tableName,
boolean versioningIter,
TimeType timeType)
Deprecated.
since 1.7.0; use
create(String, NewTableConfiguration) instead. |
void |
create(String tableName,
NewTableConfiguration ntc) |
void |
delete(String tableName)
Delete a table
|
void |
deleteRows(String tableName,
org.apache.hadoop.io.Text start,
org.apache.hadoop.io.Text end)
Delete rows between (start, end]
|
boolean |
exists(String tableName)
A method to check if a table exists in Accumulo.
|
void |
exportTable(String tableName,
String exportDir)
Exports a table.
|
void |
flush(String tableName)
Deprecated.
since 1.4; use
flush(String, Text, Text, boolean) instead |
void |
flush(String tableName,
org.apache.hadoop.io.Text start,
org.apache.hadoop.io.Text end,
boolean wait)
Flush a table's data that is currently in memory.
|
List<DiskUsage> |
getDiskUsage(Set<String> tables)
Gets the number of bytes being used in the files for a set of tables
|
IteratorSetting |
getIteratorSetting(String tableName,
String name,
org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope scope)
Get the settings for an iterator.
|
Map<String,Set<org.apache.hadoop.io.Text>> |
getLocalityGroups(String tableName)
Gets the locality groups currently set for a table.
|
org.apache.hadoop.io.Text |
getMaxRow(String tableName,
Authorizations auths,
org.apache.hadoop.io.Text startRow,
boolean startInclusive,
org.apache.hadoop.io.Text endRow,
boolean endInclusive)
Finds the max row within a given range.
|
Iterable<Map.Entry<String,String>> |
getProperties(String tableName)
Gets properties of a table.
|
Collection<org.apache.hadoop.io.Text> |
getSplits(String tableName)
Deprecated.
since 1.5.0; use
listSplits(String) instead. |
Collection<org.apache.hadoop.io.Text> |
getSplits(String tableName,
int maxSplits)
Deprecated.
since 1.5.0; use
listSplits(String, int) instead. |
void |
importDirectory(String tableName,
String dir,
String failureDir,
boolean setTime)
Bulk import all the files in a directory into a table.
|
void |
importTable(String tableName,
String importDir)
Imports a table exported via exportTable and copied via hadoop distcp.
|
SortedSet<String> |
list()
Retrieve a list of tables in Accumulo.
|
Map<String,Integer> |
listConstraints(String tableName)
List constraints on a table with their assigned numbers.
|
Map<String,EnumSet<org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope>> |
listIterators(String tableName)
Get a list of iterators for this table.
|
Collection<org.apache.hadoop.io.Text> |
listSplits(String tableName) |
Collection<org.apache.hadoop.io.Text> |
listSplits(String tableName,
int maxSplits) |
void |
merge(String tableName,
org.apache.hadoop.io.Text start,
org.apache.hadoop.io.Text end)
Merge tablets between (start, end]
|
void |
offline(String tableName)
Initiates taking a table offline, but does not wait for action to complete
|
void |
offline(String tableName,
boolean wait) |
void |
online(String tableName)
Initiates bringing a table online, but does not wait for action to complete
|
void |
online(String tableName,
boolean wait) |
void |
removeConstraint(String tableName,
int number)
Remove a constraint from a table.
|
void |
removeIterator(String tableName,
String name,
EnumSet<org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope> scopes)
Remove an iterator from a table by name.
|
void |
removeProperty(String tableName,
String property)
Removes a property from a table.
|
void |
rename(String oldTableName,
String newTableName)
Rename a table
|
void |
setLocalityGroups(String tableName,
Map<String,Set<org.apache.hadoop.io.Text>> groups)
Sets a table's locality groups.
|
void |
setProperty(String tableName,
String property,
String value)
Sets a property on a table.
|
Set<Range> |
splitRangeByTablets(String tableName,
Range range,
int maxSplits) |
Map<String,String> |
tableIdMap()
Get a mapping of table name to internal table id.
|
boolean |
testClassLoad(String tableName,
String className,
String asTypeName)
Test to see if the instance can load the given class as the given type.
|
SortedSet<String> list()
boolean exists(String tableName)
tableName
- the name of the tablevoid create(String tableName) throws AccumuloException, AccumuloSecurityException, TableExistsException
tableName
- the name of the tableAccumuloException
- if a general error occursAccumuloSecurityException
- if the user does not have permissionTableExistsException
- if the table already exists@Deprecated void create(String tableName, boolean limitVersion) throws AccumuloException, AccumuloSecurityException, TableExistsException
create(String, NewTableConfiguration)
instead.tableName
- the name of the tablelimitVersion
- Enables/disables the versioning iterator, which will limit the number of Key versions kept.AccumuloException
- if a general error occursAccumuloSecurityException
- if the user does not have permissionTableExistsException
- if the table already exists@Deprecated void create(String tableName, boolean versioningIter, TimeType timeType) throws AccumuloException, AccumuloSecurityException, TableExistsException
create(String, NewTableConfiguration)
instead.tableName
- the name of the tableversioningIter
- Enables/disables the versioning iterator, which will limit the number of Key versions kept.timeType
- specifies logical or real-time based time recording for entries in the tableAccumuloException
- if a general error occursAccumuloSecurityException
- if the user does not have permissionTableExistsException
- if the table already existsvoid create(String tableName, NewTableConfiguration ntc) throws AccumuloSecurityException, AccumuloException, TableExistsException
tableName
- the name of the tablentc
- specifies the new table's configuration variable, which are: 1. enable/disable the versioning iterator, which will limit the number of Key
versions kept; 2. specifies logical or real-time based time recording for entries in the table; 3. user defined properties to be merged into the
initial properties of the tableAccumuloException
- if a general error occursAccumuloSecurityException
- if the user does not have permissionTableExistsException
- if the table already existsvoid importTable(String tableName, String importDir) throws TableExistsException, AccumuloException, AccumuloSecurityException
tableName
- Name of a table to create and import into.importDir
- Directory that contains the files copied by distcp from exportTableTableExistsException
AccumuloException
AccumuloSecurityException
void exportTable(String tableName, String exportDir) throws TableNotFoundException, AccumuloException, AccumuloSecurityException
See docs/examples/README.export
tableName
- Name of the table to export.exportDir
- An empty directory in HDFS where files containing table metadata and list of files to distcp will be placed.TableNotFoundException
AccumuloException
AccumuloSecurityException
void addSplits(String tableName, SortedSet<org.apache.hadoop.io.Text> partitionKeys) throws TableNotFoundException, AccumuloException, AccumuloSecurityException
Note that while the documentation for Text specifies that its bytestream should be UTF-8, the encoding is not enforced by operations that work with byte arrays.
For example, you can create 256 evenly-sliced splits via the following code sample even though the given byte sequences are not valid UTF-8.
TableOperations tableOps = connector.tableOperations(); TreeSet<Text> splits = new TreeSet<Text>(); for (int i = 0; i < 256; i++) { byte[] bytes = {(byte) i}; splits.add(new Text(bytes)); } tableOps.addSplits(TABLE_NAME, splits);
tableName
- the name of the tablepartitionKeys
- a sorted set of row key values to pre-split the table onAccumuloException
- if a general error occursAccumuloSecurityException
- if the user does not have permissionTableNotFoundException
- if the table does not exist@Deprecated Collection<org.apache.hadoop.io.Text> getSplits(String tableName) throws TableNotFoundException
listSplits(String)
instead.tableName
- the name of the tableTableNotFoundException
- if the table does not existCollection<org.apache.hadoop.io.Text> listSplits(String tableName) throws TableNotFoundException, AccumuloSecurityException, AccumuloException
tableName
- the name of the tableTableNotFoundException
- if the table does not existAccumuloException
- if a general error occursAccumuloSecurityException
- if the user does not have permission@Deprecated Collection<org.apache.hadoop.io.Text> getSplits(String tableName, int maxSplits) throws TableNotFoundException
listSplits(String, int)
instead.tableName
- the name of the tablemaxSplits
- specifies the maximum number of splits to returnTableNotFoundException
Collection<org.apache.hadoop.io.Text> listSplits(String tableName, int maxSplits) throws TableNotFoundException, AccumuloSecurityException, AccumuloException
tableName
- the name of the tablemaxSplits
- specifies the maximum number of splits to returnAccumuloException
- if a general error occursAccumuloSecurityException
- if the user does not have permissionTableNotFoundException
org.apache.hadoop.io.Text getMaxRow(String tableName, Authorizations auths, org.apache.hadoop.io.Text startRow, boolean startInclusive, org.apache.hadoop.io.Text endRow, boolean endInclusive) throws TableNotFoundException, AccumuloException, AccumuloSecurityException
auths
- find the max row that can seen with these authsstartRow
- row to start looking at, null means -InfinitystartInclusive
- determines if the start row is includedendRow
- row to stop looking at, null means InfinityendInclusive
- determines if the end row is includedTableNotFoundException
AccumuloException
AccumuloSecurityException
void merge(String tableName, org.apache.hadoop.io.Text start, org.apache.hadoop.io.Text end) throws AccumuloException, AccumuloSecurityException, TableNotFoundException
tableName
- the table to mergestart
- first tablet to be merged contains the row after this row, null means the first tabletend
- last tablet to be merged contains this row, null means the last tabletAccumuloException
AccumuloSecurityException
TableNotFoundException
void deleteRows(String tableName, org.apache.hadoop.io.Text start, org.apache.hadoop.io.Text end) throws AccumuloException, AccumuloSecurityException, TableNotFoundException
tableName
- the table to mergestart
- delete rows after this, null means the first row of the tableend
- last row to be deleted, inclusive, null means the last row of the tableAccumuloException
AccumuloSecurityException
TableNotFoundException
void compact(String tableName, org.apache.hadoop.io.Text start, org.apache.hadoop.io.Text end, boolean flush, boolean wait) throws AccumuloSecurityException, TableNotFoundException, AccumuloException
tableName
- the table to compactstart
- first tablet to be compacted contains the row after this row, null means the first tablet in tableend
- last tablet to be compacted contains this row, null means the last tablet in tableflush
- when true, table memory is flushed before compaction startswait
- when true, the call will not return until compactions are finishedAccumuloSecurityException
TableNotFoundException
AccumuloException
void compact(String tableName, org.apache.hadoop.io.Text start, org.apache.hadoop.io.Text end, List<IteratorSetting> iterators, boolean flush, boolean wait) throws AccumuloSecurityException, TableNotFoundException, AccumuloException
tableName
- the table to compactstart
- first tablet to be compacted contains the row after this row, null means the first tablet in tableend
- last tablet to be compacted contains this row, null means the last tablet in tableiterators
- A set of iterators that will be applied to each tablet compacted. If two or more concurrent calls to compact pass iterators, then only one will
succeed and the others will fail.flush
- when true, table memory is flushed before compaction startswait
- when true, the call will not return until compactions are finishedAccumuloSecurityException
TableNotFoundException
AccumuloException
void compact(String tableName, CompactionConfig config) throws AccumuloSecurityException, TableNotFoundException, AccumuloException
Only one compact call at a time can pass iterators and/or a compaction strategy. If two threads call compaction with iterators and/or a copmaction strategy, then one will fail.
tableName
- the table to compactconfig
- the configuration to useAccumuloSecurityException
TableNotFoundException
AccumuloException
void cancelCompaction(String tableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException
compact(String, Text, Text, boolean, boolean)
or
compact(String, Text, Text, List, boolean, boolean)
. Compactions of tablets that are currently running may finish, but new compactions of tablets
will not start.tableName
- the name of the tableAccumuloException
- if a general error occursTableNotFoundException
- if the table does not existAccumuloSecurityException
- if the user does not have permissionvoid delete(String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException
tableName
- the name of the tableAccumuloException
- if a general error occursAccumuloSecurityException
- if the user does not have permissionTableNotFoundException
- if the table does not existvoid clone(String srcTableName, String newTableName, boolean flush, Map<String,String> propertiesToSet, Set<String> propertiesToExclude) throws AccumuloException, AccumuloSecurityException, TableNotFoundException, TableExistsException
srcTableName
- the table to clonenewTableName
- the name of the cloneflush
- determines if memory is flushed in the source table before cloning.propertiesToSet
- the sources tables properties are copied, this allows overriding of those propertiespropertiesToExclude
- do not copy these properties from the source table, just revert to system defaultsAccumuloException
AccumuloSecurityException
TableNotFoundException
TableExistsException
void rename(String oldTableName, String newTableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException, TableExistsException
oldTableName
- the old table namenewTableName
- the new table name, which must be in the same namespace as the oldTableNameAccumuloException
- if a general error occursAccumuloSecurityException
- if the user does not have permissionTableNotFoundException
- if the old table name does not existTableExistsException
- if the new table name already exists@Deprecated void flush(String tableName) throws AccumuloException, AccumuloSecurityException
flush(String, Text, Text, boolean)
insteadtableName
- the name of the tableAccumuloException
- if a general error occursAccumuloSecurityException
- if the user does not have permissionvoid flush(String tableName, org.apache.hadoop.io.Text start, org.apache.hadoop.io.Text end, boolean wait) throws AccumuloException, AccumuloSecurityException, TableNotFoundException
tableName
- the name of the tablewait
- if true the call will not return until all data present in memory when the call was is flushed if false will initiate a flush of data in memory,
but will not wait for it to completeAccumuloException
- if a general error occursAccumuloSecurityException
- if the user does not have permissionTableNotFoundException
void setProperty(String tableName, String property, String value) throws AccumuloException, AccumuloSecurityException
tableName
- the name of the tableproperty
- the name of a per-table propertyvalue
- the value to set a per-table property toAccumuloException
- if a general error occursAccumuloSecurityException
- if the user does not have permissionvoid removeProperty(String tableName, String property) throws AccumuloException, AccumuloSecurityException
tableName
- the name of the tableproperty
- the name of a per-table propertyAccumuloException
- if a general error occursAccumuloSecurityException
- if the user does not have permissionIterable<Map.Entry<String,String>> getProperties(String tableName) throws AccumuloException, TableNotFoundException
tableName
- the name of the tableTableNotFoundException
- if the table does not existAccumuloException
void setLocalityGroups(String tableName, Map<String,Set<org.apache.hadoop.io.Text>> groups) throws AccumuloException, AccumuloSecurityException, TableNotFoundException
tableName
- the name of the tablegroups
- mapping of locality group names to column families in the locality groupAccumuloException
- if a general error occursAccumuloSecurityException
- if the user does not have permissionTableNotFoundException
- if the table does not existMap<String,Set<org.apache.hadoop.io.Text>> getLocalityGroups(String tableName) throws AccumuloException, TableNotFoundException
tableName
- the name of the tableAccumuloException
- if a general error occursTableNotFoundException
- if the table does not existSet<Range> splitRangeByTablets(String tableName, Range range, int maxSplits) throws AccumuloException, AccumuloSecurityException, TableNotFoundException
tableName
- the name of the tablerange
- a range to splitmaxSplits
- the maximum number of splitsAccumuloException
- if a general error occursAccumuloSecurityException
- if the user does not have permissionTableNotFoundException
- if the table does not existvoid importDirectory(String tableName, String dir, String failureDir, boolean setTime) throws TableNotFoundException, IOException, AccumuloException, AccumuloSecurityException
tableName
- the name of the tabledir
- the HDFS directory to find files for importingfailureDir
- the HDFS directory to place files that failed to be imported, must exist and be emptysetTime
- override the time values in the input files, and use the current time for all mutationsIOException
- when there is an error reading/writing to HDFSAccumuloException
- when there is a general accumulo errorAccumuloSecurityException
- when the user does not have the proper permissionsTableNotFoundException
- when the table no longer existsvoid offline(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException
tableName
- the table to take offlineAccumuloException
- when there is a general accumulo errorAccumuloSecurityException
- when the user does not have the proper permissionsTableNotFoundException
void offline(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException
tableName
- the table to take offlinewait
- if true, then will not return until table is offlineAccumuloException
- when there is a general accumulo errorAccumuloSecurityException
- when the user does not have the proper permissionsTableNotFoundException
void online(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException
tableName
- the table to take onlineAccumuloException
- when there is a general accumulo errorAccumuloSecurityException
- when the user does not have the proper permissionsTableNotFoundException
void online(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException
tableName
- the table to take onlinewait
- if true, then will not return until table is onlineAccumuloException
- when there is a general accumulo errorAccumuloSecurityException
- when the user does not have the proper permissionsTableNotFoundException
void clearLocatorCache(String tableName) throws TableNotFoundException
tableName
- the name of the tableTableNotFoundException
- if table does not existMap<String,String> tableIdMap()
void attachIterator(String tableName, IteratorSetting setting) throws AccumuloSecurityException, AccumuloException, TableNotFoundException
tableName
- the name of the tablesetting
- object specifying the properties of the iteratorAccumuloSecurityException
- thrown if the user does not have the ability to set properties on the tableTableNotFoundException
- throw if the table no longer existsIllegalArgumentException
- if the setting conflicts with any existing iteratorsAccumuloException
void attachIterator(String tableName, IteratorSetting setting, EnumSet<org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException, TableNotFoundException
tableName
- the name of the tablesetting
- object specifying the properties of the iteratorAccumuloSecurityException
- thrown if the user does not have the ability to set properties on the tableTableNotFoundException
- throw if the table no longer existsIllegalArgumentException
- if the setting conflicts with any existing iteratorsAccumuloException
void removeIterator(String tableName, String name, EnumSet<org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException, TableNotFoundException
tableName
- the name of the tablename
- the name of the iteratorscopes
- the scopes of the iteratorAccumuloSecurityException
- thrown if the user does not have the ability to set properties on the tableTableNotFoundException
- throw if the table no longer existsAccumuloException
IteratorSetting getIteratorSetting(String tableName, String name, org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope scope) throws AccumuloSecurityException, AccumuloException, TableNotFoundException
tableName
- the name of the tablename
- the name of the iteratorscope
- the scope of the iteratorAccumuloSecurityException
- thrown if the user does not have the ability to set properties on the tableTableNotFoundException
- throw if the table no longer existsAccumuloException
Map<String,EnumSet<org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope>> listIterators(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException
tableName
- the name of the tableAccumuloSecurityException
AccumuloException
TableNotFoundException
void checkIteratorConflicts(String tableName, IteratorSetting setting, EnumSet<org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope> scopes) throws AccumuloException, TableNotFoundException
tableName
- the name of the tablesetting
- object specifying the properties of the iteratorAccumuloException
TableNotFoundException
int addConstraint(String tableName, String constraintClassName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException
tableName
- the name of the tableconstraintClassName
- the full name of the constraint classAccumuloException
- thrown if the constraint has already been added to the table or if there are errors in the configuration of existing constraintsAccumuloSecurityException
- thrown if the user doesn't have permission to add the constraintTableNotFoundException
void removeConstraint(String tableName, int number) throws AccumuloException, AccumuloSecurityException
tableName
- the name of the tablenumber
- the unique number assigned to the constraintAccumuloSecurityException
- thrown if the user doesn't have permission to remove the constraintAccumuloException
Map<String,Integer> listConstraints(String tableName) throws AccumuloException, TableNotFoundException
tableName
- the name of the tableAccumuloException
- thrown if there are errors in the configuration of existing constraintsTableNotFoundException
List<DiskUsage> getDiskUsage(Set<String> tables) throws AccumuloException, AccumuloSecurityException, TableNotFoundException
tables
- a set of tablesAccumuloException
AccumuloSecurityException
TableNotFoundException
boolean testClassLoad(String tableName, String className, String asTypeName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException
AccumuloException
AccumuloSecurityException
TableNotFoundException
Copyright © 2011–2018 The Apache Software Foundation. All rights reserved.