Author: eevans
Date: Tue May 11 17:20:56 2010
New Revision: 943191
URL: http://svn.apache.org/viewvc?rev=943191&view=rev
Log:
avro functional test reboot
Patch by eevans
Modified:
cassandra/trunk/test/system/__init__.py
cassandra/trunk/test/system/test_avro_server.py
Modified: cassandra/trunk/test/system/__init__.py
URL:
http://svn.apache.org/viewvc/cassandra/trunk/test/system/__init__.py?rev=943191&r1=943190&r2=943191&view=diff
==============================================================================
--- cassandra/trunk/test/system/__init__.py (original)
+++ cassandra/trunk/test/system/__init__.py Tue May 11 17:20:56 2010
@@ -192,20 +192,33 @@ class AvroTester(BaseTester):
def define_schema(self):
keyspace1 = dict()
keyspace1['name'] = 'Keyspace1'
- keyspace1['strategy_class'] =
'org.apache.cassandra.locator.RackUnawareStrategy'
keyspace1['replication_factor'] = 1
- cfDef = dict()
- cfDef['keyspace'] = 'Keyspace1'
- cfDef['name'] = 'Standard1'
- cfDef['column_type'] = 'Standard'
- cfDef['comparator_type'] = 'BytesType'
- cfDef['subcomparator_type'] = ''
- cfDef['comment'] = ''
- cfDef['row_cache_size'] = 0
- cfDef['preload_row_cache'] = False
- cfDef['key_cache_size'] = 200000
- keyspace1['cf_defs'] = [cfDef]
- #keyspace1['cf_defs'] = [{'keyspace': 'Keyspace1', 'name':
'Standard1'}]
+ keyspace1['strategy_class'] = \
+ 'org.apache.cassandra.locator.RackUnawareStrategy'
+
+ keyspace1['cf_defs'] = [{
+ 'keyspace': 'Keyspace1',
+ 'name': 'Standard1',
+ 'column_type': 'Standard',
+ 'comparator_type': 'BytesType',
+ 'subcomparator_type': '',
+ 'comment': '',
+ 'row_cache_size': 0,
+ 'preload_row_cache': False,
+ 'key_cache_size': 200000
+ }]
+
+ keyspace1['cf_defs'].append({
+ 'keyspace': 'Keyspace1',
+ 'name': 'Super1',
+ 'column_type': 'Super',
+ 'comparator_type': 'BytesType',
+ 'subcomparator_type': 'LongType',
+ 'comment': '',
+ 'row_cache_size': 1000,
+ 'preload_row_cache': False,
+ 'key_cache_size': 0
+ })
self.client.request('system_add_keyspace', {'ks_def': keyspace1})
Modified: cassandra/trunk/test/system/test_avro_server.py
URL:
http://svn.apache.org/viewvc/cassandra/trunk/test/system/test_avro_server.py?rev=943191&r1=943190&r2=943191&view=diff
==============================================================================
--- cassandra/trunk/test/system/test_avro_server.py (original)
+++ cassandra/trunk/test/system/test_avro_server.py Tue May 11 17:20:56 2010
@@ -18,71 +18,10 @@ from . import AvroTester
from time import time
from random import randint
from avro.ipc import AvroRemoteException
+import struct
-COLUMNS = [
- dict(name="c0", value="v0", timestamp=1L),
- dict(name="c1", value="v1", timestamp=1L),
- dict(name="c2", value="v2", timestamp=1L),
- dict(name="c3", value="v3", timestamp=1L),
- dict(name="c4", value="v4", timestamp=1L),
- dict(name="c5", value="v5", timestamp=1L),
-]
-
-SUPERCOLUMNS = [
- dict(name="sc0", columns=COLUMNS[:3]),
- dict(name="sc1", columns=COLUMNS[3:]),
-]
-
-def _insert_column(client, name, value):
- _insert_columns(client, [(name, value)])
-
-def _insert_columns(client, columns):
- params = dict()
- params['keyspace'] = 'Keyspace1'
- params['key'] = 'key1'
- params['column_path'] = dict(column_family='Standard1')
- params['consistency_level'] = 'ONE'
-
- for (name, value) in columns:
- params['column_path']['column'] = name
- params['value'] = value
- params['timestamp'] = long(time())
- client.request('insert', params)
-
-def _insert_supercolumn(client, super_name, name, value):
- params = dict()
- params['keyspace'] = 'Keyspace1'
- params['key'] = 'key1'
- params['timestamp'] = long(time())
- params['consistency_level'] = 'ONE'
-
- params['column_path'] = dict()
- params['column_path']['column_family'] = 'Super4'
- params['column_path']['super_column'] = super_name
- params['column_path']['column'] = name
- params['value'] = value
-
- client.request('insert', params)
-
-def _get_column(client, name):
- params = dict()
- params['keyspace'] = 'Keyspace1'
- params['key'] = 'key1'
- params['column_path'] = dict(column_family='Standard1', column=name)
- params['consistency_level'] = 'ONE'
- return client.request('get', params)
-
-def _get_supercolumn(client, super_name, name):
- params = dict()
- params['keyspace'] = 'Keyspace1'
- params['key'] = 'key1'
- params['column_path'] = dict()
- params['column_path']['column_family'] = 'Super4'
- params['column_path']['super_column'] = super_name
- params['column_path']['column'] = name
- params['consistency_level'] = 'ONE'
-
- return client.request('get', params)
+def i64(i):
+ return struct.pack('>q', i)
def assert_columns_match(colA, colB):
assert colA['name'] == colB['name'], \
@@ -96,85 +35,63 @@ def assert_cosc(thing, with_supercolumn=
assert thing.has_key(containing) and thing[containing].has_key('name'), \
"Invalid or missing \"%s\"" % containing
-def random_column(columns=COLUMNS):
- return columns[randint(0, len(columns)-1)]
-
-def random_supercolumn(super_columns=SUPERCOLUMNS):
- return super_columns[randint(0, len(super_columns)-1)]
-
class TestRpcOperations(AvroTester):
def test_insert_simple(self): # Also tests get
"setting and getting a simple column"
- column = random_column()
-
- _insert_column(self.client, column['name'], column['value'])
- result = _get_column(self.client, column['name'])
-
- assert_cosc(result)
- assert_columns_match(result['column'], column)
-
- def test_insert_super(self):
- "setting and getting a super column"
- sc = random_supercolumn()
- col = random_column(sc['columns'])
-
- _insert_supercolumn(self.client, sc['name'], col['name'], col['value'])
- result = _get_supercolumn(self.client, sc['name'], col['name'])
+ self.client.request('set_keyspace', {'keyspace': 'Keyspace1'})
- assert_cosc(result)
- assert_columns_match(result['column'], col)
-
- def test_batch_insert(self):
- "performing a batch insert operation"
params = dict()
- params['keyspace'] = 'Keyspace1'
params['key'] = 'key1'
+ params['column_parent'] = {'column_family': 'Standard1'}
+ params['column'] = dict()
+ params['column']['name'] = 'c1'
+ params['column']['value'] = 'v1'
+ params['column']['timestamp'] = 0
+ params['column']['ttl'] = 0
params['consistency_level'] = 'ONE'
+ self.client.request('insert', params)
- # Map<string, list<ColumnOrSuperColumn>>
- params['cfmap'] = dict()
- params['cfmap']['Standard1'] = list()
+ read_params = dict()
+ read_params['key'] = params['key']
+ read_params['column_path'] = dict()
+ read_params['column_path']['column_family'] = 'Standard1'
+ read_params['column_path']['column'] = params['column']['name']
+ read_params['consistency_level'] = 'ONE'
- for i in range(0,3):
- params['cfmap']['Standard1'].append(dict(column=COLUMNS[i]))
+ cosc = self.client.request('get', read_params)
- self.client.request('batch_insert', params)
+ assert_cosc(cosc)
+ assert_columns_match(cosc['column'], params['column'])
- for i in range(0,3):
- assert_cosc(_get_column(self.client, COLUMNS[i]['name']))
+ def test_insert_super(self):
+ "setting and getting a super column"
+ self.client.request('set_keyspace', {'keyspace': 'Keyspace1'})
- def test_batch_mutate(self):
- "performing batch mutation operations"
params = dict()
- params['keyspace'] = 'Keyspace1'
+ params['key'] = 'key1'
+ params['column_parent'] = dict()
+ params['column_parent']['column_family'] = 'Super1'
+ params['column_parent']['super_column'] = 'sc1'
+ params['column'] = dict()
+ params['column']['name'] = i64(1)
+ params['column']['value'] = 'v1'
+ params['column']['timestamp'] = 0
+ params['column']['ttl'] = 0
params['consistency_level'] = 'ONE'
+ self.client.request('insert', params)
+
+ read_params = dict()
+ read_params['key'] = params['key']
+ read_params['column_path'] = dict()
+ read_params['column_path']['column_family'] = 'Super1'
+ read_params['column_path']['super_column'] =
params['column_parent']['super_column']
+ read_params['column_path']['column'] = params['column']['name']
+ read_params['consistency_level'] = 'ONE'
- mutation_map = dict()
- mutation_map['key1'] = dict(Standard1=[
- dict(column_or_supercolumn=dict(column=COLUMNS[0])),
- dict(column_or_supercolumn=dict(column=COLUMNS[1])),
- dict(column_or_supercolumn=dict(column=COLUMNS[2]))
- ])
-
- params['mutation_map'] = mutation_map
-
- self.client.request('batch_mutate', params)
-
- for i in range(0,3):
- cosc = _get_column(self.client, COLUMNS[i]['name'])
- assert_cosc(cosc)
- assert_columns_match(cosc['column'], COLUMNS[i])
-
- # FIXME: still need to apply a mutation that deletes
-
- #try:
- # assert not _get_column(self.client, COLUMNS[1]['name']), \
- # "Mutation did not delete column %s" % COLUMNS[1]['name']
- # assert not _get_column(self.client, COLUMNS[2]['name']), \
- # "Mutation did not delete column %s" % COLUMNS[2]['name']
- #except AvroRemoteException:
- # pass
+ cosc = self.client.request('get', read_params)
+ assert_cosc(cosc)
+ assert_columns_match(cosc['column'], params['column'])
def test_get_api_version(self):
"getting the remote api version string"