You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@gora.apache.org by le...@apache.org on 2012/07/02 15:39:02 UTC

svn commit: r1356215 [3/3] - in /gora/branches/goraamazon: ./ gora-accumulo/src/main/java/org/apache/gora/accumulo/query/ gora-accumulo/src/main/java/org/apache/gora/accumulo/store/ gora-cassandra/src/main/java/org/apache/gora/cassandra/query/ gora-cas...

Added: gora/branches/goraamazon/gora-dynamodb/pom.xml
URL: http://svn.apache.org/viewvc/gora/branches/goraamazon/gora-dynamodb/pom.xml?rev=1356215&view=auto
==============================================================================
--- gora/branches/goraamazon/gora-dynamodb/pom.xml (added)
+++ gora/branches/goraamazon/gora-dynamodb/pom.xml Mon Jul  2 13:38:43 2012
@@ -0,0 +1,145 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <!--
+   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.
+   -->
+   <modelVersion>4.0.0</modelVersion>
+   <parent>
+       <groupId>org.apache.gora</groupId>
+       <artifactId>gora</artifactId>
+       <version>0.3-SNAPSHOT</version>
+       <relativePath>../</relativePath>
+   </parent>
+   <artifactId>gora-dynamodb</artifactId>
+   <packaging>bundle</packaging>
+   <name>Apache Gora :: Dynamodb</name>
+       <url>http://gora.apache.org</url>
+   <description>The Apache Gora open source framework provides an in-memory data model and 
+   persistence for big data. Gora supports persisting to column stores, key value stores, 
+   document stores and RDBMSs, and analyzing the data with extensive Apache Hadoop MapReduce 
+   support.</description>
+   <inceptionYear>2010</inceptionYear>
+   <organization>
+   	<name>The Apache Software Foundation</name>
+   	<url>http://www.apache.org/</url>
+   </organization>
+   <scm>
+   	<url>http://svn.apache.org/viewvc/gora/trunk/gora-dynamodb/</url>
+   	<connection>scm:svn:http://svn.apache.org/repos/asf/gora/trunk/gora-dynamodb/</connection>
+   	<developerConnection>scm:svn:https://svn.apache.org/repos/asf/gora/trunk/gora-dynamodb/</developerConnection>
+   </scm>
+   <issueManagement>
+   	<system>JIRA</system>
+   	<url>https://issues.apache.org/jira/browse/GORA</url>
+   </issueManagement>
+   <ciManagement>
+   	<system>Jenkins</system>
+   	<url>https://builds.apache.org/job/Gora-trunk/</url>
+   </ciManagement>
+   <properties>
+       <osgi.import>*</osgi.import>
+       <osgi.export>org.apache.gora.dynamodb*;version="${project.version}";-noimport:=true</osgi.export>
+   </properties>
+   <build>
+       <directory>target</directory>
+       <outputDirectory>target/classes</outputDirectory>
+       <finalName>${project.artifactId}-${project.version}</finalName>
+   <!--<testOutputDirectory>target/test-classes</testOutputDirectory>-->
+   <!--<testSourceDirectory>src/test/java</testSourceDirectory>-->
+       <sourceDirectory>src/main/java</sourceDirectory>
+       <resources>
+          <resource>
+            <directory>${basedir}/src/main/resources</directory>
+          </resource>
+          <resource>
+            <directory>${basedir}/conf</directory>
+          </resource>
+       </resources>
+       <plugins>
+           <plugin>
+               <groupId>org.codehaus.mojo</groupId>
+               <artifactId>build-helper-maven-plugin</artifactId>
+               <version>${build-helper-maven-plugin.version}</version>
+               <executions>
+                   <execution>
+                       <phase>generate-sources</phase>
+                       <goals>
+                           <goal>add-source</goal>
+                       </goals>
+                       <configuration>
+                           <sources>
+                               <source>src/examples/java</source>
+                           </sources>
+                       </configuration>
+                   </execution>
+               </executions>
+           </plugin>
+           <plugin>
+               <groupId>org.apache.maven.plugins</groupId>
+               <artifactId>maven-jar-plugin</artifactId>
+               <version>${maven-jar-plugin.version}</version>
+               <executions>
+                   <execution>
+                       <goals>
+                           <goal>jar</goal>
+                           <goal>test-jar</goal>
+                       </goals>
+                       <configuration>
+                       <archive>
+                           <manifestFile>${project.build.outputDirectory}/META-INF/MANIFEST.MF</manifestFile>
+                       </archive>
+                   </configuration>
+                   </execution>
+               </executions>
+           </plugin>
+       </plugins>
+   </build>
+   <dependencies>
+       <!-- Gora Internal Dependencies -->
+       <dependency>
+           <groupId>org.apache.gora</groupId>
+           <artifactId>gora-core</artifactId>
+       </dependency>
+       <dependency>
+           <groupId>org.apache.gora</groupId>
+           <artifactId>gora-core</artifactId>
+           <classifier>tests</classifier>
+       </dependency>
+       
+       <!-- Misc Dependencies -->
+       <dependency>
+       	   <groupId>com.amazonaws</groupId>
+	   <artifactId>aws-java-sdk</artifactId>
+       </dependency>
+       
+       <!-- Logging Dependencies -->
+       <dependency>
+           <groupId>org.slf4j</groupId>
+           <artifactId>slf4j-api</artifactId>
+       </dependency>
+       <dependency>
+           <groupId>org.slf4j</groupId>
+           <artifactId>slf4j-jdk14</artifactId>
+       </dependency>
+       <dependency>
+           <groupId>log4j</groupId>
+           <artifactId>log4j</artifactId>
+       </dependency>
+       <!-- Testing Dependencies -->
+       <dependency>
+           <groupId>junit</groupId>
+           <artifactId>junit</artifactId>
+       </dependency>
+    </dependencies>
+</project>

Added: gora/branches/goraamazon/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/query/DynamoDBColumn.java
URL: http://svn.apache.org/viewvc/gora/branches/goraamazon/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/query/DynamoDBColumn.java?rev=1356215&view=auto
==============================================================================
--- gora/branches/goraamazon/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/query/DynamoDBColumn.java (added)
+++ gora/branches/goraamazon/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/query/DynamoDBColumn.java Mon Jul  2 13:38:43 2012
@@ -0,0 +1,84 @@
+/**
+ * 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.
+ */
+
+package org.apache.gora.dynamodb.query;
+
+import java.nio.ByteBuffer;
+
+//import org.slf4j.Logger;
+//import org.slf4j.LoggerFactory;
+
+/**
+ * Represents a unit of data: a key value pair tagged by a family name
+ */
+public abstract class DynamoDBColumn {
+  //public static final Logger LOG = LoggerFactory.getLogger(DynamoDBColumn.class);
+
+  public static final int SUB = 0;
+  public static final int SUPER = 1;
+  
+  private String family;
+  private int type;
+  
+  public String getFamily() {
+    return family;
+  }
+  public void setFamily(String family) {
+    this.family = family;
+  }
+  public int getType() {
+    return type;
+  }
+  public void setType(int type) {
+    this.type = type;
+  }
+  
+  public abstract ByteBuffer getName();
+  public abstract Object getValue();
+  
+
+/*  protected Object fromByteBuffer(Type type, ByteBuffer byteBuffer) {
+    Object value = null;
+    switch (type) {
+      case STRING:
+        value = new Utf8(StringSerializer.get().fromByteBuffer(byteBuffer));
+        break;
+      case BYTES:
+        value = byteBuffer;
+        break;
+      case INT:
+        value = IntegerSerializer.get().fromByteBuffer(byteBuffer);
+        break;
+      case LONG:
+        value = LongSerializer.get().fromByteBuffer(byteBuffer);
+        break;
+      case FLOAT:
+        value = FloatSerializer.get().fromByteBuffer(byteBuffer);
+        break;
+      case DOUBLE:
+        value = DoubleSerializer.get().fromByteBuffer(byteBuffer);
+        break;
+
+      default:
+        LOG.info("Type is not supported: " + type);
+
+    }
+    return value;
+  }
+*/
+}

Added: gora/branches/goraamazon/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/query/DynamoDBQuery.java
URL: http://svn.apache.org/viewvc/gora/branches/goraamazon/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/query/DynamoDBQuery.java?rev=1356215&view=auto
==============================================================================
--- gora/branches/goraamazon/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/query/DynamoDBQuery.java (added)
+++ gora/branches/goraamazon/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/query/DynamoDBQuery.java Mon Jul  2 13:38:43 2012
@@ -0,0 +1,102 @@
+/**
+ * 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.
+ */
+
+package org.apache.gora.dynamodb.query;
+
+import java.util.Collection;
+
+import org.apache.gora.persistency.Persistent;
+import org.apache.gora.query.Query;
+import org.apache.gora.query.ws.impl.QueryWSBase;
+import org.apache.gora.store.DataStore;
+
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.services.dynamodb.model.QueryRequest;
+
+public class DynamoDBQuery<K, T extends Persistent> extends QueryWSBase<K, T> {
+	
+	/**
+	 * Query object to perform requests to the datastore
+	 */
+	//private Query<K, T> query;
+	
+	/**
+	 * Maps fields to DynamoDB attributes.
+	 */
+	private Collection<String> attributesColl;
+	
+	/**
+	 * Query object to perform requests to the datastore
+	 */
+	QueryRequest dynamoDBquery;
+
+	public DynamoDBQuery(){
+		super(null);
+		this.dynamoDBquery = new QueryRequest();
+	}
+  
+	public DynamoDBQuery(DataStore<K, T> dataStore) {
+		super(dataStore);
+	}
+	
+	public void setAttrCollection(Collection<String> attrsColl){
+		this.attributesColl = attrsColl;
+	}
+	
+	public Collection<String> getAttrCollection(){
+		return attributesColl;
+	}
+	public void setTableName(String tableName){
+		this.dynamoDBquery.setTableName(tableName);
+	}
+	
+	public void setLimit(int limit){
+		this.dynamoDBquery.setLimit(limit);
+	}
+	
+	public void setConsistencyRead(boolean consistency){
+		this.dynamoDBquery.setConsistentRead(consistency);
+	}
+	
+	public void setCredentials(AWSCredentials credentials){
+		this.dynamoDBquery.setRequestCredentials(credentials);
+	}
+  
+  /**
+   * @param family the family name
+   * @return an array of the query column names belonging to the family
+   */
+  public String[] getColumns(String family) {
+    
+    //List<String> columnList = attributesMap.get(family);
+    String[] columns = new String[2];
+    //for (int i = 0; i < columns.length; ++i) {
+    //  columns[i] = columnList.get(i);
+    //}
+    return columns;
+  }
+  
+  public QueryRequest getQuery() {
+    return dynamoDBquery;
+  }
+  
+  public void setQuery(Query<K, T> query) {
+    this.dynamoDBquery = (QueryRequest)query;
+  }
+
+}

Added: gora/branches/goraamazon/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/query/DynamoDBResult.java
URL: http://svn.apache.org/viewvc/gora/branches/goraamazon/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/query/DynamoDBResult.java?rev=1356215&view=auto
==============================================================================
--- gora/branches/goraamazon/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/query/DynamoDBResult.java (added)
+++ gora/branches/goraamazon/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/query/DynamoDBResult.java Mon Jul  2 13:38:43 2012
@@ -0,0 +1,63 @@
+/**
+ * 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.
+ */
+
+package org.apache.gora.dynamodb.query;
+
+import java.io.IOException;
+
+import org.apache.gora.persistency.Persistent;
+import org.apache.gora.query.Query;
+import org.apache.gora.query.ws.impl.ResultWSBase;
+import org.apache.gora.store.DataStore;
+//import org.slf4j.Logger;
+//import org.slf4j.LoggerFactory;
+
+public class DynamoDBResult<K, T extends Persistent> extends ResultWSBase<K, T> {
+  //public static final Logger LOG = LoggerFactory.getLogger(DynamoDBResult.class);
+  
+  private int rowNumber;
+
+  private DynamoDBResultSet<K> dynamoDBResultSet;
+
+  public DynamoDBResult(DataStore<K, T> dataStore, Query<K, T> query) {
+    super(dataStore, query);
+  }
+
+  public void setResultSet(DynamoDBResultSet<K> dynamoDBResultSet) {
+    this.dynamoDBResultSet = dynamoDBResultSet;
+  }
+
+@Override
+public float getProgress() throws IOException, InterruptedException, Exception {
+	// TODO Auto-generated method stub
+	return 0;
+}
+
+@Override
+public void close() throws IOException {
+	// TODO Auto-generated method stub
+	
+}
+
+@Override
+protected boolean nextInner() throws Exception {
+	// TODO Auto-generated method stub
+	return false;
+}
+
+}

Added: gora/branches/goraamazon/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/query/DynamoDBResultSet.java
URL: http://svn.apache.org/viewvc/gora/branches/goraamazon/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/query/DynamoDBResultSet.java?rev=1356215&view=auto
==============================================================================
--- gora/branches/goraamazon/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/query/DynamoDBResultSet.java (added)
+++ gora/branches/goraamazon/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/query/DynamoDBResultSet.java Mon Jul  2 13:38:43 2012
@@ -0,0 +1,54 @@
+/**
+ * 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.
+ */
+
+package org.apache.gora.dynamodb.query;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+
+/**
+ * List data structure to keep the order coming from the Cassandra selects.
+ */
+public class DynamoDBResultSet<K> extends ArrayList<DynamoDBRow<K>> {
+
+  /**
+   * 
+   */
+  private static final long serialVersionUID = -7620939600192859652L;
+
+  /**
+   * Maps keys to indices in the list.
+   */
+  private HashMap<K, Integer> indexMap = new HashMap<K, Integer>();
+
+  public DynamoDBRow<K> getRow(K key) {
+    Integer integer = this.indexMap.get(key);
+    if (integer == null) {
+      return null;
+    }
+    
+    return this.get(integer);
+  }
+
+  public void putRow(K key, DynamoDBRow<K> cassandraRow) {
+    this.add(cassandraRow);
+    this.indexMap.put(key, this.size()-1);
+  } 
+  
+
+}

Added: gora/branches/goraamazon/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/query/DynamoDBRow.java
URL: http://svn.apache.org/viewvc/gora/branches/goraamazon/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/query/DynamoDBRow.java?rev=1356215&view=auto
==============================================================================
--- gora/branches/goraamazon/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/query/DynamoDBRow.java (added)
+++ gora/branches/goraamazon/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/query/DynamoDBRow.java Mon Jul  2 13:38:43 2012
@@ -0,0 +1,42 @@
+/**
+ * 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.
+ */
+
+package org.apache.gora.dynamodb.query;
+
+import java.util.ArrayList;
+
+/**
+ * List of key value pairs representing a row, tagged by a key.
+ */
+public class DynamoDBRow<K> extends ArrayList<DynamoDBColumn> {
+
+  /**
+   * 
+   */
+  private static final long serialVersionUID = -7620939600192859652L;
+  private K key;
+
+  public K getKey() {
+    return this.key;
+  }
+
+  public void setKey(K key) {
+    this.key = key;
+  }
+
+}

Added: gora/branches/goraamazon/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/store/DynamoDBStore.java
URL: http://svn.apache.org/viewvc/gora/branches/goraamazon/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/store/DynamoDBStore.java?rev=1356215&view=auto
==============================================================================
--- gora/branches/goraamazon/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/store/DynamoDBStore.java (added)
+++ gora/branches/goraamazon/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/store/DynamoDBStore.java Mon Jul  2 13:38:43 2012
@@ -0,0 +1,427 @@
+/**
+ * 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.
+ */
+
+package org.apache.gora.dynamodb.store;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.gora.dynamodb.query.DynamoDBQuery;
+import org.apache.gora.dynamodb.query.DynamoDBResult;
+import org.apache.gora.persistency.BeanFactory;
+import org.apache.gora.persistency.Persistent;
+import org.apache.gora.query.PartitionQuery;
+import org.apache.gora.query.Query;
+import org.apache.gora.query.Result;
+import org.apache.gora.query.ws.impl.PartitionWSQueryImpl;
+import org.apache.gora.store.ws.impl.WSDataStoreBase;
+//import org.slf4j.Logger;
+//import org.slf4j.LoggerFactory;
+
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.PropertiesCredentials;
+import com.amazonaws.services.dynamodb.AmazonDynamoDBClient;
+import com.amazonaws.services.dynamodb.model.QueryResult;
+
+public class DynamoDBStore<K, T extends Persistent> extends WSDataStoreBase<K, T> {
+	
+  //public static final Logger LOG = LoggerFactory.getLogger(DynamoDBStore.class);
+  
+  /**
+   * Path where the AWS Credential will reside
+   */
+  private static String awsCredentialsProperties = "conf/AwsCredentials.properties";
+  
+  private static String wsProvider = "Amazon.Web.Services";
+	 
+  /**
+   * TODO Amazon DynamoDB decorator 
+   * because all DynamoDBAsyncClients are DynamoDBClients   
+   */
+  private AmazonDynamoDBClient dynamoDBClient;
+
+  public DynamoDBStore(){
+  }
+
+  @Override
+  public void initialize(Class<K> keyClass, Class<T> persistentClass,
+	      Properties properties) throws Exception {
+	  try {
+		File file = new File(awsCredentialsProperties);
+		AWSCredentials credentials = new PropertiesCredentials(file);
+		setConf(credentials);
+		setWsProvider(wsProvider);
+		//TODO Create decorator to create different AmazonDynamoDB clients
+		// this is because all amazonDynamoDBAsynClients are AmazonDynamoDBClients
+		dynamoDBClient = new AmazonDynamoDBClient(credentials);
+		
+	  }
+	    catch (Exception e) {
+	      throw new IOException(e.getMessage(), e);
+	  }
+  }
+  /*
+  public void close() throws IOException {
+    LOG.debug("close");
+    flush();
+  }
+
+  @Override
+  public void createSchema() {
+    LOG.debug("create schema");
+    this.cassandraClient.checkKeyspace();
+  }
+
+  @Override
+  public boolean delete(K key) throws IOException {
+    LOG.debug("delete " + key);
+    return false;
+  }
+
+  @Override
+  public long deleteByQuery(Query<K, T> query) throws IOException {
+    LOG.debug("delete by query " + query);
+    return 0;
+  }
+
+  @Override
+  public void deleteSchema() throws IOException {
+    LOG.debug("delete schema");
+    this.cassandraClient.dropKeyspace();
+  }*/
+
+  @Override
+  public Result<K, T> execute(Query<K, T> query) throws Exception {
+    
+	 DynamoDBQuery<K, T> dynamoDBQuery = new DynamoDBQuery<K, T>();
+	 DynamoDBResult<K, T> dynamoDBResult = new DynamoDBResult<K, T>(this, dynamoDBQuery);
+	 
+	 dynamoDBQuery.setQuery(query);
+	 
+	 // TODO we should return the object that the class is supposed to return
+	 QueryResult dynamodbResult = dynamoDBClient.query(dynamoDBQuery.getQuery());
+	 
+	 return (Result<K, T>)dynamoDBResult;
+  }
+
+  /*
+  private void addSubColumns(String family, DynamoDBQuery<K, T> dynamoDBQuery,
+      CassandraResultSet cassandraResultSet) {
+    // select family columns that are included in the query
+    List<Row<K, ByteBuffer, ByteBuffer>> rows = this.cassandraClient.execute(dynamoDBQuery, family);
+    
+    for (Row<K, ByteBuffer, ByteBuffer> row : rows) {
+      K key = row.getKey();
+      
+      // find associated row in the resultset
+      CassandraRow<K> cassandraRow = cassandraResultSet.getRow(key);
+      if (cassandraRow == null) {
+        cassandraRow = new CassandraRow<K>();
+        cassandraResultSet.putRow(key, cassandraRow);
+        cassandraRow.setKey(key);
+      }
+      
+      ColumnSlice<ByteBuffer, ByteBuffer> columnSlice = row.getColumnSlice();
+      
+      for (HColumn<ByteBuffer, ByteBuffer> hColumn : columnSlice.getColumns()) {
+        CassandraSubColumn cassandraSubColumn = new CassandraSubColumn();
+        cassandraSubColumn.setValue(hColumn);
+        cassandraSubColumn.setFamily(family);
+        cassandraRow.add(cassandraSubColumn);
+      }
+      
+    }
+  }
+  */
+/*
+  private void addSuperColumns(String family, CassandraQuery<K, T> cassandraQuery, 
+      CassandraResultSet cassandraResultSet) {
+    
+    List<SuperRow<K, String, ByteBuffer, ByteBuffer>> superRows = this.cassandraClient.executeSuper(cassandraQuery, family);
+    for (SuperRow<K, String, ByteBuffer, ByteBuffer> superRow: superRows) {
+      K key = superRow.getKey();
+      CassandraRow<K> cassandraRow = cassandraResultSet.getRow(key);
+      if (cassandraRow == null) {
+        cassandraRow = new CassandraRow();
+        cassandraResultSet.putRow(key, cassandraRow);
+        cassandraRow.setKey(key);
+      }
+      
+      SuperSlice<String, ByteBuffer, ByteBuffer> superSlice = superRow.getSuperSlice();
+      for (HSuperColumn<String, ByteBuffer, ByteBuffer> hSuperColumn: superSlice.getSuperColumns()) {
+        CassandraSuperColumn cassandraSuperColumn = new CassandraSuperColumn();
+        cassandraSuperColumn.setValue(hSuperColumn);
+        cassandraSuperColumn.setFamily(family);
+        cassandraRow.add(cassandraSuperColumn);
+      }
+    }
+  }
+*/
+ 
+  @Override
+  public T get(K key, String[] fields) throws Exception {
+    DynamoDBQuery<K,T> query = new DynamoDBQuery<K,T>();
+    query.setDataStore(this);
+    query.setKeyRange(key, key);
+    query.setFields(fields);
+    query.setLimit(1);
+    Result<K,T> result = execute(query);
+    boolean hasResult = result.next();
+    return hasResult ? result.get() : null;
+  }
+
+  public Query<K, T> newQuery() {
+    Query<K,T> query = new DynamoDBQuery<K, T>(this);
+   // query.setFields(getFieldsToQuery(null));
+    return query;
+  }
+
+@Override
+public String getSchemaName() {
+	// TODO Auto-generated method stub
+	return null;
+}
+
+@Override
+public void createSchema() throws Exception {
+	// TODO Auto-generated method stub
+	
+}
+
+@Override
+public void deleteSchema() throws Exception {
+	// TODO Auto-generated method stub
+	
+}
+
+@Override
+public boolean schemaExists() throws Exception {
+	// TODO Auto-generated method stub
+	return false;
+}
+
+@Override
+public K newKey() throws Exception {
+	// TODO Auto-generated method stub
+	return null;
+}
+
+@Override
+public T newPersistent() throws Exception {
+	// TODO Auto-generated method stub
+	return null;
+}
+
+@Override
+public T get(K key) throws Exception {
+	// TODO Auto-generated method stub
+	return null;
+}
+
+@Override
+public void put(K key, T obj) throws Exception {
+	// TODO Auto-generated method stub
+	
+}
+
+@Override
+public boolean delete(K key) throws Exception {
+	// TODO Auto-generated method stub
+	return false;
+}
+
+@Override
+public long deleteByQuery(Query<K, T> query) throws Exception {
+	// TODO Auto-generated method stub
+	return 0;
+}
+
+@Override
+public List<PartitionQuery<K, T>> getPartitions(Query<K, T> query)
+		throws IOException {
+	// TODO Auto-generated method stub
+	return null;
+}
+
+@Override
+public void flush() throws Exception {
+	// TODO Auto-generated method stub
+	
+}
+
+@Override
+public void setBeanFactory(BeanFactory<K, T> beanFactory) {
+	// TODO Auto-generated method stub
+	
+}
+
+@Override
+public BeanFactory<K, T> getBeanFactory() {
+	// TODO Auto-generated method stub
+	return null;
+}
+
+@Override
+public void close() throws IOException, InterruptedException, Exception {
+	// TODO Auto-generated method stub
+	
+}
+
+  /**
+   * Duplicate instance to keep all the objects in memory till flushing.
+   * @see org.apache.gora.store.DataStore#put(java.lang.Object, org.apache.gora.persistency.Persistent)
+   
+  @Override
+  public void put(K key, T value) throws IOException {
+    T p = (T) value.newInstance(new StateManagerImpl());
+    Schema schema = value.getSchema();
+    for (Field field: schema.getFields()) {
+      if (value.isDirty(field.pos())) {
+        Object fieldValue = value.get(field.pos());
+        
+        // check if field has a nested structure (array, map, or record)
+        Schema fieldSchema = field.schema();
+        Type type = fieldSchema.getType();
+        switch(type) {
+          case RECORD:
+            Persistent persistent = (Persistent) fieldValue;
+            Persistent newRecord = persistent.newInstance(new StateManagerImpl());
+            for (Field member: fieldSchema.getFields()) {
+              newRecord.put(member.pos(), persistent.get(member.pos()));
+            }
+            fieldValue = newRecord;
+            break;
+          case MAP:
+            StatefulHashMap<?, ?> map = (StatefulHashMap<?, ?>) fieldValue;
+            StatefulHashMap<?, ?> newMap = new StatefulHashMap(map);
+            fieldValue = newMap;
+            break;
+          case ARRAY:
+            GenericArray array = (GenericArray) fieldValue;
+            Type elementType = fieldSchema.getElementType().getType();
+            GenericArray newArray = new ListGenericArray(Schema.create(elementType));
+            Iterator iter = array.iterator();
+            while (iter.hasNext()) {
+              newArray.add(iter.next());
+            }
+            fieldValue = newArray;
+            break;
+        }
+        
+        p.put(field.pos(), fieldValue);
+      }
+    }
+    
+    // this performs a structural modification of the map
+    this.buffer.put(key, p);
+ }
+*/
+  /**
+   * Add a field to Cassandra according to its type.
+   * @param key     the key of the row where the field should be added
+   * @param field   the Avro field representing a datum
+   * @param value   the field value
+   
+  private void addOrUpdateField(K key, Field field, Object value) {
+    Schema schema = field.schema();
+    Type type = schema.getType();
+    switch (type) {
+      case STRING:
+      case INT:
+      case LONG:
+      case BYTES:
+      case FLOAT:
+      case DOUBLE:
+        this.cassandraClient.addColumn(key, field.name(), value);
+        break;
+      case RECORD:
+        if (value != null) {
+          if (value instanceof PersistentBase) {
+            PersistentBase persistentBase = (PersistentBase) value;
+            for (Field member: schema.getFields()) {
+              
+              // TODO: hack, do not store empty arrays
+              Object memberValue = persistentBase.get(member.pos());
+              if (memberValue instanceof GenericArray<?>) {
+                GenericArray<String> array = (GenericArray<String>) memberValue;
+                if (array.size() == 0) {
+                  continue;
+                }
+              }
+              
+              if (memberValue instanceof Utf8) {
+                memberValue = memberValue.toString();
+              }
+              this.cassandraClient.addSubColumn(key, field.name(), StringSerializer.get().toByteBuffer(member.name()), memberValue);
+            }
+          } else {
+            LOG.info("Record not supported: " + value.toString());
+            
+          }
+        }
+        break;
+      case MAP:
+        if (value != null) {
+          if (value instanceof StatefulHashMap<?, ?>) {
+            //TODO cast to stateful map and only write dirty keys
+            Map<Utf8, Object> map = (Map<Utf8, Object>) value;
+            for (Utf8 mapKey: map.keySet()) {
+              
+              // TODO: hack, do not store empty arrays
+              Object keyValue = map.get(mapKey);
+              if (keyValue instanceof GenericArray<?>) {
+                GenericArray<String> array = (GenericArray<String>) keyValue;
+                if (array.size() == 0) {
+                  continue;
+                }
+              }
+              
+              if (keyValue instanceof Utf8) {
+                keyValue = keyValue.toString();
+              }
+              this.cassandraClient.addSubColumn(key, field.name(), StringSerializer.get().toByteBuffer(mapKey.toString()), keyValue);              
+            }
+          } else {
+            LOG.info("Map not supported: " + value.toString());
+          }
+        }
+        break;
+      case ARRAY:
+        if (value != null) {
+          if (value instanceof GenericArray<?>) {
+            GenericArray<Object> array = (GenericArray<Object>) value;
+            int i= 0;
+            for (Object itemValue: array) {
+              if (itemValue instanceof Utf8) {
+                itemValue = itemValue.toString();
+              }
+              this.cassandraClient.addSubColumn(key, field.name(), IntegerSerializer.get().toByteBuffer(i++), itemValue);              
+            }
+          } else {
+            LOG.info("Array not supported: " + value.toString());
+          }
+        }
+        break;
+      default:
+        LOG.info("Type not considered: " + type.name());      
+    }
+  }
+*/
+}

Modified: gora/branches/goraamazon/gora-hbase/src/main/java/org/apache/gora/hbase/query/HBaseGetResult.java
URL: http://svn.apache.org/viewvc/gora/branches/goraamazon/gora-hbase/src/main/java/org/apache/gora/hbase/query/HBaseGetResult.java?rev=1356215&r1=1356214&r2=1356215&view=diff
==============================================================================
--- gora/branches/goraamazon/gora-hbase/src/main/java/org/apache/gora/hbase/query/HBaseGetResult.java (original)
+++ gora/branches/goraamazon/gora-hbase/src/main/java/org/apache/gora/hbase/query/HBaseGetResult.java Mon Jul  2 13:38:43 2012
@@ -22,6 +22,7 @@ import java.io.IOException;
 
 import org.apache.gora.hbase.store.HBaseStore;
 import org.apache.gora.persistency.Persistent;
+import org.apache.gora.persistency.impl.PersistentBase;
 import org.apache.gora.query.Query;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Result;
@@ -29,7 +30,7 @@ import org.apache.hadoop.hbase.client.Re
 /**
  * An {@link HBaseResult} based on the result of a HBase {@link Get} query.
  */
-public class HBaseGetResult<K, T extends Persistent> extends HBaseResult<K,T> {
+public class HBaseGetResult<K, T extends PersistentBase> extends HBaseResult<K,T> {
 
   private Result result;
   

Modified: gora/branches/goraamazon/gora-hbase/src/main/java/org/apache/gora/hbase/query/HBaseQuery.java
URL: http://svn.apache.org/viewvc/gora/branches/goraamazon/gora-hbase/src/main/java/org/apache/gora/hbase/query/HBaseQuery.java?rev=1356215&r1=1356214&r2=1356215&view=diff
==============================================================================
--- gora/branches/goraamazon/gora-hbase/src/main/java/org/apache/gora/hbase/query/HBaseQuery.java (original)
+++ gora/branches/goraamazon/gora-hbase/src/main/java/org/apache/gora/hbase/query/HBaseQuery.java Mon Jul  2 13:38:43 2012
@@ -19,6 +19,7 @@
 package org.apache.gora.hbase.query;
 
 import org.apache.gora.persistency.Persistent;
+import org.apache.gora.persistency.impl.PersistentBase;
 import org.apache.gora.query.Query;
 import org.apache.gora.query.impl.QueryBase;
 import org.apache.gora.store.DataStore;
@@ -26,7 +27,7 @@ import org.apache.gora.store.DataStore;
 /**
  * HBase specific implementation of the {@link Query} interface.
  */
-public class HBaseQuery<K, T extends Persistent> extends QueryBase<K, T> {
+public class HBaseQuery<K, T extends PersistentBase> extends QueryBase<K, T> {
 
   public HBaseQuery() {
     super(null);

Modified: gora/branches/goraamazon/gora-hbase/src/main/java/org/apache/gora/hbase/query/HBaseResult.java
URL: http://svn.apache.org/viewvc/gora/branches/goraamazon/gora-hbase/src/main/java/org/apache/gora/hbase/query/HBaseResult.java?rev=1356215&r1=1356214&r2=1356215&view=diff
==============================================================================
--- gora/branches/goraamazon/gora-hbase/src/main/java/org/apache/gora/hbase/query/HBaseResult.java (original)
+++ gora/branches/goraamazon/gora-hbase/src/main/java/org/apache/gora/hbase/query/HBaseResult.java Mon Jul  2 13:38:43 2012
@@ -24,6 +24,7 @@ import java.io.IOException;
 
 import org.apache.gora.hbase.store.HBaseStore;
 import org.apache.gora.persistency.Persistent;
+import org.apache.gora.persistency.impl.PersistentBase;
 import org.apache.gora.query.Query;
 import org.apache.gora.query.impl.ResultBase;
 import org.apache.hadoop.hbase.client.Result;
@@ -31,7 +32,7 @@ import org.apache.hadoop.hbase.client.Re
 /**
  * Base class for {@link Result} implementations for HBase.  
  */
-public abstract class HBaseResult<K, T extends Persistent> 
+public abstract class HBaseResult<K, T extends PersistentBase> 
   extends ResultBase<K, T> {
 
   public HBaseResult(HBaseStore<K,T> dataStore, Query<K, T> query) {

Modified: gora/branches/goraamazon/gora-hbase/src/main/java/org/apache/gora/hbase/query/HBaseScannerResult.java
URL: http://svn.apache.org/viewvc/gora/branches/goraamazon/gora-hbase/src/main/java/org/apache/gora/hbase/query/HBaseScannerResult.java?rev=1356215&r1=1356214&r2=1356215&view=diff
==============================================================================
--- gora/branches/goraamazon/gora-hbase/src/main/java/org/apache/gora/hbase/query/HBaseScannerResult.java (original)
+++ gora/branches/goraamazon/gora-hbase/src/main/java/org/apache/gora/hbase/query/HBaseScannerResult.java Mon Jul  2 13:38:43 2012
@@ -22,6 +22,7 @@ import java.io.IOException;
 
 import org.apache.gora.hbase.store.HBaseStore;
 import org.apache.gora.persistency.Persistent;
+import org.apache.gora.persistency.impl.PersistentBase;
 import org.apache.gora.query.Query;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
@@ -29,7 +30,7 @@ import org.apache.hadoop.hbase.client.Re
 /**
  * Result of a query based on an HBase scanner.
  */
-public class HBaseScannerResult<K, T extends Persistent> 
+public class HBaseScannerResult<K, T extends PersistentBase> 
   extends HBaseResult<K, T> {
 
   private final ResultScanner scanner;

Modified: gora/branches/goraamazon/gora-hbase/src/main/java/org/apache/gora/hbase/store/HBaseStore.java
URL: http://svn.apache.org/viewvc/gora/branches/goraamazon/gora-hbase/src/main/java/org/apache/gora/hbase/store/HBaseStore.java?rev=1356215&r1=1356214&r2=1356215&view=diff
==============================================================================
--- gora/branches/goraamazon/gora-hbase/src/main/java/org/apache/gora/hbase/store/HBaseStore.java (original)
+++ gora/branches/goraamazon/gora-hbase/src/main/java/org/apache/gora/hbase/store/HBaseStore.java Mon Jul  2 13:38:43 2012
@@ -51,6 +51,7 @@ import org.apache.gora.persistency.State
 import org.apache.gora.persistency.StateManager;
 import org.apache.gora.persistency.StatefulHashMap;
 import org.apache.gora.persistency.StatefulMap;
+import org.apache.gora.persistency.impl.PersistentBase;
 import org.apache.gora.query.PartitionQuery;
 import org.apache.gora.query.Query;
 import org.apache.gora.query.impl.PartitionQueryImpl;
@@ -77,7 +78,7 @@ import org.jdom.input.SAXBuilder;
  * DataStore for HBase. Thread safe.
  *
  */
-public class HBaseStore<K, T extends Persistent> extends DataStoreBase<K, T>
+public class HBaseStore<K, T extends PersistentBase> extends DataStoreBase<K, T>
 implements Configurable {
 
   public static final Logger LOG = LoggerFactory.getLogger(HBaseStore.class);
@@ -262,7 +263,7 @@ implements Configurable {
   }
 
   @Override
-  public long deleteByQuery(Query<K, T> query) throws IOException {
+  public long deleteByQuery(Query<K, T> query) throws IOException, Exception {
 
     String[] fields = getFieldsToQuery(query.getFields());
     //find whether all fields are queried, which means that complete

Modified: gora/branches/goraamazon/gora-sql/src/main/java/org/apache/gora/sql/query/SqlQuery.java
URL: http://svn.apache.org/viewvc/gora/branches/goraamazon/gora-sql/src/main/java/org/apache/gora/sql/query/SqlQuery.java?rev=1356215&r1=1356214&r2=1356215&view=diff
==============================================================================
--- gora/branches/goraamazon/gora-sql/src/main/java/org/apache/gora/sql/query/SqlQuery.java (original)
+++ gora/branches/goraamazon/gora-sql/src/main/java/org/apache/gora/sql/query/SqlQuery.java Mon Jul  2 13:38:43 2012
@@ -19,13 +19,14 @@
 package org.apache.gora.sql.query;
 
 import org.apache.gora.persistency.Persistent;
+import org.apache.gora.persistency.impl.PersistentBase;
 import org.apache.gora.query.impl.QueryBase;
 import org.apache.gora.sql.store.SqlStore;
 
 /**
  * Query implementation covering SQL queries
  */
-public class SqlQuery<K, T extends Persistent> extends QueryBase<K, T> {
+public class SqlQuery<K, T extends PersistentBase> extends QueryBase<K, T> {
 
   public SqlQuery() {
     super(null);

Modified: gora/branches/goraamazon/gora-sql/src/main/java/org/apache/gora/sql/query/SqlResult.java
URL: http://svn.apache.org/viewvc/gora/branches/goraamazon/gora-sql/src/main/java/org/apache/gora/sql/query/SqlResult.java?rev=1356215&r1=1356214&r2=1356215&view=diff
==============================================================================
--- gora/branches/goraamazon/gora-sql/src/main/java/org/apache/gora/sql/query/SqlResult.java (original)
+++ gora/branches/goraamazon/gora-sql/src/main/java/org/apache/gora/sql/query/SqlResult.java Mon Jul  2 13:38:43 2012
@@ -23,13 +23,14 @@ import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 
 import org.apache.gora.persistency.Persistent;
+import org.apache.gora.persistency.impl.PersistentBase;
 import org.apache.gora.query.Query;
 import org.apache.gora.query.impl.ResultBase;
 import org.apache.gora.sql.store.SqlStore;
 import org.apache.gora.sql.util.SqlUtils;
 import org.apache.gora.store.DataStore;
 
-public class SqlResult<K, T extends Persistent> extends ResultBase<K, T> {
+public class SqlResult<K, T extends PersistentBase> extends ResultBase<K, T> {
 
   private ResultSet resultSet;
   private PreparedStatement statement;

Modified: gora/branches/goraamazon/gora-sql/src/main/java/org/apache/gora/sql/statement/HSqlInsertUpdateStatement.java
URL: http://svn.apache.org/viewvc/gora/branches/goraamazon/gora-sql/src/main/java/org/apache/gora/sql/statement/HSqlInsertUpdateStatement.java?rev=1356215&r1=1356214&r2=1356215&view=diff
==============================================================================
--- gora/branches/goraamazon/gora-sql/src/main/java/org/apache/gora/sql/statement/HSqlInsertUpdateStatement.java (original)
+++ gora/branches/goraamazon/gora-sql/src/main/java/org/apache/gora/sql/statement/HSqlInsertUpdateStatement.java Mon Jul  2 13:38:43 2012
@@ -24,11 +24,12 @@ import java.sql.SQLException;
 import java.util.Map.Entry;
 
 import org.apache.gora.persistency.Persistent;
+import org.apache.gora.persistency.impl.PersistentBase;
 import org.apache.gora.sql.store.Column;
 import org.apache.gora.sql.store.SqlMapping;
 import org.apache.gora.sql.store.SqlStore;
 
-public class HSqlInsertUpdateStatement<K, T extends Persistent>
+public class HSqlInsertUpdateStatement<K, T extends PersistentBase>
 extends InsertUpdateStatement<K, T> {
 
   public HSqlInsertUpdateStatement(SqlStore<K, T> store, SqlMapping mapping,

Modified: gora/branches/goraamazon/gora-sql/src/main/java/org/apache/gora/sql/statement/InsertUpdateStatement.java
URL: http://svn.apache.org/viewvc/gora/branches/goraamazon/gora-sql/src/main/java/org/apache/gora/sql/statement/InsertUpdateStatement.java?rev=1356215&r1=1356214&r2=1356215&view=diff
==============================================================================
--- gora/branches/goraamazon/gora-sql/src/main/java/org/apache/gora/sql/statement/InsertUpdateStatement.java (original)
+++ gora/branches/goraamazon/gora-sql/src/main/java/org/apache/gora/sql/statement/InsertUpdateStatement.java Mon Jul  2 13:38:43 2012
@@ -25,11 +25,12 @@ import java.util.TreeMap;
 
 import org.apache.avro.Schema;
 import org.apache.gora.persistency.Persistent;
+import org.apache.gora.persistency.impl.PersistentBase;
 import org.apache.gora.sql.store.Column;
 import org.apache.gora.sql.store.SqlMapping;
 import org.apache.gora.sql.store.SqlStore;
 
-public abstract class InsertUpdateStatement<K, V extends Persistent> {
+public abstract class InsertUpdateStatement<K, V extends PersistentBase> {
 
   protected class ColumnData {
     protected Object object;

Modified: gora/branches/goraamazon/gora-sql/src/main/java/org/apache/gora/sql/statement/InsertUpdateStatementFactory.java
URL: http://svn.apache.org/viewvc/gora/branches/goraamazon/gora-sql/src/main/java/org/apache/gora/sql/statement/InsertUpdateStatementFactory.java?rev=1356215&r1=1356214&r2=1356215&view=diff
==============================================================================
--- gora/branches/goraamazon/gora-sql/src/main/java/org/apache/gora/sql/statement/InsertUpdateStatementFactory.java (original)
+++ gora/branches/goraamazon/gora-sql/src/main/java/org/apache/gora/sql/statement/InsertUpdateStatementFactory.java Mon Jul  2 13:38:43 2012
@@ -17,14 +17,14 @@
  */
 package org.apache.gora.sql.statement;
 
-import org.apache.gora.persistency.Persistent;
+import org.apache.gora.persistency.impl.PersistentBase;
 import org.apache.gora.sql.store.SqlMapping;
 import org.apache.gora.sql.store.SqlStore;
 import org.apache.gora.sql.store.SqlStore.DBVendor;
 
 public class InsertUpdateStatementFactory {
 
-  public static <K, T extends Persistent>
+  public static <K, T extends PersistentBase>
   InsertUpdateStatement<K, T> createStatement(SqlStore<K, T> store,
       SqlMapping mapping, DBVendor dbVendor) {
     switch(dbVendor) {

Modified: gora/branches/goraamazon/gora-sql/src/main/java/org/apache/gora/sql/statement/MySqlInsertUpdateStatement.java
URL: http://svn.apache.org/viewvc/gora/branches/goraamazon/gora-sql/src/main/java/org/apache/gora/sql/statement/MySqlInsertUpdateStatement.java?rev=1356215&r1=1356214&r2=1356215&view=diff
==============================================================================
--- gora/branches/goraamazon/gora-sql/src/main/java/org/apache/gora/sql/statement/MySqlInsertUpdateStatement.java (original)
+++ gora/branches/goraamazon/gora-sql/src/main/java/org/apache/gora/sql/statement/MySqlInsertUpdateStatement.java Mon Jul  2 13:38:43 2012
@@ -25,12 +25,13 @@ import java.util.Map.Entry;
 
 import org.apache.avro.Schema;
 import org.apache.gora.persistency.Persistent;
+import org.apache.gora.persistency.impl.PersistentBase;
 import org.apache.gora.sql.store.Column;
 import org.apache.gora.sql.store.SqlMapping;
 import org.apache.gora.sql.store.SqlStore;
 import org.apache.gora.util.StringUtils;
 
-public class MySqlInsertUpdateStatement<K, V extends Persistent> extends InsertUpdateStatement<K, V> {
+public class MySqlInsertUpdateStatement<K, V extends PersistentBase> extends InsertUpdateStatement<K, V> {
 
   public MySqlInsertUpdateStatement(SqlStore<K, V> store, SqlMapping mapping, String tableName) {
     super(store, mapping, tableName);

Modified: gora/branches/goraamazon/gora-sql/src/main/java/org/apache/gora/sql/store/SqlStore.java
URL: http://svn.apache.org/viewvc/gora/branches/goraamazon/gora-sql/src/main/java/org/apache/gora/sql/store/SqlStore.java?rev=1356215&r1=1356214&r2=1356215&view=diff
==============================================================================
--- gora/branches/goraamazon/gora-sql/src/main/java/org/apache/gora/sql/store/SqlStore.java (original)
+++ gora/branches/goraamazon/gora-sql/src/main/java/org/apache/gora/sql/store/SqlStore.java Mon Jul  2 13:38:43 2012
@@ -49,6 +49,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.gora.persistency.Persistent;
 import org.apache.gora.persistency.StateManager;
+import org.apache.gora.persistency.impl.PersistentBase;
 import org.apache.gora.query.PartitionQuery;
 import org.apache.gora.query.Query;
 import org.apache.gora.query.Result;
@@ -94,7 +95,7 @@ import org.jdom.input.SAXBuilder;
  * re-write as per GORA-86
  * Please see https://issues.apache.org/jira/browse/GORA-86
  */
-public class SqlStore<K, T extends Persistent> extends DataStoreBase<K, T> {
+public class SqlStore<K, T extends PersistentBase> extends DataStoreBase<K, T> {
 
   /** The vendor of the DB */
   public static enum DBVendor {

Modified: gora/branches/goraamazon/gora-tutorial/src/main/java/org/apache/gora/tutorial/log/LogManager.java
URL: http://svn.apache.org/viewvc/gora/branches/goraamazon/gora-tutorial/src/main/java/org/apache/gora/tutorial/log/LogManager.java?rev=1356215&r1=1356214&r2=1356215&view=diff
==============================================================================
--- gora/branches/goraamazon/gora-tutorial/src/main/java/org/apache/gora/tutorial/log/LogManager.java (original)
+++ gora/branches/goraamazon/gora-tutorial/src/main/java/org/apache/gora/tutorial/log/LogManager.java Mon Jul  2 13:38:43 2012
@@ -77,7 +77,7 @@ public class LogManager {
    * Parses a log file and store the contents at the data store.
    * @param input the input file location
    */
-  private void parse(String input) throws IOException, ParseException {
+  private void parse(String input) throws IOException, ParseException, Exception {
     log.info("Parsing file:" + input);
     BufferedReader reader = new BufferedReader(new FileReader(input));
     long lineCount = 0;
@@ -136,19 +136,19 @@ public class LogManager {
   }
   
   /** Stores the pageview object with the given key */
-  private void storePageview(long key, Pageview pageview) throws IOException {
+  private void storePageview(long key, Pageview pageview) throws IOException, Exception {
 	log.info("Storing Pageview in: " + dataStore.toString());
     dataStore.put(key, pageview);
   }
   
   /** Fetches a single pageview object and prints it*/
-  private void get(long key) throws IOException {
+  private void get(long key) throws IOException, Exception {
     Pageview pageview = dataStore.get(key);
     printPageview(pageview);
   }
   
   /** Queries and prints a single pageview object */
-  private void query(long key) throws IOException {
+  private void query(long key) throws IOException, Exception {
     //Queries are constructed from the data store
     Query<Long, Pageview> query = dataStore.newQuery();
     query.setKey(key);
@@ -160,7 +160,7 @@ public class LogManager {
   }
   
   /** Queries and prints pageview object that have keys between startKey and endKey*/
-  private void query(long startKey, long endKey) throws IOException {
+  private void query(long startKey, long endKey) throws IOException, Exception {
     Query<Long, Pageview> query = dataStore.newQuery();
     //set the properties of query
     query.setStartKey(startKey);
@@ -181,7 +181,7 @@ public class LogManager {
   }
   
   /** This method illustrates delete by query call */
-  private void deleteByQuery(long startKey, long endKey) throws IOException {
+  private void deleteByQuery(long startKey, long endKey) throws IOException, Exception {
     //Constructs a query from the dataStore. The matching rows to this query will be deleted
     Query<Long, Pageview> query = dataStore.newQuery();
     //set the properties of query
@@ -192,7 +192,7 @@ public class LogManager {
     log.info("pageviews with keys between " + startKey + " and " + endKey + " are deleted");
   }
   
-  private void printResult(Result<Long, Pageview> result) throws IOException {
+  private void printResult(Result<Long, Pageview> result) throws IOException, Exception {
     
     while(result.next()) { //advances the Result object and breaks if at end
       long resultKey = result.getKey(); //obtain current key
@@ -215,7 +215,7 @@ public class LogManager {
     }
   }
   
-  private void close() throws IOException {
+  private void close() throws IOException, Exception {
     //It is very important to close the datastore properly, otherwise
     //some data loss might occur.
     if(dataStore != null)

Modified: gora/branches/goraamazon/pom.xml
URL: http://svn.apache.org/viewvc/gora/branches/goraamazon/pom.xml?rev=1356215&r1=1356214&r2=1356215&view=diff
==============================================================================
--- gora/branches/goraamazon/pom.xml (original)
+++ gora/branches/goraamazon/pom.xml Mon Jul  2 13:38:43 2012
@@ -506,6 +506,7 @@
         <module>gora-hbase</module>
         <module>gora-accumulo</module>
         <module>gora-cassandra</module>
+        <module>gora-dynamodb</module>
         <module>gora-sql</module>
         <module>gora-tutorial</module>
         <module>sources-dist</module>
@@ -522,6 +523,8 @@
         <hbase.version>0.90.4</hbase.version>
         <avro.version>1.3.3</avro.version>
         <cxf-rt-frontend-jaxrs.version>2.5.2</cxf-rt-frontend-jaxrs.version>
+        <!-- Amazon Dependencies -->
+        <amazon.version>1.3.12</amazon.version>
         <!-- Cassandra Dependencies -->
         <cassandra.version>1.0.2</cassandra.version>
         <cassandra.thrift.version>1.0.2</cassandra.thrift.version>
@@ -617,7 +620,13 @@
             <groupId>org.apache.gora</groupId>
             <artifactId>gora-tutorial</artifactId>
             <version>${project.version}</version>
-            </dependency>
+          </dependency>
+          
+          <dependency>
+            <groupId>org.apache.gora</groupId>
+            <artifactId>gora-dynamodb</artifactId>
+            <version>${project.version}</version>
+          </dependency>
   
           <!-- Avro needs this version of jackson -->
           <dependency>
@@ -708,11 +717,19 @@
                 <version>${cassandra.thrift.version}</version>
             </dependency>
 
-          <dependency>
-            <groupId>me.prettyprint</groupId>
-            <artifactId>hector-core</artifactId>
-            <version>${hector.version}</version>
-          </dependency>
+            <dependency>
+              <groupId>me.prettyprint</groupId>
+              <artifactId>hector-core</artifactId>
+              <version>${hector.version}</version>
+            </dependency>
+            
+            <!-- Amazon Dependencies -->
+            <dependency>
+       	      <groupId>com.amazonaws</groupId>
+	      <artifactId>aws-java-sdk</artifactId>
+	      <version>${amazon.version}</version>
+            </dependency>
+          
 
             <!-- Misc Dependencies -->
             <dependency>