Welcome to Sharkbite’s documentation!

https://camo.githubusercontent.com/dbf39cef1a973d8741437693e96b59e31d9e3754/68747470733a2f2f7777772e736861726b626974652e696f2f77702d636f6e74656e742f75706c6f6164732f323031372f30322f736861726b626974652e6a7067
https://camo.githubusercontent.com/dbf39cef1a973d8741437693e96b59e31d9e3754/68747470733a2f2f7777772e736861726b626974652e696f2f77702d636f6e74656e742f75706c6f6164732f323031372f30322f736861726b626974652e6a7067

About Sharkbite

Sharkbite is an HDFS and native client for key/value stores. With initial support for Apache Accumulo Accumulo, the design can and has been used to support other key/value stores. Development began in 2014 and has slowly evolved. there is no structural specificity to Accumulo despite being the base implementation. Despite this the examples below will look very much like Accumulo due to aliasing. This is intentional.

Capabilities That will be supported in V1.0 :

  • Works with Accumulo 1.6.x, 1.7.x, 1.8.x, 1.9.x and 2.x
  • Read/Write : Reading and writing data to Accumulo is currently supported.
  • Table Operations : Most table operations are currently supported. This includes the fate operations that the normal Accumulo client performs.
  • Security Operations : Most security operations are implemented. Please let us know of any gaps.

About the name

Sharkbite’s name originated from design as a connector that abstracted components in which we tightly coupled and gripped interfaces of the underlying datastore. With an abstraction layer for access, and using cross compatible objects, the underlying interfaces are heavily coupled to each database. As a result, Sharkbite became a fitting name since interfaces exist to abstract the high coupling that exists within implementations of the API.

Installing

This python client can be installed via pip install sharkbite

A Python example is included. This is your primary example of the Python bound sharkbite library.

Sharkbite supports async iteration A simple example is provided.

Features

HDFS Client

Sharkbite supports a limited HDFS client. As this functionality grows so will the capabilities. Version 0.7 will support a complete HDFS client. Since Sharkbite it built as a python bindings around a C++ Client, the python client will mature slightly behind the C++ client, hence the delta with building this into V 0.7

Version Detection

Sharkbite detects the version of Apache Accumulo. Therefore you will be able to simply create a connector to the zookeeper instance.

Hedged Reads

Sharkbite supports hedged reads ( executing scans against RFiles when they can be accessed ) concurrently with Accumulo RPC scans. The first executor to complete will return your results. This feature is in beta and not suggested for production environments.

Enable it with the following option:

    import pysharkbite as sharkbite

    connector = sharkbite.AccumuloConnector(user, zk)

table_operations = connector.tableOps(table)

    scanner = table_operations.createScanner(auths, 2)

range = sharkbite.Range("myrow")

scanner.addRange( range )

### enable the beta option of hedged reads

scanner.setOption( sharkbite.ScannerOptions.HedgedReads )

resultset = scanner.getResultSet()

for keyvalue in resultset:
    key = keyvalue.getKey()
    value = keyvalue.getValue()

Python Iterators

We now support a beta version of python iterators. By using the cmake option PYTHON_ITERATOR_SUPPORT ( cmake -DPYTHON_ITERATOR_SUPPORT=ON ) we will build the necessary infrastructure to support python iterators

Iterators can be defined as single function lambdas or by implementing the seek or next methods.

The first example implements the seek and onNext methods. seek is optional if you don’t wish to adjust the range. Once keys are being iterated you may get the top key. You may call iterator.next() after or the infrastructure will do that for you.

class myIterator:
  def seek(iterator,soughtRange):
    range = Range("a")
    iterator.seek(range)


  def onNext(iterator):
    if (iterator.hasTop()):
        kv = KeyValue()
          key = iterator.getTopKey()
          cf = key.getColumnFamily()
          value = iterator.getTopValue()
          key.setColumnFamily("oh changed " + cf)
          iterator.next()
          return KeyValue(key,value)
    else:
      return None

If this is defined in a separate file, you may use it with the following code snippet

with open('test.iter', 'r') as file:
 iterator = file.read()
## name, iterator text, priority
 iterator = pysharkbite.PythonIterator("PythonIterator",iteratortext,100)
 scanner.addIterator(iterator)

Alternative you may use lambdas. The lambda you provide will be passed the KeyValue ( getKey() and getValue() return the constituent parts). A partial code example of setting it up is below. You may return a Key or KeyValue object. If you return the former an empty value will be return ed.

## define only the name and priority
iterator = pysharkbite.PythonIterator("PythonIterator",100)
## define a lambda to ajust the column family.
iterator = iterator.onNext("lambda x : Key( x.getKey().getRow(), 'new cf', x.getKey().getColumnQualifier()) ")

scanner.addIterator(iterator)

You may either define a python iterator as a text implementation or a lambda. Both cannot be used simulaneously.

[accumulo]: https://accumulo.apache.org

https://camo.githubusercontent.com/dbf39cef1a973d8741437693e96b59e31d9e3754/68747470733a2f2f7777772e736861726b626974652e696f2f77702d636f6e74656e742f75706c6f6164732f323031372f30322f736861726b626974652e6a7067

Async Example

Below is an example client based on pysharkbite. We’ll step through important aspects followed by the entirety of the code at the end.

The first interesting piece of code that we come across is printasync, which is intended to asynchronously print the rows from all keys. This coroutine will be used later to loop through the asynchronous iterator.

async def printasync(iter):
    async for keyvalue in iter:
        key = keyvalue.getKey()
        print(key.getRow())

After writing data the example creates a scanner. This scanner creates a range from ‘row’ to ‘row3’ then creates an async event loop to call the coroutine printasync. As stated above this enables us to asynchronously print the rows.

scanner = table_operations.createScanner(auths, 2)

range = pysharkbite.Range("row"",True,"row3"",False)

scanner.addRange( range )

resultset = scanner.getResultSet()

loop = asyncio.get_event_loop()
loop.run_until_complete(printasync(resultset))
#!/usr/bin/python
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements.  See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership.  The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License.  You may obtain a copy of the License at
#
#   http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied.  See the License for the
# specific language governing permissions and limitations
# under the License.
from ctypes import cdll
from argparse import ArgumentParser
from ctypes import cdll
import ctypes
import traceback
import time
import asyncio


"""

This is an Example of using the Python connectors. The example will accept user input
create a table writing arbitrary information to it via the BatchWriter and scanner will put the written data


"""

parser = ArgumentParser(description="This is an Apache Accummulo Python connector")

parser.add_argument("-i", "--instance", dest="instance",
                    help="Apache Accumulo Instance Name", required=True)
parser.add_argument("-z", "--zookeepers", dest="zookeepers",
                    help="Comma Separated Zookeeper List", required=True)
parser.add_argument("-u", "--username", dest="username",
                    help="User to access Apache Accumulo", required=True)
parser.add_argument("-p", "--password", dest="password",
                    help="Password to access Apache Accumulo. May also be supplied at the command line")
parser.add_argument("-t", "--table", dest="table",
                    help="Table to create/update")
args = parser.parse_args()

password = args.password
table = args.table

async def printasync(iter):
    async for keyvalue in iter:
        key = keyvalue.getKey()
        print(key.getRow())

if not password:
    print("Please enter your password")
    password = input()

if not table:
    table = "blahblahd"

import pysharkbite

configuration = pysharkbite.Configuration()

zk = pysharkbite.ZookeeperInstance(args.instance, args.zookeepers, 1000, configuration)

user = pysharkbite.AuthInfo(args.username, password, zk.getInstanceId())

try:
    connector = pysharkbite.AccumuloConnector(user, zk)


    table_operations = connector.tableOps(table)

    if not table_operations.exists(False):
        print ("Creating table " + table)
        table_operations.create(False)
    else:
        print (table + " already exists, so not creating it")


    auths = pysharkbite.Authorizations()

    """ Add authorizations """
    """ mutation.put("cf","cq","cv",1569786960) """

    writer = table_operations.createWriter(auths, 10)

    mutation = pysharkbite.Mutation("row2");

    mutation.put("cf","cq","",1569786960, "value")
    mutation.put("cf2","cq2","",1569786960, "value2")
    """ no value """
    mutation.put("cf3","cq3","",1569786960, "")

    writer.addMutation( mutation )

    writer.close()

    time.sleep(2)

    """ auths.addAuthorization("cv") """

    scanner = table_operations.createScanner(auths, 2)

    range = pysharkbite.Range("row"",True,"row3"",False)

    scanner.addRange( range )

    resultset = scanner.getResultSet()

    loop = asyncio.get_event_loop()
    loop.run_until_complete(printasync(resultset))


    """ delete your table if user did not create temp """
    if not args.table:
        table_operations.remove()

except RuntimeError as e:
    traceback.print_exc()
    print("Oops, error caused: " + str(e))
https://camo.githubusercontent.com/dbf39cef1a973d8741437693e96b59e31d9e3754/68747470733a2f2f7777772e736861726b626974652e696f2f77702d636f6e74656e742f75706c6f6164732f323031372f30322f736861726b626974652e6a7067

HDFS Client

The hdfs client is nearly full client. It lacks features found in the C++ client. These will be added over time. Please visit the link, above, to find the API of what is currently supported

An example usage of these functions is below. Note that if Opening RFiles in pysharkbite, you must specify the full path including the hdfs protocol if it is located on HDFS. This will open a full HDFS client to access these files.

import pysharkbite

hdfs = pysharkbite.Hdfs("hdfs://namenode:8020",8020);

hdfs.mkdir("/directoryA/directoryB");

hdfs.list("/");
https://camo.githubusercontent.com/dbf39cef1a973d8741437693e96b59e31d9e3754/68747470733a2f2f7777772e736861726b626974652e696f2f77702d636f6e74656e742f75706c6f6164732f323031372f30322f736861726b626974652e6a7067

Getting Statistics in Python

You can retreive Accumulo stats with the get_statistics function in the connector.

from ctypes import cdll
from argparse import ArgumentParser
from ctypes import cdll
import ctypes
import traceback
import json
import time



"""

This is an Example of using the Python connectors. The example will accept user input
create a table writing arbitrary information to it via the BatchWriter and scanner will put the written data


"""

parser = ArgumentParser(description="This is an Apache Accummulo Python connector")

parser.add_argument("-i", "--instance", dest="instance",
                    help="Apache Accumulo Instance Name", required=True)
parser.add_argument("-z", "--zookeepers", dest="zookeepers",
                    help="Comma Separated Zookeeper List", required=True)
parser.add_argument("-u", "--username", dest="username",
                    help="User to access Apache Accumulo", required=True)
parser.add_argument("-p", "--password", dest="password",
                    help="Password to access Apache Accumulo. May also be supplied at the command line")
parser.add_argument("-t", "--table", dest="table",
                    help="Table to create/update")
args = parser.parse_args()

password = args.password
table = args.table

if not password:
    print("Please enter your password")
    password = input()

if not table:
    table = "blahblahd"

import pysharkbite

configuration = pysharkbite.Configuration()

zk = pysharkbite.ZookeeperInstance(args.instance, args.zookeepers, 1000, configuration)

user = pysharkbite.AuthInfo(args.username, password, zk.getInstanceId())

try:
    connector = pysharkbite.AccumuloConnector(user, zk)

    stats = connector.getStatistics()

    print("Goal state: " + str(stats.goal_state))

    # print some server info
    for serverinfo in stats.tablet_server_info:
            print(serverinfo.name)
            print(serverinfo.last_contact)




except RuntimeError as e:
    traceback.print_exc()
    print("Oops, error caused: " + str(e))

Sharkbite Documentation

Accumulo connector plugin

class pysharkbite.AccumuloConnector

Accumulo connector

getStatistics(self: pysharkbite.AccumuloConnector)pysharkbite.AccumuloInfo

Returns Statistics for the accumulo connector

namespaceOps(self: pysharkbite.AccumuloConnector, arg0: str)pysharkbite.AccumuloNamespaceOperations

Allows the user to perform namespace operations

securityOps(self: pysharkbite.AccumuloConnector) → interconnect::PythonSecurityOperations

Return the security operations object

tableOps(self: pysharkbite.AccumuloConnector, arg0: str)pysharkbite.AccumuloTableOperations

Return the table operations object

class pysharkbite.AccumuloInfo
property bad_servers
property dead_servers
getBadTabletServers(self: pysharkbite.AccumuloInfo) → Dict[str, int]

Gets a mapping of bad tablet servers.

getDeadServers(self: pysharkbite.AccumuloInfo) → List[pysharkbite.DeadServer]

Returns a list of dead tablet servers.

getGoalState(self: pysharkbite.AccumuloInfo)pysharkbite.CoordinatorGoalState

Returns the goal state of the cluster.

getServerShuttingDown(self: pysharkbite.AccumuloInfo) → Set[str]

Returns the set of servers shutting down.

getState(self: pysharkbite.AccumuloInfo)pysharkbite.CoordinatorState

Gets the state of the accumulo cluster.

getTableMap(self: pysharkbite.AccumuloInfo) → Dict[str, pysharkbite.TableInfo]

Gets the Table map for the cluster.

getTabletServerInfo(self: pysharkbite.AccumuloInfo) → List[pysharkbite.TabletServerStatus]

Gets tablet server Info

getUnassignedTablets(self: pysharkbite.AccumuloInfo) → int

Gets the unassigned tablets

property goal_state
property servs_shutting_down
property state
property table_map
property tablet_server_info
property unassigned_tablets
class pysharkbite.AccumuloNamespaceOperations

Accumulo namespace operations. Should be accessed through ‘AccumuloConnector’

create(self: pysharkbite.AccumuloNamespaceOperations, arg0: str) → None

Creates the namespace

exists(self: pysharkbite.AccumuloNamespaceOperations, arg0: str) → bool

Returns true if the namespace exists

list(self: pysharkbite.AccumuloNamespaceOperations) → List[str]

Lists namespaces within this Accumulo instance

remove(self: pysharkbite.AccumuloNamespaceOperations, arg0: str) → bool

removes the namespace

removeProperty(self: pysharkbite.AccumuloNamespaceOperations, arg0: str, arg1: str) → None

Remove the namespace property

rename(self: pysharkbite.AccumuloNamespaceOperations, arg0: str, arg1: str) → None

Renames the namespace

setProperty(self: pysharkbite.AccumuloNamespaceOperations, arg0: str, arg1: str, arg2: str) → None

Sets a namespace property

class pysharkbite.AccumuloTableOperations

Accumulo table operations. Should be accessed through ‘AccumuloConnector’

addConstraint(self: pysharkbite.AccumuloTableOperations, arg0: str) → int

Add table constraint

addSplits(self: pysharkbite.AccumuloTableOperations, arg0: Set[str]) → None

Add splits for a table

compact(self: pysharkbite.AccumuloTableOperations, arg0: str, arg1: str, arg2: bool) → int

compact the table

create(self: pysharkbite.AccumuloTableOperations, arg0: bool) → bool

Create the table

createScanner(self: pysharkbite.AccumuloTableOperations, arg0: cclient::data::security::Authorizations, arg1: int) → scanners::Source<cclient::data::KeyValue, scanners::ResultBlock<cclient::data::KeyValue> >

createWriter scanner

createWriter(self: pysharkbite.AccumuloTableOperations, arg0: cclient::data::security::Authorizations, arg1: int) → writer::Sink<cclient::data::KeyValue>

Create writer for table

exists(self: pysharkbite.AccumuloTableOperations, arg0: bool) → bool

Returns true if the table exists

flush(self: pysharkbite.AccumuloTableOperations, arg0: str, arg1: str, arg2: bool) → int

flush the table

import(self: pysharkbite.AccumuloTableOperations, arg0: str, arg1: str, arg2: bool) → bool

import data into this directory

remove(self: pysharkbite.AccumuloTableOperations) → bool

remove the table

removeProperty(self: pysharkbite.AccumuloTableOperations, arg0: str) → int

Remove the table property

setProperty(self: pysharkbite.AccumuloTableOperations, arg0: str, arg1: str) → int

Set table property

class pysharkbite.AuthInfo

Auth info object contains accessor to an Accumulo Instance

getInstanceId(self: pysharkbite.AuthInfo) → str

Get the instance ID

getPassword(self: pysharkbite.AuthInfo) → str

Get the user’s password

getUserName(self: pysharkbite.AuthInfo) → str

Get the username

class pysharkbite.Authorizations

Authorizations object

addAuthorization(self: pysharkbite.Authorizations, arg0: str) → None

Add an authorization to be used for table operations

contains(self: pysharkbite.Authorizations, arg0: str) → bool

Returns true if the set contains an authorization

get_authorizations(self: pysharkbite.Authorizations) → List[str]

Returns an iterable of authorizations

class pysharkbite.BatchScanner

Batch Scanner to be constructed via TableOperations

addIterator(*args, **kwargs)

Overloaded function.

  1. addIterator(self: pysharkbite.BatchScanner, arg0: pysharkbite.IterInfo) -> None

Add an iterator to be run server side

  1. addIterator(self: pysharkbite.BatchScanner, arg0: pysharkbite.PythonIterator) -> None

Adds a python iterator to be run server side

addRange(self: pysharkbite.BatchScanner, arg0: pysharkbite.Range) → None

Adds a range

close(self: pysharkbite.BatchScanner) → None

closes the scanner

fetchColumn(self: pysharkbite.BatchScanner, arg0: str, arg1: str) → None

Fetches the column

getResultSet(self: pysharkbite.BatchScanner)pysharkbite.Results

Gets a result set that can be used asynchronously

removeOption(self: pysharkbite.BatchScanner, arg0: pysharkbite.ScannerOptions) → None

Removes an option

setOption(self: pysharkbite.BatchScanner, arg0: pysharkbite.ScannerOptions) → None

Sets the option

withRange(self: pysharkbite.BatchScanner, arg0: pysharkbite.Range)pysharkbite.BatchScanner
class pysharkbite.BatchWriter

Batch writer to be constructed, from TableOperations

addMutation(self: pysharkbite.BatchWriter, arg0: pysharkbite.Mutation) → bool

Adds a mutation to the batch writer

close(self: pysharkbite.BatchWriter) → None

Closes the batch writer

flush(self: pysharkbite.BatchWriter, arg0: bool) → None

Flushes the batch writer. Will be called automatically by close.

size(self: pysharkbite.BatchWriter) → int

Returns the current size to be written to Accumulo

exception pysharkbite.ClientException
class pysharkbite.Compacting
getQueued(self: pysharkbite.Compacting) → int

Gets queued compactions.

getRunning(self: pysharkbite.Compacting) → int

Gets running compactions.

property queued
property running
class pysharkbite.Configuration

Sharkbite Configuration object.

get(*args, **kwargs)

Overloaded function.

  1. get(self: pysharkbite.Configuration, arg0: str) -> str

Get the configuration option

  1. get(self: pysharkbite.Configuration, arg0: str, arg1: str) -> str

Get the configuration option with default value

getLong(self: pysharkbite.Configuration, arg0: str, arg1: int) → int

Get the integer value of a configuration item

set(self: pysharkbite.Configuration, arg0: str, arg1: str) → None

Set a configuration option

class pysharkbite.CoordinatorGoalState

Members:

CLEAN_STOP : CLEAN_STOP state

SAFE_MODE : SAFE_MODE state

NORMAL : NORMAL state

CLEAN_STOP = CoordinatorGoalState.CLEAN_STOP
NORMAL = CoordinatorGoalState.NORMAL
SAFE_MODE = CoordinatorGoalState.SAFE_MODE
__entries = {'CLEAN_STOP': (CoordinatorGoalState.CLEAN_STOP, 'CLEAN_STOP state'), 'NORMAL': (CoordinatorGoalState.NORMAL, 'NORMAL state'), 'SAFE_MODE': (CoordinatorGoalState.SAFE_MODE, 'SAFE_MODE state')}
property name

(self: handle) -> str

class pysharkbite.CoordinatorState

Members:

INITIAL : INITIAL state

HAVE_LOCK : HAVE_LOCK state

SAFE_MODE : SAFE_MODE state

NORMAL : NORMAL state

UNLOAD_METADATA_TABLETS : UNLOAD_METADATA_TABLETS state

UNLOAD_ROOT_TABLET : UNLOAD_ROOT_TABLET state

STOP : STOP state

HAVE_LOCK = CoordinatorState.HAVE_LOCK
INITIAL = CoordinatorState.INITIAL
NORMAL = CoordinatorState.NORMAL
SAFE_MODE = CoordinatorState.SAFE_MODE
STOP = CoordinatorState.STOP
UNLOAD_METADATA_TABLETS = CoordinatorState.UNLOAD_METADATA_TABLETS
UNLOAD_ROOT_TABLET = CoordinatorState.UNLOAD_ROOT_TABLET
__entries = {'HAVE_LOCK': (CoordinatorState.HAVE_LOCK, 'HAVE_LOCK state'), 'INITIAL': (CoordinatorState.INITIAL, 'INITIAL state'), 'NORMAL': (CoordinatorState.NORMAL, 'NORMAL state'), 'SAFE_MODE': (CoordinatorState.SAFE_MODE, 'SAFE_MODE state'), 'STOP': (CoordinatorState.STOP, 'STOP state'), 'UNLOAD_METADATA_TABLETS': (CoordinatorState.UNLOAD_METADATA_TABLETS, 'UNLOAD_METADATA_TABLETS state'), 'UNLOAD_ROOT_TABLET': (CoordinatorState.UNLOAD_ROOT_TABLET, 'UNLOAD_ROOT_TABLET state')}
property name

(self: handle) -> str

class pysharkbite.DeadServer
getLastContact(self: pysharkbite.DeadServer) → int

Get last contact time with this server.

getServer(self: pysharkbite.DeadServer) → str

Gets the server name.

getStatus(self: pysharkbite.DeadServer) → str

Gets the status of the server.

property last_contact
property server
property status
class pysharkbite.Hdfs

HDFS refernce object

chown(self: pysharkbite.Hdfs, arg0: str, arg1: str, arg2: str) → int

Chowns the provided path

list(self: pysharkbite.Hdfs, arg0: str) → List[pysharkbite.HdfsDirEnt]

Lists HDFS directory, returns a list of HdfsDirEnt objects

mkdir(self: pysharkbite.Hdfs, arg0: str) → int

Creates a directory on HDFS. Should be a relative path

move(self: pysharkbite.Hdfs, arg0: str, arg1: str) → int

Moves a file within a NN instance.

read(self: pysharkbite.Hdfs, arg0: str) → cclient::data::streams::InputStream

Opens a read stream to an HDFS file, creating or updating it

remove(self: pysharkbite.Hdfs, arg0: str, arg1: bool) → int

Removes a file or directory, the boolean flag, if set to true, deletes recusively

rename(self: pysharkbite.Hdfs, arg0: str, arg1: str) → int

Renames a path

write(self: pysharkbite.Hdfs, arg0: str) → cclient::data::streams::ByteOutputStream

Opens a write stream to an HDFS file, creating or updating it

class pysharkbite.HdfsDirEnt

HDFS directory entry object.

getGroup(self: pysharkbite.HdfsDirEnt) → str

Gets the group of the directory entry

getName(self: pysharkbite.HdfsDirEnt) → str

Gets the name of the directory entry

getOwner(self: pysharkbite.HdfsDirEnt) → str

Gets the owner of the directory entry

getSize(self: pysharkbite.HdfsDirEnt) → int

Returns the size of the directory entry

class pysharkbite.HdfsInputStream

HDFS Input Stream

readBytes(self: pysharkbite.HdfsInputStream, arg0: str, arg1: int) → int

Reads a character sequence from the file on HDFS

readInt(self: pysharkbite.HdfsInputStream) → int

Reads a four byte integer

readLong(self: pysharkbite.HdfsInputStream) → int

Reads an eight byte integer

readShort(self: pysharkbite.HdfsInputStream) → int

Reads a two byte integer

readString(self: pysharkbite.HdfsInputStream) → str

Reads a string

class pysharkbite.HdfsOutputStream

HDFS Output Stream

write(self: pysharkbite.HdfsOutputStream, arg0: str, arg1: int) → int

Writes an eight byte integer

writeInt(self: pysharkbite.HdfsOutputStream, arg0: int) → int

Writes a four byte integer

writeLong(self: pysharkbite.HdfsOutputStream, arg0: int) → int

Writes an eight byte integer

writeShort(self: pysharkbite.HdfsOutputStream, arg0: int) → int

Writes a two byte integer

writeString(self: pysharkbite.HdfsOutputStream, arg0: str) → int

Writes a string

class pysharkbite.Instance

Accumulo Instance

class pysharkbite.IterInfo

IterInfo is an iterator configuration

getClass(self: pysharkbite.IterInfo) → str

Get the class for this iterator

getName(self: pysharkbite.IterInfo) → str

Get the name of this iterator

getPriority(self: pysharkbite.IterInfo) → int

Get the priority for this iterator

class pysharkbite.Key

Accumulo Key

getColumnFamily(self: pysharkbite.Key) → str

Gets the Column Family

getColumnQualifier(self: pysharkbite.Key) → str

Gets the Column Qualifier

getColumnVisibility(self: pysharkbite.Key) → str

Gets the Column Visibility

getRow(self: pysharkbite.Key) → str

Gets the row

getTimestamp(self: pysharkbite.Key) → int

Gets the key timestamp

setColumnFamily(self: pysharkbite.Key, arg0: str) → None

Sets the column fmaily

setColumnQualifier(self: pysharkbite.Key, arg0: str) → None

Sets the column qualifier

setRow(self: pysharkbite.Key, arg0: str) → None

Sets the row

class pysharkbite.KeyValue
getKey(self: pysharkbite.KeyValue)pysharkbite.Key

Gets the Key from the key value object

getValue(self: pysharkbite.KeyValue)pysharkbite.Value

Gets the Value from the key value object

class pysharkbite.KeyValueIterator

Iterable object within Accumulo

getTopKey(self: pysharkbite.KeyValueIterator)pysharkbite.Key

Returns the top key

getTopValue(self: pysharkbite.KeyValueIterator)pysharkbite.Value

Returns the top value

hasNext(self: pysharkbite.KeyValueIterator) → bool

Returns true of further keys exist, false otherwise

next(self: pysharkbite.KeyValueIterator) → None

Queues the next key/value to be returned via getTopKey and getTopValue

seek(self: pysharkbite.KeyValueIterator, arg0: cclient::data::streams::StreamRelocation) → None

Seeks to a key within the iterator

class pysharkbite.LoggingConfiguration

Sharkbite Logger, enables logging at the debug and trace levels

static enableDebugLogger() → None

Enables the debug logging for all classes

static enableTraceLogger() → None

Enables trace logging for all classes

class pysharkbite.Mutation
put(*args, **kwargs)

Overloaded function.

  1. put(self: pysharkbite.Mutation, arg0: str, arg1: str, arg2: str, arg3: int, arg4: str) -> None

Adds a mutation

  1. put(self: pysharkbite.Mutation, arg0: str, arg1: str, arg2: str, arg3: int) -> None

Adds a mutation

  1. put(self: pysharkbite.Mutation, arg0: str, arg1: str, arg2: str) -> None

Adds a mutation

  1. put(self: pysharkbite.Mutation, arg0: str, arg1: str) -> None

Adds a mutation

putDelete(*args, **kwargs)

Overloaded function.

  1. putDelete(self: pysharkbite.Mutation, arg0: str, arg1: str, arg2: str, arg3: int) -> None

Adds a delete mutation

  1. putDelete(self: pysharkbite.Mutation, arg0: str, arg1: str, arg2: str) -> None

Adds a delete mutation

class pysharkbite.NamespacePermissions

Members:

READ : Enables read permission on the namespace

WRITE : Enables write permission on the namespace

ALTER_NAMESPACE : Enables the alter namespace permissions for the user

GRANT : Enables grant permission on the namespace

ALTER_TABLE : Enables alter table permissions on the namespace

CREATE_TABLE : Enables create table permissions on the namespace

DROP_TABLE : Enables drop table permissions on the namespace

BULK_IMPORT : Enables bulk import permissions on the namespace

DROP_NAMESPACE : Enables drop namespace permissions on the namespace

ALTER_NAMESPACE = NamespacePermissions.ALTER_NAMESPACE
ALTER_TABLE = NamespacePermissions.ALTER_TABLE
BULK_IMPORT = NamespacePermissions.BULK_IMPORT
CREATE_TABLE = NamespacePermissions.CREATE_TABLE
DROP_NAMESPACE = NamespacePermissions.DROP_NAMESPACE
DROP_TABLE = NamespacePermissions.DROP_TABLE
GRANT = NamespacePermissions.GRANT
READ = NamespacePermissions.READ
WRITE = NamespacePermissions.WRITE
__entries = {'ALTER_NAMESPACE': (NamespacePermissions.ALTER_NAMESPACE, 'Enables the alter namespace permissions for the user'), 'ALTER_TABLE': (NamespacePermissions.ALTER_TABLE, 'Enables alter table permissions on the namespace'), 'BULK_IMPORT': (NamespacePermissions.BULK_IMPORT, 'Enables bulk import permissions on the namespace'), 'CREATE_TABLE': (NamespacePermissions.CREATE_TABLE, 'Enables create table permissions on the namespace'), 'DROP_NAMESPACE': (NamespacePermissions.DROP_NAMESPACE, 'Enables drop namespace permissions on the namespace'), 'DROP_TABLE': (NamespacePermissions.DROP_TABLE, 'Enables drop table permissions on the namespace'), 'GRANT': (NamespacePermissions.GRANT, 'Enables grant permission on the namespace'), 'READ': (NamespacePermissions.READ, 'Enables read permission on the namespace'), 'WRITE': (NamespacePermissions.WRITE, 'Enables write permission on the namespace')}
property name

(self: handle) -> str

class pysharkbite.PythonIterator

Defines a python iterator

getClass(self: pysharkbite.PythonIterator) → str

Get the class for this iterator

getName(self: pysharkbite.PythonIterator) → str

Get the name of this iterator

getPriority(self: pysharkbite.PythonIterator) → int

Get the priority for this iterator

onNext(self: pysharkbite.PythonIterator, arg0: str)pysharkbite.PythonIterator

Lambda that is provided the accumulo key

class pysharkbite.RFile

RFile, which is an internal data structure for storing data within Accumulo

close(self: pysharkbite.RFile) → None

Closes the RFile

getTop(self: pysharkbite.RFile)pysharkbite.KeyValue

Returns the top key/value

hasNext(self: pysharkbite.RFile) → bool

Returns true of further keys exist, false otherwise

next(self: pysharkbite.RFile) → None

Queues the next key/value pair

seek(self: pysharkbite.RFile, arg0: cclient::data::streams::StreamRelocation) → None

Seek to data

class pysharkbite.RFileOperations

RFile Operations

openForWrite(self: str)pysharkbite.SequentialRFile

Opens an RFile to write

openManySequential(self: List[str], arg0: int)pysharkbite.KeyValueIterator

Opens many RFiles sequentially.

randomSeek(self: str)pysharkbite.RFile

Opens a single RFile to read, best used for random seeks

sequentialRead(self: str)pysharkbite.SequentialRFile

Opens an RFile to read sequentially

class pysharkbite.Range
after_end_key(self: pysharkbite.Range, arg0: pysharkbite.Key) → bool

Returns true if the provided key is after this range’s end key

before_start_key(self: pysharkbite.Range, arg0: pysharkbite.Key) → bool

Returns true if the provided key is before this range’s start key

get_start_key(self: pysharkbite.Range)pysharkbite.Key

Returns the start key of this range, if it exists

get_stop_key(self: pysharkbite.Range)pysharkbite.Key

Returns the stop key of this range, if it exists

inifinite_start_key(self: pysharkbite.Range) → bool

Returns true if the start key is inclusive

inifinite_stop_key(self: pysharkbite.Range) → bool

Returns true if the stop key is inclusive

start_key_inclusive(self: pysharkbite.Range) → bool

Returns true if the start key is inclusive

stop_key_inclusive(self: pysharkbite.Range) → bool

Returns true if the stop key is inclusive

class pysharkbite.RecoveryStatus
getName(self: pysharkbite.RecoveryStatus) → str

Gets the name of the recovery.

getProgress(self: pysharkbite.RecoveryStatus) → float

Gets the progress of the recovery.

getRuntime(self: pysharkbite.RecoveryStatus) → int

Gets the runtime of the recovery process.

property name
property progress
property runtime
class pysharkbite.Results
class pysharkbite.ScannerOptions

Members:

HedgedReads : Enables hedged reads

RFileScanOnly : Enables the RFile Scanner

HedgedReads = ScannerOptions.HedgedReads
RFileScanOnly = ScannerOptions.RFileScanOnly
__entries = {'HedgedReads': (ScannerOptions.HedgedReads, 'Enables hedged reads'), 'RFileScanOnly': (ScannerOptions.RFileScanOnly, 'Enables the RFile Scanner')}
property name

(self: handle) -> str

class pysharkbite.SecurityOperations

Basic security operations

change_password(self: pysharkbite.SecurityOperations, arg0: str, arg1: str) → int

Changes the user password

create_user(self: pysharkbite.SecurityOperations, arg0: str, arg1: str) → int

Creates a user

get_auths(self: pysharkbite.SecurityOperations, arg0: str)pysharkbite.Authorizations

Returns the user’s authorizations

grantAuthorizations(self: pysharkbite.SecurityOperations, arg0: pysharkbite.Authorizations, arg1: str) → int

Get user authorizations

grant_namespace_permission(self: pysharkbite.SecurityOperations, arg0: str, arg1: str, arg2: pysharkbite.NamespacePermissions) → int

Grants a namespace permission

grant_system_permission(self: pysharkbite.SecurityOperations, arg0: str, arg1: pysharkbite.SystemPermissions) → int

Grants a system permission

grant_table_permission(self: pysharkbite.SecurityOperations, arg0: str, arg1: str, arg2: pysharkbite.TablePermissions) → int

Grants a table permission

has_namespace_permission(self: pysharkbite.SecurityOperations, arg0: str, arg1: str, arg2: pysharkbite.NamespacePermissions) → bool

Has namespace permission

has_system_permission(self: pysharkbite.SecurityOperations, arg0: str, arg1: pysharkbite.SystemPermissions) → bool

Returns true if the user has the system permission

has_table_permission(self: pysharkbite.SecurityOperations, arg0: str, arg1: str, arg2: pysharkbite.TablePermissions) → bool

Has table permission

remove_user(self: pysharkbite.SecurityOperations, arg0: str) → int

Removes the user

revoke_namespace_permission(self: pysharkbite.SecurityOperations, arg0: str, arg1: str, arg2: pysharkbite.NamespacePermissions) → int

Revokes a namespace permission

revoke_system_permission(self: pysharkbite.SecurityOperations, arg0: str, arg1: pysharkbite.SystemPermissions) → int

Revokes a system permission

revoke_table_permission(self: pysharkbite.SecurityOperations, arg0: str, arg1: str, arg2: pysharkbite.TablePermissions) → int

Revokes a table permission

class pysharkbite.Seekable
getColumnFamilies(self: pysharkbite.Seekable) → List[str]

Gets the column families for this seekable

getRange(self: pysharkbite.Seekable)pysharkbite.Range

Gets this seekable range

isInclusive(self: pysharkbite.Seekable) → bool

Returns true if the column families are inclusive.

class pysharkbite.SequentialRFile

Specializd RFile, which is an internal data structure for storing data within Accumulo, to be used when much of the data is similar

addLocalityGroup(self: pysharkbite.SequentialRFile, arg0: str) → None

Adds a locality group to an RFile when writing

append(self: pysharkbite.SequentialRFile, arg0: pysharkbite.KeyValue) → bool

Appends a key and value pair to the RFile

close(self: pysharkbite.SequentialRFile) → None

Closes the RFile

getTop(self: pysharkbite.SequentialRFile)pysharkbite.KeyValue

Returns the top key/value

hasNext(self: pysharkbite.SequentialRFile) → bool

Returns true of further keys exist, false otherwise

next(self: pysharkbite.SequentialRFile) → None

Queues the next key/value pair

seek(self: pysharkbite.SequentialRFile, arg0: cclient::data::streams::StreamRelocation) → None

Seeks to the next key and value within the RFile

class pysharkbite.StreamRelocation
class pysharkbite.SystemPermissions

Members:

GRANT : Enables grant permission on the system

CREATE_TABLE : Enables create table permissions on the system

DROP_TABLE : Enables drop table permissions on the system

ALTER_TABLE : Enables alter table permissions on the system

CREATE_USER : Enables create user permissions on the system

ALTER_USER : Enables alter user permissions on the system

SYSTEM : Enables system permissions for the user

CREATE_NAMESPACE : Enables create namespace permissions for the user

DROP_NAMESPACE : Enables drop namespace permissions for the user

ALTER_NAMESPACE : Enables the alter namespace permissions for the user

ALTER_NAMESPACE = SystemPermissions.ALTER_NAMESPACE
ALTER_TABLE = SystemPermissions.ALTER_TABLE
ALTER_USER = SystemPermissions.ALTER_USER
CREATE_NAMESPACE = SystemPermissions.CREATE_NAMESPACE
CREATE_TABLE = SystemPermissions.CREATE_TABLE
CREATE_USER = SystemPermissions.CREATE_USER
DROP_NAMESPACE = SystemPermissions.DROP_NAMESPACE
DROP_TABLE = SystemPermissions.DROP_TABLE
GRANT = SystemPermissions.GRANT
SYSTEM = SystemPermissions.SYSTEM
__entries = {'ALTER_NAMESPACE': (SystemPermissions.ALTER_NAMESPACE, 'Enables the alter namespace permissions for the user'), 'ALTER_TABLE': (SystemPermissions.ALTER_TABLE, 'Enables alter table permissions on the system'), 'ALTER_USER': (SystemPermissions.ALTER_USER, 'Enables alter user permissions on the system'), 'CREATE_NAMESPACE': (SystemPermissions.CREATE_NAMESPACE, 'Enables create namespace permissions for the user'), 'CREATE_TABLE': (SystemPermissions.CREATE_TABLE, 'Enables create table permissions on the system'), 'CREATE_USER': (SystemPermissions.CREATE_USER, 'Enables create user permissions on the system'), 'DROP_NAMESPACE': (SystemPermissions.DROP_NAMESPACE, 'Enables drop namespace permissions for the user'), 'DROP_TABLE': (SystemPermissions.DROP_TABLE, 'Enables drop table permissions on the system'), 'GRANT': (SystemPermissions.GRANT, 'Enables grant permission on the system'), 'SYSTEM': (SystemPermissions.SYSTEM, 'Enables system permissions for the user')}
property name

(self: handle) -> str

exception pysharkbite.TApplicationException
class pysharkbite.TableCompactions
getMajorCompactions(self: pysharkbite.TableCompactions)pysharkbite.Compacting

Gets major compaction stats.

getMinorCompactions(self: pysharkbite.TableCompactions)pysharkbite.Compacting

Gets minor compaction stats.

getScans(self: pysharkbite.TableCompactions)pysharkbite.Compacting

Gets scan stats.

property majors
property minors
property scans
class pysharkbite.TableInfo
property compaction_info
getCompactioninfo(self: pysharkbite.TableInfo)pysharkbite.TableCompactions

Gets compaction info for the table

getOnlineTablets(self: pysharkbite.TableInfo) → int

Gets online tablets in the table

getRecords(self: pysharkbite.TableInfo) → int

Gets records in the table

getRecordsInMemory(self: pysharkbite.TableInfo) → int

Gets records in memory for a table

getTableRates(self: pysharkbite.TableInfo)pysharkbite.TableRates

Gets table rates for the table

getTablets(self: pysharkbite.TableInfo) → int

Gets tablets in the table

property online_tables
property records
property records_in_memory
property table_rates
property tablets
class pysharkbite.TablePermissions

Members:

READ : Enables read permission on the table

WRITE : Enables write permission on the table

GRANT : Enables grant permission on the table

ALTER_TABLE : Enables alter table permissions on the table

DROP_TABLE : Enables drop table permissions on the table

BULK_IMPORT : Enables bulk import permissions on the table

ALTER_TABLE = TablePermissions.ALTER_TABLE
BULK_IMPORT = TablePermissions.BULK_IMPORT
DROP_TABLE = TablePermissions.DROP_TABLE
GRANT = TablePermissions.GRANT
READ = TablePermissions.READ
WRITE = TablePermissions.WRITE
__entries = {'ALTER_TABLE': (TablePermissions.ALTER_TABLE, 'Enables alter table permissions on the table'), 'BULK_IMPORT': (TablePermissions.BULK_IMPORT, 'Enables bulk import permissions on the table'), 'DROP_TABLE': (TablePermissions.DROP_TABLE, 'Enables drop table permissions on the table'), 'GRANT': (TablePermissions.GRANT, 'Enables grant permission on the table'), 'READ': (TablePermissions.READ, 'Enables read permission on the table'), 'WRITE': (TablePermissions.WRITE, 'Enables write permission on the table')}
property name

(self: handle) -> str

class pysharkbite.TableRates
getIngestRate(self: pysharkbite.TableRates) → float

Gets the ingest rate in records.

getIngestRateByte(self: pysharkbite.TableRates) → float

Gets the ingest rate in bytes.

getQueryRate(self: pysharkbite.TableRates) → float

Gets the query rate in records.

getQueryRateByte(self: pysharkbite.TableRates) → float

Gets the query rate in bytes.

getScanRate(self: pysharkbite.TableRates) → float

Gets the scan rate.

property query_rate_byte
property scan_rate
class pysharkbite.TabletServerStatus
property data_cache_hits
property data_cache_requests
property flushes
getDataCacheHits(self: pysharkbite.TabletServerStatus) → int

Gets data cache hits against the server

getDataCacheRequests(self: pysharkbite.TabletServerStatus) → int

Gets data cache requests against the server

getFlushes(self: pysharkbite.TabletServerStatus) → int

Gets the number of flushes on the server

getHoldTime(self: pysharkbite.TabletServerStatus) → int

Gets the hold time

getIndexCacheHits(self: pysharkbite.TabletServerStatus) → int

Gets index cache hits against the server

getLastContact(self: pysharkbite.TabletServerStatus) → int

Gets the last contact time of the server

getLogSorts(self: pysharkbite.TabletServerStatus) → List[pysharkbite.RecoveryStatus]

Gets the number of log sorts

getLookups(self: pysharkbite.TabletServerStatus) → int

Gets lookups against the server

getName(self: pysharkbite.TabletServerStatus) → str

Gets the name of the server

getOsLoad(self: pysharkbite.TabletServerStatus) → float

Gets the load of the server

getSyncs(self: pysharkbite.TabletServerStatus) → int

Gets the number of syncs on the server

getTableMap(self: pysharkbite.TabletServerStatus) → Dict[str, pysharkbite.TableInfo]

Gets the table map

property hold_time
property index_cache_hits
property last_contact
property log_sorts
property lookups
property name
property os_load
property syncs
property table_map
class pysharkbite.Value

Accumulo value

get(self: pysharkbite.Value) → str

Returns the value as a UTF-8 string

get_bytes(self: pysharkbite.Value) → bytes

Returns the bytes as python bytes

class pysharkbite.ZookeeperInstance

Zookeeper instance enables connectivity to a zookeper quorum

getInstanceId(self: pysharkbite.ZookeeperInstance, retry: bool = False) → str
getInstanceName(self: pysharkbite.ZookeeperInstance) → str

Indices and tables